mirror of https://github.com/prometheus/prometheus
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 <dieter@grafana.com> * feedback Signed-off-by: Dieter Plaetinck <dieter@grafana.com> * agent: demistify seriesRefs and ChunkRefs Signed-off-by: Dieter Plaetinck <dieter@grafana.com>pull/9219/head
parent
b882d2b7c7
commit
cda025b5b5
|
@ -1363,11 +1363,11 @@ func (s *readyStorage) Appender(ctx context.Context) storage.Appender {
|
||||||
|
|
||||||
type notReadyAppender struct{}
|
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
|
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
|
return 0, tsdb.ErrNotReady
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -31,6 +31,9 @@ import (
|
||||||
"text/tabwriter"
|
"text/tabwriter"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/prometheus/prometheus/storage"
|
||||||
|
"github.com/prometheus/prometheus/tsdb/index"
|
||||||
|
|
||||||
"github.com/alecthomas/units"
|
"github.com/alecthomas/units"
|
||||||
"github.com/go-kit/log"
|
"github.com/go-kit/log"
|
||||||
"github.com/pkg/errors"
|
"github.com/pkg/errors"
|
||||||
|
@ -40,7 +43,6 @@ import (
|
||||||
"github.com/prometheus/prometheus/tsdb/chunks"
|
"github.com/prometheus/prometheus/tsdb/chunks"
|
||||||
tsdb_errors "github.com/prometheus/prometheus/tsdb/errors"
|
tsdb_errors "github.com/prometheus/prometheus/tsdb/errors"
|
||||||
"github.com/prometheus/prometheus/tsdb/fileutil"
|
"github.com/prometheus/prometheus/tsdb/fileutil"
|
||||||
"github.com/prometheus/prometheus/tsdb/index"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
const timeDelta = 30000
|
const timeDelta = 30000
|
||||||
|
@ -187,7 +189,7 @@ func (b *writeBenchmark) ingestScrapesShard(lbls []labels.Labels, scrapeCount in
|
||||||
type sample struct {
|
type sample struct {
|
||||||
labels labels.Labels
|
labels labels.Labels
|
||||||
value int64
|
value int64
|
||||||
ref *uint64
|
ref *storage.SeriesRef
|
||||||
}
|
}
|
||||||
|
|
||||||
scrape := make([]*sample, 0, len(lbls))
|
scrape := make([]*sample, 0, len(lbls))
|
||||||
|
@ -207,7 +209,7 @@ func (b *writeBenchmark) ingestScrapesShard(lbls []labels.Labels, scrapeCount in
|
||||||
for _, s := range scrape {
|
for _, s := range scrape {
|
||||||
s.value += 1000
|
s.value += 1000
|
||||||
|
|
||||||
var ref uint64
|
var ref storage.SeriesRef
|
||||||
if s.ref != nil {
|
if s.ref != nil {
|
||||||
ref = *s.ref
|
ref = *s.ref
|
||||||
}
|
}
|
||||||
|
|
|
@ -23,12 +23,13 @@ import (
|
||||||
|
|
||||||
"github.com/prometheus/prometheus/pkg/labels"
|
"github.com/prometheus/prometheus/pkg/labels"
|
||||||
"github.com/prometheus/prometheus/promql/parser"
|
"github.com/prometheus/prometheus/promql/parser"
|
||||||
|
"github.com/prometheus/prometheus/storage"
|
||||||
"github.com/prometheus/prometheus/util/teststorage"
|
"github.com/prometheus/prometheus/util/teststorage"
|
||||||
)
|
)
|
||||||
|
|
||||||
func BenchmarkRangeQuery(b *testing.B) {
|
func BenchmarkRangeQuery(b *testing.B) {
|
||||||
storage := teststorage.New(b)
|
stor := teststorage.New(b)
|
||||||
defer storage.Close()
|
defer stor.Close()
|
||||||
opts := EngineOpts{
|
opts := EngineOpts{
|
||||||
Logger: nil,
|
Logger: nil,
|
||||||
Reg: 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"))
|
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.
|
// A day of data plus 10k steps.
|
||||||
numIntervals := 8640 + 10000
|
numIntervals := 8640 + 10000
|
||||||
|
|
||||||
for s := 0; s < numIntervals; s++ {
|
for s := 0; s < numIntervals; s++ {
|
||||||
a := storage.Appender(context.Background())
|
a := stor.Appender(context.Background())
|
||||||
ts := int64(s * 10000) // 10s interval.
|
ts := int64(s * 10000) // 10s interval.
|
||||||
for i, metric := range metrics {
|
for i, metric := range metrics {
|
||||||
ref, _ := a.Append(refs[i], metric, ts, float64(s)+float64(i)/float64(len(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()
|
b.ReportAllocs()
|
||||||
for i := 0; i < b.N; i++ {
|
for i := 0; i < b.N; i++ {
|
||||||
qry, err := engine.NewRangeQuery(
|
qry, err := engine.NewRangeQuery(
|
||||||
storage, c.expr,
|
stor, c.expr,
|
||||||
time.Unix(int64((numIntervals-c.steps)*10), 0),
|
time.Unix(int64((numIntervals-c.steps)*10), 0),
|
||||||
time.Unix(int64(numIntervals*10), 0), time.Second*10)
|
time.Unix(int64(numIntervals*10), 0), time.Second*10)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
|
|
@ -30,8 +30,11 @@ func (a nopAppendable) Appender(_ context.Context) storage.Appender {
|
||||||
|
|
||||||
type nopAppender struct{}
|
type nopAppender struct{}
|
||||||
|
|
||||||
func (a nopAppender) Append(uint64, labels.Labels, int64, float64) (uint64, error) { return 0, nil }
|
func (a nopAppender) Append(storage.SeriesRef, labels.Labels, int64, float64) (storage.SeriesRef, error) {
|
||||||
func (a nopAppender) AppendExemplar(uint64, labels.Labels, exemplar.Exemplar) (uint64, error) {
|
return 0, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (a nopAppender) AppendExemplar(storage.SeriesRef, labels.Labels, exemplar.Exemplar) (storage.SeriesRef, error) {
|
||||||
return 0, nil
|
return 0, nil
|
||||||
}
|
}
|
||||||
func (a nopAppender) Commit() error { return nil }
|
func (a nopAppender) Commit() error { return nil }
|
||||||
|
@ -54,7 +57,7 @@ type collectResultAppender struct {
|
||||||
resultExemplars []exemplar.Exemplar
|
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{
|
a.pendingResult = append(a.pendingResult, sample{
|
||||||
metric: lset,
|
metric: lset,
|
||||||
t: t,
|
t: t,
|
||||||
|
@ -62,7 +65,7 @@ func (a *collectResultAppender) Append(ref uint64, lset labels.Labels, t int64,
|
||||||
})
|
})
|
||||||
|
|
||||||
if ref == 0 {
|
if ref == 0 {
|
||||||
ref = rand.Uint64()
|
ref = storage.SeriesRef(rand.Uint64())
|
||||||
}
|
}
|
||||||
if a.next == nil {
|
if a.next == nil {
|
||||||
return ref, nil
|
return ref, nil
|
||||||
|
@ -75,7 +78,7 @@ func (a *collectResultAppender) Append(ref uint64, lset labels.Labels, t int64,
|
||||||
return ref, err
|
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)
|
a.pendingExemplars = append(a.pendingExemplars, e)
|
||||||
if a.next == nil {
|
if a.next == nil {
|
||||||
return 0, nil
|
return 0, nil
|
||||||
|
|
|
@ -829,7 +829,7 @@ type loop interface {
|
||||||
}
|
}
|
||||||
|
|
||||||
type cacheEntry struct {
|
type cacheEntry struct {
|
||||||
ref uint64
|
ref storage.SeriesRef
|
||||||
lastIter uint64
|
lastIter uint64
|
||||||
hash uint64
|
hash uint64
|
||||||
lset labels.Labels
|
lset labels.Labels
|
||||||
|
@ -976,7 +976,7 @@ func (c *scrapeCache) get(met string) (*cacheEntry, bool) {
|
||||||
return e, true
|
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 {
|
if ref == 0 {
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
@ -1477,7 +1477,7 @@ loop:
|
||||||
}
|
}
|
||||||
ce, ok := sl.cache.get(yoloString(met))
|
ce, ok := sl.cache.get(yoloString(met))
|
||||||
var (
|
var (
|
||||||
ref uint64
|
ref storage.SeriesRef
|
||||||
lset labels.Labels
|
lset labels.Labels
|
||||||
mets string
|
mets string
|
||||||
hash uint64
|
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 {
|
func (sl *scrapeLoop) addReportSample(app storage.Appender, s string, t int64, v float64) error {
|
||||||
ce, ok := sl.cache.get(s)
|
ce, ok := sl.cache.get(s)
|
||||||
var ref uint64
|
var ref storage.SeriesRef
|
||||||
var lset labels.Labels
|
var lset labels.Labels
|
||||||
if ok {
|
if ok {
|
||||||
ref = ce.ref
|
ref = ce.ref
|
||||||
|
|
|
@ -1475,7 +1475,7 @@ func TestScrapeLoopAppendCacheEntryButErrNotFound(t *testing.T) {
|
||||||
false,
|
false,
|
||||||
)
|
)
|
||||||
|
|
||||||
fakeRef := uint64(1)
|
fakeRef := storage.SeriesRef(1)
|
||||||
expValue := float64(1)
|
expValue := float64(1)
|
||||||
metric := `metric{n="1"} 1`
|
metric := `metric{n="1"} 1`
|
||||||
p := textparse.New([]byte(metric), "")
|
p := textparse.New([]byte(metric), "")
|
||||||
|
@ -1962,7 +1962,7 @@ type errorAppender struct {
|
||||||
collectResultAppender
|
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) {
|
switch lset.Get(model.MetricNameLabel) {
|
||||||
case "out_of_order":
|
case "out_of_order":
|
||||||
return 0, storage.ErrOutOfOrderSample
|
return 0, storage.ErrOutOfOrderSample
|
||||||
|
|
|
@ -316,7 +316,7 @@ type limitAppender struct {
|
||||||
i int
|
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) {
|
if !value.IsStaleNaN(v) {
|
||||||
app.i++
|
app.i++
|
||||||
if app.i > app.limit {
|
if app.i > app.limit {
|
||||||
|
@ -336,7 +336,7 @@ type timeLimitAppender struct {
|
||||||
maxTime int64
|
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 {
|
if t > app.maxTime {
|
||||||
return 0, storage.ErrOutOfBounds
|
return 0, storage.ErrOutOfBounds
|
||||||
}
|
}
|
||||||
|
|
|
@ -144,7 +144,7 @@ type fanoutAppender struct {
|
||||||
secondaries []Appender
|
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)
|
ref, err := f.primary.Append(ref, l, t, v)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return ref, err
|
return ref, err
|
||||||
|
@ -158,7 +158,7 @@ func (f *fanoutAppender) Append(ref uint64, l labels.Labels, t int64, v float64)
|
||||||
return ref, nil
|
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)
|
ref, err := f.primary.AppendExemplar(ref, l, e)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return ref, err
|
return ref, err
|
||||||
|
|
|
@ -36,6 +36,11 @@ var (
|
||||||
ErrExemplarsDisabled = fmt.Errorf("exemplar storage is disabled or max exemplars is less than or equal to 0")
|
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.
|
// Appendable allows creating appenders.
|
||||||
type Appendable interface {
|
type Appendable interface {
|
||||||
// Appender returns a new appender for the storage. The implementation
|
// 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
|
// to Append() at any point. Adding the sample via Append() returns a new
|
||||||
// reference number.
|
// reference number.
|
||||||
// If the reference is 0 it must not be used for caching.
|
// 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
|
// Commit submits the collected samples and purges the batch. If Commit
|
||||||
// returns a non-nil error, it also rolls back all modifications made in
|
// 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(),
|
// 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().
|
// 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.
|
// 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
|
// 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
|
// Note that in our current implementation of Prometheus' exemplar storage
|
||||||
// calls to Append should generate the reference numbers, AppendExemplar
|
// calls to Append should generate the reference numbers, AppendExemplar
|
||||||
// generating a new reference number should be considered possible erroneous behaviour and be logged.
|
// 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.
|
// SeriesSet contains a set of series.
|
||||||
|
|
|
@ -35,6 +35,7 @@ import (
|
||||||
"github.com/prometheus/prometheus/pkg/relabel"
|
"github.com/prometheus/prometheus/pkg/relabel"
|
||||||
"github.com/prometheus/prometheus/prompb"
|
"github.com/prometheus/prometheus/prompb"
|
||||||
"github.com/prometheus/prometheus/scrape"
|
"github.com/prometheus/prometheus/scrape"
|
||||||
|
"github.com/prometheus/prometheus/tsdb/chunks"
|
||||||
"github.com/prometheus/prometheus/tsdb/record"
|
"github.com/prometheus/prometheus/tsdb/record"
|
||||||
"github.com/prometheus/prometheus/tsdb/wal"
|
"github.com/prometheus/prometheus/tsdb/wal"
|
||||||
)
|
)
|
||||||
|
@ -353,11 +354,11 @@ type QueueManager struct {
|
||||||
storeClient WriteClient
|
storeClient WriteClient
|
||||||
|
|
||||||
seriesMtx sync.Mutex // Covers seriesLabels and droppedSeries.
|
seriesMtx sync.Mutex // Covers seriesLabels and droppedSeries.
|
||||||
seriesLabels map[uint64]labels.Labels
|
seriesLabels map[chunks.HeadSeriesRef]labels.Labels
|
||||||
droppedSeries map[uint64]struct{}
|
droppedSeries map[chunks.HeadSeriesRef]struct{}
|
||||||
|
|
||||||
seriesSegmentMtx sync.Mutex // Covers seriesSegmentIndexes - if you also lock seriesMtx, take seriesMtx first.
|
seriesSegmentMtx sync.Mutex // Covers seriesSegmentIndexes - if you also lock seriesMtx, take seriesMtx first.
|
||||||
seriesSegmentIndexes map[uint64]int
|
seriesSegmentIndexes map[chunks.HeadSeriesRef]int
|
||||||
|
|
||||||
shards *shards
|
shards *shards
|
||||||
numShards int
|
numShards int
|
||||||
|
@ -406,9 +407,9 @@ func NewQueueManager(
|
||||||
storeClient: client,
|
storeClient: client,
|
||||||
sendExemplars: enableExemplarRemoteWrite,
|
sendExemplars: enableExemplarRemoteWrite,
|
||||||
|
|
||||||
seriesLabels: make(map[uint64]labels.Labels),
|
seriesLabels: make(map[chunks.HeadSeriesRef]labels.Labels),
|
||||||
seriesSegmentIndexes: make(map[uint64]int),
|
seriesSegmentIndexes: make(map[chunks.HeadSeriesRef]int),
|
||||||
droppedSeries: make(map[uint64]struct{}),
|
droppedSeries: make(map[chunks.HeadSeriesRef]struct{}),
|
||||||
|
|
||||||
numShards: cfg.MinShards,
|
numShards: cfg.MinShards,
|
||||||
reshardChan: make(chan int),
|
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,
|
// 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.
|
// 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()
|
s.mtx.RLock()
|
||||||
defer s.mtx.RUnlock()
|
defer s.mtx.RUnlock()
|
||||||
|
|
||||||
|
|
|
@ -43,6 +43,7 @@ import (
|
||||||
"github.com/prometheus/prometheus/pkg/timestamp"
|
"github.com/prometheus/prometheus/pkg/timestamp"
|
||||||
"github.com/prometheus/prometheus/prompb"
|
"github.com/prometheus/prometheus/prompb"
|
||||||
"github.com/prometheus/prometheus/scrape"
|
"github.com/prometheus/prometheus/scrape"
|
||||||
|
"github.com/prometheus/prometheus/tsdb/chunks"
|
||||||
"github.com/prometheus/prometheus/tsdb/record"
|
"github.com/prometheus/prometheus/tsdb/record"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
@ -227,12 +228,12 @@ func TestSampleDeliveryOrder(t *testing.T) {
|
||||||
for i := 0; i < n; i++ {
|
for i := 0; i < n; i++ {
|
||||||
name := fmt.Sprintf("test_metric_%d", i%ts)
|
name := fmt.Sprintf("test_metric_%d", i%ts)
|
||||||
samples = append(samples, record.RefSample{
|
samples = append(samples, record.RefSample{
|
||||||
Ref: uint64(i),
|
Ref: chunks.HeadSeriesRef(i),
|
||||||
T: int64(i),
|
T: int64(i),
|
||||||
V: float64(i),
|
V: float64(i),
|
||||||
})
|
})
|
||||||
series = append(series, record.RefSeries{
|
series = append(series, record.RefSeries{
|
||||||
Ref: uint64(i),
|
Ref: chunks.HeadSeriesRef(i),
|
||||||
Labels: labels.Labels{labels.Label{Name: "__name__", Value: name}},
|
Labels: labels.Labels{labels.Label{Name: "__name__", Value: name}},
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
@ -320,7 +321,7 @@ func TestSeriesReset(t *testing.T) {
|
||||||
for i := 0; i < numSegments; i++ {
|
for i := 0; i < numSegments; i++ {
|
||||||
series := []record.RefSeries{}
|
series := []record.RefSeries{}
|
||||||
for j := 0; j < numSeries; j++ {
|
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)
|
m.StoreSeries(series, i)
|
||||||
}
|
}
|
||||||
|
@ -411,7 +412,7 @@ func TestReleaseNoninternedString(t *testing.T) {
|
||||||
for i := 1; i < 1000; i++ {
|
for i := 1; i < 1000; i++ {
|
||||||
m.StoreSeries([]record.RefSeries{
|
m.StoreSeries([]record.RefSeries{
|
||||||
{
|
{
|
||||||
Ref: uint64(i),
|
Ref: chunks.HeadSeriesRef(i),
|
||||||
Labels: labels.Labels{
|
Labels: labels.Labels{
|
||||||
labels.Label{
|
labels.Label{
|
||||||
Name: "asdf",
|
Name: "asdf",
|
||||||
|
@ -480,13 +481,13 @@ func createTimeseries(numSamples, numSeries int, extraLabels ...labels.Label) ([
|
||||||
name := fmt.Sprintf("test_metric_%d", i)
|
name := fmt.Sprintf("test_metric_%d", i)
|
||||||
for j := 0; j < numSamples; j++ {
|
for j := 0; j < numSamples; j++ {
|
||||||
samples = append(samples, record.RefSample{
|
samples = append(samples, record.RefSample{
|
||||||
Ref: uint64(i),
|
Ref: chunks.HeadSeriesRef(i),
|
||||||
T: int64(j),
|
T: int64(j),
|
||||||
V: float64(i),
|
V: float64(i),
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
series = append(series, record.RefSeries{
|
series = append(series, record.RefSeries{
|
||||||
Ref: uint64(i),
|
Ref: chunks.HeadSeriesRef(i),
|
||||||
Labels: append(labels.Labels{{Name: "__name__", Value: name}}, extraLabels...),
|
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)
|
name := fmt.Sprintf("test_metric_%d", i)
|
||||||
for j := 0; j < numExemplars; j++ {
|
for j := 0; j < numExemplars; j++ {
|
||||||
e := record.RefExemplar{
|
e := record.RefExemplar{
|
||||||
Ref: uint64(i),
|
Ref: chunks.HeadSeriesRef(i),
|
||||||
T: int64(j),
|
T: int64(j),
|
||||||
V: float64(i),
|
V: float64(i),
|
||||||
Labels: labels.FromStrings("traceID", fmt.Sprintf("trace-%d", 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)
|
exemplars = append(exemplars, e)
|
||||||
}
|
}
|
||||||
series = append(series, record.RefSeries{
|
series = append(series, record.RefSeries{
|
||||||
Ref: uint64(i),
|
Ref: chunks.HeadSeriesRef(i),
|
||||||
Labels: labels.Labels{{Name: "__name__", Value: name}},
|
Labels: labels.Labels{{Name: "__name__", Value: name}},
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
|
@ -255,7 +255,7 @@ type timestampTracker struct {
|
||||||
}
|
}
|
||||||
|
|
||||||
// Append implements storage.Appender.
|
// 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++
|
t.samples++
|
||||||
if ts > t.highestTimestamp {
|
if ts > t.highestTimestamp {
|
||||||
t.highestTimestamp = ts
|
t.highestTimestamp = ts
|
||||||
|
@ -263,7 +263,7 @@ func (t *timestampTracker) Append(_ uint64, _ labels.Labels, ts int64, _ float64
|
||||||
return 0, nil
|
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++
|
t.exemplars++
|
||||||
return 0, nil
|
return 0, nil
|
||||||
}
|
}
|
||||||
|
|
|
@ -159,7 +159,7 @@ func (m *mockAppendable) Appender(_ context.Context) storage.Appender {
|
||||||
return m
|
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 {
|
if t < m.latestSample {
|
||||||
return 0, storage.ErrOutOfOrderSample
|
return 0, storage.ErrOutOfOrderSample
|
||||||
}
|
}
|
||||||
|
@ -177,7 +177,7 @@ func (*mockAppendable) Rollback() error {
|
||||||
return fmt.Errorf("not implemented")
|
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 {
|
if e.Ts < m.latestExemplar {
|
||||||
return 0, storage.ErrOutOfOrderExemplar
|
return 0, storage.ErrOutOfOrderExemplar
|
||||||
}
|
}
|
||||||
|
|
|
@ -33,6 +33,7 @@ import (
|
||||||
"github.com/prometheus/prometheus/storage"
|
"github.com/prometheus/prometheus/storage"
|
||||||
"github.com/prometheus/prometheus/storage/remote"
|
"github.com/prometheus/prometheus/storage/remote"
|
||||||
"github.com/prometheus/prometheus/tsdb"
|
"github.com/prometheus/prometheus/tsdb"
|
||||||
|
"github.com/prometheus/prometheus/tsdb/chunks"
|
||||||
"github.com/prometheus/prometheus/tsdb/record"
|
"github.com/prometheus/prometheus/tsdb/record"
|
||||||
"github.com/prometheus/prometheus/tsdb/wal"
|
"github.com/prometheus/prometheus/tsdb/wal"
|
||||||
)
|
)
|
||||||
|
@ -195,7 +196,7 @@ type DB struct {
|
||||||
series *stripeSeries
|
series *stripeSeries
|
||||||
// deleted is a map of (ref IDs that should be deleted from WAL) to (the WAL segment they
|
// deleted is a map of (ref IDs that should be deleted from WAL) to (the WAL segment they
|
||||||
// must be kept around to).
|
// must be kept around to).
|
||||||
deleted map[uint64]int
|
deleted map[chunks.HeadSeriesRef]int
|
||||||
|
|
||||||
donec chan struct{}
|
donec chan struct{}
|
||||||
stopc 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),
|
nextRef: atomic.NewUint64(0),
|
||||||
series: newStripeSeries(opts.StripeSize),
|
series: newStripeSeries(opts.StripeSize),
|
||||||
deleted: make(map[uint64]int),
|
deleted: make(map[chunks.HeadSeriesRef]int),
|
||||||
|
|
||||||
donec: make(chan struct{}),
|
donec: make(chan struct{}),
|
||||||
stopc: make(chan struct{}),
|
stopc: make(chan struct{}),
|
||||||
|
@ -292,7 +293,7 @@ func (db *DB) replayWAL() error {
|
||||||
return errors.Wrap(err, "find last checkpoint")
|
return errors.Wrap(err, "find last checkpoint")
|
||||||
}
|
}
|
||||||
|
|
||||||
multiRef := map[uint64]uint64{}
|
multiRef := map[chunks.HeadSeriesRef]chunks.HeadSeriesRef{}
|
||||||
|
|
||||||
if err == nil {
|
if err == nil {
|
||||||
sr, err := wal.NewSegmentsReader(dir)
|
sr, err := wal.NewSegmentsReader(dir)
|
||||||
|
@ -344,10 +345,10 @@ func (db *DB) replayWAL() error {
|
||||||
return nil
|
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 (
|
var (
|
||||||
dec record.Decoder
|
dec record.Decoder
|
||||||
lastRef uint64
|
lastRef chunks.HeadSeriesRef
|
||||||
|
|
||||||
decoded = make(chan interface{}, 10)
|
decoded = make(chan interface{}, 10)
|
||||||
errCh = make(chan error, 1)
|
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)
|
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 {
|
select {
|
||||||
case err := <-errCh:
|
case err := <-errCh:
|
||||||
|
@ -538,7 +539,7 @@ func (db *DB) truncate(mint int64) error {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
keep := func(id uint64) bool {
|
keep := func(id chunks.HeadSeriesRef) bool {
|
||||||
if db.series.GetByID(id) != nil {
|
if db.series.GetByID(id) != nil {
|
||||||
return true
|
return true
|
||||||
}
|
}
|
||||||
|
@ -650,14 +651,15 @@ type appender struct {
|
||||||
pendingSamples []record.RefSample
|
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 {
|
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()
|
hash := l.Hash()
|
||||||
series := a.series.GetByHash(hash, l)
|
series := a.series.GetByHash(hash, l)
|
||||||
if series != nil {
|
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))
|
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}
|
series = &memSeries{ref: ref, lset: l, lastTs: t}
|
||||||
|
|
||||||
a.pendingSeries = append(a.pendingSeries, record.RefSeries{
|
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
|
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)
|
series := a.series.GetByID(ref)
|
||||||
if series == nil {
|
if series == nil {
|
||||||
return storage.ErrNotFound
|
return storage.ErrNotFound
|
||||||
|
@ -718,7 +720,7 @@ func (a *appender) AddFast(ref uint64, t int64, v float64) error {
|
||||||
return nil
|
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.
|
// remote_write doesn't support exemplars yet, so do nothing here.
|
||||||
return 0, nil
|
return 0, nil
|
||||||
}
|
}
|
||||||
|
|
|
@ -17,13 +17,14 @@ import (
|
||||||
"sync"
|
"sync"
|
||||||
|
|
||||||
"github.com/prometheus/prometheus/pkg/labels"
|
"github.com/prometheus/prometheus/pkg/labels"
|
||||||
|
"github.com/prometheus/prometheus/tsdb/chunks"
|
||||||
)
|
)
|
||||||
|
|
||||||
// memSeries is a chunkless version of tsdb.memSeries.
|
// memSeries is a chunkless version of tsdb.memSeries.
|
||||||
type memSeries struct {
|
type memSeries struct {
|
||||||
sync.Mutex
|
sync.Mutex
|
||||||
|
|
||||||
ref uint64
|
ref chunks.HeadSeriesRef
|
||||||
lset labels.Labels
|
lset labels.Labels
|
||||||
lastTs int64
|
lastTs int64
|
||||||
}
|
}
|
||||||
|
@ -54,7 +55,7 @@ func (m seriesHashmap) Set(hash uint64, s *memSeries) {
|
||||||
m[hash] = append(seriesSet, s)
|
m[hash] = append(seriesSet, s)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m seriesHashmap) Delete(hash, ref uint64) {
|
func (m seriesHashmap) Delete(hash uint64, ref chunks.HeadSeriesRef) {
|
||||||
var rem []*memSeries
|
var rem []*memSeries
|
||||||
for _, s := range m[hash] {
|
for _, s := range m[hash] {
|
||||||
if s.ref != ref {
|
if s.ref != ref {
|
||||||
|
@ -74,7 +75,7 @@ func (m seriesHashmap) Delete(hash, ref uint64) {
|
||||||
// likely due to the additional pointer dereferences.
|
// likely due to the additional pointer dereferences.
|
||||||
type stripeSeries struct {
|
type stripeSeries struct {
|
||||||
size int
|
size int
|
||||||
series []map[uint64]*memSeries
|
series []map[chunks.HeadSeriesRef]*memSeries
|
||||||
hashes []seriesHashmap
|
hashes []seriesHashmap
|
||||||
locks []stripeLock
|
locks []stripeLock
|
||||||
}
|
}
|
||||||
|
@ -88,12 +89,12 @@ type stripeLock struct {
|
||||||
func newStripeSeries(stripeSize int) *stripeSeries {
|
func newStripeSeries(stripeSize int) *stripeSeries {
|
||||||
s := &stripeSeries{
|
s := &stripeSeries{
|
||||||
size: stripeSize,
|
size: stripeSize,
|
||||||
series: make([]map[uint64]*memSeries, stripeSize),
|
series: make([]map[chunks.HeadSeriesRef]*memSeries, stripeSize),
|
||||||
hashes: make([]seriesHashmap, stripeSize),
|
hashes: make([]seriesHashmap, stripeSize),
|
||||||
locks: make([]stripeLock, stripeSize),
|
locks: make([]stripeLock, stripeSize),
|
||||||
}
|
}
|
||||||
for i := range s.series {
|
for i := range s.series {
|
||||||
s.series[i] = map[uint64]*memSeries{}
|
s.series[i] = map[chunks.HeadSeriesRef]*memSeries{}
|
||||||
}
|
}
|
||||||
for i := range s.hashes {
|
for i := range s.hashes {
|
||||||
s.hashes[i] = seriesHashmap{}
|
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
|
// GC garbage collects old series that have not received a sample after mint
|
||||||
// and will fully delete them.
|
// and will fully delete them.
|
||||||
func (s *stripeSeries) GC(mint int64) map[uint64]struct{} {
|
func (s *stripeSeries) GC(mint int64) map[chunks.HeadSeriesRef]struct{} {
|
||||||
deleted := map[uint64]struct{}{}
|
deleted := map[chunks.HeadSeriesRef]struct{}{}
|
||||||
|
|
||||||
for hashLock := 0; hashLock < s.size; hashLock++ {
|
for hashLock := 0; hashLock < s.size; hashLock++ {
|
||||||
s.locks[hashLock].Lock()
|
s.locks[hashLock].Lock()
|
||||||
|
@ -143,9 +144,8 @@ func (s *stripeSeries) GC(mint int64) map[uint64]struct{} {
|
||||||
return deleted
|
return deleted
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *stripeSeries) GetByID(id uint64) *memSeries {
|
func (s *stripeSeries) GetByID(id chunks.HeadSeriesRef) *memSeries {
|
||||||
refLock := id & uint64(s.size-1)
|
refLock := uint64(id) & uint64(s.size-1)
|
||||||
|
|
||||||
s.locks[refLock].RLock()
|
s.locks[refLock].RLock()
|
||||||
defer s.locks[refLock].RUnlock()
|
defer s.locks[refLock].RUnlock()
|
||||||
return s.series[refLock][id]
|
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) {
|
func (s *stripeSeries) Set(hash uint64, series *memSeries) {
|
||||||
var (
|
var (
|
||||||
hashLock = hash & uint64(s.size-1)
|
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()
|
s.locks[hashLock].Lock()
|
||||||
defer s.locks[hashLock].Unlock()
|
defer s.locks[hashLock].Unlock()
|
||||||
|
|
|
@ -29,6 +29,7 @@ import (
|
||||||
"github.com/pkg/errors"
|
"github.com/pkg/errors"
|
||||||
|
|
||||||
"github.com/prometheus/prometheus/pkg/labels"
|
"github.com/prometheus/prometheus/pkg/labels"
|
||||||
|
"github.com/prometheus/prometheus/storage"
|
||||||
"github.com/prometheus/prometheus/tsdb/chunkenc"
|
"github.com/prometheus/prometheus/tsdb/chunkenc"
|
||||||
"github.com/prometheus/prometheus/tsdb/chunks"
|
"github.com/prometheus/prometheus/tsdb/chunks"
|
||||||
tsdb_errors "github.com/prometheus/prometheus/tsdb/errors"
|
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
|
// 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
|
// their labels. The reference numbers are used to resolve entries in postings lists
|
||||||
// that are added later.
|
// 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
|
// Close writes any finalization and closes the resources associated with
|
||||||
// the underlying writer.
|
// the underlying writer.
|
||||||
|
@ -82,7 +83,7 @@ type IndexReader interface {
|
||||||
// Series populates the given labels and chunk metas for the series identified
|
// Series populates the given labels and chunk metas for the series identified
|
||||||
// by the reference.
|
// by the reference.
|
||||||
// Returns storage.ErrNotFound if the ref does not resolve to a known series.
|
// 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 returns all the unique label names present in the index in sorted order.
|
||||||
LabelNames(matchers ...*labels.Matcher) ([]string, error)
|
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.
|
// 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
|
// If the series couldn't be found or the series doesn't have the requested label a
|
||||||
// storage.ErrNotFound is returned as error.
|
// 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.
|
// LabelNamesFor returns all the label names for the series referred to by IDs.
|
||||||
// The names returned are sorted.
|
// 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 releases the underlying resources of the reader.
|
||||||
Close() error
|
Close() error
|
||||||
|
@ -116,7 +117,7 @@ type ChunkWriter interface {
|
||||||
// ChunkReader provides reading access of serialized time series data.
|
// ChunkReader provides reading access of serialized time series data.
|
||||||
type ChunkReader interface {
|
type ChunkReader interface {
|
||||||
// Chunk returns the series data chunk with the given reference.
|
// 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 releases all underlying resources of the reader.
|
||||||
Close() error
|
Close() error
|
||||||
|
@ -472,7 +473,7 @@ func (r blockIndexReader) SortedPostings(p index.Postings) index.Postings {
|
||||||
return r.ir.SortedPostings(p)
|
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 {
|
if err := r.ir.Series(ref, lset, chks); err != nil {
|
||||||
return errors.Wrapf(err, "block: %s", r.b.Meta().ULID)
|
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.
|
// 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)
|
return r.ir.LabelValueFor(id, label)
|
||||||
}
|
}
|
||||||
|
|
||||||
// LabelNamesFor returns all the label names for the series referred to by IDs.
|
// LabelNamesFor returns all the label names for the series referred to by IDs.
|
||||||
// The names returned are sorted.
|
// 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...)
|
return r.ir.LabelNamesFor(ids...)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -558,7 +559,7 @@ Outer:
|
||||||
return p.Err()
|
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 {
|
for _, iv := range ivs {
|
||||||
stones.AddInterval(id, iv)
|
stones.AddInterval(id, iv)
|
||||||
}
|
}
|
||||||
|
@ -590,7 +591,7 @@ Outer:
|
||||||
func (pb *Block) CleanTombstones(dest string, c Compactor) (*ulid.ULID, bool, error) {
|
func (pb *Block) CleanTombstones(dest string, c Compactor) (*ulid.ULID, bool, error) {
|
||||||
numStones := 0
|
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)
|
numStones += len(ivs)
|
||||||
return nil
|
return nil
|
||||||
}); err != nil {
|
}); err != nil {
|
||||||
|
|
|
@ -522,7 +522,7 @@ func createHead(tb testing.TB, w *wal.WAL, series []storage.Series, chunkDir str
|
||||||
|
|
||||||
app := head.Appender(context.Background())
|
app := head.Appender(context.Background())
|
||||||
for _, s := range series {
|
for _, s := range series {
|
||||||
ref := uint64(0)
|
ref := storage.SeriesRef(0)
|
||||||
it := s.Iterator()
|
it := s.Iterator()
|
||||||
lset := s.Labels()
|
lset := s.Labels()
|
||||||
for it.Next() {
|
for it.Next() {
|
||||||
|
|
|
@ -54,13 +54,54 @@ const (
|
||||||
ChunkEncodingSize = 1
|
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.
|
// Meta holds information about a chunk of data.
|
||||||
type Meta struct {
|
type Meta struct {
|
||||||
// Ref and Chunk hold either a reference that can be used to retrieve
|
// Ref and Chunk hold either a reference that can be used to retrieve
|
||||||
// chunk data or the data itself.
|
// chunk data or the data itself.
|
||||||
// When it is a reference it is the segment offset at which the chunk bytes start.
|
// If Chunk is nil, call ChunkReader.Chunk(Meta.Ref) to get the chunk and assign it to the Chunk field
|
||||||
// Generally, only one of them is set.
|
Ref ChunkRef
|
||||||
Ref uint64
|
|
||||||
Chunk chunkenc.Chunk
|
Chunk chunkenc.Chunk
|
||||||
|
|
||||||
// Time range the data covers.
|
// Time range the data covers.
|
||||||
|
@ -353,16 +394,11 @@ func (w *Writer) writeChunks(chks []Meta) error {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
seq := uint64(w.seq()) << 32
|
seq := uint64(w.seq())
|
||||||
for i := range chks {
|
for i := range chks {
|
||||||
chk := &chks[i]
|
chk := &chks[i]
|
||||||
|
|
||||||
// The reference is set to the segment index and the offset where
|
chk.Ref = ChunkRef(NewBlockChunkRef(seq, uint64(w.n)))
|
||||||
// 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)
|
|
||||||
|
|
||||||
n := binary.PutUvarint(w.buf[:], uint64(len(chk.Chunk.Bytes())))
|
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.
|
// Chunk returns a chunk from a given reference.
|
||||||
func (s *Reader) Chunk(ref uint64) (chunkenc.Chunk, error) {
|
func (s *Reader) Chunk(ref ChunkRef) (chunkenc.Chunk, error) {
|
||||||
var (
|
sgmIndex, chkStart := BlockChunkRef(ref).Unpack()
|
||||||
// Get the upper 4 bytes.
|
chkCRC32 := newCRC32()
|
||||||
// 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()
|
|
||||||
)
|
|
||||||
|
|
||||||
if sgmIndex >= len(s.bs) {
|
if sgmIndex >= len(s.bs) {
|
||||||
return nil, errors.Errorf("segment index %d out of range", sgmIndex)
|
return nil, errors.Errorf("segment index %d out of range", sgmIndex)
|
||||||
|
|
|
@ -285,7 +285,7 @@ func repairLastChunkFile(files map[int]string) (_ map[int]string, returnErr erro
|
||||||
|
|
||||||
// WriteChunk writes the chunk to the disk.
|
// WriteChunk writes the chunk to the disk.
|
||||||
// The returned chunk ref is the reference from where the chunk encoding starts for the chunk.
|
// 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()
|
cdm.writePathMtx.Lock()
|
||||||
defer cdm.writePathMtx.Unlock()
|
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()))
|
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
|
bytesWritten += SeriesRefSize
|
||||||
binary.BigEndian.PutUint64(cdm.byteBuf[bytesWritten:], uint64(mint))
|
binary.BigEndian.PutUint64(cdm.byteBuf[bytesWritten:], uint64(mint))
|
||||||
bytesWritten += MintMaxtSize
|
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.
|
// 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
|
// NOTE: This method needs to be called at least once after creating ChunkDiskMapper
|
||||||
// to set the maxt of all the file.
|
// 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()
|
cdm.writePathMtx.Lock()
|
||||||
defer cdm.writePathMtx.Unlock()
|
defer cdm.writePathMtx.Unlock()
|
||||||
|
|
||||||
|
@ -627,7 +627,7 @@ func (cdm *ChunkDiskMapper) IterateAllChunks(f func(seriesRef uint64, chunkRef C
|
||||||
chunkRef := newChunkDiskMapperRef(uint64(segID), uint64(idx))
|
chunkRef := newChunkDiskMapperRef(uint64(segID), uint64(idx))
|
||||||
|
|
||||||
startIdx := 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
|
idx += SeriesRefSize
|
||||||
mint := int64(binary.BigEndian.Uint64(mmapFile.byteSlice.Range(idx, idx+MintMaxtSize)))
|
mint := int64(binary.BigEndian.Uint64(mmapFile.byteSlice.Range(idx, idx+MintMaxtSize)))
|
||||||
idx += MintMaxtSize
|
idx += MintMaxtSize
|
||||||
|
|
|
@ -38,7 +38,7 @@ func TestChunkDiskMapper_WriteChunk_Chunk_IterateChunks(t *testing.T) {
|
||||||
chkCRC32 := newCRC32()
|
chkCRC32 := newCRC32()
|
||||||
|
|
||||||
type expectedDataType struct {
|
type expectedDataType struct {
|
||||||
seriesRef uint64
|
seriesRef HeadSeriesRef
|
||||||
chunkRef ChunkDiskMapperRef
|
chunkRef ChunkDiskMapperRef
|
||||||
mint, maxt int64
|
mint, maxt int64
|
||||||
numSamples uint16
|
numSamples uint16
|
||||||
|
@ -75,7 +75,7 @@ func TestChunkDiskMapper_WriteChunk_Chunk_IterateChunks(t *testing.T) {
|
||||||
bytesWritten := 0
|
bytesWritten := 0
|
||||||
chkCRC32.Reset()
|
chkCRC32.Reset()
|
||||||
|
|
||||||
binary.BigEndian.PutUint64(buf[bytesWritten:], seriesRef)
|
binary.BigEndian.PutUint64(buf[bytesWritten:], uint64(seriesRef))
|
||||||
bytesWritten += SeriesRefSize
|
bytesWritten += SeriesRefSize
|
||||||
binary.BigEndian.PutUint64(buf[bytesWritten:], uint64(mint))
|
binary.BigEndian.PutUint64(buf[bytesWritten:], uint64(mint))
|
||||||
bytesWritten += MintMaxtSize
|
bytesWritten += MintMaxtSize
|
||||||
|
@ -133,7 +133,7 @@ func TestChunkDiskMapper_WriteChunk_Chunk_IterateChunks(t *testing.T) {
|
||||||
require.NoError(t, err)
|
require.NoError(t, err)
|
||||||
|
|
||||||
idx := 0
|
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()
|
t.Helper()
|
||||||
|
|
||||||
expData := expectedData[idx]
|
expData := expectedData[idx]
|
||||||
|
@ -221,7 +221,7 @@ func TestChunkDiskMapper_Truncate(t *testing.T) {
|
||||||
require.NoError(t, err)
|
require.NoError(t, err)
|
||||||
|
|
||||||
require.False(t, hrw.fileMaxtSet)
|
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)
|
require.True(t, hrw.fileMaxtSet)
|
||||||
|
|
||||||
verifyFiles([]int{3, 4, 5, 6, 7, 8})
|
verifyFiles([]int{3, 4, 5, 6, 7, 8})
|
||||||
|
@ -335,7 +335,7 @@ func TestHeadReadWriter_TruncateAfterFailedIterateChunks(t *testing.T) {
|
||||||
require.NoError(t, err)
|
require.NoError(t, err)
|
||||||
|
|
||||||
// Forcefully failing IterateAllChunks.
|
// 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")
|
return errors.New("random error")
|
||||||
}))
|
}))
|
||||||
|
|
||||||
|
@ -391,7 +391,7 @@ func TestHeadReadWriter_ReadRepairOnEmptyLastFile(t *testing.T) {
|
||||||
hrw, err = NewChunkDiskMapper(dir, chunkenc.NewPool(), DefaultWriteBufferSize)
|
hrw, err = NewChunkDiskMapper(dir, chunkenc.NewPool(), DefaultWriteBufferSize)
|
||||||
require.NoError(t, err)
|
require.NoError(t, err)
|
||||||
require.False(t, hrw.fileMaxtSet)
|
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)
|
require.True(t, hrw.fileMaxtSet)
|
||||||
|
|
||||||
// Removed from memory.
|
// Removed from memory.
|
||||||
|
@ -421,7 +421,7 @@ func testChunkDiskMapper(t *testing.T) *ChunkDiskMapper {
|
||||||
hrw, err := NewChunkDiskMapper(tmpdir, chunkenc.NewPool(), DefaultWriteBufferSize)
|
hrw, err := NewChunkDiskMapper(tmpdir, chunkenc.NewPool(), DefaultWriteBufferSize)
|
||||||
require.NoError(t, err)
|
require.NoError(t, err)
|
||||||
require.False(t, hrw.fileMaxtSet)
|
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)
|
require.True(t, hrw.fileMaxtSet)
|
||||||
return hrw
|
return hrw
|
||||||
}
|
}
|
||||||
|
@ -437,9 +437,9 @@ func randomChunk(t *testing.T) chunkenc.Chunk {
|
||||||
return 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
|
var err error
|
||||||
seriesRef = uint64(rand.Int63())
|
seriesRef = HeadSeriesRef(rand.Int63())
|
||||||
mint = int64((idx)*1000 + 1)
|
mint = int64((idx)*1000 + 1)
|
||||||
maxt = int64((idx + 1) * 1000)
|
maxt = int64((idx + 1) * 1000)
|
||||||
chunk = randomChunk(t)
|
chunk = randomChunk(t)
|
||||||
|
|
|
@ -745,7 +745,7 @@ func (c *LeveledCompactor) populateBlock(blocks []BlockReader, meta *BlockMeta,
|
||||||
}
|
}
|
||||||
|
|
||||||
var (
|
var (
|
||||||
ref = uint64(0)
|
ref = storage.SeriesRef(0)
|
||||||
chks []chunks.Meta
|
chks []chunks.Meta
|
||||||
)
|
)
|
||||||
|
|
||||||
|
|
|
@ -858,7 +858,7 @@ type dbAppender struct {
|
||||||
|
|
||||||
var _ storage.GetRef = dbAppender{}
|
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 {
|
if g, ok := a.Appender.(storage.GetRef); ok {
|
||||||
return g.GetRef(lset)
|
return g.GetRef(lset)
|
||||||
}
|
}
|
||||||
|
|
|
@ -1388,7 +1388,7 @@ func TestSizeRetention(t *testing.T) {
|
||||||
// Create a WAL checkpoint, and compare sizes.
|
// Create a WAL checkpoint, and compare sizes.
|
||||||
first, last, err := wal.Segments(db.Head().wal.Dir())
|
first, last, err := wal.Segments(db.Head().wal.Dir())
|
||||||
require.NoError(t, err)
|
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)
|
require.NoError(t, err)
|
||||||
blockSize = int64(prom_testutil.ToFloat64(db.metrics.blocksBytes)) // Use the actual internal metrics.
|
blockSize = int64(prom_testutil.ToFloat64(db.metrics.blocksBytes)) // Use the actual internal metrics.
|
||||||
walSize, err = db.Head().wal.Size()
|
walSize, err = db.Head().wal.Size()
|
||||||
|
|
|
@ -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`.
|
61
tsdb/head.go
61
tsdb/head.go
|
@ -80,7 +80,7 @@ type Head struct {
|
||||||
series *stripeSeries
|
series *stripeSeries
|
||||||
|
|
||||||
deletedMtx sync.Mutex
|
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.
|
postings *index.MemPostings // Postings lists for terms.
|
||||||
|
|
||||||
|
@ -236,7 +236,7 @@ func (h *Head) resetInMemoryState() error {
|
||||||
h.postings = index.NewUnorderedMemPostings()
|
h.postings = index.NewUnorderedMemPostings()
|
||||||
h.tombstones = tombstones.NewMemTombstones()
|
h.tombstones = tombstones.NewMemTombstones()
|
||||||
h.iso = newIsolation()
|
h.iso = newIsolation()
|
||||||
h.deleted = map[uint64]int{}
|
h.deleted = map[chunks.HeadSeriesRef]int{}
|
||||||
h.chunkRange.Store(h.opts.ChunkRange)
|
h.chunkRange.Store(h.opts.ChunkRange)
|
||||||
h.minTime.Store(math.MaxInt64)
|
h.minTime.Store(math.MaxInt64)
|
||||||
h.maxTime.Store(math.MinInt64)
|
h.maxTime.Store(math.MinInt64)
|
||||||
|
@ -480,7 +480,7 @@ func (h *Head) Init(minValidTime int64) error {
|
||||||
start := time.Now()
|
start := time.Now()
|
||||||
|
|
||||||
snapIdx, snapOffset := -1, 0
|
snapIdx, snapOffset := -1, 0
|
||||||
refSeries := make(map[uint64]*memSeries)
|
refSeries := make(map[chunks.HeadSeriesRef]*memSeries)
|
||||||
|
|
||||||
if h.opts.EnableMemorySnapshotOnShutdown {
|
if h.opts.EnableMemorySnapshotOnShutdown {
|
||||||
level.Info(h.logger).Log("msg", "Chunk snapshot is enabled, replaying from the snapshot")
|
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)
|
h.startWALReplayStatus(startFrom, endAt)
|
||||||
|
|
||||||
multiRef := map[uint64]uint64{}
|
multiRef := map[chunks.HeadSeriesRef]chunks.HeadSeriesRef{}
|
||||||
if err == nil && startFrom >= snapIdx {
|
if err == nil && startFrom >= snapIdx {
|
||||||
sr, err := wal.NewSegmentsReader(dir)
|
sr, err := wal.NewSegmentsReader(dir)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
@ -603,9 +603,9 @@ func (h *Head) Init(minValidTime int64) error {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (h *Head) loadMmappedChunks(refSeries map[uint64]*memSeries) (map[uint64][]*mmappedChunk, error) {
|
func (h *Head) loadMmappedChunks(refSeries map[chunks.HeadSeriesRef]*memSeries) (map[chunks.HeadSeriesRef][]*mmappedChunk, error) {
|
||||||
mmappedChunks := map[uint64][]*mmappedChunk{}
|
mmappedChunks := map[chunks.HeadSeriesRef][]*mmappedChunk{}
|
||||||
if err := h.chunkDiskMapper.IterateAllChunks(func(seriesRef uint64, chunkRef chunks.ChunkDiskMapperRef, mint, maxt int64, numSamples uint16) error {
|
if err := h.chunkDiskMapper.IterateAllChunks(func(seriesRef chunks.HeadSeriesRef, chunkRef chunks.ChunkDiskMapperRef, mint, maxt int64, numSamples uint16) error {
|
||||||
if maxt < h.minValidTime.Load() {
|
if maxt < h.minValidTime.Load() {
|
||||||
return nil
|
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
|
// removeCorruptedMmappedChunks attempts to delete the corrupted mmapped chunks and if it fails, it clears all the previously
|
||||||
// loaded mmapped chunks.
|
// 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")
|
level.Info(h.logger).Log("msg", "Deleting mmapped chunk files")
|
||||||
|
|
||||||
if err := h.chunkDiskMapper.DeleteCorrupted(err); err != nil {
|
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)
|
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")
|
level.Info(h.logger).Log("msg", "Deletion of mmap chunk files successful, reattempting m-mapping the on-disk chunks")
|
||||||
mmappedChunks, err := h.loadMmappedChunks(refSeries)
|
mmappedChunks, err := h.loadMmappedChunks(refSeries)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
level.Error(h.logger).Log("msg", "Loading on-disk chunks failed, discarding chunk files completely", "err", err)
|
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
|
return mmappedChunks
|
||||||
|
@ -925,7 +925,7 @@ func (h *Head) truncateWAL(mint int64) error {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
keep := func(id uint64) bool {
|
keep := func(id chunks.HeadSeriesRef) bool {
|
||||||
if h.series.getByID(id) != nil {
|
if h.series.getByID(id) != nil {
|
||||||
return true
|
return true
|
||||||
}
|
}
|
||||||
|
@ -1072,7 +1072,7 @@ func (h *Head) Delete(mint, maxt int64, ms ...*labels.Matcher) error {
|
||||||
|
|
||||||
var stones []tombstones.Stone
|
var stones []tombstones.Stone
|
||||||
for p.Next() {
|
for p.Next() {
|
||||||
series := h.series.getByID(p.At())
|
series := h.series.getByID(chunks.HeadSeriesRef(p.At()))
|
||||||
|
|
||||||
series.RLock()
|
series.RLock()
|
||||||
t0, t1 := series.minTime(), series.maxTime()
|
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 {
|
for _, s := range stones {
|
||||||
h.tombstones.AddInterval(s.Ref, s.Intervals[0])
|
h.tombstones.AddInterval(storage.SeriesRef(s.Ref), s.Intervals[0])
|
||||||
}
|
}
|
||||||
|
|
||||||
return nil
|
return nil
|
||||||
|
@ -1133,7 +1133,7 @@ func (h *Head) gc() int64 {
|
||||||
// that reads the WAL, wouldn't be able to use those
|
// that reads the WAL, wouldn't be able to use those
|
||||||
// samples since we would have no labels for that ref ID.
|
// samples since we would have no labels for that ref ID.
|
||||||
for ref := range deleted {
|
for ref := range deleted {
|
||||||
h.deleted[ref] = last
|
h.deleted[chunks.HeadSeriesRef(ref)] = last
|
||||||
}
|
}
|
||||||
h.deletedMtx.Unlock()
|
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.
|
// 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)
|
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 {
|
s, created, err := h.series.getOrSet(hash, lset, func() *memSeries {
|
||||||
return newMemSeries(lset, id, h.chunkRange.Load(), &h.memChunkPool)
|
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.metrics.seriesCreated.Inc()
|
||||||
h.numSeries.Inc()
|
h.numSeries.Inc()
|
||||||
|
|
||||||
h.postings.Add(id, lset)
|
h.postings.Add(storage.SeriesRef(id), lset)
|
||||||
return s, true, nil
|
return s, true, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1290,7 +1290,7 @@ const (
|
||||||
// dereferences.
|
// dereferences.
|
||||||
type stripeSeries struct {
|
type stripeSeries struct {
|
||||||
size int
|
size int
|
||||||
series []map[uint64]*memSeries
|
series []map[chunks.HeadSeriesRef]*memSeries
|
||||||
hashes []seriesHashmap
|
hashes []seriesHashmap
|
||||||
locks []stripeLock
|
locks []stripeLock
|
||||||
seriesLifecycleCallback SeriesLifecycleCallback
|
seriesLifecycleCallback SeriesLifecycleCallback
|
||||||
|
@ -1305,14 +1305,14 @@ type stripeLock struct {
|
||||||
func newStripeSeries(stripeSize int, seriesCallback SeriesLifecycleCallback) *stripeSeries {
|
func newStripeSeries(stripeSize int, seriesCallback SeriesLifecycleCallback) *stripeSeries {
|
||||||
s := &stripeSeries{
|
s := &stripeSeries{
|
||||||
size: stripeSize,
|
size: stripeSize,
|
||||||
series: make([]map[uint64]*memSeries, stripeSize),
|
series: make([]map[chunks.HeadSeriesRef]*memSeries, stripeSize),
|
||||||
hashes: make([]seriesHashmap, stripeSize),
|
hashes: make([]seriesHashmap, stripeSize),
|
||||||
locks: make([]stripeLock, stripeSize),
|
locks: make([]stripeLock, stripeSize),
|
||||||
seriesLifecycleCallback: seriesCallback,
|
seriesLifecycleCallback: seriesCallback,
|
||||||
}
|
}
|
||||||
|
|
||||||
for i := range s.series {
|
for i := range s.series {
|
||||||
s.series[i] = map[uint64]*memSeries{}
|
s.series[i] = map[chunks.HeadSeriesRef]*memSeries{}
|
||||||
}
|
}
|
||||||
for i := range s.hashes {
|
for i := range s.hashes {
|
||||||
s.hashes[i] = seriesHashmap{}
|
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
|
// gc garbage collects old chunks that are strictly before mint and removes
|
||||||
// series entirely that have no chunks left.
|
// 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 (
|
var (
|
||||||
deleted = map[uint64]struct{}{}
|
deleted = map[storage.SeriesRef]struct{}{}
|
||||||
deletedForCallback = []labels.Labels{}
|
deletedForCallback = []labels.Labels{}
|
||||||
rmChunks = 0
|
rmChunks = 0
|
||||||
actualMint int64 = math.MaxInt64
|
actualMint int64 = math.MaxInt64
|
||||||
|
@ -1359,7 +1362,7 @@ func (s *stripeSeries) gc(mint int64) (map[uint64]struct{}, int, int64) {
|
||||||
s.locks[j].Lock()
|
s.locks[j].Lock()
|
||||||
}
|
}
|
||||||
|
|
||||||
deleted[series.ref] = struct{}{}
|
deleted[storage.SeriesRef(series.ref)] = struct{}{}
|
||||||
s.hashes[i].del(hash, series.lset)
|
s.hashes[i].del(hash, series.lset)
|
||||||
delete(s.series[j], series.ref)
|
delete(s.series[j], series.ref)
|
||||||
deletedForCallback = append(deletedForCallback, series.lset)
|
deletedForCallback = append(deletedForCallback, series.lset)
|
||||||
|
@ -1385,8 +1388,8 @@ func (s *stripeSeries) gc(mint int64) (map[uint64]struct{}, int, int64) {
|
||||||
return deleted, rmChunks, actualMint
|
return deleted, rmChunks, actualMint
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *stripeSeries) getByID(id uint64) *memSeries {
|
func (s *stripeSeries) getByID(id chunks.HeadSeriesRef) *memSeries {
|
||||||
i := id & uint64(s.size-1)
|
i := uint64(id) & uint64(s.size-1)
|
||||||
|
|
||||||
s.locks[i].RLock()
|
s.locks[i].RLock()
|
||||||
series := s.series[i][id]
|
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.
|
// as any further calls to this methods would return that series.
|
||||||
s.seriesLifecycleCallback.PostCreation(series.lset)
|
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.locks[i].Lock()
|
||||||
s.series[i][series.ref] = series
|
s.series[i][series.ref] = series
|
||||||
|
@ -1461,7 +1464,7 @@ func (s sample) V() float64 { return s.v }
|
||||||
type memSeries struct {
|
type memSeries struct {
|
||||||
sync.RWMutex
|
sync.RWMutex
|
||||||
|
|
||||||
ref uint64
|
ref chunks.HeadSeriesRef
|
||||||
lset labels.Labels
|
lset labels.Labels
|
||||||
mmappedChunks []*mmappedChunk
|
mmappedChunks []*mmappedChunk
|
||||||
mmMaxTime int64 // Max time of any mmapped chunk, only used during WAL replay.
|
mmMaxTime int64 // Max time of any mmapped chunk, only used during WAL replay.
|
||||||
|
@ -1480,7 +1483,7 @@ type memSeries struct {
|
||||||
txs *txRing
|
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{
|
s := &memSeries{
|
||||||
lset: lset,
|
lset: lset,
|
||||||
ref: id,
|
ref: id,
|
||||||
|
@ -1562,7 +1565,7 @@ func overlapsClosedInterval(mint1, maxt1, mint2, maxt2 int64) bool {
|
||||||
return mint1 <= maxt2 && mint2 <= maxt1
|
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 {
|
type mmappedChunk struct {
|
||||||
ref chunks.ChunkDiskMapperRef
|
ref chunks.ChunkDiskMapperRef
|
||||||
numSamples uint16
|
numSamples uint16
|
||||||
|
|
|
@ -38,7 +38,7 @@ type initAppender struct {
|
||||||
|
|
||||||
var _ storage.GetRef = &initAppender{}
|
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 {
|
if a.app != nil {
|
||||||
return a.app.Append(ref, lset, t, v)
|
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)
|
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.
|
// Check if exemplar storage is enabled.
|
||||||
if !a.head.opts.EnableExemplarStorage || a.head.opts.MaxExemplars.Load() <= 0 {
|
if !a.head.opts.EnableExemplarStorage || a.head.opts.MaxExemplars.Load() <= 0 {
|
||||||
return 0, nil
|
return 0, nil
|
||||||
|
@ -76,7 +76,7 @@ func (h *Head) initTime(t int64) {
|
||||||
h.maxTime.CAS(math.MinInt64, t)
|
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 {
|
if g, ok := a.app.(storage.GetRef); ok {
|
||||||
return g.GetRef(lset)
|
return g.GetRef(lset)
|
||||||
}
|
}
|
||||||
|
@ -215,7 +215,7 @@ func (h *Head) putBytesBuffer(b []byte) {
|
||||||
}
|
}
|
||||||
|
|
||||||
type exemplarWithSeriesRef struct {
|
type exemplarWithSeriesRef struct {
|
||||||
ref uint64
|
ref storage.SeriesRef
|
||||||
exemplar exemplar.Exemplar
|
exemplar exemplar.Exemplar
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -233,13 +233,13 @@ type headAppender struct {
|
||||||
closed bool
|
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 {
|
if t < a.minValidTime {
|
||||||
a.head.metrics.outOfBoundSamples.Inc()
|
a.head.metrics.outOfBoundSamples.Inc()
|
||||||
return 0, storage.ErrOutOfBounds
|
return 0, storage.ErrOutOfBounds
|
||||||
}
|
}
|
||||||
|
|
||||||
s := a.head.series.getByID(ref)
|
s := a.head.series.getByID(chunks.HeadSeriesRef(ref))
|
||||||
if s == nil {
|
if s == nil {
|
||||||
// Ensure no empty labels have gotten through.
|
// Ensure no empty labels have gotten through.
|
||||||
lset = lset.WithoutEmpty()
|
lset = lset.WithoutEmpty()
|
||||||
|
@ -289,7 +289,7 @@ func (a *headAppender) Append(ref uint64, lset labels.Labels, t int64, v float64
|
||||||
V: v,
|
V: v,
|
||||||
})
|
})
|
||||||
a.sampleSeries = append(a.sampleSeries, s)
|
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.
|
// 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
|
// 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.
|
// 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.
|
// Check if exemplar storage is enabled.
|
||||||
if !a.head.opts.EnableExemplarStorage || a.head.opts.MaxExemplars.Load() <= 0 {
|
if !a.head.opts.EnableExemplarStorage || a.head.opts.MaxExemplars.Load() <= 0 {
|
||||||
return 0, nil
|
return 0, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// Get Series
|
// Get Series
|
||||||
s := a.head.series.getByID(ref)
|
s := a.head.series.getByID(chunks.HeadSeriesRef(ref))
|
||||||
if s == nil {
|
if s == nil {
|
||||||
s = a.head.series.getByHash(lset.Hash(), lset)
|
s = a.head.series.getByHash(lset.Hash(), lset)
|
||||||
if s != nil {
|
if s != nil {
|
||||||
ref = s.ref
|
ref = storage.SeriesRef(s.ref)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if s == nil {
|
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.
|
// 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})
|
a.exemplars = append(a.exemplars, exemplarWithSeriesRef{ref, e})
|
||||||
|
|
||||||
return s.ref, nil
|
return storage.SeriesRef(s.ref), nil
|
||||||
}
|
}
|
||||||
|
|
||||||
var _ storage.GetRef = &headAppender{}
|
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)
|
s := a.head.series.getByHash(lset.Hash(), lset)
|
||||||
if s == nil {
|
if s == nil {
|
||||||
return 0, nil
|
return 0, nil
|
||||||
}
|
}
|
||||||
// returned labels must be suitable to pass to Append()
|
// 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 {
|
func (a *headAppender) log() error {
|
||||||
|
@ -403,7 +403,7 @@ func exemplarsForEncoding(es []exemplarWithSeriesRef) []record.RefExemplar {
|
||||||
ret := make([]record.RefExemplar, 0, len(es))
|
ret := make([]record.RefExemplar, 0, len(es))
|
||||||
for _, e := range es {
|
for _, e := range es {
|
||||||
ret = append(ret, record.RefExemplar{
|
ret = append(ret, record.RefExemplar{
|
||||||
Ref: e.ref,
|
Ref: chunks.HeadSeriesRef(e.ref),
|
||||||
T: e.exemplar.Ts,
|
T: e.exemplar.Ts,
|
||||||
V: e.exemplar.Value,
|
V: e.exemplar.Value,
|
||||||
Labels: e.exemplar.Labels,
|
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.
|
// No errors logging to WAL, so pass the exemplars along to the in memory storage.
|
||||||
for _, e := range a.exemplars {
|
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.
|
// 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 := a.head.exemplars.AddExemplar(s.lset, e.exemplar); err != nil {
|
||||||
if err == storage.ErrOutOfOrderExemplar {
|
if err == storage.ErrOutOfOrderExemplar {
|
||||||
|
|
|
@ -115,7 +115,7 @@ func (h *headIndexReader) SortedPostings(p index.Postings) index.Postings {
|
||||||
|
|
||||||
// Fetch all the series only once.
|
// Fetch all the series only once.
|
||||||
for p.Next() {
|
for p.Next() {
|
||||||
s := h.head.series.getByID(p.At())
|
s := h.head.series.getByID(chunks.HeadSeriesRef(p.At()))
|
||||||
if s == nil {
|
if s == nil {
|
||||||
level.Debug(h.head.logger).Log("msg", "Looked up series not found")
|
level.Debug(h.head.logger).Log("msg", "Looked up series not found")
|
||||||
} else {
|
} else {
|
||||||
|
@ -131,16 +131,16 @@ func (h *headIndexReader) SortedPostings(p index.Postings) index.Postings {
|
||||||
})
|
})
|
||||||
|
|
||||||
// Convert back to list.
|
// Convert back to list.
|
||||||
ep := make([]uint64, 0, len(series))
|
ep := make([]storage.SeriesRef, 0, len(series))
|
||||||
for _, p := range series {
|
for _, p := range series {
|
||||||
ep = append(ep, p.ref)
|
ep = append(ep, storage.SeriesRef(p.ref))
|
||||||
}
|
}
|
||||||
return index.NewListPostings(ep)
|
return index.NewListPostings(ep)
|
||||||
}
|
}
|
||||||
|
|
||||||
// Series returns the series for the given reference.
|
// Series returns the series for the given reference.
|
||||||
func (h *headIndexReader) Series(ref uint64, lbls *labels.Labels, chks *[]chunks.Meta) error {
|
func (h *headIndexReader) Series(ref storage.SeriesRef, lbls *labels.Labels, chks *[]chunks.Meta) error {
|
||||||
s := h.head.series.getByID(ref)
|
s := h.head.series.getByID(chunks.HeadSeriesRef(ref))
|
||||||
|
|
||||||
if s == nil {
|
if s == nil {
|
||||||
h.head.metrics.seriesNotFound.Inc()
|
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{
|
*chks = append(*chks, chunks.Meta{
|
||||||
MinTime: c.minTime,
|
MinTime: c.minTime,
|
||||||
MaxTime: c.maxTime,
|
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) {
|
if s.headChunk != nil && s.headChunk.OverlapsClosedInterval(h.mint, h.maxt) {
|
||||||
*chks = append(*chks, chunks.Meta{
|
*chks = append(*chks, chunks.Meta{
|
||||||
MinTime: s.headChunk.minTime,
|
MinTime: s.headChunk.minTime,
|
||||||
MaxTime: math.MaxInt64, // Set the head chunks as open (being appended to).
|
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.
|
// 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) {
|
func (h *headIndexReader) LabelValueFor(id storage.SeriesRef, label string) (string, error) {
|
||||||
memSeries := h.head.series.getByID(id)
|
memSeries := h.head.series.getByID(chunks.HeadSeriesRef(id))
|
||||||
if memSeries == nil {
|
if memSeries == nil {
|
||||||
return "", storage.ErrNotFound
|
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.
|
// LabelNamesFor returns all the label names for the series referred to by IDs.
|
||||||
// The names returned are sorted.
|
// 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{})
|
namesMap := make(map[string]struct{})
|
||||||
for _, id := range ids {
|
for _, id := range ids {
|
||||||
memSeries := h.head.series.getByID(id)
|
memSeries := h.head.series.getByID(chunks.HeadSeriesRef(id))
|
||||||
if memSeries == nil {
|
if memSeries == nil {
|
||||||
return nil, storage.ErrNotFound
|
return nil, storage.ErrNotFound
|
||||||
}
|
}
|
||||||
|
@ -248,25 +248,9 @@ func (h *headChunkReader) Close() error {
|
||||||
return nil
|
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.
|
// Chunk returns the chunk for the reference number.
|
||||||
func (h *headChunkReader) Chunk(ref uint64) (chunkenc.Chunk, error) {
|
func (h *headChunkReader) Chunk(ref chunks.ChunkRef) (chunkenc.Chunk, error) {
|
||||||
sid, cid := unpackChunkID(ref)
|
sid, cid := chunks.HeadChunkRef(ref).Unpack()
|
||||||
|
|
||||||
s := h.head.series.getByID(sid)
|
s := h.head.series.getByID(sid)
|
||||||
// This means that the series has been garbage collected.
|
// This means that the series has been garbage collected.
|
||||||
|
|
|
@ -63,7 +63,7 @@ func newTestHead(t testing.TB, chunkRange int64, compressWAL bool) (*Head, *wal.
|
||||||
h, err := NewHead(nil, nil, wlog, opts, nil)
|
h, err := NewHead(nil, nil, wlog, opts, nil)
|
||||||
require.NoError(t, err)
|
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() {
|
t.Cleanup(func() {
|
||||||
require.NoError(t, os.RemoveAll(dir))
|
require.NoError(t, os.RemoveAll(dir))
|
||||||
|
@ -200,7 +200,7 @@ func BenchmarkLoadWAL(b *testing.B) {
|
||||||
for j := 1; len(lbls) < labelsPerSeries; j++ {
|
for j := 1; len(lbls) < labelsPerSeries; j++ {
|
||||||
lbls[defaultLabelName+strconv.Itoa(j)] = defaultLabelValue + strconv.Itoa(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})
|
populateTestWAL(b, w, []interface{}{refSeries})
|
||||||
}
|
}
|
||||||
|
@ -212,7 +212,7 @@ func BenchmarkLoadWAL(b *testing.B) {
|
||||||
refSamples = refSamples[:0]
|
refSamples = refSamples[:0]
|
||||||
for k := j * c.seriesPerBatch; k < (j+1)*c.seriesPerBatch; k++ {
|
for k := j * c.seriesPerBatch; k < (j+1)*c.seriesPerBatch; k++ {
|
||||||
refSamples = append(refSamples, record.RefSample{
|
refSamples = append(refSamples, record.RefSample{
|
||||||
Ref: uint64(k) * 101,
|
Ref: chunks.HeadSeriesRef(k) * 101,
|
||||||
T: int64(i) * 10,
|
T: int64(i) * 10,
|
||||||
V: float64(i) * 100,
|
V: float64(i) * 100,
|
||||||
})
|
})
|
||||||
|
@ -227,7 +227,7 @@ func BenchmarkLoadWAL(b *testing.B) {
|
||||||
require.NoError(b, err)
|
require.NoError(b, err)
|
||||||
for k := 0; k < c.batches*c.seriesPerBatch; k++ {
|
for k := 0; k < c.batches*c.seriesPerBatch; k++ {
|
||||||
// Create one mmapped chunk per series, with one sample at the given time.
|
// 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.append(c.mmappedChunkT, 42, 0, chunkDiskMapper)
|
||||||
s.mmapCurrentHeadChunk(chunkDiskMapper)
|
s.mmapCurrentHeadChunk(chunkDiskMapper)
|
||||||
}
|
}
|
||||||
|
@ -241,7 +241,7 @@ func BenchmarkLoadWAL(b *testing.B) {
|
||||||
refExemplars = refExemplars[:0]
|
refExemplars = refExemplars[:0]
|
||||||
for k := j * c.seriesPerBatch; k < (j+1)*c.seriesPerBatch; k++ {
|
for k := j * c.seriesPerBatch; k < (j+1)*c.seriesPerBatch; k++ {
|
||||||
refExemplars = append(refExemplars, record.RefExemplar{
|
refExemplars = append(refExemplars, record.RefExemplar{
|
||||||
Ref: uint64(k) * 101,
|
Ref: chunks.HeadSeriesRef(k) * 101,
|
||||||
T: int64(i) * 10,
|
T: int64(i) * 10,
|
||||||
V: float64(i) * 100,
|
V: float64(i) * 100,
|
||||||
Labels: labels.FromStrings("traceID", fmt.Sprintf("trace-%d", i)),
|
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"))
|
postingsC1, _ := index.ExpandPostings(h.postings.Get("c", "1"))
|
||||||
postingsAll, _ := index.ExpandPostings(h.postings.Get("", ""))
|
postingsAll, _ := index.ExpandPostings(h.postings.Get("", ""))
|
||||||
|
|
||||||
require.Equal(t, []uint64{s1.ref}, postingsA1)
|
require.Equal(t, []storage.SeriesRef{storage.SeriesRef(s1.ref)}, postingsA1)
|
||||||
require.Equal(t, []uint64{s2.ref}, postingsA2)
|
require.Equal(t, []storage.SeriesRef{storage.SeriesRef(s2.ref)}, postingsA2)
|
||||||
require.Equal(t, []uint64{s1.ref, s2.ref}, postingsB1)
|
require.Equal(t, []storage.SeriesRef{storage.SeriesRef(s1.ref), storage.SeriesRef(s2.ref)}, postingsB1)
|
||||||
require.Equal(t, []uint64{s1.ref, s2.ref}, postingsAll)
|
require.Equal(t, []storage.SeriesRef{storage.SeriesRef(s1.ref), storage.SeriesRef(s2.ref)}, postingsAll)
|
||||||
require.Nil(t, postingsB2)
|
require.Nil(t, postingsB2)
|
||||||
require.Nil(t, postingsC1)
|
require.Nil(t, postingsC1)
|
||||||
|
|
||||||
|
@ -2317,7 +2317,7 @@ func TestChunkNotFoundHeadGCRace(t *testing.T) {
|
||||||
|
|
||||||
var (
|
var (
|
||||||
app = db.Appender(context.Background())
|
app = db.Appender(context.Background())
|
||||||
ref = uint64(0)
|
ref = storage.SeriesRef(0)
|
||||||
mint, maxt = int64(0), int64(0)
|
mint, maxt = int64(0), int64(0)
|
||||||
err error
|
err error
|
||||||
)
|
)
|
||||||
|
@ -2382,7 +2382,7 @@ func TestDataMissingOnQueryDuringCompaction(t *testing.T) {
|
||||||
|
|
||||||
var (
|
var (
|
||||||
app = db.Appender(context.Background())
|
app = db.Appender(context.Background())
|
||||||
ref = uint64(0)
|
ref = storage.SeriesRef(0)
|
||||||
mint, maxt = int64(0), int64(0)
|
mint, maxt = int64(0), int64(0)
|
||||||
err error
|
err error
|
||||||
)
|
)
|
||||||
|
@ -2429,7 +2429,7 @@ func TestIsQuerierCollidingWithTruncation(t *testing.T) {
|
||||||
|
|
||||||
var (
|
var (
|
||||||
app = db.Appender(context.Background())
|
app = db.Appender(context.Background())
|
||||||
ref = uint64(0)
|
ref = storage.SeriesRef(0)
|
||||||
err error
|
err error
|
||||||
)
|
)
|
||||||
|
|
||||||
|
@ -2476,7 +2476,7 @@ func TestWaitForPendingReadersInTimeRange(t *testing.T) {
|
||||||
|
|
||||||
var (
|
var (
|
||||||
app = db.Appender(context.Background())
|
app = db.Appender(context.Background())
|
||||||
ref = uint64(0)
|
ref = storage.SeriesRef(0)
|
||||||
err error
|
err error
|
||||||
)
|
)
|
||||||
|
|
||||||
|
@ -2539,10 +2539,10 @@ func TestChunkSnapshot(t *testing.T) {
|
||||||
|
|
||||||
numSeries := 10
|
numSeries := 10
|
||||||
expSeries := make(map[string][]tsdbutil.Sample)
|
expSeries := make(map[string][]tsdbutil.Sample)
|
||||||
expTombstones := make(map[uint64]tombstones.Intervals)
|
expTombstones := make(map[storage.SeriesRef]tombstones.Intervals)
|
||||||
expExemplars := make([]ex, 0)
|
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{
|
e := ex{
|
||||||
seriesLabels: lbls,
|
seriesLabels: lbls,
|
||||||
e: exemplar.Exemplar{
|
e: exemplar.Exemplar{
|
||||||
|
@ -2565,8 +2565,8 @@ func TestChunkSnapshot(t *testing.T) {
|
||||||
checkTombstones := func() {
|
checkTombstones := func() {
|
||||||
tr, err := head.Tombstones()
|
tr, err := head.Tombstones()
|
||||||
require.NoError(t, err)
|
require.NoError(t, err)
|
||||||
actTombstones := make(map[uint64]tombstones.Intervals)
|
actTombstones := make(map[storage.SeriesRef]tombstones.Intervals)
|
||||||
require.NoError(t, tr.Iter(func(ref uint64, itvs tombstones.Intervals) error {
|
require.NoError(t, tr.Iter(func(ref storage.SeriesRef, itvs tombstones.Intervals) error {
|
||||||
for _, itv := range itvs {
|
for _, itv := range itvs {
|
||||||
actTombstones[ref].Add(itv)
|
actTombstones[ref].Add(itv)
|
||||||
}
|
}
|
||||||
|
@ -2640,7 +2640,7 @@ func TestChunkSnapshot(t *testing.T) {
|
||||||
// Add some tombstones.
|
// Add some tombstones.
|
||||||
var enc record.Encoder
|
var enc record.Encoder
|
||||||
for i := 1; i <= numSeries; i++ {
|
for i := 1; i <= numSeries; i++ {
|
||||||
ref := uint64(i)
|
ref := storage.SeriesRef(i)
|
||||||
itvs := tombstones.Intervals{
|
itvs := tombstones.Intervals{
|
||||||
{Mint: 1234, Maxt: 2345},
|
{Mint: 1234, Maxt: 2345},
|
||||||
{Mint: 3456, Maxt: 4567},
|
{Mint: 3456, Maxt: 4567},
|
||||||
|
@ -2701,7 +2701,7 @@ func TestChunkSnapshot(t *testing.T) {
|
||||||
// Add more tombstones.
|
// Add more tombstones.
|
||||||
var enc record.Encoder
|
var enc record.Encoder
|
||||||
for i := 1; i <= numSeries; i++ {
|
for i := 1; i <= numSeries; i++ {
|
||||||
ref := uint64(i)
|
ref := storage.SeriesRef(i)
|
||||||
itvs := tombstones.Intervals{
|
itvs := tombstones.Intervals{
|
||||||
{Mint: 12345, Maxt: 23456},
|
{Mint: 12345, Maxt: 23456},
|
||||||
{Mint: 34567, Maxt: 45678},
|
{Mint: 34567, Maxt: 45678},
|
||||||
|
|
|
@ -33,6 +33,7 @@ import (
|
||||||
"github.com/prometheus/prometheus/pkg/labels"
|
"github.com/prometheus/prometheus/pkg/labels"
|
||||||
"github.com/prometheus/prometheus/storage"
|
"github.com/prometheus/prometheus/storage"
|
||||||
"github.com/prometheus/prometheus/tsdb/chunkenc"
|
"github.com/prometheus/prometheus/tsdb/chunkenc"
|
||||||
|
"github.com/prometheus/prometheus/tsdb/chunks"
|
||||||
"github.com/prometheus/prometheus/tsdb/encoding"
|
"github.com/prometheus/prometheus/tsdb/encoding"
|
||||||
tsdb_errors "github.com/prometheus/prometheus/tsdb/errors"
|
tsdb_errors "github.com/prometheus/prometheus/tsdb/errors"
|
||||||
"github.com/prometheus/prometheus/tsdb/fileutil"
|
"github.com/prometheus/prometheus/tsdb/fileutil"
|
||||||
|
@ -41,7 +42,7 @@ import (
|
||||||
"github.com/prometheus/prometheus/tsdb/wal"
|
"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
|
// Track number of samples that referenced a series we don't know about
|
||||||
// for error reporting.
|
// for error reporting.
|
||||||
var unknownRefs atomic.Uint64
|
var unknownRefs atomic.Uint64
|
||||||
|
@ -207,8 +208,8 @@ Outer:
|
||||||
break Outer
|
break Outer
|
||||||
}
|
}
|
||||||
|
|
||||||
if h.lastSeriesID.Load() < walSeries.Ref {
|
if chunks.HeadSeriesRef(h.lastSeriesID.Load()) < walSeries.Ref {
|
||||||
h.lastSeriesID.Store(walSeries.Ref)
|
h.lastSeriesID.Store(uint64(walSeries.Ref))
|
||||||
}
|
}
|
||||||
|
|
||||||
mmc := mmappedChunks[walSeries.Ref]
|
mmc := mmappedChunks[walSeries.Ref]
|
||||||
|
@ -225,7 +226,7 @@ Outer:
|
||||||
|
|
||||||
multiRef[walSeries.Ref] = mSeries.ref
|
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
|
// 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
|
// 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.
|
// processing all old samples after emptying the buffer.
|
||||||
|
@ -297,7 +298,7 @@ Outer:
|
||||||
if r, ok := multiRef[sam.Ref]; ok {
|
if r, ok := multiRef[sam.Ref]; ok {
|
||||||
sam.Ref = r
|
sam.Ref = r
|
||||||
}
|
}
|
||||||
mod := sam.Ref % uint64(n)
|
mod := uint64(sam.Ref) % uint64(n)
|
||||||
shards[mod] = append(shards[mod], sam)
|
shards[mod] = append(shards[mod], sam)
|
||||||
}
|
}
|
||||||
for i := 0; i < n; i++ {
|
for i := 0; i < n; i++ {
|
||||||
|
@ -313,11 +314,11 @@ Outer:
|
||||||
if itv.Maxt < h.minValidTime.Load() {
|
if itv.Maxt < h.minValidTime.Load() {
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
if m := h.series.getByID(s.Ref); m == nil {
|
if m := h.series.getByID(chunks.HeadSeriesRef(s.Ref)); m == nil {
|
||||||
unknownRefs.Inc()
|
unknownRefs.Inc()
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
h.tombstones.AddInterval(s.Ref, itv)
|
h.tombstones.AddInterval(storage.SeriesRef(s.Ref), itv)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
//nolint:staticcheck // Ignore SA6002 relax staticcheck verification.
|
//nolint:staticcheck // Ignore SA6002 relax staticcheck verification.
|
||||||
|
@ -428,7 +429,7 @@ const (
|
||||||
)
|
)
|
||||||
|
|
||||||
type chunkSnapshotRecord struct {
|
type chunkSnapshotRecord struct {
|
||||||
ref uint64
|
ref chunks.HeadSeriesRef
|
||||||
lset labels.Labels
|
lset labels.Labels
|
||||||
chunkRange int64
|
chunkRange int64
|
||||||
mc *memChunk
|
mc *memChunk
|
||||||
|
@ -439,7 +440,7 @@ func (s *memSeries) encodeToSnapshotRecord(b []byte) []byte {
|
||||||
buf := encoding.Encbuf{B: b}
|
buf := encoding.Encbuf{B: b}
|
||||||
|
|
||||||
buf.PutByte(chunkSnapshotRecordTypeSeries)
|
buf.PutByte(chunkSnapshotRecordTypeSeries)
|
||||||
buf.PutBE64(s.ref)
|
buf.PutBE64(uint64(s.ref))
|
||||||
buf.PutUvarint(len(s.lset))
|
buf.PutUvarint(len(s.lset))
|
||||||
for _, l := range s.lset {
|
for _, l := range s.lset {
|
||||||
buf.PutUvarintStr(l.Name)
|
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)
|
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.
|
// The label set written to the disk is already sorted.
|
||||||
csr.lset = make(labels.Labels, dec.Uvarint())
|
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,
|
// 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.
|
// 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)
|
dir, snapIdx, snapOffset, err := LastChunkSnapshot(h.opts.ChunkDirRoot)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
if err == record.ErrNotFound {
|
if err == record.ErrNotFound {
|
||||||
|
@ -842,9 +843,9 @@ func (h *Head) loadChunkSnapshot() (int, int, map[uint64]*memSeries, error) {
|
||||||
n = runtime.GOMAXPROCS(0)
|
n = runtime.GOMAXPROCS(0)
|
||||||
wg sync.WaitGroup
|
wg sync.WaitGroup
|
||||||
recordChan = make(chan chunkSnapshotRecord, 5*n)
|
recordChan = make(chan chunkSnapshotRecord, 5*n)
|
||||||
shardedRefSeries = make([]map[uint64]*memSeries, n)
|
shardedRefSeries = make([]map[chunks.HeadSeriesRef]*memSeries, n)
|
||||||
errChan = make(chan error, n)
|
errChan = make(chan error, n)
|
||||||
refSeries map[uint64]*memSeries
|
refSeries map[chunks.HeadSeriesRef]*memSeries
|
||||||
exemplarBuf []record.RefExemplar
|
exemplarBuf []record.RefExemplar
|
||||||
dec record.Decoder
|
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]
|
localRefSeries := shardedRefSeries[idx]
|
||||||
|
|
||||||
for csr := range rc {
|
for csr := range rc {
|
||||||
|
@ -870,8 +871,8 @@ func (h *Head) loadChunkSnapshot() (int, int, map[uint64]*memSeries, error) {
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
localRefSeries[csr.ref] = series
|
localRefSeries[csr.ref] = series
|
||||||
if h.lastSeriesID.Load() < series.ref {
|
if chunks.HeadSeriesRef(h.lastSeriesID.Load()) < series.ref {
|
||||||
h.lastSeriesID.Store(series.ref)
|
h.lastSeriesID.Store(uint64(series.ref))
|
||||||
}
|
}
|
||||||
|
|
||||||
series.chunkRange = csr.chunkRange
|
series.chunkRange = csr.chunkRange
|
||||||
|
@ -926,7 +927,7 @@ Outer:
|
||||||
break 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...)
|
h.tombstones.AddInterval(ref, ivs...)
|
||||||
return nil
|
return nil
|
||||||
}); err != nil {
|
}); err != nil {
|
||||||
|
@ -940,7 +941,7 @@ Outer:
|
||||||
close(recordChan)
|
close(recordChan)
|
||||||
wg.Wait()
|
wg.Wait()
|
||||||
|
|
||||||
refSeries = make(map[uint64]*memSeries, numSeries)
|
refSeries = make(map[chunks.HeadSeriesRef]*memSeries, numSeries)
|
||||||
for _, shard := range shardedRefSeries {
|
for _, shard := range shardedRefSeries {
|
||||||
for k, v := range shard {
|
for k, v := range shard {
|
||||||
refSeries[k] = v
|
refSeries[k] = v
|
||||||
|
@ -1006,7 +1007,7 @@ Outer:
|
||||||
|
|
||||||
if len(refSeries) == 0 {
|
if len(refSeries) == 0 {
|
||||||
// We had no exemplar record, so we have to build the map here.
|
// 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 _, shard := range shardedRefSeries {
|
||||||
for k, v := range shard {
|
for k, v := range shard {
|
||||||
refSeries[k] = v
|
refSeries[k] = v
|
||||||
|
|
|
@ -143,7 +143,7 @@ type Writer struct {
|
||||||
|
|
||||||
// Hold last series to validate that clients insert new series in order.
|
// Hold last series to validate that clients insert new series in order.
|
||||||
lastSeries labels.Labels
|
lastSeries labels.Labels
|
||||||
lastRef uint64
|
lastRef storage.SeriesRef
|
||||||
|
|
||||||
crc32 hash.Hash
|
crc32 hash.Hash
|
||||||
|
|
||||||
|
@ -414,7 +414,7 @@ func (w *Writer) writeMeta() error {
|
||||||
}
|
}
|
||||||
|
|
||||||
// AddSeries adds the series one at a time along with its chunks.
|
// 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 {
|
if err := w.ensureStage(idxStageSeries); err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
@ -472,7 +472,7 @@ func (w *Writer) AddSeries(ref uint64, lset labels.Labels, chunks ...chunks.Meta
|
||||||
c := chunks[0]
|
c := chunks[0]
|
||||||
w.buf2.PutVarint64(c.MinTime)
|
w.buf2.PutVarint64(c.MinTime)
|
||||||
w.buf2.PutUvarint64(uint64(c.MaxTime - c.MinTime))
|
w.buf2.PutUvarint64(uint64(c.MaxTime - c.MinTime))
|
||||||
w.buf2.PutUvarint64(c.Ref)
|
w.buf2.PutUvarint64(uint64(c.Ref))
|
||||||
t0 := c.MaxTime
|
t0 := c.MaxTime
|
||||||
ref0 := int64(c.Ref)
|
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.
|
// LabelNamesFor returns all the label names for the series referred to by IDs.
|
||||||
// The names returned are sorted.
|
// 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
|
// Gather offsetsMap the name offsetsMap in the symbol table first
|
||||||
offsetsMap := make(map[uint32]struct{})
|
offsetsMap := make(map[uint32]struct{})
|
||||||
for _, id := range ids {
|
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.
|
// 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
|
offset := id
|
||||||
// In version 2 series IDs are no longer exact references but series are 16-byte padded
|
// 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.
|
// 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.
|
// 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
|
offset := id
|
||||||
// In version 2 series IDs are no longer exact references but series are 16-byte padded
|
// 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.
|
// 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())
|
ref0 := int64(d.Uvarint64())
|
||||||
|
|
||||||
*chks = append(*chks, chunks.Meta{
|
*chks = append(*chks, chunks.Meta{
|
||||||
Ref: uint64(ref0),
|
Ref: chunks.ChunkRef(ref0),
|
||||||
MinTime: t0,
|
MinTime: t0,
|
||||||
MaxTime: maxt,
|
MaxTime: maxt,
|
||||||
})
|
})
|
||||||
|
@ -1877,7 +1877,7 @@ func (dec *Decoder) Series(b []byte, lbls *labels.Labels, chks *[]chunks.Meta) e
|
||||||
}
|
}
|
||||||
|
|
||||||
*chks = append(*chks, chunks.Meta{
|
*chks = append(*chks, chunks.Meta{
|
||||||
Ref: uint64(ref0),
|
Ref: chunks.ChunkRef(ref0),
|
||||||
MinTime: mint,
|
MinTime: mint,
|
||||||
MaxTime: maxt,
|
MaxTime: maxt,
|
||||||
})
|
})
|
||||||
|
|
|
@ -29,6 +29,7 @@ import (
|
||||||
"go.uber.org/goleak"
|
"go.uber.org/goleak"
|
||||||
|
|
||||||
"github.com/prometheus/prometheus/pkg/labels"
|
"github.com/prometheus/prometheus/pkg/labels"
|
||||||
|
"github.com/prometheus/prometheus/storage"
|
||||||
"github.com/prometheus/prometheus/tsdb/chunkenc"
|
"github.com/prometheus/prometheus/tsdb/chunkenc"
|
||||||
"github.com/prometheus/prometheus/tsdb/chunks"
|
"github.com/prometheus/prometheus/tsdb/chunks"
|
||||||
"github.com/prometheus/prometheus/tsdb/encoding"
|
"github.com/prometheus/prometheus/tsdb/encoding"
|
||||||
|
@ -45,18 +46,18 @@ type series struct {
|
||||||
}
|
}
|
||||||
|
|
||||||
type mockIndex struct {
|
type mockIndex struct {
|
||||||
series map[uint64]series
|
series map[storage.SeriesRef]series
|
||||||
postings map[labels.Label][]uint64
|
postings map[labels.Label][]storage.SeriesRef
|
||||||
symbols map[string]struct{}
|
symbols map[string]struct{}
|
||||||
}
|
}
|
||||||
|
|
||||||
func newMockIndex() mockIndex {
|
func newMockIndex() mockIndex {
|
||||||
ix := mockIndex{
|
ix := mockIndex{
|
||||||
series: make(map[uint64]series),
|
series: make(map[storage.SeriesRef]series),
|
||||||
postings: make(map[labels.Label][]uint64),
|
postings: make(map[labels.Label][]storage.SeriesRef),
|
||||||
symbols: make(map[string]struct{}),
|
symbols: make(map[string]struct{}),
|
||||||
}
|
}
|
||||||
ix.postings[allPostingsKey] = []uint64{}
|
ix.postings[allPostingsKey] = []storage.SeriesRef{}
|
||||||
return ix
|
return ix
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -64,7 +65,7 @@ func (m mockIndex) Symbols() (map[string]struct{}, error) {
|
||||||
return m.symbols, nil
|
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 {
|
if _, ok := m.series[ref]; ok {
|
||||||
return errors.Errorf("series with reference %d already added", ref)
|
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.Name] = struct{}{}
|
||||||
m.symbols[lbl.Value] = struct{}{}
|
m.symbols[lbl.Value] = struct{}{}
|
||||||
if _, ok := m.postings[lbl]; !ok {
|
if _, ok := m.postings[lbl]; !ok {
|
||||||
m.postings[lbl] = []uint64{}
|
m.postings[lbl] = []storage.SeriesRef{}
|
||||||
}
|
}
|
||||||
m.postings[lbl] = append(m.postings[lbl], ref)
|
m.postings[lbl] = append(m.postings[lbl], ref)
|
||||||
}
|
}
|
||||||
|
@ -124,7 +125,7 @@ func (m mockIndex) SortedPostings(p Postings) Postings {
|
||||||
return NewListPostings(ep)
|
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]
|
s, ok := m.series[ref]
|
||||||
if !ok {
|
if !ok {
|
||||||
return errors.New("not found")
|
return errors.New("not found")
|
||||||
|
@ -281,7 +282,7 @@ func TestPostingsMany(t *testing.T) {
|
||||||
}
|
}
|
||||||
|
|
||||||
for i, s := range series {
|
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())
|
require.NoError(t, iw.Close())
|
||||||
|
|
||||||
|
@ -373,7 +374,7 @@ func TestPersistence_index_e2e(t *testing.T) {
|
||||||
metas = append(metas, chunks.Meta{
|
metas = append(metas, chunks.Meta{
|
||||||
MinTime: int64(j * 10000),
|
MinTime: int64(j * 10000),
|
||||||
MaxTime: int64((j + 1) * 10000),
|
MaxTime: int64((j + 1) * 10000),
|
||||||
Ref: rand.Uint64(),
|
Ref: chunks.ChunkRef(rand.Uint64()),
|
||||||
Chunk: chunkenc.NewXORChunk(),
|
Chunk: chunkenc.NewXORChunk(),
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
@ -404,9 +405,9 @@ func TestPersistence_index_e2e(t *testing.T) {
|
||||||
mi := newMockIndex()
|
mi := newMockIndex()
|
||||||
|
|
||||||
for i, s := range input {
|
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, 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 {
|
for _, l := range s.labels {
|
||||||
valset, ok := values[l.Name]
|
valset, ok := values[l.Name]
|
||||||
|
@ -416,7 +417,7 @@ func TestPersistence_index_e2e(t *testing.T) {
|
||||||
}
|
}
|
||||||
valset[l.Value] = struct{}{}
|
valset[l.Value] = struct{}{}
|
||||||
}
|
}
|
||||||
postings.Add(uint64(i), s.labels)
|
postings.Add(storage.SeriesRef(i), s.labels)
|
||||||
}
|
}
|
||||||
|
|
||||||
err = iw.Close()
|
err = iw.Close()
|
||||||
|
|
|
@ -21,6 +21,7 @@ import (
|
||||||
"sync"
|
"sync"
|
||||||
|
|
||||||
"github.com/prometheus/prometheus/pkg/labels"
|
"github.com/prometheus/prometheus/pkg/labels"
|
||||||
|
"github.com/prometheus/prometheus/storage"
|
||||||
)
|
)
|
||||||
|
|
||||||
var allPostingsKey = labels.Label{}
|
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().
|
// ensureOrderBatchPool is a pool used to recycle batches passed to workers in MemPostings.EnsureOrder().
|
||||||
var ensureOrderBatchPool = sync.Pool{
|
var ensureOrderBatchPool = sync.Pool{
|
||||||
New: func() interface{} {
|
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.
|
// unordered batch fills on startup.
|
||||||
type MemPostings struct {
|
type MemPostings struct {
|
||||||
mtx sync.RWMutex
|
mtx sync.RWMutex
|
||||||
m map[string]map[string][]uint64
|
m map[string]map[string][]storage.SeriesRef
|
||||||
ordered bool
|
ordered bool
|
||||||
}
|
}
|
||||||
|
|
||||||
// NewMemPostings returns a memPostings that's ready for reads and writes.
|
// NewMemPostings returns a memPostings that's ready for reads and writes.
|
||||||
func NewMemPostings() *MemPostings {
|
func NewMemPostings() *MemPostings {
|
||||||
return &MemPostings{
|
return &MemPostings{
|
||||||
m: make(map[string]map[string][]uint64, 512),
|
m: make(map[string]map[string][]storage.SeriesRef, 512),
|
||||||
ordered: true,
|
ordered: true,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -62,7 +63,7 @@ func NewMemPostings() *MemPostings {
|
||||||
// until EnsureOrder() was called once.
|
// until EnsureOrder() was called once.
|
||||||
func NewUnorderedMemPostings() *MemPostings {
|
func NewUnorderedMemPostings() *MemPostings {
|
||||||
return &MemPostings{
|
return &MemPostings{
|
||||||
m: make(map[string]map[string][]uint64, 512),
|
m: make(map[string]map[string][]storage.SeriesRef, 512),
|
||||||
ordered: false,
|
ordered: false,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -198,7 +199,7 @@ func (p *MemPostings) Stats(label string) *PostingsStats {
|
||||||
|
|
||||||
// Get returns a postings list for the given label pair.
|
// Get returns a postings list for the given label pair.
|
||||||
func (p *MemPostings) Get(name, value string) Postings {
|
func (p *MemPostings) Get(name, value string) Postings {
|
||||||
var lp []uint64
|
var lp []storage.SeriesRef
|
||||||
p.mtx.RLock()
|
p.mtx.RLock()
|
||||||
l := p.m[name]
|
l := p.m[name]
|
||||||
if l != nil {
|
if l != nil {
|
||||||
|
@ -228,7 +229,7 @@ func (p *MemPostings) EnsureOrder() {
|
||||||
}
|
}
|
||||||
|
|
||||||
n := runtime.GOMAXPROCS(0)
|
n := runtime.GOMAXPROCS(0)
|
||||||
workc := make(chan [][]uint64)
|
workc := make(chan [][]storage.SeriesRef)
|
||||||
|
|
||||||
var wg sync.WaitGroup
|
var wg sync.WaitGroup
|
||||||
wg.Add(n)
|
wg.Add(n)
|
||||||
|
@ -237,7 +238,7 @@ func (p *MemPostings) EnsureOrder() {
|
||||||
go func() {
|
go func() {
|
||||||
for job := range workc {
|
for job := range workc {
|
||||||
for _, l := range job {
|
for _, l := range job {
|
||||||
sort.Sort(uint64Slice(l))
|
sort.Sort(seriesRefSlice(l))
|
||||||
}
|
}
|
||||||
|
|
||||||
job = job[:0]
|
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 _, e := range p.m {
|
||||||
for _, l := range e {
|
for _, l := range e {
|
||||||
nextJob = append(nextJob, l)
|
nextJob = append(nextJob, l)
|
||||||
|
|
||||||
if len(nextJob) >= ensureOrderBatchSize {
|
if len(nextJob) >= ensureOrderBatchSize {
|
||||||
workc <- nextJob
|
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.
|
// 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
|
var keys, vals []string
|
||||||
|
|
||||||
// Collect all keys relevant for deletion once. New keys added afterwards
|
// 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()
|
p.mtx.Unlock()
|
||||||
continue
|
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] {
|
for _, id := range p.m[n][l] {
|
||||||
if _, ok := deleted[id]; !ok {
|
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.
|
// 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()
|
p.mtx.Lock()
|
||||||
|
|
||||||
for _, l := range lset {
|
for _, l := range lset {
|
||||||
|
@ -356,10 +357,10 @@ func (p *MemPostings) Add(id uint64, lset labels.Labels) {
|
||||||
p.mtx.Unlock()
|
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]
|
nm, ok := p.m[l.Name]
|
||||||
if !ok {
|
if !ok {
|
||||||
nm = map[string][]uint64{}
|
nm = map[string][]storage.SeriesRef{}
|
||||||
p.m[l.Name] = nm
|
p.m[l.Name] = nm
|
||||||
}
|
}
|
||||||
list := append(nm[l.Value], id)
|
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.
|
// 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() {
|
for p.Next() {
|
||||||
res = append(res, p.At())
|
res = append(res, p.At())
|
||||||
}
|
}
|
||||||
|
@ -395,10 +396,10 @@ type Postings interface {
|
||||||
|
|
||||||
// Seek advances the iterator to value v or greater and returns
|
// Seek advances the iterator to value v or greater and returns
|
||||||
// true if a value was found.
|
// 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 returns the value at the current iterator position.
|
||||||
At() uint64
|
At() storage.SeriesRef
|
||||||
|
|
||||||
// Err returns the last error of the iterator.
|
// Err returns the last error of the iterator.
|
||||||
Err() error
|
Err() error
|
||||||
|
@ -409,15 +410,15 @@ type errPostings struct {
|
||||||
err error
|
err error
|
||||||
}
|
}
|
||||||
|
|
||||||
func (e errPostings) Next() bool { return false }
|
func (e errPostings) Next() bool { return false }
|
||||||
func (e errPostings) Seek(uint64) bool { return false }
|
func (e errPostings) Seek(storage.SeriesRef) bool { return false }
|
||||||
func (e errPostings) At() uint64 { return 0 }
|
func (e errPostings) At() storage.SeriesRef { return 0 }
|
||||||
func (e errPostings) Err() error { return e.err }
|
func (e errPostings) Err() error { return e.err }
|
||||||
|
|
||||||
var emptyPostings = errPostings{}
|
var emptyPostings = errPostings{}
|
||||||
|
|
||||||
// EmptyPostings returns a postings list that's always empty.
|
// 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.
|
// It triggers optimized flow in other functions like Intersect, Without etc.
|
||||||
func EmptyPostings() Postings {
|
func EmptyPostings() Postings {
|
||||||
return emptyPostings
|
return emptyPostings
|
||||||
|
@ -448,14 +449,14 @@ func Intersect(its ...Postings) Postings {
|
||||||
|
|
||||||
type intersectPostings struct {
|
type intersectPostings struct {
|
||||||
arr []Postings
|
arr []Postings
|
||||||
cur uint64
|
cur storage.SeriesRef
|
||||||
}
|
}
|
||||||
|
|
||||||
func newIntersectPostings(its ...Postings) *intersectPostings {
|
func newIntersectPostings(its ...Postings) *intersectPostings {
|
||||||
return &intersectPostings{arr: its}
|
return &intersectPostings{arr: its}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (it *intersectPostings) At() uint64 {
|
func (it *intersectPostings) At() storage.SeriesRef {
|
||||||
return it.cur
|
return it.cur
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -487,7 +488,7 @@ func (it *intersectPostings) Next() bool {
|
||||||
return it.doNext()
|
return it.doNext()
|
||||||
}
|
}
|
||||||
|
|
||||||
func (it *intersectPostings) Seek(id uint64) bool {
|
func (it *intersectPostings) Seek(id storage.SeriesRef) bool {
|
||||||
it.cur = id
|
it.cur = id
|
||||||
return it.doNext()
|
return it.doNext()
|
||||||
}
|
}
|
||||||
|
@ -538,7 +539,7 @@ func (h *postingsHeap) Pop() interface{} {
|
||||||
type mergedPostings struct {
|
type mergedPostings struct {
|
||||||
h postingsHeap
|
h postingsHeap
|
||||||
initialized bool
|
initialized bool
|
||||||
cur uint64
|
cur storage.SeriesRef
|
||||||
err error
|
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 {
|
if it.h.Len() == 0 || it.err != nil {
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
|
@ -628,7 +629,7 @@ func (it *mergedPostings) Seek(id uint64) bool {
|
||||||
return true
|
return true
|
||||||
}
|
}
|
||||||
|
|
||||||
func (it mergedPostings) At() uint64 {
|
func (it mergedPostings) At() storage.SeriesRef {
|
||||||
return it.cur
|
return it.cur
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -652,7 +653,7 @@ func Without(full, drop Postings) Postings {
|
||||||
type removedPostings struct {
|
type removedPostings struct {
|
||||||
full, remove Postings
|
full, remove Postings
|
||||||
|
|
||||||
cur uint64
|
cur storage.SeriesRef
|
||||||
|
|
||||||
initialized bool
|
initialized bool
|
||||||
fok, rok 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
|
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 {
|
if rp.cur >= id {
|
||||||
return true
|
return true
|
||||||
}
|
}
|
||||||
|
@ -724,19 +725,19 @@ func (rp *removedPostings) Err() error {
|
||||||
|
|
||||||
// ListPostings implements the Postings interface over a plain list.
|
// ListPostings implements the Postings interface over a plain list.
|
||||||
type ListPostings struct {
|
type ListPostings struct {
|
||||||
list []uint64
|
list []storage.SeriesRef
|
||||||
cur uint64
|
cur storage.SeriesRef
|
||||||
}
|
}
|
||||||
|
|
||||||
func NewListPostings(list []uint64) Postings {
|
func NewListPostings(list []storage.SeriesRef) Postings {
|
||||||
return newListPostings(list...)
|
return newListPostings(list...)
|
||||||
}
|
}
|
||||||
|
|
||||||
func newListPostings(list ...uint64) *ListPostings {
|
func newListPostings(list ...storage.SeriesRef) *ListPostings {
|
||||||
return &ListPostings{list: list}
|
return &ListPostings{list: list}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (it *ListPostings) At() uint64 {
|
func (it *ListPostings) At() storage.SeriesRef {
|
||||||
return it.cur
|
return it.cur
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -750,7 +751,7 @@ func (it *ListPostings) Next() bool {
|
||||||
return false
|
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 the current value satisfies, then return.
|
||||||
if it.cur >= x {
|
if it.cur >= x {
|
||||||
return true
|
return true
|
||||||
|
@ -787,8 +788,8 @@ func newBigEndianPostings(list []byte) *bigEndianPostings {
|
||||||
return &bigEndianPostings{list: list}
|
return &bigEndianPostings{list: list}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (it *bigEndianPostings) At() uint64 {
|
func (it *bigEndianPostings) At() storage.SeriesRef {
|
||||||
return uint64(it.cur)
|
return storage.SeriesRef(it.cur)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (it *bigEndianPostings) Next() bool {
|
func (it *bigEndianPostings) Next() bool {
|
||||||
|
@ -800,8 +801,8 @@ func (it *bigEndianPostings) Next() bool {
|
||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
|
|
||||||
func (it *bigEndianPostings) Seek(x uint64) bool {
|
func (it *bigEndianPostings) Seek(x storage.SeriesRef) bool {
|
||||||
if uint64(it.cur) >= x {
|
if storage.SeriesRef(it.cur) >= x {
|
||||||
return true
|
return true
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -824,9 +825,9 @@ func (it *bigEndianPostings) Err() error {
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// uint64Slice attaches the methods of sort.Interface to []uint64, sorting in increasing order.
|
// seriesRefSlice attaches the methods of sort.Interface to []storage.SeriesRef, sorting in increasing order.
|
||||||
type uint64Slice []uint64
|
type seriesRefSlice []storage.SeriesRef
|
||||||
|
|
||||||
func (x uint64Slice) Len() int { return len(x) }
|
func (x seriesRefSlice) Len() int { return len(x) }
|
||||||
func (x uint64Slice) Less(i, j int) bool { return x[i] < x[j] }
|
func (x seriesRefSlice) 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) Swap(i, j int) { x[i], x[j] = x[j], x[i] }
|
||||||
|
|
|
@ -24,26 +24,27 @@ import (
|
||||||
"github.com/stretchr/testify/require"
|
"github.com/stretchr/testify/require"
|
||||||
|
|
||||||
"github.com/prometheus/prometheus/pkg/labels"
|
"github.com/prometheus/prometheus/pkg/labels"
|
||||||
|
"github.com/prometheus/prometheus/storage"
|
||||||
)
|
)
|
||||||
|
|
||||||
func TestMemPostings_addFor(t *testing.T) {
|
func TestMemPostings_addFor(t *testing.T) {
|
||||||
p := NewMemPostings()
|
p := NewMemPostings()
|
||||||
p.m[allPostingsKey.Name] = map[string][]uint64{}
|
p.m[allPostingsKey.Name] = map[string][]storage.SeriesRef{}
|
||||||
p.m[allPostingsKey.Name][allPostingsKey.Value] = []uint64{1, 2, 3, 4, 6, 7, 8}
|
p.m[allPostingsKey.Name][allPostingsKey.Value] = []storage.SeriesRef{1, 2, 3, 4, 6, 7, 8}
|
||||||
|
|
||||||
p.addFor(5, allPostingsKey)
|
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) {
|
func TestMemPostings_ensureOrder(t *testing.T) {
|
||||||
p := NewUnorderedMemPostings()
|
p := NewUnorderedMemPostings()
|
||||||
p.m["a"] = map[string][]uint64{}
|
p.m["a"] = map[string][]storage.SeriesRef{}
|
||||||
|
|
||||||
for i := 0; i < 100; i++ {
|
for i := 0; i < 100; i++ {
|
||||||
l := make([]uint64, 100)
|
l := make([]storage.SeriesRef, 100)
|
||||||
for j := range l {
|
for j := range l {
|
||||||
l[j] = rand.Uint64()
|
l[j] = storage.SeriesRef(rand.Uint64())
|
||||||
}
|
}
|
||||||
v := fmt.Sprintf("%d", i)
|
v := fmt.Sprintf("%d", i)
|
||||||
|
|
||||||
|
@ -94,12 +95,12 @@ func BenchmarkMemPostings_ensureOrder(b *testing.B) {
|
||||||
// Generate postings.
|
// Generate postings.
|
||||||
for l := 0; l < testData.numLabels; l++ {
|
for l := 0; l < testData.numLabels; l++ {
|
||||||
labelName := strconv.Itoa(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++ {
|
for v := 0; v < testData.numValuesPerLabel; v++ {
|
||||||
refs := make([]uint64, testData.numRefsPerValue)
|
refs := make([]storage.SeriesRef, testData.numRefsPerValue)
|
||||||
for j := range refs {
|
for j := range refs {
|
||||||
refs[j] = rand.Uint64()
|
refs[j] = storage.SeriesRef(rand.Uint64())
|
||||||
}
|
}
|
||||||
|
|
||||||
labelValue := strconv.Itoa(v)
|
labelValue := strconv.Itoa(v)
|
||||||
|
@ -237,29 +238,29 @@ func TestIntersect(t *testing.T) {
|
||||||
|
|
||||||
func TestMultiIntersect(t *testing.T) {
|
func TestMultiIntersect(t *testing.T) {
|
||||||
cases := []struct {
|
cases := []struct {
|
||||||
p [][]uint64
|
p [][]storage.SeriesRef
|
||||||
res []uint64
|
res []storage.SeriesRef
|
||||||
}{
|
}{
|
||||||
{
|
{
|
||||||
p: [][]uint64{
|
p: [][]storage.SeriesRef{
|
||||||
{1, 2, 3, 4, 5, 6, 1000, 1001},
|
{1, 2, 3, 4, 5, 6, 1000, 1001},
|
||||||
{2, 4, 5, 6, 7, 8, 999, 1001},
|
{2, 4, 5, 6, 7, 8, 999, 1001},
|
||||||
{1, 2, 5, 6, 7, 8, 1001, 1200},
|
{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:
|
// One of the reproducible cases for:
|
||||||
// https://github.com/prometheus/prometheus/issues/2616
|
// https://github.com/prometheus/prometheus/issues/2616
|
||||||
// The initialisation of intersectPostings was moving the iterator forward
|
// The initialisation of intersectPostings was moving the iterator forward
|
||||||
// prematurely making us miss some postings.
|
// prematurely making us miss some postings.
|
||||||
{
|
{
|
||||||
p: [][]uint64{
|
p: [][]storage.SeriesRef{
|
||||||
{1, 2},
|
{1, 2},
|
||||||
{1, 2},
|
{1, 2},
|
||||||
{1, 2},
|
{1, 2},
|
||||||
{2},
|
{2},
|
||||||
},
|
},
|
||||||
res: []uint64{2},
|
res: []storage.SeriesRef{2},
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -278,22 +279,22 @@ func TestMultiIntersect(t *testing.T) {
|
||||||
|
|
||||||
func BenchmarkIntersect(t *testing.B) {
|
func BenchmarkIntersect(t *testing.B) {
|
||||||
t.Run("LongPostings1", func(bench *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 {
|
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 {
|
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 {
|
for i := 5090000; i < 5090600; i += 4 {
|
||||||
b = append(b, uint64(i))
|
b = append(b, storage.SeriesRef(i))
|
||||||
}
|
}
|
||||||
for i := 4990000; i < 5100000; i++ {
|
for i := 4990000; i < 5100000; i++ {
|
||||||
c = append(c, uint64(i))
|
c = append(c, storage.SeriesRef(i))
|
||||||
}
|
}
|
||||||
for i := 4000000; i < 6000000; i++ {
|
for i := 4000000; i < 6000000; i++ {
|
||||||
d = append(d, uint64(i))
|
d = append(d, storage.SeriesRef(i))
|
||||||
}
|
}
|
||||||
|
|
||||||
i1 := newListPostings(a...)
|
i1 := newListPostings(a...)
|
||||||
|
@ -311,19 +312,19 @@ func BenchmarkIntersect(t *testing.B) {
|
||||||
})
|
})
|
||||||
|
|
||||||
t.Run("LongPostings2", func(bench *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++ {
|
for i := 0; i < 12500000; i++ {
|
||||||
a = append(a, uint64(i))
|
a = append(a, storage.SeriesRef(i))
|
||||||
}
|
}
|
||||||
for i := 7500000; i < 12500000; i++ {
|
for i := 7500000; i < 12500000; i++ {
|
||||||
b = append(b, uint64(i))
|
b = append(b, storage.SeriesRef(i))
|
||||||
}
|
}
|
||||||
for i := 9000000; i < 20000000; i++ {
|
for i := 9000000; i < 20000000; i++ {
|
||||||
c = append(c, uint64(i))
|
c = append(c, storage.SeriesRef(i))
|
||||||
}
|
}
|
||||||
for i := 10000000; i < 12000000; i++ {
|
for i := 10000000; i < 12000000; i++ {
|
||||||
d = append(d, uint64(i))
|
d = append(d, storage.SeriesRef(i))
|
||||||
}
|
}
|
||||||
|
|
||||||
i1 := newListPostings(a...)
|
i1 := newListPostings(a...)
|
||||||
|
@ -346,9 +347,9 @@ func BenchmarkIntersect(t *testing.B) {
|
||||||
|
|
||||||
// 100000 matchers(k=100000).
|
// 100000 matchers(k=100000).
|
||||||
for i := 0; i < 100000; i++ {
|
for i := 0; i < 100000; i++ {
|
||||||
var temp []uint64
|
var temp []storage.SeriesRef
|
||||||
for j := 1; j < 100; j++ {
|
for j := storage.SeriesRef(1); j < 100; j++ {
|
||||||
temp = append(temp, uint64(j))
|
temp = append(temp, j)
|
||||||
}
|
}
|
||||||
its = append(its, newListPostings(temp...))
|
its = append(its, newListPostings(temp...))
|
||||||
}
|
}
|
||||||
|
@ -370,7 +371,7 @@ func TestMultiMerge(t *testing.T) {
|
||||||
|
|
||||||
res, err := ExpandPostings(Merge(i1, i2, i3))
|
res, err := ExpandPostings(Merge(i1, i2, i3))
|
||||||
require.NoError(t, err)
|
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) {
|
func TestMergedPostings(t *testing.T) {
|
||||||
|
@ -479,43 +480,43 @@ func TestMergedPostings(t *testing.T) {
|
||||||
|
|
||||||
func TestMergedPostingsSeek(t *testing.T) {
|
func TestMergedPostingsSeek(t *testing.T) {
|
||||||
cases := []struct {
|
cases := []struct {
|
||||||
a, b []uint64
|
a, b []storage.SeriesRef
|
||||||
|
|
||||||
seek uint64
|
seek storage.SeriesRef
|
||||||
success bool
|
success bool
|
||||||
res []uint64
|
res []storage.SeriesRef
|
||||||
}{
|
}{
|
||||||
{
|
{
|
||||||
a: []uint64{2, 3, 4, 5},
|
a: []storage.SeriesRef{2, 3, 4, 5},
|
||||||
b: []uint64{6, 7, 8, 9, 10},
|
b: []storage.SeriesRef{6, 7, 8, 9, 10},
|
||||||
|
|
||||||
seek: 1,
|
seek: 1,
|
||||||
success: true,
|
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},
|
a: []storage.SeriesRef{1, 2, 3, 4, 5},
|
||||||
b: []uint64{6, 7, 8, 9, 10},
|
b: []storage.SeriesRef{6, 7, 8, 9, 10},
|
||||||
|
|
||||||
seek: 2,
|
seek: 2,
|
||||||
success: true,
|
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},
|
a: []storage.SeriesRef{1, 2, 3, 4, 5},
|
||||||
b: []uint64{4, 5, 6, 7, 8},
|
b: []storage.SeriesRef{4, 5, 6, 7, 8},
|
||||||
|
|
||||||
seek: 9,
|
seek: 9,
|
||||||
success: false,
|
success: false,
|
||||||
res: nil,
|
res: nil,
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
a: []uint64{1, 2, 3, 4, 9, 10},
|
a: []storage.SeriesRef{1, 2, 3, 4, 9, 10},
|
||||||
b: []uint64{1, 4, 5, 6, 7, 8, 10, 11},
|
b: []storage.SeriesRef{1, 4, 5, 6, 7, 8, 10, 11},
|
||||||
|
|
||||||
seek: 10,
|
seek: 10,
|
||||||
success: true,
|
success: true,
|
||||||
res: []uint64{10, 11},
|
res: []storage.SeriesRef{10, 11},
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -533,7 +534,7 @@ func TestMergedPostingsSeek(t *testing.T) {
|
||||||
lst, err := ExpandPostings(p)
|
lst, err := ExpandPostings(p)
|
||||||
require.NoError(t, err)
|
require.NoError(t, err)
|
||||||
|
|
||||||
lst = append([]uint64{start}, lst...)
|
lst = append([]storage.SeriesRef{start}, lst...)
|
||||||
require.Equal(t, c.res, lst)
|
require.Equal(t, c.res, lst)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -541,43 +542,43 @@ func TestMergedPostingsSeek(t *testing.T) {
|
||||||
|
|
||||||
func TestRemovedPostings(t *testing.T) {
|
func TestRemovedPostings(t *testing.T) {
|
||||||
cases := []struct {
|
cases := []struct {
|
||||||
a, b []uint64
|
a, b []storage.SeriesRef
|
||||||
res []uint64
|
res []storage.SeriesRef
|
||||||
}{
|
}{
|
||||||
{
|
{
|
||||||
a: nil,
|
a: nil,
|
||||||
b: 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,
|
b: nil,
|
||||||
res: []uint64{1, 2, 3, 4},
|
res: []storage.SeriesRef{1, 2, 3, 4},
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
a: nil,
|
a: nil,
|
||||||
b: []uint64{1, 2, 3, 4},
|
b: []storage.SeriesRef{1, 2, 3, 4},
|
||||||
res: []uint64(nil),
|
res: []storage.SeriesRef(nil),
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
a: []uint64{1, 2, 3, 4, 5},
|
a: []storage.SeriesRef{1, 2, 3, 4, 5},
|
||||||
b: []uint64{6, 7, 8, 9, 10},
|
b: []storage.SeriesRef{6, 7, 8, 9, 10},
|
||||||
res: []uint64{1, 2, 3, 4, 5},
|
res: []storage.SeriesRef{1, 2, 3, 4, 5},
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
a: []uint64{1, 2, 3, 4, 5},
|
a: []storage.SeriesRef{1, 2, 3, 4, 5},
|
||||||
b: []uint64{4, 5, 6, 7, 8},
|
b: []storage.SeriesRef{4, 5, 6, 7, 8},
|
||||||
res: []uint64{1, 2, 3},
|
res: []storage.SeriesRef{1, 2, 3},
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
a: []uint64{1, 2, 3, 4, 9, 10},
|
a: []storage.SeriesRef{1, 2, 3, 4, 9, 10},
|
||||||
b: []uint64{1, 4, 5, 6, 7, 8, 10, 11},
|
b: []storage.SeriesRef{1, 4, 5, 6, 7, 8, 10, 11},
|
||||||
res: []uint64{2, 3, 9},
|
res: []storage.SeriesRef{2, 3, 9},
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
a: []uint64{1, 2, 3, 4, 9, 10},
|
a: []storage.SeriesRef{1, 2, 3, 4, 9, 10},
|
||||||
b: []uint64{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11},
|
b: []storage.SeriesRef{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11},
|
||||||
res: []uint64(nil),
|
res: []storage.SeriesRef(nil),
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -592,10 +593,10 @@ func TestRemovedPostings(t *testing.T) {
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestRemovedNextStackoverflow(t *testing.T) {
|
func TestRemovedNextStackoverflow(t *testing.T) {
|
||||||
var full []uint64
|
var full []storage.SeriesRef
|
||||||
var remove []uint64
|
var remove []storage.SeriesRef
|
||||||
|
|
||||||
var i uint64
|
var i storage.SeriesRef
|
||||||
for i = 0; i < 1e7; i++ {
|
for i = 0; i < 1e7; i++ {
|
||||||
full = append(full, i)
|
full = append(full, i)
|
||||||
remove = append(remove, i)
|
remove = append(remove, i)
|
||||||
|
@ -615,67 +616,67 @@ func TestRemovedNextStackoverflow(t *testing.T) {
|
||||||
|
|
||||||
func TestRemovedPostingsSeek(t *testing.T) {
|
func TestRemovedPostingsSeek(t *testing.T) {
|
||||||
cases := []struct {
|
cases := []struct {
|
||||||
a, b []uint64
|
a, b []storage.SeriesRef
|
||||||
|
|
||||||
seek uint64
|
seek storage.SeriesRef
|
||||||
success bool
|
success bool
|
||||||
res []uint64
|
res []storage.SeriesRef
|
||||||
}{
|
}{
|
||||||
{
|
{
|
||||||
a: []uint64{2, 3, 4, 5},
|
a: []storage.SeriesRef{2, 3, 4, 5},
|
||||||
b: []uint64{6, 7, 8, 9, 10},
|
b: []storage.SeriesRef{6, 7, 8, 9, 10},
|
||||||
|
|
||||||
seek: 1,
|
seek: 1,
|
||||||
success: true,
|
success: true,
|
||||||
res: []uint64{2, 3, 4, 5},
|
res: []storage.SeriesRef{2, 3, 4, 5},
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
a: []uint64{1, 2, 3, 4, 5},
|
a: []storage.SeriesRef{1, 2, 3, 4, 5},
|
||||||
b: []uint64{6, 7, 8, 9, 10},
|
b: []storage.SeriesRef{6, 7, 8, 9, 10},
|
||||||
|
|
||||||
seek: 2,
|
seek: 2,
|
||||||
success: true,
|
success: true,
|
||||||
res: []uint64{2, 3, 4, 5},
|
res: []storage.SeriesRef{2, 3, 4, 5},
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
a: []uint64{1, 2, 3, 4, 5},
|
a: []storage.SeriesRef{1, 2, 3, 4, 5},
|
||||||
b: []uint64{4, 5, 6, 7, 8},
|
b: []storage.SeriesRef{4, 5, 6, 7, 8},
|
||||||
|
|
||||||
seek: 9,
|
seek: 9,
|
||||||
success: false,
|
success: false,
|
||||||
res: nil,
|
res: nil,
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
a: []uint64{1, 2, 3, 4, 9, 10},
|
a: []storage.SeriesRef{1, 2, 3, 4, 9, 10},
|
||||||
b: []uint64{1, 4, 5, 6, 7, 8, 10, 11},
|
b: []storage.SeriesRef{1, 4, 5, 6, 7, 8, 10, 11},
|
||||||
|
|
||||||
seek: 10,
|
seek: 10,
|
||||||
success: false,
|
success: false,
|
||||||
res: nil,
|
res: nil,
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
a: []uint64{1, 2, 3, 4, 9, 10},
|
a: []storage.SeriesRef{1, 2, 3, 4, 9, 10},
|
||||||
b: []uint64{1, 4, 5, 6, 7, 8, 11},
|
b: []storage.SeriesRef{1, 4, 5, 6, 7, 8, 11},
|
||||||
|
|
||||||
seek: 4,
|
seek: 4,
|
||||||
success: true,
|
success: true,
|
||||||
res: []uint64{9, 10},
|
res: []storage.SeriesRef{9, 10},
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
a: []uint64{1, 2, 3, 4, 9, 10},
|
a: []storage.SeriesRef{1, 2, 3, 4, 9, 10},
|
||||||
b: []uint64{1, 4, 5, 6, 7, 8, 11},
|
b: []storage.SeriesRef{1, 4, 5, 6, 7, 8, 11},
|
||||||
|
|
||||||
seek: 5,
|
seek: 5,
|
||||||
success: true,
|
success: true,
|
||||||
res: []uint64{9, 10},
|
res: []storage.SeriesRef{9, 10},
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
a: []uint64{1, 2, 3, 4, 9, 10},
|
a: []storage.SeriesRef{1, 2, 3, 4, 9, 10},
|
||||||
b: []uint64{1, 4, 5, 6, 7, 8, 11},
|
b: []storage.SeriesRef{1, 4, 5, 6, 7, 8, 11},
|
||||||
|
|
||||||
seek: 10,
|
seek: 10,
|
||||||
success: true,
|
success: true,
|
||||||
res: []uint64{10},
|
res: []storage.SeriesRef{10},
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -693,7 +694,7 @@ func TestRemovedPostingsSeek(t *testing.T) {
|
||||||
lst, err := ExpandPostings(p)
|
lst, err := ExpandPostings(p)
|
||||||
require.NoError(t, err)
|
require.NoError(t, err)
|
||||||
|
|
||||||
lst = append([]uint64{start}, lst...)
|
lst = append([]storage.SeriesRef{start}, lst...)
|
||||||
require.Equal(t, c.res, lst)
|
require.Equal(t, c.res, lst)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -718,7 +719,7 @@ func TestBigEndian(t *testing.T) {
|
||||||
bep := newBigEndianPostings(beLst)
|
bep := newBigEndianPostings(beLst)
|
||||||
for i := 0; i < num; i++ {
|
for i := 0; i < num; i++ {
|
||||||
require.True(t, bep.Next())
|
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())
|
require.False(t, bep.Next())
|
||||||
|
@ -766,8 +767,8 @@ func TestBigEndian(t *testing.T) {
|
||||||
bep := newBigEndianPostings(beLst)
|
bep := newBigEndianPostings(beLst)
|
||||||
|
|
||||||
for _, v := range table {
|
for _, v := range table {
|
||||||
require.Equal(t, v.found, bep.Seek(uint64(v.seek)))
|
require.Equal(t, v.found, bep.Seek(storage.SeriesRef(v.seek)))
|
||||||
require.Equal(t, uint64(v.val), bep.At())
|
require.Equal(t, storage.SeriesRef(v.val), bep.At())
|
||||||
require.NoError(t, bep.Err())
|
require.NoError(t, bep.Err())
|
||||||
}
|
}
|
||||||
})
|
})
|
||||||
|
@ -787,7 +788,7 @@ func TestIntersectWithMerge(t *testing.T) {
|
||||||
res, err := ExpandPostings(p)
|
res, err := ExpandPostings(p)
|
||||||
|
|
||||||
require.NoError(t, err)
|
require.NoError(t, err)
|
||||||
require.Equal(t, []uint64{30}, res)
|
require.Equal(t, []storage.SeriesRef{30}, res)
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestWithoutPostings(t *testing.T) {
|
func TestWithoutPostings(t *testing.T) {
|
||||||
|
@ -871,7 +872,7 @@ func TestWithoutPostings(t *testing.T) {
|
||||||
func BenchmarkPostings_Stats(b *testing.B) {
|
func BenchmarkPostings_Stats(b *testing.B) {
|
||||||
p := NewMemPostings()
|
p := NewMemPostings()
|
||||||
|
|
||||||
var seriesID uint64
|
var seriesID storage.SeriesRef
|
||||||
|
|
||||||
createPostingsLabelValues := func(name, valuePrefix string, count int) {
|
createPostingsLabelValues := func(name, valuePrefix string, count int) {
|
||||||
for n := 1; n < count; n++ {
|
for n := 1; n < count; n++ {
|
||||||
|
@ -906,7 +907,7 @@ func TestMemPostings_Delete(t *testing.T) {
|
||||||
p.Add(3, labels.FromStrings("lbl2", "a"))
|
p.Add(3, labels.FromStrings("lbl2", "a"))
|
||||||
|
|
||||||
before := p.Get(allPostingsKey.Name, allPostingsKey.Value)
|
before := p.Get(allPostingsKey.Name, allPostingsKey.Value)
|
||||||
p.Delete(map[uint64]struct{}{
|
p.Delete(map[storage.SeriesRef]struct{}{
|
||||||
2: {},
|
2: {},
|
||||||
})
|
})
|
||||||
after := p.Get(allPostingsKey.Name, allPostingsKey.Value)
|
after := p.Get(allPostingsKey.Name, allPostingsKey.Value)
|
||||||
|
@ -915,13 +916,13 @@ func TestMemPostings_Delete(t *testing.T) {
|
||||||
// iterated over.
|
// iterated over.
|
||||||
expanded, err := ExpandPostings(before)
|
expanded, err := ExpandPostings(before)
|
||||||
require.NoError(t, err)
|
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
|
// Make sure postings gotten after the delete have the new data when
|
||||||
// iterated over.
|
// iterated over.
|
||||||
expanded, err = ExpandPostings(after)
|
expanded, err = ExpandPostings(after)
|
||||||
require.NoError(t, err)
|
require.NoError(t, err)
|
||||||
require.Equal(t, []uint64{1, 3}, expanded)
|
require.Equal(t, []storage.SeriesRef{1, 3}, expanded)
|
||||||
|
|
||||||
deleted := p.Get("lbl1", "b")
|
deleted := p.Get("lbl1", "b")
|
||||||
expanded, err = ExpandPostings(deleted)
|
expanded, err = ExpandPostings(deleted)
|
||||||
|
|
|
@ -17,6 +17,7 @@ import (
|
||||||
"github.com/pkg/errors"
|
"github.com/pkg/errors"
|
||||||
|
|
||||||
"github.com/prometheus/prometheus/pkg/labels"
|
"github.com/prometheus/prometheus/pkg/labels"
|
||||||
|
"github.com/prometheus/prometheus/storage"
|
||||||
"github.com/prometheus/prometheus/tsdb/chunkenc"
|
"github.com/prometheus/prometheus/tsdb/chunkenc"
|
||||||
"github.com/prometheus/prometheus/tsdb/chunks"
|
"github.com/prometheus/prometheus/tsdb/chunks"
|
||||||
"github.com/prometheus/prometheus/tsdb/tombstones"
|
"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 (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.
|
// Copy chunks as their bytes are pooled.
|
||||||
chksNew := make([]chunks.Meta, len(chks))
|
chksNew := make([]chunks.Meta, len(chks))
|
||||||
for i, chk := range chks {
|
for i, chk := range chks {
|
||||||
|
|
|
@ -418,7 +418,7 @@ func labelNamesWithMatchers(r IndexReader, matchers ...*labels.Matcher) ([]strin
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
var postings []uint64
|
var postings []storage.SeriesRef
|
||||||
for p.Next() {
|
for p.Next() {
|
||||||
postings = append(postings, p.At())
|
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 }
|
func (cr nopChunkReader) Close() error { return nil }
|
||||||
|
|
|
@ -106,13 +106,13 @@ func createIdxChkReaders(t *testing.T, tc []seriesSamples) (IndexReader, ChunkRe
|
||||||
})
|
})
|
||||||
|
|
||||||
postings := index.NewMemPostings()
|
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{})
|
lblIdx := make(map[string]map[string]struct{})
|
||||||
mi := newMockIndex()
|
mi := newMockIndex()
|
||||||
blockMint := int64(math.MaxInt64)
|
blockMint := int64(math.MaxInt64)
|
||||||
blockMaxt := int64(math.MinInt64)
|
blockMaxt := int64(math.MinInt64)
|
||||||
|
|
||||||
var chunkRef uint64
|
var chunkRef chunks.ChunkRef
|
||||||
for i, s := range tc {
|
for i, s := range tc {
|
||||||
i = i + 1 // 0 is not a valid posting.
|
i = i + 1 // 0 is not a valid posting.
|
||||||
metas := make([]chunks.Meta, 0, len(s.chunks))
|
metas := make([]chunks.Meta, 0, len(s.chunks))
|
||||||
|
@ -139,9 +139,9 @@ func createIdxChkReaders(t *testing.T, tc []seriesSamples) (IndexReader, ChunkRe
|
||||||
chunkRef++
|
chunkRef++
|
||||||
}
|
}
|
||||||
ls := labels.FromMap(s.lset)
|
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 {
|
for _, l := range ls {
|
||||||
vs, present := lblIdx[l.Name]
|
vs, present := lblIdx[l.Name]
|
||||||
|
@ -607,21 +607,21 @@ func TestBlockQuerierDelete(t *testing.T) {
|
||||||
|
|
||||||
type fakeChunksReader struct {
|
type fakeChunksReader struct {
|
||||||
ChunkReader
|
ChunkReader
|
||||||
chks map[uint64]chunkenc.Chunk
|
chks map[chunks.ChunkRef]chunkenc.Chunk
|
||||||
}
|
}
|
||||||
|
|
||||||
func createFakeReaderAndNotPopulatedChunks(s ...[]tsdbutil.Sample) (*fakeChunksReader, []chunks.Meta) {
|
func createFakeReaderAndNotPopulatedChunks(s ...[]tsdbutil.Sample) (*fakeChunksReader, []chunks.Meta) {
|
||||||
f := &fakeChunksReader{
|
f := &fakeChunksReader{
|
||||||
chks: map[uint64]chunkenc.Chunk{},
|
chks: map[chunks.ChunkRef]chunkenc.Chunk{},
|
||||||
}
|
}
|
||||||
chks := make([]chunks.Meta, 0, len(s))
|
chks := make([]chunks.Meta, 0, len(s))
|
||||||
|
|
||||||
for ref, samples := range s {
|
for ref, samples := range s {
|
||||||
chk := tsdbutil.ChunkFromSamples(samples)
|
chk := tsdbutil.ChunkFromSamples(samples)
|
||||||
f.chks[uint64(ref)] = chk.Chunk
|
f.chks[chunks.ChunkRef(ref)] = chk.Chunk
|
||||||
|
|
||||||
chks = append(chks, chunks.Meta{
|
chks = append(chks, chunks.Meta{
|
||||||
Ref: uint64(ref),
|
Ref: chunks.ChunkRef(ref),
|
||||||
MinTime: chk.MinTime,
|
MinTime: chk.MinTime,
|
||||||
MaxTime: chk.MaxTime,
|
MaxTime: chk.MaxTime,
|
||||||
})
|
})
|
||||||
|
@ -629,7 +629,7 @@ func createFakeReaderAndNotPopulatedChunks(s ...[]tsdbutil.Sample) (*fakeChunksR
|
||||||
return f, chks
|
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]
|
chk, ok := r.chks[ref]
|
||||||
if !ok {
|
if !ok {
|
||||||
return nil, errors.Errorf("chunk not found at ref %v", ref)
|
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]
|
chk, ok := cr[id]
|
||||||
if ok {
|
if ok {
|
||||||
return chk, nil
|
return chk, nil
|
||||||
|
@ -1138,15 +1138,15 @@ type series struct {
|
||||||
}
|
}
|
||||||
|
|
||||||
type mockIndex struct {
|
type mockIndex struct {
|
||||||
series map[uint64]series
|
series map[storage.SeriesRef]series
|
||||||
postings map[labels.Label][]uint64
|
postings map[labels.Label][]storage.SeriesRef
|
||||||
symbols map[string]struct{}
|
symbols map[string]struct{}
|
||||||
}
|
}
|
||||||
|
|
||||||
func newMockIndex() mockIndex {
|
func newMockIndex() mockIndex {
|
||||||
ix := mockIndex{
|
ix := mockIndex{
|
||||||
series: make(map[uint64]series),
|
series: make(map[storage.SeriesRef]series),
|
||||||
postings: make(map[labels.Label][]uint64),
|
postings: make(map[labels.Label][]storage.SeriesRef),
|
||||||
symbols: make(map[string]struct{}),
|
symbols: make(map[string]struct{}),
|
||||||
}
|
}
|
||||||
return ix
|
return ix
|
||||||
|
@ -1161,7 +1161,7 @@ func (m mockIndex) Symbols() index.StringIter {
|
||||||
return index.NewStringListIter(l)
|
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 {
|
if _, ok := m.series[ref]; ok {
|
||||||
return errors.Errorf("series with reference %d already added", ref)
|
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
|
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
|
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)
|
namesMap := make(map[string]bool)
|
||||||
for _, id := range ids {
|
for _, id := range ids {
|
||||||
for _, lbl := range m.series[id].l {
|
for _, lbl := range m.series[id].l {
|
||||||
|
@ -1267,7 +1267,7 @@ func (m mockIndex) SortedPostings(p index.Postings) index.Postings {
|
||||||
return index.NewListPostings(ep)
|
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]
|
s, ok := m.series[ref]
|
||||||
if !ok {
|
if !ok {
|
||||||
return storage.ErrNotFound
|
return storage.ErrNotFound
|
||||||
|
@ -2085,11 +2085,11 @@ func (m mockMatcherIndex) LabelValues(name string, matchers ...*labels.Matcher)
|
||||||
return []string{}, errors.New("label values called")
|
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")
|
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")
|
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()
|
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
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -2151,13 +2151,13 @@ func TestBlockBaseSeriesSet(t *testing.T) {
|
||||||
lset labels.Labels
|
lset labels.Labels
|
||||||
chunks []chunks.Meta
|
chunks []chunks.Meta
|
||||||
|
|
||||||
ref uint64
|
ref storage.SeriesRef
|
||||||
}
|
}
|
||||||
|
|
||||||
cases := []struct {
|
cases := []struct {
|
||||||
series []refdSeries
|
series []refdSeries
|
||||||
// Postings should be in the sorted order of the series
|
// Postings should be in the sorted order of the series
|
||||||
postings []uint64
|
postings []storage.SeriesRef
|
||||||
|
|
||||||
expIdxs []int
|
expIdxs []int
|
||||||
}{
|
}{
|
||||||
|
@ -2196,7 +2196,7 @@ func TestBlockBaseSeriesSet(t *testing.T) {
|
||||||
ref: 108,
|
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},
|
expIdxs: []int{0, 1, 3},
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
|
@ -2214,7 +2214,7 @@ func TestBlockBaseSeriesSet(t *testing.T) {
|
||||||
ref: 3,
|
ref: 3,
|
||||||
},
|
},
|
||||||
},
|
},
|
||||||
postings: []uint64{},
|
postings: []storage.SeriesRef{},
|
||||||
expIdxs: []int{},
|
expIdxs: []int{},
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,6 +21,8 @@ import (
|
||||||
"github.com/pkg/errors"
|
"github.com/pkg/errors"
|
||||||
|
|
||||||
"github.com/prometheus/prometheus/pkg/labels"
|
"github.com/prometheus/prometheus/pkg/labels"
|
||||||
|
"github.com/prometheus/prometheus/storage"
|
||||||
|
"github.com/prometheus/prometheus/tsdb/chunks"
|
||||||
"github.com/prometheus/prometheus/tsdb/encoding"
|
"github.com/prometheus/prometheus/tsdb/encoding"
|
||||||
"github.com/prometheus/prometheus/tsdb/tombstones"
|
"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.
|
// RefSeries is the series labels with the series ID.
|
||||||
type RefSeries struct {
|
type RefSeries struct {
|
||||||
Ref uint64
|
Ref chunks.HeadSeriesRef
|
||||||
Labels labels.Labels
|
Labels labels.Labels
|
||||||
}
|
}
|
||||||
|
|
||||||
// RefSample is a timestamp/value pair associated with a reference to a series.
|
// RefSample is a timestamp/value pair associated with a reference to a series.
|
||||||
type RefSample struct {
|
type RefSample struct {
|
||||||
Ref uint64
|
Ref chunks.HeadSeriesRef
|
||||||
T int64
|
T int64
|
||||||
V float64
|
V float64
|
||||||
}
|
}
|
||||||
|
|
||||||
// RefExemplar is an exemplar with it's labels, timestamp, value the exemplar was collected/observed with, and a reference to a series.
|
// 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 {
|
type RefExemplar struct {
|
||||||
Ref uint64
|
Ref chunks.HeadSeriesRef
|
||||||
T int64
|
T int64
|
||||||
V float64
|
V float64
|
||||||
Labels labels.Labels
|
Labels labels.Labels
|
||||||
|
@ -90,7 +92,7 @@ func (d *Decoder) Series(rec []byte, series []RefSeries) ([]RefSeries, error) {
|
||||||
return nil, errors.New("invalid record type")
|
return nil, errors.New("invalid record type")
|
||||||
}
|
}
|
||||||
for len(dec.B) > 0 && dec.Err() == nil {
|
for len(dec.B) > 0 && dec.Err() == nil {
|
||||||
ref := dec.Be64()
|
ref := storage.SeriesRef(dec.Be64())
|
||||||
|
|
||||||
lset := make(labels.Labels, dec.Uvarint())
|
lset := make(labels.Labels, dec.Uvarint())
|
||||||
|
|
||||||
|
@ -101,7 +103,7 @@ func (d *Decoder) Series(rec []byte, series []RefSeries) ([]RefSeries, error) {
|
||||||
sort.Sort(lset)
|
sort.Sort(lset)
|
||||||
|
|
||||||
series = append(series, RefSeries{
|
series = append(series, RefSeries{
|
||||||
Ref: ref,
|
Ref: chunks.HeadSeriesRef(ref),
|
||||||
Labels: lset,
|
Labels: lset,
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
@ -134,7 +136,7 @@ func (d *Decoder) Samples(rec []byte, samples []RefSample) ([]RefSample, error)
|
||||||
val := dec.Be64()
|
val := dec.Be64()
|
||||||
|
|
||||||
samples = append(samples, RefSample{
|
samples = append(samples, RefSample{
|
||||||
Ref: uint64(int64(baseRef) + dref),
|
Ref: chunks.HeadSeriesRef(int64(baseRef) + dref),
|
||||||
T: baseTime + dtime,
|
T: baseTime + dtime,
|
||||||
V: math.Float64frombits(val),
|
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 {
|
for dec.Len() > 0 && dec.Err() == nil {
|
||||||
tstones = append(tstones, tombstones.Stone{
|
tstones = append(tstones, tombstones.Stone{
|
||||||
Ref: dec.Be64(),
|
Ref: storage.SeriesRef(dec.Be64()),
|
||||||
Intervals: tombstones.Intervals{
|
Intervals: tombstones.Intervals{
|
||||||
{Mint: dec.Varint64(), Maxt: dec.Varint64()},
|
{Mint: dec.Varint64(), Maxt: dec.Varint64()},
|
||||||
},
|
},
|
||||||
|
@ -204,7 +206,7 @@ func (d *Decoder) ExemplarsFromBuffer(dec *encoding.Decbuf, exemplars []RefExemp
|
||||||
sort.Sort(lset)
|
sort.Sort(lset)
|
||||||
|
|
||||||
exemplars = append(exemplars, RefExemplar{
|
exemplars = append(exemplars, RefExemplar{
|
||||||
Ref: baseRef + uint64(dref),
|
Ref: chunks.HeadSeriesRef(baseRef + uint64(dref)),
|
||||||
T: baseTime + dtime,
|
T: baseTime + dtime,
|
||||||
V: math.Float64frombits(val),
|
V: math.Float64frombits(val),
|
||||||
Labels: lset,
|
Labels: lset,
|
||||||
|
@ -230,7 +232,7 @@ func (e *Encoder) Series(series []RefSeries, b []byte) []byte {
|
||||||
buf.PutByte(byte(Series))
|
buf.PutByte(byte(Series))
|
||||||
|
|
||||||
for _, s := range series {
|
for _, s := range series {
|
||||||
buf.PutBE64(s.Ref)
|
buf.PutBE64(uint64(s.Ref))
|
||||||
buf.PutUvarint(len(s.Labels))
|
buf.PutUvarint(len(s.Labels))
|
||||||
|
|
||||||
for _, l := range 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.
|
// All samples encode their timestamp and ref as delta to those.
|
||||||
first := samples[0]
|
first := samples[0]
|
||||||
|
|
||||||
buf.PutBE64(first.Ref)
|
buf.PutBE64(uint64(first.Ref))
|
||||||
buf.PutBE64int64(first.T)
|
buf.PutBE64int64(first.T)
|
||||||
|
|
||||||
for _, s := range samples {
|
for _, s := range samples {
|
||||||
|
@ -272,7 +274,7 @@ func (e *Encoder) Tombstones(tstones []tombstones.Stone, b []byte) []byte {
|
||||||
|
|
||||||
for _, s := range tstones {
|
for _, s := range tstones {
|
||||||
for _, iv := range s.Intervals {
|
for _, iv := range s.Intervals {
|
||||||
buf.PutBE64(s.Ref)
|
buf.PutBE64(uint64(s.Ref))
|
||||||
buf.PutVarint64(iv.Mint)
|
buf.PutVarint64(iv.Mint)
|
||||||
buf.PutVarint64(iv.Maxt)
|
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.
|
// All samples encode their timestamp and ref as delta to those.
|
||||||
first := exemplars[0]
|
first := exemplars[0]
|
||||||
|
|
||||||
buf.PutBE64(first.Ref)
|
buf.PutBE64(uint64(first.Ref))
|
||||||
buf.PutBE64int64(first.T)
|
buf.PutBE64int64(first.T)
|
||||||
|
|
||||||
for _, ex := range exemplars {
|
for _, ex := range exemplars {
|
||||||
|
|
|
@ -28,6 +28,7 @@ import (
|
||||||
"github.com/go-kit/log/level"
|
"github.com/go-kit/log/level"
|
||||||
"github.com/pkg/errors"
|
"github.com/pkg/errors"
|
||||||
|
|
||||||
|
"github.com/prometheus/prometheus/storage"
|
||||||
"github.com/prometheus/prometheus/tsdb/encoding"
|
"github.com/prometheus/prometheus/tsdb/encoding"
|
||||||
tsdb_errors "github.com/prometheus/prometheus/tsdb/errors"
|
tsdb_errors "github.com/prometheus/prometheus/tsdb/errors"
|
||||||
"github.com/prometheus/prometheus/tsdb/fileutil"
|
"github.com/prometheus/prometheus/tsdb/fileutil"
|
||||||
|
@ -63,10 +64,10 @@ func newCRC32() hash.Hash32 {
|
||||||
// Reader gives access to tombstone intervals by series reference.
|
// Reader gives access to tombstone intervals by series reference.
|
||||||
type Reader interface {
|
type Reader interface {
|
||||||
// Get returns deletion intervals for the series with the given reference.
|
// 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 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 returns the total count of tombstones.
|
||||||
Total() uint64
|
Total() uint64
|
||||||
|
@ -144,9 +145,9 @@ func WriteFile(logger log.Logger, dir string, tr Reader) (int64, error) {
|
||||||
func Encode(tr Reader) ([]byte, error) {
|
func Encode(tr Reader) ([]byte, error) {
|
||||||
buf := encoding.Encbuf{}
|
buf := encoding.Encbuf{}
|
||||||
buf.PutByte(tombstoneFormatV1)
|
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 {
|
for _, iv := range ivs {
|
||||||
buf.PutUvarint64(ref)
|
buf.PutUvarint64(uint64(ref))
|
||||||
buf.PutVarint64(iv.Mint)
|
buf.PutVarint64(iv.Mint)
|
||||||
buf.PutVarint64(iv.Maxt)
|
buf.PutVarint64(iv.Maxt)
|
||||||
}
|
}
|
||||||
|
@ -169,7 +170,7 @@ func Decode(b []byte) (Reader, error) {
|
||||||
|
|
||||||
stonesMap := NewMemTombstones()
|
stonesMap := NewMemTombstones()
|
||||||
for d.Len() > 0 {
|
for d.Len() > 0 {
|
||||||
k := d.Uvarint64()
|
k := storage.SeriesRef(d.Uvarint64())
|
||||||
mint := d.Varint64()
|
mint := d.Varint64()
|
||||||
maxt := d.Varint64()
|
maxt := d.Varint64()
|
||||||
if d.Err() != nil {
|
if d.Err() != nil {
|
||||||
|
@ -184,7 +185,7 @@ func Decode(b []byte) (Reader, error) {
|
||||||
// Stone holds the information on the posting and time-range
|
// Stone holds the information on the posting and time-range
|
||||||
// that is deleted.
|
// that is deleted.
|
||||||
type Stone struct {
|
type Stone struct {
|
||||||
Ref uint64
|
Ref storage.SeriesRef
|
||||||
Intervals Intervals
|
Intervals Intervals
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -228,33 +229,33 @@ func ReadTombstones(dir string) (Reader, int64, error) {
|
||||||
}
|
}
|
||||||
|
|
||||||
type MemTombstones struct {
|
type MemTombstones struct {
|
||||||
intvlGroups map[uint64]Intervals
|
intvlGroups map[storage.SeriesRef]Intervals
|
||||||
mtx sync.RWMutex
|
mtx sync.RWMutex
|
||||||
}
|
}
|
||||||
|
|
||||||
// NewMemTombstones creates new in memory Tombstone Reader
|
// NewMemTombstones creates new in memory Tombstone Reader
|
||||||
// that allows adding new intervals.
|
// that allows adding new intervals.
|
||||||
func NewMemTombstones() *MemTombstones {
|
func NewMemTombstones() *MemTombstones {
|
||||||
return &MemTombstones{intvlGroups: make(map[uint64]Intervals)}
|
return &MemTombstones{intvlGroups: make(map[storage.SeriesRef]Intervals)}
|
||||||
}
|
}
|
||||||
|
|
||||||
func NewTestMemTombstones(intervals []Intervals) *MemTombstones {
|
func NewTestMemTombstones(intervals []Intervals) *MemTombstones {
|
||||||
ret := NewMemTombstones()
|
ret := NewMemTombstones()
|
||||||
for i, intervalsGroup := range intervals {
|
for i, intervalsGroup := range intervals {
|
||||||
for _, interval := range intervalsGroup {
|
for _, interval := range intervalsGroup {
|
||||||
ret.AddInterval(uint64(i+1), interval)
|
ret.AddInterval(storage.SeriesRef(i+1), interval)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return ret
|
return ret
|
||||||
}
|
}
|
||||||
|
|
||||||
func (t *MemTombstones) Get(ref uint64) (Intervals, error) {
|
func (t *MemTombstones) Get(ref storage.SeriesRef) (Intervals, error) {
|
||||||
t.mtx.RLock()
|
t.mtx.RLock()
|
||||||
defer t.mtx.RUnlock()
|
defer t.mtx.RUnlock()
|
||||||
return t.intvlGroups[ref], nil
|
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()
|
t.mtx.Lock()
|
||||||
defer t.mtx.Unlock()
|
defer t.mtx.Unlock()
|
||||||
for ref := range refs {
|
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()
|
t.mtx.RLock()
|
||||||
defer t.mtx.RUnlock()
|
defer t.mtx.RUnlock()
|
||||||
for ref, ivs := range t.intvlGroups {
|
for ref, ivs := range t.intvlGroups {
|
||||||
|
@ -305,7 +306,7 @@ func (t *MemTombstones) Total() uint64 {
|
||||||
}
|
}
|
||||||
|
|
||||||
// AddInterval to an existing memTombstones.
|
// 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()
|
t.mtx.Lock()
|
||||||
defer t.mtx.Unlock()
|
defer t.mtx.Unlock()
|
||||||
for _, itv := range itvs {
|
for _, itv := range itvs {
|
||||||
|
|
|
@ -25,6 +25,8 @@ import (
|
||||||
"github.com/go-kit/log"
|
"github.com/go-kit/log"
|
||||||
"github.com/stretchr/testify/require"
|
"github.com/stretchr/testify/require"
|
||||||
"go.uber.org/goleak"
|
"go.uber.org/goleak"
|
||||||
|
|
||||||
|
"github.com/prometheus/prometheus/storage"
|
||||||
)
|
)
|
||||||
|
|
||||||
func TestMain(m *testing.M) {
|
func TestMain(m *testing.M) {
|
||||||
|
@ -50,7 +52,7 @@ func TestWriteAndReadbackTombstones(t *testing.T) {
|
||||||
dranges = dranges.Add(Interval{mint, mint + rand.Int63n(1000)})
|
dranges = dranges.Add(Interval{mint, mint + rand.Int63n(1000)})
|
||||||
mint += rand.Int63n(1000) + 1
|
mint += rand.Int63n(1000) + 1
|
||||||
}
|
}
|
||||||
stones.AddInterval(ref, dranges...)
|
stones.AddInterval(storage.SeriesRef(ref), dranges...)
|
||||||
}
|
}
|
||||||
|
|
||||||
_, err := WriteFile(log.NewNopLogger(), tmpdir, stones)
|
_, err := WriteFile(log.NewNopLogger(), tmpdir, stones)
|
||||||
|
@ -66,18 +68,18 @@ func TestWriteAndReadbackTombstones(t *testing.T) {
|
||||||
func TestDeletingTombstones(t *testing.T) {
|
func TestDeletingTombstones(t *testing.T) {
|
||||||
stones := NewMemTombstones()
|
stones := NewMemTombstones()
|
||||||
|
|
||||||
ref := uint64(42)
|
ref := storage.SeriesRef(42)
|
||||||
mint := rand.Int63n(time.Now().UnixNano())
|
mint := rand.Int63n(time.Now().UnixNano())
|
||||||
dranges := make(Intervals, 0, 1)
|
dranges := make(Intervals, 0, 1)
|
||||||
dranges = dranges.Add(Interval{mint, mint + rand.Int63n(1000)})
|
dranges = dranges.Add(Interval{mint, mint + rand.Int63n(1000)})
|
||||||
stones.AddInterval(ref, dranges...)
|
stones.AddInterval(ref, dranges...)
|
||||||
stones.AddInterval(uint64(43), dranges...)
|
stones.AddInterval(storage.SeriesRef(43), dranges...)
|
||||||
|
|
||||||
intervals, err := stones.Get(ref)
|
intervals, err := stones.Get(ref)
|
||||||
require.NoError(t, err)
|
require.NoError(t, err)
|
||||||
require.Equal(t, intervals, dranges)
|
require.Equal(t, intervals, dranges)
|
||||||
|
|
||||||
stones.DeleteTombstones(map[uint64]struct{}{ref: {}})
|
stones.DeleteTombstones(map[storage.SeriesRef]struct{}{ref: {}})
|
||||||
|
|
||||||
intervals, err = stones.Get(ref)
|
intervals, err = stones.Get(ref)
|
||||||
require.NoError(t, err)
|
require.NoError(t, err)
|
||||||
|
@ -112,7 +114,7 @@ func TestTruncateBefore(t *testing.T) {
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
for _, c := range cases {
|
for _, c := range cases {
|
||||||
ref := uint64(42)
|
ref := storage.SeriesRef(42)
|
||||||
stones := NewMemTombstones()
|
stones := NewMemTombstones()
|
||||||
stones.AddInterval(ref, c.before...)
|
stones.AddInterval(ref, c.before...)
|
||||||
|
|
||||||
|
@ -231,13 +233,13 @@ func TestMemTombstonesConcurrency(t *testing.T) {
|
||||||
|
|
||||||
go func() {
|
go func() {
|
||||||
for x := 0; x < totalRuns; x++ {
|
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()
|
wg.Done()
|
||||||
}()
|
}()
|
||||||
go func() {
|
go func() {
|
||||||
for x := 0; x < totalRuns; x++ {
|
for x := 0; x < totalRuns; x++ {
|
||||||
_, err := tomb.Get(uint64(x))
|
_, err := tomb.Get(storage.SeriesRef(x))
|
||||||
require.NoError(t, err)
|
require.NoError(t, err)
|
||||||
}
|
}
|
||||||
wg.Done()
|
wg.Done()
|
||||||
|
|
|
@ -48,7 +48,7 @@ func CreateBlock(series []storage.Series, dir string, chunkRange int64, logger l
|
||||||
app := w.Appender(ctx)
|
app := w.Appender(ctx)
|
||||||
|
|
||||||
for _, s := range series {
|
for _, s := range series {
|
||||||
ref := uint64(0)
|
ref := storage.SeriesRef(0)
|
||||||
it := s.Iterator()
|
it := s.Iterator()
|
||||||
lset := s.Labels()
|
lset := s.Labels()
|
||||||
for it.Next() {
|
for it.Next() {
|
||||||
|
|
20
tsdb/wal.go
20
tsdb/wal.go
|
@ -33,6 +33,8 @@ import (
|
||||||
"github.com/prometheus/client_golang/prometheus"
|
"github.com/prometheus/client_golang/prometheus"
|
||||||
|
|
||||||
"github.com/prometheus/prometheus/pkg/labels"
|
"github.com/prometheus/prometheus/pkg/labels"
|
||||||
|
"github.com/prometheus/prometheus/storage"
|
||||||
|
"github.com/prometheus/prometheus/tsdb/chunks"
|
||||||
"github.com/prometheus/prometheus/tsdb/encoding"
|
"github.com/prometheus/prometheus/tsdb/encoding"
|
||||||
"github.com/prometheus/prometheus/tsdb/fileutil"
|
"github.com/prometheus/prometheus/tsdb/fileutil"
|
||||||
"github.com/prometheus/prometheus/tsdb/record"
|
"github.com/prometheus/prometheus/tsdb/record"
|
||||||
|
@ -113,8 +115,8 @@ type WALReader interface {
|
||||||
// the truncation threshold can be compacted.
|
// the truncation threshold can be compacted.
|
||||||
type segmentFile struct {
|
type segmentFile struct {
|
||||||
*os.File
|
*os.File
|
||||||
maxTime int64 // highest tombstone or sample timestamp in segment
|
maxTime int64 // highest tombstone or sample timestamp in segment
|
||||||
minSeries uint64 // lowerst series ID in segment
|
minSeries chunks.HeadSeriesRef // lowerst series ID in segment
|
||||||
}
|
}
|
||||||
|
|
||||||
func newSegmentFile(f *os.File) *segmentFile {
|
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
|
// Truncate deletes the values prior to mint and the series which the keep function
|
||||||
// does not indicate to preserve.
|
// 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.
|
// The last segment is always active.
|
||||||
if len(w.files) < 2 {
|
if len(w.files) < 2 {
|
||||||
return nil
|
return nil
|
||||||
|
@ -787,7 +789,7 @@ const (
|
||||||
|
|
||||||
func (w *SegmentWAL) encodeSeries(buf *encoding.Encbuf, series []record.RefSeries) uint8 {
|
func (w *SegmentWAL) encodeSeries(buf *encoding.Encbuf, series []record.RefSeries) uint8 {
|
||||||
for _, s := range series {
|
for _, s := range series {
|
||||||
buf.PutBE64(s.Ref)
|
buf.PutBE64(uint64(s.Ref))
|
||||||
buf.PutUvarint(len(s.Labels))
|
buf.PutUvarint(len(s.Labels))
|
||||||
|
|
||||||
for _, l := range 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.
|
// TODO(fabxc): optimize for all samples having the same timestamp.
|
||||||
first := samples[0]
|
first := samples[0]
|
||||||
|
|
||||||
buf.PutBE64(first.Ref)
|
buf.PutBE64(uint64(first.Ref))
|
||||||
buf.PutBE64int64(first.T)
|
buf.PutBE64int64(first.T)
|
||||||
|
|
||||||
for _, s := range samples {
|
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 {
|
func (w *SegmentWAL) encodeDeletes(buf *encoding.Encbuf, stones []tombstones.Stone) uint8 {
|
||||||
for _, s := range stones {
|
for _, s := range stones {
|
||||||
for _, iv := range s.Intervals {
|
for _, iv := range s.Intervals {
|
||||||
buf.PutBE64(s.Ref)
|
buf.PutBE64(uint64(s.Ref))
|
||||||
buf.PutVarint64(iv.Mint)
|
buf.PutVarint64(iv.Mint)
|
||||||
buf.PutVarint64(iv.Maxt)
|
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}
|
dec := encoding.Decbuf{B: b}
|
||||||
|
|
||||||
for len(dec.B) > 0 && dec.Err() == nil {
|
for len(dec.B) > 0 && dec.Err() == nil {
|
||||||
ref := dec.Be64()
|
ref := chunks.HeadSeriesRef(dec.Be64())
|
||||||
|
|
||||||
lset := make(labels.Labels, dec.Uvarint())
|
lset := make(labels.Labels, dec.Uvarint())
|
||||||
|
|
||||||
|
@ -1161,7 +1163,7 @@ func (r *walReader) decodeSamples(flag byte, b []byte, res *[]record.RefSample)
|
||||||
val := dec.Be64()
|
val := dec.Be64()
|
||||||
|
|
||||||
*res = append(*res, record.RefSample{
|
*res = append(*res, record.RefSample{
|
||||||
Ref: uint64(int64(baseRef) + dref),
|
Ref: chunks.HeadSeriesRef(int64(baseRef) + dref),
|
||||||
T: baseTime + dtime,
|
T: baseTime + dtime,
|
||||||
V: math.Float64frombits(val),
|
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 {
|
for dec.Len() > 0 && dec.Err() == nil {
|
||||||
*res = append(*res, tombstones.Stone{
|
*res = append(*res, tombstones.Stone{
|
||||||
Ref: dec.Be64(),
|
Ref: storage.SeriesRef(dec.Be64()),
|
||||||
Intervals: tombstones.Intervals{
|
Intervals: tombstones.Intervals{
|
||||||
{Mint: dec.Varint64(), Maxt: dec.Varint64()},
|
{Mint: dec.Varint64(), Maxt: dec.Varint64()},
|
||||||
},
|
},
|
||||||
|
|
|
@ -29,6 +29,7 @@ import (
|
||||||
"github.com/go-kit/log/level"
|
"github.com/go-kit/log/level"
|
||||||
"github.com/pkg/errors"
|
"github.com/pkg/errors"
|
||||||
|
|
||||||
|
"github.com/prometheus/prometheus/tsdb/chunks"
|
||||||
tsdb_errors "github.com/prometheus/prometheus/tsdb/errors"
|
tsdb_errors "github.com/prometheus/prometheus/tsdb/errors"
|
||||||
"github.com/prometheus/prometheus/tsdb/fileutil"
|
"github.com/prometheus/prometheus/tsdb/fileutil"
|
||||||
"github.com/prometheus/prometheus/tsdb/record"
|
"github.com/prometheus/prometheus/tsdb/record"
|
||||||
|
@ -90,7 +91,7 @@ const checkpointPrefix = "checkpoint."
|
||||||
// segmented format as the original WAL itself.
|
// segmented format as the original WAL itself.
|
||||||
// This makes it easy to read it through the WAL package and concatenate
|
// This makes it easy to read it through the WAL package and concatenate
|
||||||
// it with the original WAL.
|
// 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{}
|
stats := &CheckpointStats{}
|
||||||
var sgmReader io.ReadCloser
|
var sgmReader io.ReadCloser
|
||||||
|
|
||||||
|
|
|
@ -27,6 +27,7 @@ import (
|
||||||
"github.com/stretchr/testify/require"
|
"github.com/stretchr/testify/require"
|
||||||
|
|
||||||
"github.com/prometheus/prometheus/pkg/labels"
|
"github.com/prometheus/prometheus/pkg/labels"
|
||||||
|
"github.com/prometheus/prometheus/tsdb/chunks"
|
||||||
"github.com/prometheus/prometheus/tsdb/record"
|
"github.com/prometheus/prometheus/tsdb/record"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
@ -174,7 +175,7 @@ func TestCheckpoint(t *testing.T) {
|
||||||
}
|
}
|
||||||
require.NoError(t, w.Close())
|
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
|
return x%2 == 0
|
||||||
}, last/2)
|
}, last/2)
|
||||||
require.NoError(t, err)
|
require.NoError(t, err)
|
||||||
|
|
|
@ -26,6 +26,7 @@ import (
|
||||||
"github.com/stretchr/testify/require"
|
"github.com/stretchr/testify/require"
|
||||||
|
|
||||||
"github.com/prometheus/prometheus/pkg/labels"
|
"github.com/prometheus/prometheus/pkg/labels"
|
||||||
|
"github.com/prometheus/prometheus/tsdb/chunks"
|
||||||
"github.com/prometheus/prometheus/tsdb/record"
|
"github.com/prometheus/prometheus/tsdb/record"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
@ -53,7 +54,7 @@ type writeToMock struct {
|
||||||
samplesAppended int
|
samplesAppended int
|
||||||
exemplarsAppended int
|
exemplarsAppended int
|
||||||
seriesLock sync.Mutex
|
seriesLock sync.Mutex
|
||||||
seriesSegmentIndexes map[uint64]int
|
seriesSegmentIndexes map[chunks.HeadSeriesRef]int
|
||||||
}
|
}
|
||||||
|
|
||||||
func (wtm *writeToMock) Append(s []record.RefSample) bool {
|
func (wtm *writeToMock) Append(s []record.RefSample) bool {
|
||||||
|
@ -98,7 +99,7 @@ func (wtm *writeToMock) checkNumLabels() int {
|
||||||
|
|
||||||
func newWriteToMock() *writeToMock {
|
func newWriteToMock() *writeToMock {
|
||||||
return &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
|
ref := i + 100
|
||||||
series := enc.Series([]record.RefSeries{
|
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)}},
|
Labels: labels.Labels{labels.Label{Name: "__name__", Value: fmt.Sprintf("metric_%d", i)}},
|
||||||
},
|
},
|
||||||
}, nil)
|
}, nil)
|
||||||
|
@ -139,7 +140,7 @@ func TestTailSamples(t *testing.T) {
|
||||||
inner := rand.Intn(ref + 1)
|
inner := rand.Intn(ref + 1)
|
||||||
sample := enc.Samples([]record.RefSample{
|
sample := enc.Samples([]record.RefSample{
|
||||||
{
|
{
|
||||||
Ref: uint64(inner),
|
Ref: chunks.HeadSeriesRef(inner),
|
||||||
T: now.UnixNano() + 1,
|
T: now.UnixNano() + 1,
|
||||||
V: float64(i),
|
V: float64(i),
|
||||||
},
|
},
|
||||||
|
@ -151,7 +152,7 @@ func TestTailSamples(t *testing.T) {
|
||||||
inner := rand.Intn(ref + 1)
|
inner := rand.Intn(ref + 1)
|
||||||
exemplar := enc.Exemplars([]record.RefExemplar{
|
exemplar := enc.Exemplars([]record.RefExemplar{
|
||||||
{
|
{
|
||||||
Ref: uint64(inner),
|
Ref: chunks.HeadSeriesRef(inner),
|
||||||
T: now.UnixNano() + 1,
|
T: now.UnixNano() + 1,
|
||||||
V: float64(i),
|
V: float64(i),
|
||||||
Labels: labels.FromStrings("traceID", fmt.Sprintf("trace-%d", inner)),
|
Labels: labels.FromStrings("traceID", fmt.Sprintf("trace-%d", inner)),
|
||||||
|
@ -219,7 +220,7 @@ func TestReadToEndNoCheckpoint(t *testing.T) {
|
||||||
for i := 0; i < seriesCount; i++ {
|
for i := 0; i < seriesCount; i++ {
|
||||||
series := enc.Series([]record.RefSeries{
|
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)}},
|
Labels: labels.Labels{labels.Label{Name: "__name__", Value: fmt.Sprintf("metric_%d", i)}},
|
||||||
},
|
},
|
||||||
}, nil)
|
}, nil)
|
||||||
|
@ -227,7 +228,7 @@ func TestReadToEndNoCheckpoint(t *testing.T) {
|
||||||
for j := 0; j < samplesCount; j++ {
|
for j := 0; j < samplesCount; j++ {
|
||||||
sample := enc.Samples([]record.RefSample{
|
sample := enc.Samples([]record.RefSample{
|
||||||
{
|
{
|
||||||
Ref: uint64(j),
|
Ref: chunks.HeadSeriesRef(j),
|
||||||
T: int64(i),
|
T: int64(i),
|
||||||
V: float64(i),
|
V: float64(i),
|
||||||
},
|
},
|
||||||
|
@ -288,7 +289,7 @@ func TestReadToEndWithCheckpoint(t *testing.T) {
|
||||||
ref := i + 100
|
ref := i + 100
|
||||||
series := enc.Series([]record.RefSeries{
|
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)}},
|
Labels: labels.Labels{labels.Label{Name: "__name__", Value: fmt.Sprintf("metric_%d", i)}},
|
||||||
},
|
},
|
||||||
}, nil)
|
}, nil)
|
||||||
|
@ -300,7 +301,7 @@ func TestReadToEndWithCheckpoint(t *testing.T) {
|
||||||
inner := rand.Intn(ref + 1)
|
inner := rand.Intn(ref + 1)
|
||||||
sample := enc.Samples([]record.RefSample{
|
sample := enc.Samples([]record.RefSample{
|
||||||
{
|
{
|
||||||
Ref: uint64(inner),
|
Ref: chunks.HeadSeriesRef(inner),
|
||||||
T: int64(i),
|
T: int64(i),
|
||||||
V: float64(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)
|
w.Truncate(1)
|
||||||
|
|
||||||
// Write more records after checkpointing.
|
// Write more records after checkpointing.
|
||||||
for i := 0; i < seriesCount; i++ {
|
for i := 0; i < seriesCount; i++ {
|
||||||
series := enc.Series([]record.RefSeries{
|
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)}},
|
Labels: labels.Labels{labels.Label{Name: "__name__", Value: fmt.Sprintf("metric_%d", i)}},
|
||||||
},
|
},
|
||||||
}, nil)
|
}, nil)
|
||||||
|
@ -325,7 +326,7 @@ func TestReadToEndWithCheckpoint(t *testing.T) {
|
||||||
for j := 0; j < samplesCount; j++ {
|
for j := 0; j < samplesCount; j++ {
|
||||||
sample := enc.Samples([]record.RefSample{
|
sample := enc.Samples([]record.RefSample{
|
||||||
{
|
{
|
||||||
Ref: uint64(j),
|
Ref: chunks.HeadSeriesRef(j),
|
||||||
T: int64(i),
|
T: int64(i),
|
||||||
V: float64(i),
|
V: float64(i),
|
||||||
},
|
},
|
||||||
|
@ -377,7 +378,7 @@ func TestReadCheckpoint(t *testing.T) {
|
||||||
ref := i + 100
|
ref := i + 100
|
||||||
series := enc.Series([]record.RefSeries{
|
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)}},
|
Labels: labels.Labels{labels.Label{Name: "__name__", Value: fmt.Sprintf("metric_%d", i)}},
|
||||||
},
|
},
|
||||||
}, nil)
|
}, nil)
|
||||||
|
@ -387,7 +388,7 @@ func TestReadCheckpoint(t *testing.T) {
|
||||||
inner := rand.Intn(ref + 1)
|
inner := rand.Intn(ref + 1)
|
||||||
sample := enc.Samples([]record.RefSample{
|
sample := enc.Samples([]record.RefSample{
|
||||||
{
|
{
|
||||||
Ref: uint64(inner),
|
Ref: chunks.HeadSeriesRef(inner),
|
||||||
T: int64(i),
|
T: int64(i),
|
||||||
V: float64(i),
|
V: float64(i),
|
||||||
},
|
},
|
||||||
|
@ -395,7 +396,7 @@ func TestReadCheckpoint(t *testing.T) {
|
||||||
require.NoError(t, w.Log(sample))
|
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)
|
w.Truncate(32)
|
||||||
|
|
||||||
// Start read after checkpoint, no more data written.
|
// Start read after checkpoint, no more data written.
|
||||||
|
@ -441,7 +442,7 @@ func TestReadCheckpointMultipleSegments(t *testing.T) {
|
||||||
ref := j + (i * 100)
|
ref := j + (i * 100)
|
||||||
series := enc.Series([]record.RefSeries{
|
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)}},
|
Labels: labels.Labels{labels.Label{Name: "__name__", Value: fmt.Sprintf("metric_%d", j)}},
|
||||||
},
|
},
|
||||||
}, nil)
|
}, nil)
|
||||||
|
@ -451,7 +452,7 @@ func TestReadCheckpointMultipleSegments(t *testing.T) {
|
||||||
inner := rand.Intn(ref + 1)
|
inner := rand.Intn(ref + 1)
|
||||||
sample := enc.Samples([]record.RefSample{
|
sample := enc.Samples([]record.RefSample{
|
||||||
{
|
{
|
||||||
Ref: uint64(inner),
|
Ref: chunks.HeadSeriesRef(inner),
|
||||||
T: int64(i),
|
T: int64(i),
|
||||||
V: float64(i),
|
V: float64(i),
|
||||||
},
|
},
|
||||||
|
@ -521,7 +522,7 @@ func TestCheckpointSeriesReset(t *testing.T) {
|
||||||
ref := i + 100
|
ref := i + 100
|
||||||
series := enc.Series([]record.RefSeries{
|
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)}},
|
Labels: labels.Labels{labels.Label{Name: "__name__", Value: fmt.Sprintf("metric_%d", i)}},
|
||||||
},
|
},
|
||||||
}, nil)
|
}, nil)
|
||||||
|
@ -531,7 +532,7 @@ func TestCheckpointSeriesReset(t *testing.T) {
|
||||||
inner := rand.Intn(ref + 1)
|
inner := rand.Intn(ref + 1)
|
||||||
sample := enc.Samples([]record.RefSample{
|
sample := enc.Samples([]record.RefSample{
|
||||||
{
|
{
|
||||||
Ref: uint64(inner),
|
Ref: chunks.HeadSeriesRef(inner),
|
||||||
T: int64(i),
|
T: int64(i),
|
||||||
V: float64(i),
|
V: float64(i),
|
||||||
},
|
},
|
||||||
|
@ -554,7 +555,7 @@ func TestCheckpointSeriesReset(t *testing.T) {
|
||||||
})
|
})
|
||||||
require.Equal(t, seriesCount, wt.checkNumLabels())
|
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)
|
require.NoError(t, err)
|
||||||
|
|
||||||
err = w.Truncate(5)
|
err = w.Truncate(5)
|
||||||
|
|
|
@ -31,6 +31,8 @@ import (
|
||||||
"github.com/stretchr/testify/require"
|
"github.com/stretchr/testify/require"
|
||||||
|
|
||||||
"github.com/prometheus/prometheus/pkg/labels"
|
"github.com/prometheus/prometheus/pkg/labels"
|
||||||
|
"github.com/prometheus/prometheus/storage"
|
||||||
|
"github.com/prometheus/prometheus/tsdb/chunks"
|
||||||
"github.com/prometheus/prometheus/tsdb/record"
|
"github.com/prometheus/prometheus/tsdb/record"
|
||||||
"github.com/prometheus/prometheus/tsdb/tombstones"
|
"github.com/prometheus/prometheus/tsdb/tombstones"
|
||||||
"github.com/prometheus/prometheus/tsdb/wal"
|
"github.com/prometheus/prometheus/tsdb/wal"
|
||||||
|
@ -102,7 +104,7 @@ func TestSegmentWAL_Truncate(t *testing.T) {
|
||||||
var rs []record.RefSeries
|
var rs []record.RefSeries
|
||||||
|
|
||||||
for j, s := range series[i : i+batch] {
|
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)
|
err := w.LogSeries(rs)
|
||||||
require.NoError(t, err)
|
require.NoError(t, err)
|
||||||
|
@ -117,11 +119,11 @@ func TestSegmentWAL_Truncate(t *testing.T) {
|
||||||
boundarySeries := w.files[len(w.files)/2].minSeries
|
boundarySeries := w.files[len(w.files)/2].minSeries
|
||||||
|
|
||||||
// We truncate while keeping every 2nd series.
|
// We truncate while keeping every 2nd series.
|
||||||
keep := map[uint64]struct{}{}
|
keep := map[chunks.HeadSeriesRef]struct{}{}
|
||||||
for i := 1; i <= numMetrics; i += 2 {
|
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]
|
_, ok := keep[id]
|
||||||
return ok
|
return ok
|
||||||
}
|
}
|
||||||
|
@ -132,8 +134,8 @@ func TestSegmentWAL_Truncate(t *testing.T) {
|
||||||
var expected []record.RefSeries
|
var expected []record.RefSeries
|
||||||
|
|
||||||
for i := 1; i <= numMetrics; i++ {
|
for i := 1; i <= numMetrics; i++ {
|
||||||
if i%2 == 1 || uint64(i) >= boundarySeries {
|
if i%2 == 1 || chunks.HeadSeriesRef(i) >= boundarySeries {
|
||||||
expected = append(expected, record.RefSeries{Ref: uint64(i), Labels: series[i-1]})
|
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++ {
|
for j := 0; j < i*10; j++ {
|
||||||
samples = append(samples, record.RefSample{
|
samples = append(samples, record.RefSample{
|
||||||
Ref: uint64(j % 10000),
|
Ref: chunks.HeadSeriesRef(j % 10000),
|
||||||
T: int64(j * 2),
|
T: int64(j * 2),
|
||||||
V: rand.Float64(),
|
V: rand.Float64(),
|
||||||
})
|
})
|
||||||
|
@ -246,14 +248,14 @@ func TestSegmentWAL_Log_Restore(t *testing.T) {
|
||||||
|
|
||||||
for j := 0; j < i*20; j++ {
|
for j := 0; j < i*20; j++ {
|
||||||
ts := rand.Int63()
|
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]
|
lbls := series[i : i+stepSize]
|
||||||
series := make([]record.RefSeries, 0, len(series))
|
series := make([]record.RefSeries, 0, len(series))
|
||||||
for j, l := range lbls {
|
for j, l := range lbls {
|
||||||
series = append(series, record.RefSeries{
|
series = append(series, record.RefSeries{
|
||||||
Ref: uint64(i + j),
|
Ref: chunks.HeadSeriesRef(i + j),
|
||||||
Labels: l,
|
Labels: l,
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
|
@ -70,6 +70,6 @@ func (s TestStorage) ExemplarQueryable() storage.ExemplarQueryable {
|
||||||
return s.exemplarStorage
|
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)
|
return ref, s.exemplarStorage.AddExemplar(l, e)
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue