From cda025b5b50328ccd129c141e09dd77b3b1b034e Mon Sep 17 00:00:00 2001 From: Dieter Plaetinck Date: Sat, 6 Nov 2021 12:10:04 +0200 Subject: [PATCH] TSDB: demistify SeriesRefs and ChunkRefs (#9536) * TSDB: demistify seriesRefs and ChunkRefs The TSDB package contains many types of series and chunk references, all shrouded in uint types. Often the same uint value may actually mean one of different types, in non-obvious ways. This PR aims to clarify the code and help navigating to relevant docs, usage, etc much quicker. Concretely: * Use appropriately named types and document their semantics and relations. * Make multiplexing and demuxing of types explicit (on the boundaries between concrete implementations and generic interfaces). * Casting between different types should be free. None of the changes should have any impact on how the code runs. TODO: Implement BlockSeriesRef where appropriate (for a future PR) Signed-off-by: Dieter Plaetinck * feedback Signed-off-by: Dieter Plaetinck * agent: demistify seriesRefs and ChunkRefs Signed-off-by: Dieter Plaetinck --- cmd/prometheus/main.go | 4 +- cmd/promtool/tsdb.go | 8 +- promql/bench_test.go | 11 +- scrape/helpers_test.go | 13 +- scrape/scrape.go | 8 +- scrape/scrape_test.go | 4 +- scrape/target.go | 4 +- storage/fanout.go | 4 +- storage/interface.go | 11 +- storage/remote/queue_manager.go | 15 +- storage/remote/queue_manager_test.go | 17 +-- storage/remote/write.go | 4 +- storage/remote/write_handler_test.go | 4 +- tsdb/agent/db.go | 30 ++-- tsdb/agent/series.go | 22 +-- tsdb/block.go | 21 +-- tsdb/block_test.go | 2 +- tsdb/chunks/chunks.go | 69 +++++++--- tsdb/chunks/head_chunks.go | 8 +- tsdb/chunks/head_chunks_test.go | 18 +-- tsdb/compact.go | 2 +- tsdb/db.go | 2 +- tsdb/db_test.go | 2 +- tsdb/docs/refs.md | 87 ++++++++++++ tsdb/head.go | 61 +++++---- tsdb/head_append.go | 32 ++--- tsdb/head_read.go | 42 ++---- tsdb/head_test.go | 38 +++--- tsdb/head_wal.go | 39 +++--- tsdb/index/index.go | 16 +-- tsdb/index/index_test.go | 27 ++-- tsdb/index/postings.go | 93 ++++++------- tsdb/index/postings_test.go | 197 ++++++++++++++------------- tsdb/mocks_test.go | 3 +- tsdb/querier.go | 6 +- tsdb/querier_test.go | 52 +++---- tsdb/record/record.go | 26 ++-- tsdb/tombstones/tombstones.go | 27 ++-- tsdb/tombstones/tombstones_test.go | 16 ++- tsdb/tsdbblockutil.go | 2 +- tsdb/wal.go | 20 +-- tsdb/wal/checkpoint.go | 3 +- tsdb/wal/checkpoint_test.go | 3 +- tsdb/wal/watcher_test.go | 41 +++--- tsdb/wal_test.go | 20 +-- util/teststorage/storage.go | 2 +- 46 files changed, 637 insertions(+), 499 deletions(-) create mode 100644 tsdb/docs/refs.md 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) }