diff --git a/cmd/prometheus/main.go b/cmd/prometheus/main.go index 2c5ccb85a..21f60678e 100644 --- a/cmd/prometheus/main.go +++ b/cmd/prometheus/main.go @@ -1363,11 +1363,11 @@ func (s *readyStorage) Appender(ctx context.Context) storage.Appender { type notReadyAppender struct{} -func (n notReadyAppender) Append(ref uint64, l labels.Labels, t int64, v float64) (uint64, error) { +func (n notReadyAppender) Append(ref storage.SeriesRef, l labels.Labels, t int64, v float64) (storage.SeriesRef, error) { return 0, tsdb.ErrNotReady } -func (n notReadyAppender) AppendExemplar(ref uint64, l labels.Labels, e exemplar.Exemplar) (uint64, error) { +func (n notReadyAppender) AppendExemplar(ref storage.SeriesRef, l labels.Labels, e exemplar.Exemplar) (storage.SeriesRef, error) { return 0, tsdb.ErrNotReady } diff --git a/cmd/promtool/tsdb.go b/cmd/promtool/tsdb.go index 6318336ce..a3f134fd9 100644 --- a/cmd/promtool/tsdb.go +++ b/cmd/promtool/tsdb.go @@ -31,6 +31,9 @@ import ( "text/tabwriter" "time" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/index" + "github.com/alecthomas/units" "github.com/go-kit/log" "github.com/pkg/errors" @@ -40,7 +43,6 @@ import ( "github.com/prometheus/prometheus/tsdb/chunks" tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" "github.com/prometheus/prometheus/tsdb/fileutil" - "github.com/prometheus/prometheus/tsdb/index" ) const timeDelta = 30000 @@ -187,7 +189,7 @@ func (b *writeBenchmark) ingestScrapesShard(lbls []labels.Labels, scrapeCount in type sample struct { labels labels.Labels value int64 - ref *uint64 + ref *storage.SeriesRef } scrape := make([]*sample, 0, len(lbls)) @@ -207,7 +209,7 @@ func (b *writeBenchmark) ingestScrapesShard(lbls []labels.Labels, scrapeCount in for _, s := range scrape { s.value += 1000 - var ref uint64 + var ref storage.SeriesRef if s.ref != nil { ref = *s.ref } diff --git a/promql/bench_test.go b/promql/bench_test.go index eb020d2f1..6eb177250 100644 --- a/promql/bench_test.go +++ b/promql/bench_test.go @@ -23,12 +23,13 @@ import ( "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/promql/parser" + "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/util/teststorage" ) func BenchmarkRangeQuery(b *testing.B) { - storage := teststorage.New(b) - defer storage.Close() + stor := teststorage.New(b) + defer stor.Close() opts := EngineOpts{ Logger: nil, Reg: nil, @@ -62,13 +63,13 @@ func BenchmarkRangeQuery(b *testing.B) { } metrics = append(metrics, labels.FromStrings("__name__", "h_hundred", "l", strconv.Itoa(i), "le", "+Inf")) } - refs := make([]uint64, len(metrics)) + refs := make([]storage.SeriesRef, len(metrics)) // A day of data plus 10k steps. numIntervals := 8640 + 10000 for s := 0; s < numIntervals; s++ { - a := storage.Appender(context.Background()) + a := stor.Appender(context.Background()) ts := int64(s * 10000) // 10s interval. for i, metric := range metrics { ref, _ := a.Append(refs[i], metric, ts, float64(s)+float64(i)/float64(len(metrics))) @@ -216,7 +217,7 @@ func BenchmarkRangeQuery(b *testing.B) { b.ReportAllocs() for i := 0; i < b.N; i++ { qry, err := engine.NewRangeQuery( - storage, c.expr, + stor, c.expr, time.Unix(int64((numIntervals-c.steps)*10), 0), time.Unix(int64(numIntervals*10), 0), time.Second*10) if err != nil { diff --git a/scrape/helpers_test.go b/scrape/helpers_test.go index da29d6c12..7a043c0b1 100644 --- a/scrape/helpers_test.go +++ b/scrape/helpers_test.go @@ -30,8 +30,11 @@ func (a nopAppendable) Appender(_ context.Context) storage.Appender { type nopAppender struct{} -func (a nopAppender) Append(uint64, labels.Labels, int64, float64) (uint64, error) { return 0, nil } -func (a nopAppender) AppendExemplar(uint64, labels.Labels, exemplar.Exemplar) (uint64, error) { +func (a nopAppender) Append(storage.SeriesRef, labels.Labels, int64, float64) (storage.SeriesRef, error) { + return 0, nil +} + +func (a nopAppender) AppendExemplar(storage.SeriesRef, labels.Labels, exemplar.Exemplar) (storage.SeriesRef, error) { return 0, nil } func (a nopAppender) Commit() error { return nil } @@ -54,7 +57,7 @@ type collectResultAppender struct { resultExemplars []exemplar.Exemplar } -func (a *collectResultAppender) Append(ref uint64, lset labels.Labels, t int64, v float64) (uint64, error) { +func (a *collectResultAppender) Append(ref storage.SeriesRef, lset labels.Labels, t int64, v float64) (storage.SeriesRef, error) { a.pendingResult = append(a.pendingResult, sample{ metric: lset, t: t, @@ -62,7 +65,7 @@ func (a *collectResultAppender) Append(ref uint64, lset labels.Labels, t int64, }) if ref == 0 { - ref = rand.Uint64() + ref = storage.SeriesRef(rand.Uint64()) } if a.next == nil { return ref, nil @@ -75,7 +78,7 @@ func (a *collectResultAppender) Append(ref uint64, lset labels.Labels, t int64, return ref, err } -func (a *collectResultAppender) AppendExemplar(ref uint64, l labels.Labels, e exemplar.Exemplar) (uint64, error) { +func (a *collectResultAppender) AppendExemplar(ref storage.SeriesRef, l labels.Labels, e exemplar.Exemplar) (storage.SeriesRef, error) { a.pendingExemplars = append(a.pendingExemplars, e) if a.next == nil { return 0, nil diff --git a/scrape/scrape.go b/scrape/scrape.go index 590133cde..161ede6ff 100644 --- a/scrape/scrape.go +++ b/scrape/scrape.go @@ -829,7 +829,7 @@ type loop interface { } type cacheEntry struct { - ref uint64 + ref storage.SeriesRef lastIter uint64 hash uint64 lset labels.Labels @@ -976,7 +976,7 @@ func (c *scrapeCache) get(met string) (*cacheEntry, bool) { return e, true } -func (c *scrapeCache) addRef(met string, ref uint64, lset labels.Labels, hash uint64) { +func (c *scrapeCache) addRef(met string, ref storage.SeriesRef, lset labels.Labels, hash uint64) { if ref == 0 { return } @@ -1477,7 +1477,7 @@ loop: } ce, ok := sl.cache.get(yoloString(met)) var ( - ref uint64 + ref storage.SeriesRef lset labels.Labels mets string hash uint64 @@ -1727,7 +1727,7 @@ func (sl *scrapeLoop) reportStale(app storage.Appender, start time.Time) (err er func (sl *scrapeLoop) addReportSample(app storage.Appender, s string, t int64, v float64) error { ce, ok := sl.cache.get(s) - var ref uint64 + var ref storage.SeriesRef var lset labels.Labels if ok { ref = ce.ref diff --git a/scrape/scrape_test.go b/scrape/scrape_test.go index 3b34f1416..e52ea96aa 100644 --- a/scrape/scrape_test.go +++ b/scrape/scrape_test.go @@ -1475,7 +1475,7 @@ func TestScrapeLoopAppendCacheEntryButErrNotFound(t *testing.T) { false, ) - fakeRef := uint64(1) + fakeRef := storage.SeriesRef(1) expValue := float64(1) metric := `metric{n="1"} 1` p := textparse.New([]byte(metric), "") @@ -1962,7 +1962,7 @@ type errorAppender struct { collectResultAppender } -func (app *errorAppender) Append(ref uint64, lset labels.Labels, t int64, v float64) (uint64, error) { +func (app *errorAppender) Append(ref storage.SeriesRef, lset labels.Labels, t int64, v float64) (storage.SeriesRef, error) { switch lset.Get(model.MetricNameLabel) { case "out_of_order": return 0, storage.ErrOutOfOrderSample diff --git a/scrape/target.go b/scrape/target.go index 742e30663..38fea83b3 100644 --- a/scrape/target.go +++ b/scrape/target.go @@ -316,7 +316,7 @@ type limitAppender struct { i int } -func (app *limitAppender) Append(ref uint64, lset labels.Labels, t int64, v float64) (uint64, error) { +func (app *limitAppender) Append(ref storage.SeriesRef, lset labels.Labels, t int64, v float64) (storage.SeriesRef, error) { if !value.IsStaleNaN(v) { app.i++ if app.i > app.limit { @@ -336,7 +336,7 @@ type timeLimitAppender struct { maxTime int64 } -func (app *timeLimitAppender) Append(ref uint64, lset labels.Labels, t int64, v float64) (uint64, error) { +func (app *timeLimitAppender) Append(ref storage.SeriesRef, lset labels.Labels, t int64, v float64) (storage.SeriesRef, error) { if t > app.maxTime { return 0, storage.ErrOutOfBounds } diff --git a/storage/fanout.go b/storage/fanout.go index 206105fa5..17bac242d 100644 --- a/storage/fanout.go +++ b/storage/fanout.go @@ -144,7 +144,7 @@ type fanoutAppender struct { secondaries []Appender } -func (f *fanoutAppender) Append(ref uint64, l labels.Labels, t int64, v float64) (uint64, error) { +func (f *fanoutAppender) Append(ref SeriesRef, l labels.Labels, t int64, v float64) (SeriesRef, error) { ref, err := f.primary.Append(ref, l, t, v) if err != nil { return ref, err @@ -158,7 +158,7 @@ func (f *fanoutAppender) Append(ref uint64, l labels.Labels, t int64, v float64) return ref, nil } -func (f *fanoutAppender) AppendExemplar(ref uint64, l labels.Labels, e exemplar.Exemplar) (uint64, error) { +func (f *fanoutAppender) AppendExemplar(ref SeriesRef, l labels.Labels, e exemplar.Exemplar) (SeriesRef, error) { ref, err := f.primary.AppendExemplar(ref, l, e) if err != nil { return ref, err diff --git a/storage/interface.go b/storage/interface.go index 92ace42ae..fb4a2e414 100644 --- a/storage/interface.go +++ b/storage/interface.go @@ -36,6 +36,11 @@ var ( ErrExemplarsDisabled = fmt.Errorf("exemplar storage is disabled or max exemplars is less than or equal to 0") ) +// SeriesRef is a generic series reference. In prometheus it is either a +// HeadSeriesRef or BlockSeriesRef, though other implementations may have +// their own reference types. +type SeriesRef uint64 + // Appendable allows creating appenders. type Appendable interface { // Appender returns a new appender for the storage. The implementation @@ -175,7 +180,7 @@ type Appender interface { // to Append() at any point. Adding the sample via Append() returns a new // reference number. // If the reference is 0 it must not be used for caching. - Append(ref uint64, l labels.Labels, t int64, v float64) (uint64, error) + Append(ref SeriesRef, l labels.Labels, t int64, v float64) (SeriesRef, error) // Commit submits the collected samples and purges the batch. If Commit // returns a non-nil error, it also rolls back all modifications made in @@ -196,7 +201,7 @@ type GetRef interface { // Returns reference number that can be used to pass to Appender.Append(), // and a set of labels that will not cause another copy when passed to Appender.Append(). // 0 means the appender does not have a reference to this series. - GetRef(lset labels.Labels) (uint64, labels.Labels) + GetRef(lset labels.Labels) (SeriesRef, labels.Labels) } // ExemplarAppender provides an interface for adding samples to exemplar storage, which @@ -213,7 +218,7 @@ type ExemplarAppender interface { // Note that in our current implementation of Prometheus' exemplar storage // calls to Append should generate the reference numbers, AppendExemplar // generating a new reference number should be considered possible erroneous behaviour and be logged. - AppendExemplar(ref uint64, l labels.Labels, e exemplar.Exemplar) (uint64, error) + AppendExemplar(ref SeriesRef, l labels.Labels, e exemplar.Exemplar) (SeriesRef, error) } // SeriesSet contains a set of series. diff --git a/storage/remote/queue_manager.go b/storage/remote/queue_manager.go index 4cbcdd2b9..f4e74273b 100644 --- a/storage/remote/queue_manager.go +++ b/storage/remote/queue_manager.go @@ -35,6 +35,7 @@ import ( "github.com/prometheus/prometheus/pkg/relabel" "github.com/prometheus/prometheus/prompb" "github.com/prometheus/prometheus/scrape" + "github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/record" "github.com/prometheus/prometheus/tsdb/wal" ) @@ -353,11 +354,11 @@ type QueueManager struct { storeClient WriteClient seriesMtx sync.Mutex // Covers seriesLabels and droppedSeries. - seriesLabels map[uint64]labels.Labels - droppedSeries map[uint64]struct{} + seriesLabels map[chunks.HeadSeriesRef]labels.Labels + droppedSeries map[chunks.HeadSeriesRef]struct{} seriesSegmentMtx sync.Mutex // Covers seriesSegmentIndexes - if you also lock seriesMtx, take seriesMtx first. - seriesSegmentIndexes map[uint64]int + seriesSegmentIndexes map[chunks.HeadSeriesRef]int shards *shards numShards int @@ -406,9 +407,9 @@ func NewQueueManager( storeClient: client, sendExemplars: enableExemplarRemoteWrite, - seriesLabels: make(map[uint64]labels.Labels), - seriesSegmentIndexes: make(map[uint64]int), - droppedSeries: make(map[uint64]struct{}), + seriesLabels: make(map[chunks.HeadSeriesRef]labels.Labels), + seriesSegmentIndexes: make(map[chunks.HeadSeriesRef]int), + droppedSeries: make(map[chunks.HeadSeriesRef]struct{}), numShards: cfg.MinShards, reshardChan: make(chan int), @@ -997,7 +998,7 @@ func (s *shards) stop() { // enqueue data (sample or exemplar). If we are currently in the process of shutting down or resharding, // will return false; in this case, you should back off and retry. -func (s *shards) enqueue(ref uint64, data interface{}) bool { +func (s *shards) enqueue(ref chunks.HeadSeriesRef, data interface{}) bool { s.mtx.RLock() defer s.mtx.RUnlock() diff --git a/storage/remote/queue_manager_test.go b/storage/remote/queue_manager_test.go index 9a07f7c99..3706544f0 100644 --- a/storage/remote/queue_manager_test.go +++ b/storage/remote/queue_manager_test.go @@ -43,6 +43,7 @@ import ( "github.com/prometheus/prometheus/pkg/timestamp" "github.com/prometheus/prometheus/prompb" "github.com/prometheus/prometheus/scrape" + "github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/record" ) @@ -227,12 +228,12 @@ func TestSampleDeliveryOrder(t *testing.T) { for i := 0; i < n; i++ { name := fmt.Sprintf("test_metric_%d", i%ts) samples = append(samples, record.RefSample{ - Ref: uint64(i), + Ref: chunks.HeadSeriesRef(i), T: int64(i), V: float64(i), }) series = append(series, record.RefSeries{ - Ref: uint64(i), + Ref: chunks.HeadSeriesRef(i), Labels: labels.Labels{labels.Label{Name: "__name__", Value: name}}, }) } @@ -320,7 +321,7 @@ func TestSeriesReset(t *testing.T) { for i := 0; i < numSegments; i++ { series := []record.RefSeries{} for j := 0; j < numSeries; j++ { - series = append(series, record.RefSeries{Ref: uint64((i * 100) + j), Labels: labels.Labels{{Name: "a", Value: "a"}}}) + series = append(series, record.RefSeries{Ref: chunks.HeadSeriesRef((i * 100) + j), Labels: labels.Labels{{Name: "a", Value: "a"}}}) } m.StoreSeries(series, i) } @@ -411,7 +412,7 @@ func TestReleaseNoninternedString(t *testing.T) { for i := 1; i < 1000; i++ { m.StoreSeries([]record.RefSeries{ { - Ref: uint64(i), + Ref: chunks.HeadSeriesRef(i), Labels: labels.Labels{ labels.Label{ Name: "asdf", @@ -480,13 +481,13 @@ func createTimeseries(numSamples, numSeries int, extraLabels ...labels.Label) ([ name := fmt.Sprintf("test_metric_%d", i) for j := 0; j < numSamples; j++ { samples = append(samples, record.RefSample{ - Ref: uint64(i), + Ref: chunks.HeadSeriesRef(i), T: int64(j), V: float64(i), }) } series = append(series, record.RefSeries{ - Ref: uint64(i), + Ref: chunks.HeadSeriesRef(i), Labels: append(labels.Labels{{Name: "__name__", Value: name}}, extraLabels...), }) } @@ -500,7 +501,7 @@ func createExemplars(numExemplars, numSeries int) ([]record.RefExemplar, []recor name := fmt.Sprintf("test_metric_%d", i) for j := 0; j < numExemplars; j++ { e := record.RefExemplar{ - Ref: uint64(i), + Ref: chunks.HeadSeriesRef(i), T: int64(j), V: float64(i), Labels: labels.FromStrings("traceID", fmt.Sprintf("trace-%d", i)), @@ -508,7 +509,7 @@ func createExemplars(numExemplars, numSeries int) ([]record.RefExemplar, []recor exemplars = append(exemplars, e) } series = append(series, record.RefSeries{ - Ref: uint64(i), + Ref: chunks.HeadSeriesRef(i), Labels: labels.Labels{{Name: "__name__", Value: name}}, }) } diff --git a/storage/remote/write.go b/storage/remote/write.go index cbbfaf52e..b3bd983bc 100644 --- a/storage/remote/write.go +++ b/storage/remote/write.go @@ -255,7 +255,7 @@ type timestampTracker struct { } // Append implements storage.Appender. -func (t *timestampTracker) Append(_ uint64, _ labels.Labels, ts int64, _ float64) (uint64, error) { +func (t *timestampTracker) Append(_ storage.SeriesRef, _ labels.Labels, ts int64, _ float64) (storage.SeriesRef, error) { t.samples++ if ts > t.highestTimestamp { t.highestTimestamp = ts @@ -263,7 +263,7 @@ func (t *timestampTracker) Append(_ uint64, _ labels.Labels, ts int64, _ float64 return 0, nil } -func (t *timestampTracker) AppendExemplar(_ uint64, _ labels.Labels, _ exemplar.Exemplar) (uint64, error) { +func (t *timestampTracker) AppendExemplar(_ storage.SeriesRef, _ labels.Labels, _ exemplar.Exemplar) (storage.SeriesRef, error) { t.exemplars++ return 0, nil } diff --git a/storage/remote/write_handler_test.go b/storage/remote/write_handler_test.go index b4930f00b..7d467c8fb 100644 --- a/storage/remote/write_handler_test.go +++ b/storage/remote/write_handler_test.go @@ -159,7 +159,7 @@ func (m *mockAppendable) Appender(_ context.Context) storage.Appender { return m } -func (m *mockAppendable) Append(_ uint64, l labels.Labels, t int64, v float64) (uint64, error) { +func (m *mockAppendable) Append(_ storage.SeriesRef, l labels.Labels, t int64, v float64) (storage.SeriesRef, error) { if t < m.latestSample { return 0, storage.ErrOutOfOrderSample } @@ -177,7 +177,7 @@ func (*mockAppendable) Rollback() error { return fmt.Errorf("not implemented") } -func (m *mockAppendable) AppendExemplar(_ uint64, l labels.Labels, e exemplar.Exemplar) (uint64, error) { +func (m *mockAppendable) AppendExemplar(_ storage.SeriesRef, l labels.Labels, e exemplar.Exemplar) (storage.SeriesRef, error) { if e.Ts < m.latestExemplar { return 0, storage.ErrOutOfOrderExemplar } diff --git a/tsdb/agent/db.go b/tsdb/agent/db.go index 36e843e83..40161a445 100644 --- a/tsdb/agent/db.go +++ b/tsdb/agent/db.go @@ -33,6 +33,7 @@ import ( "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/storage/remote" "github.com/prometheus/prometheus/tsdb" + "github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/record" "github.com/prometheus/prometheus/tsdb/wal" ) @@ -195,7 +196,7 @@ type DB struct { series *stripeSeries // deleted is a map of (ref IDs that should be deleted from WAL) to (the WAL segment they // must be kept around to). - deleted map[uint64]int + deleted map[chunks.HeadSeriesRef]int donec chan struct{} stopc chan struct{} @@ -224,7 +225,7 @@ func Open(l log.Logger, reg prometheus.Registerer, rs *remote.Storage, dir strin nextRef: atomic.NewUint64(0), series: newStripeSeries(opts.StripeSize), - deleted: make(map[uint64]int), + deleted: make(map[chunks.HeadSeriesRef]int), donec: make(chan struct{}), stopc: make(chan struct{}), @@ -292,7 +293,7 @@ func (db *DB) replayWAL() error { return errors.Wrap(err, "find last checkpoint") } - multiRef := map[uint64]uint64{} + multiRef := map[chunks.HeadSeriesRef]chunks.HeadSeriesRef{} if err == nil { sr, err := wal.NewSegmentsReader(dir) @@ -344,10 +345,10 @@ func (db *DB) replayWAL() error { return nil } -func (db *DB) loadWAL(r *wal.Reader, multiRef map[uint64]uint64) (err error) { +func (db *DB) loadWAL(r *wal.Reader, multiRef map[chunks.HeadSeriesRef]chunks.HeadSeriesRef) (err error) { var ( dec record.Decoder - lastRef uint64 + lastRef chunks.HeadSeriesRef decoded = make(chan interface{}, 10) errCh = make(chan error, 1) @@ -455,7 +456,7 @@ func (db *DB) loadWAL(r *wal.Reader, multiRef map[uint64]uint64) (err error) { level.Warn(db.logger).Log("msg", "found sample referencing non-existing series", "skipped_series", v) } - db.nextRef.Store(lastRef) + db.nextRef.Store(uint64(lastRef)) select { case err := <-errCh: @@ -538,7 +539,7 @@ func (db *DB) truncate(mint int64) error { return nil } - keep := func(id uint64) bool { + keep := func(id chunks.HeadSeriesRef) bool { if db.series.GetByID(id) != nil { return true } @@ -650,14 +651,15 @@ type appender struct { pendingSamples []record.RefSample } -func (a *appender) Append(ref uint64, l labels.Labels, t int64, v float64) (uint64, error) { +func (a *appender) Append(ref storage.SeriesRef, l labels.Labels, t int64, v float64) (storage.SeriesRef, error) { if ref == 0 { - return a.Add(l, t, v) + r, err := a.Add(l, t, v) + return storage.SeriesRef(r), err } - return ref, a.AddFast(ref, t, v) + return ref, a.AddFast(chunks.HeadSeriesRef(ref), t, v) } -func (a *appender) Add(l labels.Labels, t int64, v float64) (uint64, error) { +func (a *appender) Add(l labels.Labels, t int64, v float64) (chunks.HeadSeriesRef, error) { hash := l.Hash() series := a.series.GetByHash(hash, l) if series != nil { @@ -675,7 +677,7 @@ func (a *appender) Add(l labels.Labels, t int64, v float64) (uint64, error) { return 0, errors.Wrap(tsdb.ErrInvalidSample, fmt.Sprintf(`label name "%s" is not unique`, lbl)) } - ref := a.nextRef.Inc() + ref := chunks.HeadSeriesRef(a.nextRef.Inc()) series = &memSeries{ref: ref, lset: l, lastTs: t} a.pendingSeries = append(a.pendingSeries, record.RefSeries{ @@ -696,7 +698,7 @@ func (a *appender) Add(l labels.Labels, t int64, v float64) (uint64, error) { return series.ref, nil } -func (a *appender) AddFast(ref uint64, t int64, v float64) error { +func (a *appender) AddFast(ref chunks.HeadSeriesRef, t int64, v float64) error { series := a.series.GetByID(ref) if series == nil { return storage.ErrNotFound @@ -718,7 +720,7 @@ func (a *appender) AddFast(ref uint64, t int64, v float64) error { return nil } -func (a *appender) AppendExemplar(ref uint64, l labels.Labels, e exemplar.Exemplar) (uint64, error) { +func (a *appender) AppendExemplar(ref storage.SeriesRef, l labels.Labels, e exemplar.Exemplar) (storage.SeriesRef, error) { // remote_write doesn't support exemplars yet, so do nothing here. return 0, nil } diff --git a/tsdb/agent/series.go b/tsdb/agent/series.go index 557f709bb..80c440ae6 100644 --- a/tsdb/agent/series.go +++ b/tsdb/agent/series.go @@ -17,13 +17,14 @@ import ( "sync" "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/tsdb/chunks" ) // memSeries is a chunkless version of tsdb.memSeries. type memSeries struct { sync.Mutex - ref uint64 + ref chunks.HeadSeriesRef lset labels.Labels lastTs int64 } @@ -54,7 +55,7 @@ func (m seriesHashmap) Set(hash uint64, s *memSeries) { m[hash] = append(seriesSet, s) } -func (m seriesHashmap) Delete(hash, ref uint64) { +func (m seriesHashmap) Delete(hash uint64, ref chunks.HeadSeriesRef) { var rem []*memSeries for _, s := range m[hash] { if s.ref != ref { @@ -74,7 +75,7 @@ func (m seriesHashmap) Delete(hash, ref uint64) { // likely due to the additional pointer dereferences. type stripeSeries struct { size int - series []map[uint64]*memSeries + series []map[chunks.HeadSeriesRef]*memSeries hashes []seriesHashmap locks []stripeLock } @@ -88,12 +89,12 @@ type stripeLock struct { func newStripeSeries(stripeSize int) *stripeSeries { s := &stripeSeries{ size: stripeSize, - series: make([]map[uint64]*memSeries, stripeSize), + series: make([]map[chunks.HeadSeriesRef]*memSeries, stripeSize), hashes: make([]seriesHashmap, stripeSize), locks: make([]stripeLock, stripeSize), } for i := range s.series { - s.series[i] = map[uint64]*memSeries{} + s.series[i] = map[chunks.HeadSeriesRef]*memSeries{} } for i := range s.hashes { s.hashes[i] = seriesHashmap{} @@ -103,8 +104,8 @@ func newStripeSeries(stripeSize int) *stripeSeries { // GC garbage collects old series that have not received a sample after mint // and will fully delete them. -func (s *stripeSeries) GC(mint int64) map[uint64]struct{} { - deleted := map[uint64]struct{}{} +func (s *stripeSeries) GC(mint int64) map[chunks.HeadSeriesRef]struct{} { + deleted := map[chunks.HeadSeriesRef]struct{}{} for hashLock := 0; hashLock < s.size; hashLock++ { s.locks[hashLock].Lock() @@ -143,9 +144,8 @@ func (s *stripeSeries) GC(mint int64) map[uint64]struct{} { return deleted } -func (s *stripeSeries) GetByID(id uint64) *memSeries { - refLock := id & uint64(s.size-1) - +func (s *stripeSeries) GetByID(id chunks.HeadSeriesRef) *memSeries { + refLock := uint64(id) & uint64(s.size-1) s.locks[refLock].RLock() defer s.locks[refLock].RUnlock() return s.series[refLock][id] @@ -162,7 +162,7 @@ func (s *stripeSeries) GetByHash(hash uint64, lset labels.Labels) *memSeries { func (s *stripeSeries) Set(hash uint64, series *memSeries) { var ( hashLock = hash & uint64(s.size-1) - refLock = series.ref & uint64(s.size-1) + refLock = uint64(series.ref) & uint64(s.size-1) ) s.locks[hashLock].Lock() defer s.locks[hashLock].Unlock() diff --git a/tsdb/block.go b/tsdb/block.go index 88c0a30f5..0f2cd3ea3 100644 --- a/tsdb/block.go +++ b/tsdb/block.go @@ -29,6 +29,7 @@ import ( "github.com/pkg/errors" "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/tsdb/chunks" tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" @@ -49,7 +50,7 @@ type IndexWriter interface { // Implementations may require series to be insert in strictly increasing order by // their labels. The reference numbers are used to resolve entries in postings lists // that are added later. - AddSeries(ref uint64, l labels.Labels, chunks ...chunks.Meta) error + AddSeries(ref storage.SeriesRef, l labels.Labels, chunks ...chunks.Meta) error // Close writes any finalization and closes the resources associated with // the underlying writer. @@ -82,7 +83,7 @@ type IndexReader interface { // Series populates the given labels and chunk metas for the series identified // by the reference. // Returns storage.ErrNotFound if the ref does not resolve to a known series. - Series(ref uint64, lset *labels.Labels, chks *[]chunks.Meta) error + Series(ref storage.SeriesRef, lset *labels.Labels, chks *[]chunks.Meta) error // LabelNames returns all the unique label names present in the index in sorted order. LabelNames(matchers ...*labels.Matcher) ([]string, error) @@ -90,11 +91,11 @@ type IndexReader interface { // LabelValueFor returns label value for the given label name in the series referred to by ID. // If the series couldn't be found or the series doesn't have the requested label a // storage.ErrNotFound is returned as error. - LabelValueFor(id uint64, label string) (string, error) + LabelValueFor(id storage.SeriesRef, label string) (string, error) // LabelNamesFor returns all the label names for the series referred to by IDs. // The names returned are sorted. - LabelNamesFor(ids ...uint64) ([]string, error) + LabelNamesFor(ids ...storage.SeriesRef) ([]string, error) // Close releases the underlying resources of the reader. Close() error @@ -116,7 +117,7 @@ type ChunkWriter interface { // ChunkReader provides reading access of serialized time series data. type ChunkReader interface { // Chunk returns the series data chunk with the given reference. - Chunk(ref uint64) (chunkenc.Chunk, error) + Chunk(ref chunks.ChunkRef) (chunkenc.Chunk, error) // Close releases all underlying resources of the reader. Close() error @@ -472,7 +473,7 @@ func (r blockIndexReader) SortedPostings(p index.Postings) index.Postings { return r.ir.SortedPostings(p) } -func (r blockIndexReader) Series(ref uint64, lset *labels.Labels, chks *[]chunks.Meta) error { +func (r blockIndexReader) Series(ref storage.SeriesRef, lset *labels.Labels, chks *[]chunks.Meta) error { if err := r.ir.Series(ref, lset, chks); err != nil { return errors.Wrapf(err, "block: %s", r.b.Meta().ULID) } @@ -485,13 +486,13 @@ func (r blockIndexReader) Close() error { } // LabelValueFor returns label value for the given label name in the series referred to by ID. -func (r blockIndexReader) LabelValueFor(id uint64, label string) (string, error) { +func (r blockIndexReader) LabelValueFor(id storage.SeriesRef, label string) (string, error) { return r.ir.LabelValueFor(id, label) } // LabelNamesFor returns all the label names for the series referred to by IDs. // The names returned are sorted. -func (r blockIndexReader) LabelNamesFor(ids ...uint64) ([]string, error) { +func (r blockIndexReader) LabelNamesFor(ids ...storage.SeriesRef) ([]string, error) { return r.ir.LabelNamesFor(ids...) } @@ -558,7 +559,7 @@ Outer: return p.Err() } - err = pb.tombstones.Iter(func(id uint64, ivs tombstones.Intervals) error { + err = pb.tombstones.Iter(func(id storage.SeriesRef, ivs tombstones.Intervals) error { for _, iv := range ivs { stones.AddInterval(id, iv) } @@ -590,7 +591,7 @@ Outer: func (pb *Block) CleanTombstones(dest string, c Compactor) (*ulid.ULID, bool, error) { numStones := 0 - if err := pb.tombstones.Iter(func(id uint64, ivs tombstones.Intervals) error { + if err := pb.tombstones.Iter(func(id storage.SeriesRef, ivs tombstones.Intervals) error { numStones += len(ivs) return nil }); err != nil { diff --git a/tsdb/block_test.go b/tsdb/block_test.go index 88ca3f1eb..2bfeb841c 100644 --- a/tsdb/block_test.go +++ b/tsdb/block_test.go @@ -522,7 +522,7 @@ func createHead(tb testing.TB, w *wal.WAL, series []storage.Series, chunkDir str app := head.Appender(context.Background()) for _, s := range series { - ref := uint64(0) + ref := storage.SeriesRef(0) it := s.Iterator() lset := s.Labels() for it.Next() { diff --git a/tsdb/chunks/chunks.go b/tsdb/chunks/chunks.go index 067c9ebc1..7554b96bb 100644 --- a/tsdb/chunks/chunks.go +++ b/tsdb/chunks/chunks.go @@ -54,13 +54,54 @@ const ( ChunkEncodingSize = 1 ) +// ChunkRef is a generic reference for reading chunk data. In prometheus it +// is either a HeadChunkRef or BlockChunkRef, though other implementations +// may have their own reference types. +type ChunkRef uint64 + +// HeadSeriesRef refers to in-memory series. +type HeadSeriesRef uint64 + +// HeadChunkRef packs a HeadSeriesRef and a ChunkID into a global 8 Byte ID. +// The HeadSeriesRef and ChunkID may not exceed 5 and 3 bytes respectively. +type HeadChunkRef uint64 + +func NewHeadChunkRef(hsr HeadSeriesRef, chunkID uint64) HeadChunkRef { + if hsr > (1<<40)-1 { + panic("series ID exceeds 5 bytes") + } + if chunkID > (1<<24)-1 { + panic("chunk ID exceeds 3 bytes") + } + return HeadChunkRef(uint64(hsr<<24) | chunkID) +} + +func (p HeadChunkRef) Unpack() (HeadSeriesRef, uint64) { + return HeadSeriesRef(p >> 24), uint64(p<<40) >> 40 +} + +// BlockChunkRef refers to a chunk within a persisted block. +// The upper 4 bytes are for the segment index and +// the lower 4 bytes are for the segment offset where the data starts for this chunk. +type BlockChunkRef uint64 + +// NewBlockChunkRef packs the file index and byte offset into a BlockChunkRef. +func NewBlockChunkRef(fileIndex, fileOffset uint64) BlockChunkRef { + return BlockChunkRef(fileIndex<<32 | fileOffset) +} + +func (b BlockChunkRef) Unpack() (int, int) { + sgmIndex := int(b >> 32) + chkStart := int((b << 32) >> 32) + return sgmIndex, chkStart +} + // Meta holds information about a chunk of data. type Meta struct { // Ref and Chunk hold either a reference that can be used to retrieve // chunk data or the data itself. - // When it is a reference it is the segment offset at which the chunk bytes start. - // Generally, only one of them is set. - Ref uint64 + // If Chunk is nil, call ChunkReader.Chunk(Meta.Ref) to get the chunk and assign it to the Chunk field + Ref ChunkRef Chunk chunkenc.Chunk // Time range the data covers. @@ -353,16 +394,11 @@ func (w *Writer) writeChunks(chks []Meta) error { return nil } - seq := uint64(w.seq()) << 32 + seq := uint64(w.seq()) for i := range chks { chk := &chks[i] - // The reference is set to the segment index and the offset where - // the data starts for this chunk. - // - // The upper 4 bytes are for the segment index and - // the lower 4 bytes are for the segment offset where to start reading this chunk. - chk.Ref = seq | uint64(w.n) + chk.Ref = ChunkRef(NewBlockChunkRef(seq, uint64(w.n))) n := binary.PutUvarint(w.buf[:], uint64(len(chk.Chunk.Bytes()))) @@ -495,16 +531,9 @@ func (s *Reader) Size() int64 { } // Chunk returns a chunk from a given reference. -func (s *Reader) Chunk(ref uint64) (chunkenc.Chunk, error) { - var ( - // Get the upper 4 bytes. - // These contain the segment index. - sgmIndex = int(ref >> 32) - // Get the lower 4 bytes. - // These contain the segment offset where the data for this chunk starts. - chkStart = int((ref << 32) >> 32) - chkCRC32 = newCRC32() - ) +func (s *Reader) Chunk(ref ChunkRef) (chunkenc.Chunk, error) { + sgmIndex, chkStart := BlockChunkRef(ref).Unpack() + chkCRC32 := newCRC32() if sgmIndex >= len(s.bs) { return nil, errors.Errorf("segment index %d out of range", sgmIndex) diff --git a/tsdb/chunks/head_chunks.go b/tsdb/chunks/head_chunks.go index cc6c27be3..31fddec7c 100644 --- a/tsdb/chunks/head_chunks.go +++ b/tsdb/chunks/head_chunks.go @@ -285,7 +285,7 @@ func repairLastChunkFile(files map[int]string) (_ map[int]string, returnErr erro // WriteChunk writes the chunk to the disk. // The returned chunk ref is the reference from where the chunk encoding starts for the chunk. -func (cdm *ChunkDiskMapper) WriteChunk(seriesRef uint64, mint, maxt int64, chk chunkenc.Chunk) (chkRef ChunkDiskMapperRef, err error) { +func (cdm *ChunkDiskMapper) WriteChunk(seriesRef HeadSeriesRef, mint, maxt int64, chk chunkenc.Chunk) (chkRef ChunkDiskMapperRef, err error) { cdm.writePathMtx.Lock() defer cdm.writePathMtx.Unlock() @@ -312,7 +312,7 @@ func (cdm *ChunkDiskMapper) WriteChunk(seriesRef uint64, mint, maxt int64, chk c chkRef = newChunkDiskMapperRef(uint64(cdm.curFileSequence), uint64(cdm.curFileSize())) - binary.BigEndian.PutUint64(cdm.byteBuf[bytesWritten:], seriesRef) + binary.BigEndian.PutUint64(cdm.byteBuf[bytesWritten:], uint64(seriesRef)) bytesWritten += SeriesRefSize binary.BigEndian.PutUint64(cdm.byteBuf[bytesWritten:], uint64(mint)) bytesWritten += MintMaxtSize @@ -579,7 +579,7 @@ func (cdm *ChunkDiskMapper) Chunk(ref ChunkDiskMapperRef) (chunkenc.Chunk, error // and runs the provided function on each chunk. It returns on the first error encountered. // NOTE: This method needs to be called at least once after creating ChunkDiskMapper // to set the maxt of all the file. -func (cdm *ChunkDiskMapper) IterateAllChunks(f func(seriesRef uint64, chunkRef ChunkDiskMapperRef, mint, maxt int64, numSamples uint16) error) (err error) { +func (cdm *ChunkDiskMapper) IterateAllChunks(f func(seriesRef HeadSeriesRef, chunkRef ChunkDiskMapperRef, mint, maxt int64, numSamples uint16) error) (err error) { cdm.writePathMtx.Lock() defer cdm.writePathMtx.Unlock() @@ -627,7 +627,7 @@ func (cdm *ChunkDiskMapper) IterateAllChunks(f func(seriesRef uint64, chunkRef C chunkRef := newChunkDiskMapperRef(uint64(segID), uint64(idx)) startIdx := idx - seriesRef := binary.BigEndian.Uint64(mmapFile.byteSlice.Range(idx, idx+SeriesRefSize)) + seriesRef := HeadSeriesRef(binary.BigEndian.Uint64(mmapFile.byteSlice.Range(idx, idx+SeriesRefSize))) idx += SeriesRefSize mint := int64(binary.BigEndian.Uint64(mmapFile.byteSlice.Range(idx, idx+MintMaxtSize))) idx += MintMaxtSize diff --git a/tsdb/chunks/head_chunks_test.go b/tsdb/chunks/head_chunks_test.go index 873a00f67..58c0f8ada 100644 --- a/tsdb/chunks/head_chunks_test.go +++ b/tsdb/chunks/head_chunks_test.go @@ -38,7 +38,7 @@ func TestChunkDiskMapper_WriteChunk_Chunk_IterateChunks(t *testing.T) { chkCRC32 := newCRC32() type expectedDataType struct { - seriesRef uint64 + seriesRef HeadSeriesRef chunkRef ChunkDiskMapperRef mint, maxt int64 numSamples uint16 @@ -75,7 +75,7 @@ func TestChunkDiskMapper_WriteChunk_Chunk_IterateChunks(t *testing.T) { bytesWritten := 0 chkCRC32.Reset() - binary.BigEndian.PutUint64(buf[bytesWritten:], seriesRef) + binary.BigEndian.PutUint64(buf[bytesWritten:], uint64(seriesRef)) bytesWritten += SeriesRefSize binary.BigEndian.PutUint64(buf[bytesWritten:], uint64(mint)) bytesWritten += MintMaxtSize @@ -133,7 +133,7 @@ func TestChunkDiskMapper_WriteChunk_Chunk_IterateChunks(t *testing.T) { require.NoError(t, err) idx := 0 - require.NoError(t, hrw.IterateAllChunks(func(seriesRef uint64, chunkRef ChunkDiskMapperRef, mint, maxt int64, numSamples uint16) error { + require.NoError(t, hrw.IterateAllChunks(func(seriesRef HeadSeriesRef, chunkRef ChunkDiskMapperRef, mint, maxt int64, numSamples uint16) error { t.Helper() expData := expectedData[idx] @@ -221,7 +221,7 @@ func TestChunkDiskMapper_Truncate(t *testing.T) { require.NoError(t, err) require.False(t, hrw.fileMaxtSet) - require.NoError(t, hrw.IterateAllChunks(func(_ uint64, _ ChunkDiskMapperRef, _, _ int64, _ uint16) error { return nil })) + require.NoError(t, hrw.IterateAllChunks(func(_ HeadSeriesRef, _ ChunkDiskMapperRef, _, _ int64, _ uint16) error { return nil })) require.True(t, hrw.fileMaxtSet) verifyFiles([]int{3, 4, 5, 6, 7, 8}) @@ -335,7 +335,7 @@ func TestHeadReadWriter_TruncateAfterFailedIterateChunks(t *testing.T) { require.NoError(t, err) // Forcefully failing IterateAllChunks. - require.Error(t, hrw.IterateAllChunks(func(_ uint64, _ ChunkDiskMapperRef, _, _ int64, _ uint16) error { + require.Error(t, hrw.IterateAllChunks(func(_ HeadSeriesRef, _ ChunkDiskMapperRef, _, _ int64, _ uint16) error { return errors.New("random error") })) @@ -391,7 +391,7 @@ func TestHeadReadWriter_ReadRepairOnEmptyLastFile(t *testing.T) { hrw, err = NewChunkDiskMapper(dir, chunkenc.NewPool(), DefaultWriteBufferSize) require.NoError(t, err) require.False(t, hrw.fileMaxtSet) - require.NoError(t, hrw.IterateAllChunks(func(_ uint64, _ ChunkDiskMapperRef, _, _ int64, _ uint16) error { return nil })) + require.NoError(t, hrw.IterateAllChunks(func(_ HeadSeriesRef, _ ChunkDiskMapperRef, _, _ int64, _ uint16) error { return nil })) require.True(t, hrw.fileMaxtSet) // Removed from memory. @@ -421,7 +421,7 @@ func testChunkDiskMapper(t *testing.T) *ChunkDiskMapper { hrw, err := NewChunkDiskMapper(tmpdir, chunkenc.NewPool(), DefaultWriteBufferSize) require.NoError(t, err) require.False(t, hrw.fileMaxtSet) - require.NoError(t, hrw.IterateAllChunks(func(_ uint64, _ ChunkDiskMapperRef, _, _ int64, _ uint16) error { return nil })) + require.NoError(t, hrw.IterateAllChunks(func(_ HeadSeriesRef, _ ChunkDiskMapperRef, _, _ int64, _ uint16) error { return nil })) require.True(t, hrw.fileMaxtSet) return hrw } @@ -437,9 +437,9 @@ func randomChunk(t *testing.T) chunkenc.Chunk { return chunk } -func createChunk(t *testing.T, idx int, hrw *ChunkDiskMapper) (seriesRef uint64, chunkRef ChunkDiskMapperRef, mint, maxt int64, chunk chunkenc.Chunk) { +func createChunk(t *testing.T, idx int, hrw *ChunkDiskMapper) (seriesRef HeadSeriesRef, chunkRef ChunkDiskMapperRef, mint, maxt int64, chunk chunkenc.Chunk) { var err error - seriesRef = uint64(rand.Int63()) + seriesRef = HeadSeriesRef(rand.Int63()) mint = int64((idx)*1000 + 1) maxt = int64((idx + 1) * 1000) chunk = randomChunk(t) diff --git a/tsdb/compact.go b/tsdb/compact.go index 3c333f858..7c12d1729 100644 --- a/tsdb/compact.go +++ b/tsdb/compact.go @@ -745,7 +745,7 @@ func (c *LeveledCompactor) populateBlock(blocks []BlockReader, meta *BlockMeta, } var ( - ref = uint64(0) + ref = storage.SeriesRef(0) chks []chunks.Meta ) diff --git a/tsdb/db.go b/tsdb/db.go index 7326b904b..c3b4af093 100644 --- a/tsdb/db.go +++ b/tsdb/db.go @@ -858,7 +858,7 @@ type dbAppender struct { var _ storage.GetRef = dbAppender{} -func (a dbAppender) GetRef(lset labels.Labels) (uint64, labels.Labels) { +func (a dbAppender) GetRef(lset labels.Labels) (storage.SeriesRef, labels.Labels) { if g, ok := a.Appender.(storage.GetRef); ok { return g.GetRef(lset) } diff --git a/tsdb/db_test.go b/tsdb/db_test.go index 3f657152f..4faf70b32 100644 --- a/tsdb/db_test.go +++ b/tsdb/db_test.go @@ -1388,7 +1388,7 @@ func TestSizeRetention(t *testing.T) { // Create a WAL checkpoint, and compare sizes. first, last, err := wal.Segments(db.Head().wal.Dir()) require.NoError(t, err) - _, err = wal.Checkpoint(log.NewNopLogger(), db.Head().wal, first, last-1, func(x uint64) bool { return false }, 0) + _, err = wal.Checkpoint(log.NewNopLogger(), db.Head().wal, first, last-1, func(x chunks.HeadSeriesRef) bool { return false }, 0) require.NoError(t, err) blockSize = int64(prom_testutil.ToFloat64(db.metrics.blocksBytes)) // Use the actual internal metrics. walSize, err = db.Head().wal.Size() diff --git a/tsdb/docs/refs.md b/tsdb/docs/refs.md new file mode 100644 index 000000000..4f745b572 --- /dev/null +++ b/tsdb/docs/refs.md @@ -0,0 +1,87 @@ +# An overview of different Series and Chunk reference types + +## Used internally in TSDB + +* `ChunkDiskMapperRef`: to load mmapped chunks from disk. + +## Used by callers of TSDB + +| Location | Series access | Chunk access | +|--------------------|--------------------------------|--------------------------------------------------------------------| +| Global interfaces | `SeriesRef` (in postings list) | `chunks.ChunkRef` (`ChunkReader` interface, `Meta.Ref`) | +| Head | `HeadSeriesRef` (autoinc) | `HeadChunkRef` (could be head chunk or mmapped chunk. 5/3B split) | +| blocks | `BlockSeriesRef` (16B aligned) | `BlockChunkRef` (4/4B split) | + +### `SeriesRef` + +Note: we cover the implementations as used in Prometheus. Other projects may use different implementations. + +#### `HeadSeriesRef` + +`HeadSeriesRef` is simply a 64bit counter that increments when a new series comes in. +Due to series churn, the set of actively used `HeadSeriesRef`s may be well above zero (e.g. 0-10M may not be used, and 10M-11M is active) + +Usage: +* [`stripeSeries`](https://github.com/prometheus/prometheus/blob/fdbc40a9efcc8197a94f23f0e479b0b56e52d424/tsdb/head.go#L1292-L1298) (note: when you don't know a `HeadSeriesRef` for a series, you can also access it by a hash of the series' labels) +* WAL +* `HeadChunkRef`s include them for addressing head chunks, as those are owned by the `memSeries`. + +Notes: +1) M-mapped Head chunks, while they use `HeadSeriesRef`s, don't contain an index and depend on the series listing in memory. +Once mmapped, chunks have `HeadSeriesRef`s inside them, allowing you to recreate the index from reading chunks +(Along with WAL which has the labels for those `HeadSeriesRef`s. It also has all those samples, but by using m-mapped chunks we can save cpu/time and not replay all of WAL on startup) + +2) During querying, `HeadSeriesRef` are limited to 2^40 (see `HeadChunkRef`) + +3) The last `HeadSeriesRef` is always replayed from the WAL and is continued from there. + +#### `BlockSeriesRef` + +Persistent blocks are independent entities and the format/structure is completely different from head block. + +In blocks, series are lexicographically ordered by labels and the byte offset in the index file (divided by 16 because they're all aligned on 16 bytes) becomes the `BlockSeriesRef`. + +They are not sequential because index entries may be multiples of 16 bytes. And they don't start from 0 because the byte offset is absolute and includes the magic number, symbols table, etc. + +`BlockSeriesRef` are only 32 bits for now, because 64 bits would slow down the postings lists disk access. (note: this limits the index size to 2^32 * 16 = 64 GB) + + +See also: +* https://ganeshvernekar.com/blog/prometheus-tsdb-persistent-block-and-its-index/#3-index +* https://ganeshvernekar.com/blog/prometheus-tsdb-persistent-block-and-its-index/#c-series + +### `ChunkRef` + +Chunk references are used to load chunk data during query execution. +Note: we cover the implementations as used in Prometheus. Other projects may use different implementations. + +#### `HeadChunkRef` + +A `HeadChunkRef` is an 8 byte integer that packs together: + +* 5 Bytes for `HeadSeriesRef`. +* 3 Bytes for `ChunkID` (uint64). This is simply an index into a slice of mmappedChunks for a given series + +There are two implications here: + +* While `HeadSeriesRef`s can during ingestion go higher, during querying they are limited to 2^40. Querying too high numbers will lead to query failures (but not impact ingestion). +* `ChunkID` keeps growing as we enter new chunks until Prometheus restarts. If Prometheus runs too long, we might hit 2^24. + ([957 years](https://www.wolframalpha.com/input/?i=2%5E24+*+120+*+15+seconds+in+years) at 1 sample per 15 seconds). If `ChunkID=len(mmappedchunks)` then it's the head chunk. + +#### `BlockChunkRef` + +A `BlockChunkRef` is an 8 byte integer. Unlike `HeadChunkRef`, it is static and independent of factors such as Prometheus restarting. + +It packs together: + +* 4 Bytes for chunk file index in the block. This number just increments. Filenames [start at 1](https://ganeshvernekar.com/blog/prometheus-tsdb-persistent-block-and-its-index/#contents-of-a-block) +but the `BlockChunkRef` start at 0. +* 4 Bytes for the byte offset within the file. + +#### Why does `HeadChunkRef` contain a series reference and `BlockChunkRef` does not? + +The `ChunkRef` types allow retrieving the chunk data as efficiently as possible. +* In the Head block the chunks are in the series struct. So we need to reach the series before we can access the chunk from it. + Hence we need to pack the `HeadSeriesRef` to get to the series. +* In persistent blocks, the chunk files are separated from the index and static. Hence you only need the co-ordinates within the `chunks` directory + to get to the chunk. Hence no need of `BlockSeriesRef`. diff --git a/tsdb/head.go b/tsdb/head.go index 8bfbb1174..02b6b7b4e 100644 --- a/tsdb/head.go +++ b/tsdb/head.go @@ -80,7 +80,7 @@ type Head struct { series *stripeSeries deletedMtx sync.Mutex - deleted map[uint64]int // Deleted series, and what WAL segment they must be kept until. + deleted map[chunks.HeadSeriesRef]int // Deleted series, and what WAL segment they must be kept until. postings *index.MemPostings // Postings lists for terms. @@ -236,7 +236,7 @@ func (h *Head) resetInMemoryState() error { h.postings = index.NewUnorderedMemPostings() h.tombstones = tombstones.NewMemTombstones() h.iso = newIsolation() - h.deleted = map[uint64]int{} + h.deleted = map[chunks.HeadSeriesRef]int{} h.chunkRange.Store(h.opts.ChunkRange) h.minTime.Store(math.MaxInt64) h.maxTime.Store(math.MinInt64) @@ -480,7 +480,7 @@ func (h *Head) Init(minValidTime int64) error { start := time.Now() snapIdx, snapOffset := -1, 0 - refSeries := make(map[uint64]*memSeries) + refSeries := make(map[chunks.HeadSeriesRef]*memSeries) if h.opts.EnableMemorySnapshotOnShutdown { level.Info(h.logger).Log("msg", "Chunk snapshot is enabled, replaying from the snapshot") @@ -533,7 +533,7 @@ func (h *Head) Init(minValidTime int64) error { h.startWALReplayStatus(startFrom, endAt) - multiRef := map[uint64]uint64{} + multiRef := map[chunks.HeadSeriesRef]chunks.HeadSeriesRef{} if err == nil && startFrom >= snapIdx { sr, err := wal.NewSegmentsReader(dir) if err != nil { @@ -603,9 +603,9 @@ func (h *Head) Init(minValidTime int64) error { return nil } -func (h *Head) loadMmappedChunks(refSeries map[uint64]*memSeries) (map[uint64][]*mmappedChunk, error) { - mmappedChunks := map[uint64][]*mmappedChunk{} - if err := h.chunkDiskMapper.IterateAllChunks(func(seriesRef uint64, chunkRef chunks.ChunkDiskMapperRef, mint, maxt int64, numSamples uint16) error { +func (h *Head) loadMmappedChunks(refSeries map[chunks.HeadSeriesRef]*memSeries) (map[chunks.HeadSeriesRef][]*mmappedChunk, error) { + mmappedChunks := map[chunks.HeadSeriesRef][]*mmappedChunk{} + if err := h.chunkDiskMapper.IterateAllChunks(func(seriesRef chunks.HeadSeriesRef, chunkRef chunks.ChunkDiskMapperRef, mint, maxt int64, numSamples uint16) error { if maxt < h.minValidTime.Load() { return nil } @@ -655,19 +655,19 @@ func (h *Head) loadMmappedChunks(refSeries map[uint64]*memSeries) (map[uint64][] // removeCorruptedMmappedChunks attempts to delete the corrupted mmapped chunks and if it fails, it clears all the previously // loaded mmapped chunks. -func (h *Head) removeCorruptedMmappedChunks(err error, refSeries map[uint64]*memSeries) map[uint64][]*mmappedChunk { +func (h *Head) removeCorruptedMmappedChunks(err error, refSeries map[chunks.HeadSeriesRef]*memSeries) map[chunks.HeadSeriesRef][]*mmappedChunk { level.Info(h.logger).Log("msg", "Deleting mmapped chunk files") if err := h.chunkDiskMapper.DeleteCorrupted(err); err != nil { level.Info(h.logger).Log("msg", "Deletion of mmap chunk files failed, discarding chunk files completely", "err", err) - return map[uint64][]*mmappedChunk{} + return map[chunks.HeadSeriesRef][]*mmappedChunk{} } level.Info(h.logger).Log("msg", "Deletion of mmap chunk files successful, reattempting m-mapping the on-disk chunks") mmappedChunks, err := h.loadMmappedChunks(refSeries) if err != nil { level.Error(h.logger).Log("msg", "Loading on-disk chunks failed, discarding chunk files completely", "err", err) - mmappedChunks = map[uint64][]*mmappedChunk{} + mmappedChunks = map[chunks.HeadSeriesRef][]*mmappedChunk{} } return mmappedChunks @@ -925,7 +925,7 @@ func (h *Head) truncateWAL(mint int64) error { return nil } - keep := func(id uint64) bool { + keep := func(id chunks.HeadSeriesRef) bool { if h.series.getByID(id) != nil { return true } @@ -1072,7 +1072,7 @@ func (h *Head) Delete(mint, maxt int64, ms ...*labels.Matcher) error { var stones []tombstones.Stone for p.Next() { - series := h.series.getByID(p.At()) + series := h.series.getByID(chunks.HeadSeriesRef(p.At())) series.RLock() t0, t1 := series.minTime(), series.maxTime() @@ -1094,7 +1094,7 @@ func (h *Head) Delete(mint, maxt int64, ms ...*labels.Matcher) error { } } for _, s := range stones { - h.tombstones.AddInterval(s.Ref, s.Intervals[0]) + h.tombstones.AddInterval(storage.SeriesRef(s.Ref), s.Intervals[0]) } return nil @@ -1133,7 +1133,7 @@ func (h *Head) gc() int64 { // that reads the WAL, wouldn't be able to use those // samples since we would have no labels for that ref ID. for ref := range deleted { - h.deleted[ref] = last + h.deleted[chunks.HeadSeriesRef(ref)] = last } h.deletedMtx.Unlock() } @@ -1216,12 +1216,12 @@ func (h *Head) getOrCreate(hash uint64, lset labels.Labels) (*memSeries, bool, e } // Optimistically assume that we are the first one to create the series. - id := h.lastSeriesID.Inc() + id := chunks.HeadSeriesRef(h.lastSeriesID.Inc()) return h.getOrCreateWithID(id, hash, lset) } -func (h *Head) getOrCreateWithID(id, hash uint64, lset labels.Labels) (*memSeries, bool, error) { +func (h *Head) getOrCreateWithID(id chunks.HeadSeriesRef, hash uint64, lset labels.Labels) (*memSeries, bool, error) { s, created, err := h.series.getOrSet(hash, lset, func() *memSeries { return newMemSeries(lset, id, h.chunkRange.Load(), &h.memChunkPool) }) @@ -1235,7 +1235,7 @@ func (h *Head) getOrCreateWithID(id, hash uint64, lset labels.Labels) (*memSerie h.metrics.seriesCreated.Inc() h.numSeries.Inc() - h.postings.Add(id, lset) + h.postings.Add(storage.SeriesRef(id), lset) return s, true, nil } @@ -1290,7 +1290,7 @@ const ( // dereferences. type stripeSeries struct { size int - series []map[uint64]*memSeries + series []map[chunks.HeadSeriesRef]*memSeries hashes []seriesHashmap locks []stripeLock seriesLifecycleCallback SeriesLifecycleCallback @@ -1305,14 +1305,14 @@ type stripeLock struct { func newStripeSeries(stripeSize int, seriesCallback SeriesLifecycleCallback) *stripeSeries { s := &stripeSeries{ size: stripeSize, - series: make([]map[uint64]*memSeries, stripeSize), + series: make([]map[chunks.HeadSeriesRef]*memSeries, stripeSize), hashes: make([]seriesHashmap, stripeSize), locks: make([]stripeLock, stripeSize), seriesLifecycleCallback: seriesCallback, } for i := range s.series { - s.series[i] = map[uint64]*memSeries{} + s.series[i] = map[chunks.HeadSeriesRef]*memSeries{} } for i := range s.hashes { s.hashes[i] = seriesHashmap{} @@ -1322,9 +1322,12 @@ func newStripeSeries(stripeSize int, seriesCallback SeriesLifecycleCallback) *st // gc garbage collects old chunks that are strictly before mint and removes // series entirely that have no chunks left. -func (s *stripeSeries) gc(mint int64) (map[uint64]struct{}, int, int64) { +// note: returning map[chunks.HeadSeriesRef]struct{} would be more accurate, +// but the returned map goes into postings.Delete() which expects a map[storage.SeriesRef]struct +// and there's no easy way to cast maps. +func (s *stripeSeries) gc(mint int64) (map[storage.SeriesRef]struct{}, int, int64) { var ( - deleted = map[uint64]struct{}{} + deleted = map[storage.SeriesRef]struct{}{} deletedForCallback = []labels.Labels{} rmChunks = 0 actualMint int64 = math.MaxInt64 @@ -1359,7 +1362,7 @@ func (s *stripeSeries) gc(mint int64) (map[uint64]struct{}, int, int64) { s.locks[j].Lock() } - deleted[series.ref] = struct{}{} + deleted[storage.SeriesRef(series.ref)] = struct{}{} s.hashes[i].del(hash, series.lset) delete(s.series[j], series.ref) deletedForCallback = append(deletedForCallback, series.lset) @@ -1385,8 +1388,8 @@ func (s *stripeSeries) gc(mint int64) (map[uint64]struct{}, int, int64) { return deleted, rmChunks, actualMint } -func (s *stripeSeries) getByID(id uint64) *memSeries { - i := id & uint64(s.size-1) +func (s *stripeSeries) getByID(id chunks.HeadSeriesRef) *memSeries { + i := uint64(id) & uint64(s.size-1) s.locks[i].RLock() series := s.series[i][id] @@ -1438,7 +1441,7 @@ func (s *stripeSeries) getOrSet(hash uint64, lset labels.Labels, createSeries fu // as any further calls to this methods would return that series. s.seriesLifecycleCallback.PostCreation(series.lset) - i = series.ref & uint64(s.size-1) + i = uint64(series.ref) & uint64(s.size-1) s.locks[i].Lock() s.series[i][series.ref] = series @@ -1461,7 +1464,7 @@ func (s sample) V() float64 { return s.v } type memSeries struct { sync.RWMutex - ref uint64 + ref chunks.HeadSeriesRef lset labels.Labels mmappedChunks []*mmappedChunk mmMaxTime int64 // Max time of any mmapped chunk, only used during WAL replay. @@ -1480,7 +1483,7 @@ type memSeries struct { txs *txRing } -func newMemSeries(lset labels.Labels, id uint64, chunkRange int64, memChunkPool *sync.Pool) *memSeries { +func newMemSeries(lset labels.Labels, id chunks.HeadSeriesRef, chunkRange int64, memChunkPool *sync.Pool) *memSeries { s := &memSeries{ lset: lset, ref: id, @@ -1562,7 +1565,7 @@ func overlapsClosedInterval(mint1, maxt1, mint2, maxt2 int64) bool { return mint1 <= maxt2 && mint2 <= maxt1 } -// mappedChunks describes chunk data on disk that can be mmapped +// mappedChunks describes a head chunk on disk that has been mmapped type mmappedChunk struct { ref chunks.ChunkDiskMapperRef numSamples uint16 diff --git a/tsdb/head_append.go b/tsdb/head_append.go index e586999f9..a0b38e728 100644 --- a/tsdb/head_append.go +++ b/tsdb/head_append.go @@ -38,7 +38,7 @@ type initAppender struct { var _ storage.GetRef = &initAppender{} -func (a *initAppender) Append(ref uint64, lset labels.Labels, t int64, v float64) (uint64, error) { +func (a *initAppender) Append(ref storage.SeriesRef, lset labels.Labels, t int64, v float64) (storage.SeriesRef, error) { if a.app != nil { return a.app.Append(ref, lset, t, v) } @@ -48,7 +48,7 @@ func (a *initAppender) Append(ref uint64, lset labels.Labels, t int64, v float64 return a.app.Append(ref, lset, t, v) } -func (a *initAppender) AppendExemplar(ref uint64, l labels.Labels, e exemplar.Exemplar) (uint64, error) { +func (a *initAppender) AppendExemplar(ref storage.SeriesRef, l labels.Labels, e exemplar.Exemplar) (storage.SeriesRef, error) { // Check if exemplar storage is enabled. if !a.head.opts.EnableExemplarStorage || a.head.opts.MaxExemplars.Load() <= 0 { return 0, nil @@ -76,7 +76,7 @@ func (h *Head) initTime(t int64) { h.maxTime.CAS(math.MinInt64, t) } -func (a *initAppender) GetRef(lset labels.Labels) (uint64, labels.Labels) { +func (a *initAppender) GetRef(lset labels.Labels) (storage.SeriesRef, labels.Labels) { if g, ok := a.app.(storage.GetRef); ok { return g.GetRef(lset) } @@ -215,7 +215,7 @@ func (h *Head) putBytesBuffer(b []byte) { } type exemplarWithSeriesRef struct { - ref uint64 + ref storage.SeriesRef exemplar exemplar.Exemplar } @@ -233,13 +233,13 @@ type headAppender struct { closed bool } -func (a *headAppender) Append(ref uint64, lset labels.Labels, t int64, v float64) (uint64, error) { +func (a *headAppender) Append(ref storage.SeriesRef, lset labels.Labels, t int64, v float64) (storage.SeriesRef, error) { if t < a.minValidTime { a.head.metrics.outOfBoundSamples.Inc() return 0, storage.ErrOutOfBounds } - s := a.head.series.getByID(ref) + s := a.head.series.getByID(chunks.HeadSeriesRef(ref)) if s == nil { // Ensure no empty labels have gotten through. lset = lset.WithoutEmpty() @@ -289,7 +289,7 @@ func (a *headAppender) Append(ref uint64, lset labels.Labels, t int64, v float64 V: v, }) a.sampleSeries = append(a.sampleSeries, s) - return s.ref, nil + return storage.SeriesRef(s.ref), nil } // appendable checks whether the given sample is valid for appending to the series. @@ -315,22 +315,22 @@ func (s *memSeries) appendable(t int64, v float64) error { // AppendExemplar for headAppender assumes the series ref already exists, and so it doesn't // use getOrCreate or make any of the lset sanity checks that Append does. -func (a *headAppender) AppendExemplar(ref uint64, lset labels.Labels, e exemplar.Exemplar) (uint64, error) { +func (a *headAppender) AppendExemplar(ref storage.SeriesRef, lset labels.Labels, e exemplar.Exemplar) (storage.SeriesRef, error) { // Check if exemplar storage is enabled. if !a.head.opts.EnableExemplarStorage || a.head.opts.MaxExemplars.Load() <= 0 { return 0, nil } // Get Series - s := a.head.series.getByID(ref) + s := a.head.series.getByID(chunks.HeadSeriesRef(ref)) if s == nil { s = a.head.series.getByHash(lset.Hash(), lset) if s != nil { - ref = s.ref + ref = storage.SeriesRef(s.ref) } } if s == nil { - return 0, fmt.Errorf("unknown series ref. when trying to add exemplar: %d", ref) + return 0, fmt.Errorf("unknown HeadSeriesRef when trying to add exemplar: %d", ref) } // Ensure no empty labels have gotten through. @@ -347,18 +347,18 @@ func (a *headAppender) AppendExemplar(ref uint64, lset labels.Labels, e exemplar a.exemplars = append(a.exemplars, exemplarWithSeriesRef{ref, e}) - return s.ref, nil + return storage.SeriesRef(s.ref), nil } var _ storage.GetRef = &headAppender{} -func (a *headAppender) GetRef(lset labels.Labels) (uint64, labels.Labels) { +func (a *headAppender) GetRef(lset labels.Labels) (storage.SeriesRef, labels.Labels) { s := a.head.series.getByHash(lset.Hash(), lset) if s == nil { return 0, nil } // returned labels must be suitable to pass to Append() - return s.ref, s.lset + return storage.SeriesRef(s.ref), s.lset } func (a *headAppender) log() error { @@ -403,7 +403,7 @@ func exemplarsForEncoding(es []exemplarWithSeriesRef) []record.RefExemplar { ret := make([]record.RefExemplar, 0, len(es)) for _, e := range es { ret = append(ret, record.RefExemplar{ - Ref: e.ref, + Ref: chunks.HeadSeriesRef(e.ref), T: e.exemplar.Ts, V: e.exemplar.Value, Labels: e.exemplar.Labels, @@ -425,7 +425,7 @@ func (a *headAppender) Commit() (err error) { // No errors logging to WAL, so pass the exemplars along to the in memory storage. for _, e := range a.exemplars { - s := a.head.series.getByID(e.ref) + s := a.head.series.getByID(chunks.HeadSeriesRef(e.ref)) // We don't instrument exemplar appends here, all is instrumented by storage. if err := a.head.exemplars.AddExemplar(s.lset, e.exemplar); err != nil { if err == storage.ErrOutOfOrderExemplar { diff --git a/tsdb/head_read.go b/tsdb/head_read.go index a337b6948..1f0f18627 100644 --- a/tsdb/head_read.go +++ b/tsdb/head_read.go @@ -115,7 +115,7 @@ func (h *headIndexReader) SortedPostings(p index.Postings) index.Postings { // Fetch all the series only once. for p.Next() { - s := h.head.series.getByID(p.At()) + s := h.head.series.getByID(chunks.HeadSeriesRef(p.At())) if s == nil { level.Debug(h.head.logger).Log("msg", "Looked up series not found") } else { @@ -131,16 +131,16 @@ func (h *headIndexReader) SortedPostings(p index.Postings) index.Postings { }) // Convert back to list. - ep := make([]uint64, 0, len(series)) + ep := make([]storage.SeriesRef, 0, len(series)) for _, p := range series { - ep = append(ep, p.ref) + ep = append(ep, storage.SeriesRef(p.ref)) } return index.NewListPostings(ep) } // Series returns the series for the given reference. -func (h *headIndexReader) Series(ref uint64, lbls *labels.Labels, chks *[]chunks.Meta) error { - s := h.head.series.getByID(ref) +func (h *headIndexReader) Series(ref storage.SeriesRef, lbls *labels.Labels, chks *[]chunks.Meta) error { + s := h.head.series.getByID(chunks.HeadSeriesRef(ref)) if s == nil { h.head.metrics.seriesNotFound.Inc() @@ -161,14 +161,14 @@ func (h *headIndexReader) Series(ref uint64, lbls *labels.Labels, chks *[]chunks *chks = append(*chks, chunks.Meta{ MinTime: c.minTime, MaxTime: c.maxTime, - Ref: packChunkID(s.ref, uint64(s.chunkID(i))), + Ref: chunks.ChunkRef(chunks.NewHeadChunkRef(s.ref, uint64(s.chunkID(i)))), }) } if s.headChunk != nil && s.headChunk.OverlapsClosedInterval(h.mint, h.maxt) { *chks = append(*chks, chunks.Meta{ MinTime: s.headChunk.minTime, MaxTime: math.MaxInt64, // Set the head chunks as open (being appended to). - Ref: packChunkID(s.ref, uint64(s.chunkID(len(s.mmappedChunks)))), + Ref: chunks.ChunkRef(chunks.NewHeadChunkRef(s.ref, uint64(s.chunkID(len(s.mmappedChunks))))), }) } @@ -180,8 +180,8 @@ func (s *memSeries) chunkID(pos int) int { } // LabelValueFor returns label value for the given label name in the series referred to by ID. -func (h *headIndexReader) LabelValueFor(id uint64, label string) (string, error) { - memSeries := h.head.series.getByID(id) +func (h *headIndexReader) LabelValueFor(id storage.SeriesRef, label string) (string, error) { + memSeries := h.head.series.getByID(chunks.HeadSeriesRef(id)) if memSeries == nil { return "", storage.ErrNotFound } @@ -196,10 +196,10 @@ func (h *headIndexReader) LabelValueFor(id uint64, label string) (string, error) // LabelNamesFor returns all the label names for the series referred to by IDs. // The names returned are sorted. -func (h *headIndexReader) LabelNamesFor(ids ...uint64) ([]string, error) { +func (h *headIndexReader) LabelNamesFor(ids ...storage.SeriesRef) ([]string, error) { namesMap := make(map[string]struct{}) for _, id := range ids { - memSeries := h.head.series.getByID(id) + memSeries := h.head.series.getByID(chunks.HeadSeriesRef(id)) if memSeries == nil { return nil, storage.ErrNotFound } @@ -248,25 +248,9 @@ func (h *headChunkReader) Close() error { return nil } -// packChunkID packs a seriesID and a chunkID within it into a global 8 byte ID. -// It panicks if the seriesID exceeds 5 bytes or the chunk ID 3 bytes. -func packChunkID(seriesID, chunkID uint64) uint64 { - if seriesID > (1<<40)-1 { - panic("series ID exceeds 5 bytes") - } - if chunkID > (1<<24)-1 { - panic("chunk ID exceeds 3 bytes") - } - return (seriesID << 24) | chunkID -} - -func unpackChunkID(id uint64) (seriesID, chunkID uint64) { - return id >> 24, (id << 40) >> 40 -} - // Chunk returns the chunk for the reference number. -func (h *headChunkReader) Chunk(ref uint64) (chunkenc.Chunk, error) { - sid, cid := unpackChunkID(ref) +func (h *headChunkReader) Chunk(ref chunks.ChunkRef) (chunkenc.Chunk, error) { + sid, cid := chunks.HeadChunkRef(ref).Unpack() s := h.head.series.getByID(sid) // This means that the series has been garbage collected. diff --git a/tsdb/head_test.go b/tsdb/head_test.go index 8ace65355..d696b55a8 100644 --- a/tsdb/head_test.go +++ b/tsdb/head_test.go @@ -63,7 +63,7 @@ func newTestHead(t testing.TB, chunkRange int64, compressWAL bool) (*Head, *wal. h, err := NewHead(nil, nil, wlog, opts, nil) require.NoError(t, err) - require.NoError(t, h.chunkDiskMapper.IterateAllChunks(func(_ uint64, _ chunks.ChunkDiskMapperRef, _, _ int64, _ uint16) error { return nil })) + require.NoError(t, h.chunkDiskMapper.IterateAllChunks(func(_ chunks.HeadSeriesRef, _ chunks.ChunkDiskMapperRef, _, _ int64, _ uint16) error { return nil })) t.Cleanup(func() { require.NoError(t, os.RemoveAll(dir)) @@ -200,7 +200,7 @@ func BenchmarkLoadWAL(b *testing.B) { for j := 1; len(lbls) < labelsPerSeries; j++ { lbls[defaultLabelName+strconv.Itoa(j)] = defaultLabelValue + strconv.Itoa(j) } - refSeries = append(refSeries, record.RefSeries{Ref: uint64(i) * 101, Labels: labels.FromMap(lbls)}) + refSeries = append(refSeries, record.RefSeries{Ref: chunks.HeadSeriesRef(i) * 101, Labels: labels.FromMap(lbls)}) } populateTestWAL(b, w, []interface{}{refSeries}) } @@ -212,7 +212,7 @@ func BenchmarkLoadWAL(b *testing.B) { refSamples = refSamples[:0] for k := j * c.seriesPerBatch; k < (j+1)*c.seriesPerBatch; k++ { refSamples = append(refSamples, record.RefSample{ - Ref: uint64(k) * 101, + Ref: chunks.HeadSeriesRef(k) * 101, T: int64(i) * 10, V: float64(i) * 100, }) @@ -227,7 +227,7 @@ func BenchmarkLoadWAL(b *testing.B) { require.NoError(b, err) for k := 0; k < c.batches*c.seriesPerBatch; k++ { // Create one mmapped chunk per series, with one sample at the given time. - s := newMemSeries(labels.Labels{}, uint64(k)*101, c.mmappedChunkT, nil) + s := newMemSeries(labels.Labels{}, chunks.HeadSeriesRef(k)*101, c.mmappedChunkT, nil) s.append(c.mmappedChunkT, 42, 0, chunkDiskMapper) s.mmapCurrentHeadChunk(chunkDiskMapper) } @@ -241,7 +241,7 @@ func BenchmarkLoadWAL(b *testing.B) { refExemplars = refExemplars[:0] for k := j * c.seriesPerBatch; k < (j+1)*c.seriesPerBatch; k++ { refExemplars = append(refExemplars, record.RefExemplar{ - Ref: uint64(k) * 101, + Ref: chunks.HeadSeriesRef(k) * 101, T: int64(i) * 10, V: float64(i) * 100, Labels: labels.FromStrings("traceID", fmt.Sprintf("trace-%d", i)), @@ -497,10 +497,10 @@ func TestHead_Truncate(t *testing.T) { postingsC1, _ := index.ExpandPostings(h.postings.Get("c", "1")) postingsAll, _ := index.ExpandPostings(h.postings.Get("", "")) - require.Equal(t, []uint64{s1.ref}, postingsA1) - require.Equal(t, []uint64{s2.ref}, postingsA2) - require.Equal(t, []uint64{s1.ref, s2.ref}, postingsB1) - require.Equal(t, []uint64{s1.ref, s2.ref}, postingsAll) + require.Equal(t, []storage.SeriesRef{storage.SeriesRef(s1.ref)}, postingsA1) + require.Equal(t, []storage.SeriesRef{storage.SeriesRef(s2.ref)}, postingsA2) + require.Equal(t, []storage.SeriesRef{storage.SeriesRef(s1.ref), storage.SeriesRef(s2.ref)}, postingsB1) + require.Equal(t, []storage.SeriesRef{storage.SeriesRef(s1.ref), storage.SeriesRef(s2.ref)}, postingsAll) require.Nil(t, postingsB2) require.Nil(t, postingsC1) @@ -2317,7 +2317,7 @@ func TestChunkNotFoundHeadGCRace(t *testing.T) { var ( app = db.Appender(context.Background()) - ref = uint64(0) + ref = storage.SeriesRef(0) mint, maxt = int64(0), int64(0) err error ) @@ -2382,7 +2382,7 @@ func TestDataMissingOnQueryDuringCompaction(t *testing.T) { var ( app = db.Appender(context.Background()) - ref = uint64(0) + ref = storage.SeriesRef(0) mint, maxt = int64(0), int64(0) err error ) @@ -2429,7 +2429,7 @@ func TestIsQuerierCollidingWithTruncation(t *testing.T) { var ( app = db.Appender(context.Background()) - ref = uint64(0) + ref = storage.SeriesRef(0) err error ) @@ -2476,7 +2476,7 @@ func TestWaitForPendingReadersInTimeRange(t *testing.T) { var ( app = db.Appender(context.Background()) - ref = uint64(0) + ref = storage.SeriesRef(0) err error ) @@ -2539,10 +2539,10 @@ func TestChunkSnapshot(t *testing.T) { numSeries := 10 expSeries := make(map[string][]tsdbutil.Sample) - expTombstones := make(map[uint64]tombstones.Intervals) + expTombstones := make(map[storage.SeriesRef]tombstones.Intervals) expExemplars := make([]ex, 0) - addExemplar := func(app storage.Appender, ref uint64, lbls labels.Labels, ts int64) { + addExemplar := func(app storage.Appender, ref storage.SeriesRef, lbls labels.Labels, ts int64) { e := ex{ seriesLabels: lbls, e: exemplar.Exemplar{ @@ -2565,8 +2565,8 @@ func TestChunkSnapshot(t *testing.T) { checkTombstones := func() { tr, err := head.Tombstones() require.NoError(t, err) - actTombstones := make(map[uint64]tombstones.Intervals) - require.NoError(t, tr.Iter(func(ref uint64, itvs tombstones.Intervals) error { + actTombstones := make(map[storage.SeriesRef]tombstones.Intervals) + require.NoError(t, tr.Iter(func(ref storage.SeriesRef, itvs tombstones.Intervals) error { for _, itv := range itvs { actTombstones[ref].Add(itv) } @@ -2640,7 +2640,7 @@ func TestChunkSnapshot(t *testing.T) { // Add some tombstones. var enc record.Encoder for i := 1; i <= numSeries; i++ { - ref := uint64(i) + ref := storage.SeriesRef(i) itvs := tombstones.Intervals{ {Mint: 1234, Maxt: 2345}, {Mint: 3456, Maxt: 4567}, @@ -2701,7 +2701,7 @@ func TestChunkSnapshot(t *testing.T) { // Add more tombstones. var enc record.Encoder for i := 1; i <= numSeries; i++ { - ref := uint64(i) + ref := storage.SeriesRef(i) itvs := tombstones.Intervals{ {Mint: 12345, Maxt: 23456}, {Mint: 34567, Maxt: 45678}, diff --git a/tsdb/head_wal.go b/tsdb/head_wal.go index 4fa030a49..5c63b3f19 100644 --- a/tsdb/head_wal.go +++ b/tsdb/head_wal.go @@ -33,6 +33,7 @@ import ( "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/encoding" tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" "github.com/prometheus/prometheus/tsdb/fileutil" @@ -41,7 +42,7 @@ import ( "github.com/prometheus/prometheus/tsdb/wal" ) -func (h *Head) loadWAL(r *wal.Reader, multiRef map[uint64]uint64, mmappedChunks map[uint64][]*mmappedChunk) (err error) { +func (h *Head) loadWAL(r *wal.Reader, multiRef map[chunks.HeadSeriesRef]chunks.HeadSeriesRef, mmappedChunks map[chunks.HeadSeriesRef][]*mmappedChunk) (err error) { // Track number of samples that referenced a series we don't know about // for error reporting. var unknownRefs atomic.Uint64 @@ -207,8 +208,8 @@ Outer: break Outer } - if h.lastSeriesID.Load() < walSeries.Ref { - h.lastSeriesID.Store(walSeries.Ref) + if chunks.HeadSeriesRef(h.lastSeriesID.Load()) < walSeries.Ref { + h.lastSeriesID.Store(uint64(walSeries.Ref)) } mmc := mmappedChunks[walSeries.Ref] @@ -225,7 +226,7 @@ Outer: multiRef[walSeries.Ref] = mSeries.ref - idx := mSeries.ref % uint64(n) + idx := uint64(mSeries.ref) % uint64(n) // It is possible that some old sample is being processed in processWALSamples that // could cause race below. So we wait for the goroutine to empty input the buffer and finish // processing all old samples after emptying the buffer. @@ -297,7 +298,7 @@ Outer: if r, ok := multiRef[sam.Ref]; ok { sam.Ref = r } - mod := sam.Ref % uint64(n) + mod := uint64(sam.Ref) % uint64(n) shards[mod] = append(shards[mod], sam) } for i := 0; i < n; i++ { @@ -313,11 +314,11 @@ Outer: if itv.Maxt < h.minValidTime.Load() { continue } - if m := h.series.getByID(s.Ref); m == nil { + if m := h.series.getByID(chunks.HeadSeriesRef(s.Ref)); m == nil { unknownRefs.Inc() continue } - h.tombstones.AddInterval(s.Ref, itv) + h.tombstones.AddInterval(storage.SeriesRef(s.Ref), itv) } } //nolint:staticcheck // Ignore SA6002 relax staticcheck verification. @@ -428,7 +429,7 @@ const ( ) type chunkSnapshotRecord struct { - ref uint64 + ref chunks.HeadSeriesRef lset labels.Labels chunkRange int64 mc *memChunk @@ -439,7 +440,7 @@ func (s *memSeries) encodeToSnapshotRecord(b []byte) []byte { buf := encoding.Encbuf{B: b} buf.PutByte(chunkSnapshotRecordTypeSeries) - buf.PutBE64(s.ref) + buf.PutBE64(uint64(s.ref)) buf.PutUvarint(len(s.lset)) for _, l := range s.lset { buf.PutUvarintStr(l.Name) @@ -474,7 +475,7 @@ func decodeSeriesFromChunkSnapshot(b []byte) (csr chunkSnapshotRecord, err error return csr, errors.Errorf("invalid record type %x", flag) } - csr.ref = dec.Be64() + csr.ref = chunks.HeadSeriesRef(dec.Be64()) // The label set written to the disk is already sorted. csr.lset = make(labels.Labels, dec.Uvarint()) @@ -816,7 +817,7 @@ func DeleteChunkSnapshots(dir string, maxIndex, maxOffset int) error { // loadChunkSnapshot replays the chunk snapshot and restores the Head state from it. If there was any error returned, // it is the responsibility of the caller to clear the contents of the Head. -func (h *Head) loadChunkSnapshot() (int, int, map[uint64]*memSeries, error) { +func (h *Head) loadChunkSnapshot() (int, int, map[chunks.HeadSeriesRef]*memSeries, error) { dir, snapIdx, snapOffset, err := LastChunkSnapshot(h.opts.ChunkDirRoot) if err != nil { if err == record.ErrNotFound { @@ -842,9 +843,9 @@ func (h *Head) loadChunkSnapshot() (int, int, map[uint64]*memSeries, error) { n = runtime.GOMAXPROCS(0) wg sync.WaitGroup recordChan = make(chan chunkSnapshotRecord, 5*n) - shardedRefSeries = make([]map[uint64]*memSeries, n) + shardedRefSeries = make([]map[chunks.HeadSeriesRef]*memSeries, n) errChan = make(chan error, n) - refSeries map[uint64]*memSeries + refSeries map[chunks.HeadSeriesRef]*memSeries exemplarBuf []record.RefExemplar dec record.Decoder ) @@ -860,7 +861,7 @@ func (h *Head) loadChunkSnapshot() (int, int, map[uint64]*memSeries, error) { } }() - shardedRefSeries[idx] = make(map[uint64]*memSeries) + shardedRefSeries[idx] = make(map[chunks.HeadSeriesRef]*memSeries) localRefSeries := shardedRefSeries[idx] for csr := range rc { @@ -870,8 +871,8 @@ func (h *Head) loadChunkSnapshot() (int, int, map[uint64]*memSeries, error) { return } localRefSeries[csr.ref] = series - if h.lastSeriesID.Load() < series.ref { - h.lastSeriesID.Store(series.ref) + if chunks.HeadSeriesRef(h.lastSeriesID.Load()) < series.ref { + h.lastSeriesID.Store(uint64(series.ref)) } series.chunkRange = csr.chunkRange @@ -926,7 +927,7 @@ Outer: break Outer } - if err = tr.Iter(func(ref uint64, ivs tombstones.Intervals) error { + if err = tr.Iter(func(ref storage.SeriesRef, ivs tombstones.Intervals) error { h.tombstones.AddInterval(ref, ivs...) return nil }); err != nil { @@ -940,7 +941,7 @@ Outer: close(recordChan) wg.Wait() - refSeries = make(map[uint64]*memSeries, numSeries) + refSeries = make(map[chunks.HeadSeriesRef]*memSeries, numSeries) for _, shard := range shardedRefSeries { for k, v := range shard { refSeries[k] = v @@ -1006,7 +1007,7 @@ Outer: if len(refSeries) == 0 { // We had no exemplar record, so we have to build the map here. - refSeries = make(map[uint64]*memSeries, numSeries) + refSeries = make(map[chunks.HeadSeriesRef]*memSeries, numSeries) for _, shard := range shardedRefSeries { for k, v := range shard { refSeries[k] = v diff --git a/tsdb/index/index.go b/tsdb/index/index.go index bfe630a7a..f5d69fceb 100644 --- a/tsdb/index/index.go +++ b/tsdb/index/index.go @@ -143,7 +143,7 @@ type Writer struct { // Hold last series to validate that clients insert new series in order. lastSeries labels.Labels - lastRef uint64 + lastRef storage.SeriesRef crc32 hash.Hash @@ -414,7 +414,7 @@ func (w *Writer) writeMeta() error { } // AddSeries adds the series one at a time along with its chunks. -func (w *Writer) AddSeries(ref uint64, lset labels.Labels, chunks ...chunks.Meta) error { +func (w *Writer) AddSeries(ref storage.SeriesRef, lset labels.Labels, chunks ...chunks.Meta) error { if err := w.ensureStage(idxStageSeries); err != nil { return err } @@ -472,7 +472,7 @@ func (w *Writer) AddSeries(ref uint64, lset labels.Labels, chunks ...chunks.Meta c := chunks[0] w.buf2.PutVarint64(c.MinTime) w.buf2.PutUvarint64(uint64(c.MaxTime - c.MinTime)) - w.buf2.PutUvarint64(c.Ref) + w.buf2.PutUvarint64(uint64(c.Ref)) t0 := c.MaxTime ref0 := int64(c.Ref) @@ -1518,7 +1518,7 @@ func (r *Reader) LabelValues(name string, matchers ...*labels.Matcher) ([]string // LabelNamesFor returns all the label names for the series referred to by IDs. // The names returned are sorted. -func (r *Reader) LabelNamesFor(ids ...uint64) ([]string, error) { +func (r *Reader) LabelNamesFor(ids ...storage.SeriesRef) ([]string, error) { // Gather offsetsMap the name offsetsMap in the symbol table first offsetsMap := make(map[uint32]struct{}) for _, id := range ids { @@ -1560,7 +1560,7 @@ func (r *Reader) LabelNamesFor(ids ...uint64) ([]string, error) { } // LabelValueFor returns label value for the given label name in the series referred to by ID. -func (r *Reader) LabelValueFor(id uint64, label string) (string, error) { +func (r *Reader) LabelValueFor(id storage.SeriesRef, label string) (string, error) { offset := id // In version 2 series IDs are no longer exact references but series are 16-byte padded // and the ID is the multiple of 16 of the actual position. @@ -1586,7 +1586,7 @@ func (r *Reader) LabelValueFor(id uint64, label string) (string, error) { } // Series reads the series with the given ID and writes its labels and chunks into lbls and chks. -func (r *Reader) Series(id uint64, lbls *labels.Labels, chks *[]chunks.Meta) error { +func (r *Reader) Series(id storage.SeriesRef, lbls *labels.Labels, chks *[]chunks.Meta) error { offset := id // In version 2 series IDs are no longer exact references but series are 16-byte padded // and the ID is the multiple of 16 of the actual position. @@ -1859,7 +1859,7 @@ func (dec *Decoder) Series(b []byte, lbls *labels.Labels, chks *[]chunks.Meta) e ref0 := int64(d.Uvarint64()) *chks = append(*chks, chunks.Meta{ - Ref: uint64(ref0), + Ref: chunks.ChunkRef(ref0), MinTime: t0, MaxTime: maxt, }) @@ -1877,7 +1877,7 @@ func (dec *Decoder) Series(b []byte, lbls *labels.Labels, chks *[]chunks.Meta) e } *chks = append(*chks, chunks.Meta{ - Ref: uint64(ref0), + Ref: chunks.ChunkRef(ref0), MinTime: mint, MaxTime: maxt, }) diff --git a/tsdb/index/index_test.go b/tsdb/index/index_test.go index 8fdc63069..be5636ed2 100644 --- a/tsdb/index/index_test.go +++ b/tsdb/index/index_test.go @@ -29,6 +29,7 @@ import ( "go.uber.org/goleak" "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/encoding" @@ -45,18 +46,18 @@ type series struct { } type mockIndex struct { - series map[uint64]series - postings map[labels.Label][]uint64 + series map[storage.SeriesRef]series + postings map[labels.Label][]storage.SeriesRef symbols map[string]struct{} } func newMockIndex() mockIndex { ix := mockIndex{ - series: make(map[uint64]series), - postings: make(map[labels.Label][]uint64), + series: make(map[storage.SeriesRef]series), + postings: make(map[labels.Label][]storage.SeriesRef), symbols: make(map[string]struct{}), } - ix.postings[allPostingsKey] = []uint64{} + ix.postings[allPostingsKey] = []storage.SeriesRef{} return ix } @@ -64,7 +65,7 @@ func (m mockIndex) Symbols() (map[string]struct{}, error) { return m.symbols, nil } -func (m mockIndex) AddSeries(ref uint64, l labels.Labels, chunks ...chunks.Meta) error { +func (m mockIndex) AddSeries(ref storage.SeriesRef, l labels.Labels, chunks ...chunks.Meta) error { if _, ok := m.series[ref]; ok { return errors.Errorf("series with reference %d already added", ref) } @@ -72,7 +73,7 @@ func (m mockIndex) AddSeries(ref uint64, l labels.Labels, chunks ...chunks.Meta) m.symbols[lbl.Name] = struct{}{} m.symbols[lbl.Value] = struct{}{} if _, ok := m.postings[lbl]; !ok { - m.postings[lbl] = []uint64{} + m.postings[lbl] = []storage.SeriesRef{} } m.postings[lbl] = append(m.postings[lbl], ref) } @@ -124,7 +125,7 @@ func (m mockIndex) SortedPostings(p Postings) Postings { return NewListPostings(ep) } -func (m mockIndex) Series(ref uint64, lset *labels.Labels, chks *[]chunks.Meta) error { +func (m mockIndex) Series(ref storage.SeriesRef, lset *labels.Labels, chks *[]chunks.Meta) error { s, ok := m.series[ref] if !ok { return errors.New("not found") @@ -281,7 +282,7 @@ func TestPostingsMany(t *testing.T) { } for i, s := range series { - require.NoError(t, iw.AddSeries(uint64(i), s)) + require.NoError(t, iw.AddSeries(storage.SeriesRef(i), s)) } require.NoError(t, iw.Close()) @@ -373,7 +374,7 @@ func TestPersistence_index_e2e(t *testing.T) { metas = append(metas, chunks.Meta{ MinTime: int64(j * 10000), MaxTime: int64((j + 1) * 10000), - Ref: rand.Uint64(), + Ref: chunks.ChunkRef(rand.Uint64()), Chunk: chunkenc.NewXORChunk(), }) } @@ -404,9 +405,9 @@ func TestPersistence_index_e2e(t *testing.T) { mi := newMockIndex() for i, s := range input { - err = iw.AddSeries(uint64(i), s.labels, s.chunks...) + err = iw.AddSeries(storage.SeriesRef(i), s.labels, s.chunks...) require.NoError(t, err) - require.NoError(t, mi.AddSeries(uint64(i), s.labels, s.chunks...)) + require.NoError(t, mi.AddSeries(storage.SeriesRef(i), s.labels, s.chunks...)) for _, l := range s.labels { valset, ok := values[l.Name] @@ -416,7 +417,7 @@ func TestPersistence_index_e2e(t *testing.T) { } valset[l.Value] = struct{}{} } - postings.Add(uint64(i), s.labels) + postings.Add(storage.SeriesRef(i), s.labels) } err = iw.Close() diff --git a/tsdb/index/postings.go b/tsdb/index/postings.go index e7f8cef8e..4a4d5506b 100644 --- a/tsdb/index/postings.go +++ b/tsdb/index/postings.go @@ -21,6 +21,7 @@ import ( "sync" "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/storage" ) var allPostingsKey = labels.Label{} @@ -36,7 +37,7 @@ const ensureOrderBatchSize = 1024 // ensureOrderBatchPool is a pool used to recycle batches passed to workers in MemPostings.EnsureOrder(). var ensureOrderBatchPool = sync.Pool{ New: func() interface{} { - return make([][]uint64, 0, ensureOrderBatchSize) + return make([][]storage.SeriesRef, 0, ensureOrderBatchSize) }, } @@ -46,14 +47,14 @@ var ensureOrderBatchPool = sync.Pool{ // unordered batch fills on startup. type MemPostings struct { mtx sync.RWMutex - m map[string]map[string][]uint64 + m map[string]map[string][]storage.SeriesRef ordered bool } // NewMemPostings returns a memPostings that's ready for reads and writes. func NewMemPostings() *MemPostings { return &MemPostings{ - m: make(map[string]map[string][]uint64, 512), + m: make(map[string]map[string][]storage.SeriesRef, 512), ordered: true, } } @@ -62,7 +63,7 @@ func NewMemPostings() *MemPostings { // until EnsureOrder() was called once. func NewUnorderedMemPostings() *MemPostings { return &MemPostings{ - m: make(map[string]map[string][]uint64, 512), + m: make(map[string]map[string][]storage.SeriesRef, 512), ordered: false, } } @@ -198,7 +199,7 @@ func (p *MemPostings) Stats(label string) *PostingsStats { // Get returns a postings list for the given label pair. func (p *MemPostings) Get(name, value string) Postings { - var lp []uint64 + var lp []storage.SeriesRef p.mtx.RLock() l := p.m[name] if l != nil { @@ -228,7 +229,7 @@ func (p *MemPostings) EnsureOrder() { } n := runtime.GOMAXPROCS(0) - workc := make(chan [][]uint64) + workc := make(chan [][]storage.SeriesRef) var wg sync.WaitGroup wg.Add(n) @@ -237,7 +238,7 @@ func (p *MemPostings) EnsureOrder() { go func() { for job := range workc { for _, l := range job { - sort.Sort(uint64Slice(l)) + sort.Sort(seriesRefSlice(l)) } job = job[:0] @@ -247,14 +248,14 @@ func (p *MemPostings) EnsureOrder() { }() } - nextJob := ensureOrderBatchPool.Get().([][]uint64) + nextJob := ensureOrderBatchPool.Get().([][]storage.SeriesRef) for _, e := range p.m { for _, l := range e { nextJob = append(nextJob, l) if len(nextJob) >= ensureOrderBatchSize { workc <- nextJob - nextJob = ensureOrderBatchPool.Get().([][]uint64) + nextJob = ensureOrderBatchPool.Get().([][]storage.SeriesRef) } } } @@ -271,7 +272,7 @@ func (p *MemPostings) EnsureOrder() { } // Delete removes all ids in the given map from the postings lists. -func (p *MemPostings) Delete(deleted map[uint64]struct{}) { +func (p *MemPostings) Delete(deleted map[storage.SeriesRef]struct{}) { var keys, vals []string // Collect all keys relevant for deletion once. New keys added afterwards @@ -307,7 +308,7 @@ func (p *MemPostings) Delete(deleted map[uint64]struct{}) { p.mtx.Unlock() continue } - repl := make([]uint64, 0, len(p.m[n][l])) + repl := make([]storage.SeriesRef, 0, len(p.m[n][l])) for _, id := range p.m[n][l] { if _, ok := deleted[id]; !ok { @@ -345,7 +346,7 @@ func (p *MemPostings) Iter(f func(labels.Label, Postings) error) error { } // Add a label set to the postings index. -func (p *MemPostings) Add(id uint64, lset labels.Labels) { +func (p *MemPostings) Add(id storage.SeriesRef, lset labels.Labels) { p.mtx.Lock() for _, l := range lset { @@ -356,10 +357,10 @@ func (p *MemPostings) Add(id uint64, lset labels.Labels) { p.mtx.Unlock() } -func (p *MemPostings) addFor(id uint64, l labels.Label) { +func (p *MemPostings) addFor(id storage.SeriesRef, l labels.Label) { nm, ok := p.m[l.Name] if !ok { - nm = map[string][]uint64{} + nm = map[string][]storage.SeriesRef{} p.m[l.Name] = nm } list := append(nm[l.Value], id) @@ -381,7 +382,7 @@ func (p *MemPostings) addFor(id uint64, l labels.Label) { } // ExpandPostings returns the postings expanded as a slice. -func ExpandPostings(p Postings) (res []uint64, err error) { +func ExpandPostings(p Postings) (res []storage.SeriesRef, err error) { for p.Next() { res = append(res, p.At()) } @@ -395,10 +396,10 @@ type Postings interface { // Seek advances the iterator to value v or greater and returns // true if a value was found. - Seek(v uint64) bool + Seek(v storage.SeriesRef) bool // At returns the value at the current iterator position. - At() uint64 + At() storage.SeriesRef // Err returns the last error of the iterator. Err() error @@ -409,15 +410,15 @@ type errPostings struct { err error } -func (e errPostings) Next() bool { return false } -func (e errPostings) Seek(uint64) bool { return false } -func (e errPostings) At() uint64 { return 0 } -func (e errPostings) Err() error { return e.err } +func (e errPostings) Next() bool { return false } +func (e errPostings) Seek(storage.SeriesRef) bool { return false } +func (e errPostings) At() storage.SeriesRef { return 0 } +func (e errPostings) Err() error { return e.err } var emptyPostings = errPostings{} // EmptyPostings returns a postings list that's always empty. -// NOTE: Returning EmptyPostings sentinel when index.Postings struct has no postings is recommended. +// NOTE: Returning EmptyPostings sentinel when Postings struct has no postings is recommended. // It triggers optimized flow in other functions like Intersect, Without etc. func EmptyPostings() Postings { return emptyPostings @@ -448,14 +449,14 @@ func Intersect(its ...Postings) Postings { type intersectPostings struct { arr []Postings - cur uint64 + cur storage.SeriesRef } func newIntersectPostings(its ...Postings) *intersectPostings { return &intersectPostings{arr: its} } -func (it *intersectPostings) At() uint64 { +func (it *intersectPostings) At() storage.SeriesRef { return it.cur } @@ -487,7 +488,7 @@ func (it *intersectPostings) Next() bool { return it.doNext() } -func (it *intersectPostings) Seek(id uint64) bool { +func (it *intersectPostings) Seek(id storage.SeriesRef) bool { it.cur = id return it.doNext() } @@ -538,7 +539,7 @@ func (h *postingsHeap) Pop() interface{} { type mergedPostings struct { h postingsHeap initialized bool - cur uint64 + cur storage.SeriesRef err error } @@ -598,7 +599,7 @@ func (it *mergedPostings) Next() bool { } } -func (it *mergedPostings) Seek(id uint64) bool { +func (it *mergedPostings) Seek(id storage.SeriesRef) bool { if it.h.Len() == 0 || it.err != nil { return false } @@ -628,7 +629,7 @@ func (it *mergedPostings) Seek(id uint64) bool { return true } -func (it mergedPostings) At() uint64 { +func (it mergedPostings) At() storage.SeriesRef { return it.cur } @@ -652,7 +653,7 @@ func Without(full, drop Postings) Postings { type removedPostings struct { full, remove Postings - cur uint64 + cur storage.SeriesRef initialized bool fok, rok bool @@ -665,7 +666,7 @@ func newRemovedPostings(full, remove Postings) *removedPostings { } } -func (rp *removedPostings) At() uint64 { +func (rp *removedPostings) At() storage.SeriesRef { return rp.cur } @@ -702,7 +703,7 @@ func (rp *removedPostings) Next() bool { } } -func (rp *removedPostings) Seek(id uint64) bool { +func (rp *removedPostings) Seek(id storage.SeriesRef) bool { if rp.cur >= id { return true } @@ -724,19 +725,19 @@ func (rp *removedPostings) Err() error { // ListPostings implements the Postings interface over a plain list. type ListPostings struct { - list []uint64 - cur uint64 + list []storage.SeriesRef + cur storage.SeriesRef } -func NewListPostings(list []uint64) Postings { +func NewListPostings(list []storage.SeriesRef) Postings { return newListPostings(list...) } -func newListPostings(list ...uint64) *ListPostings { +func newListPostings(list ...storage.SeriesRef) *ListPostings { return &ListPostings{list: list} } -func (it *ListPostings) At() uint64 { +func (it *ListPostings) At() storage.SeriesRef { return it.cur } @@ -750,7 +751,7 @@ func (it *ListPostings) Next() bool { return false } -func (it *ListPostings) Seek(x uint64) bool { +func (it *ListPostings) Seek(x storage.SeriesRef) bool { // If the current value satisfies, then return. if it.cur >= x { return true @@ -787,8 +788,8 @@ func newBigEndianPostings(list []byte) *bigEndianPostings { return &bigEndianPostings{list: list} } -func (it *bigEndianPostings) At() uint64 { - return uint64(it.cur) +func (it *bigEndianPostings) At() storage.SeriesRef { + return storage.SeriesRef(it.cur) } func (it *bigEndianPostings) Next() bool { @@ -800,8 +801,8 @@ func (it *bigEndianPostings) Next() bool { return false } -func (it *bigEndianPostings) Seek(x uint64) bool { - if uint64(it.cur) >= x { +func (it *bigEndianPostings) Seek(x storage.SeriesRef) bool { + if storage.SeriesRef(it.cur) >= x { return true } @@ -824,9 +825,9 @@ func (it *bigEndianPostings) Err() error { return nil } -// uint64Slice attaches the methods of sort.Interface to []uint64, sorting in increasing order. -type uint64Slice []uint64 +// seriesRefSlice attaches the methods of sort.Interface to []storage.SeriesRef, sorting in increasing order. +type seriesRefSlice []storage.SeriesRef -func (x uint64Slice) Len() int { return len(x) } -func (x uint64Slice) Less(i, j int) bool { return x[i] < x[j] } -func (x uint64Slice) Swap(i, j int) { x[i], x[j] = x[j], x[i] } +func (x seriesRefSlice) Len() int { return len(x) } +func (x seriesRefSlice) Less(i, j int) bool { return x[i] < x[j] } +func (x seriesRefSlice) Swap(i, j int) { x[i], x[j] = x[j], x[i] } diff --git a/tsdb/index/postings_test.go b/tsdb/index/postings_test.go index fef788c8e..8e9790f59 100644 --- a/tsdb/index/postings_test.go +++ b/tsdb/index/postings_test.go @@ -24,26 +24,27 @@ import ( "github.com/stretchr/testify/require" "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/storage" ) func TestMemPostings_addFor(t *testing.T) { p := NewMemPostings() - p.m[allPostingsKey.Name] = map[string][]uint64{} - p.m[allPostingsKey.Name][allPostingsKey.Value] = []uint64{1, 2, 3, 4, 6, 7, 8} + p.m[allPostingsKey.Name] = map[string][]storage.SeriesRef{} + p.m[allPostingsKey.Name][allPostingsKey.Value] = []storage.SeriesRef{1, 2, 3, 4, 6, 7, 8} p.addFor(5, allPostingsKey) - require.Equal(t, []uint64{1, 2, 3, 4, 5, 6, 7, 8}, p.m[allPostingsKey.Name][allPostingsKey.Value]) + require.Equal(t, []storage.SeriesRef{1, 2, 3, 4, 5, 6, 7, 8}, p.m[allPostingsKey.Name][allPostingsKey.Value]) } func TestMemPostings_ensureOrder(t *testing.T) { p := NewUnorderedMemPostings() - p.m["a"] = map[string][]uint64{} + p.m["a"] = map[string][]storage.SeriesRef{} for i := 0; i < 100; i++ { - l := make([]uint64, 100) + l := make([]storage.SeriesRef, 100) for j := range l { - l[j] = rand.Uint64() + l[j] = storage.SeriesRef(rand.Uint64()) } v := fmt.Sprintf("%d", i) @@ -94,12 +95,12 @@ func BenchmarkMemPostings_ensureOrder(b *testing.B) { // Generate postings. for l := 0; l < testData.numLabels; l++ { labelName := strconv.Itoa(l) - p.m[labelName] = map[string][]uint64{} + p.m[labelName] = map[string][]storage.SeriesRef{} for v := 0; v < testData.numValuesPerLabel; v++ { - refs := make([]uint64, testData.numRefsPerValue) + refs := make([]storage.SeriesRef, testData.numRefsPerValue) for j := range refs { - refs[j] = rand.Uint64() + refs[j] = storage.SeriesRef(rand.Uint64()) } labelValue := strconv.Itoa(v) @@ -237,29 +238,29 @@ func TestIntersect(t *testing.T) { func TestMultiIntersect(t *testing.T) { cases := []struct { - p [][]uint64 - res []uint64 + p [][]storage.SeriesRef + res []storage.SeriesRef }{ { - p: [][]uint64{ + p: [][]storage.SeriesRef{ {1, 2, 3, 4, 5, 6, 1000, 1001}, {2, 4, 5, 6, 7, 8, 999, 1001}, {1, 2, 5, 6, 7, 8, 1001, 1200}, }, - res: []uint64{2, 5, 6, 1001}, + res: []storage.SeriesRef{2, 5, 6, 1001}, }, // One of the reproducible cases for: // https://github.com/prometheus/prometheus/issues/2616 // The initialisation of intersectPostings was moving the iterator forward // prematurely making us miss some postings. { - p: [][]uint64{ + p: [][]storage.SeriesRef{ {1, 2}, {1, 2}, {1, 2}, {2}, }, - res: []uint64{2}, + res: []storage.SeriesRef{2}, }, } @@ -278,22 +279,22 @@ func TestMultiIntersect(t *testing.T) { func BenchmarkIntersect(t *testing.B) { t.Run("LongPostings1", func(bench *testing.B) { - var a, b, c, d []uint64 + var a, b, c, d []storage.SeriesRef for i := 0; i < 10000000; i += 2 { - a = append(a, uint64(i)) + a = append(a, storage.SeriesRef(i)) } for i := 5000000; i < 5000100; i += 4 { - b = append(b, uint64(i)) + b = append(b, storage.SeriesRef(i)) } for i := 5090000; i < 5090600; i += 4 { - b = append(b, uint64(i)) + b = append(b, storage.SeriesRef(i)) } for i := 4990000; i < 5100000; i++ { - c = append(c, uint64(i)) + c = append(c, storage.SeriesRef(i)) } for i := 4000000; i < 6000000; i++ { - d = append(d, uint64(i)) + d = append(d, storage.SeriesRef(i)) } i1 := newListPostings(a...) @@ -311,19 +312,19 @@ func BenchmarkIntersect(t *testing.B) { }) t.Run("LongPostings2", func(bench *testing.B) { - var a, b, c, d []uint64 + var a, b, c, d []storage.SeriesRef for i := 0; i < 12500000; i++ { - a = append(a, uint64(i)) + a = append(a, storage.SeriesRef(i)) } for i := 7500000; i < 12500000; i++ { - b = append(b, uint64(i)) + b = append(b, storage.SeriesRef(i)) } for i := 9000000; i < 20000000; i++ { - c = append(c, uint64(i)) + c = append(c, storage.SeriesRef(i)) } for i := 10000000; i < 12000000; i++ { - d = append(d, uint64(i)) + d = append(d, storage.SeriesRef(i)) } i1 := newListPostings(a...) @@ -346,9 +347,9 @@ func BenchmarkIntersect(t *testing.B) { // 100000 matchers(k=100000). for i := 0; i < 100000; i++ { - var temp []uint64 - for j := 1; j < 100; j++ { - temp = append(temp, uint64(j)) + var temp []storage.SeriesRef + for j := storage.SeriesRef(1); j < 100; j++ { + temp = append(temp, j) } its = append(its, newListPostings(temp...)) } @@ -370,7 +371,7 @@ func TestMultiMerge(t *testing.T) { res, err := ExpandPostings(Merge(i1, i2, i3)) require.NoError(t, err) - require.Equal(t, []uint64{1, 2, 3, 4, 5, 6, 7, 8, 999, 1000, 1001, 1200}, res) + require.Equal(t, []storage.SeriesRef{1, 2, 3, 4, 5, 6, 7, 8, 999, 1000, 1001, 1200}, res) } func TestMergedPostings(t *testing.T) { @@ -479,43 +480,43 @@ func TestMergedPostings(t *testing.T) { func TestMergedPostingsSeek(t *testing.T) { cases := []struct { - a, b []uint64 + a, b []storage.SeriesRef - seek uint64 + seek storage.SeriesRef success bool - res []uint64 + res []storage.SeriesRef }{ { - a: []uint64{2, 3, 4, 5}, - b: []uint64{6, 7, 8, 9, 10}, + a: []storage.SeriesRef{2, 3, 4, 5}, + b: []storage.SeriesRef{6, 7, 8, 9, 10}, seek: 1, success: true, - res: []uint64{2, 3, 4, 5, 6, 7, 8, 9, 10}, + res: []storage.SeriesRef{2, 3, 4, 5, 6, 7, 8, 9, 10}, }, { - a: []uint64{1, 2, 3, 4, 5}, - b: []uint64{6, 7, 8, 9, 10}, + a: []storage.SeriesRef{1, 2, 3, 4, 5}, + b: []storage.SeriesRef{6, 7, 8, 9, 10}, seek: 2, success: true, - res: []uint64{2, 3, 4, 5, 6, 7, 8, 9, 10}, + res: []storage.SeriesRef{2, 3, 4, 5, 6, 7, 8, 9, 10}, }, { - a: []uint64{1, 2, 3, 4, 5}, - b: []uint64{4, 5, 6, 7, 8}, + a: []storage.SeriesRef{1, 2, 3, 4, 5}, + b: []storage.SeriesRef{4, 5, 6, 7, 8}, seek: 9, success: false, res: nil, }, { - a: []uint64{1, 2, 3, 4, 9, 10}, - b: []uint64{1, 4, 5, 6, 7, 8, 10, 11}, + a: []storage.SeriesRef{1, 2, 3, 4, 9, 10}, + b: []storage.SeriesRef{1, 4, 5, 6, 7, 8, 10, 11}, seek: 10, success: true, - res: []uint64{10, 11}, + res: []storage.SeriesRef{10, 11}, }, } @@ -533,7 +534,7 @@ func TestMergedPostingsSeek(t *testing.T) { lst, err := ExpandPostings(p) require.NoError(t, err) - lst = append([]uint64{start}, lst...) + lst = append([]storage.SeriesRef{start}, lst...) require.Equal(t, c.res, lst) } } @@ -541,43 +542,43 @@ func TestMergedPostingsSeek(t *testing.T) { func TestRemovedPostings(t *testing.T) { cases := []struct { - a, b []uint64 - res []uint64 + a, b []storage.SeriesRef + res []storage.SeriesRef }{ { a: nil, b: nil, - res: []uint64(nil), + res: []storage.SeriesRef(nil), }, { - a: []uint64{1, 2, 3, 4}, + a: []storage.SeriesRef{1, 2, 3, 4}, b: nil, - res: []uint64{1, 2, 3, 4}, + res: []storage.SeriesRef{1, 2, 3, 4}, }, { a: nil, - b: []uint64{1, 2, 3, 4}, - res: []uint64(nil), + b: []storage.SeriesRef{1, 2, 3, 4}, + res: []storage.SeriesRef(nil), }, { - a: []uint64{1, 2, 3, 4, 5}, - b: []uint64{6, 7, 8, 9, 10}, - res: []uint64{1, 2, 3, 4, 5}, + a: []storage.SeriesRef{1, 2, 3, 4, 5}, + b: []storage.SeriesRef{6, 7, 8, 9, 10}, + res: []storage.SeriesRef{1, 2, 3, 4, 5}, }, { - a: []uint64{1, 2, 3, 4, 5}, - b: []uint64{4, 5, 6, 7, 8}, - res: []uint64{1, 2, 3}, + a: []storage.SeriesRef{1, 2, 3, 4, 5}, + b: []storage.SeriesRef{4, 5, 6, 7, 8}, + res: []storage.SeriesRef{1, 2, 3}, }, { - a: []uint64{1, 2, 3, 4, 9, 10}, - b: []uint64{1, 4, 5, 6, 7, 8, 10, 11}, - res: []uint64{2, 3, 9}, + a: []storage.SeriesRef{1, 2, 3, 4, 9, 10}, + b: []storage.SeriesRef{1, 4, 5, 6, 7, 8, 10, 11}, + res: []storage.SeriesRef{2, 3, 9}, }, { - a: []uint64{1, 2, 3, 4, 9, 10}, - b: []uint64{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11}, - res: []uint64(nil), + a: []storage.SeriesRef{1, 2, 3, 4, 9, 10}, + b: []storage.SeriesRef{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11}, + res: []storage.SeriesRef(nil), }, } @@ -592,10 +593,10 @@ func TestRemovedPostings(t *testing.T) { } func TestRemovedNextStackoverflow(t *testing.T) { - var full []uint64 - var remove []uint64 + var full []storage.SeriesRef + var remove []storage.SeriesRef - var i uint64 + var i storage.SeriesRef for i = 0; i < 1e7; i++ { full = append(full, i) remove = append(remove, i) @@ -615,67 +616,67 @@ func TestRemovedNextStackoverflow(t *testing.T) { func TestRemovedPostingsSeek(t *testing.T) { cases := []struct { - a, b []uint64 + a, b []storage.SeriesRef - seek uint64 + seek storage.SeriesRef success bool - res []uint64 + res []storage.SeriesRef }{ { - a: []uint64{2, 3, 4, 5}, - b: []uint64{6, 7, 8, 9, 10}, + a: []storage.SeriesRef{2, 3, 4, 5}, + b: []storage.SeriesRef{6, 7, 8, 9, 10}, seek: 1, success: true, - res: []uint64{2, 3, 4, 5}, + res: []storage.SeriesRef{2, 3, 4, 5}, }, { - a: []uint64{1, 2, 3, 4, 5}, - b: []uint64{6, 7, 8, 9, 10}, + a: []storage.SeriesRef{1, 2, 3, 4, 5}, + b: []storage.SeriesRef{6, 7, 8, 9, 10}, seek: 2, success: true, - res: []uint64{2, 3, 4, 5}, + res: []storage.SeriesRef{2, 3, 4, 5}, }, { - a: []uint64{1, 2, 3, 4, 5}, - b: []uint64{4, 5, 6, 7, 8}, + a: []storage.SeriesRef{1, 2, 3, 4, 5}, + b: []storage.SeriesRef{4, 5, 6, 7, 8}, seek: 9, success: false, res: nil, }, { - a: []uint64{1, 2, 3, 4, 9, 10}, - b: []uint64{1, 4, 5, 6, 7, 8, 10, 11}, + a: []storage.SeriesRef{1, 2, 3, 4, 9, 10}, + b: []storage.SeriesRef{1, 4, 5, 6, 7, 8, 10, 11}, seek: 10, success: false, res: nil, }, { - a: []uint64{1, 2, 3, 4, 9, 10}, - b: []uint64{1, 4, 5, 6, 7, 8, 11}, + a: []storage.SeriesRef{1, 2, 3, 4, 9, 10}, + b: []storage.SeriesRef{1, 4, 5, 6, 7, 8, 11}, seek: 4, success: true, - res: []uint64{9, 10}, + res: []storage.SeriesRef{9, 10}, }, { - a: []uint64{1, 2, 3, 4, 9, 10}, - b: []uint64{1, 4, 5, 6, 7, 8, 11}, + a: []storage.SeriesRef{1, 2, 3, 4, 9, 10}, + b: []storage.SeriesRef{1, 4, 5, 6, 7, 8, 11}, seek: 5, success: true, - res: []uint64{9, 10}, + res: []storage.SeriesRef{9, 10}, }, { - a: []uint64{1, 2, 3, 4, 9, 10}, - b: []uint64{1, 4, 5, 6, 7, 8, 11}, + a: []storage.SeriesRef{1, 2, 3, 4, 9, 10}, + b: []storage.SeriesRef{1, 4, 5, 6, 7, 8, 11}, seek: 10, success: true, - res: []uint64{10}, + res: []storage.SeriesRef{10}, }, } @@ -693,7 +694,7 @@ func TestRemovedPostingsSeek(t *testing.T) { lst, err := ExpandPostings(p) require.NoError(t, err) - lst = append([]uint64{start}, lst...) + lst = append([]storage.SeriesRef{start}, lst...) require.Equal(t, c.res, lst) } } @@ -718,7 +719,7 @@ func TestBigEndian(t *testing.T) { bep := newBigEndianPostings(beLst) for i := 0; i < num; i++ { require.True(t, bep.Next()) - require.Equal(t, uint64(ls[i]), bep.At()) + require.Equal(t, storage.SeriesRef(ls[i]), bep.At()) } require.False(t, bep.Next()) @@ -766,8 +767,8 @@ func TestBigEndian(t *testing.T) { bep := newBigEndianPostings(beLst) for _, v := range table { - require.Equal(t, v.found, bep.Seek(uint64(v.seek))) - require.Equal(t, uint64(v.val), bep.At()) + require.Equal(t, v.found, bep.Seek(storage.SeriesRef(v.seek))) + require.Equal(t, storage.SeriesRef(v.val), bep.At()) require.NoError(t, bep.Err()) } }) @@ -787,7 +788,7 @@ func TestIntersectWithMerge(t *testing.T) { res, err := ExpandPostings(p) require.NoError(t, err) - require.Equal(t, []uint64{30}, res) + require.Equal(t, []storage.SeriesRef{30}, res) } func TestWithoutPostings(t *testing.T) { @@ -871,7 +872,7 @@ func TestWithoutPostings(t *testing.T) { func BenchmarkPostings_Stats(b *testing.B) { p := NewMemPostings() - var seriesID uint64 + var seriesID storage.SeriesRef createPostingsLabelValues := func(name, valuePrefix string, count int) { for n := 1; n < count; n++ { @@ -906,7 +907,7 @@ func TestMemPostings_Delete(t *testing.T) { p.Add(3, labels.FromStrings("lbl2", "a")) before := p.Get(allPostingsKey.Name, allPostingsKey.Value) - p.Delete(map[uint64]struct{}{ + p.Delete(map[storage.SeriesRef]struct{}{ 2: {}, }) after := p.Get(allPostingsKey.Name, allPostingsKey.Value) @@ -915,13 +916,13 @@ func TestMemPostings_Delete(t *testing.T) { // iterated over. expanded, err := ExpandPostings(before) require.NoError(t, err) - require.Equal(t, []uint64{1, 2, 3}, expanded) + require.Equal(t, []storage.SeriesRef{1, 2, 3}, expanded) // Make sure postings gotten after the delete have the new data when // iterated over. expanded, err = ExpandPostings(after) require.NoError(t, err) - require.Equal(t, []uint64{1, 3}, expanded) + require.Equal(t, []storage.SeriesRef{1, 3}, expanded) deleted := p.Get("lbl1", "b") expanded, err = ExpandPostings(deleted) diff --git a/tsdb/mocks_test.go b/tsdb/mocks_test.go index ac368e7ba..1c3cc4103 100644 --- a/tsdb/mocks_test.go +++ b/tsdb/mocks_test.go @@ -17,6 +17,7 @@ import ( "github.com/pkg/errors" "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/tombstones" @@ -34,7 +35,7 @@ func copyChunk(c chunkenc.Chunk) (chunkenc.Chunk, error) { } func (mockIndexWriter) AddSymbol(string) error { return nil } -func (m *mockIndexWriter) AddSeries(_ uint64, l labels.Labels, chks ...chunks.Meta) error { +func (m *mockIndexWriter) AddSeries(_ storage.SeriesRef, l labels.Labels, chks ...chunks.Meta) error { // Copy chunks as their bytes are pooled. chksNew := make([]chunks.Meta, len(chks)) for i, chk := range chks { diff --git a/tsdb/querier.go b/tsdb/querier.go index 90ee63854..4c48b9098 100644 --- a/tsdb/querier.go +++ b/tsdb/querier.go @@ -418,7 +418,7 @@ func labelNamesWithMatchers(r IndexReader, matchers ...*labels.Matcher) ([]strin return nil, err } - var postings []uint64 + var postings []storage.SeriesRef for p.Next() { postings = append(postings, p.At()) } @@ -906,6 +906,8 @@ func newNopChunkReader() ChunkReader { } } -func (cr nopChunkReader) Chunk(ref uint64) (chunkenc.Chunk, error) { return cr.emptyChunk, nil } +func (cr nopChunkReader) Chunk(ref chunks.ChunkRef) (chunkenc.Chunk, error) { + return cr.emptyChunk, nil +} func (cr nopChunkReader) Close() error { return nil } diff --git a/tsdb/querier_test.go b/tsdb/querier_test.go index f10be88b0..804de3ab9 100644 --- a/tsdb/querier_test.go +++ b/tsdb/querier_test.go @@ -106,13 +106,13 @@ func createIdxChkReaders(t *testing.T, tc []seriesSamples) (IndexReader, ChunkRe }) postings := index.NewMemPostings() - chkReader := mockChunkReader(make(map[uint64]chunkenc.Chunk)) + chkReader := mockChunkReader(make(map[chunks.ChunkRef]chunkenc.Chunk)) lblIdx := make(map[string]map[string]struct{}) mi := newMockIndex() blockMint := int64(math.MaxInt64) blockMaxt := int64(math.MinInt64) - var chunkRef uint64 + var chunkRef chunks.ChunkRef for i, s := range tc { i = i + 1 // 0 is not a valid posting. metas := make([]chunks.Meta, 0, len(s.chunks)) @@ -139,9 +139,9 @@ func createIdxChkReaders(t *testing.T, tc []seriesSamples) (IndexReader, ChunkRe chunkRef++ } ls := labels.FromMap(s.lset) - require.NoError(t, mi.AddSeries(uint64(i), ls, metas...)) + require.NoError(t, mi.AddSeries(storage.SeriesRef(i), ls, metas...)) - postings.Add(uint64(i), ls) + postings.Add(storage.SeriesRef(i), ls) for _, l := range ls { vs, present := lblIdx[l.Name] @@ -607,21 +607,21 @@ func TestBlockQuerierDelete(t *testing.T) { type fakeChunksReader struct { ChunkReader - chks map[uint64]chunkenc.Chunk + chks map[chunks.ChunkRef]chunkenc.Chunk } func createFakeReaderAndNotPopulatedChunks(s ...[]tsdbutil.Sample) (*fakeChunksReader, []chunks.Meta) { f := &fakeChunksReader{ - chks: map[uint64]chunkenc.Chunk{}, + chks: map[chunks.ChunkRef]chunkenc.Chunk{}, } chks := make([]chunks.Meta, 0, len(s)) for ref, samples := range s { chk := tsdbutil.ChunkFromSamples(samples) - f.chks[uint64(ref)] = chk.Chunk + f.chks[chunks.ChunkRef(ref)] = chk.Chunk chks = append(chks, chunks.Meta{ - Ref: uint64(ref), + Ref: chunks.ChunkRef(ref), MinTime: chk.MinTime, MaxTime: chk.MaxTime, }) @@ -629,7 +629,7 @@ func createFakeReaderAndNotPopulatedChunks(s ...[]tsdbutil.Sample) (*fakeChunksR return f, chks } -func (r *fakeChunksReader) Chunk(ref uint64) (chunkenc.Chunk, error) { +func (r *fakeChunksReader) Chunk(ref chunks.ChunkRef) (chunkenc.Chunk, error) { chk, ok := r.chks[ref] if !ok { return nil, errors.Errorf("chunk not found at ref %v", ref) @@ -1016,9 +1016,9 @@ func BenchmarkMergedSeriesSet(b *testing.B) { } } -type mockChunkReader map[uint64]chunkenc.Chunk +type mockChunkReader map[chunks.ChunkRef]chunkenc.Chunk -func (cr mockChunkReader) Chunk(id uint64) (chunkenc.Chunk, error) { +func (cr mockChunkReader) Chunk(id chunks.ChunkRef) (chunkenc.Chunk, error) { chk, ok := cr[id] if ok { return chk, nil @@ -1138,15 +1138,15 @@ type series struct { } type mockIndex struct { - series map[uint64]series - postings map[labels.Label][]uint64 + series map[storage.SeriesRef]series + postings map[labels.Label][]storage.SeriesRef symbols map[string]struct{} } func newMockIndex() mockIndex { ix := mockIndex{ - series: make(map[uint64]series), - postings: make(map[labels.Label][]uint64), + series: make(map[storage.SeriesRef]series), + postings: make(map[labels.Label][]storage.SeriesRef), symbols: make(map[string]struct{}), } return ix @@ -1161,7 +1161,7 @@ func (m mockIndex) Symbols() index.StringIter { return index.NewStringListIter(l) } -func (m *mockIndex) AddSeries(ref uint64, l labels.Labels, chunks ...chunks.Meta) error { +func (m *mockIndex) AddSeries(ref storage.SeriesRef, l labels.Labels, chunks ...chunks.Meta) error { if _, ok := m.series[ref]; ok { return errors.Errorf("series with reference %d already added", ref) } @@ -1228,11 +1228,11 @@ func (m mockIndex) LabelValues(name string, matchers ...*labels.Matcher) ([]stri return values, nil } -func (m mockIndex) LabelValueFor(id uint64, label string) (string, error) { +func (m mockIndex) LabelValueFor(id storage.SeriesRef, label string) (string, error) { return m.series[id].l.Get(label), nil } -func (m mockIndex) LabelNamesFor(ids ...uint64) ([]string, error) { +func (m mockIndex) LabelNamesFor(ids ...storage.SeriesRef) ([]string, error) { namesMap := make(map[string]bool) for _, id := range ids { for _, lbl := range m.series[id].l { @@ -1267,7 +1267,7 @@ func (m mockIndex) SortedPostings(p index.Postings) index.Postings { return index.NewListPostings(ep) } -func (m mockIndex) Series(ref uint64, lset *labels.Labels, chks *[]chunks.Meta) error { +func (m mockIndex) Series(ref storage.SeriesRef, lset *labels.Labels, chks *[]chunks.Meta) error { s, ok := m.series[ref] if !ok { return storage.ErrNotFound @@ -2085,11 +2085,11 @@ func (m mockMatcherIndex) LabelValues(name string, matchers ...*labels.Matcher) return []string{}, errors.New("label values called") } -func (m mockMatcherIndex) LabelValueFor(id uint64, label string) (string, error) { +func (m mockMatcherIndex) LabelValueFor(id storage.SeriesRef, label string) (string, error) { return "", errors.New("label value for called") } -func (m mockMatcherIndex) LabelNamesFor(ids ...uint64) ([]string, error) { +func (m mockMatcherIndex) LabelNamesFor(ids ...storage.SeriesRef) ([]string, error) { return nil, errors.New("label names for for called") } @@ -2101,7 +2101,7 @@ func (m mockMatcherIndex) SortedPostings(p index.Postings) index.Postings { return index.EmptyPostings() } -func (m mockMatcherIndex) Series(ref uint64, lset *labels.Labels, chks *[]chunks.Meta) error { +func (m mockMatcherIndex) Series(ref storage.SeriesRef, lset *labels.Labels, chks *[]chunks.Meta) error { return nil } @@ -2151,13 +2151,13 @@ func TestBlockBaseSeriesSet(t *testing.T) { lset labels.Labels chunks []chunks.Meta - ref uint64 + ref storage.SeriesRef } cases := []struct { series []refdSeries // Postings should be in the sorted order of the series - postings []uint64 + postings []storage.SeriesRef expIdxs []int }{ @@ -2196,7 +2196,7 @@ func TestBlockBaseSeriesSet(t *testing.T) { ref: 108, }, }, - postings: []uint64{12, 13, 10, 108}, // 13 doesn't exist and should just be skipped over. + postings: []storage.SeriesRef{12, 13, 10, 108}, // 13 doesn't exist and should just be skipped over. expIdxs: []int{0, 1, 3}, }, { @@ -2214,7 +2214,7 @@ func TestBlockBaseSeriesSet(t *testing.T) { ref: 3, }, }, - postings: []uint64{}, + postings: []storage.SeriesRef{}, expIdxs: []int{}, }, } diff --git a/tsdb/record/record.go b/tsdb/record/record.go index 192c29ce7..d6fa30ac0 100644 --- a/tsdb/record/record.go +++ b/tsdb/record/record.go @@ -21,6 +21,8 @@ import ( "github.com/pkg/errors" "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/encoding" "github.com/prometheus/prometheus/tsdb/tombstones" ) @@ -46,20 +48,20 @@ var ErrNotFound = errors.New("not found") // RefSeries is the series labels with the series ID. type RefSeries struct { - Ref uint64 + Ref chunks.HeadSeriesRef Labels labels.Labels } // RefSample is a timestamp/value pair associated with a reference to a series. type RefSample struct { - Ref uint64 + Ref chunks.HeadSeriesRef T int64 V float64 } // RefExemplar is an exemplar with it's labels, timestamp, value the exemplar was collected/observed with, and a reference to a series. type RefExemplar struct { - Ref uint64 + Ref chunks.HeadSeriesRef T int64 V float64 Labels labels.Labels @@ -90,7 +92,7 @@ func (d *Decoder) Series(rec []byte, series []RefSeries) ([]RefSeries, error) { return nil, errors.New("invalid record type") } for len(dec.B) > 0 && dec.Err() == nil { - ref := dec.Be64() + ref := storage.SeriesRef(dec.Be64()) lset := make(labels.Labels, dec.Uvarint()) @@ -101,7 +103,7 @@ func (d *Decoder) Series(rec []byte, series []RefSeries) ([]RefSeries, error) { sort.Sort(lset) series = append(series, RefSeries{ - Ref: ref, + Ref: chunks.HeadSeriesRef(ref), Labels: lset, }) } @@ -134,7 +136,7 @@ func (d *Decoder) Samples(rec []byte, samples []RefSample) ([]RefSample, error) val := dec.Be64() samples = append(samples, RefSample{ - Ref: uint64(int64(baseRef) + dref), + Ref: chunks.HeadSeriesRef(int64(baseRef) + dref), T: baseTime + dtime, V: math.Float64frombits(val), }) @@ -158,7 +160,7 @@ func (d *Decoder) Tombstones(rec []byte, tstones []tombstones.Stone) ([]tombston } for dec.Len() > 0 && dec.Err() == nil { tstones = append(tstones, tombstones.Stone{ - Ref: dec.Be64(), + Ref: storage.SeriesRef(dec.Be64()), Intervals: tombstones.Intervals{ {Mint: dec.Varint64(), Maxt: dec.Varint64()}, }, @@ -204,7 +206,7 @@ func (d *Decoder) ExemplarsFromBuffer(dec *encoding.Decbuf, exemplars []RefExemp sort.Sort(lset) exemplars = append(exemplars, RefExemplar{ - Ref: baseRef + uint64(dref), + Ref: chunks.HeadSeriesRef(baseRef + uint64(dref)), T: baseTime + dtime, V: math.Float64frombits(val), Labels: lset, @@ -230,7 +232,7 @@ func (e *Encoder) Series(series []RefSeries, b []byte) []byte { buf.PutByte(byte(Series)) for _, s := range series { - buf.PutBE64(s.Ref) + buf.PutBE64(uint64(s.Ref)) buf.PutUvarint(len(s.Labels)) for _, l := range s.Labels { @@ -254,7 +256,7 @@ func (e *Encoder) Samples(samples []RefSample, b []byte) []byte { // All samples encode their timestamp and ref as delta to those. first := samples[0] - buf.PutBE64(first.Ref) + buf.PutBE64(uint64(first.Ref)) buf.PutBE64int64(first.T) for _, s := range samples { @@ -272,7 +274,7 @@ func (e *Encoder) Tombstones(tstones []tombstones.Stone, b []byte) []byte { for _, s := range tstones { for _, iv := range s.Intervals { - buf.PutBE64(s.Ref) + buf.PutBE64(uint64(s.Ref)) buf.PutVarint64(iv.Mint) buf.PutVarint64(iv.Maxt) } @@ -298,7 +300,7 @@ func (e *Encoder) EncodeExemplarsIntoBuffer(exemplars []RefExemplar, buf *encodi // All samples encode their timestamp and ref as delta to those. first := exemplars[0] - buf.PutBE64(first.Ref) + buf.PutBE64(uint64(first.Ref)) buf.PutBE64int64(first.T) for _, ex := range exemplars { diff --git a/tsdb/tombstones/tombstones.go b/tsdb/tombstones/tombstones.go index 5f5ca7d09..621e10471 100644 --- a/tsdb/tombstones/tombstones.go +++ b/tsdb/tombstones/tombstones.go @@ -28,6 +28,7 @@ import ( "github.com/go-kit/log/level" "github.com/pkg/errors" + "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/encoding" tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" "github.com/prometheus/prometheus/tsdb/fileutil" @@ -63,10 +64,10 @@ func newCRC32() hash.Hash32 { // Reader gives access to tombstone intervals by series reference. type Reader interface { // Get returns deletion intervals for the series with the given reference. - Get(ref uint64) (Intervals, error) + Get(ref storage.SeriesRef) (Intervals, error) // Iter calls the given function for each encountered interval. - Iter(func(uint64, Intervals) error) error + Iter(func(storage.SeriesRef, Intervals) error) error // Total returns the total count of tombstones. Total() uint64 @@ -144,9 +145,9 @@ func WriteFile(logger log.Logger, dir string, tr Reader) (int64, error) { func Encode(tr Reader) ([]byte, error) { buf := encoding.Encbuf{} buf.PutByte(tombstoneFormatV1) - err := tr.Iter(func(ref uint64, ivs Intervals) error { + err := tr.Iter(func(ref storage.SeriesRef, ivs Intervals) error { for _, iv := range ivs { - buf.PutUvarint64(ref) + buf.PutUvarint64(uint64(ref)) buf.PutVarint64(iv.Mint) buf.PutVarint64(iv.Maxt) } @@ -169,7 +170,7 @@ func Decode(b []byte) (Reader, error) { stonesMap := NewMemTombstones() for d.Len() > 0 { - k := d.Uvarint64() + k := storage.SeriesRef(d.Uvarint64()) mint := d.Varint64() maxt := d.Varint64() if d.Err() != nil { @@ -184,7 +185,7 @@ func Decode(b []byte) (Reader, error) { // Stone holds the information on the posting and time-range // that is deleted. type Stone struct { - Ref uint64 + Ref storage.SeriesRef Intervals Intervals } @@ -228,33 +229,33 @@ func ReadTombstones(dir string) (Reader, int64, error) { } type MemTombstones struct { - intvlGroups map[uint64]Intervals + intvlGroups map[storage.SeriesRef]Intervals mtx sync.RWMutex } // NewMemTombstones creates new in memory Tombstone Reader // that allows adding new intervals. func NewMemTombstones() *MemTombstones { - return &MemTombstones{intvlGroups: make(map[uint64]Intervals)} + return &MemTombstones{intvlGroups: make(map[storage.SeriesRef]Intervals)} } func NewTestMemTombstones(intervals []Intervals) *MemTombstones { ret := NewMemTombstones() for i, intervalsGroup := range intervals { for _, interval := range intervalsGroup { - ret.AddInterval(uint64(i+1), interval) + ret.AddInterval(storage.SeriesRef(i+1), interval) } } return ret } -func (t *MemTombstones) Get(ref uint64) (Intervals, error) { +func (t *MemTombstones) Get(ref storage.SeriesRef) (Intervals, error) { t.mtx.RLock() defer t.mtx.RUnlock() return t.intvlGroups[ref], nil } -func (t *MemTombstones) DeleteTombstones(refs map[uint64]struct{}) { +func (t *MemTombstones) DeleteTombstones(refs map[storage.SeriesRef]struct{}) { t.mtx.Lock() defer t.mtx.Unlock() for ref := range refs { @@ -282,7 +283,7 @@ func (t *MemTombstones) TruncateBefore(beforeT int64) { } } -func (t *MemTombstones) Iter(f func(uint64, Intervals) error) error { +func (t *MemTombstones) Iter(f func(storage.SeriesRef, Intervals) error) error { t.mtx.RLock() defer t.mtx.RUnlock() for ref, ivs := range t.intvlGroups { @@ -305,7 +306,7 @@ func (t *MemTombstones) Total() uint64 { } // AddInterval to an existing memTombstones. -func (t *MemTombstones) AddInterval(ref uint64, itvs ...Interval) { +func (t *MemTombstones) AddInterval(ref storage.SeriesRef, itvs ...Interval) { t.mtx.Lock() defer t.mtx.Unlock() for _, itv := range itvs { diff --git a/tsdb/tombstones/tombstones_test.go b/tsdb/tombstones/tombstones_test.go index aee8568eb..95fb12d69 100644 --- a/tsdb/tombstones/tombstones_test.go +++ b/tsdb/tombstones/tombstones_test.go @@ -25,6 +25,8 @@ import ( "github.com/go-kit/log" "github.com/stretchr/testify/require" "go.uber.org/goleak" + + "github.com/prometheus/prometheus/storage" ) func TestMain(m *testing.M) { @@ -50,7 +52,7 @@ func TestWriteAndReadbackTombstones(t *testing.T) { dranges = dranges.Add(Interval{mint, mint + rand.Int63n(1000)}) mint += rand.Int63n(1000) + 1 } - stones.AddInterval(ref, dranges...) + stones.AddInterval(storage.SeriesRef(ref), dranges...) } _, err := WriteFile(log.NewNopLogger(), tmpdir, stones) @@ -66,18 +68,18 @@ func TestWriteAndReadbackTombstones(t *testing.T) { func TestDeletingTombstones(t *testing.T) { stones := NewMemTombstones() - ref := uint64(42) + ref := storage.SeriesRef(42) mint := rand.Int63n(time.Now().UnixNano()) dranges := make(Intervals, 0, 1) dranges = dranges.Add(Interval{mint, mint + rand.Int63n(1000)}) stones.AddInterval(ref, dranges...) - stones.AddInterval(uint64(43), dranges...) + stones.AddInterval(storage.SeriesRef(43), dranges...) intervals, err := stones.Get(ref) require.NoError(t, err) require.Equal(t, intervals, dranges) - stones.DeleteTombstones(map[uint64]struct{}{ref: {}}) + stones.DeleteTombstones(map[storage.SeriesRef]struct{}{ref: {}}) intervals, err = stones.Get(ref) require.NoError(t, err) @@ -112,7 +114,7 @@ func TestTruncateBefore(t *testing.T) { }, } for _, c := range cases { - ref := uint64(42) + ref := storage.SeriesRef(42) stones := NewMemTombstones() stones.AddInterval(ref, c.before...) @@ -231,13 +233,13 @@ func TestMemTombstonesConcurrency(t *testing.T) { go func() { for x := 0; x < totalRuns; x++ { - tomb.AddInterval(uint64(x), Interval{int64(x), int64(x)}) + tomb.AddInterval(storage.SeriesRef(x), Interval{int64(x), int64(x)}) } wg.Done() }() go func() { for x := 0; x < totalRuns; x++ { - _, err := tomb.Get(uint64(x)) + _, err := tomb.Get(storage.SeriesRef(x)) require.NoError(t, err) } wg.Done() diff --git a/tsdb/tsdbblockutil.go b/tsdb/tsdbblockutil.go index 3e4ac5856..732446324 100644 --- a/tsdb/tsdbblockutil.go +++ b/tsdb/tsdbblockutil.go @@ -48,7 +48,7 @@ func CreateBlock(series []storage.Series, dir string, chunkRange int64, logger l app := w.Appender(ctx) for _, s := range series { - ref := uint64(0) + ref := storage.SeriesRef(0) it := s.Iterator() lset := s.Labels() for it.Next() { diff --git a/tsdb/wal.go b/tsdb/wal.go index 281044a95..a0672f405 100644 --- a/tsdb/wal.go +++ b/tsdb/wal.go @@ -33,6 +33,8 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/encoding" "github.com/prometheus/prometheus/tsdb/fileutil" "github.com/prometheus/prometheus/tsdb/record" @@ -113,8 +115,8 @@ type WALReader interface { // the truncation threshold can be compacted. type segmentFile struct { *os.File - maxTime int64 // highest tombstone or sample timestamp in segment - minSeries uint64 // lowerst series ID in segment + maxTime int64 // highest tombstone or sample timestamp in segment + minSeries chunks.HeadSeriesRef // lowerst series ID in segment } func newSegmentFile(f *os.File) *segmentFile { @@ -292,7 +294,7 @@ func (w *SegmentWAL) putBuffer(b *encoding.Encbuf) { // Truncate deletes the values prior to mint and the series which the keep function // does not indicate to preserve. -func (w *SegmentWAL) Truncate(mint int64, keep func(uint64) bool) error { +func (w *SegmentWAL) Truncate(mint int64, keep func(chunks.HeadSeriesRef) bool) error { // The last segment is always active. if len(w.files) < 2 { return nil @@ -787,7 +789,7 @@ const ( func (w *SegmentWAL) encodeSeries(buf *encoding.Encbuf, series []record.RefSeries) uint8 { for _, s := range series { - buf.PutBE64(s.Ref) + buf.PutBE64(uint64(s.Ref)) buf.PutUvarint(len(s.Labels)) for _, l := range s.Labels { @@ -808,7 +810,7 @@ func (w *SegmentWAL) encodeSamples(buf *encoding.Encbuf, samples []record.RefSam // TODO(fabxc): optimize for all samples having the same timestamp. first := samples[0] - buf.PutBE64(first.Ref) + buf.PutBE64(uint64(first.Ref)) buf.PutBE64int64(first.T) for _, s := range samples { @@ -822,7 +824,7 @@ func (w *SegmentWAL) encodeSamples(buf *encoding.Encbuf, samples []record.RefSam func (w *SegmentWAL) encodeDeletes(buf *encoding.Encbuf, stones []tombstones.Stone) uint8 { for _, s := range stones { for _, iv := range s.Intervals { - buf.PutBE64(s.Ref) + buf.PutBE64(uint64(s.Ref)) buf.PutVarint64(iv.Mint) buf.PutVarint64(iv.Maxt) } @@ -1120,7 +1122,7 @@ func (r *walReader) decodeSeries(flag byte, b []byte, res *[]record.RefSeries) e dec := encoding.Decbuf{B: b} for len(dec.B) > 0 && dec.Err() == nil { - ref := dec.Be64() + ref := chunks.HeadSeriesRef(dec.Be64()) lset := make(labels.Labels, dec.Uvarint()) @@ -1161,7 +1163,7 @@ func (r *walReader) decodeSamples(flag byte, b []byte, res *[]record.RefSample) val := dec.Be64() *res = append(*res, record.RefSample{ - Ref: uint64(int64(baseRef) + dref), + Ref: chunks.HeadSeriesRef(int64(baseRef) + dref), T: baseTime + dtime, V: math.Float64frombits(val), }) @@ -1181,7 +1183,7 @@ func (r *walReader) decodeDeletes(flag byte, b []byte, res *[]tombstones.Stone) for dec.Len() > 0 && dec.Err() == nil { *res = append(*res, tombstones.Stone{ - Ref: dec.Be64(), + Ref: storage.SeriesRef(dec.Be64()), Intervals: tombstones.Intervals{ {Mint: dec.Varint64(), Maxt: dec.Varint64()}, }, diff --git a/tsdb/wal/checkpoint.go b/tsdb/wal/checkpoint.go index 780aa2daf..9064beed0 100644 --- a/tsdb/wal/checkpoint.go +++ b/tsdb/wal/checkpoint.go @@ -29,6 +29,7 @@ import ( "github.com/go-kit/log/level" "github.com/pkg/errors" + "github.com/prometheus/prometheus/tsdb/chunks" tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" "github.com/prometheus/prometheus/tsdb/fileutil" "github.com/prometheus/prometheus/tsdb/record" @@ -90,7 +91,7 @@ const checkpointPrefix = "checkpoint." // segmented format as the original WAL itself. // This makes it easy to read it through the WAL package and concatenate // it with the original WAL. -func Checkpoint(logger log.Logger, w *WAL, from, to int, keep func(id uint64) bool, mint int64) (*CheckpointStats, error) { +func Checkpoint(logger log.Logger, w *WAL, from, to int, keep func(id chunks.HeadSeriesRef) bool, mint int64) (*CheckpointStats, error) { stats := &CheckpointStats{} var sgmReader io.ReadCloser diff --git a/tsdb/wal/checkpoint_test.go b/tsdb/wal/checkpoint_test.go index 2a5ff736a..eed29be94 100644 --- a/tsdb/wal/checkpoint_test.go +++ b/tsdb/wal/checkpoint_test.go @@ -27,6 +27,7 @@ import ( "github.com/stretchr/testify/require" "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/record" ) @@ -174,7 +175,7 @@ func TestCheckpoint(t *testing.T) { } require.NoError(t, w.Close()) - _, err = Checkpoint(log.NewNopLogger(), w, 100, 106, func(x uint64) bool { + _, err = Checkpoint(log.NewNopLogger(), w, 100, 106, func(x chunks.HeadSeriesRef) bool { return x%2 == 0 }, last/2) require.NoError(t, err) diff --git a/tsdb/wal/watcher_test.go b/tsdb/wal/watcher_test.go index 58800431b..d453b9da2 100644 --- a/tsdb/wal/watcher_test.go +++ b/tsdb/wal/watcher_test.go @@ -26,6 +26,7 @@ import ( "github.com/stretchr/testify/require" "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/record" ) @@ -53,7 +54,7 @@ type writeToMock struct { samplesAppended int exemplarsAppended int seriesLock sync.Mutex - seriesSegmentIndexes map[uint64]int + seriesSegmentIndexes map[chunks.HeadSeriesRef]int } func (wtm *writeToMock) Append(s []record.RefSample) bool { @@ -98,7 +99,7 @@ func (wtm *writeToMock) checkNumLabels() int { func newWriteToMock() *writeToMock { return &writeToMock{ - seriesSegmentIndexes: make(map[uint64]int), + seriesSegmentIndexes: make(map[chunks.HeadSeriesRef]int), } } @@ -129,7 +130,7 @@ func TestTailSamples(t *testing.T) { ref := i + 100 series := enc.Series([]record.RefSeries{ { - Ref: uint64(ref), + Ref: chunks.HeadSeriesRef(ref), Labels: labels.Labels{labels.Label{Name: "__name__", Value: fmt.Sprintf("metric_%d", i)}}, }, }, nil) @@ -139,7 +140,7 @@ func TestTailSamples(t *testing.T) { inner := rand.Intn(ref + 1) sample := enc.Samples([]record.RefSample{ { - Ref: uint64(inner), + Ref: chunks.HeadSeriesRef(inner), T: now.UnixNano() + 1, V: float64(i), }, @@ -151,7 +152,7 @@ func TestTailSamples(t *testing.T) { inner := rand.Intn(ref + 1) exemplar := enc.Exemplars([]record.RefExemplar{ { - Ref: uint64(inner), + Ref: chunks.HeadSeriesRef(inner), T: now.UnixNano() + 1, V: float64(i), Labels: labels.FromStrings("traceID", fmt.Sprintf("trace-%d", inner)), @@ -219,7 +220,7 @@ func TestReadToEndNoCheckpoint(t *testing.T) { for i := 0; i < seriesCount; i++ { series := enc.Series([]record.RefSeries{ { - Ref: uint64(i), + Ref: chunks.HeadSeriesRef(i), Labels: labels.Labels{labels.Label{Name: "__name__", Value: fmt.Sprintf("metric_%d", i)}}, }, }, nil) @@ -227,7 +228,7 @@ func TestReadToEndNoCheckpoint(t *testing.T) { for j := 0; j < samplesCount; j++ { sample := enc.Samples([]record.RefSample{ { - Ref: uint64(j), + Ref: chunks.HeadSeriesRef(j), T: int64(i), V: float64(i), }, @@ -288,7 +289,7 @@ func TestReadToEndWithCheckpoint(t *testing.T) { ref := i + 100 series := enc.Series([]record.RefSeries{ { - Ref: uint64(ref), + Ref: chunks.HeadSeriesRef(ref), Labels: labels.Labels{labels.Label{Name: "__name__", Value: fmt.Sprintf("metric_%d", i)}}, }, }, nil) @@ -300,7 +301,7 @@ func TestReadToEndWithCheckpoint(t *testing.T) { inner := rand.Intn(ref + 1) sample := enc.Samples([]record.RefSample{ { - Ref: uint64(inner), + Ref: chunks.HeadSeriesRef(inner), T: int64(i), V: float64(i), }, @@ -309,14 +310,14 @@ func TestReadToEndWithCheckpoint(t *testing.T) { } } - Checkpoint(log.NewNopLogger(), w, 0, 1, func(x uint64) bool { return true }, 0) + Checkpoint(log.NewNopLogger(), w, 0, 1, func(x chunks.HeadSeriesRef) bool { return true }, 0) w.Truncate(1) // Write more records after checkpointing. for i := 0; i < seriesCount; i++ { series := enc.Series([]record.RefSeries{ { - Ref: uint64(i), + Ref: chunks.HeadSeriesRef(i), Labels: labels.Labels{labels.Label{Name: "__name__", Value: fmt.Sprintf("metric_%d", i)}}, }, }, nil) @@ -325,7 +326,7 @@ func TestReadToEndWithCheckpoint(t *testing.T) { for j := 0; j < samplesCount; j++ { sample := enc.Samples([]record.RefSample{ { - Ref: uint64(j), + Ref: chunks.HeadSeriesRef(j), T: int64(i), V: float64(i), }, @@ -377,7 +378,7 @@ func TestReadCheckpoint(t *testing.T) { ref := i + 100 series := enc.Series([]record.RefSeries{ { - Ref: uint64(ref), + Ref: chunks.HeadSeriesRef(ref), Labels: labels.Labels{labels.Label{Name: "__name__", Value: fmt.Sprintf("metric_%d", i)}}, }, }, nil) @@ -387,7 +388,7 @@ func TestReadCheckpoint(t *testing.T) { inner := rand.Intn(ref + 1) sample := enc.Samples([]record.RefSample{ { - Ref: uint64(inner), + Ref: chunks.HeadSeriesRef(inner), T: int64(i), V: float64(i), }, @@ -395,7 +396,7 @@ func TestReadCheckpoint(t *testing.T) { require.NoError(t, w.Log(sample)) } } - Checkpoint(log.NewNopLogger(), w, 30, 31, func(x uint64) bool { return true }, 0) + Checkpoint(log.NewNopLogger(), w, 30, 31, func(x chunks.HeadSeriesRef) bool { return true }, 0) w.Truncate(32) // Start read after checkpoint, no more data written. @@ -441,7 +442,7 @@ func TestReadCheckpointMultipleSegments(t *testing.T) { ref := j + (i * 100) series := enc.Series([]record.RefSeries{ { - Ref: uint64(ref), + Ref: chunks.HeadSeriesRef(ref), Labels: labels.Labels{labels.Label{Name: "__name__", Value: fmt.Sprintf("metric_%d", j)}}, }, }, nil) @@ -451,7 +452,7 @@ func TestReadCheckpointMultipleSegments(t *testing.T) { inner := rand.Intn(ref + 1) sample := enc.Samples([]record.RefSample{ { - Ref: uint64(inner), + Ref: chunks.HeadSeriesRef(inner), T: int64(i), V: float64(i), }, @@ -521,7 +522,7 @@ func TestCheckpointSeriesReset(t *testing.T) { ref := i + 100 series := enc.Series([]record.RefSeries{ { - Ref: uint64(ref), + Ref: chunks.HeadSeriesRef(ref), Labels: labels.Labels{labels.Label{Name: "__name__", Value: fmt.Sprintf("metric_%d", i)}}, }, }, nil) @@ -531,7 +532,7 @@ func TestCheckpointSeriesReset(t *testing.T) { inner := rand.Intn(ref + 1) sample := enc.Samples([]record.RefSample{ { - Ref: uint64(inner), + Ref: chunks.HeadSeriesRef(inner), T: int64(i), V: float64(i), }, @@ -554,7 +555,7 @@ func TestCheckpointSeriesReset(t *testing.T) { }) require.Equal(t, seriesCount, wt.checkNumLabels()) - _, err = Checkpoint(log.NewNopLogger(), w, 2, 4, func(x uint64) bool { return true }, 0) + _, err = Checkpoint(log.NewNopLogger(), w, 2, 4, func(x chunks.HeadSeriesRef) bool { return true }, 0) require.NoError(t, err) err = w.Truncate(5) diff --git a/tsdb/wal_test.go b/tsdb/wal_test.go index ba8694b7a..21d0a8b14 100644 --- a/tsdb/wal_test.go +++ b/tsdb/wal_test.go @@ -31,6 +31,8 @@ import ( "github.com/stretchr/testify/require" "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/record" "github.com/prometheus/prometheus/tsdb/tombstones" "github.com/prometheus/prometheus/tsdb/wal" @@ -102,7 +104,7 @@ func TestSegmentWAL_Truncate(t *testing.T) { var rs []record.RefSeries for j, s := range series[i : i+batch] { - rs = append(rs, record.RefSeries{Labels: s, Ref: uint64(i+j) + 1}) + rs = append(rs, record.RefSeries{Labels: s, Ref: chunks.HeadSeriesRef(i+j) + 1}) } err := w.LogSeries(rs) require.NoError(t, err) @@ -117,11 +119,11 @@ func TestSegmentWAL_Truncate(t *testing.T) { boundarySeries := w.files[len(w.files)/2].minSeries // We truncate while keeping every 2nd series. - keep := map[uint64]struct{}{} + keep := map[chunks.HeadSeriesRef]struct{}{} for i := 1; i <= numMetrics; i += 2 { - keep[uint64(i)] = struct{}{} + keep[chunks.HeadSeriesRef(i)] = struct{}{} } - keepf := func(id uint64) bool { + keepf := func(id chunks.HeadSeriesRef) bool { _, ok := keep[id] return ok } @@ -132,8 +134,8 @@ func TestSegmentWAL_Truncate(t *testing.T) { var expected []record.RefSeries for i := 1; i <= numMetrics; i++ { - if i%2 == 1 || uint64(i) >= boundarySeries { - expected = append(expected, record.RefSeries{Ref: uint64(i), Labels: series[i-1]}) + if i%2 == 1 || chunks.HeadSeriesRef(i) >= boundarySeries { + expected = append(expected, record.RefSeries{Ref: chunks.HeadSeriesRef(i), Labels: series[i-1]}) } } @@ -238,7 +240,7 @@ func TestSegmentWAL_Log_Restore(t *testing.T) { for j := 0; j < i*10; j++ { samples = append(samples, record.RefSample{ - Ref: uint64(j % 10000), + Ref: chunks.HeadSeriesRef(j % 10000), T: int64(j * 2), V: rand.Float64(), }) @@ -246,14 +248,14 @@ func TestSegmentWAL_Log_Restore(t *testing.T) { for j := 0; j < i*20; j++ { ts := rand.Int63() - stones = append(stones, tombstones.Stone{Ref: rand.Uint64(), Intervals: tombstones.Intervals{{Mint: ts, Maxt: ts + rand.Int63n(10000)}}}) + stones = append(stones, tombstones.Stone{Ref: storage.SeriesRef(rand.Uint64()), Intervals: tombstones.Intervals{{Mint: ts, Maxt: ts + rand.Int63n(10000)}}}) } lbls := series[i : i+stepSize] series := make([]record.RefSeries, 0, len(series)) for j, l := range lbls { series = append(series, record.RefSeries{ - Ref: uint64(i + j), + Ref: chunks.HeadSeriesRef(i + j), Labels: l, }) } diff --git a/util/teststorage/storage.go b/util/teststorage/storage.go index 39aac596e..d4efcac43 100644 --- a/util/teststorage/storage.go +++ b/util/teststorage/storage.go @@ -70,6 +70,6 @@ func (s TestStorage) ExemplarQueryable() storage.ExemplarQueryable { return s.exemplarStorage } -func (s TestStorage) AppendExemplar(ref uint64, l labels.Labels, e exemplar.Exemplar) (uint64, error) { +func (s TestStorage) AppendExemplar(ref storage.SeriesRef, l labels.Labels, e exemplar.Exemplar) (storage.SeriesRef, error) { return ref, s.exemplarStorage.AddExemplar(l, e) }