Unify Iterator interfaces. All point to storage now.

This is part of https://github.com/prometheus/prometheus/pull/5882 that can be done to simplify things.
All todos I added will be fixed in follow up PRs.

* querier.Querier, querier.Appender, querier.SeriesSet, and querier.Series interfaces merged
with storage interface.go. All imports that.
* querier.SeriesIterator replaced by chunkenc.Iterator
* Added chunkenc.Iterator.Seek method and tests for xor implementation (?)
* Since we properly handle SelectParams for Select methods I adjusted min max
based on that. This should help in terms of performance for queries with functions like offset.
* added Seek to deletedIterator and test.
* storage/tsdb was removed as it was only a unnecessary glue with incompatible structs.

No logic was changed, only different source of abstractions, so no need for benchmarks.

Signed-off-by: Bartlomiej Plotka <bwplotka@gmail.com>
This commit is contained in:
Bartlomiej Plotka 2020-02-06 15:58:38 +00:00
parent 489a9aa7b9
commit 34426766d8
40 changed files with 937 additions and 1054 deletions

View file

@ -56,7 +56,7 @@ import (
"github.com/prometheus/prometheus/scrape" "github.com/prometheus/prometheus/scrape"
"github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/storage"
"github.com/prometheus/prometheus/storage/remote" "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/util/strutil"
"github.com/prometheus/prometheus/web" "github.com/prometheus/prometheus/web"
) )

View file

@ -428,10 +428,7 @@ func (t *Test) exec(tc testCommand) error {
t.clear() t.clear()
case *loadCmd: case *loadCmd:
app, err := t.storage.Appender() app := t.storage.Appender()
if err != nil {
return err
}
if err := cmd.append(app); err != nil { if err := cmd.append(app); err != nil {
app.Rollback() app.Rollback()
return err 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). // appendTill appends the defined time series to the storage till the given timestamp (in milliseconds).
func (ll *LazyLoader) appendTill(ts int64) error { func (ll *LazyLoader) appendTill(ts int64) error {
app, err := ll.storage.Appender() app := ll.storage.Appender()
if err != nil {
return err
}
for h, smpls := range ll.loadCmd.defs { for h, smpls := range ll.loadCmd.defs {
m := ll.loadCmd.metrics[h] m := ll.loadCmd.metrics[h]
for i, s := range smpls { for i, s := range smpls {

View file

@ -20,6 +20,7 @@ import (
"strings" "strings"
"github.com/pkg/errors" "github.com/pkg/errors"
"github.com/prometheus/prometheus/tsdb/chunkenc"
"github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/pkg/labels"
"github.com/prometheus/prometheus/storage" "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. // 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) return newStorageSeriesIterator(ss.series)
} }

View file

@ -587,12 +587,7 @@ func (g *Group) Eval(ctx context.Context, ts time.Time) {
numDuplicates = 0 numDuplicates = 0
) )
app, err := g.opts.Appendable.Appender() app := g.opts.Appendable.Appender()
if err != nil {
level.Warn(g.logger).Log("msg", "creating appender failed", "err", err)
return
}
seriesReturned := make(map[string]labels.Labels, len(g.seriesInPreviousEval[i])) seriesReturned := make(map[string]labels.Labels, len(g.seriesInPreviousEval[i]))
for _, s := range vector { for _, s := range vector {
if _, err := app.Add(s.Metric, s.T, s.V); err != nil { 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 { if len(g.staleSeries) == 0 {
return return
} }
app, err := g.opts.Appendable.Appender() app := g.opts.Appendable.Appender()
if err != nil {
level.Warn(g.logger).Log("msg", "creating appender failed", "err", err)
return
}
for _, s := range g.staleSeries { for _, s := range g.staleSeries {
// Rule that produced series no longer configured, mark it stale. // 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 { switch err {
case nil: case nil:
case storage.ErrOutOfOrderSample, storage.ErrDuplicateSampleForTimestamp: case storage.ErrOutOfOrderSample, storage.ErrDuplicateSampleForTimestamp:
@ -836,11 +827,6 @@ type Manager struct {
logger log.Logger 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. // NotifyFunc sends notifications about a set of alerts generated by the given expression.
type NotifyFunc func(ctx context.Context, expr string, alerts ...*Alert) type NotifyFunc func(ctx context.Context, expr string, alerts ...*Alert)
@ -850,7 +836,7 @@ type ManagerOptions struct {
QueryFunc QueryFunc QueryFunc QueryFunc
NotifyFunc NotifyFunc NotifyFunc NotifyFunc
Context context.Context Context context.Context
Appendable Appendable Appendable storage.Appendable
TSDB storage.Storage TSDB storage.Storage
Logger log.Logger Logger log.Logger
Registerer prometheus.Registerer Registerer prometheus.Registerer

View file

@ -540,7 +540,7 @@ func TestStaleness(t *testing.T) {
}) })
// A time series that has two samples and then goes stale. // 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"), 0, 1)
app.Add(labels.FromStrings(model.MetricNameLabel, "a"), 1000, 2) app.Add(labels.FromStrings(model.MetricNameLabel, "a"), 1000, 2)
app.Add(labels.FromStrings(model.MetricNameLabel, "a"), 2000, math.Float64frombits(value.StaleNaN)) app.Add(labels.FromStrings(model.MetricNameLabel, "a"), 2000, math.Float64frombits(value.StaleNaN))
@ -868,7 +868,7 @@ func TestNotify(t *testing.T) {
Opts: opts, Opts: opts,
}) })
app, _ := storage.Appender() app := storage.Appender()
app.Add(labels.FromStrings(model.MetricNameLabel, "a"), 1000, 2) 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"), 2000, 3)
app.Add(labels.FromStrings(model.MetricNameLabel, "a"), 5000, 3) app.Add(labels.FromStrings(model.MetricNameLabel, "a"), 5000, 3)

View file

@ -20,16 +20,16 @@ import (
type nopAppendable struct{} type nopAppendable struct{}
func (a nopAppendable) Appender() (storage.Appender, error) { func (a nopAppendable) Appender() storage.Appender {
return nopAppender{}, nil return nopAppender{}
} }
type nopAppender struct{} type nopAppender struct{}
func (a nopAppender) Add(labels.Labels, int64, float64) (uint64, error) { return 0, nil } 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) AddFast(uint64, int64, float64) error { return nil }
func (a nopAppender) Commit() error { return nil } func (a nopAppender) Commit() error { return nil }
func (a nopAppender) Rollback() error { return nil } func (a nopAppender) Rollback() error { return nil }
type sample struct { type sample struct {
metric labels.Labels metric labels.Labels
@ -42,18 +42,21 @@ type sample struct {
type collectResultAppender struct { type collectResultAppender struct {
next storage.Appender next storage.Appender
result []sample 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 { if a.next == nil {
return storage.ErrNotFound return storage.ErrNotFound
} }
err := a.next.AddFast(m, ref, t, v)
err := a.next.AddFast(ref, t, v)
if err != nil { if err != nil {
return err return err
} }
a.result = append(a.result, sample{ a.result = append(a.result, sample{
metric: m, metric: a.mapper[ref],
t: t, t: t,
v: v, v: v,
}) })
@ -69,7 +72,17 @@ func (a *collectResultAppender) Add(m labels.Labels, t int64, v float64) (uint64
if a.next == nil { if a.next == nil {
return 0, 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 } func (a *collectResultAppender) Commit() error { return nil }

View file

@ -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 // 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 { if logger == nil {
logger = log.NewNopLogger() logger = log.NewNopLogger()
} }
@ -127,7 +122,7 @@ func NewManager(logger log.Logger, app Appendable) *Manager {
// when receiving new target groups form the discovery manager. // when receiving new target groups form the discovery manager.
type Manager struct { type Manager struct {
logger log.Logger logger log.Logger
append Appendable append storage.Appendable
graceShut chan struct{} graceShut chan struct{}
jitterSeed uint64 // Global jitterSeed seed is used to spread scrape workload across HA setup. jitterSeed uint64 // Global jitterSeed seed is used to spread scrape workload across HA setup.

View file

@ -156,7 +156,7 @@ func init() {
// scrapePool manages scrapes for sets of targets. // scrapePool manages scrapes for sets of targets.
type scrapePool struct { type scrapePool struct {
appendable Appendable appendable storage.Appendable
logger log.Logger logger log.Logger
mtx sync.RWMutex mtx sync.RWMutex
@ -187,7 +187,7 @@ const maxAheadTime = 10 * time.Minute
type labelsMutator func(labels.Labels) labels.Labels 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() targetScrapePools.Inc()
if logger == nil { if logger == nil {
logger = log.NewNopLogger() 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) return mutateSampleLabels(l, opts.target, opts.honorLabels, opts.mrc)
}, },
func(l labels.Labels) labels.Labels { return mutateReportSampleLabels(l, opts.target) }, func(l labels.Labels) labels.Labels { return mutateReportSampleLabels(l, opts.target) },
func() storage.Appender { func() storage.Appender { return appender(app.Appender(), opts.limit) },
app, err := app.Appender()
if err != nil {
panic(err)
}
return appender(app, opts.limit)
},
cache, cache,
jitterSeed, jitterSeed,
opts.honorTimestamps, opts.honorTimestamps,
@ -1112,7 +1106,7 @@ loop:
} }
ce, ok := sl.cache.get(yoloString(met)) ce, ok := sl.cache.get(yoloString(met))
if ok { if ok {
switch err = app.AddFast(ce.lset, ce.ref, t, v); err { switch err = app.AddFast(ce.ref, t, v); err {
case nil: case nil:
if tp == nil { if tp == nil {
sl.cache.trackStaleness(ce.hash, ce.lset) 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 { func (sl *scrapeLoop) addReportSample(app storage.Appender, s string, t int64, v float64) error {
ce, ok := sl.cache.get(s) ce, ok := sl.cache.get(s)
if ok { if ok {
err := app.AddFast(ce.lset, ce.ref, t, v) err := app.AddFast(ce.ref, t, v)
switch err { switch err {
case nil: case nil:
return nil return nil

View file

@ -644,8 +644,7 @@ func TestScrapeLoopSeriesAdded(t *testing.T) {
s := teststorage.New(t) s := teststorage.New(t)
defer s.Close() defer s.Close()
app, err := s.Appender() app := s.Appender()
testutil.Ok(t, err)
ctx, cancel := context.WithCancel(context.Background()) ctx, cancel := context.WithCancel(context.Background())
sl := newScrapeLoop(ctx, sl := newScrapeLoop(ctx,
@ -788,8 +787,7 @@ func TestScrapeLoopCache(t *testing.T) {
s := teststorage.New(t) s := teststorage.New(t)
defer s.Close() defer s.Close()
sapp, err := s.Appender() sapp := s.Appender()
testutil.Ok(t, err)
appender := &collectResultAppender{next: sapp} appender := &collectResultAppender{next: sapp}
var ( var (
@ -866,8 +864,7 @@ func TestScrapeLoopCacheMemoryExhaustionProtection(t *testing.T) {
s := teststorage.New(t) s := teststorage.New(t)
defer s.Close() defer s.Close()
sapp, err := s.Appender() sapp := s.Appender()
testutil.Ok(t, err)
appender := &collectResultAppender{next: sapp} appender := &collectResultAppender{next: sapp}
var ( var (
@ -1092,8 +1089,7 @@ func TestScrapeLoop_ChangingMetricString(t *testing.T) {
s := teststorage.New(t) s := teststorage.New(t)
defer s.Close() defer s.Close()
app, err := s.Appender() app := s.Appender()
testutil.Ok(t, err)
capp := &collectResultAppender{next: app} capp := &collectResultAppender{next: app}
@ -1108,7 +1104,7 @@ func TestScrapeLoop_ChangingMetricString(t *testing.T) {
) )
now := time.Now() 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) testutil.Ok(t, err)
_, _, _, err = sl.append([]byte(`metric_a{b="1",a="1"} 2`), "", now.Add(time.Minute)) _, _, _, 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 { func (app *errorAppender) AddFast(ref uint64, t int64, v float64) error {
return app.collectResultAppender.AddFast(lset, ref, t, v) return app.collectResultAppender.AddFast(ref, t, v)
} }
func TestScrapeLoopAppendGracefullyIfAmendOrOutOfOrderOrOutOfBounds(t *testing.T) { func TestScrapeLoopAppendGracefullyIfAmendOrOutOfOrderOrOutOfBounds(t *testing.T) {
@ -1498,8 +1494,7 @@ func TestScrapeLoop_RespectTimestamps(t *testing.T) {
s := teststorage.New(t) s := teststorage.New(t)
defer s.Close() defer s.Close()
app, err := s.Appender() app := s.Appender()
testutil.Ok(t, err)
capp := &collectResultAppender{next: app} capp := &collectResultAppender{next: app}
@ -1513,7 +1508,7 @@ func TestScrapeLoop_RespectTimestamps(t *testing.T) {
) )
now := time.Now() 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) testutil.Ok(t, err)
want := []sample{ want := []sample{
@ -1530,8 +1525,7 @@ func TestScrapeLoop_DiscardTimestamps(t *testing.T) {
s := teststorage.New(t) s := teststorage.New(t)
defer s.Close() defer s.Close()
app, err := s.Appender() app := s.Appender()
testutil.Ok(t, err)
capp := &collectResultAppender{next: app} capp := &collectResultAppender{next: app}
@ -1545,7 +1539,7 @@ func TestScrapeLoop_DiscardTimestamps(t *testing.T) {
) )
now := time.Now() 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) testutil.Ok(t, err)
want := []sample{ want := []sample{
@ -1562,8 +1556,7 @@ func TestScrapeLoopDiscardDuplicateLabels(t *testing.T) {
s := teststorage.New(t) s := teststorage.New(t)
defer s.Close() defer s.Close()
app, err := s.Appender() app := s.Appender()
testutil.Ok(t, err)
ctx, cancel := context.WithCancel(context.Background()) ctx, cancel := context.WithCancel(context.Background())
sl := newScrapeLoop(ctx, sl := newScrapeLoop(ctx,
@ -1579,7 +1572,7 @@ func TestScrapeLoopDiscardDuplicateLabels(t *testing.T) {
defer cancel() defer cancel()
// We add a good and a bad metric to check that both are discarded. // 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) testutil.NotOk(t, err)
q, err := s.Querier(ctx, time.Time{}.UnixNano(), 0) q, err := s.Querier(ctx, time.Time{}.UnixNano(), 0)

View file

@ -303,14 +303,14 @@ func (app *limitAppender) Add(lset labels.Labels, t int64, v float64) (uint64, e
return ref, nil 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) { if !value.IsStaleNaN(v) {
app.i++ app.i++
if app.i > app.limit { if app.i > app.limit {
return errSampleLimit return errSampleLimit
} }
} }
err := app.Appender.AddFast(lset, ref, t, v) err := app.Appender.AddFast(ref, t, v)
return err return err
} }
@ -332,11 +332,11 @@ func (app *timeLimitAppender) Add(lset labels.Labels, t int64, v float64) (uint6
return ref, nil 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 { if t > app.maxTime {
return storage.ErrOutOfBounds return storage.ErrOutOfBounds
} }
err := app.Appender.AddFast(lset, ref, t, v) err := app.Appender.AddFast(ref, t, v)
return err return err
} }

View file

@ -15,11 +15,13 @@ package storage
import ( import (
"math" "math"
"github.com/prometheus/prometheus/tsdb/chunkenc"
) )
// BufferedSeriesIterator wraps an iterator with a look-back buffer. // BufferedSeriesIterator wraps an iterator with a look-back buffer.
type BufferedSeriesIterator struct { type BufferedSeriesIterator struct {
it SeriesIterator it chunkenc.Iterator
buf *sampleRing buf *sampleRing
delta int64 delta int64
@ -36,7 +38,7 @@ func NewBuffer(delta int64) *BufferedSeriesIterator {
// NewBufferIterator returns a new iterator that buffers the values within the // NewBufferIterator returns a new iterator that buffers the values within the
// time range of the current element and the duration of delta before. // 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{ bit := &BufferedSeriesIterator{
buf: newSampleRing(delta, 16), buf: newSampleRing(delta, 16),
delta: delta, 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 // Reset re-uses the buffer with a new iterator, resetting the buffered time
// delta to its original value. // delta to its original value.
func (b *BufferedSeriesIterator) Reset(it SeriesIterator) { func (b *BufferedSeriesIterator) Reset(it chunkenc.Iterator) {
b.it = it b.it = it
b.lastTime = math.MinInt64 b.lastTime = math.MinInt64
b.ok = true 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 // Buffer returns an iterator over the buffered data. Invalidates previously
// returned iterators. // returned iterators.
func (b *BufferedSeriesIterator) Buffer() SeriesIterator { func (b *BufferedSeriesIterator) Buffer() chunkenc.Iterator {
return b.buf.iterator() return b.buf.iterator()
} }
@ -159,7 +161,7 @@ func (r *sampleRing) reset() {
} }
// Returns the current iterator. Invalidates previously returned iterators. // 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.r = r
r.it.i = -1 r.it.i = -1
return &r.it return &r.it

View file

@ -19,6 +19,7 @@ import (
"testing" "testing"
"github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/pkg/labels"
"github.com/prometheus/prometheus/tsdb/chunkenc"
"github.com/prometheus/prometheus/util/testutil" "github.com/prometheus/prometheus/util/testutil"
) )
@ -190,7 +191,7 @@ func (m *mockSeriesIterator) Err() error { return m.err() }
type mockSeries struct { type mockSeries struct {
labels func() labels.Labels labels func() labels.Labels
iterator func() SeriesIterator iterator func() chunkenc.Iterator
} }
func newMockSeries(lset labels.Labels, samples []sample) Series { func newMockSeries(lset labels.Labels, samples []sample) Series {
@ -198,14 +199,14 @@ func newMockSeries(lset labels.Labels, samples []sample) Series {
labels: func() labels.Labels { labels: func() labels.Labels {
return lset return lset
}, },
iterator: func() SeriesIterator { iterator: func() chunkenc.Iterator {
return newListSeriesIterator(samples) return newListSeriesIterator(samples)
}, },
} }
} }
func (m *mockSeries) Labels() labels.Labels { return m.labels() } func (m *mockSeries) Labels() labels.Labels { return m.labels() }
func (m *mockSeries) Iterator() SeriesIterator { return m.iterator() } func (m *mockSeries) Iterator() chunkenc.Iterator { return m.iterator() }
type listSeriesIterator struct { type listSeriesIterator struct {
list []sample list []sample

View file

@ -24,6 +24,7 @@ import (
"github.com/pkg/errors" "github.com/pkg/errors"
"github.com/prometheus/common/model" "github.com/prometheus/common/model"
"github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/pkg/labels"
"github.com/prometheus/prometheus/tsdb/chunkenc"
) )
type fanout struct { type fanout struct {
@ -87,25 +88,17 @@ func (f *fanout) Querier(ctx context.Context, mint, maxt int64) (Querier, error)
return NewMergeQuerier(primaryQuerier, queriers), nil return NewMergeQuerier(primaryQuerier, queriers), nil
} }
func (f *fanout) Appender() (Appender, error) { func (f *fanout) Appender() Appender {
primary, err := f.primary.Appender() primary := f.primary.Appender()
if err != nil {
return nil, err
}
secondaries := make([]Appender, 0, len(f.secondaries)) secondaries := make([]Appender, 0, len(f.secondaries))
for _, storage := range f.secondaries { for _, storage := range f.secondaries {
appender, err := storage.Appender() secondaries = append(secondaries, storage.Appender())
if err != nil {
return nil, err
}
secondaries = append(secondaries, appender)
} }
return &fanoutAppender{ return &fanoutAppender{
logger: f.logger, logger: f.logger,
primary: primary, primary: primary,
secondaries: secondaries, secondaries: secondaries,
}, nil }
} }
// Close closes the storage and all its underlying resources. // 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 return ref, nil
} }
func (f *fanoutAppender) AddFast(l labels.Labels, ref uint64, t int64, v float64) error { func (f *fanoutAppender) AddFast(ref uint64, t int64, v float64) error {
if err := f.primary.AddFast(l, ref, t, v); err != nil { if err := f.primary.AddFast(ref, t, v); err != nil {
return err return err
} }
for _, appender := range f.secondaries { 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 return err
} }
} }
@ -521,8 +514,8 @@ func (m *mergeSeries) Labels() labels.Labels {
return m.labels return m.labels
} }
func (m *mergeSeries) Iterator() SeriesIterator { func (m *mergeSeries) Iterator() chunkenc.Iterator {
iterators := make([]SeriesIterator, 0, len(m.series)) iterators := make([]chunkenc.Iterator, 0, len(m.series))
for _, s := range m.series { for _, s := range m.series {
iterators = append(iterators, s.Iterator()) iterators = append(iterators, s.Iterator())
} }
@ -530,11 +523,11 @@ func (m *mergeSeries) Iterator() SeriesIterator {
} }
type mergeIterator struct { type mergeIterator struct {
iterators []SeriesIterator iterators []chunkenc.Iterator
h seriesIteratorHeap h seriesIteratorHeap
} }
func newMergeIterator(iterators []SeriesIterator) SeriesIterator { func newMergeIterator(iterators []chunkenc.Iterator) chunkenc.Iterator {
return &mergeIterator{ return &mergeIterator{
iterators: iterators, iterators: iterators,
h: nil, h: nil,
@ -581,7 +574,7 @@ func (c *mergeIterator) Next() bool {
break break
} }
iter := heap.Pop(&c.h).(SeriesIterator) iter := heap.Pop(&c.h).(chunkenc.Iterator)
if iter.Next() { if iter.Next() {
heap.Push(&c.h, iter) heap.Push(&c.h, iter)
} }
@ -599,7 +592,7 @@ func (c *mergeIterator) Err() error {
return nil return nil
} }
type seriesIteratorHeap []SeriesIterator type seriesIteratorHeap []chunkenc.Iterator
func (h seriesIteratorHeap) Len() int { return len(h) } func (h seriesIteratorHeap) Len() int { return len(h) }
func (h seriesIteratorHeap) Swap(i, j int) { h[i], h[j] = h[j], h[i] } 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{}) { func (h *seriesIteratorHeap) Push(x interface{}) {
*h = append(*h, x.(SeriesIterator)) *h = append(*h, x.(chunkenc.Iterator))
} }
func (h *seriesIteratorHeap) Pop() interface{} { func (h *seriesIteratorHeap) Pop() interface{} {

View file

@ -19,6 +19,7 @@ import (
"testing" "testing"
"github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/pkg/labels"
"github.com/prometheus/prometheus/tsdb/chunkenc"
"github.com/prometheus/prometheus/util/testutil" "github.com/prometheus/prometheus/util/testutil"
) )
@ -122,24 +123,24 @@ func TestMergeSeriesSet(t *testing.T) {
func TestMergeIterator(t *testing.T) { func TestMergeIterator(t *testing.T) {
for _, tc := range []struct { for _, tc := range []struct {
input []SeriesIterator input []chunkenc.Iterator
expected []sample expected []sample
}{ }{
{ {
input: []SeriesIterator{ input: []chunkenc.Iterator{
newListSeriesIterator([]sample{{0, 0}, {1, 1}}), newListSeriesIterator([]sample{{0, 0}, {1, 1}}),
}, },
expected: []sample{{0, 0}, {1, 1}}, expected: []sample{{0, 0}, {1, 1}},
}, },
{ {
input: []SeriesIterator{ input: []chunkenc.Iterator{
newListSeriesIterator([]sample{{0, 0}, {1, 1}}), newListSeriesIterator([]sample{{0, 0}, {1, 1}}),
newListSeriesIterator([]sample{{2, 2}, {3, 3}}), newListSeriesIterator([]sample{{2, 2}, {3, 3}}),
}, },
expected: []sample{{0, 0}, {1, 1}, {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{{0, 0}, {3, 3}}),
newListSeriesIterator([]sample{{1, 1}, {4, 4}}), newListSeriesIterator([]sample{{1, 1}, {4, 4}}),
newListSeriesIterator([]sample{{2, 2}, {5, 5}}), 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}}, 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}, {1, 1}}),
newListSeriesIterator([]sample{{0, 0}, {2, 2}}), newListSeriesIterator([]sample{{0, 0}, {2, 2}}),
newListSeriesIterator([]sample{{2, 2}, {3, 3}}), newListSeriesIterator([]sample{{2, 2}, {3, 3}}),
@ -163,19 +164,19 @@ func TestMergeIterator(t *testing.T) {
func TestMergeIteratorSeek(t *testing.T) { func TestMergeIteratorSeek(t *testing.T) {
for _, tc := range []struct { for _, tc := range []struct {
input []SeriesIterator input []chunkenc.Iterator
seek int64 seek int64
expected []sample expected []sample
}{ }{
{ {
input: []SeriesIterator{ input: []chunkenc.Iterator{
newListSeriesIterator([]sample{{0, 0}, {1, 1}, {2, 2}}), newListSeriesIterator([]sample{{0, 0}, {1, 1}, {2, 2}}),
}, },
seek: 1, seek: 1,
expected: []sample{{1, 1}, {2, 2}}, expected: []sample{{1, 1}, {2, 2}},
}, },
{ {
input: []SeriesIterator{ input: []chunkenc.Iterator{
newListSeriesIterator([]sample{{0, 0}, {1, 1}}), newListSeriesIterator([]sample{{0, 0}, {1, 1}}),
newListSeriesIterator([]sample{{2, 2}, {3, 3}}), newListSeriesIterator([]sample{{2, 2}, {3, 3}}),
}, },
@ -183,7 +184,7 @@ func TestMergeIteratorSeek(t *testing.T) {
expected: []sample{{2, 2}, {3, 3}}, expected: []sample{{2, 2}, {3, 3}},
}, },
{ {
input: []SeriesIterator{ input: []chunkenc.Iterator{
newListSeriesIterator([]sample{{0, 0}, {3, 3}}), newListSeriesIterator([]sample{{0, 0}, {3, 3}}),
newListSeriesIterator([]sample{{1, 1}, {4, 4}}), newListSeriesIterator([]sample{{1, 1}, {4, 4}}),
newListSeriesIterator([]sample{{2, 2}, {5, 5}}), 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{} result := []sample{}
for iter.Next() { for iter.Next() {
t, v := iter.At() t, v := iter.At()

View file

@ -18,6 +18,9 @@ import (
"errors" "errors"
"github.com/prometheus/prometheus/pkg/labels" "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. // The errors exposed.
@ -28,17 +31,21 @@ var (
ErrOutOfBounds = errors.New("out of bounds") 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 // Storage ingests and manages samples, along with various indexes. All methods
// are goroutine-safe. Storage implements storage.SampleAppender. // are goroutine-safe. Storage implements storage.SampleAppender.
type Storage interface { type Storage interface {
Queryable Queryable
Appendable
// StartTime returns the oldest timestamp stored in the storage. // StartTime returns the oldest timestamp stored in the storage.
StartTime() (int64, error) StartTime() (int64, error)
// Appender returns a new appender against the storage.
Appender() (Appender, error)
// Close closes the storage and all its underlying resources. // Close closes the storage and all its underlying resources.
Close() error Close() error
} }
@ -49,7 +56,8 @@ type Queryable interface {
Querier(ctx context.Context, mint, maxt int64) (Querier, error) 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 { type Querier interface {
// Select returns a set of series that matches the given label matchers. // Select returns a set of series that matches the given label matchers.
Select(*SelectParams, ...*labels.Matcher) (SeriesSet, Warnings, error) Select(*SelectParams, ...*labels.Matcher) (SeriesSet, Warnings, error)
@ -58,6 +66,7 @@ type Querier interface {
SelectSorted(*SelectParams, ...*labels.Matcher) (SeriesSet, Warnings, error) SelectSorted(*SelectParams, ...*labels.Matcher) (SeriesSet, Warnings, error)
// LabelValues returns all potential values for a label name. // 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) LabelValues(name string) ([]string, Warnings, error)
// LabelNames returns all the unique label names present in the block in sorted order. // 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 { type Appender interface {
Add(l labels.Labels, t int64, v float64) (uint64, error) 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 submits the collected samples and purges the batch.
Commit() error Commit() error
@ -108,25 +117,36 @@ type SeriesSet interface {
Err() error 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. // Series represents a single time series.
type Series interface { type Series interface {
// Labels returns the complete set of labels identifying the series. // Labels returns the complete set of labels identifying the series.
Labels() labels.Labels Labels() labels.Labels
// Iterator returns a new iterator of the data of the series. // 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. // ChunkSeriesSet exposes the chunks and intervals of a series instead of the
type SeriesIterator interface { // actual series itself.
// Seek advances the iterator forward to the value at or after // TODO(bwplotka): Move it to Series liike Iterator that iterates over chunks and avoiding loading all of them at once.
// the given timestamp. type ChunkSeriesSet interface {
Seek(t int64) bool
// At returns the current timestamp/value pair.
At() (t int64, v float64)
// Next advances the iterator by one.
Next() bool Next() bool
// Err returns the current error. At() (labels.Labels, []chunks.Meta, tombstones.Intervals)
Err() error Err() error
} }

View file

@ -254,7 +254,7 @@ func StreamChunkedReadResponses(
} }
// encodeChunks expects iterator to be ready to use (aka iter.Next() called before invoking). // 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 const maxSamplesInChunk = 120
var ( var (
@ -392,7 +392,7 @@ func (c *concreteSeries) Labels() labels.Labels {
return labels.New(c.labels...) return labels.New(c.labels...)
} }
func (c *concreteSeries) Iterator() storage.SeriesIterator { func (c *concreteSeries) Iterator() chunkenc.Iterator {
return newConcreteSeriersIterator(c) return newConcreteSeriersIterator(c)
} }
@ -402,7 +402,7 @@ type concreteSeriesIterator struct {
series *concreteSeries series *concreteSeries
} }
func newConcreteSeriersIterator(series *concreteSeries) storage.SeriesIterator { func newConcreteSeriersIterator(series *concreteSeries) chunkenc.Iterator {
return &concreteSeriesIterator{ return &concreteSeriesIterator{
cur: -1, cur: -1,
series: series, series: series,

View file

@ -149,7 +149,7 @@ func (s *Storage) Querier(ctx context.Context, mint, maxt int64) (storage.Querie
} }
// Appender implements storage.Storage. // Appender implements storage.Storage.
func (s *Storage) Appender() (storage.Appender, error) { func (s *Storage) Appender() storage.Appender {
return s.rws.Appender() return s.rws.Appender()
} }

View file

@ -183,10 +183,10 @@ func (rws *WriteStorage) ApplyConfig(conf *config.Config) error {
} }
// Appender implements storage.Storage. // Appender implements storage.Storage.
func (rws *WriteStorage) Appender() (storage.Appender, error) { func (rws *WriteStorage) Appender() storage.Appender {
return &timestampTracker{ return &timestampTracker{
writeStorage: rws, writeStorage: rws,
}, nil }
} }
// Close closes the WriteStorage. // Close closes the WriteStorage.
@ -206,7 +206,7 @@ type timestampTracker struct {
} }
// Add implements storage.Appender. // 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++ t.samples++
if ts > t.highestTimestamp { if ts > t.highestTimestamp {
t.highestTimestamp = ts t.highestTimestamp = ts
@ -215,8 +215,8 @@ func (t *timestampTracker) Add(_ labels.Labels, ts int64, v float64) (uint64, er
} }
// AddFast implements storage.Appender. // AddFast implements storage.Appender.
func (t *timestampTracker) AddFast(l labels.Labels, _ uint64, ts int64, v float64) error { func (t *timestampTracker) AddFast(_ uint64, ts int64, v float64) error {
_, err := t.Add(l, ts, v) _, err := t.Add(nil, ts, v)
return err return err
} }

View file

@ -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() }

View file

@ -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
)

View file

@ -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())
}

View file

@ -124,12 +124,6 @@ type BlockReader interface {
Meta() BlockMeta 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. // BlockMeta provides meta information about a block.
type BlockMeta struct { type BlockMeta struct {
// Unique identifier for the block and its contents. Changes on compaction. // Unique identifier for the block and its contents. Changes on compaction.

View file

@ -28,6 +28,7 @@ import (
"github.com/go-kit/kit/log" "github.com/go-kit/kit/log"
"github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/pkg/labels"
"github.com/prometheus/prometheus/storage"
"github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/chunks"
"github.com/prometheus/prometheus/tsdb/fileutil" "github.com/prometheus/prometheus/tsdb/fileutil"
"github.com/prometheus/prometheus/tsdb/tsdbutil" "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}}) 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)) files, err := sequenceFiles(chunkDir(blockDir))
testutil.Ok(t, err) testutil.Ok(t, err)
testutil.Assert(t, len(files) > 0, "No chunk created.") testutil.Assert(t, len(files) > 0, "No chunk created.")
@ -202,8 +203,9 @@ func TestCorruptedChunk(t *testing.T) {
querier, err := NewBlockQuerier(b, 0, 1) querier, err := NewBlockQuerier(b, 0, 1)
testutil.Ok(t, err) testutil.Ok(t, err)
defer func() { testutil.Ok(t, querier.Close()) }() 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.Ok(t, err)
testutil.Equals(t, 0, len(ws))
// Check query err. // Check query err.
testutil.Equals(t, false, set.Next()) 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. // 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)) 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()) 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) head, err := NewHead(nil, nil, nil, 2*60*60*1000, DefaultStripeSize)
testutil.Ok(tb, err) testutil.Ok(tb, err)
defer head.Close() defer head.Close()
@ -352,12 +354,12 @@ const (
) )
// genSeries generates series with a given number of labels and values. // 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 { if totalSeries == 0 || labelCount == 0 {
return nil return nil
} }
series := make([]Series, totalSeries) series := make([]storage.Series, totalSeries)
for i := 0; i < totalSeries; i++ { for i := 0; i < totalSeries; i++ {
lbls := make(map[string]string, labelCount) 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. // 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 { if len(lbls) == 0 {
return nil return nil
} }
series := make([]Series, 0, len(lbls)) series := make([]storage.Series, 0, len(lbls))
for _, lbl := range lbls { for _, lbl := range lbls {
if len(lbl) == 0 { if len(lbl) == 0 {
continue continue

View file

@ -15,6 +15,7 @@ package chunkenc
import ( import (
"fmt" "fmt"
"math"
"sync" "sync"
"github.com/pkg/errors" "github.com/pkg/errors"
@ -72,10 +73,22 @@ type Appender interface {
} }
// Iterator is a simple iterator that can only get the next value. // Iterator is a simple iterator that can only get the next value.
// Iterator iterates over the samples of a time series.
type Iterator interface { 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) At() (int64, float64)
Err() error // Next advances the iterator by one.
Next() bool Next() bool
// Err returns the current error.
Err() error
} }
// NewNopIterator returns a new chunk iterator that does not hold any data. // NewNopIterator returns a new chunk iterator that does not hold any data.
@ -85,7 +98,8 @@ func NewNopIterator() Iterator {
type nopIterator struct{} 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) Next() bool { return false }
func (nopIterator) Err() error { return nil } func (nopIterator) Err() error { return nil }

View file

@ -17,7 +17,6 @@ import (
"fmt" "fmt"
"io" "io"
"math/rand" "math/rand"
"reflect"
"testing" "testing"
"github.com/prometheus/prometheus/util/testutil" "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) { t.Run(fmt.Sprintf("%v", enc), func(t *testing.T) {
for range make([]struct{}, 1) { for range make([]struct{}, 1) {
c := nc() c := nc()
if err := testChunk(c); err != nil { testChunk(t, c)
t.Fatal(err)
}
} }
}) })
} }
} }
func testChunk(c Chunk) error { func testChunk(t *testing.T, c Chunk) {
app, err := c.Appender() app, err := c.Appender()
if err != nil { testutil.Ok(t, err)
return err
}
var exp []pair var exp []pair
var ( var (
@ -56,7 +51,6 @@ func testChunk(c Chunk) error {
) )
for i := 0; i < 300; i++ { for i := 0; i < 300; i++ {
ts += int64(rand.Intn(10000) + 1) ts += int64(rand.Intn(10000) + 1)
// v = rand.Float64()
if i%2 == 0 { if i%2 == 0 {
v += float64(rand.Intn(1000000)) v += float64(rand.Intn(1000000))
} else { } else {
@ -67,29 +61,52 @@ func testChunk(c Chunk) error {
// appending to a partially filled chunk. // appending to a partially filled chunk.
if i%10 == 0 { if i%10 == 0 {
app, err = c.Appender() app, err = c.Appender()
if err != nil { testutil.Ok(t, err)
return err
}
} }
app.Append(ts, v) app.Append(ts, v)
exp = append(exp, pair{t: ts, v: v}) exp = append(exp, pair{t: ts, v: v})
// fmt.Println("appended", len(c.Bytes()), c.Bytes())
} }
it := c.Iterator(nil) // 1. Expand iterator in simple case.
var res []pair it1 := c.Iterator(nil)
for it.Next() { var res1 []pair
ts, v := it.At() for it1.Next() {
res = append(res, pair{t: ts, v: v}) ts, v := it1.At()
res1 = append(res1, pair{t: ts, v: v})
} }
if it.Err() != nil { testutil.Ok(t, it1.Err())
return it.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) { testutil.Ok(t, it2.Err())
return fmt.Errorf("unexpected result\n\ngot: %v\n\nexp: %v", res, exp) 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) { func benchmarkIterator(b *testing.B, newChunk func() Chunk) {

View file

@ -253,6 +253,19 @@ type xorIterator struct {
err error 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) { func (it *xorIterator) At() (int64, float64) {
return it.t, it.val return it.t, it.val
} }

View file

@ -15,6 +15,7 @@ package main
import ( import (
"bufio" "bufio"
"context"
"fmt" "fmt"
"io" "io"
"io/ioutil" "io/ioutil"
@ -32,11 +33,12 @@ import (
"github.com/go-kit/kit/log" "github.com/go-kit/kit/log"
"github.com/pkg/errors" "github.com/pkg/errors"
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/pkg/labels"
"github.com/prometheus/prometheus/tsdb" "github.com/prometheus/prometheus/tsdb"
"github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/chunks"
tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" tsdb_errors "github.com/prometheus/prometheus/tsdb/errors"
kingpin "gopkg.in/alecthomas/kingpin.v2" "gopkg.in/alecthomas/kingpin.v2"
) )
func main() { func main() {
@ -176,8 +178,8 @@ func (b *writeBenchmark) run() error {
l := log.With(b.logger, "ts", log.DefaultTimestampUTC, "caller", log.DefaultCaller) l := log.With(b.logger, "ts", log.DefaultTimestampUTC, "caller", log.DefaultCaller)
st, err := tsdb.Open(dir, l, nil, &tsdb.Options{ st, err := tsdb.Open(dir, l, nil, &tsdb.Options{
RetentionDuration: 15 * 24 * 60 * 60 * 1000, // 15 days in milliseconds RetentionDuration: 15 * 24 * model.Duration(time.Hour),
BlockRanges: tsdb.ExponentialBlockRanges(2*60*60*1000, 5, 3), MinBlockDuration: 2 * model.Duration(time.Hour),
}) })
if err != nil { if err != nil {
return err return err
@ -604,8 +606,7 @@ func analyzeBlock(b tsdb.BlockReader, limit int) error {
} }
func dumpSamples(db *tsdb.DBReadOnly, mint, maxt int64) (err error) { func dumpSamples(db *tsdb.DBReadOnly, mint, maxt int64) (err error) {
q, err := db.Querier(context.TODO(), mint, maxt)
q, err := db.Querier(mint, maxt)
if err != nil { if err != nil {
return err return err
} }
@ -616,11 +617,19 @@ func dumpSamples(db *tsdb.DBReadOnly, mint, maxt int64) (err error) {
err = merr.Err() err = merr.Err()
}() }()
ss, err := q.Select(labels.MustNewMatcher(labels.MatchRegexp, "", ".*")) ss, ws, err := q.Select(nil, labels.MustNewMatcher(labels.MatchRegexp, "", ".*"))
if err != nil { if err != nil {
return err return err
} }
if len(ws) > 0 {
var merr tsdb_errors.MultiError
for _, w := range ws {
merr.Add(w)
}
return merr.Err()
}
for ss.Next() { for ss.Next() {
series := ss.At() series := ss.At()
labels := series.Labels() labels := series.Labels()

View file

@ -30,6 +30,7 @@ import (
"github.com/pkg/errors" "github.com/pkg/errors"
"github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/pkg/labels"
"github.com/prometheus/prometheus/storage"
"github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/tsdb/chunkenc"
"github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/chunks"
tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" tsdb_errors "github.com/prometheus/prometheus/tsdb/errors"
@ -649,7 +650,7 @@ func (c *LeveledCompactor) populateBlock(blocks []BlockReader, meta *BlockMeta,
} }
var ( var (
set ChunkSeriesSet set storage.ChunkSeriesSet
symbols index.StringIter symbols index.StringIter
closers = []io.Closer{} closers = []io.Closer{}
overlapping bool overlapping bool
@ -916,7 +917,7 @@ func (c *compactionSeriesSet) At() (labels.Labels, []chunks.Meta, tombstones.Int
} }
type compactionMerger struct { type compactionMerger struct {
a, b ChunkSeriesSet a, b storage.ChunkSeriesSet
aok, bok bool aok, bok bool
l labels.Labels l labels.Labels
@ -924,7 +925,7 @@ type compactionMerger struct {
intervals tombstones.Intervals intervals tombstones.Intervals
} }
func newCompactionMerger(a, b ChunkSeriesSet) (*compactionMerger, error) { func newCompactionMerger(a, b storage.ChunkSeriesSet) (*compactionMerger, error) {
c := &compactionMerger{ c := &compactionMerger{
a: a, a: a,
b: b, b: b,

View file

@ -895,13 +895,13 @@ func BenchmarkCompactionFromHead(b *testing.B) {
// This is needed for unit tests that rely on // This is needed for unit tests that rely on
// checking state before and after a compaction. // checking state before and after a compaction.
func TestDisableAutoCompactions(t *testing.T) { func TestDisableAutoCompactions(t *testing.T) {
db, delete := openTestDB(t, nil) db, delete := openTestDB(t, nil, nil)
defer func() { defer func() {
testutil.Ok(t, db.Close()) testutil.Ok(t, db.Close())
delete() delete()
}() }()
blockRange := DefaultOptions.BlockRanges[0] blockRange := db.compactor.(*LeveledCompactor).ranges[0]
label := labels.FromStrings("foo", "bar") label := labels.FromStrings("foo", "bar")
// Trigger a compaction to check that it was skipped and // 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. // Measure the compaction time without interrupting it.
var timeCompactionUninterrupted time.Duration 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.Ok(t, err)
testutil.Equals(t, 3, len(db.Blocks()), "initial block count mismatch") 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") 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. // 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.Ok(t, err)
testutil.Equals(t, 3, len(db.Blocks()), "initial block count mismatch") 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") 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 // TestDeleteCompactionBlockAfterFailedReload ensures that a failed reload immediately after a compaction
// deletes the resulting block to avoid creatings blocks with the same time range. // deletes the resulting block to avoid creatings blocks with the same time range.
func TestDeleteCompactionBlockAfterFailedReload(t *testing.T) { func TestDeleteCompactionBlockAfterFailedReload(t *testing.T) {
tests := map[string]func(*DB) int{ tests := map[string]func(*DB) int{
"Test Head Compaction": func(db *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") defaultLabel := labels.FromStrings("foo", "bar")
// Add some data to the head that is enough to trigger a compaction. // 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 { for title, bootStrap := range tests {
t.Run(title, func(t *testing.T) { t.Run(title, func(t *testing.T) {
db, delete := openTestDB(t, &Options{ db, delete := openTestDB(t, nil, []int64{1, 100})
BlockRanges: []int64{1, 100},
})
defer func() { defer func() {
testutil.Ok(t, db.Close()) testutil.Ok(t, db.Close())
delete() delete()

View file

@ -29,12 +29,15 @@ import (
"sync" "sync"
"time" "time"
"github.com/alecthomas/units"
"github.com/go-kit/kit/log" "github.com/go-kit/kit/log"
"github.com/go-kit/kit/log/level" "github.com/go-kit/kit/log/level"
"github.com/oklog/ulid" "github.com/oklog/ulid"
"github.com/pkg/errors" "github.com/pkg/errors"
"github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/pkg/labels"
"github.com/prometheus/prometheus/storage"
"github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/tsdb/chunkenc"
tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" tsdb_errors "github.com/prometheus/prometheus/tsdb/errors"
"github.com/prometheus/prometheus/tsdb/fileutil" "github.com/prometheus/prometheus/tsdb/fileutil"
@ -45,19 +48,27 @@ import (
// Default duration of a block in milliseconds - 2h. // Default duration of a block in milliseconds - 2h.
const ( 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 // DefaultOptions used for the DB. They are sane for setups using
// millisecond precision timestamps. // millisecond precision timestamps.
var DefaultOptions = &Options{ func DefaultOptions() *Options {
WALSegmentSize: wal.DefaultSegmentSize, return &Options{
RetentionDuration: 15 * 24 * 60 * 60 * 1000, // 15 days in milliseconds WALSegmentSize: wal.DefaultSegmentSize,
BlockRanges: ExponentialBlockRanges(DefaultBlockDuration, 3, 5), RetentionDuration: 15 * 24 * model.Duration(time.Hour),
NoLockfile: false, MinBlockDuration: model.Duration(time.Duration(DefaultBlockDuration) * time.Millisecond),
AllowOverlappingBlocks: false, MaxBlockDuration: model.Duration(time.Duration(DefaultBlockDuration) * time.Millisecond),
WALCompression: false, NoLockfile: false,
StripeSize: DefaultStripeSize, AllowOverlappingBlocks: false,
WALCompression: false,
StripeSize: DefaultStripeSize,
}
} }
// Options of the DB storage. // Options of the DB storage.
@ -66,20 +77,17 @@ type Options struct {
// WALSegmentSize = 0, segment size is default size. // WALSegmentSize = 0, segment size is default size.
// WALSegmentSize > 0, segment size is WALSegmentSize. // WALSegmentSize > 0, segment size is WALSegmentSize.
// WALSegmentSize < 0, wal is disabled. // WALSegmentSize < 0, wal is disabled.
WALSegmentSize int WALSegmentSize units.Base2Bytes
// Duration of persisted data to keep. // Duration of persisted data to keep.
RetentionDuration uint64 RetentionDuration model.Duration
// Maximum number of bytes in blocks to be retained. // Maximum number of bytes in blocks to be retained.
// 0 or less means disabled. // 0 or less means disabled.
// NOTE: For proper storage calculations need to consider // NOTE: For proper storage calculations need to consider
// the size of the WAL folder which is not added when calculating // the size of the WAL folder which is not added when calculating
// the current size of the database. // the current size of the database.
MaxBytes int64 MaxBytes units.Base2Bytes
// The sizes of the Blocks.
BlockRanges []int64
// NoLockfile disables creation and consideration of a lock file. // NoLockfile disables creation and consideration of a lock file.
NoLockfile bool 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 is the size in entries of the series hash map. Reducing the size will save memory but impact performance.
StripeSize int StripeSize int
}
// Appender allows appending a batch of data. It must be completed with a // The timestamp range of head blocks after which they get persisted.
// call to Commit or Rollback and must not be reused afterwards. // It's the minimum duration of any persisted block.
// MinBlockDuration model.Duration
// 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)
// AddFast adds a sample pair for the referenced series. It is generally // The maximum timestamp range of compacted blocks.
// faster than adding a sample by providing its full label set. MaxBlockDuration model.Duration
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
} }
// DB handles reads and writes of time series falling into // DB handles reads and writes of time series falling into
@ -168,6 +158,9 @@ type dbMetrics struct {
tombCleanTimer prometheus.Histogram tombCleanTimer prometheus.Histogram
blocksBytes prometheus.Gauge blocksBytes prometheus.Gauge
maxBytes prometheus.Gauge maxBytes prometheus.Gauge
minTime prometheus.GaugeFunc
headMaxTime prometheus.GaugeFunc
headMinTime prometheus.GaugeFunc
} }
func newDBMetrics(db *DB, r prometheus.Registerer) *dbMetrics { 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", Name: "prometheus_tsdb_size_retentions_total",
Help: "The number of times that blocks were deleted because the maximum number of bytes was exceeded.", 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 { if r != nil {
r.MustRegister( r.MustRegister(
m.loadedBlocks, m.loadedBlocks,
@ -261,6 +275,9 @@ func newDBMetrics(db *DB, r prometheus.Registerer) *dbMetrics {
m.tombCleanTimer, m.tombCleanTimer,
m.blocksBytes, m.blocksBytes,
m.maxBytes, m.maxBytes,
m.minTime,
m.headMaxTime,
m.headMinTime,
) )
} }
return m return m
@ -329,7 +346,12 @@ func (db *DBReadOnly) FlushWAL(dir string) error {
mint: mint, mint: mint,
maxt: maxt, 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 { if err != nil {
return errors.Wrap(err, "create leveled compactor") 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. // 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. // 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 { select {
case <-db.closed: case <-db.closed:
return nil, ErrClosed return nil, ErrClosed
@ -402,7 +424,7 @@ func (db *DBReadOnly) Querier(mint, maxt int64) (Querier, error) {
head: head, head: head,
} }
return dbWritable.Querier(mint, maxt) return dbWritable.Querier(context.TODO(), mint, maxt)
} }
// Blocks returns a slice of block readers for persisted blocks. // Blocks returns a slice of block readers for persisted blocks.
@ -481,20 +503,51 @@ func (db *DBReadOnly) Close() error {
return merr.Err() 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) { 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 { if err := os.MkdirAll(dir, 0777); err != nil {
return nil, err return nil, err
} }
if l == nil { if l == nil {
l = log.NewNopLogger() l = log.NewNopLogger()
} }
if opts == nil {
opts = DefaultOptions for i, v := range rngs {
} if v > int64(time.Duration(opts.MaxBlockDuration).Seconds()*1000) {
if opts.StripeSize <= 0 { rngs = rngs[:i]
opts.StripeSize = DefaultStripeSize break
}
} }
// Fixup bad format written by Prometheus 2.1. // Fixup bad format written by Prometheus 2.1.
if err := repairBadIndexVersion(l, dir); err != nil { if err := repairBadIndexVersion(l, dir); err != nil {
return nil, err 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()) 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 { if err != nil {
cancel() cancel()
return nil, errors.Wrap(err, "create leveled compactor") 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 { if opts.WALSegmentSize >= 0 {
// Wal is set to a custom size. // Wal is set to a custom size.
if opts.WALSegmentSize > 0 { if opts.WALSegmentSize > 0 {
segmentSize = opts.WALSegmentSize segmentSize = int(opts.WALSegmentSize)
} }
wlog, err = wal.NewSize(l, r, filepath.Join(dir, "wal"), segmentSize, opts.WALCompression) wlog, err = wal.NewSize(l, r, filepath.Join(dir, "wal"), segmentSize, opts.WALCompression)
if err != nil { 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 { if err != nil {
return nil, err return nil, err
} }
@ -585,6 +638,17 @@ func Open(dir string, l log.Logger, r prometheus.Registerer, opts *Options) (db
return db, nil 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. // Dir returns the directory of the database.
func (db *DB) Dir() string { func (db *DB) Dir() string {
return db.dir return db.dir
@ -630,14 +694,14 @@ func (db *DB) run() {
} }
// Appender opens a new appender against the database. // 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()} return dbAppender{db: db, Appender: db.head.Appender()}
} }
// dbAppender wraps the DB's head appender and triggers compactions on commit // dbAppender wraps the DB's head appender and triggers compactions on commit
// if necessary. // if necessary.
type dbAppender struct { type dbAppender struct {
Appender storage.Appender
db *DB db *DB
} }
@ -948,7 +1012,7 @@ func (db *DB) beyondTimeRetention(blocks []*Block) (deletable map[ulid.ULID]*Blo
for i, block := range blocks { for i, block := range blocks {
// The difference between the first block and this block is larger than // The difference between the first block and this block is larger than
// the retention period so any blocks after that are added as deletable. // 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:] { for _, b := range blocks[i:] {
deletable[b.meta.ULID] = b deletable[b.meta.ULID] = b
} }
@ -973,7 +1037,7 @@ func (db *DB) beyondSizeRetention(blocks []*Block) (deletable map[ulid.ULID]*Blo
blocksSize := walSize blocksSize := walSize
for i, block := range blocks { for i, block := range blocks {
blocksSize += block.Size() blocksSize += block.Size()
if blocksSize > db.opts.MaxBytes { if blocksSize > int64(db.opts.MaxBytes) {
// Add this and all following blocks for deletion. // Add this and all following blocks for deletion.
for _, b := range blocks[i:] { for _, b := range blocks[i:] {
deletable[b.meta.ULID] = b 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. // Querier returns a new querier over the data partition for the given time range.
// A goroutine must not handle more than one open Querier. // 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 blocks []BlockReader
var blockMetas []BlockMeta 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 { for _, b := range blocks {
q, err := NewBlockQuerier(b, mint, maxt) q, err := NewBlockQuerier(b, mint, maxt)
if err == nil { if err == nil {
@ -1413,3 +1477,88 @@ func exponential(d, min, max time.Duration) time.Duration {
} }
return d 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
}

View file

@ -14,6 +14,7 @@
package tsdb package tsdb
import ( import (
"context"
"encoding/binary" "encoding/binary"
"fmt" "fmt"
"hash/crc32" "hash/crc32"
@ -29,15 +30,18 @@ import (
"testing" "testing"
"time" "time"
"github.com/prometheus/prometheus/tsdb/fileutil" "github.com/alecthomas/units"
"github.com/go-kit/kit/log" "github.com/go-kit/kit/log"
"github.com/oklog/ulid" "github.com/oklog/ulid"
"github.com/pkg/errors" "github.com/pkg/errors"
"github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus"
prom_testutil "github.com/prometheus/client_golang/prometheus/testutil" 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/pkg/labels"
"github.com/prometheus/prometheus/storage"
"github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/chunks"
"github.com/prometheus/prometheus/tsdb/fileutil"
"github.com/prometheus/prometheus/tsdb/index" "github.com/prometheus/prometheus/tsdb/index"
"github.com/prometheus/prometheus/tsdb/record" "github.com/prometheus/prometheus/tsdb/record"
"github.com/prometheus/prometheus/tsdb/tombstones" "github.com/prometheus/prometheus/tsdb/tombstones"
@ -46,11 +50,16 @@ import (
"github.com/prometheus/prometheus/util/testutil" "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") tmpdir, err := ioutil.TempDir("", "test")
testutil.Ok(t, err) 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) testutil.Ok(t, err)
// Do not close the test database by default as it will deadlock on test failures. // 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. // 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 { func query(t testing.TB, q storage.Querier, matchers ...*labels.Matcher) map[string][]tsdbutil.Sample {
ss, err := q.Select(matchers...) ss, ws, err := q.Select(nil, matchers...)
defer func() { defer func() {
testutil.Ok(t, q.Close()) testutil.Ok(t, q.Close())
}() }()
testutil.Ok(t, err) testutil.Ok(t, err)
testutil.Equals(t, 0, len(ws))
result := map[string][]tsdbutil.Sample{} 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 // Ensure that blocks are held in memory in their time order
// and not in ULID order as they are read from the directory. // and not in ULID order as they are read from the directory.
func TestDB_reloadOrder(t *testing.T) { func TestDB_reloadOrder(t *testing.T) {
db, delete := openTestDB(t, nil) db, delete := openTestDB(t, nil, nil)
defer func() { defer func() {
testutil.Ok(t, db.Close()) testutil.Ok(t, db.Close())
delete() delete()
@ -118,7 +128,7 @@ func TestDB_reloadOrder(t *testing.T) {
} }
func TestDataAvailableOnlyAfterCommit(t *testing.T) { func TestDataAvailableOnlyAfterCommit(t *testing.T) {
db, delete := openTestDB(t, nil) db, delete := openTestDB(t, nil, nil)
defer func() { defer func() {
testutil.Ok(t, db.Close()) testutil.Ok(t, db.Close())
delete() delete()
@ -129,7 +139,7 @@ func TestDataAvailableOnlyAfterCommit(t *testing.T) {
_, err := app.Add(labels.FromStrings("foo", "bar"), 0, 0) _, err := app.Add(labels.FromStrings("foo", "bar"), 0, 0)
testutil.Ok(t, err) testutil.Ok(t, err)
querier, err := db.Querier(0, 1) querier, err := db.Querier(context.TODO(), 0, 1)
testutil.Ok(t, err) testutil.Ok(t, err)
seriesSet := query(t, querier, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) seriesSet := query(t, querier, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar"))
testutil.Equals(t, map[string][]tsdbutil.Sample{}, seriesSet) testutil.Equals(t, map[string][]tsdbutil.Sample{}, seriesSet)
@ -137,7 +147,7 @@ func TestDataAvailableOnlyAfterCommit(t *testing.T) {
err = app.Commit() err = app.Commit()
testutil.Ok(t, err) testutil.Ok(t, err)
querier, err = db.Querier(0, 1) querier, err = db.Querier(context.TODO(), 0, 1)
testutil.Ok(t, err) testutil.Ok(t, err)
defer querier.Close() defer querier.Close()
@ -147,7 +157,7 @@ func TestDataAvailableOnlyAfterCommit(t *testing.T) {
} }
func TestDataNotAvailableAfterRollback(t *testing.T) { func TestDataNotAvailableAfterRollback(t *testing.T) {
db, delete := openTestDB(t, nil) db, delete := openTestDB(t, nil, nil)
defer func() { defer func() {
testutil.Ok(t, db.Close()) testutil.Ok(t, db.Close())
delete() delete()
@ -160,7 +170,7 @@ func TestDataNotAvailableAfterRollback(t *testing.T) {
err = app.Rollback() err = app.Rollback()
testutil.Ok(t, err) testutil.Ok(t, err)
querier, err := db.Querier(0, 1) querier, err := db.Querier(context.TODO(), 0, 1)
testutil.Ok(t, err) testutil.Ok(t, err)
defer querier.Close() defer querier.Close()
@ -170,7 +180,7 @@ func TestDataNotAvailableAfterRollback(t *testing.T) {
} }
func TestDBAppenderAddRef(t *testing.T) { func TestDBAppenderAddRef(t *testing.T) {
db, delete := openTestDB(t, nil) db, delete := openTestDB(t, nil, nil)
defer func() { defer func() {
testutil.Ok(t, db.Close()) testutil.Ok(t, db.Close())
delete() delete()
@ -208,7 +218,7 @@ func TestDBAppenderAddRef(t *testing.T) {
testutil.Ok(t, app2.Commit()) testutil.Ok(t, app2.Commit())
q, err := db.Querier(0, 200) q, err := db.Querier(context.TODO(), 0, 200)
testutil.Ok(t, err) testutil.Ok(t, err)
res := query(t, q, labels.MustNewMatcher(labels.MatchEqual, "a", "b")) res := query(t, q, labels.MustNewMatcher(labels.MatchEqual, "a", "b"))
@ -225,7 +235,7 @@ func TestDBAppenderAddRef(t *testing.T) {
} }
func TestAppendEmptyLabelsIgnored(t *testing.T) { func TestAppendEmptyLabelsIgnored(t *testing.T) {
db, delete := openTestDB(t, nil) db, delete := openTestDB(t, nil, nil)
defer func() { defer func() {
testutil.Ok(t, db.Close()) testutil.Ok(t, db.Close())
delete() delete()
@ -278,7 +288,7 @@ func TestDeleteSimple(t *testing.T) {
Outer: Outer:
for _, c := range cases { for _, c := range cases {
db, delete := openTestDB(t, nil) db, delete := openTestDB(t, nil, nil)
defer func() { defer func() {
testutil.Ok(t, db.Close()) testutil.Ok(t, db.Close())
delete() delete()
@ -301,18 +311,19 @@ Outer:
} }
// Compare the result. // Compare the result.
q, err := db.Querier(0, numSamples) q, err := db.Querier(context.TODO(), 0, numSamples)
testutil.Ok(t, err) 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.Ok(t, err)
testutil.Equals(t, 0, len(ws))
expSamples := make([]tsdbutil.Sample, 0, len(c.remaint)) expSamples := make([]tsdbutil.Sample, 0, len(c.remaint))
for _, ts := range c.remaint { for _, ts := range c.remaint {
expSamples = append(expSamples, sample{ts, smpls[ts]}) expSamples = append(expSamples, sample{ts, smpls[ts]})
} }
expss := newMockSeriesSet([]Series{ expss := newMockSeriesSet([]storage.Series{
newSeries(map[string]string{"a": "b"}, expSamples), newSeries(map[string]string{"a": "b"}, expSamples),
}) })
@ -338,7 +349,7 @@ Outer:
} }
func TestAmendDatapointCausesError(t *testing.T) { func TestAmendDatapointCausesError(t *testing.T) {
db, delete := openTestDB(t, nil) db, delete := openTestDB(t, nil, nil)
defer func() { defer func() {
testutil.Ok(t, db.Close()) testutil.Ok(t, db.Close())
delete() delete()
@ -356,7 +367,7 @@ func TestAmendDatapointCausesError(t *testing.T) {
} }
func TestDuplicateNaNDatapointNoAmendError(t *testing.T) { func TestDuplicateNaNDatapointNoAmendError(t *testing.T) {
db, delete := openTestDB(t, nil) db, delete := openTestDB(t, nil, nil)
defer func() { defer func() {
testutil.Ok(t, db.Close()) testutil.Ok(t, db.Close())
delete() delete()
@ -373,7 +384,7 @@ func TestDuplicateNaNDatapointNoAmendError(t *testing.T) {
} }
func TestNonDuplicateNaNDatapointsCausesAmendError(t *testing.T) { func TestNonDuplicateNaNDatapointsCausesAmendError(t *testing.T) {
db, delete := openTestDB(t, nil) db, delete := openTestDB(t, nil, nil)
defer func() { defer func() {
testutil.Ok(t, db.Close()) testutil.Ok(t, db.Close())
delete() delete()
@ -389,7 +400,7 @@ func TestNonDuplicateNaNDatapointsCausesAmendError(t *testing.T) {
} }
func TestSkippingInvalidValuesInSameTxn(t *testing.T) { func TestSkippingInvalidValuesInSameTxn(t *testing.T) {
db, delete := openTestDB(t, nil) db, delete := openTestDB(t, nil, nil)
defer func() { defer func() {
testutil.Ok(t, db.Close()) testutil.Ok(t, db.Close())
delete() delete()
@ -404,7 +415,7 @@ func TestSkippingInvalidValuesInSameTxn(t *testing.T) {
testutil.Ok(t, app.Commit()) testutil.Ok(t, app.Commit())
// Make sure the right value is stored. // 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) testutil.Ok(t, err)
ssMap := query(t, q, labels.MustNewMatcher(labels.MatchEqual, "a", "b")) 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, err)
testutil.Ok(t, app.Commit()) testutil.Ok(t, app.Commit())
q, err = db.Querier(0, 10) q, err = db.Querier(context.TODO(), 0, 10)
testutil.Ok(t, err) testutil.Ok(t, err)
ssMap = query(t, q, labels.MustNewMatcher(labels.MatchEqual, "a", "b")) 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) { func TestDB_Snapshot(t *testing.T) {
db, delete := openTestDB(t, nil) db, delete := openTestDB(t, nil, nil)
defer delete() defer delete()
// append data // append data
@ -460,13 +471,14 @@ func TestDB_Snapshot(t *testing.T) {
testutil.Ok(t, err) testutil.Ok(t, err)
defer func() { testutil.Ok(t, db.Close()) }() 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) testutil.Ok(t, err)
defer func() { testutil.Ok(t, querier.Close()) }() defer func() { testutil.Ok(t, querier.Close()) }()
// sum values // 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.Ok(t, err)
testutil.Equals(t, 0, len(ws))
sum := 0.0 sum := 0.0
for seriesSet.Next() { for seriesSet.Next() {
@ -485,7 +497,7 @@ func TestDB_Snapshot(t *testing.T) {
// that are outside the set block time range. // that are outside the set block time range.
// See https://github.com/prometheus/prometheus/issues/5105 // See https://github.com/prometheus/prometheus/issues/5105
func TestDB_Snapshot_ChunksOutsideOfCompactedRange(t *testing.T) { func TestDB_Snapshot_ChunksOutsideOfCompactedRange(t *testing.T) {
db, delete := openTestDB(t, nil) db, delete := openTestDB(t, nil, nil)
defer delete() defer delete()
app := db.Appender() app := db.Appender()
@ -514,13 +526,14 @@ func TestDB_Snapshot_ChunksOutsideOfCompactedRange(t *testing.T) {
testutil.Ok(t, err) testutil.Ok(t, err)
defer func() { testutil.Ok(t, db.Close()) }() 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) testutil.Ok(t, err)
defer func() { testutil.Ok(t, querier.Close()) }() defer func() { testutil.Ok(t, querier.Close()) }()
// Sum values. // 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.Ok(t, err)
testutil.Equals(t, 0, len(ws))
sum := 0.0 sum := 0.0
for seriesSet.Next() { for seriesSet.Next() {
@ -540,7 +553,7 @@ func TestDB_Snapshot_ChunksOutsideOfCompactedRange(t *testing.T) {
func TestDB_SnapshotWithDelete(t *testing.T) { func TestDB_SnapshotWithDelete(t *testing.T) {
numSamples := int64(10) numSamples := int64(10)
db, delete := openTestDB(t, nil) db, delete := openTestDB(t, nil, nil)
defer delete() defer delete()
app := db.Appender() app := db.Appender()
@ -586,19 +599,20 @@ Outer:
defer func() { testutil.Ok(t, db.Close()) }() defer func() { testutil.Ok(t, db.Close()) }()
// Compare the result. // Compare the result.
q, err := db.Querier(0, numSamples) q, err := db.Querier(context.TODO(), 0, numSamples)
testutil.Ok(t, err) testutil.Ok(t, err)
defer func() { testutil.Ok(t, q.Close()) }() 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.Ok(t, err)
testutil.Equals(t, 0, len(ws))
expSamples := make([]tsdbutil.Sample, 0, len(c.remaint)) expSamples := make([]tsdbutil.Sample, 0, len(c.remaint))
for _, ts := range c.remaint { for _, ts := range c.remaint {
expSamples = append(expSamples, sample{ts, smpls[ts]}) expSamples = append(expSamples, sample{ts, smpls[ts]})
} }
expss := newMockSeriesSet([]Series{ expss := newMockSeriesSet([]storage.Series{
newSeries(map[string]string{"a": "b"}, expSamples), newSeries(map[string]string{"a": "b"}, expSamples),
}) })
@ -683,7 +697,7 @@ func TestDB_e2e(t *testing.T) {
seriesMap[labels.New(l...).String()] = []tsdbutil.Sample{} seriesMap[labels.New(l...).String()] = []tsdbutil.Sample{}
} }
db, delete := openTestDB(t, nil) db, delete := openTestDB(t, nil, nil)
defer func() { defer func() {
testutil.Ok(t, db.Close()) testutil.Ok(t, db.Close())
delete() 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) testutil.Ok(t, err)
ss, err := q.Select(qry.ms...) ss, ws, err := q.Select(nil, qry.ms...)
testutil.Ok(t, err) testutil.Ok(t, err)
testutil.Equals(t, 0, len(ws))
result := map[string][]tsdbutil.Sample{} result := map[string][]tsdbutil.Sample{}
@ -788,7 +803,7 @@ func TestDB_e2e(t *testing.T) {
} }
func TestWALFlushedOnDBClose(t *testing.T) { func TestWALFlushedOnDBClose(t *testing.T) {
db, delete := openTestDB(t, nil) db, delete := openTestDB(t, nil, nil)
defer delete() defer delete()
dirDb := db.Dir() dirDb := db.Dir()
@ -806,11 +821,12 @@ func TestWALFlushedOnDBClose(t *testing.T) {
testutil.Ok(t, err) testutil.Ok(t, err)
defer func() { testutil.Ok(t, db.Close()) }() 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) testutil.Ok(t, err)
values, err := q.LabelValues("labelname") values, ws, err := q.LabelValues("labelname")
testutil.Ok(t, err) testutil.Ok(t, err)
testutil.Equals(t, 0, len(ws))
testutil.Equals(t, []string{"labelvalue"}, values) testutil.Equals(t, []string{"labelvalue"}, values)
} }
@ -821,10 +837,10 @@ func TestWALSegmentSizeOptions(t *testing.T) {
files, err := ioutil.ReadDir(filepath.Join(dbDir, "wal")) files, err := ioutil.ReadDir(filepath.Join(dbDir, "wal"))
testutil.Ok(t, err) testutil.Ok(t, err)
for _, f := range files[:len(files)-1] { 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] 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. // Custom Wal Size.
2 * 32 * 1024: func(dbDir string, segmentSize int) { 2 * 32 * 1024: func(dbDir string, segmentSize int) {
@ -846,9 +862,9 @@ func TestWALSegmentSizeOptions(t *testing.T) {
} }
for segmentSize, testFunc := range tests { for segmentSize, testFunc := range tests {
t.Run(fmt.Sprintf("WALSegmentSize %d test", segmentSize), func(t *testing.T) { t.Run(fmt.Sprintf("WALSegmentSize %d test", segmentSize), func(t *testing.T) {
options := *DefaultOptions opts := DefaultOptions()
options.WALSegmentSize = segmentSize opts.WALSegmentSize = units.Base2Bytes(segmentSize)
db, delete := openTestDB(t, &options) db, delete := openTestDB(t, opts, nil)
defer delete() defer delete()
app := db.Appender() app := db.Appender()
for i := int64(0); i < 155; i++ { for i := int64(0); i < 155; i++ {
@ -859,7 +875,7 @@ func TestWALSegmentSizeOptions(t *testing.T) {
dbDir := db.Dir() dbDir := db.Dir()
db.Close() 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) { func TestTombstoneClean(t *testing.T) {
numSamples := int64(10) numSamples := int64(10)
db, delete := openTestDB(t, nil) db, delete := openTestDB(t, nil, nil)
defer delete() defer delete()
app := db.Appender() app := db.Appender()
@ -915,19 +931,20 @@ func TestTombstoneClean(t *testing.T) {
testutil.Ok(t, db.CleanTombstones()) testutil.Ok(t, db.CleanTombstones())
// Compare the result. // Compare the result.
q, err := db.Querier(0, numSamples) q, err := db.Querier(context.TODO(), 0, numSamples)
testutil.Ok(t, err) testutil.Ok(t, err)
defer q.Close() 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.Ok(t, err)
testutil.Equals(t, 0, len(ws))
expSamples := make([]tsdbutil.Sample, 0, len(c.remaint)) expSamples := make([]tsdbutil.Sample, 0, len(c.remaint))
for _, ts := range c.remaint { for _, ts := range c.remaint {
expSamples = append(expSamples, sample{ts, smpls[ts]}) expSamples = append(expSamples, sample{ts, smpls[ts]})
} }
expss := newMockSeriesSet([]Series{ expss := newMockSeriesSet([]storage.Series{
newSeries(map[string]string{"a": "b"}, expSamples), 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 // When TombstoneClean errors the original block that should be rebuilt doesn't get deleted so
// if TombstoneClean leaves any blocks behind these will overlap. // if TombstoneClean leaves any blocks behind these will overlap.
func TestTombstoneCleanFail(t *testing.T) { func TestTombstoneCleanFail(t *testing.T) {
db, delete := openTestDB(t, nil, nil)
db, delete := openTestDB(t, nil)
defer func() { defer func() {
testutil.Ok(t, db.Close()) testutil.Ok(t, db.Close())
delete() delete()
@ -1041,15 +1057,12 @@ func (c *mockCompactorFailing) Write(dest string, b BlockReader, mint, maxt int6
return block.Meta().ULID, nil 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 return ulid.ULID{}, nil
} }
func TestTimeRetention(t *testing.T) { func TestTimeRetention(t *testing.T) {
db, delete := openTestDB(t, &Options{ db, delete := openTestDB(t, nil, []int64{1000})
BlockRanges: []int64{1000},
})
defer func() { defer func() {
testutil.Ok(t, db.Close()) testutil.Ok(t, db.Close())
delete() delete()
@ -1068,7 +1081,7 @@ func TestTimeRetention(t *testing.T) {
testutil.Ok(t, db.reload()) // Reload the db to register the new blocks. 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. 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()) testutil.Ok(t, db.reload())
expBlocks := blocks[1:] expBlocks := blocks[1:]
@ -1081,9 +1094,7 @@ func TestTimeRetention(t *testing.T) {
} }
func TestSizeRetention(t *testing.T) { func TestSizeRetention(t *testing.T) {
db, delete := openTestDB(t, &Options{ db, delete := openTestDB(t, nil, []int64{100})
BlockRanges: []int64{100},
})
defer func() { defer func() {
testutil.Ok(t, db.Close()) testutil.Ok(t, db.Close())
delete() delete()
@ -1150,8 +1161,8 @@ func TestSizeRetention(t *testing.T) {
// Check total size, total count and check that the oldest block was deleted. // Check total size, total count and check that the oldest block was deleted.
firstBlockSize := db.Blocks()[0].Size() firstBlockSize := db.Blocks()[0].Size()
sizeLimit := actSize - firstBlockSize sizeLimit := actSize - firstBlockSize
db.opts.MaxBytes = sizeLimit // Set the new db size limit one block smaller that the actual 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. testutil.Ok(t, db.reload()) // Reload the db to register the new db size.
expBlocks := blocks[1:] expBlocks := blocks[1:]
actBlocks := db.Blocks() actBlocks := db.Blocks()
@ -1184,9 +1195,8 @@ func TestSizeRetentionMetric(t *testing.T) {
for _, c := range cases { for _, c := range cases {
db, delete := openTestDB(t, &Options{ db, delete := openTestDB(t, &Options{
BlockRanges: []int64{100}, MaxBytes: units.Base2Bytes(c.maxBytes),
MaxBytes: c.maxBytes, }, []int64{100})
})
actMaxBytes := int64(prom_testutil.ToFloat64(db.metrics.maxBytes)) actMaxBytes := int64(prom_testutil.ToFloat64(db.metrics.maxBytes))
testutil.Equals(t, actMaxBytes, c.expMaxBytes, "metric retention limit bytes mismatch") 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) { func TestNotMatcherSelectsLabelsUnsetSeries(t *testing.T) {
db, delete := openTestDB(t, nil) db, delete := openTestDB(t, nil, nil)
defer func() { defer func() {
testutil.Ok(t, db.Close()) testutil.Ok(t, db.Close())
delete() delete()
@ -1257,13 +1267,14 @@ func TestNotMatcherSelectsLabelsUnsetSeries(t *testing.T) {
series: labelpairs[:1], series: labelpairs[:1],
}} }}
q, err := db.Querier(0, 10) q, err := db.Querier(context.TODO(), 0, 10)
testutil.Ok(t, err) testutil.Ok(t, err)
defer func() { testutil.Ok(t, q.Close()) }() defer func() { testutil.Ok(t, q.Close()) }()
for _, c := range cases { for _, c := range cases {
ss, err := q.Select(c.selector...) ss, ws, err := q.Select(nil, c.selector...)
testutil.Ok(t, err) testutil.Ok(t, err)
testutil.Equals(t, 0, len(ws))
lres, err := expandSeriesSet(ss) lres, err := expandSeriesSet(ss)
testutil.Ok(t, err) 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{} result := []labels.Labels{}
for ss.Next() { for ss.Next() {
result = append(result, ss.At().Labels()) 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 // Regression test for https://github.com/prometheus/prometheus/tsdb/issues/347
func TestChunkAtBlockBoundary(t *testing.T) { func TestChunkAtBlockBoundary(t *testing.T) {
db, delete := openTestDB(t, nil) db, delete := openTestDB(t, nil, nil)
defer func() { defer func() {
testutil.Ok(t, db.Close()) testutil.Ok(t, db.Close())
delete() delete()
@ -1383,7 +1394,7 @@ func TestChunkAtBlockBoundary(t *testing.T) {
app := db.Appender() app := db.Appender()
blockRange := DefaultOptions.BlockRanges[0] blockRange := db.compactor.(*LeveledCompactor).ranges[0]
label := labels.FromStrings("foo", "bar") label := labels.FromStrings("foo", "bar")
for i := int64(0); i < 3; i++ { for i := int64(0); i < 3; i++ {
@ -1432,7 +1443,7 @@ func TestChunkAtBlockBoundary(t *testing.T) {
} }
func TestQuerierWithBoundaryChunks(t *testing.T) { func TestQuerierWithBoundaryChunks(t *testing.T) {
db, delete := openTestDB(t, nil) db, delete := openTestDB(t, nil, nil)
defer func() { defer func() {
testutil.Ok(t, db.Close()) testutil.Ok(t, db.Close())
delete() delete()
@ -1440,7 +1451,7 @@ func TestQuerierWithBoundaryChunks(t *testing.T) {
app := db.Appender() app := db.Appender()
blockRange := DefaultOptions.BlockRanges[0] blockRange := db.compactor.(*LeveledCompactor).ranges[0]
label := labels.FromStrings("foo", "bar") label := labels.FromStrings("foo", "bar")
for i := int64(0); i < 5; i++ { 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") 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) testutil.Ok(t, err)
defer q.Close() defer q.Close()
@ -1583,16 +1594,14 @@ func TestInitializeHeadTimestamp(t *testing.T) {
} }
func TestNoEmptyBlocks(t *testing.T) { func TestNoEmptyBlocks(t *testing.T) {
db, delete := openTestDB(t, &Options{ db, delete := openTestDB(t, nil, []int64{100})
BlockRanges: []int64{100},
})
defer func() { defer func() {
testutil.Ok(t, db.Close()) testutil.Ok(t, db.Close())
delete() delete()
}() }()
db.DisableCompactions() db.DisableCompactions()
rangeToTriggerCompaction := db.opts.BlockRanges[0]/2*3 - 1 rangeToTriggerCompaction := db.compactor.(*LeveledCompactor).ranges[0]/2*3 - 1
defaultLabel := labels.FromStrings("foo", "bar") defaultLabel := labels.FromStrings("foo", "bar")
defaultMatcher := labels.MustNewMatcher(labels.MatchRegexp, "", ".*") 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) { t.Run("Test no blocks remaining after deleting all samples from disk.", func(t *testing.T) {
currentTime := db.Head().MaxTime() currentTime := db.Head().MaxTime()
blocks := []*BlockMeta{ blocks := []*BlockMeta{
{MinTime: currentTime, MaxTime: currentTime + db.opts.BlockRanges[0]}, {MinTime: currentTime, MaxTime: currentTime + db.compactor.(*LeveledCompactor).ranges[0]},
{MinTime: currentTime + 100, MaxTime: currentTime + 100 + db.opts.BlockRanges[0]}, {MinTime: currentTime + 100, MaxTime: currentTime + 100 + db.compactor.(*LeveledCompactor).ranges[0]},
} }
for _, m := range blocks { for _, m := range blocks {
createBlock(t, db.Dir(), genSeries(2, 2, m.MinTime, m.MaxTime)) 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. // Appends samples into the database.
appendSamples := func(db *DB, mint, maxt int64, sampleLabels [][2]string) { appendSamples := func(db *DB, mint, maxt int64, sampleLabels [][2]string) {
t.Helper() t.Helper()
@ -1743,7 +1752,7 @@ func TestDB_LabelNames(t *testing.T) {
testutil.Ok(t, err) testutil.Ok(t, err)
} }
for _, tst := range tests { for _, tst := range tests {
db, delete := openTestDB(t, nil) db, delete := openTestDB(t, nil, nil)
defer func() { defer func() {
testutil.Ok(t, db.Close()) testutil.Ok(t, db.Close())
delete() delete()
@ -1778,23 +1787,25 @@ func TestDB_LabelNames(t *testing.T) {
appendSamples(db, 5, 9, tst.sampleLabels2) appendSamples(db, 5, 9, tst.sampleLabels2)
// Testing DB (union). // 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) testutil.Ok(t, err)
labelNames, err = q.LabelNames() var ws storage.Warnings
labelNames, ws, err = q.LabelNames()
testutil.Ok(t, err) testutil.Ok(t, err)
testutil.Equals(t, 0, len(ws))
testutil.Ok(t, q.Close()) testutil.Ok(t, q.Close())
testutil.Equals(t, tst.exp2, labelNames) testutil.Equals(t, tst.exp2, labelNames)
} }
} }
func TestCorrectNumTombstones(t *testing.T) { func TestCorrectNumTombstones(t *testing.T) {
db, delete := openTestDB(t, nil) db, delete := openTestDB(t, nil, nil)
defer func() { defer func() {
testutil.Ok(t, db.Close()) testutil.Ok(t, db.Close())
delete() delete()
}() }()
blockRange := DefaultOptions.BlockRanges[0] blockRange := db.compactor.(*LeveledCompactor).ranges[0]
defaultLabel := labels.FromStrings("foo", "bar") defaultLabel := labels.FromStrings("foo", "bar")
defaultMatcher := labels.MustNewMatcher(labels.MatchEqual, defaultLabel[0].Name, defaultLabel[0].Value) 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) { func TestVerticalCompaction(t *testing.T) {
cases := []struct { cases := []struct {
blockSeries [][]Series blockSeries [][]storage.Series
expSeries map[string][]tsdbutil.Sample expSeries map[string][]tsdbutil.Sample
expBlockNum int expBlockNum int
expOverlappingBlocks 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{ newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{
sample{0, 0}, sample{1, 0}, sample{2, 0}, sample{4, 0}, 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{ newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{
sample{0, 0}, sample{1, 0}, sample{2, 0}, sample{4, 0}, 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{ newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{
sample{0, 0}, sample{1, 0}, sample{2, 0}, sample{4, 0}, 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{ newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{
sample{0, 0}, sample{1, 0}, sample{2, 0}, sample{4, 0}, 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{ newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{
sample{0, 0}, sample{1, 0}, sample{2, 0}, sample{4, 0}, 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{ newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{
sample{0, 0}, sample{1, 0}, sample{2, 0}, sample{4, 0}, 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{ newSeries(map[string]string{"a": "b"}, []tsdbutil.Sample{
sample{0, 0}, sample{1, 0}, sample{2, 0}, sample{4, 0}, 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 { for _, series := range c.blockSeries {
createBlock(t, tmpdir, series) createBlock(t, tmpdir, series)
} }
opts := *DefaultOptions opts := DefaultOptions()
opts.AllowOverlappingBlocks = true opts.AllowOverlappingBlocks = true
db, err := Open(tmpdir, nil, nil, &opts) db, err := Open(tmpdir, nil, nil, opts)
testutil.Ok(t, err) testutil.Ok(t, err)
defer func() { defer func() {
testutil.Ok(t, db.Close()) 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].") testutil.Assert(t, len(db.blocks) == len(c.blockSeries), "Wrong number of blocks [before compact].")
// Vertical Query Merging test. // Vertical Query Merging test.
querier, err := db.Querier(0, 100) querier, err := db.Querier(context.TODO(), 0, 100)
testutil.Ok(t, err) testutil.Ok(t, err)
actSeries := query(t, querier, defaultMatcher) actSeries := query(t, querier, defaultMatcher)
testutil.Equals(t, c.expSeries, actSeries) 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") testutil.Equals(t, c.expOverlappingBlocks, int(prom_testutil.ToFloat64(lc.metrics.overlappingBlocks)), "overlapping blocks count mismatch")
// Query test after merging the overlapping blocks. // 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) testutil.Ok(t, err)
actSeries = query(t, querier, defaultMatcher) actSeries = query(t, querier, defaultMatcher)
testutil.Equals(t, c.expSeries, actSeries) testutil.Equals(t, c.expSeries, actSeries)
@ -2203,20 +2214,16 @@ func TestBlockRanges(t *testing.T) {
logger := log.NewLogfmtLogger(log.NewSyncWriter(os.Stderr)) logger := log.NewLogfmtLogger(log.NewSyncWriter(os.Stderr))
dir, err := ioutil.TempDir("", "test_storage") dir, err := ioutil.TempDir("", "test_storage")
if err != nil { testutil.Ok(t, err)
t.Fatalf("Opening test dir failed: %s", err)
}
rangeToTriggercompaction := DefaultOptions.BlockRanges[0]/2*3 + 1
// Test that the compactor doesn't create overlapping blocks // Test that the compactor doesn't create overlapping blocks
// when a non standard block already exists. // when a non standard block already exists.
firstBlockMaxT := int64(3) firstBlockMaxT := int64(3)
createBlock(t, dir, genSeries(1, 1, 0, firstBlockMaxT)) createBlock(t, dir, genSeries(1, 1, 0, firstBlockMaxT))
db, err := Open(dir, logger, nil, DefaultOptions) db, err := open(dir, logger, nil, DefaultOptions(), []int64{10000})
if err != nil { testutil.Ok(t, err)
t.Fatalf("Opening test storage failed: %s", err)
} rangeToTriggerCompaction := db.compactor.(*LeveledCompactor).ranges[0]/2*3 + 1
defer func() { defer func() {
os.RemoveAll(dir) os.RemoveAll(dir)
}() }()
@ -2230,7 +2237,7 @@ func TestBlockRanges(t *testing.T) {
testutil.Ok(t, err) testutil.Ok(t, err)
_, err = app.Add(lbl, firstBlockMaxT+2, rand.Float64()) _, err = app.Add(lbl, firstBlockMaxT+2, rand.Float64())
testutil.Ok(t, err) testutil.Ok(t, err)
secondBlockMaxt := firstBlockMaxT + rangeToTriggercompaction secondBlockMaxt := firstBlockMaxT + rangeToTriggerCompaction
_, err = app.Add(lbl, secondBlockMaxt, rand.Float64()) // Add samples to trigger a new compaction _, err = app.Add(lbl, secondBlockMaxt, rand.Float64()) // Add samples to trigger a new compaction
testutil.Ok(t, err) testutil.Ok(t, err)
@ -2264,16 +2271,15 @@ func TestBlockRanges(t *testing.T) {
thirdBlockMaxt := secondBlockMaxt + 2 thirdBlockMaxt := secondBlockMaxt + 2
createBlock(t, dir, genSeries(1, 1, secondBlockMaxt+1, thirdBlockMaxt)) createBlock(t, dir, genSeries(1, 1, secondBlockMaxt+1, thirdBlockMaxt))
db, err = Open(dir, logger, nil, DefaultOptions) db, err = open(dir, logger, nil, DefaultOptions(), []int64{10000})
if err != nil { testutil.Ok(t, err)
t.Fatalf("Opening test storage failed: %s", err)
}
defer db.Close() defer db.Close()
testutil.Equals(t, 3, len(db.Blocks()), "db doesn't include expected number of blocks") 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") testutil.Equals(t, db.Blocks()[2].Meta().MaxTime, thirdBlockMaxt, "unexpected maxt of the last block")
app = db.Appender() 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, err)
testutil.Ok(t, app.Commit()) testutil.Ok(t, app.Commit())
for x := 0; x < 100; x++ { for x := 0; x < 100; x++ {
@ -2344,7 +2350,7 @@ func TestDBReadOnly(t *testing.T) {
testutil.Ok(t, err) testutil.Ok(t, err)
testutil.Assert(t, expDbSize > dbSizeBeforeAppend, "db size didn't increase after an append") 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) testutil.Ok(t, err)
expSeries = query(t, q, matchAll) 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") 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) testutil.Ok(t, err)
readOnlySeries := query(t, q, matchAll) readOnlySeries := query(t, q, matchAll)
readOnlyDBHash := testutil.DirHash(t, dbDir) readOnlyDBHash := testutil.DirHash(t, dbDir)
@ -2393,7 +2399,7 @@ func TestDBReadOnlyClosing(t *testing.T) {
testutil.Equals(t, db.Close(), ErrClosed) testutil.Equals(t, db.Close(), ErrClosed)
_, err = db.Blocks() _, err = db.Blocks()
testutil.Equals(t, err, ErrClosed) testutil.Equals(t, err, ErrClosed)
_, err = db.Querier(0, 1) _, err = db.Querier(context.TODO(), 0, 1)
testutil.Equals(t, err, ErrClosed) testutil.Equals(t, err, ErrClosed)
} }
@ -2449,13 +2455,14 @@ func TestDBReadOnly_FlushWAL(t *testing.T) {
testutil.Ok(t, err) testutil.Ok(t, err)
testutil.Equals(t, len(blocks), 1) 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) testutil.Ok(t, err)
defer func() { testutil.Ok(t, querier.Close()) }() defer func() { testutil.Ok(t, querier.Close()) }()
// Sum the values. // 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.Ok(t, err)
testutil.Equals(t, 0, len(ws))
sum := 0.0 sum := 0.0
for seriesSet.Next() { for seriesSet.Next() {
@ -2692,3 +2699,38 @@ func TestChunkReader_ConcurrentReads(t *testing.T) {
} }
testutil.Ok(t, r.Close()) 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())
}

View file

@ -29,6 +29,7 @@ import (
"github.com/pkg/errors" "github.com/pkg/errors"
"github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/pkg/labels"
"github.com/prometheus/prometheus/storage"
"github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/tsdb/chunkenc"
"github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/chunks"
"github.com/prometheus/prometheus/tsdb/index" "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 // initAppender is a helper to initialize the time bounds of the head
// upon the first sample it receives. // upon the first sample it receives.
type initAppender struct { type initAppender struct {
app Appender app storage.Appender
head *Head head *Head
} }
@ -836,7 +837,7 @@ func (a *initAppender) Rollback() error {
} }
// Appender returns a new Appender on the database. // Appender returns a new Appender on the database.
func (h *Head) Appender() Appender { func (h *Head) Appender() storage.Appender {
h.metrics.activeAppenders.Inc() h.metrics.activeAppenders.Inc()
// The head cache might not have a starting point yet. The init appender // The head cache might not have a starting point yet. The init appender

View file

@ -28,6 +28,7 @@ import (
"github.com/pkg/errors" "github.com/pkg/errors"
prom_testutil "github.com/prometheus/client_golang/prometheus/testutil" prom_testutil "github.com/prometheus/client_golang/prometheus/testutil"
"github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/pkg/labels"
"github.com/prometheus/prometheus/storage"
"github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/tsdb/chunkenc"
"github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/chunks"
"github.com/prometheus/prometheus/tsdb/index" "github.com/prometheus/prometheus/tsdb/index"
@ -542,7 +543,7 @@ func TestHeadDeleteSimple(t *testing.T) {
testutil.Ok(t, reloadedHead.Init(0)) testutil.Ok(t, reloadedHead.Init(0))
// Compare the query results for both heads - before and after the reload. // 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 { newSeries(map[string]string{lblDefault.Name: lblDefault.Value}, func() []tsdbutil.Sample {
ss := make([]tsdbutil.Sample, 0, len(c.smplsExp)) ss := make([]tsdbutil.Sample, 0, len(c.smplsExp))
for _, s := range c.smplsExp { for _, s := range c.smplsExp {
@ -555,8 +556,9 @@ func TestHeadDeleteSimple(t *testing.T) {
for _, h := range []*Head{head, reloadedHead} { for _, h := range []*Head{head, reloadedHead} {
q, err := NewBlockQuerier(h, h.MinTime(), h.MaxTime()) q, err := NewBlockQuerier(h, h.MinTime(), h.MaxTime())
testutil.Ok(t, err) 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.Ok(t, err)
testutil.Equals(t, 0, len(ws))
for { for {
eok, rok := expSeriesSet.Next(), actSeriesSet.Next() 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. // Test the series returns no samples. The series is cleared only after compaction.
q, err := NewBlockQuerier(hb, 0, 100000) q, err := NewBlockQuerier(hb, 0, 100000)
testutil.Ok(t, err) 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.Ok(t, err)
testutil.Equals(t, 0, len(ws))
testutil.Assert(t, res.Next(), "series is not present") testutil.Assert(t, res.Next(), "series is not present")
s := res.At() s := res.At()
it := s.Iterator() it := s.Iterator()
@ -615,8 +618,9 @@ func TestDeleteUntilCurMax(t *testing.T) {
testutil.Ok(t, app.Commit()) testutil.Ok(t, app.Commit())
q, err = NewBlockQuerier(hb, 0, 100000) q, err = NewBlockQuerier(hb, 0, 100000)
testutil.Ok(t, err) 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.Ok(t, err)
testutil.Equals(t, 0, len(ws))
testutil.Assert(t, res.Next(), "series don't exist") testutil.Assert(t, res.Next(), "series don't exist")
exps := res.At() exps := res.At()
it = exps.Iterator() it = exps.Iterator()
@ -790,10 +794,11 @@ func TestDelete_e2e(t *testing.T) {
q, err := NewBlockQuerier(hb, 0, 100000) q, err := NewBlockQuerier(hb, 0, 100000)
testutil.Ok(t, err) testutil.Ok(t, err)
defer q.Close() defer q.Close()
ss, err := q.SelectSorted(del.ms...) ss, ws, err := q.SelectSorted(nil, del.ms...)
testutil.Ok(t, err) testutil.Ok(t, err)
testutil.Equals(t, 0, len(ws))
// Build the mockSeriesSet. // Build the mockSeriesSet.
matchedSeries := make([]Series, 0, len(matched)) matchedSeries := make([]storage.Series, 0, len(matched))
for _, m := range matched { for _, m := range matched {
smpls := seriesMap[m.String()] smpls := seriesMap[m.String()]
smpls = deletedSamples(smpls, del.drange) smpls = deletedSamples(smpls, del.drange)
@ -1077,8 +1082,9 @@ func TestUncommittedSamplesNotLostOnTruncate(t *testing.T) {
testutil.Ok(t, err) testutil.Ok(t, err)
defer q.Close() 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.Ok(t, err)
testutil.Equals(t, 0, len(ws))
testutil.Equals(t, true, ss.Next()) testutil.Equals(t, true, ss.Next())
} }
@ -1104,8 +1110,9 @@ func TestRemoveSeriesAfterRollbackAndTruncate(t *testing.T) {
testutil.Ok(t, err) testutil.Ok(t, err)
defer q.Close() 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.Ok(t, err)
testutil.Equals(t, 0, len(ws))
testutil.Equals(t, false, ss.Next()) testutil.Equals(t, false, ss.Next())
@ -1227,7 +1234,7 @@ func TestWalRepair_DecodingError(t *testing.T) {
// Open the db to trigger a repair. // 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) testutil.Ok(t, err)
defer func() { defer func() {
testutil.Ok(t, db.Close()) testutil.Ok(t, db.Close())

View file

@ -14,13 +14,13 @@
package tsdb package tsdb
import ( import (
"fmt"
"sort" "sort"
"strings" "strings"
"unicode/utf8" "unicode/utf8"
"github.com/pkg/errors" "github.com/pkg/errors"
"github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/pkg/labels"
"github.com/prometheus/prometheus/storage"
"github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/tsdb/chunkenc"
"github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/chunks"
tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" tsdb_errors "github.com/prometheus/prometheus/tsdb/errors"
@ -28,52 +28,25 @@ import (
"github.com/prometheus/prometheus/tsdb/tombstones" "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 // querier aggregates querying results from time blocks within
// a single partition. // a single partition.
type querier struct { 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) return q.lvals(q.blocks, n)
} }
// LabelNames returns all the unique label names present querier blocks. // 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{}) labelNamesMap := make(map[string]struct{})
var ws storage.Warnings
for _, b := range q.blocks { for _, b := range q.blocks {
names, err := b.LabelNames() names, w, err := b.LabelNames()
ws = append(ws, w...)
if err != nil { if err != nil {
return nil, errors.Wrap(err, "LabelNames() from Querier") return nil, ws, errors.Wrap(err, "LabelNames() from Querier")
} }
for _, name := range names { for _, name := range names {
labelNamesMap[name] = struct{}{} labelNamesMap[name] = struct{}{}
@ -86,51 +59,57 @@ func (q *querier) LabelNames() ([]string, error) {
} }
sort.Strings(labelNames) 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 { if len(qs) == 0 {
return nil, nil return nil, nil, nil
} }
if len(qs) == 1 { if len(qs) == 1 {
return qs[0].LabelValues(n) return qs[0].LabelValues(n)
} }
l := len(qs) / 2 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 { 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 { 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 { if len(q.blocks) != 1 {
return q.SelectSorted(ms...) return q.SelectSorted(p, ms...)
} }
// Sorting Head series is slow, and unneeded when only the // Sorting Head series is slow, and unneeded when only the
// Head is being queried. Sorting blocks is a noop. // 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 { 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 { 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 { if err != nil {
return nil, err return nil, ws, err
} }
ss[i] = s ss[i] = s
} }
return NewMergedSeriesSet(ss), nil return NewMergedSeriesSet(ss), ws, nil
} }
func (q *querier) Close() error { func (q *querier) Close() error {
@ -148,36 +127,39 @@ type verticalQuerier struct {
querier querier
} }
func (q *verticalQuerier) Select(ms ...*labels.Matcher) (SeriesSet, error) { func (q *verticalQuerier) Select(p *storage.SelectParams, ms ...*labels.Matcher) (storage.SeriesSet, storage.Warnings, error) {
return q.sel(q.blocks, ms) return q.sel(p, q.blocks, ms)
} }
func (q *verticalQuerier) SelectSorted(ms ...*labels.Matcher) (SeriesSet, error) { func (q *verticalQuerier) SelectSorted(p *storage.SelectParams, ms ...*labels.Matcher) (storage.SeriesSet, storage.Warnings, error) {
return q.sel(q.blocks, ms) 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 { if len(qs) == 0 {
return EmptySeriesSet(), nil return storage.EmptySeriesSet(), nil, nil
} }
if len(qs) == 1 { if len(qs) == 1 {
return qs[0].SelectSorted(ms...) return qs[0].SelectSorted(p, ms...)
} }
l := len(qs) / 2 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 { 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 { 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. // 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() indexr, err := b.Index()
if err != nil { if err != nil {
return nil, errors.Wrapf(err, "open index reader") return nil, errors.Wrapf(err, "open index reader")
@ -213,52 +195,64 @@ type blockQuerier struct {
mint, maxt int64 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...) base, err := LookupChunkSeries(q.index, q.tombstones, ms...)
if err != nil { 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{ return &blockSeriesSet{
set: &populatedChunkSeries{ set: &populatedChunkSeries{
set: base, set: base,
chunks: q.chunks, chunks: q.chunks,
mint: q.mint, mint: mint,
maxt: q.maxt, maxt: maxt,
}, },
mint: q.mint, mint: mint,
maxt: q.maxt, maxt: maxt,
}, nil }, 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...) base, err := LookupChunkSeriesSorted(q.index, q.tombstones, ms...)
if err != nil { 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{ return &blockSeriesSet{
set: &populatedChunkSeries{ set: &populatedChunkSeries{
set: base, set: base,
chunks: q.chunks, chunks: q.chunks,
mint: q.mint, mint: mint,
maxt: q.maxt, maxt: maxt,
}, },
mint: q.mint, mint: mint,
maxt: q.maxt, maxt: maxt,
}, nil }, nil, nil
} }
func (q *blockQuerier) LabelValues(name string) ([]string, error) { func (q *blockQuerier) LabelValues(name string) ([]string, storage.Warnings, error) {
return q.index.LabelValues(name) res, err := q.index.LabelValues(name)
return res, nil, err
} }
func (q *blockQuerier) LabelNames() ([]string, error) { func (q *blockQuerier) LabelNames() ([]string, storage.Warnings, error) {
return q.index.LabelNames() res, err := q.index.LabelNames()
} return res, nil, err
func (q *blockQuerier) LabelValuesFor(string, labels.Label) ([]string, error) {
return nil, fmt.Errorf("not implemented")
} }
func (q *blockQuerier) Close() error { func (q *blockQuerier) Close() error {
@ -495,32 +489,20 @@ func mergeStrings(a, b []string) []string {
return res 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 // mergedSeriesSet returns a series sets slice as a single series set. The input series sets
// must be sorted and sequential in time. // must be sorted and sequential in time.
// TODO(bwplotka): Merge this with merge SeriesSet available in storage package.
type mergedSeriesSet struct { type mergedSeriesSet struct {
all []SeriesSet all []storage.SeriesSet
buf []SeriesSet // A buffer for keeping the order of SeriesSet slice during forwarding the 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. ids []int // The indices of chosen SeriesSet for the current run.
done bool done bool
err error 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 { if len(all) == 1 {
return all[0] return all[0]
} }
@ -535,7 +517,7 @@ func NewMergedSeriesSet(all []SeriesSet) SeriesSet {
return s return s
} }
func (s *mergedSeriesSet) At() Series { func (s *mergedSeriesSet) At() storage.Series {
return s.cur return s.cur
} }
@ -620,7 +602,7 @@ func (s *mergedSeriesSet) Next() bool {
} }
if len(s.ids) > 1 { if len(s.ids) > 1 {
series := make([]Series, len(s.ids)) series := make([]storage.Series, len(s.ids))
for i, idx := range s.ids { for i, idx := range s.ids {
series[i] = s.all[idx].At() series[i] = s.all[idx].At()
} }
@ -632,19 +614,19 @@ func (s *mergedSeriesSet) Next() bool {
} }
type mergedVerticalSeriesSet struct { type mergedVerticalSeriesSet struct {
a, b SeriesSet a, b storage.SeriesSet
cur Series cur storage.Series
adone, bdone bool adone, bdone bool
} }
// NewMergedVerticalSeriesSet takes two series sets as a single series set. // NewMergedVerticalSeriesSet takes two series sets as a single series set.
// The input series sets must be sorted and // The input series sets must be sorted and
// the time ranges of the series can be overlapping. // 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) return newMergedVerticalSeriesSet(a, b)
} }
func newMergedVerticalSeriesSet(a, b SeriesSet) *mergedVerticalSeriesSet { func newMergedVerticalSeriesSet(a, b storage.SeriesSet) *mergedVerticalSeriesSet {
s := &mergedVerticalSeriesSet{a: a, b: b} s := &mergedVerticalSeriesSet{a: a, b: b}
// Initialize first elements of both sets as Next() needs // Initialize first elements of both sets as Next() needs
// one element look-ahead. // one element look-ahead.
@ -654,7 +636,7 @@ func newMergedVerticalSeriesSet(a, b SeriesSet) *mergedVerticalSeriesSet {
return s return s
} }
func (s *mergedVerticalSeriesSet) At() Series { func (s *mergedVerticalSeriesSet) At() storage.Series {
return s.cur return s.cur
} }
@ -690,21 +672,13 @@ func (s *mergedVerticalSeriesSet) Next() bool {
s.cur = s.a.At() s.cur = s.a.At()
s.adone = !s.a.Next() s.adone = !s.a.Next()
} else { } 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.adone = !s.a.Next()
s.bdone = !s.b.Next() s.bdone = !s.b.Next()
} }
return true 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 // 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. // list from an index. It filters out series that have labels set that should be unset.
type baseChunkSeries struct { type baseChunkSeries struct {
@ -720,17 +694,17 @@ type baseChunkSeries struct {
// LookupChunkSeries retrieves all series for the given matchers and returns a ChunkSeriesSet // LookupChunkSeries retrieves all series for the given matchers and returns a ChunkSeriesSet
// over them. It drops chunks based on tombstones in the given reader. // 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...) return lookupChunkSeries(false, ir, tr, ms...)
} }
// LookupChunkSeries retrieves all series for the given matchers and returns a ChunkSeriesSet // 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. // 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...) 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 { if tr == nil {
tr = tombstones.NewMemTombstones() 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 // with known chunk references. It filters out chunks that do not fit the
// given time range. // given time range.
type populatedChunkSeries struct { type populatedChunkSeries struct {
set ChunkSeriesSet set storage.ChunkSeriesSet
chunks ChunkReader chunks ChunkReader
mint, maxt int64 mint, maxt int64
@ -872,9 +846,9 @@ func (s *populatedChunkSeries) Next() bool {
// blockSeriesSet is a set of series from an inverted index query. // blockSeriesSet is a set of series from an inverted index query.
type blockSeriesSet struct { type blockSeriesSet struct {
set ChunkSeriesSet set storage.ChunkSeriesSet
err error err error
cur Series cur storage.Series
mint, maxt int64 mint, maxt int64
} }
@ -898,8 +872,8 @@ func (s *blockSeriesSet) Next() bool {
return false return false
} }
func (s *blockSeriesSet) At() Series { return s.cur } func (s *blockSeriesSet) At() storage.Series { return s.cur }
func (s *blockSeriesSet) Err() error { return s.err } func (s *blockSeriesSet) Err() error { return s.err }
// chunkSeries is a series that is backed by a sequence of chunks holding // chunkSeries is a series that is backed by a sequence of chunks holding
// time series data. // time series data.
@ -916,48 +890,34 @@ func (s *chunkSeries) Labels() labels.Labels {
return s.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) 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. // chainedSeries implements a series for a list of time-sorted series.
// They all must have the same labels. // They all must have the same labels.
type chainedSeries struct { type chainedSeries struct {
series []Series series []storage.Series
} }
func (s *chainedSeries) Labels() labels.Labels { func (s *chainedSeries) Labels() labels.Labels {
return s.series[0].Labels() return s.series[0].Labels()
} }
func (s *chainedSeries) Iterator() SeriesIterator { func (s *chainedSeries) Iterator() chunkenc.Iterator {
return newChainedSeriesIterator(s.series...) return newChainedSeriesIterator(s.series...)
} }
// chainedSeriesIterator implements a series iterator over a list // chainedSeriesIterator implements a series iterator over a list
// of time-sorted, non-overlapping iterators. // of time-sorted, non-overlapping iterators.
type chainedSeriesIterator struct { type chainedSeriesIterator struct {
series []Series // series in time order series []storage.Series // series in time order
i int i int
cur SeriesIterator cur chunkenc.Iterator
} }
func newChainedSeriesIterator(s ...Series) *chainedSeriesIterator { func newChainedSeriesIterator(s ...storage.Series) *chainedSeriesIterator {
return &chainedSeriesIterator{ return &chainedSeriesIterator{
series: s, series: s,
i: 0, i: 0,
@ -1008,28 +968,28 @@ func (it *chainedSeriesIterator) Err() error {
// verticalChainedSeries implements a series for a list of time-sorted, time-overlapping series. // verticalChainedSeries implements a series for a list of time-sorted, time-overlapping series.
// They all must have the same labels. // They all must have the same labels.
type verticalChainedSeries struct { type verticalChainedSeries struct {
series []Series series []storage.Series
} }
func (s *verticalChainedSeries) Labels() labels.Labels { func (s *verticalChainedSeries) Labels() labels.Labels {
return s.series[0].Labels() return s.series[0].Labels()
} }
func (s *verticalChainedSeries) Iterator() SeriesIterator { func (s *verticalChainedSeries) Iterator() chunkenc.Iterator {
return newVerticalMergeSeriesIterator(s.series...) return newVerticalMergeSeriesIterator(s.series...)
} }
// verticalMergeSeriesIterator implements a series iterator over a list // verticalMergeSeriesIterator implements a series iterator over a list
// of time-sorted, time-overlapping iterators. // of time-sorted, time-overlapping iterators.
type verticalMergeSeriesIterator struct { type verticalMergeSeriesIterator struct {
a, b SeriesIterator a, b chunkenc.Iterator
aok, bok, initialized bool aok, bok, initialized bool
curT int64 curT int64
curV float64 curV float64
} }
func newVerticalMergeSeriesIterator(s ...Series) SeriesIterator { func newVerticalMergeSeriesIterator(s ...storage.Series) chunkenc.Iterator {
if len(s) == 1 { if len(s) == 1 {
return s[0].Iterator() return s[0].Iterator()
} else if len(s) == 2 { } else if len(s) == 2 {
@ -1219,6 +1179,13 @@ func (it *deletedIterator) At() (int64, float64) {
return it.it.At() 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 { func (it *deletedIterator) Next() bool {
Outer: Outer:
for it.it.Next() { for it.it.Next() {
@ -1229,28 +1196,15 @@ Outer:
continue Outer continue Outer
} }
if ts > tr.Maxt { if ts <= tr.Maxt {
it.intervals = it.intervals[1:] return true
continue
} }
it.intervals = it.intervals[1:]
return true
} }
return true return true
} }
return false return false
} }
func (it *deletedIterator) Err() error { func (it *deletedIterator) Err() error { return it.it.Err() }
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 }

View file

@ -21,6 +21,7 @@ import (
"testing" "testing"
"github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/pkg/labels"
"github.com/prometheus/prometheus/storage"
"github.com/prometheus/prometheus/util/testutil" "github.com/prometheus/prometheus/util/testutil"
) )
@ -145,11 +146,11 @@ func BenchmarkQuerierSelect(b *testing.B) {
b.ResetTimer() b.ResetTimer()
for i := 0; i < b.N; i++ { for i := 0; i < b.N; i++ {
var ss SeriesSet var ss storage.SeriesSet
if sorted { if sorted {
ss, err = q.SelectSorted(matcher) ss, _, err = q.SelectSorted(nil, matcher)
} else { } else {
ss, err = q.Select(matcher) ss, _, err = q.Select(nil, matcher)
} }
testutil.Ok(b, err) testutil.Ok(b, err)
for ss.Next() { for ss.Next() {

View file

@ -14,6 +14,7 @@
package tsdb package tsdb
import ( import (
"context"
"fmt" "fmt"
"io/ioutil" "io/ioutil"
"math" "math"
@ -26,6 +27,7 @@ import (
"github.com/pkg/errors" "github.com/pkg/errors"
"github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/pkg/labels"
"github.com/prometheus/prometheus/storage"
"github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/tsdb/chunkenc"
"github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/chunks"
"github.com/prometheus/prometheus/tsdb/index" "github.com/prometheus/prometheus/tsdb/index"
@ -36,22 +38,22 @@ import (
type mockSeriesSet struct { type mockSeriesSet struct {
next func() bool next func() bool
series func() Series series func() storage.Series
err func() error err func() error
} }
func (m *mockSeriesSet) Next() bool { return m.next() } func (m *mockSeriesSet) Next() bool { return m.next() }
func (m *mockSeriesSet) At() Series { return m.series() } func (m *mockSeriesSet) At() storage.Series { return m.series() }
func (m *mockSeriesSet) Err() error { return m.err() } func (m *mockSeriesSet) Err() error { return m.err() }
func newMockSeriesSet(list []Series) *mockSeriesSet { func newMockSeriesSet(list []storage.Series) *mockSeriesSet {
i := -1 i := -1
return &mockSeriesSet{ return &mockSeriesSet{
next: func() bool { next: func() bool {
i++ i++
return i < len(list) return i < len(list)
}, },
series: func() Series { series: func() storage.Series {
return list[i] return list[i]
}, },
err: func() error { return nil }, err: func() error { return nil },
@ -63,20 +65,20 @@ func TestMergedSeriesSet(t *testing.T) {
cases := []struct { cases := []struct {
// The input sets in order (samples in series in b are strictly // The input sets in order (samples in series in b are strictly
// after those in a). // after those in a).
a, b SeriesSet a, b storage.SeriesSet
// The composition of a and b in the partition series set must yield // The composition of a and b in the partition series set must yield
// results equivalent to the result series set. // results equivalent to the result series set.
exp SeriesSet exp storage.SeriesSet
}{ }{
{ {
a: newMockSeriesSet([]Series{ a: newMockSeriesSet([]storage.Series{
newSeries(map[string]string{ newSeries(map[string]string{
"a": "a", "a": "a",
}, []tsdbutil.Sample{ }, []tsdbutil.Sample{
sample{t: 1, v: 1}, sample{t: 1, v: 1},
}), }),
}), }),
b: newMockSeriesSet([]Series{ b: newMockSeriesSet([]storage.Series{
newSeries(map[string]string{ newSeries(map[string]string{
"a": "a", "a": "a",
}, []tsdbutil.Sample{ }, []tsdbutil.Sample{
@ -88,7 +90,7 @@ func TestMergedSeriesSet(t *testing.T) {
sample{t: 1, v: 1}, sample{t: 1, v: 1},
}), }),
}), }),
exp: newMockSeriesSet([]Series{ exp: newMockSeriesSet([]storage.Series{
newSeries(map[string]string{ newSeries(map[string]string{
"a": "a", "a": "a",
}, []tsdbutil.Sample{ }, []tsdbutil.Sample{
@ -103,7 +105,7 @@ func TestMergedSeriesSet(t *testing.T) {
}), }),
}, },
{ {
a: newMockSeriesSet([]Series{ a: newMockSeriesSet([]storage.Series{
newSeries(map[string]string{ newSeries(map[string]string{
"handler": "prometheus", "handler": "prometheus",
"instance": "127.0.0.1:9090", "instance": "127.0.0.1:9090",
@ -117,7 +119,7 @@ func TestMergedSeriesSet(t *testing.T) {
sample{t: 1, v: 2}, sample{t: 1, v: 2},
}), }),
}), }),
b: newMockSeriesSet([]Series{ b: newMockSeriesSet([]storage.Series{
newSeries(map[string]string{ newSeries(map[string]string{
"handler": "prometheus", "handler": "prometheus",
"instance": "127.0.0.1:9090", "instance": "127.0.0.1:9090",
@ -131,7 +133,7 @@ func TestMergedSeriesSet(t *testing.T) {
sample{t: 2, v: 2}, sample{t: 2, v: 2},
}), }),
}), }),
exp: newMockSeriesSet([]Series{ exp: newMockSeriesSet([]storage.Series{
newSeries(map[string]string{ newSeries(map[string]string{
"handler": "prometheus", "handler": "prometheus",
"instance": "127.0.0.1:9090", "instance": "127.0.0.1:9090",
@ -157,7 +159,7 @@ func TestMergedSeriesSet(t *testing.T) {
Outer: Outer:
for _, c := range cases { for _, c := range cases {
res := NewMergedSeriesSet([]SeriesSet{c.a, c.b}) res := NewMergedSeriesSet([]storage.SeriesSet{c.a, c.b})
for { for {
eok, rok := c.exp.Next(), res.Next() 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() { for it.Next() {
t, v := it.At() t, v := it.At()
r = append(r, sample{t: t, v: v}) 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) { 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{ return &mockSeries{
labels: func() labels.Labels { return labels.FromMap(l) }, labels: func() labels.Labels { return labels.FromMap(l) },
iterator: func() SeriesIterator { return newListSeriesIterator(s) }, iterator: func() chunkenc.Iterator { return newListSeriesIterator(s) },
} }
} }
type query struct { type query struct {
mint, maxt int64 mint, maxt int64
ms []*labels.Matcher ms []*labels.Matcher
exp SeriesSet exp storage.SeriesSet
} }
cases := struct { cases := struct {
@ -324,25 +326,25 @@ func TestBlockQuerier(t *testing.T) {
mint: 0, mint: 0,
maxt: 0, maxt: 0,
ms: []*labels.Matcher{}, ms: []*labels.Matcher{},
exp: newMockSeriesSet([]Series{}), exp: newMockSeriesSet([]storage.Series{}),
}, },
{ {
mint: 0, mint: 0,
maxt: 0, maxt: 0,
ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "a")}, ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "a")},
exp: newMockSeriesSet([]Series{}), exp: newMockSeriesSet([]storage.Series{}),
}, },
{ {
mint: 1, mint: 1,
maxt: 0, maxt: 0,
ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "a")}, ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "a")},
exp: newMockSeriesSet([]Series{}), exp: newMockSeriesSet([]storage.Series{}),
}, },
{ {
mint: 2, mint: 2,
maxt: 6, maxt: 6,
ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "a")}, ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "a")},
exp: newMockSeriesSet([]Series{ exp: newMockSeriesSet([]storage.Series{
newSeries(map[string]string{ newSeries(map[string]string{
"a": "a", "a": "a",
}, },
@ -371,8 +373,9 @@ Outer:
maxt: c.maxt, maxt: c.maxt,
} }
res, err := querier.Select(c.ms...) res, ws, err := querier.Select(nil, c.ms...)
testutil.Ok(t, err) testutil.Ok(t, err)
testutil.Equals(t, 0, len(ws))
for { for {
eok, rok := c.exp.Next(), res.Next() eok, rok := c.exp.Next(), res.Next()
@ -396,17 +399,17 @@ Outer:
} }
func TestBlockQuerierDelete(t *testing.T) { 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{ return &mockSeries{
labels: func() labels.Labels { return labels.FromMap(l) }, labels: func() labels.Labels { return labels.FromMap(l) },
iterator: func() SeriesIterator { return newListSeriesIterator(s) }, iterator: func() chunkenc.Iterator { return newListSeriesIterator(s) },
} }
} }
type query struct { type query struct {
mint, maxt int64 mint, maxt int64
ms []*labels.Matcher ms []*labels.Matcher
exp SeriesSet exp storage.SeriesSet
} }
cases := struct { cases := struct {
@ -467,7 +470,7 @@ func TestBlockQuerierDelete(t *testing.T) {
mint: 2, mint: 2,
maxt: 7, maxt: 7,
ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "a")}, ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "a")},
exp: newMockSeriesSet([]Series{ exp: newMockSeriesSet([]storage.Series{
newSeries(map[string]string{ newSeries(map[string]string{
"a": "a", "a": "a",
}, },
@ -485,7 +488,7 @@ func TestBlockQuerierDelete(t *testing.T) {
mint: 2, mint: 2,
maxt: 7, maxt: 7,
ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "b", "b")}, ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "b", "b")},
exp: newMockSeriesSet([]Series{ exp: newMockSeriesSet([]storage.Series{
newSeries(map[string]string{ newSeries(map[string]string{
"a": "a", "a": "a",
"b": "b", "b": "b",
@ -503,7 +506,7 @@ func TestBlockQuerierDelete(t *testing.T) {
mint: 1, mint: 1,
maxt: 4, maxt: 4,
ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "a")}, ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "a")},
exp: newMockSeriesSet([]Series{ exp: newMockSeriesSet([]storage.Series{
newSeries(map[string]string{ newSeries(map[string]string{
"a": "a", "a": "a",
"b": "b", "b": "b",
@ -516,12 +519,11 @@ func TestBlockQuerierDelete(t *testing.T) {
mint: 1, mint: 1,
maxt: 3, maxt: 3,
ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "a")}, ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "a")},
exp: newMockSeriesSet([]Series{}), exp: newMockSeriesSet([]storage.Series{}),
}, },
}, },
} }
fmt.Println("tombstones", cases.tombstones)
Outer: Outer:
for _, c := range cases.queries { for _, c := range cases.queries {
ir, cr, _, _ := createIdxChkReaders(t, cases.data) ir, cr, _, _ := createIdxChkReaders(t, cases.data)
@ -534,8 +536,9 @@ Outer:
maxt: c.maxt, maxt: c.maxt,
} }
res, err := querier.Select(c.ms...) res, ws, err := querier.Select(nil, c.ms...)
testutil.Ok(t, err) testutil.Ok(t, err)
testutil.Equals(t, 0, len(ws))
for { for {
eok, rok := c.exp.Next(), res.Next() 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 { type itSeries struct {
si SeriesIterator si chunkenc.Iterator
} }
func (s itSeries) Iterator() SeriesIterator { return s.si } func (s itSeries) Iterator() chunkenc.Iterator { return s.si }
func (s itSeries) Labels() labels.Labels { return labels.Labels{} } func (s itSeries) Labels() labels.Labels { return labels.Labels{} }
func TestSeriesIterator(t *testing.T) { func TestSeriesIterator(t *testing.T) {
itcases := []struct { itcases := []struct {
@ -1003,7 +1005,7 @@ func TestSeriesIterator(t *testing.T) {
t.Run("Seek", func(t *testing.T) { t.Run("Seek", func(t *testing.T) {
for _, tc := range seekcases { for _, tc := range seekcases {
ress := []SeriesIterator{ ress := []chunkenc.Iterator{
newChainedSeriesIterator( newChainedSeriesIterator(
itSeries{newListSeriesIterator(tc.a)}, itSeries{newListSeriesIterator(tc.a)},
itSeries{newListSeriesIterator(tc.b)}, 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. // 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. // 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) { func BenchmarkMergedSeriesSet(b *testing.B) {
var sel = func(sets []SeriesSet) SeriesSet { var sel = func(sets []storage.SeriesSet) storage.SeriesSet {
return NewMergedSeriesSet(sets) return NewMergedSeriesSet(sets)
} }
@ -1185,7 +1188,7 @@ func BenchmarkMergedSeriesSet(b *testing.B) {
sort.Sort(labels.Slice(lbls)) sort.Sort(labels.Slice(lbls))
in := make([][]Series, j) in := make([][]storage.Series, j)
for _, l := range lbls { for _, l := range lbls {
l2 := l l2 := l
@ -1197,7 +1200,7 @@ func BenchmarkMergedSeriesSet(b *testing.B) {
b.ResetTimer() b.ResetTimer()
for i := 0; i < b.N; i++ { for i := 0; i < b.N; i++ {
var sets []SeriesSet var sets []storage.SeriesSet
for _, s := range in { for _, s := range in {
sets = append(sets, newMockSeriesSet(s)) sets = append(sets, newMockSeriesSet(s))
} }
@ -1257,6 +1260,104 @@ func TestDeletedIterator(t *testing.T) {
{r: tombstones.Intervals{{Mint: 1000, Maxt: 20000}}}, {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 { for _, c := range cases {
i := int64(-1) i := int64(-1)
it := &deletedIterator{it: chk.Iterator(nil), intervals: c.r[:]} it := &deletedIterator{it: chk.Iterator(nil), intervals: c.r[:]}
@ -1414,17 +1515,17 @@ func (m mockIndex) LabelNames() ([]string, error) {
type mockSeries struct { type mockSeries struct {
labels func() labels.Labels 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{ return &mockSeries{
labels: func() labels.Labels { return labels.FromMap(l) }, 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) Labels() labels.Labels { return m.labels() }
func (m *mockSeries) Iterator() SeriesIterator { return m.iterator() } func (m *mockSeries) Iterator() chunkenc.Iterator { return m.iterator() }
type listSeriesIterator struct { type listSeriesIterator struct {
list []tsdbutil.Sample list []tsdbutil.Sample
@ -1493,7 +1594,7 @@ func BenchmarkQueryIterator(b *testing.B) {
blocks []*Block blocks []*Block
overlapDelta = int64(overlapPercentage * c.numSamplesPerSeriesPerBlock / 100) overlapDelta = int64(overlapPercentage * c.numSamplesPerSeriesPerBlock / 100)
prefilledLabels []map[string]string prefilledLabels []map[string]string
generatedSeries []Series generatedSeries []storage.Series
) )
for i := int64(0); i < int64(c.numBlocks); i++ { for i := int64(0); i < int64(c.numBlocks); i++ {
offset := i * overlapDelta offset := i * overlapDelta
@ -1514,7 +1615,7 @@ func BenchmarkQueryIterator(b *testing.B) {
} }
que := &querier{ que := &querier{
blocks: make([]Querier, 0, len(blocks)), blocks: make([]storage.Querier, 0, len(blocks)),
} }
for _, blk := range blocks { for _, blk := range blocks {
q, err := NewBlockQuerier(blk, math.MinInt64, math.MaxInt64) q, err := NewBlockQuerier(blk, math.MinInt64, math.MaxInt64)
@ -1522,7 +1623,7 @@ func BenchmarkQueryIterator(b *testing.B) {
que.blocks = append(que.blocks, q) que.blocks = append(que.blocks, q)
} }
var sq Querier = que var sq storage.Querier = que
if overlapPercentage > 0 { if overlapPercentage > 0 {
sq = &verticalQuerier{ sq = &verticalQuerier{
querier: *que, querier: *que,
@ -1567,7 +1668,7 @@ func BenchmarkQuerySeek(b *testing.B) {
blocks []*Block blocks []*Block
overlapDelta = int64(overlapPercentage * c.numSamplesPerSeriesPerBlock / 100) overlapDelta = int64(overlapPercentage * c.numSamplesPerSeriesPerBlock / 100)
prefilledLabels []map[string]string prefilledLabels []map[string]string
generatedSeries []Series generatedSeries []storage.Series
) )
for i := int64(0); i < int64(c.numBlocks); i++ { for i := int64(0); i < int64(c.numBlocks); i++ {
offset := i * overlapDelta offset := i * overlapDelta
@ -1588,7 +1689,7 @@ func BenchmarkQuerySeek(b *testing.B) {
} }
que := &querier{ que := &querier{
blocks: make([]Querier, 0, len(blocks)), blocks: make([]storage.Querier, 0, len(blocks)),
} }
for _, blk := range blocks { for _, blk := range blocks {
q, err := NewBlockQuerier(blk, math.MinInt64, math.MaxInt64) q, err := NewBlockQuerier(blk, math.MinInt64, math.MaxInt64)
@ -1596,7 +1697,7 @@ func BenchmarkQuerySeek(b *testing.B) {
que.blocks = append(que.blocks, q) que.blocks = append(que.blocks, q)
} }
var sq Querier = que var sq storage.Querier = que
if overlapPercentage > 0 { if overlapPercentage > 0 {
sq = &verticalQuerier{ sq = &verticalQuerier{
querier: *que, querier: *que,
@ -1610,7 +1711,7 @@ func BenchmarkQuerySeek(b *testing.B) {
b.ResetTimer() b.ResetTimer()
b.ReportAllocs() 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() { for ss.Next() {
it := ss.At().Iterator() it := ss.At().Iterator()
for t := mint; t <= maxt; t++ { for t := mint; t <= maxt; t++ {
@ -1620,6 +1721,7 @@ func BenchmarkQuerySeek(b *testing.B) {
} }
testutil.Ok(b, ss.Err()) testutil.Ok(b, ss.Err())
testutil.Ok(b, err) testutil.Ok(b, err)
testutil.Equals(b, 0, len(ws))
}) })
} }
} }
@ -1713,7 +1815,7 @@ func BenchmarkSetMatcher(b *testing.B) {
var ( var (
blocks []*Block blocks []*Block
prefilledLabels []map[string]string prefilledLabels []map[string]string
generatedSeries []Series generatedSeries []storage.Series
) )
for i := int64(0); i < int64(c.numBlocks); i++ { for i := int64(0); i < int64(c.numBlocks); i++ {
mint := i * int64(c.numSamplesPerSeriesPerBlock) mint := i * int64(c.numSamplesPerSeriesPerBlock)
@ -1733,7 +1835,7 @@ func BenchmarkSetMatcher(b *testing.B) {
} }
que := &querier{ que := &querier{
blocks: make([]Querier, 0, len(blocks)), blocks: make([]storage.Querier, 0, len(blocks)),
} }
for _, blk := range blocks { for _, blk := range blocks {
q, err := NewBlockQuerier(blk, math.MinInt64, math.MaxInt64) q, err := NewBlockQuerier(blk, math.MinInt64, math.MaxInt64)
@ -1747,8 +1849,9 @@ func BenchmarkSetMatcher(b *testing.B) {
b.ResetTimer() b.ResetTimer()
b.ReportAllocs() b.ReportAllocs()
for n := 0; n < b.N; n++ { 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.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, 0, 10))
createBlock(t, dir, genSeries(1, 1, 10, 20)) 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 { if err != nil {
t.Fatalf("Opening test storage failed: %s", err) t.Fatalf("Opening test storage failed: %s", err)
} }
@ -2084,7 +2187,7 @@ func TestClose(t *testing.T) {
testutil.Ok(t, db.Close()) 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, err)
testutil.Ok(t, q.Close()) testutil.Ok(t, q.Close())
testutil.NotOk(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() { defer func() {
for _, q := range queryTypes { for _, q := range queryTypes {
// Can't run a check for error here as some of these will fail as // 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++ { for x := 0; x <= 10; x++ {
block, err := OpenBlock(nil, createBlock(b, dir, series), nil) block, err := OpenBlock(nil, createBlock(b, dir, series), nil)
testutil.Ok(b, err) 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.ResetTimer()
b.ReportAllocs() b.ReportAllocs()
for i := 0; i < b.N; i++ { for i := 0; i < b.N; i++ {
ss, err := q.Select(selectors...) ss, ws, err := q.Select(nil, selectors...)
testutil.Ok(b, err) testutil.Ok(b, err)
testutil.Equals(b, 0, len(ws))
var actualExpansions int var actualExpansions int
for ss.Next() { for ss.Next() {
s := ss.At() s := ss.At()

View file

@ -65,7 +65,7 @@ func CreateBlock(samples []*MetricSample, dir string, mint, maxt int64, logger l
return "", err 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 { if err != nil {
return "", err return "", err
} }

View file

@ -16,11 +16,9 @@ package teststorage
import ( import (
"io/ioutil" "io/ioutil"
"os" "os"
"time"
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/storage"
"github.com/prometheus/prometheus/storage/tsdb" "github.com/prometheus/prometheus/tsdb"
"github.com/prometheus/prometheus/util/testutil" "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 // Tests just load data for a series sequentially. Thus we
// need a long appendable window. // need a long appendable window.
db, err := tsdb.Open(dir, nil, nil, &tsdb.Options{ db, err := tsdb.Open(dir, nil, nil, nil)
MinBlockDuration: model.Duration(24 * time.Hour),
MaxBlockDuration: model.Duration(24 * time.Hour),
})
if err != nil { if err != nil {
t.Fatalf("Opening test storage failed: %s", err) 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 { type testStorage struct {

View file

@ -62,7 +62,6 @@ import (
"github.com/prometheus/prometheus/rules" "github.com/prometheus/prometheus/rules"
"github.com/prometheus/prometheus/scrape" "github.com/prometheus/prometheus/scrape"
"github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/storage"
prometheus_tsdb "github.com/prometheus/prometheus/storage/tsdb"
"github.com/prometheus/prometheus/template" "github.com/prometheus/prometheus/template"
"github.com/prometheus/prometheus/util/httputil" "github.com/prometheus/prometheus/util/httputil"
api_v1 "github.com/prometheus/prometheus/web/api/v1" api_v1 "github.com/prometheus/prometheus/web/api/v1"
@ -213,7 +212,7 @@ func (h *Handler) ApplyConfig(conf *config.Config) error {
type Options struct { type Options struct {
Context context.Context Context context.Context
TSDB func() *tsdb.DB TSDB func() *tsdb.DB
TSDBCfg prometheus_tsdb.Options TSDBCfg tsdb.Options
Storage storage.Storage Storage storage.Storage
QueryEngine *promql.Engine QueryEngine *promql.Engine
LookbackDelta time.Duration LookbackDelta time.Duration

View file

@ -33,8 +33,7 @@ import (
"github.com/prometheus/prometheus/notifier" "github.com/prometheus/prometheus/notifier"
"github.com/prometheus/prometheus/rules" "github.com/prometheus/prometheus/rules"
"github.com/prometheus/prometheus/scrape" "github.com/prometheus/prometheus/scrape"
"github.com/prometheus/prometheus/storage/tsdb" "github.com/prometheus/prometheus/tsdb"
libtsdb "github.com/prometheus/prometheus/tsdb"
"github.com/prometheus/prometheus/util/testutil" "github.com/prometheus/prometheus/util/testutil"
) )
@ -98,7 +97,7 @@ func TestReadyAndHealthy(t *testing.T) {
testutil.Ok(t, err) testutil.Ok(t, err)
defer os.RemoveAll(dbDir) defer os.RemoveAll(dbDir)
db, err := libtsdb.Open(dbDir, nil, nil, nil) db, err := tsdb.Open(dbDir, nil, nil, nil)
testutil.Ok(t, err) testutil.Ok(t, err)
@ -114,7 +113,7 @@ func TestReadyAndHealthy(t *testing.T) {
Notifier: nil, Notifier: nil,
RoutePrefix: "/", RoutePrefix: "/",
EnableAdminAPI: true, EnableAdminAPI: true,
TSDB: func() *libtsdb.DB { return db }, TSDB: func() *tsdb.DB { return db },
ExternalURL: &url.URL{ ExternalURL: &url.URL{
Scheme: "http", Scheme: "http",
Host: "localhost:9090", Host: "localhost:9090",
@ -289,7 +288,7 @@ func TestRoutePrefix(t *testing.T) {
defer os.RemoveAll(dbDir) defer os.RemoveAll(dbDir)
db, err := libtsdb.Open(dbDir, nil, nil, nil) db, err := tsdb.Open(dbDir, nil, nil, nil)
testutil.Ok(t, err) testutil.Ok(t, err)
@ -309,7 +308,7 @@ func TestRoutePrefix(t *testing.T) {
Host: "localhost.localdomain:9090", Host: "localhost.localdomain:9090",
Scheme: "http", Scheme: "http",
}, },
TSDB: func() *libtsdb.DB { return db }, TSDB: func() *tsdb.DB { return db },
} }
opts.Flags = map[string]string{} opts.Flags = map[string]string{}