From a1d088eee8a1301a33f2f29f55e8f4649998215d Mon Sep 17 00:00:00 2001 From: bwplotka Date: Fri, 28 Nov 2025 12:43:12 +0000 Subject: [PATCH] refactor(appenderV2): add TSDB AppenderV2 implementation Signed-off-by: bwplotka tmp Signed-off-by: bwplotka --- model/metadata/metadata.go | 27 +- storage/interface.go | 1 + tsdb/db.go | 37 +- tsdb/head.go | 14 + tsdb/head_append.go | 55 +- tsdb/head_append_v2.go | 2366 +++-------------------- tsdb/head_append_v2_test.go | 3595 +++++------------------------------ tsdb/head_bench_test.go | 298 ++- tsdb/head_bench_v2_test.go | 173 -- tsdb/head_test.go | 46 +- tsdb/testutil.go | 16 +- 11 files changed, 1033 insertions(+), 5595 deletions(-) delete mode 100644 tsdb/head_bench_v2_test.go diff --git a/model/metadata/metadata.go b/model/metadata/metadata.go index 1b7e63e0f3..db5d7cb6c0 100644 --- a/model/metadata/metadata.go +++ b/model/metadata/metadata.go @@ -1,4 +1,4 @@ -// Copyright 2022 The Prometheus Authors +// Copyright The Prometheus Authors // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. // You may obtain a copy of the License at @@ -13,7 +13,11 @@ package metadata -import "github.com/prometheus/common/model" +import ( + "strings" + + "github.com/prometheus/common/model" +) // Metadata stores a series' metadata information. type Metadata struct { @@ -21,3 +25,22 @@ type Metadata struct { Unit string `json:"unit"` Help string `json:"help"` } + +// IsEmpty returns true if metadata structure is empty, including unknown type case. +func (m Metadata) IsEmpty() bool { + return (m.Type == "" || m.Type == model.MetricTypeUnknown) && m.Unit == "" && m.Help == "" +} + +// Equals returns true if m is semantically the same as other metadata. +func (m Metadata) Equals(other Metadata) bool { + if strings.Compare(m.Unit, other.Unit) != 0 || strings.Compare(m.Help, other.Help) != 0 { + return false + } + + if m.Type != "" && m.Type != model.MetricTypeUnknown { + if m.Type != other.Type { + return false + } + } + return true +} diff --git a/storage/interface.go b/storage/interface.go index 3970d4ec7f..4011de8ce5 100644 --- a/storage/interface.go +++ b/storage/interface.go @@ -49,6 +49,7 @@ var ( // NOTE(bwplotka): This can be both an instrumentation failure or commonly expected // behaviour, and we currently don't have a way to determine this. As a result // it's recommended to ignore this error for now. + // TODO(bwplotka): Remove with appender v1 flow; not used in v2. ErrOutOfOrderST = errors.New("start timestamp out of order, ignoring") ErrSTNewerThanSample = errors.New("ST is newer or the same as sample's timestamp, ignoring") ) diff --git a/tsdb/db.go b/tsdb/db.go index c57ae84c9c..16f91a3b41 100644 --- a/tsdb/db.go +++ b/tsdb/db.go @@ -1131,11 +1131,16 @@ func (db *DB) run(ctx context.Context) { } } -// Appender opens a new appender against the database. +// Appender opens a new Appender against the database. func (db *DB) Appender(ctx context.Context) storage.Appender { return dbAppender{db: db, Appender: db.head.Appender(ctx)} } +// AppenderV2 opens a new AppenderV2 against the database. +func (db *DB) AppenderV2(ctx context.Context) storage.AppenderV2 { + return dbAppenderV2{db: db, AppenderV2: db.head.AppenderV2(ctx)} +} + // ApplyConfig applies a new config to the DB. // Behaviour of 'OutOfOrderTimeWindow' is as follows: // OOO enabled = oooTimeWindow > 0. OOO disabled = oooTimeWindow is 0. @@ -1249,6 +1254,36 @@ func (a dbAppender) Commit() error { return err } +// dbAppenderV2 wraps the DB's head appender and triggers compactions on commit +// if necessary. +type dbAppenderV2 struct { + storage.AppenderV2 + db *DB +} + +var _ storage.GetRef = dbAppenderV2{} + +func (a dbAppenderV2) GetRef(lset labels.Labels, hash uint64) (storage.SeriesRef, labels.Labels) { + if g, ok := a.AppenderV2.(storage.GetRef); ok { + return g.GetRef(lset, hash) + } + return 0, labels.EmptyLabels() +} + +func (a dbAppenderV2) Commit() error { + err := a.AppenderV2.Commit() + + // We could just run this check every few minutes practically. But for benchmarks + // and high frequency use cases this is the safer way. + if a.db.head.compactable() { + select { + case a.db.compactc <- struct{}{}: + default: + } + } + return err +} + // waitingForCompactionDelay returns true if the DB is waiting for the Head compaction delay. // This doesn't guarantee that the Head is really compactable. func (db *DB) waitingForCompactionDelay() bool { diff --git a/tsdb/head.go b/tsdb/head.go index cf773e82b0..25a1b88cec 100644 --- a/tsdb/head.go +++ b/tsdb/head.go @@ -187,6 +187,20 @@ type HeadOptions struct { // EnableSharding enables ShardedPostings() support in the Head. EnableSharding bool + + // EnableSTAsZeroSample represents 'created-timestamp-zero-ingestion' feature flag. + // If true, ST, if non-empty and earlier than sample timestamp, will be stored + // as a zero sample before the actual sample. + // + // The zero sample is best-effort, only debug log on failure is emitted. + // NOTE(bwplotka): This feature might be deprecated and removed once PROM-60 + // is implemented. + EnableSTAsZeroSample bool + + // EnableMetadataWALRecords represents 'metadata-wal-records' feature flag. + // NOTE(bwplotka): This feature might be deprecated and removed once PROM-60 + // is implemented. + EnableMetadataWALRecords bool } const ( diff --git a/tsdb/head_append.go b/tsdb/head_append.go index 942c3ce974..356d1c453f 100644 --- a/tsdb/head_append.go +++ b/tsdb/head_append.go @@ -165,17 +165,19 @@ func (h *Head) appender() *headAppender { minValidTime := h.appendableMinValidTime() appendID, cleanupAppendIDsBelow := h.iso.newAppendID(minValidTime) // Every appender gets an ID that is cleared upon commit/rollback. return &headAppender{ - head: h, - minValidTime: minValidTime, - mint: math.MaxInt64, - maxt: math.MinInt64, - headMaxt: h.MaxTime(), - oooTimeWindow: h.opts.OutOfOrderTimeWindow.Load(), - seriesRefs: h.getRefSeriesBuffer(), - series: h.getSeriesBuffer(), - typesInBatch: h.getTypeMap(), - appendID: appendID, - cleanupAppendIDsBelow: cleanupAppendIDsBelow, + headAppenderBase: headAppenderBase{ + head: h, + minValidTime: minValidTime, + mint: math.MaxInt64, + maxt: math.MinInt64, + headMaxt: h.MaxTime(), + oooTimeWindow: h.opts.OutOfOrderTimeWindow.Load(), + seriesRefs: h.getRefSeriesBuffer(), + series: h.getSeriesBuffer(), + typesInBatch: h.getTypeMap(), + appendID: appendID, + cleanupAppendIDsBelow: cleanupAppendIDsBelow, + }, } } @@ -382,7 +384,7 @@ func (b *appendBatch) close(h *Head) { b.exemplars = nil } -type headAppender struct { +type headAppenderBase struct { head *Head minValidTime int64 // No samples below this timestamp are allowed. mint, maxt int64 @@ -397,7 +399,10 @@ type headAppender struct { appendID, cleanupAppendIDsBelow uint64 closed bool - hints *storage.AppendOptions +} +type headAppender struct { + headAppenderBase + hints *storage.AppendOptions } func (a *headAppender) SetOptions(opts *storage.AppendOptions) { @@ -525,7 +530,7 @@ func (a *headAppender) AppendSTZeroSample(ref storage.SeriesRef, lset labels.Lab return storage.SeriesRef(s.ref), nil } -func (a *headAppender) getOrCreate(lset labels.Labels) (s *memSeries, created bool, err error) { +func (a *headAppenderBase) getOrCreate(lset labels.Labels) (s *memSeries, created bool, err error) { // Ensure no empty labels have gotten through. lset = lset.WithoutEmpty() if lset.IsEmpty() { @@ -550,7 +555,7 @@ func (a *headAppender) getOrCreate(lset labels.Labels) (s *memSeries, created bo // getCurrentBatch returns the current batch if it fits the provided sampleType // for the provided series. Otherwise, it adds a new batch and returns it. -func (a *headAppender) getCurrentBatch(st sampleType, s chunks.HeadSeriesRef) *appendBatch { +func (a *headAppenderBase) getCurrentBatch(st sampleType, s chunks.HeadSeriesRef) *appendBatch { h := a.head newBatch := func() *appendBatch { @@ -1043,7 +1048,7 @@ func (a *headAppender) UpdateMetadata(ref storage.SeriesRef, lset labels.Labels, var _ storage.GetRef = &headAppender{} -func (a *headAppender) GetRef(lset labels.Labels, hash uint64) (storage.SeriesRef, labels.Labels) { +func (a *headAppenderBase) GetRef(lset labels.Labels, hash uint64) (storage.SeriesRef, labels.Labels) { s := a.head.series.getByHash(hash, lset) if s == nil { return 0, labels.EmptyLabels() @@ -1053,7 +1058,7 @@ func (a *headAppender) GetRef(lset labels.Labels, hash uint64) (storage.SeriesRe } // log writes all headAppender's data to the WAL. -func (a *headAppender) log() error { +func (a *headAppenderBase) log() error { if a.head.wal == nil { return nil } @@ -1185,7 +1190,7 @@ type appenderCommitContext struct { } // commitExemplars adds all exemplars from the provided batch to the head's exemplar storage. -func (a *headAppender) commitExemplars(b *appendBatch) { +func (a *headAppenderBase) commitExemplars(b *appendBatch) { // No errors logging to WAL, so pass the exemplars along to the in memory storage. for _, e := range b.exemplars { s := a.head.series.getByID(chunks.HeadSeriesRef(e.ref)) @@ -1205,7 +1210,7 @@ func (a *headAppender) commitExemplars(b *appendBatch) { } } -func (acc *appenderCommitContext) collectOOORecords(a *headAppender) { +func (acc *appenderCommitContext) collectOOORecords(a *headAppenderBase) { if a.head.wbl == nil { // WBL is not enabled. So no need to collect. acc.wblSamples = nil @@ -1310,7 +1315,7 @@ func handleAppendableError(err error, appended, oooRejected, oobRejected, tooOld // operations on the series after appending the samples. // // There are also specific functions to commit histograms and float histograms. -func (a *headAppender) commitFloats(b *appendBatch, acc *appenderCommitContext) { +func (a *headAppenderBase) commitFloats(b *appendBatch, acc *appenderCommitContext) { var ok, chunkCreated bool var series *memSeries @@ -1466,7 +1471,7 @@ func (a *headAppender) commitFloats(b *appendBatch, acc *appenderCommitContext) } // For details on the commitHistograms function, see the commitFloats docs. -func (a *headAppender) commitHistograms(b *appendBatch, acc *appenderCommitContext) { +func (a *headAppenderBase) commitHistograms(b *appendBatch, acc *appenderCommitContext) { var ok, chunkCreated bool var series *memSeries @@ -1575,7 +1580,7 @@ func (a *headAppender) commitHistograms(b *appendBatch, acc *appenderCommitConte } // For details on the commitFloatHistograms function, see the commitFloats docs. -func (a *headAppender) commitFloatHistograms(b *appendBatch, acc *appenderCommitContext) { +func (a *headAppenderBase) commitFloatHistograms(b *appendBatch, acc *appenderCommitContext) { var ok, chunkCreated bool var series *memSeries @@ -1697,7 +1702,7 @@ func commitMetadata(b *appendBatch) { } } -func (a *headAppender) unmarkCreatedSeriesAsPendingCommit() { +func (a *headAppenderBase) unmarkCreatedSeriesAsPendingCommit() { for _, s := range a.series { s.Lock() s.pendingCommit = false @@ -1707,7 +1712,7 @@ func (a *headAppender) unmarkCreatedSeriesAsPendingCommit() { // Commit writes to the WAL and adds the data to the Head. // TODO(codesome): Refactor this method to reduce indentation and make it more readable. -func (a *headAppender) Commit() (err error) { +func (a *headAppenderBase) Commit() (err error) { if a.closed { return ErrAppenderClosed } @@ -2238,7 +2243,7 @@ func handleChunkWriteError(err error) { } // Rollback removes the samples and exemplars from headAppender and writes any series to WAL. -func (a *headAppender) Rollback() (err error) { +func (a *headAppenderBase) Rollback() (err error) { if a.closed { return ErrAppenderClosed } diff --git a/tsdb/head_append_v2.go b/tsdb/head_append_v2.go index 942c3ce974..4455b12be3 100644 --- a/tsdb/head_append_v2.go +++ b/tsdb/head_append_v2.go @@ -1,4 +1,4 @@ -// Copyright 2021 The Prometheus Authors +// Copyright The Prometheus Authors // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. // You may obtain a copy of the License at @@ -17,121 +17,42 @@ import ( "context" "errors" "fmt" - "log/slog" "math" "github.com/prometheus/prometheus/model/exemplar" "github.com/prometheus/prometheus/model/histogram" "github.com/prometheus/prometheus/model/labels" - "github.com/prometheus/prometheus/model/metadata" "github.com/prometheus/prometheus/model/value" "github.com/prometheus/prometheus/storage" - "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/record" ) -// initAppender is a helper to initialize the time bounds of the head +// initAppenderV2 is a helper to initialize the time bounds of the head // upon the first sample it receives. -type initAppender struct { - app storage.Appender +type initAppenderV2 struct { + app storage.AppenderV2 head *Head } -var _ storage.GetRef = &initAppender{} +var _ storage.GetRef = &initAppenderV2{} -func (a *initAppender) SetOptions(opts *storage.AppendOptions) { - if a.app != nil { - a.app.SetOptions(opts) +func (a *initAppenderV2) Append(ref storage.SeriesRef, ls labels.Labels, st, t int64, v float64, h *histogram.Histogram, fh *histogram.FloatHistogram, opts storage.AOptions) (storage.SeriesRef, error) { + if a.app == nil { + a.head.initTime(t) + a.app = a.head.appenderV2() } + return a.app.Append(ref, ls, st, t, v, h, fh, opts) } -func (a *initAppender) Append(ref storage.SeriesRef, lset labels.Labels, t int64, v float64) (storage.SeriesRef, error) { - if a.app != nil { - return a.app.Append(ref, lset, t, v) - } - - a.head.initTime(t) - a.app = a.head.appender() - return a.app.Append(ref, lset, t, v) -} - -func (a *initAppender) AppendExemplar(ref storage.SeriesRef, l labels.Labels, e exemplar.Exemplar) (storage.SeriesRef, error) { - // Check if exemplar storage is enabled. - if !a.head.opts.EnableExemplarStorage || a.head.opts.MaxExemplars.Load() <= 0 { - return 0, nil - } - - if a.app != nil { - return a.app.AppendExemplar(ref, l, e) - } - // We should never reach here given we would call Append before AppendExemplar - // and we probably want to always base head/WAL min time on sample times. - a.head.initTime(e.Ts) - a.app = a.head.appender() - - return a.app.AppendExemplar(ref, l, e) -} - -func (a *initAppender) AppendHistogram(ref storage.SeriesRef, l labels.Labels, t int64, h *histogram.Histogram, fh *histogram.FloatHistogram) (storage.SeriesRef, error) { - if a.app != nil { - return a.app.AppendHistogram(ref, l, t, h, fh) - } - a.head.initTime(t) - a.app = a.head.appender() - - return a.app.AppendHistogram(ref, l, t, h, fh) -} - -func (a *initAppender) AppendHistogramSTZeroSample(ref storage.SeriesRef, l labels.Labels, t, st int64, h *histogram.Histogram, fh *histogram.FloatHistogram) (storage.SeriesRef, error) { - if a.app != nil { - return a.app.AppendHistogramSTZeroSample(ref, l, t, st, h, fh) - } - a.head.initTime(t) - a.app = a.head.appender() - - return a.app.AppendHistogramSTZeroSample(ref, l, t, st, h, fh) -} - -func (a *initAppender) UpdateMetadata(ref storage.SeriesRef, l labels.Labels, m metadata.Metadata) (storage.SeriesRef, error) { - if a.app != nil { - return a.app.UpdateMetadata(ref, l, m) - } - - a.app = a.head.appender() - return a.app.UpdateMetadata(ref, l, m) -} - -func (a *initAppender) AppendSTZeroSample(ref storage.SeriesRef, lset labels.Labels, t, st int64) (storage.SeriesRef, error) { - if a.app != nil { - return a.app.AppendSTZeroSample(ref, lset, t, st) - } - - a.head.initTime(t) - a.app = a.head.appender() - - return a.app.AppendSTZeroSample(ref, lset, t, st) -} - -// initTime initializes a head with the first timestamp. This only needs to be called -// for a completely fresh head with an empty WAL. -func (h *Head) initTime(t int64) { - if !h.minTime.CompareAndSwap(math.MaxInt64, t) { - return - } - // Ensure that max time is initialized to at least the min time we just set. - // Concurrent appenders may already have set it to a higher value. - h.maxTime.CompareAndSwap(math.MinInt64, t) -} - -func (a *initAppender) GetRef(lset labels.Labels, hash uint64) (storage.SeriesRef, labels.Labels) { +func (a *initAppenderV2) GetRef(lset labels.Labels, hash uint64) (storage.SeriesRef, labels.Labels) { if g, ok := a.app.(storage.GetRef); ok { return g.GetRef(lset, hash) } return 0, labels.EmptyLabels() } -func (a *initAppender) Commit() error { +func (a *initAppenderV2) Commit() error { if a.app == nil { a.head.metrics.activeAppenders.Dec() return nil @@ -139,7 +60,7 @@ func (a *initAppender) Commit() error { return a.app.Commit() } -func (a *initAppender) Rollback() error { +func (a *initAppenderV2) Rollback() error { if a.app == nil { a.head.metrics.activeAppenders.Dec() return nil @@ -147,669 +68,62 @@ func (a *initAppender) Rollback() error { return a.app.Rollback() } -// Appender returns a new Appender on the database. -func (h *Head) Appender(context.Context) storage.Appender { +// AppenderV2 returns a new AppenderV2 on the database. +func (h *Head) AppenderV2(context.Context) storage.AppenderV2 { h.metrics.activeAppenders.Inc() // The head cache might not have a starting point yet. The init appender // picks up the first appended timestamp as the base. if !h.initialized() { - return &initAppender{ + return &initAppenderV2{ head: h, } } - return h.appender() + return h.appenderV2() } -func (h *Head) appender() *headAppender { +func (h *Head) appenderV2() *headAppenderV2 { minValidTime := h.appendableMinValidTime() appendID, cleanupAppendIDsBelow := h.iso.newAppendID(minValidTime) // Every appender gets an ID that is cleared upon commit/rollback. - return &headAppender{ - head: h, - minValidTime: minValidTime, - mint: math.MaxInt64, - maxt: math.MinInt64, - headMaxt: h.MaxTime(), - oooTimeWindow: h.opts.OutOfOrderTimeWindow.Load(), - seriesRefs: h.getRefSeriesBuffer(), - series: h.getSeriesBuffer(), - typesInBatch: h.getTypeMap(), - appendID: appendID, - cleanupAppendIDsBelow: cleanupAppendIDsBelow, + return &headAppenderV2{ + headAppenderBase: headAppenderBase{ + head: h, + minValidTime: minValidTime, + mint: math.MaxInt64, + maxt: math.MinInt64, + headMaxt: h.MaxTime(), + oooTimeWindow: h.opts.OutOfOrderTimeWindow.Load(), + seriesRefs: h.getRefSeriesBuffer(), + series: h.getSeriesBuffer(), + typesInBatch: h.getTypeMap(), + appendID: appendID, + cleanupAppendIDsBelow: cleanupAppendIDsBelow, + }, } } -// appendableMinValidTime returns the minimum valid timestamp for appends, -// such that samples stay ahead of prior blocks and the head compaction window. -func (h *Head) appendableMinValidTime() int64 { - // This boundary ensures that no samples will be added to the compaction window. - // This allows race-free, concurrent appending and compaction. - cwEnd := h.MaxTime() - h.chunkRange.Load()/2 - - // This boundary ensures that we avoid overlapping timeframes from one block to the next. - // While not necessary for correctness, it means we're not required to use vertical compaction. - minValid := h.minValidTime.Load() - - return max(cwEnd, minValid) +type headAppenderV2 struct { + headAppenderBase } -// AppendableMinValidTime returns the minimum valid time for samples to be appended to the Head. -// Returns false if Head hasn't been initialized yet and the minimum time isn't known yet. -func (h *Head) AppendableMinValidTime() (int64, bool) { - if !h.initialized() { - return 0, false - } - - return h.appendableMinValidTime(), true -} - -func (h *Head) getRefSeriesBuffer() []record.RefSeries { - b := h.refSeriesPool.Get() - if b == nil { - return make([]record.RefSeries, 0, 512) - } - return b -} - -func (h *Head) putRefSeriesBuffer(b []record.RefSeries) { - h.refSeriesPool.Put(b[:0]) -} - -func (h *Head) getFloatBuffer() []record.RefSample { - b := h.floatsPool.Get() - if b == nil { - return make([]record.RefSample, 0, 512) - } - return b -} - -func (h *Head) putFloatBuffer(b []record.RefSample) { - h.floatsPool.Put(b[:0]) -} - -func (h *Head) getExemplarBuffer() []exemplarWithSeriesRef { - b := h.exemplarsPool.Get() - if b == nil { - return make([]exemplarWithSeriesRef, 0, 512) - } - return b -} - -func (h *Head) putExemplarBuffer(b []exemplarWithSeriesRef) { - if b == nil { - return - } - for i := range b { // Zero out to avoid retaining label data. - b[i].exemplar.Labels = labels.EmptyLabels() - } - - h.exemplarsPool.Put(b[:0]) -} - -func (h *Head) getHistogramBuffer() []record.RefHistogramSample { - b := h.histogramsPool.Get() - if b == nil { - return make([]record.RefHistogramSample, 0, 512) - } - return b -} - -func (h *Head) putHistogramBuffer(b []record.RefHistogramSample) { - h.histogramsPool.Put(b[:0]) -} - -func (h *Head) getFloatHistogramBuffer() []record.RefFloatHistogramSample { - b := h.floatHistogramsPool.Get() - if b == nil { - return make([]record.RefFloatHistogramSample, 0, 512) - } - return b -} - -func (h *Head) putFloatHistogramBuffer(b []record.RefFloatHistogramSample) { - h.floatHistogramsPool.Put(b[:0]) -} - -func (h *Head) getMetadataBuffer() []record.RefMetadata { - b := h.metadataPool.Get() - if b == nil { - return make([]record.RefMetadata, 0, 512) - } - return b -} - -func (h *Head) putMetadataBuffer(b []record.RefMetadata) { - h.metadataPool.Put(b[:0]) -} - -func (h *Head) getSeriesBuffer() []*memSeries { - b := h.seriesPool.Get() - if b == nil { - return make([]*memSeries, 0, 512) - } - return b -} - -func (h *Head) putSeriesBuffer(b []*memSeries) { - for i := range b { // Zero out to avoid retaining data. - b[i] = nil - } - h.seriesPool.Put(b[:0]) -} - -func (h *Head) getTypeMap() map[chunks.HeadSeriesRef]sampleType { - b := h.typeMapPool.Get() - if b == nil { - return make(map[chunks.HeadSeriesRef]sampleType) - } - return b -} - -func (h *Head) putTypeMap(b map[chunks.HeadSeriesRef]sampleType) { - clear(b) - h.typeMapPool.Put(b) -} - -func (h *Head) getBytesBuffer() []byte { - b := h.bytesPool.Get() - if b == nil { - return make([]byte, 0, 1024) - } - return b -} - -func (h *Head) putBytesBuffer(b []byte) { - h.bytesPool.Put(b[:0]) -} - -type exemplarWithSeriesRef struct { - ref storage.SeriesRef - exemplar exemplar.Exemplar -} - -// sampleType describes sample types we need to distinguish for append batching. -// We need separate types for everything that goes into a different WAL record -// type or into a different chunk encoding. -type sampleType byte - -const ( - stNone sampleType = iota // To mark that the sample type does not matter. - stFloat // All simple floats (counters, gauges, untyped). Goes to `floats`. - stHistogram // Native integer histograms with a standard exponential schema. Goes to `histograms`. - stCustomBucketHistogram // Native integer histograms with custom bucket boundaries. Goes to `histograms`. - stFloatHistogram // Native float histograms. Goes to `floatHistograms`. - stCustomBucketFloatHistogram // Native float histograms with custom bucket boundaries. Goes to `floatHistograms`. -) - -// appendBatch is used to partition all the appended data into batches that are -// "type clean", i.e. every series receives only samples of one type within the -// batch. Types in this regard are defined by the sampleType enum above. -// TODO(beorn7): The same concept could be extended to make sure every series in -// the batch has at most one metadata record. This is currently not implemented -// because it is unclear if it is needed at all. (Maybe we will remove metadata -// records altogether, see issue #15911.) -type appendBatch struct { - floats []record.RefSample // New float samples held by this appender. - floatSeries []*memSeries // Float series corresponding to the samples held by this appender (using corresponding slice indices - same series may appear more than once). - histograms []record.RefHistogramSample // New histogram samples held by this appender. - histogramSeries []*memSeries // HistogramSamples series corresponding to the samples held by this appender (using corresponding slice indices - same series may appear more than once). - floatHistograms []record.RefFloatHistogramSample // New float histogram samples held by this appender. - floatHistogramSeries []*memSeries // FloatHistogramSamples series corresponding to the samples held by this appender (using corresponding slice indices - same series may appear more than once). - metadata []record.RefMetadata // New metadata held by this appender. - metadataSeries []*memSeries // Series corresponding to the metadata held by this appender. - exemplars []exemplarWithSeriesRef // New exemplars held by this appender. -} - -// close returns all the slices to the pools in Head and nil's them. -func (b *appendBatch) close(h *Head) { - h.putFloatBuffer(b.floats) - b.floats = nil - h.putSeriesBuffer(b.floatSeries) - b.floatSeries = nil - h.putHistogramBuffer(b.histograms) - b.histograms = nil - h.putSeriesBuffer(b.histogramSeries) - b.histogramSeries = nil - h.putFloatHistogramBuffer(b.floatHistograms) - b.floatHistograms = nil - h.putSeriesBuffer(b.floatHistogramSeries) - b.floatHistogramSeries = nil - h.putMetadataBuffer(b.metadata) - b.metadata = nil - h.putSeriesBuffer(b.metadataSeries) - b.metadataSeries = nil - h.putExemplarBuffer(b.exemplars) - b.exemplars = nil -} - -type headAppender struct { - head *Head - minValidTime int64 // No samples below this timestamp are allowed. - mint, maxt int64 - headMaxt int64 // We track it here to not take the lock for every sample appended. - oooTimeWindow int64 // Use the same for the entire append, and don't load the atomic for each sample. - - seriesRefs []record.RefSeries // New series records held by this appender. - series []*memSeries // New series held by this appender (using corresponding slices indexes from seriesRefs) - batches []*appendBatch // Holds all the other data to append. (In regular cases, there should be only one of these.) - - typesInBatch map[chunks.HeadSeriesRef]sampleType // Which (one) sample type each series holds in the most recent batch. - - appendID, cleanupAppendIDsBelow uint64 - closed bool - hints *storage.AppendOptions -} - -func (a *headAppender) SetOptions(opts *storage.AppendOptions) { - a.hints = opts -} - -func (a *headAppender) Append(ref storage.SeriesRef, lset labels.Labels, t int64, v float64) (storage.SeriesRef, error) { +func (a *headAppenderV2) Append(ref storage.SeriesRef, ls labels.Labels, st, t int64, v float64, h *histogram.Histogram, fh *histogram.FloatHistogram, opts storage.AOptions) (storage.SeriesRef, error) { // Fail fast if OOO is disabled and the sample is out of bounds. - // Otherwise a full check will be done later to decide if the sample is in-order or out-of-order. + // Otherwise, a full check will be done later to decide if the sample is in-order or out-of-order. if a.oooTimeWindow == 0 && t < a.minValidTime { - a.head.metrics.outOfBoundSamples.WithLabelValues(sampleMetricTypeFloat).Inc() - return 0, storage.ErrOutOfBounds - } - - s := a.head.series.getByID(chunks.HeadSeriesRef(ref)) - if s == nil { - var err error - s, _, err = a.getOrCreate(lset) - if err != nil { - return 0, err - } - } - - if value.IsStaleNaN(v) { - // If we have added a sample before with this same appender, we - // can check the previously used type and turn a stale float - // sample into a stale histogram sample or stale float histogram - // sample as appropriate. This prevents an unnecessary creation - // of a new batch. However, since other appenders might append - // to the same series concurrently, this is not perfect but just - // an optimization for the more likely case. - switch a.typesInBatch[s.ref] { - case stHistogram, stCustomBucketHistogram: - return a.AppendHistogram(ref, lset, t, &histogram.Histogram{Sum: v}, nil) - case stFloatHistogram, stCustomBucketFloatHistogram: - return a.AppendHistogram(ref, lset, t, nil, &histogram.FloatHistogram{Sum: v}) - } - // Note that a series reference not yet in the map will come out - // as stNone, but since we do not handle that case separately, - // we do not need to check for the difference between "unknown - // series" and "known series with stNone". - } - - s.Lock() - defer s.Unlock() - // TODO(codesome): If we definitely know at this point that the sample is ooo, then optimise - // to skip that sample from the WAL and write only in the WBL. - isOOO, delta, err := s.appendable(t, v, a.headMaxt, a.minValidTime, a.oooTimeWindow) - if err == nil { - if isOOO && a.hints != nil && a.hints.DiscardOutOfOrder { - a.head.metrics.outOfOrderSamples.WithLabelValues(sampleMetricTypeFloat).Inc() - return 0, storage.ErrOutOfOrderSample - } - s.pendingCommit = true - } - if delta > 0 { - a.head.metrics.oooHistogram.Observe(float64(delta) / 1000) - } - if err != nil { - switch { - case errors.Is(err, storage.ErrOutOfOrderSample): - a.head.metrics.outOfOrderSamples.WithLabelValues(sampleMetricTypeFloat).Inc() - case errors.Is(err, storage.ErrTooOldSample): - a.head.metrics.tooOldSamples.WithLabelValues(sampleMetricTypeFloat).Inc() - } - return 0, err - } - - if t < a.mint { - a.mint = t - } - if t > a.maxt { - a.maxt = t - } - - b := a.getCurrentBatch(stFloat, s.ref) - b.floats = append(b.floats, record.RefSample{ - Ref: s.ref, - T: t, - V: v, - }) - b.floatSeries = append(b.floatSeries, s) - return storage.SeriesRef(s.ref), nil -} - -// AppendSTZeroSample appends synthetic zero sample for st timestamp. It returns -// error when sample can't be appended. See -// storage.StartTimestampAppender.AppendSTZeroSample for further documentation. -func (a *headAppender) AppendSTZeroSample(ref storage.SeriesRef, lset labels.Labels, t, st int64) (storage.SeriesRef, error) { - if st >= t { - return 0, storage.ErrSTNewerThanSample - } - - s := a.head.series.getByID(chunks.HeadSeriesRef(ref)) - if s == nil { - var err error - s, _, err = a.getOrCreate(lset) - if err != nil { - return 0, err - } - } - - // Check if ST wouldn't be OOO vs samples we already might have for this series. - // NOTE(bwplotka): This will be often hit as it's expected for long living - // counters to share the same ST. - s.Lock() - isOOO, _, err := s.appendable(st, 0, a.headMaxt, a.minValidTime, a.oooTimeWindow) - if err == nil { - s.pendingCommit = true - } - s.Unlock() - if err != nil { - return 0, err - } - if isOOO { - return storage.SeriesRef(s.ref), storage.ErrOutOfOrderST - } - - if st > a.maxt { - a.maxt = st - } - b := a.getCurrentBatch(stFloat, s.ref) - b.floats = append(b.floats, record.RefSample{Ref: s.ref, T: st, V: 0.0}) - b.floatSeries = append(b.floatSeries, s) - return storage.SeriesRef(s.ref), nil -} - -func (a *headAppender) getOrCreate(lset labels.Labels) (s *memSeries, created bool, err error) { - // Ensure no empty labels have gotten through. - lset = lset.WithoutEmpty() - if lset.IsEmpty() { - return nil, false, fmt.Errorf("empty labelset: %w", ErrInvalidSample) - } - if l, dup := lset.HasDuplicateLabelNames(); dup { - return nil, false, fmt.Errorf(`label name "%s" is not unique: %w`, l, ErrInvalidSample) - } - s, created, err = a.head.getOrCreate(lset.Hash(), lset, true) - if err != nil { - return nil, false, err - } - if created { - a.seriesRefs = append(a.seriesRefs, record.RefSeries{ - Ref: s.ref, - Labels: lset, - }) - a.series = append(a.series, s) - } - return s, created, nil -} - -// getCurrentBatch returns the current batch if it fits the provided sampleType -// for the provided series. Otherwise, it adds a new batch and returns it. -func (a *headAppender) getCurrentBatch(st sampleType, s chunks.HeadSeriesRef) *appendBatch { - h := a.head - - newBatch := func() *appendBatch { - b := appendBatch{ - floats: h.getFloatBuffer(), - floatSeries: h.getSeriesBuffer(), - histograms: h.getHistogramBuffer(), - histogramSeries: h.getSeriesBuffer(), - floatHistograms: h.getFloatHistogramBuffer(), - floatHistogramSeries: h.getSeriesBuffer(), - metadata: h.getMetadataBuffer(), - metadataSeries: h.getSeriesBuffer(), - } - - // Allocate the exemplars buffer only if exemplars are enabled. - if h.opts.EnableExemplarStorage { - b.exemplars = h.getExemplarBuffer() - } - clear(a.typesInBatch) - switch st { - case stHistogram, stFloatHistogram, stCustomBucketHistogram, stCustomBucketFloatHistogram: - // We only record histogram sample types in the map. - // Floats are implicit. - a.typesInBatch[s] = st - } - a.batches = append(a.batches, &b) - return &b - } - - // First batch ever. Create it. - if len(a.batches) == 0 { - return newBatch() - } - - // TODO(beorn7): If we ever see that the a.typesInBatch map grows so - // large that it matters for total memory consumption, we could limit - // the batch size here, i.e. cut a new batch even without a type change. - // Something like: - // if len(a.typesInBatch > limit) { - // return newBatch() - // } - - lastBatch := a.batches[len(a.batches)-1] - if st == stNone { - // Type doesn't matter, last batch will always do. - return lastBatch - } - prevST, ok := a.typesInBatch[s] - switch { - case prevST == st: - // An old series of some histogram type with the same type being appended. - // Continue the batch. - return lastBatch - case !ok && st == stFloat: - // A new float series, or an old float series that gets floats appended. - // Note that we do not track stFloat in typesInBatch. - // Continue the batch. - return lastBatch - case st == stFloat: - // A float being appended to a histogram series. - // Start a new batch. - return newBatch() - case !ok: - // A new series of some histogram type, or some histogram type - // being appended to on old float series. Even in the latter - // case, we don't need to start a new batch because histograms - // after floats are fine. - // Add new sample type to the map and continue batch. - a.typesInBatch[s] = st - return lastBatch - default: - // One histogram type changed to another. - // Start a new batch. - return newBatch() - } -} - -// appendable checks whether the given sample is valid for appending to the series. -// If the sample is valid and in-order, it returns false with no error. -// If the sample belongs to the out-of-order chunk, it returns true with no error. -// If the sample cannot be handled, it returns an error. -func (s *memSeries) appendable(t int64, v float64, headMaxt, minValidTime, oooTimeWindow int64) (isOOO bool, oooDelta int64, err error) { - // Check if we can append in the in-order chunk. - if t >= minValidTime { - if s.headChunks == nil { - // The series has no sample and was freshly created. - return false, 0, nil - } - msMaxt := s.maxTime() - if t > msMaxt { - return false, 0, nil - } - if t == msMaxt { - // We are allowing exact duplicates as we can encounter them in valid cases - // like federation and erroring out at that time would be extremely noisy. - // This only checks against the latest in-order sample. - // The OOO headchunk has its own method to detect these duplicates. - if s.lastHistogramValue != nil || s.lastFloatHistogramValue != nil { - return false, 0, storage.NewDuplicateHistogramToFloatErr(t, v) - } - if math.Float64bits(s.lastValue) != math.Float64bits(v) { - return false, 0, storage.NewDuplicateFloatErr(t, s.lastValue, v) - } - // Sample is identical (ts + value) with most current (highest ts) sample in sampleBuf. - return false, 0, nil - } - } - - // The sample cannot go in the in-order chunk. Check if it can go in the out-of-order chunk. - if oooTimeWindow > 0 && t >= headMaxt-oooTimeWindow { - return true, headMaxt - t, nil - } - - // The sample cannot go in both in-order and out-of-order chunk. - if oooTimeWindow > 0 { - return true, headMaxt - t, storage.ErrTooOldSample - } - if t < minValidTime { - return false, headMaxt - t, storage.ErrOutOfBounds - } - return false, headMaxt - t, storage.ErrOutOfOrderSample -} - -// appendableHistogram checks whether the given histogram sample is valid for appending to the series. (if we return false and no error) -// The sample belongs to the out of order chunk if we return true and no error. -// An error signifies the sample cannot be handled. -func (s *memSeries) appendableHistogram(t int64, h *histogram.Histogram, headMaxt, minValidTime, oooTimeWindow int64) (isOOO bool, oooDelta int64, err error) { - // Check if we can append in the in-order chunk. - if t >= minValidTime { - if s.headChunks == nil { - // The series has no sample and was freshly created. - return false, 0, nil - } - msMaxt := s.maxTime() - if t > msMaxt { - return false, 0, nil - } - if t == msMaxt { - // We are allowing exact duplicates as we can encounter them in valid cases - // like federation and erroring out at that time would be extremely noisy. - // This only checks against the latest in-order sample. - // The OOO headchunk has its own method to detect these duplicates. - if !h.Equals(s.lastHistogramValue) { - return false, 0, storage.ErrDuplicateSampleForTimestamp - } - // Sample is identical (ts + value) with most current (highest ts) sample in sampleBuf. - return false, 0, nil - } - } - - // The sample cannot go in the in-order chunk. Check if it can go in the out-of-order chunk. - if oooTimeWindow > 0 && t >= headMaxt-oooTimeWindow { - return true, headMaxt - t, nil - } - - // The sample cannot go in both in-order and out-of-order chunk. - if oooTimeWindow > 0 { - return true, headMaxt - t, storage.ErrTooOldSample - } - if t < minValidTime { - return false, headMaxt - t, storage.ErrOutOfBounds - } - return false, headMaxt - t, storage.ErrOutOfOrderSample -} - -// appendableFloatHistogram checks whether the given float histogram sample is valid for appending to the series. (if we return false and no error) -// The sample belongs to the out of order chunk if we return true and no error. -// An error signifies the sample cannot be handled. -func (s *memSeries) appendableFloatHistogram(t int64, fh *histogram.FloatHistogram, headMaxt, minValidTime, oooTimeWindow int64) (isOOO bool, oooDelta int64, err error) { - // Check if we can append in the in-order chunk. - if t >= minValidTime { - if s.headChunks == nil { - // The series has no sample and was freshly created. - return false, 0, nil - } - msMaxt := s.maxTime() - if t > msMaxt { - return false, 0, nil - } - if t == msMaxt { - // We are allowing exact duplicates as we can encounter them in valid cases - // like federation and erroring out at that time would be extremely noisy. - // This only checks against the latest in-order sample. - // The OOO headchunk has its own method to detect these duplicates. - if !fh.Equals(s.lastFloatHistogramValue) { - return false, 0, storage.ErrDuplicateSampleForTimestamp - } - // Sample is identical (ts + value) with most current (highest ts) sample in sampleBuf. - return false, 0, nil - } - } - - // The sample cannot go in the in-order chunk. Check if it can go in the out-of-order chunk. - if oooTimeWindow > 0 && t >= headMaxt-oooTimeWindow { - return true, headMaxt - t, nil - } - - // The sample cannot go in both in-order and out-of-order chunk. - if oooTimeWindow > 0 { - return true, headMaxt - t, storage.ErrTooOldSample - } - if t < minValidTime { - return false, headMaxt - t, storage.ErrOutOfBounds - } - return false, headMaxt - t, storage.ErrOutOfOrderSample -} - -// AppendExemplar for headAppender assumes the series ref already exists, and so it doesn't -// use getOrCreate or make any of the lset validity checks that Append does. -func (a *headAppender) AppendExemplar(ref storage.SeriesRef, lset labels.Labels, e exemplar.Exemplar) (storage.SeriesRef, error) { - // Check if exemplar storage is enabled. - if !a.head.opts.EnableExemplarStorage || a.head.opts.MaxExemplars.Load() <= 0 { - return 0, nil - } - - // Get Series - s := a.head.series.getByID(chunks.HeadSeriesRef(ref)) - if s == nil { - s = a.head.series.getByHash(lset.Hash(), lset) - if s != nil { - ref = storage.SeriesRef(s.ref) - } - } - if s == nil { - return 0, fmt.Errorf("unknown HeadSeriesRef when trying to add exemplar: %d", ref) - } - - // Ensure no empty labels have gotten through. - e.Labels = e.Labels.WithoutEmpty() - - err := a.head.exemplars.ValidateExemplar(s.labels(), e) - if err != nil { - if errors.Is(err, storage.ErrDuplicateExemplar) || errors.Is(err, storage.ErrExemplarsDisabled) { - // Duplicate, don't return an error but don't accept the exemplar. - return 0, nil - } - return 0, err - } - - b := a.getCurrentBatch(stNone, chunks.HeadSeriesRef(ref)) - b.exemplars = append(b.exemplars, exemplarWithSeriesRef{ref, e}) - - return storage.SeriesRef(s.ref), nil -} - -func (a *headAppender) AppendHistogram(ref storage.SeriesRef, lset labels.Labels, t int64, h *histogram.Histogram, fh *histogram.FloatHistogram) (storage.SeriesRef, error) { - // Fail fast if OOO is disabled and the sample is out of bounds. - // Otherwise a full check will be done later to decide if the sample is in-order or out-of-order. - if a.oooTimeWindow == 0 && t < a.minValidTime { - a.head.metrics.outOfBoundSamples.WithLabelValues(sampleMetricTypeHistogram).Inc() + if h != nil || fh != nil { + a.head.metrics.outOfBoundSamples.WithLabelValues(sampleMetricTypeHistogram).Inc() + } else { + a.head.metrics.outOfBoundSamples.WithLabelValues(sampleMetricTypeFloat).Inc() + } return 0, storage.ErrOutOfBounds } + // Validate samples before touching memSeries. if h != nil { if err := h.Validate(); err != nil { return 0, err } } - if fh != nil { if err := fh.Validate(); err != nil { return 0, err @@ -819,26 +133,20 @@ func (a *headAppender) AppendHistogram(ref storage.SeriesRef, lset labels.Labels s := a.head.series.getByID(chunks.HeadSeriesRef(ref)) if s == nil { var err error - s, _, err = a.getOrCreate(lset) + s, _, err = a.getOrCreate(ls) if err != nil { return 0, err } } + // TODO(PROM-60): Handle ST natively. + if a.head.opts.EnableSTAsZeroSample && st != 0 { + a.bestEffortAppendSTZeroSample(s, st, t, h, fh) + } + switch { case h != nil: - s.Lock() - // TODO(codesome): If we definitely know at this point that the sample is ooo, then optimise - // to skip that sample from the WAL and write only in the WBL. - _, delta, err := s.appendableHistogram(t, h, a.headMaxt, a.minValidTime, a.oooTimeWindow) - if err != nil { - s.pendingCommit = true - } - s.Unlock() - if delta > 0 { - a.head.metrics.oooHistogram.Observe(float64(delta) / 1000) - } - if err != nil { + if err := a.appendHistogram(s, t, h, opts.RejectOutOfOrder); err != nil { switch { case errors.Is(err, storage.ErrOutOfOrderSample): a.head.metrics.outOfOrderSamples.WithLabelValues(sampleMetricTypeHistogram).Inc() @@ -847,30 +155,8 @@ func (a *headAppender) AppendHistogram(ref storage.SeriesRef, lset labels.Labels } return 0, err } - st := stHistogram - if h.UsesCustomBuckets() { - st = stCustomBucketHistogram - } - b := a.getCurrentBatch(st, s.ref) - b.histograms = append(b.histograms, record.RefHistogramSample{ - Ref: s.ref, - T: t, - H: h, - }) - b.histogramSeries = append(b.histogramSeries, s) case fh != nil: - s.Lock() - // TODO(codesome): If we definitely know at this point that the sample is ooo, then optimise - // to skip that sample from the WAL and write only in the WBL. - _, delta, err := s.appendableFloatHistogram(t, fh, a.headMaxt, a.minValidTime, a.oooTimeWindow) - if err == nil { - s.pendingCommit = true - } - s.Unlock() - if delta > 0 { - a.head.metrics.oooHistogram.Observe(float64(delta) / 1000) - } - if err != nil { + if err := a.appendFloatHistogram(s, t, fh, opts.RejectOutOfOrder); err != nil { switch { case errors.Is(err, storage.ErrOutOfOrderSample): a.head.metrics.outOfOrderSamples.WithLabelValues(sampleMetricTypeHistogram).Inc() @@ -879,17 +165,41 @@ func (a *headAppender) AppendHistogram(ref storage.SeriesRef, lset labels.Labels } return 0, err } - st := stFloatHistogram - if fh.UsesCustomBuckets() { - st = stCustomBucketFloatHistogram + default: + if value.IsStaleNaN(v) { + // If we have added a sample before with this same appender, we + // can check the previously used type and turn a stale float + // sample into a stale histogram sample or stale float histogram + // sample as appropriate. This prevents an unnecessary creation + // of a new batch. However, since other appenders might append + // to the same series concurrently, this is not perfect but just + // an optimization for the more likely case. + switch a.typesInBatch[s.ref] { + case stHistogram, stCustomBucketHistogram: + return a.Append(ref, ls, st, t, 0, &histogram.Histogram{Sum: v}, nil, storage.AOptions{ + // TODO(bwplotka): What to do with metadata and exemplars in this case? Handle correctly, for now mimicking old behaviour (empty). + RejectOutOfOrder: opts.RejectOutOfOrder, + }) + case stFloatHistogram, stCustomBucketFloatHistogram: + return a.Append(ref, ls, st, t, 0, nil, &histogram.FloatHistogram{Sum: v}, storage.AOptions{ + // TODO(bwplotka): What to do with metadata and exemplars in this case? Handle correctly, for now mimicking old behaviour (empty). + RejectOutOfOrder: opts.RejectOutOfOrder, + }) + } + // Note that a series reference not yet in the map will come out + // as stNone, but since we do not handle that case separately, + // we do not need to check for the difference between "unknown + // series" and "known series with stNone". + } + if err := a.appendFloat(s, t, v, opts.RejectOutOfOrder); err != nil { + switch { + case errors.Is(err, storage.ErrOutOfOrderSample): + a.head.metrics.outOfOrderSamples.WithLabelValues(sampleMetricTypeFloat).Inc() + case errors.Is(err, storage.ErrTooOldSample): + a.head.metrics.tooOldSamples.WithLabelValues(sampleMetricTypeFloat).Inc() + } + return 0, err } - b := a.getCurrentBatch(st, s.ref) - b.floatHistograms = append(b.floatHistograms, record.RefFloatHistogramSample{ - Ref: s.ref, - T: t, - FH: fh, - }) - b.floatHistogramSeries = append(b.floatHistogramSeries, s) } if t < a.mint { @@ -899,23 +209,156 @@ func (a *headAppender) AppendHistogram(ref storage.SeriesRef, lset labels.Labels a.maxt = t } - return storage.SeriesRef(s.ref), nil -} - -func (a *headAppender) AppendHistogramSTZeroSample(ref storage.SeriesRef, lset labels.Labels, t, st int64, h *histogram.Histogram, fh *histogram.FloatHistogram) (storage.SeriesRef, error) { - if st >= t { - return 0, storage.ErrSTNewerThanSample + var partialErr error + // Append exemplars if any and if storage was configured for it. + if len(opts.Exemplars) > 0 && a.head.opts.EnableExemplarStorage && a.head.opts.MaxExemplars.Load() > 0 { + // Currently only exemplars can return partial errors. + partialErr = a.appendExemplars(s, opts.Exemplars) } - s := a.head.series.getByID(chunks.HeadSeriesRef(ref)) - if s == nil { - var err error - s, _, err = a.getOrCreate(lset) - if err != nil { - return 0, err + // TODO(bwplotka): Move/reuse metadata tests from scrape, once scrape adopts AppenderV2. + // Currently tsdb package does not test metadata. + if a.head.opts.EnableMetadataWALRecords && !opts.Metadata.IsEmpty() { + s.Lock() + metaChanged := !s.meta.Equals(opts.Metadata) + s.Unlock() + if metaChanged { + b := a.getCurrentBatch(stNone, s.ref) + b.metadata = append(b.metadata, record.RefMetadata{ + Ref: s.ref, + Type: record.GetMetricType(opts.Metadata.Type), + Unit: opts.Metadata.Unit, + Help: opts.Metadata.Help, + }) + b.metadataSeries = append(b.metadataSeries, s) } } + return storage.SeriesRef(s.ref), partialErr +} +func (a *headAppenderV2) appendFloat(s *memSeries, t int64, v float64, fastRejectOOO bool) error { + s.Lock() + // TODO(codesome): If we definitely know at this point that the sample is ooo, then optimise + // to skip that sample from the WAL and write only in the WBL. + isOOO, delta, err := s.appendable(t, v, a.headMaxt, a.minValidTime, a.oooTimeWindow) + if isOOO && fastRejectOOO { + s.Unlock() + return storage.ErrOutOfOrderSample + } + if err == nil { + s.pendingCommit = true + } + s.Unlock() + if delta > 0 { + a.head.metrics.oooHistogram.Observe(float64(delta) / 1000) + } + if err != nil { + return err + } + + b := a.getCurrentBatch(stFloat, s.ref) + b.floats = append(b.floats, record.RefSample{Ref: s.ref, T: t, V: v}) + b.floatSeries = append(b.floatSeries, s) + return nil +} + +func (a *headAppenderV2) appendHistogram(s *memSeries, t int64, h *histogram.Histogram, fastRejectOOO bool) error { + s.Lock() + // TODO(codesome): If we definitely know at this point that the sample is ooo, then optimise + // to skip that sample from the WAL and write only in the WBL. + isOOO, delta, err := s.appendableHistogram(t, h, a.headMaxt, a.minValidTime, a.oooTimeWindow) + if isOOO && fastRejectOOO { + s.Unlock() + return storage.ErrOutOfOrderSample + } + if err == nil { + s.pendingCommit = true + } + s.Unlock() + if delta > 0 { + a.head.metrics.oooHistogram.Observe(float64(delta) / 1000) + } + if err != nil { + return err + } + st := stHistogram + if h.UsesCustomBuckets() { + st = stCustomBucketHistogram + } + b := a.getCurrentBatch(st, s.ref) + b.histograms = append(b.histograms, record.RefHistogramSample{Ref: s.ref, T: t, H: h}) + b.histogramSeries = append(b.histogramSeries, s) + return nil +} + +func (a *headAppenderV2) appendFloatHistogram(s *memSeries, t int64, fh *histogram.FloatHistogram, fastRejectOOO bool) error { + s.Lock() + // TODO(codesome): If we definitely know at this point that the sample is ooo, then optimise + // to skip that sample from the WAL and write only in the WBL. + isOOO, delta, err := s.appendableFloatHistogram(t, fh, a.headMaxt, a.minValidTime, a.oooTimeWindow) + if isOOO && fastRejectOOO { + s.Unlock() + return storage.ErrOutOfOrderSample + } + if err == nil { + s.pendingCommit = true + } + s.Unlock() + if delta > 0 { + a.head.metrics.oooHistogram.Observe(float64(delta) / 1000) + } + if err != nil { + return err + } + st := stFloatHistogram + if fh.UsesCustomBuckets() { + st = stCustomBucketFloatHistogram + } + b := a.getCurrentBatch(st, s.ref) + b.floatHistograms = append(b.floatHistograms, record.RefFloatHistogramSample{Ref: s.ref, T: t, FH: fh}) + b.floatHistogramSeries = append(b.floatHistogramSeries, s) + return nil +} + +func (a *headAppenderV2) appendExemplars(s *memSeries, exemplar []exemplar.Exemplar) error { + var errs []error + for _, e := range exemplar { + // Ensure no empty labels have gotten through. + e.Labels = e.Labels.WithoutEmpty() + if err := a.head.exemplars.ValidateExemplar(s.labels(), e); err != nil { + if !errors.Is(err, storage.ErrDuplicateExemplar) && !errors.Is(err, storage.ErrExemplarsDisabled) { + // Except duplicates, return partial errors. + errs = append(errs, err) + } + if !errors.Is(err, storage.ErrOutOfOrderExemplar) { + a.head.logger.Debug("Error while adding an exemplar on AppendSample", "exemplars", fmt.Sprintf("%+v", e), "err", e) + } + continue + } + b := a.getCurrentBatch(stNone, s.ref) + b.exemplars = append(b.exemplars, exemplarWithSeriesRef{storage.SeriesRef(s.ref), e}) + } + if len(errs) > 0 { + return &storage.AppendPartialError{ExemplarErrors: errs} + } + return nil +} + +// NOTE(bwplotka): This feature might be deprecated and removed once PROM-60 +// is implemented. +// +// ST is an experimental feature, we don't fail the append on errors, just debug log. +func (a *headAppenderV2) bestEffortAppendSTZeroSample(s *memSeries, st, t int64, h *histogram.Histogram, fh *histogram.FloatHistogram) { + if st >= t { + a.head.logger.Debug("Error when appending ST", "series", s.lset.String(), "st", st, "t", t, "err", storage.ErrSTNewerThanSample) + return + } + if st < a.minValidTime { + a.head.logger.Debug("Error when appending ST", "series", s.lset.String(), "st", st, "t", t, "err", storage.ErrOutOfBounds) + return + } + + var err error switch { case h != nil: zeroHistogram := &histogram.Histogram{ @@ -926,39 +369,7 @@ func (a *headAppender) AppendHistogramSTZeroSample(ref storage.SeriesRef, lset l ZeroThreshold: h.ZeroThreshold, CustomValues: h.CustomValues, } - s.Lock() - // For STZeroSamples OOO is not allowed. - // We set it to true to make this implementation as close as possible to the float implementation. - isOOO, _, err := s.appendableHistogram(st, zeroHistogram, a.headMaxt, a.minValidTime, a.oooTimeWindow) - if err != nil { - s.Unlock() - if errors.Is(err, storage.ErrOutOfOrderSample) { - return 0, storage.ErrOutOfOrderST - } - - return 0, err - } - - // OOO is not allowed because after the first scrape, ST will be the same for most (if not all) future samples. - // This is to prevent the injected zero from being marked as OOO forever. - if isOOO { - s.Unlock() - return 0, storage.ErrOutOfOrderST - } - - s.pendingCommit = true - s.Unlock() - sTyp := stHistogram - if h.UsesCustomBuckets() { - sTyp = stCustomBucketHistogram - } - b := a.getCurrentBatch(sTyp, s.ref) - b.histograms = append(b.histograms, record.RefHistogramSample{ - Ref: s.ref, - T: st, - H: zeroHistogram, - }) - b.histogramSeries = append(b.histogramSeries, s) + err = a.appendHistogram(s, st, zeroHistogram, true) case fh != nil: zeroFloatHistogram := &histogram.FloatHistogram{ // The STZeroSample represents a counter reset by definition. @@ -968,1318 +379,23 @@ func (a *headAppender) AppendHistogramSTZeroSample(ref storage.SeriesRef, lset l ZeroThreshold: fh.ZeroThreshold, CustomValues: fh.CustomValues, } - s.Lock() - // We set it to true to make this implementation as close as possible to the float implementation. - isOOO, _, err := s.appendableFloatHistogram(st, zeroFloatHistogram, a.headMaxt, a.minValidTime, a.oooTimeWindow) // OOO is not allowed for STZeroSamples. - if err != nil { - s.Unlock() - if errors.Is(err, storage.ErrOutOfOrderSample) { - return 0, storage.ErrOutOfOrderST - } + err = a.appendFloatHistogram(s, st, zeroFloatHistogram, true) + default: + err = a.appendFloat(s, st, 0, true) + } - return 0, err + if err != nil { + if errors.Is(err, storage.ErrOutOfOrderSample) { + // OOO errors are common and expected (cumulative). Explicitly ignored. + return } - - // OOO is not allowed because after the first scrape, ST will be the same for most (if not all) future samples. - // This is to prevent the injected zero from being marked as OOO forever. - if isOOO { - s.Unlock() - return 0, storage.ErrOutOfOrderST - } - - s.pendingCommit = true - s.Unlock() - sTyp := stFloatHistogram - if fh.UsesCustomBuckets() { - sTyp = stCustomBucketFloatHistogram - } - b := a.getCurrentBatch(sTyp, s.ref) - b.floatHistograms = append(b.floatHistograms, record.RefFloatHistogramSample{ - Ref: s.ref, - T: st, - FH: zeroFloatHistogram, - }) - b.floatHistogramSeries = append(b.floatHistogramSeries, s) + a.head.logger.Debug("Error when appending ST", "series", s.lset.String(), "st", st, "t", t, "err", err) + return } if st > a.maxt { a.maxt = st } - - return storage.SeriesRef(s.ref), nil } -// UpdateMetadata for headAppender assumes the series ref already exists, and so it doesn't -// use getOrCreate or make any of the lset sanity checks that Append does. -func (a *headAppender) UpdateMetadata(ref storage.SeriesRef, lset labels.Labels, meta metadata.Metadata) (storage.SeriesRef, error) { - s := a.head.series.getByID(chunks.HeadSeriesRef(ref)) - if s == nil { - s = a.head.series.getByHash(lset.Hash(), lset) - if s != nil { - ref = storage.SeriesRef(s.ref) - } - } - if s == nil { - return 0, fmt.Errorf("unknown series when trying to add metadata with HeadSeriesRef: %d and labels: %s", ref, lset) - } - - s.Lock() - hasNewMetadata := s.meta == nil || *s.meta != meta - s.Unlock() - - if hasNewMetadata { - b := a.getCurrentBatch(stNone, s.ref) - b.metadata = append(b.metadata, record.RefMetadata{ - Ref: s.ref, - Type: record.GetMetricType(meta.Type), - Unit: meta.Unit, - Help: meta.Help, - }) - b.metadataSeries = append(b.metadataSeries, s) - } - - return ref, nil -} - -var _ storage.GetRef = &headAppender{} - -func (a *headAppender) GetRef(lset labels.Labels, hash uint64) (storage.SeriesRef, labels.Labels) { - s := a.head.series.getByHash(hash, lset) - if s == nil { - return 0, labels.EmptyLabels() - } - // returned labels must be suitable to pass to Append() - return storage.SeriesRef(s.ref), s.labels() -} - -// log writes all headAppender's data to the WAL. -func (a *headAppender) log() error { - if a.head.wal == nil { - return nil - } - - buf := a.head.getBytesBuffer() - defer func() { a.head.putBytesBuffer(buf) }() - - var rec []byte - var enc record.Encoder - - if len(a.seriesRefs) > 0 { - rec = enc.Series(a.seriesRefs, buf) - buf = rec[:0] - - if err := a.head.wal.Log(rec); err != nil { - return fmt.Errorf("log series: %w", err) - } - } - for _, b := range a.batches { - if len(b.metadata) > 0 { - rec = enc.Metadata(b.metadata, buf) - buf = rec[:0] - - if err := a.head.wal.Log(rec); err != nil { - return fmt.Errorf("log metadata: %w", err) - } - } - // It's important to do (float) Samples before histogram samples - // to end up with the correct order. - if len(b.floats) > 0 { - rec = enc.Samples(b.floats, buf) - buf = rec[:0] - - if err := a.head.wal.Log(rec); err != nil { - return fmt.Errorf("log samples: %w", err) - } - } - if len(b.histograms) > 0 { - var customBucketsHistograms []record.RefHistogramSample - rec, customBucketsHistograms = enc.HistogramSamples(b.histograms, buf) - buf = rec[:0] - if len(rec) > 0 { - if err := a.head.wal.Log(rec); err != nil { - return fmt.Errorf("log histograms: %w", err) - } - } - - if len(customBucketsHistograms) > 0 { - rec = enc.CustomBucketsHistogramSamples(customBucketsHistograms, buf) - if err := a.head.wal.Log(rec); err != nil { - return fmt.Errorf("log custom buckets histograms: %w", err) - } - } - } - if len(b.floatHistograms) > 0 { - var customBucketsFloatHistograms []record.RefFloatHistogramSample - rec, customBucketsFloatHistograms = enc.FloatHistogramSamples(b.floatHistograms, buf) - buf = rec[:0] - if len(rec) > 0 { - if err := a.head.wal.Log(rec); err != nil { - return fmt.Errorf("log float histograms: %w", err) - } - } - - if len(customBucketsFloatHistograms) > 0 { - rec = enc.CustomBucketsFloatHistogramSamples(customBucketsFloatHistograms, buf) - if err := a.head.wal.Log(rec); err != nil { - return fmt.Errorf("log custom buckets float histograms: %w", err) - } - } - } - // Exemplars should be logged after samples (float/native histogram/etc), - // otherwise it might happen that we send the exemplars in a remote write - // batch before the samples, which in turn means the exemplar is rejected - // for missing series, since series are created due to samples. - if len(b.exemplars) > 0 { - rec = enc.Exemplars(exemplarsForEncoding(b.exemplars), buf) - buf = rec[:0] - - if err := a.head.wal.Log(rec); err != nil { - return fmt.Errorf("log exemplars: %w", err) - } - } - } - return nil -} - -func exemplarsForEncoding(es []exemplarWithSeriesRef) []record.RefExemplar { - ret := make([]record.RefExemplar, 0, len(es)) - for _, e := range es { - ret = append(ret, record.RefExemplar{ - Ref: chunks.HeadSeriesRef(e.ref), - T: e.exemplar.Ts, - V: e.exemplar.Value, - Labels: e.exemplar.Labels, - }) - } - return ret -} - -type appenderCommitContext struct { - floatsAppended int - histogramsAppended int - // Number of samples out of order but accepted: with ooo enabled and within time window. - oooFloatsAccepted int - oooHistogramAccepted int - // Number of samples rejected due to: out of order but OOO support disabled. - floatOOORejected int - histoOOORejected int - // Number of samples rejected due to: out of order but too old (OOO support enabled, but outside time window). - floatTooOldRejected int - histoTooOldRejected int - // Number of samples rejected due to: out of bounds: with t < minValidTime (OOO support disabled). - floatOOBRejected int - histoOOBRejected int - inOrderMint int64 - inOrderMaxt int64 - oooMinT int64 - oooMaxT int64 - wblSamples []record.RefSample - wblHistograms []record.RefHistogramSample - wblFloatHistograms []record.RefFloatHistogramSample - oooMmapMarkers map[chunks.HeadSeriesRef][]chunks.ChunkDiskMapperRef - oooMmapMarkersCount int - oooRecords [][]byte - oooCapMax int64 - appendChunkOpts chunkOpts - enc record.Encoder -} - -// commitExemplars adds all exemplars from the provided batch to the head's exemplar storage. -func (a *headAppender) commitExemplars(b *appendBatch) { - // No errors logging to WAL, so pass the exemplars along to the in memory storage. - for _, e := range b.exemplars { - s := a.head.series.getByID(chunks.HeadSeriesRef(e.ref)) - if s == nil { - // This is very unlikely to happen, but we have seen it in the wild. - // It means that the series was truncated between AppendExemplar and Commit. - // See TestHeadCompactionWhileAppendAndCommitExemplar. - continue - } - // We don't instrument exemplar appends here, all is instrumented by storage. - if err := a.head.exemplars.AddExemplar(s.labels(), e.exemplar); err != nil { - if errors.Is(err, storage.ErrOutOfOrderExemplar) { - continue - } - a.head.logger.Debug("Unknown error while adding exemplar", "err", err) - } - } -} - -func (acc *appenderCommitContext) collectOOORecords(a *headAppender) { - if a.head.wbl == nil { - // WBL is not enabled. So no need to collect. - acc.wblSamples = nil - acc.wblHistograms = nil - acc.wblFloatHistograms = nil - acc.oooMmapMarkers = nil - acc.oooMmapMarkersCount = 0 - return - } - - // The m-map happens before adding a new sample. So we collect - // the m-map markers first, and then samples. - // WBL Graphically: - // WBL Before this Commit(): [old samples before this commit for chunk 1] - // WBL After this Commit(): [old samples before this commit for chunk 1][new samples in this commit for chunk 1]mmapmarker1[samples for chunk 2]mmapmarker2[samples for chunk 3] - if acc.oooMmapMarkers != nil { - markers := make([]record.RefMmapMarker, 0, acc.oooMmapMarkersCount) - for ref, mmapRefs := range acc.oooMmapMarkers { - for _, mmapRef := range mmapRefs { - markers = append(markers, record.RefMmapMarker{ - Ref: ref, - MmapRef: mmapRef, - }) - } - } - r := acc.enc.MmapMarkers(markers, a.head.getBytesBuffer()) - acc.oooRecords = append(acc.oooRecords, r) - } - - if len(acc.wblSamples) > 0 { - r := acc.enc.Samples(acc.wblSamples, a.head.getBytesBuffer()) - acc.oooRecords = append(acc.oooRecords, r) - } - if len(acc.wblHistograms) > 0 { - r, customBucketsHistograms := acc.enc.HistogramSamples(acc.wblHistograms, a.head.getBytesBuffer()) - if len(r) > 0 { - acc.oooRecords = append(acc.oooRecords, r) - } - if len(customBucketsHistograms) > 0 { - r := acc.enc.CustomBucketsHistogramSamples(customBucketsHistograms, a.head.getBytesBuffer()) - acc.oooRecords = append(acc.oooRecords, r) - } - } - if len(acc.wblFloatHistograms) > 0 { - r, customBucketsFloatHistograms := acc.enc.FloatHistogramSamples(acc.wblFloatHistograms, a.head.getBytesBuffer()) - if len(r) > 0 { - acc.oooRecords = append(acc.oooRecords, r) - } - if len(customBucketsFloatHistograms) > 0 { - r := acc.enc.CustomBucketsFloatHistogramSamples(customBucketsFloatHistograms, a.head.getBytesBuffer()) - acc.oooRecords = append(acc.oooRecords, r) - } - } - - acc.wblSamples = nil - acc.wblHistograms = nil - acc.wblFloatHistograms = nil - acc.oooMmapMarkers = nil -} - -// handleAppendableError processes errors encountered during sample appending and updates -// the provided counters accordingly. -// -// Parameters: -// - err: The error encountered during appending. -// - appended: Pointer to the counter tracking the number of successfully appended samples. -// - oooRejected: Pointer to the counter tracking the number of out-of-order samples rejected. -// - oobRejected: Pointer to the counter tracking the number of out-of-bounds samples rejected. -// - tooOldRejected: Pointer to the counter tracking the number of too-old samples rejected. -func handleAppendableError(err error, appended, oooRejected, oobRejected, tooOldRejected *int) { - switch { - case errors.Is(err, storage.ErrOutOfOrderSample): - *appended-- - *oooRejected++ - case errors.Is(err, storage.ErrOutOfBounds): - *appended-- - *oobRejected++ - case errors.Is(err, storage.ErrTooOldSample): - *appended-- - *tooOldRejected++ - default: - *appended-- - } -} - -// commitFloats processes and commits the samples in the provided batch to the -// series. It handles both in-order and out-of-order samples, updating the -// appenderCommitContext with the results of the append operations. -// -// The function iterates over the samples in the headAppender and attempts to append each sample -// to its corresponding series. It handles various error cases such as out-of-order samples, -// out-of-bounds samples, and too-old samples, updating the appenderCommitContext accordingly. -// -// For out-of-order samples, it checks if the sample can be inserted into the series and updates -// the out-of-order mmap markers if necessary. It also updates the write-ahead log (WBL) samples -// and the minimum and maximum timestamps for out-of-order samples. -// -// For in-order samples, it attempts to append the sample to the series and updates the minimum -// and maximum timestamps for in-order samples. -// -// The function also increments the chunk metrics if a new chunk is created and performs cleanup -// operations on the series after appending the samples. -// -// There are also specific functions to commit histograms and float histograms. -func (a *headAppender) commitFloats(b *appendBatch, acc *appenderCommitContext) { - var ok, chunkCreated bool - var series *memSeries - - for i, s := range b.floats { - series = b.floatSeries[i] - series.Lock() - - if value.IsStaleNaN(s.V) { - // If a float staleness marker had been appended for a - // series that got a histogram or float histogram - // appended before via this same appender, it would not - // show up here because we had already converted it. We - // end up here for two reasons: (1) This is the very - // first sample for this series appended via this - // appender. (2) A float sample was appended to this - // series before via this same appender. - // - // In either case, we need to check the previous sample - // in the memSeries to append the appropriately typed - // staleness marker. This is obviously so in case (1). - // In case (2), we would usually expect a float sample - // as the previous sample, but there might be concurrent - // appends that have added a histogram sample in the - // meantime. (This will probably lead to OOO shenanigans - // anyway, but that's a different story.) - // - // If the last sample in the memSeries is indeed a - // float, we don't have to do anything special here and - // just go on with the normal commit for a float sample. - // However, if the last sample in the memSeries is a - // histogram or float histogram, we have to convert the - // staleness marker to a histogram (or float histogram, - // respectively), and just add it at the end of the - // histograms (or float histograms) in the same batch, - // to be committed later in commitHistograms (or - // commitFloatHistograms). The latter is fine because we - // know there is no other histogram (or float histogram) - // sample for this same series in this same batch - // (because any such sample would have triggered a new - // batch). - switch { - case series.lastHistogramValue != nil: - b.histograms = append(b.histograms, record.RefHistogramSample{ - Ref: series.ref, - T: s.T, - H: &histogram.Histogram{Sum: s.V}, - }) - b.histogramSeries = append(b.histogramSeries, series) - // This sample was counted as a float but is now a histogram. - acc.floatsAppended-- - acc.histogramsAppended++ - series.Unlock() - continue - case series.lastFloatHistogramValue != nil: - b.floatHistograms = append(b.floatHistograms, record.RefFloatHistogramSample{ - Ref: series.ref, - T: s.T, - FH: &histogram.FloatHistogram{Sum: s.V}, - }) - b.floatHistogramSeries = append(b.floatHistogramSeries, series) - // This sample was counted as a float but is now a float histogram. - acc.floatsAppended-- - acc.histogramsAppended++ - series.Unlock() - continue - } - } - oooSample, _, err := series.appendable(s.T, s.V, a.headMaxt, a.minValidTime, a.oooTimeWindow) - if err != nil { - handleAppendableError(err, &acc.floatsAppended, &acc.floatOOORejected, &acc.floatOOBRejected, &acc.floatTooOldRejected) - } - - switch { - case err != nil: - // Do nothing here. - case oooSample: - // Sample is OOO and OOO handling is enabled - // and the delta is within the OOO tolerance. - var mmapRefs []chunks.ChunkDiskMapperRef - ok, chunkCreated, mmapRefs = series.insert(s.T, s.V, nil, nil, a.head.chunkDiskMapper, acc.oooCapMax, a.head.logger) - if chunkCreated { - r, ok := acc.oooMmapMarkers[series.ref] - if !ok || r != nil { - // !ok means there are no markers collected for these samples yet. So we first flush the samples - // before setting this m-map marker. - - // r != nil means we have already m-mapped a chunk for this series in the same Commit(). - // Hence, before we m-map again, we should add the samples and m-map markers - // seen till now to the WBL records. - acc.collectOOORecords(a) - } - - if acc.oooMmapMarkers == nil { - acc.oooMmapMarkers = make(map[chunks.HeadSeriesRef][]chunks.ChunkDiskMapperRef) - } - if len(mmapRefs) > 0 { - acc.oooMmapMarkers[series.ref] = mmapRefs - acc.oooMmapMarkersCount += len(mmapRefs) - } else { - // No chunk was written to disk, so we need to set an initial marker for this series. - acc.oooMmapMarkers[series.ref] = []chunks.ChunkDiskMapperRef{0} - acc.oooMmapMarkersCount++ - } - } - if ok { - acc.wblSamples = append(acc.wblSamples, s) - if s.T < acc.oooMinT { - acc.oooMinT = s.T - } - if s.T > acc.oooMaxT { - acc.oooMaxT = s.T - } - acc.oooFloatsAccepted++ - } else { - // Sample is an exact duplicate of the last sample. - // NOTE: We can only detect updates if they clash with a sample in the OOOHeadChunk, - // not with samples in already flushed OOO chunks. - // TODO(codesome): Add error reporting? It depends on addressing https://github.com/prometheus/prometheus/discussions/10305. - acc.floatsAppended-- - } - default: - newlyStale := !value.IsStaleNaN(series.lastValue) && value.IsStaleNaN(s.V) - staleToNonStale := value.IsStaleNaN(series.lastValue) && !value.IsStaleNaN(s.V) - ok, chunkCreated = series.append(s.T, s.V, a.appendID, acc.appendChunkOpts) - if ok { - if s.T < acc.inOrderMint { - acc.inOrderMint = s.T - } - if s.T > acc.inOrderMaxt { - acc.inOrderMaxt = s.T - } - if newlyStale { - a.head.numStaleSeries.Inc() - } - if staleToNonStale { - a.head.numStaleSeries.Dec() - } - } else { - // The sample is an exact duplicate, and should be silently dropped. - acc.floatsAppended-- - } - } - - if chunkCreated { - a.head.metrics.chunks.Inc() - a.head.metrics.chunksCreated.Inc() - } - - series.cleanupAppendIDsBelow(a.cleanupAppendIDsBelow) - series.pendingCommit = false - series.Unlock() - } -} - -// For details on the commitHistograms function, see the commitFloats docs. -func (a *headAppender) commitHistograms(b *appendBatch, acc *appenderCommitContext) { - var ok, chunkCreated bool - var series *memSeries - - for i, s := range b.histograms { - series = b.histogramSeries[i] - series.Lock() - - // At this point, we could encounter a histogram staleness - // marker that should better be a float staleness marker or a - // float histogram staleness marker. This can only happen with - // concurrent appenders appending to the same series _and_ doing - // so in a mixed-type scenario. This case is expected to be very - // rare, so we do not bother here to convert the staleness - // marker. The worst case is that we need to cut a new chunk - // just for the staleness marker. - - oooSample, _, err := series.appendableHistogram(s.T, s.H, a.headMaxt, a.minValidTime, a.oooTimeWindow) - if err != nil { - handleAppendableError(err, &acc.histogramsAppended, &acc.histoOOORejected, &acc.histoOOBRejected, &acc.histoTooOldRejected) - } - - switch { - case err != nil: - // Do nothing here. - case oooSample: - // Sample is OOO and OOO handling is enabled - // and the delta is within the OOO tolerance. - var mmapRefs []chunks.ChunkDiskMapperRef - ok, chunkCreated, mmapRefs = series.insert(s.T, 0, s.H, nil, a.head.chunkDiskMapper, acc.oooCapMax, a.head.logger) - if chunkCreated { - r, ok := acc.oooMmapMarkers[series.ref] - if !ok || r != nil { - // !ok means there are no markers collected for these samples yet. So we first flush the samples - // before setting this m-map marker. - - // r != 0 means we have already m-mapped a chunk for this series in the same Commit(). - // Hence, before we m-map again, we should add the samples and m-map markers - // seen till now to the WBL records. - acc.collectOOORecords(a) - } - - if acc.oooMmapMarkers == nil { - acc.oooMmapMarkers = make(map[chunks.HeadSeriesRef][]chunks.ChunkDiskMapperRef) - } - if len(mmapRefs) > 0 { - acc.oooMmapMarkers[series.ref] = mmapRefs - acc.oooMmapMarkersCount += len(mmapRefs) - } else { - // No chunk was written to disk, so we need to set an initial marker for this series. - acc.oooMmapMarkers[series.ref] = []chunks.ChunkDiskMapperRef{0} - acc.oooMmapMarkersCount++ - } - } - if ok { - acc.wblHistograms = append(acc.wblHistograms, s) - if s.T < acc.oooMinT { - acc.oooMinT = s.T - } - if s.T > acc.oooMaxT { - acc.oooMaxT = s.T - } - acc.oooHistogramAccepted++ - } else { - // Sample is an exact duplicate of the last sample. - // NOTE: We can only detect updates if they clash with a sample in the OOOHeadChunk, - // not with samples in already flushed OOO chunks. - // TODO(codesome): Add error reporting? It depends on addressing https://github.com/prometheus/prometheus/discussions/10305. - acc.histogramsAppended-- - } - default: - newlyStale := value.IsStaleNaN(s.H.Sum) - staleToNonStale := false - if series.lastHistogramValue != nil { - newlyStale = newlyStale && !value.IsStaleNaN(series.lastHistogramValue.Sum) - staleToNonStale = value.IsStaleNaN(series.lastHistogramValue.Sum) && !value.IsStaleNaN(s.H.Sum) - } - ok, chunkCreated = series.appendHistogram(s.T, s.H, a.appendID, acc.appendChunkOpts) - if ok { - if s.T < acc.inOrderMint { - acc.inOrderMint = s.T - } - if s.T > acc.inOrderMaxt { - acc.inOrderMaxt = s.T - } - if newlyStale { - a.head.numStaleSeries.Inc() - } - if staleToNonStale { - a.head.numStaleSeries.Dec() - } - } else { - acc.histogramsAppended-- - acc.histoOOORejected++ - } - } - - if chunkCreated { - a.head.metrics.chunks.Inc() - a.head.metrics.chunksCreated.Inc() - } - - series.cleanupAppendIDsBelow(a.cleanupAppendIDsBelow) - series.pendingCommit = false - series.Unlock() - } -} - -// For details on the commitFloatHistograms function, see the commitFloats docs. -func (a *headAppender) commitFloatHistograms(b *appendBatch, acc *appenderCommitContext) { - var ok, chunkCreated bool - var series *memSeries - - for i, s := range b.floatHistograms { - series = b.floatHistogramSeries[i] - series.Lock() - - // At this point, we could encounter a float histogram staleness - // marker that should better be a float staleness marker or an - // integer histogram staleness marker. This can only happen with - // concurrent appenders appending to the same series _and_ doing - // so in a mixed-type scenario. This case is expected to be very - // rare, so we do not bother here to convert the staleness - // marker. The worst case is that we need to cut a new chunk - // just for the staleness marker. - - oooSample, _, err := series.appendableFloatHistogram(s.T, s.FH, a.headMaxt, a.minValidTime, a.oooTimeWindow) - if err != nil { - handleAppendableError(err, &acc.histogramsAppended, &acc.histoOOORejected, &acc.histoOOBRejected, &acc.histoTooOldRejected) - } - - switch { - case err != nil: - // Do nothing here. - case oooSample: - // Sample is OOO and OOO handling is enabled - // and the delta is within the OOO tolerance. - var mmapRefs []chunks.ChunkDiskMapperRef - ok, chunkCreated, mmapRefs = series.insert(s.T, 0, nil, s.FH, a.head.chunkDiskMapper, acc.oooCapMax, a.head.logger) - if chunkCreated { - r, ok := acc.oooMmapMarkers[series.ref] - if !ok || r != nil { - // !ok means there are no markers collected for these samples yet. So we first flush the samples - // before setting this m-map marker. - - // r != 0 means we have already m-mapped a chunk for this series in the same Commit(). - // Hence, before we m-map again, we should add the samples and m-map markers - // seen till now to the WBL records. - acc.collectOOORecords(a) - } - - if acc.oooMmapMarkers == nil { - acc.oooMmapMarkers = make(map[chunks.HeadSeriesRef][]chunks.ChunkDiskMapperRef) - } - if len(mmapRefs) > 0 { - acc.oooMmapMarkers[series.ref] = mmapRefs - acc.oooMmapMarkersCount += len(mmapRefs) - } else { - // No chunk was written to disk, so we need to set an initial marker for this series. - acc.oooMmapMarkers[series.ref] = []chunks.ChunkDiskMapperRef{0} - acc.oooMmapMarkersCount++ - } - } - if ok { - acc.wblFloatHistograms = append(acc.wblFloatHistograms, s) - if s.T < acc.oooMinT { - acc.oooMinT = s.T - } - if s.T > acc.oooMaxT { - acc.oooMaxT = s.T - } - acc.oooHistogramAccepted++ - } else { - // Sample is an exact duplicate of the last sample. - // NOTE: We can only detect updates if they clash with a sample in the OOOHeadChunk, - // not with samples in already flushed OOO chunks. - // TODO(codesome): Add error reporting? It depends on addressing https://github.com/prometheus/prometheus/discussions/10305. - acc.histogramsAppended-- - } - default: - newlyStale := value.IsStaleNaN(s.FH.Sum) - staleToNonStale := false - if series.lastFloatHistogramValue != nil { - newlyStale = newlyStale && !value.IsStaleNaN(series.lastFloatHistogramValue.Sum) - staleToNonStale = value.IsStaleNaN(series.lastFloatHistogramValue.Sum) && !value.IsStaleNaN(s.FH.Sum) - } - ok, chunkCreated = series.appendFloatHistogram(s.T, s.FH, a.appendID, acc.appendChunkOpts) - if ok { - if s.T < acc.inOrderMint { - acc.inOrderMint = s.T - } - if s.T > acc.inOrderMaxt { - acc.inOrderMaxt = s.T - } - if newlyStale { - a.head.numStaleSeries.Inc() - } - if staleToNonStale { - a.head.numStaleSeries.Dec() - } - } else { - acc.histogramsAppended-- - acc.histoOOORejected++ - } - } - - if chunkCreated { - a.head.metrics.chunks.Inc() - a.head.metrics.chunksCreated.Inc() - } - - series.cleanupAppendIDsBelow(a.cleanupAppendIDsBelow) - series.pendingCommit = false - series.Unlock() - } -} - -// commitMetadata commits the metadata for each series in the provided batch. -// It iterates over the metadata slice and updates the corresponding series -// with the new metadata information. The series is locked during the update -// to ensure thread safety. -func commitMetadata(b *appendBatch) { - var series *memSeries - for i, m := range b.metadata { - series = b.metadataSeries[i] - series.Lock() - series.meta = &metadata.Metadata{Type: record.ToMetricType(m.Type), Unit: m.Unit, Help: m.Help} - series.Unlock() - } -} - -func (a *headAppender) unmarkCreatedSeriesAsPendingCommit() { - for _, s := range a.series { - s.Lock() - s.pendingCommit = false - s.Unlock() - } -} - -// Commit writes to the WAL and adds the data to the Head. -// TODO(codesome): Refactor this method to reduce indentation and make it more readable. -func (a *headAppender) Commit() (err error) { - if a.closed { - return ErrAppenderClosed - } - - h := a.head - - defer func() { - if a.closed { - // Don't double-close in case Rollback() was called. - return - } - h.putRefSeriesBuffer(a.seriesRefs) - h.putSeriesBuffer(a.series) - h.putTypeMap(a.typesInBatch) - a.closed = true - }() - - if err := a.log(); err != nil { - _ = a.Rollback() // Most likely the same error will happen again. - return fmt.Errorf("write to WAL: %w", err) - } - - if h.writeNotified != nil { - h.writeNotified.Notify() - } - - acc := &appenderCommitContext{ - inOrderMint: math.MaxInt64, - inOrderMaxt: math.MinInt64, - oooMinT: math.MaxInt64, - oooMaxT: math.MinInt64, - oooCapMax: h.opts.OutOfOrderCapMax.Load(), - appendChunkOpts: chunkOpts{ - chunkDiskMapper: h.chunkDiskMapper, - chunkRange: h.chunkRange.Load(), - samplesPerChunk: h.opts.SamplesPerChunk, - }, - } - - for _, b := range a.batches { - acc.floatsAppended += len(b.floats) - acc.histogramsAppended += len(b.histograms) + len(b.floatHistograms) - a.commitExemplars(b) - defer b.close(h) - } - defer h.metrics.activeAppenders.Dec() - defer h.iso.closeAppend(a.appendID) - - defer func() { - for i := range acc.oooRecords { - h.putBytesBuffer(acc.oooRecords[i][:0]) - } - }() - - for _, b := range a.batches { - // Do not change the order of these calls. We depend on it for - // correct commit order of samples and for the staleness marker - // handling. - a.commitFloats(b, acc) - a.commitHistograms(b, acc) - a.commitFloatHistograms(b, acc) - commitMetadata(b) - } - // Unmark all series as pending commit after all samples have been committed. - a.unmarkCreatedSeriesAsPendingCommit() - - h.metrics.outOfOrderSamples.WithLabelValues(sampleMetricTypeFloat).Add(float64(acc.floatOOORejected)) - h.metrics.outOfOrderSamples.WithLabelValues(sampleMetricTypeHistogram).Add(float64(acc.histoOOORejected)) - h.metrics.outOfBoundSamples.WithLabelValues(sampleMetricTypeFloat).Add(float64(acc.floatOOBRejected)) - h.metrics.tooOldSamples.WithLabelValues(sampleMetricTypeFloat).Add(float64(acc.floatTooOldRejected)) - h.metrics.samplesAppended.WithLabelValues(sampleMetricTypeFloat).Add(float64(acc.floatsAppended)) - h.metrics.samplesAppended.WithLabelValues(sampleMetricTypeHistogram).Add(float64(acc.histogramsAppended)) - h.metrics.outOfOrderSamplesAppended.WithLabelValues(sampleMetricTypeFloat).Add(float64(acc.oooFloatsAccepted)) - h.metrics.outOfOrderSamplesAppended.WithLabelValues(sampleMetricTypeHistogram).Add(float64(acc.oooHistogramAccepted)) - h.updateMinMaxTime(acc.inOrderMint, acc.inOrderMaxt) - h.updateMinOOOMaxOOOTime(acc.oooMinT, acc.oooMaxT) - - acc.collectOOORecords(a) - if h.wbl != nil { - if err := h.wbl.Log(acc.oooRecords...); err != nil { - // TODO(codesome): Currently WBL logging of ooo samples is best effort here since we cannot try logging - // until we have found what samples become OOO. We can try having a metric for this failure. - // Returning the error here is not correct because we have already put the samples into the memory, - // hence the append/insert was a success. - h.logger.Error("Failed to log out of order samples into the WAL", "err", err) - } - } - return nil -} - -// insert is like append, except it inserts. Used for OOO samples. -func (s *memSeries) insert(t int64, v float64, h *histogram.Histogram, fh *histogram.FloatHistogram, chunkDiskMapper *chunks.ChunkDiskMapper, oooCapMax int64, logger *slog.Logger) (inserted, chunkCreated bool, mmapRefs []chunks.ChunkDiskMapperRef) { - if s.ooo == nil { - s.ooo = &memSeriesOOOFields{} - } - c := s.ooo.oooHeadChunk - if c == nil || c.chunk.NumSamples() == int(oooCapMax) { - // Note: If no new samples come in then we rely on compaction to clean up stale in-memory OOO chunks. - c, mmapRefs = s.cutNewOOOHeadChunk(t, chunkDiskMapper, logger) - chunkCreated = true - } - - ok := c.chunk.Insert(t, v, h, fh) - if ok { - if chunkCreated || t < c.minTime { - c.minTime = t - } - if chunkCreated || t > c.maxTime { - c.maxTime = t - } - } - return ok, chunkCreated, mmapRefs -} - -// chunkOpts are chunk-level options that are passed when appending to a memSeries. -type chunkOpts struct { - chunkDiskMapper *chunks.ChunkDiskMapper - chunkRange int64 - samplesPerChunk int -} - -// append adds the sample (t, v) to the series. The caller also has to provide -// the appendID for isolation. (The appendID can be zero, which results in no -// isolation for this append.) -// Series lock must be held when calling. -func (s *memSeries) append(t int64, v float64, appendID uint64, o chunkOpts) (sampleInOrder, chunkCreated bool) { - c, sampleInOrder, chunkCreated := s.appendPreprocessor(t, chunkenc.EncXOR, o) - if !sampleInOrder { - return sampleInOrder, chunkCreated - } - s.app.Append(t, v) - - c.maxTime = t - - s.lastValue = v - s.lastHistogramValue = nil - s.lastFloatHistogramValue = nil - - if appendID > 0 { - s.txs.add(appendID) - } - - return true, chunkCreated -} - -// appendHistogram adds the histogram. -// It is unsafe to call this concurrently with s.iterator(...) without holding the series lock. -// In case of recoding the existing chunk, a new chunk is allocated and the old chunk is dropped. -// To keep the meaning of prometheus_tsdb_head_chunks and prometheus_tsdb_head_chunks_created_total -// consistent, we return chunkCreated=false in this case. -func (s *memSeries) appendHistogram(t int64, h *histogram.Histogram, appendID uint64, o chunkOpts) (sampleInOrder, chunkCreated bool) { - // Head controls the execution of recoding, so that we own the proper - // chunk reference afterwards and mmap used up chunks. - - // Ignoring ok is ok, since we don't want to compare to the wrong previous appender anyway. - prevApp, _ := s.app.(*chunkenc.HistogramAppender) - - c, sampleInOrder, chunkCreated := s.histogramsAppendPreprocessor(t, chunkenc.EncHistogram, o) - if !sampleInOrder { - return sampleInOrder, chunkCreated - } - - var ( - newChunk chunkenc.Chunk - recoded bool - ) - - if !chunkCreated { - // Ignore the previous appender if we continue the current chunk. - prevApp = nil - } - - newChunk, recoded, s.app, _ = s.app.AppendHistogram(prevApp, t, h, false) // false=request a new chunk if needed - - s.lastHistogramValue = h - s.lastFloatHistogramValue = nil - - if appendID > 0 { - s.txs.add(appendID) - } - - if newChunk == nil { // Sample was appended to existing chunk or is the first sample in a new chunk. - c.maxTime = t - return true, chunkCreated - } - - if recoded { // The appender needed to recode the chunk. - c.maxTime = t - c.chunk = newChunk - return true, false - } - - s.headChunks = &memChunk{ - chunk: newChunk, - minTime: t, - maxTime: t, - prev: s.headChunks, - } - s.nextAt = rangeForTimestamp(t, o.chunkRange) - return true, true -} - -// appendFloatHistogram adds the float histogram. -// It is unsafe to call this concurrently with s.iterator(...) without holding the series lock. -// In case of recoding the existing chunk, a new chunk is allocated and the old chunk is dropped. -// To keep the meaning of prometheus_tsdb_head_chunks and prometheus_tsdb_head_chunks_created_total -// consistent, we return chunkCreated=false in this case. -func (s *memSeries) appendFloatHistogram(t int64, fh *histogram.FloatHistogram, appendID uint64, o chunkOpts) (sampleInOrder, chunkCreated bool) { - // Head controls the execution of recoding, so that we own the proper - // chunk reference afterwards and mmap used up chunks. - - // Ignoring ok is ok, since we don't want to compare to the wrong previous appender anyway. - prevApp, _ := s.app.(*chunkenc.FloatHistogramAppender) - - c, sampleInOrder, chunkCreated := s.histogramsAppendPreprocessor(t, chunkenc.EncFloatHistogram, o) - if !sampleInOrder { - return sampleInOrder, chunkCreated - } - - var ( - newChunk chunkenc.Chunk - recoded bool - ) - - if !chunkCreated { - // Ignore the previous appender if we continue the current chunk. - prevApp = nil - } - - newChunk, recoded, s.app, _ = s.app.AppendFloatHistogram(prevApp, t, fh, false) // False means request a new chunk if needed. - - s.lastHistogramValue = nil - s.lastFloatHistogramValue = fh - - if appendID > 0 { - s.txs.add(appendID) - } - - if newChunk == nil { // Sample was appended to existing chunk or is the first sample in a new chunk. - c.maxTime = t - return true, chunkCreated - } - - if recoded { // The appender needed to recode the chunk. - c.maxTime = t - c.chunk = newChunk - return true, false - } - - s.headChunks = &memChunk{ - chunk: newChunk, - minTime: t, - maxTime: t, - prev: s.headChunks, - } - s.nextAt = rangeForTimestamp(t, o.chunkRange) - return true, true -} - -// appendPreprocessor takes care of cutting new XOR chunks and m-mapping old ones. XOR chunks are cut based on the -// number of samples they contain with a soft cap in bytes. -// It is unsafe to call this concurrently with s.iterator(...) without holding the series lock. -// This should be called only when appending data. -func (s *memSeries) appendPreprocessor(t int64, e chunkenc.Encoding, o chunkOpts) (c *memChunk, sampleInOrder, chunkCreated bool) { - // We target chunkenc.MaxBytesPerXORChunk as a hard for the size of an XOR chunk. We must determine whether to cut - // a new head chunk without knowing the size of the next sample, however, so we assume the next sample will be a - // maximally-sized sample (19 bytes). - const maxBytesPerXORChunk = chunkenc.MaxBytesPerXORChunk - 19 - - c = s.headChunks - - if c == nil { - if len(s.mmappedChunks) > 0 && s.mmappedChunks[len(s.mmappedChunks)-1].maxTime >= t { - // Out of order sample. Sample timestamp is already in the mmapped chunks, so ignore it. - return c, false, false - } - // There is no head chunk in this series yet, create the first chunk for the sample. - c = s.cutNewHeadChunk(t, e, o.chunkRange) - chunkCreated = true - } - - // Out of order sample. - if c.maxTime >= t { - return c, false, chunkCreated - } - - // Check the chunk size, unless we just created it and if the chunk is too large, cut a new one. - if !chunkCreated && len(c.chunk.Bytes()) > maxBytesPerXORChunk { - c = s.cutNewHeadChunk(t, e, o.chunkRange) - chunkCreated = true - } - - if c.chunk.Encoding() != e { - // The chunk encoding expected by this append is different than the head chunk's - // encoding. So we cut a new chunk with the expected encoding. - c = s.cutNewHeadChunk(t, e, o.chunkRange) - chunkCreated = true - } - - numSamples := c.chunk.NumSamples() - if numSamples == 0 { - // It could be the new chunk created after reading the chunk snapshot, - // hence we fix the minTime of the chunk here. - c.minTime = t - s.nextAt = rangeForTimestamp(c.minTime, o.chunkRange) - } - - // If we reach 25% of a chunk's desired sample count, predict an end time - // for this chunk that will try to make samples equally distributed within - // the remaining chunks in the current chunk range. - // At latest it must happen at the timestamp set when the chunk was cut. - if numSamples == o.samplesPerChunk/4 { - s.nextAt = computeChunkEndTime(c.minTime, c.maxTime, s.nextAt, 4) - } - // If numSamples > samplesPerChunk*2 then our previous prediction was invalid, - // most likely because samples rate has changed and now they are arriving more frequently. - // Since we assume that the rate is higher, we're being conservative and cutting at 2*samplesPerChunk - // as we expect more chunks to come. - // Note that next chunk will have its nextAt recalculated for the new rate. - if t >= s.nextAt || numSamples >= o.samplesPerChunk*2 { - c = s.cutNewHeadChunk(t, e, o.chunkRange) - chunkCreated = true - } - - return c, true, chunkCreated -} - -// histogramsAppendPreprocessor takes care of cutting new histogram chunks and m-mapping old ones. Histogram chunks are -// cut based on their size in bytes. -// It is unsafe to call this concurrently with s.iterator(...) without holding the series lock. -// This should be called only when appending data. -func (s *memSeries) histogramsAppendPreprocessor(t int64, e chunkenc.Encoding, o chunkOpts) (c *memChunk, sampleInOrder, chunkCreated bool) { - c = s.headChunks - - if c == nil { - if len(s.mmappedChunks) > 0 && s.mmappedChunks[len(s.mmappedChunks)-1].maxTime >= t { - // Out of order sample. Sample timestamp is already in the mmapped chunks, so ignore it. - return c, false, false - } - // There is no head chunk in this series yet, create the first chunk for the sample. - c = s.cutNewHeadChunk(t, e, o.chunkRange) - chunkCreated = true - } - - // Out of order sample. - if c.maxTime >= t { - return c, false, chunkCreated - } - - if c.chunk.Encoding() != e { - // The chunk encoding expected by this append is different than the head chunk's - // encoding. So we cut a new chunk with the expected encoding. - c = s.cutNewHeadChunk(t, e, o.chunkRange) - chunkCreated = true - } - - numSamples := c.chunk.NumSamples() - targetBytes := chunkenc.TargetBytesPerHistogramChunk - numBytes := len(c.chunk.Bytes()) - - if numSamples == 0 { - // It could be the new chunk created after reading the chunk snapshot, - // hence we fix the minTime of the chunk here. - c.minTime = t - s.nextAt = rangeForTimestamp(c.minTime, o.chunkRange) - } - - // Below, we will enforce chunkenc.MinSamplesPerHistogramChunk. There are, however, two cases that supersede it: - // - The current chunk range is ending before chunkenc.MinSamplesPerHistogramChunk will be satisfied. - // - s.nextAt was set while loading a chunk snapshot with the intent that a new chunk be cut on the next append. - var nextChunkRangeStart int64 - if s.histogramChunkHasComputedEndTime { - nextChunkRangeStart = rangeForTimestamp(c.minTime, o.chunkRange) - } else { - // If we haven't yet computed an end time yet, s.nextAt is either set to - // rangeForTimestamp(c.minTime, o.chunkRange) or was set while loading a chunk snapshot. Either way, we want to - // skip enforcing chunkenc.MinSamplesPerHistogramChunk. - nextChunkRangeStart = s.nextAt - } - - // If we reach 25% of a chunk's desired maximum size, predict an end time - // for this chunk that will try to make samples equally distributed within - // the remaining chunks in the current chunk range. - // At the latest it must happen at the timestamp set when the chunk was cut. - if !s.histogramChunkHasComputedEndTime && numBytes >= targetBytes/4 { - ratioToFull := float64(targetBytes) / float64(numBytes) - s.nextAt = computeChunkEndTime(c.minTime, c.maxTime, s.nextAt, ratioToFull) - s.histogramChunkHasComputedEndTime = true - } - // If numBytes > targetBytes*2 then our previous prediction was invalid. This could happen if the sample rate has - // increased or if the bucket/span count has increased. - // Note that next chunk will have its nextAt recalculated for the new rate. - if (t >= s.nextAt || numBytes >= targetBytes*2) && (numSamples >= chunkenc.MinSamplesPerHistogramChunk || t >= nextChunkRangeStart) { - c = s.cutNewHeadChunk(t, e, o.chunkRange) - chunkCreated = true - } - - // The new chunk will also need a new computed end time. - if chunkCreated { - s.histogramChunkHasComputedEndTime = false - } - - return c, true, chunkCreated -} - -// computeChunkEndTime estimates the end timestamp based the beginning of a -// chunk, its current timestamp and the upper bound up to which we insert data. -// It assumes that the time range is 1/ratioToFull full. -// Assuming that the samples will keep arriving at the same rate, it will make the -// remaining n chunks within this chunk range (before max) equally sized. -func computeChunkEndTime(start, cur, maxT int64, ratioToFull float64) int64 { - n := float64(maxT-start) / (float64(cur-start+1) * ratioToFull) - if n <= 1 { - return maxT - } - return int64(float64(start) + float64(maxT-start)/math.Floor(n)) -} - -func (s *memSeries) cutNewHeadChunk(mint int64, e chunkenc.Encoding, chunkRange int64) *memChunk { - // When cutting a new head chunk we create a new memChunk instance with .prev - // pointing at the current .headChunks, so it forms a linked list. - // All but first headChunks list elements will be m-mapped as soon as possible - // so this is a single element list most of the time. - s.headChunks = &memChunk{ - minTime: mint, - maxTime: math.MinInt64, - prev: s.headChunks, - } - - if chunkenc.IsValidEncoding(e) { - var err error - s.headChunks.chunk, err = chunkenc.NewEmptyChunk(e) - if err != nil { - panic(err) // This should never happen. - } - } else { - s.headChunks.chunk = chunkenc.NewXORChunk() - } - - // Set upper bound on when the next chunk must be started. An earlier timestamp - // may be chosen dynamically at a later point. - s.nextAt = rangeForTimestamp(mint, chunkRange) - - app, err := s.headChunks.chunk.Appender() - if err != nil { - panic(err) - } - s.app = app - return s.headChunks -} - -// cutNewOOOHeadChunk cuts a new OOO chunk and m-maps the old chunk. -// The caller must ensure that s is locked and s.ooo is not nil. -func (s *memSeries) cutNewOOOHeadChunk(mint int64, chunkDiskMapper *chunks.ChunkDiskMapper, logger *slog.Logger) (*oooHeadChunk, []chunks.ChunkDiskMapperRef) { - ref := s.mmapCurrentOOOHeadChunk(chunkDiskMapper, logger) - - s.ooo.oooHeadChunk = &oooHeadChunk{ - chunk: NewOOOChunk(), - minTime: mint, - maxTime: math.MinInt64, - } - - return s.ooo.oooHeadChunk, ref -} - -// s must be locked when calling. -func (s *memSeries) mmapCurrentOOOHeadChunk(chunkDiskMapper *chunks.ChunkDiskMapper, logger *slog.Logger) []chunks.ChunkDiskMapperRef { - if s.ooo == nil || s.ooo.oooHeadChunk == nil { - // OOO is not enabled or there is no head chunk, so nothing to m-map here. - return nil - } - chks, err := s.ooo.oooHeadChunk.chunk.ToEncodedChunks(math.MinInt64, math.MaxInt64) - if err != nil { - handleChunkWriteError(err) - return nil - } - chunkRefs := make([]chunks.ChunkDiskMapperRef, 0, len(chks)) - for _, memchunk := range chks { - if len(s.ooo.oooMmappedChunks) >= (oooChunkIDMask - 1) { - logger.Error("Too many OOO chunks, dropping data", "series", s.lset.String()) - break - } - chunkRef := chunkDiskMapper.WriteChunk(s.ref, memchunk.minTime, memchunk.maxTime, memchunk.chunk, true, handleChunkWriteError) - chunkRefs = append(chunkRefs, chunkRef) - s.ooo.oooMmappedChunks = append(s.ooo.oooMmappedChunks, &mmappedChunk{ - ref: chunkRef, - numSamples: uint16(memchunk.chunk.NumSamples()), - minTime: memchunk.minTime, - maxTime: memchunk.maxTime, - }) - } - s.ooo.oooHeadChunk = nil - return chunkRefs -} - -// mmapChunks will m-map all but first chunk on s.headChunks list. -func (s *memSeries) mmapChunks(chunkDiskMapper *chunks.ChunkDiskMapper) (count int) { - if s.headChunks == nil || s.headChunks.prev == nil { - // There is none or only one head chunk, so nothing to m-map here. - return count - } - - // Write chunks starting from the oldest one and stop before we get to current s.headChunks. - // If we have this chain: s.headChunks{t4} -> t3 -> t2 -> t1 -> t0 - // then we need to write chunks t0 to t3, but skip s.headChunks. - for i := s.headChunks.len() - 1; i > 0; i-- { - chk := s.headChunks.atOffset(i) - chunkRef := chunkDiskMapper.WriteChunk(s.ref, chk.minTime, chk.maxTime, chk.chunk, false, handleChunkWriteError) - s.mmappedChunks = append(s.mmappedChunks, &mmappedChunk{ - ref: chunkRef, - numSamples: uint16(chk.chunk.NumSamples()), - minTime: chk.minTime, - maxTime: chk.maxTime, - }) - count++ - } - - // Once we've written out all chunks except s.headChunks we need to unlink these from s.headChunk. - s.headChunks.prev = nil - - return count -} - -func handleChunkWriteError(err error) { - if err != nil && !errors.Is(err, chunks.ErrChunkDiskMapperClosed) { - panic(err) - } -} - -// Rollback removes the samples and exemplars from headAppender and writes any series to WAL. -func (a *headAppender) Rollback() (err error) { - if a.closed { - return ErrAppenderClosed - } - h := a.head - defer func() { - a.unmarkCreatedSeriesAsPendingCommit() - h.iso.closeAppend(a.appendID) - h.metrics.activeAppenders.Dec() - a.closed = true - h.putRefSeriesBuffer(a.seriesRefs) - h.putSeriesBuffer(a.series) - h.putTypeMap(a.typesInBatch) - }() - - var series *memSeries - for _, b := range a.batches { - for i := range b.floats { - series = b.floatSeries[i] - series.Lock() - series.cleanupAppendIDsBelow(a.cleanupAppendIDsBelow) - series.pendingCommit = false - series.Unlock() - } - for i := range b.histograms { - series = b.histogramSeries[i] - series.Lock() - series.cleanupAppendIDsBelow(a.cleanupAppendIDsBelow) - series.pendingCommit = false - series.Unlock() - } - for i := range b.floatHistograms { - series = b.floatHistogramSeries[i] - series.Lock() - series.cleanupAppendIDsBelow(a.cleanupAppendIDsBelow) - series.pendingCommit = false - series.Unlock() - } - b.close(h) - } - a.batches = a.batches[:0] - // Series are created in the head memory regardless of rollback. Thus we have - // to log them to the WAL in any case. - return a.log() -} +var _ storage.GetRef = &headAppenderV2{} diff --git a/tsdb/head_append_v2_test.go b/tsdb/head_append_v2_test.go index 552db13d07..e9ea701a20 100644 --- a/tsdb/head_append_v2_test.go +++ b/tsdb/head_append_v2_test.go @@ -1,4 +1,4 @@ -// Copyright 2017 The Prometheus Authors +// Copyright The Prometheus Authors // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. // You may obtain a copy of the License at @@ -35,7 +35,6 @@ import ( "github.com/prometheus/client_golang/prometheus" prom_testutil "github.com/prometheus/client_golang/prometheus/testutil" dto "github.com/prometheus/client_model/go" - "github.com/prometheus/common/promslog" "github.com/stretchr/testify/require" "go.uber.org/atomic" "golang.org/x/sync/errgroup" @@ -48,8 +47,6 @@ import ( "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/tsdb/chunks" - "github.com/prometheus/prometheus/tsdb/fileutil" - "github.com/prometheus/prometheus/tsdb/index" "github.com/prometheus/prometheus/tsdb/record" "github.com/prometheus/prometheus/tsdb/tombstones" "github.com/prometheus/prometheus/tsdb/tsdbutil" @@ -58,463 +55,11 @@ import ( "github.com/prometheus/prometheus/util/testutil" ) -// newTestHeadDefaultOptions returns the HeadOptions that should be used by default in unit tests. -func newTestHeadDefaultOptions(chunkRange int64, oooEnabled bool) *HeadOptions { - opts := DefaultHeadOptions() - opts.ChunkRange = chunkRange - opts.EnableExemplarStorage = true - opts.MaxExemplars.Store(config.DefaultExemplarsConfig.MaxExemplars) - if oooEnabled { - opts.OutOfOrderTimeWindow.Store(10 * time.Minute.Milliseconds()) - } - return opts -} - -func newTestHead(t testing.TB, chunkRange int64, compressWAL compression.Type, oooEnabled bool) (*Head, *wlog.WL) { - return newTestHeadWithOptions(t, compressWAL, newTestHeadDefaultOptions(chunkRange, oooEnabled)) -} - -func newTestHeadWithOptions(t testing.TB, compressWAL compression.Type, opts *HeadOptions) (*Head, *wlog.WL) { - dir := t.TempDir() - wal, err := wlog.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, compressWAL) - require.NoError(t, err) - - // Override the chunks dir with the testing one. - opts.ChunkDirRoot = dir - - h, err := NewHead(nil, nil, wal, nil, opts, nil) - require.NoError(t, err) - - require.NoError(t, h.chunkDiskMapper.IterateAllChunks(func(chunks.HeadSeriesRef, chunks.ChunkDiskMapperRef, int64, int64, uint16, chunkenc.Encoding, bool) error { - return nil - })) - - return h, wal -} - -func BenchmarkCreateSeries(b *testing.B) { - series := genSeries(b.N, 10, 0, 0) - h, _ := newTestHead(b, 10000, compression.None, false) - b.Cleanup(func() { - require.NoError(b, h.Close()) - }) - - b.ReportAllocs() - b.ResetTimer() - - for _, s := range series { - h.getOrCreate(s.Labels().Hash(), s.Labels(), false) - } -} - -func BenchmarkHeadAppender_Append_Commit_ExistingSeries(b *testing.B) { - seriesCounts := []int{100, 1000, 10000} - series := genSeries(10000, 10, 0, 0) - - for _, seriesCount := range seriesCounts { - b.Run(fmt.Sprintf("%d series", seriesCount), func(b *testing.B) { - for _, samplesPerAppend := range []int64{1, 2, 5, 100} { - b.Run(fmt.Sprintf("%d samples per append", samplesPerAppend), func(b *testing.B) { - h, _ := newTestHead(b, 10000, compression.None, false) - b.Cleanup(func() { require.NoError(b, h.Close()) }) - - ts := int64(1000) - appendSamples := func() error { - var err error - app := h.Appender(context.Background()) - for _, s := range series[:seriesCount] { - var ref storage.SeriesRef - for sampleIndex := range samplesPerAppend { - ref, err = app.Append(ref, s.Labels(), ts+sampleIndex, float64(ts+sampleIndex)) - if err != nil { - return err - } - } - } - ts += 1000 // should increment more than highest samplesPerAppend - return app.Commit() - } - - // Init series, that's not what we're benchmarking here. - require.NoError(b, appendSamples()) - - b.ReportAllocs() - b.ResetTimer() - - for b.Loop() { - require.NoError(b, appendSamples()) - } - }) - } - }) - } -} - -func populateTestWL(t testing.TB, w *wlog.WL, recs []any, buf []byte) []byte { - var enc record.Encoder - for _, r := range recs { - buf = buf[:0] - switch v := r.(type) { - case []record.RefSeries: - buf = enc.Series(v, buf) - case []record.RefSample: - buf = enc.Samples(v, buf) - case []tombstones.Stone: - buf = enc.Tombstones(v, buf) - case []record.RefExemplar: - buf = enc.Exemplars(v, buf) - case []record.RefHistogramSample: - buf, _ = enc.HistogramSamples(v, buf) - case []record.RefFloatHistogramSample: - buf, _ = enc.FloatHistogramSamples(v, buf) - case []record.RefMmapMarker: - buf = enc.MmapMarkers(v, buf) - case []record.RefMetadata: - buf = enc.Metadata(v, buf) - default: - continue - } - require.NoError(t, w.Log(buf)) - } - return buf -} - -func readTestWAL(t testing.TB, dir string) (recs []any) { - sr, err := wlog.NewSegmentsReader(dir) - require.NoError(t, err) - defer func() { - require.NoError(t, sr.Close()) - }() - - dec := record.NewDecoder(labels.NewSymbolTable(), promslog.NewNopLogger()) - r := wlog.NewReader(sr) - - for r.Next() { - rec := r.Record() - - switch dec.Type(rec) { - case record.Series: - series, err := dec.Series(rec, nil) - require.NoError(t, err) - recs = append(recs, series) - case record.Samples: - samples, err := dec.Samples(rec, nil) - require.NoError(t, err) - recs = append(recs, samples) - case record.HistogramSamples, record.CustomBucketsHistogramSamples: - samples, err := dec.HistogramSamples(rec, nil) - require.NoError(t, err) - recs = append(recs, samples) - case record.FloatHistogramSamples, record.CustomBucketsFloatHistogramSamples: - samples, err := dec.FloatHistogramSamples(rec, nil) - require.NoError(t, err) - recs = append(recs, samples) - case record.Tombstones: - tstones, err := dec.Tombstones(rec, nil) - require.NoError(t, err) - recs = append(recs, tstones) - case record.Metadata: - meta, err := dec.Metadata(rec, nil) - require.NoError(t, err) - recs = append(recs, meta) - case record.Exemplars: - exemplars, err := dec.Exemplars(rec, nil) - require.NoError(t, err) - recs = append(recs, exemplars) - default: - require.Fail(t, "unknown record type") - } - } - require.NoError(t, r.Err()) - return recs -} - -func BenchmarkLoadWLs(b *testing.B) { - cases := []struct { - // Total series is (batches*seriesPerBatch). - batches int - seriesPerBatch int - samplesPerSeries int - mmappedChunkT int64 - // The first oooSeriesPct*seriesPerBatch series in a batch are selected as "OOO" series. - oooSeriesPct float64 - // The first oooSamplesPct*samplesPerSeries samples in an OOO series are written as OOO samples. - oooSamplesPct float64 - oooCapMax int64 - }{ - { // Less series and more samples. 2 hour WAL with 1 second scrape interval. - batches: 10, - seriesPerBatch: 100, - samplesPerSeries: 7200, - }, - { // More series and less samples. - batches: 10, - seriesPerBatch: 10000, - samplesPerSeries: 50, - }, - { // In between. - batches: 10, - seriesPerBatch: 1000, - samplesPerSeries: 480, - }, - { // 2 hour WAL with 15 second scrape interval, and mmapped chunks up to last 100 samples. - batches: 100, - seriesPerBatch: 1000, - samplesPerSeries: 480, - mmappedChunkT: 3800, - }, - { // A lot of OOO samples (50% series with 50% of samples being OOO). - batches: 10, - seriesPerBatch: 1000, - samplesPerSeries: 480, - oooSeriesPct: 0.5, - oooSamplesPct: 0.5, - oooCapMax: DefaultOutOfOrderCapMax, - }, - { // Fewer OOO samples (10% of series with 10% of samples being OOO). - batches: 10, - seriesPerBatch: 1000, - samplesPerSeries: 480, - oooSeriesPct: 0.1, - oooSamplesPct: 0.1, - }, - { // 2 hour WAL with 15 second scrape interval, and mmapped chunks up to last 100 samples. - // Four mmap markers per OOO series: 480 * 0.3 = 144, 144 / 32 (DefaultOutOfOrderCapMax) = 4. - batches: 100, - seriesPerBatch: 1000, - samplesPerSeries: 480, - mmappedChunkT: 3800, - oooSeriesPct: 0.2, - oooSamplesPct: 0.3, - oooCapMax: DefaultOutOfOrderCapMax, - }, - } - - labelsPerSeries := 5 - // Rough estimates of most common % of samples that have an exemplar for each scrape. - exemplarsPercentages := []float64{0, 0.5, 1, 5} - lastExemplarsPerSeries := -1 - for _, c := range cases { - missingSeriesPercentages := []float64{0, 0.1} - for _, missingSeriesPct := range missingSeriesPercentages { - for _, p := range exemplarsPercentages { - exemplarsPerSeries := int(math.RoundToEven(float64(c.samplesPerSeries) * p / 100)) - // For tests with low samplesPerSeries we could end up testing with 0 exemplarsPerSeries - // multiple times without this check. - if exemplarsPerSeries == lastExemplarsPerSeries { - continue - } - lastExemplarsPerSeries = exemplarsPerSeries - b.Run(fmt.Sprintf("batches=%d,seriesPerBatch=%d,samplesPerSeries=%d,exemplarsPerSeries=%d,mmappedChunkT=%d,oooSeriesPct=%.3f,oooSamplesPct=%.3f,oooCapMax=%d,missingSeriesPct=%.3f", c.batches, c.seriesPerBatch, c.samplesPerSeries, exemplarsPerSeries, c.mmappedChunkT, c.oooSeriesPct, c.oooSamplesPct, c.oooCapMax, missingSeriesPct), - func(b *testing.B) { - dir := b.TempDir() - - wal, err := wlog.New(nil, nil, dir, compression.None) - require.NoError(b, err) - var wbl *wlog.WL - if c.oooSeriesPct != 0 { - wbl, err = wlog.New(nil, nil, dir, compression.None) - require.NoError(b, err) - } - - // Write series. - refSeries := make([]record.RefSeries, 0, c.seriesPerBatch) - var buf []byte - builder := labels.NewBuilder(labels.EmptyLabels()) - for j := 1; j < labelsPerSeries; j++ { - builder.Set(defaultLabelName+strconv.Itoa(j), defaultLabelValue+strconv.Itoa(j)) - } - for k := 0; k < c.batches; k++ { - refSeries = refSeries[:0] - for i := k * c.seriesPerBatch; i < (k+1)*c.seriesPerBatch; i++ { - builder.Set(defaultLabelName, strconv.Itoa(i)) - refSeries = append(refSeries, record.RefSeries{Ref: chunks.HeadSeriesRef(i) * 101, Labels: builder.Labels()}) - } - - writeSeries := refSeries - if missingSeriesPct > 0 { - newWriteSeries := make([]record.RefSeries, 0, int(float64(len(refSeries))*(1.0-missingSeriesPct))) - keepRatio := 1.0 - missingSeriesPct - // Keep approximately every 1/keepRatio series. - for i, s := range refSeries { - if int(float64(i)*keepRatio) != int(float64(i+1)*keepRatio) { - newWriteSeries = append(newWriteSeries, s) - } - } - writeSeries = newWriteSeries - } - - buf = populateTestWL(b, wal, []any{writeSeries}, buf) - } - - // Write samples. - refSamples := make([]record.RefSample, 0, c.seriesPerBatch) - - oooSeriesPerBatch := int(float64(c.seriesPerBatch) * c.oooSeriesPct) - oooSamplesPerSeries := int(float64(c.samplesPerSeries) * c.oooSamplesPct) - - for i := 0; i < c.samplesPerSeries; i++ { - for j := 0; j < c.batches; j++ { - refSamples = refSamples[:0] - - k := j * c.seriesPerBatch - // Skip appending the first oooSamplesPerSeries samples for the series in the batch that - // should have OOO samples. OOO samples are appended after all the in-order samples. - if i < oooSamplesPerSeries { - k += oooSeriesPerBatch - } - for ; k < (j+1)*c.seriesPerBatch; k++ { - refSamples = append(refSamples, record.RefSample{ - Ref: chunks.HeadSeriesRef(k) * 101, - T: int64(i) * 10, - V: float64(i) * 100, - }) - } - buf = populateTestWL(b, wal, []any{refSamples}, buf) - } - } - - // Write mmapped chunks. - if c.mmappedChunkT != 0 { - chunkDiskMapper, err := chunks.NewChunkDiskMapper(nil, mmappedChunksDir(dir), chunkenc.NewPool(), chunks.DefaultWriteBufferSize, chunks.DefaultWriteQueueSize) - require.NoError(b, err) - cOpts := chunkOpts{ - chunkDiskMapper: chunkDiskMapper, - chunkRange: c.mmappedChunkT, - samplesPerChunk: DefaultSamplesPerChunk, - } - for k := 0; k < c.batches*c.seriesPerBatch; k++ { - // Create one mmapped chunk per series, with one sample at the given time. - s := newMemSeries(labels.Labels{}, chunks.HeadSeriesRef(k)*101, 0, defaultIsolationDisabled, false) - s.append(c.mmappedChunkT, 42, 0, cOpts) - // There's only one head chunk because only a single sample is appended. mmapChunks() - // ignores the latest chunk, so we need to cut a new head chunk to guarantee the chunk with - // the sample at c.mmappedChunkT is mmapped. - s.cutNewHeadChunk(c.mmappedChunkT, chunkenc.EncXOR, c.mmappedChunkT) - s.mmapChunks(chunkDiskMapper) - } - require.NoError(b, chunkDiskMapper.Close()) - } - - // Write exemplars. - refExemplars := make([]record.RefExemplar, 0, c.seriesPerBatch) - for i := range exemplarsPerSeries { - for j := 0; j < c.batches; j++ { - refExemplars = refExemplars[:0] - for k := j * c.seriesPerBatch; k < (j+1)*c.seriesPerBatch; k++ { - refExemplars = append(refExemplars, record.RefExemplar{ - Ref: chunks.HeadSeriesRef(k) * 101, - T: int64(i) * 10, - V: float64(i) * 100, - Labels: labels.FromStrings("trace_id", fmt.Sprintf("trace-%d", i)), - }) - } - buf = populateTestWL(b, wal, []any{refExemplars}, buf) - } - } - - // Write OOO samples and mmap markers. - refMarkers := make([]record.RefMmapMarker, 0, oooSeriesPerBatch) - refSamples = make([]record.RefSample, 0, oooSeriesPerBatch) - for i := range oooSamplesPerSeries { - shouldAddMarkers := c.oooCapMax != 0 && i != 0 && int64(i)%c.oooCapMax == 0 - - for j := 0; j < c.batches; j++ { - refSamples = refSamples[:0] - if shouldAddMarkers { - refMarkers = refMarkers[:0] - } - for k := j * c.seriesPerBatch; k < (j*c.seriesPerBatch)+oooSeriesPerBatch; k++ { - ref := chunks.HeadSeriesRef(k) * 101 - if shouldAddMarkers { - // loadWBL() checks that the marker's MmapRef is less than or equal to the ref - // for the last mmap chunk. Setting MmapRef to 0 to always pass that check. - refMarkers = append(refMarkers, record.RefMmapMarker{Ref: ref, MmapRef: 0}) - } - refSamples = append(refSamples, record.RefSample{ - Ref: ref, - T: int64(i) * 10, - V: float64(i) * 100, - }) - } - if shouldAddMarkers { - populateTestWL(b, wbl, []any{refMarkers}, buf) - } - buf = populateTestWL(b, wal, []any{refSamples}, buf) - buf = populateTestWL(b, wbl, []any{refSamples}, buf) - } - } - - b.ResetTimer() - - // Load the WAL. - for b.Loop() { - opts := DefaultHeadOptions() - opts.ChunkRange = 1000 - opts.ChunkDirRoot = dir - if c.oooCapMax > 0 { - opts.OutOfOrderCapMax.Store(c.oooCapMax) - } - h, err := NewHead(nil, nil, wal, wbl, opts, nil) - require.NoError(b, err) - h.Init(0) - } - b.StopTimer() - wal.Close() - if wbl != nil { - wbl.Close() - } - }) - } - } - } -} - -// BenchmarkLoadRealWLs will be skipped unless the BENCHMARK_LOAD_REAL_WLS_DIR environment variable is set. -// BENCHMARK_LOAD_REAL_WLS_DIR should be the folder where `wal` and `chunks_head` are located. -// -// Using an absolute path for BENCHMARK_LOAD_REAL_WLS_DIR is recommended. -// -// Because WLs loading may alter BENCHMARK_LOAD_REAL_WLS_DIR which can affect benchmark results and to ensure consistency, -// a copy of BENCHMARK_LOAD_REAL_WLS_DIR is made for each iteration and deleted at the end. -// Make sure there is sufficient disk space for that. -func BenchmarkLoadRealWLs(b *testing.B) { - srcDir := os.Getenv("BENCHMARK_LOAD_REAL_WLS_DIR") - if srcDir == "" { - b.SkipNow() - } - - // Load the WAL. - for b.Loop() { - b.StopTimer() - dir := b.TempDir() - require.NoError(b, fileutil.CopyDirs(srcDir, dir)) - - wal, err := wlog.New(nil, nil, filepath.Join(dir, "wal"), compression.None) - require.NoError(b, err) - b.Cleanup(func() { wal.Close() }) - - wbl, err := wlog.New(nil, nil, filepath.Join(dir, "wbl"), compression.None) - require.NoError(b, err) - b.Cleanup(func() { wbl.Close() }) - b.StartTimer() - - opts := DefaultHeadOptions() - opts.ChunkDirRoot = dir - h, err := NewHead(nil, nil, wal, wbl, opts, nil) - require.NoError(b, err) - require.NoError(b, h.Init(0)) - - b.StopTimer() - require.NoError(b, os.RemoveAll(dir)) - } -} - -// TestHead_HighConcurrencyReadAndWrite generates 1000 series with a step of 15s and fills a whole block with samples, +// TestHeadAppenderV2_HighConcurrencyReadAndWrite generates 1000 series with a step of 15s and fills a whole block with samples, // this means in total it generates 4000 chunks because with a step of 15s there are 4 chunks per block per series. // While appending the samples to the head it concurrently queries them from multiple go routines and verifies that the // returned results are correct. -func TestHead_HighConcurrencyReadAndWrite(t *testing.T) { +func TestHeadAppenderV2_HighConcurrencyReadAndWrite(t *testing.T) { head, _ := newTestHead(t, DefaultBlockDuration, compression.None, false) defer func() { require.NoError(t, head.Close()) @@ -579,10 +124,10 @@ func TestHead_HighConcurrencyReadAndWrite(t *testing.T) { return false, nil } - app := head.Appender(ctx) + app := head.AppenderV2(ctx) for i := range workerLabelSets { // We also use the timestamp as the sample value. - _, err := app.Append(0, workerLabelSets[i], int64(ts), float64(ts)) + _, err := app.Append(0, workerLabelSets[i], 0, int64(ts), float64(ts), nil, nil, storage.AOptions{}) if err != nil { return false, fmt.Errorf("Error when appending to head: %w", err) } @@ -704,160 +249,35 @@ func TestHead_HighConcurrencyReadAndWrite(t *testing.T) { require.NoError(t, g.Wait()) } -func TestHead_ReadWAL(t *testing.T) { - for _, compress := range []compression.Type{compression.None, compression.Snappy, compression.Zstd} { - t.Run(fmt.Sprintf("compress=%s", compress), func(t *testing.T) { - entries := []any{ - []record.RefSeries{ - {Ref: 10, Labels: labels.FromStrings("a", "1")}, - {Ref: 11, Labels: labels.FromStrings("a", "2")}, - {Ref: 100, Labels: labels.FromStrings("a", "3")}, - }, - []record.RefSample{ - {Ref: 0, T: 99, V: 1}, - {Ref: 10, T: 100, V: 2}, - {Ref: 100, T: 100, V: 3}, - }, - []record.RefSeries{ - {Ref: 50, Labels: labels.FromStrings("a", "4")}, - // This series has two refs pointing to it. - {Ref: 101, Labels: labels.FromStrings("a", "3")}, - }, - []record.RefSample{ - {Ref: 10, T: 101, V: 5}, - {Ref: 50, T: 101, V: 6}, - // Sample for duplicate series record. - {Ref: 101, T: 101, V: 7}, - }, - []tombstones.Stone{ - {Ref: 0, Intervals: []tombstones.Interval{{Mint: 99, Maxt: 101}}}, - // Tombstone for duplicate series record. - {Ref: 101, Intervals: []tombstones.Interval{{Mint: 0, Maxt: 100}}}, - }, - []record.RefExemplar{ - {Ref: 10, T: 100, V: 1, Labels: labels.FromStrings("trace_id", "asdf")}, - // Exemplar for duplicate series record. - {Ref: 101, T: 101, V: 7, Labels: labels.FromStrings("trace_id", "zxcv")}, - }, - []record.RefMetadata{ - // Metadata for duplicate series record. - {Ref: 101, Type: uint8(record.Counter), Unit: "foo", Help: "total foo"}, - }, - } - - head, w := newTestHead(t, 1000, compress, false) - defer func() { - require.NoError(t, head.Close()) - }() - - populateTestWL(t, w, entries, nil) - - require.NoError(t, head.Init(math.MinInt64)) - require.Equal(t, uint64(101), head.lastSeriesID.Load()) - - s10 := head.series.getByID(10) - s11 := head.series.getByID(11) - s50 := head.series.getByID(50) - s100 := head.series.getByID(100) - s101 := head.series.getByID(101) - - testutil.RequireEqual(t, labels.FromStrings("a", "1"), s10.lset) - require.Nil(t, s11) // Series without samples should be garbage collected at head.Init(). - testutil.RequireEqual(t, labels.FromStrings("a", "4"), s50.lset) - testutil.RequireEqual(t, labels.FromStrings("a", "3"), s100.lset) - - // Duplicate series record should not be written to the head. - require.Nil(t, s101) - // But it should have a WAL expiry set. - keepUntil, ok := head.getWALExpiry(101) - require.True(t, ok) - require.Equal(t, int64(101), keepUntil) - // Only the duplicate series record should have a WAL expiry set. - _, ok = head.getWALExpiry(50) - require.False(t, ok) - - expandChunk := func(c chunkenc.Iterator) (x []sample) { - for c.Next() == chunkenc.ValFloat { - t, v := c.At() - x = append(x, sample{t: t, f: v}) - } - require.NoError(t, c.Err()) - return x - } - - // Verify samples and exemplar for series 10. - c, _, _, err := s10.chunk(0, head.chunkDiskMapper, &head.memChunkPool) - require.NoError(t, err) - require.Equal(t, []sample{{100, 2, nil, nil}, {101, 5, nil, nil}}, expandChunk(c.chunk.Iterator(nil))) - - q, err := head.ExemplarQuerier(context.Background()) - require.NoError(t, err) - e, err := q.Select(0, 1000, []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "1")}) - require.NoError(t, err) - require.NotEmpty(t, e) - require.NotEmpty(t, e[0].Exemplars) - require.True(t, exemplar.Exemplar{Ts: 100, Value: 1, Labels: labels.FromStrings("trace_id", "asdf")}.Equals(e[0].Exemplars[0])) - - // Verify samples for series 50 - c, _, _, err = s50.chunk(0, head.chunkDiskMapper, &head.memChunkPool) - require.NoError(t, err) - require.Equal(t, []sample{{101, 6, nil, nil}}, expandChunk(c.chunk.Iterator(nil))) - - // Verify records for series 100 and its duplicate, series 101. - // The samples before the new series record should be discarded since a duplicate record - // is only possible when old samples were compacted. - c, _, _, err = s100.chunk(0, head.chunkDiskMapper, &head.memChunkPool) - require.NoError(t, err) - require.Equal(t, []sample{{101, 7, nil, nil}}, expandChunk(c.chunk.Iterator(nil))) - - q, err = head.ExemplarQuerier(context.Background()) - require.NoError(t, err) - e, err = q.Select(0, 1000, []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "3")}) - require.NoError(t, err) - require.NotEmpty(t, e) - require.NotEmpty(t, e[0].Exemplars) - require.True(t, exemplar.Exemplar{Ts: 101, Value: 7, Labels: labels.FromStrings("trace_id", "zxcv")}.Equals(e[0].Exemplars[0])) - - require.NotNil(t, s100.meta) - require.Equal(t, "foo", s100.meta.Unit) - require.Equal(t, "total foo", s100.meta.Help) - - intervals, err := head.tombstones.Get(storage.SeriesRef(s100.ref)) - require.NoError(t, err) - require.Equal(t, tombstones.Intervals{{Mint: 0, Maxt: 100}}, intervals) - }) - } -} - -func TestHead_WALMultiRef(t *testing.T) { +func TestHeadAppenderV2_WALMultiRef(t *testing.T) { head, w := newTestHead(t, 1000, compression.None, false) require.NoError(t, head.Init(0)) - app := head.Appender(context.Background()) - ref1, err := app.Append(0, labels.FromStrings("foo", "bar"), 100, 1) + app := head.AppenderV2(context.Background()) + ref1, err := app.Append(0, labels.FromStrings("foo", "bar"), 0, 100, 1, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) require.Equal(t, 1.0, prom_testutil.ToFloat64(head.metrics.chunksCreated)) // Add another sample outside chunk range to mmap a chunk. - app = head.Appender(context.Background()) - _, err = app.Append(0, labels.FromStrings("foo", "bar"), 1500, 2) + app = head.AppenderV2(context.Background()) + _, err = app.Append(0, labels.FromStrings("foo", "bar"), 0, 1500, 2, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) require.Equal(t, 2.0, prom_testutil.ToFloat64(head.metrics.chunksCreated)) require.NoError(t, head.Truncate(1600)) - app = head.Appender(context.Background()) - ref2, err := app.Append(0, labels.FromStrings("foo", "bar"), 1700, 3) + app = head.AppenderV2(context.Background()) + ref2, err := app.Append(0, labels.FromStrings("foo", "bar"), 0, 1700, 3, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) require.Equal(t, 3.0, prom_testutil.ToFloat64(head.metrics.chunksCreated)) // Add another sample outside chunk range to mmap a chunk. - app = head.Appender(context.Background()) - _, err = app.Append(0, labels.FromStrings("foo", "bar"), 2000, 4) + app = head.AppenderV2(context.Background()) + _, err = app.Append(0, labels.FromStrings("foo", "bar"), 0, 2000, 4, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) require.Equal(t, 4.0, prom_testutil.ToFloat64(head.metrics.chunksCreated)) @@ -889,344 +309,40 @@ func TestHead_WALMultiRef(t *testing.T) { }}, series) } -func TestHead_WALCheckpointMultiRef(t *testing.T) { - cases := []struct { - name string - walEntries []any - expectedWalExpiry int64 - walTruncateMinT int64 - expectedWalEntries []any - }{ - { - name: "Samples only; keep needed duplicate series record", - walEntries: []any{ - []record.RefSeries{ - {Ref: 1, Labels: labels.FromStrings("a", "1")}, - {Ref: 2, Labels: labels.FromStrings("a", "1")}, - }, - []record.RefSample{ - {Ref: 1, T: 100, V: 1}, - {Ref: 2, T: 200, V: 2}, - {Ref: 2, T: 500, V: 3}, - }, - }, - expectedWalExpiry: 500, - walTruncateMinT: 500, - expectedWalEntries: []any{ - []record.RefSeries{ - {Ref: 1, Labels: labels.FromStrings("a", "1")}, - {Ref: 2, Labels: labels.FromStrings("a", "1")}, - }, - []record.RefSample{ - {Ref: 2, T: 500, V: 3}, - }, - }, - }, - { - name: "Tombstones only; keep needed duplicate series record", - walEntries: []any{ - []record.RefSeries{ - {Ref: 1, Labels: labels.FromStrings("a", "1")}, - {Ref: 2, Labels: labels.FromStrings("a", "1")}, - }, - []tombstones.Stone{ - {Ref: 1, Intervals: []tombstones.Interval{{Mint: 0, Maxt: 100}}}, - {Ref: 2, Intervals: []tombstones.Interval{{Mint: 0, Maxt: 200}}}, - {Ref: 2, Intervals: []tombstones.Interval{{Mint: 0, Maxt: 500}}}, - }, - }, - expectedWalExpiry: 500, - walTruncateMinT: 500, - expectedWalEntries: []any{ - []record.RefSeries{ - {Ref: 1, Labels: labels.FromStrings("a", "1")}, - {Ref: 2, Labels: labels.FromStrings("a", "1")}, - }, - []tombstones.Stone{ - {Ref: 2, Intervals: []tombstones.Interval{{Mint: 0, Maxt: 500}}}, - }, - }, - }, - { - name: "Exemplars only; keep needed duplicate series record", - walEntries: []any{ - []record.RefSeries{ - {Ref: 1, Labels: labels.FromStrings("a", "1")}, - {Ref: 2, Labels: labels.FromStrings("a", "1")}, - }, - []record.RefExemplar{ - {Ref: 1, T: 100, V: 1, Labels: labels.FromStrings("trace_id", "asdf")}, - {Ref: 2, T: 200, V: 2, Labels: labels.FromStrings("trace_id", "asdf")}, - {Ref: 2, T: 500, V: 3, Labels: labels.FromStrings("trace_id", "asdf")}, - }, - }, - expectedWalExpiry: 500, - walTruncateMinT: 500, - expectedWalEntries: []any{ - []record.RefSeries{ - {Ref: 1, Labels: labels.FromStrings("a", "1")}, - {Ref: 2, Labels: labels.FromStrings("a", "1")}, - }, - []record.RefExemplar{ - {Ref: 2, T: 500, V: 3, Labels: labels.FromStrings("trace_id", "asdf")}, - }, - }, - }, - { - name: "Histograms only; keep needed duplicate series record", - walEntries: []any{ - []record.RefSeries{ - {Ref: 1, Labels: labels.FromStrings("a", "1")}, - {Ref: 2, Labels: labels.FromStrings("a", "1")}, - }, - []record.RefHistogramSample{ - {Ref: 1, T: 100, H: &histogram.Histogram{}}, - {Ref: 2, T: 200, H: &histogram.Histogram{}}, - {Ref: 2, T: 500, H: &histogram.Histogram{}}, - }, - }, - expectedWalExpiry: 500, - walTruncateMinT: 500, - expectedWalEntries: []any{ - []record.RefSeries{ - {Ref: 1, Labels: labels.FromStrings("a", "1")}, - {Ref: 2, Labels: labels.FromStrings("a", "1")}, - }, - []record.RefHistogramSample{ - {Ref: 2, T: 500, H: &histogram.Histogram{}}, - }, - }, - }, - { - name: "Float histograms only; keep needed duplicate series record", - walEntries: []any{ - []record.RefSeries{ - {Ref: 1, Labels: labels.FromStrings("a", "1")}, - {Ref: 2, Labels: labels.FromStrings("a", "1")}, - }, - []record.RefFloatHistogramSample{ - {Ref: 1, T: 100, FH: &histogram.FloatHistogram{}}, - {Ref: 2, T: 200, FH: &histogram.FloatHistogram{}}, - {Ref: 2, T: 500, FH: &histogram.FloatHistogram{}}, - }, - }, - expectedWalExpiry: 500, - walTruncateMinT: 500, - expectedWalEntries: []any{ - []record.RefSeries{ - {Ref: 1, Labels: labels.FromStrings("a", "1")}, - {Ref: 2, Labels: labels.FromStrings("a", "1")}, - }, - []record.RefFloatHistogramSample{ - {Ref: 2, T: 500, FH: &histogram.FloatHistogram{}}, - }, - }, - }, - { - name: "All record types; keep needed duplicate series record until last record", - // Series with 2 refs and samples for both - walEntries: []any{ - []record.RefSeries{ - {Ref: 1, Labels: labels.FromStrings("a", "1")}, - {Ref: 2, Labels: labels.FromStrings("a", "1")}, - }, - []record.RefSample{ - {Ref: 2, T: 500, V: 3}, - }, - []tombstones.Stone{ - {Ref: 2, Intervals: []tombstones.Interval{{Mint: 0, Maxt: 500}}}, - }, - []record.RefExemplar{ - {Ref: 2, T: 800, V: 2, Labels: labels.FromStrings("trace_id", "asdf")}, - }, - []record.RefHistogramSample{ - {Ref: 2, T: 500, H: &histogram.Histogram{}}, - }, - []record.RefFloatHistogramSample{ - {Ref: 2, T: 500, FH: &histogram.FloatHistogram{}}, - }, - }, - expectedWalExpiry: 800, - walTruncateMinT: 700, - expectedWalEntries: []any{ - []record.RefSeries{ - {Ref: 1, Labels: labels.FromStrings("a", "1")}, - {Ref: 2, Labels: labels.FromStrings("a", "1")}, - }, - []record.RefExemplar{ - {Ref: 2, T: 800, V: 2, Labels: labels.FromStrings("trace_id", "asdf")}, - }, - }, - }, - { - name: "All record types; drop expired duplicate series record", - // Series with 2 refs and samples for both - walEntries: []any{ - []record.RefSeries{ - {Ref: 1, Labels: labels.FromStrings("a", "1")}, - {Ref: 2, Labels: labels.FromStrings("a", "1")}, - }, - []record.RefSample{ - {Ref: 2, T: 500, V: 2}, - {Ref: 1, T: 900, V: 3}, - }, - []tombstones.Stone{ - {Ref: 2, Intervals: []tombstones.Interval{{Mint: 0, Maxt: 750}}}, - }, - []record.RefExemplar{ - {Ref: 2, T: 800, V: 2, Labels: labels.FromStrings("trace_id", "asdf")}, - }, - []record.RefHistogramSample{ - {Ref: 2, T: 600, H: &histogram.Histogram{}}, - }, - []record.RefFloatHistogramSample{ - {Ref: 2, T: 700, FH: &histogram.FloatHistogram{}}, - }, - }, - expectedWalExpiry: 800, - walTruncateMinT: 900, - expectedWalEntries: []any{ - []record.RefSeries{ - {Ref: 1, Labels: labels.FromStrings("a", "1")}, - }, - []record.RefSample{ - {Ref: 1, T: 900, V: 3}, - }, - }, - }, - } - - for _, tc := range cases { - t.Run(tc.name, func(t *testing.T) { - h, w := newTestHead(t, 1000, compression.None, false) - t.Cleanup(func() { - require.NoError(t, h.Close()) - }) - - populateTestWL(t, w, tc.walEntries, nil) - first, _, err := wlog.Segments(w.Dir()) - require.NoError(t, err) - - require.NoError(t, h.Init(0)) - - keepUntil, ok := h.getWALExpiry(2) - require.True(t, ok) - require.Equal(t, tc.expectedWalExpiry, keepUntil) - - // Each truncation creates a new segment, so attempt truncations until a checkpoint is created - for { - h.lastWALTruncationTime.Store(0) // Reset so that it's always time to truncate the WAL - err := h.truncateWAL(tc.walTruncateMinT) - require.NoError(t, err) - f, _, err := wlog.Segments(w.Dir()) - require.NoError(t, err) - if f > first { - break - } - } - - // Read test WAL , checkpoint first - checkpointDir, _, err := wlog.LastCheckpoint(w.Dir()) - require.NoError(t, err) - cprecs := readTestWAL(t, checkpointDir) - recs := readTestWAL(t, w.Dir()) - recs = append(cprecs, recs...) - - // Use testutil.RequireEqual which handles labels properly with dedupelabels - testutil.RequireEqual(t, tc.expectedWalEntries, recs) - }) - } -} - -func TestHead_KeepSeriesInWALCheckpoint(t *testing.T) { - existingRef := 1 - existingLbls := labels.FromStrings("foo", "bar") - keepUntil := int64(10) - - cases := []struct { - name string - prepare func(t *testing.T, h *Head) - mint int64 - expected bool - }{ - { - name: "keep series still in the head", - prepare: func(t *testing.T, h *Head) { - _, _, err := h.getOrCreateWithOptionalID(chunks.HeadSeriesRef(existingRef), existingLbls.Hash(), existingLbls, false) - require.NoError(t, err) - }, - expected: true, - }, - { - name: "keep series with keepUntil > mint", - mint: keepUntil - 1, - expected: true, - }, - { - name: "keep series with keepUntil = mint", - mint: keepUntil, - expected: true, - }, - { - name: "drop series with keepUntil < mint", - mint: keepUntil + 1, - expected: false, - }, - } - - for _, tc := range cases { - t.Run(tc.name, func(t *testing.T) { - h, _ := newTestHead(t, 1000, compression.None, false) - t.Cleanup(func() { - require.NoError(t, h.Close()) - }) - - if tc.prepare != nil { - tc.prepare(t, h) - } else { - h.updateWALExpiry(chunks.HeadSeriesRef(existingRef), keepUntil) - } - - keep := h.keepSeriesInWALCheckpointFn(tc.mint) - require.Equal(t, tc.expected, keep(chunks.HeadSeriesRef(existingRef))) - }) - } -} - -func TestHead_ActiveAppenders(t *testing.T) { +func TestHeadAppenderV2_ActiveAppenders(t *testing.T) { head, _ := newTestHead(t, 1000, compression.None, false) defer head.Close() require.NoError(t, head.Init(0)) // First rollback with no samples. - app := head.Appender(context.Background()) + app := head.AppenderV2(context.Background()) require.Equal(t, 1.0, prom_testutil.ToFloat64(head.metrics.activeAppenders)) require.NoError(t, app.Rollback()) require.Equal(t, 0.0, prom_testutil.ToFloat64(head.metrics.activeAppenders)) // Then commit with no samples. - app = head.Appender(context.Background()) + app = head.AppenderV2(context.Background()) require.NoError(t, app.Commit()) require.Equal(t, 0.0, prom_testutil.ToFloat64(head.metrics.activeAppenders)) // Now rollback with one sample. - app = head.Appender(context.Background()) - _, err := app.Append(0, labels.FromStrings("foo", "bar"), 100, 1) + app = head.AppenderV2(context.Background()) + _, err := app.Append(0, labels.FromStrings("foo", "bar"), 0, 100, 1, nil, nil, storage.AOptions{}) require.NoError(t, err) require.Equal(t, 1.0, prom_testutil.ToFloat64(head.metrics.activeAppenders)) require.NoError(t, app.Rollback()) require.Equal(t, 0.0, prom_testutil.ToFloat64(head.metrics.activeAppenders)) // Now commit with one sample. - app = head.Appender(context.Background()) - _, err = app.Append(0, labels.FromStrings("foo", "bar"), 100, 1) + app = head.AppenderV2(context.Background()) + _, err = app.Append(0, labels.FromStrings("foo", "bar"), 0, 100, 1, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) require.Equal(t, 0.0, prom_testutil.ToFloat64(head.metrics.activeAppenders)) } -func TestHead_RaceBetweenSeriesCreationAndGC(t *testing.T) { +func TestHeadAppenderV2_RaceBetweenSeriesCreationAndGC(t *testing.T) { head, _ := newTestHead(t, 1000, compression.None, false) t.Cleanup(func() { _ = head.Close() }) require.NoError(t, head.Init(0)) @@ -1240,14 +356,14 @@ func TestHead_RaceBetweenSeriesCreationAndGC(t *testing.T) { go func() { defer done.Store(true) - app := head.Appender(context.Background()) + app := head.AppenderV2(context.Background()) defer func() { if err := app.Commit(); err != nil { t.Errorf("Failed to commit: %v", err) } }() for i := range totalSeries { - _, err := app.Append(0, series[i], 100, 1) + _, err := app.Append(0, series[i], 0, 100, 1, nil, nil, storage.AOptions{}) if err != nil { t.Errorf("Failed to append: %v", err) return @@ -1263,10 +379,10 @@ func TestHead_RaceBetweenSeriesCreationAndGC(t *testing.T) { require.Equal(t, totalSeries, int(head.NumSeries())) } -func TestHead_CanGarbagecollectSeriesCreatedWithoutSamples(t *testing.T) { - for op, finishTxn := range map[string]func(app storage.Appender) error{ - "after commit": func(app storage.Appender) error { return app.Commit() }, - "after rollback": func(app storage.Appender) error { return app.Rollback() }, +func TestHeadAppenderV2_CanGCSeriesCreatedWithoutSamples(t *testing.T) { + for op, finishTxn := range map[string]func(app storage.AppenderTransaction) error{ + "after commit": func(app storage.AppenderTransaction) error { return app.Commit() }, + "after rollback": func(app storage.AppenderTransaction) error { return app.Rollback() }, } { t.Run(op, func(t *testing.T) { chunkRange := time.Hour.Milliseconds() @@ -1278,8 +394,8 @@ func TestHead_CanGarbagecollectSeriesCreatedWithoutSamples(t *testing.T) { firstSampleTime := 10 * chunkRange { // Append first sample, it should init head max time to firstSampleTime. - app := head.Appender(context.Background()) - _, err := app.Append(0, labels.FromStrings("lbl", "ok"), firstSampleTime, 1) + app := head.AppenderV2(context.Background()) + _, err := app.Append(0, labels.FromStrings("lbl", "ok"), 0, firstSampleTime, 1, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) require.Equal(t, 1, int(head.NumSeries())) @@ -1287,9 +403,9 @@ func TestHead_CanGarbagecollectSeriesCreatedWithoutSamples(t *testing.T) { // Append a sample in a time range that is not covered by the chunk range, // We would create series first and then append no sample. - app := head.Appender(context.Background()) + app := head.AppenderV2(context.Background()) invalidSampleTime := firstSampleTime - chunkRange - _, err := app.Append(0, labels.FromStrings("foo", "bar"), invalidSampleTime, 2) + _, err := app.Append(0, labels.FromStrings("foo", "bar"), 0, invalidSampleTime, 2, nil, nil, storage.AOptions{}) require.Error(t, err) // These are our assumptions: we're not testing them, we're just checking them to make debugging a failed // test easier if someone refactors the code and breaks these assumptions. @@ -1306,428 +422,7 @@ func TestHead_CanGarbagecollectSeriesCreatedWithoutSamples(t *testing.T) { } } -func TestHead_UnknownWALRecord(t *testing.T) { - head, w := newTestHead(t, 1000, compression.None, false) - w.Log([]byte{255, 42}) - require.NoError(t, head.Init(0)) - require.NoError(t, head.Close()) -} - -// BenchmarkHead_Truncate is quite heavy, so consider running it with -// -benchtime=10x or similar to get more stable and comparable results. -func BenchmarkHead_Truncate(b *testing.B) { - const total = 1e6 - - prepare := func(b *testing.B, churn int) *Head { - h, _ := newTestHead(b, 1000, compression.None, false) - b.Cleanup(func() { - require.NoError(b, h.Close()) - }) - - h.initTime(0) - - internedItoa := map[int]string{} - var mtx sync.RWMutex - itoa := func(i int) string { - mtx.RLock() - s, ok := internedItoa[i] - mtx.RUnlock() - if ok { - return s - } - mtx.Lock() - s = strconv.Itoa(i) - internedItoa[i] = s - mtx.Unlock() - return s - } - - allSeries := [total]labels.Labels{} - nameValues := make([]string, 0, 100) - for i := range int(total) { - nameValues = nameValues[:0] - - // A thousand labels like lbl_x_of_1000, each with total/1000 values - thousand := "lbl_" + itoa(i%1000) + "_of_1000" - nameValues = append(nameValues, thousand, itoa(i/1000)) - // A hundred labels like lbl_x_of_100, each with total/100 values. - hundred := "lbl_" + itoa(i%100) + "_of_100" - nameValues = append(nameValues, hundred, itoa(i/100)) - - if i%13 == 0 { - ten := "lbl_" + itoa(i%10) + "_of_10" - nameValues = append(nameValues, ten, itoa(i%10)) - } - - allSeries[i] = labels.FromStrings(append(nameValues, "first", "a", "second", "a", "third", "a")...) - s, _, _ := h.getOrCreate(allSeries[i].Hash(), allSeries[i], false) - s.mmappedChunks = []*mmappedChunk{ - {minTime: 1000 * int64(i/churn), maxTime: 999 + 1000*int64(i/churn)}, - } - } - - return h - } - - for _, churn := range []int{10, 100, 1000} { - b.Run(fmt.Sprintf("churn=%d", churn), func(b *testing.B) { - if b.N > total/churn { - // Just to make sure that benchmark still makes sense. - panic("benchmark not prepared") - } - h := prepare(b, churn) - b.ResetTimer() - - for i := 0; b.Loop(); i++ { - require.NoError(b, h.Truncate(1000*int64(i))) - // Make sure the benchmark is meaningful and it's actually truncating the expected amount of series. - require.Equal(b, total-churn*i, int(h.NumSeries())) - } - }) - } -} - -func TestHead_Truncate(t *testing.T) { - h, _ := newTestHead(t, 1000, compression.None, false) - defer func() { - require.NoError(t, h.Close()) - }() - - h.initTime(0) - - ctx := context.Background() - - s1, _, _ := h.getOrCreate(1, labels.FromStrings("a", "1", "b", "1"), false) - s2, _, _ := h.getOrCreate(2, labels.FromStrings("a", "2", "b", "1"), false) - s3, _, _ := h.getOrCreate(3, labels.FromStrings("a", "1", "b", "2"), false) - s4, _, _ := h.getOrCreate(4, labels.FromStrings("a", "2", "b", "2", "c", "1"), false) - - s1.mmappedChunks = []*mmappedChunk{ - {minTime: 0, maxTime: 999}, - {minTime: 1000, maxTime: 1999}, - {minTime: 2000, maxTime: 2999}, - } - s2.mmappedChunks = []*mmappedChunk{ - {minTime: 1000, maxTime: 1999}, - {minTime: 2000, maxTime: 2999}, - {minTime: 3000, maxTime: 3999}, - } - s3.mmappedChunks = []*mmappedChunk{ - {minTime: 0, maxTime: 999}, - {minTime: 1000, maxTime: 1999}, - } - s4.mmappedChunks = []*mmappedChunk{} - - // Truncation need not be aligned. - require.NoError(t, h.Truncate(1)) - - require.NoError(t, h.Truncate(2000)) - - require.Equal(t, []*mmappedChunk{ - {minTime: 2000, maxTime: 2999}, - }, h.series.getByID(s1.ref).mmappedChunks) - - require.Equal(t, []*mmappedChunk{ - {minTime: 2000, maxTime: 2999}, - {minTime: 3000, maxTime: 3999}, - }, h.series.getByID(s2.ref).mmappedChunks) - - require.Nil(t, h.series.getByID(s3.ref)) - require.Nil(t, h.series.getByID(s4.ref)) - - postingsA1, _ := index.ExpandPostings(h.postings.Postings(ctx, "a", "1")) - postingsA2, _ := index.ExpandPostings(h.postings.Postings(ctx, "a", "2")) - postingsB1, _ := index.ExpandPostings(h.postings.Postings(ctx, "b", "1")) - postingsB2, _ := index.ExpandPostings(h.postings.Postings(ctx, "b", "2")) - postingsC1, _ := index.ExpandPostings(h.postings.Postings(ctx, "c", "1")) - postingsAll, _ := index.ExpandPostings(h.postings.Postings(ctx, "", "")) - - require.Equal(t, []storage.SeriesRef{storage.SeriesRef(s1.ref)}, postingsA1) - require.Equal(t, []storage.SeriesRef{storage.SeriesRef(s2.ref)}, postingsA2) - require.Equal(t, []storage.SeriesRef{storage.SeriesRef(s1.ref), storage.SeriesRef(s2.ref)}, postingsB1) - require.Equal(t, []storage.SeriesRef{storage.SeriesRef(s1.ref), storage.SeriesRef(s2.ref)}, postingsAll) - require.Nil(t, postingsB2) - require.Nil(t, postingsC1) - - iter := h.postings.Symbols() - symbols := []string{} - for iter.Next() { - symbols = append(symbols, iter.At()) - } - require.Equal(t, - []string{"" /* from 'all' postings list */, "1", "2", "a", "b"}, - symbols) - - values := map[string]map[string]struct{}{} - for _, name := range h.postings.LabelNames() { - ss, ok := values[name] - if !ok { - ss = map[string]struct{}{} - values[name] = ss - } - for _, value := range h.postings.LabelValues(ctx, name, nil) { - ss[value] = struct{}{} - } - } - require.Equal(t, map[string]map[string]struct{}{ - "a": {"1": struct{}{}, "2": struct{}{}}, - "b": {"1": struct{}{}}, - }, values) -} - -// Validate various behaviors brought on by firstChunkID accounting for -// garbage collected chunks. -func TestMemSeries_truncateChunks(t *testing.T) { - dir := t.TempDir() - // This is usually taken from the Head, but passing manually here. - chunkDiskMapper, err := chunks.NewChunkDiskMapper(nil, dir, chunkenc.NewPool(), chunks.DefaultWriteBufferSize, chunks.DefaultWriteQueueSize) - require.NoError(t, err) - defer func() { - require.NoError(t, chunkDiskMapper.Close()) - }() - cOpts := chunkOpts{ - chunkDiskMapper: chunkDiskMapper, - chunkRange: 2000, - samplesPerChunk: DefaultSamplesPerChunk, - } - - memChunkPool := sync.Pool{ - New: func() any { - return &memChunk{} - }, - } - - s := newMemSeries(labels.FromStrings("a", "b"), 1, 0, defaultIsolationDisabled, false) - - for i := 0; i < 4000; i += 5 { - ok, _ := s.append(int64(i), float64(i), 0, cOpts) - require.True(t, ok, "sample append failed") - } - s.mmapChunks(chunkDiskMapper) - - // Check that truncate removes half of the chunks and afterwards - // that the ID of the last chunk still gives us the same chunk afterwards. - countBefore := len(s.mmappedChunks) + 1 // +1 for the head chunk. - lastID := s.headChunkID(countBefore - 1) - lastChunk, _, _, err := s.chunk(lastID, chunkDiskMapper, &memChunkPool) - require.NoError(t, err) - require.NotNil(t, lastChunk) - - chk, _, _, err := s.chunk(0, chunkDiskMapper, &memChunkPool) - require.NotNil(t, chk) - require.NoError(t, err) - - s.truncateChunksBefore(2000, 0) - - require.Equal(t, int64(2000), s.mmappedChunks[0].minTime) - _, _, _, err = s.chunk(0, chunkDiskMapper, &memChunkPool) - require.Equal(t, storage.ErrNotFound, err, "first chunks not gone") - require.Equal(t, countBefore/2, len(s.mmappedChunks)+1) // +1 for the head chunk. - chk, _, _, err = s.chunk(lastID, chunkDiskMapper, &memChunkPool) - require.NoError(t, err) - require.Equal(t, lastChunk, chk) -} - -func TestMemSeries_truncateChunks_scenarios(t *testing.T) { - const chunkRange = 100 - const chunkStep = 5 - - tests := []struct { - name string - headChunks int // the number of head chunks to create on memSeries by appending enough samples - mmappedChunks int // the number of mmapped chunks to create on memSeries by appending enough samples - truncateBefore int64 // the mint to pass to truncateChunksBefore() - expectedTruncated int // the number of chunks that we're expecting be truncated and returned by truncateChunksBefore() - expectedHead int // the expected number of head chunks after truncation - expectedMmap int // the expected number of mmapped chunks after truncation - expectedFirstChunkID chunks.HeadChunkID // the expected series.firstChunkID after truncation - }{ - { - name: "empty memSeries", - truncateBefore: chunkRange * 10, - }, - { - name: "single head chunk, not truncated", - headChunks: 1, - expectedHead: 1, - }, - { - name: "single head chunk, truncated", - headChunks: 1, - truncateBefore: chunkRange, - expectedTruncated: 1, - expectedHead: 0, - expectedFirstChunkID: 1, - }, - { - name: "2 head chunks, not truncated", - headChunks: 2, - expectedHead: 2, - }, - { - name: "2 head chunks, first truncated", - headChunks: 2, - truncateBefore: chunkRange, - expectedTruncated: 1, - expectedHead: 1, - expectedFirstChunkID: 1, - }, - { - name: "2 head chunks, everything truncated", - headChunks: 2, - truncateBefore: chunkRange * 2, - expectedTruncated: 2, - expectedHead: 0, - expectedFirstChunkID: 2, - }, - { - name: "no head chunks, 3 mmap chunks, second mmap truncated", - headChunks: 0, - mmappedChunks: 3, - truncateBefore: chunkRange * 2, - expectedTruncated: 2, - expectedHead: 0, - expectedMmap: 1, - expectedFirstChunkID: 2, - }, - { - name: "single head chunk, single mmap chunk, not truncated", - headChunks: 1, - mmappedChunks: 1, - expectedHead: 1, - expectedMmap: 1, - }, - { - name: "single head chunk, single mmap chunk, mmap truncated", - headChunks: 1, - mmappedChunks: 1, - truncateBefore: chunkRange, - expectedTruncated: 1, - expectedHead: 1, - expectedMmap: 0, - expectedFirstChunkID: 1, - }, - { - name: "5 head chunk, 5 mmap chunk, third head truncated", - headChunks: 5, - mmappedChunks: 5, - truncateBefore: chunkRange * 7, - expectedTruncated: 7, - expectedHead: 3, - expectedMmap: 0, - expectedFirstChunkID: 7, - }, - { - name: "2 head chunks, 3 mmap chunks, second mmap truncated", - headChunks: 2, - mmappedChunks: 3, - truncateBefore: chunkRange * 2, - expectedTruncated: 2, - expectedHead: 2, - expectedMmap: 1, - expectedFirstChunkID: 2, - }, - } - - for _, tc := range tests { - t.Run(tc.name, func(t *testing.T) { - dir := t.TempDir() - chunkDiskMapper, err := chunks.NewChunkDiskMapper(nil, dir, chunkenc.NewPool(), chunks.DefaultWriteBufferSize, chunks.DefaultWriteQueueSize) - require.NoError(t, err) - defer func() { - require.NoError(t, chunkDiskMapper.Close()) - }() - - series := newMemSeries(labels.EmptyLabels(), 1, 0, true, false) - - cOpts := chunkOpts{ - chunkDiskMapper: chunkDiskMapper, - chunkRange: chunkRange, - samplesPerChunk: DefaultSamplesPerChunk, - } - - var headStart int - if tc.mmappedChunks > 0 { - headStart = (tc.mmappedChunks + 1) * chunkRange - for i := 0; i < (tc.mmappedChunks+1)*chunkRange; i += chunkStep { - ok, _ := series.append(int64(i), float64(i), 0, cOpts) - require.True(t, ok, "sample append failed") - } - series.mmapChunks(chunkDiskMapper) - } - - if tc.headChunks == 0 { - series.headChunks = nil - } else { - for i := headStart; i < chunkRange*(tc.mmappedChunks+tc.headChunks); i += chunkStep { - ok, _ := series.append(int64(i), float64(i), 0, cOpts) - require.True(t, ok, "sample append failed: %d", i) - } - } - - if tc.headChunks > 0 { - require.NotNil(t, series.headChunks, "head chunk is missing") - require.Equal(t, tc.headChunks, series.headChunks.len(), "wrong number of head chunks") - } else { - require.Nil(t, series.headChunks, "head chunk is present") - } - require.Len(t, series.mmappedChunks, tc.mmappedChunks, "wrong number of mmapped chunks") - - truncated := series.truncateChunksBefore(tc.truncateBefore, 0) - require.Equal(t, tc.expectedTruncated, truncated, "wrong number of truncated chunks returned") - - require.Len(t, series.mmappedChunks, tc.expectedMmap, "wrong number of mmappedChunks after truncation") - - if tc.expectedHead > 0 { - require.NotNil(t, series.headChunks, "headChunks should is nil after truncation") - require.Equal(t, tc.expectedHead, series.headChunks.len(), "wrong number of head chunks after truncation") - require.Nil(t, series.headChunks.oldest().prev, "last head chunk cannot have any next chunk set") - } else { - require.Nil(t, series.headChunks, "headChunks should is non-nil after truncation") - } - - if series.headChunks != nil || len(series.mmappedChunks) > 0 { - require.GreaterOrEqual(t, series.maxTime(), tc.truncateBefore, "wrong value of series.maxTime() after truncation") - } else { - require.Equal(t, int64(math.MinInt64), series.maxTime(), "wrong value of series.maxTime() after truncation") - } - - require.Equal(t, tc.expectedFirstChunkID, series.firstChunkID, "wrong firstChunkID after truncation") - }) - } -} - -func TestHeadDeleteSeriesWithoutSamples(t *testing.T) { - for _, compress := range []compression.Type{compression.None, compression.Snappy, compression.Zstd} { - t.Run(fmt.Sprintf("compress=%s", compress), func(t *testing.T) { - entries := []any{ - []record.RefSeries{ - {Ref: 10, Labels: labels.FromStrings("a", "1")}, - }, - []record.RefSample{}, - []record.RefSeries{ - {Ref: 50, Labels: labels.FromStrings("a", "2")}, - }, - []record.RefSample{ - {Ref: 50, T: 80, V: 1}, - {Ref: 50, T: 90, V: 1}, - }, - } - head, w := newTestHead(t, 1000, compress, false) - defer func() { - require.NoError(t, head.Close()) - }() - - populateTestWL(t, w, entries, nil) - - require.NoError(t, head.Init(math.MinInt64)) - - require.NoError(t, head.Delete(context.Background(), 0, 100, labels.MustNewMatcher(labels.MatchEqual, "a", "1"))) - }) - } -} - -func TestHeadDeleteSimple(t *testing.T) { +func TestHeadAppenderV2_DeleteSimple(t *testing.T) { buildSmpls := func(s []int64) []sample { ss := make([]sample, 0, len(s)) for _, t := range s { @@ -1784,9 +479,9 @@ func TestHeadDeleteSimple(t *testing.T) { head, w := newTestHead(t, 1000, compress, false) require.NoError(t, head.Init(0)) - app := head.Appender(context.Background()) + app := head.AppenderV2(context.Background()) for _, smpl := range smplsAll { - _, err := app.Append(0, lblsDefault, smpl.t, smpl.f) + _, err := app.Append(0, lblsDefault, 0, smpl.t, smpl.f, nil, nil, storage.AOptions{}) require.NoError(t, err) } require.NoError(t, app.Commit()) @@ -1797,9 +492,9 @@ func TestHeadDeleteSimple(t *testing.T) { } // Add more samples. - app = head.Appender(context.Background()) + app = head.AppenderV2(context.Background()) for _, smpl := range c.addSamples { - _, err := app.Append(0, lblsDefault, smpl.t, smpl.f) + _, err := app.Append(0, lblsDefault, 0, smpl.t, smpl.f, nil, nil, storage.AOptions{}) require.NoError(t, err) } require.NoError(t, app.Commit()) @@ -1859,18 +554,18 @@ func TestHeadDeleteSimple(t *testing.T) { } } -func TestDeleteUntilCurMax(t *testing.T) { +func TestHeadAppenderV2_DeleteUntilCurrMax(t *testing.T) { hb, _ := newTestHead(t, 1000000, compression.None, false) defer func() { require.NoError(t, hb.Close()) }() numSamples := int64(10) - app := hb.Appender(context.Background()) + app := hb.AppenderV2(context.Background()) smpls := make([]float64, numSamples) for i := range numSamples { smpls[i] = rand.Float64() - _, err := app.Append(0, labels.FromStrings("a", "b"), i, smpls[i]) + _, err := app.Append(0, labels.FromStrings("a", "b"), 0, i, smpls[i], nil, nil, storage.AOptions{}) require.NoError(t, err) } require.NoError(t, app.Commit()) @@ -1890,8 +585,8 @@ func TestDeleteUntilCurMax(t *testing.T) { require.Empty(t, res.Warnings()) // Add again and test for presence. - app = hb.Appender(context.Background()) - _, err = app.Append(0, labels.FromStrings("a", "b"), 11, 1) + app = hb.AppenderV2(context.Background()) + _, err = app.Append(0, labels.FromStrings("a", "b"), 0, 11, 1, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) q, err = NewBlockQuerier(hb, 0, 100000) @@ -1909,15 +604,15 @@ func TestDeleteUntilCurMax(t *testing.T) { require.Empty(t, res.Warnings()) } -func TestDeletedSamplesAndSeriesStillInWALAfterCheckpoint(t *testing.T) { +func TestHeadAppenderV2_DeleteSamplesAndSeriesStillInWALAfterCheckpoint(t *testing.T) { numSamples := 10000 // Enough samples to cause a checkpoint. hb, w := newTestHead(t, int64(numSamples)*10, compression.None, false) for i := range numSamples { - app := hb.Appender(context.Background()) - _, err := app.Append(0, labels.FromStrings("a", "b"), int64(i), 0) + app := hb.AppenderV2(context.Background()) + _, err := app.Append(0, labels.FromStrings("a", "b"), 0, int64(i), 0, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) } @@ -1953,7 +648,7 @@ func TestDeletedSamplesAndSeriesStillInWALAfterCheckpoint(t *testing.T) { require.Equal(t, 0, metadata) } -func TestDelete_e2e(t *testing.T) { +func TestHeadAppenderV2_Delete_e2e(t *testing.T) { numDatapoints := 1000 numRanges := 1000 timeInterval := int64(2) @@ -2010,14 +705,14 @@ func TestDelete_e2e(t *testing.T) { require.NoError(t, hb.Close()) }() - app := hb.Appender(context.Background()) + app := hb.AppenderV2(context.Background()) for _, l := range lbls { ls := labels.New(l...) series := []chunks.Sample{} ts := rand.Int63n(300) for range numDatapoints { v := rand.Float64() - _, err := app.Append(0, ls, ts, v) + _, err := app.Append(0, ls, 0, ts, v, nil, nil, storage.AOptions{}) require.NoError(t, err) series = append(series, sample{ts, v, nil, nil}) ts += rand.Int63n(timeInterval) + 1 @@ -2112,385 +807,7 @@ func TestDelete_e2e(t *testing.T) { } } -func boundedSamples(full []chunks.Sample, mint, maxt int64) []chunks.Sample { - for len(full) > 0 { - if full[0].T() >= mint { - break - } - full = full[1:] - } - for i, s := range full { - // labels.Labelinate on the first sample larger than maxt. - if s.T() > maxt { - return full[:i] - } - } - // maxt is after highest sample. - return full -} - -func deletedSamples(full []chunks.Sample, dranges tombstones.Intervals) []chunks.Sample { - ds := make([]chunks.Sample, 0, len(full)) -Outer: - for _, s := range full { - for _, r := range dranges { - if r.InBounds(s.T()) { - continue Outer - } - } - ds = append(ds, s) - } - - return ds -} - -func TestComputeChunkEndTime(t *testing.T) { - cases := map[string]struct { - start, cur, max int64 - ratioToFull float64 - res int64 - }{ - "exactly 1/4 full, even increment": { - start: 0, - cur: 250, - max: 1000, - ratioToFull: 4, - res: 1000, - }, - "exactly 1/4 full, uneven increment": { - start: 100, - cur: 200, - max: 1000, - ratioToFull: 4, - res: 550, - }, - "decimal ratio to full": { - start: 5000, - cur: 5110, - max: 10000, - ratioToFull: 4.2, - res: 5500, - }, - // Case where we fit floored 0 chunks. Must catch division by 0 - // and default to maximum time. - "fit floored 0 chunks": { - start: 0, - cur: 500, - max: 1000, - ratioToFull: 4, - res: 1000, - }, - // Catch division by zero for cur == start. Strictly not a possible case. - "cur == start": { - start: 100, - cur: 100, - max: 1000, - ratioToFull: 4, - res: 104, - }, - } - - for testName, tc := range cases { - t.Run(testName, func(t *testing.T) { - got := computeChunkEndTime(tc.start, tc.cur, tc.max, tc.ratioToFull) - require.Equal(t, tc.res, got, "(start: %d, cur: %d, max: %d)", tc.start, tc.cur, tc.max) - }) - } -} - -func TestMemSeries_append(t *testing.T) { - dir := t.TempDir() - // This is usually taken from the Head, but passing manually here. - chunkDiskMapper, err := chunks.NewChunkDiskMapper(nil, dir, chunkenc.NewPool(), chunks.DefaultWriteBufferSize, chunks.DefaultWriteQueueSize) - require.NoError(t, err) - defer func() { - require.NoError(t, chunkDiskMapper.Close()) - }() - cOpts := chunkOpts{ - chunkDiskMapper: chunkDiskMapper, - chunkRange: 500, - samplesPerChunk: DefaultSamplesPerChunk, - } - - s := newMemSeries(labels.Labels{}, 1, 0, defaultIsolationDisabled, false) - - // Add first two samples at the very end of a chunk range and the next two - // on and after it. - // New chunk must correctly be cut at 1000. - ok, chunkCreated := s.append(998, 1, 0, cOpts) - require.True(t, ok, "append failed") - require.True(t, chunkCreated, "first sample created chunk") - - ok, chunkCreated = s.append(999, 2, 0, cOpts) - require.True(t, ok, "append failed") - require.False(t, chunkCreated, "second sample should use same chunk") - s.mmapChunks(chunkDiskMapper) - - ok, chunkCreated = s.append(1000, 3, 0, cOpts) - require.True(t, ok, "append failed") - require.True(t, chunkCreated, "expected new chunk on boundary") - - ok, chunkCreated = s.append(1001, 4, 0, cOpts) - require.True(t, ok, "append failed") - require.False(t, chunkCreated, "second sample should use same chunk") - - s.mmapChunks(chunkDiskMapper) - require.Len(t, s.mmappedChunks, 1, "there should be only 1 mmapped chunk") - require.Equal(t, int64(998), s.mmappedChunks[0].minTime, "wrong chunk range") - require.Equal(t, int64(999), s.mmappedChunks[0].maxTime, "wrong chunk range") - require.Equal(t, int64(1000), s.headChunks.minTime, "wrong chunk range") - require.Equal(t, int64(1001), s.headChunks.maxTime, "wrong chunk range") - - // Fill the range [1000,2000) with many samples. Intermediate chunks should be cut - // at approximately 120 samples per chunk. - for i := 1; i < 1000; i++ { - ok, _ := s.append(1001+int64(i), float64(i), 0, cOpts) - require.True(t, ok, "append failed") - } - s.mmapChunks(chunkDiskMapper) - - require.Greater(t, len(s.mmappedChunks)+1, 7, "expected intermediate chunks") - - // All chunks but the first and last should now be moderately full. - for i, c := range s.mmappedChunks[1:] { - chk, err := chunkDiskMapper.Chunk(c.ref) - require.NoError(t, err) - require.Greater(t, chk.NumSamples(), 100, "unexpected small chunk %d of length %d", i, chk.NumSamples()) - } -} - -func TestMemSeries_appendHistogram(t *testing.T) { - dir := t.TempDir() - // This is usually taken from the Head, but passing manually here. - chunkDiskMapper, err := chunks.NewChunkDiskMapper(nil, dir, chunkenc.NewPool(), chunks.DefaultWriteBufferSize, chunks.DefaultWriteQueueSize) - require.NoError(t, err) - defer func() { - require.NoError(t, chunkDiskMapper.Close()) - }() - cOpts := chunkOpts{ - chunkDiskMapper: chunkDiskMapper, - chunkRange: int64(1000), - samplesPerChunk: DefaultSamplesPerChunk, - } - - s := newMemSeries(labels.Labels{}, 1, 0, defaultIsolationDisabled, false) - - histograms := tsdbutil.GenerateTestHistograms(4) - histogramWithOneMoreBucket := histograms[3].Copy() - histogramWithOneMoreBucket.Count++ - histogramWithOneMoreBucket.Sum += 1.23 - histogramWithOneMoreBucket.PositiveSpans[1].Length = 3 - histogramWithOneMoreBucket.PositiveBuckets = append(histogramWithOneMoreBucket.PositiveBuckets, 1) - - // Add first two samples at the very end of a chunk range and the next two - // on and after it. - // New chunk must correctly be cut at 1000. - ok, chunkCreated := s.appendHistogram(998, histograms[0], 0, cOpts) - require.True(t, ok, "append failed") - require.True(t, chunkCreated, "first sample created chunk") - - ok, chunkCreated = s.appendHistogram(999, histograms[1], 0, cOpts) - require.True(t, ok, "append failed") - require.False(t, chunkCreated, "second sample should use same chunk") - - ok, chunkCreated = s.appendHistogram(1000, histograms[2], 0, cOpts) - require.True(t, ok, "append failed") - require.True(t, chunkCreated, "expected new chunk on boundary") - - ok, chunkCreated = s.appendHistogram(1001, histograms[3], 0, cOpts) - require.True(t, ok, "append failed") - require.False(t, chunkCreated, "second sample should use same chunk") - - s.mmapChunks(chunkDiskMapper) - require.Len(t, s.mmappedChunks, 1, "there should be only 1 mmapped chunk") - require.Equal(t, int64(998), s.mmappedChunks[0].minTime, "wrong chunk range") - require.Equal(t, int64(999), s.mmappedChunks[0].maxTime, "wrong chunk range") - require.Equal(t, int64(1000), s.headChunks.minTime, "wrong chunk range") - require.Equal(t, int64(1001), s.headChunks.maxTime, "wrong chunk range") - - ok, chunkCreated = s.appendHistogram(1002, histogramWithOneMoreBucket, 0, cOpts) - require.True(t, ok, "append failed") - require.False(t, chunkCreated, "third sample should trigger a re-encoded chunk") - - s.mmapChunks(chunkDiskMapper) - require.Len(t, s.mmappedChunks, 1, "there should be only 1 mmapped chunk") - require.Equal(t, int64(998), s.mmappedChunks[0].minTime, "wrong chunk range") - require.Equal(t, int64(999), s.mmappedChunks[0].maxTime, "wrong chunk range") - require.Equal(t, int64(1000), s.headChunks.minTime, "wrong chunk range") - require.Equal(t, int64(1002), s.headChunks.maxTime, "wrong chunk range") -} - -func TestMemSeries_append_atVariableRate(t *testing.T) { - const samplesPerChunk = 120 - dir := t.TempDir() - // This is usually taken from the Head, but passing manually here. - chunkDiskMapper, err := chunks.NewChunkDiskMapper(nil, dir, chunkenc.NewPool(), chunks.DefaultWriteBufferSize, chunks.DefaultWriteQueueSize) - require.NoError(t, err) - t.Cleanup(func() { - require.NoError(t, chunkDiskMapper.Close()) - }) - cOpts := chunkOpts{ - chunkDiskMapper: chunkDiskMapper, - chunkRange: DefaultBlockDuration, - samplesPerChunk: samplesPerChunk, - } - - s := newMemSeries(labels.Labels{}, 1, 0, defaultIsolationDisabled, false) - - // At this slow rate, we will fill the chunk in two block durations. - slowRate := (DefaultBlockDuration * 2) / samplesPerChunk - - var nextTs int64 - var totalAppendedSamples int - for i := range samplesPerChunk / 4 { - ok, _ := s.append(nextTs, float64(i), 0, cOpts) - require.Truef(t, ok, "slow sample %d was not appended", i) - nextTs += slowRate - totalAppendedSamples++ - } - require.Equal(t, DefaultBlockDuration, s.nextAt, "after appending a samplesPerChunk/4 samples at a slow rate, we should aim to cut a new block at the default block duration %d, but it's set to %d", DefaultBlockDuration, s.nextAt) - - // Suddenly, the rate increases and we receive a sample every millisecond. - for i := range math.MaxUint16 { - ok, _ := s.append(nextTs, float64(i), 0, cOpts) - require.Truef(t, ok, "quick sample %d was not appended", i) - nextTs++ - totalAppendedSamples++ - } - ok, chunkCreated := s.append(DefaultBlockDuration, float64(0), 0, cOpts) - require.True(t, ok, "new chunk sample was not appended") - require.True(t, chunkCreated, "sample at block duration timestamp should create a new chunk") - - s.mmapChunks(chunkDiskMapper) - var totalSamplesInChunks int - for i, c := range s.mmappedChunks { - totalSamplesInChunks += int(c.numSamples) - require.LessOrEqualf(t, c.numSamples, uint16(2*samplesPerChunk), "mmapped chunk %d has more than %d samples", i, 2*samplesPerChunk) - } - require.Equal(t, totalAppendedSamples, totalSamplesInChunks, "wrong number of samples in %d mmapped chunks", len(s.mmappedChunks)) -} - -func TestGCChunkAccess(t *testing.T) { - // Put a chunk, select it. GC it and then access it. - const chunkRange = 1000 - h, _ := newTestHead(t, chunkRange, compression.None, false) - defer func() { - require.NoError(t, h.Close()) - }() - - cOpts := chunkOpts{ - chunkDiskMapper: h.chunkDiskMapper, - chunkRange: chunkRange, - samplesPerChunk: DefaultSamplesPerChunk, - } - - h.initTime(0) - - s, _, _ := h.getOrCreate(1, labels.FromStrings("a", "1"), false) - - // Appending 2 samples for the first chunk. - ok, chunkCreated := s.append(0, 0, 0, cOpts) - require.True(t, ok, "series append failed") - require.True(t, chunkCreated, "chunks was not created") - ok, chunkCreated = s.append(999, 999, 0, cOpts) - require.True(t, ok, "series append failed") - require.False(t, chunkCreated, "chunks was created") - - // A new chunks should be created here as it's beyond the chunk range. - ok, chunkCreated = s.append(1000, 1000, 0, cOpts) - require.True(t, ok, "series append failed") - require.True(t, chunkCreated, "chunks was not created") - ok, chunkCreated = s.append(1999, 1999, 0, cOpts) - require.True(t, ok, "series append failed") - require.False(t, chunkCreated, "chunks was created") - - idx := h.indexRange(0, 1500) - var ( - chunks []chunks.Meta - builder labels.ScratchBuilder - ) - require.NoError(t, idx.Series(1, &builder, &chunks)) - - require.Equal(t, labels.FromStrings("a", "1"), builder.Labels()) - require.Len(t, chunks, 2) - - cr, err := h.chunksRange(0, 1500, nil) - require.NoError(t, err) - _, _, err = cr.ChunkOrIterable(chunks[0]) - require.NoError(t, err) - _, _, err = cr.ChunkOrIterable(chunks[1]) - require.NoError(t, err) - - require.NoError(t, h.Truncate(1500)) // Remove a chunk. - - _, _, err = cr.ChunkOrIterable(chunks[0]) - require.Equal(t, storage.ErrNotFound, err) - _, _, err = cr.ChunkOrIterable(chunks[1]) - require.NoError(t, err) -} - -func TestGCSeriesAccess(t *testing.T) { - // Put a series, select it. GC it and then access it. - const chunkRange = 1000 - h, _ := newTestHead(t, chunkRange, compression.None, false) - defer func() { - require.NoError(t, h.Close()) - }() - - cOpts := chunkOpts{ - chunkDiskMapper: h.chunkDiskMapper, - chunkRange: chunkRange, - samplesPerChunk: DefaultSamplesPerChunk, - } - - h.initTime(0) - - s, _, _ := h.getOrCreate(1, labels.FromStrings("a", "1"), false) - - // Appending 2 samples for the first chunk. - ok, chunkCreated := s.append(0, 0, 0, cOpts) - require.True(t, ok, "series append failed") - require.True(t, chunkCreated, "chunks was not created") - ok, chunkCreated = s.append(999, 999, 0, cOpts) - require.True(t, ok, "series append failed") - require.False(t, chunkCreated, "chunks was created") - - // A new chunks should be created here as it's beyond the chunk range. - ok, chunkCreated = s.append(1000, 1000, 0, cOpts) - require.True(t, ok, "series append failed") - require.True(t, chunkCreated, "chunks was not created") - ok, chunkCreated = s.append(1999, 1999, 0, cOpts) - require.True(t, ok, "series append failed") - require.False(t, chunkCreated, "chunks was created") - - idx := h.indexRange(0, 2000) - var ( - chunks []chunks.Meta - builder labels.ScratchBuilder - ) - require.NoError(t, idx.Series(1, &builder, &chunks)) - - require.Equal(t, labels.FromStrings("a", "1"), builder.Labels()) - require.Len(t, chunks, 2) - - cr, err := h.chunksRange(0, 2000, nil) - require.NoError(t, err) - _, _, err = cr.ChunkOrIterable(chunks[0]) - require.NoError(t, err) - _, _, err = cr.ChunkOrIterable(chunks[1]) - require.NoError(t, err) - - require.NoError(t, h.Truncate(2000)) // Remove the series. - - require.Equal(t, (*memSeries)(nil), h.series.getByID(1)) - - _, _, err = cr.ChunkOrIterable(chunks[0]) - require.Equal(t, storage.ErrNotFound, err) - _, _, err = cr.ChunkOrIterable(chunks[1]) - require.Equal(t, storage.ErrNotFound, err) -} - -func TestUncommittedSamplesNotLostOnTruncate(t *testing.T) { +func TestHeadAppenderV2_UncommittedSamplesNotLostOnTruncate(t *testing.T) { h, _ := newTestHead(t, 1000, compression.None, false) defer func() { require.NoError(t, h.Close()) @@ -2498,9 +815,9 @@ func TestUncommittedSamplesNotLostOnTruncate(t *testing.T) { h.initTime(0) - app := h.appender() + app := h.appenderV2() lset := labels.FromStrings("a", "1") - _, err := app.Append(0, lset, 2100, 1) + _, err := app.Append(0, lset, 0, 2100, 1, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, h.Truncate(2000)) @@ -2520,7 +837,7 @@ func TestUncommittedSamplesNotLostOnTruncate(t *testing.T) { require.Empty(t, ss.Warnings()) } -func TestRemoveSeriesAfterRollbackAndTruncate(t *testing.T) { +func TestHeadAppenderV2_TestRemoveSeriesAfterRollbackAndTruncate(t *testing.T) { h, _ := newTestHead(t, 1000, compression.None, false) defer func() { require.NoError(t, h.Close()) @@ -2528,9 +845,9 @@ func TestRemoveSeriesAfterRollbackAndTruncate(t *testing.T) { h.initTime(0) - app := h.appender() + app := h.appenderV2() lset := labels.FromStrings("a", "1") - _, err := app.Append(0, lset, 2100, 1) + _, err := app.Append(0, lset, 0, 2100, 1, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, h.Truncate(2000)) @@ -2551,7 +868,7 @@ func TestRemoveSeriesAfterRollbackAndTruncate(t *testing.T) { require.Equal(t, (*memSeries)(nil), h.series.getByHash(lset.Hash(), lset)) } -func TestHead_LogRollback(t *testing.T) { +func TestHeadAppenderV2_LogRollback(t *testing.T) { for _, compress := range []compression.Type{compression.None, compression.Snappy, compression.Zstd} { t.Run(fmt.Sprintf("compress=%s", compress), func(t *testing.T) { h, w := newTestHead(t, 1000, compress, false) @@ -2559,8 +876,8 @@ func TestHead_LogRollback(t *testing.T) { require.NoError(t, h.Close()) }() - app := h.Appender(context.Background()) - _, err := app.Append(0, labels.FromStrings("a", "b"), 1, 2) + app := h.AppenderV2(context.Background()) + _, err := app.Append(0, labels.FromStrings("a", "b"), 0, 1, 2, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Rollback()) @@ -2575,7 +892,7 @@ func TestHead_LogRollback(t *testing.T) { } } -func TestHead_ReturnsSortedLabelValues(t *testing.T) { +func TestHeadAppenderV2_ReturnsSortedLabelValues(t *testing.T) { h, _ := newTestHead(t, 1000, compression.None, false) defer func() { require.NoError(t, h.Close()) @@ -2583,14 +900,14 @@ func TestHead_ReturnsSortedLabelValues(t *testing.T) { h.initTime(0) - app := h.appender() + app := h.appenderV2() for i := 100; i > 0; i-- { for j := range 10 { lset := labels.FromStrings( "__name__", fmt.Sprintf("metric_%d", i), "label", fmt.Sprintf("value_%d", j), ) - _, err := app.Append(0, lset, 2100, 1) + _, err := app.Append(0, lset, 0, 2100, 1, nil, nil, storage.AOptions{}) require.NoError(t, err) } } @@ -2605,257 +922,14 @@ func TestHead_ReturnsSortedLabelValues(t *testing.T) { require.NoError(t, q.Close()) } -// TestWalRepair_DecodingError ensures that a repair is run for an error -// when decoding a record. -func TestWalRepair_DecodingError(t *testing.T) { - var enc record.Encoder - for name, test := range map[string]struct { - corrFunc func(rec []byte) []byte // Func that applies the corruption to a record. - rec []byte - totalRecs int - expRecs int - }{ - "decode_series": { - func(rec []byte) []byte { - return rec[:3] - }, - enc.Series([]record.RefSeries{{Ref: 1, Labels: labels.FromStrings("a", "b")}}, []byte{}), - 9, - 5, - }, - "decode_samples": { - func(rec []byte) []byte { - return rec[:3] - }, - enc.Samples([]record.RefSample{{Ref: 0, T: 99, V: 1}}, []byte{}), - 9, - 5, - }, - "decode_tombstone": { - func(rec []byte) []byte { - return rec[:3] - }, - enc.Tombstones([]tombstones.Stone{{Ref: 1, Intervals: tombstones.Intervals{}}}, []byte{}), - 9, - 5, - }, - } { - for _, compress := range []compression.Type{compression.None, compression.Snappy, compression.Zstd} { - t.Run(fmt.Sprintf("%s,compress=%s", name, compress), func(t *testing.T) { - dir := t.TempDir() - - // Fill the wal and corrupt it. - { - w, err := wlog.New(nil, nil, filepath.Join(dir, "wal"), compress) - require.NoError(t, err) - - for i := 1; i <= test.totalRecs; i++ { - // At this point insert a corrupted record. - if i-1 == test.expRecs { - require.NoError(t, w.Log(test.corrFunc(test.rec))) - continue - } - require.NoError(t, w.Log(test.rec)) - } - - opts := DefaultHeadOptions() - opts.ChunkRange = 1 - opts.ChunkDirRoot = w.Dir() - h, err := NewHead(nil, nil, w, nil, opts, nil) - require.NoError(t, err) - require.Equal(t, 0.0, prom_testutil.ToFloat64(h.metrics.walCorruptionsTotal)) - initErr := h.Init(math.MinInt64) - - var cerr *wlog.CorruptionErr - require.ErrorAs(t, initErr, &cerr, "reading the wal didn't return corruption error") - require.NoError(t, h.Close()) // Head will close the wal as well. - } - - // Open the db to trigger a repair. - { - db, err := Open(dir, nil, nil, DefaultOptions(), nil) - require.NoError(t, err) - defer func() { - require.NoError(t, db.Close()) - }() - require.Equal(t, 1.0, prom_testutil.ToFloat64(db.head.metrics.walCorruptionsTotal)) - } - - // Read the wal content after the repair. - { - sr, err := wlog.NewSegmentsReader(filepath.Join(dir, "wal")) - require.NoError(t, err) - defer sr.Close() - r := wlog.NewReader(sr) - - var actRec int - for r.Next() { - actRec++ - } - require.NoError(t, r.Err()) - require.Equal(t, test.expRecs, actRec, "Wrong number of intact records") - } - }) - } - } -} - -// TestWblRepair_DecodingError ensures that a repair is run for an error -// when decoding a record. -func TestWblRepair_DecodingError(t *testing.T) { - var enc record.Encoder - corrFunc := func(rec []byte) []byte { - return rec[:3] - } - rec := enc.Samples([]record.RefSample{{Ref: 0, T: 99, V: 1}}, []byte{}) - totalRecs := 9 - expRecs := 5 - dir := t.TempDir() - - // Fill the wbl and corrupt it. - { - wal, err := wlog.New(nil, nil, filepath.Join(dir, "wal"), compression.None) - require.NoError(t, err) - wbl, err := wlog.New(nil, nil, filepath.Join(dir, "wbl"), compression.None) - require.NoError(t, err) - - for i := 1; i <= totalRecs; i++ { - // At this point insert a corrupted record. - if i-1 == expRecs { - require.NoError(t, wbl.Log(corrFunc(rec))) - continue - } - require.NoError(t, wbl.Log(rec)) - } - - opts := DefaultHeadOptions() - opts.ChunkRange = 1 - opts.ChunkDirRoot = wal.Dir() - opts.OutOfOrderCapMax.Store(30) - opts.OutOfOrderTimeWindow.Store(1000 * time.Minute.Milliseconds()) - h, err := NewHead(nil, nil, wal, wbl, opts, nil) - require.NoError(t, err) - require.Equal(t, 0.0, prom_testutil.ToFloat64(h.metrics.walCorruptionsTotal)) - initErr := h.Init(math.MinInt64) - - var elb *errLoadWbl - require.ErrorAs(t, initErr, &elb) // Wbl errors are wrapped into errLoadWbl, make sure we can unwrap it. - - var cerr *wlog.CorruptionErr - require.ErrorAs(t, initErr, &cerr, "reading the wal didn't return corruption error") - require.NoError(t, h.Close()) // Head will close the wal as well. - } - - // Open the db to trigger a repair. - { - db, err := Open(dir, nil, nil, DefaultOptions(), nil) - require.NoError(t, err) - defer func() { - require.NoError(t, db.Close()) - }() - require.Equal(t, 1.0, prom_testutil.ToFloat64(db.head.metrics.walCorruptionsTotal)) - } - - // Read the wbl content after the repair. - { - sr, err := wlog.NewSegmentsReader(filepath.Join(dir, "wbl")) - require.NoError(t, err) - defer sr.Close() - r := wlog.NewReader(sr) - - var actRec int - for r.Next() { - actRec++ - } - require.NoError(t, r.Err()) - require.Equal(t, expRecs, actRec, "Wrong number of intact records") - } -} - -func TestHeadReadWriterRepair(t *testing.T) { - dir := t.TempDir() - - const chunkRange = 1000 - - walDir := filepath.Join(dir, "wal") - // Fill the chunk segments and corrupt it. - { - w, err := wlog.New(nil, nil, walDir, compression.None) - require.NoError(t, err) - - opts := DefaultHeadOptions() - opts.ChunkRange = chunkRange - opts.ChunkDirRoot = dir - opts.ChunkWriteQueueSize = 1 // We need to set this option so that we use the async queue. Upstream prometheus uses the queue directly. - h, err := NewHead(nil, nil, w, nil, opts, nil) - require.NoError(t, err) - require.Equal(t, 0.0, prom_testutil.ToFloat64(h.metrics.mmapChunkCorruptionTotal)) - require.NoError(t, h.Init(math.MinInt64)) - - cOpts := chunkOpts{ - chunkDiskMapper: h.chunkDiskMapper, - chunkRange: chunkRange, - samplesPerChunk: DefaultSamplesPerChunk, - } - - s, created, _ := h.getOrCreate(1, labels.FromStrings("a", "1"), false) - require.True(t, created, "series was not created") - - for i := range 7 { - ok, chunkCreated := s.append(int64(i*chunkRange), float64(i*chunkRange), 0, cOpts) - require.True(t, ok, "series append failed") - require.True(t, chunkCreated, "chunk was not created") - ok, chunkCreated = s.append(int64(i*chunkRange)+chunkRange-1, float64(i*chunkRange), 0, cOpts) - require.True(t, ok, "series append failed") - require.False(t, chunkCreated, "chunk was created") - h.chunkDiskMapper.CutNewFile() - s.mmapChunks(h.chunkDiskMapper) - } - require.NoError(t, h.Close()) - - // Verify that there are 6 segment files. - // It should only be 6 because the last call to .CutNewFile() won't - // take effect without another chunk being written. - files, err := os.ReadDir(mmappedChunksDir(dir)) - require.NoError(t, err) - require.Len(t, files, 6) - - // Corrupt the 4th file by writing a random byte to series ref. - f, err := os.OpenFile(filepath.Join(mmappedChunksDir(dir), files[3].Name()), os.O_WRONLY, 0o666) - require.NoError(t, err) - n, err := f.WriteAt([]byte{67, 88}, chunks.HeadChunkFileHeaderSize+2) - require.NoError(t, err) - require.Equal(t, 2, n) - require.NoError(t, f.Close()) - } - - // Open the db to trigger a repair. - { - db, err := Open(dir, nil, nil, DefaultOptions(), nil) - require.NoError(t, err) - defer func() { - require.NoError(t, db.Close()) - }() - require.Equal(t, 1.0, prom_testutil.ToFloat64(db.head.metrics.mmapChunkCorruptionTotal)) - } - - // Verify that there are 3 segment files after the repair. - // The segments from the corrupt segment should be removed. - { - files, err := os.ReadDir(mmappedChunksDir(dir)) - require.NoError(t, err) - require.Len(t, files, 3) - } -} - -func TestNewWalSegmentOnTruncate(t *testing.T) { +func TestHeadAppenderV2_NewWalSegmentOnTruncate(t *testing.T) { h, wal := newTestHead(t, 1000, compression.None, false) defer func() { require.NoError(t, h.Close()) }() add := func(ts int64) { - app := h.Appender(context.Background()) - _, err := app.Append(0, labels.FromStrings("a", "b"), ts, 0) + app := h.AppenderV2(context.Background()) + _, err := app.Append(0, labels.FromStrings("a", "b"), 0, ts, 0, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) } @@ -2878,15 +952,15 @@ func TestNewWalSegmentOnTruncate(t *testing.T) { require.Equal(t, 2, last) } -func TestAddDuplicateLabelName(t *testing.T) { +func TestHeadAppenderV2_Append_DuplicateLabelName(t *testing.T) { h, _ := newTestHead(t, 1000, compression.None, false) defer func() { require.NoError(t, h.Close()) }() add := func(labels labels.Labels, labelName string) { - app := h.Appender(context.Background()) - _, err := app.Append(0, labels, 0, 0) + app := h.AppenderV2(context.Background()) + _, err := app.Append(0, labels, 0, 0, 0, nil, nil, storage.AOptions{}) require.EqualError(t, err, fmt.Sprintf(`label name "%s" is not unique: invalid sample`, labelName)) } @@ -2895,7 +969,7 @@ func TestAddDuplicateLabelName(t *testing.T) { add(labels.FromStrings("__name__", "up", "job", "prometheus", "le", "500", "le", "400", "unit", "s"), "le") } -func TestMemSeriesIsolation(t *testing.T) { +func TestHeadAppenderV2_MemSeriesIsolation(t *testing.T) { if defaultIsolationDisabled { t.Skip("skipping test since tsdb isolation is disabled") } @@ -2940,17 +1014,17 @@ func TestMemSeriesIsolation(t *testing.T) { addSamples := func(h *Head) int { i := 1 for ; i <= 1000; i++ { - var app storage.Appender + var app storage.AppenderV2 // To initialize bounds. if h.MinTime() == math.MaxInt64 { - app = &initAppender{head: h} + app = &initAppenderV2{head: h} } else { - a := h.appender() + a := h.appenderV2() a.cleanupAppendIDsBelow = 0 app = a } - _, err := app.Append(0, labels.FromStrings("foo", "bar"), int64(i), float64(i)) + _, err := app.Append(0, labels.FromStrings("foo", "bar"), 0, int64(i), float64(i), nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) h.mmapHeadChunks() @@ -2977,9 +1051,9 @@ func TestMemSeriesIsolation(t *testing.T) { require.Equal(t, 999, lastValue(hb, 999)) // Cleanup appendIDs below 500. - app := hb.appender() + app := hb.appenderV2() app.cleanupAppendIDsBelow = 500 - _, err := app.Append(0, labels.FromStrings("foo", "bar"), int64(i), float64(i)) + _, err := app.Append(0, labels.FromStrings("foo", "bar"), 0, int64(i), float64(i), nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) i++ @@ -2996,9 +1070,9 @@ func TestMemSeriesIsolation(t *testing.T) { // Cleanup appendIDs below 1000, which means the sample buffer is // the only thing with appendIDs. - app = hb.appender() + app = hb.appenderV2() app.cleanupAppendIDsBelow = 1000 - _, err = app.Append(0, labels.FromStrings("foo", "bar"), int64(i), float64(i)) + _, err = app.Append(0, labels.FromStrings("foo", "bar"), 0, int64(i), float64(i), nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) require.Equal(t, 999, lastValue(hb, 998)) @@ -3010,9 +1084,9 @@ func TestMemSeriesIsolation(t *testing.T) { i++ // Cleanup appendIDs below 1001, but with a rollback. - app = hb.appender() + app = hb.appenderV2() app.cleanupAppendIDsBelow = 1001 - _, err = app.Append(0, labels.FromStrings("foo", "bar"), int64(i), float64(i)) + _, err = app.Append(0, labels.FromStrings("foo", "bar"), 0, int64(i), float64(i), nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Rollback()) require.Equal(t, 1000, lastValue(hb, 999)) @@ -3047,8 +1121,8 @@ func TestMemSeriesIsolation(t *testing.T) { // Cleanup appendIDs below 1000, which means the sample buffer is // the only thing with appendIDs. - app = hb.appender() - _, err = app.Append(0, labels.FromStrings("foo", "bar"), int64(i), float64(i)) + app = hb.appenderV2() + _, err = app.Append(0, labels.FromStrings("foo", "bar"), 0, int64(i), float64(i), nil, nil, storage.AOptions{}) i++ require.NoError(t, err) require.NoError(t, app.Commit()) @@ -3060,8 +1134,8 @@ func TestMemSeriesIsolation(t *testing.T) { require.Equal(t, 1001, lastValue(hb, 1003)) // Cleanup appendIDs below 1002, but with a rollback. - app = hb.appender() - _, err = app.Append(0, labels.FromStrings("foo", "bar"), int64(i), float64(i)) + app = hb.appenderV2() + _, err = app.Append(0, labels.FromStrings("foo", "bar"), 0, int64(i), float64(i), nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Rollback()) require.Equal(t, 1001, lastValue(hb, 999)) @@ -3071,7 +1145,7 @@ func TestMemSeriesIsolation(t *testing.T) { require.Equal(t, 1001, lastValue(hb, 1003)) } -func TestIsolationRollback(t *testing.T) { +func TestHeadAppenderV2_IsolationRollback(t *testing.T) { if defaultIsolationDisabled { t.Skip("skipping test since tsdb isolation is disabled") } @@ -3082,28 +1156,28 @@ func TestIsolationRollback(t *testing.T) { require.NoError(t, hb.Close()) }() - app := hb.Appender(context.Background()) - _, err := app.Append(0, labels.FromStrings("foo", "bar"), 0, 0) + app := hb.AppenderV2(context.Background()) + _, err := app.Append(0, labels.FromStrings("foo", "bar"), 0, 0, 0, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) require.Equal(t, uint64(1), hb.iso.lowWatermark()) - app = hb.Appender(context.Background()) - _, err = app.Append(0, labels.FromStrings("foo", "bar"), 1, 1) + app = hb.AppenderV2(context.Background()) + _, err = app.Append(0, labels.FromStrings("foo", "bar"), 0, 1, 1, nil, nil, storage.AOptions{}) require.NoError(t, err) - _, err = app.Append(0, labels.FromStrings("foo", "bar", "foo", "baz"), 2, 2) + _, err = app.Append(0, labels.FromStrings("foo", "bar", "foo", "baz"), 0, 2, 2, nil, nil, storage.AOptions{}) require.Error(t, err) require.NoError(t, app.Rollback()) require.Equal(t, uint64(2), hb.iso.lowWatermark()) - app = hb.Appender(context.Background()) - _, err = app.Append(0, labels.FromStrings("foo", "bar"), 3, 3) + app = hb.AppenderV2(context.Background()) + _, err = app.Append(0, labels.FromStrings("foo", "bar"), 0, 3, 3, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) require.Equal(t, uint64(3), hb.iso.lowWatermark(), "Low watermark should proceed to 3 even if append #2 was rolled back.") } -func TestIsolationLowWatermarkMonotonous(t *testing.T) { +func TestHeadAppenderV2_IsolationLowWatermarkMonotonous(t *testing.T) { if defaultIsolationDisabled { t.Skip("skipping test since tsdb isolation is disabled") } @@ -3113,19 +1187,19 @@ func TestIsolationLowWatermarkMonotonous(t *testing.T) { require.NoError(t, hb.Close()) }() - app1 := hb.Appender(context.Background()) - _, err := app1.Append(0, labels.FromStrings("foo", "bar"), 0, 0) + app1 := hb.AppenderV2(context.Background()) + _, err := app1.Append(0, labels.FromStrings("foo", "bar"), 0, 0, 0, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app1.Commit()) require.Equal(t, uint64(1), hb.iso.lowWatermark(), "Low watermark should by 1 after 1st append.") - app1 = hb.Appender(context.Background()) - _, err = app1.Append(0, labels.FromStrings("foo", "bar"), 1, 1) + app1 = hb.AppenderV2(context.Background()) + _, err = app1.Append(0, labels.FromStrings("foo", "bar"), 0, 1, 1, nil, nil, storage.AOptions{}) require.NoError(t, err) require.Equal(t, uint64(2), hb.iso.lowWatermark(), "Low watermark should be two, even if append is not committed yet.") - app2 := hb.Appender(context.Background()) - _, err = app2.Append(0, labels.FromStrings("foo", "baz"), 1, 1) + app2 := hb.AppenderV2(context.Background()) + _, err = app2.Append(0, labels.FromStrings("foo", "baz"), 0, 1, 1, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app2.Commit()) require.Equal(t, uint64(2), hb.iso.lowWatermark(), "Low watermark should stay two because app1 is not committed yet.") @@ -3140,39 +1214,7 @@ func TestIsolationLowWatermarkMonotonous(t *testing.T) { require.Equal(t, uint64(3), hb.iso.lowWatermark(), "After read has finished (iso state closed), low watermark should jump to three.") } -func TestIsolationAppendIDZeroIsNoop(t *testing.T) { - if defaultIsolationDisabled { - t.Skip("skipping test since tsdb isolation is disabled") - } - - h, _ := newTestHead(t, 1000, compression.None, false) - defer func() { - require.NoError(t, h.Close()) - }() - - h.initTime(0) - - cOpts := chunkOpts{ - chunkDiskMapper: h.chunkDiskMapper, - chunkRange: h.chunkRange.Load(), - samplesPerChunk: DefaultSamplesPerChunk, - } - - s, _, _ := h.getOrCreate(1, labels.FromStrings("a", "1"), false) - - ok, _ := s.append(0, 0, 0, cOpts) - require.True(t, ok, "Series append failed.") - require.Equal(t, 0, int(s.txs.txIDCount), "Series should not have an appendID after append with appendID=0.") -} - -func TestHeadSeriesChunkRace(t *testing.T) { - t.Parallel() - for range 100 { - testHeadSeriesChunkRace(t) - } -} - -func TestIsolationWithoutAdd(t *testing.T) { +func TestHeadAppenderV2_IsolationWithoutAdd(t *testing.T) { if defaultIsolationDisabled { t.Skip("skipping test since tsdb isolation is disabled") } @@ -3182,41 +1224,18 @@ func TestIsolationWithoutAdd(t *testing.T) { require.NoError(t, hb.Close()) }() - app := hb.Appender(context.Background()) + app := hb.AppenderV2(context.Background()) require.NoError(t, app.Commit()) - app = hb.Appender(context.Background()) - _, err := app.Append(0, labels.FromStrings("foo", "baz"), 1, 1) + app = hb.AppenderV2(context.Background()) + _, err := app.Append(0, labels.FromStrings("foo", "baz"), 0, 1, 1, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) require.Equal(t, hb.iso.lastAppendID(), hb.iso.lowWatermark(), "High watermark should be equal to the low watermark") } -func TestOutOfOrderSamplesMetric(t *testing.T) { - t.Parallel() - for name, scenario := range sampleTypeScenarios { - t.Run(name, func(t *testing.T) { - options := DefaultOptions() - testOutOfOrderSamplesMetric(t, scenario, options, storage.ErrOutOfOrderSample) - }) - } -} - -func TestOutOfOrderSamplesMetricNativeHistogramOOODisabled(t *testing.T) { - for name, scenario := range sampleTypeScenarios { - if scenario.sampleType != "histogram" { - continue - } - t.Run(name, func(t *testing.T) { - options := DefaultOptions() - options.OutOfOrderTimeWindow = 0 - testOutOfOrderSamplesMetric(t, scenario, options, storage.ErrOutOfOrderSample) - }) - } -} - -func testOutOfOrderSamplesMetric(t *testing.T, scenario sampleTypeScenario, options *Options, expectOutOfOrderError error) { +func testHeadAppenderV2OutOfOrderSamplesMetric(t *testing.T, scenario sampleTypeScenario, options *Options, expectOutOfOrderError error) { dir := t.TempDir() db, err := Open(dir, nil, nil, options, nil) require.NoError(t, err) @@ -3225,13 +1244,14 @@ func testOutOfOrderSamplesMetric(t *testing.T, scenario sampleTypeScenario, opti }() db.DisableCompactions() - appendSample := func(appender storage.Appender, ts int64) (storage.SeriesRef, error) { - ref, _, err := scenario.appendFunc(appender, labels.FromStrings("a", "b"), ts, 99) + appendSample := func(app storage.AppenderV2, ts int64) (storage.SeriesRef, error) { + // TODO(bwplotka): Migrate to V2 natively. + ref, _, err := scenario.appendFunc(storage.AppenderV2AsLimitedV1(app), labels.FromStrings("a", "b"), ts, 99) return ref, err } ctx := context.Background() - app := db.Appender(ctx) + app := db.AppenderV2(ctx) for i := 1; i <= 5; i++ { _, err = appendSample(app, int64(i)) require.NoError(t, err) @@ -3240,7 +1260,7 @@ func testOutOfOrderSamplesMetric(t *testing.T, scenario sampleTypeScenario, opti // Test out of order metric. require.Equal(t, 0.0, prom_testutil.ToFloat64(db.head.metrics.outOfOrderSamples.WithLabelValues(scenario.sampleType))) - app = db.Appender(ctx) + app = db.AppenderV2(ctx) _, err = appendSample(app, 2) require.Equal(t, expectOutOfOrderError, err) require.Equal(t, 1.0, prom_testutil.ToFloat64(db.head.metrics.outOfOrderSamples.WithLabelValues(scenario.sampleType))) @@ -3255,7 +1275,7 @@ func testOutOfOrderSamplesMetric(t *testing.T, scenario sampleTypeScenario, opti require.NoError(t, app.Commit()) // Compact Head to test out of bound metric. - app = db.Appender(ctx) + app = db.AppenderV2(ctx) _, err = appendSample(app, DefaultBlockDuration*2) require.NoError(t, err) require.NoError(t, app.Commit()) @@ -3263,8 +1283,9 @@ func testOutOfOrderSamplesMetric(t *testing.T, scenario sampleTypeScenario, opti require.Equal(t, int64(math.MinInt64), db.head.minValidTime.Load()) require.NoError(t, db.Compact(ctx)) require.Positive(t, db.head.minValidTime.Load()) + require.Equal(t, 0.0, prom_testutil.ToFloat64(db.head.metrics.outOfBoundSamples.WithLabelValues(scenario.sampleType))) - app = db.Appender(ctx) + app = db.AppenderV2(ctx) _, err = appendSample(app, db.head.minValidTime.Load()-2) require.Equal(t, storage.ErrOutOfBounds, err) require.Equal(t, 1.0, prom_testutil.ToFloat64(db.head.metrics.outOfBoundSamples.WithLabelValues(scenario.sampleType))) @@ -3275,7 +1296,7 @@ func testOutOfOrderSamplesMetric(t *testing.T, scenario sampleTypeScenario, opti require.NoError(t, app.Commit()) // Some more valid samples for out of order. - app = db.Appender(ctx) + app = db.AppenderV2(ctx) for i := 1; i <= 5; i++ { _, err = appendSample(app, db.head.minValidTime.Load()+DefaultBlockDuration+int64(i)) require.NoError(t, err) @@ -3283,7 +1304,7 @@ func testOutOfOrderSamplesMetric(t *testing.T, scenario sampleTypeScenario, opti require.NoError(t, app.Commit()) // Test out of order metric. - app = db.Appender(ctx) + app = db.AppenderV2(ctx) _, err = appendSample(app, db.head.minValidTime.Load()+DefaultBlockDuration+2) require.Equal(t, expectOutOfOrderError, err) require.Equal(t, 4.0, prom_testutil.ToFloat64(db.head.metrics.outOfOrderSamples.WithLabelValues(scenario.sampleType))) @@ -3298,356 +1319,74 @@ func testOutOfOrderSamplesMetric(t *testing.T, scenario sampleTypeScenario, opti require.NoError(t, app.Commit()) } -func testHeadSeriesChunkRace(t *testing.T) { - h, _ := newTestHead(t, 1000, compression.None, false) - defer func() { - require.NoError(t, h.Close()) - }() - require.NoError(t, h.Init(0)) - app := h.Appender(context.Background()) - - s2, err := app.Append(0, labels.FromStrings("foo2", "bar"), 5, 0) - require.NoError(t, err) - for ts := int64(6); ts < 11; ts++ { - _, err = app.Append(s2, labels.EmptyLabels(), ts, 0) - require.NoError(t, err) - } - require.NoError(t, app.Commit()) - - matcher := labels.MustNewMatcher(labels.MatchEqual, "", "") - q, err := NewBlockQuerier(h, 18, 22) - require.NoError(t, err) - defer q.Close() - - var wg sync.WaitGroup - wg.Add(1) - go func() { - defer wg.Done() - h.updateMinMaxTime(20, 25) - h.gc() - }() - ss := q.Select(context.Background(), false, nil, matcher) - for ss.Next() { - } - require.NoError(t, ss.Err()) - wg.Wait() -} - -func TestHeadLabelNamesValuesWithMinMaxRange(t *testing.T) { - head, _ := newTestHead(t, 1000, compression.None, false) - defer func() { - require.NoError(t, head.Close()) - }() - - const ( - firstSeriesTimestamp int64 = 100 - secondSeriesTimestamp int64 = 200 - lastSeriesTimestamp int64 = 300 - ) - var ( - seriesTimestamps = []int64{ - firstSeriesTimestamp, - secondSeriesTimestamp, - lastSeriesTimestamp, - } - expectedLabelNames = []string{"a", "b", "c"} - expectedLabelValues = []string{"d", "e", "f"} - ctx = context.Background() - ) - - app := head.Appender(ctx) - for i, name := range expectedLabelNames { - _, err := app.Append(0, labels.FromStrings(name, expectedLabelValues[i]), seriesTimestamps[i], 0) - require.NoError(t, err) - } - require.NoError(t, app.Commit()) - require.Equal(t, firstSeriesTimestamp, head.MinTime()) - require.Equal(t, lastSeriesTimestamp, head.MaxTime()) - - testCases := []struct { - name string - mint int64 - maxt int64 - expectedNames []string - expectedValues []string - }{ - {"maxt less than head min", head.MaxTime() - 10, head.MinTime() - 10, []string{}, []string{}}, - {"mint less than head max", head.MaxTime() + 10, head.MinTime() + 10, []string{}, []string{}}, - {"mint and maxt outside head", head.MaxTime() + 10, head.MinTime() - 10, []string{}, []string{}}, - {"mint and maxt within head", head.MaxTime() - 10, head.MinTime() + 10, expectedLabelNames, expectedLabelValues}, - } - - for _, tt := range testCases { - t.Run(tt.name, func(t *testing.T) { - headIdxReader := head.indexRange(tt.mint, tt.maxt) - actualLabelNames, err := headIdxReader.LabelNames(ctx) - require.NoError(t, err) - require.Equal(t, tt.expectedNames, actualLabelNames) - if len(tt.expectedValues) > 0 { - for i, name := range expectedLabelNames { - actualLabelValue, err := headIdxReader.SortedLabelValues(ctx, name, nil) - require.NoError(t, err) - require.Equal(t, []string{tt.expectedValues[i]}, actualLabelValue) - } - } +func TestHeadAppenderV2_Append_OutOfOrderSamplesMetric(t *testing.T) { + t.Parallel() + for name, scenario := range sampleTypeScenarios { + t.Run(name, func(t *testing.T) { + options := DefaultOptions() + testHeadAppenderV2OutOfOrderSamplesMetric(t, scenario, options, storage.ErrOutOfOrderSample) }) } } -func TestHeadLabelValuesWithMatchers(t *testing.T) { - head, _ := newTestHead(t, 1000, compression.None, false) - t.Cleanup(func() { require.NoError(t, head.Close()) }) - - ctx := context.Background() - - app := head.Appender(context.Background()) - for i := range 100 { - _, err := app.Append(0, labels.FromStrings( - "tens", fmt.Sprintf("value%d", i/10), - "unique", fmt.Sprintf("value%d", i), - ), 100, 0) - require.NoError(t, err) - } - require.NoError(t, app.Commit()) - - var uniqueWithout30s []string - for i := range 100 { - if i/10 != 3 { - uniqueWithout30s = append(uniqueWithout30s, fmt.Sprintf("value%d", i)) +func TestHeadAppenderV2_Append_OutOfOrderSamplesMetricNativeHistogramOOODisabled(t *testing.T) { + for name, scenario := range sampleTypeScenarios { + if scenario.sampleType != "histogram" { + continue } - } - sort.Strings(uniqueWithout30s) - testCases := []struct { - name string - labelName string - matchers []*labels.Matcher - expectedValues []string - }{ - { - name: "get tens based on unique id", - labelName: "tens", - matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "unique", "value35")}, - expectedValues: []string{"value3"}, - }, { - name: "get unique ids based on a ten", - labelName: "unique", - matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "tens", "value1")}, - expectedValues: []string{"value10", "value11", "value12", "value13", "value14", "value15", "value16", "value17", "value18", "value19"}, - }, { - name: "get tens by pattern matching on unique id", - labelName: "tens", - matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchRegexp, "unique", "value[5-7]5")}, - expectedValues: []string{"value5", "value6", "value7"}, - }, { - name: "get tens by matching for presence of unique label", - labelName: "tens", - matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchNotEqual, "unique", "")}, - expectedValues: []string{"value0", "value1", "value2", "value3", "value4", "value5", "value6", "value7", "value8", "value9"}, - }, { - name: "get unique IDs based on tens not being equal to a certain value, while not empty", - labelName: "unique", - matchers: []*labels.Matcher{ - labels.MustNewMatcher(labels.MatchNotEqual, "tens", "value3"), - labels.MustNewMatcher(labels.MatchNotEqual, "tens", ""), - }, - expectedValues: uniqueWithout30s, - }, - } - - for _, tt := range testCases { - t.Run(tt.name, func(t *testing.T) { - headIdxReader := head.indexRange(0, 200) - - actualValues, err := headIdxReader.SortedLabelValues(ctx, tt.labelName, nil, tt.matchers...) - require.NoError(t, err) - require.Equal(t, tt.expectedValues, actualValues) - - actualValues, err = headIdxReader.LabelValues(ctx, tt.labelName, nil, tt.matchers...) - sort.Strings(actualValues) - require.NoError(t, err) - require.Equal(t, tt.expectedValues, actualValues) + t.Run(name, func(t *testing.T) { + options := DefaultOptions() + options.OutOfOrderTimeWindow = 0 + testHeadAppenderV2OutOfOrderSamplesMetric(t, scenario, options, storage.ErrOutOfOrderSample) }) } } -func TestHeadLabelNamesWithMatchers(t *testing.T) { +func TestHeadAppenderV2_ErrReuse(t *testing.T) { head, _ := newTestHead(t, 1000, compression.None, false) defer func() { require.NoError(t, head.Close()) }() - app := head.Appender(context.Background()) - for i := range 100 { - _, err := app.Append(0, labels.FromStrings( - "unique", fmt.Sprintf("value%d", i), - ), 100, 0) - require.NoError(t, err) - - if i%10 == 0 { - _, err := app.Append(0, labels.FromStrings( - "tens", fmt.Sprintf("value%d", i/10), - "unique", fmt.Sprintf("value%d", i), - ), 100, 0) - require.NoError(t, err) - } - - if i%20 == 0 { - _, err := app.Append(0, labels.FromStrings( - "tens", fmt.Sprintf("value%d", i/10), - "twenties", fmt.Sprintf("value%d", i/20), - "unique", fmt.Sprintf("value%d", i), - ), 100, 0) - require.NoError(t, err) - } - } - require.NoError(t, app.Commit()) - - testCases := []struct { - name string - labelName string - matchers []*labels.Matcher - expectedNames []string - }{ - { - name: "get with non-empty unique: all", - matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchNotEqual, "unique", "")}, - expectedNames: []string{"tens", "twenties", "unique"}, - }, { - name: "get with unique ending in 1: only unique", - matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchRegexp, "unique", "value.*1")}, - expectedNames: []string{"unique"}, - }, { - name: "get with unique = value20: all", - matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "unique", "value20")}, - expectedNames: []string{"tens", "twenties", "unique"}, - }, { - name: "get tens = 1: unique & tens", - matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "tens", "value1")}, - expectedNames: []string{"tens", "unique"}, - }, - } - - for _, tt := range testCases { - t.Run(tt.name, func(t *testing.T) { - headIdxReader := head.indexRange(0, 200) - - actualNames, err := headIdxReader.LabelNames(context.Background(), tt.matchers...) - require.NoError(t, err) - require.Equal(t, tt.expectedNames, actualNames) - }) - } -} - -func TestHeadShardedPostings(t *testing.T) { - headOpts := newTestHeadDefaultOptions(1000, false) - headOpts.EnableSharding = true - head, _ := newTestHeadWithOptions(t, compression.None, headOpts) - defer func() { - require.NoError(t, head.Close()) - }() - - ctx := context.Background() - - // Append some series. - app := head.Appender(ctx) - for i := range 100 { - _, err := app.Append(0, labels.FromStrings("unique", fmt.Sprintf("value%d", i), "const", "1"), 100, 0) - require.NoError(t, err) - } - require.NoError(t, app.Commit()) - - ir := head.indexRange(0, 200) - - // List all postings for a given label value. This is what we expect to get - // in output from all shards. - p, err := ir.Postings(ctx, "const", "1") - require.NoError(t, err) - - var expected []storage.SeriesRef - for p.Next() { - expected = append(expected, p.At()) - } - require.NoError(t, p.Err()) - require.NotEmpty(t, expected) - - // Query the same postings for each shard. - const shardCount = uint64(4) - actualShards := make(map[uint64][]storage.SeriesRef) - actualPostings := make([]storage.SeriesRef, 0, len(expected)) - - for shardIndex := range shardCount { - p, err = ir.Postings(ctx, "const", "1") - require.NoError(t, err) - - p = ir.ShardedPostings(p, shardIndex, shardCount) - for p.Next() { - ref := p.At() - - actualShards[shardIndex] = append(actualShards[shardIndex], ref) - actualPostings = append(actualPostings, ref) - } - require.NoError(t, p.Err()) - } - - // We expect the postings merged out of shards is the exact same of the non sharded ones. - require.ElementsMatch(t, expected, actualPostings) - - // We expect the series in each shard are the expected ones. - for shardIndex, ids := range actualShards { - for _, id := range ids { - var lbls labels.ScratchBuilder - - require.NoError(t, ir.Series(id, &lbls, nil)) - require.Equal(t, shardIndex, labels.StableHash(lbls.Labels())%shardCount) - } - } -} - -func TestErrReuseAppender(t *testing.T) { - head, _ := newTestHead(t, 1000, compression.None, false) - defer func() { - require.NoError(t, head.Close()) - }() - - app := head.Appender(context.Background()) - _, err := app.Append(0, labels.FromStrings("test", "test"), 0, 0) + app := head.AppenderV2(context.Background()) + _, err := app.Append(0, labels.FromStrings("test", "test"), 0, 0, 0, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) require.Error(t, app.Commit()) require.Error(t, app.Rollback()) - app = head.Appender(context.Background()) - _, err = app.Append(0, labels.FromStrings("test", "test"), 1, 0) + app = head.AppenderV2(context.Background()) + _, err = app.Append(0, labels.FromStrings("test", "test"), 0, 1, 0, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Rollback()) require.Error(t, app.Rollback()) require.Error(t, app.Commit()) - app = head.Appender(context.Background()) - _, err = app.Append(0, labels.FromStrings("test", "test"), 2, 0) + app = head.AppenderV2(context.Background()) + _, err = app.Append(0, labels.FromStrings("test", "test"), 0, 2, 0, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) require.Error(t, app.Rollback()) require.Error(t, app.Commit()) - app = head.Appender(context.Background()) - _, err = app.Append(0, labels.FromStrings("test", "test"), 3, 0) + app = head.AppenderV2(context.Background()) + _, err = app.Append(0, labels.FromStrings("test", "test"), 0, 3, 0, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Rollback()) require.Error(t, app.Commit()) require.Error(t, app.Rollback()) } -func TestHeadMintAfterTruncation(t *testing.T) { +func TestHeadAppenderV2_MinTimeAfterTruncation(t *testing.T) { chunkRange := int64(2000) head, _ := newTestHead(t, chunkRange, compression.None, false) - app := head.Appender(context.Background()) - _, err := app.Append(0, labels.FromStrings("a", "b"), 100, 100) + app := head.AppenderV2(context.Background()) + _, err := app.Append(0, labels.FromStrings("a", "b"), 0, 100, 100, nil, nil, storage.AOptions{}) require.NoError(t, err) - _, err = app.Append(0, labels.FromStrings("a", "b"), 4000, 200) + _, err = app.Append(0, labels.FromStrings("a", "b"), 0, 4000, 200, nil, nil, storage.AOptions{}) require.NoError(t, err) - _, err = app.Append(0, labels.FromStrings("a", "b"), 8000, 300) + _, err = app.Append(0, labels.FromStrings("a", "b"), 0, 8000, 300, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) @@ -3672,202 +1411,33 @@ func TestHeadMintAfterTruncation(t *testing.T) { require.NoError(t, head.Close()) } -func TestHeadExemplars(t *testing.T) { +func TestHeadAppenderV2_AppendExemplars(t *testing.T) { chunkRange := int64(2000) head, _ := newTestHead(t, chunkRange, compression.None, false) - app := head.Appender(context.Background()) + app := head.AppenderV2(context.Background()) l := labels.FromStrings("trace_id", "123") + // It is perfectly valid to add Exemplars before the current start time - // histogram buckets that haven't been update in a while could still be // exported exemplars from an hour ago. - ref, err := app.Append(0, labels.FromStrings("a", "b"), 100, 100) - require.NoError(t, err) - _, err = app.AppendExemplar(ref, l, exemplar.Exemplar{ - Labels: l, - HasTs: true, - Ts: -1000, - Value: 1, + _, err := app.Append(0, labels.FromStrings("a", "b"), 0, 100, 100, nil, nil, storage.AOptions{ + Exemplars: []exemplar.Exemplar{{Labels: l, HasTs: true, Ts: -1000, Value: 1}}, }) require.NoError(t, err) require.NoError(t, app.Commit()) require.NoError(t, head.Close()) } -func BenchmarkHeadLabelValuesWithMatchers(b *testing.B) { - chunkRange := int64(2000) - head, _ := newTestHead(b, chunkRange, compression.None, false) - b.Cleanup(func() { require.NoError(b, head.Close()) }) - - ctx := context.Background() - - app := head.Appender(context.Background()) - - metricCount := 1000000 - for i := range metricCount { - _, err := app.Append(0, labels.FromStrings( - "a_unique", fmt.Sprintf("value%d", i), - "b_tens", fmt.Sprintf("value%d", i/(metricCount/10)), - "c_ninety", fmt.Sprintf("value%d", i/(metricCount/10)/9), // "0" for the first 90%, then "1" - ), 100, 0) - require.NoError(b, err) - } - require.NoError(b, app.Commit()) - - headIdxReader := head.indexRange(0, 200) - matchers := []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "c_ninety", "value0")} - - b.ReportAllocs() - - for b.Loop() { - actualValues, err := headIdxReader.LabelValues(ctx, "b_tens", nil, matchers...) - require.NoError(b, err) - require.Len(b, actualValues, 9) - } -} - -func TestIteratorSeekIntoBuffer(t *testing.T) { - dir := t.TempDir() - // This is usually taken from the Head, but passing manually here. - chunkDiskMapper, err := chunks.NewChunkDiskMapper(nil, dir, chunkenc.NewPool(), chunks.DefaultWriteBufferSize, chunks.DefaultWriteQueueSize) - require.NoError(t, err) - defer func() { - require.NoError(t, chunkDiskMapper.Close()) - }() - cOpts := chunkOpts{ - chunkDiskMapper: chunkDiskMapper, - chunkRange: 500, - samplesPerChunk: DefaultSamplesPerChunk, - } - - s := newMemSeries(labels.Labels{}, 1, 0, defaultIsolationDisabled, false) - - for i := range 7 { - ok, _ := s.append(int64(i), float64(i), 0, cOpts) - require.True(t, ok, "sample append failed") - } - - c, _, _, err := s.chunk(0, chunkDiskMapper, &sync.Pool{ - New: func() any { - return &memChunk{} - }, - }) - require.NoError(t, err) - it := c.chunk.Iterator(nil) - - // First point. - require.Equal(t, chunkenc.ValFloat, it.Seek(0)) - ts, val := it.At() - require.Equal(t, int64(0), ts) - require.Equal(t, float64(0), val) - - // Advance one point. - require.Equal(t, chunkenc.ValFloat, it.Next()) - ts, val = it.At() - require.Equal(t, int64(1), ts) - require.Equal(t, float64(1), val) - - // Seeking an older timestamp shouldn't cause the iterator to go backwards. - require.Equal(t, chunkenc.ValFloat, it.Seek(0)) - ts, val = it.At() - require.Equal(t, int64(1), ts) - require.Equal(t, float64(1), val) - - // Seek into the buffer. - require.Equal(t, chunkenc.ValFloat, it.Seek(3)) - ts, val = it.At() - require.Equal(t, int64(3), ts) - require.Equal(t, float64(3), val) - - // Iterate through the rest of the buffer. - for i := 4; i < 7; i++ { - require.Equal(t, chunkenc.ValFloat, it.Next()) - ts, val = it.At() - require.Equal(t, int64(i), ts) - require.Equal(t, float64(i), val) - } - - // Run out of elements in the iterator. - require.Equal(t, chunkenc.ValNone, it.Next()) - require.Equal(t, chunkenc.ValNone, it.Seek(7)) -} - -// Tests https://github.com/prometheus/prometheus/issues/8221. -func TestChunkNotFoundHeadGCRace(t *testing.T) { - t.Parallel() - db := newTestDB(t) - db.DisableCompactions() - ctx := context.Background() - - var ( - app = db.Appender(context.Background()) - ref = storage.SeriesRef(0) - mint, maxt = int64(0), int64(0) - err error - ) - - // Appends samples to span over 1.5 block ranges. - // 7 chunks with 15s scrape interval. - for i := int64(0); i <= 120*7; i++ { - ts := i * DefaultBlockDuration / (4 * 120) - ref, err = app.Append(ref, labels.FromStrings("a", "b"), ts, float64(i)) - require.NoError(t, err) - maxt = ts - } - require.NoError(t, app.Commit()) - - // Get a querier before compaction (or when compaction is about to begin). - q, err := db.Querier(mint, maxt) - require.NoError(t, err) - - // Query the compacted range and get the first series before compaction. - ss := q.Select(context.Background(), true, nil, labels.MustNewMatcher(labels.MatchEqual, "a", "b")) - require.True(t, ss.Next()) - s := ss.At() - - var wg sync.WaitGroup - wg.Add(1) - go func() { - defer wg.Done() - // Compacting head while the querier spans the compaction time. - require.NoError(t, db.Compact(ctx)) - require.NotEmpty(t, db.Blocks()) - }() - - // Give enough time for compaction to finish. - // We expect it to be blocked until querier is closed. - <-time.After(3 * time.Second) - - // Now consume after compaction when it's gone. - it := s.Iterator(nil) - for it.Next() == chunkenc.ValFloat { - _, _ = it.At() - } - // It should error here without any fix for the mentioned issue. - require.NoError(t, it.Err()) - for ss.Next() { - s = ss.At() - it = s.Iterator(it) - for it.Next() == chunkenc.ValFloat { - _, _ = it.At() - } - require.NoError(t, it.Err()) - } - require.NoError(t, ss.Err()) - - require.NoError(t, q.Close()) - wg.Wait() -} - // Tests https://github.com/prometheus/prometheus/issues/9079. -func TestDataMissingOnQueryDuringCompaction(t *testing.T) { +func TestDataMissingOnQueryDuringCompaction_AppenderV2(t *testing.T) { t.Parallel() db := newTestDB(t) db.DisableCompactions() ctx := context.Background() var ( - app = db.Appender(context.Background()) + app = db.AppenderV2(context.Background()) ref = storage.SeriesRef(0) mint, maxt = int64(0), int64(0) err error @@ -3878,7 +1448,7 @@ func TestDataMissingOnQueryDuringCompaction(t *testing.T) { // 7 chunks with 15s scrape interval. for i := int64(0); i <= 120*7; i++ { ts := i * DefaultBlockDuration / (4 * 120) - ref, err = app.Append(ref, labels.FromStrings("a", "b"), ts, float64(i)) + ref, err = app.Append(ref, labels.FromStrings("a", "b"), 0, ts, float64(i), nil, nil, storage.AOptions{}) require.NoError(t, err) maxt = ts expSamples = append(expSamples, sample{ts, float64(i), nil, nil}) @@ -3909,18 +1479,18 @@ func TestDataMissingOnQueryDuringCompaction(t *testing.T) { wg.Wait() } -func TestIsQuerierCollidingWithTruncation(t *testing.T) { +func TestIsQuerierCollidingWithTruncation_AppenderV2(t *testing.T) { db := newTestDB(t) db.DisableCompactions() var ( - app = db.Appender(context.Background()) + app = db.AppenderV2(context.Background()) ref = storage.SeriesRef(0) err error ) for i := int64(0); i <= 3000; i++ { - ref, err = app.Append(ref, labels.FromStrings("a", "b"), i, float64(i)) + ref, err = app.Append(ref, labels.FromStrings("a", "b"), 0, i, float64(i), nil, nil, storage.AOptions{}) require.NoError(t, err) } require.NoError(t, app.Commit()) @@ -3954,7 +1524,7 @@ func TestIsQuerierCollidingWithTruncation(t *testing.T) { } } -func TestWaitForPendingReadersInTimeRange(t *testing.T) { +func TestWaitForPendingReadersInTimeRange_AppenderV2(t *testing.T) { t.Parallel() db := newTestDB(t) db.DisableCompactions() @@ -3962,14 +1532,14 @@ func TestWaitForPendingReadersInTimeRange(t *testing.T) { sampleTs := func(i int64) int64 { return i * DefaultBlockDuration / (4 * 120) } var ( - app = db.Appender(context.Background()) + app = db.AppenderV2(context.Background()) ref = storage.SeriesRef(0) err error ) for i := int64(0); i <= 3000; i++ { ts := sampleTs(i) - ref, err = app.Append(ref, labels.FromStrings("a", "b"), ts, float64(i)) + ref, err = app.Append(ref, labels.FromStrings("a", "b"), 0, ts, float64(i), nil, nil, storage.AOptions{}) require.NoError(t, err) } require.NoError(t, app.Commit()) @@ -4012,31 +1582,8 @@ func TestWaitForPendingReadersInTimeRange(t *testing.T) { } } -func TestQueryOOOHeadDuringTruncate(t *testing.T) { - testQueryOOOHeadDuringTruncate(t, - func(db *DB, minT, maxT int64) (storage.LabelQuerier, error) { - return db.Querier(minT, maxT) - }, - func(t *testing.T, lq storage.LabelQuerier, minT, _ int64) { - // Samples - q, ok := lq.(storage.Querier) - require.True(t, ok) - ss := q.Select(context.Background(), false, nil, labels.MustNewMatcher(labels.MatchEqual, "a", "b")) - require.True(t, ss.Next()) - s := ss.At() - require.False(t, ss.Next()) // One series. - it := s.Iterator(nil) - require.NotEqual(t, chunkenc.ValNone, it.Next()) // Has some data. - require.Equal(t, minT, it.AtT()) // It is an in-order sample. - require.NotEqual(t, chunkenc.ValNone, it.Next()) // Has some data. - require.Equal(t, minT+50, it.AtT()) // it is an out-of-order sample. - require.NoError(t, it.Err()) - }, - ) -} - -func TestChunkQueryOOOHeadDuringTruncate(t *testing.T) { - testQueryOOOHeadDuringTruncate(t, +func TestChunkQueryOOOHeadDuringTruncate_AppenderV2(t *testing.T) { + testQueryOOOHeadDuringTruncateAppenderV2(t, func(db *DB, minT, maxT int64) (storage.LabelQuerier, error) { return db.ChunkQuerier(minT, maxT) }, @@ -4062,7 +1609,7 @@ func TestChunkQueryOOOHeadDuringTruncate(t *testing.T) { ) } -func testQueryOOOHeadDuringTruncate(t *testing.T, makeQuerier func(db *DB, minT, maxT int64) (storage.LabelQuerier, error), verify func(t *testing.T, q storage.LabelQuerier, minT, maxT int64)) { +func testQueryOOOHeadDuringTruncateAppenderV2(t *testing.T, makeQuerier func(db *DB, minT, maxT int64) (storage.LabelQuerier, error), verify func(t *testing.T, q storage.LabelQuerier, minT, maxT int64)) { const maxT int64 = 6000 dir := t.TempDir() @@ -4079,16 +1626,16 @@ func testQueryOOOHeadDuringTruncate(t *testing.T, makeQuerier func(db *DB, minT, var ( ref = storage.SeriesRef(0) - app = db.Appender(context.Background()) + app = db.AppenderV2(context.Background()) ) // Add in-order samples at every 100ms starting at 0ms. for i := int64(0); i < maxT; i += 100 { - _, err := app.Append(ref, labels.FromStrings("a", "b"), i, 0) + _, err := app.Append(ref, labels.FromStrings("a", "b"), 0, i, 0, nil, nil, storage.AOptions{}) require.NoError(t, err) } // Add out-of-order samples at every 100ms starting at 50ms. for i := int64(50); i < maxT; i += 100 { - _, err := app.Append(ref, labels.FromStrings("a", "b"), i, 0) + _, err := app.Append(ref, labels.FromStrings("a", "b"), 0, i, 0, nil, nil, storage.AOptions{}) require.NoError(t, err) } require.NoError(t, app.Commit()) @@ -4138,7 +1685,7 @@ func testQueryOOOHeadDuringTruncate(t *testing.T, makeQuerier func(db *DB, minT, <-compactionFinished // Wait for compaction otherwise Go test finds stray goroutines. } -func TestAppendHistogram(t *testing.T) { +func TestHeadAppenderV2_Append_Histogram(t *testing.T) { l := labels.FromStrings("a", "b") for _, numHistograms := range []int{1, 10, 150, 200, 250, 300} { t.Run(strconv.Itoa(numHistograms), func(t *testing.T) { @@ -4149,31 +1696,31 @@ func TestAppendHistogram(t *testing.T) { require.NoError(t, head.Init(0)) ingestTs := int64(0) - app := head.Appender(context.Background()) + app := head.AppenderV2(context.Background()) expHistograms := make([]chunks.Sample, 0, 2*numHistograms) // Counter integer histograms. for _, h := range tsdbutil.GenerateTestHistograms(numHistograms) { - _, err := app.AppendHistogram(0, l, ingestTs, h, nil) + _, err := app.Append(0, l, 0, ingestTs, 0, h, nil, storage.AOptions{}) require.NoError(t, err) expHistograms = append(expHistograms, sample{t: ingestTs, h: h}) ingestTs++ if ingestTs%50 == 0 { require.NoError(t, app.Commit()) - app = head.Appender(context.Background()) + app = head.AppenderV2(context.Background()) } } // Gauge integer histograms. for _, h := range tsdbutil.GenerateTestGaugeHistograms(numHistograms) { - _, err := app.AppendHistogram(0, l, ingestTs, h, nil) + _, err := app.Append(0, l, 0, ingestTs, 0, h, nil, storage.AOptions{}) require.NoError(t, err) expHistograms = append(expHistograms, sample{t: ingestTs, h: h}) ingestTs++ if ingestTs%50 == 0 { require.NoError(t, app.Commit()) - app = head.Appender(context.Background()) + app = head.AppenderV2(context.Background()) } } @@ -4181,25 +1728,25 @@ func TestAppendHistogram(t *testing.T) { // Counter float histograms. for _, fh := range tsdbutil.GenerateTestFloatHistograms(numHistograms) { - _, err := app.AppendHistogram(0, l, ingestTs, nil, fh) + _, err := app.Append(0, l, 0, ingestTs, 0, nil, fh, storage.AOptions{}) require.NoError(t, err) expFloatHistograms = append(expFloatHistograms, sample{t: ingestTs, fh: fh}) ingestTs++ if ingestTs%50 == 0 { require.NoError(t, app.Commit()) - app = head.Appender(context.Background()) + app = head.AppenderV2(context.Background()) } } // Gauge float histograms. for _, fh := range tsdbutil.GenerateTestGaugeFloatHistograms(numHistograms) { - _, err := app.AppendHistogram(0, l, ingestTs, nil, fh) + _, err := app.Append(0, l, 0, ingestTs, 0, nil, fh, storage.AOptions{}) require.NoError(t, err) expFloatHistograms = append(expFloatHistograms, sample{t: ingestTs, fh: fh}) ingestTs++ if ingestTs%50 == 0 { require.NoError(t, app.Commit()) - app = head.Appender(context.Background()) + app = head.AppenderV2(context.Background()) } } @@ -4245,7 +1792,7 @@ func TestAppendHistogram(t *testing.T) { } } -func TestHistogramInWALAndMmapChunk(t *testing.T) { +func TestHistogramInWALAndMmapChunk_AppenderV2(t *testing.T) { head, _ := newTestHead(t, 3000, compression.None, false) t.Cleanup(func() { require.NoError(t, head.Close()) @@ -4258,9 +1805,9 @@ func TestHistogramInWALAndMmapChunk(t *testing.T) { numHistograms := 300 exp := map[string][]chunks.Sample{} ts := int64(0) - var app storage.Appender + var app storage.AppenderV2 for _, gauge := range []bool{true, false} { - app = head.Appender(context.Background()) + app = head.AppenderV2(context.Background()) var hists []*histogram.Histogram if gauge { hists = tsdbutil.GenerateTestGaugeHistograms(numHistograms) @@ -4270,19 +1817,19 @@ func TestHistogramInWALAndMmapChunk(t *testing.T) { for _, h := range hists { h.NegativeSpans = h.PositiveSpans h.NegativeBuckets = h.PositiveBuckets - _, err := app.AppendHistogram(0, s1, ts, h, nil) + _, err := app.Append(0, s1, 0, ts, 0, h, nil, storage.AOptions{}) require.NoError(t, err) exp[k1] = append(exp[k1], sample{t: ts, h: h.Copy()}) ts++ if ts%5 == 0 { require.NoError(t, app.Commit()) - app = head.Appender(context.Background()) + app = head.AppenderV2(context.Background()) } } require.NoError(t, app.Commit()) } for _, gauge := range []bool{true, false} { - app = head.Appender(context.Background()) + app = head.AppenderV2(context.Background()) var hists []*histogram.FloatHistogram if gauge { hists = tsdbutil.GenerateTestGaugeFloatHistograms(numHistograms) @@ -4292,13 +1839,13 @@ func TestHistogramInWALAndMmapChunk(t *testing.T) { for _, h := range hists { h.NegativeSpans = h.PositiveSpans h.NegativeBuckets = h.PositiveBuckets - _, err := app.AppendHistogram(0, s1, ts, nil, h) + _, err := app.Append(0, s1, 0, ts, 0, nil, h, storage.AOptions{}) require.NoError(t, err) exp[k1] = append(exp[k1], sample{t: ts, fh: h.Copy()}) ts++ if ts%5 == 0 { require.NoError(t, app.Commit()) - app = head.Appender(context.Background()) + app = head.AppenderV2(context.Background()) } } require.NoError(t, app.Commit()) @@ -4322,7 +1869,7 @@ func TestHistogramInWALAndMmapChunk(t *testing.T) { k2 := s2.String() ts = 0 for _, gauge := range []bool{true, false} { - app = head.Appender(context.Background()) + app = head.AppenderV2(context.Background()) var hists []*histogram.Histogram if gauge { hists = tsdbutil.GenerateTestGaugeHistograms(100) @@ -4333,7 +1880,7 @@ func TestHistogramInWALAndMmapChunk(t *testing.T) { ts++ h.NegativeSpans = h.PositiveSpans h.NegativeBuckets = h.PositiveBuckets - _, err := app.AppendHistogram(0, s2, ts, h, nil) + _, err := app.Append(0, s2, 0, ts, 0, h, nil, storage.AOptions{}) require.NoError(t, err) eh := h.Copy() if !gauge && ts > 30 && (ts-10)%20 == 1 { @@ -4343,22 +1890,22 @@ func TestHistogramInWALAndMmapChunk(t *testing.T) { exp[k2] = append(exp[k2], sample{t: ts, h: eh}) if ts%20 == 0 { require.NoError(t, app.Commit()) - app = head.Appender(context.Background()) + app = head.AppenderV2(context.Background()) // Add some float. for range 10 { ts++ - _, err := app.Append(0, s2, ts, float64(ts)) + _, err := app.Append(0, s2, 0, ts, float64(ts), nil, nil, storage.AOptions{}) require.NoError(t, err) exp[k2] = append(exp[k2], sample{t: ts, f: float64(ts)}) } require.NoError(t, app.Commit()) - app = head.Appender(context.Background()) + app = head.AppenderV2(context.Background()) } } require.NoError(t, app.Commit()) } for _, gauge := range []bool{true, false} { - app = head.Appender(context.Background()) + app = head.AppenderV2(context.Background()) var hists []*histogram.FloatHistogram if gauge { hists = tsdbutil.GenerateTestGaugeFloatHistograms(100) @@ -4369,7 +1916,7 @@ func TestHistogramInWALAndMmapChunk(t *testing.T) { ts++ h.NegativeSpans = h.PositiveSpans h.NegativeBuckets = h.PositiveBuckets - _, err := app.AppendHistogram(0, s2, ts, nil, h) + _, err := app.Append(0, s2, 0, ts, 0, nil, h, storage.AOptions{}) require.NoError(t, err) eh := h.Copy() if !gauge && ts > 30 && (ts-10)%20 == 1 { @@ -4379,16 +1926,16 @@ func TestHistogramInWALAndMmapChunk(t *testing.T) { exp[k2] = append(exp[k2], sample{t: ts, fh: eh}) if ts%20 == 0 { require.NoError(t, app.Commit()) - app = head.Appender(context.Background()) + app = head.AppenderV2(context.Background()) // Add some float. for range 10 { ts++ - _, err := app.Append(0, s2, ts, float64(ts)) + _, err := app.Append(0, s2, 0, ts, float64(ts), nil, nil, storage.AOptions{}) require.NoError(t, err) exp[k2] = append(exp[k2], sample{t: ts, f: float64(ts)}) } require.NoError(t, app.Commit()) - app = head.Appender(context.Background()) + app = head.AppenderV2(context.Background()) } } require.NoError(t, app.Commit()) @@ -4425,7 +1972,7 @@ func TestHistogramInWALAndMmapChunk(t *testing.T) { testQuery() } -func TestChunkSnapshot(t *testing.T) { +func TestChunkSnapshot_AppenderV2(t *testing.T) { head, _ := newTestHead(t, 120*4, compression.None, false) defer func() { head.opts.EnableMemorySnapshotOnShutdown = false @@ -4446,7 +1993,7 @@ func TestChunkSnapshot(t *testing.T) { histograms := tsdbutil.GenerateTestGaugeHistograms(481) floatHistogram := tsdbutil.GenerateTestGaugeFloatHistograms(481) - addExemplar := func(app storage.Appender, ref storage.SeriesRef, lbls labels.Labels, ts int64) { + newExemplar := func(lbls labels.Labels, ts int64) exemplar.Exemplar { e := ex{ seriesLabels: lbls, e: exemplar.Exemplar{ @@ -4456,8 +2003,7 @@ func TestChunkSnapshot(t *testing.T) { }, } expExemplars = append(expExemplars, e) - _, err := app.AppendExemplar(ref, e.seriesLabels, e.e) - require.NoError(t, err) + return e.e } checkSamples := func() { @@ -4534,7 +2080,7 @@ func TestChunkSnapshot(t *testing.T) { { // Initial data that goes into snapshot. // Add some initial samples with >=1 m-map chunk. - app := head.Appender(context.Background()) + app := head.AppenderV2(context.Background()) for i := 1; i <= numSeries; i++ { lbls := labels.FromStrings("foo", fmt.Sprintf("bar%d", i)) lblStr := lbls.String() @@ -4545,26 +2091,30 @@ func TestChunkSnapshot(t *testing.T) { // 240 samples should m-map at least 1 chunk. for ts := int64(1); ts <= 240; ts++ { + // Add an exemplar, but only to float sample. + aOpts := storage.AOptions{} + if ts%10 == 0 { + aOpts.Exemplars = []exemplar.Exemplar{newExemplar(lbls, ts)} + } val := rand.Float64() expSeries[lblStr] = append(expSeries[lblStr], sample{ts, val, nil, nil}) - ref, err := app.Append(0, lbls, ts, val) + _, err := app.Append(0, lbls, 0, ts, val, nil, nil, aOpts) require.NoError(t, err) hist := histograms[int(ts)] expHist[lblsHistStr] = append(expHist[lblsHistStr], sample{ts, 0, hist, nil}) - _, err = app.AppendHistogram(0, lblsHist, ts, hist, nil) + _, err = app.Append(0, lblsHist, 0, ts, 0, hist, nil, storage.AOptions{}) require.NoError(t, err) floatHist := floatHistogram[int(ts)] expFloatHist[lblsFloatHistStr] = append(expFloatHist[lblsFloatHistStr], sample{ts, 0, nil, floatHist}) - _, err = app.AppendHistogram(0, lblsFloatHist, ts, nil, floatHist) + _, err = app.Append(0, lblsFloatHist, 0, ts, 0, nil, floatHist, storage.AOptions{}) require.NoError(t, err) - // Add an exemplar and to create multiple WAL records. + // Create multiple WAL records (commit). if ts%10 == 0 { - addExemplar(app, ref, lbls, ts) require.NoError(t, app.Commit()) - app = head.Appender(context.Background()) + app = head.AppenderV2(context.Background()) } } } @@ -4608,7 +2158,7 @@ func TestChunkSnapshot(t *testing.T) { { // Additional data to only include in WAL and m-mapped chunks and not snapshot. This mimics having an old snapshot on disk. // Add more samples. - app := head.Appender(context.Background()) + app := head.AppenderV2(context.Background()) for i := 1; i <= numSeries; i++ { lbls := labels.FromStrings("foo", fmt.Sprintf("bar%d", i)) lblStr := lbls.String() @@ -4619,26 +2169,30 @@ func TestChunkSnapshot(t *testing.T) { // 240 samples should m-map at least 1 chunk. for ts := int64(241); ts <= 480; ts++ { + // Add an exemplar, but only to float sample. + aOpts := storage.AOptions{} + if ts%10 == 0 { + aOpts.Exemplars = []exemplar.Exemplar{newExemplar(lbls, ts)} + } val := rand.Float64() expSeries[lblStr] = append(expSeries[lblStr], sample{ts, val, nil, nil}) - ref, err := app.Append(0, lbls, ts, val) + _, err := app.Append(0, lbls, 0, ts, val, nil, nil, aOpts) require.NoError(t, err) hist := histograms[int(ts)] expHist[lblsHistStr] = append(expHist[lblsHistStr], sample{ts, 0, hist, nil}) - _, err = app.AppendHistogram(0, lblsHist, ts, hist, nil) + _, err = app.Append(0, lblsHist, 0, ts, 0, hist, nil, storage.AOptions{}) require.NoError(t, err) floatHist := floatHistogram[int(ts)] expFloatHist[lblsFloatHistStr] = append(expFloatHist[lblsFloatHistStr], sample{ts, 0, nil, floatHist}) - _, err = app.AppendHistogram(0, lblsFloatHist, ts, nil, floatHist) + _, err = app.Append(0, lblsFloatHist, 0, ts, 0, nil, floatHist, storage.AOptions{}) require.NoError(t, err) - // Add an exemplar and to create multiple WAL records. + // Create multiple WAL records (commit). if ts%10 == 0 { - addExemplar(app, ref, lbls, ts) require.NoError(t, app.Commit()) - app = head.Appender(context.Background()) + app = head.AppenderV2(context.Background()) } } } @@ -4712,7 +2266,7 @@ func TestChunkSnapshot(t *testing.T) { } } -func TestSnapshotError(t *testing.T) { +func TestSnapshotError_AppenderV2(t *testing.T) { head, _ := newTestHead(t, 120*4, compression.None, false) defer func() { head.opts.EnableMemorySnapshotOnShutdown = false @@ -4720,9 +2274,9 @@ func TestSnapshotError(t *testing.T) { }() // Add a sample. - app := head.Appender(context.Background()) + app := head.AppenderV2(context.Background()) lbls := labels.FromStrings("foo", "bar") - _, err := app.Append(0, lbls, 99, 99) + _, err := app.Append(0, lbls, 0, 99, 99, nil, nil, storage.AOptions{}) require.NoError(t, err) // Add histograms @@ -4731,10 +2285,10 @@ func TestSnapshotError(t *testing.T) { lblsHist := labels.FromStrings("hist", "bar") lblsFloatHist := labels.FromStrings("floathist", "bar") - _, err = app.AppendHistogram(0, lblsHist, 99, hist, nil) + _, err = app.Append(0, lblsHist, 0, 99, 0, hist, nil, storage.AOptions{}) require.NoError(t, err) - _, err = app.AppendHistogram(0, lblsFloatHist, 99, nil, floatHist) + _, err = app.Append(0, lblsFloatHist, 0, 99, 0, nil, floatHist, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) @@ -4822,7 +2376,7 @@ func TestSnapshotError(t *testing.T) { require.Equal(t, 2.0, prom_testutil.ToFloat64(head.metrics.seriesCreated)) } -func TestHistogramMetrics(t *testing.T) { +func TestHeadAppenderV2_Append_HistogramSamplesAppendedMetric(t *testing.T) { numHistograms := 10 head, _ := newTestHead(t, 1000, compression.None, false) t.Cleanup(func() { @@ -4836,15 +2390,15 @@ func TestHistogramMetrics(t *testing.T) { expHSeries++ l := labels.FromStrings("a", fmt.Sprintf("b%d", x)) for i, h := range tsdbutil.GenerateTestHistograms(numHistograms) { - app := head.Appender(context.Background()) - _, err := app.AppendHistogram(0, l, int64(i), h, nil) + app := head.AppenderV2(context.Background()) + _, err := app.Append(0, l, 0, int64(i), 0, h, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) expHSamples++ } for i, fh := range tsdbutil.GenerateTestFloatHistograms(numHistograms) { - app := head.Appender(context.Background()) - _, err := app.AppendHistogram(0, l, int64(numHistograms+i), nil, fh) + app := head.AppenderV2(context.Background()) + _, err := app.Append(0, l, 0, int64(numHistograms+i), 0, nil, fh, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) expHSamples++ @@ -4863,16 +2417,16 @@ func TestHistogramMetrics(t *testing.T) { require.Equal(t, float64(0), prom_testutil.ToFloat64(head.metrics.samplesAppended.WithLabelValues(sampleMetricTypeHistogram))) // Counter reset. } -func TestHistogramStaleSample(t *testing.T) { +func TestHeadAppenderV2_Append_StaleHistogram(t *testing.T) { t.Run("integer histogram", func(t *testing.T) { - testHistogramStaleSampleHelper(t, false) + testHeadAppenderV2AppendStaleHistogram(t, false) }) t.Run("float histogram", func(t *testing.T) { - testHistogramStaleSampleHelper(t, true) + testHeadAppenderV2AppendStaleHistogram(t, true) }) } -func testHistogramStaleSampleHelper(t *testing.T, floatHistogram bool) { +func testHeadAppenderV2AppendStaleHistogram(t *testing.T, floatHistogram bool) { t.Helper() l := labels.FromStrings("a", "b") numHistograms := 20 @@ -4958,20 +2512,20 @@ func testHistogramStaleSampleHelper(t *testing.T, floatHistogram bool) { } // Adding stale in the same appender. - app := head.Appender(context.Background()) + app := head.AppenderV2(context.Background()) for _, h := range tsdbutil.GenerateTestHistograms(numHistograms) { var err error if floatHistogram { - _, err = app.AppendHistogram(0, l, 100*int64(len(expHistograms)), nil, h.ToFloat(nil)) + _, err = app.Append(0, l, 0, 100*int64(len(expHistograms)), 0, nil, h.ToFloat(nil), storage.AOptions{}) expHistograms = append(expHistograms, timedHistogram{t: 100 * int64(len(expHistograms)), fh: h.ToFloat(nil)}) } else { - _, err = app.AppendHistogram(0, l, 100*int64(len(expHistograms)), h, nil) + _, err = app.Append(0, l, 0, 100*int64(len(expHistograms)), 0, h, nil, storage.AOptions{}) expHistograms = append(expHistograms, timedHistogram{t: 100 * int64(len(expHistograms)), h: h}) } require.NoError(t, err) } // +1 so that delta-of-delta is not 0. - _, err := app.Append(0, l, 100*int64(len(expHistograms))+1, math.Float64frombits(value.StaleNaN)) + _, err := app.Append(0, l, 0, 100*int64(len(expHistograms))+1, math.Float64frombits(value.StaleNaN), nil, nil, storage.AOptions{}) require.NoError(t, err) if floatHistogram { expHistograms = append(expHistograms, timedHistogram{t: 100*int64(len(expHistograms)) + 1, fh: &histogram.FloatHistogram{Sum: math.Float64frombits(value.StaleNaN)}}) @@ -4989,23 +2543,23 @@ func testHistogramStaleSampleHelper(t *testing.T, floatHistogram bool) { testQuery(1) // Adding stale in different appender and continuing series after a stale sample. - app = head.Appender(context.Background()) + app = head.AppenderV2(context.Background()) for _, h := range tsdbutil.GenerateTestHistograms(2 * numHistograms)[numHistograms:] { var err error if floatHistogram { - _, err = app.AppendHistogram(0, l, 100*int64(len(expHistograms)), nil, h.ToFloat(nil)) + _, err = app.Append(0, l, 0, 100*int64(len(expHistograms)), 0, nil, h.ToFloat(nil), storage.AOptions{}) expHistograms = append(expHistograms, timedHistogram{t: 100 * int64(len(expHistograms)), fh: h.ToFloat(nil)}) } else { - _, err = app.AppendHistogram(0, l, 100*int64(len(expHistograms)), h, nil) + _, err = app.Append(0, l, 0, 100*int64(len(expHistograms)), 0, h, nil, storage.AOptions{}) expHistograms = append(expHistograms, timedHistogram{t: 100 * int64(len(expHistograms)), h: h}) } require.NoError(t, err) } require.NoError(t, app.Commit()) - app = head.Appender(context.Background()) + app = head.AppenderV2(context.Background()) // +1 so that delta-of-delta is not 0. - _, err = app.Append(0, l, 100*int64(len(expHistograms))+1, math.Float64frombits(value.StaleNaN)) + _, err = app.Append(0, l, 0, 100*int64(len(expHistograms))+1, math.Float64frombits(value.StaleNaN), nil, nil, storage.AOptions{}) require.NoError(t, err) if floatHistogram { expHistograms = append(expHistograms, timedHistogram{t: 100*int64(len(expHistograms)) + 1, fh: &histogram.FloatHistogram{Sum: math.Float64frombits(value.StaleNaN)}}) @@ -5024,7 +2578,7 @@ func testHistogramStaleSampleHelper(t *testing.T, floatHistogram bool) { testQuery(2) } -func TestHistogramCounterResetHeader(t *testing.T) { +func TestHeadAppenderV2_Append_CounterResetHeader(t *testing.T) { for _, floatHisto := range []bool{true} { // FIXME t.Run(fmt.Sprintf("floatHistogram=%t", floatHisto), func(t *testing.T) { l := labels.FromStrings("a", "b") @@ -5037,12 +2591,12 @@ func TestHistogramCounterResetHeader(t *testing.T) { ts := int64(0) appendHistogram := func(h *histogram.Histogram) { ts++ - app := head.Appender(context.Background()) + app := head.AppenderV2(context.Background()) var err error if floatHisto { - _, err = app.AppendHistogram(0, l, ts, nil, h.ToFloat(nil)) + _, err = app.Append(0, l, 0, ts, 0, nil, h.ToFloat(nil), storage.AOptions{}) } else { - _, err = app.AppendHistogram(0, l, ts, h.Copy(), nil) + _, err = app.Append(0, l, 0, ts, 0, h.Copy(), nil, storage.AOptions{}) } require.NoError(t, err) require.NoError(t, app.Commit()) @@ -5145,7 +2699,7 @@ func TestHistogramCounterResetHeader(t *testing.T) { } } -func TestOOOHistogramCounterResetHeaders(t *testing.T) { +func TestHeadAppenderV2_Append_OOOHistogramCounterResetHeaders(t *testing.T) { for _, floatHisto := range []bool{true, false} { t.Run(fmt.Sprintf("floatHistogram=%t", floatHisto), func(t *testing.T) { l := labels.FromStrings("a", "b") @@ -5158,12 +2712,12 @@ func TestOOOHistogramCounterResetHeaders(t *testing.T) { require.NoError(t, head.Init(0)) appendHistogram := func(ts int64, h *histogram.Histogram) { - app := head.Appender(context.Background()) + app := head.AppenderV2(context.Background()) var err error if floatHisto { - _, err = app.AppendHistogram(0, l, ts, nil, h.ToFloat(nil)) + _, err = app.Append(0, l, 0, ts, 0, nil, h.ToFloat(nil), storage.AOptions{}) } else { - _, err = app.AppendHistogram(0, l, ts, h.Copy(), nil) + _, err = app.Append(0, l, 0, ts, 0, h.Copy(), nil, storage.AOptions{}) } require.NoError(t, err) require.NoError(t, app.Commit()) @@ -5305,7 +2859,7 @@ func TestOOOHistogramCounterResetHeaders(t *testing.T) { } } -func TestAppendingDifferentEncodingToSameSeries(t *testing.T) { +func TestHeadAppenderV2_Append_DifferentEncodingSameSeries(t *testing.T) { dir := t.TempDir() opts := DefaultOptions() db, err := Open(dir, nil, nil, opts, nil) @@ -5398,13 +2952,13 @@ func TestAppendingDifferentEncodingToSameSeries(t *testing.T) { } for _, a := range appends { - app := db.Appender(context.Background()) + app := db.AppenderV2(context.Background()) for _, s := range a.samples { var err error if s.H() != nil || s.FH() != nil { - _, err = app.AppendHistogram(0, lbls, s.T(), s.H(), s.FH()) + _, err = app.Append(0, lbls, 0, s.T(), 0, s.H(), s.FH(), storage.AOptions{}) } else { - _, err = app.Append(0, lbls, s.T(), s.F()) + _, err = app.Append(0, lbls, 0, s.T(), s.F(), nil, nil, storage.AOptions{}) } require.Equal(t, a.err, err) } @@ -5434,75 +2988,7 @@ func TestAppendingDifferentEncodingToSameSeries(t *testing.T) { require.Equal(t, map[string][]chunks.Sample{lbls.String(): expResult}, series) } -// Tests https://github.com/prometheus/prometheus/issues/9725. -func TestChunkSnapshotReplayBug(t *testing.T) { - dir := t.TempDir() - wal, err := wlog.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, compression.Snappy) - require.NoError(t, err) - - // Write few series records and samples such that the series references are not in order in the WAL - // for status_code="200". - var buf []byte - for i := 1; i <= 1000; i++ { - var ref chunks.HeadSeriesRef - if i <= 500 { - ref = chunks.HeadSeriesRef(i * 100) - } else { - ref = chunks.HeadSeriesRef((i - 500) * 50) - } - seriesRec := record.RefSeries{ - Ref: ref, - Labels: labels.FromStrings( - "__name__", "request_duration", - "status_code", "200", - "foo", fmt.Sprintf("baz%d", rand.Int()), - ), - } - // Add a sample so that the series is not garbage collected. - samplesRec := record.RefSample{Ref: ref, T: 1000, V: 1000} - var enc record.Encoder - - rec := enc.Series([]record.RefSeries{seriesRec}, buf) - buf = rec[:0] - require.NoError(t, wal.Log(rec)) - rec = enc.Samples([]record.RefSample{samplesRec}, buf) - buf = rec[:0] - require.NoError(t, wal.Log(rec)) - } - - // Write a corrupt snapshot to fail the replay on startup. - snapshotName := chunkSnapshotDir(0, 100) - cpdir := filepath.Join(dir, snapshotName) - require.NoError(t, os.MkdirAll(cpdir, 0o777)) - - err = os.WriteFile(filepath.Join(cpdir, "00000000"), []byte{1, 5, 3, 5, 6, 7, 4, 2, 2}, 0o777) - require.NoError(t, err) - - opts := DefaultHeadOptions() - opts.ChunkDirRoot = dir - opts.EnableMemorySnapshotOnShutdown = true - head, err := NewHead(nil, nil, wal, nil, opts, nil) - require.NoError(t, err) - require.NoError(t, head.Init(math.MinInt64)) - defer func() { - require.NoError(t, head.Close()) - }() - - // Snapshot replay should error out. - require.Equal(t, 1.0, prom_testutil.ToFloat64(head.metrics.snapshotReplayErrorTotal)) - - // Querying `request_duration{status_code!="200"}` should return no series since all of - // them have status_code="200". - q, err := NewBlockQuerier(head, math.MinInt64, math.MaxInt64) - require.NoError(t, err) - series := query(t, q, - labels.MustNewMatcher(labels.MatchEqual, "__name__", "request_duration"), - labels.MustNewMatcher(labels.MatchNotEqual, "status_code", "200"), - ) - require.Empty(t, series, "there should be no series found") -} - -func TestChunkSnapshotTakenAfterIncompleteSnapshot(t *testing.T) { +func TestChunkSnapshotTakenAfterIncompleteSnapshot_AppenderV2(t *testing.T) { dir := t.TempDir() wlTemp, err := wlog.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, compression.Snappy) require.NoError(t, err) @@ -5522,8 +3008,8 @@ func TestChunkSnapshotTakenAfterIncompleteSnapshot(t *testing.T) { require.Equal(t, 0.0, prom_testutil.ToFloat64(head.metrics.snapshotReplayErrorTotal)) // Add some samples for the snapshot. - app := head.Appender(context.Background()) - _, err = app.Append(0, labels.FromStrings("foo", "bar"), 10, 10) + app := head.AppenderV2(context.Background()) + _, err = app.Append(0, labels.FromStrings("foo", "bar"), 0, 10, 10, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) @@ -5539,15 +3025,15 @@ func TestChunkSnapshotTakenAfterIncompleteSnapshot(t *testing.T) { } // TestWBLReplay checks the replay at a low level. -func TestWBLReplay(t *testing.T) { +func TestWBLReplay_AppenderV2(t *testing.T) { for name, scenario := range sampleTypeScenarios { t.Run(name, func(t *testing.T) { - testWBLReplay(t, scenario) + testWBLReplayAppenderV2(t, scenario) }) } } -func testWBLReplay(t *testing.T, scenario sampleTypeScenario) { +func testWBLReplayAppenderV2(t *testing.T, scenario sampleTypeScenario) { dir := t.TempDir() wal, err := wlog.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, compression.Snappy) require.NoError(t, err) @@ -5566,8 +3052,8 @@ func testWBLReplay(t *testing.T, scenario sampleTypeScenario) { var expOOOSamples []chunks.Sample l := labels.FromStrings("foo", "bar") appendSample := func(mins int64, _ float64, isOOO bool) { - app := h.Appender(context.Background()) - _, s, err := scenario.appendFunc(app, l, mins*time.Minute.Milliseconds(), mins) + app := h.AppenderV2(context.Background()) + _, s, err := scenario.appendFunc(storage.AppenderV2AsLimitedV1(app), l, mins*time.Minute.Milliseconds(), mins) require.NoError(t, err) require.NoError(t, app.Commit()) @@ -5631,15 +3117,15 @@ func testWBLReplay(t *testing.T, scenario sampleTypeScenario) { } // TestOOOMmapReplay checks the replay at a low level. -func TestOOOMmapReplay(t *testing.T) { +func TestOOOMmapReplay_AppenderV2(t *testing.T) { for name, scenario := range sampleTypeScenarios { t.Run(name, func(t *testing.T) { - testOOOMmapReplay(t, scenario) + testOOOMmapReplayAppenderV2(t, scenario) }) } } -func testOOOMmapReplay(t *testing.T, scenario sampleTypeScenario) { +func testOOOMmapReplayAppenderV2(t *testing.T, scenario sampleTypeScenario) { dir := t.TempDir() wal, err := wlog.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, compression.Snappy) require.NoError(t, err) @@ -5658,8 +3144,8 @@ func testOOOMmapReplay(t *testing.T, scenario sampleTypeScenario) { l := labels.FromStrings("foo", "bar") appendSample := func(mins int64) { - app := h.Appender(context.Background()) - _, _, err := scenario.appendFunc(app, l, mins*time.Minute.Milliseconds(), mins) + app := h.AppenderV2(context.Background()) + _, _, err := scenario.appendFunc(storage.AppenderV2AsLimitedV1(app), l, mins*time.Minute.Milliseconds(), mins) require.NoError(t, err) require.NoError(t, app.Commit()) } @@ -5721,7 +3207,7 @@ func testOOOMmapReplay(t *testing.T, scenario sampleTypeScenario) { require.NoError(t, h.Close()) } -func TestHeadInit_DiscardChunksWithUnsupportedEncoding(t *testing.T) { +func TestHead_Init_DiscardChunksWithUnsupportedEncoding(t *testing.T) { h, _ := newTestHead(t, 1000, compression.None, false) defer func() { require.NoError(t, h.Close()) @@ -5730,12 +3216,12 @@ func TestHeadInit_DiscardChunksWithUnsupportedEncoding(t *testing.T) { require.NoError(t, h.Init(0)) ctx := context.Background() - app := h.Appender(ctx) + app := h.AppenderV2(ctx) seriesLabels := labels.FromStrings("a", "1") var seriesRef storage.SeriesRef var err error for i := range 400 { - seriesRef, err = app.Append(0, seriesLabels, int64(i), float64(i)) + seriesRef, err = app.Append(0, seriesLabels, 0, int64(i), float64(i), nil, nil, storage.AOptions{}) require.NoError(t, err) } @@ -5746,9 +3232,9 @@ func TestHeadInit_DiscardChunksWithUnsupportedEncoding(t *testing.T) { // Make this chunk not overlap with the previous and the next h.chunkDiskMapper.WriteChunk(chunks.HeadSeriesRef(seriesRef), 500, 600, uc, false, func(err error) { require.NoError(t, err) }) - app = h.Appender(ctx) + app = h.AppenderV2(ctx) for i := 700; i < 1200; i++ { - _, err := app.Append(0, seriesLabels, int64(i), float64(i)) + _, err := app.Append(0, seriesLabels, 0, int64(i), float64(i), nil, nil, storage.AOptions{}) require.NoError(t, err) } @@ -5780,26 +3266,8 @@ func TestHeadInit_DiscardChunksWithUnsupportedEncoding(t *testing.T) { require.Equal(t, expChunks, series.mmappedChunks) } -const ( - UnsupportedMask = 0b10000000 - EncUnsupportedXOR = chunkenc.EncXOR | UnsupportedMask -) - -// unsupportedChunk holds a XORChunk and overrides the Encoding() method. -type unsupportedChunk struct { - *chunkenc.XORChunk -} - -func newUnsupportedChunk() *unsupportedChunk { - return &unsupportedChunk{chunkenc.NewXORChunk()} -} - -func (*unsupportedChunk) Encoding() chunkenc.Encoding { - return EncUnsupportedXOR -} - // Tests https://github.com/prometheus/prometheus/issues/10277. -func TestMmapPanicAfterMmapReplayCorruption(t *testing.T) { +func TestMmapPanicAfterMmapReplayCorruption_AppenderV2(t *testing.T) { dir := t.TempDir() wal, err := wlog.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, compression.None) require.NoError(t, err) @@ -5819,13 +3287,13 @@ func TestMmapPanicAfterMmapReplayCorruption(t *testing.T) { lbls := labels.FromStrings("__name__", "testing", "foo", "bar") addChunks := func() { interval := DefaultBlockDuration / (4 * 120) - app := h.Appender(context.Background()) + app := h.AppenderV2(context.Background()) for i := range 250 { - ref, err = app.Append(ref, lbls, lastTs, float64(lastTs)) + ref, err = app.Append(ref, lbls, 0, lastTs, float64(lastTs), nil, nil, storage.AOptions{}) lastTs += interval if i%10 == 0 { require.NoError(t, app.Commit()) - app = h.Appender(context.Background()) + app = h.AppenderV2(context.Background()) } } require.NoError(t, app.Commit()) @@ -5854,7 +3322,7 @@ func TestMmapPanicAfterMmapReplayCorruption(t *testing.T) { } // Tests https://github.com/prometheus/prometheus/issues/10277. -func TestReplayAfterMmapReplayError(t *testing.T) { +func TestReplayAfterMmapReplayError_AppenderV2(t *testing.T) { dir := t.TempDir() var h *Head var err error @@ -5881,16 +3349,16 @@ func TestReplayAfterMmapReplayError(t *testing.T) { lbls := labels.FromStrings("__name__", "testing", "foo", "bar") var expSamples []chunks.Sample addSamples := func(numSamples int) { - app := h.Appender(context.Background()) + app := h.AppenderV2(context.Background()) var ref storage.SeriesRef for i := range numSamples { - ref, err = app.Append(ref, lbls, lastTs, float64(lastTs)) + ref, err = app.Append(ref, lbls, 0, lastTs, float64(lastTs), nil, nil, storage.AOptions{}) expSamples = append(expSamples, sample{t: lastTs, f: float64(lastTs)}) require.NoError(t, err) lastTs += itvl if i%10 == 0 { require.NoError(t, app.Commit()) - app = h.Appender(context.Background()) + app = h.AppenderV2(context.Background()) } } require.NoError(t, app.Commit()) @@ -5933,15 +3401,15 @@ func TestReplayAfterMmapReplayError(t *testing.T) { require.NoError(t, h.Close()) } -func TestOOOAppendWithNoSeries(t *testing.T) { +func TestHeadAppenderV2_Append_OOOWithNoSeries(t *testing.T) { for name, scenario := range sampleTypeScenarios { t.Run(name, func(t *testing.T) { - testOOOAppendWithNoSeries(t, scenario.appendFunc) + testHeadAppenderV2AppendOOOWithNoSeries(t, scenario.appendFunc) }) } } -func testOOOAppendWithNoSeries(t *testing.T, appendFunc func(appender storage.Appender, lbls labels.Labels, ts, value int64) (storage.SeriesRef, sample, error)) { +func testHeadAppenderV2AppendOOOWithNoSeries(t *testing.T, appendFunc func(appender storage.LimitedAppenderV1, lbls labels.Labels, ts, value int64) (storage.SeriesRef, sample, error)) { dir := t.TempDir() wal, err := wlog.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, compression.Snappy) require.NoError(t, err) @@ -5961,8 +3429,8 @@ func testOOOAppendWithNoSeries(t *testing.T, appendFunc func(appender storage.Ap require.NoError(t, h.Init(0)) appendSample := func(lbls labels.Labels, ts int64) { - app := h.Appender(context.Background()) - _, _, err := appendFunc(app, lbls, ts*time.Minute.Milliseconds(), ts) + app := h.AppenderV2(context.Background()) + _, _, err := appendFunc(storage.AppenderV2AsLimitedV1(app), lbls, ts*time.Minute.Milliseconds(), ts) require.NoError(t, err) require.NoError(t, app.Commit()) } @@ -6009,8 +3477,8 @@ func testOOOAppendWithNoSeries(t *testing.T, appendFunc func(appender storage.Ap // Now 179m is too old. s4 := newLabels(4) - app := h.Appender(context.Background()) - _, _, err = appendFunc(app, s4, 179*time.Minute.Milliseconds(), 179) + app := h.AppenderV2(context.Background()) + _, _, err = appendFunc(storage.AppenderV2AsLimitedV1(app), s4, 179*time.Minute.Milliseconds(), 179) require.Equal(t, storage.ErrTooOldSample, err) require.NoError(t, app.Rollback()) verifyOOOSamples(s3, 1) @@ -6022,17 +3490,17 @@ func testOOOAppendWithNoSeries(t *testing.T, appendFunc func(appender storage.Ap verifyInOrderSamples(s5, 1) } -func TestHeadMinOOOTimeUpdate(t *testing.T) { +func TestHead_MinOOOTime_Update_AppenderV2(t *testing.T) { for name, scenario := range sampleTypeScenarios { t.Run(name, func(t *testing.T) { if scenario.sampleType == sampleMetricTypeFloat { - testHeadMinOOOTimeUpdate(t, scenario) + testHeadMinOOOTimeUpdateAppenderV2(t, scenario) } }) } } -func testHeadMinOOOTimeUpdate(t *testing.T, scenario sampleTypeScenario) { +func testHeadMinOOOTimeUpdateAppenderV2(t *testing.T, scenario sampleTypeScenario) { dir := t.TempDir() wal, err := wlog.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, compression.Snappy) require.NoError(t, err) @@ -6051,8 +3519,8 @@ func testHeadMinOOOTimeUpdate(t *testing.T, scenario sampleTypeScenario) { require.NoError(t, h.Init(0)) appendSample := func(ts int64) { - app := h.Appender(context.Background()) - _, _, err = scenario.appendFunc(app, labels.FromStrings("a", "b"), ts*time.Minute.Milliseconds(), ts) + app := h.AppenderV2(context.Background()) + _, _, err = scenario.appendFunc(storage.AppenderV2AsLimitedV1(app), labels.FromStrings("a", "b"), ts*time.Minute.Milliseconds(), ts) require.NoError(t, err) require.NoError(t, app.Commit()) } @@ -6077,7 +3545,7 @@ func testHeadMinOOOTimeUpdate(t *testing.T, scenario sampleTypeScenario) { require.Equal(t, 295*time.Minute.Milliseconds(), h.MinOOOTime()) } -func TestGaugeHistogramWALAndChunkHeader(t *testing.T) { +func TestGaugeHistogramWALAndChunkHeader_AppenderV2(t *testing.T) { l := labels.FromStrings("a", "b") head, _ := newTestHead(t, 1000, compression.None, false) t.Cleanup(func() { @@ -6088,8 +3556,8 @@ func TestGaugeHistogramWALAndChunkHeader(t *testing.T) { ts := int64(0) appendHistogram := func(h *histogram.Histogram) { ts++ - app := head.Appender(context.Background()) - _, err := app.AppendHistogram(0, l, ts, h.Copy(), nil) + app := head.AppenderV2(context.Background()) + _, err := app.Append(0, l, 0, ts, 0, h.Copy(), nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) } @@ -6153,7 +3621,7 @@ func TestGaugeHistogramWALAndChunkHeader(t *testing.T) { checkHeaders() } -func TestGaugeFloatHistogramWALAndChunkHeader(t *testing.T) { +func TestGaugeFloatHistogramWALAndChunkHeader_AppenderV2(t *testing.T) { l := labels.FromStrings("a", "b") head, _ := newTestHead(t, 1000, compression.None, false) t.Cleanup(func() { @@ -6164,8 +3632,8 @@ func TestGaugeFloatHistogramWALAndChunkHeader(t *testing.T) { ts := int64(0) appendHistogram := func(h *histogram.FloatHistogram) { ts++ - app := head.Appender(context.Background()) - _, err := app.AppendHistogram(0, l, ts, nil, h.Copy()) + app := head.AppenderV2(context.Background()) + _, err := app.Append(0, l, 0, ts, 0, nil, h.Copy(), storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) } @@ -6229,12 +3697,12 @@ func TestGaugeFloatHistogramWALAndChunkHeader(t *testing.T) { checkHeaders() } -func TestSnapshotAheadOfWALError(t *testing.T) { +func TestSnapshotAheadOfWALError_AppenderV2(t *testing.T) { head, _ := newTestHead(t, 120*4, compression.None, false) head.opts.EnableMemorySnapshotOnShutdown = true // Add a sample to fill WAL. - app := head.Appender(context.Background()) - _, err := app.Append(0, labels.FromStrings("foo", "bar"), 10, 10) + app := head.AppenderV2(context.Background()) + _, err := app.Append(0, labels.FromStrings("foo", "bar"), 0, 10, 10, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) @@ -6257,8 +3725,8 @@ func TestSnapshotAheadOfWALError(t *testing.T) { head, err = NewHead(nil, nil, w, nil, head.opts, nil) require.NoError(t, err) // Add a sample to fill WAL. - app = head.Appender(context.Background()) - _, err = app.Append(0, labels.FromStrings("foo", "bar"), 10, 10) + app = head.AppenderV2(context.Background()) + _, err = app.Append(0, labels.FromStrings("foo", "bar"), 0, 10, 10, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) lastSegment, _, _ := w.LastSegmentAndOffset() @@ -6284,31 +3752,7 @@ func TestSnapshotAheadOfWALError(t *testing.T) { require.NoError(t, head.Close()) } -func BenchmarkCuttingHeadHistogramChunks(b *testing.B) { - const ( - numSamples = 50000 - numBuckets = 100 - ) - samples := histogram.GenerateBigTestHistograms(numSamples, numBuckets) - - h, _ := newTestHead(b, DefaultBlockDuration, compression.None, false) - defer func() { - require.NoError(b, h.Close()) - }() - - a := h.Appender(context.Background()) - ts := time.Now().UnixMilli() - lbls := labels.FromStrings("foo", "bar") - - b.ResetTimer() - - for _, s := range samples { - _, err := a.AppendHistogram(0, lbls, ts, s, nil) - require.NoError(b, err) - } -} - -func TestCuttingNewHeadChunks(t *testing.T) { +func TestCuttingNewHeadChunks_AppenderV2(t *testing.T) { ctx := context.Background() testCases := map[string]struct { numTotalSamples int @@ -6413,7 +3857,7 @@ func TestCuttingNewHeadChunks(t *testing.T) { require.NoError(t, h.Close()) }() - a := h.Appender(context.Background()) + a := h.AppenderV2(context.Background()) ts := int64(10000) lbls := labels.FromStrings("foo", "bar") @@ -6421,10 +3865,10 @@ func TestCuttingNewHeadChunks(t *testing.T) { for i := 0; i < tc.numTotalSamples; i++ { if tc.floatValFunc != nil { - _, err := a.Append(0, lbls, ts, tc.floatValFunc(i)) + _, err := a.Append(0, lbls, 0, ts, tc.floatValFunc(i), nil, nil, storage.AOptions{}) require.NoError(t, err) } else if tc.histValFunc != nil { - _, err := a.AppendHistogram(0, lbls, ts, tc.histValFunc(i), nil) + _, err := a.Append(0, lbls, 0, ts, 0, tc.histValFunc(i), nil, storage.AOptions{}) require.NoError(t, err) } @@ -6472,7 +3916,7 @@ func TestCuttingNewHeadChunks(t *testing.T) { // is appended to the head, right when the head chunk is at the size limit. // The test adds all samples as duplicate, thus expecting that the result has // exactly half of the samples. -func TestHeadDetectsDuplicateSampleAtSizeLimit(t *testing.T) { +func TestHeadDetectsDuplicateSampleAtSizeLimit_AppenderV2(t *testing.T) { numSamples := 1000 baseTS := int64(1695209650) @@ -6481,15 +3925,15 @@ func TestHeadDetectsDuplicateSampleAtSizeLimit(t *testing.T) { require.NoError(t, h.Close()) }() - a := h.Appender(context.Background()) + a := h.AppenderV2(context.Background()) var err error vals := []float64{math.MaxFloat64, 0x00} // Use the worst case scenario for the XOR encoding. Otherwise we hit the sample limit before the size limit. for i := range numSamples { ts := baseTS + int64(i/2)*10000 - a.Append(0, labels.FromStrings("foo", "bar"), ts, vals[(i/2)%len(vals)]) + a.Append(0, labels.FromStrings("foo", "bar"), 0, ts, vals[(i/2)%len(vals)], nil, nil, storage.AOptions{}) err = a.Commit() require.NoError(t, err) - a = h.Appender(context.Background()) + a = h.AppenderV2(context.Background()) } indexReader, err := h.Index() @@ -6515,27 +3959,27 @@ func TestHeadDetectsDuplicateSampleAtSizeLimit(t *testing.T) { require.Equal(t, numSamples/2, storedSampleCount) } -func TestWALSampleAndExemplarOrder(t *testing.T) { +func TestWALSampleAndExemplarOrder_AppenderV2(t *testing.T) { lbls := labels.FromStrings("foo", "bar") testcases := map[string]struct { - appendF func(app storage.Appender, ts int64) (storage.SeriesRef, error) + appendF func(app storage.AppenderV2, ts int64) (storage.SeriesRef, error) expectedType reflect.Type }{ "float sample": { - appendF: func(app storage.Appender, ts int64) (storage.SeriesRef, error) { - return app.Append(0, lbls, ts, 1.0) + appendF: func(app storage.AppenderV2, ts int64) (storage.SeriesRef, error) { + return app.Append(0, lbls, 0, ts, 1.0, nil, nil, storage.AOptions{Exemplars: []exemplar.Exemplar{{Value: 1.0, Ts: 5}}}) }, expectedType: reflect.TypeOf([]record.RefSample{}), }, "histogram sample": { - appendF: func(app storage.Appender, ts int64) (storage.SeriesRef, error) { - return app.AppendHistogram(0, lbls, ts, tsdbutil.GenerateTestHistogram(1), nil) + appendF: func(app storage.AppenderV2, ts int64) (storage.SeriesRef, error) { + return app.Append(0, lbls, 0, ts, 0, tsdbutil.GenerateTestHistogram(1), nil, storage.AOptions{Exemplars: []exemplar.Exemplar{{Value: 1.0, Ts: 5}}}) }, expectedType: reflect.TypeOf([]record.RefHistogramSample{}), }, "float histogram sample": { - appendF: func(app storage.Appender, ts int64) (storage.SeriesRef, error) { - return app.AppendHistogram(0, lbls, ts, nil, tsdbutil.GenerateTestFloatHistogram(1)) + appendF: func(app storage.AppenderV2, ts int64) (storage.SeriesRef, error) { + return app.Append(0, lbls, 0, ts, 0, nil, tsdbutil.GenerateTestFloatHistogram(1), storage.AOptions{Exemplars: []exemplar.Exemplar{{Value: 1.0, Ts: 5}}}) }, expectedType: reflect.TypeOf([]record.RefFloatHistogramSample{}), }, @@ -6548,12 +3992,11 @@ func TestWALSampleAndExemplarOrder(t *testing.T) { require.NoError(t, h.Close()) }() - app := h.Appender(context.Background()) - ref, err := tc.appendF(app, 10) + app := h.AppenderV2(context.Background()) + _, err := tc.appendF(app, 10) require.NoError(t, err) - app.AppendExemplar(ref, lbls, exemplar.Exemplar{Value: 1.0, Ts: 5}) - app.Commit() + require.NoError(t, app.Commit()) recs := readTestWAL(t, w.Dir()) require.Len(t, recs, 3) @@ -6567,126 +4010,7 @@ func TestWALSampleAndExemplarOrder(t *testing.T) { } } -// TestHeadCompactionWhileAppendAndCommitExemplar simulates a use case where -// a series is removed from the head while an exemplar is being appended to it. -// This can happen in theory by compacting the head at the right time due to -// a series being idle. -// The test cheats a little bit by not appending a sample with the exemplar. -// If you also add a sample and run Truncate in a concurrent goroutine and run -// the test around a million(!) times, you can get -// `unknown HeadSeriesRef when trying to add exemplar: 1` error on push. -// It is likely that running the test for much longer and with more time variations -// would trigger the -// `signal SIGSEGV: segmentation violation code=0x1 addr=0x20 pc=0xbb03d1` -// panic, that we have seen in the wild once. -func TestHeadCompactionWhileAppendAndCommitExemplar(t *testing.T) { - h, _ := newTestHead(t, DefaultBlockDuration, compression.None, false) - app := h.Appender(context.Background()) - lbls := labels.FromStrings("foo", "bar") - ref, err := app.Append(0, lbls, 1, 1) - require.NoError(t, err) - app.Commit() - // Not adding a sample here to trigger the fault. - app = h.Appender(context.Background()) - _, err = app.AppendExemplar(ref, lbls, exemplar.Exemplar{Value: 1, Ts: 20}) - require.NoError(t, err) - h.Truncate(10) - app.Commit() - h.Close() -} - -func labelsWithHashCollision() (labels.Labels, labels.Labels) { - // These two series have the same XXHash; thanks to https://github.com/pstibrany/labels_hash_collisions - ls1 := labels.FromStrings("__name__", "metric", "lbl", "HFnEaGl") - ls2 := labels.FromStrings("__name__", "metric", "lbl", "RqcXatm") - - if ls1.Hash() != ls2.Hash() { - // These ones are the same when using -tags slicelabels - ls1 = labels.FromStrings("__name__", "metric", "lbl1", "value", "lbl2", "l6CQ5y") - ls2 = labels.FromStrings("__name__", "metric", "lbl1", "value", "lbl2", "v7uDlF") - } - - if ls1.Hash() != ls2.Hash() { - panic("This code needs to be updated: find new labels with colliding hash values.") - } - - return ls1, ls2 -} - -// stripeSeriesWithCollidingSeries returns a stripeSeries with two memSeries having the same, colliding, hash. -func stripeSeriesWithCollidingSeries(t *testing.T) (*stripeSeries, *memSeries, *memSeries) { - t.Helper() - - lbls1, lbls2 := labelsWithHashCollision() - ms1 := memSeries{ - lset: lbls1, - } - ms2 := memSeries{ - lset: lbls2, - } - hash := lbls1.Hash() - s := newStripeSeries(1, noopSeriesLifecycleCallback{}) - - got, created := s.setUnlessAlreadySet(hash, lbls1, &ms1) - require.True(t, created) - require.Same(t, &ms1, got) - - // Add a conflicting series - got, created = s.setUnlessAlreadySet(hash, lbls2, &ms2) - require.True(t, created) - require.Same(t, &ms2, got) - - return s, &ms1, &ms2 -} - -func TestStripeSeries_getOrSet(t *testing.T) { - s, ms1, ms2 := stripeSeriesWithCollidingSeries(t) - hash := ms1.lset.Hash() - - // Verify that we can get both of the series despite the hash collision - got := s.getByHash(hash, ms1.lset) - require.Same(t, ms1, got) - got = s.getByHash(hash, ms2.lset) - require.Same(t, ms2, got) -} - -func TestStripeSeries_gc(t *testing.T) { - s, ms1, ms2 := stripeSeriesWithCollidingSeries(t) - hash := ms1.lset.Hash() - - s.gc(0, 0, nil) - - // Verify that we can get neither ms1 nor ms2 after gc-ing corresponding series - got := s.getByHash(hash, ms1.lset) - require.Nil(t, got) - got = s.getByHash(hash, ms2.lset) - require.Nil(t, got) -} - -func TestPostingsCardinalityStats(t *testing.T) { - head := &Head{postings: index.NewMemPostings()} - head.postings.Add(1, labels.FromStrings(labels.MetricName, "t", "n", "v1")) - head.postings.Add(2, labels.FromStrings(labels.MetricName, "t", "n", "v2")) - - statsForMetricName := head.PostingsCardinalityStats(labels.MetricName, 10) - head.postings.Add(3, labels.FromStrings(labels.MetricName, "t", "n", "v3")) - // Using cache. - require.Equal(t, statsForMetricName, head.PostingsCardinalityStats(labels.MetricName, 10)) - - statsForSomeLabel := head.PostingsCardinalityStats("n", 10) - // Cache should be evicted because of the change of label name. - require.NotEqual(t, statsForMetricName, statsForSomeLabel) - head.postings.Add(4, labels.FromStrings(labels.MetricName, "t", "n", "v4")) - // Using cache. - require.Equal(t, statsForSomeLabel, head.PostingsCardinalityStats("n", 10)) - // Cache should be evicted because of the change of limit parameter. - statsForSomeLabel1 := head.PostingsCardinalityStats("n", 1) - require.NotEqual(t, statsForSomeLabel1, statsForSomeLabel) - // Using cache. - require.Equal(t, statsForSomeLabel1, head.PostingsCardinalityStats("n", 1)) -} - -func TestHeadAppender_AppendFloatWithSameTimestampAsPreviousHistogram(t *testing.T) { +func TestHeadAppenderV2_Append_FloatWithSameTimestampAsPreviousHistogram(t *testing.T) { head, _ := newTestHead(t, DefaultBlockDuration, compression.None, false) t.Cleanup(func() { head.Close() }) @@ -6694,28 +4018,29 @@ func TestHeadAppender_AppendFloatWithSameTimestampAsPreviousHistogram(t *testing { // Append a float 10.0 @ 1_000 - app := head.Appender(context.Background()) - _, err := app.Append(0, ls, 1_000, 10.0) + app := head.AppenderV2(context.Background()) + _, err := app.Append(0, ls, 0, 1_000, 10.0, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) } { // Append a float histogram @ 2_000 - app := head.Appender(context.Background()) + app := head.AppenderV2(context.Background()) h := tsdbutil.GenerateTestHistogram(1) - _, err := app.AppendHistogram(0, ls, 2_000, h, nil) + _, err := app.Append(0, ls, 0, 2_000, 0, h, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) } - app := head.Appender(context.Background()) - _, err := app.Append(0, ls, 2_000, 10.0) + app := head.AppenderV2(context.Background()) + _, err := app.Append(0, ls, 0, 2_000, 10.0, nil, nil, storage.AOptions{}) require.Error(t, err) require.ErrorIs(t, err, storage.NewDuplicateHistogramToFloatErr(2_000, 10.0)) } -func TestHeadAppender_AppendST(t *testing.T) { +func TestHeadAppenderV2_Append_EnableSTAsZeroSample(t *testing.T) { + // Make sure counter resets hints are non-zero, so we can detect ST histogram samples. testHistogram := tsdbutil.GenerateTestHistogram(1) testHistogram.CounterResetHint = histogram.NotCounterReset testFloatHistogram := tsdbutil.GenerateTestFloatHistogram(1) @@ -6909,30 +4234,121 @@ func TestHeadAppender_AppendST(t *testing.T) { } }(), }, + { + name: "ST lower than minValidTime/float", + appendableSamples: []appendableSamples{ + {ts: 100, fSample: 10, st: -1}, + }, + // ST results ErrOutOfBounds, but ST append is best effort, so + // ST should be ignored, but sample appended. + expectedSamples: func() []chunks.Sample { + return []chunks.Sample{ + sample{t: 100, f: 10}, + } + }(), + }, + { + name: "ST lower than minValidTime/histogram", + appendableSamples: []appendableSamples{ + {ts: 100, h: testHistogram, st: -1}, + }, + // ST results ErrOutOfBounds, but ST append is best effort, so + // ST should be ignored, but sample appended. + expectedSamples: func() []chunks.Sample { + // NOTE: Without ST, on query, first histogram sample will get + // CounterReset adjusted to 0. + firstSample := testHistogram.Copy() + firstSample.CounterResetHint = histogram.UnknownCounterReset + return []chunks.Sample{ + sample{t: 100, h: firstSample}, + } + }(), + }, + { + name: "ST lower than minValidTime/floathistogram", + appendableSamples: []appendableSamples{ + {ts: 100, fh: testFloatHistogram, st: -1}, + }, + // ST results ErrOutOfBounds, but ST append is best effort, so + // ST should be ignored, but sample appended. + expectedSamples: func() []chunks.Sample { + // NOTE: Without ST, on query, first histogram sample will get + // CounterReset adjusted to 0. + firstSample := testFloatHistogram.Copy() + firstSample.CounterResetHint = histogram.UnknownCounterReset + return []chunks.Sample{ + sample{t: 100, fh: firstSample}, + } + }(), + }, + { + name: "ST duplicates an existing sample/float", + appendableSamples: []appendableSamples{ + {ts: 100, fSample: 10}, + {ts: 200, fSample: 10, st: 100}, + }, + // ST results ErrOutOfBounds, but ST append is best effort, so + // ST should be ignored, but sample appended. + expectedSamples: func() []chunks.Sample { + return []chunks.Sample{ + sample{t: 100, f: 10}, + sample{t: 200, f: 10}, + } + }(), + }, + { + name: "ST duplicates an existing sample/histogram", + appendableSamples: []appendableSamples{ + {ts: 100, h: testHistogram}, + {ts: 200, h: testHistogram, st: 100}, + }, + // ST results ErrDuplicateSampleForTimestamp, but ST append is best effort, so + // ST should be ignored, but sample appended. + expectedSamples: func() []chunks.Sample { + // NOTE: Without ST, on query, first histogram sample will get + // CounterReset adjusted to 0. + firstSample := testHistogram.Copy() + firstSample.CounterResetHint = histogram.UnknownCounterReset + return []chunks.Sample{ + sample{t: 100, h: firstSample}, + sample{t: 200, h: testHistogram}, + } + }(), + }, + { + name: "ST duplicates an existing sample/floathistogram", + appendableSamples: []appendableSamples{ + {ts: 100, fh: testFloatHistogram}, + {ts: 200, fh: testFloatHistogram, st: 100}, + }, + // ST results ErrDuplicateSampleForTimestamp, but ST append is best effort, so + // ST should ignored, but sample appended. + expectedSamples: func() []chunks.Sample { + // NOTE: Without ST, on query, first histogram sample will get + // CounterReset adjusted to 0. + firstSample := testFloatHistogram.Copy() + firstSample.CounterResetHint = histogram.UnknownCounterReset + return []chunks.Sample{ + sample{t: 100, fh: firstSample}, + sample{t: 200, fh: testFloatHistogram}, + } + }(), + }, } { t.Run(tc.name, func(t *testing.T) { - h, _ := newTestHead(t, DefaultBlockDuration, compression.None, false) + opts := newTestHeadDefaultOptions(DefaultBlockDuration, false) + opts.EnableSTAsZeroSample = true + h, _ := newTestHeadWithOptions(t, compression.None, opts) defer func() { require.NoError(t, h.Close()) }() - a := h.Appender(context.Background()) - lbls := labels.FromStrings("foo", "bar") - for _, sample := range tc.appendableSamples { - // Append float if it's a float test case - if sample.fSample != 0 { - _, err := a.AppendSTZeroSample(0, lbls, sample.ts, sample.st) - require.NoError(t, err) - _, err = a.Append(0, lbls, sample.ts, sample.fSample) - require.NoError(t, err) - } - // Append histograms if it's a histogram test case - if sample.h != nil || sample.fh != nil { - ref, err := a.AppendHistogramSTZeroSample(0, lbls, sample.ts, sample.st, sample.h, sample.fh) - require.NoError(t, err) - _, err = a.AppendHistogram(ref, lbls, sample.ts, sample.h, sample.fh) - require.NoError(t, err) - } + a := h.AppenderV2(context.Background()) + lbls := labels.FromStrings("foo", "bar") + + for _, s := range tc.appendableSamples { + _, err := a.Append(0, lbls, s.st, s.ts, s.fSample, s.h, s.fh, storage.AOptions{}) + require.NoError(t, err) } require.NoError(t, a.Commit()) @@ -6944,122 +4360,29 @@ func TestHeadAppender_AppendST(t *testing.T) { } } -func TestHeadAppender_AppendHistogramSTZeroSample(t *testing.T) { - type appendableSamples struct { - ts int64 - h *histogram.Histogram - fh *histogram.FloatHistogram - st int64 // 0 if no created timestamp. - } - for _, tc := range []struct { - name string - appendableSamples []appendableSamples - expectedError error - }{ - { - name: "integer histogram ST lower than minValidTime initiates ErrOutOfBounds", - appendableSamples: []appendableSamples{ - {ts: 100, h: tsdbutil.GenerateTestHistogram(1), st: -1}, - }, - expectedError: storage.ErrOutOfBounds, - }, - { - name: "float histograms ST lower than minValidTime initiates ErrOutOfBounds", - appendableSamples: []appendableSamples{ - {ts: 100, fh: tsdbutil.GenerateTestFloatHistogram(1), st: -1}, - }, - expectedError: storage.ErrOutOfBounds, - }, - { - name: "integer histogram ST duplicates an existing sample", - appendableSamples: []appendableSamples{ - {ts: 100, h: tsdbutil.GenerateTestHistogram(1)}, - {ts: 200, h: tsdbutil.GenerateTestHistogram(1), st: 100}, - }, - expectedError: storage.ErrDuplicateSampleForTimestamp, - }, - { - name: "float histogram ST duplicates an existing sample", - appendableSamples: []appendableSamples{ - {ts: 100, fh: tsdbutil.GenerateTestFloatHistogram(1)}, - {ts: 200, fh: tsdbutil.GenerateTestFloatHistogram(1), st: 100}, - }, - expectedError: storage.ErrDuplicateSampleForTimestamp, - }, - } { - t.Run(tc.name, func(t *testing.T) { - h, _ := newTestHead(t, DefaultBlockDuration, compression.None, false) - - defer func() { - require.NoError(t, h.Close()) - }() - - lbls := labels.FromStrings("foo", "bar") - - var ref storage.SeriesRef - for _, sample := range tc.appendableSamples { - a := h.Appender(context.Background()) - var err error - if sample.st != 0 { - ref, err = a.AppendHistogramSTZeroSample(ref, lbls, sample.ts, sample.st, sample.h, sample.fh) - require.ErrorIs(t, err, tc.expectedError) - } - - ref, err = a.AppendHistogram(ref, lbls, sample.ts, sample.h, sample.fh) - require.NoError(t, err) - require.NoError(t, a.Commit()) - } - }) - } -} - -func TestHeadCompactableDoesNotCompactEmptyHead(t *testing.T) { - // Use a chunk range of 1 here so that if we attempted to determine if the head - // was compactable using default values for min and max times, `Head.compactable()` - // would return true which is incorrect. This test verifies that we short-circuit - // the check when the head has not yet had any samples added. - head, _ := newTestHead(t, 1, compression.None, false) - defer func() { - require.NoError(t, head.Close()) - }() - - require.False(t, head.compactable()) -} - -type countSeriesLifecycleCallback struct { - created atomic.Int64 - deleted atomic.Int64 -} - -func (*countSeriesLifecycleCallback) PreCreation(labels.Labels) error { return nil } -func (c *countSeriesLifecycleCallback) PostCreation(labels.Labels) { c.created.Inc() } -func (c *countSeriesLifecycleCallback) PostDeletion(s map[chunks.HeadSeriesRef]labels.Labels) { - c.deleted.Add(int64(len(s))) -} - // Regression test for data race https://github.com/prometheus/prometheus/issues/15139. -func TestHeadAppendHistogramAndCommitConcurrency(t *testing.T) { +func TestHeadAppenderV2_Append_HistogramAndCommitConcurrency(t *testing.T) { h := tsdbutil.GenerateTestHistogram(1) fh := tsdbutil.GenerateTestFloatHistogram(1) - testCases := map[string]func(storage.Appender, int) error{ - "integer histogram": func(app storage.Appender, i int) error { - _, err := app.AppendHistogram(0, labels.FromStrings("foo", "bar", "serial", strconv.Itoa(i)), 1, h, nil) + testCases := map[string]func(storage.AppenderV2, int) error{ + "integer histogram": func(app storage.AppenderV2, i int) error { + _, err := app.Append(0, labels.FromStrings("foo", "bar", "serial", strconv.Itoa(i)), 0, 1, 0, h, nil, storage.AOptions{}) return err }, - "float histogram": func(app storage.Appender, i int) error { - _, err := app.AppendHistogram(0, labels.FromStrings("foo", "bar", "serial", strconv.Itoa(i)), 1, nil, fh) + "float histogram": func(app storage.AppenderV2, i int) error { + _, err := app.Append(0, labels.FromStrings("foo", "bar", "serial", strconv.Itoa(i)), 0, 1, 0, nil, fh, storage.AOptions{}) return err }, } for name, tc := range testCases { t.Run(name, func(t *testing.T) { - testHeadAppendHistogramAndCommitConcurrency(t, tc) + testHeadAppenderV2AppendHistogramAndCommitConcurrency(t, tc) }) } } -func testHeadAppendHistogramAndCommitConcurrency(t *testing.T, appendFn func(storage.Appender, int) error) { +func testHeadAppenderV2AppendHistogramAndCommitConcurrency(t *testing.T, appendFn func(storage.AppenderV2, int) error) { head, _ := newTestHead(t, 1000, compression.None, false) defer func() { require.NoError(t, head.Close()) @@ -7076,7 +4399,7 @@ func testHeadAppendHistogramAndCommitConcurrency(t *testing.T, appendFn func(sto go func() { defer wg.Done() for i := range 10000 { - app := head.Appender(context.Background()) + app := head.AppenderV2(context.Background()) require.NoError(t, appendFn(app, i)) require.NoError(t, app.Commit()) } @@ -7085,7 +4408,7 @@ func testHeadAppendHistogramAndCommitConcurrency(t *testing.T, appendFn func(sto go func() { defer wg.Done() for i := range 10000 { - app := head.Appender(context.Background()) + app := head.AppenderV2(context.Background()) require.NoError(t, appendFn(app, i)) require.NoError(t, app.Commit()) } @@ -7094,7 +4417,7 @@ func testHeadAppendHistogramAndCommitConcurrency(t *testing.T, appendFn func(sto wg.Wait() } -func TestHead_NumStaleSeries(t *testing.T) { +func TestHeadAppenderV2_NumStaleSeries(t *testing.T) { head, _ := newTestHead(t, 1000, compression.None, false) t.Cleanup(func() { require.NoError(t, head.Close()) @@ -7105,20 +4428,20 @@ func TestHead_NumStaleSeries(t *testing.T) { require.Equal(t, uint64(0), head.NumStaleSeries()) appendSample := func(lbls labels.Labels, ts int64, val float64) { - app := head.Appender(context.Background()) - _, err := app.Append(0, lbls, ts, val) + app := head.AppenderV2(context.Background()) + _, err := app.Append(0, lbls, 0, ts, val, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) } appendHistogram := func(lbls labels.Labels, ts int64, val *histogram.Histogram) { - app := head.Appender(context.Background()) - _, err := app.AppendHistogram(0, lbls, ts, val, nil) + app := head.AppenderV2(context.Background()) + _, err := app.Append(0, lbls, 0, ts, 0, val, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) } appendFloatHistogram := func(lbls labels.Labels, ts int64, val *histogram.FloatHistogram) { - app := head.Appender(context.Background()) - _, err := app.AppendHistogram(0, lbls, ts, nil, val) + app := head.AppenderV2(context.Background()) + _, err := app.Append(0, lbls, 0, ts, 0, nil, val, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) } @@ -7244,22 +4567,22 @@ func TestHead_NumStaleSeries(t *testing.T) { // TestHistogramStalenessConversionMetrics verifies that staleness marker conversion correctly // increments the right appender metrics for both histogram and float histogram scenarios. -func TestHistogramStalenessConversionMetrics(t *testing.T) { +func TestHeadAppenderV2_Append_HistogramStalenessConversionMetrics(t *testing.T) { testCases := []struct { name string - setupHistogram func(app storage.Appender, lbls labels.Labels) error + setupHistogram func(app storage.AppenderV2, lbls labels.Labels) error }{ { name: "float_staleness_to_histogram", - setupHistogram: func(app storage.Appender, lbls labels.Labels) error { - _, err := app.AppendHistogram(0, lbls, 1000, tsdbutil.GenerateTestHistograms(1)[0], nil) + setupHistogram: func(app storage.AppenderV2, lbls labels.Labels) error { + _, err := app.Append(0, lbls, 0, 1000, 0, tsdbutil.GenerateTestHistograms(1)[0], nil, storage.AOptions{}) return err }, }, { name: "float_staleness_to_float_histogram", - setupHistogram: func(app storage.Appender, lbls labels.Labels) error { - _, err := app.AppendHistogram(0, lbls, 1000, nil, tsdbutil.GenerateTestFloatHistograms(1)[0]) + setupHistogram: func(app storage.AppenderV2, lbls labels.Labels) error { + _, err := app.Append(0, lbls, 0, 1000, 0, nil, tsdbutil.GenerateTestFloatHistograms(1)[0], storage.AOptions{}) return err }, }, @@ -7283,14 +4606,14 @@ func TestHistogramStalenessConversionMetrics(t *testing.T) { } // Step 1: Establish a series with histogram data - app := head.Appender(context.Background()) + app := head.AppenderV2(context.Background()) err := tc.setupHistogram(app, lbls) require.NoError(t, err) require.NoError(t, app.Commit()) // Step 2: Add a float staleness marker - app = head.Appender(context.Background()) - _, err = app.Append(0, lbls, 2000, math.Float64frombits(value.StaleNaN)) + app = head.AppenderV2(context.Background()) + _, err = app.Append(0, lbls, 0, 2000, math.Float64frombits(value.StaleNaN), nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) diff --git a/tsdb/head_bench_test.go b/tsdb/head_bench_test.go index c98fb6613d..347a6bce18 100644 --- a/tsdb/head_bench_test.go +++ b/tsdb/head_bench_test.go @@ -14,7 +14,6 @@ package tsdb import ( - "context" "errors" "fmt" "math/rand" @@ -32,6 +31,223 @@ import ( "github.com/prometheus/prometheus/util/compression" ) +type benchAppendFunc func(b *testing.B, h *Head, ts int64, series []storage.Series, samplesPerAppend int64) storage.AppenderTransaction + +func appendV1Float(b *testing.B, h *Head, ts int64, series []storage.Series, samplesPerAppend int64) storage.AppenderTransaction { + var err error + app := h.Appender(b.Context()) + for _, s := range series { + var ref storage.SeriesRef + for sampleIndex := range samplesPerAppend { + ref, err = app.Append(ref, s.Labels(), ts+sampleIndex, float64(ts+sampleIndex)) + require.NoError(b, err) + } + } + return app +} + +func appendV2Float(b *testing.B, h *Head, ts int64, series []storage.Series, samplesPerAppend int64) storage.AppenderTransaction { + var err error + app := h.AppenderV2(b.Context()) + for _, s := range series { + var ref storage.SeriesRef + for sampleIndex := range samplesPerAppend { + ref, err = app.Append(ref, s.Labels(), 0, ts+sampleIndex, float64(ts+sampleIndex), nil, nil, storage.AOptions{}) + require.NoError(b, err) + } + } + return app +} + +func appendV1FloatOrHistogramWithExemplars(b *testing.B, h *Head, ts int64, series []storage.Series, samplesPerAppend int64) storage.AppenderTransaction { + var err error + app := h.Appender(b.Context()) + for i, s := range series { + var ref storage.SeriesRef + for sampleIndex := range samplesPerAppend { + // if i is even, append a sample, else append a histogram. + if i%2 == 0 { + ref, err = app.Append(ref, s.Labels(), ts+sampleIndex, float64(ts+sampleIndex)) + require.NoError(b, err) + // Every sample also has an exemplar attached. + _, err = app.AppendExemplar(ref, s.Labels(), exemplar.Exemplar{ + Labels: labels.FromStrings("trace_id", strconv.Itoa(rand.Int())), + Value: rand.Float64(), + Ts: ts + sampleIndex, + }) + require.NoError(b, err) + continue + } + + h := &histogram.Histogram{ + Count: 7 + uint64(ts*5), + ZeroCount: 2 + uint64(ts), + ZeroThreshold: 0.001, + Sum: 18.4 * rand.Float64(), + Schema: 1, + PositiveSpans: []histogram.Span{ + {Offset: 0, Length: 2}, + {Offset: 1, Length: 2}, + }, + PositiveBuckets: []int64{ts + 1, 1, -1, 0}, + } + ref, err = app.AppendHistogram(ref, s.Labels(), ts, h, nil) + require.NoError(b, err) + // Every histogram sample also has 3 exemplars attached. + _, err = app.AppendExemplar(ref, s.Labels(), exemplar.Exemplar{ + Labels: labels.FromStrings("trace_id", strconv.Itoa(rand.Int())), + Value: rand.Float64(), + Ts: ts + sampleIndex, + }) + require.NoError(b, err) + _, err = app.AppendExemplar(ref, s.Labels(), exemplar.Exemplar{ + Labels: labels.FromStrings("trace_id", strconv.Itoa(rand.Int())), + Value: rand.Float64(), + Ts: ts + sampleIndex, + }) + require.NoError(b, err) + _, err = app.AppendExemplar(ref, s.Labels(), exemplar.Exemplar{ + Labels: labels.FromStrings("trace_id", strconv.Itoa(rand.Int())), + Value: rand.Float64(), + Ts: ts + sampleIndex, + }) + require.NoError(b, err) + } + } + return app +} + +func appendV2FloatOrHistogramWithExemplars(b *testing.B, h *Head, ts int64, series []storage.Series, samplesPerAppend int64) storage.AppenderTransaction { + var ( + err error + ex = make([]exemplar.Exemplar, 3) + ) + + app := h.AppenderV2(b.Context()) + for i, s := range series { + var ref storage.SeriesRef + for sampleIndex := range samplesPerAppend { + aOpts := storage.AOptions{Exemplars: ex[:0]} + + // if i is even, append a sample, else append a histogram. + if i%2 == 0 { + // Every sample also has an exemplar attached. + aOpts.Exemplars = append(aOpts.Exemplars, exemplar.Exemplar{ + Labels: labels.FromStrings("trace_id", strconv.Itoa(rand.Int())), + Value: rand.Float64(), + Ts: ts + sampleIndex, + }) + ref, err = app.Append(ref, s.Labels(), 0, ts, float64(ts), nil, nil, aOpts) + require.NoError(b, err) + continue + } + h := &histogram.Histogram{ + Count: 7 + uint64(ts*5), + ZeroCount: 2 + uint64(ts), + ZeroThreshold: 0.001, + Sum: 18.4 * rand.Float64(), + Schema: 1, + PositiveSpans: []histogram.Span{ + {Offset: 0, Length: 2}, + {Offset: 1, Length: 2}, + }, + PositiveBuckets: []int64{ts + 1, 1, -1, 0}, + } + + // Every histogram sample also has 3 exemplars attached. + aOpts.Exemplars = append(aOpts.Exemplars, + exemplar.Exemplar{ + Labels: labels.FromStrings("trace_id", strconv.Itoa(rand.Int())), + Value: rand.Float64(), + Ts: ts + sampleIndex, + }, + exemplar.Exemplar{ + Labels: labels.FromStrings("trace_id", strconv.Itoa(rand.Int())), + Value: rand.Float64(), + Ts: ts + sampleIndex, + }, + exemplar.Exemplar{ + Labels: labels.FromStrings("trace_id", strconv.Itoa(rand.Int())), + Value: rand.Float64(), + Ts: ts + sampleIndex, + }, + ) + ref, err = app.Append(ref, s.Labels(), 0, ts, 0, h, nil, aOpts) + require.NoError(b, err) + } + } + return app +} + +type appendCase struct { + name string + appendFunc benchAppendFunc +} + +func appendCases() []appendCase { + return []appendCase{ + { + name: "appender=v1/case=floats", + appendFunc: appendV1Float, + }, + { + name: "appender=v2/case=floats", + appendFunc: appendV2Float, + }, + { + name: "appender=v1/case=floatsHistogramsExemplars", + appendFunc: appendV1FloatOrHistogramWithExemplars, + }, + { + name: "appender=v2/case=floatsHistogramsExemplars", + appendFunc: appendV2FloatOrHistogramWithExemplars, + }, + } +} + +/* + export bench=append && go test \ + -run '^$' -bench '^BenchmarkHeadAppender_AppendCommit$' \ + -benchtime 5s -count 6 -cpu 2 -timeout 999m \ + | tee ${bench}.txt +*/ +func BenchmarkHeadAppender_AppendCommit(b *testing.B) { + // NOTE(bwplotka): Previously we also had 1k and 10k series case. There is nothing + // special happening in 100 vs 1k vs 10k, so let's save considerable amount of benchmark time + // for quicker feedback. In return, we add more sample type cases. + // Similarly, we removed the 2 sample in append case. + seriesCounts := []int{10, 100} + series := genSeries(100, 10, 0, 0) // Only using the generated labels. + for _, appendCase := range appendCases() { + for _, seriesCount := range seriesCounts { + for _, samplesPerAppend := range []int64{1, 5, 100} { + b.Run(fmt.Sprintf("%s/series=%d/samples_per_append=%d", appendCase.name, seriesCount, samplesPerAppend), func(b *testing.B) { + opts := newTestHeadDefaultOptions(10000, false) + opts.EnableExemplarStorage = true // We benchmark with exemplars, benchmark with them. + h, _ := newTestHeadWithOptions(b, compression.None, opts) + b.Cleanup(func() { require.NoError(b, h.Close()) }) + + ts := int64(1000) + + // Init series, that's not what we're benchmarking here. + app := appendCase.appendFunc(b, h, ts, series[:seriesCount], samplesPerAppend) + require.NoError(b, app.Commit()) + ts += 1000 // should increment more than highest samplesPerAppend + + b.ReportAllocs() + b.ResetTimer() + + for b.Loop() { + app := appendCase.appendFunc(b, h, ts, series[:seriesCount], samplesPerAppend) + require.NoError(b, app.Commit()) + ts += 1000 // should increment more than highest samplesPerAppend + } + }) + } + } + } +} + func BenchmarkHeadStripeSeriesCreate(b *testing.B) { chunkDir := b.TempDir() // Put a series, select it. GC it and then access it. @@ -86,86 +302,6 @@ func BenchmarkHeadStripeSeriesCreate_PreCreationFailure(b *testing.B) { } } -func BenchmarkHead_WalCommit(b *testing.B) { - seriesCounts := []int{100, 1000, 10000} - series := genSeries(10000, 10, 0, 0) // Only using the generated labels. - - appendSamples := func(b *testing.B, app storage.Appender, seriesCount int, ts int64) { - var err error - for i, s := range series[:seriesCount] { - var ref storage.SeriesRef - // if i is even, append a sample, else append a histogram. - if i%2 == 0 { - ref, err = app.Append(ref, s.Labels(), ts, float64(ts)) - } else { - h := &histogram.Histogram{ - Count: 7 + uint64(ts*5), - ZeroCount: 2 + uint64(ts), - ZeroThreshold: 0.001, - Sum: 18.4 * rand.Float64(), - Schema: 1, - PositiveSpans: []histogram.Span{ - {Offset: 0, Length: 2}, - {Offset: 1, Length: 2}, - }, - PositiveBuckets: []int64{ts + 1, 1, -1, 0}, - } - ref, err = app.AppendHistogram(ref, s.Labels(), ts, h, nil) - } - require.NoError(b, err) - - _, err = app.AppendExemplar(ref, s.Labels(), exemplar.Exemplar{ - Labels: labels.FromStrings("trace_id", strconv.Itoa(rand.Int())), - Value: rand.Float64(), - Ts: ts, - }) - require.NoError(b, err) - } - } - - for _, seriesCount := range seriesCounts { - b.Run(fmt.Sprintf("%d series", seriesCount), func(b *testing.B) { - for _, commits := range []int64{1, 2} { // To test commits that create new series and when the series already exists. - b.Run(fmt.Sprintf("%d commits", commits), func(b *testing.B) { - b.ReportAllocs() - b.ResetTimer() - - for b.Loop() { - b.StopTimer() - h, w := newTestHead(b, 10000, compression.None, false) - b.Cleanup(func() { - if h != nil { - h.Close() - } - if w != nil { - w.Close() - } - }) - app := h.Appender(context.Background()) - - appendSamples(b, app, seriesCount, 0) - - b.StartTimer() - require.NoError(b, app.Commit()) - if commits == 2 { - b.StopTimer() - app = h.Appender(context.Background()) - appendSamples(b, app, seriesCount, 1) - b.StartTimer() - require.NoError(b, app.Commit()) - } - b.StopTimer() - h.Close() - h = nil - w.Close() - w = nil - } - }) - } - }) - } -} - type failingSeriesLifecycleCallback struct{} func (failingSeriesLifecycleCallback) PreCreation(labels.Labels) error { return errors.New("failed") } diff --git a/tsdb/head_bench_v2_test.go b/tsdb/head_bench_v2_test.go deleted file mode 100644 index c98fb6613d..0000000000 --- a/tsdb/head_bench_v2_test.go +++ /dev/null @@ -1,173 +0,0 @@ -// Copyright 2018 The Prometheus Authors -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package tsdb - -import ( - "context" - "errors" - "fmt" - "math/rand" - "strconv" - "testing" - - "github.com/stretchr/testify/require" - "go.uber.org/atomic" - - "github.com/prometheus/prometheus/model/exemplar" - "github.com/prometheus/prometheus/model/histogram" - "github.com/prometheus/prometheus/model/labels" - "github.com/prometheus/prometheus/storage" - "github.com/prometheus/prometheus/tsdb/chunks" - "github.com/prometheus/prometheus/util/compression" -) - -func BenchmarkHeadStripeSeriesCreate(b *testing.B) { - chunkDir := b.TempDir() - // Put a series, select it. GC it and then access it. - opts := DefaultHeadOptions() - opts.ChunkRange = 1000 - opts.ChunkDirRoot = chunkDir - h, err := NewHead(nil, nil, nil, nil, opts, nil) - require.NoError(b, err) - defer h.Close() - - for i := 0; b.Loop(); i++ { - h.getOrCreate(uint64(i), labels.FromStrings("a", strconv.Itoa(i)), false) - } -} - -func BenchmarkHeadStripeSeriesCreateParallel(b *testing.B) { - chunkDir := b.TempDir() - // Put a series, select it. GC it and then access it. - opts := DefaultHeadOptions() - opts.ChunkRange = 1000 - opts.ChunkDirRoot = chunkDir - h, err := NewHead(nil, nil, nil, nil, opts, nil) - require.NoError(b, err) - defer h.Close() - - var count atomic.Int64 - - b.RunParallel(func(pb *testing.PB) { - for pb.Next() { - i := count.Inc() - h.getOrCreate(uint64(i), labels.FromStrings("a", strconv.Itoa(int(i))), false) - } - }) -} - -func BenchmarkHeadStripeSeriesCreate_PreCreationFailure(b *testing.B) { - chunkDir := b.TempDir() - // Put a series, select it. GC it and then access it. - opts := DefaultHeadOptions() - opts.ChunkRange = 1000 - opts.ChunkDirRoot = chunkDir - - // Mock the PreCreation() callback to fail on each series. - opts.SeriesCallback = failingSeriesLifecycleCallback{} - - h, err := NewHead(nil, nil, nil, nil, opts, nil) - require.NoError(b, err) - defer h.Close() - - for i := 0; b.Loop(); i++ { - h.getOrCreate(uint64(i), labels.FromStrings("a", strconv.Itoa(i)), false) - } -} - -func BenchmarkHead_WalCommit(b *testing.B) { - seriesCounts := []int{100, 1000, 10000} - series := genSeries(10000, 10, 0, 0) // Only using the generated labels. - - appendSamples := func(b *testing.B, app storage.Appender, seriesCount int, ts int64) { - var err error - for i, s := range series[:seriesCount] { - var ref storage.SeriesRef - // if i is even, append a sample, else append a histogram. - if i%2 == 0 { - ref, err = app.Append(ref, s.Labels(), ts, float64(ts)) - } else { - h := &histogram.Histogram{ - Count: 7 + uint64(ts*5), - ZeroCount: 2 + uint64(ts), - ZeroThreshold: 0.001, - Sum: 18.4 * rand.Float64(), - Schema: 1, - PositiveSpans: []histogram.Span{ - {Offset: 0, Length: 2}, - {Offset: 1, Length: 2}, - }, - PositiveBuckets: []int64{ts + 1, 1, -1, 0}, - } - ref, err = app.AppendHistogram(ref, s.Labels(), ts, h, nil) - } - require.NoError(b, err) - - _, err = app.AppendExemplar(ref, s.Labels(), exemplar.Exemplar{ - Labels: labels.FromStrings("trace_id", strconv.Itoa(rand.Int())), - Value: rand.Float64(), - Ts: ts, - }) - require.NoError(b, err) - } - } - - for _, seriesCount := range seriesCounts { - b.Run(fmt.Sprintf("%d series", seriesCount), func(b *testing.B) { - for _, commits := range []int64{1, 2} { // To test commits that create new series and when the series already exists. - b.Run(fmt.Sprintf("%d commits", commits), func(b *testing.B) { - b.ReportAllocs() - b.ResetTimer() - - for b.Loop() { - b.StopTimer() - h, w := newTestHead(b, 10000, compression.None, false) - b.Cleanup(func() { - if h != nil { - h.Close() - } - if w != nil { - w.Close() - } - }) - app := h.Appender(context.Background()) - - appendSamples(b, app, seriesCount, 0) - - b.StartTimer() - require.NoError(b, app.Commit()) - if commits == 2 { - b.StopTimer() - app = h.Appender(context.Background()) - appendSamples(b, app, seriesCount, 1) - b.StartTimer() - require.NoError(b, app.Commit()) - } - b.StopTimer() - h.Close() - h = nil - w.Close() - w = nil - } - }) - } - }) - } -} - -type failingSeriesLifecycleCallback struct{} - -func (failingSeriesLifecycleCallback) PreCreation(labels.Labels) error { return errors.New("failed") } -func (failingSeriesLifecycleCallback) PostCreation(labels.Labels) {} -func (failingSeriesLifecycleCallback) PostDeletion(map[chunks.HeadSeriesRef]labels.Labels) {} diff --git a/tsdb/head_test.go b/tsdb/head_test.go index 552db13d07..fefe0f200d 100644 --- a/tsdb/head_test.go +++ b/tsdb/head_test.go @@ -107,49 +107,6 @@ func BenchmarkCreateSeries(b *testing.B) { } } -func BenchmarkHeadAppender_Append_Commit_ExistingSeries(b *testing.B) { - seriesCounts := []int{100, 1000, 10000} - series := genSeries(10000, 10, 0, 0) - - for _, seriesCount := range seriesCounts { - b.Run(fmt.Sprintf("%d series", seriesCount), func(b *testing.B) { - for _, samplesPerAppend := range []int64{1, 2, 5, 100} { - b.Run(fmt.Sprintf("%d samples per append", samplesPerAppend), func(b *testing.B) { - h, _ := newTestHead(b, 10000, compression.None, false) - b.Cleanup(func() { require.NoError(b, h.Close()) }) - - ts := int64(1000) - appendSamples := func() error { - var err error - app := h.Appender(context.Background()) - for _, s := range series[:seriesCount] { - var ref storage.SeriesRef - for sampleIndex := range samplesPerAppend { - ref, err = app.Append(ref, s.Labels(), ts+sampleIndex, float64(ts+sampleIndex)) - if err != nil { - return err - } - } - } - ts += 1000 // should increment more than highest samplesPerAppend - return app.Commit() - } - - // Init series, that's not what we're benchmarking here. - require.NoError(b, appendSamples()) - - b.ReportAllocs() - b.ResetTimer() - - for b.Loop() { - require.NoError(b, appendSamples()) - } - }) - } - }) - } -} - func populateTestWL(t testing.TB, w *wlog.WL, recs []any, buf []byte) []byte { var enc record.Encoder for _, r := range recs { @@ -5941,7 +5898,7 @@ func TestOOOAppendWithNoSeries(t *testing.T) { } } -func testOOOAppendWithNoSeries(t *testing.T, appendFunc func(appender storage.Appender, lbls labels.Labels, ts, value int64) (storage.SeriesRef, sample, error)) { +func testOOOAppendWithNoSeries(t *testing.T, appendFunc func(appender storage.LimitedAppenderV1, lbls labels.Labels, ts, value int64) (storage.SeriesRef, sample, error)) { dir := t.TempDir() wal, err := wlog.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, compression.Snappy) require.NoError(t, err) @@ -6284,6 +6241,7 @@ func TestSnapshotAheadOfWALError(t *testing.T) { require.NoError(t, head.Close()) } +// TODO(bwplotka): Bad benchmark (no b.Loop/b.N), fix or remove. func BenchmarkCuttingHeadHistogramChunks(b *testing.B) { const ( numSamples = 50000 diff --git a/tsdb/testutil.go b/tsdb/testutil.go index 4d413322c8..d41591750b 100644 --- a/tsdb/testutil.go +++ b/tsdb/testutil.go @@ -44,14 +44,14 @@ type testValue struct { type sampleTypeScenario struct { sampleType string - appendFunc func(appender storage.Appender, lbls labels.Labels, ts, value int64) (storage.SeriesRef, sample, error) + appendFunc func(appender storage.LimitedAppenderV1, lbls labels.Labels, ts, value int64) (storage.SeriesRef, sample, error) sampleFunc func(ts, value int64) sample } var sampleTypeScenarios = map[string]sampleTypeScenario{ float: { sampleType: sampleMetricTypeFloat, - appendFunc: func(appender storage.Appender, lbls labels.Labels, ts, value int64) (storage.SeriesRef, sample, error) { + appendFunc: func(appender storage.LimitedAppenderV1, lbls labels.Labels, ts, value int64) (storage.SeriesRef, sample, error) { s := sample{t: ts, f: float64(value)} ref, err := appender.Append(0, lbls, ts, s.f) return ref, s, err @@ -62,7 +62,7 @@ var sampleTypeScenarios = map[string]sampleTypeScenario{ }, intHistogram: { sampleType: sampleMetricTypeHistogram, - appendFunc: func(appender storage.Appender, lbls labels.Labels, ts, value int64) (storage.SeriesRef, sample, error) { + appendFunc: func(appender storage.LimitedAppenderV1, lbls labels.Labels, ts, value int64) (storage.SeriesRef, sample, error) { s := sample{t: ts, h: tsdbutil.GenerateTestHistogram(value)} ref, err := appender.AppendHistogram(0, lbls, ts, s.h, nil) return ref, s, err @@ -73,7 +73,7 @@ var sampleTypeScenarios = map[string]sampleTypeScenario{ }, floatHistogram: { sampleType: sampleMetricTypeHistogram, - appendFunc: func(appender storage.Appender, lbls labels.Labels, ts, value int64) (storage.SeriesRef, sample, error) { + appendFunc: func(appender storage.LimitedAppenderV1, lbls labels.Labels, ts, value int64) (storage.SeriesRef, sample, error) { s := sample{t: ts, fh: tsdbutil.GenerateTestFloatHistogram(value)} ref, err := appender.AppendHistogram(0, lbls, ts, nil, s.fh) return ref, s, err @@ -84,7 +84,7 @@ var sampleTypeScenarios = map[string]sampleTypeScenario{ }, customBucketsIntHistogram: { sampleType: sampleMetricTypeHistogram, - appendFunc: func(appender storage.Appender, lbls labels.Labels, ts, value int64) (storage.SeriesRef, sample, error) { + appendFunc: func(appender storage.LimitedAppenderV1, lbls labels.Labels, ts, value int64) (storage.SeriesRef, sample, error) { s := sample{t: ts, h: tsdbutil.GenerateTestCustomBucketsHistogram(value)} ref, err := appender.AppendHistogram(0, lbls, ts, s.h, nil) return ref, s, err @@ -95,7 +95,7 @@ var sampleTypeScenarios = map[string]sampleTypeScenario{ }, customBucketsFloatHistogram: { sampleType: sampleMetricTypeHistogram, - appendFunc: func(appender storage.Appender, lbls labels.Labels, ts, value int64) (storage.SeriesRef, sample, error) { + appendFunc: func(appender storage.LimitedAppenderV1, lbls labels.Labels, ts, value int64) (storage.SeriesRef, sample, error) { s := sample{t: ts, fh: tsdbutil.GenerateTestCustomBucketsFloatHistogram(value)} ref, err := appender.AppendHistogram(0, lbls, ts, nil, s.fh) return ref, s, err @@ -106,7 +106,7 @@ var sampleTypeScenarios = map[string]sampleTypeScenario{ }, gaugeIntHistogram: { sampleType: sampleMetricTypeHistogram, - appendFunc: func(appender storage.Appender, lbls labels.Labels, ts, value int64) (storage.SeriesRef, sample, error) { + appendFunc: func(appender storage.LimitedAppenderV1, lbls labels.Labels, ts, value int64) (storage.SeriesRef, sample, error) { s := sample{t: ts, h: tsdbutil.GenerateTestGaugeHistogram(value)} ref, err := appender.AppendHistogram(0, lbls, ts, s.h, nil) return ref, s, err @@ -117,7 +117,7 @@ var sampleTypeScenarios = map[string]sampleTypeScenario{ }, gaugeFloatHistogram: { sampleType: sampleMetricTypeHistogram, - appendFunc: func(appender storage.Appender, lbls labels.Labels, ts, value int64) (storage.SeriesRef, sample, error) { + appendFunc: func(appender storage.LimitedAppenderV1, lbls labels.Labels, ts, value int64) (storage.SeriesRef, sample, error) { s := sample{t: ts, fh: tsdbutil.GenerateTestGaugeFloatHistogram(value)} ref, err := appender.AppendHistogram(0, lbls, ts, nil, s.fh) return ref, s, err