diff --git a/cmd/prometheus/main.go b/cmd/prometheus/main.go index ab20d65940..521201d876 100644 --- a/cmd/prometheus/main.go +++ b/cmd/prometheus/main.go @@ -56,7 +56,7 @@ import ( "github.com/prometheus/prometheus/scrape" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/storage/remote" - "github.com/prometheus/prometheus/storage/tsdb" + "github.com/prometheus/prometheus/tsdb" "github.com/prometheus/prometheus/util/strutil" "github.com/prometheus/prometheus/web" ) diff --git a/promql/test.go b/promql/test.go index 8c2b0497fc..07508f13a9 100644 --- a/promql/test.go +++ b/promql/test.go @@ -428,10 +428,7 @@ func (t *Test) exec(tc testCommand) error { t.clear() case *loadCmd: - app, err := t.storage.Appender() - if err != nil { - return err - } + app := t.storage.Appender() if err := cmd.append(app); err != nil { app.Rollback() return err @@ -641,10 +638,7 @@ func (ll *LazyLoader) clear() { // appendTill appends the defined time series to the storage till the given timestamp (in milliseconds). func (ll *LazyLoader) appendTill(ts int64) error { - app, err := ll.storage.Appender() - if err != nil { - return err - } + app := ll.storage.Appender() for h, smpls := range ll.loadCmd.defs { m := ll.loadCmd.metrics[h] for i, s := range smpls { diff --git a/promql/value.go b/promql/value.go index 6993ee8b07..3ef629fd45 100644 --- a/promql/value.go +++ b/promql/value.go @@ -20,6 +20,7 @@ import ( "strings" "github.com/pkg/errors" + "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/storage" @@ -275,7 +276,7 @@ func (ss *StorageSeries) Labels() labels.Labels { } // Iterator returns a new iterator of the data of the series. -func (ss *StorageSeries) Iterator() storage.SeriesIterator { +func (ss *StorageSeries) Iterator() chunkenc.Iterator { return newStorageSeriesIterator(ss.series) } diff --git a/rules/manager.go b/rules/manager.go index 9ae3b69b76..16966ad8a3 100644 --- a/rules/manager.go +++ b/rules/manager.go @@ -587,12 +587,7 @@ func (g *Group) Eval(ctx context.Context, ts time.Time) { numDuplicates = 0 ) - app, err := g.opts.Appendable.Appender() - if err != nil { - level.Warn(g.logger).Log("msg", "creating appender failed", "err", err) - return - } - + app := g.opts.Appendable.Appender() seriesReturned := make(map[string]labels.Labels, len(g.seriesInPreviousEval[i])) for _, s := range vector { if _, err := app.Add(s.Metric, s.T, s.V); err != nil { @@ -645,14 +640,10 @@ func (g *Group) cleanupStaleSeries(ts time.Time) { if len(g.staleSeries) == 0 { return } - app, err := g.opts.Appendable.Appender() - if err != nil { - level.Warn(g.logger).Log("msg", "creating appender failed", "err", err) - return - } + app := g.opts.Appendable.Appender() for _, s := range g.staleSeries { // Rule that produced series no longer configured, mark it stale. - _, err = app.Add(s, timestamp.FromTime(ts), math.Float64frombits(value.StaleNaN)) + _, err := app.Add(s, timestamp.FromTime(ts), math.Float64frombits(value.StaleNaN)) switch err { case nil: case storage.ErrOutOfOrderSample, storage.ErrDuplicateSampleForTimestamp: @@ -836,11 +827,6 @@ type Manager struct { logger log.Logger } -// Appendable returns an Appender. -type Appendable interface { - Appender() (storage.Appender, error) -} - // NotifyFunc sends notifications about a set of alerts generated by the given expression. type NotifyFunc func(ctx context.Context, expr string, alerts ...*Alert) @@ -850,7 +836,7 @@ type ManagerOptions struct { QueryFunc QueryFunc NotifyFunc NotifyFunc Context context.Context - Appendable Appendable + Appendable storage.Appendable TSDB storage.Storage Logger log.Logger Registerer prometheus.Registerer diff --git a/rules/manager_test.go b/rules/manager_test.go index 61b6cf5703..53c843c746 100644 --- a/rules/manager_test.go +++ b/rules/manager_test.go @@ -540,7 +540,7 @@ func TestStaleness(t *testing.T) { }) // A time series that has two samples and then goes stale. - app, _ := storage.Appender() + app := storage.Appender() app.Add(labels.FromStrings(model.MetricNameLabel, "a"), 0, 1) app.Add(labels.FromStrings(model.MetricNameLabel, "a"), 1000, 2) app.Add(labels.FromStrings(model.MetricNameLabel, "a"), 2000, math.Float64frombits(value.StaleNaN)) @@ -868,7 +868,7 @@ func TestNotify(t *testing.T) { Opts: opts, }) - app, _ := storage.Appender() + app := storage.Appender() app.Add(labels.FromStrings(model.MetricNameLabel, "a"), 1000, 2) app.Add(labels.FromStrings(model.MetricNameLabel, "a"), 2000, 3) app.Add(labels.FromStrings(model.MetricNameLabel, "a"), 5000, 3) diff --git a/scrape/helpers_test.go b/scrape/helpers_test.go index 903c40688b..0dae21e770 100644 --- a/scrape/helpers_test.go +++ b/scrape/helpers_test.go @@ -20,16 +20,16 @@ import ( type nopAppendable struct{} -func (a nopAppendable) Appender() (storage.Appender, error) { - return nopAppender{}, nil +func (a nopAppendable) Appender() storage.Appender { + return nopAppender{} } type nopAppender struct{} -func (a nopAppender) Add(labels.Labels, int64, float64) (uint64, error) { return 0, nil } -func (a nopAppender) AddFast(labels.Labels, uint64, int64, float64) error { return nil } -func (a nopAppender) Commit() error { return nil } -func (a nopAppender) Rollback() error { return nil } +func (a nopAppender) Add(labels.Labels, int64, float64) (uint64, error) { return 0, nil } +func (a nopAppender) AddFast(uint64, int64, float64) error { return nil } +func (a nopAppender) Commit() error { return nil } +func (a nopAppender) Rollback() error { return nil } type sample struct { metric labels.Labels @@ -42,18 +42,21 @@ type sample struct { type collectResultAppender struct { next storage.Appender result []sample + + mapper map[uint64]labels.Labels } -func (a *collectResultAppender) AddFast(m labels.Labels, ref uint64, t int64, v float64) error { +func (a *collectResultAppender) AddFast(ref uint64, t int64, v float64) error { if a.next == nil { return storage.ErrNotFound } - err := a.next.AddFast(m, ref, t, v) + + err := a.next.AddFast(ref, t, v) if err != nil { return err } a.result = append(a.result, sample{ - metric: m, + metric: a.mapper[ref], t: t, v: v, }) @@ -69,7 +72,17 @@ func (a *collectResultAppender) Add(m labels.Labels, t int64, v float64) (uint64 if a.next == nil { return 0, nil } - return a.next.Add(m, t, v) + + if a.mapper == nil { + a.mapper = map[uint64]labels.Labels{} + } + + ref, err := a.next.Add(m, t, v) + if err != nil { + return 0, err + } + a.mapper[ref] = m + return ref, nil } func (a *collectResultAppender) Commit() error { return nil } diff --git a/scrape/manager.go b/scrape/manager.go index 2da5e66e10..26bb0d1a57 100644 --- a/scrape/manager.go +++ b/scrape/manager.go @@ -100,13 +100,8 @@ func (mc *MetadataMetricsCollector) Collect(ch chan<- prometheus.Metric) { } } -// Appendable returns an Appender. -type Appendable interface { - Appender() (storage.Appender, error) -} - // NewManager is the Manager constructor -func NewManager(logger log.Logger, app Appendable) *Manager { +func NewManager(logger log.Logger, app storage.Appendable) *Manager { if logger == nil { logger = log.NewNopLogger() } @@ -127,7 +122,7 @@ func NewManager(logger log.Logger, app Appendable) *Manager { // when receiving new target groups form the discovery manager. type Manager struct { logger log.Logger - append Appendable + append storage.Appendable graceShut chan struct{} jitterSeed uint64 // Global jitterSeed seed is used to spread scrape workload across HA setup. diff --git a/scrape/scrape.go b/scrape/scrape.go index c216f03f1a..99bc85fbbb 100644 --- a/scrape/scrape.go +++ b/scrape/scrape.go @@ -156,7 +156,7 @@ func init() { // scrapePool manages scrapes for sets of targets. type scrapePool struct { - appendable Appendable + appendable storage.Appendable logger log.Logger mtx sync.RWMutex @@ -187,7 +187,7 @@ const maxAheadTime = 10 * time.Minute type labelsMutator func(labels.Labels) labels.Labels -func newScrapePool(cfg *config.ScrapeConfig, app Appendable, jitterSeed uint64, logger log.Logger) (*scrapePool, error) { +func newScrapePool(cfg *config.ScrapeConfig, app storage.Appendable, jitterSeed uint64, logger log.Logger) (*scrapePool, error) { targetScrapePools.Inc() if logger == nil { logger = log.NewNopLogger() @@ -228,13 +228,7 @@ func newScrapePool(cfg *config.ScrapeConfig, app Appendable, jitterSeed uint64, return mutateSampleLabels(l, opts.target, opts.honorLabels, opts.mrc) }, func(l labels.Labels) labels.Labels { return mutateReportSampleLabels(l, opts.target) }, - func() storage.Appender { - app, err := app.Appender() - if err != nil { - panic(err) - } - return appender(app, opts.limit) - }, + func() storage.Appender { return appender(app.Appender(), opts.limit) }, cache, jitterSeed, opts.honorTimestamps, @@ -1112,7 +1106,7 @@ loop: } ce, ok := sl.cache.get(yoloString(met)) if ok { - switch err = app.AddFast(ce.lset, ce.ref, t, v); err { + switch err = app.AddFast(ce.ref, t, v); err { case nil: if tp == nil { sl.cache.trackStaleness(ce.hash, ce.lset) @@ -1323,7 +1317,7 @@ func (sl *scrapeLoop) reportStale(start time.Time) error { func (sl *scrapeLoop) addReportSample(app storage.Appender, s string, t int64, v float64) error { ce, ok := sl.cache.get(s) if ok { - err := app.AddFast(ce.lset, ce.ref, t, v) + err := app.AddFast(ce.ref, t, v) switch err { case nil: return nil diff --git a/scrape/scrape_test.go b/scrape/scrape_test.go index fd0490e181..f0009f29e7 100644 --- a/scrape/scrape_test.go +++ b/scrape/scrape_test.go @@ -644,8 +644,7 @@ func TestScrapeLoopSeriesAdded(t *testing.T) { s := teststorage.New(t) defer s.Close() - app, err := s.Appender() - testutil.Ok(t, err) + app := s.Appender() ctx, cancel := context.WithCancel(context.Background()) sl := newScrapeLoop(ctx, @@ -788,8 +787,7 @@ func TestScrapeLoopCache(t *testing.T) { s := teststorage.New(t) defer s.Close() - sapp, err := s.Appender() - testutil.Ok(t, err) + sapp := s.Appender() appender := &collectResultAppender{next: sapp} var ( @@ -866,8 +864,7 @@ func TestScrapeLoopCacheMemoryExhaustionProtection(t *testing.T) { s := teststorage.New(t) defer s.Close() - sapp, err := s.Appender() - testutil.Ok(t, err) + sapp := s.Appender() appender := &collectResultAppender{next: sapp} var ( @@ -1092,8 +1089,7 @@ func TestScrapeLoop_ChangingMetricString(t *testing.T) { s := teststorage.New(t) defer s.Close() - app, err := s.Appender() - testutil.Ok(t, err) + app := s.Appender() capp := &collectResultAppender{next: app} @@ -1108,7 +1104,7 @@ func TestScrapeLoop_ChangingMetricString(t *testing.T) { ) now := time.Now() - _, _, _, err = sl.append([]byte(`metric_a{a="1",b="1"} 1`), "", now) + _, _, _, err := sl.append([]byte(`metric_a{a="1",b="1"} 1`), "", now) testutil.Ok(t, err) _, _, _, err = sl.append([]byte(`metric_a{b="1",a="1"} 2`), "", now.Add(time.Minute)) @@ -1273,8 +1269,8 @@ func (app *errorAppender) Add(lset labels.Labels, t int64, v float64) (uint64, e } } -func (app *errorAppender) AddFast(lset labels.Labels, ref uint64, t int64, v float64) error { - return app.collectResultAppender.AddFast(lset, ref, t, v) +func (app *errorAppender) AddFast(ref uint64, t int64, v float64) error { + return app.collectResultAppender.AddFast(ref, t, v) } func TestScrapeLoopAppendGracefullyIfAmendOrOutOfOrderOrOutOfBounds(t *testing.T) { @@ -1498,8 +1494,7 @@ func TestScrapeLoop_RespectTimestamps(t *testing.T) { s := teststorage.New(t) defer s.Close() - app, err := s.Appender() - testutil.Ok(t, err) + app := s.Appender() capp := &collectResultAppender{next: app} @@ -1513,7 +1508,7 @@ func TestScrapeLoop_RespectTimestamps(t *testing.T) { ) now := time.Now() - _, _, _, err = sl.append([]byte(`metric_a{a="1",b="1"} 1 0`), "", now) + _, _, _, err := sl.append([]byte(`metric_a{a="1",b="1"} 1 0`), "", now) testutil.Ok(t, err) want := []sample{ @@ -1530,8 +1525,7 @@ func TestScrapeLoop_DiscardTimestamps(t *testing.T) { s := teststorage.New(t) defer s.Close() - app, err := s.Appender() - testutil.Ok(t, err) + app := s.Appender() capp := &collectResultAppender{next: app} @@ -1545,7 +1539,7 @@ func TestScrapeLoop_DiscardTimestamps(t *testing.T) { ) now := time.Now() - _, _, _, err = sl.append([]byte(`metric_a{a="1",b="1"} 1 0`), "", now) + _, _, _, err := sl.append([]byte(`metric_a{a="1",b="1"} 1 0`), "", now) testutil.Ok(t, err) want := []sample{ @@ -1562,8 +1556,7 @@ func TestScrapeLoopDiscardDuplicateLabels(t *testing.T) { s := teststorage.New(t) defer s.Close() - app, err := s.Appender() - testutil.Ok(t, err) + app := s.Appender() ctx, cancel := context.WithCancel(context.Background()) sl := newScrapeLoop(ctx, @@ -1579,7 +1572,7 @@ func TestScrapeLoopDiscardDuplicateLabels(t *testing.T) { defer cancel() // We add a good and a bad metric to check that both are discarded. - _, _, _, err = sl.append([]byte("test_metric{le=\"500\"} 1\ntest_metric{le=\"600\",le=\"700\"} 1\n"), "", time.Time{}) + _, _, _, err := sl.append([]byte("test_metric{le=\"500\"} 1\ntest_metric{le=\"600\",le=\"700\"} 1\n"), "", time.Time{}) testutil.NotOk(t, err) q, err := s.Querier(ctx, time.Time{}.UnixNano(), 0) diff --git a/scrape/target.go b/scrape/target.go index 678438e969..ac9cad0316 100644 --- a/scrape/target.go +++ b/scrape/target.go @@ -303,14 +303,14 @@ func (app *limitAppender) Add(lset labels.Labels, t int64, v float64) (uint64, e return ref, nil } -func (app *limitAppender) AddFast(lset labels.Labels, ref uint64, t int64, v float64) error { +func (app *limitAppender) AddFast(ref uint64, t int64, v float64) error { if !value.IsStaleNaN(v) { app.i++ if app.i > app.limit { return errSampleLimit } } - err := app.Appender.AddFast(lset, ref, t, v) + err := app.Appender.AddFast(ref, t, v) return err } @@ -332,11 +332,11 @@ func (app *timeLimitAppender) Add(lset labels.Labels, t int64, v float64) (uint6 return ref, nil } -func (app *timeLimitAppender) AddFast(lset labels.Labels, ref uint64, t int64, v float64) error { +func (app *timeLimitAppender) AddFast(ref uint64, t int64, v float64) error { if t > app.maxTime { return storage.ErrOutOfBounds } - err := app.Appender.AddFast(lset, ref, t, v) + err := app.Appender.AddFast(ref, t, v) return err } diff --git a/storage/buffer.go b/storage/buffer.go index 1b1d09bf59..874fdba958 100644 --- a/storage/buffer.go +++ b/storage/buffer.go @@ -15,11 +15,13 @@ package storage import ( "math" + + "github.com/prometheus/prometheus/tsdb/chunkenc" ) // BufferedSeriesIterator wraps an iterator with a look-back buffer. type BufferedSeriesIterator struct { - it SeriesIterator + it chunkenc.Iterator buf *sampleRing delta int64 @@ -36,7 +38,7 @@ func NewBuffer(delta int64) *BufferedSeriesIterator { // NewBufferIterator returns a new iterator that buffers the values within the // time range of the current element and the duration of delta before. -func NewBufferIterator(it SeriesIterator, delta int64) *BufferedSeriesIterator { +func NewBufferIterator(it chunkenc.Iterator, delta int64) *BufferedSeriesIterator { bit := &BufferedSeriesIterator{ buf: newSampleRing(delta, 16), delta: delta, @@ -48,7 +50,7 @@ func NewBufferIterator(it SeriesIterator, delta int64) *BufferedSeriesIterator { // Reset re-uses the buffer with a new iterator, resetting the buffered time // delta to its original value. -func (b *BufferedSeriesIterator) Reset(it SeriesIterator) { +func (b *BufferedSeriesIterator) Reset(it chunkenc.Iterator) { b.it = it b.lastTime = math.MinInt64 b.ok = true @@ -70,7 +72,7 @@ func (b *BufferedSeriesIterator) PeekBack(n int) (t int64, v float64, ok bool) { // Buffer returns an iterator over the buffered data. Invalidates previously // returned iterators. -func (b *BufferedSeriesIterator) Buffer() SeriesIterator { +func (b *BufferedSeriesIterator) Buffer() chunkenc.Iterator { return b.buf.iterator() } @@ -159,7 +161,7 @@ func (r *sampleRing) reset() { } // Returns the current iterator. Invalidates previously returned iterators. -func (r *sampleRing) iterator() SeriesIterator { +func (r *sampleRing) iterator() chunkenc.Iterator { r.it.r = r r.it.i = -1 return &r.it diff --git a/storage/buffer_test.go b/storage/buffer_test.go index 9ef7f5da9a..4ec4591a4a 100644 --- a/storage/buffer_test.go +++ b/storage/buffer_test.go @@ -19,6 +19,7 @@ import ( "testing" "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/util/testutil" ) @@ -190,7 +191,7 @@ func (m *mockSeriesIterator) Err() error { return m.err() } type mockSeries struct { labels func() labels.Labels - iterator func() SeriesIterator + iterator func() chunkenc.Iterator } func newMockSeries(lset labels.Labels, samples []sample) Series { @@ -198,14 +199,14 @@ func newMockSeries(lset labels.Labels, samples []sample) Series { labels: func() labels.Labels { return lset }, - iterator: func() SeriesIterator { + iterator: func() chunkenc.Iterator { return newListSeriesIterator(samples) }, } } -func (m *mockSeries) Labels() labels.Labels { return m.labels() } -func (m *mockSeries) Iterator() SeriesIterator { return m.iterator() } +func (m *mockSeries) Labels() labels.Labels { return m.labels() } +func (m *mockSeries) Iterator() chunkenc.Iterator { return m.iterator() } type listSeriesIterator struct { list []sample diff --git a/storage/fanout.go b/storage/fanout.go index 344200eb99..bad6a8df12 100644 --- a/storage/fanout.go +++ b/storage/fanout.go @@ -24,6 +24,7 @@ import ( "github.com/pkg/errors" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/tsdb/chunkenc" ) type fanout struct { @@ -87,25 +88,17 @@ func (f *fanout) Querier(ctx context.Context, mint, maxt int64) (Querier, error) return NewMergeQuerier(primaryQuerier, queriers), nil } -func (f *fanout) Appender() (Appender, error) { - primary, err := f.primary.Appender() - if err != nil { - return nil, err - } - +func (f *fanout) Appender() Appender { + primary := f.primary.Appender() secondaries := make([]Appender, 0, len(f.secondaries)) for _, storage := range f.secondaries { - appender, err := storage.Appender() - if err != nil { - return nil, err - } - secondaries = append(secondaries, appender) + secondaries = append(secondaries, storage.Appender()) } return &fanoutAppender{ logger: f.logger, primary: primary, secondaries: secondaries, - }, nil + } } // Close closes the storage and all its underlying resources. @@ -146,13 +139,13 @@ func (f *fanoutAppender) Add(l labels.Labels, t int64, v float64) (uint64, error return ref, nil } -func (f *fanoutAppender) AddFast(l labels.Labels, ref uint64, t int64, v float64) error { - if err := f.primary.AddFast(l, ref, t, v); err != nil { +func (f *fanoutAppender) AddFast(ref uint64, t int64, v float64) error { + if err := f.primary.AddFast(ref, t, v); err != nil { return err } for _, appender := range f.secondaries { - if _, err := appender.Add(l, t, v); err != nil { + if err := appender.AddFast(ref, t, v); err != nil { return err } } @@ -521,8 +514,8 @@ func (m *mergeSeries) Labels() labels.Labels { return m.labels } -func (m *mergeSeries) Iterator() SeriesIterator { - iterators := make([]SeriesIterator, 0, len(m.series)) +func (m *mergeSeries) Iterator() chunkenc.Iterator { + iterators := make([]chunkenc.Iterator, 0, len(m.series)) for _, s := range m.series { iterators = append(iterators, s.Iterator()) } @@ -530,11 +523,11 @@ func (m *mergeSeries) Iterator() SeriesIterator { } type mergeIterator struct { - iterators []SeriesIterator + iterators []chunkenc.Iterator h seriesIteratorHeap } -func newMergeIterator(iterators []SeriesIterator) SeriesIterator { +func newMergeIterator(iterators []chunkenc.Iterator) chunkenc.Iterator { return &mergeIterator{ iterators: iterators, h: nil, @@ -581,7 +574,7 @@ func (c *mergeIterator) Next() bool { break } - iter := heap.Pop(&c.h).(SeriesIterator) + iter := heap.Pop(&c.h).(chunkenc.Iterator) if iter.Next() { heap.Push(&c.h, iter) } @@ -599,7 +592,7 @@ func (c *mergeIterator) Err() error { return nil } -type seriesIteratorHeap []SeriesIterator +type seriesIteratorHeap []chunkenc.Iterator func (h seriesIteratorHeap) Len() int { return len(h) } func (h seriesIteratorHeap) Swap(i, j int) { h[i], h[j] = h[j], h[i] } @@ -611,7 +604,7 @@ func (h seriesIteratorHeap) Less(i, j int) bool { } func (h *seriesIteratorHeap) Push(x interface{}) { - *h = append(*h, x.(SeriesIterator)) + *h = append(*h, x.(chunkenc.Iterator)) } func (h *seriesIteratorHeap) Pop() interface{} { diff --git a/storage/fanout_test.go b/storage/fanout_test.go index dda57515c7..97ffcddb29 100644 --- a/storage/fanout_test.go +++ b/storage/fanout_test.go @@ -19,6 +19,7 @@ import ( "testing" "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/util/testutil" ) @@ -122,24 +123,24 @@ func TestMergeSeriesSet(t *testing.T) { func TestMergeIterator(t *testing.T) { for _, tc := range []struct { - input []SeriesIterator + input []chunkenc.Iterator expected []sample }{ { - input: []SeriesIterator{ + input: []chunkenc.Iterator{ newListSeriesIterator([]sample{{0, 0}, {1, 1}}), }, expected: []sample{{0, 0}, {1, 1}}, }, { - input: []SeriesIterator{ + input: []chunkenc.Iterator{ newListSeriesIterator([]sample{{0, 0}, {1, 1}}), newListSeriesIterator([]sample{{2, 2}, {3, 3}}), }, expected: []sample{{0, 0}, {1, 1}, {2, 2}, {3, 3}}, }, { - input: []SeriesIterator{ + input: []chunkenc.Iterator{ newListSeriesIterator([]sample{{0, 0}, {3, 3}}), newListSeriesIterator([]sample{{1, 1}, {4, 4}}), newListSeriesIterator([]sample{{2, 2}, {5, 5}}), @@ -147,7 +148,7 @@ func TestMergeIterator(t *testing.T) { expected: []sample{{0, 0}, {1, 1}, {2, 2}, {3, 3}, {4, 4}, {5, 5}}, }, { - input: []SeriesIterator{ + input: []chunkenc.Iterator{ newListSeriesIterator([]sample{{0, 0}, {1, 1}}), newListSeriesIterator([]sample{{0, 0}, {2, 2}}), newListSeriesIterator([]sample{{2, 2}, {3, 3}}), @@ -163,19 +164,19 @@ func TestMergeIterator(t *testing.T) { func TestMergeIteratorSeek(t *testing.T) { for _, tc := range []struct { - input []SeriesIterator + input []chunkenc.Iterator seek int64 expected []sample }{ { - input: []SeriesIterator{ + input: []chunkenc.Iterator{ newListSeriesIterator([]sample{{0, 0}, {1, 1}, {2, 2}}), }, seek: 1, expected: []sample{{1, 1}, {2, 2}}, }, { - input: []SeriesIterator{ + input: []chunkenc.Iterator{ newListSeriesIterator([]sample{{0, 0}, {1, 1}}), newListSeriesIterator([]sample{{2, 2}, {3, 3}}), }, @@ -183,7 +184,7 @@ func TestMergeIteratorSeek(t *testing.T) { expected: []sample{{2, 2}, {3, 3}}, }, { - input: []SeriesIterator{ + input: []chunkenc.Iterator{ newListSeriesIterator([]sample{{0, 0}, {3, 3}}), newListSeriesIterator([]sample{{1, 1}, {4, 4}}), newListSeriesIterator([]sample{{2, 2}, {5, 5}}), @@ -203,7 +204,7 @@ func TestMergeIteratorSeek(t *testing.T) { } } -func drainSamples(iter SeriesIterator) []sample { +func drainSamples(iter chunkenc.Iterator) []sample { result := []sample{} for iter.Next() { t, v := iter.At() diff --git a/storage/interface.go b/storage/interface.go index 0ff3da99a2..6d91ff4248 100644 --- a/storage/interface.go +++ b/storage/interface.go @@ -18,6 +18,9 @@ import ( "errors" "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. @@ -28,17 +31,21 @@ var ( ErrOutOfBounds = errors.New("out of bounds") ) +// Appendable allows creating appenders. +type Appendable interface { + // Appender returns a new appender against the storage. + Appender() Appender +} + // Storage ingests and manages samples, along with various indexes. All methods // are goroutine-safe. Storage implements storage.SampleAppender. type Storage interface { Queryable + Appendable // StartTime returns the oldest timestamp stored in the storage. StartTime() (int64, error) - // Appender returns a new appender against the storage. - Appender() (Appender, error) - // Close closes the storage and all its underlying resources. Close() error } @@ -49,7 +56,8 @@ type Queryable interface { Querier(ctx context.Context, mint, maxt int64) (Querier, error) } -// Querier provides reading access to time series data. +// Querier provides querying access over time series data of a fixed +// time range. type Querier interface { // Select returns a set of series that matches the given label matchers. Select(*SelectParams, ...*labels.Matcher) (SeriesSet, Warnings, error) @@ -58,6 +66,7 @@ type Querier interface { SelectSorted(*SelectParams, ...*labels.Matcher) (SeriesSet, Warnings, error) // LabelValues returns all potential values for a label name. + // It is not safe to use the strings beyond the lifefime of the querier. LabelValues(name string) ([]string, Warnings, error) // LabelNames returns all the unique label names present in the block in sorted order. @@ -93,7 +102,7 @@ func (f QueryableFunc) Querier(ctx context.Context, mint, maxt int64) (Querier, type Appender interface { Add(l labels.Labels, t int64, v float64) (uint64, error) - AddFast(l labels.Labels, ref uint64, t int64, v float64) error + AddFast(ref uint64, t int64, v float64) error // Commit submits the collected samples and purges the batch. Commit() error @@ -108,25 +117,36 @@ type SeriesSet interface { Err() error } +var emptySeriesSet = errSeriesSet{} + +// EmptySeriesSet returns a series set that's always empty. +func EmptySeriesSet() SeriesSet { + return emptySeriesSet +} + +type errSeriesSet struct { + err error +} + +func (s errSeriesSet) Next() bool { return false } +func (s errSeriesSet) At() Series { return nil } +func (s errSeriesSet) Err() error { return s.err } + // Series represents a single time series. type Series interface { // Labels returns the complete set of labels identifying the series. Labels() labels.Labels // Iterator returns a new iterator of the data of the series. - Iterator() SeriesIterator + Iterator() chunkenc.Iterator } -// SeriesIterator iterates over the data of a time series. -type SeriesIterator interface { - // Seek advances the iterator forward to the value at or after - // the given timestamp. - Seek(t int64) bool - // At returns the current timestamp/value pair. - At() (t int64, v float64) - // Next advances the iterator by one. +// ChunkSeriesSet exposes the chunks and intervals of a series instead of the +// actual series itself. +// TODO(bwplotka): Move it to Series liike Iterator that iterates over chunks and avoiding loading all of them at once. +type ChunkSeriesSet interface { Next() bool - // Err returns the current error. + At() (labels.Labels, []chunks.Meta, tombstones.Intervals) Err() error } diff --git a/storage/remote/codec.go b/storage/remote/codec.go index 10500f06a3..c3e5403a5b 100644 --- a/storage/remote/codec.go +++ b/storage/remote/codec.go @@ -254,7 +254,7 @@ func StreamChunkedReadResponses( } // encodeChunks expects iterator to be ready to use (aka iter.Next() called before invoking). -func encodeChunks(iter storage.SeriesIterator, chks []prompb.Chunk, frameBytesLeft int) ([]prompb.Chunk, error) { +func encodeChunks(iter chunkenc.Iterator, chks []prompb.Chunk, frameBytesLeft int) ([]prompb.Chunk, error) { const maxSamplesInChunk = 120 var ( @@ -392,7 +392,7 @@ func (c *concreteSeries) Labels() labels.Labels { return labels.New(c.labels...) } -func (c *concreteSeries) Iterator() storage.SeriesIterator { +func (c *concreteSeries) Iterator() chunkenc.Iterator { return newConcreteSeriersIterator(c) } @@ -402,7 +402,7 @@ type concreteSeriesIterator struct { series *concreteSeries } -func newConcreteSeriersIterator(series *concreteSeries) storage.SeriesIterator { +func newConcreteSeriersIterator(series *concreteSeries) chunkenc.Iterator { return &concreteSeriesIterator{ cur: -1, series: series, diff --git a/storage/remote/storage.go b/storage/remote/storage.go index dd486b83c4..81883e4fe9 100644 --- a/storage/remote/storage.go +++ b/storage/remote/storage.go @@ -149,7 +149,7 @@ func (s *Storage) Querier(ctx context.Context, mint, maxt int64) (storage.Querie } // Appender implements storage.Storage. -func (s *Storage) Appender() (storage.Appender, error) { +func (s *Storage) Appender() storage.Appender { return s.rws.Appender() } diff --git a/storage/remote/write.go b/storage/remote/write.go index c2a0a5d53f..665eb2b071 100644 --- a/storage/remote/write.go +++ b/storage/remote/write.go @@ -183,10 +183,10 @@ func (rws *WriteStorage) ApplyConfig(conf *config.Config) error { } // Appender implements storage.Storage. -func (rws *WriteStorage) Appender() (storage.Appender, error) { +func (rws *WriteStorage) Appender() storage.Appender { return ×tampTracker{ writeStorage: rws, - }, nil + } } // Close closes the WriteStorage. @@ -206,7 +206,7 @@ type timestampTracker struct { } // Add implements storage.Appender. -func (t *timestampTracker) Add(_ labels.Labels, ts int64, v float64) (uint64, error) { +func (t *timestampTracker) Add(_ labels.Labels, ts int64, _ float64) (uint64, error) { t.samples++ if ts > t.highestTimestamp { t.highestTimestamp = ts @@ -215,8 +215,8 @@ func (t *timestampTracker) Add(_ labels.Labels, ts int64, v float64) (uint64, er } // AddFast implements storage.Appender. -func (t *timestampTracker) AddFast(l labels.Labels, _ uint64, ts int64, v float64) error { - _, err := t.Add(l, ts, v) +func (t *timestampTracker) AddFast(_ uint64, ts int64, v float64) error { + _, err := t.Add(nil, ts, v) return err } diff --git a/storage/tsdb/tsdb.go b/storage/tsdb/tsdb.go deleted file mode 100644 index 1fa3557612..0000000000 --- a/storage/tsdb/tsdb.go +++ /dev/null @@ -1,323 +0,0 @@ -// Copyright 2017 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 tsdb - -import ( - "context" - "sync" - "time" - - "github.com/alecthomas/units" - "github.com/go-kit/kit/log" - "github.com/pkg/errors" - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/common/model" - "github.com/prometheus/prometheus/pkg/labels" - "github.com/prometheus/prometheus/storage" - "github.com/prometheus/prometheus/tsdb" -) - -// ErrNotReady is returned if the underlying storage is not ready yet. -var ErrNotReady = errors.New("TSDB not ready") - -// ReadyStorage implements the Storage interface while allowing to set the actual -// storage at a later point in time. -type ReadyStorage struct { - mtx sync.RWMutex - a *adapter -} - -// Set the storage. -func (s *ReadyStorage) Set(db *tsdb.DB, startTimeMargin int64) { - s.mtx.Lock() - defer s.mtx.Unlock() - - s.a = &adapter{db: db, startTimeMargin: startTimeMargin} -} - -// Get the storage. -func (s *ReadyStorage) Get() *tsdb.DB { - if x := s.get(); x != nil { - return x.db - } - return nil -} - -func (s *ReadyStorage) get() *adapter { - s.mtx.RLock() - x := s.a - s.mtx.RUnlock() - return x -} - -// StartTime implements the Storage interface. -func (s *ReadyStorage) StartTime() (int64, error) { - if x := s.get(); x != nil { - return x.StartTime() - } - return int64(model.Latest), ErrNotReady -} - -// Querier implements the Storage interface. -func (s *ReadyStorage) Querier(ctx context.Context, mint, maxt int64) (storage.Querier, error) { - if x := s.get(); x != nil { - return x.Querier(ctx, mint, maxt) - } - return nil, ErrNotReady -} - -// Appender implements the Storage interface. -func (s *ReadyStorage) Appender() (storage.Appender, error) { - if x := s.get(); x != nil { - return x.Appender() - } - return nil, ErrNotReady -} - -// Close implements the Storage interface. -func (s *ReadyStorage) Close() error { - if x := s.Get(); x != nil { - return x.Close() - } - return nil -} - -// Adapter return an adapter as storage.Storage. -func Adapter(db *tsdb.DB, startTimeMargin int64) storage.Storage { - return &adapter{db: db, startTimeMargin: startTimeMargin} -} - -// adapter implements a storage.Storage around TSDB. -type adapter struct { - db *tsdb.DB - startTimeMargin int64 -} - -// Options of the DB storage. -type Options struct { - // The timestamp range of head blocks after which they get persisted. - // It's the minimum duration of any persisted block. - MinBlockDuration model.Duration - - // The maximum timestamp range of compacted blocks. - MaxBlockDuration model.Duration - - // The maximum size of each WAL segment file. - WALSegmentSize units.Base2Bytes - - // Duration for how long to retain data. - RetentionDuration model.Duration - - // Maximum number of bytes to be retained. - MaxBytes units.Base2Bytes - - // Disable creation and consideration of lockfile. - NoLockfile bool - - // When true it disables the overlapping blocks check. - // This in-turn enables vertical compaction and vertical query merge. - AllowOverlappingBlocks bool - - // When true records in the WAL will be compressed. - WALCompression bool -} - -var ( - startTime prometheus.GaugeFunc - headMaxTime prometheus.GaugeFunc - headMinTime prometheus.GaugeFunc -) - -func registerMetrics(db *tsdb.DB, r prometheus.Registerer) { - - startTime = prometheus.NewGaugeFunc(prometheus.GaugeOpts{ - Name: "prometheus_tsdb_lowest_timestamp_seconds", - Help: "Lowest timestamp value stored in the database.", - }, func() float64 { - bb := db.Blocks() - if len(bb) == 0 { - return float64(db.Head().MinTime()) / 1000 - } - return float64(db.Blocks()[0].Meta().MinTime) / 1000 - }) - headMinTime = prometheus.NewGaugeFunc(prometheus.GaugeOpts{ - Name: "prometheus_tsdb_head_min_time_seconds", - Help: "Minimum time bound of the head block.", - }, func() float64 { - return float64(db.Head().MinTime()) / 1000 - }) - headMaxTime = prometheus.NewGaugeFunc(prometheus.GaugeOpts{ - Name: "prometheus_tsdb_head_max_time_seconds", - Help: "Maximum timestamp of the head block.", - }, func() float64 { - return float64(db.Head().MaxTime()) / 1000 - }) - - if r != nil { - r.MustRegister( - startTime, - headMaxTime, - headMinTime, - ) - } -} - -// Open returns a new storage backed by a TSDB database that is configured for Prometheus. -func Open(path string, l log.Logger, r prometheus.Registerer, opts *Options) (*tsdb.DB, error) { - if opts.MinBlockDuration > opts.MaxBlockDuration { - opts.MaxBlockDuration = opts.MinBlockDuration - } - // Start with smallest block duration and create exponential buckets until the exceed the - // configured maximum block duration. - rngs := tsdb.ExponentialBlockRanges(int64(time.Duration(opts.MinBlockDuration).Seconds()*1000), 10, 3) - - for i, v := range rngs { - if v > int64(time.Duration(opts.MaxBlockDuration).Seconds()*1000) { - rngs = rngs[:i] - break - } - } - - db, err := tsdb.Open(path, l, r, &tsdb.Options{ - WALSegmentSize: int(opts.WALSegmentSize), - RetentionDuration: uint64(time.Duration(opts.RetentionDuration).Seconds() * 1000), - MaxBytes: int64(opts.MaxBytes), - BlockRanges: rngs, - NoLockfile: opts.NoLockfile, - AllowOverlappingBlocks: opts.AllowOverlappingBlocks, - WALCompression: opts.WALCompression, - }) - if err != nil { - return nil, err - } - registerMetrics(db, r) - - return db, nil -} - -// StartTime implements the Storage interface. -func (a adapter) StartTime() (int64, error) { - var startTime int64 - - if len(a.db.Blocks()) > 0 { - startTime = a.db.Blocks()[0].Meta().MinTime - } else { - startTime = time.Now().Unix() * 1000 - } - - // Add a safety margin as it may take a few minutes for everything to spin up. - return startTime + a.startTimeMargin, nil -} - -func (a adapter) Querier(_ context.Context, mint, maxt int64) (storage.Querier, error) { - q, err := a.db.Querier(mint, maxt) - if err != nil { - return nil, err - } - return querier{q: q}, nil -} - -// Appender returns a new appender against the storage. -func (a adapter) Appender() (storage.Appender, error) { - return appender{a: a.db.Appender()}, nil -} - -// Close closes the storage and all its underlying resources. -func (a adapter) Close() error { - return a.db.Close() -} - -type querier struct { - q tsdb.Querier -} - -func (q querier) Select(_ *storage.SelectParams, ms ...*labels.Matcher) (storage.SeriesSet, storage.Warnings, error) { - set, err := q.q.Select(ms...) - if err != nil { - return nil, nil, err - } - return seriesSet{set: set}, nil, nil -} - -func (q querier) SelectSorted(_ *storage.SelectParams, ms ...*labels.Matcher) (storage.SeriesSet, storage.Warnings, error) { - set, err := q.q.SelectSorted(ms...) - if err != nil { - return nil, nil, err - } - return seriesSet{set: set}, nil, nil -} - -func (q querier) LabelValues(name string) ([]string, storage.Warnings, error) { - v, err := q.q.LabelValues(name) - return v, nil, err -} -func (q querier) LabelNames() ([]string, storage.Warnings, error) { - v, err := q.q.LabelNames() - return v, nil, err -} -func (q querier) Close() error { return q.q.Close() } - -type seriesSet struct { - set tsdb.SeriesSet -} - -func (s seriesSet) Next() bool { return s.set.Next() } -func (s seriesSet) Err() error { return s.set.Err() } -func (s seriesSet) At() storage.Series { return series{s: s.set.At()} } - -type series struct { - s tsdb.Series -} - -func (s series) Labels() labels.Labels { return s.s.Labels() } -func (s series) Iterator() storage.SeriesIterator { return s.s.Iterator() } - -type appender struct { - a tsdb.Appender -} - -func (a appender) Add(lset labels.Labels, t int64, v float64) (uint64, error) { - ref, err := a.a.Add(lset, t, v) - - switch errors.Cause(err) { - case tsdb.ErrNotFound: - return 0, storage.ErrNotFound - case tsdb.ErrOutOfOrderSample: - return 0, storage.ErrOutOfOrderSample - case tsdb.ErrAmendSample: - return 0, storage.ErrDuplicateSampleForTimestamp - case tsdb.ErrOutOfBounds: - return 0, storage.ErrOutOfBounds - } - return ref, err -} - -func (a appender) AddFast(_ labels.Labels, ref uint64, t int64, v float64) error { - err := a.a.AddFast(ref, t, v) - - switch errors.Cause(err) { - case tsdb.ErrNotFound: - return storage.ErrNotFound - case tsdb.ErrOutOfOrderSample: - return storage.ErrOutOfOrderSample - case tsdb.ErrAmendSample: - return storage.ErrDuplicateSampleForTimestamp - case tsdb.ErrOutOfBounds: - return storage.ErrOutOfBounds - } - return err -} - -func (a appender) Commit() error { return a.a.Commit() } -func (a appender) Rollback() error { return a.a.Rollback() } diff --git a/storage/tsdb/tsdb_export_test.go b/storage/tsdb/tsdb_export_test.go deleted file mode 100644 index 5821e94256..0000000000 --- a/storage/tsdb/tsdb_export_test.go +++ /dev/null @@ -1,21 +0,0 @@ -// Copyright 2017 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 tsdb - -// Export the internal variables only for tests. -var ( - StartTime = &startTime - HeadMaxTime = &headMaxTime - HeadMinTime = &headMinTime -) diff --git a/storage/tsdb/tsdb_test.go b/storage/tsdb/tsdb_test.go deleted file mode 100644 index c7813a0ea8..0000000000 --- a/storage/tsdb/tsdb_test.go +++ /dev/null @@ -1,64 +0,0 @@ -// Copyright 2017 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 tsdb_test - -import ( - "testing" - - dto "github.com/prometheus/client_model/go" - "github.com/prometheus/common/model" - "github.com/prometheus/prometheus/pkg/labels" - "github.com/prometheus/prometheus/storage/tsdb" - "github.com/prometheus/prometheus/util/teststorage" - "github.com/prometheus/prometheus/util/testutil" -) - -func TestMetrics(t *testing.T) { - db := teststorage.New(t) - defer db.Close() - - metrics := &dto.Metric{} - startTime := *tsdb.StartTime - headMinTime := *tsdb.HeadMinTime - headMaxTime := *tsdb.HeadMaxTime - - // Check initial values. - testutil.Ok(t, startTime.Write(metrics)) - testutil.Equals(t, float64(model.Latest)/1000, metrics.Gauge.GetValue()) - - testutil.Ok(t, headMinTime.Write(metrics)) - testutil.Equals(t, float64(model.Latest)/1000, metrics.Gauge.GetValue()) - - testutil.Ok(t, headMaxTime.Write(metrics)) - testutil.Equals(t, float64(model.Earliest)/1000, metrics.Gauge.GetValue()) - - app, err := db.Appender() - testutil.Ok(t, err) - - app.Add(labels.FromStrings(model.MetricNameLabel, "a"), 1, 1) - app.Add(labels.FromStrings(model.MetricNameLabel, "a"), 2, 1) - app.Add(labels.FromStrings(model.MetricNameLabel, "a"), 3, 1) - testutil.Ok(t, app.Commit()) - - // Check after adding some samples. - testutil.Ok(t, startTime.Write(metrics)) - testutil.Equals(t, 0.001, metrics.Gauge.GetValue()) - - testutil.Ok(t, headMinTime.Write(metrics)) - testutil.Equals(t, 0.001, metrics.Gauge.GetValue()) - - testutil.Ok(t, headMaxTime.Write(metrics)) - testutil.Equals(t, 0.003, metrics.Gauge.GetValue()) - -} diff --git a/tsdb/block.go b/tsdb/block.go index 48ebb348b3..e302ce8fa1 100644 --- a/tsdb/block.go +++ b/tsdb/block.go @@ -124,12 +124,6 @@ type BlockReader interface { Meta() BlockMeta } -// Appendable defines an entity to which data can be appended. -type Appendable interface { - // Appender returns a new Appender against an underlying store. - Appender() Appender -} - // BlockMeta provides meta information about a block. type BlockMeta struct { // Unique identifier for the block and its contents. Changes on compaction. diff --git a/tsdb/block_test.go b/tsdb/block_test.go index 3a2fe3af91..d1ef6c524f 100644 --- a/tsdb/block_test.go +++ b/tsdb/block_test.go @@ -28,6 +28,7 @@ import ( "github.com/go-kit/kit/log" "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/fileutil" "github.com/prometheus/prometheus/tsdb/tsdbutil" @@ -179,7 +180,7 @@ func TestCorruptedChunk(t *testing.T) { }() series := newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{sample{1, 1}}) - blockDir := createBlock(t, tmpdir, []Series{series}) + blockDir := createBlock(t, tmpdir, []storage.Series{series}) files, err := sequenceFiles(chunkDir(blockDir)) testutil.Ok(t, err) testutil.Assert(t, len(files) > 0, "No chunk created.") @@ -202,8 +203,9 @@ func TestCorruptedChunk(t *testing.T) { querier, err := NewBlockQuerier(b, 0, 1) testutil.Ok(t, err) defer func() { testutil.Ok(t, querier.Close()) }() - set, err := querier.Select(labels.MustNewMatcher(labels.MatchEqual, "a", "b")) + set, ws, err := querier.Select(nil, labels.MustNewMatcher(labels.MatchEqual, "a", "b")) testutil.Ok(t, err) + testutil.Equals(t, 0, len(ws)) // Check query err. testutil.Equals(t, false, set.Next()) @@ -302,7 +304,7 @@ func TestReadIndexFormatV1(t *testing.T) { } // createBlock creates a block with given set of series and returns its dir. -func createBlock(tb testing.TB, dir string, series []Series) string { +func createBlock(tb testing.TB, dir string, series []storage.Series) string { return createBlockFromHead(tb, dir, createHead(tb, series)) } @@ -319,7 +321,7 @@ func createBlockFromHead(tb testing.TB, dir string, head *Head) string { return filepath.Join(dir, ulid.String()) } -func createHead(tb testing.TB, series []Series) *Head { +func createHead(tb testing.TB, series []storage.Series) *Head { head, err := NewHead(nil, nil, nil, 2*60*60*1000, DefaultStripeSize) testutil.Ok(tb, err) defer head.Close() @@ -352,12 +354,12 @@ const ( ) // genSeries generates series with a given number of labels and values. -func genSeries(totalSeries, labelCount int, mint, maxt int64) []Series { +func genSeries(totalSeries, labelCount int, mint, maxt int64) []storage.Series { if totalSeries == 0 || labelCount == 0 { return nil } - series := make([]Series, totalSeries) + series := make([]storage.Series, totalSeries) for i := 0; i < totalSeries; i++ { lbls := make(map[string]string, labelCount) @@ -375,12 +377,12 @@ func genSeries(totalSeries, labelCount int, mint, maxt int64) []Series { } // populateSeries generates series from given labels, mint and maxt. -func populateSeries(lbls []map[string]string, mint, maxt int64) []Series { +func populateSeries(lbls []map[string]string, mint, maxt int64) []storage.Series { if len(lbls) == 0 { return nil } - series := make([]Series, 0, len(lbls)) + series := make([]storage.Series, 0, len(lbls)) for _, lbl := range lbls { if len(lbl) == 0 { continue diff --git a/tsdb/chunkenc/chunk.go b/tsdb/chunkenc/chunk.go index 5f228df48c..e7cb16ede8 100644 --- a/tsdb/chunkenc/chunk.go +++ b/tsdb/chunkenc/chunk.go @@ -15,6 +15,7 @@ package chunkenc import ( "fmt" + "math" "sync" "github.com/pkg/errors" @@ -72,10 +73,22 @@ type Appender interface { } // Iterator is a simple iterator that can only get the next value. +// Iterator iterates over the samples of a time series. type Iterator interface { + // Seek advances the iterator forward to the sample with the timestamp t or first value after t. + // Seek has no effect if requested timestamp is the same or lower than the current iterator position. + // Seek returns false if there is no such sample with the timestamp equal or larger than t. + // Iterator is exhausted when the Seek returns false. + // TODO(bwplotka): Verify above statements on all implementations with unit test. + Seek(t int64) bool + // At returns the current timestamp/value pair. + // At returns (math.MinInt64, 0.0) before the iterator has advanced. + // TODO(bwplotka): Verify above statement on all implementations with unit test. At() (int64, float64) - Err() error + // Next advances the iterator by one. Next() bool + // Err returns the current error. + Err() error } // NewNopIterator returns a new chunk iterator that does not hold any data. @@ -85,7 +98,8 @@ func NewNopIterator() Iterator { type nopIterator struct{} -func (nopIterator) At() (int64, float64) { return 0, 0 } +func (nopIterator) Seek(int64) bool { return false } +func (nopIterator) At() (int64, float64) { return math.MinInt64, 0 } func (nopIterator) Next() bool { return false } func (nopIterator) Err() error { return nil } diff --git a/tsdb/chunkenc/chunk_test.go b/tsdb/chunkenc/chunk_test.go index bd344882ae..beda72e276 100644 --- a/tsdb/chunkenc/chunk_test.go +++ b/tsdb/chunkenc/chunk_test.go @@ -17,7 +17,6 @@ import ( "fmt" "io" "math/rand" - "reflect" "testing" "github.com/prometheus/prometheus/util/testutil" @@ -35,19 +34,15 @@ func TestChunk(t *testing.T) { t.Run(fmt.Sprintf("%v", enc), func(t *testing.T) { for range make([]struct{}, 1) { c := nc() - if err := testChunk(c); err != nil { - t.Fatal(err) - } + testChunk(t, c) } }) } } -func testChunk(c Chunk) error { +func testChunk(t *testing.T, c Chunk) { app, err := c.Appender() - if err != nil { - return err - } + testutil.Ok(t, err) var exp []pair var ( @@ -56,7 +51,6 @@ func testChunk(c Chunk) error { ) for i := 0; i < 300; i++ { ts += int64(rand.Intn(10000) + 1) - // v = rand.Float64() if i%2 == 0 { v += float64(rand.Intn(1000000)) } else { @@ -67,29 +61,52 @@ func testChunk(c Chunk) error { // appending to a partially filled chunk. if i%10 == 0 { app, err = c.Appender() - if err != nil { - return err - } + testutil.Ok(t, err) } app.Append(ts, v) exp = append(exp, pair{t: ts, v: v}) - // fmt.Println("appended", len(c.Bytes()), c.Bytes()) } - it := c.Iterator(nil) - var res []pair - for it.Next() { - ts, v := it.At() - res = append(res, pair{t: ts, v: v}) + // 1. Expand iterator in simple case. + it1 := c.Iterator(nil) + var res1 []pair + for it1.Next() { + ts, v := it1.At() + res1 = append(res1, pair{t: ts, v: v}) } - if it.Err() != nil { - return it.Err() + testutil.Ok(t, it1.Err()) + testutil.Equals(t, exp, res1) + + // 2. Expand second iterator while reusing first one. + it2 := c.Iterator(it1) + var res2 []pair + for it2.Next() { + ts, v := it2.At() + res2 = append(res2, pair{t: ts, v: v}) } - if !reflect.DeepEqual(exp, res) { - return fmt.Errorf("unexpected result\n\ngot: %v\n\nexp: %v", res, exp) + testutil.Ok(t, it2.Err()) + testutil.Equals(t, exp, res2) + + // 3. Test Iterator Seek. + mid := len(exp) / 2 + + it3 := c.Iterator(nil) + var res3 []pair + testutil.Equals(t, true, it3.Seek(exp[mid].t)) + // Below ones should not matter. + testutil.Equals(t, true, it3.Seek(exp[mid].t)) + testutil.Equals(t, true, it3.Seek(exp[mid].t)) + ts, v = it3.At() + res3 = append(res3, pair{t: ts, v: v}) + + for it3.Next() { + ts, v := it3.At() + res3 = append(res3, pair{t: ts, v: v}) } - return nil + testutil.Ok(t, it3.Err()) + testutil.Equals(t, exp[mid:], res3) + testutil.Equals(t, false, it3.Seek(exp[len(exp)-1].t+1)) } func benchmarkIterator(b *testing.B, newChunk func() Chunk) { diff --git a/tsdb/chunkenc/xor.go b/tsdb/chunkenc/xor.go index 5401261d9d..2f76c08ca3 100644 --- a/tsdb/chunkenc/xor.go +++ b/tsdb/chunkenc/xor.go @@ -253,6 +253,19 @@ type xorIterator struct { err error } +func (it *xorIterator) Seek(t int64) bool { + if it.err != nil { + return false + } + + for t > it.t || it.numRead == 0 { + if !it.Next() { + return false + } + } + return true +} + func (it *xorIterator) At() (int64, float64) { return it.t, it.val } diff --git a/tsdb/cmd/tsdb/main.go b/tsdb/cmd/tsdb/main.go index 3b26493f6e..236ec437d6 100644 --- a/tsdb/cmd/tsdb/main.go +++ b/tsdb/cmd/tsdb/main.go @@ -15,6 +15,7 @@ package main import ( "bufio" + "context" "fmt" "io" "io/ioutil" @@ -32,11 +33,12 @@ import ( "github.com/go-kit/kit/log" "github.com/pkg/errors" + "github.com/prometheus/common/model" "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/tsdb" "github.com/prometheus/prometheus/tsdb/chunks" tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" - kingpin "gopkg.in/alecthomas/kingpin.v2" + "gopkg.in/alecthomas/kingpin.v2" ) func main() { @@ -176,8 +178,8 @@ func (b *writeBenchmark) run() error { l := log.With(b.logger, "ts", log.DefaultTimestampUTC, "caller", log.DefaultCaller) st, err := tsdb.Open(dir, l, nil, &tsdb.Options{ - RetentionDuration: 15 * 24 * 60 * 60 * 1000, // 15 days in milliseconds - BlockRanges: tsdb.ExponentialBlockRanges(2*60*60*1000, 5, 3), + RetentionDuration: 15 * 24 * model.Duration(time.Hour), + MinBlockDuration: 2 * model.Duration(time.Hour), }) if err != nil { return err @@ -604,8 +606,7 @@ func analyzeBlock(b tsdb.BlockReader, limit int) error { } func dumpSamples(db *tsdb.DBReadOnly, mint, maxt int64) (err error) { - - q, err := db.Querier(mint, maxt) + q, err := db.Querier(context.TODO(), mint, maxt) if err != nil { return err } @@ -616,11 +617,19 @@ func dumpSamples(db *tsdb.DBReadOnly, mint, maxt int64) (err error) { err = merr.Err() }() - ss, err := q.Select(labels.MustNewMatcher(labels.MatchRegexp, "", ".*")) + ss, ws, err := q.Select(nil, labels.MustNewMatcher(labels.MatchRegexp, "", ".*")) if err != nil { return err } + if len(ws) > 0 { + var merr tsdb_errors.MultiError + for _, w := range ws { + merr.Add(w) + } + return merr.Err() + } + for ss.Next() { series := ss.At() labels := series.Labels() diff --git a/tsdb/compact.go b/tsdb/compact.go index 47151dad6a..2d7376e1af 100644 --- a/tsdb/compact.go +++ b/tsdb/compact.go @@ -30,6 +30,7 @@ import ( "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" "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" @@ -649,7 +650,7 @@ func (c *LeveledCompactor) populateBlock(blocks []BlockReader, meta *BlockMeta, } var ( - set ChunkSeriesSet + set storage.ChunkSeriesSet symbols index.StringIter closers = []io.Closer{} overlapping bool @@ -916,7 +917,7 @@ func (c *compactionSeriesSet) At() (labels.Labels, []chunks.Meta, tombstones.Int } type compactionMerger struct { - a, b ChunkSeriesSet + a, b storage.ChunkSeriesSet aok, bok bool l labels.Labels @@ -924,7 +925,7 @@ type compactionMerger struct { intervals tombstones.Intervals } -func newCompactionMerger(a, b ChunkSeriesSet) (*compactionMerger, error) { +func newCompactionMerger(a, b storage.ChunkSeriesSet) (*compactionMerger, error) { c := &compactionMerger{ a: a, b: b, diff --git a/tsdb/compact_test.go b/tsdb/compact_test.go index 515c3f4ef5..71b1ab48c1 100644 --- a/tsdb/compact_test.go +++ b/tsdb/compact_test.go @@ -895,13 +895,13 @@ func BenchmarkCompactionFromHead(b *testing.B) { // This is needed for unit tests that rely on // checking state before and after a compaction. func TestDisableAutoCompactions(t *testing.T) { - db, delete := openTestDB(t, nil) + db, delete := openTestDB(t, nil, nil) defer func() { testutil.Ok(t, db.Close()) delete() }() - blockRange := DefaultOptions.BlockRanges[0] + blockRange := db.compactor.(*LeveledCompactor).ranges[0] label := labels.FromStrings("foo", "bar") // Trigger a compaction to check that it was skipped and @@ -971,7 +971,7 @@ func TestCancelCompactions(t *testing.T) { // Measure the compaction time without interrupting it. var timeCompactionUninterrupted time.Duration { - db, err := Open(tmpdir, log.NewNopLogger(), nil, &Options{BlockRanges: []int64{1, 2000}}) + db, err := open(tmpdir, log.NewNopLogger(), nil, DefaultOptions(), []int64{1, 2000}) testutil.Ok(t, err) testutil.Equals(t, 3, len(db.Blocks()), "initial block count mismatch") testutil.Equals(t, 0.0, prom_testutil.ToFloat64(db.compactor.(*LeveledCompactor).metrics.ran), "initial compaction counter mismatch") @@ -991,7 +991,7 @@ func TestCancelCompactions(t *testing.T) { } // Measure the compaction time when closing the db in the middle of compaction. { - db, err := Open(tmpdirCopy, log.NewNopLogger(), nil, &Options{BlockRanges: []int64{1, 2000}}) + db, err := open(tmpdirCopy, log.NewNopLogger(), nil, nil, []int64{1, 2000}) testutil.Ok(t, err) testutil.Equals(t, 3, len(db.Blocks()), "initial block count mismatch") testutil.Equals(t, 0.0, prom_testutil.ToFloat64(db.compactor.(*LeveledCompactor).metrics.ran), "initial compaction counter mismatch") @@ -1017,10 +1017,9 @@ func TestCancelCompactions(t *testing.T) { // TestDeleteCompactionBlockAfterFailedReload ensures that a failed reload immediately after a compaction // deletes the resulting block to avoid creatings blocks with the same time range. func TestDeleteCompactionBlockAfterFailedReload(t *testing.T) { - tests := map[string]func(*DB) int{ "Test Head Compaction": func(db *DB) int { - rangeToTriggerCompaction := db.opts.BlockRanges[0]/2*3 - 1 + rangeToTriggerCompaction := db.compactor.(*LeveledCompactor).ranges[0]/2*3 - 1 defaultLabel := labels.FromStrings("foo", "bar") // Add some data to the head that is enough to trigger a compaction. @@ -1053,9 +1052,7 @@ func TestDeleteCompactionBlockAfterFailedReload(t *testing.T) { for title, bootStrap := range tests { t.Run(title, func(t *testing.T) { - db, delete := openTestDB(t, &Options{ - BlockRanges: []int64{1, 100}, - }) + db, delete := openTestDB(t, nil, []int64{1, 100}) defer func() { testutil.Ok(t, db.Close()) delete() diff --git a/tsdb/db.go b/tsdb/db.go index 53257b7985..df1d70b492 100644 --- a/tsdb/db.go +++ b/tsdb/db.go @@ -29,12 +29,15 @@ import ( "sync" "time" + "github.com/alecthomas/units" "github.com/go-kit/kit/log" "github.com/go-kit/kit/log/level" "github.com/oklog/ulid" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/common/model" "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunkenc" tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" "github.com/prometheus/prometheus/tsdb/fileutil" @@ -45,19 +48,27 @@ import ( // Default duration of a block in milliseconds - 2h. const ( - DefaultBlockDuration = int64(2 * 60 * 60 * 1000) + DefaultBlockDuration = int64(2 * 60 * 60 * 1000) // 2h in miliseconds. +) + +var ( + // ErrNotReady is returned if the underlying storage is not ready yet. + ErrNotReady = errors.New("TSDB not ready") ) // DefaultOptions used for the DB. They are sane for setups using // millisecond precision timestamps. -var DefaultOptions = &Options{ - WALSegmentSize: wal.DefaultSegmentSize, - RetentionDuration: 15 * 24 * 60 * 60 * 1000, // 15 days in milliseconds - BlockRanges: ExponentialBlockRanges(DefaultBlockDuration, 3, 5), - NoLockfile: false, - AllowOverlappingBlocks: false, - WALCompression: false, - StripeSize: DefaultStripeSize, +func DefaultOptions() *Options { + return &Options{ + WALSegmentSize: wal.DefaultSegmentSize, + RetentionDuration: 15 * 24 * model.Duration(time.Hour), + MinBlockDuration: model.Duration(time.Duration(DefaultBlockDuration) * time.Millisecond), + MaxBlockDuration: model.Duration(time.Duration(DefaultBlockDuration) * time.Millisecond), + NoLockfile: false, + AllowOverlappingBlocks: false, + WALCompression: false, + StripeSize: DefaultStripeSize, + } } // Options of the DB storage. @@ -66,20 +77,17 @@ type Options struct { // WALSegmentSize = 0, segment size is default size. // WALSegmentSize > 0, segment size is WALSegmentSize. // WALSegmentSize < 0, wal is disabled. - WALSegmentSize int + WALSegmentSize units.Base2Bytes // Duration of persisted data to keep. - RetentionDuration uint64 + RetentionDuration model.Duration // Maximum number of bytes in blocks to be retained. // 0 or less means disabled. // NOTE: For proper storage calculations need to consider // the size of the WAL folder which is not added when calculating // the current size of the database. - MaxBytes int64 - - // The sizes of the Blocks. - BlockRanges []int64 + MaxBytes units.Base2Bytes // NoLockfile disables creation and consideration of a lock file. NoLockfile bool @@ -93,31 +101,13 @@ type Options struct { // StripeSize is the size in entries of the series hash map. Reducing the size will save memory but impact performance. StripeSize int -} -// Appender allows appending a batch of data. It must be completed with a -// call to Commit or Rollback and must not be reused afterwards. -// -// Operations on the Appender interface are not goroutine-safe. -type Appender interface { - // Add adds a sample pair for the given series. A reference number is - // returned which can be used to add further samples in the same or later - // transactions. - // Returned reference numbers are ephemeral and may be rejected in calls - // to AddFast() at any point. Adding the sample via Add() returns a new - // reference number. - // If the reference is 0 it must not be used for caching. - Add(l labels.Labels, t int64, v float64) (uint64, error) + // The timestamp range of head blocks after which they get persisted. + // It's the minimum duration of any persisted block. + MinBlockDuration model.Duration - // AddFast adds a sample pair for the referenced series. It is generally - // faster than adding a sample by providing its full label set. - AddFast(ref uint64, t int64, v float64) error - - // Commit submits the collected samples and purges the batch. - Commit() error - - // Rollback rolls back all modifications made in the appender so far. - Rollback() error + // The maximum timestamp range of compacted blocks. + MaxBlockDuration model.Duration } // DB handles reads and writes of time series falling into @@ -168,6 +158,9 @@ type dbMetrics struct { tombCleanTimer prometheus.Histogram blocksBytes prometheus.Gauge maxBytes prometheus.Gauge + minTime prometheus.GaugeFunc + headMaxTime prometheus.GaugeFunc + headMinTime prometheus.GaugeFunc } func newDBMetrics(db *DB, r prometheus.Registerer) *dbMetrics { @@ -245,7 +238,28 @@ func newDBMetrics(db *DB, r prometheus.Registerer) *dbMetrics { Name: "prometheus_tsdb_size_retentions_total", Help: "The number of times that blocks were deleted because the maximum number of bytes was exceeded.", }) - + m.minTime = prometheus.NewGaugeFunc(prometheus.GaugeOpts{ + Name: "prometheus_tsdb_lowest_timestamp_seconds", + Help: "Lowest timestamp value stored in the database.", + }, func() float64 { + bb := db.Blocks() + if len(bb) == 0 { + return float64(db.Head().MinTime()) / 1000 + } + return float64(db.Blocks()[0].Meta().MinTime) / 1000 + }) + m.headMinTime = prometheus.NewGaugeFunc(prometheus.GaugeOpts{ + Name: "prometheus_tsdb_head_min_time_seconds", + Help: "Minimum time bound of the head block.", + }, func() float64 { + return float64(db.Head().MinTime()) / 1000 + }) + m.headMaxTime = prometheus.NewGaugeFunc(prometheus.GaugeOpts{ + Name: "prometheus_tsdb_head_max_time_seconds", + Help: "Maximum timestamp of the head block.", + }, func() float64 { + return float64(db.Head().MaxTime()) / 1000 + }) if r != nil { r.MustRegister( m.loadedBlocks, @@ -261,6 +275,9 @@ func newDBMetrics(db *DB, r prometheus.Registerer) *dbMetrics { m.tombCleanTimer, m.blocksBytes, m.maxBytes, + m.minTime, + m.headMaxTime, + m.headMinTime, ) } return m @@ -329,7 +346,12 @@ func (db *DBReadOnly) FlushWAL(dir string) error { mint: mint, maxt: maxt, } - compactor, err := NewLeveledCompactor(context.Background(), nil, db.logger, DefaultOptions.BlockRanges, chunkenc.NewPool()) + compactor, err := NewLeveledCompactor( + context.Background(), + nil, + db.logger, + ExponentialBlockRanges(time.Duration(DefaultOptions().MinBlockDuration).Milliseconds(), 3, 5), chunkenc.NewPool(), + ) if err != nil { return errors.Wrap(err, "create leveled compactor") } @@ -341,7 +363,7 @@ func (db *DBReadOnly) FlushWAL(dir string) error { // 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(mint, maxt int64) (Querier, error) { +func (db *DBReadOnly) Querier(_ context.Context, mint, maxt int64) (storage.Querier, error) { select { case <-db.closed: return nil, ErrClosed @@ -402,7 +424,7 @@ func (db *DBReadOnly) Querier(mint, maxt int64) (Querier, error) { head: head, } - return dbWritable.Querier(mint, maxt) + return dbWritable.Querier(context.TODO(), mint, maxt) } // Blocks returns a slice of block readers for persisted blocks. @@ -481,20 +503,51 @@ func (db *DBReadOnly) Close() error { return merr.Err() } -// Open returns a new DB in the given directory. +// Open returns a new DB in the given directory. If options are empty, default DefaultOptions will be used. func Open(dir string, l log.Logger, r prometheus.Registerer, opts *Options) (db *DB, err error) { + var rngs []int64 + opts, rngs = validateOpts(opts, nil) + return open(dir, l, r, opts, rngs) +} + +func validateOpts(opts *Options, rngs []int64) (*Options, []int64) { + if opts == nil { + opts = DefaultOptions() + } + if opts.StripeSize <= 0 { + opts.StripeSize = DefaultStripeSize + } + + if opts.MinBlockDuration <= 0 { + opts.MinBlockDuration = model.Duration(time.Duration(DefaultBlockDuration) * time.Millisecond) + } + if opts.MinBlockDuration > opts.MaxBlockDuration { + opts.MaxBlockDuration = opts.MinBlockDuration + } + + if len(rngs) == 0 { + // Start with smallest block duration and create exponential buckets until the exceed the + // configured maximum block duration. + rngs = ExponentialBlockRanges(int64(time.Duration(opts.MinBlockDuration).Seconds()*1000), 10, 3) + } + return opts, rngs +} + +func open(dir string, l log.Logger, r prometheus.Registerer, opts *Options, rngs []int64) (db *DB, err error) { if err := os.MkdirAll(dir, 0777); err != nil { return nil, err } if l == nil { l = log.NewNopLogger() } - if opts == nil { - opts = DefaultOptions - } - if opts.StripeSize <= 0 { - opts.StripeSize = DefaultStripeSize + + for i, v := range rngs { + if v > int64(time.Duration(opts.MaxBlockDuration).Seconds()*1000) { + rngs = rngs[:i] + break + } } + // Fixup bad format written by Prometheus 2.1. if err := repairBadIndexVersion(l, dir); err != nil { return nil, err @@ -535,7 +588,7 @@ func Open(dir string, l log.Logger, r prometheus.Registerer, opts *Options) (db } ctx, cancel := context.WithCancel(context.Background()) - db.compactor, err = NewLeveledCompactor(ctx, r, l, opts.BlockRanges, db.chunkPool) + db.compactor, err = NewLeveledCompactor(ctx, r, l, rngs, db.chunkPool) if err != nil { cancel() return nil, errors.Wrap(err, "create leveled compactor") @@ -548,7 +601,7 @@ func Open(dir string, l log.Logger, r prometheus.Registerer, opts *Options) (db if opts.WALSegmentSize >= 0 { // Wal is set to a custom size. if opts.WALSegmentSize > 0 { - segmentSize = opts.WALSegmentSize + segmentSize = int(opts.WALSegmentSize) } wlog, err = wal.NewSize(l, r, filepath.Join(dir, "wal"), segmentSize, opts.WALCompression) if err != nil { @@ -556,7 +609,7 @@ func Open(dir string, l log.Logger, r prometheus.Registerer, opts *Options) (db } } - db.head, err = NewHead(r, l, wlog, opts.BlockRanges[0], opts.StripeSize) + db.head, err = NewHead(r, l, wlog, rngs[0], opts.StripeSize) if err != nil { return nil, err } @@ -585,6 +638,17 @@ func Open(dir string, l log.Logger, r prometheus.Registerer, opts *Options) (db return db, nil } +// StartTime implements the Storage interface. +func (db *DB) StartTime() (int64, error) { + db.mtx.RLock() + defer db.mtx.RUnlock() + + if len(db.blocks) > 0 { + return db.blocks[0].Meta().MinTime, nil + } + return db.head.MinTime(), nil +} + // Dir returns the directory of the database. func (db *DB) Dir() string { return db.dir @@ -630,14 +694,14 @@ func (db *DB) run() { } // Appender opens a new appender against the database. -func (db *DB) Appender() Appender { +func (db *DB) Appender() storage.Appender { return dbAppender{db: db, Appender: db.head.Appender()} } // dbAppender wraps the DB's head appender and triggers compactions on commit // if necessary. type dbAppender struct { - Appender + storage.Appender db *DB } @@ -948,7 +1012,7 @@ func (db *DB) beyondTimeRetention(blocks []*Block) (deletable map[ulid.ULID]*Blo for i, block := range blocks { // The difference between the first block and this block is larger than // the retention period so any blocks after that are added as deletable. - if i > 0 && blocks[0].Meta().MaxTime-block.Meta().MaxTime > int64(db.opts.RetentionDuration) { + if i > 0 && blocks[0].Meta().MaxTime-block.Meta().MaxTime > (int64(db.opts.RetentionDuration)/int64(time.Millisecond)) { for _, b := range blocks[i:] { deletable[b.meta.ULID] = b } @@ -973,7 +1037,7 @@ func (db *DB) beyondSizeRetention(blocks []*Block) (deletable map[ulid.ULID]*Blo blocksSize := walSize for i, block := range blocks { blocksSize += block.Size() - if blocksSize > db.opts.MaxBytes { + if blocksSize > int64(db.opts.MaxBytes) { // Add this and all following blocks for deletion. for _, b := range blocks[i:] { deletable[b.meta.ULID] = b @@ -1227,7 +1291,7 @@ 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(mint, maxt int64) (Querier, error) { +func (db *DB) Querier(_ context.Context, mint, maxt int64) (storage.Querier, error) { var blocks []BlockReader var blockMetas []BlockMeta @@ -1248,7 +1312,7 @@ func (db *DB) Querier(mint, maxt int64) (Querier, error) { }) } - blockQueriers := make([]Querier, 0, len(blocks)) + blockQueriers := make([]storage.Querier, 0, len(blocks)) for _, b := range blocks { q, err := NewBlockQuerier(b, mint, maxt) if err == nil { @@ -1413,3 +1477,88 @@ func exponential(d, min, max time.Duration) time.Duration { } return d } + +// ReadyStorage implements the Storage interface while allowing to set the actual +// storage at a later point in time. +type ReadyStorage struct { + mtx sync.RWMutex + db *DB + startTimeMargin int64 +} + +// Set the storage. +func (s *ReadyStorage) Set(db *DB, startTimeMargin int64) { + s.mtx.Lock() + defer s.mtx.Unlock() + + s.db = db + s.startTimeMargin = startTimeMargin +} + +// Get the storage. +func (s *ReadyStorage) Get() *DB { + if x := s.get(); x != nil { + return x + } + return nil +} + +func (s *ReadyStorage) get() *DB { + s.mtx.RLock() + x := s.db + s.mtx.RUnlock() + return x +} + +// StartTime implements the Storage interface. +func (s *ReadyStorage) StartTime() (int64, error) { + if x := s.get(); x != nil { + var startTime int64 + + if len(x.Blocks()) > 0 { + startTime = x.Blocks()[0].Meta().MinTime + } else { + startTime = time.Now().Unix() * 1000 + } + // Add a safety margin as it may take a few minutes for everything to spin up. + return startTime + s.startTimeMargin, nil + } + + return int64(model.Latest), ErrNotReady +} + +// Querier implements the Storage interface. +func (s *ReadyStorage) Querier(ctx context.Context, mint, maxt int64) (storage.Querier, error) { + if x := s.get(); x != nil { + return x.Querier(ctx, mint, maxt) + } + return nil, ErrNotReady +} + +// Appender implements the Storage interface. +func (s *ReadyStorage) Appender() storage.Appender { + if x := s.get(); x != nil { + return x.Appender() + } + return notReadyAppender{} +} + +type notReadyAppender struct{} + +func (n notReadyAppender) Add(l labels.Labels, t int64, v float64) (uint64, error) { + return 0, ErrNotReady +} + +func (n notReadyAppender) AddFast(ref uint64, t int64, v float64) error { return ErrNotReady } + +func (n notReadyAppender) Commit() error { return ErrNotReady } + +func (n notReadyAppender) Rollback() error { return ErrNotReady } + +// Close implements the Storage interface. +func (s *ReadyStorage) Close() error { + if x := s.Get(); x != nil { + return x.Close() + } + return nil +} diff --git a/tsdb/db_test.go b/tsdb/db_test.go index 4b965d2b12..c2d5b450fb 100644 --- a/tsdb/db_test.go +++ b/tsdb/db_test.go @@ -14,6 +14,7 @@ package tsdb import ( + "context" "encoding/binary" "fmt" "hash/crc32" @@ -29,15 +30,18 @@ import ( "testing" "time" - "github.com/prometheus/prometheus/tsdb/fileutil" - + "github.com/alecthomas/units" "github.com/go-kit/kit/log" "github.com/oklog/ulid" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" prom_testutil "github.com/prometheus/client_golang/prometheus/testutil" + dto "github.com/prometheus/client_model/go" + "github.com/prometheus/common/model" "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunks" + "github.com/prometheus/prometheus/tsdb/fileutil" "github.com/prometheus/prometheus/tsdb/index" "github.com/prometheus/prometheus/tsdb/record" "github.com/prometheus/prometheus/tsdb/tombstones" @@ -46,11 +50,16 @@ import ( "github.com/prometheus/prometheus/util/testutil" ) -func openTestDB(t testing.TB, opts *Options) (db *DB, close func()) { +func openTestDB(t testing.TB, opts *Options, rngs []int64) (db *DB, close func()) { tmpdir, err := ioutil.TempDir("", "test") testutil.Ok(t, err) - db, err = Open(tmpdir, nil, nil, opts) + if len(rngs) == 0 { + db, err = Open(tmpdir, nil, nil, opts) + } else { + opts, rngs = validateOpts(opts, rngs) + db, err = open(tmpdir, nil, nil, opts, rngs) + } testutil.Ok(t, err) // Do not close the test database by default as it will deadlock on test failures. @@ -60,12 +69,13 @@ func openTestDB(t testing.TB, opts *Options) (db *DB, close func()) { } // query runs a matcher query against the querier and fully expands its data. -func query(t testing.TB, q Querier, matchers ...*labels.Matcher) map[string][]tsdbutil.Sample { - ss, err := q.Select(matchers...) +func query(t testing.TB, q storage.Querier, matchers ...*labels.Matcher) map[string][]tsdbutil.Sample { + ss, ws, err := q.Select(nil, matchers...) defer func() { testutil.Ok(t, q.Close()) }() testutil.Ok(t, err) + testutil.Equals(t, 0, len(ws)) result := map[string][]tsdbutil.Sample{} @@ -91,7 +101,7 @@ func query(t testing.TB, q Querier, matchers ...*labels.Matcher) map[string][]ts // 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) { - db, delete := openTestDB(t, nil) + db, delete := openTestDB(t, nil, nil) defer func() { testutil.Ok(t, db.Close()) delete() @@ -118,7 +128,7 @@ func TestDB_reloadOrder(t *testing.T) { } func TestDataAvailableOnlyAfterCommit(t *testing.T) { - db, delete := openTestDB(t, nil) + db, delete := openTestDB(t, nil, nil) defer func() { testutil.Ok(t, db.Close()) delete() @@ -129,7 +139,7 @@ func TestDataAvailableOnlyAfterCommit(t *testing.T) { _, err := app.Add(labels.FromStrings("foo", "bar"), 0, 0) testutil.Ok(t, err) - querier, err := db.Querier(0, 1) + querier, err := db.Querier(context.TODO(), 0, 1) testutil.Ok(t, err) seriesSet := query(t, querier, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) testutil.Equals(t, map[string][]tsdbutil.Sample{}, seriesSet) @@ -137,7 +147,7 @@ func TestDataAvailableOnlyAfterCommit(t *testing.T) { err = app.Commit() testutil.Ok(t, err) - querier, err = db.Querier(0, 1) + querier, err = db.Querier(context.TODO(), 0, 1) testutil.Ok(t, err) defer querier.Close() @@ -147,7 +157,7 @@ func TestDataAvailableOnlyAfterCommit(t *testing.T) { } func TestDataNotAvailableAfterRollback(t *testing.T) { - db, delete := openTestDB(t, nil) + db, delete := openTestDB(t, nil, nil) defer func() { testutil.Ok(t, db.Close()) delete() @@ -160,7 +170,7 @@ func TestDataNotAvailableAfterRollback(t *testing.T) { err = app.Rollback() testutil.Ok(t, err) - querier, err := db.Querier(0, 1) + querier, err := db.Querier(context.TODO(), 0, 1) testutil.Ok(t, err) defer querier.Close() @@ -170,7 +180,7 @@ func TestDataNotAvailableAfterRollback(t *testing.T) { } func TestDBAppenderAddRef(t *testing.T) { - db, delete := openTestDB(t, nil) + db, delete := openTestDB(t, nil, nil) defer func() { testutil.Ok(t, db.Close()) delete() @@ -208,7 +218,7 @@ func TestDBAppenderAddRef(t *testing.T) { testutil.Ok(t, app2.Commit()) - q, err := db.Querier(0, 200) + q, err := db.Querier(context.TODO(), 0, 200) testutil.Ok(t, err) res := query(t, q, labels.MustNewMatcher(labels.MatchEqual, "a", "b")) @@ -225,7 +235,7 @@ func TestDBAppenderAddRef(t *testing.T) { } func TestAppendEmptyLabelsIgnored(t *testing.T) { - db, delete := openTestDB(t, nil) + db, delete := openTestDB(t, nil, nil) defer func() { testutil.Ok(t, db.Close()) delete() @@ -278,7 +288,7 @@ func TestDeleteSimple(t *testing.T) { Outer: for _, c := range cases { - db, delete := openTestDB(t, nil) + db, delete := openTestDB(t, nil, nil) defer func() { testutil.Ok(t, db.Close()) delete() @@ -301,18 +311,19 @@ Outer: } // Compare the result. - q, err := db.Querier(0, numSamples) + q, err := db.Querier(context.TODO(), 0, numSamples) testutil.Ok(t, err) - res, err := q.Select(labels.MustNewMatcher(labels.MatchEqual, "a", "b")) + res, ws, err := q.Select(nil, labels.MustNewMatcher(labels.MatchEqual, "a", "b")) testutil.Ok(t, err) + testutil.Equals(t, 0, len(ws)) expSamples := make([]tsdbutil.Sample, 0, len(c.remaint)) for _, ts := range c.remaint { expSamples = append(expSamples, sample{ts, smpls[ts]}) } - expss := newMockSeriesSet([]Series{ + expss := newMockSeriesSet([]storage.Series{ newSeries(map[string]string{"a": "b"}, expSamples), }) @@ -338,7 +349,7 @@ Outer: } func TestAmendDatapointCausesError(t *testing.T) { - db, delete := openTestDB(t, nil) + db, delete := openTestDB(t, nil, nil) defer func() { testutil.Ok(t, db.Close()) delete() @@ -356,7 +367,7 @@ func TestAmendDatapointCausesError(t *testing.T) { } func TestDuplicateNaNDatapointNoAmendError(t *testing.T) { - db, delete := openTestDB(t, nil) + db, delete := openTestDB(t, nil, nil) defer func() { testutil.Ok(t, db.Close()) delete() @@ -373,7 +384,7 @@ func TestDuplicateNaNDatapointNoAmendError(t *testing.T) { } func TestNonDuplicateNaNDatapointsCausesAmendError(t *testing.T) { - db, delete := openTestDB(t, nil) + db, delete := openTestDB(t, nil, nil) defer func() { testutil.Ok(t, db.Close()) delete() @@ -389,7 +400,7 @@ func TestNonDuplicateNaNDatapointsCausesAmendError(t *testing.T) { } func TestSkippingInvalidValuesInSameTxn(t *testing.T) { - db, delete := openTestDB(t, nil) + db, delete := openTestDB(t, nil, nil) defer func() { testutil.Ok(t, db.Close()) delete() @@ -404,7 +415,7 @@ func TestSkippingInvalidValuesInSameTxn(t *testing.T) { testutil.Ok(t, app.Commit()) // Make sure the right value is stored. - q, err := db.Querier(0, 10) + q, err := db.Querier(context.TODO(), 0, 10) testutil.Ok(t, err) ssMap := query(t, q, labels.MustNewMatcher(labels.MatchEqual, "a", "b")) @@ -421,7 +432,7 @@ func TestSkippingInvalidValuesInSameTxn(t *testing.T) { testutil.Ok(t, err) testutil.Ok(t, app.Commit()) - q, err = db.Querier(0, 10) + q, err = db.Querier(context.TODO(), 0, 10) testutil.Ok(t, err) ssMap = query(t, q, labels.MustNewMatcher(labels.MatchEqual, "a", "b")) @@ -432,7 +443,7 @@ func TestSkippingInvalidValuesInSameTxn(t *testing.T) { } func TestDB_Snapshot(t *testing.T) { - db, delete := openTestDB(t, nil) + db, delete := openTestDB(t, nil, nil) defer delete() // append data @@ -460,13 +471,14 @@ func TestDB_Snapshot(t *testing.T) { testutil.Ok(t, err) defer func() { testutil.Ok(t, db.Close()) }() - querier, err := db.Querier(mint, mint+1000) + querier, err := db.Querier(context.TODO(), mint, mint+1000) testutil.Ok(t, err) defer func() { testutil.Ok(t, querier.Close()) }() // sum values - seriesSet, err := querier.Select(labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + seriesSet, ws, err := querier.Select(nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) testutil.Ok(t, err) + testutil.Equals(t, 0, len(ws)) sum := 0.0 for seriesSet.Next() { @@ -485,7 +497,7 @@ func TestDB_Snapshot(t *testing.T) { // that are outside the set block time range. // See https://github.com/prometheus/prometheus/issues/5105 func TestDB_Snapshot_ChunksOutsideOfCompactedRange(t *testing.T) { - db, delete := openTestDB(t, nil) + db, delete := openTestDB(t, nil, nil) defer delete() app := db.Appender() @@ -514,13 +526,14 @@ func TestDB_Snapshot_ChunksOutsideOfCompactedRange(t *testing.T) { testutil.Ok(t, err) defer func() { testutil.Ok(t, db.Close()) }() - querier, err := db.Querier(mint, mint+1000) + querier, err := db.Querier(context.TODO(), mint, mint+1000) testutil.Ok(t, err) defer func() { testutil.Ok(t, querier.Close()) }() // Sum values. - seriesSet, err := querier.Select(labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + seriesSet, ws, err := querier.Select(nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) testutil.Ok(t, err) + testutil.Equals(t, 0, len(ws)) sum := 0.0 for seriesSet.Next() { @@ -540,7 +553,7 @@ func TestDB_Snapshot_ChunksOutsideOfCompactedRange(t *testing.T) { func TestDB_SnapshotWithDelete(t *testing.T) { numSamples := int64(10) - db, delete := openTestDB(t, nil) + db, delete := openTestDB(t, nil, nil) defer delete() app := db.Appender() @@ -586,19 +599,20 @@ Outer: defer func() { testutil.Ok(t, db.Close()) }() // Compare the result. - q, err := db.Querier(0, numSamples) + q, err := db.Querier(context.TODO(), 0, numSamples) testutil.Ok(t, err) defer func() { testutil.Ok(t, q.Close()) }() - res, err := q.Select(labels.MustNewMatcher(labels.MatchEqual, "a", "b")) + res, ws, err := q.Select(nil, labels.MustNewMatcher(labels.MatchEqual, "a", "b")) testutil.Ok(t, err) + testutil.Equals(t, 0, len(ws)) expSamples := make([]tsdbutil.Sample, 0, len(c.remaint)) for _, ts := range c.remaint { expSamples = append(expSamples, sample{ts, smpls[ts]}) } - expss := newMockSeriesSet([]Series{ + expss := newMockSeriesSet([]storage.Series{ newSeries(map[string]string{"a": "b"}, expSamples), }) @@ -683,7 +697,7 @@ func TestDB_e2e(t *testing.T) { seriesMap[labels.New(l...).String()] = []tsdbutil.Sample{} } - db, delete := openTestDB(t, nil) + db, delete := openTestDB(t, nil, nil) defer func() { testutil.Ok(t, db.Close()) delete() @@ -760,11 +774,12 @@ func TestDB_e2e(t *testing.T) { } } - q, err := db.Querier(mint, maxt) + q, err := db.Querier(context.TODO(), mint, maxt) testutil.Ok(t, err) - ss, err := q.Select(qry.ms...) + ss, ws, err := q.Select(nil, qry.ms...) testutil.Ok(t, err) + testutil.Equals(t, 0, len(ws)) result := map[string][]tsdbutil.Sample{} @@ -788,7 +803,7 @@ func TestDB_e2e(t *testing.T) { } func TestWALFlushedOnDBClose(t *testing.T) { - db, delete := openTestDB(t, nil) + db, delete := openTestDB(t, nil, nil) defer delete() dirDb := db.Dir() @@ -806,11 +821,12 @@ func TestWALFlushedOnDBClose(t *testing.T) { testutil.Ok(t, err) defer func() { testutil.Ok(t, db.Close()) }() - q, err := db.Querier(0, 1) + q, err := db.Querier(context.TODO(), 0, 1) testutil.Ok(t, err) - values, err := q.LabelValues("labelname") + values, ws, err := q.LabelValues("labelname") testutil.Ok(t, err) + testutil.Equals(t, 0, len(ws)) testutil.Equals(t, []string{"labelvalue"}, values) } @@ -821,10 +837,10 @@ func TestWALSegmentSizeOptions(t *testing.T) { files, err := ioutil.ReadDir(filepath.Join(dbDir, "wal")) testutil.Ok(t, err) for _, f := range files[:len(files)-1] { - testutil.Equals(t, int64(DefaultOptions.WALSegmentSize), f.Size(), "WAL file size doesn't match WALSegmentSize option, filename: %v", f.Name()) + testutil.Equals(t, int64(DefaultOptions().WALSegmentSize), f.Size(), "WAL file size doesn't match WALSegmentSize option, filename: %v", f.Name()) } lastFile := files[len(files)-1] - testutil.Assert(t, int64(DefaultOptions.WALSegmentSize) > lastFile.Size(), "last WAL file size is not smaller than the WALSegmentSize option, filename: %v", lastFile.Name()) + testutil.Assert(t, int64(DefaultOptions().WALSegmentSize) > lastFile.Size(), "last WAL file size is not smaller than the WALSegmentSize option, filename: %v", lastFile.Name()) }, // Custom Wal Size. 2 * 32 * 1024: func(dbDir string, segmentSize int) { @@ -846,9 +862,9 @@ func TestWALSegmentSizeOptions(t *testing.T) { } for segmentSize, testFunc := range tests { t.Run(fmt.Sprintf("WALSegmentSize %d test", segmentSize), func(t *testing.T) { - options := *DefaultOptions - options.WALSegmentSize = segmentSize - db, delete := openTestDB(t, &options) + opts := DefaultOptions() + opts.WALSegmentSize = units.Base2Bytes(segmentSize) + db, delete := openTestDB(t, opts, nil) defer delete() app := db.Appender() for i := int64(0); i < 155; i++ { @@ -859,7 +875,7 @@ func TestWALSegmentSizeOptions(t *testing.T) { dbDir := db.Dir() db.Close() - testFunc(dbDir, options.WALSegmentSize) + testFunc(dbDir, int(opts.WALSegmentSize)) }) } } @@ -867,7 +883,7 @@ func TestWALSegmentSizeOptions(t *testing.T) { func TestTombstoneClean(t *testing.T) { numSamples := int64(10) - db, delete := openTestDB(t, nil) + db, delete := openTestDB(t, nil, nil) defer delete() app := db.Appender() @@ -915,19 +931,20 @@ func TestTombstoneClean(t *testing.T) { testutil.Ok(t, db.CleanTombstones()) // Compare the result. - q, err := db.Querier(0, numSamples) + q, err := db.Querier(context.TODO(), 0, numSamples) testutil.Ok(t, err) defer q.Close() - res, err := q.Select(labels.MustNewMatcher(labels.MatchEqual, "a", "b")) + res, ws, err := q.Select(nil, labels.MustNewMatcher(labels.MatchEqual, "a", "b")) testutil.Ok(t, err) + testutil.Equals(t, 0, len(ws)) expSamples := make([]tsdbutil.Sample, 0, len(c.remaint)) for _, ts := range c.remaint { expSamples = append(expSamples, sample{ts, smpls[ts]}) } - expss := newMockSeriesSet([]Series{ + expss := newMockSeriesSet([]storage.Series{ newSeries(map[string]string{"a": "b"}, expSamples), }) @@ -965,8 +982,7 @@ func TestTombstoneClean(t *testing.T) { // When TombstoneClean errors the original block that should be rebuilt doesn't get deleted so // if TombstoneClean leaves any blocks behind these will overlap. func TestTombstoneCleanFail(t *testing.T) { - - db, delete := openTestDB(t, nil) + db, delete := openTestDB(t, nil, nil) defer func() { testutil.Ok(t, db.Close()) delete() @@ -1041,15 +1057,12 @@ func (c *mockCompactorFailing) Write(dest string, b BlockReader, mint, maxt int6 return block.Meta().ULID, nil } -func (*mockCompactorFailing) Compact(dest string, dirs []string, open []*Block) (ulid.ULID, error) { +func (*mockCompactorFailing) Compact(string, []string, []*Block) (ulid.ULID, error) { return ulid.ULID{}, nil - } func TestTimeRetention(t *testing.T) { - db, delete := openTestDB(t, &Options{ - BlockRanges: []int64{1000}, - }) + db, delete := openTestDB(t, nil, []int64{1000}) defer func() { testutil.Ok(t, db.Close()) delete() @@ -1068,7 +1081,7 @@ func TestTimeRetention(t *testing.T) { testutil.Ok(t, db.reload()) // Reload the db to register the new blocks. testutil.Equals(t, len(blocks), len(db.Blocks())) // Ensure all blocks are registered. - db.opts.RetentionDuration = uint64(blocks[2].MaxTime - blocks[1].MinTime) + db.opts.RetentionDuration = model.Duration(time.Duration(blocks[2].MaxTime-blocks[1].MinTime) * time.Millisecond) testutil.Ok(t, db.reload()) expBlocks := blocks[1:] @@ -1081,9 +1094,7 @@ func TestTimeRetention(t *testing.T) { } func TestSizeRetention(t *testing.T) { - db, delete := openTestDB(t, &Options{ - BlockRanges: []int64{100}, - }) + db, delete := openTestDB(t, nil, []int64{100}) defer func() { testutil.Ok(t, db.Close()) delete() @@ -1150,8 +1161,8 @@ func TestSizeRetention(t *testing.T) { // Check total size, total count and check that the oldest block was deleted. firstBlockSize := db.Blocks()[0].Size() sizeLimit := actSize - firstBlockSize - db.opts.MaxBytes = sizeLimit // Set the new db size limit one block smaller that the actual size. - testutil.Ok(t, db.reload()) // Reload the db to register the new db size. + db.opts.MaxBytes = units.Base2Bytes(sizeLimit) // Set the new db size limit one block smaller that the actual size. + testutil.Ok(t, db.reload()) // Reload the db to register the new db size. expBlocks := blocks[1:] actBlocks := db.Blocks() @@ -1184,9 +1195,8 @@ func TestSizeRetentionMetric(t *testing.T) { for _, c := range cases { db, delete := openTestDB(t, &Options{ - BlockRanges: []int64{100}, - MaxBytes: c.maxBytes, - }) + MaxBytes: units.Base2Bytes(c.maxBytes), + }, []int64{100}) actMaxBytes := int64(prom_testutil.ToFloat64(db.metrics.maxBytes)) testutil.Equals(t, actMaxBytes, c.expMaxBytes, "metric retention limit bytes mismatch") @@ -1197,7 +1207,7 @@ func TestSizeRetentionMetric(t *testing.T) { } func TestNotMatcherSelectsLabelsUnsetSeries(t *testing.T) { - db, delete := openTestDB(t, nil) + db, delete := openTestDB(t, nil, nil) defer func() { testutil.Ok(t, db.Close()) delete() @@ -1257,13 +1267,14 @@ func TestNotMatcherSelectsLabelsUnsetSeries(t *testing.T) { series: labelpairs[:1], }} - q, err := db.Querier(0, 10) + q, err := db.Querier(context.TODO(), 0, 10) testutil.Ok(t, err) defer func() { testutil.Ok(t, q.Close()) }() for _, c := range cases { - ss, err := q.Select(c.selector...) + ss, ws, err := q.Select(nil, c.selector...) testutil.Ok(t, err) + testutil.Equals(t, 0, len(ws)) lres, err := expandSeriesSet(ss) testutil.Ok(t, err) @@ -1272,7 +1283,7 @@ func TestNotMatcherSelectsLabelsUnsetSeries(t *testing.T) { } } -func expandSeriesSet(ss SeriesSet) ([]labels.Labels, error) { +func expandSeriesSet(ss storage.SeriesSet) ([]labels.Labels, error) { result := []labels.Labels{} for ss.Next() { result = append(result, ss.At().Labels()) @@ -1375,7 +1386,7 @@ func TestOverlappingBlocksDetectsAllOverlaps(t *testing.T) { // Regression test for https://github.com/prometheus/prometheus/tsdb/issues/347 func TestChunkAtBlockBoundary(t *testing.T) { - db, delete := openTestDB(t, nil) + db, delete := openTestDB(t, nil, nil) defer func() { testutil.Ok(t, db.Close()) delete() @@ -1383,7 +1394,7 @@ func TestChunkAtBlockBoundary(t *testing.T) { app := db.Appender() - blockRange := DefaultOptions.BlockRanges[0] + blockRange := db.compactor.(*LeveledCompactor).ranges[0] label := labels.FromStrings("foo", "bar") for i := int64(0); i < 3; i++ { @@ -1432,7 +1443,7 @@ func TestChunkAtBlockBoundary(t *testing.T) { } func TestQuerierWithBoundaryChunks(t *testing.T) { - db, delete := openTestDB(t, nil) + db, delete := openTestDB(t, nil, nil) defer func() { testutil.Ok(t, db.Close()) delete() @@ -1440,7 +1451,7 @@ func TestQuerierWithBoundaryChunks(t *testing.T) { app := db.Appender() - blockRange := DefaultOptions.BlockRanges[0] + blockRange := db.compactor.(*LeveledCompactor).ranges[0] label := labels.FromStrings("foo", "bar") for i := int64(0); i < 5; i++ { @@ -1456,7 +1467,7 @@ func TestQuerierWithBoundaryChunks(t *testing.T) { testutil.Assert(t, len(db.blocks) >= 3, "invalid test, less than three blocks in DB") - q, err := db.Querier(blockRange, 2*blockRange) + q, err := db.Querier(context.TODO(), blockRange, 2*blockRange) testutil.Ok(t, err) defer q.Close() @@ -1583,16 +1594,14 @@ func TestInitializeHeadTimestamp(t *testing.T) { } func TestNoEmptyBlocks(t *testing.T) { - db, delete := openTestDB(t, &Options{ - BlockRanges: []int64{100}, - }) + db, delete := openTestDB(t, nil, []int64{100}) defer func() { testutil.Ok(t, db.Close()) delete() }() db.DisableCompactions() - rangeToTriggerCompaction := db.opts.BlockRanges[0]/2*3 - 1 + rangeToTriggerCompaction := db.compactor.(*LeveledCompactor).ranges[0]/2*3 - 1 defaultLabel := labels.FromStrings("foo", "bar") defaultMatcher := labels.MustNewMatcher(labels.MatchRegexp, "", ".*") @@ -1666,8 +1675,8 @@ func TestNoEmptyBlocks(t *testing.T) { t.Run("Test no blocks remaining after deleting all samples from disk.", func(t *testing.T) { currentTime := db.Head().MaxTime() blocks := []*BlockMeta{ - {MinTime: currentTime, MaxTime: currentTime + db.opts.BlockRanges[0]}, - {MinTime: currentTime + 100, MaxTime: currentTime + 100 + db.opts.BlockRanges[0]}, + {MinTime: currentTime, MaxTime: currentTime + db.compactor.(*LeveledCompactor).ranges[0]}, + {MinTime: currentTime + 100, MaxTime: currentTime + 100 + db.compactor.(*LeveledCompactor).ranges[0]}, } for _, m := range blocks { createBlock(t, db.Dir(), genSeries(2, 2, m.MinTime, m.MaxTime)) @@ -1727,7 +1736,7 @@ func TestDB_LabelNames(t *testing.T) { }, } - blockRange := DefaultOptions.BlockRanges[0] + blockRange := int64(1000) // Appends samples into the database. appendSamples := func(db *DB, mint, maxt int64, sampleLabels [][2]string) { t.Helper() @@ -1743,7 +1752,7 @@ func TestDB_LabelNames(t *testing.T) { testutil.Ok(t, err) } for _, tst := range tests { - db, delete := openTestDB(t, nil) + db, delete := openTestDB(t, nil, nil) defer func() { testutil.Ok(t, db.Close()) delete() @@ -1778,23 +1787,25 @@ func TestDB_LabelNames(t *testing.T) { appendSamples(db, 5, 9, tst.sampleLabels2) // Testing DB (union). - q, err := db.Querier(math.MinInt64, math.MaxInt64) + q, err := db.Querier(context.TODO(), math.MinInt64, math.MaxInt64) testutil.Ok(t, err) - labelNames, err = q.LabelNames() + var ws storage.Warnings + labelNames, ws, err = q.LabelNames() testutil.Ok(t, err) + testutil.Equals(t, 0, len(ws)) testutil.Ok(t, q.Close()) testutil.Equals(t, tst.exp2, labelNames) } } func TestCorrectNumTombstones(t *testing.T) { - db, delete := openTestDB(t, nil) + db, delete := openTestDB(t, nil, nil) defer func() { testutil.Ok(t, db.Close()) delete() }() - blockRange := DefaultOptions.BlockRanges[0] + blockRange := db.compactor.(*LeveledCompactor).ranges[0] defaultLabel := labels.FromStrings("foo", "bar") defaultMatcher := labels.MustNewMatcher(labels.MatchEqual, defaultLabel[0].Name, defaultLabel[0].Value) @@ -1827,7 +1838,7 @@ func TestCorrectNumTombstones(t *testing.T) { func TestVerticalCompaction(t *testing.T) { cases := []struct { - blockSeries [][]Series + blockSeries [][]storage.Series expSeries map[string][]tsdbutil.Sample expBlockNum int expOverlappingBlocks int @@ -1836,7 +1847,7 @@ func TestVerticalCompaction(t *testing.T) { // |--------------| // |----------------| { - blockSeries: [][]Series{ + blockSeries: [][]storage.Series{ { newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{ sample{0, 0}, sample{1, 0}, sample{2, 0}, sample{4, 0}, @@ -1864,7 +1875,7 @@ func TestVerticalCompaction(t *testing.T) { // |-------------------------------| // |----------------| { - blockSeries: [][]Series{ + blockSeries: [][]storage.Series{ { newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{ sample{0, 0}, sample{1, 0}, sample{2, 0}, sample{4, 0}, @@ -1893,7 +1904,7 @@ func TestVerticalCompaction(t *testing.T) { // |------------| // |--------------------| { - blockSeries: [][]Series{ + blockSeries: [][]storage.Series{ { newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{ sample{0, 0}, sample{1, 0}, sample{2, 0}, sample{4, 0}, @@ -1929,7 +1940,7 @@ func TestVerticalCompaction(t *testing.T) { // |--------------------| // |----------------| { - blockSeries: [][]Series{ + blockSeries: [][]storage.Series{ { newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{ sample{0, 0}, sample{1, 0}, sample{2, 0}, sample{4, 0}, @@ -1965,7 +1976,7 @@ func TestVerticalCompaction(t *testing.T) { // |------------| // |-------------------------| { - blockSeries: [][]Series{ + blockSeries: [][]storage.Series{ { newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{ sample{0, 0}, sample{1, 0}, sample{2, 0}, sample{4, 0}, @@ -2003,7 +2014,7 @@ func TestVerticalCompaction(t *testing.T) { // |------------| // |-------------------------| { - blockSeries: [][]Series{ + blockSeries: [][]storage.Series{ { newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{ sample{0, 0}, sample{1, 0}, sample{2, 0}, sample{4, 0}, @@ -2100,7 +2111,7 @@ func TestVerticalCompaction(t *testing.T) { // |--------------| // |----------------| { - blockSeries: [][]Series{ + blockSeries: [][]storage.Series{ { newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{ sample{0, 0}, sample{1, 0}, sample{2, 0}, sample{4, 0}, @@ -2154,9 +2165,9 @@ func TestVerticalCompaction(t *testing.T) { for _, series := range c.blockSeries { createBlock(t, tmpdir, series) } - opts := *DefaultOptions + opts := DefaultOptions() opts.AllowOverlappingBlocks = true - db, err := Open(tmpdir, nil, nil, &opts) + db, err := Open(tmpdir, nil, nil, opts) testutil.Ok(t, err) defer func() { testutil.Ok(t, db.Close()) @@ -2165,7 +2176,7 @@ func TestVerticalCompaction(t *testing.T) { testutil.Assert(t, len(db.blocks) == len(c.blockSeries), "Wrong number of blocks [before compact].") // Vertical Query Merging test. - querier, err := db.Querier(0, 100) + querier, err := db.Querier(context.TODO(), 0, 100) testutil.Ok(t, err) actSeries := query(t, querier, defaultMatcher) testutil.Equals(t, c.expSeries, actSeries) @@ -2180,7 +2191,7 @@ func TestVerticalCompaction(t *testing.T) { 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(0, 100) + querier, err = db.Querier(context.TODO(), 0, 100) testutil.Ok(t, err) actSeries = query(t, querier, defaultMatcher) testutil.Equals(t, c.expSeries, actSeries) @@ -2203,20 +2214,16 @@ func TestBlockRanges(t *testing.T) { logger := log.NewLogfmtLogger(log.NewSyncWriter(os.Stderr)) dir, err := ioutil.TempDir("", "test_storage") - if err != nil { - t.Fatalf("Opening test dir failed: %s", err) - } - - rangeToTriggercompaction := DefaultOptions.BlockRanges[0]/2*3 + 1 + testutil.Ok(t, err) // Test that the compactor doesn't create overlapping blocks // when a non standard block already exists. firstBlockMaxT := int64(3) createBlock(t, dir, genSeries(1, 1, 0, firstBlockMaxT)) - db, err := Open(dir, logger, nil, DefaultOptions) - if err != nil { - t.Fatalf("Opening test storage failed: %s", err) - } + db, err := open(dir, logger, nil, DefaultOptions(), []int64{10000}) + testutil.Ok(t, err) + + rangeToTriggerCompaction := db.compactor.(*LeveledCompactor).ranges[0]/2*3 + 1 defer func() { os.RemoveAll(dir) }() @@ -2230,7 +2237,7 @@ func TestBlockRanges(t *testing.T) { testutil.Ok(t, err) _, err = app.Add(lbl, firstBlockMaxT+2, rand.Float64()) testutil.Ok(t, err) - secondBlockMaxt := firstBlockMaxT + rangeToTriggercompaction + secondBlockMaxt := firstBlockMaxT + rangeToTriggerCompaction _, err = app.Add(lbl, secondBlockMaxt, rand.Float64()) // Add samples to trigger a new compaction testutil.Ok(t, err) @@ -2264,16 +2271,15 @@ func TestBlockRanges(t *testing.T) { thirdBlockMaxt := secondBlockMaxt + 2 createBlock(t, dir, genSeries(1, 1, secondBlockMaxt+1, thirdBlockMaxt)) - db, err = Open(dir, logger, nil, DefaultOptions) - if err != nil { - t.Fatalf("Opening test storage failed: %s", err) - } + db, err = open(dir, logger, nil, DefaultOptions(), []int64{10000}) + testutil.Ok(t, err) + defer db.Close() testutil.Equals(t, 3, len(db.Blocks()), "db doesn't include expected number of blocks") testutil.Equals(t, db.Blocks()[2].Meta().MaxTime, thirdBlockMaxt, "unexpected maxt of the last block") app = db.Appender() - _, err = app.Add(lbl, thirdBlockMaxt+rangeToTriggercompaction, rand.Float64()) // Trigger a compaction + _, err = app.Add(lbl, thirdBlockMaxt+rangeToTriggerCompaction, rand.Float64()) // Trigger a compaction testutil.Ok(t, err) testutil.Ok(t, app.Commit()) for x := 0; x < 100; x++ { @@ -2344,7 +2350,7 @@ func TestDBReadOnly(t *testing.T) { testutil.Ok(t, err) testutil.Assert(t, expDbSize > dbSizeBeforeAppend, "db size didn't increase after an append") - q, err := dbWritable.Querier(math.MinInt64, math.MaxInt64) + q, err := dbWritable.Querier(context.TODO(), math.MinInt64, math.MaxInt64) testutil.Ok(t, err) expSeries = query(t, q, matchAll) @@ -2367,7 +2373,7 @@ func TestDBReadOnly(t *testing.T) { testutil.Equals(t, expBlock.Meta(), blocks[i].Meta(), "block meta mismatch") } - q, err := dbReadOnly.Querier(math.MinInt64, math.MaxInt64) + q, err := dbReadOnly.Querier(context.TODO(), math.MinInt64, math.MaxInt64) testutil.Ok(t, err) readOnlySeries := query(t, q, matchAll) readOnlyDBHash := testutil.DirHash(t, dbDir) @@ -2393,7 +2399,7 @@ func TestDBReadOnlyClosing(t *testing.T) { testutil.Equals(t, db.Close(), ErrClosed) _, err = db.Blocks() testutil.Equals(t, err, ErrClosed) - _, err = db.Querier(0, 1) + _, err = db.Querier(context.TODO(), 0, 1) testutil.Equals(t, err, ErrClosed) } @@ -2449,13 +2455,14 @@ func TestDBReadOnly_FlushWAL(t *testing.T) { testutil.Ok(t, err) testutil.Equals(t, len(blocks), 1) - querier, err := db.Querier(0, int64(maxt)-1) + querier, err := db.Querier(context.TODO(), 0, int64(maxt)-1) testutil.Ok(t, err) defer func() { testutil.Ok(t, querier.Close()) }() // Sum the values. - seriesSet, err := querier.Select(labels.MustNewMatcher(labels.MatchEqual, defaultLabelName, "flush")) + seriesSet, ws, err := querier.Select(nil, labels.MustNewMatcher(labels.MatchEqual, defaultLabelName, "flush")) testutil.Ok(t, err) + testutil.Equals(t, 0, len(ws)) sum := 0.0 for seriesSet.Next() { @@ -2692,3 +2699,38 @@ func TestChunkReader_ConcurrentReads(t *testing.T) { } testutil.Ok(t, r.Close()) } + +func TestTimeMetrics(t *testing.T) { + db, closeFn := openTestDB(t, nil, nil) + defer closeFn() + + metrics := &dto.Metric{} + + // Check initial values. + testutil.Ok(t, db.metrics.minTime.Write(metrics)) + testutil.Equals(t, float64(model.Latest)/1000, metrics.Gauge.GetValue()) + + testutil.Ok(t, db.metrics.headMinTime.Write(metrics)) + testutil.Equals(t, float64(model.Latest)/1000, metrics.Gauge.GetValue()) + + testutil.Ok(t, db.metrics.headMaxTime.Write(metrics)) + testutil.Equals(t, float64(model.Earliest)/1000, metrics.Gauge.GetValue()) + + app := db.Appender() + + app.Add(labels.FromStrings(model.MetricNameLabel, "a"), 1, 1) + app.Add(labels.FromStrings(model.MetricNameLabel, "a"), 2, 1) + app.Add(labels.FromStrings(model.MetricNameLabel, "a"), 3, 1) + testutil.Ok(t, app.Commit()) + + // Check after adding some samples. + testutil.Ok(t, db.metrics.minTime.Write(metrics)) + testutil.Equals(t, 0.001, metrics.Gauge.GetValue()) + + testutil.Ok(t, db.metrics.headMinTime.Write(metrics)) + testutil.Equals(t, 0.001, metrics.Gauge.GetValue()) + + testutil.Ok(t, db.metrics.headMaxTime.Write(metrics)) + testutil.Equals(t, 0.003, metrics.Gauge.GetValue()) + +} diff --git a/tsdb/head.go b/tsdb/head.go index f00abde979..abf356e1ac 100644 --- a/tsdb/head.go +++ b/tsdb/head.go @@ -29,6 +29,7 @@ import ( "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/index" @@ -800,7 +801,7 @@ func (h *RangeHead) Meta() BlockMeta { // initAppender is a helper to initialize the time bounds of the head // upon the first sample it receives. type initAppender struct { - app Appender + app storage.Appender head *Head } @@ -836,7 +837,7 @@ func (a *initAppender) Rollback() error { } // Appender returns a new Appender on the database. -func (h *Head) Appender() Appender { +func (h *Head) Appender() storage.Appender { h.metrics.activeAppenders.Inc() // The head cache might not have a starting point yet. The init appender diff --git a/tsdb/head_test.go b/tsdb/head_test.go index 93006e3210..cba30ef386 100644 --- a/tsdb/head_test.go +++ b/tsdb/head_test.go @@ -28,6 +28,7 @@ import ( "github.com/pkg/errors" prom_testutil "github.com/prometheus/client_golang/prometheus/testutil" "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/index" @@ -542,7 +543,7 @@ func TestHeadDeleteSimple(t *testing.T) { testutil.Ok(t, reloadedHead.Init(0)) // Compare the query results for both heads - before and after the reload. - expSeriesSet := newMockSeriesSet([]Series{ + expSeriesSet := newMockSeriesSet([]storage.Series{ newSeries(map[string]string{lblDefault.Name: lblDefault.Value}, func() []tsdbutil.Sample { ss := make([]tsdbutil.Sample, 0, len(c.smplsExp)) for _, s := range c.smplsExp { @@ -555,8 +556,9 @@ func TestHeadDeleteSimple(t *testing.T) { for _, h := range []*Head{head, reloadedHead} { q, err := NewBlockQuerier(h, h.MinTime(), h.MaxTime()) testutil.Ok(t, err) - actSeriesSet, err := q.Select(labels.MustNewMatcher(labels.MatchEqual, lblDefault.Name, lblDefault.Value)) + actSeriesSet, ws, err := q.Select(nil, labels.MustNewMatcher(labels.MatchEqual, lblDefault.Name, lblDefault.Value)) testutil.Ok(t, err) + testutil.Equals(t, 0, len(ws)) for { eok, rok := expSeriesSet.Next(), actSeriesSet.Next() @@ -601,8 +603,9 @@ func TestDeleteUntilCurMax(t *testing.T) { // Test the series returns no samples. The series is cleared only after compaction. q, err := NewBlockQuerier(hb, 0, 100000) testutil.Ok(t, err) - res, err := q.Select(labels.MustNewMatcher(labels.MatchEqual, "a", "b")) + res, ws, err := q.Select(nil, labels.MustNewMatcher(labels.MatchEqual, "a", "b")) testutil.Ok(t, err) + testutil.Equals(t, 0, len(ws)) testutil.Assert(t, res.Next(), "series is not present") s := res.At() it := s.Iterator() @@ -615,8 +618,9 @@ func TestDeleteUntilCurMax(t *testing.T) { testutil.Ok(t, app.Commit()) q, err = NewBlockQuerier(hb, 0, 100000) testutil.Ok(t, err) - res, err = q.Select(labels.MustNewMatcher(labels.MatchEqual, "a", "b")) + res, ws, err = q.Select(nil, labels.MustNewMatcher(labels.MatchEqual, "a", "b")) testutil.Ok(t, err) + testutil.Equals(t, 0, len(ws)) testutil.Assert(t, res.Next(), "series don't exist") exps := res.At() it = exps.Iterator() @@ -790,10 +794,11 @@ func TestDelete_e2e(t *testing.T) { q, err := NewBlockQuerier(hb, 0, 100000) testutil.Ok(t, err) defer q.Close() - ss, err := q.SelectSorted(del.ms...) + ss, ws, err := q.SelectSorted(nil, del.ms...) testutil.Ok(t, err) + testutil.Equals(t, 0, len(ws)) // Build the mockSeriesSet. - matchedSeries := make([]Series, 0, len(matched)) + matchedSeries := make([]storage.Series, 0, len(matched)) for _, m := range matched { smpls := seriesMap[m.String()] smpls = deletedSamples(smpls, del.drange) @@ -1077,8 +1082,9 @@ func TestUncommittedSamplesNotLostOnTruncate(t *testing.T) { testutil.Ok(t, err) defer q.Close() - ss, err := q.Select(labels.MustNewMatcher(labels.MatchEqual, "a", "1")) + ss, ws, err := q.Select(nil, labels.MustNewMatcher(labels.MatchEqual, "a", "1")) testutil.Ok(t, err) + testutil.Equals(t, 0, len(ws)) testutil.Equals(t, true, ss.Next()) } @@ -1104,8 +1110,9 @@ func TestRemoveSeriesAfterRollbackAndTruncate(t *testing.T) { testutil.Ok(t, err) defer q.Close() - ss, err := q.Select(labels.MustNewMatcher(labels.MatchEqual, "a", "1")) + ss, ws, err := q.Select(nil, labels.MustNewMatcher(labels.MatchEqual, "a", "1")) testutil.Ok(t, err) + testutil.Equals(t, 0, len(ws)) testutil.Equals(t, false, ss.Next()) @@ -1227,7 +1234,7 @@ func TestWalRepair_DecodingError(t *testing.T) { // Open the db to trigger a repair. { - db, err := Open(dir, nil, nil, DefaultOptions) + db, err := Open(dir, nil, nil, DefaultOptions()) testutil.Ok(t, err) defer func() { testutil.Ok(t, db.Close()) diff --git a/tsdb/querier.go b/tsdb/querier.go index dbde6ef762..ed5f7351cb 100644 --- a/tsdb/querier.go +++ b/tsdb/querier.go @@ -14,13 +14,13 @@ package tsdb import ( - "fmt" "sort" "strings" "unicode/utf8" "github.com/pkg/errors" "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" @@ -28,52 +28,25 @@ import ( "github.com/prometheus/prometheus/tsdb/tombstones" ) -// Querier provides querying access over time series data of a fixed -// time range. -type Querier interface { - // Select returns a set of series that matches the given label matchers. - Select(...*labels.Matcher) (SeriesSet, error) - - // SelectSorted returns a sorted set of series that matches the given label matcher. - SelectSorted(...*labels.Matcher) (SeriesSet, error) - - // LabelValues returns all potential values for a label name. - // It is not safe to use the strings beyond the lifefime of the querier. - LabelValues(string) ([]string, error) - - // LabelNames returns all the unique label names present in the block in sorted order. - LabelNames() ([]string, error) - - // Close releases the resources of the Querier. - Close() error -} - -// Series exposes a single time series. -type Series interface { - // Labels returns the complete set of labels identifying the series. - Labels() labels.Labels - - // Iterator returns a new iterator of the data of the series. - Iterator() SeriesIterator -} - // querier aggregates querying results from time blocks within // a single partition. type querier struct { - blocks []Querier + blocks []storage.Querier } -func (q *querier) LabelValues(n string) ([]string, error) { +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, error) { +func (q *querier) LabelNames() ([]string, storage.Warnings, error) { labelNamesMap := make(map[string]struct{}) + var ws storage.Warnings for _, b := range q.blocks { - names, err := b.LabelNames() + names, w, err := b.LabelNames() + ws = append(ws, w...) if err != nil { - return nil, errors.Wrap(err, "LabelNames() from Querier") + return nil, ws, errors.Wrap(err, "LabelNames() from Querier") } for _, name := range names { labelNamesMap[name] = struct{}{} @@ -86,51 +59,57 @@ func (q *querier) LabelNames() ([]string, error) { } sort.Strings(labelNames) - return labelNames, nil + return labelNames, ws, nil } -func (q *querier) lvals(qs []Querier, n string) ([]string, error) { +func (q *querier) lvals(qs []storage.Querier, n string) ([]string, storage.Warnings, error) { if len(qs) == 0 { - return nil, nil + return nil, nil, nil } if len(qs) == 1 { return qs[0].LabelValues(n) } l := len(qs) / 2 - s1, err := q.lvals(qs[:l], n) + + var ws storage.Warnings + s1, w, err := q.lvals(qs[:l], n) + ws = append(ws, w...) if err != nil { - return nil, err + return nil, ws, err } - s2, err := q.lvals(qs[l:], n) + s2, ws, err := q.lvals(qs[l:], n) + ws = append(ws, w...) if err != nil { - return nil, err + return nil, ws, err } - return mergeStrings(s1, s2), nil + return mergeStrings(s1, s2), ws, nil } -func (q *querier) Select(ms ...*labels.Matcher) (SeriesSet, error) { +func (q *querier) Select(p *storage.SelectParams, ms ...*labels.Matcher) (storage.SeriesSet, storage.Warnings, error) { if len(q.blocks) != 1 { - return q.SelectSorted(ms...) + return q.SelectSorted(p, ms...) } // Sorting Head series is slow, and unneeded when only the // Head is being queried. Sorting blocks is a noop. - return q.blocks[0].Select(ms...) + return q.blocks[0].Select(p, ms...) } -func (q *querier) SelectSorted(ms ...*labels.Matcher) (SeriesSet, error) { +func (q *querier) SelectSorted(p *storage.SelectParams, ms ...*labels.Matcher) (storage.SeriesSet, storage.Warnings, error) { if len(q.blocks) == 0 { - return EmptySeriesSet(), nil + return storage.EmptySeriesSet(), nil, nil } - ss := make([]SeriesSet, len(q.blocks)) + ss := make([]storage.SeriesSet, len(q.blocks)) + var ws storage.Warnings for i, b := range q.blocks { - s, err := b.SelectSorted(ms...) + s, w, err := b.SelectSorted(p, ms...) + ws = append(ws, w...) if err != nil { - return nil, err + return nil, ws, err } ss[i] = s } - return NewMergedSeriesSet(ss), nil + return NewMergedSeriesSet(ss), ws, nil } func (q *querier) Close() error { @@ -148,36 +127,39 @@ type verticalQuerier struct { querier } -func (q *verticalQuerier) Select(ms ...*labels.Matcher) (SeriesSet, error) { - return q.sel(q.blocks, ms) +func (q *verticalQuerier) Select(p *storage.SelectParams, ms ...*labels.Matcher) (storage.SeriesSet, storage.Warnings, error) { + return q.sel(p, q.blocks, ms) } -func (q *verticalQuerier) SelectSorted(ms ...*labels.Matcher) (SeriesSet, error) { - return q.sel(q.blocks, ms) +func (q *verticalQuerier) SelectSorted(p *storage.SelectParams, ms ...*labels.Matcher) (storage.SeriesSet, storage.Warnings, error) { + return q.sel(p, q.blocks, ms) } -func (q *verticalQuerier) sel(qs []Querier, ms []*labels.Matcher) (SeriesSet, error) { +func (q *verticalQuerier) sel(p *storage.SelectParams, qs []storage.Querier, ms []*labels.Matcher) (storage.SeriesSet, storage.Warnings, error) { if len(qs) == 0 { - return EmptySeriesSet(), nil + return storage.EmptySeriesSet(), nil, nil } if len(qs) == 1 { - return qs[0].SelectSorted(ms...) + return qs[0].SelectSorted(p, ms...) } l := len(qs) / 2 - a, err := q.sel(qs[:l], ms) + var ws storage.Warnings + a, w, err := q.sel(p, qs[:l], ms) + ws = append(ws, w...) if err != nil { - return nil, err + return nil, ws, err } - b, err := q.sel(qs[l:], ms) + b, w, err := q.sel(p, qs[l:], ms) + ws = append(ws, w...) if err != nil { - return nil, err + return nil, ws, err } - return newMergedVerticalSeriesSet(a, b), nil + return newMergedVerticalSeriesSet(a, b), ws, nil } // NewBlockQuerier returns a querier against the reader. -func NewBlockQuerier(b BlockReader, mint, maxt int64) (Querier, error) { +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") @@ -213,52 +195,64 @@ type blockQuerier struct { mint, maxt int64 } -func (q *blockQuerier) Select(ms ...*labels.Matcher) (SeriesSet, error) { +func (q *blockQuerier) Select(p *storage.SelectParams, ms ...*labels.Matcher) (storage.SeriesSet, storage.Warnings, error) { base, err := LookupChunkSeries(q.index, q.tombstones, ms...) if err != nil { - return nil, err + return nil, nil, err + } + + mint := q.mint + maxt := q.maxt + if p != nil { + mint = p.Start + maxt = p.End } return &blockSeriesSet{ set: &populatedChunkSeries{ set: base, chunks: q.chunks, - mint: q.mint, - maxt: q.maxt, + mint: mint, + maxt: maxt, }, - mint: q.mint, - maxt: q.maxt, - }, nil + mint: mint, + maxt: maxt, + }, nil, nil } -func (q *blockQuerier) SelectSorted(ms ...*labels.Matcher) (SeriesSet, error) { +func (q *blockQuerier) SelectSorted(p *storage.SelectParams, ms ...*labels.Matcher) (storage.SeriesSet, storage.Warnings, error) { base, err := LookupChunkSeriesSorted(q.index, q.tombstones, ms...) if err != nil { - return nil, err + return nil, nil, err + } + + mint := q.mint + maxt := q.maxt + if p != nil { + mint = p.Start + maxt = p.End } return &blockSeriesSet{ set: &populatedChunkSeries{ set: base, chunks: q.chunks, - mint: q.mint, - maxt: q.maxt, + mint: mint, + maxt: maxt, }, - mint: q.mint, - maxt: q.maxt, - }, nil + mint: mint, + maxt: maxt, + }, nil, nil } -func (q *blockQuerier) LabelValues(name string) ([]string, error) { - return q.index.LabelValues(name) +func (q *blockQuerier) LabelValues(name string) ([]string, storage.Warnings, error) { + res, err := q.index.LabelValues(name) + return res, nil, err } -func (q *blockQuerier) LabelNames() ([]string, error) { - return q.index.LabelNames() -} - -func (q *blockQuerier) LabelValuesFor(string, labels.Label) ([]string, error) { - return nil, fmt.Errorf("not implemented") +func (q *blockQuerier) LabelNames() ([]string, storage.Warnings, error) { + res, err := q.index.LabelNames() + return res, nil, err } func (q *blockQuerier) Close() error { @@ -495,32 +489,20 @@ func mergeStrings(a, b []string) []string { return res } -// SeriesSet contains a set of series. -type SeriesSet interface { - Next() bool - At() Series - Err() error -} - -var emptySeriesSet = errSeriesSet{} - -// EmptySeriesSet returns a series set that's always empty. -func EmptySeriesSet() SeriesSet { - return emptySeriesSet -} - // 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 []SeriesSet - buf []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. + 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 Series + cur storage.Series } -func NewMergedSeriesSet(all []SeriesSet) SeriesSet { +// 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] } @@ -535,7 +517,7 @@ func NewMergedSeriesSet(all []SeriesSet) SeriesSet { return s } -func (s *mergedSeriesSet) At() Series { +func (s *mergedSeriesSet) At() storage.Series { return s.cur } @@ -620,7 +602,7 @@ func (s *mergedSeriesSet) Next() bool { } if len(s.ids) > 1 { - series := make([]Series, len(s.ids)) + series := make([]storage.Series, len(s.ids)) for i, idx := range s.ids { series[i] = s.all[idx].At() } @@ -632,19 +614,19 @@ func (s *mergedSeriesSet) Next() bool { } type mergedVerticalSeriesSet struct { - a, b SeriesSet - cur Series + 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 SeriesSet) SeriesSet { +func NewMergedVerticalSeriesSet(a, b storage.SeriesSet) storage.SeriesSet { return newMergedVerticalSeriesSet(a, b) } -func newMergedVerticalSeriesSet(a, b SeriesSet) *mergedVerticalSeriesSet { +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. @@ -654,7 +636,7 @@ func newMergedVerticalSeriesSet(a, b SeriesSet) *mergedVerticalSeriesSet { return s } -func (s *mergedVerticalSeriesSet) At() Series { +func (s *mergedVerticalSeriesSet) At() storage.Series { return s.cur } @@ -690,21 +672,13 @@ func (s *mergedVerticalSeriesSet) Next() bool { s.cur = s.a.At() s.adone = !s.a.Next() } else { - s.cur = &verticalChainedSeries{series: []Series{s.a.At(), s.b.At()}} + s.cur = &verticalChainedSeries{series: []storage.Series{s.a.At(), s.b.At()}} s.adone = !s.a.Next() s.bdone = !s.b.Next() } return true } -// ChunkSeriesSet exposes the chunks and intervals of a series instead of the -// actual series itself. -type ChunkSeriesSet interface { - Next() bool - At() (labels.Labels, []chunks.Meta, tombstones.Intervals) - Err() error -} - // 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 { @@ -720,17 +694,17 @@ type baseChunkSeries struct { // 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) (ChunkSeriesSet, error) { +func LookupChunkSeries(ir IndexReader, tr tombstones.Reader, ms ...*labels.Matcher) (storage.ChunkSeriesSet, 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) (ChunkSeriesSet, error) { +func LookupChunkSeriesSorted(ir IndexReader, tr tombstones.Reader, ms ...*labels.Matcher) (storage.ChunkSeriesSet, error) { return lookupChunkSeries(true, ir, tr, ms...) } -func lookupChunkSeries(sorted bool, ir IndexReader, tr tombstones.Reader, ms ...*labels.Matcher) (ChunkSeriesSet, error) { +func lookupChunkSeries(sorted bool, ir IndexReader, tr tombstones.Reader, ms ...*labels.Matcher) (storage.ChunkSeriesSet, error) { if tr == nil { tr = tombstones.NewMemTombstones() } @@ -804,7 +778,7 @@ func (s *baseChunkSeries) Next() bool { // with known chunk references. It filters out chunks that do not fit the // given time range. type populatedChunkSeries struct { - set ChunkSeriesSet + set storage.ChunkSeriesSet chunks ChunkReader mint, maxt int64 @@ -872,9 +846,9 @@ func (s *populatedChunkSeries) Next() bool { // blockSeriesSet is a set of series from an inverted index query. type blockSeriesSet struct { - set ChunkSeriesSet + set storage.ChunkSeriesSet err error - cur Series + cur storage.Series mint, maxt int64 } @@ -898,8 +872,8 @@ func (s *blockSeriesSet) Next() bool { return false } -func (s *blockSeriesSet) At() Series { return s.cur } -func (s *blockSeriesSet) Err() error { return s.err } +func (s *blockSeriesSet) At() storage.Series { return s.cur } +func (s *blockSeriesSet) Err() error { return s.err } // chunkSeries is a series that is backed by a sequence of chunks holding // time series data. @@ -916,48 +890,34 @@ func (s *chunkSeries) Labels() labels.Labels { return s.labels } -func (s *chunkSeries) Iterator() SeriesIterator { +func (s *chunkSeries) Iterator() chunkenc.Iterator { return newChunkSeriesIterator(s.chunks, s.intervals, s.mint, s.maxt) } -// SeriesIterator iterates over the data of a time series. -type SeriesIterator interface { - // Seek advances the iterator forward to the given timestamp. - // If there's no value exactly at t, it advances to the first value - // after t. - Seek(t int64) bool - // At returns the current timestamp/value pair. - At() (t int64, v float64) - // Next advances the iterator by one. - Next() bool - // Err returns the current error. - Err() error -} - // chainedSeries implements a series for a list of time-sorted series. // They all must have the same labels. type chainedSeries struct { - series []Series + series []storage.Series } func (s *chainedSeries) Labels() labels.Labels { return s.series[0].Labels() } -func (s *chainedSeries) Iterator() SeriesIterator { +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 []Series // series in time order + series []storage.Series // series in time order i int - cur SeriesIterator + cur chunkenc.Iterator } -func newChainedSeriesIterator(s ...Series) *chainedSeriesIterator { +func newChainedSeriesIterator(s ...storage.Series) *chainedSeriesIterator { return &chainedSeriesIterator{ series: s, i: 0, @@ -1008,28 +968,28 @@ func (it *chainedSeriesIterator) Err() error { // verticalChainedSeries implements a series for a list of time-sorted, time-overlapping series. // They all must have the same labels. type verticalChainedSeries struct { - series []Series + series []storage.Series } func (s *verticalChainedSeries) Labels() labels.Labels { return s.series[0].Labels() } -func (s *verticalChainedSeries) Iterator() SeriesIterator { +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 SeriesIterator + a, b chunkenc.Iterator aok, bok, initialized bool curT int64 curV float64 } -func newVerticalMergeSeriesIterator(s ...Series) SeriesIterator { +func newVerticalMergeSeriesIterator(s ...storage.Series) chunkenc.Iterator { if len(s) == 1 { return s[0].Iterator() } else if len(s) == 2 { @@ -1219,6 +1179,13 @@ func (it *deletedIterator) At() (int64, float64) { return it.it.At() } +func (it *deletedIterator) Seek(t int64) bool { + if it.it.Err() != nil { + return false + } + return it.it.Seek(t) +} + func (it *deletedIterator) Next() bool { Outer: for it.it.Next() { @@ -1229,28 +1196,15 @@ Outer: continue Outer } - if ts > tr.Maxt { - it.intervals = it.intervals[1:] - continue + if ts <= tr.Maxt { + return true + } - - return true + it.intervals = it.intervals[1:] } - return true } - return false } -func (it *deletedIterator) Err() error { - return it.it.Err() -} - -type errSeriesSet struct { - err error -} - -func (s errSeriesSet) Next() bool { return false } -func (s errSeriesSet) At() Series { return nil } -func (s errSeriesSet) Err() error { return s.err } +func (it *deletedIterator) Err() error { return it.it.Err() } diff --git a/tsdb/querier_bench_test.go b/tsdb/querier_bench_test.go index bd7364b451..2cad9d9e1e 100644 --- a/tsdb/querier_bench_test.go +++ b/tsdb/querier_bench_test.go @@ -21,6 +21,7 @@ import ( "testing" "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/util/testutil" ) @@ -145,11 +146,11 @@ func BenchmarkQuerierSelect(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { - var ss SeriesSet + var ss storage.SeriesSet if sorted { - ss, err = q.SelectSorted(matcher) + ss, _, err = q.SelectSorted(nil, matcher) } else { - ss, err = q.Select(matcher) + ss, _, err = q.Select(nil, matcher) } testutil.Ok(b, err) for ss.Next() { diff --git a/tsdb/querier_test.go b/tsdb/querier_test.go index f1fe53b730..66c02b90b0 100644 --- a/tsdb/querier_test.go +++ b/tsdb/querier_test.go @@ -14,6 +14,7 @@ package tsdb import ( + "context" "fmt" "io/ioutil" "math" @@ -26,6 +27,7 @@ import ( "github.com/pkg/errors" "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/index" @@ -36,22 +38,22 @@ import ( type mockSeriesSet struct { next func() bool - series func() Series + series func() storage.Series err func() error } -func (m *mockSeriesSet) Next() bool { return m.next() } -func (m *mockSeriesSet) At() Series { return m.series() } -func (m *mockSeriesSet) Err() error { return m.err() } +func (m *mockSeriesSet) Next() bool { return m.next() } +func (m *mockSeriesSet) At() storage.Series { return m.series() } +func (m *mockSeriesSet) Err() error { return m.err() } -func newMockSeriesSet(list []Series) *mockSeriesSet { +func newMockSeriesSet(list []storage.Series) *mockSeriesSet { i := -1 return &mockSeriesSet{ next: func() bool { i++ return i < len(list) }, - series: func() Series { + series: func() storage.Series { return list[i] }, err: func() error { return nil }, @@ -63,20 +65,20 @@ 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 SeriesSet + 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 SeriesSet + exp storage.SeriesSet }{ { - a: newMockSeriesSet([]Series{ + a: newMockSeriesSet([]storage.Series{ newSeries(map[string]string{ "a": "a", }, []tsdbutil.Sample{ sample{t: 1, v: 1}, }), }), - b: newMockSeriesSet([]Series{ + b: newMockSeriesSet([]storage.Series{ newSeries(map[string]string{ "a": "a", }, []tsdbutil.Sample{ @@ -88,7 +90,7 @@ func TestMergedSeriesSet(t *testing.T) { sample{t: 1, v: 1}, }), }), - exp: newMockSeriesSet([]Series{ + exp: newMockSeriesSet([]storage.Series{ newSeries(map[string]string{ "a": "a", }, []tsdbutil.Sample{ @@ -103,7 +105,7 @@ func TestMergedSeriesSet(t *testing.T) { }), }, { - a: newMockSeriesSet([]Series{ + a: newMockSeriesSet([]storage.Series{ newSeries(map[string]string{ "handler": "prometheus", "instance": "127.0.0.1:9090", @@ -117,7 +119,7 @@ func TestMergedSeriesSet(t *testing.T) { sample{t: 1, v: 2}, }), }), - b: newMockSeriesSet([]Series{ + b: newMockSeriesSet([]storage.Series{ newSeries(map[string]string{ "handler": "prometheus", "instance": "127.0.0.1:9090", @@ -131,7 +133,7 @@ func TestMergedSeriesSet(t *testing.T) { sample{t: 2, v: 2}, }), }), - exp: newMockSeriesSet([]Series{ + exp: newMockSeriesSet([]storage.Series{ newSeries(map[string]string{ "handler": "prometheus", "instance": "127.0.0.1:9090", @@ -157,7 +159,7 @@ func TestMergedSeriesSet(t *testing.T) { Outer: for _, c := range cases { - res := NewMergedSeriesSet([]SeriesSet{c.a, c.b}) + res := NewMergedSeriesSet([]storage.SeriesSet{c.a, c.b}) for { eok, rok := c.exp.Next(), res.Next() @@ -180,7 +182,7 @@ Outer: } } -func expandSeriesIterator(it SeriesIterator) (r []tsdbutil.Sample, err 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}) @@ -258,17 +260,17 @@ func createIdxChkReaders(t *testing.T, tc []seriesSamples) (IndexReader, ChunkRe } func TestBlockQuerier(t *testing.T) { - newSeries := func(l map[string]string, s []tsdbutil.Sample) Series { + newSeries := func(l map[string]string, s []tsdbutil.Sample) storage.Series { return &mockSeries{ labels: func() labels.Labels { return labels.FromMap(l) }, - iterator: func() SeriesIterator { return newListSeriesIterator(s) }, + iterator: func() chunkenc.Iterator { return newListSeriesIterator(s) }, } } type query struct { mint, maxt int64 ms []*labels.Matcher - exp SeriesSet + exp storage.SeriesSet } cases := struct { @@ -324,25 +326,25 @@ func TestBlockQuerier(t *testing.T) { mint: 0, maxt: 0, ms: []*labels.Matcher{}, - exp: newMockSeriesSet([]Series{}), + exp: newMockSeriesSet([]storage.Series{}), }, { mint: 0, maxt: 0, ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "a")}, - exp: newMockSeriesSet([]Series{}), + exp: newMockSeriesSet([]storage.Series{}), }, { mint: 1, maxt: 0, ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "a")}, - exp: newMockSeriesSet([]Series{}), + exp: newMockSeriesSet([]storage.Series{}), }, { mint: 2, maxt: 6, ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "a")}, - exp: newMockSeriesSet([]Series{ + exp: newMockSeriesSet([]storage.Series{ newSeries(map[string]string{ "a": "a", }, @@ -371,8 +373,9 @@ Outer: maxt: c.maxt, } - res, err := querier.Select(c.ms...) + res, ws, err := querier.Select(nil, c.ms...) testutil.Ok(t, err) + testutil.Equals(t, 0, len(ws)) for { eok, rok := c.exp.Next(), res.Next() @@ -396,17 +399,17 @@ Outer: } func TestBlockQuerierDelete(t *testing.T) { - newSeries := func(l map[string]string, s []tsdbutil.Sample) Series { + newSeries := func(l map[string]string, s []tsdbutil.Sample) storage.Series { return &mockSeries{ labels: func() labels.Labels { return labels.FromMap(l) }, - iterator: func() SeriesIterator { return newListSeriesIterator(s) }, + iterator: func() chunkenc.Iterator { return newListSeriesIterator(s) }, } } type query struct { mint, maxt int64 ms []*labels.Matcher - exp SeriesSet + exp storage.SeriesSet } cases := struct { @@ -467,7 +470,7 @@ func TestBlockQuerierDelete(t *testing.T) { mint: 2, maxt: 7, ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "a")}, - exp: newMockSeriesSet([]Series{ + exp: newMockSeriesSet([]storage.Series{ newSeries(map[string]string{ "a": "a", }, @@ -485,7 +488,7 @@ func TestBlockQuerierDelete(t *testing.T) { mint: 2, maxt: 7, ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "b", "b")}, - exp: newMockSeriesSet([]Series{ + exp: newMockSeriesSet([]storage.Series{ newSeries(map[string]string{ "a": "a", "b": "b", @@ -503,7 +506,7 @@ func TestBlockQuerierDelete(t *testing.T) { mint: 1, maxt: 4, ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "a")}, - exp: newMockSeriesSet([]Series{ + exp: newMockSeriesSet([]storage.Series{ newSeries(map[string]string{ "a": "a", "b": "b", @@ -516,12 +519,11 @@ func TestBlockQuerierDelete(t *testing.T) { mint: 1, maxt: 3, ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "a")}, - exp: newMockSeriesSet([]Series{}), + exp: newMockSeriesSet([]storage.Series{}), }, }, } - fmt.Println("tombstones", cases.tombstones) Outer: for _, c := range cases.queries { ir, cr, _, _ := createIdxChkReaders(t, cases.data) @@ -534,8 +536,9 @@ Outer: maxt: c.maxt, } - res, err := querier.Select(c.ms...) + res, ws, err := querier.Select(nil, c.ms...) testutil.Ok(t, err) + testutil.Equals(t, 0, len(ws)) for { eok, rok := c.exp.Next(), res.Next() @@ -654,13 +657,12 @@ func TestBaseChunkSeries(t *testing.T) { } } -// TODO: Remove after simpleSeries is merged type itSeries struct { - si SeriesIterator + si chunkenc.Iterator } -func (s itSeries) Iterator() SeriesIterator { return s.si } -func (s itSeries) Labels() labels.Labels { return labels.Labels{} } +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 { @@ -1003,7 +1005,7 @@ func TestSeriesIterator(t *testing.T) { t.Run("Seek", func(t *testing.T) { for _, tc := range seekcases { - ress := []SeriesIterator{ + ress := []chunkenc.Iterator{ newChainedSeriesIterator( itSeries{newListSeriesIterator(tc.a)}, itSeries{newListSeriesIterator(tc.b)}, @@ -1167,8 +1169,9 @@ func (m *mockChunkSeriesSet) Err() error { // Test the cost of merging series sets for different number of merged sets and their size. // The subset are all equivalent so this does not capture merging of partial or non-overlapping sets well. +// TODO(bwplotka): Merge with storage merged series set benchmark. func BenchmarkMergedSeriesSet(b *testing.B) { - var sel = func(sets []SeriesSet) SeriesSet { + var sel = func(sets []storage.SeriesSet) storage.SeriesSet { return NewMergedSeriesSet(sets) } @@ -1185,7 +1188,7 @@ func BenchmarkMergedSeriesSet(b *testing.B) { sort.Sort(labels.Slice(lbls)) - in := make([][]Series, j) + in := make([][]storage.Series, j) for _, l := range lbls { l2 := l @@ -1197,7 +1200,7 @@ func BenchmarkMergedSeriesSet(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { - var sets []SeriesSet + var sets []storage.SeriesSet for _, s := range in { sets = append(sets, newMockSeriesSet(s)) } @@ -1257,6 +1260,104 @@ func TestDeletedIterator(t *testing.T) { {r: tombstones.Intervals{{Mint: 1000, Maxt: 20000}}}, } + for _, c := range cases { + t.Run("Simple", func(t *testing.T) { + i := int64(-1) + it := &deletedIterator{it: chk.Iterator(nil), intervals: c.r[:]} + ranges := c.r[:] + for it.Next() { + i++ + for _, tr := range ranges { + if tr.InBounds(i) { + i = tr.Maxt + 1 + ranges = ranges[1:] + } + } + + testutil.Assert(t, i < 1000, "") + + ts, v := it.At() + testutil.Equals(t, act[i].t, ts) + testutil.Equals(t, act[i].v, v) + } + // There has been an extra call to Next(). + i++ + for _, tr := range ranges { + if tr.InBounds(i) { + i = tr.Maxt + 1 + ranges = ranges[1:] + } + } + + testutil.Assert(t, i >= 1000, "") + testutil.Ok(t, it.Err()) + }) + t.Run("Seek", func(t *testing.T) { + const seek = 10 + + i := int64(seek) + it := &deletedIterator{it: chk.Iterator(nil), intervals: c.r[:]} + ranges := c.r[:] + + testutil.Assert(t, it.Seek(seek), "") + for it.Next() { + i++ + for _, tr := range ranges { + if tr.InBounds(i) { + i = tr.Maxt + 1 + ranges = ranges[1:] + } + } + + testutil.Assert(t, i < 1000, "") + + ts, v := it.At() + testutil.Equals(t, act[i].t, ts) + testutil.Equals(t, act[i].v, v) + } + // There has been an extra call to Next(). + i++ + for _, tr := range ranges { + if tr.InBounds(i) { + i = tr.Maxt + 1 + ranges = ranges[1:] + } + } + + testutil.Assert(t, i >= 1000, "") + testutil.Ok(t, it.Err()) + }) + } +} + +func TestDeletedIterator_WithSeek(t *testing.T) { + chk := chunkenc.NewXORChunk() + app, err := chk.Appender() + testutil.Ok(t, err) + // Insert random stuff from (0, 1000). + act := make([]sample, 1000) + for i := 0; i < 1000; i++ { + act[i].t = int64(i) + act[i].v = rand.Float64() + app.Append(act[i].t, act[i].v) + } + + cases := []struct { + r tombstones.Intervals + seek int64 + }{ + {r: tombstones.Intervals{{Mint: 1, Maxt: 20}}}, + {r: tombstones.Intervals{{Mint: 1, Maxt: 10}, {Mint: 12, Maxt: 20}, {Mint: 21, Maxt: 23}, {Mint: 25, Maxt: 30}}}, + {r: tombstones.Intervals{{Mint: 1, Maxt: 10}, {Mint: 12, Maxt: 20}, {Mint: 20, Maxt: 30}}}, + {r: tombstones.Intervals{{Mint: 1, Maxt: 10}, {Mint: 12, Maxt: 23}, {Mint: 25, Maxt: 30}}}, + {r: tombstones.Intervals{{Mint: 1, Maxt: 23}, {Mint: 12, Maxt: 20}, {Mint: 25, Maxt: 30}}}, + {r: tombstones.Intervals{{Mint: 1, Maxt: 23}, {Mint: 12, Maxt: 20}, {Mint: 25, Maxt: 3000}}}, + {r: tombstones.Intervals{{Mint: 0, Maxt: 2000}}}, + {r: tombstones.Intervals{{Mint: 500, Maxt: 2000}}}, + {r: tombstones.Intervals{{Mint: 0, Maxt: 200}}}, + {r: tombstones.Intervals{{Mint: 1000, Maxt: 20000}}}, + } + for _, c := range cases { i := int64(-1) it := &deletedIterator{it: chk.Iterator(nil), intervals: c.r[:]} @@ -1414,17 +1515,17 @@ func (m mockIndex) LabelNames() ([]string, error) { type mockSeries struct { labels func() labels.Labels - iterator func() SeriesIterator + iterator func() chunkenc.Iterator } -func newSeries(l map[string]string, s []tsdbutil.Sample) Series { +func newSeries(l map[string]string, s []tsdbutil.Sample) storage.Series { return &mockSeries{ labels: func() labels.Labels { return labels.FromMap(l) }, - iterator: func() SeriesIterator { return newListSeriesIterator(s) }, + iterator: func() chunkenc.Iterator { return newListSeriesIterator(s) }, } } -func (m *mockSeries) Labels() labels.Labels { return m.labels() } -func (m *mockSeries) Iterator() SeriesIterator { return m.iterator() } +func (m *mockSeries) Labels() labels.Labels { return m.labels() } +func (m *mockSeries) Iterator() chunkenc.Iterator { return m.iterator() } type listSeriesIterator struct { list []tsdbutil.Sample @@ -1493,7 +1594,7 @@ func BenchmarkQueryIterator(b *testing.B) { blocks []*Block overlapDelta = int64(overlapPercentage * c.numSamplesPerSeriesPerBlock / 100) prefilledLabels []map[string]string - generatedSeries []Series + generatedSeries []storage.Series ) for i := int64(0); i < int64(c.numBlocks); i++ { offset := i * overlapDelta @@ -1514,7 +1615,7 @@ func BenchmarkQueryIterator(b *testing.B) { } que := &querier{ - blocks: make([]Querier, 0, len(blocks)), + blocks: make([]storage.Querier, 0, len(blocks)), } for _, blk := range blocks { q, err := NewBlockQuerier(blk, math.MinInt64, math.MaxInt64) @@ -1522,7 +1623,7 @@ func BenchmarkQueryIterator(b *testing.B) { que.blocks = append(que.blocks, q) } - var sq Querier = que + var sq storage.Querier = que if overlapPercentage > 0 { sq = &verticalQuerier{ querier: *que, @@ -1567,7 +1668,7 @@ func BenchmarkQuerySeek(b *testing.B) { blocks []*Block overlapDelta = int64(overlapPercentage * c.numSamplesPerSeriesPerBlock / 100) prefilledLabels []map[string]string - generatedSeries []Series + generatedSeries []storage.Series ) for i := int64(0); i < int64(c.numBlocks); i++ { offset := i * overlapDelta @@ -1588,7 +1689,7 @@ func BenchmarkQuerySeek(b *testing.B) { } que := &querier{ - blocks: make([]Querier, 0, len(blocks)), + blocks: make([]storage.Querier, 0, len(blocks)), } for _, blk := range blocks { q, err := NewBlockQuerier(blk, math.MinInt64, math.MaxInt64) @@ -1596,7 +1697,7 @@ func BenchmarkQuerySeek(b *testing.B) { que.blocks = append(que.blocks, q) } - var sq Querier = que + var sq storage.Querier = que if overlapPercentage > 0 { sq = &verticalQuerier{ querier: *que, @@ -1610,7 +1711,7 @@ func BenchmarkQuerySeek(b *testing.B) { b.ResetTimer() b.ReportAllocs() - ss, err := sq.Select(labels.MustNewMatcher(labels.MatchRegexp, "__name__", ".*")) + ss, ws, err := sq.Select(nil, labels.MustNewMatcher(labels.MatchRegexp, "__name__", ".*")) for ss.Next() { it := ss.At().Iterator() for t := mint; t <= maxt; t++ { @@ -1620,6 +1721,7 @@ func BenchmarkQuerySeek(b *testing.B) { } testutil.Ok(b, ss.Err()) testutil.Ok(b, err) + testutil.Equals(b, 0, len(ws)) }) } } @@ -1713,7 +1815,7 @@ func BenchmarkSetMatcher(b *testing.B) { var ( blocks []*Block prefilledLabels []map[string]string - generatedSeries []Series + generatedSeries []storage.Series ) for i := int64(0); i < int64(c.numBlocks); i++ { mint := i * int64(c.numSamplesPerSeriesPerBlock) @@ -1733,7 +1835,7 @@ func BenchmarkSetMatcher(b *testing.B) { } que := &querier{ - blocks: make([]Querier, 0, len(blocks)), + blocks: make([]storage.Querier, 0, len(blocks)), } for _, blk := range blocks { q, err := NewBlockQuerier(blk, math.MinInt64, math.MaxInt64) @@ -1747,8 +1849,9 @@ func BenchmarkSetMatcher(b *testing.B) { b.ResetTimer() b.ReportAllocs() for n := 0; n < b.N; n++ { - _, err := que.Select(labels.MustNewMatcher(labels.MatchRegexp, "test", c.pattern)) + _, ws, err := que.Select(nil, labels.MustNewMatcher(labels.MatchRegexp, "test", c.pattern)) testutil.Ok(b, err) + testutil.Equals(b, 0, len(ws)) } }) } @@ -2076,7 +2179,7 @@ func TestClose(t *testing.T) { createBlock(t, dir, genSeries(1, 1, 0, 10)) createBlock(t, dir, genSeries(1, 1, 10, 20)) - db, err := Open(dir, nil, nil, DefaultOptions) + db, err := Open(dir, nil, nil, DefaultOptions()) if err != nil { t.Fatalf("Opening test storage failed: %s", err) } @@ -2084,7 +2187,7 @@ func TestClose(t *testing.T) { testutil.Ok(t, db.Close()) }() - q, err := db.Querier(0, 20) + q, err := db.Querier(context.TODO(), 0, 20) testutil.Ok(t, err) testutil.Ok(t, q.Close()) testutil.NotOk(t, q.Close()) @@ -2119,7 +2222,7 @@ func BenchmarkQueries(b *testing.B) { }, } - queryTypes := make(map[string]Querier) + queryTypes := make(map[string]storage.Querier) defer func() { for _, q := range queryTypes { // Can't run a check for error here as some of these will fail as @@ -2162,7 +2265,7 @@ func BenchmarkQueries(b *testing.B) { } } - qs := []Querier{} + qs := make([]storage.Querier, 0, 10) for x := 0; x <= 10; x++ { block, err := OpenBlock(nil, createBlock(b, dir, series), nil) testutil.Ok(b, err) @@ -2191,12 +2294,13 @@ func BenchmarkQueries(b *testing.B) { } } -func benchQuery(b *testing.B, expExpansions int, q Querier, selectors labels.Selector) { +func benchQuery(b *testing.B, expExpansions int, q storage.Querier, selectors labels.Selector) { b.ResetTimer() b.ReportAllocs() for i := 0; i < b.N; i++ { - ss, err := q.Select(selectors...) + ss, ws, err := q.Select(nil, selectors...) testutil.Ok(b, err) + testutil.Equals(b, 0, len(ws)) var actualExpansions int for ss.Next() { s := ss.At() diff --git a/tsdb/tsdbblockutil.go b/tsdb/tsdbblockutil.go index 891876bcaa..297dc3107c 100644 --- a/tsdb/tsdbblockutil.go +++ b/tsdb/tsdbblockutil.go @@ -65,7 +65,7 @@ func CreateBlock(samples []*MetricSample, dir string, mint, maxt int64, logger l return "", err } - compactor, err := NewLeveledCompactor(context.Background(), nil, logger, DefaultOptions.BlockRanges, nil) + compactor, err := NewLeveledCompactor(context.Background(), nil, logger, ExponentialBlockRanges(DefaultBlockDuration, 3, 5), nil) if err != nil { return "", err } diff --git a/util/teststorage/storage.go b/util/teststorage/storage.go index 9b1fcb88b3..633112d013 100644 --- a/util/teststorage/storage.go +++ b/util/teststorage/storage.go @@ -16,11 +16,9 @@ package teststorage import ( "io/ioutil" "os" - "time" - "github.com/prometheus/common/model" "github.com/prometheus/prometheus/storage" - "github.com/prometheus/prometheus/storage/tsdb" + "github.com/prometheus/prometheus/tsdb" "github.com/prometheus/prometheus/util/testutil" ) @@ -34,14 +32,11 @@ func New(t testutil.T) storage.Storage { // Tests just load data for a series sequentially. Thus we // need a long appendable window. - db, err := tsdb.Open(dir, nil, nil, &tsdb.Options{ - MinBlockDuration: model.Duration(24 * time.Hour), - MaxBlockDuration: model.Duration(24 * time.Hour), - }) + db, err := tsdb.Open(dir, nil, nil, nil) if err != nil { t.Fatalf("Opening test storage failed: %s", err) } - return testStorage{Storage: tsdb.Adapter(db, int64(0)), dir: dir} + return testStorage{Storage: db, dir: dir} } type testStorage struct { diff --git a/web/web.go b/web/web.go index 3a3fedfb46..9274176ae0 100644 --- a/web/web.go +++ b/web/web.go @@ -62,7 +62,6 @@ import ( "github.com/prometheus/prometheus/rules" "github.com/prometheus/prometheus/scrape" "github.com/prometheus/prometheus/storage" - prometheus_tsdb "github.com/prometheus/prometheus/storage/tsdb" "github.com/prometheus/prometheus/template" "github.com/prometheus/prometheus/util/httputil" api_v1 "github.com/prometheus/prometheus/web/api/v1" @@ -213,7 +212,7 @@ func (h *Handler) ApplyConfig(conf *config.Config) error { type Options struct { Context context.Context TSDB func() *tsdb.DB - TSDBCfg prometheus_tsdb.Options + TSDBCfg tsdb.Options Storage storage.Storage QueryEngine *promql.Engine LookbackDelta time.Duration diff --git a/web/web_test.go b/web/web_test.go index f9e6b1413a..718923cab1 100644 --- a/web/web_test.go +++ b/web/web_test.go @@ -33,8 +33,7 @@ import ( "github.com/prometheus/prometheus/notifier" "github.com/prometheus/prometheus/rules" "github.com/prometheus/prometheus/scrape" - "github.com/prometheus/prometheus/storage/tsdb" - libtsdb "github.com/prometheus/prometheus/tsdb" + "github.com/prometheus/prometheus/tsdb" "github.com/prometheus/prometheus/util/testutil" ) @@ -98,7 +97,7 @@ func TestReadyAndHealthy(t *testing.T) { testutil.Ok(t, err) defer os.RemoveAll(dbDir) - db, err := libtsdb.Open(dbDir, nil, nil, nil) + db, err := tsdb.Open(dbDir, nil, nil, nil) testutil.Ok(t, err) @@ -114,7 +113,7 @@ func TestReadyAndHealthy(t *testing.T) { Notifier: nil, RoutePrefix: "/", EnableAdminAPI: true, - TSDB: func() *libtsdb.DB { return db }, + TSDB: func() *tsdb.DB { return db }, ExternalURL: &url.URL{ Scheme: "http", Host: "localhost:9090", @@ -289,7 +288,7 @@ func TestRoutePrefix(t *testing.T) { defer os.RemoveAll(dbDir) - db, err := libtsdb.Open(dbDir, nil, nil, nil) + db, err := tsdb.Open(dbDir, nil, nil, nil) testutil.Ok(t, err) @@ -309,7 +308,7 @@ func TestRoutePrefix(t *testing.T) { Host: "localhost.localdomain:9090", Scheme: "http", }, - TSDB: func() *libtsdb.DB { return db }, + TSDB: func() *tsdb.DB { return db }, } opts.Flags = map[string]string{}