From 04696703fe253001571b4dbcb012a575f3eec18b Mon Sep 17 00:00:00 2001 From: bwplotka Date: Fri, 28 Nov 2025 12:13:38 +0000 Subject: [PATCH 1/6] refactor(appenderV2): add AppenderV2 interface Signed-off-by: bwplotka --- storage/interface.go | 47 ++++++---- storage/interface_append.go | 169 ++++++++++++++++++++++++++++++++++++ 2 files changed, 198 insertions(+), 18 deletions(-) create mode 100644 storage/interface_append.go diff --git a/storage/interface.go b/storage/interface.go index 19b4db4210..fe9b3fa6e8 100644 --- a/storage/interface.go +++ b/storage/interface.go @@ -58,11 +58,14 @@ var ( // their own reference types. type SeriesRef uint64 -// Appendable allows creating appenders. +// Appendable allows creating Appender. +// +// WARNING: Work AppendableV2 is in progress. Appendable will be removed soon (ETA: Q2 2026). type Appendable interface { - // Appender returns a new appender for the storage. The implementation - // can choose whether or not to use the context, for deadlines or to check - // for errors. + // Appender returns a new appender for the storage. + // + // Implementations CAN choose whether to use the context e.g. for deadlines, + // but it's not mandatory. Appender(ctx context.Context) Appender } @@ -255,7 +258,13 @@ func (f QueryableFunc) Querier(mint, maxt int64) (Querier, error) { return f(mint, maxt) } +// AppendOptions provides options for implementations of the Appender interface. +// +// WARNING: Work AppendableV2 is in progress. Appendable will be removed soon (ETA: Q2 2026). type AppendOptions struct { + // DiscardOutOfOrder tells implementation that this append should not be out + // of order. An OOO append MUST be rejected with storage.ErrOutOfOrderSample + // error. DiscardOutOfOrder bool } @@ -267,7 +276,11 @@ type AppendOptions struct { // The order of samples appended via the Appender is preserved within each // series. I.e. samples are not reordered per timestamp, or by float/histogram // type. +// +// WARNING: Work AppendableV2 is in progress. Appendable will be removed soon (ETA: Q2 2026). type Appender interface { + AppenderTransaction + // Append adds a sample pair for the given series. // An optional series reference can be provided to accelerate calls. // A series reference number is returned which can be used to add further @@ -278,16 +291,6 @@ type Appender interface { // If the reference is 0 it must not be used for caching. Append(ref SeriesRef, l labels.Labels, t int64, v float64) (SeriesRef, error) - // Commit submits the collected samples and purges the batch. If Commit - // returns a non-nil error, it also rolls back all modifications made in - // the appender so far, as Rollback would do. In any case, an Appender - // must not be used anymore after Commit has been called. - Commit() error - - // Rollback rolls back all modifications made in the appender so far. - // Appender has to be discarded after rollback. - Rollback() error - // SetOptions configures the appender with specific append options such as // discarding out-of-order samples even if out-of-order is enabled in the TSDB. SetOptions(opts *AppendOptions) @@ -301,8 +304,8 @@ type Appender interface { // GetRef is an extra interface on Appenders used by downstream projects // (e.g. Cortex) to avoid maintaining a parallel set of references. type GetRef interface { - // Returns reference number that can be used to pass to Appender.Append(), - // and a set of labels that will not cause another copy when passed to Appender.Append(). + // GetRef returns a reference number that can be used to pass to AppenderV2.Append(), + // and a set of labels that will not cause another copy when passed to AppenderV2.Append(). // 0 means the appender does not have a reference to this series. // hash should be a hash of lset. GetRef(lset labels.Labels, hash uint64) (SeriesRef, labels.Labels) @@ -310,6 +313,8 @@ type GetRef interface { // ExemplarAppender provides an interface for adding samples to exemplar storage, which // within Prometheus is in-memory only. +// +// WARNING: Work AppendableV2 is in progress. Appendable will be removed soon (ETA: Q2 2026). type ExemplarAppender interface { // AppendExemplar adds an exemplar for the given series labels. // An optional reference number can be provided to accelerate calls. @@ -326,6 +331,8 @@ type ExemplarAppender interface { } // HistogramAppender provides an interface for appending histograms to the storage. +// +// WARNING: Work AppendableV2 is in progress. Appendable will be removed soon (ETA: Q2 2026). type HistogramAppender interface { // AppendHistogram adds a histogram for the given series labels. An // optional reference number can be provided to accelerate calls. A @@ -356,6 +363,8 @@ type HistogramAppender interface { } // MetadataUpdater provides an interface for associating metadata to stored series. +// +// WARNING: Work AppendableV2 is in progress. Appendable will be removed soon (ETA: Q2 2026). type MetadataUpdater interface { // UpdateMetadata updates a metadata entry for the given series and labels. // A series reference number is returned which can be used to modify the @@ -368,6 +377,8 @@ type MetadataUpdater interface { } // StartTimestampAppender provides an interface for appending ST to storage. +// +// WARNING: Work AppendableV2 is in progress. Appendable will be removed soon (ETA: Q2 2026). type StartTimestampAppender interface { // AppendSTZeroSample adds synthetic zero sample for the given st timestamp, // which will be associated with given series, labels and the incoming @@ -390,10 +401,10 @@ type SeriesSet interface { Next() bool // At returns full series. Returned series should be iterable even after Next is called. At() Series - // The error that iteration has failed with. + // Err returns the error that iteration has failed with. // When an error occurs, set cannot continue to iterate. Err() error - // A collection of warnings for the whole set. + // Warnings returns a collection of warnings for the whole set. // Warnings could be return even iteration has not failed with error. Warnings() annotations.Annotations } diff --git a/storage/interface_append.go b/storage/interface_append.go new file mode 100644 index 0000000000..c8d1b46ce8 --- /dev/null +++ b/storage/interface_append.go @@ -0,0 +1,169 @@ +package storage + +import ( + "context" + "errors" + + "github.com/prometheus/prometheus/model/exemplar" + "github.com/prometheus/prometheus/model/histogram" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/model/metadata" +) + +// AppendableV2 allows creating AppenderV2. +type AppendableV2 interface { + // AppenderV2 returns a new appender for the storage. + // + // Implementations CAN choose whether to use the context e.g. for deadlines, + // but it's not mandatory. + AppenderV2(ctx context.Context) AppenderV2 +} + +// AOptions is a shorthand for AppendV2Options. +// NOTE: AppendOption is used already. +type AOptions = AppendV2Options + +// AppendV2Options provides optional, auxiliary data and configuration for AppenderV2.Append. +type AppendV2Options struct { + // MetricFamilyName (optional) provides metric family name for the appended sample's + // series. If the client of the AppenderV2 has this information + // (e.g. from scrape) it's recommended to pass it to the appender. + // + // Provided string bytes are unsafe to reuse, it only lives for the duration of the Append call. + // + // Some implementations use this to avoid slow and prone to error metric family detection for: + // * Metadata per metric family storages (e.g. Prometheus metadata WAL/API/RW1) + // * Strictly complex types storages (e.g. OpenTelemetry Collector). + // + // NOTE(krajorama): Example purpose is highlighted in OTLP ingestion: OTLP calculates the + // metric family name for all metrics and uses it for generating summary, + // histogram series by adding the magic suffixes. The metric family name is + // passed down to the appender in case the storage needs it for metadata updates. + // Known user of this is Mimir that implements /api/v1/metadata and uses + // Remote-Write 1.0 for this. Might be removed later if no longer + // needed by any downstream project. + // NOTE(bwplotka): Long term, once Prometheus uses complex types on storage level + // the MetricFamilyName can be removed as MetricFamilyName will equal to __name__ always. + MetricFamilyName string + + // Metadata (optional) attached to the appended sample. + // Metadata strings are safe for reuse. + // IMPORTANT: Appender v1 was only providing update. This field MUST be + // set (if known) even if it didn't change since the last iteration. + // This moves the responsibility for metadata storage options to TSDB. + Metadata metadata.Metadata + + // Exemplars (optional) attached to the appended sample. + // Exemplar slice MUST be sorted by Exemplar.TS. + // Exemplar slice is unsafe for reuse. + Exemplars []exemplar.Exemplar + + // RejectOutOfOrder tells implementation that this append should not be out + // of order. An OOO append MUST be rejected with storage.ErrOutOfOrderSample + // error. + RejectOutOfOrder bool +} + +// AppendPartialError represents an AppenderV2.Append error that tells +// callers sample was written but some auxiliary optional data (e.g. exemplars) +// was not (or partially written) +// +// It's up to the caller to decide if it's an ignorable error or not, plus +// it allows extra reporting (e.g. for Remote Write 2.0 X-Remote-Write-Written headers). +type AppendPartialError struct { + ExemplarErrors []error +} + +// Error returns combined error string. +func (e *AppendPartialError) Error() string { + errs := errors.Join(e.ExemplarErrors...) + if errs == nil { + return "" + } + return errs.Error() +} + +var _ error = &AppendPartialError{} + +// AppenderV2 provides appends against a storage for all types of samples. +// It must be completed with a call to Commit or Rollback and must not be reused afterwards. +// +// Operations on the AppenderV2 interface are not goroutine-safe. +// +// The order of samples appended via the AppenderV2 is preserved within each +// series. I.e. samples are not reordered per timestamp, or by float/histogram +// type. +type AppenderV2 interface { + AppenderTransaction + + // Append appends a sample and related exemplars, metadata, and start timestamp (st) to the storage. + // + // ref (optional) represents the stable ID for the given series identified by ls (excluding metadata). + // Callers MAY provide the ref to help implementation avoid ls -> ref computation, otherwise ref MUST be 0 (unknown). + // + // ls represents labels for the sample's series. + // + // st (optional) represents sample start timestamp. 0 means unknown. Implementations + // are responsible for any potential ST storage logic (e.g. ST zero injections). + // + // t represents sample timestamp. + // + // v, h, fh represents sample value for each sample type. + // Callers MUST only provide one of the sample types (either v, h or fh). + // Implementations can detect the type of the sample with the following switch: + // + // switch { + // case fh != nil: It's a float histogram append. + // case h != nil: It's a histogram append. + // default: It's a float append. + // } + // TODO(bwplotka): We plan to experiment on using generics for complex sampleType, but do it after we unify interface (derisk) and before we add native summaries. + // + // Implementations MUST attempt to append sample even if metadata, exemplar or (st) start timestamp appends fail. + // Implementations MAY return AppendPartialError as an error. Use errors.As to detect. + // For the successful Append, Implementations MUST return valid SeriesRef that represents ls. + // NOTE(bwplotka): Given OTLP and native histograms and the relaxation of the requirement for + // type and unit suffixes in metric names we start to hit cases of ls being not enough for id + // of the series (metadata matters). Current solution is to enable 'type-and-unit-label' features for those cases, but we may + // start to extend the id with metadata one day. + Append(ref SeriesRef, ls labels.Labels, st, t int64, v float64, h *histogram.Histogram, fh *histogram.FloatHistogram, opts AppendV2Options) (SeriesRef, error) +} + +// AppenderTransaction allows transactional appends. +type AppenderTransaction interface { + // Commit submits the collected samples and purges the batch. If Commit + // returns a non-nil error, it also rolls back all modifications made in + // the appender so far, as Rollback would do. In any case, an Appender + // must not be used anymore after Commit has been called. + Commit() error + + // Rollback rolls back all modifications made in the appender so far. + // Appender has to be discarded after rollback. + Rollback() error +} + +// LimitedAppenderV1 is an Appender that only supports appending float and histogram samples. +// This is to support migration to AppenderV2. +// TODO(bwplotka): Remove once migration to AppenderV2 is fully complete. +type LimitedAppenderV1 interface { + Append(ref SeriesRef, l labels.Labels, t int64, v float64) (SeriesRef, error) + AppendHistogram(ref SeriesRef, l labels.Labels, t int64, h *histogram.Histogram, fh *histogram.FloatHistogram) (SeriesRef, error) +} + +// AppenderV2AsLimitedV1 returns appender that exposes AppenderV2 as LimitedAppenderV1 +// TODO(bwplotka): Remove once migration to AppenderV2 is fully complete. +func AppenderV2AsLimitedV1(app AppenderV2) LimitedAppenderV1 { + return &limitedAppenderV1{AppenderV2: app} +} + +type limitedAppenderV1 struct { + AppenderV2 +} + +func (a *limitedAppenderV1) Append(ref SeriesRef, l labels.Labels, t int64, v float64) (SeriesRef, error) { + return a.AppenderV2.Append(ref, l, 0, t, v, nil, nil, AppendV2Options{}) +} + +func (a *limitedAppenderV1) AppendHistogram(ref SeriesRef, l labels.Labels, t int64, h *histogram.Histogram, fh *histogram.FloatHistogram) (SeriesRef, error) { + return a.AppenderV2.Append(ref, l, 0, t, 0, h, fh, AppendV2Options{}) +} From 129650df9d910e6288f37754140f842046a92d60 Mon Sep 17 00:00:00 2001 From: bwplotka Date: Fri, 28 Nov 2025 12:41:41 +0000 Subject: [PATCH 2/6] refactor(appenderV2): 1:1 copy of head_append.go -> head_append_v2.go (starting point) Signed-off-by: bwplotka --- storage/interface_append.go | 13 + tsdb/head_append_v2.go | 2285 +++++++++++++++++++++++++++++++++++ 2 files changed, 2298 insertions(+) create mode 100644 tsdb/head_append_v2.go diff --git a/storage/interface_append.go b/storage/interface_append.go index c8d1b46ce8..880e57f194 100644 --- a/storage/interface_append.go +++ b/storage/interface_append.go @@ -1,3 +1,16 @@ +// 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 +// +// 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 storage import ( diff --git a/tsdb/head_append_v2.go b/tsdb/head_append_v2.go new file mode 100644 index 0000000000..942c3ce974 --- /dev/null +++ b/tsdb/head_append_v2.go @@ -0,0 +1,2285 @@ +// Copyright 2021 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" + "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 +// upon the first sample it receives. +type initAppender struct { + app storage.Appender + head *Head +} + +var _ storage.GetRef = &initAppender{} + +func (a *initAppender) SetOptions(opts *storage.AppendOptions) { + if a.app != nil { + a.app.SetOptions(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) { + if g, ok := a.app.(storage.GetRef); ok { + return g.GetRef(lset, hash) + } + return 0, labels.EmptyLabels() +} + +func (a *initAppender) Commit() error { + if a.app == nil { + a.head.metrics.activeAppenders.Dec() + return nil + } + return a.app.Commit() +} + +func (a *initAppender) Rollback() error { + if a.app == nil { + a.head.metrics.activeAppenders.Dec() + return nil + } + return a.app.Rollback() +} + +// Appender returns a new Appender on the database. +func (h *Head) Appender(context.Context) storage.Appender { + 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{ + head: h, + } + } + return h.appender() +} + +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, + } +} + +// 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) +} + +// 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) { + // 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(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() + return 0, storage.ErrOutOfBounds + } + + if h != nil { + if err := h.Validate(); err != nil { + return 0, err + } + } + + if fh != nil { + if err := fh.Validate(); err != nil { + return 0, err + } + } + + 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 + } + } + + 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 { + switch { + case errors.Is(err, storage.ErrOutOfOrderSample): + a.head.metrics.outOfOrderSamples.WithLabelValues(sampleMetricTypeHistogram).Inc() + case errors.Is(err, storage.ErrTooOldSample): + a.head.metrics.tooOldSamples.WithLabelValues(sampleMetricTypeHistogram).Inc() + } + 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 { + switch { + case errors.Is(err, storage.ErrOutOfOrderSample): + a.head.metrics.outOfOrderSamples.WithLabelValues(sampleMetricTypeHistogram).Inc() + case errors.Is(err, storage.ErrTooOldSample): + a.head.metrics.tooOldSamples.WithLabelValues(sampleMetricTypeHistogram).Inc() + } + return 0, 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) + } + + if t < a.mint { + a.mint = t + } + if t > a.maxt { + 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 + } + + 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 + } + } + + switch { + case h != nil: + zeroHistogram := &histogram.Histogram{ + // The STZeroSample represents a counter reset by definition. + CounterResetHint: histogram.CounterReset, + // Replicate other fields to avoid needless chunk creation. + Schema: h.Schema, + 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) + case fh != nil: + zeroFloatHistogram := &histogram.FloatHistogram{ + // The STZeroSample represents a counter reset by definition. + CounterResetHint: histogram.CounterReset, + // Replicate other fields to avoid needless chunk creation. + Schema: fh.Schema, + 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 + } + + 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 := 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) + } + + 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() +} From efdfb8fed626ebbcddeeb60ea9ba8b29c13dbdf2 Mon Sep 17 00:00:00 2001 From: bwplotka Date: Fri, 28 Nov 2025 12:47:43 +0000 Subject: [PATCH 3/6] refactor(appenderV2): 1:1 copy of head append test files for v2 (starting point) Signed-off-by: bwplotka --- tsdb/head_append_v2_test.go | 7332 +++++++++++++++++++++++++++++++++++ tsdb/head_bench_v2_test.go | 173 + 2 files changed, 7505 insertions(+) create mode 100644 tsdb/head_append_v2_test.go create mode 100644 tsdb/head_bench_v2_test.go diff --git a/tsdb/head_append_v2_test.go b/tsdb/head_append_v2_test.go new file mode 100644 index 0000000000..552db13d07 --- /dev/null +++ b/tsdb/head_append_v2_test.go @@ -0,0 +1,7332 @@ +// Copyright 2017 The Prometheus Authors +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package tsdb + +import ( + "context" + "fmt" + "io" + "math" + "math/rand" + "os" + "path" + "path/filepath" + "reflect" + "slices" + "sort" + "strconv" + "strings" + "sync" + "testing" + "time" + + "github.com/google/go-cmp/cmp" + "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" + + "github.com/prometheus/prometheus/config" + "github.com/prometheus/prometheus/model/exemplar" + "github.com/prometheus/prometheus/model/histogram" + "github.com/prometheus/prometheus/model/labels" + "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/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" + "github.com/prometheus/prometheus/tsdb/wlog" + "github.com/prometheus/prometheus/util/compression" + "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, +// 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) { + head, _ := newTestHead(t, DefaultBlockDuration, compression.None, false) + defer func() { + require.NoError(t, head.Close()) + }() + + seriesCnt := 1000 + readConcurrency := 2 + writeConcurrency := 10 + startTs := uint64(DefaultBlockDuration) // start at the second block relative to the unix epoch. + qryRange := uint64(5 * time.Minute.Milliseconds()) + step := uint64(15 * time.Second / time.Millisecond) + endTs := startTs + uint64(DefaultBlockDuration) + + labelSets := make([]labels.Labels, seriesCnt) + for i := range seriesCnt { + labelSets[i] = labels.FromStrings("seriesId", strconv.Itoa(i)) + } + + head.Init(0) + + g, ctx := errgroup.WithContext(context.Background()) + whileNotCanceled := func(f func() (bool, error)) error { + for ctx.Err() == nil { + cont, err := f() + if err != nil { + return err + } + if !cont { + return nil + } + } + return nil + } + + // Create one channel for each write worker, the channels will be used by the coordinator + // go routine to coordinate which timestamps each write worker has to write. + writerTsCh := make([]chan uint64, writeConcurrency) + for writerTsChIdx := range writerTsCh { + writerTsCh[writerTsChIdx] = make(chan uint64) + } + + // workerReadyWg is used to synchronize the start of the test, + // we only start the test once all workers signal that they're ready. + var workerReadyWg sync.WaitGroup + workerReadyWg.Add(writeConcurrency + readConcurrency) + + // Start the write workers. + for wid := range writeConcurrency { + // Create copy of workerID to be used by worker routine. + workerID := wid + + g.Go(func() error { + // The label sets which this worker will write. + workerLabelSets := labelSets[(seriesCnt/writeConcurrency)*workerID : (seriesCnt/writeConcurrency)*(workerID+1)] + + // Signal that this worker is ready. + workerReadyWg.Done() + + return whileNotCanceled(func() (bool, error) { + ts, ok := <-writerTsCh[workerID] + if !ok { + return false, nil + } + + app := head.Appender(ctx) + for i := range workerLabelSets { + // We also use the timestamp as the sample value. + _, err := app.Append(0, workerLabelSets[i], int64(ts), float64(ts)) + if err != nil { + return false, fmt.Errorf("Error when appending to head: %w", err) + } + } + + return true, app.Commit() + }) + }) + } + + // queryHead is a helper to query the head for a given time range and labelset. + queryHead := func(mint, maxt uint64, label labels.Label) (map[string][]chunks.Sample, error) { + q, err := NewBlockQuerier(head, int64(mint), int64(maxt)) + if err != nil { + return nil, err + } + return query(t, q, labels.MustNewMatcher(labels.MatchEqual, label.Name, label.Value)), nil + } + + // readerTsCh will be used by the coordinator go routine to coordinate which timestamps the reader should read. + readerTsCh := make(chan uint64) + + // Start the read workers. + for wid := range readConcurrency { + // Create copy of threadID to be used by worker routine. + workerID := wid + + g.Go(func() error { + querySeriesRef := (seriesCnt / readConcurrency) * workerID + + // Signal that this worker is ready. + workerReadyWg.Done() + + return whileNotCanceled(func() (bool, error) { + ts, ok := <-readerTsCh + if !ok { + return false, nil + } + + querySeriesRef = (querySeriesRef + 1) % seriesCnt + lbls := labelSets[querySeriesRef] + // lbls has a single entry; extract it so we can run a query. + var lbl labels.Label + lbls.Range(func(l labels.Label) { + lbl = l + }) + samples, err := queryHead(ts-qryRange, ts, lbl) + if err != nil { + return false, err + } + + if len(samples) != 1 { + return false, fmt.Errorf("expected 1 series, got %d", len(samples)) + } + + series := lbls.String() + expectSampleCnt := qryRange/step + 1 + if expectSampleCnt != uint64(len(samples[series])) { + return false, fmt.Errorf("expected %d samples, got %d", expectSampleCnt, len(samples[series])) + } + + for sampleIdx, sample := range samples[series] { + expectedValue := ts - qryRange + (uint64(sampleIdx) * step) + if sample.T() != int64(expectedValue) { + return false, fmt.Errorf("expected sample %d to have ts %d, got %d", sampleIdx, expectedValue, sample.T()) + } + if sample.F() != float64(expectedValue) { + return false, fmt.Errorf("expected sample %d to have value %d, got %f", sampleIdx, expectedValue, sample.F()) + } + } + + return true, nil + }) + }) + } + + // Start the coordinator go routine. + g.Go(func() error { + currTs := startTs + + defer func() { + // End of the test, close all channels to stop the workers. + for _, ch := range writerTsCh { + close(ch) + } + close(readerTsCh) + }() + + // Wait until all workers are ready to start the test. + workerReadyWg.Wait() + return whileNotCanceled(func() (bool, error) { + // Send the current timestamp to each of the writers. + for _, ch := range writerTsCh { + select { + case ch <- currTs: + case <-ctx.Done(): + return false, nil + } + } + + // Once data for at least has been ingested, send the current timestamp to the readers. + if currTs > startTs+qryRange { + select { + case readerTsCh <- currTs - step: + case <-ctx.Done(): + return false, nil + } + } + + currTs += step + if currTs > endTs { + return false, nil + } + + return true, nil + }) + }) + + 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) { + 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) + 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) + 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) + 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) + require.NoError(t, err) + require.NoError(t, app.Commit()) + require.Equal(t, 4.0, prom_testutil.ToFloat64(head.metrics.chunksCreated)) + + require.NotEqual(t, ref1, ref2, "Refs are the same") + require.NoError(t, head.Close()) + + w, err = wlog.New(nil, nil, w.Dir(), compression.None) + require.NoError(t, err) + + opts := DefaultHeadOptions() + opts.ChunkRange = 1000 + opts.ChunkDirRoot = head.opts.ChunkDirRoot + head, err = NewHead(nil, nil, w, nil, opts, nil) + require.NoError(t, err) + require.NoError(t, head.Init(0)) + defer func() { + require.NoError(t, head.Close()) + }() + + q, err := NewBlockQuerier(head, 0, 2100) + require.NoError(t, err) + series := query(t, q, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + // The samples before the new ref should be discarded since Head truncation + // happens only after compacting the Head. + require.Equal(t, map[string][]chunks.Sample{`{foo="bar"}`: { + sample{1700, 3, nil, nil}, + sample{2000, 4, nil, nil}, + }}, 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) { + 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()) + 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()) + 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) + 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) + 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) { + head, _ := newTestHead(t, 1000, compression.None, false) + t.Cleanup(func() { _ = head.Close() }) + require.NoError(t, head.Init(0)) + + const totalSeries = 100_000 + series := make([]labels.Labels, totalSeries) + for i := range totalSeries { + series[i] = labels.FromStrings("foo", strconv.Itoa(i)) + } + done := atomic.NewBool(false) + + go func() { + defer done.Store(true) + app := head.Appender(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) + if err != nil { + t.Errorf("Failed to append: %v", err) + return + } + } + }() + + // Don't check the atomic.Bool on all iterations in order to perform more gc iterations and make the race condition more likely. + for i := 1; i%128 != 0 || !done.Load(); i++ { + head.gc() + } + + 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() }, + } { + t.Run(op, func(t *testing.T) { + chunkRange := time.Hour.Milliseconds() + head, _ := newTestHead(t, chunkRange, compression.None, true) + t.Cleanup(func() { _ = head.Close() }) + + require.NoError(t, head.Init(0)) + + 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) + require.NoError(t, err) + require.NoError(t, app.Commit()) + require.Equal(t, 1, int(head.NumSeries())) + } + + // 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()) + invalidSampleTime := firstSampleTime - chunkRange + _, err := app.Append(0, labels.FromStrings("foo", "bar"), invalidSampleTime, 2) + 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. + // If these assumptions fail after a refactor, feel free to remove them but make sure that the test is still what we intended to test. + require.NotErrorIs(t, err, storage.ErrOutOfBounds, "Failed to append sample shouldn't take the shortcut that returns storage.ErrOutOfBounds") + require.ErrorIs(t, err, storage.ErrTooOldSample, "Failed to append sample should return storage.ErrTooOldSample, because OOO window was enabled but this sample doesn't fall into it.") + // Do commit or rollback, depending on what we're testing. + require.NoError(t, finishTxn(app)) + + // Garbage-collect, since we finished the transaction and series has no samples, it should be collectable. + head.gc() + require.Equal(t, 1, int(head.NumSeries())) + }) + } +} + +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) { + buildSmpls := func(s []int64) []sample { + ss := make([]sample, 0, len(s)) + for _, t := range s { + ss = append(ss, sample{t: t, f: float64(t)}) + } + return ss + } + smplsAll := buildSmpls([]int64{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) + lblDefault := labels.Label{Name: "a", Value: "b"} + lblsDefault := labels.FromStrings("a", "b") + + cases := []struct { + dranges tombstones.Intervals + addSamples []sample // Samples to add after delete. + smplsExp []sample + }{ + { + dranges: tombstones.Intervals{{Mint: 0, Maxt: 3}}, + smplsExp: buildSmpls([]int64{4, 5, 6, 7, 8, 9}), + }, + { + dranges: tombstones.Intervals{{Mint: 1, Maxt: 3}}, + smplsExp: buildSmpls([]int64{0, 4, 5, 6, 7, 8, 9}), + }, + { + dranges: tombstones.Intervals{{Mint: 1, Maxt: 3}, {Mint: 4, Maxt: 7}}, + smplsExp: buildSmpls([]int64{0, 8, 9}), + }, + { + dranges: tombstones.Intervals{{Mint: 1, Maxt: 3}, {Mint: 4, Maxt: 700}}, + smplsExp: buildSmpls([]int64{0}), + }, + { // This case is to ensure that labels and symbols are deleted. + dranges: tombstones.Intervals{{Mint: 0, Maxt: 9}}, + smplsExp: buildSmpls([]int64{}), + }, + { + dranges: tombstones.Intervals{{Mint: 1, Maxt: 3}}, + addSamples: buildSmpls([]int64{11, 13, 15}), + smplsExp: buildSmpls([]int64{0, 4, 5, 6, 7, 8, 9, 11, 13, 15}), + }, + { + // After delete, the appended samples in the deleted range should be visible + // as the tombstones are clamped to head min/max time. + dranges: tombstones.Intervals{{Mint: 7, Maxt: 20}}, + addSamples: buildSmpls([]int64{11, 13, 15}), + smplsExp: buildSmpls([]int64{0, 1, 2, 3, 4, 5, 6, 11, 13, 15}), + }, + } + + for _, compress := range []compression.Type{compression.None, compression.Snappy, compression.Zstd} { + t.Run(fmt.Sprintf("compress=%s", compress), func(t *testing.T) { + for _, c := range cases { + head, w := newTestHead(t, 1000, compress, false) + require.NoError(t, head.Init(0)) + + app := head.Appender(context.Background()) + for _, smpl := range smplsAll { + _, err := app.Append(0, lblsDefault, smpl.t, smpl.f) + require.NoError(t, err) + } + require.NoError(t, app.Commit()) + + // Delete the ranges. + for _, r := range c.dranges { + require.NoError(t, head.Delete(context.Background(), r.Mint, r.Maxt, labels.MustNewMatcher(labels.MatchEqual, lblDefault.Name, lblDefault.Value))) + } + + // Add more samples. + app = head.Appender(context.Background()) + for _, smpl := range c.addSamples { + _, err := app.Append(0, lblsDefault, smpl.t, smpl.f) + require.NoError(t, err) + } + require.NoError(t, app.Commit()) + + // Compare the samples for both heads - before and after the reloadBlocks. + reloadedW, err := wlog.New(nil, nil, w.Dir(), compress) // Use a new wal to ensure deleted samples are gone even after a reloadBlocks. + require.NoError(t, err) + opts := DefaultHeadOptions() + opts.ChunkRange = 1000 + opts.ChunkDirRoot = reloadedW.Dir() + reloadedHead, err := NewHead(nil, nil, reloadedW, nil, opts, nil) + require.NoError(t, err) + require.NoError(t, reloadedHead.Init(0)) + + // Compare the query results for both heads - before and after the reloadBlocks. + Outer: + for _, h := range []*Head{head, reloadedHead} { + q, err := NewBlockQuerier(h, h.MinTime(), h.MaxTime()) + require.NoError(t, err) + actSeriesSet := q.Select(context.Background(), false, nil, labels.MustNewMatcher(labels.MatchEqual, lblDefault.Name, lblDefault.Value)) + require.NoError(t, q.Close()) + expSeriesSet := newMockSeriesSet([]storage.Series{ + storage.NewListSeries(lblsDefault, func() []chunks.Sample { + ss := make([]chunks.Sample, 0, len(c.smplsExp)) + for _, s := range c.smplsExp { + ss = append(ss, s) + } + return ss + }(), + ), + }) + + for { + eok, rok := expSeriesSet.Next(), actSeriesSet.Next() + require.Equal(t, eok, rok) + + if !eok { + require.NoError(t, h.Close()) + require.NoError(t, actSeriesSet.Err()) + require.Empty(t, actSeriesSet.Warnings()) + continue Outer + } + expSeries := expSeriesSet.At() + actSeries := actSeriesSet.At() + + require.Equal(t, expSeries.Labels(), actSeries.Labels()) + + smplExp, errExp := storage.ExpandSamples(expSeries.Iterator(nil), nil) + smplRes, errRes := storage.ExpandSamples(actSeries.Iterator(nil), nil) + + require.Equal(t, errExp, errRes) + require.Equal(t, smplExp, smplRes) + } + } + } + }) + } +} + +func TestDeleteUntilCurMax(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()) + smpls := make([]float64, numSamples) + for i := range numSamples { + smpls[i] = rand.Float64() + _, err := app.Append(0, labels.FromStrings("a", "b"), i, smpls[i]) + require.NoError(t, err) + } + require.NoError(t, app.Commit()) + require.NoError(t, hb.Delete(context.Background(), 0, 10000, labels.MustNewMatcher(labels.MatchEqual, "a", "b"))) + + // Test the series returns no samples. The series is cleared only after compaction. + q, err := NewBlockQuerier(hb, 0, 100000) + require.NoError(t, err) + res := q.Select(context.Background(), false, nil, labels.MustNewMatcher(labels.MatchEqual, "a", "b")) + require.True(t, res.Next(), "series is not present") + s := res.At() + it := s.Iterator(nil) + require.Equal(t, chunkenc.ValNone, it.Next(), "expected no samples") + for res.Next() { + } + require.NoError(t, res.Err()) + 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) + require.NoError(t, err) + require.NoError(t, app.Commit()) + q, err = NewBlockQuerier(hb, 0, 100000) + require.NoError(t, err) + res = q.Select(context.Background(), false, nil, labels.MustNewMatcher(labels.MatchEqual, "a", "b")) + require.True(t, res.Next(), "series don't exist") + exps := res.At() + it = exps.Iterator(nil) + resSamples, err := storage.ExpandSamples(it, newSample) + require.NoError(t, err) + require.Equal(t, []chunks.Sample{sample{11, 1, nil, nil}}, resSamples) + for res.Next() { + } + require.NoError(t, res.Err()) + require.Empty(t, res.Warnings()) +} + +func TestDeletedSamplesAndSeriesStillInWALAfterCheckpoint(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) + require.NoError(t, err) + require.NoError(t, app.Commit()) + } + require.NoError(t, hb.Delete(context.Background(), 0, int64(numSamples), labels.MustNewMatcher(labels.MatchEqual, "a", "b"))) + require.NoError(t, hb.Truncate(1)) + require.NoError(t, hb.Close()) + + // Confirm there's been a checkpoint. + cdir, _, err := wlog.LastCheckpoint(w.Dir()) + require.NoError(t, err) + // Read in checkpoint and WAL. + recs := readTestWAL(t, cdir) + recs = append(recs, readTestWAL(t, w.Dir())...) + + var series, samples, stones, metadata int + for _, rec := range recs { + switch rec.(type) { + case []record.RefSeries: + series++ + case []record.RefSample: + samples++ + case []tombstones.Stone: + stones++ + case []record.RefMetadata: + metadata++ + default: + require.Fail(t, "unknown record type") + } + } + require.Equal(t, 1, series) + require.Equal(t, 9999, samples) + require.Equal(t, 1, stones) + require.Equal(t, 0, metadata) +} + +func TestDelete_e2e(t *testing.T) { + numDatapoints := 1000 + numRanges := 1000 + timeInterval := int64(2) + // Create 8 series with 1000 data-points of different ranges, delete and run queries. + lbls := [][]labels.Label{ + { + {Name: "a", Value: "b"}, + {Name: "instance", Value: "localhost:9090"}, + {Name: "job", Value: "prometheus"}, + }, + { + {Name: "a", Value: "b"}, + {Name: "instance", Value: "127.0.0.1:9090"}, + {Name: "job", Value: "prometheus"}, + }, + { + {Name: "a", Value: "b"}, + {Name: "instance", Value: "127.0.0.1:9090"}, + {Name: "job", Value: "prom-k8s"}, + }, + { + {Name: "a", Value: "b"}, + {Name: "instance", Value: "localhost:9090"}, + {Name: "job", Value: "prom-k8s"}, + }, + { + {Name: "a", Value: "c"}, + {Name: "instance", Value: "localhost:9090"}, + {Name: "job", Value: "prometheus"}, + }, + { + {Name: "a", Value: "c"}, + {Name: "instance", Value: "127.0.0.1:9090"}, + {Name: "job", Value: "prometheus"}, + }, + { + {Name: "a", Value: "c"}, + {Name: "instance", Value: "127.0.0.1:9090"}, + {Name: "job", Value: "prom-k8s"}, + }, + { + {Name: "a", Value: "c"}, + {Name: "instance", Value: "localhost:9090"}, + {Name: "job", Value: "prom-k8s"}, + }, + } + seriesMap := map[string][]chunks.Sample{} + for _, l := range lbls { + seriesMap[labels.New(l...).String()] = []chunks.Sample{} + } + + hb, _ := newTestHead(t, 100000, compression.None, false) + defer func() { + require.NoError(t, hb.Close()) + }() + + app := hb.Appender(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) + require.NoError(t, err) + series = append(series, sample{ts, v, nil, nil}) + ts += rand.Int63n(timeInterval) + 1 + } + seriesMap[labels.New(l...).String()] = series + } + require.NoError(t, app.Commit()) + // Delete a time-range from each-selector. + dels := []struct { + ms []*labels.Matcher + drange tombstones.Intervals + }{ + { + ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "b")}, + drange: tombstones.Intervals{{Mint: 300, Maxt: 500}, {Mint: 600, Maxt: 670}}, + }, + { + ms: []*labels.Matcher{ + labels.MustNewMatcher(labels.MatchEqual, "a", "b"), + labels.MustNewMatcher(labels.MatchEqual, "job", "prom-k8s"), + }, + drange: tombstones.Intervals{{Mint: 300, Maxt: 500}, {Mint: 100, Maxt: 670}}, + }, + { + ms: []*labels.Matcher{ + labels.MustNewMatcher(labels.MatchEqual, "a", "c"), + labels.MustNewMatcher(labels.MatchEqual, "instance", "localhost:9090"), + labels.MustNewMatcher(labels.MatchEqual, "job", "prometheus"), + }, + drange: tombstones.Intervals{{Mint: 300, Maxt: 400}, {Mint: 100, Maxt: 6700}}, + }, + // TODO: Add Regexp Matchers. + } + for _, del := range dels { + for _, r := range del.drange { + require.NoError(t, hb.Delete(context.Background(), r.Mint, r.Maxt, del.ms...)) + } + matched := labels.Slice{} + for _, l := range lbls { + s := labels.Selector(del.ms) + ls := labels.New(l...) + if s.Matches(ls) { + matched = append(matched, ls) + } + } + sort.Sort(matched) + for range numRanges { + q, err := NewBlockQuerier(hb, 0, 100000) + require.NoError(t, err) + ss := q.Select(context.Background(), true, nil, del.ms...) + // Build the mockSeriesSet. + matchedSeries := make([]storage.Series, 0, len(matched)) + for _, m := range matched { + smpls := seriesMap[m.String()] + smpls = deletedSamples(smpls, del.drange) + // Only append those series for which samples exist as mockSeriesSet + // doesn't skip series with no samples. + // TODO: But sometimes SeriesSet returns an empty chunkenc.Iterator + if len(smpls) > 0 { + matchedSeries = append(matchedSeries, storage.NewListSeries(m, smpls)) + } + } + expSs := newMockSeriesSet(matchedSeries) + // Compare both SeriesSets. + for { + eok, rok := expSs.Next(), ss.Next() + // Skip a series if iterator is empty. + if rok { + for ss.At().Iterator(nil).Next() == chunkenc.ValNone { + rok = ss.Next() + if !rok { + break + } + } + } + require.Equal(t, eok, rok) + if !eok { + break + } + sexp := expSs.At() + sres := ss.At() + require.Equal(t, sexp.Labels(), sres.Labels()) + smplExp, errExp := storage.ExpandSamples(sexp.Iterator(nil), nil) + smplRes, errRes := storage.ExpandSamples(sres.Iterator(nil), nil) + require.Equal(t, errExp, errRes) + require.Equal(t, smplExp, smplRes) + } + require.NoError(t, ss.Err()) + require.Empty(t, ss.Warnings()) + require.NoError(t, q.Close()) + } + } +} + +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) { + h, _ := newTestHead(t, 1000, compression.None, false) + defer func() { + require.NoError(t, h.Close()) + }() + + h.initTime(0) + + app := h.appender() + lset := labels.FromStrings("a", "1") + _, err := app.Append(0, lset, 2100, 1) + require.NoError(t, err) + + require.NoError(t, h.Truncate(2000)) + require.NotNil(t, h.series.getByHash(lset.Hash(), lset), "series should not have been garbage collected") + + require.NoError(t, app.Commit()) + + q, err := NewBlockQuerier(h, 1500, 2500) + require.NoError(t, err) + defer q.Close() + + ss := q.Select(context.Background(), false, nil, labels.MustNewMatcher(labels.MatchEqual, "a", "1")) + require.True(t, ss.Next()) + for ss.Next() { + } + require.NoError(t, ss.Err()) + require.Empty(t, ss.Warnings()) +} + +func TestRemoveSeriesAfterRollbackAndTruncate(t *testing.T) { + h, _ := newTestHead(t, 1000, compression.None, false) + defer func() { + require.NoError(t, h.Close()) + }() + + h.initTime(0) + + app := h.appender() + lset := labels.FromStrings("a", "1") + _, err := app.Append(0, lset, 2100, 1) + require.NoError(t, err) + + require.NoError(t, h.Truncate(2000)) + require.NotNil(t, h.series.getByHash(lset.Hash(), lset), "series should not have been garbage collected") + + require.NoError(t, app.Rollback()) + + q, err := NewBlockQuerier(h, 1500, 2500) + require.NoError(t, err) + + ss := q.Select(context.Background(), false, nil, labels.MustNewMatcher(labels.MatchEqual, "a", "1")) + require.False(t, ss.Next()) + require.Empty(t, ss.Warnings()) + require.NoError(t, q.Close()) + + // Truncate again, this time the series should be deleted + require.NoError(t, h.Truncate(2050)) + require.Equal(t, (*memSeries)(nil), h.series.getByHash(lset.Hash(), lset)) +} + +func TestHead_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) + defer func() { + require.NoError(t, h.Close()) + }() + + app := h.Appender(context.Background()) + _, err := app.Append(0, labels.FromStrings("a", "b"), 1, 2) + require.NoError(t, err) + + require.NoError(t, app.Rollback()) + recs := readTestWAL(t, w.Dir()) + + require.Len(t, recs, 1) + + series, ok := recs[0].([]record.RefSeries) + require.True(t, ok, "expected series record but got %+v", recs[0]) + require.Equal(t, []record.RefSeries{{Ref: 1, Labels: labels.FromStrings("a", "b")}}, series) + }) + } +} + +func TestHead_ReturnsSortedLabelValues(t *testing.T) { + h, _ := newTestHead(t, 1000, compression.None, false) + defer func() { + require.NoError(t, h.Close()) + }() + + h.initTime(0) + + app := h.appender() + 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) + require.NoError(t, err) + } + } + + q, err := NewBlockQuerier(h, 1500, 2500) + require.NoError(t, err) + + res, _, err := q.LabelValues(context.Background(), "__name__", nil) + require.NoError(t, err) + + require.True(t, slices.IsSorted(res)) + 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) { + 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) + require.NoError(t, err) + require.NoError(t, app.Commit()) + } + + add(0) + _, last, err := wlog.Segments(wal.Dir()) + require.NoError(t, err) + require.Equal(t, 0, last) + + add(1) + require.NoError(t, h.Truncate(1)) + _, last, err = wlog.Segments(wal.Dir()) + require.NoError(t, err) + require.Equal(t, 1, last) + + add(2) + require.NoError(t, h.Truncate(2)) + _, last, err = wlog.Segments(wal.Dir()) + require.NoError(t, err) + require.Equal(t, 2, last) +} + +func TestAddDuplicateLabelName(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) + require.EqualError(t, err, fmt.Sprintf(`label name "%s" is not unique: invalid sample`, labelName)) + } + + add(labels.FromStrings("a", "c", "a", "b"), "a") + add(labels.FromStrings("a", "c", "a", "c"), "a") + add(labels.FromStrings("__name__", "up", "job", "prometheus", "le", "500", "le", "400", "unit", "s"), "le") +} + +func TestMemSeriesIsolation(t *testing.T) { + if defaultIsolationDisabled { + t.Skip("skipping test since tsdb isolation is disabled") + } + + // Put a series, select it. GC it and then access it. + lastValue := func(h *Head, maxAppendID uint64) int { + idx, err := h.Index() + + require.NoError(t, err) + + iso := h.iso.State(math.MinInt64, math.MaxInt64) + iso.maxAppendID = maxAppendID + + chunks, err := h.chunksRange(math.MinInt64, math.MaxInt64, iso) + require.NoError(t, err) + // Hm.. here direct block chunk querier might be required? + querier := blockQuerier{ + blockBaseQuerier: &blockBaseQuerier{ + index: idx, + chunks: chunks, + tombstones: tombstones.NewMemTombstones(), + + mint: 0, + maxt: 10000, + }, + } + + require.NoError(t, err) + defer querier.Close() + + ss := querier.Select(context.Background(), false, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + _, seriesSet, ws, err := expandSeriesSet(ss) + require.NoError(t, err) + require.Empty(t, ws) + + for _, series := range seriesSet { + return int(series[len(series)-1].f) + } + return -1 + } + + addSamples := func(h *Head) int { + i := 1 + for ; i <= 1000; i++ { + var app storage.Appender + // To initialize bounds. + if h.MinTime() == math.MaxInt64 { + app = &initAppender{head: h} + } else { + a := h.appender() + a.cleanupAppendIDsBelow = 0 + app = a + } + + _, err := app.Append(0, labels.FromStrings("foo", "bar"), int64(i), float64(i)) + require.NoError(t, err) + require.NoError(t, app.Commit()) + h.mmapHeadChunks() + } + return i + } + + testIsolation := func(*Head, int) { + } + + // Test isolation without restart of Head. + hb, _ := newTestHead(t, 1000, compression.None, false) + i := addSamples(hb) + testIsolation(hb, i) + + // Test simple cases in different chunks when no appendID cleanup has been performed. + require.Equal(t, 10, lastValue(hb, 10)) + require.Equal(t, 130, lastValue(hb, 130)) + require.Equal(t, 160, lastValue(hb, 160)) + require.Equal(t, 240, lastValue(hb, 240)) + require.Equal(t, 500, lastValue(hb, 500)) + require.Equal(t, 750, lastValue(hb, 750)) + require.Equal(t, 995, lastValue(hb, 995)) + require.Equal(t, 999, lastValue(hb, 999)) + + // Cleanup appendIDs below 500. + app := hb.appender() + app.cleanupAppendIDsBelow = 500 + _, err := app.Append(0, labels.FromStrings("foo", "bar"), int64(i), float64(i)) + require.NoError(t, err) + require.NoError(t, app.Commit()) + i++ + + // We should not get queries with a maxAppendID below 500 after the cleanup, + // but they only take the remaining appendIDs into account. + require.Equal(t, 499, lastValue(hb, 10)) + require.Equal(t, 499, lastValue(hb, 130)) + require.Equal(t, 499, lastValue(hb, 160)) + require.Equal(t, 499, lastValue(hb, 240)) + require.Equal(t, 500, lastValue(hb, 500)) + require.Equal(t, 995, lastValue(hb, 995)) + require.Equal(t, 999, lastValue(hb, 999)) + + // Cleanup appendIDs below 1000, which means the sample buffer is + // the only thing with appendIDs. + app = hb.appender() + app.cleanupAppendIDsBelow = 1000 + _, err = app.Append(0, labels.FromStrings("foo", "bar"), int64(i), float64(i)) + require.NoError(t, err) + require.NoError(t, app.Commit()) + require.Equal(t, 999, lastValue(hb, 998)) + require.Equal(t, 999, lastValue(hb, 999)) + require.Equal(t, 1000, lastValue(hb, 1000)) + require.Equal(t, 1001, lastValue(hb, 1001)) + require.Equal(t, 1002, lastValue(hb, 1002)) + require.Equal(t, 1002, lastValue(hb, 1003)) + + i++ + // Cleanup appendIDs below 1001, but with a rollback. + app = hb.appender() + app.cleanupAppendIDsBelow = 1001 + _, err = app.Append(0, labels.FromStrings("foo", "bar"), int64(i), float64(i)) + require.NoError(t, err) + require.NoError(t, app.Rollback()) + require.Equal(t, 1000, lastValue(hb, 999)) + require.Equal(t, 1000, lastValue(hb, 1000)) + require.Equal(t, 1001, lastValue(hb, 1001)) + require.Equal(t, 1002, lastValue(hb, 1002)) + require.Equal(t, 1002, lastValue(hb, 1003)) + + require.NoError(t, hb.Close()) + + // Test isolation with restart of Head. This is to verify the num samples of chunks after m-map chunk replay. + hb, w := newTestHead(t, 1000, compression.None, false) + i = addSamples(hb) + require.NoError(t, hb.Close()) + + wal, err := wlog.NewSize(nil, nil, w.Dir(), 32768, compression.None) + require.NoError(t, err) + opts := DefaultHeadOptions() + opts.ChunkRange = 1000 + opts.ChunkDirRoot = wal.Dir() + hb, err = NewHead(nil, nil, wal, nil, opts, nil) + defer func() { require.NoError(t, hb.Close()) }() + require.NoError(t, err) + require.NoError(t, hb.Init(0)) + + // No appends after restarting. Hence all should return the last value. + require.Equal(t, 1000, lastValue(hb, 10)) + require.Equal(t, 1000, lastValue(hb, 130)) + require.Equal(t, 1000, lastValue(hb, 160)) + require.Equal(t, 1000, lastValue(hb, 240)) + require.Equal(t, 1000, lastValue(hb, 500)) + + // 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)) + i++ + require.NoError(t, err) + require.NoError(t, app.Commit()) + require.Equal(t, 1001, lastValue(hb, 998)) + require.Equal(t, 1001, lastValue(hb, 999)) + require.Equal(t, 1001, lastValue(hb, 1000)) + require.Equal(t, 1001, lastValue(hb, 1001)) + require.Equal(t, 1001, lastValue(hb, 1002)) + 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)) + require.NoError(t, err) + require.NoError(t, app.Rollback()) + require.Equal(t, 1001, lastValue(hb, 999)) + require.Equal(t, 1001, lastValue(hb, 1000)) + require.Equal(t, 1001, lastValue(hb, 1001)) + require.Equal(t, 1001, lastValue(hb, 1002)) + require.Equal(t, 1001, lastValue(hb, 1003)) +} + +func TestIsolationRollback(t *testing.T) { + if defaultIsolationDisabled { + t.Skip("skipping test since tsdb isolation is disabled") + } + + // Rollback after a failed append and test if the low watermark has progressed anyway. + hb, _ := newTestHead(t, 1000, compression.None, false) + defer func() { + require.NoError(t, hb.Close()) + }() + + app := hb.Appender(context.Background()) + _, err := app.Append(0, labels.FromStrings("foo", "bar"), 0, 0) + 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) + require.NoError(t, err) + _, err = app.Append(0, labels.FromStrings("foo", "bar", "foo", "baz"), 2, 2) + 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) + 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) { + if defaultIsolationDisabled { + t.Skip("skipping test since tsdb isolation is disabled") + } + + hb, _ := newTestHead(t, 1000, compression.None, false) + defer func() { + require.NoError(t, hb.Close()) + }() + + app1 := hb.Appender(context.Background()) + _, err := app1.Append(0, labels.FromStrings("foo", "bar"), 0, 0) + 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) + 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) + 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.") + + is := hb.iso.State(math.MinInt64, math.MaxInt64) + require.Equal(t, uint64(2), hb.iso.lowWatermark(), "After simulated read (iso state retrieved), low watermark should stay at 2.") + + require.NoError(t, app1.Commit()) + require.Equal(t, uint64(2), hb.iso.lowWatermark(), "Even after app1 is committed, low watermark should stay at 2 because read is still ongoing.") + + is.Close() + 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) { + if defaultIsolationDisabled { + t.Skip("skipping test since tsdb isolation is disabled") + } + + hb, _ := newTestHead(t, 1000, compression.None, false) + defer func() { + require.NoError(t, hb.Close()) + }() + + app := hb.Appender(context.Background()) + require.NoError(t, app.Commit()) + + app = hb.Appender(context.Background()) + _, err := app.Append(0, labels.FromStrings("foo", "baz"), 1, 1) + 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) { + dir := t.TempDir() + db, err := Open(dir, nil, nil, options, nil) + require.NoError(t, err) + defer func() { + require.NoError(t, db.Close()) + }() + db.DisableCompactions() + + appendSample := func(appender storage.Appender, ts int64) (storage.SeriesRef, error) { + ref, _, err := scenario.appendFunc(appender, labels.FromStrings("a", "b"), ts, 99) + return ref, err + } + + ctx := context.Background() + app := db.Appender(ctx) + for i := 1; i <= 5; i++ { + _, err = appendSample(app, int64(i)) + require.NoError(t, err) + } + require.NoError(t, app.Commit()) + + // Test out of order metric. + require.Equal(t, 0.0, prom_testutil.ToFloat64(db.head.metrics.outOfOrderSamples.WithLabelValues(scenario.sampleType))) + app = db.Appender(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))) + + _, err = appendSample(app, 3) + require.Equal(t, expectOutOfOrderError, err) + require.Equal(t, 2.0, prom_testutil.ToFloat64(db.head.metrics.outOfOrderSamples.WithLabelValues(scenario.sampleType))) + + _, err = appendSample(app, 4) + require.Equal(t, expectOutOfOrderError, err) + require.Equal(t, 3.0, prom_testutil.ToFloat64(db.head.metrics.outOfOrderSamples.WithLabelValues(scenario.sampleType))) + require.NoError(t, app.Commit()) + + // Compact Head to test out of bound metric. + app = db.Appender(ctx) + _, err = appendSample(app, DefaultBlockDuration*2) + require.NoError(t, err) + require.NoError(t, app.Commit()) + + require.Equal(t, int64(math.MinInt64), db.head.minValidTime.Load()) + require.NoError(t, db.Compact(ctx)) + require.Positive(t, db.head.minValidTime.Load()) + + app = db.Appender(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))) + + _, err = appendSample(app, db.head.minValidTime.Load()-1) + require.Equal(t, storage.ErrOutOfBounds, err) + require.Equal(t, 2.0, prom_testutil.ToFloat64(db.head.metrics.outOfBoundSamples.WithLabelValues(scenario.sampleType))) + require.NoError(t, app.Commit()) + + // Some more valid samples for out of order. + app = db.Appender(ctx) + for i := 1; i <= 5; i++ { + _, err = appendSample(app, db.head.minValidTime.Load()+DefaultBlockDuration+int64(i)) + require.NoError(t, err) + } + require.NoError(t, app.Commit()) + + // Test out of order metric. + app = db.Appender(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))) + + _, err = appendSample(app, db.head.minValidTime.Load()+DefaultBlockDuration+3) + require.Equal(t, expectOutOfOrderError, err) + require.Equal(t, 5.0, prom_testutil.ToFloat64(db.head.metrics.outOfOrderSamples.WithLabelValues(scenario.sampleType))) + + _, err = appendSample(app, db.head.minValidTime.Load()+DefaultBlockDuration+4) + require.Equal(t, expectOutOfOrderError, err) + require.Equal(t, 6.0, prom_testutil.ToFloat64(db.head.metrics.outOfOrderSamples.WithLabelValues(scenario.sampleType))) + 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 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)) + } + } + 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) + }) + } +} + +func TestHeadLabelNamesWithMatchers(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) + 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) + 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) + 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) + require.NoError(t, err) + require.NoError(t, app.Rollback()) + require.Error(t, app.Commit()) + require.Error(t, app.Rollback()) +} + +func TestHeadMintAfterTruncation(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) + require.NoError(t, err) + _, err = app.Append(0, labels.FromStrings("a", "b"), 4000, 200) + require.NoError(t, err) + _, err = app.Append(0, labels.FromStrings("a", "b"), 8000, 300) + require.NoError(t, err) + require.NoError(t, app.Commit()) + + // Truncating outside the appendable window and actual mint being outside + // appendable window should leave mint at the actual mint. + require.NoError(t, head.Truncate(3500)) + require.Equal(t, int64(4000), head.MinTime()) + require.Equal(t, int64(4000), head.minValidTime.Load()) + + // After truncation outside the appendable window if the actual min time + // is in the appendable window then we should leave mint at the start of appendable window. + require.NoError(t, head.Truncate(5000)) + require.Equal(t, head.appendableMinValidTime(), head.MinTime()) + require.Equal(t, head.appendableMinValidTime(), head.minValidTime.Load()) + + // If the truncation time is inside the appendable window, then the min time + // should be the truncation time. + require.NoError(t, head.Truncate(7500)) + require.Equal(t, int64(7500), head.MinTime()) + require.Equal(t, int64(7500), head.minValidTime.Load()) + + require.NoError(t, head.Close()) +} + +func TestHeadExemplars(t *testing.T) { + chunkRange := int64(2000) + head, _ := newTestHead(t, chunkRange, compression.None, false) + app := head.Appender(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, + }) + 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) { + 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. + expSamples := make([]chunks.Sample, 0) + // 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 + expSamples = append(expSamples, sample{ts, float64(i), nil, nil}) + } + 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) + + 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) + + // Querying the querier that was got before compaction. + series := query(t, q, labels.MustNewMatcher(labels.MatchEqual, "a", "b")) + require.Equal(t, map[string][]chunks.Sample{`{a="b"}`: expSamples}, series) + + wg.Wait() +} + +func TestIsQuerierCollidingWithTruncation(t *testing.T) { + db := newTestDB(t) + db.DisableCompactions() + + var ( + app = db.Appender(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)) + require.NoError(t, err) + } + require.NoError(t, app.Commit()) + + // This mocks truncation. + db.head.memTruncationInProcess.Store(true) + db.head.lastMemoryTruncationTime.Store(2000) + + // Test that IsQuerierValid suggests correct querier ranges. + cases := []struct { + mint, maxt int64 // For the querier. + expShouldClose, expGetNew bool + expNewMint int64 + }{ + {-200, -100, true, false, 0}, + {-200, 300, true, false, 0}, + {100, 1900, true, false, 0}, + {1900, 2200, true, true, 2000}, + {2000, 2500, false, false, 0}, + } + + for _, c := range cases { + t.Run(fmt.Sprintf("mint=%d,maxt=%d", c.mint, c.maxt), func(t *testing.T) { + shouldClose, getNew, newMint := db.head.IsQuerierCollidingWithTruncation(c.mint, c.maxt) + require.Equal(t, c.expShouldClose, shouldClose) + require.Equal(t, c.expGetNew, getNew) + if getNew { + require.Equal(t, c.expNewMint, newMint) + } + }) + } +} + +func TestWaitForPendingReadersInTimeRange(t *testing.T) { + t.Parallel() + db := newTestDB(t) + db.DisableCompactions() + + sampleTs := func(i int64) int64 { return i * DefaultBlockDuration / (4 * 120) } + + var ( + app = db.Appender(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)) + require.NoError(t, err) + } + require.NoError(t, app.Commit()) + + truncMint, truncMaxt := int64(1000), int64(2000) + cases := []struct { + mint, maxt int64 + shouldWait bool + }{ + {0, 500, false}, // Before truncation range. + {500, 1500, true}, // Overlaps with truncation at the start. + {1200, 1700, true}, // Within truncation range. + {1800, 2500, true}, // Overlaps with truncation at the end. + {2000, 2500, false}, // After truncation range. + {2100, 2500, false}, // After truncation range. + } + for _, c := range cases { + t.Run(fmt.Sprintf("mint=%d,maxt=%d,shouldWait=%t", c.mint, c.maxt, c.shouldWait), func(t *testing.T) { + checkWaiting := func(cl io.Closer) { + var waitOver atomic.Bool + go func() { + db.head.WaitForPendingReadersInTimeRange(truncMint, truncMaxt) + waitOver.Store(true) + }() + <-time.After(550 * time.Millisecond) + require.Equal(t, !c.shouldWait, waitOver.Load()) + require.NoError(t, cl.Close()) + <-time.After(550 * time.Millisecond) + require.True(t, waitOver.Load()) + } + + q, err := db.Querier(c.mint, c.maxt) + require.NoError(t, err) + checkWaiting(q) + + cq, err := db.ChunkQuerier(c.mint, c.maxt) + require.NoError(t, err) + checkWaiting(cq) + }) + } +} + +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(db *DB, minT, maxT int64) (storage.LabelQuerier, error) { + return db.ChunkQuerier(minT, maxT) + }, + func(t *testing.T, lq storage.LabelQuerier, minT, _ int64) { + // Chunks + q, ok := lq.(storage.ChunkQuerier) + 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. + metaIt := s.Iterator(nil) + require.True(t, metaIt.Next()) + meta := metaIt.At() + // Samples + it := meta.Chunk.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 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)) { + const maxT int64 = 6000 + + dir := t.TempDir() + opts := DefaultOptions() + opts.OutOfOrderTimeWindow = maxT + opts.MinBlockDuration = maxT / 2 // So that head will compact up to 3000. + + db, err := Open(dir, nil, nil, opts, nil) + require.NoError(t, err) + t.Cleanup(func() { + require.NoError(t, db.Close()) + }) + db.DisableCompactions() + + var ( + ref = storage.SeriesRef(0) + app = db.Appender(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) + 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) + require.NoError(t, err) + } + require.NoError(t, app.Commit()) + + requireEqualOOOSamples(t, int(maxT/100-1), db) + + // Synchronization points. + allowQueryToStart := make(chan struct{}) + queryStarted := make(chan struct{}) + compactionFinished := make(chan struct{}) + + db.head.memTruncationCallBack = func() { + // Compaction has started, let the query start and wait for it to actually start to simulate race condition. + allowQueryToStart <- struct{}{} + <-queryStarted + } + + go func() { + db.Compact(context.Background()) // Compact and write blocks up to 3000 (maxtT/2). + compactionFinished <- struct{}{} + }() + + // Wait for the compaction to start. + <-allowQueryToStart + + q, err := makeQuerier(db, 1500, 2500) + require.NoError(t, err) + queryStarted <- struct{}{} // Unblock the compaction. + ctx := context.Background() + + // Label names. + res, annots, err := q.LabelNames(ctx, nil, labels.MustNewMatcher(labels.MatchEqual, "a", "b")) + require.NoError(t, err) + require.Empty(t, annots) + require.Equal(t, []string{"a"}, res) + + // Label values. + res, annots, err = q.LabelValues(ctx, "a", nil, labels.MustNewMatcher(labels.MatchEqual, "a", "b")) + require.NoError(t, err) + require.Empty(t, annots) + require.Equal(t, []string{"b"}, res) + + verify(t, q, 1500, 2500) + + require.NoError(t, q.Close()) // Cannot be deferred as the compaction waits for queries to close before finishing. + + <-compactionFinished // Wait for compaction otherwise Go test finds stray goroutines. +} + +func TestAppendHistogram(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) { + head, _ := newTestHead(t, 1000, compression.None, false) + t.Cleanup(func() { + require.NoError(t, head.Close()) + }) + + require.NoError(t, head.Init(0)) + ingestTs := int64(0) + app := head.Appender(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) + 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()) + } + } + + // Gauge integer histograms. + for _, h := range tsdbutil.GenerateTestGaugeHistograms(numHistograms) { + _, err := app.AppendHistogram(0, l, ingestTs, h, nil) + 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()) + } + } + + expFloatHistograms := make([]chunks.Sample, 0, 2*numHistograms) + + // Counter float histograms. + for _, fh := range tsdbutil.GenerateTestFloatHistograms(numHistograms) { + _, err := app.AppendHistogram(0, l, ingestTs, nil, fh) + 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()) + } + } + + // Gauge float histograms. + for _, fh := range tsdbutil.GenerateTestGaugeFloatHistograms(numHistograms) { + _, err := app.AppendHistogram(0, l, ingestTs, nil, fh) + 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()) + } + } + + require.NoError(t, app.Commit()) + + q, err := NewBlockQuerier(head, head.MinTime(), head.MaxTime()) + require.NoError(t, err) + t.Cleanup(func() { + require.NoError(t, q.Close()) + }) + + 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()) + + it := s.Iterator(nil) + actHistograms := make([]chunks.Sample, 0, len(expHistograms)) + actFloatHistograms := make([]chunks.Sample, 0, len(expFloatHistograms)) + for typ := it.Next(); typ != chunkenc.ValNone; typ = it.Next() { + switch typ { + case chunkenc.ValHistogram: + ts, h := it.AtHistogram(nil) + actHistograms = append(actHistograms, sample{t: ts, h: h}) + case chunkenc.ValFloatHistogram: + ts, fh := it.AtFloatHistogram(nil) + actFloatHistograms = append(actFloatHistograms, sample{t: ts, fh: fh}) + } + } + + compareSeries( + t, + map[string][]chunks.Sample{"dummy": expHistograms}, + map[string][]chunks.Sample{"dummy": actHistograms}, + ) + compareSeries( + t, + map[string][]chunks.Sample{"dummy": expFloatHistograms}, + map[string][]chunks.Sample{"dummy": actFloatHistograms}, + ) + }) + } +} + +func TestHistogramInWALAndMmapChunk(t *testing.T) { + head, _ := newTestHead(t, 3000, compression.None, false) + t.Cleanup(func() { + require.NoError(t, head.Close()) + }) + require.NoError(t, head.Init(0)) + + // Series with only histograms. + s1 := labels.FromStrings("a", "b1") + k1 := s1.String() + numHistograms := 300 + exp := map[string][]chunks.Sample{} + ts := int64(0) + var app storage.Appender + for _, gauge := range []bool{true, false} { + app = head.Appender(context.Background()) + var hists []*histogram.Histogram + if gauge { + hists = tsdbutil.GenerateTestGaugeHistograms(numHistograms) + } else { + hists = tsdbutil.GenerateTestHistograms(numHistograms) + } + for _, h := range hists { + h.NegativeSpans = h.PositiveSpans + h.NegativeBuckets = h.PositiveBuckets + _, err := app.AppendHistogram(0, s1, ts, h, nil) + 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()) + } + } + require.NoError(t, app.Commit()) + } + for _, gauge := range []bool{true, false} { + app = head.Appender(context.Background()) + var hists []*histogram.FloatHistogram + if gauge { + hists = tsdbutil.GenerateTestGaugeFloatHistograms(numHistograms) + } else { + hists = tsdbutil.GenerateTestFloatHistograms(numHistograms) + } + for _, h := range hists { + h.NegativeSpans = h.PositiveSpans + h.NegativeBuckets = h.PositiveBuckets + _, err := app.AppendHistogram(0, s1, ts, nil, h) + 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()) + } + } + require.NoError(t, app.Commit()) + head.mmapHeadChunks() + } + + // There should be 20 mmap chunks in s1. + ms := head.series.getByHash(s1.Hash(), s1) + require.Len(t, ms.mmappedChunks, 25) + expMmapChunks := make([]*mmappedChunk, 0, 20) + for _, mmap := range ms.mmappedChunks { + require.Positive(t, mmap.numSamples) + cpy := *mmap + expMmapChunks = append(expMmapChunks, &cpy) + } + expHeadChunkSamples := ms.headChunks.chunk.NumSamples() + require.Positive(t, expHeadChunkSamples) + + // Series with mix of histograms and float. + s2 := labels.FromStrings("a", "b2") + k2 := s2.String() + ts = 0 + for _, gauge := range []bool{true, false} { + app = head.Appender(context.Background()) + var hists []*histogram.Histogram + if gauge { + hists = tsdbutil.GenerateTestGaugeHistograms(100) + } else { + hists = tsdbutil.GenerateTestHistograms(100) + } + for _, h := range hists { + ts++ + h.NegativeSpans = h.PositiveSpans + h.NegativeBuckets = h.PositiveBuckets + _, err := app.AppendHistogram(0, s2, ts, h, nil) + require.NoError(t, err) + eh := h.Copy() + if !gauge && ts > 30 && (ts-10)%20 == 1 { + // Need "unknown" hint after float sample. + eh.CounterResetHint = histogram.UnknownCounterReset + } + exp[k2] = append(exp[k2], sample{t: ts, h: eh}) + if ts%20 == 0 { + require.NoError(t, app.Commit()) + app = head.Appender(context.Background()) + // Add some float. + for range 10 { + ts++ + _, err := app.Append(0, s2, ts, float64(ts)) + 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()) + } + } + require.NoError(t, app.Commit()) + } + for _, gauge := range []bool{true, false} { + app = head.Appender(context.Background()) + var hists []*histogram.FloatHistogram + if gauge { + hists = tsdbutil.GenerateTestGaugeFloatHistograms(100) + } else { + hists = tsdbutil.GenerateTestFloatHistograms(100) + } + for _, h := range hists { + ts++ + h.NegativeSpans = h.PositiveSpans + h.NegativeBuckets = h.PositiveBuckets + _, err := app.AppendHistogram(0, s2, ts, nil, h) + require.NoError(t, err) + eh := h.Copy() + if !gauge && ts > 30 && (ts-10)%20 == 1 { + // Need "unknown" hint after float sample. + eh.CounterResetHint = histogram.UnknownCounterReset + } + exp[k2] = append(exp[k2], sample{t: ts, fh: eh}) + if ts%20 == 0 { + require.NoError(t, app.Commit()) + app = head.Appender(context.Background()) + // Add some float. + for range 10 { + ts++ + _, err := app.Append(0, s2, ts, float64(ts)) + 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()) + } + } + require.NoError(t, app.Commit()) + } + + // Restart head. + require.NoError(t, head.Close()) + startHead := func() { + w, err := wlog.NewSize(nil, nil, head.wal.Dir(), 32768, compression.None) + require.NoError(t, err) + head, err = NewHead(nil, nil, w, nil, head.opts, nil) + require.NoError(t, err) + require.NoError(t, head.Init(0)) + } + startHead() + + // Checking contents of s1. + ms = head.series.getByHash(s1.Hash(), s1) + require.Equal(t, expMmapChunks, ms.mmappedChunks) + require.Equal(t, expHeadChunkSamples, ms.headChunks.chunk.NumSamples()) + + testQuery := func() { + q, err := NewBlockQuerier(head, head.MinTime(), head.MaxTime()) + require.NoError(t, err) + act := query(t, q, labels.MustNewMatcher(labels.MatchRegexp, "a", "b.*")) + compareSeries(t, exp, act) + } + testQuery() + + // Restart with no mmap chunks to test WAL replay. + require.NoError(t, head.Close()) + require.NoError(t, os.RemoveAll(mmappedChunksDir(head.opts.ChunkDirRoot))) + startHead() + testQuery() +} + +func TestChunkSnapshot(t *testing.T) { + head, _ := newTestHead(t, 120*4, compression.None, false) + defer func() { + head.opts.EnableMemorySnapshotOnShutdown = false + require.NoError(t, head.Close()) + }() + + type ex struct { + seriesLabels labels.Labels + e exemplar.Exemplar + } + + numSeries := 10 + expSeries := make(map[string][]chunks.Sample) + expHist := make(map[string][]chunks.Sample) + expFloatHist := make(map[string][]chunks.Sample) + expTombstones := make(map[storage.SeriesRef]tombstones.Intervals) + expExemplars := make([]ex, 0) + histograms := tsdbutil.GenerateTestGaugeHistograms(481) + floatHistogram := tsdbutil.GenerateTestGaugeFloatHistograms(481) + + addExemplar := func(app storage.Appender, ref storage.SeriesRef, lbls labels.Labels, ts int64) { + e := ex{ + seriesLabels: lbls, + e: exemplar.Exemplar{ + Labels: labels.FromStrings("trace_id", strconv.Itoa(rand.Int())), + Value: rand.Float64(), + Ts: ts, + }, + } + expExemplars = append(expExemplars, e) + _, err := app.AppendExemplar(ref, e.seriesLabels, e.e) + require.NoError(t, err) + } + + checkSamples := func() { + q, err := NewBlockQuerier(head, math.MinInt64, math.MaxInt64) + require.NoError(t, err) + series := query(t, q, labels.MustNewMatcher(labels.MatchRegexp, "foo", "bar.*")) + require.Equal(t, expSeries, series) + } + checkHistograms := func() { + q, err := NewBlockQuerier(head, math.MinInt64, math.MaxInt64) + require.NoError(t, err) + series := query(t, q, labels.MustNewMatcher(labels.MatchRegexp, "hist", "baz.*")) + require.Equal(t, expHist, series) + } + checkFloatHistograms := func() { + q, err := NewBlockQuerier(head, math.MinInt64, math.MaxInt64) + require.NoError(t, err) + series := query(t, q, labels.MustNewMatcher(labels.MatchRegexp, "floathist", "bat.*")) + require.Equal(t, expFloatHist, series) + } + checkTombstones := func() { + tr, err := head.Tombstones() + require.NoError(t, err) + actTombstones := make(map[storage.SeriesRef]tombstones.Intervals) + require.NoError(t, tr.Iter(func(ref storage.SeriesRef, itvs tombstones.Intervals) error { + for _, itv := range itvs { + actTombstones[ref].Add(itv) + } + return nil + })) + require.Equal(t, expTombstones, actTombstones) + } + checkExemplars := func() { + actExemplars := make([]ex, 0, len(expExemplars)) + err := head.exemplars.IterateExemplars(func(seriesLabels labels.Labels, e exemplar.Exemplar) error { + actExemplars = append(actExemplars, ex{ + seriesLabels: seriesLabels, + e: e, + }) + return nil + }) + require.NoError(t, err) + // Verifies both existence of right exemplars and order of exemplars in the buffer. + testutil.RequireEqualWithOptions(t, expExemplars, actExemplars, []cmp.Option{cmp.AllowUnexported(ex{})}) + } + + var ( + wlast, woffset int + err error + ) + + closeHeadAndCheckSnapshot := func() { + require.NoError(t, head.Close()) + + _, sidx, soffset, err := LastChunkSnapshot(head.opts.ChunkDirRoot) + require.NoError(t, err) + require.Equal(t, wlast, sidx) + require.Equal(t, woffset, soffset) + } + + openHeadAndCheckReplay := func() { + w, err := wlog.NewSize(nil, nil, head.wal.Dir(), 32768, compression.None) + require.NoError(t, err) + head, err = NewHead(nil, nil, w, nil, head.opts, nil) + require.NoError(t, err) + require.NoError(t, head.Init(math.MinInt64)) + + checkSamples() + checkHistograms() + checkFloatHistograms() + checkTombstones() + checkExemplars() + } + + { // Initial data that goes into snapshot. + // Add some initial samples with >=1 m-map chunk. + app := head.Appender(context.Background()) + for i := 1; i <= numSeries; i++ { + lbls := labels.FromStrings("foo", fmt.Sprintf("bar%d", i)) + lblStr := lbls.String() + lblsHist := labels.FromStrings("hist", fmt.Sprintf("baz%d", i)) + lblsHistStr := lblsHist.String() + lblsFloatHist := labels.FromStrings("floathist", fmt.Sprintf("bat%d", i)) + lblsFloatHistStr := lblsFloatHist.String() + + // 240 samples should m-map at least 1 chunk. + for ts := int64(1); ts <= 240; ts++ { + val := rand.Float64() + expSeries[lblStr] = append(expSeries[lblStr], sample{ts, val, nil, nil}) + ref, err := app.Append(0, lbls, ts, val) + 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) + 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) + require.NoError(t, err) + + // Add an exemplar and to create multiple WAL records. + if ts%10 == 0 { + addExemplar(app, ref, lbls, ts) + require.NoError(t, app.Commit()) + app = head.Appender(context.Background()) + } + } + } + require.NoError(t, app.Commit()) + + // Add some tombstones. + var enc record.Encoder + for i := 1; i <= numSeries; i++ { + ref := storage.SeriesRef(i) + itvs := tombstones.Intervals{ + {Mint: 1234, Maxt: 2345}, + {Mint: 3456, Maxt: 4567}, + } + for _, itv := range itvs { + expTombstones[ref].Add(itv) + } + head.tombstones.AddInterval(ref, itvs...) + err := head.wal.Log(enc.Tombstones([]tombstones.Stone{ + {Ref: ref, Intervals: itvs}, + }, nil)) + require.NoError(t, err) + } + } + + // These references should be the ones used for the snapshot. + wlast, woffset, err = head.wal.LastSegmentAndOffset() + require.NoError(t, err) + if woffset != 0 && woffset < 32*1024 { + // The page is always filled before taking the snapshot. + woffset = 32 * 1024 + } + + { + // Creating snapshot and verifying it. + head.opts.EnableMemorySnapshotOnShutdown = true + closeHeadAndCheckSnapshot() // This will create a snapshot. + + // Test the replay of snapshot. + openHeadAndCheckReplay() + } + + { // 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()) + for i := 1; i <= numSeries; i++ { + lbls := labels.FromStrings("foo", fmt.Sprintf("bar%d", i)) + lblStr := lbls.String() + lblsHist := labels.FromStrings("hist", fmt.Sprintf("baz%d", i)) + lblsHistStr := lblsHist.String() + lblsFloatHist := labels.FromStrings("floathist", fmt.Sprintf("bat%d", i)) + lblsFloatHistStr := lblsFloatHist.String() + + // 240 samples should m-map at least 1 chunk. + for ts := int64(241); ts <= 480; ts++ { + val := rand.Float64() + expSeries[lblStr] = append(expSeries[lblStr], sample{ts, val, nil, nil}) + ref, err := app.Append(0, lbls, ts, val) + 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) + 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) + require.NoError(t, err) + + // Add an exemplar and to create multiple WAL records. + if ts%10 == 0 { + addExemplar(app, ref, lbls, ts) + require.NoError(t, app.Commit()) + app = head.Appender(context.Background()) + } + } + } + require.NoError(t, app.Commit()) + + // Add more tombstones. + var enc record.Encoder + for i := 1; i <= numSeries; i++ { + ref := storage.SeriesRef(i) + itvs := tombstones.Intervals{ + {Mint: 12345, Maxt: 23456}, + {Mint: 34567, Maxt: 45678}, + } + for _, itv := range itvs { + expTombstones[ref].Add(itv) + } + head.tombstones.AddInterval(ref, itvs...) + err := head.wal.Log(enc.Tombstones([]tombstones.Stone{ + {Ref: ref, Intervals: itvs}, + }, nil)) + require.NoError(t, err) + } + } + { + // Close Head and verify that new snapshot was not created. + head.opts.EnableMemorySnapshotOnShutdown = false + closeHeadAndCheckSnapshot() // This should not create a snapshot. + + // Test the replay of snapshot, m-map chunks, and WAL. + head.opts.EnableMemorySnapshotOnShutdown = true // Enabled to read from snapshot. + openHeadAndCheckReplay() + } + + // Creating another snapshot should delete the older snapshot and replay still works fine. + wlast, woffset, err = head.wal.LastSegmentAndOffset() + require.NoError(t, err) + if woffset != 0 && woffset < 32*1024 { + // The page is always filled before taking the snapshot. + woffset = 32 * 1024 + } + + { + // Close Head and verify that new snapshot was created. + closeHeadAndCheckSnapshot() + + // Verify that there is only 1 snapshot. + files, err := os.ReadDir(head.opts.ChunkDirRoot) + require.NoError(t, err) + snapshots := 0 + for i := len(files) - 1; i >= 0; i-- { + fi := files[i] + if strings.HasPrefix(fi.Name(), chunkSnapshotPrefix) { + snapshots++ + require.Equal(t, chunkSnapshotDir(wlast, woffset), fi.Name()) + } + } + require.Equal(t, 1, snapshots) + + // Test the replay of snapshot. + head.opts.EnableMemorySnapshotOnShutdown = true // Enabled to read from snapshot. + + // Disabling exemplars to check that it does not hard fail replay + // https://github.com/prometheus/prometheus/issues/9437#issuecomment-933285870. + head.opts.EnableExemplarStorage = false + head.opts.MaxExemplars.Store(0) + expExemplars = expExemplars[:0] + + openHeadAndCheckReplay() + + require.Equal(t, 0.0, prom_testutil.ToFloat64(head.metrics.snapshotReplayErrorTotal)) + } +} + +func TestSnapshotError(t *testing.T) { + head, _ := newTestHead(t, 120*4, compression.None, false) + defer func() { + head.opts.EnableMemorySnapshotOnShutdown = false + require.NoError(t, head.Close()) + }() + + // Add a sample. + app := head.Appender(context.Background()) + lbls := labels.FromStrings("foo", "bar") + _, err := app.Append(0, lbls, 99, 99) + require.NoError(t, err) + + // Add histograms + hist := tsdbutil.GenerateTestGaugeHistograms(1)[0] + floatHist := tsdbutil.GenerateTestGaugeFloatHistograms(1)[0] + lblsHist := labels.FromStrings("hist", "bar") + lblsFloatHist := labels.FromStrings("floathist", "bar") + + _, err = app.AppendHistogram(0, lblsHist, 99, hist, nil) + require.NoError(t, err) + + _, err = app.AppendHistogram(0, lblsFloatHist, 99, nil, floatHist) + require.NoError(t, err) + + require.NoError(t, app.Commit()) + + // Add some tombstones. + itvs := tombstones.Intervals{ + {Mint: 1234, Maxt: 2345}, + {Mint: 3456, Maxt: 4567}, + } + head.tombstones.AddInterval(1, itvs...) + + // Check existence of data. + require.NotNil(t, head.series.getByHash(lbls.Hash(), lbls)) + tm, err := head.tombstones.Get(1) + require.NoError(t, err) + require.NotEmpty(t, tm) + + head.opts.EnableMemorySnapshotOnShutdown = true + require.NoError(t, head.Close()) // This will create a snapshot. + + // Remove the WAL so that we don't load from it. + require.NoError(t, os.RemoveAll(head.wal.Dir())) + + // Corrupt the snapshot. + snapDir, _, _, err := LastChunkSnapshot(head.opts.ChunkDirRoot) + require.NoError(t, err) + files, err := os.ReadDir(snapDir) + require.NoError(t, err) + f, err := os.OpenFile(path.Join(snapDir, files[0].Name()), os.O_RDWR, 0) + require.NoError(t, err) + // Create snapshot backup to be restored on future test cases. + snapshotBackup, err := io.ReadAll(f) + require.NoError(t, err) + _, err = f.WriteAt([]byte{0b11111111}, 18) + require.NoError(t, err) + require.NoError(t, f.Close()) + + // Create new Head which should replay this snapshot. + w, err := wlog.NewSize(nil, nil, head.wal.Dir(), 32768, compression.None) + require.NoError(t, err) + // Testing https://github.com/prometheus/prometheus/issues/9437 with the registry. + head, err = NewHead(prometheus.NewRegistry(), nil, w, nil, head.opts, nil) + require.NoError(t, err) + require.NoError(t, head.Init(math.MinInt64)) + + // There should be no series in the memory after snapshot error since WAL was removed. + require.Equal(t, 1.0, prom_testutil.ToFloat64(head.metrics.snapshotReplayErrorTotal)) + require.Equal(t, uint64(0), head.NumSeries()) + require.Nil(t, head.series.getByHash(lbls.Hash(), lbls)) + tm, err = head.tombstones.Get(1) + require.NoError(t, err) + require.Empty(t, tm) + require.NoError(t, head.Close()) + + // Test corruption in the middle of the snapshot. + f, err = os.OpenFile(path.Join(snapDir, files[0].Name()), os.O_RDWR, 0) + require.NoError(t, err) + _, err = f.WriteAt(snapshotBackup, 0) + require.NoError(t, err) + _, err = f.WriteAt([]byte{0b11111111}, 300) + require.NoError(t, err) + require.NoError(t, f.Close()) + + c := &countSeriesLifecycleCallback{} + opts := head.opts + opts.SeriesCallback = c + + w, err = wlog.NewSize(nil, nil, head.wal.Dir(), 32768, compression.None) + require.NoError(t, err) + head, err = NewHead(prometheus.NewRegistry(), nil, w, nil, head.opts, nil) + require.NoError(t, err) + require.NoError(t, head.Init(math.MinInt64)) + + // There should be no series in the memory after snapshot error since WAL was removed. + require.Equal(t, 1.0, prom_testutil.ToFloat64(head.metrics.snapshotReplayErrorTotal)) + require.Nil(t, head.series.getByHash(lbls.Hash(), lbls)) + require.Equal(t, uint64(0), head.NumSeries()) + + // Since the snapshot could replay certain series, we continue invoking the create hooks. + // In such instances, we need to ensure that we also trigger the delete hooks when resetting the memory. + require.Equal(t, int64(2), c.created.Load()) + require.Equal(t, int64(2), c.deleted.Load()) + + require.Equal(t, 2.0, prom_testutil.ToFloat64(head.metrics.seriesRemoved)) + require.Equal(t, 2.0, prom_testutil.ToFloat64(head.metrics.seriesCreated)) +} + +func TestHistogramMetrics(t *testing.T) { + numHistograms := 10 + head, _ := newTestHead(t, 1000, compression.None, false) + t.Cleanup(func() { + require.NoError(t, head.Close()) + }) + require.NoError(t, head.Init(0)) + + expHSeries, expHSamples := 0, 0 + + for x := range 5 { + 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) + 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) + require.NoError(t, err) + require.NoError(t, app.Commit()) + expHSamples++ + } + } + + require.Equal(t, float64(expHSamples), prom_testutil.ToFloat64(head.metrics.samplesAppended.WithLabelValues(sampleMetricTypeHistogram))) + + require.NoError(t, head.Close()) + w, err := wlog.NewSize(nil, nil, head.wal.Dir(), 32768, compression.None) + require.NoError(t, err) + head, err = NewHead(nil, nil, w, nil, head.opts, nil) + require.NoError(t, err) + require.NoError(t, head.Init(0)) + + require.Equal(t, float64(0), prom_testutil.ToFloat64(head.metrics.samplesAppended.WithLabelValues(sampleMetricTypeHistogram))) // Counter reset. +} + +func TestHistogramStaleSample(t *testing.T) { + t.Run("integer histogram", func(t *testing.T) { + testHistogramStaleSampleHelper(t, false) + }) + t.Run("float histogram", func(t *testing.T) { + testHistogramStaleSampleHelper(t, true) + }) +} + +func testHistogramStaleSampleHelper(t *testing.T, floatHistogram bool) { + t.Helper() + l := labels.FromStrings("a", "b") + numHistograms := 20 + head, _ := newTestHead(t, 100000, compression.None, false) + t.Cleanup(func() { + require.NoError(t, head.Close()) + }) + require.NoError(t, head.Init(0)) + + type timedHistogram struct { + t int64 + h *histogram.Histogram + fh *histogram.FloatHistogram + } + expHistograms := make([]timedHistogram, 0, numHistograms) + + testQuery := func(numStale int) { + q, err := NewBlockQuerier(head, head.MinTime(), head.MaxTime()) + require.NoError(t, err) + t.Cleanup(func() { + require.NoError(t, q.Close()) + }) + + 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()) + + it := s.Iterator(nil) + actHistograms := make([]timedHistogram, 0, len(expHistograms)) + for typ := it.Next(); typ != chunkenc.ValNone; typ = it.Next() { + switch typ { + case chunkenc.ValHistogram: + t, h := it.AtHistogram(nil) + actHistograms = append(actHistograms, timedHistogram{t: t, h: h}) + case chunkenc.ValFloatHistogram: + t, h := it.AtFloatHistogram(nil) + actHistograms = append(actHistograms, timedHistogram{t: t, fh: h}) + } + } + + // We cannot compare StaleNAN with require.Equal, hence checking each histogram manually. + require.Len(t, actHistograms, len(expHistograms)) + actNumStale := 0 + for i, eh := range expHistograms { + ah := actHistograms[i] + if floatHistogram { + switch { + case value.IsStaleNaN(eh.fh.Sum): + actNumStale++ + require.True(t, value.IsStaleNaN(ah.fh.Sum)) + // To make require.Equal work. + ah.fh.Sum = 0 + eh.fh = eh.fh.Copy() + eh.fh.Sum = 0 + case i > 0: + prev := expHistograms[i-1] + if prev.fh == nil || value.IsStaleNaN(prev.fh.Sum) { + eh.fh.CounterResetHint = histogram.UnknownCounterReset + } + } + require.Equal(t, eh, ah) + } else { + switch { + case value.IsStaleNaN(eh.h.Sum): + actNumStale++ + require.True(t, value.IsStaleNaN(ah.h.Sum)) + // To make require.Equal work. + ah.h.Sum = 0 + eh.h = eh.h.Copy() + eh.h.Sum = 0 + case i > 0: + prev := expHistograms[i-1] + if prev.h == nil || value.IsStaleNaN(prev.h.Sum) { + eh.h.CounterResetHint = histogram.UnknownCounterReset + } + } + require.Equal(t, eh, ah) + } + } + require.Equal(t, numStale, actNumStale) + } + + // Adding stale in the same appender. + app := head.Appender(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)) + 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) + 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)) + require.NoError(t, err) + if floatHistogram { + expHistograms = append(expHistograms, timedHistogram{t: 100*int64(len(expHistograms)) + 1, fh: &histogram.FloatHistogram{Sum: math.Float64frombits(value.StaleNaN)}}) + } else { + expHistograms = append(expHistograms, timedHistogram{t: 100*int64(len(expHistograms)) + 1, h: &histogram.Histogram{Sum: math.Float64frombits(value.StaleNaN)}}) + } + require.NoError(t, app.Commit()) + + // Only 1 chunk in the memory, no m-mapped chunk. + s := head.series.getByHash(l.Hash(), l) + require.NotNil(t, s) + require.NotNil(t, s.headChunks) + require.Equal(t, 1, s.headChunks.len()) + require.Empty(t, s.mmappedChunks) + testQuery(1) + + // Adding stale in different appender and continuing series after a stale sample. + app = head.Appender(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)) + 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) + 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()) + // +1 so that delta-of-delta is not 0. + _, err = app.Append(0, l, 100*int64(len(expHistograms))+1, math.Float64frombits(value.StaleNaN)) + require.NoError(t, err) + if floatHistogram { + expHistograms = append(expHistograms, timedHistogram{t: 100*int64(len(expHistograms)) + 1, fh: &histogram.FloatHistogram{Sum: math.Float64frombits(value.StaleNaN)}}) + } else { + expHistograms = append(expHistograms, timedHistogram{t: 100*int64(len(expHistograms)) + 1, h: &histogram.Histogram{Sum: math.Float64frombits(value.StaleNaN)}}) + } + require.NoError(t, app.Commit()) + head.mmapHeadChunks() + + // Total 2 chunks, 1 m-mapped. + s = head.series.getByHash(l.Hash(), l) + require.NotNil(t, s) + require.NotNil(t, s.headChunks) + require.Equal(t, 1, s.headChunks.len()) + require.Len(t, s.mmappedChunks, 1) + testQuery(2) +} + +func TestHistogramCounterResetHeader(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") + head, _ := newTestHead(t, 1000, compression.None, false) + t.Cleanup(func() { + require.NoError(t, head.Close()) + }) + require.NoError(t, head.Init(0)) + + ts := int64(0) + appendHistogram := func(h *histogram.Histogram) { + ts++ + app := head.Appender(context.Background()) + var err error + if floatHisto { + _, err = app.AppendHistogram(0, l, ts, nil, h.ToFloat(nil)) + } else { + _, err = app.AppendHistogram(0, l, ts, h.Copy(), nil) + } + require.NoError(t, err) + require.NoError(t, app.Commit()) + } + + var expHeaders []chunkenc.CounterResetHeader + checkExpCounterResetHeader := func(newHeaders ...chunkenc.CounterResetHeader) { + expHeaders = append(expHeaders, newHeaders...) + + ms, _, err := head.getOrCreate(l.Hash(), l, false) + require.NoError(t, err) + ms.mmapChunks(head.chunkDiskMapper) + require.Len(t, ms.mmappedChunks, len(expHeaders)-1) // One is the head chunk. + + for i, mmapChunk := range ms.mmappedChunks { + chk, err := head.chunkDiskMapper.Chunk(mmapChunk.ref) + require.NoError(t, err) + if floatHisto { + require.Equal(t, expHeaders[i], chk.(*chunkenc.FloatHistogramChunk).GetCounterResetHeader()) + } else { + require.Equal(t, expHeaders[i], chk.(*chunkenc.HistogramChunk).GetCounterResetHeader()) + } + } + if floatHisto { + require.Equal(t, expHeaders[len(expHeaders)-1], ms.headChunks.chunk.(*chunkenc.FloatHistogramChunk).GetCounterResetHeader()) + } else { + require.Equal(t, expHeaders[len(expHeaders)-1], ms.headChunks.chunk.(*chunkenc.HistogramChunk).GetCounterResetHeader()) + } + } + + h := tsdbutil.GenerateTestHistograms(1)[0] + h.PositiveBuckets = []int64{100, 1, 1, 1} + h.NegativeBuckets = []int64{100, 1, 1, 1} + h.Count = 1000 + + // First histogram is UnknownCounterReset. + appendHistogram(h) + checkExpCounterResetHeader(chunkenc.UnknownCounterReset) + + // Another normal histogram. + h.Count++ + appendHistogram(h) + checkExpCounterResetHeader() + + // Counter reset via Count. + h.Count-- + appendHistogram(h) + checkExpCounterResetHeader(chunkenc.CounterReset) + + // Add 2 non-counter reset histogram chunks (each chunk targets 1024 bytes which contains ~500 int histogram + // samples or ~1000 float histogram samples). + numAppend := 2000 + if floatHisto { + numAppend = 1000 + } + for i := 0; i < numAppend; i++ { + appendHistogram(h) + } + + checkExpCounterResetHeader(chunkenc.NotCounterReset, chunkenc.NotCounterReset) + + // Changing schema will cut a new chunk with unknown counter reset. + h.Schema++ + appendHistogram(h) + checkExpCounterResetHeader(chunkenc.UnknownCounterReset) + + // Changing schema will zero threshold a new chunk with unknown counter reset. + h.ZeroThreshold += 0.01 + appendHistogram(h) + checkExpCounterResetHeader(chunkenc.UnknownCounterReset) + + // Counter reset by removing a positive bucket. + h.PositiveSpans[1].Length-- + h.PositiveBuckets = h.PositiveBuckets[1:] + appendHistogram(h) + checkExpCounterResetHeader(chunkenc.CounterReset) + + // Counter reset by removing a negative bucket. + h.NegativeSpans[1].Length-- + h.NegativeBuckets = h.NegativeBuckets[1:] + appendHistogram(h) + checkExpCounterResetHeader(chunkenc.CounterReset) + + // Add 2 non-counter reset histogram chunks. Just to have some non-counter reset chunks in between. + for range 2000 { + appendHistogram(h) + } + checkExpCounterResetHeader(chunkenc.NotCounterReset, chunkenc.NotCounterReset) + + // Counter reset with counter reset in a positive bucket. + h.PositiveBuckets[len(h.PositiveBuckets)-1]-- + appendHistogram(h) + checkExpCounterResetHeader(chunkenc.CounterReset) + + // Counter reset with counter reset in a negative bucket. + h.NegativeBuckets[len(h.NegativeBuckets)-1]-- + appendHistogram(h) + checkExpCounterResetHeader(chunkenc.CounterReset) + }) + } +} + +func TestOOOHistogramCounterResetHeaders(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") + head, _ := newTestHead(t, 1000, compression.None, true) + head.opts.OutOfOrderCapMax.Store(5) + + t.Cleanup(func() { + require.NoError(t, head.Close()) + }) + require.NoError(t, head.Init(0)) + + appendHistogram := func(ts int64, h *histogram.Histogram) { + app := head.Appender(context.Background()) + var err error + if floatHisto { + _, err = app.AppendHistogram(0, l, ts, nil, h.ToFloat(nil)) + } else { + _, err = app.AppendHistogram(0, l, ts, h.Copy(), nil) + } + require.NoError(t, err) + require.NoError(t, app.Commit()) + } + + type expOOOMmappedChunks struct { + header chunkenc.CounterResetHeader + mint, maxt int64 + numSamples uint16 + } + + var expChunks []expOOOMmappedChunks + checkOOOExpCounterResetHeader := func(newChunks ...expOOOMmappedChunks) { + expChunks = append(expChunks, newChunks...) + + ms, _, err := head.getOrCreate(l.Hash(), l, false) + require.NoError(t, err) + + require.Len(t, ms.ooo.oooMmappedChunks, len(expChunks)) + + for i, mmapChunk := range ms.ooo.oooMmappedChunks { + chk, err := head.chunkDiskMapper.Chunk(mmapChunk.ref) + require.NoError(t, err) + if floatHisto { + require.Equal(t, expChunks[i].header, chk.(*chunkenc.FloatHistogramChunk).GetCounterResetHeader()) + } else { + require.Equal(t, expChunks[i].header, chk.(*chunkenc.HistogramChunk).GetCounterResetHeader()) + } + require.Equal(t, expChunks[i].mint, mmapChunk.minTime) + require.Equal(t, expChunks[i].maxt, mmapChunk.maxTime) + require.Equal(t, expChunks[i].numSamples, mmapChunk.numSamples) + } + } + + // Append an in-order histogram, so the rest of the samples can be detected as OOO. + appendHistogram(1000, tsdbutil.GenerateTestHistogram(1000)) + + // OOO histogram + for i := 1; i <= 5; i++ { + appendHistogram(100+int64(i), tsdbutil.GenerateTestHistogram(1000+int64(i))) + } + // Nothing mmapped yet. + checkOOOExpCounterResetHeader() + + // 6th observation (which triggers a head chunk mmapping). + appendHistogram(int64(112), tsdbutil.GenerateTestHistogram(1002)) + + // One mmapped chunk with (ts, val) [(101, 1001), (102, 1002), (103, 1003), (104, 1004), (105, 1005)]. + checkOOOExpCounterResetHeader(expOOOMmappedChunks{ + header: chunkenc.UnknownCounterReset, + mint: 101, + maxt: 105, + numSamples: 5, + }) + + // Add more samples, there's a counter reset at ts 122. + appendHistogram(int64(110), tsdbutil.GenerateTestHistogram(1001)) + appendHistogram(int64(124), tsdbutil.GenerateTestHistogram(904)) + appendHistogram(int64(123), tsdbutil.GenerateTestHistogram(903)) + appendHistogram(int64(122), tsdbutil.GenerateTestHistogram(902)) + + // New samples not mmapped yet. + checkOOOExpCounterResetHeader() + + // 11th observation (which triggers another head chunk mmapping). + appendHistogram(int64(200), tsdbutil.GenerateTestHistogram(2000)) + + // Two new mmapped chunks [(110, 1001), (112, 1002)], [(122, 902), (123, 903), (124, 904)]. + checkOOOExpCounterResetHeader( + expOOOMmappedChunks{ + header: chunkenc.UnknownCounterReset, + mint: 110, + maxt: 112, + numSamples: 2, + }, + expOOOMmappedChunks{ + header: chunkenc.CounterReset, + mint: 122, + maxt: 124, + numSamples: 3, + }, + ) + + // Count is lower than previous sample at ts 200, and NotCounterReset is always ignored on append. + appendHistogram(int64(205), tsdbutil.SetHistogramNotCounterReset(tsdbutil.GenerateTestHistogram(1000))) + + appendHistogram(int64(210), tsdbutil.SetHistogramCounterReset(tsdbutil.GenerateTestHistogram(2010))) + + appendHistogram(int64(220), tsdbutil.GenerateTestHistogram(2020)) + + appendHistogram(int64(215), tsdbutil.GenerateTestHistogram(2005)) + + // 16th observation (which triggers another head chunk mmapping). + appendHistogram(int64(350), tsdbutil.GenerateTestHistogram(4000)) + + // Four new mmapped chunks: [(200, 2000)] [(205, 1000)], [(210, 2010)], [(215, 2015), (220, 2020)] + checkOOOExpCounterResetHeader( + expOOOMmappedChunks{ + header: chunkenc.UnknownCounterReset, + mint: 200, + maxt: 200, + numSamples: 1, + }, + expOOOMmappedChunks{ + header: chunkenc.CounterReset, + mint: 205, + maxt: 205, + numSamples: 1, + }, + expOOOMmappedChunks{ + header: chunkenc.CounterReset, + mint: 210, + maxt: 210, + numSamples: 1, + }, + expOOOMmappedChunks{ + header: chunkenc.CounterReset, + mint: 215, + maxt: 220, + numSamples: 2, + }, + ) + + // Adding five more samples (21 in total), so another mmapped chunk is created. + appendHistogram(300, tsdbutil.SetHistogramCounterReset(tsdbutil.GenerateTestHistogram(3000))) + + for i := 1; i <= 4; i++ { + appendHistogram(300+int64(i), tsdbutil.GenerateTestHistogram(3000+int64(i))) + } + + // One mmapped chunk with (ts, val) [(300, 3000), (301, 3001), (302, 3002), (303, 3003), (350, 4000)]. + checkOOOExpCounterResetHeader(expOOOMmappedChunks{ + header: chunkenc.CounterReset, + mint: 300, + maxt: 350, + numSamples: 5, + }) + }) + } +} + +func TestAppendingDifferentEncodingToSameSeries(t *testing.T) { + dir := t.TempDir() + opts := DefaultOptions() + db, err := Open(dir, nil, nil, opts, nil) + require.NoError(t, err) + t.Cleanup(func() { + require.NoError(t, db.Close()) + }) + db.DisableCompactions() + + hists := tsdbutil.GenerateTestHistograms(10) + floatHists := tsdbutil.GenerateTestFloatHistograms(10) + lbls := labels.FromStrings("a", "b") + + var expResult []chunks.Sample + checkExpChunks := func(count int) { + ms, created, err := db.Head().getOrCreate(lbls.Hash(), lbls, false) + require.NoError(t, err) + require.False(t, created) + require.NotNil(t, ms) + require.Equal(t, count, ms.headChunks.len()) + } + + appends := []struct { + samples []chunks.Sample + expChunks int + err error + }{ + // Histograms that end up in the expected samples are copied here so that we + // can independently set the CounterResetHint later. + { + samples: []chunks.Sample{sample{t: 100, h: hists[0].Copy()}}, + expChunks: 1, + }, + { + samples: []chunks.Sample{sample{t: 200, f: 2}}, + expChunks: 2, + }, + { + samples: []chunks.Sample{sample{t: 210, fh: floatHists[0].Copy()}}, + expChunks: 3, + }, + { + samples: []chunks.Sample{sample{t: 220, h: hists[1].Copy()}}, + expChunks: 4, + }, + { + samples: []chunks.Sample{sample{t: 230, fh: floatHists[3].Copy()}}, + expChunks: 5, + }, + { + samples: []chunks.Sample{sample{t: 100, h: hists[2].Copy()}}, + err: storage.ErrOutOfOrderSample, + }, + { + samples: []chunks.Sample{sample{t: 300, h: hists[3].Copy()}}, + expChunks: 6, + }, + { + samples: []chunks.Sample{sample{t: 100, f: 2}}, + err: storage.ErrOutOfOrderSample, + }, + { + samples: []chunks.Sample{sample{t: 100, fh: floatHists[4].Copy()}}, + err: storage.ErrOutOfOrderSample, + }, + // The three next tests all failed before #15177 was fixed. + { + samples: []chunks.Sample{ + sample{t: 400, f: 4}, + sample{t: 500, h: hists[5]}, + sample{t: 600, f: 6}, + }, + expChunks: 9, // Each of the three samples above creates a new chunk because the type changes. + }, + { + samples: []chunks.Sample{ + sample{t: 700, h: hists[7]}, + sample{t: 800, f: 8}, + sample{t: 900, h: hists[9]}, + }, + expChunks: 12, // Again each sample creates a new chunk. + }, + { + samples: []chunks.Sample{ + sample{t: 1000, fh: floatHists[7]}, + sample{t: 1100, h: hists[9]}, + }, + expChunks: 14, // Even changes between float and integer histogram create new chunks. + }, + } + + for _, a := range appends { + app := db.Appender(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()) + } else { + _, err = app.Append(0, lbls, s.T(), s.F()) + } + require.Equal(t, a.err, err) + } + + if a.err == nil { + require.NoError(t, app.Commit()) + expResult = append(expResult, a.samples...) + checkExpChunks(a.expChunks) + } else { + require.NoError(t, app.Rollback()) + } + } + for i, s := range expResult[1:] { + switch { + case s.H() != nil && expResult[i].H() == nil: + s.(sample).h.CounterResetHint = histogram.UnknownCounterReset + case s.FH() != nil && expResult[i].FH() == nil: + s.(sample).fh.CounterResetHint = histogram.UnknownCounterReset + } + } + + // Query back and expect same order of samples. + q, err := db.Querier(math.MinInt64, math.MaxInt64) + require.NoError(t, err) + + series := query(t, q, labels.MustNewMatcher(labels.MatchEqual, "a", "b")) + 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) { + dir := t.TempDir() + wlTemp, err := wlog.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, compression.Snappy) + require.NoError(t, err) + + // Write a snapshot with .tmp suffix. This used to fail taking any further snapshots or replay of snapshots. + snapshotName := chunkSnapshotDir(0, 100) + ".tmp" + cpdir := filepath.Join(dir, snapshotName) + require.NoError(t, os.MkdirAll(cpdir, 0o777)) + + opts := DefaultHeadOptions() + opts.ChunkDirRoot = dir + opts.EnableMemorySnapshotOnShutdown = true + head, err := NewHead(nil, nil, wlTemp, nil, opts, nil) + require.NoError(t, err) + require.NoError(t, head.Init(math.MinInt64)) + + 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) + require.NoError(t, err) + require.NoError(t, app.Commit()) + + // Should not return any error for a successful snapshot. + require.NoError(t, head.Close()) + + // Verify the snapshot. + name, idx, offset, err := LastChunkSnapshot(dir) + require.NoError(t, err) + require.NotEmpty(t, name) + require.Equal(t, 0, idx) + require.Positive(t, offset) +} + +// TestWBLReplay checks the replay at a low level. +func TestWBLReplay(t *testing.T) { + for name, scenario := range sampleTypeScenarios { + t.Run(name, func(t *testing.T) { + testWBLReplay(t, scenario) + }) + } +} + +func testWBLReplay(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) + oooWlog, err := wlog.NewSize(nil, nil, filepath.Join(dir, wlog.WblDirName), 32768, compression.Snappy) + require.NoError(t, err) + + opts := DefaultHeadOptions() + opts.ChunkRange = 1000 + opts.ChunkDirRoot = dir + opts.OutOfOrderTimeWindow.Store(30 * time.Minute.Milliseconds()) + + h, err := NewHead(nil, nil, wal, oooWlog, opts, nil) + require.NoError(t, err) + require.NoError(t, h.Init(0)) + + 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) + require.NoError(t, err) + require.NoError(t, app.Commit()) + + if isOOO { + expOOOSamples = append(expOOOSamples, s) + } + } + + // In-order sample. + appendSample(60, 60, false) + + // Out of order samples. + appendSample(40, 40, true) + appendSample(35, 35, true) + appendSample(50, 50, true) + appendSample(55, 55, true) + appendSample(59, 59, true) + appendSample(31, 31, true) + + // Check that Head's time ranges are set properly. + require.Equal(t, 60*time.Minute.Milliseconds(), h.MinTime()) + require.Equal(t, 60*time.Minute.Milliseconds(), h.MaxTime()) + require.Equal(t, 31*time.Minute.Milliseconds(), h.MinOOOTime()) + require.Equal(t, 59*time.Minute.Milliseconds(), h.MaxOOOTime()) + + // Restart head. + require.NoError(t, h.Close()) + wal, err = wlog.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, compression.Snappy) + require.NoError(t, err) + oooWlog, err = wlog.NewSize(nil, nil, filepath.Join(dir, wlog.WblDirName), 32768, compression.Snappy) + require.NoError(t, err) + h, err = NewHead(nil, nil, wal, oooWlog, opts, nil) + require.NoError(t, err) + require.NoError(t, h.Init(0)) // Replay happens here. + + // Get the ooo samples from the Head. + ms, ok, err := h.getOrCreate(l.Hash(), l, false) + require.NoError(t, err) + require.False(t, ok) + require.NotNil(t, ms) + + chks, err := ms.ooo.oooHeadChunk.chunk.ToEncodedChunks(math.MinInt64, math.MaxInt64) + require.NoError(t, err) + require.Len(t, chks, 1) + + it := chks[0].chunk.Iterator(nil) + actOOOSamples, err := storage.ExpandSamples(it, nil) + require.NoError(t, err) + + // OOO chunk will be sorted. Hence sort the expected samples. + sort.Slice(expOOOSamples, func(i, j int) bool { + return expOOOSamples[i].T() < expOOOSamples[j].T() + }) + + // Passing in true for the 'ignoreCounterResets' parameter prevents differences in counter reset headers + // from being factored in to the sample comparison + // TODO(fionaliao): understand counter reset behaviour, might want to modify this later + requireEqualSamples(t, l.String(), expOOOSamples, actOOOSamples, requireEqualSamplesIgnoreCounterResets) + + require.NoError(t, h.Close()) +} + +// TestOOOMmapReplay checks the replay at a low level. +func TestOOOMmapReplay(t *testing.T) { + for name, scenario := range sampleTypeScenarios { + t.Run(name, func(t *testing.T) { + testOOOMmapReplay(t, scenario) + }) + } +} + +func testOOOMmapReplay(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) + oooWlog, err := wlog.NewSize(nil, nil, filepath.Join(dir, wlog.WblDirName), 32768, compression.Snappy) + require.NoError(t, err) + + opts := DefaultHeadOptions() + opts.ChunkRange = 1000 + opts.ChunkDirRoot = dir + opts.OutOfOrderCapMax.Store(30) + opts.OutOfOrderTimeWindow.Store(1000 * time.Minute.Milliseconds()) + + h, err := NewHead(nil, nil, wal, oooWlog, opts, nil) + require.NoError(t, err) + require.NoError(t, h.Init(0)) + + l := labels.FromStrings("foo", "bar") + appendSample := func(mins int64) { + app := h.Appender(context.Background()) + _, _, err := scenario.appendFunc(app, l, mins*time.Minute.Milliseconds(), mins) + require.NoError(t, err) + require.NoError(t, app.Commit()) + } + + // In-order sample. + appendSample(200) + + // Out of order samples. 92 samples to create 3 m-map chunks. + for mins := int64(100); mins <= 191; mins++ { + appendSample(mins) + } + + ms, ok, err := h.getOrCreate(l.Hash(), l, false) + require.NoError(t, err) + require.False(t, ok) + require.NotNil(t, ms) + + require.Len(t, ms.ooo.oooMmappedChunks, 3) + // Verify that we can access the chunks without error. + for _, m := range ms.ooo.oooMmappedChunks { + chk, err := h.chunkDiskMapper.Chunk(m.ref) + require.NoError(t, err) + require.Equal(t, int(m.numSamples), chk.NumSamples()) + } + + expMmapChunks := make([]*mmappedChunk, 3) + copy(expMmapChunks, ms.ooo.oooMmappedChunks) + + // Restart head. + require.NoError(t, h.Close()) + + wal, err = wlog.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, compression.Snappy) + require.NoError(t, err) + oooWlog, err = wlog.NewSize(nil, nil, filepath.Join(dir, wlog.WblDirName), 32768, compression.Snappy) + require.NoError(t, err) + h, err = NewHead(nil, nil, wal, oooWlog, opts, nil) + require.NoError(t, err) + require.NoError(t, h.Init(0)) // Replay happens here. + + // Get the mmap chunks from the Head. + ms, ok, err = h.getOrCreate(l.Hash(), l, false) + require.NoError(t, err) + require.False(t, ok) + require.NotNil(t, ms) + + require.Len(t, ms.ooo.oooMmappedChunks, len(expMmapChunks)) + // Verify that we can access the chunks without error. + for _, m := range ms.ooo.oooMmappedChunks { + chk, err := h.chunkDiskMapper.Chunk(m.ref) + require.NoError(t, err) + require.Equal(t, int(m.numSamples), chk.NumSamples()) + } + + actMmapChunks := make([]*mmappedChunk, len(expMmapChunks)) + copy(actMmapChunks, ms.ooo.oooMmappedChunks) + + require.Equal(t, expMmapChunks, actMmapChunks) + + require.NoError(t, h.Close()) +} + +func TestHeadInit_DiscardChunksWithUnsupportedEncoding(t *testing.T) { + h, _ := newTestHead(t, 1000, compression.None, false) + defer func() { + require.NoError(t, h.Close()) + }() + + require.NoError(t, h.Init(0)) + + ctx := context.Background() + app := h.Appender(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)) + require.NoError(t, err) + } + + require.NoError(t, app.Commit()) + require.Greater(t, prom_testutil.ToFloat64(h.metrics.chunksCreated), 1.0) + + uc := newUnsupportedChunk() + // 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) + for i := 700; i < 1200; i++ { + _, err := app.Append(0, seriesLabels, int64(i), float64(i)) + require.NoError(t, err) + } + + require.NoError(t, app.Commit()) + require.Greater(t, prom_testutil.ToFloat64(h.metrics.chunksCreated), 4.0) + + series, created, err := h.getOrCreate(seriesLabels.Hash(), seriesLabels, false) + require.NoError(t, err) + require.False(t, created, "should already exist") + require.NotNil(t, series, "should return the series we created above") + + series.mmapChunks(h.chunkDiskMapper) + expChunks := make([]*mmappedChunk, len(series.mmappedChunks)) + copy(expChunks, series.mmappedChunks) + + require.NoError(t, h.Close()) + + wal, err := wlog.NewSize(nil, nil, filepath.Join(h.opts.ChunkDirRoot, "wal"), 32768, compression.None) + require.NoError(t, err) + h, err = NewHead(nil, nil, wal, nil, h.opts, nil) + require.NoError(t, err) + require.NoError(t, h.Init(0)) + + series, created, err = h.getOrCreate(seriesLabels.Hash(), seriesLabels, false) + require.NoError(t, err) + require.False(t, created, "should already exist") + require.NotNil(t, series, "should return the series we created above") + + 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) { + dir := t.TempDir() + wal, err := wlog.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, compression.None) + require.NoError(t, err) + + opts := DefaultHeadOptions() + opts.ChunkRange = DefaultBlockDuration + opts.ChunkDirRoot = dir + opts.EnableExemplarStorage = true + opts.MaxExemplars.Store(config.DefaultExemplarsConfig.MaxExemplars) + + h, err := NewHead(nil, nil, wal, nil, opts, nil) + require.NoError(t, err) + require.NoError(t, h.Init(0)) + + lastTs := int64(0) + var ref storage.SeriesRef + lbls := labels.FromStrings("__name__", "testing", "foo", "bar") + addChunks := func() { + interval := DefaultBlockDuration / (4 * 120) + app := h.Appender(context.Background()) + for i := range 250 { + ref, err = app.Append(ref, lbls, lastTs, float64(lastTs)) + lastTs += interval + if i%10 == 0 { + require.NoError(t, app.Commit()) + app = h.Appender(context.Background()) + } + } + require.NoError(t, app.Commit()) + } + + addChunks() + + require.NoError(t, h.Close()) + wal, err = wlog.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, compression.None) + require.NoError(t, err) + + mmapFilePath := filepath.Join(dir, "chunks_head", "000001") + f, err := os.OpenFile(mmapFilePath, os.O_WRONLY, 0o666) + require.NoError(t, err) + _, err = f.WriteAt([]byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11}, 17) + require.NoError(t, err) + require.NoError(t, f.Close()) + + h, err = NewHead(nil, nil, wal, nil, opts, nil) + require.NoError(t, err) + require.NoError(t, h.Init(0)) + + addChunks() + + require.NoError(t, h.Close()) +} + +// Tests https://github.com/prometheus/prometheus/issues/10277. +func TestReplayAfterMmapReplayError(t *testing.T) { + dir := t.TempDir() + var h *Head + var err error + + openHead := func() { + wal, err := wlog.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, compression.None) + require.NoError(t, err) + + opts := DefaultHeadOptions() + opts.ChunkRange = DefaultBlockDuration + opts.ChunkDirRoot = dir + opts.EnableMemorySnapshotOnShutdown = true + opts.MaxExemplars.Store(config.DefaultExemplarsConfig.MaxExemplars) + + h, err = NewHead(nil, nil, wal, nil, opts, nil) + require.NoError(t, err) + require.NoError(t, h.Init(0)) + } + + openHead() + + itvl := int64(15 * time.Second / time.Millisecond) + lastTs := int64(0) + lbls := labels.FromStrings("__name__", "testing", "foo", "bar") + var expSamples []chunks.Sample + addSamples := func(numSamples int) { + app := h.Appender(context.Background()) + var ref storage.SeriesRef + for i := range numSamples { + ref, err = app.Append(ref, lbls, lastTs, float64(lastTs)) + 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()) + } + } + require.NoError(t, app.Commit()) + } + + // Creating multiple m-map files. + for i := range 5 { + addSamples(250) + require.NoError(t, h.Close()) + if i != 4 { + // Don't open head for the last iteration. + openHead() + } + } + + files, err := os.ReadDir(filepath.Join(dir, "chunks_head")) + require.Len(t, files, 5) + + // Corrupt a m-map file. + mmapFilePath := filepath.Join(dir, "chunks_head", "000002") + f, err := os.OpenFile(mmapFilePath, os.O_WRONLY, 0o666) + require.NoError(t, err) + _, err = f.WriteAt([]byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11}, 17) + require.NoError(t, err) + require.NoError(t, f.Close()) + + openHead() + h.mmapHeadChunks() + + // There should be less m-map files due to corruption. + files, err = os.ReadDir(filepath.Join(dir, "chunks_head")) + require.Len(t, files, 2) + + // Querying should not panic. + q, err := NewBlockQuerier(h, 0, lastTs) + require.NoError(t, err) + res := query(t, q, labels.MustNewMatcher(labels.MatchEqual, "__name__", "testing")) + require.Equal(t, map[string][]chunks.Sample{lbls.String(): expSamples}, res) + + require.NoError(t, h.Close()) +} + +func TestOOOAppendWithNoSeries(t *testing.T) { + for name, scenario := range sampleTypeScenarios { + t.Run(name, func(t *testing.T) { + testOOOAppendWithNoSeries(t, scenario.appendFunc) + }) + } +} + +func testOOOAppendWithNoSeries(t *testing.T, appendFunc func(appender storage.Appender, 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) + oooWlog, err := wlog.NewSize(nil, nil, filepath.Join(dir, wlog.WblDirName), 32768, compression.Snappy) + require.NoError(t, err) + + opts := DefaultHeadOptions() + opts.ChunkDirRoot = dir + opts.OutOfOrderCapMax.Store(30) + opts.OutOfOrderTimeWindow.Store(120 * time.Minute.Milliseconds()) + + h, err := NewHead(nil, nil, wal, oooWlog, opts, nil) + require.NoError(t, err) + t.Cleanup(func() { + require.NoError(t, h.Close()) + }) + 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) + require.NoError(t, err) + require.NoError(t, app.Commit()) + } + + verifyOOOSamples := func(lbls labels.Labels, expSamples int) { + ms, created, err := h.getOrCreate(lbls.Hash(), lbls, false) + require.NoError(t, err) + require.False(t, created) + require.NotNil(t, ms) + + require.Nil(t, ms.headChunks) + require.NotNil(t, ms.ooo.oooHeadChunk) + require.Equal(t, expSamples, ms.ooo.oooHeadChunk.chunk.NumSamples()) + } + + verifyInOrderSamples := func(lbls labels.Labels, expSamples int) { + ms, created, err := h.getOrCreate(lbls.Hash(), lbls, false) + require.NoError(t, err) + require.False(t, created) + require.NotNil(t, ms) + + require.Nil(t, ms.ooo) + require.NotNil(t, ms.headChunks) + require.Equal(t, expSamples, ms.headChunks.chunk.NumSamples()) + } + + newLabels := func(idx int) labels.Labels { return labels.FromStrings("foo", strconv.Itoa(idx)) } + + s1 := newLabels(1) + appendSample(s1, 300) // At 300m. + verifyInOrderSamples(s1, 1) + + // At 239m, the sample cannot be appended to in-order chunk since it is + // beyond the minValidTime. So it should go in OOO chunk. + // Series does not exist for s2 yet. + s2 := newLabels(2) + appendSample(s2, 239) // OOO sample. + verifyOOOSamples(s2, 1) + + // Similar for 180m. + s3 := newLabels(3) + appendSample(s3, 180) // OOO sample. + verifyOOOSamples(s3, 1) + + // Now 179m is too old. + s4 := newLabels(4) + app := h.Appender(context.Background()) + _, _, err = appendFunc(app, s4, 179*time.Minute.Milliseconds(), 179) + require.Equal(t, storage.ErrTooOldSample, err) + require.NoError(t, app.Rollback()) + verifyOOOSamples(s3, 1) + + // Samples still go into in-order chunk for samples within + // appendable minValidTime. + s5 := newLabels(5) + appendSample(s5, 240) + verifyInOrderSamples(s5, 1) +} + +func TestHeadMinOOOTimeUpdate(t *testing.T) { + for name, scenario := range sampleTypeScenarios { + t.Run(name, func(t *testing.T) { + if scenario.sampleType == sampleMetricTypeFloat { + testHeadMinOOOTimeUpdate(t, scenario) + } + }) + } +} + +func testHeadMinOOOTimeUpdate(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) + oooWlog, err := wlog.NewSize(nil, nil, filepath.Join(dir, wlog.WblDirName), 32768, compression.Snappy) + require.NoError(t, err) + + opts := DefaultHeadOptions() + opts.ChunkDirRoot = dir + opts.OutOfOrderTimeWindow.Store(10 * time.Minute.Milliseconds()) + + h, err := NewHead(nil, nil, wal, oooWlog, opts, nil) + require.NoError(t, err) + t.Cleanup(func() { + require.NoError(t, h.Close()) + }) + 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) + require.NoError(t, err) + require.NoError(t, app.Commit()) + } + + appendSample(300) // In-order sample. + require.Equal(t, int64(math.MaxInt64), h.MinOOOTime()) + + appendSample(295) // OOO sample. + require.Equal(t, 295*time.Minute.Milliseconds(), h.MinOOOTime()) + + // Allowed window for OOO is >=290, which is before the earliest ooo sample 295, so it gets set to the lower value. + require.NoError(t, h.truncateOOO(0, 1)) + require.Equal(t, 290*time.Minute.Milliseconds(), h.MinOOOTime()) + + appendSample(310) // In-order sample. + appendSample(305) // OOO sample. + require.Equal(t, 290*time.Minute.Milliseconds(), h.MinOOOTime()) + + // Now the OOO sample 295 was not gc'ed yet. And allowed window for OOO is now >=300. + // So the lowest among them, 295, is set as minOOOTime. + require.NoError(t, h.truncateOOO(0, 2)) + require.Equal(t, 295*time.Minute.Milliseconds(), h.MinOOOTime()) +} + +func TestGaugeHistogramWALAndChunkHeader(t *testing.T) { + l := labels.FromStrings("a", "b") + head, _ := newTestHead(t, 1000, compression.None, false) + t.Cleanup(func() { + require.NoError(t, head.Close()) + }) + require.NoError(t, head.Init(0)) + + ts := int64(0) + appendHistogram := func(h *histogram.Histogram) { + ts++ + app := head.Appender(context.Background()) + _, err := app.AppendHistogram(0, l, ts, h.Copy(), nil) + require.NoError(t, err) + require.NoError(t, app.Commit()) + } + + hists := tsdbutil.GenerateTestGaugeHistograms(5) + hists[0].CounterResetHint = histogram.UnknownCounterReset + appendHistogram(hists[0]) + appendHistogram(hists[1]) + appendHistogram(hists[2]) + hists[3].CounterResetHint = histogram.UnknownCounterReset + appendHistogram(hists[3]) + appendHistogram(hists[3]) + appendHistogram(hists[4]) + + checkHeaders := func() { + head.mmapHeadChunks() + ms, _, err := head.getOrCreate(l.Hash(), l, false) + require.NoError(t, err) + require.Len(t, ms.mmappedChunks, 3) + expHeaders := []chunkenc.CounterResetHeader{ + chunkenc.UnknownCounterReset, + chunkenc.GaugeType, + chunkenc.NotCounterReset, + chunkenc.GaugeType, + } + for i, mmapChunk := range ms.mmappedChunks { + chk, err := head.chunkDiskMapper.Chunk(mmapChunk.ref) + require.NoError(t, err) + require.Equal(t, expHeaders[i], chk.(*chunkenc.HistogramChunk).GetCounterResetHeader()) + } + require.Equal(t, expHeaders[len(expHeaders)-1], ms.headChunks.chunk.(*chunkenc.HistogramChunk).GetCounterResetHeader()) + } + checkHeaders() + + recs := readTestWAL(t, head.wal.Dir()) + require.Equal(t, []any{ + []record.RefSeries{ + { + Ref: 1, + Labels: labels.FromStrings("a", "b"), + }, + }, + []record.RefHistogramSample{{Ref: 1, T: 1, H: hists[0]}}, + []record.RefHistogramSample{{Ref: 1, T: 2, H: hists[1]}}, + []record.RefHistogramSample{{Ref: 1, T: 3, H: hists[2]}}, + []record.RefHistogramSample{{Ref: 1, T: 4, H: hists[3]}}, + []record.RefHistogramSample{{Ref: 1, T: 5, H: hists[3]}}, + []record.RefHistogramSample{{Ref: 1, T: 6, H: hists[4]}}, + }, recs) + + // Restart Head without mmap chunks to expect the WAL replay to recognize gauge histograms. + require.NoError(t, head.Close()) + require.NoError(t, os.RemoveAll(mmappedChunksDir(head.opts.ChunkDirRoot))) + + w, err := wlog.NewSize(nil, nil, head.wal.Dir(), 32768, compression.None) + require.NoError(t, err) + head, err = NewHead(nil, nil, w, nil, head.opts, nil) + require.NoError(t, err) + require.NoError(t, head.Init(0)) + + checkHeaders() +} + +func TestGaugeFloatHistogramWALAndChunkHeader(t *testing.T) { + l := labels.FromStrings("a", "b") + head, _ := newTestHead(t, 1000, compression.None, false) + t.Cleanup(func() { + require.NoError(t, head.Close()) + }) + require.NoError(t, head.Init(0)) + + ts := int64(0) + appendHistogram := func(h *histogram.FloatHistogram) { + ts++ + app := head.Appender(context.Background()) + _, err := app.AppendHistogram(0, l, ts, nil, h.Copy()) + require.NoError(t, err) + require.NoError(t, app.Commit()) + } + + hists := tsdbutil.GenerateTestGaugeFloatHistograms(5) + hists[0].CounterResetHint = histogram.UnknownCounterReset + appendHistogram(hists[0]) + appendHistogram(hists[1]) + appendHistogram(hists[2]) + hists[3].CounterResetHint = histogram.UnknownCounterReset + appendHistogram(hists[3]) + appendHistogram(hists[3]) + appendHistogram(hists[4]) + + checkHeaders := func() { + ms, _, err := head.getOrCreate(l.Hash(), l, false) + require.NoError(t, err) + head.mmapHeadChunks() + require.Len(t, ms.mmappedChunks, 3) + expHeaders := []chunkenc.CounterResetHeader{ + chunkenc.UnknownCounterReset, + chunkenc.GaugeType, + chunkenc.UnknownCounterReset, + chunkenc.GaugeType, + } + for i, mmapChunk := range ms.mmappedChunks { + chk, err := head.chunkDiskMapper.Chunk(mmapChunk.ref) + require.NoError(t, err) + require.Equal(t, expHeaders[i], chk.(*chunkenc.FloatHistogramChunk).GetCounterResetHeader()) + } + require.Equal(t, expHeaders[len(expHeaders)-1], ms.headChunks.chunk.(*chunkenc.FloatHistogramChunk).GetCounterResetHeader()) + } + checkHeaders() + + recs := readTestWAL(t, head.wal.Dir()) + require.Equal(t, []any{ + []record.RefSeries{ + { + Ref: 1, + Labels: labels.FromStrings("a", "b"), + }, + }, + []record.RefFloatHistogramSample{{Ref: 1, T: 1, FH: hists[0]}}, + []record.RefFloatHistogramSample{{Ref: 1, T: 2, FH: hists[1]}}, + []record.RefFloatHistogramSample{{Ref: 1, T: 3, FH: hists[2]}}, + []record.RefFloatHistogramSample{{Ref: 1, T: 4, FH: hists[3]}}, + []record.RefFloatHistogramSample{{Ref: 1, T: 5, FH: hists[3]}}, + []record.RefFloatHistogramSample{{Ref: 1, T: 6, FH: hists[4]}}, + }, recs) + + // Restart Head without mmap chunks to expect the WAL replay to recognize gauge histograms. + require.NoError(t, head.Close()) + require.NoError(t, os.RemoveAll(mmappedChunksDir(head.opts.ChunkDirRoot))) + + w, err := wlog.NewSize(nil, nil, head.wal.Dir(), 32768, compression.None) + require.NoError(t, err) + head, err = NewHead(nil, nil, w, nil, head.opts, nil) + require.NoError(t, err) + require.NoError(t, head.Init(0)) + + checkHeaders() +} + +func TestSnapshotAheadOfWALError(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) + require.NoError(t, err) + require.NoError(t, app.Commit()) + + // Increment snapshot index to create sufficiently large difference. + for range 2 { + _, err = head.wal.NextSegment() + require.NoError(t, err) + } + require.NoError(t, head.Close()) // This will create a snapshot. + + _, idx, _, err := LastChunkSnapshot(head.opts.ChunkDirRoot) + require.NoError(t, err) + require.Equal(t, 2, idx) + + // Restart the WAL while keeping the old snapshot. The new head is created manually in this case in order + // to keep using the same snapshot directory instead of a random one. + require.NoError(t, os.RemoveAll(head.wal.Dir())) + head.opts.EnableMemorySnapshotOnShutdown = false + w, _ := wlog.NewSize(nil, nil, head.wal.Dir(), 32768, compression.None) + 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) + require.NoError(t, err) + require.NoError(t, app.Commit()) + lastSegment, _, _ := w.LastSegmentAndOffset() + require.Equal(t, 0, lastSegment) + require.NoError(t, head.Close()) + + // New WAL is saved, but old snapshot still exists. + _, idx, _, err = LastChunkSnapshot(head.opts.ChunkDirRoot) + require.NoError(t, err) + require.Equal(t, 2, idx) + + // Create new Head which should detect the incorrect index and delete the snapshot. + head.opts.EnableMemorySnapshotOnShutdown = true + w, _ = wlog.NewSize(nil, nil, head.wal.Dir(), 32768, compression.None) + head, err = NewHead(nil, nil, w, nil, head.opts, nil) + require.NoError(t, err) + require.NoError(t, head.Init(math.MinInt64)) + + // Verify that snapshot directory does not exist anymore. + _, _, _, err = LastChunkSnapshot(head.opts.ChunkDirRoot) + require.Equal(t, record.ErrNotFound, err) + + 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) { + ctx := context.Background() + testCases := map[string]struct { + numTotalSamples int + timestampJitter bool + floatValFunc func(i int) float64 + histValFunc func(i int) *histogram.Histogram + expectedChks []struct { + numSamples int + numBytes int + } + }{ + "float samples": { + numTotalSamples: 180, + floatValFunc: func(int) float64 { + return 1. + }, + expectedChks: []struct { + numSamples int + numBytes int + }{ + {numSamples: 120, numBytes: 46}, + {numSamples: 60, numBytes: 32}, + }, + }, + "large float samples": { + // Normally 120 samples would fit into a single chunk but these chunks violate the 1005 byte soft cap. + numTotalSamples: 120, + timestampJitter: true, + floatValFunc: func(i int) float64 { + // Flipping between these two make each sample val take at least 64 bits. + vals := []float64{math.MaxFloat64, 0x00} + return vals[i%len(vals)] + }, + expectedChks: []struct { + numSamples int + numBytes int + }{ + {99, 1008}, + {21, 219}, + }, + }, + "small histograms": { + numTotalSamples: 240, + histValFunc: func() func(i int) *histogram.Histogram { + hists := histogram.GenerateBigTestHistograms(240, 10) + return func(i int) *histogram.Histogram { + return hists[i] + } + }(), + expectedChks: []struct { + numSamples int + numBytes int + }{ + {120, 1087}, + {120, 1039}, + }, + }, + "large histograms": { + numTotalSamples: 240, + histValFunc: func() func(i int) *histogram.Histogram { + hists := histogram.GenerateBigTestHistograms(240, 100) + return func(i int) *histogram.Histogram { + return hists[i] + } + }(), + expectedChks: []struct { + numSamples int + numBytes int + }{ + {40, 896}, + {40, 899}, + {40, 896}, + {30, 690}, + {30, 691}, + {30, 694}, + {30, 693}, + }, + }, + "really large histograms": { + // Really large histograms; each chunk can only contain a single histogram but we have a 10 sample minimum + // per chunk. + numTotalSamples: 11, + histValFunc: func() func(i int) *histogram.Histogram { + hists := histogram.GenerateBigTestHistograms(11, 100000) + return func(i int) *histogram.Histogram { + return hists[i] + } + }(), + expectedChks: []struct { + numSamples int + numBytes int + }{ + {10, 200103}, + {1, 87540}, + }, + }, + } + for testName, tc := range testCases { + t.Run(testName, func(t *testing.T) { + h, _ := newTestHead(t, DefaultBlockDuration, compression.None, false) + defer func() { + require.NoError(t, h.Close()) + }() + + a := h.Appender(context.Background()) + + ts := int64(10000) + lbls := labels.FromStrings("foo", "bar") + jitter := []int64{0, 1} // A bit of jitter to prevent dod=0. + + for i := 0; i < tc.numTotalSamples; i++ { + if tc.floatValFunc != nil { + _, err := a.Append(0, lbls, ts, tc.floatValFunc(i)) + require.NoError(t, err) + } else if tc.histValFunc != nil { + _, err := a.AppendHistogram(0, lbls, ts, tc.histValFunc(i), nil) + require.NoError(t, err) + } + + ts += int64(60 * time.Second / time.Millisecond) + if tc.timestampJitter { + ts += jitter[i%len(jitter)] + } + } + + require.NoError(t, a.Commit()) + + idxReader, err := h.Index() + require.NoError(t, err) + + chkReader, err := h.Chunks() + require.NoError(t, err) + + p, err := idxReader.Postings(ctx, "foo", "bar") + require.NoError(t, err) + + var lblBuilder labels.ScratchBuilder + + for p.Next() { + sRef := p.At() + + chkMetas := make([]chunks.Meta, len(tc.expectedChks)) + require.NoError(t, idxReader.Series(sRef, &lblBuilder, &chkMetas)) + + require.Len(t, chkMetas, len(tc.expectedChks)) + + for i, expected := range tc.expectedChks { + chk, iterable, err := chkReader.ChunkOrIterable(chkMetas[i]) + require.NoError(t, err) + require.Nil(t, iterable) + + require.Equal(t, expected.numSamples, chk.NumSamples()) + require.Len(t, chk.Bytes(), expected.numBytes) + } + } + }) + } +} + +// TestHeadDetectsDuplicateSampleAtSizeLimit tests a regression where a duplicate sample +// 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) { + numSamples := 1000 + baseTS := int64(1695209650) + + h, _ := newTestHead(t, DefaultBlockDuration, compression.None, false) + defer func() { + require.NoError(t, h.Close()) + }() + + a := h.Appender(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)]) + err = a.Commit() + require.NoError(t, err) + a = h.Appender(context.Background()) + } + + indexReader, err := h.Index() + require.NoError(t, err) + + var ( + chunks []chunks.Meta + builder labels.ScratchBuilder + ) + require.NoError(t, indexReader.Series(1, &builder, &chunks)) + + chunkReader, err := h.Chunks() + require.NoError(t, err) + + storedSampleCount := 0 + for _, chunkMeta := range chunks { + chunk, iterable, err := chunkReader.ChunkOrIterable(chunkMeta) + require.NoError(t, err) + require.Nil(t, iterable) + storedSampleCount += chunk.NumSamples() + } + + require.Equal(t, numSamples/2, storedSampleCount) +} + +func TestWALSampleAndExemplarOrder(t *testing.T) { + lbls := labels.FromStrings("foo", "bar") + testcases := map[string]struct { + appendF func(app storage.Appender, 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) + }, + 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) + }, + 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)) + }, + expectedType: reflect.TypeOf([]record.RefFloatHistogramSample{}), + }, + } + + for testName, tc := range testcases { + t.Run(testName, func(t *testing.T) { + h, w := newTestHead(t, 1000, compression.None, false) + defer func() { + require.NoError(t, h.Close()) + }() + + app := h.Appender(context.Background()) + ref, err := tc.appendF(app, 10) + require.NoError(t, err) + app.AppendExemplar(ref, lbls, exemplar.Exemplar{Value: 1.0, Ts: 5}) + + app.Commit() + + recs := readTestWAL(t, w.Dir()) + require.Len(t, recs, 3) + _, ok := recs[0].([]record.RefSeries) + require.True(t, ok, "expected first record to be a RefSeries") + actualType := reflect.TypeOf(recs[1]) + require.Equal(t, tc.expectedType, actualType, "expected second record to be a %s", tc.expectedType) + _, ok = recs[2].([]record.RefExemplar) + require.True(t, ok, "expected third record to be a RefExemplar") + }) + } +} + +// 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) { + head, _ := newTestHead(t, DefaultBlockDuration, compression.None, false) + t.Cleanup(func() { head.Close() }) + + ls := labels.FromStrings(labels.MetricName, "test") + + { + // Append a float 10.0 @ 1_000 + app := head.Appender(context.Background()) + _, err := app.Append(0, ls, 1_000, 10.0) + require.NoError(t, err) + require.NoError(t, app.Commit()) + } + + { + // Append a float histogram @ 2_000 + app := head.Appender(context.Background()) + h := tsdbutil.GenerateTestHistogram(1) + _, err := app.AppendHistogram(0, ls, 2_000, h, nil) + require.NoError(t, err) + require.NoError(t, app.Commit()) + } + + app := head.Appender(context.Background()) + _, err := app.Append(0, ls, 2_000, 10.0) + require.Error(t, err) + require.ErrorIs(t, err, storage.NewDuplicateHistogramToFloatErr(2_000, 10.0)) +} + +func TestHeadAppender_AppendST(t *testing.T) { + testHistogram := tsdbutil.GenerateTestHistogram(1) + testHistogram.CounterResetHint = histogram.NotCounterReset + testFloatHistogram := tsdbutil.GenerateTestFloatHistogram(1) + testFloatHistogram.CounterResetHint = histogram.NotCounterReset + // TODO(beorn7): Once issue #15346 is fixed, the CounterResetHint of the + // following two zero histograms should be histogram.CounterReset. + testZeroHistogram := &histogram.Histogram{ + Schema: testHistogram.Schema, + ZeroThreshold: testHistogram.ZeroThreshold, + PositiveSpans: testHistogram.PositiveSpans, + NegativeSpans: testHistogram.NegativeSpans, + PositiveBuckets: []int64{0, 0, 0, 0}, + NegativeBuckets: []int64{0, 0, 0, 0}, + } + testZeroFloatHistogram := &histogram.FloatHistogram{ + Schema: testFloatHistogram.Schema, + ZeroThreshold: testFloatHistogram.ZeroThreshold, + PositiveSpans: testFloatHistogram.PositiveSpans, + NegativeSpans: testFloatHistogram.NegativeSpans, + PositiveBuckets: []float64{0, 0, 0, 0}, + NegativeBuckets: []float64{0, 0, 0, 0}, + } + type appendableSamples struct { + ts int64 + fSample float64 + h *histogram.Histogram + fh *histogram.FloatHistogram + st int64 + } + for _, tc := range []struct { + name string + appendableSamples []appendableSamples + expectedSamples []chunks.Sample + }{ + { + name: "In order ct+normal sample/floatSample", + appendableSamples: []appendableSamples{ + {ts: 100, fSample: 10, st: 1}, + {ts: 101, fSample: 10, st: 1}, + }, + expectedSamples: []chunks.Sample{ + sample{t: 1, f: 0}, + sample{t: 100, f: 10}, + sample{t: 101, f: 10}, + }, + }, + { + name: "In order ct+normal sample/histogram", + appendableSamples: []appendableSamples{ + {ts: 100, h: testHistogram, st: 1}, + {ts: 101, h: testHistogram, st: 1}, + }, + expectedSamples: func() []chunks.Sample { + return []chunks.Sample{ + sample{t: 1, h: testZeroHistogram}, + sample{t: 100, h: testHistogram}, + sample{t: 101, h: testHistogram}, + } + }(), + }, + { + name: "In order ct+normal sample/floathistogram", + appendableSamples: []appendableSamples{ + {ts: 100, fh: testFloatHistogram, st: 1}, + {ts: 101, fh: testFloatHistogram, st: 1}, + }, + expectedSamples: func() []chunks.Sample { + return []chunks.Sample{ + sample{t: 1, fh: testZeroFloatHistogram}, + sample{t: 100, fh: testFloatHistogram}, + sample{t: 101, fh: testFloatHistogram}, + } + }(), + }, + { + name: "Consecutive appends with same st ignore st/floatSample", + appendableSamples: []appendableSamples{ + {ts: 100, fSample: 10, st: 1}, + {ts: 101, fSample: 10, st: 1}, + }, + expectedSamples: []chunks.Sample{ + sample{t: 1, f: 0}, + sample{t: 100, f: 10}, + sample{t: 101, f: 10}, + }, + }, + { + name: "Consecutive appends with same st ignore st/histogram", + appendableSamples: []appendableSamples{ + {ts: 100, h: testHistogram, st: 1}, + {ts: 101, h: testHistogram, st: 1}, + }, + expectedSamples: func() []chunks.Sample { + return []chunks.Sample{ + sample{t: 1, h: testZeroHistogram}, + sample{t: 100, h: testHistogram}, + sample{t: 101, h: testHistogram}, + } + }(), + }, + { + name: "Consecutive appends with same st ignore st/floathistogram", + appendableSamples: []appendableSamples{ + {ts: 100, fh: testFloatHistogram, st: 1}, + {ts: 101, fh: testFloatHistogram, st: 1}, + }, + expectedSamples: func() []chunks.Sample { + return []chunks.Sample{ + sample{t: 1, fh: testZeroFloatHistogram}, + sample{t: 100, fh: testFloatHistogram}, + sample{t: 101, fh: testFloatHistogram}, + } + }(), + }, + { + name: "Consecutive appends with newer st do not ignore st/floatSample", + appendableSamples: []appendableSamples{ + {ts: 100, fSample: 10, st: 1}, + {ts: 102, fSample: 10, st: 101}, + }, + expectedSamples: []chunks.Sample{ + sample{t: 1, f: 0}, + sample{t: 100, f: 10}, + sample{t: 101, f: 0}, + sample{t: 102, f: 10}, + }, + }, + { + name: "Consecutive appends with newer st do not ignore st/histogram", + appendableSamples: []appendableSamples{ + {ts: 100, h: testHistogram, st: 1}, + {ts: 102, h: testHistogram, st: 101}, + }, + expectedSamples: []chunks.Sample{ + sample{t: 1, h: testZeroHistogram}, + sample{t: 100, h: testHistogram}, + sample{t: 101, h: testZeroHistogram}, + sample{t: 102, h: testHistogram}, + }, + }, + { + name: "Consecutive appends with newer st do not ignore st/floathistogram", + appendableSamples: []appendableSamples{ + {ts: 100, fh: testFloatHistogram, st: 1}, + {ts: 102, fh: testFloatHistogram, st: 101}, + }, + expectedSamples: []chunks.Sample{ + sample{t: 1, fh: testZeroFloatHistogram}, + sample{t: 100, fh: testFloatHistogram}, + sample{t: 101, fh: testZeroFloatHistogram}, + sample{t: 102, fh: testFloatHistogram}, + }, + }, + { + name: "ST equals to previous sample timestamp is ignored/floatSample", + appendableSamples: []appendableSamples{ + {ts: 100, fSample: 10, st: 1}, + {ts: 101, fSample: 10, st: 100}, + }, + expectedSamples: []chunks.Sample{ + sample{t: 1, f: 0}, + sample{t: 100, f: 10}, + sample{t: 101, f: 10}, + }, + }, + { + name: "ST equals to previous sample timestamp is ignored/histogram", + appendableSamples: []appendableSamples{ + {ts: 100, h: testHistogram, st: 1}, + {ts: 101, h: testHistogram, st: 100}, + }, + expectedSamples: func() []chunks.Sample { + return []chunks.Sample{ + sample{t: 1, h: testZeroHistogram}, + sample{t: 100, h: testHistogram}, + sample{t: 101, h: testHistogram}, + } + }(), + }, + { + name: "ST equals to previous sample timestamp is ignored/floathistogram", + appendableSamples: []appendableSamples{ + {ts: 100, fh: testFloatHistogram, st: 1}, + {ts: 101, fh: testFloatHistogram, st: 100}, + }, + expectedSamples: func() []chunks.Sample { + return []chunks.Sample{ + sample{t: 1, fh: testZeroFloatHistogram}, + sample{t: 100, fh: testFloatHistogram}, + sample{t: 101, fh: testFloatHistogram}, + } + }(), + }, + } { + t.Run(tc.name, func(t *testing.T) { + h, _ := newTestHead(t, DefaultBlockDuration, compression.None, false) + 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) + } + } + require.NoError(t, a.Commit()) + + q, err := NewBlockQuerier(h, math.MinInt64, math.MaxInt64) + require.NoError(t, err) + result := query(t, q, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + require.Equal(t, tc.expectedSamples, result[`{foo="bar"}`]) + }) + } +} + +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) { + 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) + 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) + return err + }, + } + for name, tc := range testCases { + t.Run(name, func(t *testing.T) { + testHeadAppendHistogramAndCommitConcurrency(t, tc) + }) + } +} + +func testHeadAppendHistogramAndCommitConcurrency(t *testing.T, appendFn func(storage.Appender, int) error) { + head, _ := newTestHead(t, 1000, compression.None, false) + defer func() { + require.NoError(t, head.Close()) + }() + + wg := sync.WaitGroup{} + wg.Add(2) + + // How this works: Commit() should be atomic, thus one of the commits will + // be first and the other second. The first commit will create a new series + // and write a sample. The second commit will see an exact duplicate sample + // which it should ignore. Unless there's a race that causes the + // memSeries.lastHistogram to be corrupt and fail the duplicate check. + go func() { + defer wg.Done() + for i := range 10000 { + app := head.Appender(context.Background()) + require.NoError(t, appendFn(app, i)) + require.NoError(t, app.Commit()) + } + }() + + go func() { + defer wg.Done() + for i := range 10000 { + app := head.Appender(context.Background()) + require.NoError(t, appendFn(app, i)) + require.NoError(t, app.Commit()) + } + }() + + wg.Wait() +} + +func TestHead_NumStaleSeries(t *testing.T) { + head, _ := newTestHead(t, 1000, compression.None, false) + t.Cleanup(func() { + require.NoError(t, head.Close()) + }) + require.NoError(t, head.Init(0)) + + // Initially, no series should be stale. + 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) + 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) + 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) + require.NoError(t, err) + require.NoError(t, app.Commit()) + } + + verifySeriesCounts := func(numStaleSeries, numSeries int) { + require.Equal(t, uint64(numStaleSeries), head.NumStaleSeries()) + require.Equal(t, uint64(numSeries), head.NumSeries()) + } + + restartHeadAndVerifySeriesCounts := func(numStaleSeries, numSeries int) { + verifySeriesCounts(numStaleSeries, numSeries) + + require.NoError(t, head.Close()) + + wal, err := wlog.NewSize(nil, nil, filepath.Join(head.opts.ChunkDirRoot, "wal"), 32768, compression.None) + require.NoError(t, err) + head, err = NewHead(nil, nil, wal, nil, head.opts, nil) + require.NoError(t, err) + require.NoError(t, head.Init(0)) + + verifySeriesCounts(numStaleSeries, numSeries) + } + + // Create some series with normal samples. + series1 := labels.FromStrings("name", "series1", "label", "value1") + series2 := labels.FromStrings("name", "series2", "label", "value2") + series3 := labels.FromStrings("name", "series3", "label", "value3") + + // Add normal samples to all series. + appendSample(series1, 100, 1) + appendSample(series2, 100, 2) + appendSample(series3, 100, 3) + // Still no stale series. + verifySeriesCounts(0, 3) + + // Make series1 stale by appending a stale sample. Now we should have 1 stale series. + appendSample(series1, 200, math.Float64frombits(value.StaleNaN)) + verifySeriesCounts(1, 3) + + // Make series2 stale as well. + appendSample(series2, 200, math.Float64frombits(value.StaleNaN)) + verifySeriesCounts(2, 3) + restartHeadAndVerifySeriesCounts(2, 3) + + // Add a non-stale sample to series1. It should not be counted as stale now. + appendSample(series1, 300, 10) + verifySeriesCounts(1, 3) + restartHeadAndVerifySeriesCounts(1, 3) + + // Test that series3 doesn't become stale when we add another normal sample. + appendSample(series3, 200, 10) + verifySeriesCounts(1, 3) + + // Test histogram stale samples as well. + series4 := labels.FromStrings("name", "series4", "type", "histogram") + h := tsdbutil.GenerateTestHistograms(1)[0] + appendHistogram(series4, 100, h) + verifySeriesCounts(1, 4) + + // Make histogram series stale. + staleHist := h.Copy() + staleHist.Sum = math.Float64frombits(value.StaleNaN) + appendHistogram(series4, 200, staleHist) + verifySeriesCounts(2, 4) + + // Test float histogram stale samples. + series5 := labels.FromStrings("name", "series5", "type", "float_histogram") + fh := tsdbutil.GenerateTestFloatHistograms(1)[0] + appendFloatHistogram(series5, 100, fh) + verifySeriesCounts(2, 5) + restartHeadAndVerifySeriesCounts(2, 5) + + // Make float histogram series stale. + staleFH := fh.Copy() + staleFH.Sum = math.Float64frombits(value.StaleNaN) + appendFloatHistogram(series5, 200, staleFH) + verifySeriesCounts(3, 5) + + // Make histogram sample non-stale and stale back again. + appendHistogram(series4, 210, h) + verifySeriesCounts(2, 5) + appendHistogram(series4, 220, staleHist) + verifySeriesCounts(3, 5) + + // Make float histogram sample non-stale and stale back again. + appendFloatHistogram(series5, 210, fh) + verifySeriesCounts(2, 5) + appendFloatHistogram(series5, 220, staleFH) + verifySeriesCounts(3, 5) + + // Series 1 and 3 are not stale at this point. Add a new sample to series 1 and series 5, + // so after the GC and removing series 2, 3, 4, we should be left with 1 stale and 1 non-stale series. + appendSample(series1, 400, 10) + appendFloatHistogram(series5, 400, staleFH) + restartHeadAndVerifySeriesCounts(3, 5) + + // This will test restarting with snapshot. + head.opts.EnableMemorySnapshotOnShutdown = true + restartHeadAndVerifySeriesCounts(3, 5) + + // Test garbage collection behavior - stale series should be decremented when GC'd. + // Force a garbage collection by truncating old data. + require.NoError(t, head.Truncate(300)) + + // After truncation, run GC to collect old chunks/series. + head.gc() + + // series 1 and series 5 are left. + verifySeriesCounts(1, 2) + + // Test creating a new series for each of float, histogram, float histogram that starts as stale. + // This should be counted as stale. + series6 := labels.FromStrings("name", "series6", "direct", "stale") + series7 := labels.FromStrings("name", "series7", "direct", "stale") + series8 := labels.FromStrings("name", "series8", "direct", "stale") + appendSample(series6, 400, math.Float64frombits(value.StaleNaN)) + verifySeriesCounts(2, 3) + appendHistogram(series7, 400, staleHist) + verifySeriesCounts(3, 4) + appendFloatHistogram(series8, 400, staleFH) + verifySeriesCounts(4, 5) +} + +// TestHistogramStalenessConversionMetrics verifies that staleness marker conversion correctly +// increments the right appender metrics for both histogram and float histogram scenarios. +func TestHistogramStalenessConversionMetrics(t *testing.T) { + testCases := []struct { + name string + setupHistogram func(app storage.Appender, 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) + 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]) + return err + }, + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + head, _ := newTestHead(t, 1000, compression.None, false) + defer func() { + require.NoError(t, head.Close()) + }() + + lbls := labels.FromStrings("name", tc.name) + + // Helper to get counter values + getSampleCounter := func(sampleType string) float64 { + metric := &dto.Metric{} + err := head.metrics.samplesAppended.WithLabelValues(sampleType).Write(metric) + require.NoError(t, err) + return metric.GetCounter().GetValue() + } + + // Step 1: Establish a series with histogram data + app := head.Appender(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)) + require.NoError(t, err) + require.NoError(t, app.Commit()) + + // Count what was actually stored by querying the series + q, err := NewBlockQuerier(head, 0, 3000) + require.NoError(t, err) + defer q.Close() + + ss := q.Select(context.Background(), false, nil, labels.MustNewMatcher(labels.MatchEqual, "name", tc.name)) + require.True(t, ss.Next()) + series := ss.At() + + it := series.Iterator(nil) + + actualFloatSamples := 0 + actualHistogramSamples := 0 + + for valType := it.Next(); valType != chunkenc.ValNone; valType = it.Next() { + switch valType { + case chunkenc.ValFloat: + actualFloatSamples++ + case chunkenc.ValHistogram, chunkenc.ValFloatHistogram: + actualHistogramSamples++ + } + } + require.NoError(t, it.Err()) + + // Verify what was actually stored - should be 0 floats, 2 histograms (original + converted staleness marker) + require.Equal(t, 0, actualFloatSamples, "Should have 0 float samples stored") + require.Equal(t, 2, actualHistogramSamples, "Should have 2 histogram samples: original + converted staleness marker") + + // The metrics should match what was actually stored + require.Equal(t, float64(actualFloatSamples), getSampleCounter(sampleMetricTypeFloat), + "Float counter should match actual float samples stored") + require.Equal(t, float64(actualHistogramSamples), getSampleCounter(sampleMetricTypeHistogram), + "Histogram counter should match actual histogram samples stored") + }) + } +} diff --git a/tsdb/head_bench_v2_test.go b/tsdb/head_bench_v2_test.go new file mode 100644 index 0000000000..c98fb6613d --- /dev/null +++ b/tsdb/head_bench_v2_test.go @@ -0,0 +1,173 @@ +// 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) {} From a41e1144ddefafbc3de344b9d1c0b7f4b0f237af Mon Sep 17 00:00:00 2001 From: bwplotka Date: Tue, 2 Dec 2025 13:41:18 +0000 Subject: [PATCH 4/6] refactor(appenderV2): 1:1 copy of db_test.go to db_append_v2_test.go (starting point) Signed-off-by: bwplotka --- tsdb/db_append_v2_test.go | 9286 +++++++++++++++++++++++++++++++++++++ 1 file changed, 9286 insertions(+) create mode 100644 tsdb/db_append_v2_test.go diff --git a/tsdb/db_append_v2_test.go b/tsdb/db_append_v2_test.go new file mode 100644 index 0000000000..4e084ef0d8 --- /dev/null +++ b/tsdb/db_append_v2_test.go @@ -0,0 +1,9286 @@ +// Copyright 2017 The Prometheus Authors +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package tsdb + +import ( + "bufio" + "bytes" + "context" + "encoding/binary" + "errors" + "flag" + "fmt" + "hash/crc32" + "log/slog" + "math" + "math/rand" + "net/http" + "net/http/httptest" + "os" + "path" + "path/filepath" + "runtime" + "sort" + "strconv" + "sync" + "testing" + "time" + + "github.com/gogo/protobuf/proto" + "github.com/golang/snappy" + "github.com/oklog/ulid/v2" + "github.com/prometheus/client_golang/prometheus" + prom_testutil "github.com/prometheus/client_golang/prometheus/testutil" + "github.com/prometheus/common/model" + "github.com/prometheus/common/promslog" + "github.com/stretchr/testify/require" + "go.uber.org/atomic" + "go.uber.org/goleak" + + "github.com/prometheus/prometheus/config" + "github.com/prometheus/prometheus/model/histogram" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/model/metadata" + "github.com/prometheus/prometheus/prompb" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/storage/remote" + "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" + "github.com/prometheus/prometheus/tsdb/wlog" + "github.com/prometheus/prometheus/util/annotations" + "github.com/prometheus/prometheus/util/compression" + "github.com/prometheus/prometheus/util/testutil" +) + +func TestMain(m *testing.M) { + var isolationEnabled bool + flag.BoolVar(&isolationEnabled, "test.tsdb-isolation", true, "enable isolation") + flag.Parse() + defaultIsolationDisabled = !isolationEnabled + + goleak.VerifyTestMain(m, + goleak.IgnoreTopFunction("github.com/prometheus/prometheus/tsdb.(*SegmentWAL).cut.func1"), + goleak.IgnoreTopFunction("github.com/prometheus/prometheus/tsdb.(*SegmentWAL).cut.func2"), + goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start")) +} + +type testDBOptions struct { + dir string + opts *Options + rngs []int64 +} +type testDBOpt func(o *testDBOptions) + +func withDir(dir string) testDBOpt { + return func(o *testDBOptions) { + o.dir = dir + } +} + +func withOpts(opts *Options) testDBOpt { + return func(o *testDBOptions) { + o.opts = opts + } +} + +func withRngs(rngs ...int64) testDBOpt { + return func(o *testDBOptions) { + o.rngs = rngs + } +} + +func newTestDB(t testing.TB, opts ...testDBOpt) (db *DB) { + var o testDBOptions + for _, opt := range opts { + opt(&o) + } + if o.opts == nil { + o.opts = DefaultOptions() + } + if o.dir == "" { + o.dir = t.TempDir() + } + + var err error + if len(o.rngs) == 0 { + db, err = Open(o.dir, nil, nil, o.opts, nil) + } else { + o.opts, o.rngs = validateOpts(o.opts, o.rngs) + db, err = open(o.dir, nil, nil, o.opts, o.rngs, nil) + } + require.NoError(t, err) + t.Cleanup(func() { + // Always close. DB is safe for close-after-close. + require.NoError(t, db.Close()) + }) + return db +} + +func TestDBClose_AfterClose(t *testing.T) { + db := newTestDB(t) + require.NoError(t, db.Close()) + require.NoError(t, db.Close()) + + // Double check if we are closing correct DB after reuse. + db = newTestDB(t) + require.NoError(t, db.Close()) + require.NoError(t, db.Close()) +} + +// query runs a matcher query against the querier and fully expands its data. +func query(t testing.TB, q storage.Querier, matchers ...*labels.Matcher) map[string][]chunks.Sample { + ss := q.Select(context.Background(), false, nil, matchers...) + defer func() { + require.NoError(t, q.Close()) + }() + + var it chunkenc.Iterator + result := map[string][]chunks.Sample{} + for ss.Next() { + series := ss.At() + + it = series.Iterator(it) + samples, err := storage.ExpandSamples(it, newSample) + require.NoError(t, err) + require.NoError(t, it.Err()) + + if len(samples) == 0 { + continue + } + + name := series.Labels().String() + result[name] = samples + } + require.NoError(t, ss.Err()) + require.Empty(t, ss.Warnings()) + + return result +} + +// queryAndExpandChunks runs a matcher query against the querier and fully expands its data into samples. +func queryAndExpandChunks(t testing.TB, q storage.ChunkQuerier, matchers ...*labels.Matcher) map[string][][]chunks.Sample { + s := queryChunks(t, q, matchers...) + + res := make(map[string][][]chunks.Sample) + for k, v := range s { + var samples [][]chunks.Sample + for _, chk := range v { + sam, err := storage.ExpandSamples(chk.Chunk.Iterator(nil), nil) + require.NoError(t, err) + samples = append(samples, sam) + } + res[k] = samples + } + + return res +} + +// queryChunks runs a matcher query against the querier and expands its data. +func queryChunks(t testing.TB, q storage.ChunkQuerier, matchers ...*labels.Matcher) map[string][]chunks.Meta { + ss := q.Select(context.Background(), false, nil, matchers...) + defer func() { + require.NoError(t, q.Close()) + }() + + var it chunks.Iterator + result := map[string][]chunks.Meta{} + for ss.Next() { + series := ss.At() + + chks := []chunks.Meta{} + it = series.Iterator(it) + for it.Next() { + chks = append(chks, it.At()) + } + require.NoError(t, it.Err()) + + if len(chks) == 0 { + continue + } + + name := series.Labels().String() + result[name] = chks + } + require.NoError(t, ss.Err()) + require.Empty(t, ss.Warnings()) + return result +} + +// Ensure that blocks are held in memory in their time order +// and not in ULID order as they are read from the directory. +func TestDB_reloadOrder(t *testing.T) { + db := newTestDB(t) + + metas := []BlockMeta{ + {MinTime: 90, MaxTime: 100}, + {MinTime: 70, MaxTime: 80}, + {MinTime: 100, MaxTime: 110}, + } + for _, m := range metas { + createBlock(t, db.Dir(), genSeries(1, 1, m.MinTime, m.MaxTime)) + } + + require.NoError(t, db.reloadBlocks()) + blocks := db.Blocks() + require.Len(t, blocks, 3) + require.Equal(t, metas[1].MinTime, blocks[0].Meta().MinTime) + require.Equal(t, metas[1].MaxTime, blocks[0].Meta().MaxTime) + require.Equal(t, metas[0].MinTime, blocks[1].Meta().MinTime) + require.Equal(t, metas[0].MaxTime, blocks[1].Meta().MaxTime) + require.Equal(t, metas[2].MinTime, blocks[2].Meta().MinTime) + require.Equal(t, metas[2].MaxTime, blocks[2].Meta().MaxTime) +} + +func TestDataAvailableOnlyAfterCommit(t *testing.T) { + db := newTestDB(t) + + ctx := context.Background() + app := db.Appender(ctx) + + _, err := app.Append(0, labels.FromStrings("foo", "bar"), 0, 0) + require.NoError(t, err) + + querier, err := db.Querier(0, 1) + require.NoError(t, err) + seriesSet := query(t, querier, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + require.Equal(t, map[string][]chunks.Sample{}, seriesSet) + + err = app.Commit() + require.NoError(t, err) + + querier, err = db.Querier(0, 1) + require.NoError(t, err) + defer querier.Close() + + seriesSet = query(t, querier, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + + require.Equal(t, map[string][]chunks.Sample{`{foo="bar"}`: {sample{t: 0, f: 0}}}, seriesSet) +} + +// TestNoPanicAfterWALCorruption ensures that querying the db after a WAL corruption doesn't cause a panic. +// https://github.com/prometheus/prometheus/issues/7548 +func TestNoPanicAfterWALCorruption(t *testing.T) { + db := newTestDB(t, withOpts(&Options{WALSegmentSize: 32 * 1024})) + + // Append until the first mmapped head chunk. + // This is to ensure that all samples can be read from the mmapped chunks when the WAL is corrupted. + var expSamples []chunks.Sample + var maxt int64 + ctx := context.Background() + { + // Appending 121 samples because on the 121st a new chunk will be created. + for range 121 { + app := db.Appender(ctx) + _, err := app.Append(0, labels.FromStrings("foo", "bar"), maxt, 0) + expSamples = append(expSamples, sample{t: maxt, f: 0}) + require.NoError(t, err) + require.NoError(t, app.Commit()) + maxt++ + } + require.NoError(t, db.Close()) + } + + // Corrupt the WAL after the first sample of the series so that it has at least one sample and + // it is not garbage collected. + // The repair deletes all WAL records after the corrupted record and these are read from the mmapped chunk. + { + walFiles, err := os.ReadDir(path.Join(db.Dir(), "wal")) + require.NoError(t, err) + f, err := os.OpenFile(path.Join(db.Dir(), "wal", walFiles[0].Name()), os.O_RDWR, 0o666) + require.NoError(t, err) + r := wlog.NewReader(bufio.NewReader(f)) + require.True(t, r.Next(), "reading the series record") + require.True(t, r.Next(), "reading the first sample record") + // Write an invalid record header to corrupt everything after the first wal sample. + _, err = f.WriteAt([]byte{99}, r.Offset()) + require.NoError(t, err) + f.Close() + } + + // Query the data. + { + db := newTestDB(t, withDir(db.Dir())) + require.Equal(t, 1.0, prom_testutil.ToFloat64(db.head.metrics.walCorruptionsTotal), "WAL corruption count mismatch") + + querier, err := db.Querier(0, maxt) + require.NoError(t, err) + seriesSet := query(t, querier, labels.MustNewMatcher(labels.MatchEqual, "", "")) + // The last sample should be missing as it was after the WAL segment corruption. + require.Equal(t, map[string][]chunks.Sample{`{foo="bar"}`: expSamples[0 : len(expSamples)-1]}, seriesSet) + } +} + +func TestDataNotAvailableAfterRollback(t *testing.T) { + db := newTestDB(t) + + app := db.Appender(context.Background()) + _, err := app.Append(0, labels.FromStrings("type", "float"), 0, 0) + require.NoError(t, err) + + _, err = app.AppendHistogram( + 0, labels.FromStrings("type", "histogram"), 0, + &histogram.Histogram{Count: 42, Sum: math.NaN()}, nil, + ) + require.NoError(t, err) + + _, err = app.AppendHistogram( + 0, labels.FromStrings("type", "floathistogram"), 0, + nil, &histogram.FloatHistogram{Count: 42, Sum: math.NaN()}, + ) + require.NoError(t, err) + + err = app.Rollback() + require.NoError(t, err) + + for _, typ := range []string{"float", "histogram", "floathistogram"} { + querier, err := db.Querier(0, 1) + require.NoError(t, err) + seriesSet := query(t, querier, labels.MustNewMatcher(labels.MatchEqual, "type", typ)) + require.Equal(t, map[string][]chunks.Sample{}, seriesSet) + } + + sr, err := wlog.NewSegmentsReader(db.head.wal.Dir()) + require.NoError(t, err) + defer func() { + require.NoError(t, sr.Close()) + }() + + // Read records from WAL and check for expected count of series and samples. + var ( + r = wlog.NewReader(sr) + dec = record.NewDecoder(labels.NewSymbolTable(), promslog.NewNopLogger()) + + walSeriesCount, walSamplesCount, walHistogramCount, walFloatHistogramCount, walExemplarsCount int + ) + for r.Next() { + rec := r.Record() + switch dec.Type(rec) { + case record.Series: + var series []record.RefSeries + series, err = dec.Series(rec, series) + require.NoError(t, err) + walSeriesCount += len(series) + + case record.Samples: + var samples []record.RefSample + samples, err = dec.Samples(rec, samples) + require.NoError(t, err) + walSamplesCount += len(samples) + + case record.Exemplars: + var exemplars []record.RefExemplar + exemplars, err = dec.Exemplars(rec, exemplars) + require.NoError(t, err) + walExemplarsCount += len(exemplars) + + case record.HistogramSamples, record.CustomBucketsHistogramSamples: + var histograms []record.RefHistogramSample + histograms, err = dec.HistogramSamples(rec, histograms) + require.NoError(t, err) + walHistogramCount += len(histograms) + + case record.FloatHistogramSamples, record.CustomBucketsFloatHistogramSamples: + var floatHistograms []record.RefFloatHistogramSample + floatHistograms, err = dec.FloatHistogramSamples(rec, floatHistograms) + require.NoError(t, err) + walFloatHistogramCount += len(floatHistograms) + + default: + } + } + + // Check that only series get stored after calling Rollback. + require.Equal(t, 3, walSeriesCount, "series should have been written to WAL") + require.Equal(t, 0, walSamplesCount, "samples should not have been written to WAL") + require.Equal(t, 0, walExemplarsCount, "exemplars should not have been written to WAL") + require.Equal(t, 0, walHistogramCount, "histograms should not have been written to WAL") + require.Equal(t, 0, walFloatHistogramCount, "float histograms should not have been written to WAL") +} + +func TestDBAppenderAddRef(t *testing.T) { + db := newTestDB(t) + + ctx := context.Background() + app1 := db.Appender(ctx) + + ref1, err := app1.Append(0, labels.FromStrings("a", "b"), 123, 0) + require.NoError(t, err) + + // Reference should already work before commit. + ref2, err := app1.Append(ref1, labels.EmptyLabels(), 124, 1) + require.NoError(t, err) + require.Equal(t, ref1, ref2) + + err = app1.Commit() + require.NoError(t, err) + + app2 := db.Appender(ctx) + + // first ref should already work in next transaction. + ref3, err := app2.Append(ref1, labels.EmptyLabels(), 125, 0) + require.NoError(t, err) + require.Equal(t, ref1, ref3) + + ref4, err := app2.Append(ref1, labels.FromStrings("a", "b"), 133, 1) + require.NoError(t, err) + require.Equal(t, ref1, ref4) + + // Reference must be valid to add another sample. + ref5, err := app2.Append(ref2, labels.EmptyLabels(), 143, 2) + require.NoError(t, err) + require.Equal(t, ref1, ref5) + + // Missing labels & invalid refs should fail. + _, err = app2.Append(9999999, labels.EmptyLabels(), 1, 1) + require.ErrorIs(t, err, ErrInvalidSample) + + require.NoError(t, app2.Commit()) + + q, err := db.Querier(0, 200) + require.NoError(t, err) + + res := query(t, q, labels.MustNewMatcher(labels.MatchEqual, "a", "b")) + + require.Equal(t, map[string][]chunks.Sample{ + labels.FromStrings("a", "b").String(): { + sample{t: 123, f: 0}, + sample{t: 124, f: 1}, + sample{t: 125, f: 0}, + sample{t: 133, f: 1}, + sample{t: 143, f: 2}, + }, + }, res) +} + +func TestAppendEmptyLabelsIgnored(t *testing.T) { + db := newTestDB(t) + + ctx := context.Background() + app1 := db.Appender(ctx) + + ref1, err := app1.Append(0, labels.FromStrings("a", "b"), 123, 0) + require.NoError(t, err) + + // Add with empty label. + ref2, err := app1.Append(0, labels.FromStrings("a", "b", "c", ""), 124, 0) + require.NoError(t, err) + + // Should be the same series. + require.Equal(t, ref1, ref2) + + err = app1.Commit() + require.NoError(t, err) +} + +func TestDeleteSimple(t *testing.T) { + const numSamples int64 = 10 + + cases := []struct { + Intervals tombstones.Intervals + remaint []int64 + }{ + { + Intervals: tombstones.Intervals{{Mint: 0, Maxt: 3}}, + remaint: []int64{4, 5, 6, 7, 8, 9}, + }, + { + Intervals: tombstones.Intervals{{Mint: 1, Maxt: 3}}, + remaint: []int64{0, 4, 5, 6, 7, 8, 9}, + }, + { + Intervals: tombstones.Intervals{{Mint: 1, Maxt: 3}, {Mint: 4, Maxt: 7}}, + remaint: []int64{0, 8, 9}, + }, + { + Intervals: tombstones.Intervals{{Mint: 1, Maxt: 3}, {Mint: 4, Maxt: 700}}, + remaint: []int64{0}, + }, + { // This case is to ensure that labels and symbols are deleted. + Intervals: tombstones.Intervals{{Mint: 0, Maxt: 9}}, + remaint: []int64{}, + }, + } + + for _, c := range cases { + t.Run("", func(t *testing.T) { + db := newTestDB(t) + + ctx := context.Background() + app := db.Appender(ctx) + + smpls := make([]float64, numSamples) + for i := range numSamples { + smpls[i] = rand.Float64() + app.Append(0, labels.FromStrings("a", "b"), i, smpls[i]) + } + + require.NoError(t, app.Commit()) + + // TODO(gouthamve): Reset the tombstones somehow. + // Delete the ranges. + for _, r := range c.Intervals { + require.NoError(t, db.Delete(ctx, r.Mint, r.Maxt, labels.MustNewMatcher(labels.MatchEqual, "a", "b"))) + } + + // Compare the result. + q, err := db.Querier(0, numSamples) + require.NoError(t, err) + + res := q.Select(ctx, false, nil, labels.MustNewMatcher(labels.MatchEqual, "a", "b")) + + expSamples := make([]chunks.Sample, 0, len(c.remaint)) + for _, ts := range c.remaint { + expSamples = append(expSamples, sample{ts, smpls[ts], nil, nil}) + } + + expss := newMockSeriesSet([]storage.Series{ + storage.NewListSeries(labels.FromStrings("a", "b"), expSamples), + }) + + for { + eok, rok := expss.Next(), res.Next() + require.Equal(t, eok, rok) + + if !eok { + require.Empty(t, res.Warnings()) + break + } + sexp := expss.At() + sres := res.At() + + require.Equal(t, sexp.Labels(), sres.Labels()) + + smplExp, errExp := storage.ExpandSamples(sexp.Iterator(nil), nil) + smplRes, errRes := storage.ExpandSamples(sres.Iterator(nil), nil) + + require.Equal(t, errExp, errRes) + require.Equal(t, smplExp, smplRes) + } + }) + } +} + +func TestAmendHistogramDatapointCausesError(t *testing.T) { + db := newTestDB(t) + + ctx := context.Background() + app := db.Appender(ctx) + _, err := app.Append(0, labels.FromStrings("a", "b"), 0, 0) + require.NoError(t, err) + require.NoError(t, app.Commit()) + + app = db.Appender(ctx) + _, err = app.Append(0, labels.FromStrings("a", "b"), 0, 0) + require.NoError(t, err) + _, err = app.Append(0, labels.FromStrings("a", "b"), 0, 1) + require.ErrorIs(t, err, storage.ErrDuplicateSampleForTimestamp) + require.NoError(t, app.Rollback()) + + h := histogram.Histogram{ + Schema: 3, + Count: 52, + Sum: 2.7, + ZeroThreshold: 0.1, + ZeroCount: 42, + PositiveSpans: []histogram.Span{ + {Offset: 0, Length: 4}, + {Offset: 10, Length: 3}, + }, + PositiveBuckets: []int64{1, 2, -2, 1, -1, 0, 0}, + } + fh := h.ToFloat(nil) + + app = db.Appender(ctx) + _, err = app.AppendHistogram(0, labels.FromStrings("a", "c"), 0, h.Copy(), nil) + require.NoError(t, err) + require.NoError(t, app.Commit()) + + app = db.Appender(ctx) + _, err = app.AppendHistogram(0, labels.FromStrings("a", "c"), 0, h.Copy(), nil) + require.NoError(t, err) + h.Schema = 2 + _, err = app.AppendHistogram(0, labels.FromStrings("a", "c"), 0, h.Copy(), nil) + require.Equal(t, storage.ErrDuplicateSampleForTimestamp, err) + require.NoError(t, app.Rollback()) + + // Float histogram. + app = db.Appender(ctx) + _, err = app.AppendHistogram(0, labels.FromStrings("a", "d"), 0, nil, fh.Copy()) + require.NoError(t, err) + require.NoError(t, app.Commit()) + + app = db.Appender(ctx) + _, err = app.AppendHistogram(0, labels.FromStrings("a", "d"), 0, nil, fh.Copy()) + require.NoError(t, err) + fh.Schema = 2 + _, err = app.AppendHistogram(0, labels.FromStrings("a", "d"), 0, nil, fh.Copy()) + require.Equal(t, storage.ErrDuplicateSampleForTimestamp, err) + require.NoError(t, app.Rollback()) +} + +func TestDuplicateNaNDatapointNoAmendError(t *testing.T) { + db := newTestDB(t) + + ctx := context.Background() + app := db.Appender(ctx) + _, err := app.Append(0, labels.FromStrings("a", "b"), 0, math.NaN()) + require.NoError(t, err) + require.NoError(t, app.Commit()) + + app = db.Appender(ctx) + _, err = app.Append(0, labels.FromStrings("a", "b"), 0, math.NaN()) + require.NoError(t, err) +} + +func TestNonDuplicateNaNDatapointsCausesAmendError(t *testing.T) { + db := newTestDB(t) + + ctx := context.Background() + app := db.Appender(ctx) + _, err := app.Append(0, labels.FromStrings("a", "b"), 0, math.Float64frombits(0x7ff0000000000001)) + require.NoError(t, err) + require.NoError(t, app.Commit()) + + app = db.Appender(ctx) + _, err = app.Append(0, labels.FromStrings("a", "b"), 0, math.Float64frombits(0x7ff0000000000002)) + require.ErrorIs(t, err, storage.ErrDuplicateSampleForTimestamp) +} + +func TestEmptyLabelsetCausesError(t *testing.T) { + db := newTestDB(t) + + ctx := context.Background() + app := db.Appender(ctx) + _, err := app.Append(0, labels.Labels{}, 0, 0) + require.Error(t, err) + require.Equal(t, "empty labelset: invalid sample", err.Error()) +} + +func TestSkippingInvalidValuesInSameTxn(t *testing.T) { + db := newTestDB(t) + + // Append AmendedValue. + ctx := context.Background() + app := db.Appender(ctx) + _, err := app.Append(0, labels.FromStrings("a", "b"), 0, 1) + require.NoError(t, err) + _, err = app.Append(0, labels.FromStrings("a", "b"), 0, 2) + require.NoError(t, err) + require.NoError(t, app.Commit()) + + // Make sure the right value is stored. + q, err := db.Querier(0, 10) + require.NoError(t, err) + + ssMap := query(t, q, labels.MustNewMatcher(labels.MatchEqual, "a", "b")) + + require.Equal(t, map[string][]chunks.Sample{ + labels.New(labels.Label{Name: "a", Value: "b"}).String(): {sample{0, 1, nil, nil}}, + }, ssMap) + + // Append Out of Order Value. + app = db.Appender(ctx) + _, err = app.Append(0, labels.FromStrings("a", "b"), 10, 3) + require.NoError(t, err) + _, err = app.Append(0, labels.FromStrings("a", "b"), 7, 5) + require.NoError(t, err) + require.NoError(t, app.Commit()) + + q, err = db.Querier(0, 10) + require.NoError(t, err) + + ssMap = query(t, q, labels.MustNewMatcher(labels.MatchEqual, "a", "b")) + + require.Equal(t, map[string][]chunks.Sample{ + labels.New(labels.Label{Name: "a", Value: "b"}).String(): {sample{0, 1, nil, nil}, sample{10, 3, nil, nil}}, + }, ssMap) +} + +func TestDB_Snapshot(t *testing.T) { + db := newTestDB(t) + + // append data + ctx := context.Background() + app := db.Appender(ctx) + mint := int64(1414141414000) + for i := range 1000 { + _, err := app.Append(0, labels.FromStrings("foo", "bar"), mint+int64(i), 1.0) + require.NoError(t, err) + } + require.NoError(t, app.Commit()) + + // create snapshot + snap := t.TempDir() + require.NoError(t, db.Snapshot(snap, true)) + require.NoError(t, db.Close()) + + // reopen DB from snapshot + db = newTestDB(t, withDir(snap)) + + querier, err := db.Querier(mint, mint+1000) + require.NoError(t, err) + defer func() { require.NoError(t, querier.Close()) }() + + // sum values + seriesSet := querier.Select(context.Background(), false, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + var series chunkenc.Iterator + sum := 0.0 + for seriesSet.Next() { + series = seriesSet.At().Iterator(series) + for series.Next() == chunkenc.ValFloat { + _, v := series.At() + sum += v + } + require.NoError(t, series.Err()) + } + require.NoError(t, seriesSet.Err()) + require.Empty(t, seriesSet.Warnings()) + require.Equal(t, 1000.0, sum) +} + +// TestDB_Snapshot_ChunksOutsideOfCompactedRange ensures that a snapshot removes chunks samples +// that are outside the set block time range. +// See https://github.com/prometheus/prometheus/issues/5105 +func TestDB_Snapshot_ChunksOutsideOfCompactedRange(t *testing.T) { + db := newTestDB(t) + + ctx := context.Background() + app := db.Appender(ctx) + mint := int64(1414141414000) + for i := range 1000 { + _, err := app.Append(0, labels.FromStrings("foo", "bar"), mint+int64(i), 1.0) + require.NoError(t, err) + } + require.NoError(t, app.Commit()) + + snap := t.TempDir() + + // Hackingly introduce "race", by having lower max time then maxTime in last chunk. + db.head.maxTime.Sub(10) + + require.NoError(t, db.Snapshot(snap, true)) + require.NoError(t, db.Close()) + + // reopen DB from snapshot + db = newTestDB(t, withDir(snap)) + + querier, err := db.Querier(mint, mint+1000) + require.NoError(t, err) + defer func() { require.NoError(t, querier.Close()) }() + + // Sum values. + seriesSet := querier.Select(ctx, false, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + var series chunkenc.Iterator + sum := 0.0 + for seriesSet.Next() { + series = seriesSet.At().Iterator(series) + for series.Next() == chunkenc.ValFloat { + _, v := series.At() + sum += v + } + require.NoError(t, series.Err()) + } + require.NoError(t, seriesSet.Err()) + require.Empty(t, seriesSet.Warnings()) + + // Since we snapshotted with MaxTime - 10, so expect 10 less samples. + require.Equal(t, 1000.0-10, sum) +} + +func TestDB_SnapshotWithDelete(t *testing.T) { + const numSamples int64 = 10 + + db := newTestDB(t) + + ctx := context.Background() + app := db.Appender(ctx) + + smpls := make([]float64, numSamples) + for i := range numSamples { + smpls[i] = rand.Float64() + app.Append(0, labels.FromStrings("a", "b"), i, smpls[i]) + } + + require.NoError(t, app.Commit()) + cases := []struct { + intervals tombstones.Intervals + remaint []int64 + }{ + { + intervals: tombstones.Intervals{{Mint: 1, Maxt: 3}, {Mint: 4, Maxt: 7}}, + remaint: []int64{0, 8, 9}, + }, + } + + for _, c := range cases { + t.Run("", func(t *testing.T) { + // TODO(gouthamve): Reset the tombstones somehow. + // Delete the ranges. + for _, r := range c.intervals { + require.NoError(t, db.Delete(ctx, r.Mint, r.Maxt, labels.MustNewMatcher(labels.MatchEqual, "a", "b"))) + } + + // create snapshot + snap := t.TempDir() + + require.NoError(t, db.Snapshot(snap, true)) + + // reopen DB from snapshot + db := newTestDB(t, withDir(snap)) + + // Compare the result. + q, err := db.Querier(0, numSamples) + require.NoError(t, err) + defer func() { require.NoError(t, q.Close()) }() + + res := q.Select(ctx, false, nil, labels.MustNewMatcher(labels.MatchEqual, "a", "b")) + + expSamples := make([]chunks.Sample, 0, len(c.remaint)) + for _, ts := range c.remaint { + expSamples = append(expSamples, sample{ts, smpls[ts], nil, nil}) + } + + expss := newMockSeriesSet([]storage.Series{ + storage.NewListSeries(labels.FromStrings("a", "b"), expSamples), + }) + + if len(expSamples) == 0 { + require.False(t, res.Next()) + return + } + + for { + eok, rok := expss.Next(), res.Next() + require.Equal(t, eok, rok) + + if !eok { + require.Empty(t, res.Warnings()) + break + } + sexp := expss.At() + sres := res.At() + + require.Equal(t, sexp.Labels(), sres.Labels()) + + smplExp, errExp := storage.ExpandSamples(sexp.Iterator(nil), nil) + smplRes, errRes := storage.ExpandSamples(sres.Iterator(nil), nil) + + require.Equal(t, errExp, errRes) + require.Equal(t, smplExp, smplRes) + } + }) + } +} + +func TestDB_e2e(t *testing.T) { + const ( + numDatapoints = 1000 + numRanges = 1000 + timeInterval = int64(3) + ) + // Create 8 series with 1000 data-points of different ranges and run queries. + lbls := [][]labels.Label{ + { + {Name: "a", Value: "b"}, + {Name: "instance", Value: "localhost:9090"}, + {Name: "job", Value: "prometheus"}, + }, + { + {Name: "a", Value: "b"}, + {Name: "instance", Value: "127.0.0.1:9090"}, + {Name: "job", Value: "prometheus"}, + }, + { + {Name: "a", Value: "b"}, + {Name: "instance", Value: "127.0.0.1:9090"}, + {Name: "job", Value: "prom-k8s"}, + }, + { + {Name: "a", Value: "b"}, + {Name: "instance", Value: "localhost:9090"}, + {Name: "job", Value: "prom-k8s"}, + }, + { + {Name: "a", Value: "c"}, + {Name: "instance", Value: "localhost:9090"}, + {Name: "job", Value: "prometheus"}, + }, + { + {Name: "a", Value: "c"}, + {Name: "instance", Value: "127.0.0.1:9090"}, + {Name: "job", Value: "prometheus"}, + }, + { + {Name: "a", Value: "c"}, + {Name: "instance", Value: "127.0.0.1:9090"}, + {Name: "job", Value: "prom-k8s"}, + }, + { + {Name: "a", Value: "c"}, + {Name: "instance", Value: "localhost:9090"}, + {Name: "job", Value: "prom-k8s"}, + }, + } + + seriesMap := map[string][]chunks.Sample{} + for _, l := range lbls { + seriesMap[labels.New(l...).String()] = []chunks.Sample{} + } + + db := newTestDB(t) + + ctx := context.Background() + app := db.Appender(ctx) + + for _, l := range lbls { + lset := labels.New(l...) + series := []chunks.Sample{} + + ts := rand.Int63n(300) + for range numDatapoints { + v := rand.Float64() + + series = append(series, sample{ts, v, nil, nil}) + + _, err := app.Append(0, lset, ts, v) + require.NoError(t, err) + + ts += rand.Int63n(timeInterval) + 1 + } + + seriesMap[lset.String()] = series + } + + require.NoError(t, app.Commit()) + + // Query each selector on 1000 random time-ranges. + queries := []struct { + ms []*labels.Matcher + }{ + { + ms: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "a", "b")}, + }, + { + ms: []*labels.Matcher{ + labels.MustNewMatcher(labels.MatchEqual, "a", "b"), + labels.MustNewMatcher(labels.MatchEqual, "job", "prom-k8s"), + }, + }, + { + ms: []*labels.Matcher{ + labels.MustNewMatcher(labels.MatchEqual, "a", "c"), + labels.MustNewMatcher(labels.MatchEqual, "instance", "localhost:9090"), + labels.MustNewMatcher(labels.MatchEqual, "job", "prometheus"), + }, + }, + // TODO: Add Regexp Matchers. + } + + for _, qry := range queries { + matched := labels.Slice{} + for _, l := range lbls { + s := labels.Selector(qry.ms) + ls := labels.New(l...) + if s.Matches(ls) { + matched = append(matched, ls) + } + } + + sort.Sort(matched) + + for range numRanges { + mint := rand.Int63n(300) + maxt := mint + rand.Int63n(timeInterval*int64(numDatapoints)) + + expected := map[string][]chunks.Sample{} + + // Build the mockSeriesSet. + for _, m := range matched { + smpls := boundedSamples(seriesMap[m.String()], mint, maxt) + if len(smpls) > 0 { + expected[m.String()] = smpls + } + } + + q, err := db.Querier(mint, maxt) + require.NoError(t, err) + + ss := q.Select(ctx, false, nil, qry.ms...) + result := map[string][]chunks.Sample{} + + for ss.Next() { + x := ss.At() + + smpls, err := storage.ExpandSamples(x.Iterator(nil), newSample) + require.NoError(t, err) + + if len(smpls) > 0 { + result[x.Labels().String()] = smpls + } + } + + require.NoError(t, ss.Err()) + require.Empty(t, ss.Warnings()) + require.Equal(t, expected, result) + + q.Close() + } + } +} + +func TestWALFlushedOnDBClose(t *testing.T) { + db := newTestDB(t) + + lbls := labels.FromStrings("labelname", "labelvalue") + + ctx := context.Background() + app := db.Appender(ctx) + _, err := app.Append(0, lbls, 0, 1) + require.NoError(t, err) + require.NoError(t, app.Commit()) + + require.NoError(t, db.Close()) + + db = newTestDB(t, withDir(db.Dir())) + + q, err := db.Querier(0, 1) + require.NoError(t, err) + + values, ws, err := q.LabelValues(ctx, "labelname", nil) + require.NoError(t, err) + require.Empty(t, ws) + require.Equal(t, []string{"labelvalue"}, values) +} + +func TestWALSegmentSizeOptions(t *testing.T) { + tests := map[int]func(dbdir string, segmentSize int){ + // Default Wal Size. + 0: func(dbDir string, _ int) { + filesAndDir, err := os.ReadDir(filepath.Join(dbDir, "wal")) + require.NoError(t, err) + files := []os.FileInfo{} + for _, f := range filesAndDir { + if !f.IsDir() { + fi, err := f.Info() + require.NoError(t, err) + files = append(files, fi) + } + } + // All the full segment files (all but the last) should match the segment size option. + for _, f := range files[:len(files)-1] { + require.Equal(t, int64(DefaultOptions().WALSegmentSize), f.Size(), "WAL file size doesn't match WALSegmentSize option, filename: %v", f.Name()) + } + lastFile := files[len(files)-1] + require.Greater(t, int64(DefaultOptions().WALSegmentSize), lastFile.Size(), "last WAL file size is not smaller than the WALSegmentSize option, filename: %v", lastFile.Name()) + }, + // Custom Wal Size. + 2 * 32 * 1024: func(dbDir string, segmentSize int) { + filesAndDir, err := os.ReadDir(filepath.Join(dbDir, "wal")) + require.NoError(t, err) + files := []os.FileInfo{} + for _, f := range filesAndDir { + if !f.IsDir() { + fi, err := f.Info() + require.NoError(t, err) + files = append(files, fi) + } + } + require.NotEmpty(t, files, "current WALSegmentSize should result in more than a single WAL file.") + // All the full segment files (all but the last) should match the segment size option. + for _, f := range files[:len(files)-1] { + require.Equal(t, int64(segmentSize), f.Size(), "WAL file size doesn't match WALSegmentSize option, filename: %v", f.Name()) + } + lastFile := files[len(files)-1] + require.Greater(t, int64(segmentSize), lastFile.Size(), "last WAL file size is not smaller than the WALSegmentSize option, filename: %v", lastFile.Name()) + }, + // Wal disabled. + -1: func(dbDir string, _ int) { + // Check that WAL dir is not there. + _, err := os.Stat(filepath.Join(dbDir, "wal")) + require.Error(t, err) + // Check that there is chunks dir. + _, err = os.Stat(mmappedChunksDir(dbDir)) + require.NoError(t, err) + }, + } + for segmentSize, testFunc := range tests { + t.Run(fmt.Sprintf("WALSegmentSize %d test", segmentSize), func(t *testing.T) { + opts := DefaultOptions() + opts.WALSegmentSize = segmentSize + db := newTestDB(t, withOpts(opts)) + + for i := range int64(155) { + app := db.Appender(context.Background()) + ref, err := app.Append(0, labels.FromStrings("wal"+strconv.Itoa(int(i)), "size"), i, rand.Float64()) + require.NoError(t, err) + for j := int64(1); j <= 78; j++ { + _, err := app.Append(ref, labels.EmptyLabels(), i+j, rand.Float64()) + require.NoError(t, err) + } + require.NoError(t, app.Commit()) + } + + require.NoError(t, db.Close()) + testFunc(db.Dir(), opts.WALSegmentSize) + }) + } +} + +// https://github.com/prometheus/prometheus/issues/9846 +// https://github.com/prometheus/prometheus/issues/9859 +func TestWALReplayRaceOnSamplesLoggedBeforeSeries(t *testing.T) { + const ( + numRuns = 1 + numSamplesBeforeSeriesCreation = 1000 + ) + + // We test both with few and many samples appended after series creation. If samples are < 120 then there's no + // mmap-ed chunk, otherwise there's at least 1 mmap-ed chunk when replaying the WAL. + for _, numSamplesAfterSeriesCreation := range []int{1, 1000} { + for run := 1; run <= numRuns; run++ { + t.Run(fmt.Sprintf("samples after series creation = %d, run = %d", numSamplesAfterSeriesCreation, run), func(t *testing.T) { + testWALReplayRaceOnSamplesLoggedBeforeSeries(t, numSamplesBeforeSeriesCreation, numSamplesAfterSeriesCreation) + }) + } + } +} + +func testWALReplayRaceOnSamplesLoggedBeforeSeries(t *testing.T, numSamplesBeforeSeriesCreation, numSamplesAfterSeriesCreation int) { + const numSeries = 1000 + + db := newTestDB(t) + db.DisableCompactions() + + for seriesRef := 1; seriesRef <= numSeries; seriesRef++ { + // Log samples before the series is logged to the WAL. + var enc record.Encoder + var samples []record.RefSample + + for ts := range numSamplesBeforeSeriesCreation { + samples = append(samples, record.RefSample{ + Ref: chunks.HeadSeriesRef(uint64(seriesRef)), + T: int64(ts), + V: float64(ts), + }) + } + + err := db.Head().wal.Log(enc.Samples(samples, nil)) + require.NoError(t, err) + + // Add samples via appender so that they're logged after the series in the WAL. + app := db.Appender(context.Background()) + lbls := labels.FromStrings("series_id", strconv.Itoa(seriesRef)) + + for ts := numSamplesBeforeSeriesCreation; ts < numSamplesBeforeSeriesCreation+numSamplesAfterSeriesCreation; ts++ { + _, err := app.Append(0, lbls, int64(ts), float64(ts)) + require.NoError(t, err) + } + require.NoError(t, app.Commit()) + } + + require.NoError(t, db.Close()) + + // Reopen the DB, replaying the WAL. + db = newTestDB(t, withDir(db.Dir())) + + // Query back chunks for all series. + q, err := db.ChunkQuerier(math.MinInt64, math.MaxInt64) + require.NoError(t, err) + + set := q.Select(context.Background(), false, nil, labels.MustNewMatcher(labels.MatchRegexp, "series_id", ".+")) + actualSeries := 0 + var chunksIt chunks.Iterator + + for set.Next() { + actualSeries++ + actualChunks := 0 + + chunksIt = set.At().Iterator(chunksIt) + for chunksIt.Next() { + actualChunks++ + } + require.NoError(t, chunksIt.Err()) + + // We expect 1 chunk every 120 samples after series creation. + require.Equalf(t, (numSamplesAfterSeriesCreation/120)+1, actualChunks, "series: %s", set.At().Labels().String()) + } + + require.NoError(t, set.Err()) + require.Equal(t, numSeries, actualSeries) +} + +func TestTombstoneClean(t *testing.T) { + t.Parallel() + const numSamples int64 = 10 + + db := newTestDB(t) + + ctx := context.Background() + app := db.Appender(ctx) + + smpls := make([]float64, numSamples) + for i := range numSamples { + smpls[i] = rand.Float64() + app.Append(0, labels.FromStrings("a", "b"), i, smpls[i]) + } + + require.NoError(t, app.Commit()) + cases := []struct { + intervals tombstones.Intervals + remaint []int64 + }{ + { + intervals: tombstones.Intervals{{Mint: 1, Maxt: 3}, {Mint: 4, Maxt: 7}}, + remaint: []int64{0, 8, 9}, + }, + } + + for _, c := range cases { + // Delete the ranges. + + // Create snapshot. + snap := t.TempDir() + require.NoError(t, db.Snapshot(snap, true)) + require.NoError(t, db.Close()) + + // Reopen DB from snapshot. + db := newTestDB(t, withDir(snap)) + + for _, r := range c.intervals { + require.NoError(t, db.Delete(ctx, r.Mint, r.Maxt, labels.MustNewMatcher(labels.MatchEqual, "a", "b"))) + } + + // All of the setup for THIS line. + require.NoError(t, db.CleanTombstones()) + + // Compare the result. + q, err := db.Querier(0, numSamples) + require.NoError(t, err) + defer q.Close() + + res := q.Select(ctx, false, nil, labels.MustNewMatcher(labels.MatchEqual, "a", "b")) + + expSamples := make([]chunks.Sample, 0, len(c.remaint)) + for _, ts := range c.remaint { + expSamples = append(expSamples, sample{ts, smpls[ts], nil, nil}) + } + + expss := newMockSeriesSet([]storage.Series{ + storage.NewListSeries(labels.FromStrings("a", "b"), expSamples), + }) + + if len(expSamples) == 0 { + require.False(t, res.Next()) + continue + } + + for { + eok, rok := expss.Next(), res.Next() + require.Equal(t, eok, rok) + + if !eok { + break + } + sexp := expss.At() + sres := res.At() + + require.Equal(t, sexp.Labels(), sres.Labels()) + + smplExp, errExp := storage.ExpandSamples(sexp.Iterator(nil), nil) + smplRes, errRes := storage.ExpandSamples(sres.Iterator(nil), nil) + + require.Equal(t, errExp, errRes) + require.Equal(t, smplExp, smplRes) + } + require.Empty(t, res.Warnings()) + + for _, b := range db.Blocks() { + require.Equal(t, tombstones.NewMemTombstones(), b.tombstones) + } + } +} + +// TestTombstoneCleanResultEmptyBlock tests that a TombstoneClean that results in empty blocks (no timeseries) +// will also delete the resultant block. +func TestTombstoneCleanResultEmptyBlock(t *testing.T) { + t.Parallel() + numSamples := int64(10) + + db := newTestDB(t) + + ctx := context.Background() + app := db.Appender(ctx) + + smpls := make([]float64, numSamples) + for i := range numSamples { + smpls[i] = rand.Float64() + app.Append(0, labels.FromStrings("a", "b"), i, smpls[i]) + } + + require.NoError(t, app.Commit()) + // Interval should cover the whole block. + intervals := tombstones.Intervals{{Mint: 0, Maxt: numSamples}} + + // Create snapshot. + snap := t.TempDir() + require.NoError(t, db.Snapshot(snap, true)) + require.NoError(t, db.Close()) + + // Reopen DB from snapshot. + db = newTestDB(t, withDir(snap)) + + // Create tombstones by deleting all samples. + for _, r := range intervals { + require.NoError(t, db.Delete(ctx, r.Mint, r.Maxt, labels.MustNewMatcher(labels.MatchEqual, "a", "b"))) + } + + require.NoError(t, db.CleanTombstones()) + + // After cleaning tombstones that covers the entire block, no blocks should be left behind. + actualBlockDirs, err := blockDirs(db.Dir()) + require.NoError(t, err) + require.Empty(t, actualBlockDirs) +} + +// TestTombstoneCleanFail tests that a failing TombstoneClean doesn't leave any blocks behind. +// When TombstoneClean errors the original block that should be rebuilt doesn't get deleted so +// if TombstoneClean leaves any blocks behind these will overlap. +func TestTombstoneCleanFail(t *testing.T) { + t.Parallel() + db := newTestDB(t) + + var oldBlockDirs []string + + // Create some blocks pending for compaction. + // totalBlocks should be >=2 so we have enough blocks to trigger compaction failure. + totalBlocks := 2 + for i := range totalBlocks { + blockDir := createBlock(t, db.Dir(), genSeries(1, 1, int64(i), int64(i)+1)) + block, err := OpenBlock(nil, blockDir, nil, nil) + require.NoError(t, err) + // Add some fake tombstones to trigger the compaction. + tomb := tombstones.NewMemTombstones() + tomb.AddInterval(0, tombstones.Interval{Mint: int64(i), Maxt: int64(i) + 1}) + block.tombstones = tomb + + db.blocks = append(db.blocks, block) + oldBlockDirs = append(oldBlockDirs, blockDir) + } + + // Initialize the mockCompactorFailing with a room for a single compaction iteration. + // mockCompactorFailing will fail on the second iteration so we can check if the cleanup works as expected. + db.compactor = &mockCompactorFailing{ + t: t, + blocks: db.blocks, + max: totalBlocks + 1, + } + + // The compactor should trigger a failure here. + require.Error(t, db.CleanTombstones()) + + // Now check that the CleanTombstones replaced the old block even after a failure. + actualBlockDirs, err := blockDirs(db.Dir()) + require.NoError(t, err) + // Only one block should have been replaced by a new block. + require.Len(t, actualBlockDirs, len(oldBlockDirs)) + require.Len(t, intersection(oldBlockDirs, actualBlockDirs), len(actualBlockDirs)-1) +} + +func intersection(oldBlocks, actualBlocks []string) (intersection []string) { + hash := make(map[string]bool) + for _, e := range oldBlocks { + hash[e] = true + } + for _, e := range actualBlocks { + // If block present in the hashmap then append intersection list. + if hash[e] { + intersection = append(intersection, e) + } + } + return intersection +} + +// mockCompactorFailing creates a new empty block on every write and fails when reached the max allowed total. +// For CompactOOO, it always fails. +type mockCompactorFailing struct { + t *testing.T + blocks []*Block + max int +} + +func (*mockCompactorFailing) Plan(string) ([]string, error) { + return nil, nil +} + +func (c *mockCompactorFailing) Write(dest string, _ BlockReader, _, _ int64, _ *BlockMeta) ([]ulid.ULID, error) { + if len(c.blocks) >= c.max { + return []ulid.ULID{}, errors.New("the compactor already did the maximum allowed blocks so it is time to fail") + } + + block, err := OpenBlock(nil, createBlock(c.t, dest, genSeries(1, 1, 0, 1)), nil, nil) + require.NoError(c.t, err) + require.NoError(c.t, block.Close()) // Close block as we won't be using anywhere. + c.blocks = append(c.blocks, block) + + // Now check that all expected blocks are actually persisted on disk. + // This way we make sure that we have some blocks that are supposed to be removed. + var expectedBlocks []string + for _, b := range c.blocks { + expectedBlocks = append(expectedBlocks, filepath.Join(dest, b.Meta().ULID.String())) + } + actualBlockDirs, err := blockDirs(dest) + require.NoError(c.t, err) + + require.Equal(c.t, expectedBlocks, actualBlockDirs) + + return []ulid.ULID{block.Meta().ULID}, nil +} + +func (*mockCompactorFailing) Compact(string, []string, []*Block) ([]ulid.ULID, error) { + return []ulid.ULID{}, nil +} + +func (*mockCompactorFailing) CompactOOO(string, *OOOCompactionHead) (result []ulid.ULID, err error) { + return nil, errors.New("mock compaction failing CompactOOO") +} + +func TestTimeRetention(t *testing.T) { + t.Parallel() + testCases := []struct { + name string + blocks []*BlockMeta + expBlocks []*BlockMeta + retentionDuration int64 + }{ + { + name: "Block max time delta greater than retention duration", + blocks: []*BlockMeta{ + {MinTime: 500, MaxTime: 900}, // Oldest block, beyond retention + {MinTime: 1000, MaxTime: 1500}, + {MinTime: 1500, MaxTime: 2000}, // Newest block + }, + expBlocks: []*BlockMeta{ + {MinTime: 1000, MaxTime: 1500}, + {MinTime: 1500, MaxTime: 2000}, + }, + retentionDuration: 1000, + }, + { + name: "Block max time delta equal to retention duration", + blocks: []*BlockMeta{ + {MinTime: 500, MaxTime: 900}, // Oldest block + {MinTime: 1000, MaxTime: 1500}, // Coinciding exactly with the retention duration. + {MinTime: 1500, MaxTime: 2000}, // Newest block + }, + expBlocks: []*BlockMeta{ + {MinTime: 1500, MaxTime: 2000}, + }, + retentionDuration: 500, + }, + } + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + db := newTestDB(t, withRngs(1000)) + + for _, m := range tc.blocks { + createBlock(t, db.Dir(), genSeries(10, 10, m.MinTime, m.MaxTime)) + } + + require.NoError(t, db.reloadBlocks()) // Reload the db to register the new blocks. + require.Len(t, db.Blocks(), len(tc.blocks)) // Ensure all blocks are registered. + + db.opts.RetentionDuration = tc.retentionDuration + // Reloading should truncate the blocks which are >= the retention duration vs the first block. + require.NoError(t, db.reloadBlocks()) + + actBlocks := db.Blocks() + + require.Equal(t, 1, int(prom_testutil.ToFloat64(db.metrics.timeRetentionCount)), "metric retention count mismatch") + require.Len(t, actBlocks, len(tc.expBlocks)) + for i, eb := range tc.expBlocks { + require.Equal(t, eb.MinTime, actBlocks[i].meta.MinTime) + require.Equal(t, eb.MaxTime, actBlocks[i].meta.MaxTime) + } + }) + } +} + +func TestRetentionDurationMetric(t *testing.T) { + db := newTestDB(t, withOpts(&Options{ + RetentionDuration: 1000, + }), withRngs(100)) + + expRetentionDuration := 1.0 + actRetentionDuration := prom_testutil.ToFloat64(db.metrics.retentionDuration) + require.Equal(t, expRetentionDuration, actRetentionDuration, "metric retention duration mismatch") +} + +func TestSizeRetention(t *testing.T) { + t.Parallel() + opts := DefaultOptions() + opts.OutOfOrderTimeWindow = 100 + db := newTestDB(t, withOpts(opts), withRngs(100)) + + blocks := []*BlockMeta{ + {MinTime: 100, MaxTime: 200}, // Oldest block + {MinTime: 200, MaxTime: 300}, + {MinTime: 300, MaxTime: 400}, + {MinTime: 400, MaxTime: 500}, + {MinTime: 500, MaxTime: 600}, // Newest Block + } + + for _, m := range blocks { + createBlock(t, db.Dir(), genSeries(100, 10, m.MinTime, m.MaxTime)) + } + + headBlocks := []*BlockMeta{ + {MinTime: 700, MaxTime: 800}, + } + + // Add some data to the WAL. + headApp := db.Head().Appender(context.Background()) + var aSeries labels.Labels + var it chunkenc.Iterator + for _, m := range headBlocks { + series := genSeries(100, 10, m.MinTime, m.MaxTime+1) + for _, s := range series { + aSeries = s.Labels() + it = s.Iterator(it) + for it.Next() == chunkenc.ValFloat { + tim, v := it.At() + _, err := headApp.Append(0, s.Labels(), tim, v) + require.NoError(t, err) + } + require.NoError(t, it.Err()) + } + } + require.NoError(t, headApp.Commit()) + db.Head().mmapHeadChunks() + + require.Eventually(t, func() bool { + return db.Head().chunkDiskMapper.IsQueueEmpty() + }, 2*time.Second, 100*time.Millisecond) + + // Test that registered size matches the actual disk size. + require.NoError(t, db.reloadBlocks()) // Reload the db to register the new db size. + require.Len(t, db.Blocks(), len(blocks)) // Ensure all blocks are registered. + blockSize := int64(prom_testutil.ToFloat64(db.metrics.blocksBytes)) // Use the actual internal metrics. + walSize, err := db.Head().wal.Size() + require.NoError(t, err) + cdmSize, err := db.Head().chunkDiskMapper.Size() + require.NoError(t, err) + require.NotZero(t, cdmSize) + // Expected size should take into account block size + WAL size + Head + // chunks size + expSize := blockSize + walSize + cdmSize + actSize, err := fileutil.DirSize(db.Dir()) + require.NoError(t, err) + require.Equal(t, expSize, actSize, "registered size doesn't match actual disk size") + + // Create a WAL checkpoint, and compare sizes. + first, last, err := wlog.Segments(db.Head().wal.Dir()) + require.NoError(t, err) + _, err = wlog.Checkpoint(promslog.NewNopLogger(), db.Head().wal, first, last-1, func(chunks.HeadSeriesRef) bool { return false }, 0) + require.NoError(t, err) + blockSize = int64(prom_testutil.ToFloat64(db.metrics.blocksBytes)) // Use the actual internal metrics. + walSize, err = db.Head().wal.Size() + require.NoError(t, err) + cdmSize, err = db.Head().chunkDiskMapper.Size() + require.NoError(t, err) + require.NotZero(t, cdmSize) + expSize = blockSize + walSize + cdmSize + actSize, err = fileutil.DirSize(db.Dir()) + require.NoError(t, err) + require.Equal(t, expSize, actSize, "registered size doesn't match actual disk size") + + // Truncate Chunk Disk Mapper and compare sizes. + require.NoError(t, db.Head().chunkDiskMapper.Truncate(900)) + cdmSize, err = db.Head().chunkDiskMapper.Size() + require.NoError(t, err) + require.NotZero(t, cdmSize) + expSize = blockSize + walSize + cdmSize + actSize, err = fileutil.DirSize(db.Dir()) + require.NoError(t, err) + require.Equal(t, expSize, actSize, "registered size doesn't match actual disk size") + + // Add some out of order samples to check the size of WBL. + headApp = db.Head().Appender(context.Background()) + for ts := int64(750); ts < 800; ts++ { + _, err := headApp.Append(0, aSeries, ts, float64(ts)) + require.NoError(t, err) + } + require.NoError(t, headApp.Commit()) + + walSize, err = db.Head().wal.Size() + require.NoError(t, err) + wblSize, err := db.Head().wbl.Size() + require.NoError(t, err) + require.NotZero(t, wblSize) + cdmSize, err = db.Head().chunkDiskMapper.Size() + require.NoError(t, err) + expSize = blockSize + walSize + wblSize + cdmSize + actSize, err = fileutil.DirSize(db.Dir()) + require.NoError(t, err) + require.Equal(t, expSize, actSize, "registered size doesn't match actual disk size") + + // Decrease the max bytes limit so that a delete is triggered. + // Check total size, total count and check that the oldest block was deleted. + firstBlockSize := db.Blocks()[0].Size() + sizeLimit := actSize - firstBlockSize + db.opts.MaxBytes = sizeLimit // Set the new db size limit one block smaller that the actual size. + require.NoError(t, db.reloadBlocks()) // Reload the db to register the new db size. + + expBlocks := blocks[1:] + actBlocks := db.Blocks() + blockSize = int64(prom_testutil.ToFloat64(db.metrics.blocksBytes)) + walSize, err = db.Head().wal.Size() + require.NoError(t, err) + cdmSize, err = db.Head().chunkDiskMapper.Size() + require.NoError(t, err) + require.NotZero(t, cdmSize) + // Expected size should take into account block size + WAL size + WBL size + expSize = blockSize + walSize + wblSize + cdmSize + actRetentionCount := int(prom_testutil.ToFloat64(db.metrics.sizeRetentionCount)) + actSize, err = fileutil.DirSize(db.Dir()) + require.NoError(t, err) + + require.Equal(t, 1, actRetentionCount, "metric retention count mismatch") + require.Equal(t, expSize, actSize, "metric db size doesn't match actual disk size") + require.LessOrEqual(t, expSize, sizeLimit, "actual size (%v) is expected to be less than or equal to limit (%v)", expSize, sizeLimit) + require.Len(t, actBlocks, len(blocks)-1, "new block count should be decreased from:%v to:%v", len(blocks), len(blocks)-1) + require.Equal(t, expBlocks[0].MaxTime, actBlocks[0].meta.MaxTime, "maxT mismatch of the first block") + require.Equal(t, expBlocks[len(expBlocks)-1].MaxTime, actBlocks[len(actBlocks)-1].meta.MaxTime, "maxT mismatch of the last block") +} + +func TestSizeRetentionMetric(t *testing.T) { + cases := []struct { + maxBytes int64 + expMaxBytes int64 + }{ + {maxBytes: 1000, expMaxBytes: 1000}, + {maxBytes: 0, expMaxBytes: 0}, + {maxBytes: -1000, expMaxBytes: 0}, + } + + for _, c := range cases { + db := newTestDB(t, withOpts(&Options{ + MaxBytes: c.maxBytes, + }), withRngs(100)) + + actMaxBytes := int64(prom_testutil.ToFloat64(db.metrics.maxBytes)) + require.Equal(t, c.expMaxBytes, actMaxBytes, "metric retention limit bytes mismatch") + } +} + +// TestRuntimeRetentionConfigChange tests that retention configuration can be +// changed at runtime via ApplyConfig and that the retention logic properly +// deletes blocks when retention is shortened. This test also ensures race-free +// concurrent access to retention settings. +func TestRuntimeRetentionConfigChange(t *testing.T) { + const ( + initialRetentionDuration = int64(10 * time.Hour / time.Millisecond) // 10 hours + shorterRetentionDuration = int64(1 * time.Hour / time.Millisecond) // 1 hour + ) + + db := newTestDB(t, withOpts(&Options{ + RetentionDuration: initialRetentionDuration, + }), withRngs(100)) + + nineHoursMs := int64(9 * time.Hour / time.Millisecond) + nineAndHalfHoursMs := int64((9*time.Hour + 30*time.Minute) / time.Millisecond) + blocks := []*BlockMeta{ + {MinTime: 0, MaxTime: 100}, // 10 hours old (beyond new retention) + {MinTime: 100, MaxTime: 200}, // 9.9 hours old (beyond new retention) + {MinTime: nineHoursMs, MaxTime: nineAndHalfHoursMs}, // 1 hour old (within new retention) + {MinTime: nineAndHalfHoursMs, MaxTime: initialRetentionDuration}, // 0.5 hours old (within new retention) + } + + for _, m := range blocks { + createBlock(t, db.Dir(), genSeriesFromSampleGenerator(10, 10, m.MinTime, m.MaxTime, int64(time.Minute/time.Millisecond), func(ts int64) chunks.Sample { + return sample{t: ts, f: rand.Float64()} + })) + } + + // Reload blocks and verify all are loaded. + require.NoError(t, db.reloadBlocks()) + require.Len(t, db.Blocks(), len(blocks), "expected all blocks to be loaded initially") + + cfg := &config.Config{ + StorageConfig: config.StorageConfig{ + TSDBConfig: &config.TSDBConfig{ + Retention: &config.TSDBRetentionConfig{ + Time: model.Duration(shorterRetentionDuration), + }, + }, + }, + } + + require.NoError(t, db.ApplyConfig(cfg), "ApplyConfig should succeed") + + actualRetention := db.getRetentionDuration() + require.Equal(t, shorterRetentionDuration, actualRetention, "retention duration should be updated") + + expectedRetentionSeconds := (time.Duration(shorterRetentionDuration) * time.Millisecond).Seconds() + actualRetentionSeconds := prom_testutil.ToFloat64(db.metrics.retentionDuration) + require.Equal(t, expectedRetentionSeconds, actualRetentionSeconds, "retention duration metric should be updated") + + require.NoError(t, db.reloadBlocks()) + + // Verify that blocks beyond the new retention were deleted. + // We expect only the last 2 blocks to remain (those within 1 hour). + actBlocks := db.Blocks() + require.Len(t, actBlocks, 2, "expected old blocks to be deleted after retention change") + + // Verify the remaining blocks are the newest ones. + require.Equal(t, nineHoursMs, actBlocks[0].meta.MinTime, "first remaining block should be within retention") + require.Equal(t, nineAndHalfHoursMs, actBlocks[1].meta.MinTime, "last remaining block should be the newest") + + require.Positive(t, int(prom_testutil.ToFloat64(db.metrics.timeRetentionCount)), "time retention count should be incremented") +} + +func TestNotMatcherSelectsLabelsUnsetSeries(t *testing.T) { + db := newTestDB(t) + + labelpairs := []labels.Labels{ + labels.FromStrings("a", "abcd", "b", "abcde"), + labels.FromStrings("labelname", "labelvalue"), + } + + ctx := context.Background() + app := db.Appender(ctx) + for _, lbls := range labelpairs { + _, err := app.Append(0, lbls, 0, 1) + require.NoError(t, err) + } + require.NoError(t, app.Commit()) + + cases := []struct { + selector labels.Selector + series []labels.Labels + }{{ + selector: labels.Selector{ + labels.MustNewMatcher(labels.MatchNotEqual, "lname", "lvalue"), + }, + series: labelpairs, + }, { + selector: labels.Selector{ + labels.MustNewMatcher(labels.MatchEqual, "a", "abcd"), + labels.MustNewMatcher(labels.MatchNotEqual, "b", "abcde"), + }, + series: []labels.Labels{}, + }, { + selector: labels.Selector{ + labels.MustNewMatcher(labels.MatchEqual, "a", "abcd"), + labels.MustNewMatcher(labels.MatchNotEqual, "b", "abc"), + }, + series: []labels.Labels{labelpairs[0]}, + }, { + selector: labels.Selector{ + labels.MustNewMatcher(labels.MatchNotRegexp, "a", "abd.*"), + }, + series: labelpairs, + }, { + selector: labels.Selector{ + labels.MustNewMatcher(labels.MatchNotRegexp, "a", "abc.*"), + }, + series: labelpairs[1:], + }, { + selector: labels.Selector{ + labels.MustNewMatcher(labels.MatchNotRegexp, "c", "abd.*"), + }, + series: labelpairs, + }, { + selector: labels.Selector{ + labels.MustNewMatcher(labels.MatchNotRegexp, "labelname", "labelvalue"), + }, + series: labelpairs[:1], + }} + + q, err := db.Querier(0, 10) + require.NoError(t, err) + defer func() { require.NoError(t, q.Close()) }() + + for _, c := range cases { + ss := q.Select(ctx, false, nil, c.selector...) + lres, _, ws, err := expandSeriesSet(ss) + require.NoError(t, err) + require.Empty(t, ws) + require.Equal(t, c.series, lres) + } +} + +// expandSeriesSet returns the raw labels in the order they are retrieved from +// the series set and the samples keyed by Labels().String(). +func expandSeriesSet(ss storage.SeriesSet) ([]labels.Labels, map[string][]sample, annotations.Annotations, error) { + resultLabels := []labels.Labels{} + resultSamples := map[string][]sample{} + var it chunkenc.Iterator + for ss.Next() { + series := ss.At() + samples := []sample{} + it = series.Iterator(it) + for it.Next() == chunkenc.ValFloat { + t, v := it.At() + samples = append(samples, sample{t: t, f: v}) + } + resultLabels = append(resultLabels, series.Labels()) + resultSamples[series.Labels().String()] = samples + } + return resultLabels, resultSamples, ss.Warnings(), ss.Err() +} + +func TestOverlappingBlocksDetectsAllOverlaps(t *testing.T) { + // Create 10 blocks that does not overlap (0-10, 10-20, ..., 100-110) but in reverse order to ensure our algorithm + // will handle that. + metas := make([]BlockMeta, 11) + for i := 10; i >= 0; i-- { + metas[i] = BlockMeta{MinTime: int64(i * 10), MaxTime: int64((i + 1) * 10)} + } + + require.Empty(t, OverlappingBlocks(metas), "we found unexpected overlaps") + + // Add overlapping blocks. We've to establish order again since we aren't interested + // in trivial overlaps caused by unorderedness. + add := func(ms ...BlockMeta) []BlockMeta { + repl := append(append([]BlockMeta{}, metas...), ms...) + sort.Slice(repl, func(i, j int) bool { + return repl[i].MinTime < repl[j].MinTime + }) + return repl + } + + // o1 overlaps with 10-20. + o1 := BlockMeta{MinTime: 15, MaxTime: 17} + require.Equal(t, Overlaps{ + {Min: 15, Max: 17}: {metas[1], o1}, + }, OverlappingBlocks(add(o1))) + + // o2 overlaps with 20-30 and 30-40. + o2 := BlockMeta{MinTime: 21, MaxTime: 31} + require.Equal(t, Overlaps{ + {Min: 21, Max: 30}: {metas[2], o2}, + {Min: 30, Max: 31}: {o2, metas[3]}, + }, OverlappingBlocks(add(o2))) + + // o3a and o3b overlaps with 30-40 and each other. + o3a := BlockMeta{MinTime: 33, MaxTime: 39} + o3b := BlockMeta{MinTime: 34, MaxTime: 36} + require.Equal(t, Overlaps{ + {Min: 34, Max: 36}: {metas[3], o3a, o3b}, + }, OverlappingBlocks(add(o3a, o3b))) + + // o4 is 1:1 overlap with 50-60. + o4 := BlockMeta{MinTime: 50, MaxTime: 60} + require.Equal(t, Overlaps{ + {Min: 50, Max: 60}: {metas[5], o4}, + }, OverlappingBlocks(add(o4))) + + // o5 overlaps with 60-70, 70-80 and 80-90. + o5 := BlockMeta{MinTime: 61, MaxTime: 85} + require.Equal(t, Overlaps{ + {Min: 61, Max: 70}: {metas[6], o5}, + {Min: 70, Max: 80}: {o5, metas[7]}, + {Min: 80, Max: 85}: {o5, metas[8]}, + }, OverlappingBlocks(add(o5))) + + // o6a overlaps with 90-100, 100-110 and o6b, o6b overlaps with 90-100 and o6a. + o6a := BlockMeta{MinTime: 92, MaxTime: 105} + o6b := BlockMeta{MinTime: 94, MaxTime: 99} + require.Equal(t, Overlaps{ + {Min: 94, Max: 99}: {metas[9], o6a, o6b}, + {Min: 100, Max: 105}: {o6a, metas[10]}, + }, OverlappingBlocks(add(o6a, o6b))) + + // All together. + require.Equal(t, Overlaps{ + {Min: 15, Max: 17}: {metas[1], o1}, + {Min: 21, Max: 30}: {metas[2], o2}, {Min: 30, Max: 31}: {o2, metas[3]}, + {Min: 34, Max: 36}: {metas[3], o3a, o3b}, + {Min: 50, Max: 60}: {metas[5], o4}, + {Min: 61, Max: 70}: {metas[6], o5}, {Min: 70, Max: 80}: {o5, metas[7]}, {Min: 80, Max: 85}: {o5, metas[8]}, + {Min: 94, Max: 99}: {metas[9], o6a, o6b}, {Min: 100, Max: 105}: {o6a, metas[10]}, + }, OverlappingBlocks(add(o1, o2, o3a, o3b, o4, o5, o6a, o6b))) + + // Additional case. + var nc1 []BlockMeta + nc1 = append(nc1, BlockMeta{MinTime: 1, MaxTime: 5}) + nc1 = append(nc1, BlockMeta{MinTime: 2, MaxTime: 3}) + nc1 = append(nc1, BlockMeta{MinTime: 2, MaxTime: 3}) + nc1 = append(nc1, BlockMeta{MinTime: 2, MaxTime: 3}) + nc1 = append(nc1, BlockMeta{MinTime: 2, MaxTime: 3}) + nc1 = append(nc1, BlockMeta{MinTime: 2, MaxTime: 6}) + nc1 = append(nc1, BlockMeta{MinTime: 3, MaxTime: 5}) + nc1 = append(nc1, BlockMeta{MinTime: 5, MaxTime: 7}) + nc1 = append(nc1, BlockMeta{MinTime: 7, MaxTime: 10}) + nc1 = append(nc1, BlockMeta{MinTime: 8, MaxTime: 9}) + require.Equal(t, Overlaps{ + {Min: 2, Max: 3}: {nc1[0], nc1[1], nc1[2], nc1[3], nc1[4], nc1[5]}, // 1-5, 2-3, 2-3, 2-3, 2-3, 2,6 + {Min: 3, Max: 5}: {nc1[0], nc1[5], nc1[6]}, // 1-5, 2-6, 3-5 + {Min: 5, Max: 6}: {nc1[5], nc1[7]}, // 2-6, 5-7 + {Min: 8, Max: 9}: {nc1[8], nc1[9]}, // 7-10, 8-9 + }, OverlappingBlocks(nc1)) +} + +// Regression test for https://github.com/prometheus/tsdb/issues/347 +func TestChunkAtBlockBoundary(t *testing.T) { + t.Parallel() + db := newTestDB(t) + + ctx := context.Background() + app := db.Appender(ctx) + + blockRange := db.compactor.(*LeveledCompactor).ranges[0] + label := labels.FromStrings("foo", "bar") + + for i := range int64(3) { + _, err := app.Append(0, label, i*blockRange, 0) + require.NoError(t, err) + _, err = app.Append(0, label, i*blockRange+1000, 0) + require.NoError(t, err) + } + + err := app.Commit() + require.NoError(t, err) + + err = db.Compact(ctx) + require.NoError(t, err) + + var builder labels.ScratchBuilder + + for _, block := range db.Blocks() { + r, err := block.Index() + require.NoError(t, err) + defer r.Close() + + meta := block.Meta() + + k, v := index.AllPostingsKey() + p, err := r.Postings(ctx, k, v) + require.NoError(t, err) + + var chks []chunks.Meta + + chunkCount := 0 + + for p.Next() { + err = r.Series(p.At(), &builder, &chks) + require.NoError(t, err) + for _, c := range chks { + require.True(t, meta.MinTime <= c.MinTime && c.MaxTime <= meta.MaxTime, + "chunk spans beyond block boundaries: [block.MinTime=%d, block.MaxTime=%d]; [chunk.MinTime=%d, chunk.MaxTime=%d]", + meta.MinTime, meta.MaxTime, c.MinTime, c.MaxTime) + chunkCount++ + } + } + require.Equal(t, 1, chunkCount, "expected 1 chunk in block %s, got %d", meta.ULID, chunkCount) + } +} + +func TestQuerierWithBoundaryChunks(t *testing.T) { + t.Parallel() + db := newTestDB(t) + + ctx := context.Background() + app := db.Appender(ctx) + + blockRange := db.compactor.(*LeveledCompactor).ranges[0] + label := labels.FromStrings("foo", "bar") + + for i := range int64(5) { + _, err := app.Append(0, label, i*blockRange, 0) + require.NoError(t, err) + _, err = app.Append(0, labels.FromStrings("blockID", strconv.FormatInt(i, 10)), i*blockRange, 0) + require.NoError(t, err) + } + + err := app.Commit() + require.NoError(t, err) + + err = db.Compact(ctx) + require.NoError(t, err) + + require.GreaterOrEqual(t, len(db.blocks), 3, "invalid test, less than three blocks in DB") + + q, err := db.Querier(blockRange, 2*blockRange) + require.NoError(t, err) + defer q.Close() + + // The requested interval covers 2 blocks, so the querier's label values for blockID should give us 2 values, one from each block. + b, ws, err := q.LabelValues(ctx, "blockID", nil) + require.NoError(t, err) + var nilAnnotations annotations.Annotations + require.Equal(t, nilAnnotations, ws) + require.Equal(t, []string{"1", "2"}, b) +} + +// TestInitializeHeadTimestamp ensures that the h.minTime is set properly. +// - no blocks no WAL: set to the time of the first appended sample +// - no blocks with WAL: set to the smallest sample from the WAL +// - with blocks no WAL: set to the last block maxT +// - with blocks with WAL: same as above +func TestInitializeHeadTimestamp(t *testing.T) { + t.Parallel() + t.Run("clean", func(t *testing.T) { + db := newTestDB(t) + + // Should be set to init values if no WAL or blocks exist so far. + require.Equal(t, int64(math.MaxInt64), db.head.MinTime()) + require.Equal(t, int64(math.MinInt64), db.head.MaxTime()) + require.False(t, db.head.initialized()) + + // First added sample initializes the writable range. + ctx := context.Background() + app := db.Appender(ctx) + _, err := app.Append(0, labels.FromStrings("a", "b"), 1000, 1) + require.NoError(t, err) + + require.Equal(t, int64(1000), db.head.MinTime()) + require.Equal(t, int64(1000), db.head.MaxTime()) + require.True(t, db.head.initialized()) + }) + t.Run("wal-only", func(t *testing.T) { + dir := t.TempDir() + + require.NoError(t, os.MkdirAll(path.Join(dir, "wal"), 0o777)) + w, err := wlog.New(nil, nil, path.Join(dir, "wal"), compression.None) + require.NoError(t, err) + + var enc record.Encoder + err = w.Log( + enc.Series([]record.RefSeries{ + {Ref: 123, Labels: labels.FromStrings("a", "1")}, + {Ref: 124, Labels: labels.FromStrings("a", "2")}, + }, nil), + enc.Samples([]record.RefSample{ + {Ref: 123, T: 5000, V: 1}, + {Ref: 124, T: 15000, V: 1}, + }, nil), + ) + require.NoError(t, err) + require.NoError(t, w.Close()) + + db := newTestDB(t, withDir(dir)) + + require.Equal(t, int64(5000), db.head.MinTime()) + require.Equal(t, int64(15000), db.head.MaxTime()) + require.True(t, db.head.initialized()) + }) + t.Run("existing-block", func(t *testing.T) { + dir := t.TempDir() + + createBlock(t, dir, genSeries(1, 1, 1000, 2000)) + + db := newTestDB(t, withDir(dir)) + + require.Equal(t, int64(2000), db.head.MinTime()) + require.Equal(t, int64(2000), db.head.MaxTime()) + require.True(t, db.head.initialized()) + }) + t.Run("existing-block-and-wal", func(t *testing.T) { + dir := t.TempDir() + + createBlock(t, dir, genSeries(1, 1, 1000, 6000)) + + require.NoError(t, os.MkdirAll(path.Join(dir, "wal"), 0o777)) + w, err := wlog.New(nil, nil, path.Join(dir, "wal"), compression.None) + require.NoError(t, err) + + var enc record.Encoder + err = w.Log( + enc.Series([]record.RefSeries{ + {Ref: 123, Labels: labels.FromStrings("a", "1")}, + {Ref: 124, Labels: labels.FromStrings("a", "2")}, + }, nil), + enc.Samples([]record.RefSample{ + {Ref: 123, T: 5000, V: 1}, + {Ref: 124, T: 15000, V: 1}, + }, nil), + ) + require.NoError(t, err) + require.NoError(t, w.Close()) + + db := newTestDB(t, withDir(dir)) + + require.Equal(t, int64(6000), db.head.MinTime()) + require.Equal(t, int64(15000), db.head.MaxTime()) + require.True(t, db.head.initialized()) + // Check that old series has been GCed. + require.Equal(t, 1.0, prom_testutil.ToFloat64(db.head.metrics.series)) + }) +} + +func TestNoEmptyBlocks(t *testing.T) { + t.Parallel() + db := newTestDB(t, withRngs(100)) + ctx := context.Background() + + db.DisableCompactions() + + rangeToTriggerCompaction := db.compactor.(*LeveledCompactor).ranges[0]/2*3 - 1 + defaultLabel := labels.FromStrings("foo", "bar") + defaultMatcher := labels.MustNewMatcher(labels.MatchRegexp, "", ".*") + + t.Run("Test no blocks after compact with empty head.", func(t *testing.T) { + require.NoError(t, db.Compact(ctx)) + actBlocks, err := blockDirs(db.Dir()) + require.NoError(t, err) + require.Len(t, actBlocks, len(db.Blocks())) + require.Empty(t, actBlocks) + require.Equal(t, 0, int(prom_testutil.ToFloat64(db.compactor.(*LeveledCompactor).metrics.Ran)), "no compaction should be triggered here") + }) + + t.Run("Test no blocks after deleting all samples from head.", func(t *testing.T) { + app := db.Appender(ctx) + _, err := app.Append(0, defaultLabel, 1, 0) + require.NoError(t, err) + _, err = app.Append(0, defaultLabel, 2, 0) + require.NoError(t, err) + _, err = app.Append(0, defaultLabel, 3+rangeToTriggerCompaction, 0) + require.NoError(t, err) + require.NoError(t, app.Commit()) + require.NoError(t, db.Delete(ctx, math.MinInt64, math.MaxInt64, defaultMatcher)) + require.NoError(t, db.Compact(ctx)) + require.Equal(t, 1, int(prom_testutil.ToFloat64(db.compactor.(*LeveledCompactor).metrics.Ran)), "compaction should have been triggered here") + + actBlocks, err := blockDirs(db.Dir()) + require.NoError(t, err) + require.Len(t, actBlocks, len(db.Blocks())) + require.Empty(t, actBlocks) + + app = db.Appender(ctx) + _, err = app.Append(0, defaultLabel, 1, 0) + require.Equal(t, storage.ErrOutOfBounds, err, "the head should be truncated so no samples in the past should be allowed") + + // Adding new blocks. + currentTime := db.Head().MaxTime() + _, err = app.Append(0, defaultLabel, currentTime, 0) + require.NoError(t, err) + _, err = app.Append(0, defaultLabel, currentTime+1, 0) + require.NoError(t, err) + _, err = app.Append(0, defaultLabel, currentTime+rangeToTriggerCompaction, 0) + require.NoError(t, err) + require.NoError(t, app.Commit()) + + require.NoError(t, db.Compact(ctx)) + require.Equal(t, 2, int(prom_testutil.ToFloat64(db.compactor.(*LeveledCompactor).metrics.Ran)), "compaction should have been triggered here") + actBlocks, err = blockDirs(db.Dir()) + require.NoError(t, err) + require.Len(t, actBlocks, len(db.Blocks())) + require.Len(t, actBlocks, 1, "No blocks created when compacting with >0 samples") + }) + + t.Run(`When no new block is created from head, and there are some blocks on disk + compaction should not run into infinite loop (was seen during development).`, func(t *testing.T) { + oldBlocks := db.Blocks() + app := db.Appender(ctx) + currentTime := db.Head().MaxTime() + _, err := app.Append(0, defaultLabel, currentTime, 0) + require.NoError(t, err) + _, err = app.Append(0, defaultLabel, currentTime+1, 0) + require.NoError(t, err) + _, err = app.Append(0, defaultLabel, currentTime+rangeToTriggerCompaction, 0) + require.NoError(t, err) + require.NoError(t, app.Commit()) + require.NoError(t, db.head.Delete(ctx, math.MinInt64, math.MaxInt64, defaultMatcher)) + require.NoError(t, db.Compact(ctx)) + require.Equal(t, 3, int(prom_testutil.ToFloat64(db.compactor.(*LeveledCompactor).metrics.Ran)), "compaction should have been triggered here") + require.Equal(t, oldBlocks, db.Blocks()) + }) + + t.Run("Test no blocks remaining after deleting all samples from disk.", func(t *testing.T) { + currentTime := db.Head().MaxTime() + blocks := []*BlockMeta{ + {MinTime: currentTime, MaxTime: currentTime + db.compactor.(*LeveledCompactor).ranges[0]}, + {MinTime: currentTime + 100, MaxTime: currentTime + 100 + db.compactor.(*LeveledCompactor).ranges[0]}, + } + for _, m := range blocks { + createBlock(t, db.Dir(), genSeries(2, 2, m.MinTime, m.MaxTime)) + } + + oldBlocks := db.Blocks() + require.NoError(t, db.reloadBlocks()) // Reload the db to register the new blocks. + require.Len(t, db.Blocks(), len(blocks)+len(oldBlocks)) // Ensure all blocks are registered. + require.NoError(t, db.Delete(ctx, math.MinInt64, math.MaxInt64, defaultMatcher)) + require.NoError(t, db.Compact(ctx)) + require.Equal(t, 5, int(prom_testutil.ToFloat64(db.compactor.(*LeveledCompactor).metrics.Ran)), "compaction should have been triggered here once for each block that have tombstones") + + actBlocks, err := blockDirs(db.Dir()) + require.NoError(t, err) + require.Len(t, actBlocks, len(db.Blocks())) + require.Len(t, actBlocks, 1, "All samples are deleted. Only the most recent block should remain after compaction.") + }) +} + +func TestDB_LabelNames(t *testing.T) { + ctx := context.Background() + tests := []struct { + // Add 'sampleLabels1' -> Test Head -> Compact -> Test Disk -> + // -> Add 'sampleLabels2' -> Test Head+Disk + + sampleLabels1 [][2]string // For checking head and disk separately. + // To test Head+Disk, sampleLabels2 should have + // at least 1 unique label name which is not in sampleLabels1. + sampleLabels2 [][2]string // For checking head and disk together. + exp1 []string // after adding sampleLabels1. + exp2 []string // after adding sampleLabels1 and sampleLabels2. + }{ + { + sampleLabels1: [][2]string{ + {"name1", "1"}, + {"name3", "3"}, + {"name2", "2"}, + }, + sampleLabels2: [][2]string{ + {"name4", "4"}, + {"name1", "1"}, + }, + exp1: []string{"name1", "name2", "name3"}, + exp2: []string{"name1", "name2", "name3", "name4"}, + }, + { + sampleLabels1: [][2]string{ + {"name2", "2"}, + {"name1", "1"}, + {"name2", "2"}, + }, + sampleLabels2: [][2]string{ + {"name6", "6"}, + {"name0", "0"}, + }, + exp1: []string{"name1", "name2"}, + exp2: []string{"name0", "name1", "name2", "name6"}, + }, + } + + blockRange := int64(1000) + // Appends samples into the database. + appendSamples := func(db *DB, mint, maxt int64, sampleLabels [][2]string) { + t.Helper() + app := db.Appender(ctx) + for i := mint; i <= maxt; i++ { + for _, tuple := range sampleLabels { + label := labels.FromStrings(tuple[0], tuple[1]) + _, err := app.Append(0, label, i*blockRange, 0) + require.NoError(t, err) + } + } + err := app.Commit() + require.NoError(t, err) + } + for _, tst := range tests { + t.Run("", func(t *testing.T) { + ctx := context.Background() + db := newTestDB(t) + + appendSamples(db, 0, 4, tst.sampleLabels1) + + // Testing head. + headIndexr, err := db.head.Index() + require.NoError(t, err) + labelNames, err := headIndexr.LabelNames(ctx) + require.NoError(t, err) + require.Equal(t, tst.exp1, labelNames) + require.NoError(t, headIndexr.Close()) + + // Testing disk. + err = db.Compact(ctx) + require.NoError(t, err) + // All blocks have same label names, hence check them individually. + // No need to aggregate and check. + for _, b := range db.Blocks() { + blockIndexr, err := b.Index() + require.NoError(t, err) + labelNames, err = blockIndexr.LabelNames(ctx) + require.NoError(t, err) + require.Equal(t, tst.exp1, labelNames) + require.NoError(t, blockIndexr.Close()) + } + + // Adding more samples to head with new label names + // so that we can test (head+disk).LabelNames(ctx) (the union). + appendSamples(db, 5, 9, tst.sampleLabels2) + + // Testing DB (union). + q, err := db.Querier(math.MinInt64, math.MaxInt64) + require.NoError(t, err) + var ws annotations.Annotations + labelNames, ws, err = q.LabelNames(ctx, nil) + require.NoError(t, err) + require.Empty(t, ws) + require.NoError(t, q.Close()) + require.Equal(t, tst.exp2, labelNames) + }) + } +} + +func TestCorrectNumTombstones(t *testing.T) { + t.Parallel() + db := newTestDB(t) + + blockRange := db.compactor.(*LeveledCompactor).ranges[0] + name, value := "foo", "bar" + defaultLabel := labels.FromStrings(name, value) + defaultMatcher := labels.MustNewMatcher(labels.MatchEqual, name, value) + + ctx := context.Background() + app := db.Appender(ctx) + for i := range int64(3) { + for j := range int64(15) { + _, err := app.Append(0, defaultLabel, i*blockRange+j, 0) + require.NoError(t, err) + } + } + require.NoError(t, app.Commit()) + + err := db.Compact(ctx) + require.NoError(t, err) + require.Len(t, db.blocks, 1) + + require.NoError(t, db.Delete(ctx, 0, 1, defaultMatcher)) + require.Equal(t, uint64(1), db.blocks[0].meta.Stats.NumTombstones) + + // {0, 1} and {2, 3} are merged to form 1 tombstone. + require.NoError(t, db.Delete(ctx, 2, 3, defaultMatcher)) + require.Equal(t, uint64(1), db.blocks[0].meta.Stats.NumTombstones) + + require.NoError(t, db.Delete(ctx, 5, 6, defaultMatcher)) + require.Equal(t, uint64(2), db.blocks[0].meta.Stats.NumTombstones) + + require.NoError(t, db.Delete(ctx, 9, 11, defaultMatcher)) + require.Equal(t, uint64(3), db.blocks[0].meta.Stats.NumTombstones) +} + +// TestBlockRanges checks the following use cases: +// - No samples can be added with timestamps lower than the last block maxt. +// - The compactor doesn't create overlapping blocks +// +// even when the last blocks is not within the default boundaries. +// - Lower boundary is based on the smallest sample in the head and +// +// upper boundary is rounded to the configured block range. +// +// This ensures that a snapshot that includes the head and creates a block with a custom time range +// will not overlap with the first block created by the next compaction. +func TestBlockRanges(t *testing.T) { + t.Parallel() + logger := promslog.New(&promslog.Config{}) + ctx := context.Background() + + dir := t.TempDir() + + // Test that the compactor doesn't create overlapping blocks + // when a non standard block already exists. + firstBlockMaxT := int64(3) + createBlock(t, dir, genSeries(1, 1, 0, firstBlockMaxT)) + db, err := open(dir, logger, nil, DefaultOptions(), []int64{10000}, nil) + require.NoError(t, err) + + rangeToTriggerCompaction := db.compactor.(*LeveledCompactor).ranges[0]/2*3 + 1 + + app := db.Appender(ctx) + lbl := labels.FromStrings("a", "b") + _, err = app.Append(0, lbl, firstBlockMaxT-1, rand.Float64()) + require.Error(t, err, "appending a sample with a timestamp covered by a previous block shouldn't be possible") + _, err = app.Append(0, lbl, firstBlockMaxT+1, rand.Float64()) + require.NoError(t, err) + _, err = app.Append(0, lbl, firstBlockMaxT+2, rand.Float64()) + require.NoError(t, err) + secondBlockMaxt := firstBlockMaxT + rangeToTriggerCompaction + _, err = app.Append(0, lbl, secondBlockMaxt, rand.Float64()) // Add samples to trigger a new compaction + + require.NoError(t, err) + require.NoError(t, app.Commit()) + for range 100 { + if len(db.Blocks()) == 2 { + break + } + time.Sleep(100 * time.Millisecond) + } + require.Len(t, db.Blocks(), 2, "no new block created after the set timeout") + + require.LessOrEqual(t, db.Blocks()[1].Meta().MinTime, db.Blocks()[0].Meta().MaxTime, + "new block overlaps old:%v,new:%v", db.Blocks()[0].Meta(), db.Blocks()[1].Meta()) + + // Test that wal records are skipped when an existing block covers the same time ranges + // and compaction doesn't create an overlapping block. + app = db.Appender(ctx) + db.DisableCompactions() + _, err = app.Append(0, lbl, secondBlockMaxt+1, rand.Float64()) + require.NoError(t, err) + _, err = app.Append(0, lbl, secondBlockMaxt+2, rand.Float64()) + require.NoError(t, err) + _, err = app.Append(0, lbl, secondBlockMaxt+3, rand.Float64()) + require.NoError(t, err) + _, err = app.Append(0, lbl, secondBlockMaxt+4, rand.Float64()) + require.NoError(t, err) + require.NoError(t, app.Commit()) + require.NoError(t, db.Close()) + + thirdBlockMaxt := secondBlockMaxt + 2 + createBlock(t, dir, genSeries(1, 1, secondBlockMaxt+1, thirdBlockMaxt)) + + db, err = open(dir, logger, nil, DefaultOptions(), []int64{10000}, nil) + require.NoError(t, err) + + defer db.Close() + require.Len(t, db.Blocks(), 3, "db doesn't include expected number of blocks") + require.Equal(t, db.Blocks()[2].Meta().MaxTime, thirdBlockMaxt, "unexpected maxt of the last block") + + app = db.Appender(ctx) + _, err = app.Append(0, lbl, thirdBlockMaxt+rangeToTriggerCompaction, rand.Float64()) // Trigger a compaction + require.NoError(t, err) + require.NoError(t, app.Commit()) + for range 100 { + if len(db.Blocks()) == 4 { + break + } + time.Sleep(100 * time.Millisecond) + } + + require.Len(t, db.Blocks(), 4, "no new block created after the set timeout") + + require.LessOrEqual(t, db.Blocks()[3].Meta().MinTime, db.Blocks()[2].Meta().MaxTime, + "new block overlaps old:%v,new:%v", db.Blocks()[2].Meta(), db.Blocks()[3].Meta()) +} + +// TestDBReadOnly ensures that opening a DB in readonly mode doesn't modify any files on the disk. +// It also checks that the API calls return equivalent results as a normal db.Open() mode. +func TestDBReadOnly(t *testing.T) { + t.Parallel() + var ( + dbDir = t.TempDir() + expBlocks []*Block + expBlock *Block + expSeries map[string][]chunks.Sample + expChunks map[string][][]chunks.Sample + expDBHash []byte + matchAll = labels.MustNewMatcher(labels.MatchEqual, "", "") + err error + ) + + // Bootstrap the db. + { + dbBlocks := []*BlockMeta{ + // Create three 2-sample blocks. + {MinTime: 10, MaxTime: 12}, + {MinTime: 12, MaxTime: 14}, + {MinTime: 14, MaxTime: 16}, + } + + for _, m := range dbBlocks { + _ = createBlock(t, dbDir, genSeries(1, 1, m.MinTime, m.MaxTime)) + } + + // Add head to test DBReadOnly WAL reading capabilities. + w, err := wlog.New(nil, nil, filepath.Join(dbDir, "wal"), compression.Snappy) + require.NoError(t, err) + h := createHead(t, w, genSeries(1, 1, 16, 18), dbDir) + require.NoError(t, h.Close()) + } + + // Open a normal db to use for a comparison. + { + dbWritable := newTestDB(t, withDir(dbDir)) + dbWritable.DisableCompactions() + + dbSizeBeforeAppend, err := fileutil.DirSize(dbWritable.Dir()) + require.NoError(t, err) + app := dbWritable.Appender(context.Background()) + _, err = app.Append(0, labels.FromStrings("foo", "bar"), dbWritable.Head().MaxTime()+1, 0) + require.NoError(t, err) + require.NoError(t, app.Commit()) + + expBlocks = dbWritable.Blocks() + expBlock = expBlocks[0] + expDbSize, err := fileutil.DirSize(dbWritable.Dir()) + require.NoError(t, err) + require.Greater(t, expDbSize, dbSizeBeforeAppend, "db size didn't increase after an append") + + q, err := dbWritable.Querier(math.MinInt64, math.MaxInt64) + require.NoError(t, err) + expSeries = query(t, q, matchAll) + cq, err := dbWritable.ChunkQuerier(math.MinInt64, math.MaxInt64) + require.NoError(t, err) + expChunks = queryAndExpandChunks(t, cq, matchAll) + + require.NoError(t, dbWritable.Close()) // Close here to allow getting the dir hash for windows. + expDBHash = testutil.DirHash(t, dbWritable.Dir()) + } + + // Open a read only db and ensure that the API returns the same result as the normal DB. + dbReadOnly, err := OpenDBReadOnly(dbDir, "", nil) + require.NoError(t, err) + defer func() { require.NoError(t, dbReadOnly.Close()) }() + + t.Run("blocks", func(t *testing.T) { + blocks, err := dbReadOnly.Blocks() + require.NoError(t, err) + require.Len(t, blocks, len(expBlocks)) + for i, expBlock := range expBlocks { + require.Equal(t, expBlock.Meta(), blocks[i].Meta(), "block meta mismatch") + } + }) + t.Run("block", func(t *testing.T) { + blockID := expBlock.meta.ULID.String() + block, err := dbReadOnly.Block(blockID, nil) + require.NoError(t, err) + require.Equal(t, expBlock.Meta(), block.Meta(), "block meta mismatch") + }) + t.Run("invalid block ID", func(t *testing.T) { + blockID := "01GTDVZZF52NSWB5SXQF0P2PGF" + _, err := dbReadOnly.Block(blockID, nil) + require.Error(t, err) + }) + t.Run("last block ID", func(t *testing.T) { + blockID, err := dbReadOnly.LastBlockID() + require.NoError(t, err) + require.Equal(t, expBlocks[2].Meta().ULID.String(), blockID) + }) + t.Run("querier", func(t *testing.T) { + // Open a read only db and ensure that the API returns the same result as the normal DB. + q, err := dbReadOnly.Querier(math.MinInt64, math.MaxInt64) + require.NoError(t, err) + readOnlySeries := query(t, q, matchAll) + readOnlyDBHash := testutil.DirHash(t, dbDir) + + require.Len(t, readOnlySeries, len(expSeries), "total series mismatch") + require.Equal(t, expSeries, readOnlySeries, "series mismatch") + require.Equal(t, expDBHash, readOnlyDBHash, "after all read operations the db hash should remain the same") + }) + t.Run("chunk querier", func(t *testing.T) { + cq, err := dbReadOnly.ChunkQuerier(math.MinInt64, math.MaxInt64) + require.NoError(t, err) + readOnlySeries := queryAndExpandChunks(t, cq, matchAll) + readOnlyDBHash := testutil.DirHash(t, dbDir) + + require.Len(t, readOnlySeries, len(expChunks), "total series mismatch") + require.Equal(t, expChunks, readOnlySeries, "series chunks mismatch") + require.Equal(t, expDBHash, readOnlyDBHash, "after all read operations the db hash should remain the same") + }) +} + +// TestDBReadOnlyClosing ensures that after closing the db +// all api methods return an ErrClosed. +func TestDBReadOnlyClosing(t *testing.T) { + t.Parallel() + sandboxDir := t.TempDir() + db, err := OpenDBReadOnly(t.TempDir(), sandboxDir, promslog.New(&promslog.Config{})) + require.NoError(t, err) + // The sandboxDir was there. + require.DirExists(t, db.sandboxDir) + require.NoError(t, db.Close()) + // The sandboxDir was deleted when closing. + require.NoDirExists(t, db.sandboxDir) + require.Equal(t, db.Close(), ErrClosed) + _, err = db.Blocks() + require.Equal(t, err, ErrClosed) + _, err = db.Querier(0, 1) + require.Equal(t, err, ErrClosed) +} + +func TestDBReadOnly_FlushWAL(t *testing.T) { + t.Parallel() + var ( + dbDir = t.TempDir() + err error + maxt int + ctx = context.Background() + ) + + // Bootstrap the db. + { + // Append data to the WAL. + db := newTestDB(t, withDir(dbDir)) + db.DisableCompactions() + app := db.Appender(ctx) + maxt = 1000 + for i := 0; i < maxt; i++ { + _, err := app.Append(0, labels.FromStrings(defaultLabelName, "flush"), int64(i), 1.0) + require.NoError(t, err) + } + require.NoError(t, app.Commit()) + require.NoError(t, db.Close()) + } + + // Flush WAL. + db, err := OpenDBReadOnly(dbDir, "", nil) + require.NoError(t, err) + + flush := t.TempDir() + require.NoError(t, db.FlushWAL(flush)) + require.NoError(t, db.Close()) + + // Reopen the DB from the flushed WAL block. + db, err = OpenDBReadOnly(flush, "", nil) + require.NoError(t, err) + defer func() { require.NoError(t, db.Close()) }() + blocks, err := db.Blocks() + require.NoError(t, err) + require.Len(t, blocks, 1) + + querier, err := db.Querier(0, int64(maxt)-1) + require.NoError(t, err) + defer func() { require.NoError(t, querier.Close()) }() + + // Sum the values. + seriesSet := querier.Select(ctx, false, nil, labels.MustNewMatcher(labels.MatchEqual, defaultLabelName, "flush")) + var series chunkenc.Iterator + + sum := 0.0 + for seriesSet.Next() { + series = seriesSet.At().Iterator(series) + for series.Next() == chunkenc.ValFloat { + _, v := series.At() + sum += v + } + require.NoError(t, series.Err()) + } + require.NoError(t, seriesSet.Err()) + require.Empty(t, seriesSet.Warnings()) + require.Equal(t, 1000.0, sum) +} + +func TestDBReadOnly_Querier_NoAlteration(t *testing.T) { + countChunks := func(dir string) int { + files, err := os.ReadDir(mmappedChunksDir(dir)) + require.NoError(t, err) + return len(files) + } + + dirHash := func(dir string) (hash []byte) { + // Windows requires the DB to be closed: "xxx\lock: The process cannot access the file because it is being used by another process." + // But closing the DB alters the directory in this case (it'll cut a new chunk). + if runtime.GOOS != "windows" { + hash = testutil.DirHash(t, dir) + } + return hash + } + + spinUpQuerierAndCheck := func(dir, sandboxDir string, chunksCount int) { + dBDirHash := dirHash(dir) + // Bootstrap a RO db from the same dir and set up a querier. + dbReadOnly, err := OpenDBReadOnly(dir, sandboxDir, nil) + require.NoError(t, err) + require.Equal(t, chunksCount, countChunks(dir)) + q, err := dbReadOnly.Querier(math.MinInt, math.MaxInt) + require.NoError(t, err) + require.NoError(t, q.Close()) + require.NoError(t, dbReadOnly.Close()) + // The RO Head doesn't alter RW db chunks_head/. + require.Equal(t, chunksCount, countChunks(dir)) + require.Equal(t, dirHash(dir), dBDirHash) + } + + t.Run("doesn't cut chunks while replaying WAL", func(t *testing.T) { + db := newTestDB(t) + + // Append until the first mmapped head chunk. + for i := range 121 { + app := db.Appender(context.Background()) + _, err := app.Append(0, labels.FromStrings("foo", "bar"), int64(i), 0) + require.NoError(t, err) + require.NoError(t, app.Commit()) + } + + spinUpQuerierAndCheck(db.Dir(), t.TempDir(), 0) + + // The RW Head should have no problem cutting its own chunk, + // this also proves that a chunk needed to be cut. + require.NotPanics(t, func() { db.ForceHeadMMap() }) + require.Equal(t, 1, countChunks(db.Dir())) + }) + + t.Run("doesn't truncate corrupted chunks", func(t *testing.T) { + db := newTestDB(t) + require.NoError(t, db.Close()) + + // Simulate a corrupted chunk: without a header. + chunk, err := os.Create(path.Join(mmappedChunksDir(db.Dir()), "000001")) + require.NoError(t, err) + require.NoError(t, chunk.Close()) + + spinUpQuerierAndCheck(db.Dir(), t.TempDir(), 1) + + // The RW Head should have no problem truncating its corrupted file: + // this proves that the chunk needed to be truncated. + db = newTestDB(t, withDir(db.Dir())) + + require.NoError(t, err) + require.Equal(t, 0, countChunks(db.Dir())) + }) +} + +func TestDBCannotSeePartialCommits(t *testing.T) { + if defaultIsolationDisabled { + t.Skip("skipping test since tsdb isolation is disabled") + } + + db := newTestDB(t) + + stop := make(chan struct{}) + firstInsert := make(chan struct{}) + ctx := context.Background() + + // Insert data in batches. + go func() { + iter := 0 + for { + app := db.Appender(ctx) + + for j := range 100 { + _, err := app.Append(0, labels.FromStrings("foo", "bar", "a", strconv.Itoa(j)), int64(iter), float64(iter)) + require.NoError(t, err) + } + require.NoError(t, app.Commit()) + + if iter == 0 { + close(firstInsert) + } + iter++ + + select { + case <-stop: + return + default: + } + } + }() + + <-firstInsert + + // This is a race condition, so do a few tests to tickle it. + // Usually most will fail. + inconsistencies := 0 + for range 10 { + func() { + querier, err := db.Querier(0, 1000000) + require.NoError(t, err) + defer querier.Close() + + ss := querier.Select(ctx, false, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + _, seriesSet, ws, err := expandSeriesSet(ss) + require.NoError(t, err) + require.Empty(t, ws) + + values := map[float64]struct{}{} + for _, series := range seriesSet { + values[series[len(series)-1].f] = struct{}{} + } + if len(values) != 1 { + inconsistencies++ + } + }() + } + stop <- struct{}{} + + require.Equal(t, 0, inconsistencies, "Some queries saw inconsistent results.") +} + +func TestDBQueryDoesntSeeAppendsAfterCreation(t *testing.T) { + if defaultIsolationDisabled { + t.Skip("skipping test since tsdb isolation is disabled") + } + + db := newTestDB(t) + querierBeforeAdd, err := db.Querier(0, 1000000) + require.NoError(t, err) + defer querierBeforeAdd.Close() + + ctx := context.Background() + app := db.Appender(ctx) + _, err = app.Append(0, labels.FromStrings("foo", "bar"), 0, 0) + require.NoError(t, err) + + querierAfterAddButBeforeCommit, err := db.Querier(0, 1000000) + require.NoError(t, err) + defer querierAfterAddButBeforeCommit.Close() + + // None of the queriers should return anything after the Add but before the commit. + ss := querierBeforeAdd.Select(ctx, false, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + _, seriesSet, ws, err := expandSeriesSet(ss) + require.NoError(t, err) + require.Empty(t, ws) + require.Equal(t, map[string][]sample{}, seriesSet) + + ss = querierAfterAddButBeforeCommit.Select(ctx, false, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + _, seriesSet, ws, err = expandSeriesSet(ss) + require.NoError(t, err) + require.Empty(t, ws) + require.Equal(t, map[string][]sample{}, seriesSet) + + // This commit is after the queriers are created, so should not be returned. + err = app.Commit() + require.NoError(t, err) + + // Nothing returned for querier created before the Add. + ss = querierBeforeAdd.Select(ctx, false, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + _, seriesSet, ws, err = expandSeriesSet(ss) + require.NoError(t, err) + require.Empty(t, ws) + require.Equal(t, map[string][]sample{}, seriesSet) + + // Series exists but has no samples for querier created after Add. + ss = querierAfterAddButBeforeCommit.Select(ctx, false, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + _, seriesSet, ws, err = expandSeriesSet(ss) + require.NoError(t, err) + require.Empty(t, ws) + require.Equal(t, map[string][]sample{`{foo="bar"}`: {}}, seriesSet) + + querierAfterCommit, err := db.Querier(0, 1000000) + require.NoError(t, err) + defer querierAfterCommit.Close() + + // Samples are returned for querier created after Commit. + ss = querierAfterCommit.Select(ctx, false, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + _, seriesSet, ws, err = expandSeriesSet(ss) + require.NoError(t, err) + require.Empty(t, ws) + require.Equal(t, map[string][]sample{`{foo="bar"}`: {{t: 0, f: 0}}}, seriesSet) +} + +func assureChunkFromSamples(t *testing.T, samples []chunks.Sample) chunks.Meta { + chks, err := chunks.ChunkFromSamples(samples) + require.NoError(t, err) + return chks +} + +// TestChunkWriter_ReadAfterWrite ensures that chunk segment are cut at the set segment size and +// that the resulted segments includes the expected chunks data. +func TestChunkWriter_ReadAfterWrite(t *testing.T) { + chk1 := assureChunkFromSamples(t, []chunks.Sample{sample{1, 1, nil, nil}}) + chk2 := assureChunkFromSamples(t, []chunks.Sample{sample{1, 2, nil, nil}}) + chk3 := assureChunkFromSamples(t, []chunks.Sample{sample{1, 3, nil, nil}}) + chk4 := assureChunkFromSamples(t, []chunks.Sample{sample{1, 4, nil, nil}}) + chk5 := assureChunkFromSamples(t, []chunks.Sample{sample{1, 5, nil, nil}}) + chunkSize := len(chk1.Chunk.Bytes()) + chunks.MaxChunkLengthFieldSize + chunks.ChunkEncodingSize + crc32.Size + + tests := []struct { + chks [][]chunks.Meta + segmentSize, + expSegmentsCount int + expSegmentSizes []int + }{ + // 0:Last chunk ends at the segment boundary so + // all chunks should fit in a single segment. + { + chks: [][]chunks.Meta{ + { + chk1, + chk2, + chk3, + }, + }, + segmentSize: 3 * chunkSize, + expSegmentSizes: []int{3 * chunkSize}, + expSegmentsCount: 1, + }, + // 1:Two chunks can fit in a single segment so the last one should result in a new segment. + { + chks: [][]chunks.Meta{ + { + chk1, + chk2, + chk3, + chk4, + chk5, + }, + }, + segmentSize: 2 * chunkSize, + expSegmentSizes: []int{2 * chunkSize, 2 * chunkSize, chunkSize}, + expSegmentsCount: 3, + }, + // 2:When the segment size is smaller than the size of 2 chunks + // the last segment should still create a new segment. + { + chks: [][]chunks.Meta{ + { + chk1, + chk2, + chk3, + }, + }, + segmentSize: 2*chunkSize - 1, + expSegmentSizes: []int{chunkSize, chunkSize, chunkSize}, + expSegmentsCount: 3, + }, + // 3:When the segment is smaller than a single chunk + // it should still be written by ignoring the max segment size. + { + chks: [][]chunks.Meta{ + { + chk1, + }, + }, + segmentSize: chunkSize - 1, + expSegmentSizes: []int{chunkSize}, + expSegmentsCount: 1, + }, + // 4:All chunks are bigger than the max segment size, but + // these should still be written even when this will result in bigger segment than the set size. + // Each segment will hold a single chunk. + { + chks: [][]chunks.Meta{ + { + chk1, + chk2, + chk3, + }, + }, + segmentSize: 1, + expSegmentSizes: []int{chunkSize, chunkSize, chunkSize}, + expSegmentsCount: 3, + }, + // 5:Adding multiple batches of chunks. + { + chks: [][]chunks.Meta{ + { + chk1, + chk2, + chk3, + }, + { + chk4, + chk5, + }, + }, + segmentSize: 3 * chunkSize, + expSegmentSizes: []int{3 * chunkSize, 2 * chunkSize}, + expSegmentsCount: 2, + }, + // 6:Adding multiple batches of chunks. + { + chks: [][]chunks.Meta{ + { + chk1, + }, + { + chk2, + chk3, + }, + { + chk4, + }, + }, + segmentSize: 2 * chunkSize, + expSegmentSizes: []int{2 * chunkSize, 2 * chunkSize}, + expSegmentsCount: 2, + }, + } + + for i, test := range tests { + t.Run(strconv.Itoa(i), func(t *testing.T) { + tempDir := t.TempDir() + + chunkw, err := chunks.NewWriter(tempDir, chunks.WithSegmentSize(chunks.SegmentHeaderSize+int64(test.segmentSize))) + require.NoError(t, err) + + for _, chks := range test.chks { + require.NoError(t, chunkw.WriteChunks(chks...)) + } + require.NoError(t, chunkw.Close()) + + files, err := os.ReadDir(tempDir) + require.NoError(t, err) + require.Len(t, files, test.expSegmentsCount, "expected segments count mismatch") + + // Verify that all data is written to the segments. + sizeExp := 0 + sizeAct := 0 + + for _, chks := range test.chks { + for _, chk := range chks { + l := make([]byte, binary.MaxVarintLen32) + sizeExp += binary.PutUvarint(l, uint64(len(chk.Chunk.Bytes()))) // The length field. + sizeExp += chunks.ChunkEncodingSize + sizeExp += len(chk.Chunk.Bytes()) // The data itself. + sizeExp += crc32.Size // The 4 bytes of crc32 + } + } + sizeExp += test.expSegmentsCount * chunks.SegmentHeaderSize // The segment header bytes. + + for i, f := range files { + fi, err := f.Info() + require.NoError(t, err) + size := int(fi.Size()) + // Verify that the segment is the same or smaller than the expected size. + require.GreaterOrEqual(t, chunks.SegmentHeaderSize+test.expSegmentSizes[i], size, "Segment:%v should NOT be bigger than:%v actual:%v", i, chunks.SegmentHeaderSize+test.expSegmentSizes[i], size) + + sizeAct += size + } + require.Equal(t, sizeExp, sizeAct) + + // Check the content of the chunks. + r, err := chunks.NewDirReader(tempDir, nil) + require.NoError(t, err) + defer func() { require.NoError(t, r.Close()) }() + + for _, chks := range test.chks { + for _, chkExp := range chks { + chkAct, iterable, err := r.ChunkOrIterable(chkExp) + require.NoError(t, err) + require.Nil(t, iterable) + require.Equal(t, chkExp.Chunk.Bytes(), chkAct.Bytes()) + } + } + }) + } +} + +func TestRangeForTimestamp(t *testing.T) { + type args struct { + t int64 + width int64 + } + tests := []struct { + args args + expected int64 + }{ + {args{0, 5}, 5}, + {args{1, 5}, 5}, + {args{5, 5}, 10}, + {args{6, 5}, 10}, + {args{13, 5}, 15}, + {args{95, 5}, 100}, + } + for _, tt := range tests { + got := rangeForTimestamp(tt.args.t, tt.args.width) + require.Equal(t, tt.expected, got) + } +} + +// TestChunkReader_ConcurrentReads checks that the chunk result can be read concurrently. +// Regression test for https://github.com/prometheus/prometheus/pull/6514. +func TestChunkReader_ConcurrentReads(t *testing.T) { + t.Parallel() + chks := []chunks.Meta{ + assureChunkFromSamples(t, []chunks.Sample{sample{1, 1, nil, nil}}), + assureChunkFromSamples(t, []chunks.Sample{sample{1, 2, nil, nil}}), + assureChunkFromSamples(t, []chunks.Sample{sample{1, 3, nil, nil}}), + assureChunkFromSamples(t, []chunks.Sample{sample{1, 4, nil, nil}}), + assureChunkFromSamples(t, []chunks.Sample{sample{1, 5, nil, nil}}), + } + + tempDir := t.TempDir() + + chunkw, err := chunks.NewWriter(tempDir) + require.NoError(t, err) + + require.NoError(t, chunkw.WriteChunks(chks...)) + require.NoError(t, chunkw.Close()) + + r, err := chunks.NewDirReader(tempDir, nil) + require.NoError(t, err) + + var wg sync.WaitGroup + for _, chk := range chks { + for range 100 { + wg.Add(1) + go func(chunk chunks.Meta) { + defer wg.Done() + + chkAct, iterable, err := r.ChunkOrIterable(chunk) + require.NoError(t, err) + require.Nil(t, iterable) + require.Equal(t, chunk.Chunk.Bytes(), chkAct.Bytes()) + }(chk) + } + wg.Wait() + } + require.NoError(t, r.Close()) +} + +// TestCompactHead ensures that the head compaction +// creates a block that is ready for loading and +// does not cause data loss. +// This test: +// * opens a storage; +// * appends values; +// * compacts the head; and +// * queries the db to ensure the samples are present from the compacted head. +func TestCompactHead(t *testing.T) { + t.Parallel() + + // Open a DB and append data to the WAL. + opts := &Options{ + RetentionDuration: int64(time.Hour * 24 * 15 / time.Millisecond), + NoLockfile: true, + MinBlockDuration: int64(time.Hour * 2 / time.Millisecond), + MaxBlockDuration: int64(time.Hour * 2 / time.Millisecond), + WALCompression: compression.Snappy, + } + db := newTestDB(t, withOpts(opts)) + ctx := context.Background() + app := db.Appender(ctx) + var expSamples []sample + maxt := 100 + for i := range maxt { + val := rand.Float64() + _, err := app.Append(0, labels.FromStrings("a", "b"), int64(i), val) + require.NoError(t, err) + expSamples = append(expSamples, sample{int64(i), val, nil, nil}) + } + require.NoError(t, app.Commit()) + + // Compact the Head to create a new block. + require.NoError(t, db.CompactHead(NewRangeHead(db.Head(), 0, int64(maxt)-1))) + require.NoError(t, db.Close()) + + // Delete everything but the new block and + // reopen the db to query it to ensure it includes the head data. + require.NoError(t, deleteNonBlocks(db.Dir())) + db = newTestDB(t, withDir(db.Dir()), withOpts(opts)) + require.Len(t, db.Blocks(), 1) + require.Equal(t, int64(maxt), db.Head().MinTime()) + defer func() { require.NoError(t, db.Close()) }() + querier, err := db.Querier(0, int64(maxt)-1) + require.NoError(t, err) + defer func() { require.NoError(t, querier.Close()) }() + + seriesSet := querier.Select(ctx, false, nil, &labels.Matcher{Type: labels.MatchEqual, Name: "a", Value: "b"}) + var series chunkenc.Iterator + var actSamples []sample + + for seriesSet.Next() { + series = seriesSet.At().Iterator(series) + for series.Next() == chunkenc.ValFloat { + time, val := series.At() + actSamples = append(actSamples, sample{time, val, nil, nil}) + } + require.NoError(t, series.Err()) + } + require.Equal(t, expSamples, actSamples) + require.NoError(t, seriesSet.Err()) +} + +// TestCompactHeadWithDeletion tests https://github.com/prometheus/prometheus/issues/11585. +func TestCompactHeadWithDeletion(t *testing.T) { + db := newTestDB(t) + + ctx := context.Background() + + app := db.Appender(ctx) + _, err := app.Append(0, labels.FromStrings("a", "b"), 10, rand.Float64()) + require.NoError(t, err) + require.NoError(t, app.Commit()) + + err = db.Delete(ctx, 0, 100, labels.MustNewMatcher(labels.MatchEqual, "a", "b")) + require.NoError(t, err) + + // This recreates the bug. + require.NoError(t, db.CompactHead(NewRangeHead(db.Head(), 0, 100))) +} + +func deleteNonBlocks(dbDir string) error { + dirs, err := os.ReadDir(dbDir) + if err != nil { + return err + } + for _, dir := range dirs { + if ok := isBlockDir(dir); !ok { + if err := os.RemoveAll(filepath.Join(dbDir, dir.Name())); err != nil { + return err + } + } + } + dirs, err = os.ReadDir(dbDir) + if err != nil { + return err + } + for _, dir := range dirs { + if ok := isBlockDir(dir); !ok { + return fmt.Errorf("root folder:%v still hase non block directory:%v", dbDir, dir.Name()) + } + } + return nil +} + +func TestOpen_VariousBlockStates(t *testing.T) { + tmpDir := t.TempDir() + + var ( + expectedLoadedDirs = map[string]struct{}{} + expectedRemovedDirs = map[string]struct{}{} + expectedIgnoredDirs = map[string]struct{}{} + ) + + { + // Ok blocks; should be loaded. + expectedLoadedDirs[createBlock(t, tmpDir, genSeries(10, 2, 0, 10))] = struct{}{} + expectedLoadedDirs[createBlock(t, tmpDir, genSeries(10, 2, 10, 20))] = struct{}{} + } + { + // Block to repair; should be repaired & loaded. + dbDir := filepath.Join("testdata", "repair_index_version", "01BZJ9WJQPWHGNC2W4J9TA62KC") + outDir := filepath.Join(tmpDir, "01BZJ9WJQPWHGNC2W4J9TA62KC") + expectedLoadedDirs[outDir] = struct{}{} + + // Touch chunks dir in block. + require.NoError(t, os.MkdirAll(filepath.Join(dbDir, "chunks"), 0o777)) + defer func() { + require.NoError(t, os.RemoveAll(filepath.Join(dbDir, "chunks"))) + }() + require.NoError(t, os.Mkdir(outDir, os.ModePerm)) + require.NoError(t, fileutil.CopyDirs(dbDir, outDir)) + } + { + // Missing meta.json; should be ignored and only logged. + // TODO(bwplotka): Probably add metric. + dir := createBlock(t, tmpDir, genSeries(10, 2, 20, 30)) + expectedIgnoredDirs[dir] = struct{}{} + require.NoError(t, os.Remove(filepath.Join(dir, metaFilename))) + } + { + // Tmp blocks during creation; those should be removed on start. + dir := createBlock(t, tmpDir, genSeries(10, 2, 30, 40)) + require.NoError(t, fileutil.Replace(dir, dir+tmpForCreationBlockDirSuffix)) + expectedRemovedDirs[dir+tmpForCreationBlockDirSuffix] = struct{}{} + + // Tmp blocks during deletion; those should be removed on start. + dir = createBlock(t, tmpDir, genSeries(10, 2, 40, 50)) + require.NoError(t, fileutil.Replace(dir, dir+tmpForDeletionBlockDirSuffix)) + expectedRemovedDirs[dir+tmpForDeletionBlockDirSuffix] = struct{}{} + + // Pre-2.21 tmp blocks; those should be removed on start. + dir = createBlock(t, tmpDir, genSeries(10, 2, 50, 60)) + require.NoError(t, fileutil.Replace(dir, dir+tmpLegacy)) + expectedRemovedDirs[dir+tmpLegacy] = struct{}{} + } + { + // One ok block; but two should be replaced. + dir := createBlock(t, tmpDir, genSeries(10, 2, 50, 60)) + expectedLoadedDirs[dir] = struct{}{} + + m, _, err := readMetaFile(dir) + require.NoError(t, err) + + compacted := createBlock(t, tmpDir, genSeries(10, 2, 50, 55)) + expectedRemovedDirs[compacted] = struct{}{} + + m.Compaction.Parents = append(m.Compaction.Parents, + BlockDesc{ULID: ulid.MustParse(filepath.Base(compacted))}, + BlockDesc{ULID: ulid.MustNew(1, nil)}, + BlockDesc{ULID: ulid.MustNew(123, nil)}, + ) + + // Regression test: Already removed parent can be still in list, which was causing Open errors. + m.Compaction.Parents = append(m.Compaction.Parents, BlockDesc{ULID: ulid.MustParse(filepath.Base(compacted))}) + m.Compaction.Parents = append(m.Compaction.Parents, BlockDesc{ULID: ulid.MustParse(filepath.Base(compacted))}) + _, err = writeMetaFile(promslog.New(&promslog.Config{}), dir, m) + require.NoError(t, err) + } + tmpCheckpointDir := path.Join(tmpDir, "wal/checkpoint.00000001.tmp") + err := os.MkdirAll(tmpCheckpointDir, 0o777) + require.NoError(t, err) + tmpChunkSnapshotDir := path.Join(tmpDir, chunkSnapshotPrefix+"0000.00000001.tmp") + err = os.MkdirAll(tmpChunkSnapshotDir, 0o777) + require.NoError(t, err) + + opts := DefaultOptions() + opts.RetentionDuration = 0 + db := newTestDB(t, withDir(tmpDir), withOpts(opts)) + loadedBlocks := db.Blocks() + + var loaded int + for _, l := range loadedBlocks { + _, ok := expectedLoadedDirs[filepath.Join(tmpDir, l.meta.ULID.String())] + require.True(t, ok, "unexpected block", l.meta.ULID, "was loaded") + loaded++ + } + require.Len(t, expectedLoadedDirs, loaded) + require.NoError(t, db.Close()) + + files, err := os.ReadDir(tmpDir) + require.NoError(t, err) + + var ignored int + for _, f := range files { + _, ok := expectedRemovedDirs[filepath.Join(tmpDir, f.Name())] + require.False(t, ok, "expected", filepath.Join(tmpDir, f.Name()), "to be removed, but still exists") + if _, ok := expectedIgnoredDirs[filepath.Join(tmpDir, f.Name())]; ok { + ignored++ + } + } + require.Len(t, expectedIgnoredDirs, ignored) + _, err = os.Stat(tmpCheckpointDir) + require.True(t, os.IsNotExist(err)) + _, err = os.Stat(tmpChunkSnapshotDir) + require.True(t, os.IsNotExist(err)) +} + +func TestOneCheckpointPerCompactCall(t *testing.T) { + t.Parallel() + blockRange := int64(1000) + opts := &Options{ + RetentionDuration: blockRange * 1000, + NoLockfile: true, + MinBlockDuration: blockRange, + MaxBlockDuration: blockRange, + } + + ctx := context.Background() + + db := newTestDB(t, withOpts(opts)) + db.DisableCompactions() + + // Case 1: Lot's of uncompacted data in Head. + + lbls := labels.FromStrings("foo_d", "choco_bar") + // Append samples spanning 59 block ranges. + app := db.Appender(context.Background()) + for i := range int64(60) { + _, err := app.Append(0, lbls, blockRange*i, rand.Float64()) + require.NoError(t, err) + _, err = app.Append(0, lbls, (blockRange*i)+blockRange/2, rand.Float64()) + require.NoError(t, err) + // Rotate the WAL file so that there is >3 files for checkpoint to happen. + _, err = db.head.wal.NextSegment() + require.NoError(t, err) + } + require.NoError(t, app.Commit()) + + // Check the existing WAL files. + first, last, err := wlog.Segments(db.head.wal.Dir()) + require.NoError(t, err) + require.Equal(t, 0, first) + require.Equal(t, 60, last) + + require.Equal(t, 0.0, prom_testutil.ToFloat64(db.head.metrics.checkpointCreationTotal)) + require.NoError(t, db.Compact(ctx)) + require.Equal(t, 1.0, prom_testutil.ToFloat64(db.head.metrics.checkpointCreationTotal)) + + // As the data spans for 59 blocks, 58 go to disk and 1 remains in Head. + require.Len(t, db.Blocks(), 58) + // Though WAL was truncated only once, head should be truncated after each compaction. + require.Equal(t, 58.0, prom_testutil.ToFloat64(db.head.metrics.headTruncateTotal)) + + // The compaction should have only truncated first 2/3 of WAL (while also rotating the files). + first, last, err = wlog.Segments(db.head.wal.Dir()) + require.NoError(t, err) + require.Equal(t, 40, first) + require.Equal(t, 61, last) + + // The first checkpoint would be for first 2/3rd of WAL, hence till 39. + // That should be the last checkpoint. + _, cno, err := wlog.LastCheckpoint(db.head.wal.Dir()) + require.NoError(t, err) + require.Equal(t, 39, cno) + + // Case 2: Old blocks on disk. + // The above blocks will act as old blocks. + + // Creating a block to cover the data in the Head so that + // Head will skip the data during replay and start fresh. + blocks := db.Blocks() + newBlockMint := blocks[len(blocks)-1].Meta().MaxTime + newBlockMaxt := db.Head().MaxTime() + 1 + require.NoError(t, db.Close()) + + createBlock(t, db.Dir(), genSeries(1, 1, newBlockMint, newBlockMaxt)) + + db = newTestDB(t, withDir(db.Dir()), withOpts(opts)) + db.DisableCompactions() + + // 1 block more. + require.Len(t, db.Blocks(), 59) + // No series in Head because of this new block. + require.Equal(t, 0, int(db.head.NumSeries())) + + // Adding sample way into the future. + app = db.Appender(context.Background()) + _, err = app.Append(0, lbls, blockRange*120, rand.Float64()) + require.NoError(t, err) + require.NoError(t, app.Commit()) + + // The mint of head is the last block maxt, that means the gap between mint and maxt + // of Head is too large. This will trigger many compactions. + require.Equal(t, newBlockMaxt, db.head.MinTime()) + + // Another WAL file was rotated. + first, last, err = wlog.Segments(db.head.wal.Dir()) + require.NoError(t, err) + require.Equal(t, 40, first) + require.Equal(t, 62, last) + + require.Equal(t, 0.0, prom_testutil.ToFloat64(db.head.metrics.checkpointCreationTotal)) + require.NoError(t, db.Compact(ctx)) + require.Equal(t, 1.0, prom_testutil.ToFloat64(db.head.metrics.checkpointCreationTotal)) + + // No new blocks should be created as there was not data in between the new samples and the blocks. + require.Len(t, db.Blocks(), 59) + + // The compaction should have only truncated first 2/3 of WAL (while also rotating the files). + first, last, err = wlog.Segments(db.head.wal.Dir()) + require.NoError(t, err) + require.Equal(t, 55, first) + require.Equal(t, 63, last) + + // The first checkpoint would be for first 2/3rd of WAL, hence till 54. + // That should be the last checkpoint. + _, cno, err = wlog.LastCheckpoint(db.head.wal.Dir()) + require.NoError(t, err) + require.Equal(t, 54, cno) +} + +func TestNoPanicOnTSDBOpenError(t *testing.T) { + tmpdir := t.TempDir() + + // Taking the lock will cause a TSDB startup error. + l, err := tsdbutil.NewDirLocker(tmpdir, "tsdb", promslog.NewNopLogger(), nil) + require.NoError(t, err) + require.NoError(t, l.Lock()) + + _, err = Open(tmpdir, nil, nil, DefaultOptions(), nil) + require.Error(t, err) + + require.NoError(t, l.Release()) +} + +func TestLockfile(t *testing.T) { + tsdbutil.TestDirLockerUsage(t, func(t *testing.T, data string, createLock bool) (*tsdbutil.DirLocker, testutil.Closer) { + opts := DefaultOptions() + opts.NoLockfile = !createLock + + // Create the DB. This should create lockfile and its metrics. + db, err := Open(data, nil, nil, opts, nil) + require.NoError(t, err) + + return db.locker, testutil.NewCallbackCloser(func() { + require.NoError(t, db.Close()) + }) + }) +} + +func TestQuerier_ShouldNotPanicIfHeadChunkIsTruncatedWhileReadingQueriedChunks(t *testing.T) { + t.Skip("TODO: investigate why process crash in CI") + + const numRuns = 5 + + for i := 1; i <= numRuns; i++ { + t.Run(strconv.Itoa(i), func(t *testing.T) { + testQuerierShouldNotPanicIfHeadChunkIsTruncatedWhileReadingQueriedChunks(t) + }) + } +} + +func testQuerierShouldNotPanicIfHeadChunkIsTruncatedWhileReadingQueriedChunks(t *testing.T) { + const ( + numSeries = 1000 + numStressIterations = 10000 + minStressAllocationBytes = 128 * 1024 + maxStressAllocationBytes = 512 * 1024 + ) + + db := newTestDB(t) + + // Disable compactions so we can control it. + db.DisableCompactions() + + // Generate the metrics we're going to append. + metrics := make([]labels.Labels, 0, numSeries) + for i := range numSeries { + metrics = append(metrics, labels.FromStrings(labels.MetricName, fmt.Sprintf("test_%d", i))) + } + + // Push 1 sample every 15s for 2x the block duration period. + ctx := context.Background() + interval := int64(15 * time.Second / time.Millisecond) + ts := int64(0) + + for ; ts < 2*DefaultBlockDuration; ts += interval { + app := db.Appender(ctx) + + for _, metric := range metrics { + _, err := app.Append(0, metric, ts, float64(ts)) + require.NoError(t, err) + } + + require.NoError(t, app.Commit()) + } + + // Compact the TSDB head for the first time. We expect the head chunks file has been cut. + require.NoError(t, db.Compact(ctx)) + require.Equal(t, float64(1), prom_testutil.ToFloat64(db.Head().metrics.headTruncateTotal)) + + // Push more samples for another 1x block duration period. + for ; ts < 3*DefaultBlockDuration; ts += interval { + app := db.Appender(ctx) + + for _, metric := range metrics { + _, err := app.Append(0, metric, ts, float64(ts)) + require.NoError(t, err) + } + + require.NoError(t, app.Commit()) + } + + // At this point we expect 2 mmap-ed head chunks. + + // Get a querier and make sure it's closed only once the test is over. + querier, err := db.Querier(0, math.MaxInt64) + require.NoError(t, err) + defer func() { + require.NoError(t, querier.Close()) + }() + + // Query back all series. + hints := &storage.SelectHints{Start: 0, End: math.MaxInt64, Step: interval} + seriesSet := querier.Select(ctx, true, hints, labels.MustNewMatcher(labels.MatchRegexp, labels.MetricName, ".+")) + + // Fetch samples iterators from all series. + var iterators []chunkenc.Iterator + actualSeries := 0 + for seriesSet.Next() { + actualSeries++ + + // Get the iterator and call Next() so that we're sure the chunk is loaded. + it := seriesSet.At().Iterator(nil) + it.Next() + it.At() + + iterators = append(iterators, it) + } + require.NoError(t, seriesSet.Err()) + require.Equal(t, numSeries, actualSeries) + + // Compact the TSDB head again. + require.NoError(t, db.Compact(ctx)) + require.Equal(t, float64(2), prom_testutil.ToFloat64(db.Head().metrics.headTruncateTotal)) + + // At this point we expect 1 head chunk has been deleted. + + // Stress the memory and call GC. This is required to increase the chances + // the chunk memory area is released to the kernel. + var buf []byte + for i := range numStressIterations { + //nolint:staticcheck + buf = append(buf, make([]byte, minStressAllocationBytes+rand.Int31n(maxStressAllocationBytes-minStressAllocationBytes))...) + if i%1000 == 0 { + buf = nil + } + } + + // Iterate samples. Here we're summing it just to make sure no golang compiler + // optimization triggers in case we discard the result of it.At(). + var sum float64 + var firstErr error + for _, it := range iterators { + for it.Next() == chunkenc.ValFloat { + _, v := it.At() + sum += v + } + + if err := it.Err(); err != nil { + firstErr = err + } + } + + // After having iterated all samples we also want to be sure no error occurred or + // the "cannot populate chunk XXX: not found" error occurred. This error can occur + // when the iterator tries to fetch an head chunk which has been offloaded because + // of the head compaction in the meanwhile. + if firstErr != nil { + require.ErrorContains(t, firstErr, "cannot populate chunk") + } +} + +func TestChunkQuerier_ShouldNotPanicIfHeadChunkIsTruncatedWhileReadingQueriedChunks(t *testing.T) { + t.Skip("TODO: investigate why process crash in CI") + + const numRuns = 5 + + for i := 1; i <= numRuns; i++ { + t.Run(strconv.Itoa(i), func(t *testing.T) { + testChunkQuerierShouldNotPanicIfHeadChunkIsTruncatedWhileReadingQueriedChunks(t) + }) + } +} + +func testChunkQuerierShouldNotPanicIfHeadChunkIsTruncatedWhileReadingQueriedChunks(t *testing.T) { + const ( + numSeries = 1000 + numStressIterations = 10000 + minStressAllocationBytes = 128 * 1024 + maxStressAllocationBytes = 512 * 1024 + ) + + db := newTestDB(t) + + // Disable compactions so we can control it. + db.DisableCompactions() + + // Generate the metrics we're going to append. + metrics := make([]labels.Labels, 0, numSeries) + for i := range numSeries { + metrics = append(metrics, labels.FromStrings(labels.MetricName, fmt.Sprintf("test_%d", i))) + } + + // Push 1 sample every 15s for 2x the block duration period. + ctx := context.Background() + interval := int64(15 * time.Second / time.Millisecond) + ts := int64(0) + + for ; ts < 2*DefaultBlockDuration; ts += interval { + app := db.Appender(ctx) + + for _, metric := range metrics { + _, err := app.Append(0, metric, ts, float64(ts)) + require.NoError(t, err) + } + + require.NoError(t, app.Commit()) + } + + // Compact the TSDB head for the first time. We expect the head chunks file has been cut. + require.NoError(t, db.Compact(ctx)) + require.Equal(t, float64(1), prom_testutil.ToFloat64(db.Head().metrics.headTruncateTotal)) + + // Push more samples for another 1x block duration period. + for ; ts < 3*DefaultBlockDuration; ts += interval { + app := db.Appender(ctx) + + for _, metric := range metrics { + _, err := app.Append(0, metric, ts, float64(ts)) + require.NoError(t, err) + } + + require.NoError(t, app.Commit()) + } + + // At this point we expect 2 mmap-ed head chunks. + + // Get a querier and make sure it's closed only once the test is over. + querier, err := db.ChunkQuerier(0, math.MaxInt64) + require.NoError(t, err) + defer func() { + require.NoError(t, querier.Close()) + }() + + // Query back all series. + hints := &storage.SelectHints{Start: 0, End: math.MaxInt64, Step: interval} + seriesSet := querier.Select(ctx, true, hints, labels.MustNewMatcher(labels.MatchRegexp, labels.MetricName, ".+")) + + // Iterate all series and get their chunks. + var it chunks.Iterator + var chunks []chunkenc.Chunk + actualSeries := 0 + for seriesSet.Next() { + actualSeries++ + it = seriesSet.At().Iterator(it) + for it.Next() { + chunks = append(chunks, it.At().Chunk) + } + } + require.NoError(t, seriesSet.Err()) + require.Equal(t, numSeries, actualSeries) + + // Compact the TSDB head again. + require.NoError(t, db.Compact(ctx)) + require.Equal(t, float64(2), prom_testutil.ToFloat64(db.Head().metrics.headTruncateTotal)) + + // At this point we expect 1 head chunk has been deleted. + + // Stress the memory and call GC. This is required to increase the chances + // the chunk memory area is released to the kernel. + var buf []byte + for i := range numStressIterations { + //nolint:staticcheck + buf = append(buf, make([]byte, minStressAllocationBytes+rand.Int31n(maxStressAllocationBytes-minStressAllocationBytes))...) + if i%1000 == 0 { + buf = nil + } + } + + // Iterate chunks and read their bytes slice. Here we're computing the CRC32 + // just to iterate through the bytes slice. We don't really care the reason why + // we read this data, we just need to read it to make sure the memory address + // of the []byte is still valid. + chkCRC32 := crc32.New(crc32.MakeTable(crc32.Castagnoli)) + for _, chunk := range chunks { + chkCRC32.Reset() + _, err := chkCRC32.Write(chunk.Bytes()) + require.NoError(t, err) + } +} + +func TestQuerierShouldNotFailIfOOOCompactionOccursAfterRetrievingQuerier(t *testing.T) { + opts := DefaultOptions() + opts.OutOfOrderTimeWindow = 3 * DefaultBlockDuration + db := newTestDB(t, withOpts(opts)) + + // Disable compactions so we can control it. + db.DisableCompactions() + + metric := labels.FromStrings(labels.MetricName, "test_metric") + ctx := context.Background() + interval := int64(15 * time.Second / time.Millisecond) + ts := int64(0) + samplesWritten := 0 + + // Capture the first timestamp - this will be the timestamp of the OOO sample we'll append below. + oooTS := ts + ts += interval + + // Push samples after the OOO sample we'll write below. + for ; ts < 10*interval; ts += interval { + app := db.Appender(ctx) + _, err := app.Append(0, metric, ts, float64(ts)) + require.NoError(t, err) + require.NoError(t, app.Commit()) + samplesWritten++ + } + + // Push a single OOO sample. + app := db.Appender(ctx) + _, err := app.Append(0, metric, oooTS, float64(ts)) + require.NoError(t, err) + require.NoError(t, app.Commit()) + samplesWritten++ + + // Get a querier. + querierCreatedBeforeCompaction, err := db.ChunkQuerier(0, math.MaxInt64) + require.NoError(t, err) + + // Start OOO head compaction. + compactionComplete := atomic.NewBool(false) + go func() { + defer compactionComplete.Store(true) + + require.NoError(t, db.CompactOOOHead(ctx)) + require.Equal(t, float64(1), prom_testutil.ToFloat64(db.Head().metrics.chunksRemoved)) + }() + + // Give CompactOOOHead time to start work. + // If it does not wait for querierCreatedBeforeCompaction to be closed, then the query will return incorrect results or fail. + time.Sleep(time.Second) + require.False(t, compactionComplete.Load(), "compaction completed before reading chunks or closing querier created before compaction") + + // Get another querier. This one should only use the compacted blocks from disk and ignore the chunks that will be garbage collected. + querierCreatedAfterCompaction, err := db.ChunkQuerier(0, math.MaxInt64) + require.NoError(t, err) + + testQuerier := func(q storage.ChunkQuerier) { + // Query back the series. + hints := &storage.SelectHints{Start: 0, End: math.MaxInt64, Step: interval} + seriesSet := q.Select(ctx, true, hints, labels.MustNewMatcher(labels.MatchEqual, labels.MetricName, "test_metric")) + + // Collect the iterator for the series. + var iterators []chunks.Iterator + for seriesSet.Next() { + iterators = append(iterators, seriesSet.At().Iterator(nil)) + } + require.NoError(t, seriesSet.Err()) + require.Len(t, iterators, 1) + iterator := iterators[0] + + // Check that we can still successfully read all samples. + samplesRead := 0 + for iterator.Next() { + samplesRead += iterator.At().Chunk.NumSamples() + } + + require.NoError(t, iterator.Err()) + require.Equal(t, samplesWritten, samplesRead) + } + + testQuerier(querierCreatedBeforeCompaction) + + require.False(t, compactionComplete.Load(), "compaction completed before closing querier created before compaction") + require.NoError(t, querierCreatedBeforeCompaction.Close()) + require.Eventually(t, compactionComplete.Load, time.Second, 10*time.Millisecond, "compaction should complete after querier created before compaction was closed, and not wait for querier created after compaction") + + // Use the querier created after compaction and confirm it returns the expected results (ie. from the disk block created from OOO head and in-order head) without error. + testQuerier(querierCreatedAfterCompaction) + require.NoError(t, querierCreatedAfterCompaction.Close()) +} + +func TestQuerierShouldNotFailIfOOOCompactionOccursAfterSelecting(t *testing.T) { + opts := DefaultOptions() + opts.OutOfOrderTimeWindow = 3 * DefaultBlockDuration + db := newTestDB(t, withOpts(opts)) + + // Disable compactions so we can control it. + db.DisableCompactions() + + metric := labels.FromStrings(labels.MetricName, "test_metric") + ctx := context.Background() + interval := int64(15 * time.Second / time.Millisecond) + ts := int64(0) + samplesWritten := 0 + + // Capture the first timestamp - this will be the timestamp of the OOO sample we'll append below. + oooTS := ts + ts += interval + + // Push samples after the OOO sample we'll write below. + for ; ts < 10*interval; ts += interval { + app := db.Appender(ctx) + _, err := app.Append(0, metric, ts, float64(ts)) + require.NoError(t, err) + require.NoError(t, app.Commit()) + samplesWritten++ + } + + // Push a single OOO sample. + app := db.Appender(ctx) + _, err := app.Append(0, metric, oooTS, float64(ts)) + require.NoError(t, err) + require.NoError(t, app.Commit()) + samplesWritten++ + + // Get a querier. + querier, err := db.ChunkQuerier(0, math.MaxInt64) + require.NoError(t, err) + + // Query back the series. + hints := &storage.SelectHints{Start: 0, End: math.MaxInt64, Step: interval} + seriesSet := querier.Select(ctx, true, hints, labels.MustNewMatcher(labels.MatchEqual, labels.MetricName, "test_metric")) + + // Start OOO head compaction. + compactionComplete := atomic.NewBool(false) + go func() { + defer compactionComplete.Store(true) + + require.NoError(t, db.CompactOOOHead(ctx)) + require.Equal(t, float64(1), prom_testutil.ToFloat64(db.Head().metrics.chunksRemoved)) + }() + + // Give CompactOOOHead time to start work. + // If it does not wait for the querier to be closed, then the query will return incorrect results or fail. + time.Sleep(time.Second) + require.False(t, compactionComplete.Load(), "compaction completed before reading chunks or closing querier") + + // Collect the iterator for the series. + var iterators []chunks.Iterator + for seriesSet.Next() { + iterators = append(iterators, seriesSet.At().Iterator(nil)) + } + require.NoError(t, seriesSet.Err()) + require.Len(t, iterators, 1) + iterator := iterators[0] + + // Check that we can still successfully read all samples. + samplesRead := 0 + for iterator.Next() { + samplesRead += iterator.At().Chunk.NumSamples() + } + + require.NoError(t, iterator.Err()) + require.Equal(t, samplesWritten, samplesRead) + + require.False(t, compactionComplete.Load(), "compaction completed before closing querier") + require.NoError(t, querier.Close()) + require.Eventually(t, compactionComplete.Load, time.Second, 10*time.Millisecond, "compaction should complete after querier was closed") +} + +func TestQuerierShouldNotFailIfOOOCompactionOccursAfterRetrievingIterators(t *testing.T) { + opts := DefaultOptions() + opts.OutOfOrderTimeWindow = 3 * DefaultBlockDuration + db := newTestDB(t, withOpts(opts)) + + // Disable compactions so we can control it. + db.DisableCompactions() + + metric := labels.FromStrings(labels.MetricName, "test_metric") + ctx := context.Background() + interval := int64(15 * time.Second / time.Millisecond) + ts := int64(0) + samplesWritten := 0 + + // Capture the first timestamp - this will be the timestamp of the OOO sample we'll append below. + oooTS := ts + ts += interval + + // Push samples after the OOO sample we'll write below. + for ; ts < 10*interval; ts += interval { + app := db.Appender(ctx) + _, err := app.Append(0, metric, ts, float64(ts)) + require.NoError(t, err) + require.NoError(t, app.Commit()) + samplesWritten++ + } + + // Push a single OOO sample. + app := db.Appender(ctx) + _, err := app.Append(0, metric, oooTS, float64(ts)) + require.NoError(t, err) + require.NoError(t, app.Commit()) + samplesWritten++ + + // Get a querier. + querier, err := db.ChunkQuerier(0, math.MaxInt64) + require.NoError(t, err) + + // Query back the series. + hints := &storage.SelectHints{Start: 0, End: math.MaxInt64, Step: interval} + seriesSet := querier.Select(ctx, true, hints, labels.MustNewMatcher(labels.MatchEqual, labels.MetricName, "test_metric")) + + // Collect the iterator for the series. + var iterators []chunks.Iterator + for seriesSet.Next() { + iterators = append(iterators, seriesSet.At().Iterator(nil)) + } + require.NoError(t, seriesSet.Err()) + require.Len(t, iterators, 1) + iterator := iterators[0] + + // Start OOO head compaction. + compactionComplete := atomic.NewBool(false) + go func() { + defer compactionComplete.Store(true) + + require.NoError(t, db.CompactOOOHead(ctx)) + require.Equal(t, float64(1), prom_testutil.ToFloat64(db.Head().metrics.chunksRemoved)) + }() + + // Give CompactOOOHead time to start work. + // If it does not wait for the querier to be closed, then the query will return incorrect results or fail. + time.Sleep(time.Second) + require.False(t, compactionComplete.Load(), "compaction completed before reading chunks or closing querier") + + // Check that we can still successfully read all samples. + samplesRead := 0 + for iterator.Next() { + samplesRead += iterator.At().Chunk.NumSamples() + } + + require.NoError(t, iterator.Err()) + require.Equal(t, samplesWritten, samplesRead) + + require.False(t, compactionComplete.Load(), "compaction completed before closing querier") + require.NoError(t, querier.Close()) + require.Eventually(t, compactionComplete.Load, time.Second, 10*time.Millisecond, "compaction should complete after querier was closed") +} + +func TestOOOWALWrite(t *testing.T) { + minutes := func(m int64) int64 { return m * time.Minute.Milliseconds() } + + s := labels.NewSymbolTable() + scratchBuilder1 := labels.NewScratchBuilderWithSymbolTable(s, 1) + scratchBuilder1.Add("l", "v1") + s1 := scratchBuilder1.Labels() + scratchBuilder2 := labels.NewScratchBuilderWithSymbolTable(s, 1) + scratchBuilder2.Add("l", "v2") + s2 := scratchBuilder2.Labels() + + scenarios := map[string]struct { + appendSample func(app storage.Appender, l labels.Labels, mins int64) (storage.SeriesRef, error) + expectedOOORecords []any + expectedInORecords []any + }{ + "float": { + appendSample: func(app storage.Appender, l labels.Labels, mins int64) (storage.SeriesRef, error) { + seriesRef, err := app.Append(0, l, minutes(mins), float64(mins)) + require.NoError(t, err) + return seriesRef, nil + }, + expectedOOORecords: []any{ + // The MmapRef in this are not hand calculated, and instead taken from the test run. + // What is important here is the order of records, and that MmapRef increases for each record. + []record.RefMmapMarker{ + {Ref: 1}, + }, + []record.RefSample{ + {Ref: 1, T: minutes(40), V: 40}, + }, + + []record.RefMmapMarker{ + {Ref: 2}, + }, + []record.RefSample{ + {Ref: 2, T: minutes(42), V: 42}, + }, + + []record.RefSample{ + {Ref: 2, T: minutes(45), V: 45}, + {Ref: 1, T: minutes(35), V: 35}, + }, + []record.RefMmapMarker{ // 3rd sample, hence m-mapped. + {Ref: 1, MmapRef: 0x100000000 + 8}, + }, + []record.RefSample{ + {Ref: 1, T: minutes(36), V: 36}, + {Ref: 1, T: minutes(37), V: 37}, + }, + + []record.RefMmapMarker{ // 3rd sample, hence m-mapped. + {Ref: 1, MmapRef: 0x100000000 + 58}, + }, + []record.RefSample{ // Does not contain the in-order sample here. + {Ref: 1, T: minutes(50), V: 50}, + }, + + // Single commit but multiple OOO records. + []record.RefMmapMarker{ + {Ref: 2, MmapRef: 0x100000000 + 107}, + }, + []record.RefSample{ + {Ref: 2, T: minutes(50), V: 50}, + {Ref: 2, T: minutes(51), V: 51}, + }, + []record.RefMmapMarker{ + {Ref: 2, MmapRef: 0x100000000 + 156}, + }, + []record.RefSample{ + {Ref: 2, T: minutes(52), V: 52}, + {Ref: 2, T: minutes(53), V: 53}, + }, + }, + expectedInORecords: []any{ + []record.RefSeries{ + {Ref: 1, Labels: s1}, + {Ref: 2, Labels: s2}, + }, + []record.RefSample{ + {Ref: 1, T: minutes(60), V: 60}, + {Ref: 2, T: minutes(60), V: 60}, + }, + []record.RefSample{ + {Ref: 1, T: minutes(40), V: 40}, + }, + []record.RefSample{ + {Ref: 2, T: minutes(42), V: 42}, + }, + []record.RefSample{ + {Ref: 2, T: minutes(45), V: 45}, + {Ref: 1, T: minutes(35), V: 35}, + {Ref: 1, T: minutes(36), V: 36}, + {Ref: 1, T: minutes(37), V: 37}, + }, + []record.RefSample{ // Contains both in-order and ooo sample. + {Ref: 1, T: minutes(50), V: 50}, + {Ref: 2, T: minutes(65), V: 65}, + }, + []record.RefSample{ + {Ref: 2, T: minutes(50), V: 50}, + {Ref: 2, T: minutes(51), V: 51}, + {Ref: 2, T: minutes(52), V: 52}, + {Ref: 2, T: minutes(53), V: 53}, + }, + }, + }, + "integer histogram": { + appendSample: func(app storage.Appender, l labels.Labels, mins int64) (storage.SeriesRef, error) { + seriesRef, err := app.AppendHistogram(0, l, minutes(mins), tsdbutil.GenerateTestHistogram(mins), nil) + require.NoError(t, err) + return seriesRef, nil + }, + expectedOOORecords: []any{ + // The MmapRef in this are not hand calculated, and instead taken from the test run. + // What is important here is the order of records, and that MmapRef increases for each record. + []record.RefMmapMarker{ + {Ref: 1}, + }, + []record.RefHistogramSample{ + {Ref: 1, T: minutes(40), H: tsdbutil.GenerateTestHistogram(40)}, + }, + + []record.RefMmapMarker{ + {Ref: 2}, + }, + []record.RefHistogramSample{ + {Ref: 2, T: minutes(42), H: tsdbutil.GenerateTestHistogram(42)}, + }, + + []record.RefHistogramSample{ + {Ref: 2, T: minutes(45), H: tsdbutil.GenerateTestHistogram(45)}, + {Ref: 1, T: minutes(35), H: tsdbutil.GenerateTestHistogram(35)}, + }, + []record.RefMmapMarker{ // 3rd sample, hence m-mapped. + {Ref: 1, MmapRef: 0x100000000 + 8}, + }, + []record.RefHistogramSample{ + {Ref: 1, T: minutes(36), H: tsdbutil.GenerateTestHistogram(36)}, + {Ref: 1, T: minutes(37), H: tsdbutil.GenerateTestHistogram(37)}, + }, + + []record.RefMmapMarker{ // 3rd sample, hence m-mapped. + {Ref: 1, MmapRef: 0x100000000 + 89}, + }, + []record.RefHistogramSample{ // Does not contain the in-order sample here. + {Ref: 1, T: minutes(50), H: tsdbutil.GenerateTestHistogram(50)}, + }, + + // Single commit but multiple OOO records. + []record.RefMmapMarker{ + {Ref: 2, MmapRef: 0x100000000 + 172}, + }, + []record.RefHistogramSample{ + {Ref: 2, T: minutes(50), H: tsdbutil.GenerateTestHistogram(50)}, + {Ref: 2, T: minutes(51), H: tsdbutil.GenerateTestHistogram(51)}, + }, + []record.RefMmapMarker{ + {Ref: 2, MmapRef: 0x100000000 + 257}, + }, + []record.RefHistogramSample{ + {Ref: 2, T: minutes(52), H: tsdbutil.GenerateTestHistogram(52)}, + {Ref: 2, T: minutes(53), H: tsdbutil.GenerateTestHistogram(53)}, + }, + }, + expectedInORecords: []any{ + []record.RefSeries{ + {Ref: 1, Labels: s1}, + {Ref: 2, Labels: s2}, + }, + []record.RefHistogramSample{ + {Ref: 1, T: minutes(60), H: tsdbutil.GenerateTestHistogram(60)}, + {Ref: 2, T: minutes(60), H: tsdbutil.GenerateTestHistogram(60)}, + }, + []record.RefHistogramSample{ + {Ref: 1, T: minutes(40), H: tsdbutil.GenerateTestHistogram(40)}, + }, + []record.RefHistogramSample{ + {Ref: 2, T: minutes(42), H: tsdbutil.GenerateTestHistogram(42)}, + }, + []record.RefHistogramSample{ + {Ref: 2, T: minutes(45), H: tsdbutil.GenerateTestHistogram(45)}, + {Ref: 1, T: minutes(35), H: tsdbutil.GenerateTestHistogram(35)}, + {Ref: 1, T: minutes(36), H: tsdbutil.GenerateTestHistogram(36)}, + {Ref: 1, T: minutes(37), H: tsdbutil.GenerateTestHistogram(37)}, + }, + []record.RefHistogramSample{ // Contains both in-order and ooo sample. + {Ref: 1, T: minutes(50), H: tsdbutil.GenerateTestHistogram(50)}, + {Ref: 2, T: minutes(65), H: tsdbutil.GenerateTestHistogram(65)}, + }, + []record.RefHistogramSample{ + {Ref: 2, T: minutes(50), H: tsdbutil.GenerateTestHistogram(50)}, + {Ref: 2, T: minutes(51), H: tsdbutil.GenerateTestHistogram(51)}, + {Ref: 2, T: minutes(52), H: tsdbutil.GenerateTestHistogram(52)}, + {Ref: 2, T: minutes(53), H: tsdbutil.GenerateTestHistogram(53)}, + }, + }, + }, + "float histogram": { + appendSample: func(app storage.Appender, l labels.Labels, mins int64) (storage.SeriesRef, error) { + seriesRef, err := app.AppendHistogram(0, l, minutes(mins), nil, tsdbutil.GenerateTestFloatHistogram(mins)) + require.NoError(t, err) + return seriesRef, nil + }, + expectedOOORecords: []any{ + // The MmapRef in this are not hand calculated, and instead taken from the test run. + // What is important here is the order of records, and that MmapRef increases for each record. + []record.RefMmapMarker{ + {Ref: 1}, + }, + []record.RefFloatHistogramSample{ + {Ref: 1, T: minutes(40), FH: tsdbutil.GenerateTestFloatHistogram(40)}, + }, + + []record.RefMmapMarker{ + {Ref: 2}, + }, + []record.RefFloatHistogramSample{ + {Ref: 2, T: minutes(42), FH: tsdbutil.GenerateTestFloatHistogram(42)}, + }, + + []record.RefFloatHistogramSample{ + {Ref: 2, T: minutes(45), FH: tsdbutil.GenerateTestFloatHistogram(45)}, + {Ref: 1, T: minutes(35), FH: tsdbutil.GenerateTestFloatHistogram(35)}, + }, + []record.RefMmapMarker{ // 3rd sample, hence m-mapped. + {Ref: 1, MmapRef: 0x100000000 + 8}, + }, + []record.RefFloatHistogramSample{ + {Ref: 1, T: minutes(36), FH: tsdbutil.GenerateTestFloatHistogram(36)}, + {Ref: 1, T: minutes(37), FH: tsdbutil.GenerateTestFloatHistogram(37)}, + }, + + []record.RefMmapMarker{ // 3rd sample, hence m-mapped. + {Ref: 1, MmapRef: 0x100000000 + 177}, + }, + []record.RefFloatHistogramSample{ // Does not contain the in-order sample here. + {Ref: 1, T: minutes(50), FH: tsdbutil.GenerateTestFloatHistogram(50)}, + }, + + // Single commit but multiple OOO records. + []record.RefMmapMarker{ + {Ref: 2, MmapRef: 0x100000000 + 348}, + }, + []record.RefFloatHistogramSample{ + {Ref: 2, T: minutes(50), FH: tsdbutil.GenerateTestFloatHistogram(50)}, + {Ref: 2, T: minutes(51), FH: tsdbutil.GenerateTestFloatHistogram(51)}, + }, + []record.RefMmapMarker{ + {Ref: 2, MmapRef: 0x100000000 + 521}, + }, + []record.RefFloatHistogramSample{ + {Ref: 2, T: minutes(52), FH: tsdbutil.GenerateTestFloatHistogram(52)}, + {Ref: 2, T: minutes(53), FH: tsdbutil.GenerateTestFloatHistogram(53)}, + }, + }, + expectedInORecords: []any{ + []record.RefSeries{ + {Ref: 1, Labels: s1}, + {Ref: 2, Labels: s2}, + }, + []record.RefFloatHistogramSample{ + {Ref: 1, T: minutes(60), FH: tsdbutil.GenerateTestFloatHistogram(60)}, + {Ref: 2, T: minutes(60), FH: tsdbutil.GenerateTestFloatHistogram(60)}, + }, + []record.RefFloatHistogramSample{ + {Ref: 1, T: minutes(40), FH: tsdbutil.GenerateTestFloatHistogram(40)}, + }, + []record.RefFloatHistogramSample{ + {Ref: 2, T: minutes(42), FH: tsdbutil.GenerateTestFloatHistogram(42)}, + }, + []record.RefFloatHistogramSample{ + {Ref: 2, T: minutes(45), FH: tsdbutil.GenerateTestFloatHistogram(45)}, + {Ref: 1, T: minutes(35), FH: tsdbutil.GenerateTestFloatHistogram(35)}, + {Ref: 1, T: minutes(36), FH: tsdbutil.GenerateTestFloatHistogram(36)}, + {Ref: 1, T: minutes(37), FH: tsdbutil.GenerateTestFloatHistogram(37)}, + }, + []record.RefFloatHistogramSample{ // Contains both in-order and ooo sample. + {Ref: 1, T: minutes(50), FH: tsdbutil.GenerateTestFloatHistogram(50)}, + {Ref: 2, T: minutes(65), FH: tsdbutil.GenerateTestFloatHistogram(65)}, + }, + []record.RefFloatHistogramSample{ + {Ref: 2, T: minutes(50), FH: tsdbutil.GenerateTestFloatHistogram(50)}, + {Ref: 2, T: minutes(51), FH: tsdbutil.GenerateTestFloatHistogram(51)}, + {Ref: 2, T: minutes(52), FH: tsdbutil.GenerateTestFloatHistogram(52)}, + {Ref: 2, T: minutes(53), FH: tsdbutil.GenerateTestFloatHistogram(53)}, + }, + }, + }, + "custom buckets histogram": { + appendSample: func(app storage.Appender, l labels.Labels, mins int64) (storage.SeriesRef, error) { + seriesRef, err := app.AppendHistogram(0, l, minutes(mins), tsdbutil.GenerateTestCustomBucketsHistogram(mins), nil) + require.NoError(t, err) + return seriesRef, nil + }, + expectedOOORecords: []any{ + // The MmapRef in this are not hand calculated, and instead taken from the test run. + // What is important here is the order of records, and that MmapRef increases for each record. + []record.RefMmapMarker{ + {Ref: 1}, + }, + []record.RefHistogramSample{ + {Ref: 1, T: minutes(40), H: tsdbutil.GenerateTestCustomBucketsHistogram(40)}, + }, + + []record.RefMmapMarker{ + {Ref: 2}, + }, + []record.RefHistogramSample{ + {Ref: 2, T: minutes(42), H: tsdbutil.GenerateTestCustomBucketsHistogram(42)}, + }, + + []record.RefHistogramSample{ + {Ref: 2, T: minutes(45), H: tsdbutil.GenerateTestCustomBucketsHistogram(45)}, + {Ref: 1, T: minutes(35), H: tsdbutil.GenerateTestCustomBucketsHistogram(35)}, + }, + []record.RefMmapMarker{ // 3rd sample, hence m-mapped. + {Ref: 1, MmapRef: 0x100000000 + 8}, + }, + []record.RefHistogramSample{ + {Ref: 1, T: minutes(36), H: tsdbutil.GenerateTestCustomBucketsHistogram(36)}, + {Ref: 1, T: minutes(37), H: tsdbutil.GenerateTestCustomBucketsHistogram(37)}, + }, + + []record.RefMmapMarker{ // 3rd sample, hence m-mapped. + {Ref: 1, MmapRef: 0x100000000 + 82}, + }, + []record.RefHistogramSample{ // Does not contain the in-order sample here. + {Ref: 1, T: minutes(50), H: tsdbutil.GenerateTestCustomBucketsHistogram(50)}, + }, + + // Single commit but multiple OOO records. + []record.RefMmapMarker{ + {Ref: 2, MmapRef: 0x100000000 + 160}, + }, + []record.RefHistogramSample{ + {Ref: 2, T: minutes(50), H: tsdbutil.GenerateTestCustomBucketsHistogram(50)}, + {Ref: 2, T: minutes(51), H: tsdbutil.GenerateTestCustomBucketsHistogram(51)}, + }, + []record.RefMmapMarker{ + {Ref: 2, MmapRef: 0x100000000 + 239}, + }, + []record.RefHistogramSample{ + {Ref: 2, T: minutes(52), H: tsdbutil.GenerateTestCustomBucketsHistogram(52)}, + {Ref: 2, T: minutes(53), H: tsdbutil.GenerateTestCustomBucketsHistogram(53)}, + }, + }, + expectedInORecords: []any{ + []record.RefSeries{ + {Ref: 1, Labels: s1}, + {Ref: 2, Labels: s2}, + }, + []record.RefHistogramSample{ + {Ref: 1, T: minutes(60), H: tsdbutil.GenerateTestCustomBucketsHistogram(60)}, + {Ref: 2, T: minutes(60), H: tsdbutil.GenerateTestCustomBucketsHistogram(60)}, + }, + []record.RefHistogramSample{ + {Ref: 1, T: minutes(40), H: tsdbutil.GenerateTestCustomBucketsHistogram(40)}, + }, + []record.RefHistogramSample{ + {Ref: 2, T: minutes(42), H: tsdbutil.GenerateTestCustomBucketsHistogram(42)}, + }, + []record.RefHistogramSample{ + {Ref: 2, T: minutes(45), H: tsdbutil.GenerateTestCustomBucketsHistogram(45)}, + {Ref: 1, T: minutes(35), H: tsdbutil.GenerateTestCustomBucketsHistogram(35)}, + {Ref: 1, T: minutes(36), H: tsdbutil.GenerateTestCustomBucketsHistogram(36)}, + {Ref: 1, T: minutes(37), H: tsdbutil.GenerateTestCustomBucketsHistogram(37)}, + }, + []record.RefHistogramSample{ // Contains both in-order and ooo sample. + {Ref: 1, T: minutes(50), H: tsdbutil.GenerateTestCustomBucketsHistogram(50)}, + {Ref: 2, T: minutes(65), H: tsdbutil.GenerateTestCustomBucketsHistogram(65)}, + }, + []record.RefHistogramSample{ + {Ref: 2, T: minutes(50), H: tsdbutil.GenerateTestCustomBucketsHistogram(50)}, + {Ref: 2, T: minutes(51), H: tsdbutil.GenerateTestCustomBucketsHistogram(51)}, + {Ref: 2, T: minutes(52), H: tsdbutil.GenerateTestCustomBucketsHistogram(52)}, + {Ref: 2, T: minutes(53), H: tsdbutil.GenerateTestCustomBucketsHistogram(53)}, + }, + }, + }, + "custom buckets float histogram": { + appendSample: func(app storage.Appender, l labels.Labels, mins int64) (storage.SeriesRef, error) { + seriesRef, err := app.AppendHistogram(0, l, minutes(mins), nil, tsdbutil.GenerateTestCustomBucketsFloatHistogram(mins)) + require.NoError(t, err) + return seriesRef, nil + }, + expectedOOORecords: []any{ + // The MmapRef in this are not hand calculated, and instead taken from the test run. + // What is important here is the order of records, and that MmapRef increases for each record. + []record.RefMmapMarker{ + {Ref: 1}, + }, + []record.RefFloatHistogramSample{ + {Ref: 1, T: minutes(40), FH: tsdbutil.GenerateTestCustomBucketsFloatHistogram(40)}, + }, + + []record.RefMmapMarker{ + {Ref: 2}, + }, + []record.RefFloatHistogramSample{ + {Ref: 2, T: minutes(42), FH: tsdbutil.GenerateTestCustomBucketsFloatHistogram(42)}, + }, + + []record.RefFloatHistogramSample{ + {Ref: 2, T: minutes(45), FH: tsdbutil.GenerateTestCustomBucketsFloatHistogram(45)}, + {Ref: 1, T: minutes(35), FH: tsdbutil.GenerateTestCustomBucketsFloatHistogram(35)}, + }, + []record.RefMmapMarker{ // 3rd sample, hence m-mapped. + {Ref: 1, MmapRef: 0x100000000 + 8}, + }, + []record.RefFloatHistogramSample{ + {Ref: 1, T: minutes(36), FH: tsdbutil.GenerateTestCustomBucketsFloatHistogram(36)}, + {Ref: 1, T: minutes(37), FH: tsdbutil.GenerateTestCustomBucketsFloatHistogram(37)}, + }, + + []record.RefMmapMarker{ // 3rd sample, hence m-mapped. + {Ref: 1, MmapRef: 0x100000000 + 134}, + }, + []record.RefFloatHistogramSample{ // Does not contain the in-order sample here. + {Ref: 1, T: minutes(50), FH: tsdbutil.GenerateTestCustomBucketsFloatHistogram(50)}, + }, + + // Single commit but multiple OOO records. + []record.RefMmapMarker{ + {Ref: 2, MmapRef: 0x100000000 + 263}, + }, + []record.RefFloatHistogramSample{ + {Ref: 2, T: minutes(50), FH: tsdbutil.GenerateTestCustomBucketsFloatHistogram(50)}, + {Ref: 2, T: minutes(51), FH: tsdbutil.GenerateTestCustomBucketsFloatHistogram(51)}, + }, + []record.RefMmapMarker{ + {Ref: 2, MmapRef: 0x100000000 + 393}, + }, + []record.RefFloatHistogramSample{ + {Ref: 2, T: minutes(52), FH: tsdbutil.GenerateTestCustomBucketsFloatHistogram(52)}, + {Ref: 2, T: minutes(53), FH: tsdbutil.GenerateTestCustomBucketsFloatHistogram(53)}, + }, + }, + expectedInORecords: []any{ + []record.RefSeries{ + {Ref: 1, Labels: s1}, + {Ref: 2, Labels: s2}, + }, + []record.RefFloatHistogramSample{ + {Ref: 1, T: minutes(60), FH: tsdbutil.GenerateTestCustomBucketsFloatHistogram(60)}, + {Ref: 2, T: minutes(60), FH: tsdbutil.GenerateTestCustomBucketsFloatHistogram(60)}, + }, + []record.RefFloatHistogramSample{ + {Ref: 1, T: minutes(40), FH: tsdbutil.GenerateTestCustomBucketsFloatHistogram(40)}, + }, + []record.RefFloatHistogramSample{ + {Ref: 2, T: minutes(42), FH: tsdbutil.GenerateTestCustomBucketsFloatHistogram(42)}, + }, + []record.RefFloatHistogramSample{ + {Ref: 2, T: minutes(45), FH: tsdbutil.GenerateTestCustomBucketsFloatHistogram(45)}, + {Ref: 1, T: minutes(35), FH: tsdbutil.GenerateTestCustomBucketsFloatHistogram(35)}, + {Ref: 1, T: minutes(36), FH: tsdbutil.GenerateTestCustomBucketsFloatHistogram(36)}, + {Ref: 1, T: minutes(37), FH: tsdbutil.GenerateTestCustomBucketsFloatHistogram(37)}, + }, + []record.RefFloatHistogramSample{ // Contains both in-order and ooo sample. + {Ref: 1, T: minutes(50), FH: tsdbutil.GenerateTestCustomBucketsFloatHistogram(50)}, + {Ref: 2, T: minutes(65), FH: tsdbutil.GenerateTestCustomBucketsFloatHistogram(65)}, + }, + []record.RefFloatHistogramSample{ + {Ref: 2, T: minutes(50), FH: tsdbutil.GenerateTestCustomBucketsFloatHistogram(50)}, + {Ref: 2, T: minutes(51), FH: tsdbutil.GenerateTestCustomBucketsFloatHistogram(51)}, + {Ref: 2, T: minutes(52), FH: tsdbutil.GenerateTestCustomBucketsFloatHistogram(52)}, + {Ref: 2, T: minutes(53), FH: tsdbutil.GenerateTestCustomBucketsFloatHistogram(53)}, + }, + }, + }, + } + for name, scenario := range scenarios { + t.Run(name, func(t *testing.T) { + testOOOWALWrite(t, scenario.appendSample, scenario.expectedOOORecords, scenario.expectedInORecords) + }) + } +} + +func testOOOWALWrite(t *testing.T, + appendSample func(app storage.Appender, l labels.Labels, mins int64) (storage.SeriesRef, error), + expectedOOORecords []any, + expectedInORecords []any, +) { + opts := DefaultOptions() + opts.OutOfOrderCapMax = 2 + opts.OutOfOrderTimeWindow = 30 * time.Minute.Milliseconds() + db := newTestDB(t, withOpts(opts)) + + s1, s2 := labels.FromStrings("l", "v1"), labels.FromStrings("l", "v2") + + // Ingest sample at 1h. + app := db.Appender(context.Background()) + appendSample(app, s1, 60) + appendSample(app, s2, 60) + require.NoError(t, app.Commit()) + + // OOO for s1. + app = db.Appender(context.Background()) + appendSample(app, s1, 40) + require.NoError(t, app.Commit()) + + // OOO for s2. + app = db.Appender(context.Background()) + appendSample(app, s2, 42) + require.NoError(t, app.Commit()) + + // OOO for both s1 and s2 in the same commit. + app = db.Appender(context.Background()) + appendSample(app, s2, 45) + appendSample(app, s1, 35) + appendSample(app, s1, 36) // m-maps. + appendSample(app, s1, 37) + require.NoError(t, app.Commit()) + + // OOO for s1 but not for s2 in the same commit. + app = db.Appender(context.Background()) + appendSample(app, s1, 50) // m-maps. + appendSample(app, s2, 65) + require.NoError(t, app.Commit()) + + // Single commit has 2 times m-mapping and more samples after m-map. + app = db.Appender(context.Background()) + appendSample(app, s2, 50) // m-maps. + appendSample(app, s2, 51) + appendSample(app, s2, 52) // m-maps. + appendSample(app, s2, 53) + require.NoError(t, app.Commit()) + + getRecords := func(walDir string) []any { + sr, err := wlog.NewSegmentsReader(walDir) + require.NoError(t, err) + r := wlog.NewReader(sr) + defer func() { + require.NoError(t, sr.Close()) + }() + + var records []any + dec := record.NewDecoder(nil, promslog.NewNopLogger()) + for r.Next() { + rec := r.Record() + switch typ := dec.Type(rec); typ { + case record.Series: + series, err := dec.Series(rec, nil) + require.NoError(t, err) + records = append(records, series) + case record.Samples: + samples, err := dec.Samples(rec, nil) + require.NoError(t, err) + records = append(records, samples) + case record.MmapMarkers: + markers, err := dec.MmapMarkers(rec, nil) + require.NoError(t, err) + records = append(records, markers) + case record.HistogramSamples, record.CustomBucketsHistogramSamples: + histogramSamples, err := dec.HistogramSamples(rec, nil) + require.NoError(t, err) + records = append(records, histogramSamples) + case record.FloatHistogramSamples, record.CustomBucketsFloatHistogramSamples: + floatHistogramSamples, err := dec.FloatHistogramSamples(rec, nil) + require.NoError(t, err) + records = append(records, floatHistogramSamples) + default: + t.Fatalf("got a WAL record that is not series or samples: %v", typ) + } + } + + return records + } + + // The normal WAL. + actRecs := getRecords(path.Join(db.Dir(), "wal")) + require.Equal(t, expectedInORecords, actRecs) + + // The WBL. + actRecs = getRecords(path.Join(db.Dir(), wlog.WblDirName)) + require.Equal(t, expectedOOORecords, actRecs) +} + +// Tests https://github.com/prometheus/prometheus/issues/10291#issuecomment-1044373110. +func TestDBPanicOnMmappingHeadChunk(t *testing.T) { + var err error + ctx := context.Background() + + db := newTestDB(t) + db.DisableCompactions() + + // Choosing scrape interval of 45s to have chunk larger than 1h. + itvl := int64(45 * time.Second / time.Millisecond) + + lastTs := int64(0) + addSamples := func(numSamples int) { + app := db.Appender(context.Background()) + var ref storage.SeriesRef + lbls := labels.FromStrings("__name__", "testing", "foo", "bar") + for i := range numSamples { + ref, err = app.Append(ref, lbls, lastTs, float64(lastTs)) + require.NoError(t, err) + lastTs += itvl + if i%10 == 0 { + require.NoError(t, app.Commit()) + app = db.Appender(context.Background()) + } + } + require.NoError(t, app.Commit()) + } + + // Ingest samples upto 2h50m to make the head "about to compact". + numSamples := int(170*time.Minute/time.Millisecond) / int(itvl) + addSamples(numSamples) + + require.Empty(t, db.Blocks()) + require.NoError(t, db.Compact(ctx)) + require.Empty(t, db.Blocks()) + + // Restarting. + require.NoError(t, db.Close()) + + db = newTestDB(t, withDir(db.Dir())) + db.DisableCompactions() + + // Ingest samples upto 20m more to make the head compact. + numSamples = int(20*time.Minute/time.Millisecond) / int(itvl) + addSamples(numSamples) + + require.Empty(t, db.Blocks()) + require.NoError(t, db.Compact(ctx)) + require.Len(t, db.Blocks(), 1) + + // More samples to m-map and panic. + numSamples = int(120*time.Minute/time.Millisecond) / int(itvl) + addSamples(numSamples) + + require.NoError(t, db.Close()) +} + +func TestMetadataInWAL(t *testing.T) { + updateMetadata := func(t *testing.T, app storage.Appender, s labels.Labels, m metadata.Metadata) { + _, err := app.UpdateMetadata(0, s, m) + require.NoError(t, err) + } + + db := newTestDB(t) + ctx := context.Background() + + // Add some series so we can append metadata to them. + app := db.Appender(ctx) + s1 := labels.FromStrings("a", "b") + s2 := labels.FromStrings("c", "d") + s3 := labels.FromStrings("e", "f") + s4 := labels.FromStrings("g", "h") + + for _, s := range []labels.Labels{s1, s2, s3, s4} { + _, err := app.Append(0, s, 0, 0) + require.NoError(t, err) + } + require.NoError(t, app.Commit()) + + // Add a first round of metadata to the first three series. + // Re-take the Appender, as the previous Commit will have it closed. + m1 := metadata.Metadata{Type: "gauge", Unit: "unit_1", Help: "help_1"} + m2 := metadata.Metadata{Type: "gauge", Unit: "unit_2", Help: "help_2"} + m3 := metadata.Metadata{Type: "gauge", Unit: "unit_3", Help: "help_3"} + app = db.Appender(ctx) + updateMetadata(t, app, s1, m1) + updateMetadata(t, app, s2, m2) + updateMetadata(t, app, s3, m3) + require.NoError(t, app.Commit()) + + // Add a replicated metadata entry to the first series, + // a completely new metadata entry for the fourth series, + // and a changed metadata entry to the second series. + m4 := metadata.Metadata{Type: "counter", Unit: "unit_4", Help: "help_4"} + m5 := metadata.Metadata{Type: "counter", Unit: "unit_5", Help: "help_5"} + app = db.Appender(ctx) + updateMetadata(t, app, s1, m1) + updateMetadata(t, app, s4, m4) + updateMetadata(t, app, s2, m5) + require.NoError(t, app.Commit()) + + // Read the WAL to see if the disk storage format is correct. + recs := readTestWAL(t, path.Join(db.Dir(), "wal")) + var gotMetadataBlocks [][]record.RefMetadata + for _, rec := range recs { + if mr, ok := rec.([]record.RefMetadata); ok { + gotMetadataBlocks = append(gotMetadataBlocks, mr) + } + } + + expectedMetadata := []record.RefMetadata{ + {Ref: 1, Type: record.GetMetricType(m1.Type), Unit: m1.Unit, Help: m1.Help}, + {Ref: 2, Type: record.GetMetricType(m2.Type), Unit: m2.Unit, Help: m2.Help}, + {Ref: 3, Type: record.GetMetricType(m3.Type), Unit: m3.Unit, Help: m3.Help}, + {Ref: 4, Type: record.GetMetricType(m4.Type), Unit: m4.Unit, Help: m4.Help}, + {Ref: 2, Type: record.GetMetricType(m5.Type), Unit: m5.Unit, Help: m5.Help}, + } + require.Len(t, gotMetadataBlocks, 2) + require.Equal(t, expectedMetadata[:3], gotMetadataBlocks[0]) + require.Equal(t, expectedMetadata[3:], gotMetadataBlocks[1]) +} + +func TestMetadataCheckpointingOnlyKeepsLatestEntry(t *testing.T) { + updateMetadata := func(t *testing.T, app storage.Appender, s labels.Labels, m metadata.Metadata) { + _, err := app.UpdateMetadata(0, s, m) + require.NoError(t, err) + } + + ctx := context.Background() + numSamples := 10000 + hb, w := newTestHead(t, int64(numSamples)*10, compression.None, false) + + // Add some series so we can append metadata to them. + app := hb.Appender(ctx) + s1 := labels.FromStrings("a", "b") + s2 := labels.FromStrings("c", "d") + s3 := labels.FromStrings("e", "f") + s4 := labels.FromStrings("g", "h") + + for _, s := range []labels.Labels{s1, s2, s3, s4} { + _, err := app.Append(0, s, 0, 0) + require.NoError(t, err) + } + require.NoError(t, app.Commit()) + + // Add a first round of metadata to the first three series. + // Re-take the Appender, as the previous Commit will have it closed. + m1 := metadata.Metadata{Type: "gauge", Unit: "unit_1", Help: "help_1"} + m2 := metadata.Metadata{Type: "gauge", Unit: "unit_2", Help: "help_2"} + m3 := metadata.Metadata{Type: "gauge", Unit: "unit_3", Help: "help_3"} + m4 := metadata.Metadata{Type: "gauge", Unit: "unit_4", Help: "help_4"} + app = hb.Appender(ctx) + updateMetadata(t, app, s1, m1) + updateMetadata(t, app, s2, m2) + updateMetadata(t, app, s3, m3) + updateMetadata(t, app, s4, m4) + require.NoError(t, app.Commit()) + + // Update metadata for first series. + m5 := metadata.Metadata{Type: "counter", Unit: "unit_5", Help: "help_5"} + app = hb.Appender(ctx) + updateMetadata(t, app, s1, m5) + require.NoError(t, app.Commit()) + + // Switch back-and-forth metadata for second series. + // Since it ended on a new metadata record, we expect a single new entry. + m6 := metadata.Metadata{Type: "counter", Unit: "unit_6", Help: "help_6"} + + app = hb.Appender(ctx) + updateMetadata(t, app, s2, m6) + require.NoError(t, app.Commit()) + + app = hb.Appender(ctx) + updateMetadata(t, app, s2, m2) + require.NoError(t, app.Commit()) + + app = hb.Appender(ctx) + updateMetadata(t, app, s2, m6) + require.NoError(t, app.Commit()) + + app = hb.Appender(ctx) + updateMetadata(t, app, s2, m2) + require.NoError(t, app.Commit()) + + app = hb.Appender(ctx) + updateMetadata(t, app, s2, m6) + require.NoError(t, app.Commit()) + + // Let's create a checkpoint. + first, last, err := wlog.Segments(w.Dir()) + require.NoError(t, err) + keep := func(id chunks.HeadSeriesRef) bool { + return id != 3 + } + _, err = wlog.Checkpoint(promslog.NewNopLogger(), w, first, last-1, keep, 0) + require.NoError(t, err) + + // Confirm there's been a checkpoint. + cdir, _, err := wlog.LastCheckpoint(w.Dir()) + require.NoError(t, err) + + // Read in checkpoint and WAL. + recs := readTestWAL(t, cdir) + var gotMetadataBlocks [][]record.RefMetadata + for _, rec := range recs { + if mr, ok := rec.([]record.RefMetadata); ok { + gotMetadataBlocks = append(gotMetadataBlocks, mr) + } + } + + // There should only be 1 metadata block present, with only the latest + // metadata kept around. + wantMetadata := []record.RefMetadata{ + {Ref: 1, Type: record.GetMetricType(m5.Type), Unit: m5.Unit, Help: m5.Help}, + {Ref: 2, Type: record.GetMetricType(m6.Type), Unit: m6.Unit, Help: m6.Help}, + {Ref: 4, Type: record.GetMetricType(m4.Type), Unit: m4.Unit, Help: m4.Help}, + } + require.Len(t, gotMetadataBlocks, 1) + require.Len(t, gotMetadataBlocks[0], 3) + gotMetadataBlock := gotMetadataBlocks[0] + + sort.Slice(gotMetadataBlock, func(i, j int) bool { return gotMetadataBlock[i].Ref < gotMetadataBlock[j].Ref }) + require.Equal(t, wantMetadata, gotMetadataBlock) + require.NoError(t, hb.Close()) +} + +func TestMetadataAssertInMemoryData(t *testing.T) { + updateMetadata := func(t *testing.T, app storage.Appender, s labels.Labels, m metadata.Metadata) { + _, err := app.UpdateMetadata(0, s, m) + require.NoError(t, err) + } + + db := newTestDB(t) + ctx := context.Background() + + // Add some series so we can append metadata to them. + app := db.Appender(ctx) + s1 := labels.FromStrings("a", "b") + s2 := labels.FromStrings("c", "d") + s3 := labels.FromStrings("e", "f") + s4 := labels.FromStrings("g", "h") + + for _, s := range []labels.Labels{s1, s2, s3, s4} { + _, err := app.Append(0, s, 0, 0) + require.NoError(t, err) + } + require.NoError(t, app.Commit()) + + // Add a first round of metadata to the first three series. + // The in-memory data held in the db Head should hold the metadata. + m1 := metadata.Metadata{Type: "gauge", Unit: "unit_1", Help: "help_1"} + m2 := metadata.Metadata{Type: "gauge", Unit: "unit_2", Help: "help_2"} + m3 := metadata.Metadata{Type: "gauge", Unit: "unit_3", Help: "help_3"} + app = db.Appender(ctx) + updateMetadata(t, app, s1, m1) + updateMetadata(t, app, s2, m2) + updateMetadata(t, app, s3, m3) + require.NoError(t, app.Commit()) + + series1 := db.head.series.getByHash(s1.Hash(), s1) + series2 := db.head.series.getByHash(s2.Hash(), s2) + series3 := db.head.series.getByHash(s3.Hash(), s3) + series4 := db.head.series.getByHash(s4.Hash(), s4) + require.Equal(t, *series1.meta, m1) + require.Equal(t, *series2.meta, m2) + require.Equal(t, *series3.meta, m3) + require.Nil(t, series4.meta) + + // Add a replicated metadata entry to the first series, + // a changed metadata entry to the second series, + // and a completely new metadata entry for the fourth series. + // The in-memory data held in the db Head should be correctly updated. + m4 := metadata.Metadata{Type: "counter", Unit: "unit_4", Help: "help_4"} + m5 := metadata.Metadata{Type: "counter", Unit: "unit_5", Help: "help_5"} + app = db.Appender(ctx) + updateMetadata(t, app, s1, m1) + updateMetadata(t, app, s4, m4) + updateMetadata(t, app, s2, m5) + require.NoError(t, app.Commit()) + + series1 = db.head.series.getByHash(s1.Hash(), s1) + series2 = db.head.series.getByHash(s2.Hash(), s2) + series3 = db.head.series.getByHash(s3.Hash(), s3) + series4 = db.head.series.getByHash(s4.Hash(), s4) + require.Equal(t, *series1.meta, m1) + require.Equal(t, *series2.meta, m5) + require.Equal(t, *series3.meta, m3) + require.Equal(t, *series4.meta, m4) + + require.NoError(t, db.Close()) + + // Reopen the DB, replaying the WAL. The Head must have been replayed + // correctly in memory. + db = newTestDB(t, withDir(db.Dir())) + _, err := db.head.wal.Size() + require.NoError(t, err) + + require.Equal(t, *db.head.series.getByHash(s1.Hash(), s1).meta, m1) + require.Equal(t, *db.head.series.getByHash(s2.Hash(), s2).meta, m5) + require.Equal(t, *db.head.series.getByHash(s3.Hash(), s3).meta, m3) + require.Equal(t, *db.head.series.getByHash(s4.Hash(), s4).meta, m4) +} + +// TestMultipleEncodingsCommitOrder mainly serves to demonstrate when happens when committing a batch of samples for the +// same series when there are multiple encodings. With issue #15177 fixed, this now all works as expected. +func TestMultipleEncodingsCommitOrder(t *testing.T) { + opts := DefaultOptions() + opts.OutOfOrderCapMax = 30 + opts.OutOfOrderTimeWindow = 24 * time.Hour.Milliseconds() + + db := newTestDB(t, withOpts(opts)) + db.DisableCompactions() + + series1 := labels.FromStrings("foo", "bar1") + addSample := func(app storage.Appender, ts int64, valType chunkenc.ValueType) chunks.Sample { + if valType == chunkenc.ValFloat { + _, err := app.Append(0, labels.FromStrings("foo", "bar1"), ts, float64(ts)) + require.NoError(t, err) + return sample{t: ts, f: float64(ts)} + } + if valType == chunkenc.ValHistogram { + h := tsdbutil.GenerateTestHistogram(ts) + _, err := app.AppendHistogram(0, labels.FromStrings("foo", "bar1"), ts, h, nil) + require.NoError(t, err) + return sample{t: ts, h: h} + } + fh := tsdbutil.GenerateTestFloatHistogram(ts) + _, err := app.AppendHistogram(0, labels.FromStrings("foo", "bar1"), ts, nil, fh) + require.NoError(t, err) + return sample{t: ts, fh: fh} + } + + verifySamples := func(minT, maxT int64, expSamples []chunks.Sample, oooCount int) { + requireEqualOOOSamples(t, oooCount, db) + + // Verify samples querier. + querier, err := db.Querier(minT, maxT) + require.NoError(t, err) + defer querier.Close() + + seriesSet := query(t, querier, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar1")) + require.Len(t, seriesSet, 1) + gotSamples := seriesSet[series1.String()] + requireEqualSamples(t, series1.String(), expSamples, gotSamples, requireEqualSamplesIgnoreCounterResets) + + // Verify chunks querier. + chunkQuerier, err := db.ChunkQuerier(minT, maxT) + require.NoError(t, err) + defer chunkQuerier.Close() + + chks := queryChunks(t, chunkQuerier, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar1")) + require.NotNil(t, chks[series1.String()]) + require.Len(t, chks, 1) + var gotChunkSamples []chunks.Sample + for _, chunk := range chks[series1.String()] { + it := chunk.Chunk.Iterator(nil) + smpls, err := storage.ExpandSamples(it, newSample) + require.NoError(t, err) + gotChunkSamples = append(gotChunkSamples, smpls...) + require.NoError(t, it.Err()) + } + requireEqualSamples(t, series1.String(), expSamples, gotChunkSamples, requireEqualSamplesIgnoreCounterResets) + } + + var expSamples []chunks.Sample + + // Append samples with different encoding types and then commit them at once. + app := db.Appender(context.Background()) + + for i := 100; i < 105; i++ { + s := addSample(app, int64(i), chunkenc.ValFloat) + expSamples = append(expSamples, s) + } + for i := 110; i < 120; i++ { + s := addSample(app, int64(i), chunkenc.ValHistogram) + expSamples = append(expSamples, s) + } + for i := 120; i < 130; i++ { + s := addSample(app, int64(i), chunkenc.ValFloatHistogram) + expSamples = append(expSamples, s) + } + for i := 140; i < 150; i++ { + s := addSample(app, int64(i), chunkenc.ValFloatHistogram) + expSamples = append(expSamples, s) + } + // These samples will be marked as out-of-order. + for i := 130; i < 135; i++ { + s := addSample(app, int64(i), chunkenc.ValFloat) + expSamples = append(expSamples, s) + } + + require.NoError(t, app.Commit()) + + sort.Slice(expSamples, func(i, j int) bool { + return expSamples[i].T() < expSamples[j].T() + }) + + // oooCount = 5 for the samples 130 to 134. + verifySamples(100, 150, expSamples, 5) + + // Append and commit some in-order histograms by themselves. + app = db.Appender(context.Background()) + for i := 150; i < 160; i++ { + s := addSample(app, int64(i), chunkenc.ValHistogram) + expSamples = append(expSamples, s) + } + require.NoError(t, app.Commit()) + + // oooCount remains at 5. + verifySamples(100, 160, expSamples, 5) + + // Append and commit samples for all encoding types. This time all samples will be treated as OOO because samples + // with newer timestamps have already been committed. + app = db.Appender(context.Background()) + for i := 50; i < 55; i++ { + s := addSample(app, int64(i), chunkenc.ValFloat) + expSamples = append(expSamples, s) + } + for i := 60; i < 70; i++ { + s := addSample(app, int64(i), chunkenc.ValHistogram) + expSamples = append(expSamples, s) + } + for i := 70; i < 75; i++ { + s := addSample(app, int64(i), chunkenc.ValFloat) + expSamples = append(expSamples, s) + } + for i := 80; i < 90; i++ { + s := addSample(app, int64(i), chunkenc.ValFloatHistogram) + expSamples = append(expSamples, s) + } + require.NoError(t, app.Commit()) + + // Sort samples again because OOO samples have been added. + sort.Slice(expSamples, func(i, j int) bool { + return expSamples[i].T() < expSamples[j].T() + }) + + // oooCount = 35 as we've added 30 more OOO samples. + verifySamples(50, 160, expSamples, 35) +} + +// TODO(codesome): test more samples incoming once compaction has started. To verify new samples after the start +// +// are not included in this compaction. +func TestOOOCompaction(t *testing.T) { + for name, scenario := range sampleTypeScenarios { + t.Run(name, func(t *testing.T) { + testOOOCompaction(t, scenario, false) + }) + t.Run(name+"+extra", func(t *testing.T) { + testOOOCompaction(t, scenario, true) + }) + } +} + +func testOOOCompaction(t *testing.T, scenario sampleTypeScenario, addExtraSamples bool) { + ctx := context.Background() + + opts := DefaultOptions() + opts.OutOfOrderCapMax = 30 + opts.OutOfOrderTimeWindow = 300 * time.Minute.Milliseconds() + db := newTestDB(t, withOpts(opts)) + db.DisableCompactions() // We want to manually call it. + + series1 := labels.FromStrings("foo", "bar1") + series2 := labels.FromStrings("foo", "bar2") + + addSample := func(fromMins, toMins int64) { + app := db.Appender(context.Background()) + for m := fromMins; m <= toMins; m++ { + ts := m * time.Minute.Milliseconds() + _, _, err := scenario.appendFunc(app, series1, ts, ts) + require.NoError(t, err) + _, _, err = scenario.appendFunc(app, series2, ts, 2*ts) + require.NoError(t, err) + } + require.NoError(t, app.Commit()) + } + + // Add an in-order samples. + addSample(250, 300) + + // Verify that the in-memory ooo chunk is empty. + checkEmptyOOOChunk := func(lbls labels.Labels) { + ms, created, err := db.head.getOrCreate(lbls.Hash(), lbls, false) + require.NoError(t, err) + require.False(t, created) + require.Nil(t, ms.ooo) + } + checkEmptyOOOChunk(series1) + checkEmptyOOOChunk(series2) + + // Add ooo samples that creates multiple chunks. + // 90 to 300 spans across 3 block ranges: [0, 120), [120, 240), [240, 360) + addSample(90, 300) + // Adding same samples to create overlapping chunks. + // Since the active chunk won't start at 90 again, all the new + // chunks will have different time ranges than the previous chunks. + addSample(90, 300) + + var highest int64 = 300 + + verifyDBSamples := func() { + var series1Samples, series2Samples []chunks.Sample + for _, r := range [][2]int64{{90, 119}, {120, 239}, {240, highest}} { + fromMins, toMins := r[0], r[1] + for m := fromMins; m <= toMins; m++ { + ts := m * time.Minute.Milliseconds() + series1Samples = append(series1Samples, scenario.sampleFunc(ts, ts)) + series2Samples = append(series2Samples, scenario.sampleFunc(ts, 2*ts)) + } + } + expRes := map[string][]chunks.Sample{ + series1.String(): series1Samples, + series2.String(): series2Samples, + } + + q, err := db.Querier(math.MinInt64, math.MaxInt64) + require.NoError(t, err) + + actRes := query(t, q, labels.MustNewMatcher(labels.MatchRegexp, "foo", "bar.*")) + requireEqualSeries(t, expRes, actRes, true) + } + + verifyDBSamples() // Before any compaction. + + // Verify that the in-memory ooo chunk is not empty. + checkNonEmptyOOOChunk := func(lbls labels.Labels) { + ms, created, err := db.head.getOrCreate(lbls.Hash(), lbls, false) + require.NoError(t, err) + require.False(t, created) + require.Positive(t, ms.ooo.oooHeadChunk.chunk.NumSamples()) + require.Len(t, ms.ooo.oooMmappedChunks, 13) // 7 original, 6 duplicate. + } + checkNonEmptyOOOChunk(series1) + checkNonEmptyOOOChunk(series2) + + // No blocks before compaction. + require.Empty(t, db.Blocks()) + + // There is a 0th WBL file. + require.NoError(t, db.head.wbl.Sync()) // syncing to make sure wbl is flushed in windows + files, err := os.ReadDir(db.head.wbl.Dir()) + require.NoError(t, err) + require.Len(t, files, 1) + require.Equal(t, "00000000", files[0].Name()) + f, err := files[0].Info() + require.NoError(t, err) + require.Greater(t, f.Size(), int64(100)) + + if addExtraSamples { + compactOOOHeadTestingCallback = func() { + addSample(90, 120) // Back in time, to generate a new OOO chunk. + addSample(300, 330) // Now some samples after the previous highest timestamp. + addSample(300, 330) // Repeat to generate an OOO chunk at these timestamps. + } + highest = 330 + } + + // OOO compaction happens here. + require.NoError(t, db.CompactOOOHead(ctx)) + + // 3 blocks exist now. [0, 120), [120, 240), [240, 360) + require.Len(t, db.Blocks(), 3) + + verifyDBSamples() // Blocks created out of OOO head now. + + // 0th WBL file will be deleted and 1st will be the only present. + files, err = os.ReadDir(db.head.wbl.Dir()) + require.NoError(t, err) + require.Len(t, files, 1) + require.Equal(t, "00000001", files[0].Name()) + f, err = files[0].Info() + require.NoError(t, err) + + if !addExtraSamples { + require.Equal(t, int64(0), f.Size()) + // OOO stuff should not be present in the Head now. + checkEmptyOOOChunk(series1) + checkEmptyOOOChunk(series2) + } + + verifySamples := func(block *Block, fromMins, toMins int64) { + series1Samples := make([]chunks.Sample, 0, toMins-fromMins+1) + series2Samples := make([]chunks.Sample, 0, toMins-fromMins+1) + for m := fromMins; m <= toMins; m++ { + ts := m * time.Minute.Milliseconds() + series1Samples = append(series1Samples, scenario.sampleFunc(ts, ts)) + series2Samples = append(series2Samples, scenario.sampleFunc(ts, 2*ts)) + } + expRes := map[string][]chunks.Sample{ + series1.String(): series1Samples, + series2.String(): series2Samples, + } + + q, err := NewBlockQuerier(block, math.MinInt64, math.MaxInt64) + require.NoError(t, err) + + actRes := query(t, q, labels.MustNewMatcher(labels.MatchRegexp, "foo", "bar.*")) + requireEqualSeries(t, expRes, actRes, true) + } + + // Checking for expected data in the blocks. + verifySamples(db.Blocks()[0], 90, 119) + verifySamples(db.Blocks()[1], 120, 239) + verifySamples(db.Blocks()[2], 240, 299) + + // There should be a single m-map file. + mmapDir := mmappedChunksDir(db.head.opts.ChunkDirRoot) + files, err = os.ReadDir(mmapDir) + require.NoError(t, err) + require.Len(t, files, 1) + + // Compact the in-order head and expect another block. + // Since this is a forced compaction, this block is not aligned with 2h. + err = db.CompactHead(NewRangeHead(db.head, 250*time.Minute.Milliseconds(), 350*time.Minute.Milliseconds())) + require.NoError(t, err) + require.Len(t, db.Blocks(), 4) // [0, 120), [120, 240), [240, 360), [250, 351) + verifySamples(db.Blocks()[3], 250, highest) + + verifyDBSamples() // Blocks created out of normal and OOO head now. But not merged. + + // The compaction also clears out the old m-map files. Including + // the file that has ooo chunks. + files, err = os.ReadDir(mmapDir) + require.NoError(t, err) + require.Len(t, files, 1) + require.Equal(t, "000001", files[0].Name()) + + // This will merge overlapping block. + require.NoError(t, db.Compact(ctx)) + + require.Len(t, db.Blocks(), 3) // [0, 120), [120, 240), [240, 360) + verifySamples(db.Blocks()[0], 90, 119) + verifySamples(db.Blocks()[1], 120, 239) + verifySamples(db.Blocks()[2], 240, highest) // Merged block. + + verifyDBSamples() // Final state. Blocks from normal and OOO head are merged. +} + +// TestOOOCompactionWithNormalCompaction tests if OOO compaction is performed +// when the normal head's compaction is done. +func TestOOOCompactionWithNormalCompaction(t *testing.T) { + t.Parallel() + for name, scenario := range sampleTypeScenarios { + t.Run(name, func(t *testing.T) { + testOOOCompactionWithNormalCompaction(t, scenario) + }) + } +} + +func testOOOCompactionWithNormalCompaction(t *testing.T, scenario sampleTypeScenario) { + t.Parallel() + ctx := context.Background() + + opts := DefaultOptions() + opts.OutOfOrderCapMax = 30 + opts.OutOfOrderTimeWindow = 300 * time.Minute.Milliseconds() + + db := newTestDB(t, withOpts(opts)) + db.DisableCompactions() // We want to manually call it. + + series1 := labels.FromStrings("foo", "bar1") + series2 := labels.FromStrings("foo", "bar2") + + addSamples := func(fromMins, toMins int64) { + app := db.Appender(context.Background()) + for m := fromMins; m <= toMins; m++ { + ts := m * time.Minute.Milliseconds() + _, _, err := scenario.appendFunc(app, series1, ts, ts) + require.NoError(t, err) + _, _, err = scenario.appendFunc(app, series2, ts, 2*ts) + require.NoError(t, err) + } + require.NoError(t, app.Commit()) + } + + // Add an in-order samples. + addSamples(250, 350) + + // Add ooo samples that will result into a single block. + addSamples(90, 110) + + // Checking that ooo chunk is not empty. + for _, lbls := range []labels.Labels{series1, series2} { + ms, created, err := db.head.getOrCreate(lbls.Hash(), lbls, false) + require.NoError(t, err) + require.False(t, created) + require.Positive(t, ms.ooo.oooHeadChunk.chunk.NumSamples()) + } + + // If the normal Head is not compacted, the OOO head compaction does not take place. + require.NoError(t, db.Compact(ctx)) + require.Empty(t, db.Blocks()) + + // Add more in-order samples in future that would trigger the compaction. + addSamples(400, 450) + + // No blocks before compaction. + require.Empty(t, db.Blocks()) + + // Compacts normal and OOO head. + require.NoError(t, db.Compact(ctx)) + + // 2 blocks exist now. [0, 120), [250, 360) + require.Len(t, db.Blocks(), 2) + require.Equal(t, int64(0), db.Blocks()[0].MinTime()) + require.Equal(t, 120*time.Minute.Milliseconds(), db.Blocks()[0].MaxTime()) + require.Equal(t, 250*time.Minute.Milliseconds(), db.Blocks()[1].MinTime()) + require.Equal(t, 360*time.Minute.Milliseconds(), db.Blocks()[1].MaxTime()) + + // Checking that ooo chunk is empty. + for _, lbls := range []labels.Labels{series1, series2} { + ms, created, err := db.head.getOrCreate(lbls.Hash(), lbls, false) + require.NoError(t, err) + require.False(t, created) + require.Nil(t, ms.ooo) + } + + verifySamples := func(block *Block, fromMins, toMins int64) { + series1Samples := make([]chunks.Sample, 0, toMins-fromMins+1) + series2Samples := make([]chunks.Sample, 0, toMins-fromMins+1) + for m := fromMins; m <= toMins; m++ { + ts := m * time.Minute.Milliseconds() + series1Samples = append(series1Samples, scenario.sampleFunc(ts, ts)) + series2Samples = append(series2Samples, scenario.sampleFunc(ts, 2*ts)) + } + expRes := map[string][]chunks.Sample{ + series1.String(): series1Samples, + series2.String(): series2Samples, + } + + q, err := NewBlockQuerier(block, math.MinInt64, math.MaxInt64) + require.NoError(t, err) + + actRes := query(t, q, labels.MustNewMatcher(labels.MatchRegexp, "foo", "bar.*")) + requireEqualSeries(t, expRes, actRes, true) + } + + // Checking for expected data in the blocks. + verifySamples(db.Blocks()[0], 90, 110) + verifySamples(db.Blocks()[1], 250, 350) +} + +// TestOOOCompactionWithDisabledWriteLog tests the scenario where the TSDB is +// configured to not have wal and wbl but its able to compact both the in-order +// and out-of-order head. +func TestOOOCompactionWithDisabledWriteLog(t *testing.T) { + t.Parallel() + for name, scenario := range sampleTypeScenarios { + t.Run(name, func(t *testing.T) { + testOOOCompactionWithDisabledWriteLog(t, scenario) + }) + } +} + +func testOOOCompactionWithDisabledWriteLog(t *testing.T, scenario sampleTypeScenario) { + t.Parallel() + ctx := context.Background() + + opts := DefaultOptions() + opts.OutOfOrderCapMax = 30 + opts.OutOfOrderTimeWindow = 300 * time.Minute.Milliseconds() + opts.WALSegmentSize = -1 // disabled WAL and WBL + + db := newTestDB(t, withOpts(opts)) + db.DisableCompactions() // We want to manually call it. + + series1 := labels.FromStrings("foo", "bar1") + series2 := labels.FromStrings("foo", "bar2") + + addSamples := func(fromMins, toMins int64) { + app := db.Appender(context.Background()) + for m := fromMins; m <= toMins; m++ { + ts := m * time.Minute.Milliseconds() + _, _, err := scenario.appendFunc(app, series1, ts, ts) + require.NoError(t, err) + _, _, err = scenario.appendFunc(app, series2, ts, 2*ts) + require.NoError(t, err) + } + require.NoError(t, app.Commit()) + } + + // Add an in-order samples. + addSamples(250, 350) + + // Add ooo samples that will result into a single block. + addSamples(90, 110) + + // Checking that ooo chunk is not empty. + for _, lbls := range []labels.Labels{series1, series2} { + ms, created, err := db.head.getOrCreate(lbls.Hash(), lbls, false) + require.NoError(t, err) + require.False(t, created) + require.Positive(t, ms.ooo.oooHeadChunk.chunk.NumSamples()) + } + + // If the normal Head is not compacted, the OOO head compaction does not take place. + require.NoError(t, db.Compact(ctx)) + require.Empty(t, db.Blocks()) + + // Add more in-order samples in future that would trigger the compaction. + addSamples(400, 450) + + // No blocks before compaction. + require.Empty(t, db.Blocks()) + + // Compacts normal and OOO head. + require.NoError(t, db.Compact(ctx)) + + // 2 blocks exist now. [0, 120), [250, 360) + require.Len(t, db.Blocks(), 2) + require.Equal(t, int64(0), db.Blocks()[0].MinTime()) + require.Equal(t, 120*time.Minute.Milliseconds(), db.Blocks()[0].MaxTime()) + require.Equal(t, 250*time.Minute.Milliseconds(), db.Blocks()[1].MinTime()) + require.Equal(t, 360*time.Minute.Milliseconds(), db.Blocks()[1].MaxTime()) + + // Checking that ooo chunk is empty. + for _, lbls := range []labels.Labels{series1, series2} { + ms, created, err := db.head.getOrCreate(lbls.Hash(), lbls, false) + require.NoError(t, err) + require.False(t, created) + require.Nil(t, ms.ooo) + } + + verifySamples := func(block *Block, fromMins, toMins int64) { + series1Samples := make([]chunks.Sample, 0, toMins-fromMins+1) + series2Samples := make([]chunks.Sample, 0, toMins-fromMins+1) + for m := fromMins; m <= toMins; m++ { + ts := m * time.Minute.Milliseconds() + series1Samples = append(series1Samples, scenario.sampleFunc(ts, ts)) + series2Samples = append(series2Samples, scenario.sampleFunc(ts, 2*ts)) + } + expRes := map[string][]chunks.Sample{ + series1.String(): series1Samples, + series2.String(): series2Samples, + } + + q, err := NewBlockQuerier(block, math.MinInt64, math.MaxInt64) + require.NoError(t, err) + + actRes := query(t, q, labels.MustNewMatcher(labels.MatchRegexp, "foo", "bar.*")) + requireEqualSeries(t, expRes, actRes, true) + } + + // Checking for expected data in the blocks. + verifySamples(db.Blocks()[0], 90, 110) + verifySamples(db.Blocks()[1], 250, 350) +} + +// TestOOOQueryAfterRestartWithSnapshotAndRemovedWBL tests the scenario where the WBL goes +// missing after a restart while snapshot was enabled, but the query still returns the right +// data from the mmap chunks. +func TestOOOQueryAfterRestartWithSnapshotAndRemovedWBL(t *testing.T) { + t.Parallel() + for name, scenario := range sampleTypeScenarios { + t.Run(name, func(t *testing.T) { + testOOOQueryAfterRestartWithSnapshotAndRemovedWBL(t, scenario) + }) + } +} + +func testOOOQueryAfterRestartWithSnapshotAndRemovedWBL(t *testing.T, scenario sampleTypeScenario) { + ctx := context.Background() + + opts := DefaultOptions() + opts.OutOfOrderCapMax = 10 + opts.OutOfOrderTimeWindow = 300 * time.Minute.Milliseconds() + opts.EnableMemorySnapshotOnShutdown = true + + db := newTestDB(t, withOpts(opts)) + db.DisableCompactions() // We want to manually call it. + + series1 := labels.FromStrings("foo", "bar1") + series2 := labels.FromStrings("foo", "bar2") + + addSamples := func(fromMins, toMins int64) { + app := db.Appender(context.Background()) + for m := fromMins; m <= toMins; m++ { + ts := m * time.Minute.Milliseconds() + _, _, err := scenario.appendFunc(app, series1, ts, ts) + require.NoError(t, err) + _, _, err = scenario.appendFunc(app, series2, ts, 2*ts) + require.NoError(t, err) + } + require.NoError(t, app.Commit()) + } + + // Add an in-order samples. + addSamples(250, 350) + + // Add ooo samples that will result into a single block. + addSamples(90, 110) // The sample 110 will not be in m-map chunks. + + // Checking that there are some ooo m-map chunks. + for _, lbls := range []labels.Labels{series1, series2} { + ms, created, err := db.head.getOrCreate(lbls.Hash(), lbls, false) + require.NoError(t, err) + require.False(t, created) + require.Len(t, ms.ooo.oooMmappedChunks, 2) + require.NotNil(t, ms.ooo.oooHeadChunk) + } + + // Restart DB. + require.NoError(t, db.Close()) + + // For some reason wbl goes missing. + require.NoError(t, os.RemoveAll(path.Join(db.Dir(), "wbl"))) + + db = newTestDB(t, withDir(db.Dir())) + db.DisableCompactions() // We want to manually call it. + + // Check ooo m-map chunks again. + for _, lbls := range []labels.Labels{series1, series2} { + ms, created, err := db.head.getOrCreate(lbls.Hash(), lbls, false) + require.NoError(t, err) + require.False(t, created) + require.Len(t, ms.ooo.oooMmappedChunks, 2) + require.Equal(t, 109*time.Minute.Milliseconds(), ms.ooo.oooMmappedChunks[1].maxTime) + require.Nil(t, ms.ooo.oooHeadChunk) // Because of missing wbl. + } + + verifySamples := func(fromMins, toMins int64) { + series1Samples := make([]chunks.Sample, 0, toMins-fromMins+1) + series2Samples := make([]chunks.Sample, 0, toMins-fromMins+1) + for m := fromMins; m <= toMins; m++ { + ts := m * time.Minute.Milliseconds() + series1Samples = append(series1Samples, scenario.sampleFunc(ts, ts)) + series2Samples = append(series2Samples, scenario.sampleFunc(ts, ts*2)) + } + expRes := map[string][]chunks.Sample{ + series1.String(): series1Samples, + series2.String(): series2Samples, + } + + q, err := db.Querier(fromMins*time.Minute.Milliseconds(), toMins*time.Minute.Milliseconds()) + require.NoError(t, err) + + actRes := query(t, q, labels.MustNewMatcher(labels.MatchRegexp, "foo", "bar.*")) + requireEqualSeries(t, expRes, actRes, true) + } + + // Checking for expected ooo data from mmap chunks. + verifySamples(90, 109) + + // Compaction should also work fine. + require.Empty(t, db.Blocks()) + require.NoError(t, db.CompactOOOHead(ctx)) + require.Len(t, db.Blocks(), 1) // One block from OOO data. + require.Equal(t, int64(0), db.Blocks()[0].MinTime()) + require.Equal(t, 120*time.Minute.Milliseconds(), db.Blocks()[0].MaxTime()) + + // Checking that ooo chunk is empty in Head. + for _, lbls := range []labels.Labels{series1, series2} { + ms, created, err := db.head.getOrCreate(lbls.Hash(), lbls, false) + require.NoError(t, err) + require.False(t, created) + require.Nil(t, ms.ooo) + } + + verifySamples(90, 109) +} + +func TestQuerierOOOQuery(t *testing.T) { + scenarios := map[string]struct { + appendFunc func(app storage.Appender, ts int64, counterReset bool) (storage.SeriesRef, error) + sampleFunc func(ts int64) chunks.Sample + }{ + "float": { + appendFunc: func(app storage.Appender, ts int64, _ bool) (storage.SeriesRef, error) { + return app.Append(0, labels.FromStrings("foo", "bar1"), ts, float64(ts)) + }, + sampleFunc: func(ts int64) chunks.Sample { + return sample{t: ts, f: float64(ts)} + }, + }, + "integer histogram": { + appendFunc: func(app storage.Appender, ts int64, counterReset bool) (storage.SeriesRef, error) { + h := tsdbutil.GenerateTestHistogram(ts) + if counterReset { + h.CounterResetHint = histogram.CounterReset + } + return app.AppendHistogram(0, labels.FromStrings("foo", "bar1"), ts, h, nil) + }, + sampleFunc: func(ts int64) chunks.Sample { + return sample{t: ts, h: tsdbutil.GenerateTestHistogram(ts)} + }, + }, + "float histogram": { + appendFunc: func(app storage.Appender, ts int64, counterReset bool) (storage.SeriesRef, error) { + fh := tsdbutil.GenerateTestFloatHistogram(ts) + if counterReset { + fh.CounterResetHint = histogram.CounterReset + } + return app.AppendHistogram(0, labels.FromStrings("foo", "bar1"), ts, nil, fh) + }, + sampleFunc: func(ts int64) chunks.Sample { + return sample{t: ts, fh: tsdbutil.GenerateTestFloatHistogram(ts)} + }, + }, + "integer histogram counter resets": { + // Adding counter reset to all histograms means each histogram will have its own chunk. + appendFunc: func(app storage.Appender, ts int64, _ bool) (storage.SeriesRef, error) { + h := tsdbutil.GenerateTestHistogram(ts) + h.CounterResetHint = histogram.CounterReset // For this scenario, ignore the counterReset argument. + return app.AppendHistogram(0, labels.FromStrings("foo", "bar1"), ts, h, nil) + }, + sampleFunc: func(ts int64) chunks.Sample { + return sample{t: ts, h: tsdbutil.GenerateTestHistogram(ts)} + }, + }, + } + + for name, scenario := range scenarios { + t.Run(name, func(t *testing.T) { + testQuerierOOOQuery(t, scenario.appendFunc, scenario.sampleFunc) + }) + } +} + +func testQuerierOOOQuery(t *testing.T, + appendFunc func(app storage.Appender, ts int64, counterReset bool) (storage.SeriesRef, error), + sampleFunc func(ts int64) chunks.Sample, +) { + opts := DefaultOptions() + opts.OutOfOrderTimeWindow = 24 * time.Hour.Milliseconds() + + series1 := labels.FromStrings("foo", "bar1") + + type filterFunc func(t int64) bool + defaultFilterFunc := func(int64) bool { return true } + + minutes := func(m int64) int64 { return m * time.Minute.Milliseconds() } + addSample := func(db *DB, fromMins, toMins, queryMinT, queryMaxT int64, expSamples []chunks.Sample, filter filterFunc, counterReset bool) ([]chunks.Sample, int) { + app := db.Appender(context.Background()) + totalAppended := 0 + for m := fromMins; m <= toMins; m += time.Minute.Milliseconds() { + if !filter(m / time.Minute.Milliseconds()) { + continue + } + _, err := appendFunc(app, m, counterReset) + if m >= queryMinT && m <= queryMaxT { + expSamples = append(expSamples, sampleFunc(m)) + } + require.NoError(t, err) + totalAppended++ + } + require.NoError(t, app.Commit()) + require.Positive(t, totalAppended, 0) // Sanity check that filter is not too zealous. + return expSamples, totalAppended + } + + type sampleBatch struct { + minT int64 + maxT int64 + filter filterFunc + counterReset bool + isOOO bool + } + + tests := []struct { + name string + oooCap int64 + queryMinT int64 + queryMaxT int64 + batches []sampleBatch + }{ + { + name: "query interval covering ooomint and inordermaxt returns all ingested samples", + oooCap: 30, + queryMinT: minutes(0), + queryMaxT: minutes(200), + batches: []sampleBatch{ + { + minT: minutes(100), + maxT: minutes(200), + filter: defaultFilterFunc, + }, + { + minT: minutes(0), + maxT: minutes(99), + filter: defaultFilterFunc, + isOOO: true, + }, + }, + }, + { + name: "partial query interval returns only samples within interval", + oooCap: 30, + queryMinT: minutes(20), + queryMaxT: minutes(180), + batches: []sampleBatch{ + { + minT: minutes(100), + maxT: minutes(200), + filter: defaultFilterFunc, + }, + { + minT: minutes(0), + maxT: minutes(99), + filter: defaultFilterFunc, + isOOO: true, + }, + }, + }, + { + name: "alternating OOO batches", // In order: 100-200 normal. out of order first path: 0, 2, 4, ... 98 (no counter reset), second pass: 1, 3, 5, ... 99 (with counter reset). + queryMinT: minutes(0), + queryMaxT: minutes(200), + batches: []sampleBatch{ + { + minT: minutes(100), + maxT: minutes(200), + filter: defaultFilterFunc, + }, + { + minT: minutes(0), + maxT: minutes(99), + filter: func(t int64) bool { return t%2 == 0 }, + isOOO: true, + }, + { + minT: minutes(0), + maxT: minutes(99), + filter: func(t int64) bool { return t%2 == 1 }, + counterReset: true, + isOOO: true, + }, + }, + }, + { + name: "query overlapping inorder and ooo samples returns all ingested samples at the end of the interval", + oooCap: 30, + queryMinT: minutes(0), + queryMaxT: minutes(200), + batches: []sampleBatch{ + { + minT: minutes(100), + maxT: minutes(200), + filter: func(t int64) bool { return t%2 == 0 }, + isOOO: false, + }, + { + minT: minutes(170), + maxT: minutes(180), + filter: func(t int64) bool { return t%2 == 1 }, + isOOO: true, + }, + }, + }, + { + name: "query overlapping inorder and ooo in-memory samples returns all ingested samples at the beginning of the interval", + oooCap: 30, + queryMinT: minutes(0), + queryMaxT: minutes(200), + batches: []sampleBatch{ + { + minT: minutes(100), + maxT: minutes(200), + filter: func(t int64) bool { return t%2 == 0 }, + isOOO: false, + }, + { + minT: minutes(100), + maxT: minutes(110), + filter: func(t int64) bool { return t%2 == 1 }, + isOOO: true, + }, + }, + }, + { + name: "query inorder contain ooo mmapped samples returns all ingested samples at the beginning of the interval", + oooCap: 5, + queryMinT: minutes(0), + queryMaxT: minutes(200), + batches: []sampleBatch{ + { + minT: minutes(100), + maxT: minutes(200), + filter: func(t int64) bool { return t%2 == 0 }, + isOOO: false, + }, + { + minT: minutes(101), + maxT: minutes(101 + (5-1)*2), // Append samples to fit in a single mmapped OOO chunk and fit inside the first in-order mmapped chunk. + filter: func(t int64) bool { return t%2 == 1 }, + isOOO: true, + }, + { + minT: minutes(191), + maxT: minutes(193), // Append some more OOO samples to trigger mapping the OOO chunk, but use time 151 to not overlap with in-order head chunk. + filter: func(t int64) bool { return t%2 == 1 }, + isOOO: true, + }, + }, + }, + { + name: "query overlapping inorder and ooo mmapped samples returns all ingested samples at the beginning of the interval", + oooCap: 30, + queryMinT: minutes(0), + queryMaxT: minutes(200), + batches: []sampleBatch{ + { + minT: minutes(100), + maxT: minutes(200), + filter: func(t int64) bool { return t%2 == 0 }, + isOOO: false, + }, + { + minT: minutes(101), + maxT: minutes(101 + (30-1)*2), // Append samples to fit in a single mmapped OOO chunk and overlap the first in-order mmapped chunk. + filter: func(t int64) bool { return t%2 == 1 }, + isOOO: true, + }, + { + minT: minutes(191), + maxT: minutes(193), // Append some more OOO samples to trigger mapping the OOO chunk, but use time 151 to not overlap with in-order head chunk. + filter: func(t int64) bool { return t%2 == 1 }, + isOOO: true, + }, + }, + }, + } + for _, tc := range tests { + t.Run(fmt.Sprintf("name=%s", tc.name), func(t *testing.T) { + opts.OutOfOrderCapMax = tc.oooCap + db := newTestDB(t, withOpts(opts)) + db.DisableCompactions() + + var expSamples []chunks.Sample + var oooSamples, appendedCount int + + for _, batch := range tc.batches { + expSamples, appendedCount = addSample(db, batch.minT, batch.maxT, tc.queryMinT, tc.queryMaxT, expSamples, batch.filter, batch.counterReset) + if batch.isOOO { + oooSamples += appendedCount + } + } + + sort.Slice(expSamples, func(i, j int) bool { + return expSamples[i].T() < expSamples[j].T() + }) + + querier, err := db.Querier(tc.queryMinT, tc.queryMaxT) + require.NoError(t, err) + defer querier.Close() + + seriesSet := query(t, querier, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar1")) + gotSamples := seriesSet[series1.String()] + require.NotNil(t, gotSamples) + require.Len(t, seriesSet, 1) + requireEqualSamples(t, series1.String(), expSamples, gotSamples, requireEqualSamplesIgnoreCounterResets) + requireEqualOOOSamples(t, oooSamples, db) + }) + } +} + +func TestChunkQuerierOOOQuery(t *testing.T) { + nBucketHistogram := func(n int64) *histogram.Histogram { + h := &histogram.Histogram{ + Count: uint64(n), + Sum: float64(n), + } + if n == 0 { + h.PositiveSpans = []histogram.Span{} + h.PositiveBuckets = []int64{} + return h + } + h.PositiveSpans = []histogram.Span{{Offset: 0, Length: uint32(n)}} + h.PositiveBuckets = make([]int64, n) + h.PositiveBuckets[0] = 1 + return h + } + + scenarios := map[string]struct { + appendFunc func(app storage.Appender, ts int64, counterReset bool) (storage.SeriesRef, error) + sampleFunc func(ts int64) chunks.Sample + checkInUseBucket bool + }{ + "float": { + appendFunc: func(app storage.Appender, ts int64, _ bool) (storage.SeriesRef, error) { + return app.Append(0, labels.FromStrings("foo", "bar1"), ts, float64(ts)) + }, + sampleFunc: func(ts int64) chunks.Sample { + return sample{t: ts, f: float64(ts)} + }, + }, + "integer histogram": { + appendFunc: func(app storage.Appender, ts int64, counterReset bool) (storage.SeriesRef, error) { + h := tsdbutil.GenerateTestHistogram(ts) + if counterReset { + h.CounterResetHint = histogram.CounterReset + } + return app.AppendHistogram(0, labels.FromStrings("foo", "bar1"), ts, h, nil) + }, + sampleFunc: func(ts int64) chunks.Sample { + return sample{t: ts, h: tsdbutil.GenerateTestHistogram(ts)} + }, + }, + "float histogram": { + appendFunc: func(app storage.Appender, ts int64, counterReset bool) (storage.SeriesRef, error) { + fh := tsdbutil.GenerateTestFloatHistogram(ts) + if counterReset { + fh.CounterResetHint = histogram.CounterReset + } + return app.AppendHistogram(0, labels.FromStrings("foo", "bar1"), ts, nil, fh) + }, + sampleFunc: func(ts int64) chunks.Sample { + return sample{t: ts, fh: tsdbutil.GenerateTestFloatHistogram(ts)} + }, + }, + "integer histogram counter resets": { + // Adding counter reset to all histograms means each histogram will have its own chunk. + appendFunc: func(app storage.Appender, ts int64, _ bool) (storage.SeriesRef, error) { + h := tsdbutil.GenerateTestHistogram(ts) + h.CounterResetHint = histogram.CounterReset // For this scenario, ignore the counterReset argument. + return app.AppendHistogram(0, labels.FromStrings("foo", "bar1"), ts, h, nil) + }, + sampleFunc: func(ts int64) chunks.Sample { + return sample{t: ts, h: tsdbutil.GenerateTestHistogram(ts)} + }, + }, + "integer histogram with recode": { + // Histograms have increasing number of buckets so their chunks are recoded. + appendFunc: func(app storage.Appender, ts int64, _ bool) (storage.SeriesRef, error) { + n := ts / time.Minute.Milliseconds() + return app.AppendHistogram(0, labels.FromStrings("foo", "bar1"), ts, nBucketHistogram(n), nil) + }, + sampleFunc: func(ts int64) chunks.Sample { + n := ts / time.Minute.Milliseconds() + return sample{t: ts, h: nBucketHistogram(n)} + }, + // Only check in-use buckets for this scenario. + // Recoding adds empty buckets. + checkInUseBucket: true, + }, + } + for name, scenario := range scenarios { + t.Run(name, func(t *testing.T) { + testChunkQuerierOOOQuery(t, scenario.appendFunc, scenario.sampleFunc, scenario.checkInUseBucket) + }) + } +} + +func testChunkQuerierOOOQuery(t *testing.T, + appendFunc func(app storage.Appender, ts int64, counterReset bool) (storage.SeriesRef, error), + sampleFunc func(ts int64) chunks.Sample, + checkInUseBuckets bool, +) { + opts := DefaultOptions() + opts.OutOfOrderCapMax = 30 + opts.OutOfOrderTimeWindow = 24 * time.Hour.Milliseconds() + + series1 := labels.FromStrings("foo", "bar1") + + type filterFunc func(t int64) bool + defaultFilterFunc := func(int64) bool { return true } + + minutes := func(m int64) int64 { return m * time.Minute.Milliseconds() } + addSample := func(db *DB, fromMins, toMins, queryMinT, queryMaxT int64, expSamples []chunks.Sample, filter filterFunc, counterReset bool) ([]chunks.Sample, int) { + app := db.Appender(context.Background()) + totalAppended := 0 + for m := fromMins; m <= toMins; m += time.Minute.Milliseconds() { + if !filter(m / time.Minute.Milliseconds()) { + continue + } + _, err := appendFunc(app, m, counterReset) + if m >= queryMinT && m <= queryMaxT { + expSamples = append(expSamples, sampleFunc(m)) + } + require.NoError(t, err) + totalAppended++ + } + require.NoError(t, app.Commit()) + require.Positive(t, totalAppended) // Sanity check that filter is not too zealous. + return expSamples, totalAppended + } + + type sampleBatch struct { + minT int64 + maxT int64 + filter filterFunc + counterReset bool + isOOO bool + } + + tests := []struct { + name string + oooCap int64 + queryMinT int64 + queryMaxT int64 + batches []sampleBatch + }{ + { + name: "query interval covering ooomint and inordermaxt returns all ingested samples", + oooCap: 30, + queryMinT: minutes(0), + queryMaxT: minutes(200), + batches: []sampleBatch{ + { + minT: minutes(100), + maxT: minutes(200), + filter: defaultFilterFunc, + }, + { + minT: minutes(0), + maxT: minutes(99), + filter: defaultFilterFunc, + isOOO: true, + }, + }, + }, + { + name: "partial query interval returns only samples within interval", + oooCap: 30, + queryMinT: minutes(20), + queryMaxT: minutes(180), + batches: []sampleBatch{ + { + minT: minutes(100), + maxT: minutes(200), + filter: defaultFilterFunc, + }, + { + minT: minutes(0), + maxT: minutes(99), + filter: defaultFilterFunc, + isOOO: true, + }, + }, + }, + { + name: "alternating OOO batches", // In order: 100-200 normal. out of order first path: 0, 2, 4, ... 98 (no counter reset), second pass: 1, 3, 5, ... 99 (with counter reset). + queryMinT: minutes(0), + queryMaxT: minutes(200), + batches: []sampleBatch{ + { + minT: minutes(100), + maxT: minutes(200), + filter: defaultFilterFunc, + }, + { + minT: minutes(0), + maxT: minutes(99), + filter: func(t int64) bool { return t%2 == 0 }, + isOOO: true, + }, + { + minT: minutes(0), + maxT: minutes(99), + filter: func(t int64) bool { return t%2 == 1 }, + counterReset: true, + isOOO: true, + }, + }, + }, + { + name: "query overlapping inorder and ooo samples returns all ingested samples at the end of the interval", + oooCap: 30, + queryMinT: minutes(0), + queryMaxT: minutes(200), + batches: []sampleBatch{ + { + minT: minutes(100), + maxT: minutes(200), + filter: func(t int64) bool { return t%2 == 0 }, + isOOO: false, + }, + { + minT: minutes(170), + maxT: minutes(180), + filter: func(t int64) bool { return t%2 == 1 }, + isOOO: true, + }, + }, + }, + { + name: "query overlapping inorder and ooo in-memory samples returns all ingested samples at the beginning of the interval", + oooCap: 30, + queryMinT: minutes(0), + queryMaxT: minutes(200), + batches: []sampleBatch{ + { + minT: minutes(100), + maxT: minutes(200), + filter: func(t int64) bool { return t%2 == 0 }, + isOOO: false, + }, + { + minT: minutes(100), + maxT: minutes(110), + filter: func(t int64) bool { return t%2 == 1 }, + isOOO: true, + }, + }, + }, + { + name: "query inorder contain ooo mmapped samples returns all ingested samples at the beginning of the interval", + oooCap: 5, + queryMinT: minutes(0), + queryMaxT: minutes(200), + batches: []sampleBatch{ + { + minT: minutes(100), + maxT: minutes(200), + filter: func(t int64) bool { return t%2 == 0 }, + isOOO: false, + }, + { + minT: minutes(101), + maxT: minutes(101 + (5-1)*2), // Append samples to fit in a single mmapped OOO chunk and fit inside the first in-order mmapped chunk. + filter: func(t int64) bool { return t%2 == 1 }, + isOOO: true, + }, + { + minT: minutes(191), + maxT: minutes(193), // Append some more OOO samples to trigger mapping the OOO chunk, but use time 151 to not overlap with in-order head chunk. + filter: func(t int64) bool { return t%2 == 1 }, + isOOO: true, + }, + }, + }, + { + name: "query overlapping inorder and ooo mmapped samples returns all ingested samples at the beginning of the interval", + oooCap: 30, + queryMinT: minutes(0), + queryMaxT: minutes(200), + batches: []sampleBatch{ + { + minT: minutes(100), + maxT: minutes(200), + filter: func(t int64) bool { return t%2 == 0 }, + isOOO: false, + }, + { + minT: minutes(101), + maxT: minutes(101 + (30-1)*2), // Append samples to fit in a single mmapped OOO chunk and overlap the first in-order mmapped chunk. + filter: func(t int64) bool { return t%2 == 1 }, + isOOO: true, + }, + { + minT: minutes(191), + maxT: minutes(193), // Append some more OOO samples to trigger mapping the OOO chunk, but use time 151 to not overlap with in-order head chunk. + filter: func(t int64) bool { return t%2 == 1 }, + isOOO: true, + }, + }, + }, + } + for _, tc := range tests { + t.Run(fmt.Sprintf("name=%s", tc.name), func(t *testing.T) { + opts.OutOfOrderCapMax = tc.oooCap + db := newTestDB(t, withOpts(opts)) + db.DisableCompactions() + + var expSamples []chunks.Sample + var oooSamples, appendedCount int + + for _, batch := range tc.batches { + expSamples, appendedCount = addSample(db, batch.minT, batch.maxT, tc.queryMinT, tc.queryMaxT, expSamples, batch.filter, batch.counterReset) + if batch.isOOO { + oooSamples += appendedCount + } + } + + sort.Slice(expSamples, func(i, j int) bool { + return expSamples[i].T() < expSamples[j].T() + }) + + querier, err := db.ChunkQuerier(tc.queryMinT, tc.queryMaxT) + require.NoError(t, err) + defer querier.Close() + + chks := queryChunks(t, querier, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar1")) + require.NotNil(t, chks[series1.String()]) + require.Len(t, chks, 1) + requireEqualOOOSamples(t, oooSamples, db) + var gotSamples []chunks.Sample + for _, chunk := range chks[series1.String()] { + it := chunk.Chunk.Iterator(nil) + smpls, err := storage.ExpandSamples(it, newSample) + require.NoError(t, err) + + // Verify that no sample is outside the chunk's time range. + for i, s := range smpls { + switch i { + case 0: + require.Equal(t, chunk.MinTime, s.T(), "first sample %v not at chunk min time %v", s, chunk.MinTime) + case len(smpls) - 1: + require.Equal(t, chunk.MaxTime, s.T(), "last sample %v not at chunk max time %v", s, chunk.MaxTime) + default: + require.GreaterOrEqual(t, s.T(), chunk.MinTime, "sample %v before chunk min time %v", s, chunk.MinTime) + require.LessOrEqual(t, s.T(), chunk.MaxTime, "sample %v after chunk max time %v", s, chunk.MaxTime) + } + } + + gotSamples = append(gotSamples, smpls...) + require.NoError(t, it.Err()) + } + if checkInUseBuckets { + requireEqualSamples(t, series1.String(), expSamples, gotSamples, requireEqualSamplesIgnoreCounterResets, requireEqualSamplesInUseBucketCompare) + } else { + requireEqualSamples(t, series1.String(), expSamples, gotSamples, requireEqualSamplesIgnoreCounterResets) + } + }) + } +} + +// TestOOONativeHistogramsWithCounterResets verifies the counter reset headers for in-order and out-of-order samples +// upon ingestion. Note that when the counter reset(s) occur in OOO samples, the header is set to UnknownCounterReset +// rather than CounterReset. This is because with OOO native histogram samples, it cannot be definitely +// determined if a counter reset occurred because the samples are not consecutive, and another sample +// could potentially come in that would change the status of the header. In this case, the UnknownCounterReset +// headers would be re-checked at query time and updated as needed. However, this test is checking the counter +// reset headers at the time of storage. +func TestOOONativeHistogramsWithCounterResets(t *testing.T) { + for name, scenario := range sampleTypeScenarios { + t.Run(name, func(t *testing.T) { + if name == intHistogram || name == floatHistogram { + testOOONativeHistogramsWithCounterResets(t, scenario) + } + }) + } +} + +func testOOONativeHistogramsWithCounterResets(t *testing.T, scenario sampleTypeScenario) { + opts := DefaultOptions() + opts.OutOfOrderCapMax = 30 + opts.OutOfOrderTimeWindow = 24 * time.Hour.Milliseconds() + + type resetFunc func(v int64) bool + defaultResetFunc := func(int64) bool { return false } + + lbls := labels.FromStrings("foo", "bar1") + minutes := func(m int64) int64 { return m * time.Minute.Milliseconds() } + + type sampleBatch struct { + from int64 + until int64 + shouldReset resetFunc + expCounterResetHints []histogram.CounterResetHint + } + + tests := []struct { + name string + queryMin int64 + queryMax int64 + batches []sampleBatch + expectedSamples []chunks.Sample + }{ + { + name: "Counter reset within in-order samples", + queryMin: minutes(40), + queryMax: minutes(55), + batches: []sampleBatch{ + // In-order samples + { + from: 40, + until: 50, + shouldReset: func(v int64) bool { + return v == 45 + }, + expCounterResetHints: []histogram.CounterResetHint{histogram.UnknownCounterReset, histogram.NotCounterReset, histogram.NotCounterReset, histogram.NotCounterReset, histogram.NotCounterReset, histogram.UnknownCounterReset, histogram.NotCounterReset, histogram.NotCounterReset, histogram.NotCounterReset, histogram.NotCounterReset}, + }, + }, + }, + { + name: "Counter reset right at beginning of OOO samples", + queryMin: minutes(40), + queryMax: minutes(55), + batches: []sampleBatch{ + // In-order samples + { + from: 40, + until: 45, + shouldReset: defaultResetFunc, + expCounterResetHints: []histogram.CounterResetHint{histogram.UnknownCounterReset, histogram.NotCounterReset, histogram.NotCounterReset, histogram.NotCounterReset, histogram.NotCounterReset}, + }, + { + from: 50, + until: 55, + shouldReset: defaultResetFunc, + expCounterResetHints: []histogram.CounterResetHint{histogram.UnknownCounterReset, histogram.NotCounterReset, histogram.NotCounterReset, histogram.NotCounterReset, histogram.NotCounterReset}, + }, + // OOO samples + { + from: 45, + until: 50, + shouldReset: func(v int64) bool { + return v == 45 + }, + expCounterResetHints: []histogram.CounterResetHint{histogram.UnknownCounterReset, histogram.NotCounterReset, histogram.NotCounterReset, histogram.NotCounterReset, histogram.NotCounterReset}, + }, + }, + }, + { + name: "Counter resets in both in-order and OOO samples", + queryMin: minutes(40), + queryMax: minutes(55), + batches: []sampleBatch{ + // In-order samples + { + from: 40, + until: 45, + shouldReset: func(v int64) bool { + return v == 44 + }, + expCounterResetHints: []histogram.CounterResetHint{histogram.UnknownCounterReset, histogram.NotCounterReset, histogram.NotCounterReset, histogram.NotCounterReset, histogram.UnknownCounterReset}, + }, + { + from: 50, + until: 55, + shouldReset: defaultResetFunc, + expCounterResetHints: []histogram.CounterResetHint{histogram.UnknownCounterReset, histogram.NotCounterReset, histogram.NotCounterReset, histogram.NotCounterReset, histogram.NotCounterReset}, + }, + // OOO samples + { + from: 45, + until: 50, + shouldReset: func(v int64) bool { + return v == 49 + }, + expCounterResetHints: []histogram.CounterResetHint{histogram.UnknownCounterReset, histogram.NotCounterReset, histogram.NotCounterReset, histogram.NotCounterReset, histogram.UnknownCounterReset}, + }, + }, + }, + } + for _, tc := range tests { + t.Run(fmt.Sprintf("name=%s", tc.name), func(t *testing.T) { + db := newTestDB(t, withOpts(opts)) + db.DisableCompactions() + + app := db.Appender(context.Background()) + + expSamples := make(map[string][]chunks.Sample) + + for _, batch := range tc.batches { + j := batch.from + smplIdx := 0 + for i := batch.from; i < batch.until; i++ { + resetCount := batch.shouldReset(i) + if resetCount { + j = 0 + } + _, s, err := scenario.appendFunc(app, lbls, minutes(i), j) + require.NoError(t, err) + if s.Type() == chunkenc.ValHistogram { + s.H().CounterResetHint = batch.expCounterResetHints[smplIdx] + } else if s.Type() == chunkenc.ValFloatHistogram { + s.FH().CounterResetHint = batch.expCounterResetHints[smplIdx] + } + expSamples[lbls.String()] = append(expSamples[lbls.String()], s) + j++ + smplIdx++ + } + } + + require.NoError(t, app.Commit()) + + for k, v := range expSamples { + sort.Slice(v, func(i, j int) bool { + return v[i].T() < v[j].T() + }) + expSamples[k] = v + } + + querier, err := db.Querier(tc.queryMin, tc.queryMax) + require.NoError(t, err) + defer querier.Close() + + seriesSet := query(t, querier, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar1")) + require.NotNil(t, seriesSet[lbls.String()]) + require.Len(t, seriesSet, 1) + requireEqualSeries(t, expSamples, seriesSet, false) + }) + } +} + +func TestOOOInterleavedImplicitCounterResets(t *testing.T) { + for name, scenario := range sampleTypeScenarios { + t.Run(name, func(t *testing.T) { + testOOOInterleavedImplicitCounterResets(t, name, scenario) + }) + } +} + +func testOOOInterleavedImplicitCounterResets(t *testing.T, name string, scenario sampleTypeScenario) { + var appendFunc func(app storage.Appender, ts, v int64) error + + if scenario.sampleType != sampleMetricTypeHistogram { + return + } + + switch name { + case intHistogram: + appendFunc = func(app storage.Appender, ts, v int64) error { + h := &histogram.Histogram{ + Count: uint64(v), + Sum: float64(v), + PositiveSpans: []histogram.Span{{Offset: 0, Length: 1}}, + PositiveBuckets: []int64{v}, + } + _, err := app.AppendHistogram(0, labels.FromStrings("foo", "bar1"), ts, h, nil) + return err + } + case floatHistogram: + appendFunc = func(app storage.Appender, ts, v int64) error { + fh := &histogram.FloatHistogram{ + Count: float64(v), + Sum: float64(v), + PositiveSpans: []histogram.Span{{Offset: 0, Length: 1}}, + PositiveBuckets: []float64{float64(v)}, + } + _, err := app.AppendHistogram(0, labels.FromStrings("foo", "bar1"), ts, nil, fh) + return err + } + case customBucketsIntHistogram: + appendFunc = func(app storage.Appender, ts, v int64) error { + h := &histogram.Histogram{ + Schema: -53, + Count: uint64(v), + Sum: float64(v), + PositiveSpans: []histogram.Span{{Offset: 0, Length: 1}}, + PositiveBuckets: []int64{v}, + CustomValues: []float64{float64(1), float64(2), float64(3)}, + } + _, err := app.AppendHistogram(0, labels.FromStrings("foo", "bar1"), ts, h, nil) + return err + } + case customBucketsFloatHistogram: + appendFunc = func(app storage.Appender, ts, v int64) error { + fh := &histogram.FloatHistogram{ + Schema: -53, + Count: float64(v), + Sum: float64(v), + PositiveSpans: []histogram.Span{{Offset: 0, Length: 1}}, + PositiveBuckets: []float64{float64(v)}, + CustomValues: []float64{float64(1), float64(2), float64(3)}, + } + _, err := app.AppendHistogram(0, labels.FromStrings("foo", "bar1"), ts, nil, fh) + return err + } + case gaugeIntHistogram, gaugeFloatHistogram: + return + } + + // Not a sample, we're encoding an integer counter that we convert to a + // histogram with a single bucket. + type tsValue struct { + ts int64 + v int64 + } + + type expectedTsValue struct { + ts int64 + v int64 + hint histogram.CounterResetHint + } + + type expectedChunk struct { + hint histogram.CounterResetHint + size int + } + + cases := map[string]struct { + samples []tsValue + oooCap int64 + // The expected samples with counter reset. + expectedSamples []expectedTsValue + // The expected counter reset hint for each chunk. + expectedChunks []expectedChunk + }{ + "counter reset in-order cleared by in-memory OOO chunk": { + samples: []tsValue{ + {1, 40}, // New in In-order. I1. + {4, 30}, // In-order counter reset. I2. + {2, 40}, // New in OOO. O1. + {3, 10}, // OOO counter reset. O2. + }, + oooCap: 30, + // Expect all to be set to UnknownCounterReset because we switch between + // in-order and out-of-order samples. + expectedSamples: []expectedTsValue{ + {1, 40, histogram.UnknownCounterReset}, // I1. + {2, 40, histogram.UnknownCounterReset}, // O1. + {3, 10, histogram.UnknownCounterReset}, // O2. + {4, 30, histogram.UnknownCounterReset}, // I2. Counter reset cleared by iterator change. + }, + expectedChunks: []expectedChunk{ + {histogram.UnknownCounterReset, 1}, // I1. + {histogram.UnknownCounterReset, 1}, // O1. + {histogram.UnknownCounterReset, 1}, // O2. + {histogram.UnknownCounterReset, 1}, // I2. + }, + }, + "counter reset in OOO mmapped chunk cleared by in-memory ooo chunk": { + samples: []tsValue{ + {8, 30}, // In-order, new chunk. I1. + {1, 10}, // OOO, new chunk (will be mmapped). MO1. + {2, 20}, // OOO, no reset (will be mmapped). MO1. + {3, 30}, // OOO, no reset (will be mmapped). MO1. + {5, 20}, // OOO, reset (will be mmapped). MO2. + {6, 10}, // OOO, reset (will be mmapped). MO3. + {7, 20}, // OOO, no reset (will be mmapped). MO3. + {4, 10}, // OOO, inserted into memory, triggers mmap. O1. + }, + oooCap: 6, + expectedSamples: []expectedTsValue{ + {1, 10, histogram.UnknownCounterReset}, // MO1. + {2, 20, histogram.NotCounterReset}, // MO1. + {3, 30, histogram.NotCounterReset}, // MO1. + {4, 10, histogram.UnknownCounterReset}, // O1. Counter reset cleared by iterator change. + {5, 20, histogram.UnknownCounterReset}, // MO2. + {6, 10, histogram.UnknownCounterReset}, // MO3. + {7, 20, histogram.NotCounterReset}, // MO3. + {8, 30, histogram.UnknownCounterReset}, // I1. + }, + expectedChunks: []expectedChunk{ + {histogram.UnknownCounterReset, 3}, // MO1. + {histogram.UnknownCounterReset, 1}, // O1. + {histogram.UnknownCounterReset, 1}, // MO2. + {histogram.UnknownCounterReset, 2}, // MO3. + {histogram.UnknownCounterReset, 1}, // I1. + }, + }, + "counter reset in OOO mmapped chunk cleared by another OOO mmapped chunk": { + samples: []tsValue{ + {8, 100}, // In-order, new chunk. I1. + {1, 50}, // OOO, new chunk (will be mmapped). MO1. + {5, 40}, // OOO, reset (will be mmapped). MO2. + {6, 50}, // OOO, no reset (will be mmapped). MO2. + {2, 10}, // OOO, new chunk no reset (will be mmapped). MO3. + {3, 20}, // OOO, no reset (will be mmapped). MO3. + {4, 30}, // OOO, no reset (will be mmapped). MO3. + {7, 60}, // OOO, no reset in memory. O1. + }, + oooCap: 3, + expectedSamples: []expectedTsValue{ + {1, 50, histogram.UnknownCounterReset}, // MO1. + {2, 10, histogram.UnknownCounterReset}, // MO3. + {3, 20, histogram.NotCounterReset}, // MO3. + {4, 30, histogram.NotCounterReset}, // MO3. + {5, 40, histogram.UnknownCounterReset}, // MO2. + {6, 50, histogram.NotCounterReset}, // MO2. + {7, 60, histogram.UnknownCounterReset}, // O1. + {8, 100, histogram.UnknownCounterReset}, // I1. + }, + expectedChunks: []expectedChunk{ + {histogram.UnknownCounterReset, 1}, // MO1. + {histogram.UnknownCounterReset, 3}, // MO3. + {histogram.UnknownCounterReset, 2}, // MO2. + {histogram.UnknownCounterReset, 1}, // O1. + {histogram.UnknownCounterReset, 1}, // I1. + }, + }, + } + + for tcName, tc := range cases { + t.Run(tcName, func(t *testing.T) { + opts := DefaultOptions() + opts.OutOfOrderCapMax = tc.oooCap + opts.OutOfOrderTimeWindow = 24 * time.Hour.Milliseconds() + + db := newTestDB(t, withOpts(opts)) + db.DisableCompactions() + + app := db.Appender(context.Background()) + for _, s := range tc.samples { + require.NoError(t, appendFunc(app, s.ts, s.v)) + } + require.NoError(t, app.Commit()) + + t.Run("querier", func(t *testing.T) { + querier, err := db.Querier(0, 10) + require.NoError(t, err) + defer querier.Close() + + seriesSet := query(t, querier, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar1")) + require.Len(t, seriesSet, 1) + samples, ok := seriesSet["{foo=\"bar1\"}"] + require.True(t, ok) + require.Len(t, samples, len(tc.samples)) + require.Len(t, samples, len(tc.expectedSamples)) + + // We expect all unknown counter resets because we clear the counter reset + // hint when we switch between in-order and out-of-order samples. + for i, s := range samples { + switch name { + case intHistogram: + require.Equal(t, tc.expectedSamples[i].hint, s.H().CounterResetHint, "sample %d", i) + require.Equal(t, tc.expectedSamples[i].v, int64(s.H().Count), "sample %d", i) + case floatHistogram: + require.Equal(t, tc.expectedSamples[i].hint, s.FH().CounterResetHint, "sample %d", i) + require.Equal(t, tc.expectedSamples[i].v, int64(s.FH().Count), "sample %d", i) + case customBucketsIntHistogram: + require.Equal(t, tc.expectedSamples[i].hint, s.H().CounterResetHint, "sample %d", i) + require.Equal(t, tc.expectedSamples[i].v, int64(s.H().Count), "sample %d", i) + case customBucketsFloatHistogram: + require.Equal(t, tc.expectedSamples[i].hint, s.FH().CounterResetHint, "sample %d", i) + require.Equal(t, tc.expectedSamples[i].v, int64(s.FH().Count), "sample %d", i) + default: + t.Fatalf("unexpected sample type %s", name) + } + } + }) + + t.Run("chunk-querier", func(t *testing.T) { + querier, err := db.ChunkQuerier(0, 10) + require.NoError(t, err) + defer querier.Close() + + chunkSet := queryAndExpandChunks(t, querier, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar1")) + require.Len(t, chunkSet, 1) + chunks, ok := chunkSet["{foo=\"bar1\"}"] + require.True(t, ok) + require.Len(t, chunks, len(tc.expectedChunks)) + idx := 0 + for i, samples := range chunks { + require.Len(t, samples, tc.expectedChunks[i].size) + for j, s := range samples { + expectHint := tc.expectedChunks[i].hint + if j > 0 { + expectHint = histogram.NotCounterReset + } + switch name { + case intHistogram: + require.Equal(t, expectHint, s.H().CounterResetHint, "sample %d", idx) + require.Equal(t, tc.expectedSamples[idx].v, int64(s.H().Count), "sample %d", idx) + case floatHistogram: + require.Equal(t, expectHint, s.FH().CounterResetHint, "sample %d", idx) + require.Equal(t, tc.expectedSamples[idx].v, int64(s.FH().Count), "sample %d", idx) + case customBucketsIntHistogram: + require.Equal(t, expectHint, s.H().CounterResetHint, "sample %d", idx) + require.Equal(t, tc.expectedSamples[idx].v, int64(s.H().Count), "sample %d", idx) + case customBucketsFloatHistogram: + require.Equal(t, expectHint, s.FH().CounterResetHint, "sample %d", idx) + require.Equal(t, tc.expectedSamples[idx].v, int64(s.FH().Count), "sample %d", idx) + default: + t.Fatalf("unexpected sample type %s", name) + } + idx++ + } + } + }) + }) + } +} + +func TestOOOAppendAndQuery(t *testing.T) { + for name, scenario := range sampleTypeScenarios { + t.Run(name, func(t *testing.T) { + testOOOAppendAndQuery(t, scenario) + }) + } +} + +func testOOOAppendAndQuery(t *testing.T, scenario sampleTypeScenario) { + opts := DefaultOptions() + opts.OutOfOrderCapMax = 30 + opts.OutOfOrderTimeWindow = 4 * time.Hour.Milliseconds() + + db := newTestDB(t, withOpts(opts)) + db.DisableCompactions() + + s1 := labels.FromStrings("foo", "bar1") + s2 := labels.FromStrings("foo", "bar2") + + minutes := func(m int64) int64 { return m * time.Minute.Milliseconds() } + appendedSamples := make(map[string][]chunks.Sample) + totalSamples := 0 + addSample := func(lbls labels.Labels, fromMins, toMins int64, faceError bool) { + app := db.Appender(context.Background()) + key := lbls.String() + from, to := minutes(fromMins), minutes(toMins) + for m := from; m <= to; m += time.Minute.Milliseconds() { + val := rand.Intn(1000) + _, s, err := scenario.appendFunc(app, lbls, m, int64(val)) + if faceError { + require.Error(t, err) + } else { + require.NoError(t, err) + appendedSamples[key] = append(appendedSamples[key], s) + totalSamples++ + } + } + if faceError { + require.NoError(t, app.Rollback()) + } else { + require.NoError(t, app.Commit()) + } + } + + testQuery := func(from, to int64) { + querier, err := db.Querier(from, to) + require.NoError(t, err) + + seriesSet := query(t, querier, labels.MustNewMatcher(labels.MatchRegexp, "foo", "bar.")) + + for k, v := range appendedSamples { + sort.Slice(v, func(i, j int) bool { + return v[i].T() < v[j].T() + }) + appendedSamples[k] = v + } + + expSamples := make(map[string][]chunks.Sample) + for k, samples := range appendedSamples { + for _, s := range samples { + if s.T() < from { + continue + } + if s.T() > to { + continue + } + expSamples[k] = append(expSamples[k], s) + } + } + requireEqualSeries(t, expSamples, seriesSet, true) + requireEqualOOOSamples(t, totalSamples-2, db) + } + + verifyOOOMinMaxTimes := func(expMin, expMax int64) { + require.Equal(t, minutes(expMin), db.head.MinOOOTime()) + require.Equal(t, minutes(expMax), db.head.MaxOOOTime()) + } + + // In-order samples. + addSample(s1, 300, 300, false) + addSample(s2, 290, 290, false) + require.Equal(t, float64(2), prom_testutil.ToFloat64(db.head.metrics.chunksCreated)) + testQuery(math.MinInt64, math.MaxInt64) + + // Some ooo samples. + addSample(s1, 250, 260, false) + addSample(s2, 255, 265, false) + verifyOOOMinMaxTimes(250, 265) + testQuery(math.MinInt64, math.MaxInt64) + testQuery(minutes(250), minutes(265)) // Test querying ooo data time range. + testQuery(minutes(290), minutes(300)) // Test querying in-order data time range. + testQuery(minutes(250), minutes(300)) // Test querying the entire range. + + // Out of time window. + addSample(s1, 59, 59, true) + addSample(s2, 49, 49, true) + verifyOOOMinMaxTimes(250, 265) + testQuery(math.MinInt64, math.MaxInt64) + + // At the edge of time window, also it would be "out of bound" without the ooo support. + addSample(s1, 60, 65, false) + verifyOOOMinMaxTimes(60, 265) + testQuery(math.MinInt64, math.MaxInt64) + + // This sample is not within the time window w.r.t. the head's maxt, but it is within the window + // w.r.t. the series' maxt. But we consider only head's maxt. + addSample(s2, 59, 59, true) + verifyOOOMinMaxTimes(60, 265) + testQuery(math.MinInt64, math.MaxInt64) + + // Now the sample is within time window w.r.t. the head's maxt. + addSample(s2, 60, 65, false) + verifyOOOMinMaxTimes(60, 265) + testQuery(math.MinInt64, math.MaxInt64) + + // Out of time window again. + addSample(s1, 59, 59, true) + addSample(s2, 49, 49, true) + testQuery(math.MinInt64, math.MaxInt64) + + // Generating some m-map chunks. The m-map chunks here are in such a way + // that when sorted w.r.t. mint, the last chunk's maxt is not the overall maxt + // of the merged chunk. This tests a bug fixed in https://github.com/grafana/mimir-prometheus/pull/238/. + require.Equal(t, float64(4), prom_testutil.ToFloat64(db.head.metrics.chunksCreated)) + addSample(s1, 180, 249, false) + require.Equal(t, float64(6), prom_testutil.ToFloat64(db.head.metrics.chunksCreated)) + verifyOOOMinMaxTimes(60, 265) + testQuery(math.MinInt64, math.MaxInt64) +} + +func TestOOODisabled(t *testing.T) { + for name, scenario := range sampleTypeScenarios { + t.Run(name, func(t *testing.T) { + testOOODisabled(t, scenario) + }) + } +} + +func testOOODisabled(t *testing.T, scenario sampleTypeScenario) { + opts := DefaultOptions() + opts.OutOfOrderTimeWindow = 0 + db := newTestDB(t, withOpts(opts)) + db.DisableCompactions() + + s1 := labels.FromStrings("foo", "bar1") + minutes := func(m int64) int64 { return m * time.Minute.Milliseconds() } + expSamples := make(map[string][]chunks.Sample) + totalSamples := 0 + failedSamples := 0 + + addSample := func(db *DB, lbls labels.Labels, fromMins, toMins int64, faceError bool) { + app := db.Appender(context.Background()) + key := lbls.String() + from, to := minutes(fromMins), minutes(toMins) + for m := from; m <= to; m += time.Minute.Milliseconds() { + _, _, err := scenario.appendFunc(app, lbls, m, m) + if faceError { + require.Error(t, err) + failedSamples++ + } else { + require.NoError(t, err) + expSamples[key] = append(expSamples[key], scenario.sampleFunc(m, m)) + totalSamples++ + } + } + if faceError { + require.NoError(t, app.Rollback()) + } else { + require.NoError(t, app.Commit()) + } + } + + addSample(db, s1, 300, 300, false) // In-order samples. + addSample(db, s1, 250, 260, true) // Some ooo samples. + addSample(db, s1, 59, 59, true) // Out of time window. + addSample(db, s1, 60, 65, true) // At the edge of time window, also it would be "out of bound" without the ooo support. + addSample(db, s1, 59, 59, true) // Out of time window again. + addSample(db, s1, 301, 310, false) // More in-order samples. + + querier, err := db.Querier(math.MinInt64, math.MaxInt64) + require.NoError(t, err) + + seriesSet := query(t, querier, labels.MustNewMatcher(labels.MatchRegexp, "foo", "bar.")) + requireEqualSeries(t, expSamples, seriesSet, true) + requireEqualOOOSamples(t, 0, db) + require.Equal(t, float64(failedSamples), + prom_testutil.ToFloat64(db.head.metrics.outOfOrderSamples.WithLabelValues(scenario.sampleType))+prom_testutil.ToFloat64(db.head.metrics.outOfBoundSamples.WithLabelValues(scenario.sampleType)), + "number of ooo/oob samples mismatch") + + // Verifying that no OOO artifacts were generated. + _, err = os.ReadDir(path.Join(db.Dir(), wlog.WblDirName)) + require.True(t, os.IsNotExist(err)) + + ms, created, err := db.head.getOrCreate(s1.Hash(), s1, false) + require.NoError(t, err) + require.False(t, created) + require.NotNil(t, ms) + require.Nil(t, ms.ooo) +} + +func TestWBLAndMmapReplay(t *testing.T) { + for name, scenario := range sampleTypeScenarios { + t.Run(name, func(t *testing.T) { + testWBLAndMmapReplay(t, scenario) + }) + } +} + +func testWBLAndMmapReplay(t *testing.T, scenario sampleTypeScenario) { + opts := DefaultOptions() + opts.OutOfOrderCapMax = 30 + opts.OutOfOrderTimeWindow = 4 * time.Hour.Milliseconds() + + db := newTestDB(t, withOpts(opts)) + db.DisableCompactions() + + s1 := labels.FromStrings("foo", "bar1") + + minutes := func(m int64) int64 { return m * time.Minute.Milliseconds() } + expSamples := make(map[string][]chunks.Sample) + totalSamples := 0 + addSample := func(lbls labels.Labels, fromMins, toMins int64) { + app := db.Appender(context.Background()) + key := lbls.String() + from, to := minutes(fromMins), minutes(toMins) + for m := from; m <= to; m += time.Minute.Milliseconds() { + val := rand.Intn(1000) + _, s, err := scenario.appendFunc(app, lbls, m, int64(val)) + require.NoError(t, err) + expSamples[key] = append(expSamples[key], s) + totalSamples++ + } + require.NoError(t, app.Commit()) + } + + testQuery := func(exp map[string][]chunks.Sample) { + querier, err := db.Querier(math.MinInt64, math.MaxInt64) + require.NoError(t, err) + + seriesSet := query(t, querier, labels.MustNewMatcher(labels.MatchRegexp, "foo", "bar.")) + + for k, v := range exp { + sort.Slice(v, func(i, j int) bool { + return v[i].T() < v[j].T() + }) + exp[k] = v + } + requireEqualSeries(t, exp, seriesSet, true) + } + + // In-order samples. + addSample(s1, 300, 300) + require.Equal(t, float64(1), prom_testutil.ToFloat64(db.head.metrics.chunksCreated)) + + // Some ooo samples. + addSample(s1, 250, 260) + addSample(s1, 195, 249) // This creates some m-map chunks. + require.Equal(t, float64(4), prom_testutil.ToFloat64(db.head.metrics.chunksCreated)) + testQuery(expSamples) + oooMint, oooMaxt := minutes(195), minutes(260) + + // Collect the samples only present in the ooo m-map chunks. + ms, created, err := db.head.getOrCreate(s1.Hash(), s1, false) + require.False(t, created) + require.NoError(t, err) + var s1MmapSamples []chunks.Sample + for _, mc := range ms.ooo.oooMmappedChunks { + chk, err := db.head.chunkDiskMapper.Chunk(mc.ref) + require.NoError(t, err) + it := chk.Iterator(nil) + smpls, err := storage.ExpandSamples(it, newSample) + require.NoError(t, err) + s1MmapSamples = append(s1MmapSamples, smpls...) + } + require.NotEmpty(t, s1MmapSamples) + + require.NoError(t, db.Close()) + + // Making a copy of original state of WBL and Mmap files to use it later. + mmapDir := mmappedChunksDir(db.head.opts.ChunkDirRoot) + wblDir := db.head.wbl.Dir() + originalWblDir := filepath.Join(t.TempDir(), "original_wbl") + originalMmapDir := filepath.Join(t.TempDir(), "original_mmap") + require.NoError(t, fileutil.CopyDirs(wblDir, originalWblDir)) + require.NoError(t, fileutil.CopyDirs(mmapDir, originalMmapDir)) + resetWBLToOriginal := func() { + require.NoError(t, os.RemoveAll(wblDir)) + require.NoError(t, fileutil.CopyDirs(originalWblDir, wblDir)) + } + resetMmapToOriginal := func() { + require.NoError(t, os.RemoveAll(mmapDir)) + require.NoError(t, fileutil.CopyDirs(originalMmapDir, mmapDir)) + } + + t.Run("Restart DB with both WBL and M-map files for ooo data", func(t *testing.T) { + db = newTestDB(t, withDir(db.Dir()), withOpts(opts)) + require.Equal(t, oooMint, db.head.MinOOOTime()) + require.Equal(t, oooMaxt, db.head.MaxOOOTime()) + testQuery(expSamples) + }) + + t.Run("Restart DB with only WBL for ooo data", func(t *testing.T) { + require.NoError(t, os.RemoveAll(mmapDir)) + + db = newTestDB(t, withDir(db.Dir()), withOpts(opts)) + require.Equal(t, oooMint, db.head.MinOOOTime()) + require.Equal(t, oooMaxt, db.head.MaxOOOTime()) + testQuery(expSamples) + }) + + t.Run("Restart DB with only M-map files for ooo data", func(t *testing.T) { + require.NoError(t, os.RemoveAll(wblDir)) + resetMmapToOriginal() + + db = newTestDB(t, withDir(db.Dir()), withOpts(opts)) + require.Equal(t, oooMint, db.head.MinOOOTime()) + require.Equal(t, oooMaxt, db.head.MaxOOOTime()) + inOrderSample := expSamples[s1.String()][len(expSamples[s1.String()])-1] + testQuery(map[string][]chunks.Sample{ + s1.String(): append(s1MmapSamples, inOrderSample), + }) + }) + + t.Run("Restart DB with WBL+Mmap while increasing the OOOCapMax", func(t *testing.T) { + resetWBLToOriginal() + resetMmapToOriginal() + + opts.OutOfOrderCapMax = 60 + db = newTestDB(t, withDir(db.Dir()), withOpts(opts)) + require.NoError(t, err) + require.Equal(t, oooMint, db.head.MinOOOTime()) + require.Equal(t, oooMaxt, db.head.MaxOOOTime()) + testQuery(expSamples) + }) + + t.Run("Restart DB with WBL+Mmap while decreasing the OOOCapMax", func(t *testing.T) { + resetMmapToOriginal() // We need to reset because new duplicate chunks can be written above. + + opts.OutOfOrderCapMax = 10 + db = newTestDB(t, withDir(db.Dir()), withOpts(opts)) + require.NoError(t, err) + require.Equal(t, oooMint, db.head.MinOOOTime()) + require.Equal(t, oooMaxt, db.head.MaxOOOTime()) + testQuery(expSamples) + }) + + t.Run("Restart DB with WBL+Mmap while having no m-map markers in WBL", func(t *testing.T) { + resetMmapToOriginal() // We neet to reset because new duplicate chunks can be written above. + + // Removing m-map markers in WBL by rewriting it. + newWbl, err := wlog.New(promslog.NewNopLogger(), nil, filepath.Join(t.TempDir(), "new_wbl"), compression.None) + require.NoError(t, err) + sr, err := wlog.NewSegmentsReader(originalWblDir) + require.NoError(t, err) + dec := record.NewDecoder(labels.NewSymbolTable(), promslog.NewNopLogger()) + r, markers, addedRecs := wlog.NewReader(sr), 0, 0 + for r.Next() { + rec := r.Record() + if dec.Type(rec) == record.MmapMarkers { + markers++ + continue + } + addedRecs++ + require.NoError(t, newWbl.Log(rec)) + } + require.Positive(t, markers) + require.Positive(t, addedRecs) + require.NoError(t, newWbl.Close()) + require.NoError(t, sr.Close()) + require.NoError(t, os.RemoveAll(wblDir)) + require.NoError(t, os.Rename(newWbl.Dir(), wblDir)) + + opts.OutOfOrderCapMax = 30 + db = newTestDB(t, withDir(db.Dir()), withOpts(opts)) + require.NoError(t, err) + require.Equal(t, oooMint, db.head.MinOOOTime()) + require.Equal(t, oooMaxt, db.head.MaxOOOTime()) + testQuery(expSamples) + }) +} + +func TestOOOHistogramCompactionWithCounterResets(t *testing.T) { + for _, floatHistogram := range []bool{false, true} { + ctx := context.Background() + + opts := DefaultOptions() + opts.OutOfOrderCapMax = 30 + opts.OutOfOrderTimeWindow = 500 * time.Minute.Milliseconds() + + db := newTestDB(t, withOpts(opts)) + db.DisableCompactions() // We want to manually call it. + + series1 := labels.FromStrings("foo", "bar1") + series2 := labels.FromStrings("foo", "bar2") + + var series1ExpSamplesPreCompact, series2ExpSamplesPreCompact, series1ExpSamplesPostCompact, series2ExpSamplesPostCompact []chunks.Sample + + addSample := func(ts int64, l labels.Labels, val int, hint histogram.CounterResetHint) sample { + app := db.Appender(context.Background()) + tsMs := ts * time.Minute.Milliseconds() + if floatHistogram { + h := tsdbutil.GenerateTestFloatHistogram(int64(val)) + h.CounterResetHint = hint + _, err := app.AppendHistogram(0, l, tsMs, nil, h) + require.NoError(t, err) + require.NoError(t, app.Commit()) + return sample{t: tsMs, fh: h.Copy()} + } + + h := tsdbutil.GenerateTestHistogram(int64(val)) + h.CounterResetHint = hint + _, err := app.AppendHistogram(0, l, tsMs, h, nil) + require.NoError(t, err) + require.NoError(t, app.Commit()) + return sample{t: tsMs, h: h.Copy()} + } + + // Add an in-order sample to each series. + s := addSample(520, series1, 1000000, histogram.UnknownCounterReset) + series1ExpSamplesPreCompact = append(series1ExpSamplesPreCompact, s) + series1ExpSamplesPostCompact = append(series1ExpSamplesPostCompact, s) + + s = addSample(520, series2, 1000000, histogram.UnknownCounterReset) + series2ExpSamplesPreCompact = append(series2ExpSamplesPreCompact, s) + series2ExpSamplesPostCompact = append(series2ExpSamplesPostCompact, s) + + // Verify that the in-memory ooo chunk is empty. + checkEmptyOOOChunk := func(lbls labels.Labels) { + ms, created, err := db.head.getOrCreate(lbls.Hash(), lbls, false) + require.NoError(t, err) + require.False(t, created) + require.Nil(t, ms.ooo) + } + + checkEmptyOOOChunk(series1) + checkEmptyOOOChunk(series2) + + // Add samples for series1. There are three head chunks that will be created: + // Chunk 1 - Samples between 100 - 440. One explicit counter reset at ts 250. + // Chunk 2 - Samples between 105 - 395. Overlaps with Chunk 1. One detected counter reset at ts 165. + // Chunk 3 - Samples between 480 - 509. All within one block boundary. One detected counter reset at 490. + + // Chunk 1. + // First add 10 samples. + for i := 100; i < 200; i += 10 { + s = addSample(int64(i), series1, 100000+i, histogram.UnknownCounterReset) + // Before compaction, all the samples have UnknownCounterReset even though they've been added to the same + // chunk. This is because they overlap with the samples from chunk two and when merging two chunks on read, + // the header is set as unknown when the next sample is not in the same chunk as the previous one. + series1ExpSamplesPreCompact = append(series1ExpSamplesPreCompact, s) + // After compaction, samples from multiple mmapped chunks will be merged, so there won't be any overlapping + // chunks. Therefore, most samples will have the NotCounterReset header. + // 100 is the first sample in the first chunk in the blocks, so is still set to UnknownCounterReset. + // 120 is a block boundary - after compaction, 120 will be the first sample in a chunk, so is still set to + // UnknownCounterReset. + if i > 100 && i != 120 { + s = copyWithCounterReset(s, histogram.NotCounterReset) + } + series1ExpSamplesPostCompact = append(series1ExpSamplesPostCompact, s) + } + // Explicit counter reset - the counter reset header is set to CounterReset but the value is higher + // than for the previous timestamp. Explicit counter reset headers are actually ignored though, so when reading + // the sample back you actually get unknown/not counter reset. This is as the chainSampleIterator ignores + // existing headers and sets the header as UnknownCounterReset if the next sample is not in the same chunk as + // the previous one, and counter resets always create a new chunk. + // This case has been added to document what's happening, though it might not be the ideal behavior. + s = addSample(250, series1, 100000+250, histogram.CounterReset) + series1ExpSamplesPreCompact = append(series1ExpSamplesPreCompact, copyWithCounterReset(s, histogram.UnknownCounterReset)) + series1ExpSamplesPostCompact = append(series1ExpSamplesPostCompact, copyWithCounterReset(s, histogram.NotCounterReset)) + + // Add 19 more samples to complete a chunk. + for i := 260; i < 450; i += 10 { + s = addSample(int64(i), series1, 100000+i, histogram.UnknownCounterReset) + // The samples with timestamp less than 410 overlap with the samples from chunk 2, so before compaction, + // they're all UnknownCounterReset. Samples greater than or equal to 410 don't overlap with other chunks + // so they're always detected as NotCounterReset pre and post compaction. + if i >= 410 { + s = copyWithCounterReset(s, histogram.NotCounterReset) + } + series1ExpSamplesPreCompact = append(series1ExpSamplesPreCompact, s) + // + // 360 is a block boundary, so after compaction its header is still UnknownCounterReset. + if i != 360 { + s = copyWithCounterReset(s, histogram.NotCounterReset) + } + series1ExpSamplesPostCompact = append(series1ExpSamplesPostCompact, s) + } + + // Chunk 2. + // Add six OOO samples. + for i := 105; i < 165; i += 10 { + s = addSample(int64(i), series1, 100000+i, histogram.UnknownCounterReset) + // Samples overlap with chunk 1 so before compaction all headers are UnknownCounterReset. + series1ExpSamplesPreCompact = append(series1ExpSamplesPreCompact, s) + series1ExpSamplesPostCompact = append(series1ExpSamplesPostCompact, copyWithCounterReset(s, histogram.NotCounterReset)) + } + + // Add sample that will be detected as a counter reset. + s = addSample(165, series1, 100000, histogram.UnknownCounterReset) + // Before compaction, sample has an UnknownCounterReset header due to the chainSampleIterator. + series1ExpSamplesPreCompact = append(series1ExpSamplesPreCompact, s) + // After compaction, the sample's counter reset is still UnknownCounterReset as we cannot trust CounterReset + // headers in chunks at the moment, so when reading the first sample in a chunk, its hint is set to + // UnknownCounterReset. + series1ExpSamplesPostCompact = append(series1ExpSamplesPostCompact, s) + + // Add 23 more samples to complete a chunk. + for i := 175; i < 405; i += 10 { + s = addSample(int64(i), series1, 100000+i, histogram.UnknownCounterReset) + // Samples between 205-255 overlap with chunk 1 so before compaction those samples will have the + // UnknownCounterReset header. + if i >= 205 && i < 255 { + s = copyWithCounterReset(s, histogram.NotCounterReset) + } + series1ExpSamplesPreCompact = append(series1ExpSamplesPreCompact, s) + // 245 is the first sample >= the block boundary at 240, so it's still UnknownCounterReset after compaction. + if i != 245 { + s = copyWithCounterReset(s, histogram.NotCounterReset) + } else { + s = copyWithCounterReset(s, histogram.UnknownCounterReset) + } + series1ExpSamplesPostCompact = append(series1ExpSamplesPostCompact, s) + } + + // Chunk 3. + for i := 480; i < 490; i++ { + s = addSample(int64(i), series1, 100000+i, histogram.UnknownCounterReset) + // No overlapping samples in other chunks, so all other samples will already be detected as NotCounterReset + // before compaction. + if i > 480 { + s = copyWithCounterReset(s, histogram.NotCounterReset) + } + series1ExpSamplesPreCompact = append(series1ExpSamplesPreCompact, s) + // 480 is block boundary. + if i == 480 { + s = copyWithCounterReset(s, histogram.UnknownCounterReset) + } + series1ExpSamplesPostCompact = append(series1ExpSamplesPostCompact, s) + } + // Counter reset. + s = addSample(int64(490), series1, 100000, histogram.UnknownCounterReset) + series1ExpSamplesPreCompact = append(series1ExpSamplesPreCompact, s) + series1ExpSamplesPostCompact = append(series1ExpSamplesPostCompact, s) + // Add some more samples after the counter reset. + for i := 491; i < 510; i++ { + s = addSample(int64(i), series1, 100000+i, histogram.UnknownCounterReset) + s = copyWithCounterReset(s, histogram.NotCounterReset) + series1ExpSamplesPreCompact = append(series1ExpSamplesPreCompact, s) + series1ExpSamplesPostCompact = append(series1ExpSamplesPostCompact, s) + } + + // Add samples for series2 - one chunk with one detected counter reset at 300. + for i := 200; i < 300; i += 10 { + s = addSample(int64(i), series2, 100000+i, histogram.UnknownCounterReset) + if i > 200 { + s = copyWithCounterReset(s, histogram.NotCounterReset) + } + series2ExpSamplesPreCompact = append(series2ExpSamplesPreCompact, s) + if i == 240 { + s = copyWithCounterReset(s, histogram.UnknownCounterReset) + } + series2ExpSamplesPostCompact = append(series2ExpSamplesPostCompact, s) + } + // Counter reset. + s = addSample(int64(300), series2, 100000, histogram.UnknownCounterReset) + series2ExpSamplesPreCompact = append(series2ExpSamplesPreCompact, s) + series2ExpSamplesPostCompact = append(series2ExpSamplesPostCompact, s) + // Add some more samples after the counter reset. + for i := 310; i < 500; i += 10 { + s := addSample(int64(i), series2, 100000+i, histogram.UnknownCounterReset) + s = copyWithCounterReset(s, histogram.NotCounterReset) + series2ExpSamplesPreCompact = append(series2ExpSamplesPreCompact, s) + // 360 and 480 are block boundaries. + if i == 360 || i == 480 { + s = copyWithCounterReset(s, histogram.UnknownCounterReset) + } + series2ExpSamplesPostCompact = append(series2ExpSamplesPostCompact, s) + } + + // Sort samples (as OOO samples not added in time-order). + sort.Slice(series1ExpSamplesPreCompact, func(i, j int) bool { + return series1ExpSamplesPreCompact[i].T() < series1ExpSamplesPreCompact[j].T() + }) + sort.Slice(series1ExpSamplesPostCompact, func(i, j int) bool { + return series1ExpSamplesPostCompact[i].T() < series1ExpSamplesPostCompact[j].T() + }) + sort.Slice(series2ExpSamplesPreCompact, func(i, j int) bool { + return series2ExpSamplesPreCompact[i].T() < series2ExpSamplesPreCompact[j].T() + }) + sort.Slice(series2ExpSamplesPostCompact, func(i, j int) bool { + return series2ExpSamplesPostCompact[i].T() < series2ExpSamplesPostCompact[j].T() + }) + + verifyDBSamples := func(s1Samples, s2Samples []chunks.Sample) { + expRes := map[string][]chunks.Sample{ + series1.String(): s1Samples, + series2.String(): s2Samples, + } + + q, err := db.Querier(math.MinInt64, math.MaxInt64) + require.NoError(t, err) + actRes := query(t, q, labels.MustNewMatcher(labels.MatchRegexp, "foo", "bar.*")) + requireEqualSeries(t, expRes, actRes, false) + } + + // Verify DB samples before compaction. + verifyDBSamples(series1ExpSamplesPreCompact, series2ExpSamplesPreCompact) + + // Verify that the in-memory ooo chunk is not empty. + checkNonEmptyOOOChunk := func(lbls labels.Labels) { + ms, created, err := db.head.getOrCreate(lbls.Hash(), lbls, false) + require.NoError(t, err) + require.False(t, created) + require.Positive(t, ms.ooo.oooHeadChunk.chunk.NumSamples()) + } + + checkNonEmptyOOOChunk(series1) + checkNonEmptyOOOChunk(series2) + + // No blocks before compaction. + require.Empty(t, db.Blocks()) + + // There is a 0th WBL file. + require.NoError(t, db.head.wbl.Sync()) // syncing to make sure wbl is flushed in windows + files, err := os.ReadDir(db.head.wbl.Dir()) + require.NoError(t, err) + require.Len(t, files, 1) + require.Equal(t, "00000000", files[0].Name()) + f, err := files[0].Info() + require.NoError(t, err) + require.Greater(t, f.Size(), int64(100)) + + // OOO compaction happens here. + require.NoError(t, db.CompactOOOHead(ctx)) + + // Check that blocks are created after compaction. + require.Len(t, db.Blocks(), 5) + + // Check samples after compaction. + verifyDBSamples(series1ExpSamplesPostCompact, series2ExpSamplesPostCompact) + + // 0th WBL file will be deleted and 1st will be the only present. + files, err = os.ReadDir(db.head.wbl.Dir()) + require.NoError(t, err) + require.Len(t, files, 1) + require.Equal(t, "00000001", files[0].Name()) + f, err = files[0].Info() + require.NoError(t, err) + require.Equal(t, int64(0), f.Size()) + + // OOO stuff should not be present in the Head now. + checkEmptyOOOChunk(series1) + checkEmptyOOOChunk(series2) + + verifyBlockSamples := func(block *Block, fromMins, toMins int64) { + var series1Samples, series2Samples []chunks.Sample + + for _, s := range series1ExpSamplesPostCompact { + if s.T() >= fromMins*time.Minute.Milliseconds() { + // Samples should be sorted, so break out of loop when we reach a timestamp that's too big. + if s.T() > toMins*time.Minute.Milliseconds() { + break + } + series1Samples = append(series1Samples, s) + } + } + for _, s := range series2ExpSamplesPostCompact { + if s.T() >= fromMins*time.Minute.Milliseconds() { + // Samples should be sorted, so break out of loop when we reach a timestamp that's too big. + if s.T() > toMins*time.Minute.Milliseconds() { + break + } + series2Samples = append(series2Samples, s) + } + } + + expRes := map[string][]chunks.Sample{} + if len(series1Samples) != 0 { + expRes[series1.String()] = series1Samples + } + if len(series2Samples) != 0 { + expRes[series2.String()] = series2Samples + } + + q, err := NewBlockQuerier(block, math.MinInt64, math.MaxInt64) + require.NoError(t, err) + + actRes := query(t, q, labels.MustNewMatcher(labels.MatchRegexp, "foo", "bar.*")) + requireEqualSeries(t, expRes, actRes, false) + } + + // Checking for expected data in the blocks. + verifyBlockSamples(db.Blocks()[0], 100, 119) + verifyBlockSamples(db.Blocks()[1], 120, 239) + verifyBlockSamples(db.Blocks()[2], 240, 359) + verifyBlockSamples(db.Blocks()[3], 360, 479) + verifyBlockSamples(db.Blocks()[4], 480, 509) + + // There should be a single m-map file. + mmapDir := mmappedChunksDir(db.head.opts.ChunkDirRoot) + files, err = os.ReadDir(mmapDir) + require.NoError(t, err) + require.Len(t, files, 1) + + // Compact the in-order head and expect another block. + // Since this is a forced compaction, this block is not aligned with 2h. + err = db.CompactHead(NewRangeHead(db.head, 500*time.Minute.Milliseconds(), 550*time.Minute.Milliseconds())) + require.NoError(t, err) + require.Len(t, db.Blocks(), 6) + verifyBlockSamples(db.Blocks()[5], 520, 520) + + // Blocks created out of normal and OOO head now. But not merged. + verifyDBSamples(series1ExpSamplesPostCompact, series2ExpSamplesPostCompact) + + // The compaction also clears out the old m-map files. Including + // the file that has ooo chunks. + files, err = os.ReadDir(mmapDir) + require.NoError(t, err) + require.Len(t, files, 1) + require.Equal(t, "000001", files[0].Name()) + + // This will merge overlapping block. + require.NoError(t, db.Compact(ctx)) + + require.Len(t, db.Blocks(), 5) + verifyBlockSamples(db.Blocks()[0], 100, 119) + verifyBlockSamples(db.Blocks()[1], 120, 239) + verifyBlockSamples(db.Blocks()[2], 240, 359) + verifyBlockSamples(db.Blocks()[3], 360, 479) + verifyBlockSamples(db.Blocks()[4], 480, 520) // Merged block. + + // Final state. Blocks from normal and OOO head are merged. + verifyDBSamples(series1ExpSamplesPostCompact, series2ExpSamplesPostCompact) + } +} + +func TestInterleavedInOrderAndOOOHistogramCompactionWithCounterResets(t *testing.T) { + for _, floatHistogram := range []bool{false, true} { + ctx := context.Background() + + opts := DefaultOptions() + opts.OutOfOrderCapMax = 30 + opts.OutOfOrderTimeWindow = 500 * time.Minute.Milliseconds() + + db := newTestDB(t, withOpts(opts)) + db.DisableCompactions() // We want to manually call it. + + series1 := labels.FromStrings("foo", "bar1") + + addSample := func(ts int64, l labels.Labels, val int) sample { + app := db.Appender(context.Background()) + tsMs := ts + if floatHistogram { + h := tsdbutil.GenerateTestFloatHistogram(int64(val)) + _, err := app.AppendHistogram(0, l, tsMs, nil, h) + require.NoError(t, err) + require.NoError(t, app.Commit()) + return sample{t: tsMs, fh: h.Copy()} + } + + h := tsdbutil.GenerateTestHistogram(int64(val)) + _, err := app.AppendHistogram(0, l, tsMs, h, nil) + require.NoError(t, err) + require.NoError(t, app.Commit()) + return sample{t: tsMs, h: h.Copy()} + } + + var expSamples []chunks.Sample + + s := addSample(0, series1, 0) + expSamples = append(expSamples, s) + s = addSample(1, series1, 10) + expSamples = append(expSamples, copyWithCounterReset(s, histogram.NotCounterReset)) + s = addSample(3, series1, 3) + expSamples = append(expSamples, copyWithCounterReset(s, histogram.UnknownCounterReset)) + s = addSample(2, series1, 0) + expSamples = append(expSamples, copyWithCounterReset(s, histogram.UnknownCounterReset)) + + // Sort samples (as OOO samples not added in time-order). + sort.Slice(expSamples, func(i, j int) bool { + return expSamples[i].T() < expSamples[j].T() + }) + + verifyDBSamples := func(s1Samples []chunks.Sample) { + t.Helper() + expRes := map[string][]chunks.Sample{ + series1.String(): s1Samples, + } + + q, err := db.Querier(math.MinInt64, math.MaxInt64) + require.NoError(t, err) + actRes := query(t, q, labels.MustNewMatcher(labels.MatchRegexp, "foo", "bar.*")) + requireEqualSeries(t, expRes, actRes, false) + } + + // Verify DB samples before compaction. + verifyDBSamples(expSamples) + + require.NoError(t, db.CompactOOOHead(ctx)) + + // Check samples after OOO compaction. + verifyDBSamples(expSamples) + + // Checking for expected data in the blocks. + // Check that blocks are created after compaction. + require.Len(t, db.Blocks(), 1) + + // Compact the in-order head and expect another block. + // Since this is a forced compaction, this block is not aligned with 2h. + require.NoError(t, db.CompactHead(NewRangeHead(db.head, 0, 3))) + require.Len(t, db.Blocks(), 2) + + // Blocks created out of normal and OOO head now. But not merged. + verifyDBSamples(expSamples) + + // This will merge overlapping block. + require.NoError(t, db.Compact(ctx)) + + require.Len(t, db.Blocks(), 1) + + // Final state. Blocks from normal and OOO head are merged. + verifyDBSamples(expSamples) + } +} + +func copyWithCounterReset(s sample, hint histogram.CounterResetHint) sample { + if s.h != nil { + h := s.h.Copy() + h.CounterResetHint = hint + return sample{t: s.t, h: h} + } + + h := s.fh.Copy() + h.CounterResetHint = hint + return sample{t: s.t, fh: h} +} + +func TestOOOCompactionFailure(t *testing.T) { + t.Parallel() + for name, scenario := range sampleTypeScenarios { + t.Run(name, func(t *testing.T) { + testOOOCompactionFailure(t, scenario) + }) + } +} + +func testOOOCompactionFailure(t *testing.T, scenario sampleTypeScenario) { + ctx := context.Background() + + opts := DefaultOptions() + opts.OutOfOrderCapMax = 30 + opts.OutOfOrderTimeWindow = 300 * time.Minute.Milliseconds() + db := newTestDB(t, withOpts(opts)) + db.DisableCompactions() // We want to manually call it. + + series1 := labels.FromStrings("foo", "bar1") + + addSample := func(fromMins, toMins int64) { + app := db.Appender(context.Background()) + for m := fromMins; m <= toMins; m++ { + ts := m * time.Minute.Milliseconds() + _, _, err := scenario.appendFunc(app, series1, ts, ts) + require.NoError(t, err) + } + require.NoError(t, app.Commit()) + } + + // Add an in-order samples. + addSample(250, 350) + + // Add ooo samples that creates multiple chunks. + addSample(90, 310) + + // No blocks before compaction. + require.Empty(t, db.Blocks()) + + // There is a 0th WBL file. + verifyFirstWBLFileIs0 := func(count int) { + require.NoError(t, db.head.wbl.Sync()) // Syncing to make sure wbl is flushed in windows. + files, err := os.ReadDir(db.head.wbl.Dir()) + require.NoError(t, err) + require.Len(t, files, count) + require.Equal(t, "00000000", files[0].Name()) + f, err := files[0].Info() + require.NoError(t, err) + require.Greater(t, f.Size(), int64(100)) + } + verifyFirstWBLFileIs0(1) + + verifyMmapFiles := func(exp ...string) { + mmapDir := mmappedChunksDir(db.head.opts.ChunkDirRoot) + files, err := os.ReadDir(mmapDir) + require.NoError(t, err) + require.Len(t, files, len(exp)) + for i, f := range files { + require.Equal(t, exp[i], f.Name()) + } + } + + verifyMmapFiles("000001") + + // OOO compaction fails 5 times. + originalCompactor := db.compactor + db.compactor = &mockCompactorFailing{t: t} + for range 5 { + require.Error(t, db.CompactOOOHead(ctx)) + } + require.Empty(t, db.Blocks()) + + // M-map files don't change after failed compaction. + verifyMmapFiles("000001") + + // Because of 5 compaction attempts, there are 6 files now. + verifyFirstWBLFileIs0(6) + + db.compactor = originalCompactor + require.NoError(t, db.CompactOOOHead(ctx)) + oldBlocks := db.Blocks() + require.Len(t, db.Blocks(), 3) + + // Check that the ooo chunks were removed. + ms, created, err := db.head.getOrCreate(series1.Hash(), series1, false) + require.NoError(t, err) + require.False(t, created) + require.Nil(t, ms.ooo) + + // The failed compaction should not have left the ooo Head corrupted. + // Hence, expect no new blocks with another OOO compaction call. + require.NoError(t, db.CompactOOOHead(ctx)) + require.Len(t, db.Blocks(), 3) + require.Equal(t, oldBlocks, db.Blocks()) + + // There should be a single m-map file. + verifyMmapFiles("000001") + + // All but last WBL file will be deleted. + // 8 files in total (starting at 0) because of 7 compaction calls. + files, err := os.ReadDir(db.head.wbl.Dir()) + require.NoError(t, err) + require.Len(t, files, 1) + require.Equal(t, "00000007", files[0].Name()) + f, err := files[0].Info() + require.NoError(t, err) + require.Equal(t, int64(0), f.Size()) + + verifySamples := func(block *Block, fromMins, toMins int64) { + series1Samples := make([]chunks.Sample, 0, toMins-fromMins+1) + for m := fromMins; m <= toMins; m++ { + ts := m * time.Minute.Milliseconds() + series1Samples = append(series1Samples, scenario.sampleFunc(ts, ts)) + } + expRes := map[string][]chunks.Sample{ + series1.String(): series1Samples, + } + + q, err := NewBlockQuerier(block, math.MinInt64, math.MaxInt64) + require.NoError(t, err) + actRes := query(t, q, labels.MustNewMatcher(labels.MatchRegexp, "foo", "bar.*")) + requireEqualSeries(t, expRes, actRes, true) + } + + // Checking for expected data in the blocks. + verifySamples(db.Blocks()[0], 90, 119) + verifySamples(db.Blocks()[1], 120, 239) + verifySamples(db.Blocks()[2], 240, 310) + + // Compact the in-order head and expect another block. + // Since this is a forced compaction, this block is not aligned with 2h. + err = db.CompactHead(NewRangeHead(db.head, 250*time.Minute.Milliseconds(), 350*time.Minute.Milliseconds())) + require.NoError(t, err) + require.Len(t, db.Blocks(), 4) // [0, 120), [120, 240), [240, 360), [250, 351) + verifySamples(db.Blocks()[3], 250, 350) + + // The compaction also clears out the old m-map files. Including + // the file that has ooo chunks. + verifyMmapFiles("000001") +} + +func TestWBLCorruption(t *testing.T) { + opts := DefaultOptions() + opts.OutOfOrderCapMax = 30 + opts.OutOfOrderTimeWindow = 300 * time.Minute.Milliseconds() + + db := newTestDB(t, withOpts(opts)) + + series1 := labels.FromStrings("foo", "bar1") + var allSamples, expAfterRestart []chunks.Sample + addSamples := func(fromMins, toMins int64, afterRestart bool) { + app := db.Appender(context.Background()) + for m := fromMins; m <= toMins; m++ { + ts := m * time.Minute.Milliseconds() + _, err := app.Append(0, series1, ts, float64(ts)) + require.NoError(t, err) + allSamples = append(allSamples, sample{t: ts, f: float64(ts)}) + if afterRestart { + expAfterRestart = append(expAfterRestart, sample{t: ts, f: float64(ts)}) + } + } + require.NoError(t, app.Commit()) + } + + // Add an in-order samples. + addSamples(340, 350, true) + + // OOO samples. + addSamples(90, 99, true) + addSamples(100, 119, true) + addSamples(120, 130, true) + + // Moving onto the second file. + _, err := db.head.wbl.NextSegment() + require.NoError(t, err) + + // More OOO samples. + addSamples(200, 230, true) + addSamples(240, 255, true) + + // We corrupt WBL after the sample at 255. So everything added later + // should be deleted after replay. + + // Checking where we corrupt it. + require.NoError(t, db.head.wbl.Sync()) // Syncing to make sure wbl is flushed in windows. + files, err := os.ReadDir(db.head.wbl.Dir()) + require.NoError(t, err) + require.Len(t, files, 2) + f1, err := files[1].Info() + require.NoError(t, err) + corruptIndex := f1.Size() + corruptFilePath := path.Join(db.head.wbl.Dir(), files[1].Name()) + + // Corrupt the WBL by adding a malformed record. + require.NoError(t, db.head.wbl.Log([]byte{byte(record.Samples), 99, 9, 99, 9, 99, 9, 99})) + + // More samples after the corruption point. + addSamples(260, 280, false) + addSamples(290, 300, false) + + // Another file. + _, err = db.head.wbl.NextSegment() + require.NoError(t, err) + + addSamples(310, 320, false) + + // Verifying that we have data after corruption point. + require.NoError(t, db.head.wbl.Sync()) // Syncing to make sure wbl is flushed in windows. + files, err = os.ReadDir(db.head.wbl.Dir()) + require.NoError(t, err) + require.Len(t, files, 3) + f1, err = files[1].Info() + require.NoError(t, err) + require.Greater(t, f1.Size(), corruptIndex) + f0, err := files[0].Info() + require.NoError(t, err) + require.Greater(t, f0.Size(), int64(100)) + f2, err := files[2].Info() + require.NoError(t, err) + require.Greater(t, f2.Size(), int64(100)) + + verifySamples := func(expSamples []chunks.Sample) { + sort.Slice(expSamples, func(i, j int) bool { + return expSamples[i].T() < expSamples[j].T() + }) + + expRes := map[string][]chunks.Sample{ + series1.String(): expSamples, + } + + q, err := db.Querier(math.MinInt64, math.MaxInt64) + require.NoError(t, err) + + actRes := query(t, q, labels.MustNewMatcher(labels.MatchRegexp, "foo", "bar.*")) + require.Equal(t, expRes, actRes) + } + + verifySamples(allSamples) + + require.NoError(t, db.Close()) + + // We want everything to be replayed from the WBL. So we delete the m-map files. + require.NoError(t, os.RemoveAll(mmappedChunksDir(db.head.opts.ChunkDirRoot))) + + // Restart does the replay and repair. + db = newTestDB(t, withDir(db.Dir()), withOpts(opts)) + require.NoError(t, err) + require.Equal(t, 1.0, prom_testutil.ToFloat64(db.head.metrics.walCorruptionsTotal)) + require.Less(t, len(expAfterRestart), len(allSamples)) + verifySamples(expAfterRestart) + + // Verify that it did the repair on disk. + files, err = os.ReadDir(db.head.wbl.Dir()) + require.NoError(t, err) + require.Len(t, files, 3) + f0, err = files[0].Info() + require.NoError(t, err) + require.Greater(t, f0.Size(), int64(100)) + f2, err = files[2].Info() + require.NoError(t, err) + require.Equal(t, int64(0), f2.Size()) + require.Equal(t, corruptFilePath, path.Join(db.head.wbl.Dir(), files[1].Name())) + + // Verifying that everything after the corruption point is set to 0. + b, err := os.ReadFile(corruptFilePath) + require.NoError(t, err) + sum := 0 + for _, val := range b[corruptIndex:] { + sum += int(val) + } + require.Equal(t, 0, sum) + + // Another restart, everything normal with no repair. + require.NoError(t, db.Close()) + db = newTestDB(t, withDir(db.Dir()), withOpts(opts)) + require.NoError(t, err) + require.Equal(t, 0.0, prom_testutil.ToFloat64(db.head.metrics.walCorruptionsTotal)) + verifySamples(expAfterRestart) +} + +func TestOOOMmapCorruption(t *testing.T) { + for name, scenario := range sampleTypeScenarios { + t.Run(name, func(t *testing.T) { + testOOOMmapCorruption(t, scenario) + }) + } +} + +func testOOOMmapCorruption(t *testing.T, scenario sampleTypeScenario) { + opts := DefaultOptions() + opts.OutOfOrderCapMax = 10 + opts.OutOfOrderTimeWindow = 300 * time.Minute.Milliseconds() + + db := newTestDB(t, withOpts(opts)) + + series1 := labels.FromStrings("foo", "bar1") + var allSamples, expInMmapChunks []chunks.Sample + addSamples := func(fromMins, toMins int64, inMmapAfterCorruption bool) { + app := db.Appender(context.Background()) + for m := fromMins; m <= toMins; m++ { + ts := m * time.Minute.Milliseconds() + _, s, err := scenario.appendFunc(app, series1, ts, ts) + require.NoError(t, err) + allSamples = append(allSamples, s) + if inMmapAfterCorruption { + expInMmapChunks = append(expInMmapChunks, s) + } + } + require.NoError(t, app.Commit()) + } + + // Add an in-order samples. + addSamples(340, 350, true) + + // OOO samples. + addSamples(90, 99, true) + addSamples(100, 109, true) + // This sample m-maps a chunk. But 120 goes into a new chunk. + addSamples(120, 120, false) + + // Second m-map file. We will corrupt this file. Sample 120 goes into this new file. + db.head.chunkDiskMapper.CutNewFile() + + // More OOO samples. + addSamples(200, 230, false) + addSamples(240, 255, false) + + db.head.chunkDiskMapper.CutNewFile() + addSamples(260, 290, false) + + verifySamples := func(expSamples []chunks.Sample) { + sort.Slice(expSamples, func(i, j int) bool { + return expSamples[i].T() < expSamples[j].T() + }) + + expRes := map[string][]chunks.Sample{ + series1.String(): expSamples, + } + + q, err := db.Querier(math.MinInt64, math.MaxInt64) + require.NoError(t, err) + + actRes := query(t, q, labels.MustNewMatcher(labels.MatchRegexp, "foo", "bar.*")) + requireEqualSeries(t, expRes, actRes, true) + } + + verifySamples(allSamples) + + // Verifying existing files. + mmapDir := mmappedChunksDir(db.head.opts.ChunkDirRoot) + files, err := os.ReadDir(mmapDir) + require.NoError(t, err) + require.Len(t, files, 3) + + // Corrupting the 2nd file. + f, err := os.OpenFile(path.Join(mmapDir, files[1].Name()), os.O_RDWR, 0o666) + require.NoError(t, err) + _, err = f.WriteAt([]byte{99, 9, 99, 9, 99}, 20) + require.NoError(t, err) + require.NoError(t, f.Close()) + firstFileName := files[0].Name() + + require.NoError(t, db.Close()) + + // Moving OOO WBL to use it later. + wblDir := db.head.wbl.Dir() + wblDirTmp := path.Join(t.TempDir(), "wbl_tmp") + require.NoError(t, os.Rename(wblDir, wblDirTmp)) + + // Restart does the replay and repair of m-map files. + db = newTestDB(t, withDir(db.Dir()), withOpts(opts)) + require.NoError(t, err) + require.Equal(t, 1.0, prom_testutil.ToFloat64(db.head.metrics.mmapChunkCorruptionTotal)) + require.Less(t, len(expInMmapChunks), len(allSamples)) + + // Since there is no WBL, only samples from m-map chunks comes in the query. + verifySamples(expInMmapChunks) + + // Verify that it did the repair on disk. All files from the point of corruption + // should be deleted. + files, err = os.ReadDir(mmapDir) + require.NoError(t, err) + require.Len(t, files, 1) + f0, err := files[0].Info() + require.NoError(t, err) + require.Greater(t, f0.Size(), int64(100)) + require.Equal(t, firstFileName, files[0].Name()) + + // Another restart, everything normal with no repair. + require.NoError(t, db.Close()) + db = newTestDB(t, withDir(db.Dir()), withOpts(opts)) + require.NoError(t, err) + require.Equal(t, 0.0, prom_testutil.ToFloat64(db.head.metrics.mmapChunkCorruptionTotal)) + verifySamples(expInMmapChunks) + + // Restart again with the WBL, all samples should be present now. + require.NoError(t, db.Close()) + require.NoError(t, os.RemoveAll(wblDir)) + require.NoError(t, os.Rename(wblDirTmp, wblDir)) + db = newTestDB(t, withDir(db.Dir()), withOpts(opts)) + require.NoError(t, err) + verifySamples(allSamples) +} + +func TestOutOfOrderRuntimeConfig(t *testing.T) { + t.Parallel() + for name, scenario := range sampleTypeScenarios { + t.Run(name, func(t *testing.T) { + testOutOfOrderRuntimeConfig(t, scenario) + }) + } +} + +func testOutOfOrderRuntimeConfig(t *testing.T, scenario sampleTypeScenario) { + ctx := context.Background() + + getDB := func(oooTimeWindow int64) *DB { + opts := DefaultOptions() + opts.OutOfOrderTimeWindow = oooTimeWindow + db := newTestDB(t, withOpts(opts)) + db.DisableCompactions() + return db + } + + makeConfig := func(oooTimeWindow int) *config.Config { + return &config.Config{ + StorageConfig: config.StorageConfig{ + TSDBConfig: &config.TSDBConfig{ + OutOfOrderTimeWindow: int64(oooTimeWindow) * time.Minute.Milliseconds(), + }, + }, + } + } + + series1 := labels.FromStrings("foo", "bar1") + addSamples := func(t *testing.T, db *DB, fromMins, toMins int64, success bool, allSamples []chunks.Sample) []chunks.Sample { + app := db.Appender(context.Background()) + for m := fromMins; m <= toMins; m++ { + ts := m * time.Minute.Milliseconds() + _, s, err := scenario.appendFunc(app, series1, ts, ts) + if success { + require.NoError(t, err) + allSamples = append(allSamples, s) + } else { + require.Error(t, err) + } + } + require.NoError(t, app.Commit()) + return allSamples + } + + verifySamples := func(t *testing.T, db *DB, expSamples []chunks.Sample) { + sort.Slice(expSamples, func(i, j int) bool { + return expSamples[i].T() < expSamples[j].T() + }) + + expRes := map[string][]chunks.Sample{ + series1.String(): expSamples, + } + + q, err := db.Querier(math.MinInt64, math.MaxInt64) + require.NoError(t, err) + + actRes := query(t, q, labels.MustNewMatcher(labels.MatchRegexp, "foo", "bar.*")) + requireEqualSeries(t, expRes, actRes, true) + } + + doOOOCompaction := func(t *testing.T, db *DB) { + // WBL is not empty. + size, err := db.head.wbl.Size() + require.NoError(t, err) + require.Positive(t, size) + + require.Empty(t, db.Blocks()) + require.NoError(t, db.compactOOOHead(ctx)) + require.NotEmpty(t, db.Blocks()) + + // WBL is empty. + size, err = db.head.wbl.Size() + require.NoError(t, err) + require.Equal(t, int64(0), size) + } + + t.Run("increase time window", func(t *testing.T) { + var allSamples []chunks.Sample + db := getDB(30 * time.Minute.Milliseconds()) + + // In-order. + allSamples = addSamples(t, db, 300, 310, true, allSamples) + + // OOO upto 30m old is success. + allSamples = addSamples(t, db, 281, 290, true, allSamples) + + // OOO of 59m old fails. + s := addSamples(t, db, 251, 260, false, nil) + require.Empty(t, s) + verifySamples(t, db, allSamples) + + oldWblPtr := fmt.Sprintf("%p", db.head.wbl) + + // Increase time window and try adding again. + err := db.ApplyConfig(makeConfig(60)) + require.NoError(t, err) + allSamples = addSamples(t, db, 251, 260, true, allSamples) + + // WBL does not change. + newWblPtr := fmt.Sprintf("%p", db.head.wbl) + require.Equal(t, oldWblPtr, newWblPtr) + + doOOOCompaction(t, db) + verifySamples(t, db, allSamples) + }) + + t.Run("decrease time window and increase again", func(t *testing.T) { + var allSamples []chunks.Sample + db := getDB(60 * time.Minute.Milliseconds()) + + // In-order. + allSamples = addSamples(t, db, 300, 310, true, allSamples) + + // OOO upto 59m old is success. + allSamples = addSamples(t, db, 251, 260, true, allSamples) + + oldWblPtr := fmt.Sprintf("%p", db.head.wbl) + // Decrease time window. + err := db.ApplyConfig(makeConfig(30)) + require.NoError(t, err) + + // OOO of 49m old fails. + s := addSamples(t, db, 261, 270, false, nil) + require.Empty(t, s) + + // WBL does not change. + newWblPtr := fmt.Sprintf("%p", db.head.wbl) + require.Equal(t, oldWblPtr, newWblPtr) + + verifySamples(t, db, allSamples) + + // Increase time window again and check + err = db.ApplyConfig(makeConfig(60)) + require.NoError(t, err) + allSamples = addSamples(t, db, 261, 270, true, allSamples) + verifySamples(t, db, allSamples) + + // WBL does not change. + newWblPtr = fmt.Sprintf("%p", db.head.wbl) + require.Equal(t, oldWblPtr, newWblPtr) + + doOOOCompaction(t, db) + verifySamples(t, db, allSamples) + }) + + t.Run("disabled to enabled", func(t *testing.T) { + var allSamples []chunks.Sample + db := getDB(0) + + // In-order. + allSamples = addSamples(t, db, 300, 310, true, allSamples) + + // OOO fails. + s := addSamples(t, db, 251, 260, false, nil) + require.Empty(t, s) + verifySamples(t, db, allSamples) + + require.Nil(t, db.head.wbl) + + // Increase time window and try adding again. + err := db.ApplyConfig(makeConfig(60)) + require.NoError(t, err) + allSamples = addSamples(t, db, 251, 260, true, allSamples) + + // WBL gets created. + require.NotNil(t, db.head.wbl) + + verifySamples(t, db, allSamples) + + // OOO compaction works now. + doOOOCompaction(t, db) + verifySamples(t, db, allSamples) + }) + + t.Run("enabled to disabled", func(t *testing.T) { + var allSamples []chunks.Sample + db := getDB(60 * time.Minute.Milliseconds()) + + // In-order. + allSamples = addSamples(t, db, 300, 310, true, allSamples) + + // OOO upto 59m old is success. + allSamples = addSamples(t, db, 251, 260, true, allSamples) + + oldWblPtr := fmt.Sprintf("%p", db.head.wbl) + // Time Window to 0, hence disabled. + err := db.ApplyConfig(makeConfig(0)) + require.NoError(t, err) + + // OOO within old time window fails. + s := addSamples(t, db, 290, 309, false, nil) + require.Empty(t, s) + + // WBL does not change and is not removed. + newWblPtr := fmt.Sprintf("%p", db.head.wbl) + require.Equal(t, oldWblPtr, newWblPtr) + + verifySamples(t, db, allSamples) + + // Compaction still works after disabling with WBL cleanup. + doOOOCompaction(t, db) + verifySamples(t, db, allSamples) + }) + + t.Run("disabled to disabled", func(t *testing.T) { + var allSamples []chunks.Sample + db := getDB(0) + + // In-order. + allSamples = addSamples(t, db, 300, 310, true, allSamples) + + // OOO fails. + s := addSamples(t, db, 290, 309, false, nil) + require.Empty(t, s) + verifySamples(t, db, allSamples) + require.Nil(t, db.head.wbl) + + // Time window to 0. + err := db.ApplyConfig(makeConfig(0)) + require.NoError(t, err) + + // OOO still fails. + s = addSamples(t, db, 290, 309, false, nil) + require.Empty(t, s) + verifySamples(t, db, allSamples) + require.Nil(t, db.head.wbl) + }) +} + +func TestNoGapAfterRestartWithOOO(t *testing.T) { + t.Parallel() + for name, scenario := range sampleTypeScenarios { + t.Run(name, func(t *testing.T) { + testNoGapAfterRestartWithOOO(t, scenario) + }) + } +} + +func testNoGapAfterRestartWithOOO(t *testing.T, scenario sampleTypeScenario) { + series1 := labels.FromStrings("foo", "bar1") + addSamples := func(t *testing.T, db *DB, fromMins, toMins int64, success bool) { + app := db.Appender(context.Background()) + for m := fromMins; m <= toMins; m++ { + ts := m * time.Minute.Milliseconds() + _, _, err := scenario.appendFunc(app, series1, ts, ts) + if success { + require.NoError(t, err) + } else { + require.Error(t, err) + } + } + require.NoError(t, app.Commit()) + } + + verifySamples := func(t *testing.T, db *DB, fromMins, toMins int64) { + var expSamples []chunks.Sample + for m := fromMins; m <= toMins; m++ { + ts := m * time.Minute.Milliseconds() + expSamples = append(expSamples, scenario.sampleFunc(ts, ts)) + } + + expRes := map[string][]chunks.Sample{ + series1.String(): expSamples, + } + + q, err := db.Querier(math.MinInt64, math.MaxInt64) + require.NoError(t, err) + + actRes := query(t, q, labels.MustNewMatcher(labels.MatchRegexp, "foo", "bar.*")) + requireEqualSeries(t, expRes, actRes, true) + } + + cases := []struct { + inOrderMint, inOrderMaxt int64 + oooMint, oooMaxt int64 + // After compaction. + blockRanges [][2]int64 + headMint, headMaxt int64 + }{ + { + 300, 490, + 489, 489, + [][2]int64{{300, 360}, {480, 600}}, + 360, 490, + }, + { + 300, 490, + 479, 479, + [][2]int64{{300, 360}, {360, 480}}, + 360, 490, + }, + } + + for i, c := range cases { + t.Run(fmt.Sprintf("case=%d", i), func(t *testing.T) { + ctx := context.Background() + + opts := DefaultOptions() + opts.OutOfOrderTimeWindow = 30 * time.Minute.Milliseconds() + db := newTestDB(t, withOpts(opts)) + db.DisableCompactions() + + // 3h10m=190m worth in-order data. + addSamples(t, db, c.inOrderMint, c.inOrderMaxt, true) + verifySamples(t, db, c.inOrderMint, c.inOrderMaxt) + + // One ooo samples. + addSamples(t, db, c.oooMint, c.oooMaxt, true) + verifySamples(t, db, c.inOrderMint, c.inOrderMaxt) + + // We get 2 blocks. 1 from OOO, 1 from in-order. + require.NoError(t, db.Compact(ctx)) + verifyBlockRanges := func() { + blocks := db.Blocks() + require.Len(t, blocks, len(c.blockRanges)) + for j, br := range c.blockRanges { + require.Equal(t, br[0]*time.Minute.Milliseconds(), blocks[j].MinTime()) + require.Equal(t, br[1]*time.Minute.Milliseconds(), blocks[j].MaxTime()) + } + } + verifyBlockRanges() + require.Equal(t, c.headMint*time.Minute.Milliseconds(), db.head.MinTime()) + require.Equal(t, c.headMaxt*time.Minute.Milliseconds(), db.head.MaxTime()) + + // Restart and expect all samples to be present. + require.NoError(t, db.Close()) + + db = newTestDB(t, withDir(db.Dir()), withOpts(opts)) + db.DisableCompactions() + + verifyBlockRanges() + require.Equal(t, c.headMint*time.Minute.Milliseconds(), db.head.MinTime()) + require.Equal(t, c.headMaxt*time.Minute.Milliseconds(), db.head.MaxTime()) + verifySamples(t, db, c.inOrderMint, c.inOrderMaxt) + }) + } +} + +func TestWblReplayAfterOOODisableAndRestart(t *testing.T) { + for name, scenario := range sampleTypeScenarios { + t.Run(name, func(t *testing.T) { + testWblReplayAfterOOODisableAndRestart(t, scenario) + }) + } +} + +func testWblReplayAfterOOODisableAndRestart(t *testing.T, scenario sampleTypeScenario) { + opts := DefaultOptions() + opts.OutOfOrderTimeWindow = 60 * time.Minute.Milliseconds() + + db := newTestDB(t, withOpts(opts)) + + series1 := labels.FromStrings("foo", "bar1") + var allSamples []chunks.Sample + addSamples := func(fromMins, toMins int64) { + app := db.Appender(context.Background()) + for m := fromMins; m <= toMins; m++ { + ts := m * time.Minute.Milliseconds() + _, s, err := scenario.appendFunc(app, series1, ts, ts) + require.NoError(t, err) + allSamples = append(allSamples, s) + } + require.NoError(t, app.Commit()) + } + + // In-order samples. + addSamples(290, 300) + // OOO samples. + addSamples(250, 260) + + verifySamples := func(expSamples []chunks.Sample) { + sort.Slice(expSamples, func(i, j int) bool { + return expSamples[i].T() < expSamples[j].T() + }) + + expRes := map[string][]chunks.Sample{ + series1.String(): expSamples, + } + + q, err := db.Querier(math.MinInt64, math.MaxInt64) + require.NoError(t, err) + + actRes := query(t, q, labels.MustNewMatcher(labels.MatchRegexp, "foo", "bar.*")) + requireEqualSeries(t, expRes, actRes, true) + } + + verifySamples(allSamples) + + // Restart DB with OOO disabled. + require.NoError(t, db.Close()) + + opts.OutOfOrderTimeWindow = 0 + db = newTestDB(t, withDir(db.Dir()), withOpts(opts)) + + // We can still query OOO samples when OOO is disabled. + verifySamples(allSamples) +} + +func TestPanicOnApplyConfig(t *testing.T) { + for name, scenario := range sampleTypeScenarios { + t.Run(name, func(t *testing.T) { + testPanicOnApplyConfig(t, scenario) + }) + } +} + +func testPanicOnApplyConfig(t *testing.T, scenario sampleTypeScenario) { + opts := DefaultOptions() + opts.OutOfOrderTimeWindow = 60 * time.Minute.Milliseconds() + + db := newTestDB(t, withOpts(opts)) + + series1 := labels.FromStrings("foo", "bar1") + var allSamples []chunks.Sample + addSamples := func(fromMins, toMins int64) { + app := db.Appender(context.Background()) + for m := fromMins; m <= toMins; m++ { + ts := m * time.Minute.Milliseconds() + _, s, err := scenario.appendFunc(app, series1, ts, ts) + require.NoError(t, err) + allSamples = append(allSamples, s) + } + require.NoError(t, app.Commit()) + } + + // In-order samples. + addSamples(290, 300) + // OOO samples. + addSamples(250, 260) + + // Restart DB with OOO disabled. + require.NoError(t, db.Close()) + + opts.OutOfOrderTimeWindow = 0 + db = newTestDB(t, withDir(db.Dir()), withOpts(opts)) + + // ApplyConfig with OOO enabled and expect no panic. + err := db.ApplyConfig(&config.Config{ + StorageConfig: config.StorageConfig{ + TSDBConfig: &config.TSDBConfig{ + OutOfOrderTimeWindow: 60 * time.Minute.Milliseconds(), + }, + }, + }) + require.NoError(t, err) +} + +func TestDiskFillingUpAfterDisablingOOO(t *testing.T) { + t.Parallel() + for name, scenario := range sampleTypeScenarios { + t.Run(name, func(t *testing.T) { + testDiskFillingUpAfterDisablingOOO(t, scenario) + }) + } +} + +func testDiskFillingUpAfterDisablingOOO(t *testing.T, scenario sampleTypeScenario) { + t.Parallel() + ctx := context.Background() + + opts := DefaultOptions() + opts.OutOfOrderTimeWindow = 60 * time.Minute.Milliseconds() + + db := newTestDB(t, withOpts(opts)) + db.DisableCompactions() + + series1 := labels.FromStrings("foo", "bar1") + var allSamples []chunks.Sample + addSamples := func(fromMins, toMins int64) { + app := db.Appender(context.Background()) + for m := fromMins; m <= toMins; m++ { + ts := m * time.Minute.Milliseconds() + _, s, err := scenario.appendFunc(app, series1, ts, ts) + require.NoError(t, err) + allSamples = append(allSamples, s) + } + require.NoError(t, app.Commit()) + } + + // In-order samples. + addSamples(290, 300) + // OOO samples. + addSamples(250, 299) + + // Restart DB with OOO disabled. + require.NoError(t, db.Close()) + + opts.OutOfOrderTimeWindow = 0 + db = newTestDB(t, withDir(db.Dir()), withOpts(opts)) + db.DisableCompactions() + + ms := db.head.series.getByHash(series1.Hash(), series1) + require.NotEmpty(t, ms.ooo.oooMmappedChunks, "OOO mmap chunk was not replayed") + + checkMmapFileContents := func(contains, notContains []string) { + mmapDir := mmappedChunksDir(db.head.opts.ChunkDirRoot) + files, err := os.ReadDir(mmapDir) + require.NoError(t, err) + + fnames := make([]string, 0, len(files)) + for _, f := range files { + fnames = append(fnames, f.Name()) + } + + for _, f := range contains { + require.Contains(t, fnames, f) + } + for _, f := range notContains { + require.NotContains(t, fnames, f) + } + } + + // Add in-order samples until ready for compaction.. + addSamples(301, 500) + + // Check that m-map files gets deleted properly after compactions. + + db.head.mmapHeadChunks() + checkMmapFileContents([]string{"000001", "000002"}, nil) + require.NoError(t, db.Compact(ctx)) + checkMmapFileContents([]string{"000002"}, []string{"000001"}) + require.Nil(t, ms.ooo, "OOO mmap chunk was not compacted") + + addSamples(501, 650) + db.head.mmapHeadChunks() + checkMmapFileContents([]string{"000002", "000003"}, []string{"000001"}) + require.NoError(t, db.Compact(ctx)) + checkMmapFileContents(nil, []string{"000001", "000002", "000003"}) + + // Verify that WBL is empty. + files, err := os.ReadDir(db.head.wbl.Dir()) + require.NoError(t, err) + require.Len(t, files, 1) // Last empty file after compaction. + finfo, err := files[0].Info() + require.NoError(t, err) + require.Equal(t, int64(0), finfo.Size()) +} + +func TestHistogramAppendAndQuery(t *testing.T) { + t.Run("integer histograms", func(t *testing.T) { + testHistogramAppendAndQueryHelper(t, false) + }) + t.Run("float histograms", func(t *testing.T) { + testHistogramAppendAndQueryHelper(t, true) + }) +} + +func testHistogramAppendAndQueryHelper(t *testing.T, floatHistogram bool) { + t.Helper() + db := newTestDB(t) + minute := func(m int) int64 { return int64(m) * time.Minute.Milliseconds() } + + ctx := context.Background() + appendHistogram := func(t *testing.T, + lbls labels.Labels, tsMinute int, h *histogram.Histogram, + exp *[]chunks.Sample, expCRH histogram.CounterResetHint, + ) { + t.Helper() + var err error + app := db.Appender(ctx) + if floatHistogram { + _, err = app.AppendHistogram(0, lbls, minute(tsMinute), nil, h.ToFloat(nil)) + efh := h.ToFloat(nil) + efh.CounterResetHint = expCRH + *exp = append(*exp, sample{t: minute(tsMinute), fh: efh}) + } else { + _, err = app.AppendHistogram(0, lbls, minute(tsMinute), h.Copy(), nil) + eh := h.Copy() + eh.CounterResetHint = expCRH + *exp = append(*exp, sample{t: minute(tsMinute), h: eh}) + } + require.NoError(t, err) + require.NoError(t, app.Commit()) + } + appendFloat := func(t *testing.T, lbls labels.Labels, tsMinute int, val float64, exp *[]chunks.Sample) { + t.Helper() + app := db.Appender(ctx) + _, err := app.Append(0, lbls, minute(tsMinute), val) + require.NoError(t, err) + require.NoError(t, app.Commit()) + *exp = append(*exp, sample{t: minute(tsMinute), f: val}) + } + + testQuery := func(t *testing.T, name, value string, exp map[string][]chunks.Sample) { + t.Helper() + q, err := db.Querier(math.MinInt64, math.MaxInt64) + require.NoError(t, err) + act := query(t, q, labels.MustNewMatcher(labels.MatchRegexp, name, value)) + require.Equal(t, exp, act) + } + + baseH := &histogram.Histogram{ + Count: 15, + ZeroCount: 4, + ZeroThreshold: 0.001, + Sum: 35.5, + Schema: 1, + PositiveSpans: []histogram.Span{ + {Offset: 0, Length: 2}, + {Offset: 2, Length: 2}, + }, + PositiveBuckets: []int64{1, 1, -1, 0}, + NegativeSpans: []histogram.Span{ + {Offset: 0, Length: 1}, + {Offset: 1, Length: 2}, + }, + NegativeBuckets: []int64{1, 2, -1}, + } + + var ( + series1 = labels.FromStrings("foo", "bar1") + series2 = labels.FromStrings("foo", "bar2") + series3 = labels.FromStrings("foo", "bar3") + series4 = labels.FromStrings("foo", "bar4") + exp1, exp2, exp3, exp4 []chunks.Sample + ) + + // TODO(codesome): test everything for negative buckets as well. + t.Run("series with only histograms", func(t *testing.T) { + h := baseH.Copy() // This is shared across all sub tests. + + appendHistogram(t, series1, 100, h, &exp1, histogram.UnknownCounterReset) + testQuery(t, "foo", "bar1", map[string][]chunks.Sample{series1.String(): exp1}) + + h.PositiveBuckets[0]++ + h.NegativeBuckets[0] += 2 + h.Count += 10 + appendHistogram(t, series1, 101, h, &exp1, histogram.NotCounterReset) + testQuery(t, "foo", "bar1", map[string][]chunks.Sample{series1.String(): exp1}) + + t.Run("changing schema", func(t *testing.T) { + h.Schema = 2 + appendHistogram(t, series1, 102, h, &exp1, histogram.UnknownCounterReset) + testQuery(t, "foo", "bar1", map[string][]chunks.Sample{series1.String(): exp1}) + + // Schema back to old. + h.Schema = 1 + appendHistogram(t, series1, 103, h, &exp1, histogram.UnknownCounterReset) + testQuery(t, "foo", "bar1", map[string][]chunks.Sample{series1.String(): exp1}) + }) + + t.Run("new buckets incoming", func(t *testing.T) { + // In the previous unit test, during the last histogram append, we + // changed the schema and that caused a new chunk creation. Because + // of the next append the layout of the last histogram will change + // because the chunk will be re-encoded. So this forces us to modify + // the last histogram in exp1 so when we query we get the expected + // results. + if floatHistogram { + lh := exp1[len(exp1)-1].FH().Copy() + lh.PositiveSpans[1].Length++ + lh.PositiveBuckets = append(lh.PositiveBuckets, 0) + exp1[len(exp1)-1] = sample{t: exp1[len(exp1)-1].T(), fh: lh} + } else { + lh := exp1[len(exp1)-1].H().Copy() + lh.PositiveSpans[1].Length++ + lh.PositiveBuckets = append(lh.PositiveBuckets, -2) // -2 makes the last bucket 0. + exp1[len(exp1)-1] = sample{t: exp1[len(exp1)-1].T(), h: lh} + } + + // This histogram with new bucket at the end causes the re-encoding of the previous histogram. + // Hence the previous histogram is recoded into this new layout. + // But the query returns the histogram from the in-memory buffer, hence we don't see the recode here yet. + h.PositiveSpans[1].Length++ + h.PositiveBuckets = append(h.PositiveBuckets, 1) + h.Count += 3 + appendHistogram(t, series1, 104, h, &exp1, histogram.NotCounterReset) + testQuery(t, "foo", "bar1", map[string][]chunks.Sample{series1.String(): exp1}) + + // Because of the previous two histograms being on the active chunk, + // and the next append is only adding a new bucket, the active chunk + // will be re-encoded to the new layout. + if floatHistogram { + lh := exp1[len(exp1)-2].FH().Copy() + lh.PositiveSpans[0].Length++ + lh.PositiveSpans[1].Offset-- + lh.PositiveBuckets = []float64{2, 3, 0, 2, 2, 0} + exp1[len(exp1)-2] = sample{t: exp1[len(exp1)-2].T(), fh: lh} + + lh = exp1[len(exp1)-1].FH().Copy() + lh.PositiveSpans[0].Length++ + lh.PositiveSpans[1].Offset-- + lh.PositiveBuckets = []float64{2, 3, 0, 2, 2, 3} + exp1[len(exp1)-1] = sample{t: exp1[len(exp1)-1].T(), fh: lh} + } else { + lh := exp1[len(exp1)-2].H().Copy() + lh.PositiveSpans[0].Length++ + lh.PositiveSpans[1].Offset-- + lh.PositiveBuckets = []int64{2, 1, -3, 2, 0, -2} + exp1[len(exp1)-2] = sample{t: exp1[len(exp1)-2].T(), h: lh} + + lh = exp1[len(exp1)-1].H().Copy() + lh.PositiveSpans[0].Length++ + lh.PositiveSpans[1].Offset-- + lh.PositiveBuckets = []int64{2, 1, -3, 2, 0, 1} + exp1[len(exp1)-1] = sample{t: exp1[len(exp1)-1].T(), h: lh} + } + + // Now we add the new buckets in between. Empty bucket is again not present for the old histogram. + h.PositiveSpans[0].Length++ + h.PositiveSpans[1].Offset-- + h.Count += 3 + // {2, 1, -1, 0, 1} -> {2, 1, 0, -1, 0, 1} + h.PositiveBuckets = append(h.PositiveBuckets[:2], append([]int64{0}, h.PositiveBuckets[2:]...)...) + appendHistogram(t, series1, 105, h, &exp1, histogram.NotCounterReset) + testQuery(t, "foo", "bar1", map[string][]chunks.Sample{series1.String(): exp1}) + + // We add 4 more histograms to clear out the buffer and see the re-encoded histograms. + appendHistogram(t, series1, 106, h, &exp1, histogram.NotCounterReset) + appendHistogram(t, series1, 107, h, &exp1, histogram.NotCounterReset) + appendHistogram(t, series1, 108, h, &exp1, histogram.NotCounterReset) + appendHistogram(t, series1, 109, h, &exp1, histogram.NotCounterReset) + + // Update the expected histograms to reflect the re-encoding. + if floatHistogram { + l := len(exp1) + h7 := exp1[l-7].FH() + h7.PositiveSpans = exp1[l-1].FH().PositiveSpans + h7.PositiveBuckets = []float64{2, 3, 0, 2, 2, 0} + exp1[l-7] = sample{t: exp1[l-7].T(), fh: h7} + + h6 := exp1[l-6].FH() + h6.PositiveSpans = exp1[l-1].FH().PositiveSpans + h6.PositiveBuckets = []float64{2, 3, 0, 2, 2, 3} + exp1[l-6] = sample{t: exp1[l-6].T(), fh: h6} + } else { + l := len(exp1) + h7 := exp1[l-7].H() + h7.PositiveSpans = exp1[l-1].H().PositiveSpans + h7.PositiveBuckets = []int64{2, 1, -3, 2, 0, -2} // -3 and -2 are the empty buckets. + exp1[l-7] = sample{t: exp1[l-7].T(), h: h7} + + h6 := exp1[l-6].H() + h6.PositiveSpans = exp1[l-1].H().PositiveSpans + h6.PositiveBuckets = []int64{2, 1, -3, 2, 0, 1} // -3 is the empty bucket. + exp1[l-6] = sample{t: exp1[l-6].T(), h: h6} + } + + testQuery(t, "foo", "bar1", map[string][]chunks.Sample{series1.String(): exp1}) + }) + + t.Run("buckets disappearing", func(t *testing.T) { + h.PositiveSpans[1].Length-- + h.PositiveBuckets = h.PositiveBuckets[:len(h.PositiveBuckets)-1] + h.Count -= 3 + appendHistogram(t, series1, 110, h, &exp1, histogram.UnknownCounterReset) + testQuery(t, "foo", "bar1", map[string][]chunks.Sample{series1.String(): exp1}) + }) + }) + + t.Run("series starting with float and then getting histograms", func(t *testing.T) { + appendFloat(t, series2, 100, 100, &exp2) + appendFloat(t, series2, 101, 101, &exp2) + appendFloat(t, series2, 102, 102, &exp2) + testQuery(t, "foo", "bar2", map[string][]chunks.Sample{series2.String(): exp2}) + + h := baseH.Copy() + appendHistogram(t, series2, 103, h, &exp2, histogram.UnknownCounterReset) + appendHistogram(t, series2, 104, h, &exp2, histogram.NotCounterReset) + appendHistogram(t, series2, 105, h, &exp2, histogram.NotCounterReset) + testQuery(t, "foo", "bar2", map[string][]chunks.Sample{series2.String(): exp2}) + + // Switching between float and histograms again. + appendFloat(t, series2, 106, 106, &exp2) + appendFloat(t, series2, 107, 107, &exp2) + testQuery(t, "foo", "bar2", map[string][]chunks.Sample{series2.String(): exp2}) + + appendHistogram(t, series2, 108, h, &exp2, histogram.UnknownCounterReset) + appendHistogram(t, series2, 109, h, &exp2, histogram.NotCounterReset) + testQuery(t, "foo", "bar2", map[string][]chunks.Sample{series2.String(): exp2}) + }) + + t.Run("series starting with histogram and then getting float", func(t *testing.T) { + h := baseH.Copy() + appendHistogram(t, series3, 101, h, &exp3, histogram.UnknownCounterReset) + appendHistogram(t, series3, 102, h, &exp3, histogram.NotCounterReset) + appendHistogram(t, series3, 103, h, &exp3, histogram.NotCounterReset) + testQuery(t, "foo", "bar3", map[string][]chunks.Sample{series3.String(): exp3}) + + appendFloat(t, series3, 104, 100, &exp3) + appendFloat(t, series3, 105, 101, &exp3) + appendFloat(t, series3, 106, 102, &exp3) + testQuery(t, "foo", "bar3", map[string][]chunks.Sample{series3.String(): exp3}) + + // Switching between histogram and float again. + appendHistogram(t, series3, 107, h, &exp3, histogram.UnknownCounterReset) + appendHistogram(t, series3, 108, h, &exp3, histogram.NotCounterReset) + testQuery(t, "foo", "bar3", map[string][]chunks.Sample{series3.String(): exp3}) + + appendFloat(t, series3, 109, 106, &exp3) + appendFloat(t, series3, 110, 107, &exp3) + testQuery(t, "foo", "bar3", map[string][]chunks.Sample{series3.String(): exp3}) + }) + + t.Run("query mix of histogram and float series", func(t *testing.T) { + // A float only series. + appendFloat(t, series4, 100, 100, &exp4) + appendFloat(t, series4, 101, 101, &exp4) + appendFloat(t, series4, 102, 102, &exp4) + + testQuery(t, "foo", "bar.*", map[string][]chunks.Sample{ + series1.String(): exp1, + series2.String(): exp2, + series3.String(): exp3, + series4.String(): exp4, + }) + }) +} + +func TestQueryHistogramFromBlocksWithCompaction(t *testing.T) { + t.Parallel() + minute := func(m int) int64 { return int64(m) * time.Minute.Milliseconds() } + + testBlockQuerying := func(t *testing.T, blockSeries ...[]storage.Series) { + t.Helper() + + opts := DefaultOptions() + db := newTestDB(t, withOpts(opts)) + + var it chunkenc.Iterator + exp := make(map[string][]chunks.Sample) + for _, series := range blockSeries { + createBlock(t, db.Dir(), series) + + for _, s := range series { + lbls := s.Labels().String() + slice := exp[lbls] + it = s.Iterator(it) + smpls, err := storage.ExpandSamples(it, nil) + require.NoError(t, err) + slice = append(slice, smpls...) + sort.Slice(slice, func(i, j int) bool { + return slice[i].T() < slice[j].T() + }) + exp[lbls] = slice + } + } + + require.Empty(t, db.Blocks()) + require.NoError(t, db.reload()) + require.Len(t, db.Blocks(), len(blockSeries)) + + q, err := db.Querier(math.MinInt64, math.MaxInt64) + require.NoError(t, err) + res := query(t, q, labels.MustNewMatcher(labels.MatchRegexp, "__name__", ".*")) + compareSeries(t, exp, res) + + // Compact all the blocks together and query again. + blocks := db.Blocks() + blockDirs := make([]string, 0, len(blocks)) + for _, b := range blocks { + blockDirs = append(blockDirs, b.Dir()) + } + ids, err := db.compactor.Compact(db.Dir(), blockDirs, blocks) + require.NoError(t, err) + require.Len(t, ids, 1) + require.NoError(t, db.reload()) + require.Len(t, db.Blocks(), 1) + + q, err = db.Querier(math.MinInt64, math.MaxInt64) + require.NoError(t, err) + res = query(t, q, labels.MustNewMatcher(labels.MatchRegexp, "__name__", ".*")) + + // After compaction, we do not require "unknown" counter resets + // due to origin from different overlapping chunks anymore. + for _, ss := range exp { + for i, s := range ss[1:] { + if s.Type() == chunkenc.ValHistogram && ss[i].Type() == chunkenc.ValHistogram && s.H().CounterResetHint == histogram.UnknownCounterReset { + s.H().CounterResetHint = histogram.NotCounterReset + } + if s.Type() == chunkenc.ValFloatHistogram && ss[i].Type() == chunkenc.ValFloatHistogram && s.FH().CounterResetHint == histogram.UnknownCounterReset { + s.FH().CounterResetHint = histogram.NotCounterReset + } + } + } + compareSeries(t, exp, res) + } + + for _, floatHistogram := range []bool{false, true} { + t.Run(fmt.Sprintf("floatHistogram=%t", floatHistogram), func(t *testing.T) { + t.Run("serial blocks with only histograms", func(t *testing.T) { + testBlockQuerying(t, + genHistogramSeries(10, 5, minute(0), minute(119), minute(1), floatHistogram), + genHistogramSeries(10, 5, minute(120), minute(239), minute(1), floatHistogram), + genHistogramSeries(10, 5, minute(240), minute(359), minute(1), floatHistogram), + ) + }) + + t.Run("serial blocks with either histograms or floats in a block and not both", func(t *testing.T) { + testBlockQuerying(t, + genHistogramSeries(10, 5, minute(0), minute(119), minute(1), floatHistogram), + genSeriesFromSampleGenerator(10, 5, minute(120), minute(239), minute(1), func(ts int64) chunks.Sample { + return sample{t: ts, f: rand.Float64()} + }), + genHistogramSeries(10, 5, minute(240), minute(359), minute(1), floatHistogram), + ) + }) + + t.Run("serial blocks with mix of histograms and float64", func(t *testing.T) { + testBlockQuerying(t, + genHistogramAndFloatSeries(10, 5, minute(0), minute(60), minute(1), floatHistogram), + genHistogramSeries(10, 5, minute(61), minute(120), minute(1), floatHistogram), + genHistogramAndFloatSeries(10, 5, minute(121), minute(180), minute(1), floatHistogram), + genSeriesFromSampleGenerator(10, 5, minute(181), minute(240), minute(1), func(ts int64) chunks.Sample { + return sample{t: ts, f: rand.Float64()} + }), + ) + }) + + t.Run("overlapping blocks with only histograms", func(t *testing.T) { + testBlockQuerying(t, + genHistogramSeries(10, 5, minute(0), minute(120), minute(3), floatHistogram), + genHistogramSeries(10, 5, minute(1), minute(120), minute(3), floatHistogram), + genHistogramSeries(10, 5, minute(2), minute(120), minute(3), floatHistogram), + ) + }) + + t.Run("overlapping blocks with only histograms and only float in a series", func(t *testing.T) { + testBlockQuerying(t, + genHistogramSeries(10, 5, minute(0), minute(120), minute(3), floatHistogram), + genSeriesFromSampleGenerator(10, 5, minute(1), minute(120), minute(3), func(ts int64) chunks.Sample { + return sample{t: ts, f: rand.Float64()} + }), + genHistogramSeries(10, 5, minute(2), minute(120), minute(3), floatHistogram), + ) + }) + + t.Run("overlapping blocks with mix of histograms and float64", func(t *testing.T) { + testBlockQuerying(t, + genHistogramAndFloatSeries(10, 5, minute(0), minute(60), minute(3), floatHistogram), + genHistogramSeries(10, 5, minute(46), minute(100), minute(3), floatHistogram), + genHistogramAndFloatSeries(10, 5, minute(89), minute(140), minute(3), floatHistogram), + genSeriesFromSampleGenerator(10, 5, minute(126), minute(200), minute(3), func(ts int64) chunks.Sample { + return sample{t: ts, f: rand.Float64()} + }), + ) + }) + }) + } +} + +func TestOOONativeHistogramsSettings(t *testing.T) { + h := &histogram.Histogram{ + Count: 9, + ZeroCount: 4, + ZeroThreshold: 0.001, + Sum: 35.5, + Schema: 1, + PositiveSpans: []histogram.Span{ + {Offset: 0, Length: 2}, + {Offset: 2, Length: 2}, + }, + PositiveBuckets: []int64{1, 1, -1, 0}, + } + + l := labels.FromStrings("foo", "bar") + + t.Run("Test OOO native histograms if OOO is disabled and Native Histograms is enabled", func(t *testing.T) { + opts := DefaultOptions() + opts.OutOfOrderTimeWindow = 0 + db := newTestDB(t, withOpts(opts), withRngs(100)) + + app := db.Appender(context.Background()) + _, err := app.AppendHistogram(0, l, 100, h, nil) + require.NoError(t, err) + + _, err = app.AppendHistogram(0, l, 50, h, nil) + require.NoError(t, err) // The OOO sample is not detected until it is committed, so no error is returned + + require.NoError(t, app.Commit()) + + q, err := db.Querier(math.MinInt, math.MaxInt64) + require.NoError(t, err) + act := query(t, q, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + require.Equal(t, map[string][]chunks.Sample{ + l.String(): {sample{t: 100, h: h}}, + }, act) + }) + t.Run("Test OOO native histograms when both OOO and Native Histograms are enabled", func(t *testing.T) { + opts := DefaultOptions() + opts.OutOfOrderTimeWindow = 100 + db := newTestDB(t, withOpts(opts), withRngs(100)) + + // Add in-order samples + app := db.Appender(context.Background()) + _, err := app.AppendHistogram(0, l, 200, h, nil) + require.NoError(t, err) + + // Add OOO samples + _, err = app.AppendHistogram(0, l, 100, h, nil) + require.NoError(t, err) + _, err = app.AppendHistogram(0, l, 150, h, nil) + require.NoError(t, err) + + require.NoError(t, app.Commit()) + + q, err := db.Querier(math.MinInt, math.MaxInt64) + require.NoError(t, err) + act := query(t, q, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + requireEqualSeries(t, map[string][]chunks.Sample{ + l.String(): {sample{t: 100, h: h}, sample{t: 150, h: h}, sample{t: 200, h: h}}, + }, act, true) + }) +} + +// compareSeries essentially replaces `require.Equal(t, expected, actual)` in +// situations where the actual series might contain more counter reset hints +// "unknown" than the expected series. This can easily happen for long series +// that trigger new chunks. This function therefore tolerates counter reset +// hints "CounterReset" and "NotCounterReset" in an expected series where the +// actual series contains a counter reset hint "UnknownCounterReset". +// "GaugeType" hints are still strictly checked, and any "UnknownCounterReset" +// in an expected series has to be matched precisely by the actual series. +func compareSeries(t require.TestingT, expected, actual map[string][]chunks.Sample) { + if len(expected) != len(actual) { + // The reason for the difference is not the counter reset hints + // (alone), so let's use the pretty diffing by the require + // package. + require.Equal(t, expected, actual, "number of series differs") + } + for key, expSamples := range expected { + actSamples, ok := actual[key] + if !ok { + require.Equal(t, expected, actual, "expected series %q not found", key) + } + if len(expSamples) != len(actSamples) { + require.Equal(t, expSamples, actSamples, "number of samples for series %q differs", key) + } + + for i, eS := range expSamples { + aS := actSamples[i] + + // Must use the interface as Equal does not work when actual types differ + // not only does the type differ, but chunk.Sample.FH() interface may auto convert from chunk.Sample.H()! + require.Equal(t, eS.T(), aS.T(), "timestamp of sample %d in series %q differs", i, key) + + require.Equal(t, eS.Type(), aS.Type(), "type of sample %d in series %q differs", i, key) + + switch eS.Type() { + case chunkenc.ValFloat: + require.Equal(t, eS.F(), aS.F(), "sample %d in series %q differs", i, key) + case chunkenc.ValHistogram: + eH, aH := eS.H(), aS.H() + if aH.CounterResetHint == histogram.UnknownCounterReset { + eH = eH.Copy() + // It is always safe to set the counter reset hint to UnknownCounterReset + eH.CounterResetHint = histogram.UnknownCounterReset + eS = sample{t: eS.T(), h: eH} + } + require.Equal(t, eH, aH, "histogram sample %d in series %q differs", i, key) + + case chunkenc.ValFloatHistogram: + eFH, aFH := eS.FH(), aS.FH() + if aFH.CounterResetHint == histogram.UnknownCounterReset { + eFH = eFH.Copy() + // It is always safe to set the counter reset hint to UnknownCounterReset + eFH.CounterResetHint = histogram.UnknownCounterReset + eS = sample{t: eS.T(), fh: eFH} + } + require.Equal(t, eFH, aFH, "float histogram sample %d in series %q differs", i, key) + } + } + } +} + +// TestChunkQuerierReadWriteRace looks for any possible race between appending +// samples and reading chunks because the head chunk that is being appended to +// can be read in parallel and we should be able to make a copy of the chunk without +// worrying about the parallel write. +func TestChunkQuerierReadWriteRace(t *testing.T) { + t.Parallel() + db := newTestDB(t) + + lbls := labels.FromStrings("foo", "bar") + + writer := func() error { + <-time.After(5 * time.Millisecond) // Initial pause while readers start. + ts := 0 + for range 500 { + app := db.Appender(context.Background()) + for range 10 { + ts++ + _, err := app.Append(0, lbls, int64(ts), float64(ts*100)) + if err != nil { + return err + } + } + err := app.Commit() + if err != nil { + return err + } + <-time.After(time.Millisecond) + } + return nil + } + + reader := func() { + querier, err := db.ChunkQuerier(math.MinInt64, math.MaxInt64) + require.NoError(t, err) + defer func(q storage.ChunkQuerier) { + require.NoError(t, q.Close()) + }(querier) + ss := querier.Select(context.Background(), false, nil, labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")) + for ss.Next() { + cs := ss.At() + it := cs.Iterator(nil) + for it.Next() { + m := it.At() + b := m.Chunk.Bytes() + bb := make([]byte, len(b)) + copy(bb, b) // This copying of chunk bytes detects any race. + } + } + require.NoError(t, ss.Err()) + } + + ch := make(chan struct{}) + var writerErr error + go func() { + defer close(ch) + writerErr = writer() + }() + +Outer: + for { + reader() + select { + case <-ch: + break Outer + default: + } + } + + require.NoError(t, writerErr) +} + +type mockCompactorFn struct { + planFn func() ([]string, error) + compactFn func() ([]ulid.ULID, error) + writeFn func() ([]ulid.ULID, error) +} + +func (c *mockCompactorFn) Plan(string) ([]string, error) { + return c.planFn() +} + +func (c *mockCompactorFn) Compact(string, []string, []*Block) ([]ulid.ULID, error) { + return c.compactFn() +} + +func (c *mockCompactorFn) Write(string, BlockReader, int64, int64, *BlockMeta) ([]ulid.ULID, error) { + return c.writeFn() +} + +// Regression test for https://github.com/prometheus/prometheus/pull/13754 +func TestAbortBlockCompactions(t *testing.T) { + // Create a test DB + db := newTestDB(t) + // It should NOT be compactable at the beginning of the test + require.False(t, db.head.compactable(), "head should NOT be compactable") + + // Track the number of compactions run inside db.compactBlocks() + var compactions int + + // Use a mock compactor with custom Plan() implementation + db.compactor = &mockCompactorFn{ + planFn: func() ([]string, error) { + // On every Plan() run increment compactions. After 4 compactions + // update HEAD to make it compactable to force an exit from db.compactBlocks() loop. + compactions++ + if compactions > 3 { + chunkRange := db.head.chunkRange.Load() + db.head.minTime.Store(0) + db.head.maxTime.Store(chunkRange * 2) + require.True(t, db.head.compactable(), "head should be compactable") + } + // Our custom Plan() will always return something to compact. + return []string{"1", "2", "3"}, nil + }, + compactFn: func() ([]ulid.ULID, error) { + return []ulid.ULID{}, nil + }, + writeFn: func() ([]ulid.ULID, error) { + return []ulid.ULID{}, nil + }, + } + + err := db.Compact(context.Background()) + require.NoError(t, err) + require.True(t, db.head.compactable(), "head should be compactable") + require.Equal(t, 4, compactions, "expected 4 compactions to be completed") +} + +func TestNewCompactorFunc(t *testing.T) { + opts := DefaultOptions() + block1 := ulid.MustNew(1, nil) + block2 := ulid.MustNew(2, nil) + opts.NewCompactorFunc = func(context.Context, prometheus.Registerer, *slog.Logger, []int64, chunkenc.Pool, *Options) (Compactor, error) { + return &mockCompactorFn{ + planFn: func() ([]string, error) { + return []string{block1.String(), block2.String()}, nil + }, + compactFn: func() ([]ulid.ULID, error) { + return []ulid.ULID{block1}, nil + }, + writeFn: func() ([]ulid.ULID, error) { + return []ulid.ULID{block2}, nil + }, + }, nil + } + db := newTestDB(t, withOpts(opts)) + + plans, err := db.compactor.Plan("") + require.NoError(t, err) + require.Equal(t, []string{block1.String(), block2.String()}, plans) + ulids, err := db.compactor.Compact("", nil, nil) + require.NoError(t, err) + require.Len(t, ulids, 1) + require.Equal(t, block1, ulids[0]) + ulids, err = db.compactor.Write("", nil, 0, 1, nil) + require.NoError(t, err) + require.Len(t, ulids, 1) + require.Equal(t, block2, ulids[0]) +} + +func TestBlockQuerierAndBlockChunkQuerier(t *testing.T) { + opts := DefaultOptions() + opts.BlockQuerierFunc = func(b BlockReader, mint, maxt int64) (storage.Querier, error) { + // Only block with hints can be queried. + if len(b.Meta().Compaction.Hints) > 0 { + return NewBlockQuerier(b, mint, maxt) + } + return storage.NoopQuerier(), nil + } + opts.BlockChunkQuerierFunc = func(b BlockReader, mint, maxt int64) (storage.ChunkQuerier, error) { + // Only level 4 compaction block can be queried. + if b.Meta().Compaction.Level == 4 { + return NewBlockChunkQuerier(b, mint, maxt) + } + return storage.NoopChunkedQuerier(), nil + } + + db := newTestDB(t, withOpts(opts)) + + metas := []BlockMeta{ + {Compaction: BlockMetaCompaction{Hints: []string{"test-hint"}}}, + {Compaction: BlockMetaCompaction{Level: 4}}, + } + for i := range metas { + // Include blockID into series to identify which block got touched. + serieses := []storage.Series{storage.NewListSeries(labels.FromMap(map[string]string{"block": fmt.Sprintf("block-%d", i), labels.MetricName: "test_metric"}), []chunks.Sample{sample{t: 0, f: 1}})} + blockDir := createBlock(t, db.Dir(), serieses) + b, err := OpenBlock(db.logger, blockDir, db.chunkPool, nil) + require.NoError(t, err) + + // Overwrite meta.json with compaction section for testing purpose. + b.meta.Compaction = metas[i].Compaction + _, err = writeMetaFile(db.logger, blockDir, &b.meta) + require.NoError(t, err) + require.NoError(t, b.Close()) + } + require.NoError(t, db.reloadBlocks()) + require.Len(t, db.Blocks(), 2) + + querier, err := db.Querier(0, 500) + require.NoError(t, err) + defer querier.Close() + matcher := labels.MustNewMatcher(labels.MatchEqual, labels.MetricName, "test_metric") + seriesSet := querier.Select(context.Background(), false, nil, matcher) + count := 0 + var lbls labels.Labels + for seriesSet.Next() { + count++ + lbls = seriesSet.At().Labels() + } + require.NoError(t, seriesSet.Err()) + require.Equal(t, 1, count) + // Make sure only block-0 is queried. + require.Equal(t, "block-0", lbls.Get("block")) + + chunkQuerier, err := db.ChunkQuerier(0, 500) + require.NoError(t, err) + defer chunkQuerier.Close() + css := chunkQuerier.Select(context.Background(), false, nil, matcher) + count = 0 + // Reset lbls variable. + lbls = labels.EmptyLabels() + for css.Next() { + count++ + lbls = css.At().Labels() + } + require.NoError(t, css.Err()) + require.Equal(t, 1, count) + // Make sure only block-1 is queried. + require.Equal(t, "block-1", lbls.Get("block")) +} + +func TestGenerateCompactionDelay(t *testing.T) { + assertDelay := func(delay time.Duration, expectedMaxPercentDelay int) { + t.Helper() + require.GreaterOrEqual(t, delay, time.Duration(0)) + // Expect to generate a delay up to MaxPercentDelay of the head chunk range + require.LessOrEqual(t, delay, (time.Duration(60000*expectedMaxPercentDelay/100) * time.Millisecond)) + } + + opts := DefaultOptions() + cases := []struct { + compactionDelayPercent int + }{ + { + compactionDelayPercent: 1, + }, + { + compactionDelayPercent: 10, + }, + { + compactionDelayPercent: 60, + }, + { + compactionDelayPercent: 100, + }, + } + + opts.EnableDelayedCompaction = true + + for _, c := range cases { + opts.CompactionDelayMaxPercent = c.compactionDelayPercent + db := newTestDB(t, withOpts(opts), withRngs(60000)) + + // The offset is generated and changed while opening. + assertDelay(db.opts.CompactionDelay, c.compactionDelayPercent) + + for range 1000 { + assertDelay(db.generateCompactionDelay(), c.compactionDelayPercent) + } + } +} + +type blockedResponseRecorder struct { + r *httptest.ResponseRecorder + + // writeBlocked is used to block writing until the test wants it to resume. + writeBlocked chan struct{} + // writeStarted is closed by blockedResponseRecorder to signal that writing has started. + writeStarted chan struct{} +} + +func (br *blockedResponseRecorder) Write(buf []byte) (int, error) { + select { + case <-br.writeStarted: + default: + close(br.writeStarted) + } + + <-br.writeBlocked + return br.r.Write(buf) +} + +func (br *blockedResponseRecorder) Header() http.Header { return br.r.Header() } + +func (br *blockedResponseRecorder) WriteHeader(code int) { br.r.WriteHeader(code) } + +func (br *blockedResponseRecorder) Flush() { br.r.Flush() } + +// TestBlockClosingBlockedDuringRemoteRead ensures that a TSDB Block is not closed while it is being queried +// through remote read. This is a regression test for https://github.com/prometheus/prometheus/issues/14422. +// TODO: Ideally, this should reside in storage/remote/read_handler_test.go once the necessary TSDB utils are accessible there. +func TestBlockClosingBlockedDuringRemoteRead(t *testing.T) { + dir := t.TempDir() + + createBlock(t, dir, genSeries(2, 1, 0, 10)) + + // Not using newTestDB as db.Close is expected to return error. + db, err := Open(dir, nil, nil, nil, nil) + require.NoError(t, err) + defer db.Close() + + readAPI := remote.NewReadHandler( + nil, nil, db, + func() config.Config { + return config.Config{} + }, 0, 1, 0, + ) + + matcher, err := labels.NewMatcher(labels.MatchRegexp, "__name__", ".*") + require.NoError(t, err) + + query, err := remote.ToQuery(0, 10, []*labels.Matcher{matcher}, nil) + require.NoError(t, err) + + req := &prompb.ReadRequest{ + Queries: []*prompb.Query{query}, + AcceptedResponseTypes: []prompb.ReadRequest_ResponseType{prompb.ReadRequest_STREAMED_XOR_CHUNKS}, + } + data, err := proto.Marshal(req) + require.NoError(t, err) + + request, err := http.NewRequest(http.MethodPost, "", bytes.NewBuffer(snappy.Encode(nil, data))) + require.NoError(t, err) + + blockedRecorder := &blockedResponseRecorder{ + r: httptest.NewRecorder(), + writeBlocked: make(chan struct{}), + writeStarted: make(chan struct{}), + } + + readDone := make(chan struct{}) + go func() { + readAPI.ServeHTTP(blockedRecorder, request) + require.Equal(t, http.StatusOK, blockedRecorder.r.Code) + close(readDone) + }() + + // Wait for the read API to start streaming data. + <-blockedRecorder.writeStarted + + // Try to close the queried block. + blockClosed := make(chan struct{}) + go func() { + for _, block := range db.Blocks() { + block.Close() + } + close(blockClosed) + }() + + // Closing the queried block should block. + // Wait a little bit to make sure of that. + select { + case <-time.After(100 * time.Millisecond): + case <-readDone: + require.Fail(t, "read API should still be streaming data.") + case <-blockClosed: + require.Fail(t, "Block shouldn't get closed while being queried.") + } + + // Resume the read API data streaming. + close(blockedRecorder.writeBlocked) + <-readDone + + // The block should be no longer needed and closing it should end. + select { + case <-time.After(10 * time.Millisecond): + require.Fail(t, "Closing the block timed out.") + case <-blockClosed: + } +} From 0b70a0757263930f035e48290826ac3d63b3d5ec Mon Sep 17 00:00:00 2001 From: bwplotka Date: Fri, 28 Nov 2025 12:43:12 +0000 Subject: [PATCH 5/6] refactor(appenderV2): add TSDB AppenderV2 implementation Signed-off-by: bwplotka tmp Signed-off-by: bwplotka --- model/metadata/metadata.go | 26 +- model/metadata/metadata_test.go | 116 + storage/interface.go | 1 + tsdb/db.go | 37 +- tsdb/head.go | 14 + tsdb/head_append.go | 55 +- tsdb/head_append_v2.go | 2391 +++------------------ tsdb/head_append_v2_test.go | 3532 ++++--------------------------- tsdb/head_bench_test.go | 303 ++- tsdb/head_bench_v2_test.go | 173 -- tsdb/head_test.go | 48 +- tsdb/testutil.go | 16 +- 12 files changed, 1168 insertions(+), 5544 deletions(-) create mode 100644 model/metadata/metadata_test.go delete mode 100644 tsdb/head_bench_v2_test.go diff --git a/model/metadata/metadata.go b/model/metadata/metadata.go index 1b7e63e0f3..d2a91bb560 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,21 @@ 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 + } + + // Unknown means the same as empty string. + if m.Type == "" || m.Type == model.MetricTypeUnknown { + return other.Type == "" || other.Type == model.MetricTypeUnknown + } + return m.Type == other.Type +} diff --git a/model/metadata/metadata_test.go b/model/metadata/metadata_test.go new file mode 100644 index 0000000000..169cd60c2e --- /dev/null +++ b/model/metadata/metadata_test.go @@ -0,0 +1,116 @@ +// 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 +// +// 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 metadata + +import ( + "testing" + + "github.com/prometheus/common/model" + "github.com/stretchr/testify/require" +) + +func TestMetadata_IsEmpty(t *testing.T) { + for _, tt := range []struct { + name string + m Metadata + expected bool + }{ + { + name: "empty struct", expected: true, + }, + { + name: "unknown type with empty fields", expected: true, + m: Metadata{Type: model.MetricTypeUnknown}, + }, + { + name: "type", expected: false, + m: Metadata{Type: model.MetricTypeCounter}, + }, + { + name: "unit", expected: false, + m: Metadata{Unit: "seconds"}, + }, + { + name: "help", expected: false, + m: Metadata{Help: "help text"}, + }, + { + name: "unknown type with help", expected: false, + m: Metadata{Type: model.MetricTypeUnknown, Help: "help text"}, + }, + } { + t.Run(tt.name, func(t *testing.T) { + require.Equal(t, tt.expected, tt.m.IsEmpty()) + }) + } +} + +func TestMetadata_Equals(t *testing.T) { + for _, tt := range []struct { + name string + m Metadata + other Metadata + expected bool + }{ + { + name: "same empty", expected: true, + }, + { + name: "same", expected: true, + m: Metadata{Type: model.MetricTypeCounter, Unit: "s", Help: "doc"}, + other: Metadata{Type: model.MetricTypeCounter, Unit: "s", Help: "doc"}, + }, + { + name: "same unknown type", expected: true, + m: Metadata{Type: model.MetricTypeUnknown, Unit: "s", Help: "doc"}, + other: Metadata{Type: model.MetricTypeUnknown, Unit: "s", Help: "doc"}, + }, + { + name: "same mixed unknown type", expected: true, + m: Metadata{Type: "", Unit: "s", Help: "doc"}, + other: Metadata{Type: model.MetricTypeUnknown, Unit: "s", Help: "doc"}, + }, + { + name: "different unit", expected: false, + m: Metadata{Type: model.MetricTypeCounter, Unit: "s", Help: "doc"}, + other: Metadata{Type: model.MetricTypeCounter, Unit: "bytes", Help: "doc"}, + }, + { + name: "different help", expected: false, + m: Metadata{Type: model.MetricTypeCounter, Unit: "s", Help: "doc"}, + other: Metadata{Type: model.MetricTypeCounter, Unit: "s", Help: "other doc"}, + }, + { + name: "different type", expected: false, + m: Metadata{Type: model.MetricTypeCounter, Unit: "s", Help: "doc"}, + other: Metadata{Type: model.MetricTypeGauge, Unit: "s", Help: "doc"}, + }, + { + name: "different type with unknown", expected: false, + m: Metadata{Type: model.MetricTypeUnknown, Unit: "s", Help: "doc"}, + other: Metadata{Type: model.MetricTypeCounter, Unit: "s", Help: "doc"}, + }, + { + name: "different type with empty", expected: false, + m: Metadata{Type: "", Unit: "s", Help: "doc"}, + other: Metadata{Type: model.MetricTypeCounter, Unit: "s", Help: "doc"}, + }, + } { + t.Run(tt.name, func(t *testing.T) { + if got := tt.m.Equals(tt.other); got != tt.expected { + t.Errorf("Metadata.Equals() = %v, expected %v", got, tt.expected) + } + }) + } +} diff --git a/storage/interface.go b/storage/interface.go index fe9b3fa6e8..f7d7953de4 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 dac5689b09..c4f29c225f 100644 --- a/tsdb/db.go +++ b/tsdb/db.go @@ -1136,11 +1136,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. @@ -1254,6 +1259,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..c5ed9898e9 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,323 +68,129 @@ 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 +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) { + var ( + // Avoid shadowing err variables for reliability. + valErr, appErr, partialErr error + sampleMetricType = sampleMetricTypeFloat + isStale bool + ) + // Fail fast on incorrect histograms. + + switch { + case fh != nil: + sampleMetricType = sampleMetricTypeHistogram + valErr = fh.Validate() + case h != nil: + sampleMetricType = sampleMetricTypeHistogram + valErr = h.Validate() + } + if valErr != nil { + return 0, valErr } - 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) { // 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() + a.head.metrics.outOfBoundSamples.WithLabelValues(sampleMetricType).Inc() return 0, storage.ErrOutOfBounds } 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 } } - 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". + // TODO(bwplotka): Handle ST natively (as per PROM-60). + if a.head.opts.EnableSTAsZeroSample && st != 0 { + a.bestEffortAppendSTZeroSample(s, st, t, h, fh) } - 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 + switch { + case fh != nil: + isStale = value.IsStaleNaN(fh.Sum) + appErr = a.appendFloatHistogram(s, t, fh, opts.RejectOutOfOrder) + case h != nil: + isStale = value.IsStaleNaN(h.Sum) + appErr = a.appendHistogram(s, t, h, opts.RejectOutOfOrder) + default: + isStale = value.IsStaleNaN(v) + if isStale { + // 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{ + RejectOutOfOrder: opts.RejectOutOfOrder, + }) + case stFloatHistogram, stCustomBucketFloatHistogram: + return a.Append(ref, ls, st, t, 0, nil, &histogram.FloatHistogram{Sum: v}, storage.AOptions{ + 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". } - s.pendingCommit = true + appErr = a.appendFloat(s, t, v, opts.RejectOutOfOrder) } - if delta > 0 { - a.head.metrics.oooHistogram.Observe(float64(delta) / 1000) - } - if err != nil { + // Handle append error, if any. + if appErr != 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() + case errors.Is(appErr, storage.ErrOutOfOrderSample): + a.head.metrics.outOfOrderSamples.WithLabelValues(sampleMetricType).Inc() + case errors.Is(appErr, storage.ErrTooOldSample): + a.head.metrics.tooOldSamples.WithLabelValues(sampleMetricType).Inc() } - return 0, err + return 0, appErr } if t < a.mint { @@ -473,492 +200,161 @@ func (a *headAppender) Append(ref storage.SeriesRef, lset labels.Labels, t int64 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 + if isStale { + // For stale values we never attempt to process metadata/exemplars, claim the success. + return ref, nil } - 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 + // 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) + } + + // 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 == nil || !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 +} - // 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. +func (a *headAppenderV2) appendFloat(s *memSeries, t int64, v float64, fastRejectOOO bool) error { s.Lock() - isOOO, _, err := s.appendable(st, 0, a.headMaxt, a.minValidTime, a.oooTimeWindow) + // 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 err != nil { - return 0, err + if delta > 0 { + a.head.metrics.oooHistogram.Observe(float64(delta) / 1000) } - if isOOO { - return storage.SeriesRef(s.ref), storage.ErrOutOfOrderST + if err != nil { + return err } - 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.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 + return 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() - return 0, storage.ErrOutOfBounds - } - - if h != nil { - if err := h.Validate(); err != nil { - return 0, err - } - } - - if fh != nil { - if err := fh.Validate(); err != nil { - return 0, err - } - } - - 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 - } - } - - 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 - } +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() - 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(sampleMetricTypeHistogram).Inc() - case errors.Is(err, storage.ErrTooOldSample): - a.head.metrics.tooOldSamples.WithLabelValues(sampleMetricTypeHistogram).Inc() - } - 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 { - switch { - case errors.Is(err, storage.ErrOutOfOrderSample): - a.head.metrics.outOfOrderSamples.WithLabelValues(sampleMetricTypeHistogram).Inc() - case errors.Is(err, storage.ErrTooOldSample): - a.head.metrics.tooOldSamples.WithLabelValues(sampleMetricTypeHistogram).Inc() - } - return 0, 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 storage.ErrOutOfOrderSample } - - if t < a.mint { - a.mint = t - } - if t > a.maxt { - 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 - } - - 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 - } - } - - switch { - case h != nil: - zeroHistogram := &histogram.Histogram{ - // The STZeroSample represents a counter reset by definition. - CounterResetHint: histogram.CounterReset, - // Replicate other fields to avoid needless chunk creation. - Schema: h.Schema, - 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 - } - + 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() - sTyp := stHistogram - if h.UsesCustomBuckets() { - sTyp = stCustomBucketHistogram + 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(sTyp, s.ref) - b.histograms = append(b.histograms, record.RefHistogramSample{ - Ref: s.ref, - T: st, - H: zeroHistogram, - }) - b.histogramSeries = append(b.histogramSeries, s) + 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 fh != nil: zeroFloatHistogram := &histogram.FloatHistogram{ // The STZeroSample represents a counter reset by definition. @@ -968,1318 +364,33 @@ 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 - } - - return 0, err + err = a.appendFloatHistogram(s, st, zeroFloatHistogram, true) + case h != nil: + zeroHistogram := &histogram.Histogram{ + // The STZeroSample represents a counter reset by definition. + CounterResetHint: histogram.CounterReset, + // Replicate other fields to avoid needless chunk creation. + Schema: h.Schema, + ZeroThreshold: h.ZeroThreshold, + CustomValues: h.CustomValues, } + err = a.appendHistogram(s, st, zeroHistogram, true) + default: + err = a.appendFloat(s, st, 0, true) + } - // 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 + if err != nil { + if errors.Is(err, storage.ErrOutOfOrderSample) { + // OOO errors are common and expected (cumulative). Explicitly ignored. + return } - - 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..33bc3aec38 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,19 @@ 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 -} +// TODO(bwplotka): Ensure non-ported tests are not deleted from db_test.go when removing AppenderV1 flow (#17632), +// for example: +// * TestChunkNotFoundHeadGCRace +// * TestHeadSeriesChunkRace +// * TestHeadLabelValuesWithMatchers +// * TestHeadLabelNamesWithMatchers +// * TestHeadShardedPostings -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 +132,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 +257,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 +317,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 +364,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 +387,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 +402,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 +411,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 +430,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 +487,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 +500,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 +562,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 +593,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 +612,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 +656,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 +713,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 +815,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 +823,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 +845,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 +853,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 +876,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 +884,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 +900,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 +908,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 +930,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 +960,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 +977,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 +1022,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 +1059,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 +1078,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 +1092,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 +1129,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 +1142,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 +1153,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 +1164,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 +1195,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 +1222,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,28 +1232,28 @@ 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) { +func TestHeadAppenderV2_Append_OutOfOrderSamplesMetric(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) + testHeadAppenderV2OutOfOrderSamplesMetric(t, scenario, options, storage.ErrOutOfOrderSample) }) } } -func TestOutOfOrderSamplesMetricNativeHistogramOOODisabled(t *testing.T) { +func TestHeadAppenderV2_Append_OutOfOrderSamplesMetricNativeHistogramOOODisabled(t *testing.T) { for name, scenario := range sampleTypeScenarios { if scenario.sampleType != "histogram" { continue @@ -3211,12 +1261,12 @@ func TestOutOfOrderSamplesMetricNativeHistogramOOODisabled(t *testing.T) { t.Run(name, func(t *testing.T) { options := DefaultOptions() options.OutOfOrderTimeWindow = 0 - testOutOfOrderSamplesMetric(t, scenario, options, storage.ErrOutOfOrderSample) + testHeadAppenderV2OutOfOrderSamplesMetric(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 +1275,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 +1291,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 +1306,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 +1314,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 +1327,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 +1335,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,42 +1350,7 @@ 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) { +func TestHeadLabelNamesValuesWithMinMaxRange_AppenderV2(t *testing.T) { head, _ := newTestHead(t, 1000, compression.None, false) defer func() { require.NoError(t, head.Close()) @@ -3355,9 +1372,9 @@ func TestHeadLabelNamesValuesWithMinMaxRange(t *testing.T) { ctx = context.Background() ) - app := head.Appender(ctx) + app := head.AppenderV2(ctx) for i, name := range expectedLabelNames { - _, err := app.Append(0, labels.FromStrings(name, expectedLabelValues[i]), seriesTimestamps[i], 0) + _, err := app.Append(0, labels.FromStrings(name, expectedLabelValues[i]), 0, seriesTimestamps[i], 0, nil, nil, storage.AOptions{}) require.NoError(t, err) } require.NoError(t, app.Commit()) @@ -3394,260 +1411,51 @@ func TestHeadLabelNamesValuesWithMinMaxRange(t *testing.T) { } } -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)) - } - } - 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) - }) - } -} - -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 +1480,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 +1517,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 +1548,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 +1593,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 +1601,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 +1651,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 +1678,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 +1695,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 +1754,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 +1765,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 +1797,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 +1861,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 +1874,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 +1886,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 +1908,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 +1938,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 +1949,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 +1959,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 +1985,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 +1995,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 +2041,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 +2062,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 +2072,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 +2149,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 +2160,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 +2227,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 +2238,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 +2335,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 +2343,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 +2354,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 +2445,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 +2459,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 +2486,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 +2581,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 +2612,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 +2647,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 +2660,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 +2768,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 +2781,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 +2928,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 +3021,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 +3057,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 +3077,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 +3094,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 +3121,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 +3186,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 +3213,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 +3276,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 +3285,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 +3301,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 +3335,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 +3356,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 +3391,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 +3418,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 +3470,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 +3498,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 +3546,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 +3559,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 +3588,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 +3614,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 +3625,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 +3690,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 +3701,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 +3766,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 +3794,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 +3821,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 +3926,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 +3934,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 +3985,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 +3994,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,29 +4028,29 @@ 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{}), + expectedType: reflect.TypeFor[[]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{}), + expectedType: reflect.TypeFor[[]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{}), + expectedType: reflect.TypeFor[[]record.RefFloatHistogramSample](), }, } @@ -6548,12 +4061,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 +4079,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 +4087,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 +4303,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 +4429,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 +4468,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 +4477,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 +4486,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 +4497,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 +4636,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 +4675,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..a63b0ced50 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,228 @@ 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. + // + // TODO(bwplotka): This still takes ~6500s (~2h) for -benchtime 5s -count 6 to complete. + // We might want to reduce the time bit more. 5s is really important as the slowest + // case (appender=v1/case=floatsHistogramsExemplars/series=100/samples_per_append=100-2) + // in 5s yields only 255 iters 23184892 ns/op. Perhaps -benchtime=300x would be better? + 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 +307,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..84605d31fa 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 @@ -6579,6 +6537,8 @@ func TestWALSampleAndExemplarOrder(t *testing.T) { // would trigger the // `signal SIGSEGV: segmentation violation code=0x1 addr=0x20 pc=0xbb03d1` // panic, that we have seen in the wild once. +// +// TODO(bwplotka): This no longer can happen in AppenderV2, remove once AppenderV1 is removed, see #17632. func TestHeadCompactionWhileAppendAndCommitExemplar(t *testing.T) { h, _ := newTestHead(t, DefaultBlockDuration, compression.None, false) app := h.Appender(context.Background()) 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 From e7e45090e4561775350e8ee5e23a398916165046 Mon Sep 17 00:00:00 2001 From: bwplotka Date: Tue, 2 Dec 2025 14:35:53 +0000 Subject: [PATCH 6/6] refactor(appenderV2): port TSDB non-head tests Signed-off-by: bwplotka --- tsdb/blockwriter.go | 6 + tsdb/blockwriter_test.go | 35 + tsdb/db.go | 16 + tsdb/db_append_v2_test.go | 2683 +++++++------------------------------ 4 files changed, 566 insertions(+), 2174 deletions(-) diff --git a/tsdb/blockwriter.go b/tsdb/blockwriter.go index 14137f12cc..e038812224 100644 --- a/tsdb/blockwriter.go +++ b/tsdb/blockwriter.go @@ -86,6 +86,12 @@ func (w *BlockWriter) Appender(ctx context.Context) storage.Appender { return w.head.Appender(ctx) } +// AppenderV2 returns a new appender on the database. +// AppenderV2 can't be called concurrently. However, the returned AppenderV2 can safely be used concurrently. +func (w *BlockWriter) AppenderV2(ctx context.Context) storage.AppenderV2 { + return w.head.AppenderV2(ctx) +} + // Flush implements the Writer interface. This is where actual block writing // happens. After flush completes, no writes can be done. func (w *BlockWriter) Flush(ctx context.Context) (ulid.ULID, error) { diff --git a/tsdb/blockwriter_test.go b/tsdb/blockwriter_test.go index e7c3146247..becae6aa04 100644 --- a/tsdb/blockwriter_test.go +++ b/tsdb/blockwriter_test.go @@ -23,6 +23,7 @@ import ( "github.com/stretchr/testify/require" "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunks" ) @@ -59,3 +60,37 @@ func TestBlockWriter(t *testing.T) { require.NoError(t, w.Close()) } + +func TestBlockWriter_AppenderV2(t *testing.T) { + ctx := context.Background() + outputDir := t.TempDir() + w, err := NewBlockWriter(promslog.NewNopLogger(), outputDir, DefaultBlockDuration) + require.NoError(t, err) + + // Add some series. + app := w.AppenderV2(ctx) + ts1, v1 := int64(44), float64(7) + _, err = app.Append(0, labels.FromStrings("a", "b"), 0, ts1, v1, nil, nil, storage.AOptions{}) + require.NoError(t, err) + ts2, v2 := int64(55), float64(12) + _, err = app.Append(0, labels.FromStrings("c", "d"), 0, ts2, v2, nil, nil, storage.AOptions{}) + require.NoError(t, err) + require.NoError(t, app.Commit()) + id, err := w.Flush(ctx) + require.NoError(t, err) + + // Confirm the block has the correct data. + blockpath := filepath.Join(outputDir, id.String()) + b, err := OpenBlock(nil, blockpath, nil, nil) + require.NoError(t, err) + defer func() { require.NoError(t, b.Close()) }() + q, err := NewBlockQuerier(b, math.MinInt64, math.MaxInt64) + require.NoError(t, err) + series := query(t, q, labels.MustNewMatcher(labels.MatchRegexp, "", ".*")) + sample1 := []chunks.Sample{sample{t: ts1, f: v1}} + sample2 := []chunks.Sample{sample{t: ts2, f: v2}} + expectedSeries := map[string][]chunks.Sample{"{a=\"b\"}": sample1, "{c=\"d\"}": sample2} + require.Equal(t, expectedSeries, series) + + require.NoError(t, w.Close()) +} diff --git a/tsdb/db.go b/tsdb/db.go index c4f29c225f..73300d74f1 100644 --- a/tsdb/db.go +++ b/tsdb/db.go @@ -220,6 +220,20 @@ type Options struct { // UseUncachedIO allows bypassing the page cache when appropriate. UseUncachedIO bool + // EnableSTAsZeroSample represents 'created-timestamp-zero-ingestion' feature flag. + // If true, ST, if non-zero 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 + // BlockCompactionExcludeFunc is a function which returns true for blocks that should NOT be compacted. // It's passed down to the TSDB compactor. BlockCompactionExcludeFunc BlockExcludeFilterFunc @@ -973,6 +987,8 @@ func open(dir string, l *slog.Logger, r prometheus.Registerer, opts *Options, rn headOpts.OutOfOrderTimeWindow.Store(opts.OutOfOrderTimeWindow) headOpts.OutOfOrderCapMax.Store(opts.OutOfOrderCapMax) headOpts.EnableSharding = opts.EnableSharding + headOpts.EnableSTAsZeroSample = opts.EnableSTAsZeroSample + headOpts.EnableMetadataWALRecords = opts.EnableMetadataWALRecords if opts.WALReplayConcurrency > 0 { headOpts.WALReplayConcurrency = opts.WALReplayConcurrency } diff --git a/tsdb/db_append_v2_test.go b/tsdb/db_append_v2_test.go index 4e084ef0d8..344b1d6943 100644 --- a/tsdb/db_append_v2_test.go +++ b/tsdb/db_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 @@ -15,46 +15,32 @@ package tsdb import ( "bufio" - "bytes" "context" - "encoding/binary" - "errors" - "flag" "fmt" - "hash/crc32" "log/slog" "math" "math/rand" - "net/http" - "net/http/httptest" "os" "path" "path/filepath" "runtime" "sort" "strconv" - "sync" "testing" "time" - "github.com/gogo/protobuf/proto" - "github.com/golang/snappy" "github.com/oklog/ulid/v2" "github.com/prometheus/client_golang/prometheus" prom_testutil "github.com/prometheus/client_golang/prometheus/testutil" - "github.com/prometheus/common/model" "github.com/prometheus/common/promslog" "github.com/stretchr/testify/require" "go.uber.org/atomic" - "go.uber.org/goleak" "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/model/histogram" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/model/metadata" - "github.com/prometheus/prometheus/prompb" "github.com/prometheus/prometheus/storage" - "github.com/prometheus/prometheus/storage/remote" "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/fileutil" @@ -68,192 +54,20 @@ import ( "github.com/prometheus/prometheus/util/testutil" ) -func TestMain(m *testing.M) { - var isolationEnabled bool - flag.BoolVar(&isolationEnabled, "test.tsdb-isolation", true, "enable isolation") - flag.Parse() - defaultIsolationDisabled = !isolationEnabled +// TODO(bwplotka): Ensure non-ported tests are not deleted from db_test.go when removing AppenderV1 flow (#17632): +// * TestQuerier_ShouldNotPanicIfHeadChunkIsTruncatedWhileReadingQueriedChunks +// * TestChunkQuerier_ShouldNotPanicIfHeadChunkIsTruncatedWhileReadingQueriedChunks +// * TestEmptyLabelsetCausesError +// * TestQueryHistogramFromBlocksWithCompaction - goleak.VerifyTestMain(m, - goleak.IgnoreTopFunction("github.com/prometheus/prometheus/tsdb.(*SegmentWAL).cut.func1"), - goleak.IgnoreTopFunction("github.com/prometheus/prometheus/tsdb.(*SegmentWAL).cut.func2"), - goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start")) -} - -type testDBOptions struct { - dir string - opts *Options - rngs []int64 -} -type testDBOpt func(o *testDBOptions) - -func withDir(dir string) testDBOpt { - return func(o *testDBOptions) { - o.dir = dir - } -} - -func withOpts(opts *Options) testDBOpt { - return func(o *testDBOptions) { - o.opts = opts - } -} - -func withRngs(rngs ...int64) testDBOpt { - return func(o *testDBOptions) { - o.rngs = rngs - } -} - -func newTestDB(t testing.TB, opts ...testDBOpt) (db *DB) { - var o testDBOptions - for _, opt := range opts { - opt(&o) - } - if o.opts == nil { - o.opts = DefaultOptions() - } - if o.dir == "" { - o.dir = t.TempDir() - } - - var err error - if len(o.rngs) == 0 { - db, err = Open(o.dir, nil, nil, o.opts, nil) - } else { - o.opts, o.rngs = validateOpts(o.opts, o.rngs) - db, err = open(o.dir, nil, nil, o.opts, o.rngs, nil) - } - require.NoError(t, err) - t.Cleanup(func() { - // Always close. DB is safe for close-after-close. - require.NoError(t, db.Close()) - }) - return db -} - -func TestDBClose_AfterClose(t *testing.T) { - db := newTestDB(t) - require.NoError(t, db.Close()) - require.NoError(t, db.Close()) - - // Double check if we are closing correct DB after reuse. - db = newTestDB(t) - require.NoError(t, db.Close()) - require.NoError(t, db.Close()) -} - -// query runs a matcher query against the querier and fully expands its data. -func query(t testing.TB, q storage.Querier, matchers ...*labels.Matcher) map[string][]chunks.Sample { - ss := q.Select(context.Background(), false, nil, matchers...) - defer func() { - require.NoError(t, q.Close()) - }() - - var it chunkenc.Iterator - result := map[string][]chunks.Sample{} - for ss.Next() { - series := ss.At() - - it = series.Iterator(it) - samples, err := storage.ExpandSamples(it, newSample) - require.NoError(t, err) - require.NoError(t, it.Err()) - - if len(samples) == 0 { - continue - } - - name := series.Labels().String() - result[name] = samples - } - require.NoError(t, ss.Err()) - require.Empty(t, ss.Warnings()) - - return result -} - -// queryAndExpandChunks runs a matcher query against the querier and fully expands its data into samples. -func queryAndExpandChunks(t testing.TB, q storage.ChunkQuerier, matchers ...*labels.Matcher) map[string][][]chunks.Sample { - s := queryChunks(t, q, matchers...) - - res := make(map[string][][]chunks.Sample) - for k, v := range s { - var samples [][]chunks.Sample - for _, chk := range v { - sam, err := storage.ExpandSamples(chk.Chunk.Iterator(nil), nil) - require.NoError(t, err) - samples = append(samples, sam) - } - res[k] = samples - } - - return res -} - -// queryChunks runs a matcher query against the querier and expands its data. -func queryChunks(t testing.TB, q storage.ChunkQuerier, matchers ...*labels.Matcher) map[string][]chunks.Meta { - ss := q.Select(context.Background(), false, nil, matchers...) - defer func() { - require.NoError(t, q.Close()) - }() - - var it chunks.Iterator - result := map[string][]chunks.Meta{} - for ss.Next() { - series := ss.At() - - chks := []chunks.Meta{} - it = series.Iterator(it) - for it.Next() { - chks = append(chks, it.At()) - } - require.NoError(t, it.Err()) - - if len(chks) == 0 { - continue - } - - name := series.Labels().String() - result[name] = chks - } - require.NoError(t, ss.Err()) - require.Empty(t, ss.Warnings()) - return result -} - -// Ensure that blocks are held in memory in their time order -// and not in ULID order as they are read from the directory. -func TestDB_reloadOrder(t *testing.T) { - db := newTestDB(t) - - metas := []BlockMeta{ - {MinTime: 90, MaxTime: 100}, - {MinTime: 70, MaxTime: 80}, - {MinTime: 100, MaxTime: 110}, - } - for _, m := range metas { - createBlock(t, db.Dir(), genSeries(1, 1, m.MinTime, m.MaxTime)) - } - - require.NoError(t, db.reloadBlocks()) - blocks := db.Blocks() - require.Len(t, blocks, 3) - require.Equal(t, metas[1].MinTime, blocks[0].Meta().MinTime) - require.Equal(t, metas[1].MaxTime, blocks[0].Meta().MaxTime) - require.Equal(t, metas[0].MinTime, blocks[1].Meta().MinTime) - require.Equal(t, metas[0].MaxTime, blocks[1].Meta().MaxTime) - require.Equal(t, metas[2].MinTime, blocks[2].Meta().MinTime) - require.Equal(t, metas[2].MaxTime, blocks[2].Meta().MaxTime) -} - -func TestDataAvailableOnlyAfterCommit(t *testing.T) { +// TODO(krajorama): Add histograms test cases. +func TestDataAvailableOnlyAfterCommit_AppendV2(t *testing.T) { db := newTestDB(t) ctx := context.Background() - app := db.Appender(ctx) + app := db.AppenderV2(ctx) - _, err := app.Append(0, labels.FromStrings("foo", "bar"), 0, 0) + _, err := app.Append(0, labels.FromStrings("foo", "bar"), 0, 0, 0, nil, nil, storage.AOptions{}) require.NoError(t, err) querier, err := db.Querier(0, 1) @@ -275,7 +89,7 @@ func TestDataAvailableOnlyAfterCommit(t *testing.T) { // TestNoPanicAfterWALCorruption ensures that querying the db after a WAL corruption doesn't cause a panic. // https://github.com/prometheus/prometheus/issues/7548 -func TestNoPanicAfterWALCorruption(t *testing.T) { +func TestNoPanicAfterWALCorruption_AppendV2(t *testing.T) { db := newTestDB(t, withOpts(&Options{WALSegmentSize: 32 * 1024})) // Append until the first mmapped head chunk. @@ -286,8 +100,8 @@ func TestNoPanicAfterWALCorruption(t *testing.T) { { // Appending 121 samples because on the 121st a new chunk will be created. for range 121 { - app := db.Appender(ctx) - _, err := app.Append(0, labels.FromStrings("foo", "bar"), maxt, 0) + app := db.AppenderV2(ctx) + _, err := app.Append(0, labels.FromStrings("foo", "bar"), 0, maxt, 0, nil, nil, storage.AOptions{}) expSamples = append(expSamples, sample{t: maxt, f: 0}) require.NoError(t, err) require.NoError(t, app.Commit()) @@ -326,22 +140,24 @@ func TestNoPanicAfterWALCorruption(t *testing.T) { } } -func TestDataNotAvailableAfterRollback(t *testing.T) { +func TestDataNotAvailableAfterRollback_AppendV2(t *testing.T) { db := newTestDB(t) - app := db.Appender(context.Background()) - _, err := app.Append(0, labels.FromStrings("type", "float"), 0, 0) + app := db.AppenderV2(context.Background()) + _, err := app.Append(0, labels.FromStrings("type", "float"), 0, 0, 0, nil, nil, storage.AOptions{}) require.NoError(t, err) - _, err = app.AppendHistogram( - 0, labels.FromStrings("type", "histogram"), 0, + _, err = app.Append( + 0, labels.FromStrings("type", "histogram"), 0, 0, 0, &histogram.Histogram{Count: 42, Sum: math.NaN()}, nil, + storage.AOptions{}, ) require.NoError(t, err) - _, err = app.AppendHistogram( - 0, labels.FromStrings("type", "floathistogram"), 0, + _, err = app.Append( + 0, labels.FromStrings("type", "floathistogram"), 0, 0, 0, nil, &histogram.FloatHistogram{Count: 42, Sum: math.NaN()}, + storage.AOptions{}, ) require.NoError(t, err) @@ -413,41 +229,41 @@ func TestDataNotAvailableAfterRollback(t *testing.T) { require.Equal(t, 0, walFloatHistogramCount, "float histograms should not have been written to WAL") } -func TestDBAppenderAddRef(t *testing.T) { +func TestDBAppenderV2_AddRef(t *testing.T) { db := newTestDB(t) ctx := context.Background() - app1 := db.Appender(ctx) + app1 := db.AppenderV2(ctx) - ref1, err := app1.Append(0, labels.FromStrings("a", "b"), 123, 0) + ref1, err := app1.Append(0, labels.FromStrings("a", "b"), 0, 123, 0, nil, nil, storage.AOptions{}) require.NoError(t, err) // Reference should already work before commit. - ref2, err := app1.Append(ref1, labels.EmptyLabels(), 124, 1) + ref2, err := app1.Append(ref1, labels.EmptyLabels(), 0, 124, 1, nil, nil, storage.AOptions{}) require.NoError(t, err) require.Equal(t, ref1, ref2) err = app1.Commit() require.NoError(t, err) - app2 := db.Appender(ctx) + app2 := db.AppenderV2(ctx) // first ref should already work in next transaction. - ref3, err := app2.Append(ref1, labels.EmptyLabels(), 125, 0) + ref3, err := app2.Append(ref1, labels.EmptyLabels(), 0, 125, 0, nil, nil, storage.AOptions{}) require.NoError(t, err) require.Equal(t, ref1, ref3) - ref4, err := app2.Append(ref1, labels.FromStrings("a", "b"), 133, 1) + ref4, err := app2.Append(ref1, labels.FromStrings("a", "b"), 0, 133, 1, nil, nil, storage.AOptions{}) require.NoError(t, err) require.Equal(t, ref1, ref4) // Reference must be valid to add another sample. - ref5, err := app2.Append(ref2, labels.EmptyLabels(), 143, 2) + ref5, err := app2.Append(ref2, labels.EmptyLabels(), 0, 143, 2, nil, nil, storage.AOptions{}) require.NoError(t, err) require.Equal(t, ref1, ref5) // Missing labels & invalid refs should fail. - _, err = app2.Append(9999999, labels.EmptyLabels(), 1, 1) + _, err = app2.Append(9999999, labels.EmptyLabels(), 0, 1, 1, nil, nil, storage.AOptions{}) require.ErrorIs(t, err, ErrInvalidSample) require.NoError(t, app2.Commit()) @@ -468,17 +284,17 @@ func TestDBAppenderAddRef(t *testing.T) { }, res) } -func TestAppendEmptyLabelsIgnored(t *testing.T) { +func TestDBAppenderV2_EmptyLabelsIgnored(t *testing.T) { db := newTestDB(t) ctx := context.Background() - app1 := db.Appender(ctx) + app1 := db.AppenderV2(ctx) - ref1, err := app1.Append(0, labels.FromStrings("a", "b"), 123, 0) + ref1, err := app1.Append(0, labels.FromStrings("a", "b"), 0, 123, 0, nil, nil, storage.AOptions{}) require.NoError(t, err) // Add with empty label. - ref2, err := app1.Append(0, labels.FromStrings("a", "b", "c", ""), 124, 0) + ref2, err := app1.Append(0, labels.FromStrings("a", "b", "c", ""), 0, 124, 0, nil, nil, storage.AOptions{}) require.NoError(t, err) // Should be the same series. @@ -488,7 +304,17 @@ func TestAppendEmptyLabelsIgnored(t *testing.T) { require.NoError(t, err) } -func TestDeleteSimple(t *testing.T) { +func TestDBAppenderV2_EmptyLabelsetCausesError(t *testing.T) { + db := newTestDB(t) + + ctx := context.Background() + app := db.AppenderV2(ctx) + _, err := app.Append(0, labels.Labels{}, 0, 0, 0, nil, nil, storage.AOptions{}) + require.Error(t, err) + require.Equal(t, "empty labelset: invalid sample", err.Error()) +} + +func TestDeleteSimple_AppendV2(t *testing.T) { const numSamples int64 = 10 cases := []struct { @@ -522,12 +348,12 @@ func TestDeleteSimple(t *testing.T) { db := newTestDB(t) ctx := context.Background() - app := db.Appender(ctx) + app := db.AppenderV2(ctx) smpls := make([]float64, numSamples) for i := range numSamples { smpls[i] = rand.Float64() - app.Append(0, labels.FromStrings("a", "b"), i, smpls[i]) + app.Append(0, labels.FromStrings("a", "b"), 0, i, smpls[i], nil, nil, storage.AOptions{}) } require.NoError(t, app.Commit()) @@ -576,19 +402,19 @@ func TestDeleteSimple(t *testing.T) { } } -func TestAmendHistogramDatapointCausesError(t *testing.T) { +func TestAmendHistogramDatapointCausesError_AppendV2(t *testing.T) { db := newTestDB(t) ctx := context.Background() - app := db.Appender(ctx) - _, err := app.Append(0, labels.FromStrings("a", "b"), 0, 0) + app := db.AppenderV2(ctx) + _, err := app.Append(0, labels.FromStrings("a", "b"), 0, 0, 0, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) - app = db.Appender(ctx) - _, err = app.Append(0, labels.FromStrings("a", "b"), 0, 0) + app = db.AppenderV2(ctx) + _, err = app.Append(0, labels.FromStrings("a", "b"), 0, 0, 0, nil, nil, storage.AOptions{}) require.NoError(t, err) - _, err = app.Append(0, labels.FromStrings("a", "b"), 0, 1) + _, err = app.Append(0, labels.FromStrings("a", "b"), 0, 0, 1, nil, nil, storage.AOptions{}) require.ErrorIs(t, err, storage.ErrDuplicateSampleForTimestamp) require.NoError(t, app.Rollback()) @@ -606,81 +432,71 @@ func TestAmendHistogramDatapointCausesError(t *testing.T) { } fh := h.ToFloat(nil) - app = db.Appender(ctx) - _, err = app.AppendHistogram(0, labels.FromStrings("a", "c"), 0, h.Copy(), nil) + app = db.AppenderV2(ctx) + _, err = app.Append(0, labels.FromStrings("a", "c"), 0, 0, 0, h.Copy(), nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) - app = db.Appender(ctx) - _, err = app.AppendHistogram(0, labels.FromStrings("a", "c"), 0, h.Copy(), nil) + app = db.AppenderV2(ctx) + _, err = app.Append(0, labels.FromStrings("a", "c"), 0, 0, 0, h.Copy(), nil, storage.AOptions{}) require.NoError(t, err) h.Schema = 2 - _, err = app.AppendHistogram(0, labels.FromStrings("a", "c"), 0, h.Copy(), nil) + _, err = app.Append(0, labels.FromStrings("a", "c"), 0, 0, 0, h.Copy(), nil, storage.AOptions{}) require.Equal(t, storage.ErrDuplicateSampleForTimestamp, err) require.NoError(t, app.Rollback()) // Float histogram. - app = db.Appender(ctx) - _, err = app.AppendHistogram(0, labels.FromStrings("a", "d"), 0, nil, fh.Copy()) + app = db.AppenderV2(ctx) + _, err = app.Append(0, labels.FromStrings("a", "d"), 0, 0, 0, nil, fh.Copy(), storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) - app = db.Appender(ctx) - _, err = app.AppendHistogram(0, labels.FromStrings("a", "d"), 0, nil, fh.Copy()) + app = db.AppenderV2(ctx) + _, err = app.Append(0, labels.FromStrings("a", "d"), 0, 0, 0, nil, fh.Copy(), storage.AOptions{}) require.NoError(t, err) fh.Schema = 2 - _, err = app.AppendHistogram(0, labels.FromStrings("a", "d"), 0, nil, fh.Copy()) + _, err = app.Append(0, labels.FromStrings("a", "d"), 0, 0, 0, nil, fh.Copy(), storage.AOptions{}) require.Equal(t, storage.ErrDuplicateSampleForTimestamp, err) require.NoError(t, app.Rollback()) } -func TestDuplicateNaNDatapointNoAmendError(t *testing.T) { +func TestDuplicateNaNDatapointNoAmendError_AppendV2(t *testing.T) { db := newTestDB(t) ctx := context.Background() - app := db.Appender(ctx) - _, err := app.Append(0, labels.FromStrings("a", "b"), 0, math.NaN()) + app := db.AppenderV2(ctx) + _, err := app.Append(0, labels.FromStrings("a", "b"), 0, 0, math.NaN(), nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) - app = db.Appender(ctx) - _, err = app.Append(0, labels.FromStrings("a", "b"), 0, math.NaN()) + app = db.AppenderV2(ctx) + _, err = app.Append(0, labels.FromStrings("a", "b"), 0, 0, math.NaN(), nil, nil, storage.AOptions{}) require.NoError(t, err) } -func TestNonDuplicateNaNDatapointsCausesAmendError(t *testing.T) { +func TestNonDuplicateNaNDatapointsCausesAmendError_AppendV2(t *testing.T) { db := newTestDB(t) ctx := context.Background() - app := db.Appender(ctx) - _, err := app.Append(0, labels.FromStrings("a", "b"), 0, math.Float64frombits(0x7ff0000000000001)) + app := db.AppenderV2(ctx) + _, err := app.Append(0, labels.FromStrings("a", "b"), 0, 0, math.Float64frombits(0x7ff0000000000001), nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) - app = db.Appender(ctx) - _, err = app.Append(0, labels.FromStrings("a", "b"), 0, math.Float64frombits(0x7ff0000000000002)) + app = db.AppenderV2(ctx) + _, err = app.Append(0, labels.FromStrings("a", "b"), 0, 0, math.Float64frombits(0x7ff0000000000002), nil, nil, storage.AOptions{}) require.ErrorIs(t, err, storage.ErrDuplicateSampleForTimestamp) } -func TestEmptyLabelsetCausesError(t *testing.T) { - db := newTestDB(t) - - ctx := context.Background() - app := db.Appender(ctx) - _, err := app.Append(0, labels.Labels{}, 0, 0) - require.Error(t, err) - require.Equal(t, "empty labelset: invalid sample", err.Error()) -} - -func TestSkippingInvalidValuesInSameTxn(t *testing.T) { +func TestSkippingInvalidValuesInSameTxn_AppendV2(t *testing.T) { db := newTestDB(t) // Append AmendedValue. ctx := context.Background() - app := db.Appender(ctx) - _, err := app.Append(0, labels.FromStrings("a", "b"), 0, 1) + app := db.AppenderV2(ctx) + _, err := app.Append(0, labels.FromStrings("a", "b"), 0, 0, 1, nil, nil, storage.AOptions{}) require.NoError(t, err) - _, err = app.Append(0, labels.FromStrings("a", "b"), 0, 2) + _, err = app.Append(0, labels.FromStrings("a", "b"), 0, 0, 2, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) @@ -695,10 +511,10 @@ func TestSkippingInvalidValuesInSameTxn(t *testing.T) { }, ssMap) // Append Out of Order Value. - app = db.Appender(ctx) - _, err = app.Append(0, labels.FromStrings("a", "b"), 10, 3) + app = db.AppenderV2(ctx) + _, err = app.Append(0, labels.FromStrings("a", "b"), 0, 10, 3, nil, nil, storage.AOptions{}) require.NoError(t, err) - _, err = app.Append(0, labels.FromStrings("a", "b"), 7, 5) + _, err = app.Append(0, labels.FromStrings("a", "b"), 0, 7, 5, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) @@ -712,15 +528,15 @@ func TestSkippingInvalidValuesInSameTxn(t *testing.T) { }, ssMap) } -func TestDB_Snapshot(t *testing.T) { +func TestDB_Snapshot_AppendV2(t *testing.T) { db := newTestDB(t) // append data ctx := context.Background() - app := db.Appender(ctx) + app := db.AppenderV2(ctx) mint := int64(1414141414000) for i := range 1000 { - _, err := app.Append(0, labels.FromStrings("foo", "bar"), mint+int64(i), 1.0) + _, err := app.Append(0, labels.FromStrings("foo", "bar"), 0, mint+int64(i), 1.0, nil, nil, storage.AOptions{}) require.NoError(t, err) } require.NoError(t, app.Commit()) @@ -757,14 +573,14 @@ func TestDB_Snapshot(t *testing.T) { // TestDB_Snapshot_ChunksOutsideOfCompactedRange ensures that a snapshot removes chunks samples // that are outside the set block time range. // See https://github.com/prometheus/prometheus/issues/5105 -func TestDB_Snapshot_ChunksOutsideOfCompactedRange(t *testing.T) { +func TestDB_Snapshot_ChunksOutsideOfCompactedRange_AppendV2(t *testing.T) { db := newTestDB(t) ctx := context.Background() - app := db.Appender(ctx) + app := db.AppenderV2(ctx) mint := int64(1414141414000) for i := range 1000 { - _, err := app.Append(0, labels.FromStrings("foo", "bar"), mint+int64(i), 1.0) + _, err := app.Append(0, labels.FromStrings("foo", "bar"), 0, mint+int64(i), 1.0, nil, nil, storage.AOptions{}) require.NoError(t, err) } require.NoError(t, app.Commit()) @@ -803,18 +619,18 @@ func TestDB_Snapshot_ChunksOutsideOfCompactedRange(t *testing.T) { require.Equal(t, 1000.0-10, sum) } -func TestDB_SnapshotWithDelete(t *testing.T) { +func TestDB_SnapshotWithDelete_AppendV2(t *testing.T) { const numSamples int64 = 10 db := newTestDB(t) ctx := context.Background() - app := db.Appender(ctx) + app := db.AppenderV2(ctx) smpls := make([]float64, numSamples) for i := range numSamples { smpls[i] = rand.Float64() - app.Append(0, labels.FromStrings("a", "b"), i, smpls[i]) + app.Append(0, labels.FromStrings("a", "b"), 0, i, smpls[i], nil, nil, storage.AOptions{}) } require.NoError(t, app.Commit()) @@ -888,7 +704,7 @@ func TestDB_SnapshotWithDelete(t *testing.T) { } } -func TestDB_e2e(t *testing.T) { +func TestDB_e2e_AppendV2(t *testing.T) { const ( numDatapoints = 1000 numRanges = 1000 @@ -946,7 +762,7 @@ func TestDB_e2e(t *testing.T) { db := newTestDB(t) ctx := context.Background() - app := db.Appender(ctx) + app := db.AppenderV2(ctx) for _, l := range lbls { lset := labels.New(l...) @@ -958,7 +774,7 @@ func TestDB_e2e(t *testing.T) { series = append(series, sample{ts, v, nil, nil}) - _, err := app.Append(0, lset, ts, v) + _, err := app.Append(0, lset, 0, ts, v, nil, nil, storage.AOptions{}) require.NoError(t, err) ts += rand.Int63n(timeInterval) + 1 @@ -1044,14 +860,14 @@ func TestDB_e2e(t *testing.T) { } } -func TestWALFlushedOnDBClose(t *testing.T) { +func TestWALFlushedOnDBClose_AppendV2(t *testing.T) { db := newTestDB(t) lbls := labels.FromStrings("labelname", "labelvalue") ctx := context.Background() - app := db.Appender(ctx) - _, err := app.Append(0, lbls, 0, 1) + app := db.AppenderV2(ctx) + _, err := app.Append(0, lbls, 0, 0, 1, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) @@ -1068,7 +884,7 @@ func TestWALFlushedOnDBClose(t *testing.T) { require.Equal(t, []string{"labelvalue"}, values) } -func TestWALSegmentSizeOptions(t *testing.T) { +func TestWALSegmentSizeOptions_AppendV2(t *testing.T) { tests := map[int]func(dbdir string, segmentSize int){ // Default Wal Size. 0: func(dbDir string, _ int) { @@ -1126,11 +942,11 @@ func TestWALSegmentSizeOptions(t *testing.T) { db := newTestDB(t, withOpts(opts)) for i := range int64(155) { - app := db.Appender(context.Background()) - ref, err := app.Append(0, labels.FromStrings("wal"+strconv.Itoa(int(i)), "size"), i, rand.Float64()) + app := db.AppenderV2(context.Background()) + ref, err := app.Append(0, labels.FromStrings("wal"+strconv.Itoa(int(i)), "size"), 0, i, rand.Float64(), nil, nil, storage.AOptions{}) require.NoError(t, err) for j := int64(1); j <= 78; j++ { - _, err := app.Append(ref, labels.EmptyLabels(), i+j, rand.Float64()) + _, err := app.Append(ref, labels.EmptyLabels(), 0, i+j, rand.Float64(), nil, nil, storage.AOptions{}) require.NoError(t, err) } require.NoError(t, app.Commit()) @@ -1144,7 +960,7 @@ func TestWALSegmentSizeOptions(t *testing.T) { // https://github.com/prometheus/prometheus/issues/9846 // https://github.com/prometheus/prometheus/issues/9859 -func TestWALReplayRaceOnSamplesLoggedBeforeSeries(t *testing.T) { +func TestWALReplayRaceOnSamplesLoggedBeforeSeries_AppendV2(t *testing.T) { const ( numRuns = 1 numSamplesBeforeSeriesCreation = 1000 @@ -1155,13 +971,13 @@ func TestWALReplayRaceOnSamplesLoggedBeforeSeries(t *testing.T) { for _, numSamplesAfterSeriesCreation := range []int{1, 1000} { for run := 1; run <= numRuns; run++ { t.Run(fmt.Sprintf("samples after series creation = %d, run = %d", numSamplesAfterSeriesCreation, run), func(t *testing.T) { - testWALReplayRaceOnSamplesLoggedBeforeSeries(t, numSamplesBeforeSeriesCreation, numSamplesAfterSeriesCreation) + testWALReplayRaceOnSamplesLoggedBeforeSeriesAppendV2(t, numSamplesBeforeSeriesCreation, numSamplesAfterSeriesCreation) }) } } } -func testWALReplayRaceOnSamplesLoggedBeforeSeries(t *testing.T, numSamplesBeforeSeriesCreation, numSamplesAfterSeriesCreation int) { +func testWALReplayRaceOnSamplesLoggedBeforeSeriesAppendV2(t *testing.T, numSamplesBeforeSeriesCreation, numSamplesAfterSeriesCreation int) { const numSeries = 1000 db := newTestDB(t) @@ -1184,11 +1000,11 @@ func testWALReplayRaceOnSamplesLoggedBeforeSeries(t *testing.T, numSamplesBefore require.NoError(t, err) // Add samples via appender so that they're logged after the series in the WAL. - app := db.Appender(context.Background()) + app := db.AppenderV2(context.Background()) lbls := labels.FromStrings("series_id", strconv.Itoa(seriesRef)) for ts := numSamplesBeforeSeriesCreation; ts < numSamplesBeforeSeriesCreation+numSamplesAfterSeriesCreation; ts++ { - _, err := app.Append(0, lbls, int64(ts), float64(ts)) + _, err := app.Append(0, lbls, 0, int64(ts), float64(ts), nil, nil, storage.AOptions{}) require.NoError(t, err) } require.NoError(t, app.Commit()) @@ -1225,19 +1041,19 @@ func testWALReplayRaceOnSamplesLoggedBeforeSeries(t *testing.T, numSamplesBefore require.Equal(t, numSeries, actualSeries) } -func TestTombstoneClean(t *testing.T) { +func TestTombstoneClean_AppendV2(t *testing.T) { t.Parallel() const numSamples int64 = 10 db := newTestDB(t) ctx := context.Background() - app := db.Appender(ctx) + app := db.AppenderV2(ctx) smpls := make([]float64, numSamples) for i := range numSamples { smpls[i] = rand.Float64() - app.Append(0, labels.FromStrings("a", "b"), i, smpls[i]) + app.Append(0, labels.FromStrings("a", "b"), 0, i, smpls[i], nil, nil, storage.AOptions{}) } require.NoError(t, app.Commit()) @@ -1318,19 +1134,19 @@ func TestTombstoneClean(t *testing.T) { // TestTombstoneCleanResultEmptyBlock tests that a TombstoneClean that results in empty blocks (no timeseries) // will also delete the resultant block. -func TestTombstoneCleanResultEmptyBlock(t *testing.T) { +func TestTombstoneCleanResultEmptyBlock_AppendV2(t *testing.T) { t.Parallel() numSamples := int64(10) db := newTestDB(t) ctx := context.Background() - app := db.Appender(ctx) + app := db.AppenderV2(ctx) smpls := make([]float64, numSamples) for i := range numSamples { smpls[i] = rand.Float64() - app.Append(0, labels.FromStrings("a", "b"), i, smpls[i]) + app.Append(0, labels.FromStrings("a", "b"), 0, i, smpls[i], nil, nil, storage.AOptions{}) } require.NoError(t, app.Commit()) @@ -1358,180 +1174,7 @@ func TestTombstoneCleanResultEmptyBlock(t *testing.T) { require.Empty(t, actualBlockDirs) } -// TestTombstoneCleanFail tests that a failing TombstoneClean doesn't leave any blocks behind. -// When TombstoneClean errors the original block that should be rebuilt doesn't get deleted so -// if TombstoneClean leaves any blocks behind these will overlap. -func TestTombstoneCleanFail(t *testing.T) { - t.Parallel() - db := newTestDB(t) - - var oldBlockDirs []string - - // Create some blocks pending for compaction. - // totalBlocks should be >=2 so we have enough blocks to trigger compaction failure. - totalBlocks := 2 - for i := range totalBlocks { - blockDir := createBlock(t, db.Dir(), genSeries(1, 1, int64(i), int64(i)+1)) - block, err := OpenBlock(nil, blockDir, nil, nil) - require.NoError(t, err) - // Add some fake tombstones to trigger the compaction. - tomb := tombstones.NewMemTombstones() - tomb.AddInterval(0, tombstones.Interval{Mint: int64(i), Maxt: int64(i) + 1}) - block.tombstones = tomb - - db.blocks = append(db.blocks, block) - oldBlockDirs = append(oldBlockDirs, blockDir) - } - - // Initialize the mockCompactorFailing with a room for a single compaction iteration. - // mockCompactorFailing will fail on the second iteration so we can check if the cleanup works as expected. - db.compactor = &mockCompactorFailing{ - t: t, - blocks: db.blocks, - max: totalBlocks + 1, - } - - // The compactor should trigger a failure here. - require.Error(t, db.CleanTombstones()) - - // Now check that the CleanTombstones replaced the old block even after a failure. - actualBlockDirs, err := blockDirs(db.Dir()) - require.NoError(t, err) - // Only one block should have been replaced by a new block. - require.Len(t, actualBlockDirs, len(oldBlockDirs)) - require.Len(t, intersection(oldBlockDirs, actualBlockDirs), len(actualBlockDirs)-1) -} - -func intersection(oldBlocks, actualBlocks []string) (intersection []string) { - hash := make(map[string]bool) - for _, e := range oldBlocks { - hash[e] = true - } - for _, e := range actualBlocks { - // If block present in the hashmap then append intersection list. - if hash[e] { - intersection = append(intersection, e) - } - } - return intersection -} - -// mockCompactorFailing creates a new empty block on every write and fails when reached the max allowed total. -// For CompactOOO, it always fails. -type mockCompactorFailing struct { - t *testing.T - blocks []*Block - max int -} - -func (*mockCompactorFailing) Plan(string) ([]string, error) { - return nil, nil -} - -func (c *mockCompactorFailing) Write(dest string, _ BlockReader, _, _ int64, _ *BlockMeta) ([]ulid.ULID, error) { - if len(c.blocks) >= c.max { - return []ulid.ULID{}, errors.New("the compactor already did the maximum allowed blocks so it is time to fail") - } - - block, err := OpenBlock(nil, createBlock(c.t, dest, genSeries(1, 1, 0, 1)), nil, nil) - require.NoError(c.t, err) - require.NoError(c.t, block.Close()) // Close block as we won't be using anywhere. - c.blocks = append(c.blocks, block) - - // Now check that all expected blocks are actually persisted on disk. - // This way we make sure that we have some blocks that are supposed to be removed. - var expectedBlocks []string - for _, b := range c.blocks { - expectedBlocks = append(expectedBlocks, filepath.Join(dest, b.Meta().ULID.String())) - } - actualBlockDirs, err := blockDirs(dest) - require.NoError(c.t, err) - - require.Equal(c.t, expectedBlocks, actualBlockDirs) - - return []ulid.ULID{block.Meta().ULID}, nil -} - -func (*mockCompactorFailing) Compact(string, []string, []*Block) ([]ulid.ULID, error) { - return []ulid.ULID{}, nil -} - -func (*mockCompactorFailing) CompactOOO(string, *OOOCompactionHead) (result []ulid.ULID, err error) { - return nil, errors.New("mock compaction failing CompactOOO") -} - -func TestTimeRetention(t *testing.T) { - t.Parallel() - testCases := []struct { - name string - blocks []*BlockMeta - expBlocks []*BlockMeta - retentionDuration int64 - }{ - { - name: "Block max time delta greater than retention duration", - blocks: []*BlockMeta{ - {MinTime: 500, MaxTime: 900}, // Oldest block, beyond retention - {MinTime: 1000, MaxTime: 1500}, - {MinTime: 1500, MaxTime: 2000}, // Newest block - }, - expBlocks: []*BlockMeta{ - {MinTime: 1000, MaxTime: 1500}, - {MinTime: 1500, MaxTime: 2000}, - }, - retentionDuration: 1000, - }, - { - name: "Block max time delta equal to retention duration", - blocks: []*BlockMeta{ - {MinTime: 500, MaxTime: 900}, // Oldest block - {MinTime: 1000, MaxTime: 1500}, // Coinciding exactly with the retention duration. - {MinTime: 1500, MaxTime: 2000}, // Newest block - }, - expBlocks: []*BlockMeta{ - {MinTime: 1500, MaxTime: 2000}, - }, - retentionDuration: 500, - }, - } - for _, tc := range testCases { - t.Run(tc.name, func(t *testing.T) { - db := newTestDB(t, withRngs(1000)) - - for _, m := range tc.blocks { - createBlock(t, db.Dir(), genSeries(10, 10, m.MinTime, m.MaxTime)) - } - - require.NoError(t, db.reloadBlocks()) // Reload the db to register the new blocks. - require.Len(t, db.Blocks(), len(tc.blocks)) // Ensure all blocks are registered. - - db.opts.RetentionDuration = tc.retentionDuration - // Reloading should truncate the blocks which are >= the retention duration vs the first block. - require.NoError(t, db.reloadBlocks()) - - actBlocks := db.Blocks() - - require.Equal(t, 1, int(prom_testutil.ToFloat64(db.metrics.timeRetentionCount)), "metric retention count mismatch") - require.Len(t, actBlocks, len(tc.expBlocks)) - for i, eb := range tc.expBlocks { - require.Equal(t, eb.MinTime, actBlocks[i].meta.MinTime) - require.Equal(t, eb.MaxTime, actBlocks[i].meta.MaxTime) - } - }) - } -} - -func TestRetentionDurationMetric(t *testing.T) { - db := newTestDB(t, withOpts(&Options{ - RetentionDuration: 1000, - }), withRngs(100)) - - expRetentionDuration := 1.0 - actRetentionDuration := prom_testutil.ToFloat64(db.metrics.retentionDuration) - require.Equal(t, expRetentionDuration, actRetentionDuration, "metric retention duration mismatch") -} - -func TestSizeRetention(t *testing.T) { +func TestSizeRetention_AppendV2(t *testing.T) { t.Parallel() opts := DefaultOptions() opts.OutOfOrderTimeWindow = 100 @@ -1554,7 +1197,7 @@ func TestSizeRetention(t *testing.T) { } // Add some data to the WAL. - headApp := db.Head().Appender(context.Background()) + headApp := db.Head().AppenderV2(context.Background()) var aSeries labels.Labels var it chunkenc.Iterator for _, m := range headBlocks { @@ -1564,7 +1207,7 @@ func TestSizeRetention(t *testing.T) { it = s.Iterator(it) for it.Next() == chunkenc.ValFloat { tim, v := it.At() - _, err := headApp.Append(0, s.Labels(), tim, v) + _, err := headApp.Append(0, s.Labels(), 0, tim, v, nil, nil, storage.AOptions{}) require.NoError(t, err) } require.NoError(t, it.Err()) @@ -1620,9 +1263,9 @@ func TestSizeRetention(t *testing.T) { require.Equal(t, expSize, actSize, "registered size doesn't match actual disk size") // Add some out of order samples to check the size of WBL. - headApp = db.Head().Appender(context.Background()) + headApp = db.Head().AppenderV2(context.Background()) for ts := int64(750); ts < 800; ts++ { - _, err := headApp.Append(0, aSeries, ts, float64(ts)) + _, err := headApp.Append(0, aSeries, 0, ts, float64(ts), nil, nil, storage.AOptions{}) require.NoError(t, err) } require.NoError(t, headApp.Commit()) @@ -1668,93 +1311,7 @@ func TestSizeRetention(t *testing.T) { require.Equal(t, expBlocks[len(expBlocks)-1].MaxTime, actBlocks[len(actBlocks)-1].meta.MaxTime, "maxT mismatch of the last block") } -func TestSizeRetentionMetric(t *testing.T) { - cases := []struct { - maxBytes int64 - expMaxBytes int64 - }{ - {maxBytes: 1000, expMaxBytes: 1000}, - {maxBytes: 0, expMaxBytes: 0}, - {maxBytes: -1000, expMaxBytes: 0}, - } - - for _, c := range cases { - db := newTestDB(t, withOpts(&Options{ - MaxBytes: c.maxBytes, - }), withRngs(100)) - - actMaxBytes := int64(prom_testutil.ToFloat64(db.metrics.maxBytes)) - require.Equal(t, c.expMaxBytes, actMaxBytes, "metric retention limit bytes mismatch") - } -} - -// TestRuntimeRetentionConfigChange tests that retention configuration can be -// changed at runtime via ApplyConfig and that the retention logic properly -// deletes blocks when retention is shortened. This test also ensures race-free -// concurrent access to retention settings. -func TestRuntimeRetentionConfigChange(t *testing.T) { - const ( - initialRetentionDuration = int64(10 * time.Hour / time.Millisecond) // 10 hours - shorterRetentionDuration = int64(1 * time.Hour / time.Millisecond) // 1 hour - ) - - db := newTestDB(t, withOpts(&Options{ - RetentionDuration: initialRetentionDuration, - }), withRngs(100)) - - nineHoursMs := int64(9 * time.Hour / time.Millisecond) - nineAndHalfHoursMs := int64((9*time.Hour + 30*time.Minute) / time.Millisecond) - blocks := []*BlockMeta{ - {MinTime: 0, MaxTime: 100}, // 10 hours old (beyond new retention) - {MinTime: 100, MaxTime: 200}, // 9.9 hours old (beyond new retention) - {MinTime: nineHoursMs, MaxTime: nineAndHalfHoursMs}, // 1 hour old (within new retention) - {MinTime: nineAndHalfHoursMs, MaxTime: initialRetentionDuration}, // 0.5 hours old (within new retention) - } - - for _, m := range blocks { - createBlock(t, db.Dir(), genSeriesFromSampleGenerator(10, 10, m.MinTime, m.MaxTime, int64(time.Minute/time.Millisecond), func(ts int64) chunks.Sample { - return sample{t: ts, f: rand.Float64()} - })) - } - - // Reload blocks and verify all are loaded. - require.NoError(t, db.reloadBlocks()) - require.Len(t, db.Blocks(), len(blocks), "expected all blocks to be loaded initially") - - cfg := &config.Config{ - StorageConfig: config.StorageConfig{ - TSDBConfig: &config.TSDBConfig{ - Retention: &config.TSDBRetentionConfig{ - Time: model.Duration(shorterRetentionDuration), - }, - }, - }, - } - - require.NoError(t, db.ApplyConfig(cfg), "ApplyConfig should succeed") - - actualRetention := db.getRetentionDuration() - require.Equal(t, shorterRetentionDuration, actualRetention, "retention duration should be updated") - - expectedRetentionSeconds := (time.Duration(shorterRetentionDuration) * time.Millisecond).Seconds() - actualRetentionSeconds := prom_testutil.ToFloat64(db.metrics.retentionDuration) - require.Equal(t, expectedRetentionSeconds, actualRetentionSeconds, "retention duration metric should be updated") - - require.NoError(t, db.reloadBlocks()) - - // Verify that blocks beyond the new retention were deleted. - // We expect only the last 2 blocks to remain (those within 1 hour). - actBlocks := db.Blocks() - require.Len(t, actBlocks, 2, "expected old blocks to be deleted after retention change") - - // Verify the remaining blocks are the newest ones. - require.Equal(t, nineHoursMs, actBlocks[0].meta.MinTime, "first remaining block should be within retention") - require.Equal(t, nineAndHalfHoursMs, actBlocks[1].meta.MinTime, "last remaining block should be the newest") - - require.Positive(t, int(prom_testutil.ToFloat64(db.metrics.timeRetentionCount)), "time retention count should be incremented") -} - -func TestNotMatcherSelectsLabelsUnsetSeries(t *testing.T) { +func TestNotMatcherSelectsLabelsUnsetSeries_AppendV2(t *testing.T) { db := newTestDB(t) labelpairs := []labels.Labels{ @@ -1763,9 +1320,9 @@ func TestNotMatcherSelectsLabelsUnsetSeries(t *testing.T) { } ctx := context.Background() - app := db.Appender(ctx) + app := db.AppenderV2(ctx) for _, lbls := range labelpairs { - _, err := app.Append(0, lbls, 0, 1) + _, err := app.Append(0, lbls, 0, 0, 1, nil, nil, storage.AOptions{}) require.NoError(t, err) } require.NoError(t, app.Commit()) @@ -1825,133 +1382,21 @@ func TestNotMatcherSelectsLabelsUnsetSeries(t *testing.T) { } } -// expandSeriesSet returns the raw labels in the order they are retrieved from -// the series set and the samples keyed by Labels().String(). -func expandSeriesSet(ss storage.SeriesSet) ([]labels.Labels, map[string][]sample, annotations.Annotations, error) { - resultLabels := []labels.Labels{} - resultSamples := map[string][]sample{} - var it chunkenc.Iterator - for ss.Next() { - series := ss.At() - samples := []sample{} - it = series.Iterator(it) - for it.Next() == chunkenc.ValFloat { - t, v := it.At() - samples = append(samples, sample{t: t, f: v}) - } - resultLabels = append(resultLabels, series.Labels()) - resultSamples[series.Labels().String()] = samples - } - return resultLabels, resultSamples, ss.Warnings(), ss.Err() -} - -func TestOverlappingBlocksDetectsAllOverlaps(t *testing.T) { - // Create 10 blocks that does not overlap (0-10, 10-20, ..., 100-110) but in reverse order to ensure our algorithm - // will handle that. - metas := make([]BlockMeta, 11) - for i := 10; i >= 0; i-- { - metas[i] = BlockMeta{MinTime: int64(i * 10), MaxTime: int64((i + 1) * 10)} - } - - require.Empty(t, OverlappingBlocks(metas), "we found unexpected overlaps") - - // Add overlapping blocks. We've to establish order again since we aren't interested - // in trivial overlaps caused by unorderedness. - add := func(ms ...BlockMeta) []BlockMeta { - repl := append(append([]BlockMeta{}, metas...), ms...) - sort.Slice(repl, func(i, j int) bool { - return repl[i].MinTime < repl[j].MinTime - }) - return repl - } - - // o1 overlaps with 10-20. - o1 := BlockMeta{MinTime: 15, MaxTime: 17} - require.Equal(t, Overlaps{ - {Min: 15, Max: 17}: {metas[1], o1}, - }, OverlappingBlocks(add(o1))) - - // o2 overlaps with 20-30 and 30-40. - o2 := BlockMeta{MinTime: 21, MaxTime: 31} - require.Equal(t, Overlaps{ - {Min: 21, Max: 30}: {metas[2], o2}, - {Min: 30, Max: 31}: {o2, metas[3]}, - }, OverlappingBlocks(add(o2))) - - // o3a and o3b overlaps with 30-40 and each other. - o3a := BlockMeta{MinTime: 33, MaxTime: 39} - o3b := BlockMeta{MinTime: 34, MaxTime: 36} - require.Equal(t, Overlaps{ - {Min: 34, Max: 36}: {metas[3], o3a, o3b}, - }, OverlappingBlocks(add(o3a, o3b))) - - // o4 is 1:1 overlap with 50-60. - o4 := BlockMeta{MinTime: 50, MaxTime: 60} - require.Equal(t, Overlaps{ - {Min: 50, Max: 60}: {metas[5], o4}, - }, OverlappingBlocks(add(o4))) - - // o5 overlaps with 60-70, 70-80 and 80-90. - o5 := BlockMeta{MinTime: 61, MaxTime: 85} - require.Equal(t, Overlaps{ - {Min: 61, Max: 70}: {metas[6], o5}, - {Min: 70, Max: 80}: {o5, metas[7]}, - {Min: 80, Max: 85}: {o5, metas[8]}, - }, OverlappingBlocks(add(o5))) - - // o6a overlaps with 90-100, 100-110 and o6b, o6b overlaps with 90-100 and o6a. - o6a := BlockMeta{MinTime: 92, MaxTime: 105} - o6b := BlockMeta{MinTime: 94, MaxTime: 99} - require.Equal(t, Overlaps{ - {Min: 94, Max: 99}: {metas[9], o6a, o6b}, - {Min: 100, Max: 105}: {o6a, metas[10]}, - }, OverlappingBlocks(add(o6a, o6b))) - - // All together. - require.Equal(t, Overlaps{ - {Min: 15, Max: 17}: {metas[1], o1}, - {Min: 21, Max: 30}: {metas[2], o2}, {Min: 30, Max: 31}: {o2, metas[3]}, - {Min: 34, Max: 36}: {metas[3], o3a, o3b}, - {Min: 50, Max: 60}: {metas[5], o4}, - {Min: 61, Max: 70}: {metas[6], o5}, {Min: 70, Max: 80}: {o5, metas[7]}, {Min: 80, Max: 85}: {o5, metas[8]}, - {Min: 94, Max: 99}: {metas[9], o6a, o6b}, {Min: 100, Max: 105}: {o6a, metas[10]}, - }, OverlappingBlocks(add(o1, o2, o3a, o3b, o4, o5, o6a, o6b))) - - // Additional case. - var nc1 []BlockMeta - nc1 = append(nc1, BlockMeta{MinTime: 1, MaxTime: 5}) - nc1 = append(nc1, BlockMeta{MinTime: 2, MaxTime: 3}) - nc1 = append(nc1, BlockMeta{MinTime: 2, MaxTime: 3}) - nc1 = append(nc1, BlockMeta{MinTime: 2, MaxTime: 3}) - nc1 = append(nc1, BlockMeta{MinTime: 2, MaxTime: 3}) - nc1 = append(nc1, BlockMeta{MinTime: 2, MaxTime: 6}) - nc1 = append(nc1, BlockMeta{MinTime: 3, MaxTime: 5}) - nc1 = append(nc1, BlockMeta{MinTime: 5, MaxTime: 7}) - nc1 = append(nc1, BlockMeta{MinTime: 7, MaxTime: 10}) - nc1 = append(nc1, BlockMeta{MinTime: 8, MaxTime: 9}) - require.Equal(t, Overlaps{ - {Min: 2, Max: 3}: {nc1[0], nc1[1], nc1[2], nc1[3], nc1[4], nc1[5]}, // 1-5, 2-3, 2-3, 2-3, 2-3, 2,6 - {Min: 3, Max: 5}: {nc1[0], nc1[5], nc1[6]}, // 1-5, 2-6, 3-5 - {Min: 5, Max: 6}: {nc1[5], nc1[7]}, // 2-6, 5-7 - {Min: 8, Max: 9}: {nc1[8], nc1[9]}, // 7-10, 8-9 - }, OverlappingBlocks(nc1)) -} - // Regression test for https://github.com/prometheus/tsdb/issues/347 -func TestChunkAtBlockBoundary(t *testing.T) { +func TestChunkAtBlockBoundary_AppendV2(t *testing.T) { t.Parallel() db := newTestDB(t) ctx := context.Background() - app := db.Appender(ctx) + app := db.AppenderV2(ctx) blockRange := db.compactor.(*LeveledCompactor).ranges[0] label := labels.FromStrings("foo", "bar") for i := range int64(3) { - _, err := app.Append(0, label, i*blockRange, 0) + _, err := app.Append(0, label, 0, i*blockRange, 0, nil, nil, storage.AOptions{}) require.NoError(t, err) - _, err = app.Append(0, label, i*blockRange+1000, 0) + _, err = app.Append(0, label, 0, i*blockRange+1000, 0, nil, nil, storage.AOptions{}) require.NoError(t, err) } @@ -1992,20 +1437,20 @@ func TestChunkAtBlockBoundary(t *testing.T) { } } -func TestQuerierWithBoundaryChunks(t *testing.T) { +func TestQuerierWithBoundaryChunks_AppendV2(t *testing.T) { t.Parallel() db := newTestDB(t) ctx := context.Background() - app := db.Appender(ctx) + app := db.AppenderV2(ctx) blockRange := db.compactor.(*LeveledCompactor).ranges[0] label := labels.FromStrings("foo", "bar") for i := range int64(5) { - _, err := app.Append(0, label, i*blockRange, 0) + _, err := app.Append(0, label, 0, i*blockRange, 0, nil, nil, storage.AOptions{}) require.NoError(t, err) - _, err = app.Append(0, labels.FromStrings("blockID", strconv.FormatInt(i, 10)), i*blockRange, 0) + _, err = app.Append(0, labels.FromStrings("blockID", strconv.FormatInt(i, 10)), 0, i*blockRange, 0, nil, nil, storage.AOptions{}) require.NoError(t, err) } @@ -2034,7 +1479,7 @@ func TestQuerierWithBoundaryChunks(t *testing.T) { // - no blocks with WAL: set to the smallest sample from the WAL // - with blocks no WAL: set to the last block maxT // - with blocks with WAL: same as above -func TestInitializeHeadTimestamp(t *testing.T) { +func TestInitializeHeadTimestamp_AppendV2(t *testing.T) { t.Parallel() t.Run("clean", func(t *testing.T) { db := newTestDB(t) @@ -2046,8 +1491,8 @@ func TestInitializeHeadTimestamp(t *testing.T) { // First added sample initializes the writable range. ctx := context.Background() - app := db.Appender(ctx) - _, err := app.Append(0, labels.FromStrings("a", "b"), 1000, 1) + app := db.AppenderV2(ctx) + _, err := app.Append(0, labels.FromStrings("a", "b"), 0, 1000, 1, nil, nil, storage.AOptions{}) require.NoError(t, err) require.Equal(t, int64(1000), db.head.MinTime()) @@ -2125,7 +1570,7 @@ func TestInitializeHeadTimestamp(t *testing.T) { }) } -func TestNoEmptyBlocks(t *testing.T) { +func TestNoEmptyBlocks_AppendV2(t *testing.T) { t.Parallel() db := newTestDB(t, withRngs(100)) ctx := context.Background() @@ -2146,12 +1591,12 @@ func TestNoEmptyBlocks(t *testing.T) { }) t.Run("Test no blocks after deleting all samples from head.", func(t *testing.T) { - app := db.Appender(ctx) - _, err := app.Append(0, defaultLabel, 1, 0) + app := db.AppenderV2(ctx) + _, err := app.Append(0, defaultLabel, 0, 1, 0, nil, nil, storage.AOptions{}) require.NoError(t, err) - _, err = app.Append(0, defaultLabel, 2, 0) + _, err = app.Append(0, defaultLabel, 0, 2, 0, nil, nil, storage.AOptions{}) require.NoError(t, err) - _, err = app.Append(0, defaultLabel, 3+rangeToTriggerCompaction, 0) + _, err = app.Append(0, defaultLabel, 0, 3+rangeToTriggerCompaction, 0, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) require.NoError(t, db.Delete(ctx, math.MinInt64, math.MaxInt64, defaultMatcher)) @@ -2163,17 +1608,17 @@ func TestNoEmptyBlocks(t *testing.T) { require.Len(t, actBlocks, len(db.Blocks())) require.Empty(t, actBlocks) - app = db.Appender(ctx) - _, err = app.Append(0, defaultLabel, 1, 0) + app = db.AppenderV2(ctx) + _, err = app.Append(0, defaultLabel, 0, 1, 0, nil, nil, storage.AOptions{}) require.Equal(t, storage.ErrOutOfBounds, err, "the head should be truncated so no samples in the past should be allowed") // Adding new blocks. currentTime := db.Head().MaxTime() - _, err = app.Append(0, defaultLabel, currentTime, 0) + _, err = app.Append(0, defaultLabel, 0, currentTime, 0, nil, nil, storage.AOptions{}) require.NoError(t, err) - _, err = app.Append(0, defaultLabel, currentTime+1, 0) + _, err = app.Append(0, defaultLabel, 0, currentTime+1, 0, nil, nil, storage.AOptions{}) require.NoError(t, err) - _, err = app.Append(0, defaultLabel, currentTime+rangeToTriggerCompaction, 0) + _, err = app.Append(0, defaultLabel, 0, currentTime+rangeToTriggerCompaction, 0, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) @@ -2188,13 +1633,13 @@ func TestNoEmptyBlocks(t *testing.T) { t.Run(`When no new block is created from head, and there are some blocks on disk compaction should not run into infinite loop (was seen during development).`, func(t *testing.T) { oldBlocks := db.Blocks() - app := db.Appender(ctx) + app := db.AppenderV2(ctx) currentTime := db.Head().MaxTime() - _, err := app.Append(0, defaultLabel, currentTime, 0) + _, err := app.Append(0, defaultLabel, 0, currentTime, 0, nil, nil, storage.AOptions{}) require.NoError(t, err) - _, err = app.Append(0, defaultLabel, currentTime+1, 0) + _, err = app.Append(0, defaultLabel, 0, currentTime+1, 0, nil, nil, storage.AOptions{}) require.NoError(t, err) - _, err = app.Append(0, defaultLabel, currentTime+rangeToTriggerCompaction, 0) + _, err = app.Append(0, defaultLabel, 0, currentTime+rangeToTriggerCompaction, 0, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) require.NoError(t, db.head.Delete(ctx, math.MinInt64, math.MaxInt64, defaultMatcher)) @@ -2227,7 +1672,7 @@ func TestNoEmptyBlocks(t *testing.T) { }) } -func TestDB_LabelNames(t *testing.T) { +func TestDB_LabelNames_AppendV2(t *testing.T) { ctx := context.Background() tests := []struct { // Add 'sampleLabels1' -> Test Head -> Compact -> Test Disk -> @@ -2272,11 +1717,11 @@ func TestDB_LabelNames(t *testing.T) { // Appends samples into the database. appendSamples := func(db *DB, mint, maxt int64, sampleLabels [][2]string) { t.Helper() - app := db.Appender(ctx) + app := db.AppenderV2(ctx) for i := mint; i <= maxt; i++ { for _, tuple := range sampleLabels { label := labels.FromStrings(tuple[0], tuple[1]) - _, err := app.Append(0, label, i*blockRange, 0) + _, err := app.Append(0, label, 0, i*blockRange, 0, nil, nil, storage.AOptions{}) require.NoError(t, err) } } @@ -2329,7 +1774,7 @@ func TestDB_LabelNames(t *testing.T) { } } -func TestCorrectNumTombstones(t *testing.T) { +func TestCorrectNumTombstones_AppendV2(t *testing.T) { t.Parallel() db := newTestDB(t) @@ -2339,10 +1784,10 @@ func TestCorrectNumTombstones(t *testing.T) { defaultMatcher := labels.MustNewMatcher(labels.MatchEqual, name, value) ctx := context.Background() - app := db.Appender(ctx) + app := db.AppenderV2(ctx) for i := range int64(3) { for j := range int64(15) { - _, err := app.Append(0, defaultLabel, i*blockRange+j, 0) + _, err := app.Append(0, defaultLabel, 0, i*blockRange+j, 0, nil, nil, storage.AOptions{}) require.NoError(t, err) } } @@ -2377,7 +1822,7 @@ func TestCorrectNumTombstones(t *testing.T) { // // This ensures that a snapshot that includes the head and creates a block with a custom time range // will not overlap with the first block created by the next compaction. -func TestBlockRanges(t *testing.T) { +func TestBlockRanges_AppendV2(t *testing.T) { t.Parallel() logger := promslog.New(&promslog.Config{}) ctx := context.Background() @@ -2393,16 +1838,16 @@ func TestBlockRanges(t *testing.T) { rangeToTriggerCompaction := db.compactor.(*LeveledCompactor).ranges[0]/2*3 + 1 - app := db.Appender(ctx) + app := db.AppenderV2(ctx) lbl := labels.FromStrings("a", "b") - _, err = app.Append(0, lbl, firstBlockMaxT-1, rand.Float64()) + _, err = app.Append(0, lbl, 0, firstBlockMaxT-1, rand.Float64(), nil, nil, storage.AOptions{}) require.Error(t, err, "appending a sample with a timestamp covered by a previous block shouldn't be possible") - _, err = app.Append(0, lbl, firstBlockMaxT+1, rand.Float64()) + _, err = app.Append(0, lbl, 0, firstBlockMaxT+1, rand.Float64(), nil, nil, storage.AOptions{}) require.NoError(t, err) - _, err = app.Append(0, lbl, firstBlockMaxT+2, rand.Float64()) + _, err = app.Append(0, lbl, 0, firstBlockMaxT+2, rand.Float64(), nil, nil, storage.AOptions{}) require.NoError(t, err) secondBlockMaxt := firstBlockMaxT + rangeToTriggerCompaction - _, err = app.Append(0, lbl, secondBlockMaxt, rand.Float64()) // Add samples to trigger a new compaction + _, err = app.Append(0, lbl, 0, secondBlockMaxt, rand.Float64(), nil, nil, storage.AOptions{}) // Add samples to trigger a new compaction require.NoError(t, err) require.NoError(t, app.Commit()) @@ -2419,15 +1864,15 @@ func TestBlockRanges(t *testing.T) { // Test that wal records are skipped when an existing block covers the same time ranges // and compaction doesn't create an overlapping block. - app = db.Appender(ctx) + app = db.AppenderV2(ctx) db.DisableCompactions() - _, err = app.Append(0, lbl, secondBlockMaxt+1, rand.Float64()) + _, err = app.Append(0, lbl, 0, secondBlockMaxt+1, rand.Float64(), nil, nil, storage.AOptions{}) require.NoError(t, err) - _, err = app.Append(0, lbl, secondBlockMaxt+2, rand.Float64()) + _, err = app.Append(0, lbl, 0, secondBlockMaxt+2, rand.Float64(), nil, nil, storage.AOptions{}) require.NoError(t, err) - _, err = app.Append(0, lbl, secondBlockMaxt+3, rand.Float64()) + _, err = app.Append(0, lbl, 0, secondBlockMaxt+3, rand.Float64(), nil, nil, storage.AOptions{}) require.NoError(t, err) - _, err = app.Append(0, lbl, secondBlockMaxt+4, rand.Float64()) + _, err = app.Append(0, lbl, 0, secondBlockMaxt+4, rand.Float64(), nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) require.NoError(t, db.Close()) @@ -2442,8 +1887,8 @@ func TestBlockRanges(t *testing.T) { require.Len(t, db.Blocks(), 3, "db doesn't include expected number of blocks") require.Equal(t, db.Blocks()[2].Meta().MaxTime, thirdBlockMaxt, "unexpected maxt of the last block") - app = db.Appender(ctx) - _, err = app.Append(0, lbl, thirdBlockMaxt+rangeToTriggerCompaction, rand.Float64()) // Trigger a compaction + app = db.AppenderV2(ctx) + _, err = app.Append(0, lbl, 0, thirdBlockMaxt+rangeToTriggerCompaction, rand.Float64(), nil, nil, storage.AOptions{}) // Trigger a compaction require.NoError(t, err) require.NoError(t, app.Commit()) for range 100 { @@ -2461,10 +1906,11 @@ func TestBlockRanges(t *testing.T) { // TestDBReadOnly ensures that opening a DB in readonly mode doesn't modify any files on the disk. // It also checks that the API calls return equivalent results as a normal db.Open() mode. -func TestDBReadOnly(t *testing.T) { +func TestDBReadOnly_AppendV2(t *testing.T) { t.Parallel() var ( dbDir = t.TempDir() + logger = promslog.New(&promslog.Config{}) expBlocks []*Block expBlock *Block expSeries map[string][]chunks.Sample @@ -2488,7 +1934,7 @@ func TestDBReadOnly(t *testing.T) { } // Add head to test DBReadOnly WAL reading capabilities. - w, err := wlog.New(nil, nil, filepath.Join(dbDir, "wal"), compression.Snappy) + w, err := wlog.New(logger, nil, filepath.Join(dbDir, "wal"), compression.Snappy) require.NoError(t, err) h := createHead(t, w, genSeries(1, 1, 16, 18), dbDir) require.NoError(t, h.Close()) @@ -2501,8 +1947,8 @@ func TestDBReadOnly(t *testing.T) { dbSizeBeforeAppend, err := fileutil.DirSize(dbWritable.Dir()) require.NoError(t, err) - app := dbWritable.Appender(context.Background()) - _, err = app.Append(0, labels.FromStrings("foo", "bar"), dbWritable.Head().MaxTime()+1, 0) + app := dbWritable.AppenderV2(context.Background()) + _, err = app.Append(0, labels.FromStrings("foo", "bar"), 0, dbWritable.Head().MaxTime()+1, 0, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) @@ -2524,7 +1970,7 @@ func TestDBReadOnly(t *testing.T) { } // Open a read only db and ensure that the API returns the same result as the normal DB. - dbReadOnly, err := OpenDBReadOnly(dbDir, "", nil) + dbReadOnly, err := OpenDBReadOnly(dbDir, "", logger) require.NoError(t, err) defer func() { require.NoError(t, dbReadOnly.Close()) }() @@ -2575,32 +2021,14 @@ func TestDBReadOnly(t *testing.T) { }) } -// TestDBReadOnlyClosing ensures that after closing the db -// all api methods return an ErrClosed. -func TestDBReadOnlyClosing(t *testing.T) { - t.Parallel() - sandboxDir := t.TempDir() - db, err := OpenDBReadOnly(t.TempDir(), sandboxDir, promslog.New(&promslog.Config{})) - require.NoError(t, err) - // The sandboxDir was there. - require.DirExists(t, db.sandboxDir) - require.NoError(t, db.Close()) - // The sandboxDir was deleted when closing. - require.NoDirExists(t, db.sandboxDir) - require.Equal(t, db.Close(), ErrClosed) - _, err = db.Blocks() - require.Equal(t, err, ErrClosed) - _, err = db.Querier(0, 1) - require.Equal(t, err, ErrClosed) -} - -func TestDBReadOnly_FlushWAL(t *testing.T) { +func TestDBReadOnly_FlushWAL_AppendV2(t *testing.T) { t.Parallel() var ( - dbDir = t.TempDir() - err error - maxt int - ctx = context.Background() + dbDir = t.TempDir() + logger = promslog.New(&promslog.Config{}) + err error + maxt int + ctx = context.Background() ) // Bootstrap the db. @@ -2608,10 +2036,10 @@ func TestDBReadOnly_FlushWAL(t *testing.T) { // Append data to the WAL. db := newTestDB(t, withDir(dbDir)) db.DisableCompactions() - app := db.Appender(ctx) + app := db.AppenderV2(ctx) maxt = 1000 for i := 0; i < maxt; i++ { - _, err := app.Append(0, labels.FromStrings(defaultLabelName, "flush"), int64(i), 1.0) + _, err := app.Append(0, labels.FromStrings(defaultLabelName, "flush"), 0, int64(i), 1.0, nil, nil, storage.AOptions{}) require.NoError(t, err) } require.NoError(t, app.Commit()) @@ -2619,7 +2047,7 @@ func TestDBReadOnly_FlushWAL(t *testing.T) { } // Flush WAL. - db, err := OpenDBReadOnly(dbDir, "", nil) + db, err := OpenDBReadOnly(dbDir, "", logger) require.NoError(t, err) flush := t.TempDir() @@ -2627,7 +2055,7 @@ func TestDBReadOnly_FlushWAL(t *testing.T) { require.NoError(t, db.Close()) // Reopen the DB from the flushed WAL block. - db, err = OpenDBReadOnly(flush, "", nil) + db, err = OpenDBReadOnly(flush, "", logger) require.NoError(t, err) defer func() { require.NoError(t, db.Close()) }() blocks, err := db.Blocks() @@ -2656,7 +2084,7 @@ func TestDBReadOnly_FlushWAL(t *testing.T) { require.Equal(t, 1000.0, sum) } -func TestDBReadOnly_Querier_NoAlteration(t *testing.T) { +func TestDBReadOnly_Querier_NoAlteration_AppendV2(t *testing.T) { countChunks := func(dir string) int { files, err := os.ReadDir(mmappedChunksDir(dir)) require.NoError(t, err) @@ -2692,8 +2120,8 @@ func TestDBReadOnly_Querier_NoAlteration(t *testing.T) { // Append until the first mmapped head chunk. for i := range 121 { - app := db.Appender(context.Background()) - _, err := app.Append(0, labels.FromStrings("foo", "bar"), int64(i), 0) + app := db.AppenderV2(context.Background()) + _, err := app.Append(0, labels.FromStrings("foo", "bar"), 0, int64(i), 0, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) } @@ -2726,7 +2154,7 @@ func TestDBReadOnly_Querier_NoAlteration(t *testing.T) { }) } -func TestDBCannotSeePartialCommits(t *testing.T) { +func TestDBCannotSeePartialCommits_AppendV2(t *testing.T) { if defaultIsolationDisabled { t.Skip("skipping test since tsdb isolation is disabled") } @@ -2741,10 +2169,10 @@ func TestDBCannotSeePartialCommits(t *testing.T) { go func() { iter := 0 for { - app := db.Appender(ctx) + app := db.AppenderV2(ctx) for j := range 100 { - _, err := app.Append(0, labels.FromStrings("foo", "bar", "a", strconv.Itoa(j)), int64(iter), float64(iter)) + _, err := app.Append(0, labels.FromStrings("foo", "bar", "a", strconv.Itoa(j)), 0, int64(iter), float64(iter), nil, nil, storage.AOptions{}) require.NoError(t, err) } require.NoError(t, app.Commit()) @@ -2792,7 +2220,7 @@ func TestDBCannotSeePartialCommits(t *testing.T) { require.Equal(t, 0, inconsistencies, "Some queries saw inconsistent results.") } -func TestDBQueryDoesntSeeAppendsAfterCreation(t *testing.T) { +func TestDBQueryDoesntSeeAppendsAfterCreation_AppendV2(t *testing.T) { if defaultIsolationDisabled { t.Skip("skipping test since tsdb isolation is disabled") } @@ -2803,8 +2231,8 @@ func TestDBQueryDoesntSeeAppendsAfterCreation(t *testing.T) { defer querierBeforeAdd.Close() ctx := context.Background() - app := db.Appender(ctx) - _, err = app.Append(0, labels.FromStrings("foo", "bar"), 0, 0) + app := db.AppenderV2(ctx) + _, err = app.Append(0, labels.FromStrings("foo", "bar"), 0, 0, 0, nil, nil, storage.AOptions{}) require.NoError(t, err) querierAfterAddButBeforeCommit, err := db.Querier(0, 1000000) @@ -2854,257 +2282,6 @@ func TestDBQueryDoesntSeeAppendsAfterCreation(t *testing.T) { require.Equal(t, map[string][]sample{`{foo="bar"}`: {{t: 0, f: 0}}}, seriesSet) } -func assureChunkFromSamples(t *testing.T, samples []chunks.Sample) chunks.Meta { - chks, err := chunks.ChunkFromSamples(samples) - require.NoError(t, err) - return chks -} - -// TestChunkWriter_ReadAfterWrite ensures that chunk segment are cut at the set segment size and -// that the resulted segments includes the expected chunks data. -func TestChunkWriter_ReadAfterWrite(t *testing.T) { - chk1 := assureChunkFromSamples(t, []chunks.Sample{sample{1, 1, nil, nil}}) - chk2 := assureChunkFromSamples(t, []chunks.Sample{sample{1, 2, nil, nil}}) - chk3 := assureChunkFromSamples(t, []chunks.Sample{sample{1, 3, nil, nil}}) - chk4 := assureChunkFromSamples(t, []chunks.Sample{sample{1, 4, nil, nil}}) - chk5 := assureChunkFromSamples(t, []chunks.Sample{sample{1, 5, nil, nil}}) - chunkSize := len(chk1.Chunk.Bytes()) + chunks.MaxChunkLengthFieldSize + chunks.ChunkEncodingSize + crc32.Size - - tests := []struct { - chks [][]chunks.Meta - segmentSize, - expSegmentsCount int - expSegmentSizes []int - }{ - // 0:Last chunk ends at the segment boundary so - // all chunks should fit in a single segment. - { - chks: [][]chunks.Meta{ - { - chk1, - chk2, - chk3, - }, - }, - segmentSize: 3 * chunkSize, - expSegmentSizes: []int{3 * chunkSize}, - expSegmentsCount: 1, - }, - // 1:Two chunks can fit in a single segment so the last one should result in a new segment. - { - chks: [][]chunks.Meta{ - { - chk1, - chk2, - chk3, - chk4, - chk5, - }, - }, - segmentSize: 2 * chunkSize, - expSegmentSizes: []int{2 * chunkSize, 2 * chunkSize, chunkSize}, - expSegmentsCount: 3, - }, - // 2:When the segment size is smaller than the size of 2 chunks - // the last segment should still create a new segment. - { - chks: [][]chunks.Meta{ - { - chk1, - chk2, - chk3, - }, - }, - segmentSize: 2*chunkSize - 1, - expSegmentSizes: []int{chunkSize, chunkSize, chunkSize}, - expSegmentsCount: 3, - }, - // 3:When the segment is smaller than a single chunk - // it should still be written by ignoring the max segment size. - { - chks: [][]chunks.Meta{ - { - chk1, - }, - }, - segmentSize: chunkSize - 1, - expSegmentSizes: []int{chunkSize}, - expSegmentsCount: 1, - }, - // 4:All chunks are bigger than the max segment size, but - // these should still be written even when this will result in bigger segment than the set size. - // Each segment will hold a single chunk. - { - chks: [][]chunks.Meta{ - { - chk1, - chk2, - chk3, - }, - }, - segmentSize: 1, - expSegmentSizes: []int{chunkSize, chunkSize, chunkSize}, - expSegmentsCount: 3, - }, - // 5:Adding multiple batches of chunks. - { - chks: [][]chunks.Meta{ - { - chk1, - chk2, - chk3, - }, - { - chk4, - chk5, - }, - }, - segmentSize: 3 * chunkSize, - expSegmentSizes: []int{3 * chunkSize, 2 * chunkSize}, - expSegmentsCount: 2, - }, - // 6:Adding multiple batches of chunks. - { - chks: [][]chunks.Meta{ - { - chk1, - }, - { - chk2, - chk3, - }, - { - chk4, - }, - }, - segmentSize: 2 * chunkSize, - expSegmentSizes: []int{2 * chunkSize, 2 * chunkSize}, - expSegmentsCount: 2, - }, - } - - for i, test := range tests { - t.Run(strconv.Itoa(i), func(t *testing.T) { - tempDir := t.TempDir() - - chunkw, err := chunks.NewWriter(tempDir, chunks.WithSegmentSize(chunks.SegmentHeaderSize+int64(test.segmentSize))) - require.NoError(t, err) - - for _, chks := range test.chks { - require.NoError(t, chunkw.WriteChunks(chks...)) - } - require.NoError(t, chunkw.Close()) - - files, err := os.ReadDir(tempDir) - require.NoError(t, err) - require.Len(t, files, test.expSegmentsCount, "expected segments count mismatch") - - // Verify that all data is written to the segments. - sizeExp := 0 - sizeAct := 0 - - for _, chks := range test.chks { - for _, chk := range chks { - l := make([]byte, binary.MaxVarintLen32) - sizeExp += binary.PutUvarint(l, uint64(len(chk.Chunk.Bytes()))) // The length field. - sizeExp += chunks.ChunkEncodingSize - sizeExp += len(chk.Chunk.Bytes()) // The data itself. - sizeExp += crc32.Size // The 4 bytes of crc32 - } - } - sizeExp += test.expSegmentsCount * chunks.SegmentHeaderSize // The segment header bytes. - - for i, f := range files { - fi, err := f.Info() - require.NoError(t, err) - size := int(fi.Size()) - // Verify that the segment is the same or smaller than the expected size. - require.GreaterOrEqual(t, chunks.SegmentHeaderSize+test.expSegmentSizes[i], size, "Segment:%v should NOT be bigger than:%v actual:%v", i, chunks.SegmentHeaderSize+test.expSegmentSizes[i], size) - - sizeAct += size - } - require.Equal(t, sizeExp, sizeAct) - - // Check the content of the chunks. - r, err := chunks.NewDirReader(tempDir, nil) - require.NoError(t, err) - defer func() { require.NoError(t, r.Close()) }() - - for _, chks := range test.chks { - for _, chkExp := range chks { - chkAct, iterable, err := r.ChunkOrIterable(chkExp) - require.NoError(t, err) - require.Nil(t, iterable) - require.Equal(t, chkExp.Chunk.Bytes(), chkAct.Bytes()) - } - } - }) - } -} - -func TestRangeForTimestamp(t *testing.T) { - type args struct { - t int64 - width int64 - } - tests := []struct { - args args - expected int64 - }{ - {args{0, 5}, 5}, - {args{1, 5}, 5}, - {args{5, 5}, 10}, - {args{6, 5}, 10}, - {args{13, 5}, 15}, - {args{95, 5}, 100}, - } - for _, tt := range tests { - got := rangeForTimestamp(tt.args.t, tt.args.width) - require.Equal(t, tt.expected, got) - } -} - -// TestChunkReader_ConcurrentReads checks that the chunk result can be read concurrently. -// Regression test for https://github.com/prometheus/prometheus/pull/6514. -func TestChunkReader_ConcurrentReads(t *testing.T) { - t.Parallel() - chks := []chunks.Meta{ - assureChunkFromSamples(t, []chunks.Sample{sample{1, 1, nil, nil}}), - assureChunkFromSamples(t, []chunks.Sample{sample{1, 2, nil, nil}}), - assureChunkFromSamples(t, []chunks.Sample{sample{1, 3, nil, nil}}), - assureChunkFromSamples(t, []chunks.Sample{sample{1, 4, nil, nil}}), - assureChunkFromSamples(t, []chunks.Sample{sample{1, 5, nil, nil}}), - } - - tempDir := t.TempDir() - - chunkw, err := chunks.NewWriter(tempDir) - require.NoError(t, err) - - require.NoError(t, chunkw.WriteChunks(chks...)) - require.NoError(t, chunkw.Close()) - - r, err := chunks.NewDirReader(tempDir, nil) - require.NoError(t, err) - - var wg sync.WaitGroup - for _, chk := range chks { - for range 100 { - wg.Add(1) - go func(chunk chunks.Meta) { - defer wg.Done() - - chkAct, iterable, err := r.ChunkOrIterable(chunk) - require.NoError(t, err) - require.Nil(t, iterable) - require.Equal(t, chunk.Chunk.Bytes(), chkAct.Bytes()) - }(chk) - } - wg.Wait() - } - require.NoError(t, r.Close()) -} - // TestCompactHead ensures that the head compaction // creates a block that is ready for loading and // does not cause data loss. @@ -3113,7 +2290,7 @@ func TestChunkReader_ConcurrentReads(t *testing.T) { // * appends values; // * compacts the head; and // * queries the db to ensure the samples are present from the compacted head. -func TestCompactHead(t *testing.T) { +func TestCompactHead_AppendV2(t *testing.T) { t.Parallel() // Open a DB and append data to the WAL. @@ -3126,12 +2303,12 @@ func TestCompactHead(t *testing.T) { } db := newTestDB(t, withOpts(opts)) ctx := context.Background() - app := db.Appender(ctx) + app := db.AppenderV2(ctx) var expSamples []sample maxt := 100 for i := range maxt { val := rand.Float64() - _, err := app.Append(0, labels.FromStrings("a", "b"), int64(i), val) + _, err := app.Append(0, labels.FromStrings("a", "b"), 0, int64(i), val, nil, nil, storage.AOptions{}) require.NoError(t, err) expSamples = append(expSamples, sample{int64(i), val, nil, nil}) } @@ -3169,13 +2346,13 @@ func TestCompactHead(t *testing.T) { } // TestCompactHeadWithDeletion tests https://github.com/prometheus/prometheus/issues/11585. -func TestCompactHeadWithDeletion(t *testing.T) { +func TestCompactHeadWithDeletion_AppendV2(t *testing.T) { db := newTestDB(t) ctx := context.Background() - app := db.Appender(ctx) - _, err := app.Append(0, labels.FromStrings("a", "b"), 10, rand.Float64()) + app := db.AppenderV2(ctx) + _, err := app.Append(0, labels.FromStrings("a", "b"), 0, 10, rand.Float64(), nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) @@ -3184,146 +2361,10 @@ func TestCompactHeadWithDeletion(t *testing.T) { // This recreates the bug. require.NoError(t, db.CompactHead(NewRangeHead(db.Head(), 0, 100))) -} - -func deleteNonBlocks(dbDir string) error { - dirs, err := os.ReadDir(dbDir) - if err != nil { - return err - } - for _, dir := range dirs { - if ok := isBlockDir(dir); !ok { - if err := os.RemoveAll(filepath.Join(dbDir, dir.Name())); err != nil { - return err - } - } - } - dirs, err = os.ReadDir(dbDir) - if err != nil { - return err - } - for _, dir := range dirs { - if ok := isBlockDir(dir); !ok { - return fmt.Errorf("root folder:%v still hase non block directory:%v", dbDir, dir.Name()) - } - } - return nil -} - -func TestOpen_VariousBlockStates(t *testing.T) { - tmpDir := t.TempDir() - - var ( - expectedLoadedDirs = map[string]struct{}{} - expectedRemovedDirs = map[string]struct{}{} - expectedIgnoredDirs = map[string]struct{}{} - ) - - { - // Ok blocks; should be loaded. - expectedLoadedDirs[createBlock(t, tmpDir, genSeries(10, 2, 0, 10))] = struct{}{} - expectedLoadedDirs[createBlock(t, tmpDir, genSeries(10, 2, 10, 20))] = struct{}{} - } - { - // Block to repair; should be repaired & loaded. - dbDir := filepath.Join("testdata", "repair_index_version", "01BZJ9WJQPWHGNC2W4J9TA62KC") - outDir := filepath.Join(tmpDir, "01BZJ9WJQPWHGNC2W4J9TA62KC") - expectedLoadedDirs[outDir] = struct{}{} - - // Touch chunks dir in block. - require.NoError(t, os.MkdirAll(filepath.Join(dbDir, "chunks"), 0o777)) - defer func() { - require.NoError(t, os.RemoveAll(filepath.Join(dbDir, "chunks"))) - }() - require.NoError(t, os.Mkdir(outDir, os.ModePerm)) - require.NoError(t, fileutil.CopyDirs(dbDir, outDir)) - } - { - // Missing meta.json; should be ignored and only logged. - // TODO(bwplotka): Probably add metric. - dir := createBlock(t, tmpDir, genSeries(10, 2, 20, 30)) - expectedIgnoredDirs[dir] = struct{}{} - require.NoError(t, os.Remove(filepath.Join(dir, metaFilename))) - } - { - // Tmp blocks during creation; those should be removed on start. - dir := createBlock(t, tmpDir, genSeries(10, 2, 30, 40)) - require.NoError(t, fileutil.Replace(dir, dir+tmpForCreationBlockDirSuffix)) - expectedRemovedDirs[dir+tmpForCreationBlockDirSuffix] = struct{}{} - - // Tmp blocks during deletion; those should be removed on start. - dir = createBlock(t, tmpDir, genSeries(10, 2, 40, 50)) - require.NoError(t, fileutil.Replace(dir, dir+tmpForDeletionBlockDirSuffix)) - expectedRemovedDirs[dir+tmpForDeletionBlockDirSuffix] = struct{}{} - - // Pre-2.21 tmp blocks; those should be removed on start. - dir = createBlock(t, tmpDir, genSeries(10, 2, 50, 60)) - require.NoError(t, fileutil.Replace(dir, dir+tmpLegacy)) - expectedRemovedDirs[dir+tmpLegacy] = struct{}{} - } - { - // One ok block; but two should be replaced. - dir := createBlock(t, tmpDir, genSeries(10, 2, 50, 60)) - expectedLoadedDirs[dir] = struct{}{} - - m, _, err := readMetaFile(dir) - require.NoError(t, err) - - compacted := createBlock(t, tmpDir, genSeries(10, 2, 50, 55)) - expectedRemovedDirs[compacted] = struct{}{} - - m.Compaction.Parents = append(m.Compaction.Parents, - BlockDesc{ULID: ulid.MustParse(filepath.Base(compacted))}, - BlockDesc{ULID: ulid.MustNew(1, nil)}, - BlockDesc{ULID: ulid.MustNew(123, nil)}, - ) - - // Regression test: Already removed parent can be still in list, which was causing Open errors. - m.Compaction.Parents = append(m.Compaction.Parents, BlockDesc{ULID: ulid.MustParse(filepath.Base(compacted))}) - m.Compaction.Parents = append(m.Compaction.Parents, BlockDesc{ULID: ulid.MustParse(filepath.Base(compacted))}) - _, err = writeMetaFile(promslog.New(&promslog.Config{}), dir, m) - require.NoError(t, err) - } - tmpCheckpointDir := path.Join(tmpDir, "wal/checkpoint.00000001.tmp") - err := os.MkdirAll(tmpCheckpointDir, 0o777) - require.NoError(t, err) - tmpChunkSnapshotDir := path.Join(tmpDir, chunkSnapshotPrefix+"0000.00000001.tmp") - err = os.MkdirAll(tmpChunkSnapshotDir, 0o777) - require.NoError(t, err) - - opts := DefaultOptions() - opts.RetentionDuration = 0 - db := newTestDB(t, withDir(tmpDir), withOpts(opts)) - loadedBlocks := db.Blocks() - - var loaded int - for _, l := range loadedBlocks { - _, ok := expectedLoadedDirs[filepath.Join(tmpDir, l.meta.ULID.String())] - require.True(t, ok, "unexpected block", l.meta.ULID, "was loaded") - loaded++ - } - require.Len(t, expectedLoadedDirs, loaded) require.NoError(t, db.Close()) - - files, err := os.ReadDir(tmpDir) - require.NoError(t, err) - - var ignored int - for _, f := range files { - _, ok := expectedRemovedDirs[filepath.Join(tmpDir, f.Name())] - require.False(t, ok, "expected", filepath.Join(tmpDir, f.Name()), "to be removed, but still exists") - if _, ok := expectedIgnoredDirs[filepath.Join(tmpDir, f.Name())]; ok { - ignored++ - } - } - require.Len(t, expectedIgnoredDirs, ignored) - _, err = os.Stat(tmpCheckpointDir) - require.True(t, os.IsNotExist(err)) - _, err = os.Stat(tmpChunkSnapshotDir) - require.True(t, os.IsNotExist(err)) } -func TestOneCheckpointPerCompactCall(t *testing.T) { +func TestOneCheckpointPerCompactCall_AppendV2(t *testing.T) { t.Parallel() blockRange := int64(1000) opts := &Options{ @@ -3342,11 +2383,11 @@ func TestOneCheckpointPerCompactCall(t *testing.T) { lbls := labels.FromStrings("foo_d", "choco_bar") // Append samples spanning 59 block ranges. - app := db.Appender(context.Background()) + app := db.AppenderV2(context.Background()) for i := range int64(60) { - _, err := app.Append(0, lbls, blockRange*i, rand.Float64()) + _, err := app.Append(0, lbls, 0, blockRange*i, rand.Float64(), nil, nil, storage.AOptions{}) require.NoError(t, err) - _, err = app.Append(0, lbls, (blockRange*i)+blockRange/2, rand.Float64()) + _, err = app.Append(0, lbls, 0, (blockRange*i)+blockRange/2, rand.Float64(), nil, nil, storage.AOptions{}) require.NoError(t, err) // Rotate the WAL file so that there is >3 files for checkpoint to happen. _, err = db.head.wal.NextSegment() @@ -3402,8 +2443,8 @@ func TestOneCheckpointPerCompactCall(t *testing.T) { require.Equal(t, 0, int(db.head.NumSeries())) // Adding sample way into the future. - app = db.Appender(context.Background()) - _, err = app.Append(0, lbls, blockRange*120, rand.Float64()) + app = db.AppenderV2(context.Background()) + _, err = app.Append(0, lbls, 0, blockRange*120, rand.Float64(), nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) @@ -3437,288 +2478,7 @@ func TestOneCheckpointPerCompactCall(t *testing.T) { require.Equal(t, 54, cno) } -func TestNoPanicOnTSDBOpenError(t *testing.T) { - tmpdir := t.TempDir() - - // Taking the lock will cause a TSDB startup error. - l, err := tsdbutil.NewDirLocker(tmpdir, "tsdb", promslog.NewNopLogger(), nil) - require.NoError(t, err) - require.NoError(t, l.Lock()) - - _, err = Open(tmpdir, nil, nil, DefaultOptions(), nil) - require.Error(t, err) - - require.NoError(t, l.Release()) -} - -func TestLockfile(t *testing.T) { - tsdbutil.TestDirLockerUsage(t, func(t *testing.T, data string, createLock bool) (*tsdbutil.DirLocker, testutil.Closer) { - opts := DefaultOptions() - opts.NoLockfile = !createLock - - // Create the DB. This should create lockfile and its metrics. - db, err := Open(data, nil, nil, opts, nil) - require.NoError(t, err) - - return db.locker, testutil.NewCallbackCloser(func() { - require.NoError(t, db.Close()) - }) - }) -} - -func TestQuerier_ShouldNotPanicIfHeadChunkIsTruncatedWhileReadingQueriedChunks(t *testing.T) { - t.Skip("TODO: investigate why process crash in CI") - - const numRuns = 5 - - for i := 1; i <= numRuns; i++ { - t.Run(strconv.Itoa(i), func(t *testing.T) { - testQuerierShouldNotPanicIfHeadChunkIsTruncatedWhileReadingQueriedChunks(t) - }) - } -} - -func testQuerierShouldNotPanicIfHeadChunkIsTruncatedWhileReadingQueriedChunks(t *testing.T) { - const ( - numSeries = 1000 - numStressIterations = 10000 - minStressAllocationBytes = 128 * 1024 - maxStressAllocationBytes = 512 * 1024 - ) - - db := newTestDB(t) - - // Disable compactions so we can control it. - db.DisableCompactions() - - // Generate the metrics we're going to append. - metrics := make([]labels.Labels, 0, numSeries) - for i := range numSeries { - metrics = append(metrics, labels.FromStrings(labels.MetricName, fmt.Sprintf("test_%d", i))) - } - - // Push 1 sample every 15s for 2x the block duration period. - ctx := context.Background() - interval := int64(15 * time.Second / time.Millisecond) - ts := int64(0) - - for ; ts < 2*DefaultBlockDuration; ts += interval { - app := db.Appender(ctx) - - for _, metric := range metrics { - _, err := app.Append(0, metric, ts, float64(ts)) - require.NoError(t, err) - } - - require.NoError(t, app.Commit()) - } - - // Compact the TSDB head for the first time. We expect the head chunks file has been cut. - require.NoError(t, db.Compact(ctx)) - require.Equal(t, float64(1), prom_testutil.ToFloat64(db.Head().metrics.headTruncateTotal)) - - // Push more samples for another 1x block duration period. - for ; ts < 3*DefaultBlockDuration; ts += interval { - app := db.Appender(ctx) - - for _, metric := range metrics { - _, err := app.Append(0, metric, ts, float64(ts)) - require.NoError(t, err) - } - - require.NoError(t, app.Commit()) - } - - // At this point we expect 2 mmap-ed head chunks. - - // Get a querier and make sure it's closed only once the test is over. - querier, err := db.Querier(0, math.MaxInt64) - require.NoError(t, err) - defer func() { - require.NoError(t, querier.Close()) - }() - - // Query back all series. - hints := &storage.SelectHints{Start: 0, End: math.MaxInt64, Step: interval} - seriesSet := querier.Select(ctx, true, hints, labels.MustNewMatcher(labels.MatchRegexp, labels.MetricName, ".+")) - - // Fetch samples iterators from all series. - var iterators []chunkenc.Iterator - actualSeries := 0 - for seriesSet.Next() { - actualSeries++ - - // Get the iterator and call Next() so that we're sure the chunk is loaded. - it := seriesSet.At().Iterator(nil) - it.Next() - it.At() - - iterators = append(iterators, it) - } - require.NoError(t, seriesSet.Err()) - require.Equal(t, numSeries, actualSeries) - - // Compact the TSDB head again. - require.NoError(t, db.Compact(ctx)) - require.Equal(t, float64(2), prom_testutil.ToFloat64(db.Head().metrics.headTruncateTotal)) - - // At this point we expect 1 head chunk has been deleted. - - // Stress the memory and call GC. This is required to increase the chances - // the chunk memory area is released to the kernel. - var buf []byte - for i := range numStressIterations { - //nolint:staticcheck - buf = append(buf, make([]byte, minStressAllocationBytes+rand.Int31n(maxStressAllocationBytes-minStressAllocationBytes))...) - if i%1000 == 0 { - buf = nil - } - } - - // Iterate samples. Here we're summing it just to make sure no golang compiler - // optimization triggers in case we discard the result of it.At(). - var sum float64 - var firstErr error - for _, it := range iterators { - for it.Next() == chunkenc.ValFloat { - _, v := it.At() - sum += v - } - - if err := it.Err(); err != nil { - firstErr = err - } - } - - // After having iterated all samples we also want to be sure no error occurred or - // the "cannot populate chunk XXX: not found" error occurred. This error can occur - // when the iterator tries to fetch an head chunk which has been offloaded because - // of the head compaction in the meanwhile. - if firstErr != nil { - require.ErrorContains(t, firstErr, "cannot populate chunk") - } -} - -func TestChunkQuerier_ShouldNotPanicIfHeadChunkIsTruncatedWhileReadingQueriedChunks(t *testing.T) { - t.Skip("TODO: investigate why process crash in CI") - - const numRuns = 5 - - for i := 1; i <= numRuns; i++ { - t.Run(strconv.Itoa(i), func(t *testing.T) { - testChunkQuerierShouldNotPanicIfHeadChunkIsTruncatedWhileReadingQueriedChunks(t) - }) - } -} - -func testChunkQuerierShouldNotPanicIfHeadChunkIsTruncatedWhileReadingQueriedChunks(t *testing.T) { - const ( - numSeries = 1000 - numStressIterations = 10000 - minStressAllocationBytes = 128 * 1024 - maxStressAllocationBytes = 512 * 1024 - ) - - db := newTestDB(t) - - // Disable compactions so we can control it. - db.DisableCompactions() - - // Generate the metrics we're going to append. - metrics := make([]labels.Labels, 0, numSeries) - for i := range numSeries { - metrics = append(metrics, labels.FromStrings(labels.MetricName, fmt.Sprintf("test_%d", i))) - } - - // Push 1 sample every 15s for 2x the block duration period. - ctx := context.Background() - interval := int64(15 * time.Second / time.Millisecond) - ts := int64(0) - - for ; ts < 2*DefaultBlockDuration; ts += interval { - app := db.Appender(ctx) - - for _, metric := range metrics { - _, err := app.Append(0, metric, ts, float64(ts)) - require.NoError(t, err) - } - - require.NoError(t, app.Commit()) - } - - // Compact the TSDB head for the first time. We expect the head chunks file has been cut. - require.NoError(t, db.Compact(ctx)) - require.Equal(t, float64(1), prom_testutil.ToFloat64(db.Head().metrics.headTruncateTotal)) - - // Push more samples for another 1x block duration period. - for ; ts < 3*DefaultBlockDuration; ts += interval { - app := db.Appender(ctx) - - for _, metric := range metrics { - _, err := app.Append(0, metric, ts, float64(ts)) - require.NoError(t, err) - } - - require.NoError(t, app.Commit()) - } - - // At this point we expect 2 mmap-ed head chunks. - - // Get a querier and make sure it's closed only once the test is over. - querier, err := db.ChunkQuerier(0, math.MaxInt64) - require.NoError(t, err) - defer func() { - require.NoError(t, querier.Close()) - }() - - // Query back all series. - hints := &storage.SelectHints{Start: 0, End: math.MaxInt64, Step: interval} - seriesSet := querier.Select(ctx, true, hints, labels.MustNewMatcher(labels.MatchRegexp, labels.MetricName, ".+")) - - // Iterate all series and get their chunks. - var it chunks.Iterator - var chunks []chunkenc.Chunk - actualSeries := 0 - for seriesSet.Next() { - actualSeries++ - it = seriesSet.At().Iterator(it) - for it.Next() { - chunks = append(chunks, it.At().Chunk) - } - } - require.NoError(t, seriesSet.Err()) - require.Equal(t, numSeries, actualSeries) - - // Compact the TSDB head again. - require.NoError(t, db.Compact(ctx)) - require.Equal(t, float64(2), prom_testutil.ToFloat64(db.Head().metrics.headTruncateTotal)) - - // At this point we expect 1 head chunk has been deleted. - - // Stress the memory and call GC. This is required to increase the chances - // the chunk memory area is released to the kernel. - var buf []byte - for i := range numStressIterations { - //nolint:staticcheck - buf = append(buf, make([]byte, minStressAllocationBytes+rand.Int31n(maxStressAllocationBytes-minStressAllocationBytes))...) - if i%1000 == 0 { - buf = nil - } - } - - // Iterate chunks and read their bytes slice. Here we're computing the CRC32 - // just to iterate through the bytes slice. We don't really care the reason why - // we read this data, we just need to read it to make sure the memory address - // of the []byte is still valid. - chkCRC32 := crc32.New(crc32.MakeTable(crc32.Castagnoli)) - for _, chunk := range chunks { - chkCRC32.Reset() - _, err := chkCRC32.Write(chunk.Bytes()) - require.NoError(t, err) - } -} - -func TestQuerierShouldNotFailIfOOOCompactionOccursAfterRetrievingQuerier(t *testing.T) { +func TestQuerierShouldNotFailIfOOOCompactionOccursAfterRetrievingQuerier_AppendV2(t *testing.T) { opts := DefaultOptions() opts.OutOfOrderTimeWindow = 3 * DefaultBlockDuration db := newTestDB(t, withOpts(opts)) @@ -3738,16 +2498,16 @@ func TestQuerierShouldNotFailIfOOOCompactionOccursAfterRetrievingQuerier(t *test // Push samples after the OOO sample we'll write below. for ; ts < 10*interval; ts += interval { - app := db.Appender(ctx) - _, err := app.Append(0, metric, ts, float64(ts)) + app := db.AppenderV2(ctx) + _, err := app.Append(0, metric, 0, ts, float64(ts), nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) samplesWritten++ } // Push a single OOO sample. - app := db.Appender(ctx) - _, err := app.Append(0, metric, oooTS, float64(ts)) + app := db.AppenderV2(ctx) + _, err := app.Append(0, metric, 0, oooTS, float64(ts), nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) samplesWritten++ @@ -3809,7 +2569,7 @@ func TestQuerierShouldNotFailIfOOOCompactionOccursAfterRetrievingQuerier(t *test require.NoError(t, querierCreatedAfterCompaction.Close()) } -func TestQuerierShouldNotFailIfOOOCompactionOccursAfterSelecting(t *testing.T) { +func TestQuerierShouldNotFailIfOOOCompactionOccursAfterSelecting_AppendV2(t *testing.T) { opts := DefaultOptions() opts.OutOfOrderTimeWindow = 3 * DefaultBlockDuration db := newTestDB(t, withOpts(opts)) @@ -3829,16 +2589,16 @@ func TestQuerierShouldNotFailIfOOOCompactionOccursAfterSelecting(t *testing.T) { // Push samples after the OOO sample we'll write below. for ; ts < 10*interval; ts += interval { - app := db.Appender(ctx) - _, err := app.Append(0, metric, ts, float64(ts)) + app := db.AppenderV2(ctx) + _, err := app.Append(0, metric, 0, ts, float64(ts), nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) samplesWritten++ } // Push a single OOO sample. - app := db.Appender(ctx) - _, err := app.Append(0, metric, oooTS, float64(ts)) + app := db.AppenderV2(ctx) + _, err := app.Append(0, metric, 0, oooTS, float64(ts), nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) samplesWritten++ @@ -3888,7 +2648,7 @@ func TestQuerierShouldNotFailIfOOOCompactionOccursAfterSelecting(t *testing.T) { require.Eventually(t, compactionComplete.Load, time.Second, 10*time.Millisecond, "compaction should complete after querier was closed") } -func TestQuerierShouldNotFailIfOOOCompactionOccursAfterRetrievingIterators(t *testing.T) { +func TestQuerierShouldNotFailIfOOOCompactionOccursAfterRetrievingIterators_AppendV2(t *testing.T) { opts := DefaultOptions() opts.OutOfOrderTimeWindow = 3 * DefaultBlockDuration db := newTestDB(t, withOpts(opts)) @@ -3908,16 +2668,16 @@ func TestQuerierShouldNotFailIfOOOCompactionOccursAfterRetrievingIterators(t *te // Push samples after the OOO sample we'll write below. for ; ts < 10*interval; ts += interval { - app := db.Appender(ctx) - _, err := app.Append(0, metric, ts, float64(ts)) + app := db.AppenderV2(ctx) + _, err := app.Append(0, metric, 0, ts, float64(ts), nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) samplesWritten++ } // Push a single OOO sample. - app := db.Appender(ctx) - _, err := app.Append(0, metric, oooTS, float64(ts)) + app := db.AppenderV2(ctx) + _, err := app.Append(0, metric, 0, oooTS, float64(ts), nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) samplesWritten++ @@ -3967,7 +2727,7 @@ func TestQuerierShouldNotFailIfOOOCompactionOccursAfterRetrievingIterators(t *te require.Eventually(t, compactionComplete.Load, time.Second, 10*time.Millisecond, "compaction should complete after querier was closed") } -func TestOOOWALWrite(t *testing.T) { +func TestOOOWALWrite_AppendV2(t *testing.T) { minutes := func(m int64) int64 { return m * time.Minute.Milliseconds() } s := labels.NewSymbolTable() @@ -3979,13 +2739,13 @@ func TestOOOWALWrite(t *testing.T) { s2 := scratchBuilder2.Labels() scenarios := map[string]struct { - appendSample func(app storage.Appender, l labels.Labels, mins int64) (storage.SeriesRef, error) + appendSample func(app storage.AppenderV2, l labels.Labels, mins int64) (storage.SeriesRef, error) expectedOOORecords []any expectedInORecords []any }{ "float": { - appendSample: func(app storage.Appender, l labels.Labels, mins int64) (storage.SeriesRef, error) { - seriesRef, err := app.Append(0, l, minutes(mins), float64(mins)) + appendSample: func(app storage.AppenderV2, l labels.Labels, mins int64) (storage.SeriesRef, error) { + seriesRef, err := app.Append(0, l, 0, minutes(mins), float64(mins), nil, nil, storage.AOptions{}) require.NoError(t, err) return seriesRef, nil }, @@ -4075,8 +2835,8 @@ func TestOOOWALWrite(t *testing.T) { }, }, "integer histogram": { - appendSample: func(app storage.Appender, l labels.Labels, mins int64) (storage.SeriesRef, error) { - seriesRef, err := app.AppendHistogram(0, l, minutes(mins), tsdbutil.GenerateTestHistogram(mins), nil) + appendSample: func(app storage.AppenderV2, l labels.Labels, mins int64) (storage.SeriesRef, error) { + seriesRef, err := app.Append(0, l, 0, minutes(mins), 0, tsdbutil.GenerateTestHistogram(mins), nil, storage.AOptions{}) require.NoError(t, err) return seriesRef, nil }, @@ -4166,8 +2926,8 @@ func TestOOOWALWrite(t *testing.T) { }, }, "float histogram": { - appendSample: func(app storage.Appender, l labels.Labels, mins int64) (storage.SeriesRef, error) { - seriesRef, err := app.AppendHistogram(0, l, minutes(mins), nil, tsdbutil.GenerateTestFloatHistogram(mins)) + appendSample: func(app storage.AppenderV2, l labels.Labels, mins int64) (storage.SeriesRef, error) { + seriesRef, err := app.Append(0, l, 0, minutes(mins), 0, nil, tsdbutil.GenerateTestFloatHistogram(mins), storage.AOptions{}) require.NoError(t, err) return seriesRef, nil }, @@ -4257,8 +3017,8 @@ func TestOOOWALWrite(t *testing.T) { }, }, "custom buckets histogram": { - appendSample: func(app storage.Appender, l labels.Labels, mins int64) (storage.SeriesRef, error) { - seriesRef, err := app.AppendHistogram(0, l, minutes(mins), tsdbutil.GenerateTestCustomBucketsHistogram(mins), nil) + appendSample: func(app storage.AppenderV2, l labels.Labels, mins int64) (storage.SeriesRef, error) { + seriesRef, err := app.Append(0, l, 0, minutes(mins), 0, tsdbutil.GenerateTestCustomBucketsHistogram(mins), nil, storage.AOptions{}) require.NoError(t, err) return seriesRef, nil }, @@ -4348,8 +3108,8 @@ func TestOOOWALWrite(t *testing.T) { }, }, "custom buckets float histogram": { - appendSample: func(app storage.Appender, l labels.Labels, mins int64) (storage.SeriesRef, error) { - seriesRef, err := app.AppendHistogram(0, l, minutes(mins), nil, tsdbutil.GenerateTestCustomBucketsFloatHistogram(mins)) + appendSample: func(app storage.AppenderV2, l labels.Labels, mins int64) (storage.SeriesRef, error) { + seriesRef, err := app.Append(0, l, 0, minutes(mins), 0, nil, tsdbutil.GenerateTestCustomBucketsFloatHistogram(mins), storage.AOptions{}) require.NoError(t, err) return seriesRef, nil }, @@ -4441,13 +3201,13 @@ func TestOOOWALWrite(t *testing.T) { } for name, scenario := range scenarios { t.Run(name, func(t *testing.T) { - testOOOWALWrite(t, scenario.appendSample, scenario.expectedOOORecords, scenario.expectedInORecords) + testOOOWALWriteAppendV2(t, scenario.appendSample, scenario.expectedOOORecords, scenario.expectedInORecords) }) } } -func testOOOWALWrite(t *testing.T, - appendSample func(app storage.Appender, l labels.Labels, mins int64) (storage.SeriesRef, error), +func testOOOWALWriteAppendV2(t *testing.T, + appendSample func(app storage.AppenderV2, l labels.Labels, mins int64) (storage.SeriesRef, error), expectedOOORecords []any, expectedInORecords []any, ) { @@ -4459,23 +3219,23 @@ func testOOOWALWrite(t *testing.T, s1, s2 := labels.FromStrings("l", "v1"), labels.FromStrings("l", "v2") // Ingest sample at 1h. - app := db.Appender(context.Background()) + app := db.AppenderV2(context.Background()) appendSample(app, s1, 60) appendSample(app, s2, 60) require.NoError(t, app.Commit()) // OOO for s1. - app = db.Appender(context.Background()) + app = db.AppenderV2(context.Background()) appendSample(app, s1, 40) require.NoError(t, app.Commit()) // OOO for s2. - app = db.Appender(context.Background()) + app = db.AppenderV2(context.Background()) appendSample(app, s2, 42) require.NoError(t, app.Commit()) // OOO for both s1 and s2 in the same commit. - app = db.Appender(context.Background()) + app = db.AppenderV2(context.Background()) appendSample(app, s2, 45) appendSample(app, s1, 35) appendSample(app, s1, 36) // m-maps. @@ -4483,13 +3243,13 @@ func testOOOWALWrite(t *testing.T, require.NoError(t, app.Commit()) // OOO for s1 but not for s2 in the same commit. - app = db.Appender(context.Background()) + app = db.AppenderV2(context.Background()) appendSample(app, s1, 50) // m-maps. appendSample(app, s2, 65) require.NoError(t, app.Commit()) // Single commit has 2 times m-mapping and more samples after m-map. - app = db.Appender(context.Background()) + app = db.AppenderV2(context.Background()) appendSample(app, s2, 50) // m-maps. appendSample(app, s2, 51) appendSample(app, s2, 52) // m-maps. @@ -4547,7 +3307,7 @@ func testOOOWALWrite(t *testing.T, } // Tests https://github.com/prometheus/prometheus/issues/10291#issuecomment-1044373110. -func TestDBPanicOnMmappingHeadChunk(t *testing.T) { +func TestDBPanicOnMmappingHeadChunk_AppendV2(t *testing.T) { var err error ctx := context.Background() @@ -4559,16 +3319,16 @@ func TestDBPanicOnMmappingHeadChunk(t *testing.T) { lastTs := int64(0) addSamples := func(numSamples int) { - app := db.Appender(context.Background()) + app := db.AppenderV2(context.Background()) var ref storage.SeriesRef lbls := labels.FromStrings("__name__", "testing", "foo", "bar") 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{}) require.NoError(t, err) lastTs += itvl if i%10 == 0 { require.NoError(t, app.Commit()) - app = db.Appender(context.Background()) + app = db.AppenderV2(context.Background()) } } require.NoError(t, app.Commit()) @@ -4603,37 +3363,34 @@ func TestDBPanicOnMmappingHeadChunk(t *testing.T) { require.NoError(t, db.Close()) } -func TestMetadataInWAL(t *testing.T) { - updateMetadata := func(t *testing.T, app storage.Appender, s labels.Labels, m metadata.Metadata) { - _, err := app.UpdateMetadata(0, s, m) - require.NoError(t, err) - } - - db := newTestDB(t) +// TODO(bwplotka): Add cases ensuring stale sample appends will skipp metadata persisting. +func TestMetadataInWAL_AppenderV2(t *testing.T) { + opts := DefaultOptions() + opts.EnableMetadataWALRecords = true + db := newTestDB(t, withOpts(opts)) ctx := context.Background() - // Add some series so we can append metadata to them. - app := db.Appender(ctx) + // Add some series so we can attach metadata to them. s1 := labels.FromStrings("a", "b") s2 := labels.FromStrings("c", "d") s3 := labels.FromStrings("e", "f") s4 := labels.FromStrings("g", "h") - for _, s := range []labels.Labels{s1, s2, s3, s4} { - _, err := app.Append(0, s, 0, 0) - require.NoError(t, err) - } - require.NoError(t, app.Commit()) - // Add a first round of metadata to the first three series. - // Re-take the Appender, as the previous Commit will have it closed. m1 := metadata.Metadata{Type: "gauge", Unit: "unit_1", Help: "help_1"} m2 := metadata.Metadata{Type: "gauge", Unit: "unit_2", Help: "help_2"} m3 := metadata.Metadata{Type: "gauge", Unit: "unit_3", Help: "help_3"} - app = db.Appender(ctx) - updateMetadata(t, app, s1, m1) - updateMetadata(t, app, s2, m2) - updateMetadata(t, app, s3, m3) + + app := db.AppenderV2(ctx) + ts := int64(0) + _, err := app.Append(0, s1, 0, ts, 0, nil, nil, storage.AOptions{Metadata: m1}) + require.NoError(t, err) + _, err = app.Append(0, s2, 0, ts, 0, nil, nil, storage.AOptions{Metadata: m2}) + require.NoError(t, err) + _, err = app.Append(0, s3, 0, ts, 0, nil, nil, storage.AOptions{Metadata: m3}) + require.NoError(t, err) + _, err = app.Append(0, s4, 0, ts, 0, nil, nil, storage.AOptions{}) + require.NoError(t, err) require.NoError(t, app.Commit()) // Add a replicated metadata entry to the first series, @@ -4641,10 +3398,14 @@ func TestMetadataInWAL(t *testing.T) { // and a changed metadata entry to the second series. m4 := metadata.Metadata{Type: "counter", Unit: "unit_4", Help: "help_4"} m5 := metadata.Metadata{Type: "counter", Unit: "unit_5", Help: "help_5"} - app = db.Appender(ctx) - updateMetadata(t, app, s1, m1) - updateMetadata(t, app, s4, m4) - updateMetadata(t, app, s2, m5) + app = db.AppenderV2(ctx) + ts++ + _, err = app.Append(0, s1, 0, ts, 0, nil, nil, storage.AOptions{Metadata: m1}) + require.NoError(t, err) + _, err = app.Append(0, s4, 0, ts, 0, nil, nil, storage.AOptions{Metadata: m4}) + require.NoError(t, err) + _, err = app.Append(0, s2, 0, ts, 0, nil, nil, storage.AOptions{Metadata: m5}) + require.NoError(t, err) require.NoError(t, app.Commit()) // Read the WAL to see if the disk storage format is correct. @@ -4668,70 +3429,75 @@ func TestMetadataInWAL(t *testing.T) { require.Equal(t, expectedMetadata[3:], gotMetadataBlocks[1]) } -func TestMetadataCheckpointingOnlyKeepsLatestEntry(t *testing.T) { - updateMetadata := func(t *testing.T, app storage.Appender, s labels.Labels, m metadata.Metadata) { - _, err := app.UpdateMetadata(0, s, m) - require.NoError(t, err) - } - +func TestMetadataCheckpointingOnlyKeepsLatestEntry_AppendV2(t *testing.T) { ctx := context.Background() numSamples := 10000 hb, w := newTestHead(t, int64(numSamples)*10, compression.None, false) + hb.opts.EnableMetadataWALRecords = true // Add some series so we can append metadata to them. - app := hb.Appender(ctx) s1 := labels.FromStrings("a", "b") s2 := labels.FromStrings("c", "d") s3 := labels.FromStrings("e", "f") s4 := labels.FromStrings("g", "h") - for _, s := range []labels.Labels{s1, s2, s3, s4} { - _, err := app.Append(0, s, 0, 0) - require.NoError(t, err) - } - require.NoError(t, app.Commit()) - - // Add a first round of metadata to the first three series. - // Re-take the Appender, as the previous Commit will have it closed. m1 := metadata.Metadata{Type: "gauge", Unit: "unit_1", Help: "help_1"} m2 := metadata.Metadata{Type: "gauge", Unit: "unit_2", Help: "help_2"} m3 := metadata.Metadata{Type: "gauge", Unit: "unit_3", Help: "help_3"} m4 := metadata.Metadata{Type: "gauge", Unit: "unit_4", Help: "help_4"} - app = hb.Appender(ctx) - updateMetadata(t, app, s1, m1) - updateMetadata(t, app, s2, m2) - updateMetadata(t, app, s3, m3) - updateMetadata(t, app, s4, m4) + + app := hb.AppenderV2(ctx) + ts := int64(0) + _, err := app.Append(0, s1, 0, ts, 0, nil, nil, storage.AOptions{Metadata: m1}) + require.NoError(t, err) + _, err = app.Append(0, s2, 0, ts, 0, nil, nil, storage.AOptions{Metadata: m2}) + require.NoError(t, err) + _, err = app.Append(0, s3, 0, ts, 0, nil, nil, storage.AOptions{Metadata: m3}) + require.NoError(t, err) + _, err = app.Append(0, s4, 0, ts, 0, nil, nil, storage.AOptions{Metadata: m4}) + require.NoError(t, err) require.NoError(t, app.Commit()) // Update metadata for first series. m5 := metadata.Metadata{Type: "counter", Unit: "unit_5", Help: "help_5"} - app = hb.Appender(ctx) - updateMetadata(t, app, s1, m5) + app = hb.AppenderV2(ctx) + ts++ + _, err = app.Append(0, s1, 0, ts, 0, nil, nil, storage.AOptions{Metadata: m5}) + require.NoError(t, err) require.NoError(t, app.Commit()) // Switch back-and-forth metadata for second series. // Since it ended on a new metadata record, we expect a single new entry. m6 := metadata.Metadata{Type: "counter", Unit: "unit_6", Help: "help_6"} - app = hb.Appender(ctx) - updateMetadata(t, app, s2, m6) + app = hb.AppenderV2(ctx) + ts++ + _, err = app.Append(0, s2, 0, ts, 0, nil, nil, storage.AOptions{Metadata: m6}) + require.NoError(t, err) require.NoError(t, app.Commit()) - app = hb.Appender(ctx) - updateMetadata(t, app, s2, m2) + app = hb.AppenderV2(ctx) + ts++ + _, err = app.Append(0, s2, 0, ts, 0, nil, nil, storage.AOptions{Metadata: m2}) + require.NoError(t, err) require.NoError(t, app.Commit()) - app = hb.Appender(ctx) - updateMetadata(t, app, s2, m6) + app = hb.AppenderV2(ctx) + ts++ + _, err = app.Append(0, s2, 0, ts, 0, nil, nil, storage.AOptions{Metadata: m6}) + require.NoError(t, err) require.NoError(t, app.Commit()) - app = hb.Appender(ctx) - updateMetadata(t, app, s2, m2) + app = hb.AppenderV2(ctx) + ts++ + _, err = app.Append(0, s2, 0, ts, 0, nil, nil, storage.AOptions{Metadata: m2}) + require.NoError(t, err) require.NoError(t, app.Commit()) - app = hb.Appender(ctx) - updateMetadata(t, app, s2, m6) + app = hb.AppenderV2(ctx) + ts++ + _, err = app.Append(0, s2, 0, ts, 0, nil, nil, storage.AOptions{Metadata: m6}) + require.NoError(t, err) require.NoError(t, app.Commit()) // Let's create a checkpoint. @@ -4772,37 +3538,34 @@ func TestMetadataCheckpointingOnlyKeepsLatestEntry(t *testing.T) { require.NoError(t, hb.Close()) } -func TestMetadataAssertInMemoryData(t *testing.T) { - updateMetadata := func(t *testing.T, app storage.Appender, s labels.Labels, m metadata.Metadata) { - _, err := app.UpdateMetadata(0, s, m) - require.NoError(t, err) - } - - db := newTestDB(t) +func TestMetadataAssertInMemoryData_AppendV2(t *testing.T) { + opts := DefaultOptions() + opts.EnableMetadataWALRecords = true + db := newTestDB(t, withOpts(opts)) ctx := context.Background() // Add some series so we can append metadata to them. - app := db.Appender(ctx) s1 := labels.FromStrings("a", "b") s2 := labels.FromStrings("c", "d") s3 := labels.FromStrings("e", "f") s4 := labels.FromStrings("g", "h") - for _, s := range []labels.Labels{s1, s2, s3, s4} { - _, err := app.Append(0, s, 0, 0) - require.NoError(t, err) - } - require.NoError(t, app.Commit()) - // Add a first round of metadata to the first three series. // The in-memory data held in the db Head should hold the metadata. m1 := metadata.Metadata{Type: "gauge", Unit: "unit_1", Help: "help_1"} m2 := metadata.Metadata{Type: "gauge", Unit: "unit_2", Help: "help_2"} m3 := metadata.Metadata{Type: "gauge", Unit: "unit_3", Help: "help_3"} - app = db.Appender(ctx) - updateMetadata(t, app, s1, m1) - updateMetadata(t, app, s2, m2) - updateMetadata(t, app, s3, m3) + + app := db.AppenderV2(ctx) + ts := int64(0) + _, err := app.Append(0, s1, 0, ts, 0, nil, nil, storage.AOptions{Metadata: m1}) + require.NoError(t, err) + _, err = app.Append(0, s2, 0, ts, 0, nil, nil, storage.AOptions{Metadata: m2}) + require.NoError(t, err) + _, err = app.Append(0, s3, 0, ts, 0, nil, nil, storage.AOptions{Metadata: m3}) + require.NoError(t, err) + _, err = app.Append(0, s4, 0, ts, 0, nil, nil, storage.AOptions{}) + require.NoError(t, err) require.NoError(t, app.Commit()) series1 := db.head.series.getByHash(s1.Hash(), s1) @@ -4820,10 +3583,14 @@ func TestMetadataAssertInMemoryData(t *testing.T) { // The in-memory data held in the db Head should be correctly updated. m4 := metadata.Metadata{Type: "counter", Unit: "unit_4", Help: "help_4"} m5 := metadata.Metadata{Type: "counter", Unit: "unit_5", Help: "help_5"} - app = db.Appender(ctx) - updateMetadata(t, app, s1, m1) - updateMetadata(t, app, s4, m4) - updateMetadata(t, app, s2, m5) + app = db.AppenderV2(ctx) + ts++ + _, err = app.Append(0, s1, 0, ts, 0, nil, nil, storage.AOptions{Metadata: m1}) + require.NoError(t, err) + _, err = app.Append(0, s4, 0, ts, 0, nil, nil, storage.AOptions{Metadata: m4}) + require.NoError(t, err) + _, err = app.Append(0, s2, 0, ts, 0, nil, nil, storage.AOptions{Metadata: m5}) + require.NoError(t, err) require.NoError(t, app.Commit()) series1 = db.head.series.getByHash(s1.Hash(), s1) @@ -4839,19 +3606,24 @@ func TestMetadataAssertInMemoryData(t *testing.T) { // Reopen the DB, replaying the WAL. The Head must have been replayed // correctly in memory. - db = newTestDB(t, withDir(db.Dir())) - _, err := db.head.wal.Size() + reopenDB, err := Open(db.Dir(), nil, nil, nil, nil) + require.NoError(t, err) + t.Cleanup(func() { + require.NoError(t, reopenDB.Close()) + }) + + _, err = reopenDB.head.wal.Size() require.NoError(t, err) - require.Equal(t, *db.head.series.getByHash(s1.Hash(), s1).meta, m1) - require.Equal(t, *db.head.series.getByHash(s2.Hash(), s2).meta, m5) - require.Equal(t, *db.head.series.getByHash(s3.Hash(), s3).meta, m3) - require.Equal(t, *db.head.series.getByHash(s4.Hash(), s4).meta, m4) + require.Equal(t, *reopenDB.head.series.getByHash(s1.Hash(), s1).meta, m1) + require.Equal(t, *reopenDB.head.series.getByHash(s2.Hash(), s2).meta, m5) + require.Equal(t, *reopenDB.head.series.getByHash(s3.Hash(), s3).meta, m3) + require.Equal(t, *reopenDB.head.series.getByHash(s4.Hash(), s4).meta, m4) } // TestMultipleEncodingsCommitOrder mainly serves to demonstrate when happens when committing a batch of samples for the // same series when there are multiple encodings. With issue #15177 fixed, this now all works as expected. -func TestMultipleEncodingsCommitOrder(t *testing.T) { +func TestMultipleEncodingsCommitOrder_AppendV2(t *testing.T) { opts := DefaultOptions() opts.OutOfOrderCapMax = 30 opts.OutOfOrderTimeWindow = 24 * time.Hour.Milliseconds() @@ -4860,20 +3632,20 @@ func TestMultipleEncodingsCommitOrder(t *testing.T) { db.DisableCompactions() series1 := labels.FromStrings("foo", "bar1") - addSample := func(app storage.Appender, ts int64, valType chunkenc.ValueType) chunks.Sample { + addSample := func(app storage.AppenderV2, ts int64, valType chunkenc.ValueType) chunks.Sample { if valType == chunkenc.ValFloat { - _, err := app.Append(0, labels.FromStrings("foo", "bar1"), ts, float64(ts)) + _, err := app.Append(0, labels.FromStrings("foo", "bar1"), 0, ts, float64(ts), nil, nil, storage.AOptions{}) require.NoError(t, err) return sample{t: ts, f: float64(ts)} } if valType == chunkenc.ValHistogram { h := tsdbutil.GenerateTestHistogram(ts) - _, err := app.AppendHistogram(0, labels.FromStrings("foo", "bar1"), ts, h, nil) + _, err := app.Append(0, labels.FromStrings("foo", "bar1"), 0, ts, 0, h, nil, storage.AOptions{}) require.NoError(t, err) return sample{t: ts, h: h} } fh := tsdbutil.GenerateTestFloatHistogram(ts) - _, err := app.AppendHistogram(0, labels.FromStrings("foo", "bar1"), ts, nil, fh) + _, err := app.Append(0, labels.FromStrings("foo", "bar1"), 0, ts, 0, nil, fh, storage.AOptions{}) require.NoError(t, err) return sample{t: ts, fh: fh} } @@ -4913,7 +3685,7 @@ func TestMultipleEncodingsCommitOrder(t *testing.T) { var expSamples []chunks.Sample // Append samples with different encoding types and then commit them at once. - app := db.Appender(context.Background()) + app := db.AppenderV2(context.Background()) for i := 100; i < 105; i++ { s := addSample(app, int64(i), chunkenc.ValFloat) @@ -4947,7 +3719,7 @@ func TestMultipleEncodingsCommitOrder(t *testing.T) { verifySamples(100, 150, expSamples, 5) // Append and commit some in-order histograms by themselves. - app = db.Appender(context.Background()) + app = db.AppenderV2(context.Background()) for i := 150; i < 160; i++ { s := addSample(app, int64(i), chunkenc.ValHistogram) expSamples = append(expSamples, s) @@ -4959,7 +3731,7 @@ func TestMultipleEncodingsCommitOrder(t *testing.T) { // Append and commit samples for all encoding types. This time all samples will be treated as OOO because samples // with newer timestamps have already been committed. - app = db.Appender(context.Background()) + app = db.AppenderV2(context.Background()) for i := 50; i < 55; i++ { s := addSample(app, int64(i), chunkenc.ValFloat) expSamples = append(expSamples, s) @@ -4990,18 +3762,18 @@ func TestMultipleEncodingsCommitOrder(t *testing.T) { // TODO(codesome): test more samples incoming once compaction has started. To verify new samples after the start // // are not included in this compaction. -func TestOOOCompaction(t *testing.T) { +func TestOOOCompaction_AppendV2(t *testing.T) { for name, scenario := range sampleTypeScenarios { t.Run(name, func(t *testing.T) { - testOOOCompaction(t, scenario, false) + testOOOCompactionAppenderV2(t, scenario, false) }) t.Run(name+"+extra", func(t *testing.T) { - testOOOCompaction(t, scenario, true) + testOOOCompactionAppenderV2(t, scenario, true) }) } } -func testOOOCompaction(t *testing.T, scenario sampleTypeScenario, addExtraSamples bool) { +func testOOOCompactionAppenderV2(t *testing.T, scenario sampleTypeScenario, addExtraSamples bool) { ctx := context.Background() opts := DefaultOptions() @@ -5014,12 +3786,12 @@ func testOOOCompaction(t *testing.T, scenario sampleTypeScenario, addExtraSample series2 := labels.FromStrings("foo", "bar2") addSample := func(fromMins, toMins int64) { - app := db.Appender(context.Background()) + app := db.AppenderV2(context.Background()) for m := fromMins; m <= toMins; m++ { ts := m * time.Minute.Milliseconds() - _, _, err := scenario.appendFunc(app, series1, ts, ts) + _, _, err := scenario.appendFunc(storage.AppenderV2AsLimitedV1(app), series1, ts, ts) require.NoError(t, err) - _, _, err = scenario.appendFunc(app, series2, ts, 2*ts) + _, _, err = scenario.appendFunc(storage.AppenderV2AsLimitedV1(app), series2, ts, 2*ts) require.NoError(t, err) } require.NoError(t, app.Commit()) @@ -5188,16 +3960,16 @@ func testOOOCompaction(t *testing.T, scenario sampleTypeScenario, addExtraSample // TestOOOCompactionWithNormalCompaction tests if OOO compaction is performed // when the normal head's compaction is done. -func TestOOOCompactionWithNormalCompaction(t *testing.T) { +func TestOOOCompactionWithNormalCompaction_AppendV2(t *testing.T) { t.Parallel() for name, scenario := range sampleTypeScenarios { t.Run(name, func(t *testing.T) { - testOOOCompactionWithNormalCompaction(t, scenario) + testOOOCompactionWithNormalCompactionAppendV2(t, scenario) }) } } -func testOOOCompactionWithNormalCompaction(t *testing.T, scenario sampleTypeScenario) { +func testOOOCompactionWithNormalCompactionAppendV2(t *testing.T, scenario sampleTypeScenario) { t.Parallel() ctx := context.Background() @@ -5212,12 +3984,12 @@ func testOOOCompactionWithNormalCompaction(t *testing.T, scenario sampleTypeScen series2 := labels.FromStrings("foo", "bar2") addSamples := func(fromMins, toMins int64) { - app := db.Appender(context.Background()) + app := db.AppenderV2(context.Background()) for m := fromMins; m <= toMins; m++ { ts := m * time.Minute.Milliseconds() - _, _, err := scenario.appendFunc(app, series1, ts, ts) + _, _, err := scenario.appendFunc(storage.AppenderV2AsLimitedV1(app), series1, ts, ts) require.NoError(t, err) - _, _, err = scenario.appendFunc(app, series2, ts, 2*ts) + _, _, err = scenario.appendFunc(storage.AppenderV2AsLimitedV1(app), series2, ts, 2*ts) require.NoError(t, err) } require.NoError(t, app.Commit()) @@ -5293,16 +4065,16 @@ func testOOOCompactionWithNormalCompaction(t *testing.T, scenario sampleTypeScen // TestOOOCompactionWithDisabledWriteLog tests the scenario where the TSDB is // configured to not have wal and wbl but its able to compact both the in-order // and out-of-order head. -func TestOOOCompactionWithDisabledWriteLog(t *testing.T) { +func TestOOOCompactionWithDisabledWriteLog_AppendV2(t *testing.T) { t.Parallel() for name, scenario := range sampleTypeScenarios { t.Run(name, func(t *testing.T) { - testOOOCompactionWithDisabledWriteLog(t, scenario) + testOOOCompactionWithDisabledWriteLogAppend2(t, scenario) }) } } -func testOOOCompactionWithDisabledWriteLog(t *testing.T, scenario sampleTypeScenario) { +func testOOOCompactionWithDisabledWriteLogAppend2(t *testing.T, scenario sampleTypeScenario) { t.Parallel() ctx := context.Background() @@ -5318,12 +4090,12 @@ func testOOOCompactionWithDisabledWriteLog(t *testing.T, scenario sampleTypeScen series2 := labels.FromStrings("foo", "bar2") addSamples := func(fromMins, toMins int64) { - app := db.Appender(context.Background()) + app := db.AppenderV2(context.Background()) for m := fromMins; m <= toMins; m++ { ts := m * time.Minute.Milliseconds() - _, _, err := scenario.appendFunc(app, series1, ts, ts) + _, _, err := scenario.appendFunc(storage.AppenderV2AsLimitedV1(app), series1, ts, ts) require.NoError(t, err) - _, _, err = scenario.appendFunc(app, series2, ts, 2*ts) + _, _, err = scenario.appendFunc(storage.AppenderV2AsLimitedV1(app), series2, ts, 2*ts) require.NoError(t, err) } require.NoError(t, app.Commit()) @@ -5396,19 +4168,19 @@ func testOOOCompactionWithDisabledWriteLog(t *testing.T, scenario sampleTypeScen verifySamples(db.Blocks()[1], 250, 350) } -// TestOOOQueryAfterRestartWithSnapshotAndRemovedWBL tests the scenario where the WBL goes +// TestOOOQueryAfterRestartWithSnapshotAndRemovedWBL_AppendV2 tests the scenario where the WBL goes // missing after a restart while snapshot was enabled, but the query still returns the right // data from the mmap chunks. -func TestOOOQueryAfterRestartWithSnapshotAndRemovedWBL(t *testing.T) { +func TestOOOQueryAfterRestartWithSnapshotAndRemovedWBL_AppendV2(t *testing.T) { t.Parallel() for name, scenario := range sampleTypeScenarios { t.Run(name, func(t *testing.T) { - testOOOQueryAfterRestartWithSnapshotAndRemovedWBL(t, scenario) + testOOOQueryAfterRestartWithSnapshotAndRemovedWBLAppendV2(t, scenario) }) } } -func testOOOQueryAfterRestartWithSnapshotAndRemovedWBL(t *testing.T, scenario sampleTypeScenario) { +func testOOOQueryAfterRestartWithSnapshotAndRemovedWBLAppendV2(t *testing.T, scenario sampleTypeScenario) { ctx := context.Background() opts := DefaultOptions() @@ -5423,12 +4195,12 @@ func testOOOQueryAfterRestartWithSnapshotAndRemovedWBL(t *testing.T, scenario sa series2 := labels.FromStrings("foo", "bar2") addSamples := func(fromMins, toMins int64) { - app := db.Appender(context.Background()) + app := db.AppenderV2(context.Background()) for m := fromMins; m <= toMins; m++ { ts := m * time.Minute.Milliseconds() - _, _, err := scenario.appendFunc(app, series1, ts, ts) + _, _, err := scenario.appendFunc(storage.AppenderV2AsLimitedV1(app), series1, ts, ts) require.NoError(t, err) - _, _, err = scenario.appendFunc(app, series2, ts, 2*ts) + _, _, err = scenario.appendFunc(storage.AppenderV2AsLimitedV1(app), series2, ts, 2*ts) require.NoError(t, err) } require.NoError(t, app.Commit()) @@ -5509,38 +4281,38 @@ func testOOOQueryAfterRestartWithSnapshotAndRemovedWBL(t *testing.T, scenario sa verifySamples(90, 109) } -func TestQuerierOOOQuery(t *testing.T) { +func TestQuerierOOOQuery_AppendV2(t *testing.T) { scenarios := map[string]struct { - appendFunc func(app storage.Appender, ts int64, counterReset bool) (storage.SeriesRef, error) + appendFunc func(app storage.AppenderV2, ts int64, counterReset bool) (storage.SeriesRef, error) sampleFunc func(ts int64) chunks.Sample }{ "float": { - appendFunc: func(app storage.Appender, ts int64, _ bool) (storage.SeriesRef, error) { - return app.Append(0, labels.FromStrings("foo", "bar1"), ts, float64(ts)) + appendFunc: func(app storage.AppenderV2, ts int64, _ bool) (storage.SeriesRef, error) { + return app.Append(0, labels.FromStrings("foo", "bar1"), 0, ts, float64(ts), nil, nil, storage.AOptions{}) }, sampleFunc: func(ts int64) chunks.Sample { return sample{t: ts, f: float64(ts)} }, }, "integer histogram": { - appendFunc: func(app storage.Appender, ts int64, counterReset bool) (storage.SeriesRef, error) { + appendFunc: func(app storage.AppenderV2, ts int64, counterReset bool) (storage.SeriesRef, error) { h := tsdbutil.GenerateTestHistogram(ts) if counterReset { h.CounterResetHint = histogram.CounterReset } - return app.AppendHistogram(0, labels.FromStrings("foo", "bar1"), ts, h, nil) + return app.Append(0, labels.FromStrings("foo", "bar1"), 0, ts, 0, h, nil, storage.AOptions{}) }, sampleFunc: func(ts int64) chunks.Sample { return sample{t: ts, h: tsdbutil.GenerateTestHistogram(ts)} }, }, "float histogram": { - appendFunc: func(app storage.Appender, ts int64, counterReset bool) (storage.SeriesRef, error) { + appendFunc: func(app storage.AppenderV2, ts int64, counterReset bool) (storage.SeriesRef, error) { fh := tsdbutil.GenerateTestFloatHistogram(ts) if counterReset { fh.CounterResetHint = histogram.CounterReset } - return app.AppendHistogram(0, labels.FromStrings("foo", "bar1"), ts, nil, fh) + return app.Append(0, labels.FromStrings("foo", "bar1"), 0, ts, 0, nil, fh, storage.AOptions{}) }, sampleFunc: func(ts int64) chunks.Sample { return sample{t: ts, fh: tsdbutil.GenerateTestFloatHistogram(ts)} @@ -5548,10 +4320,10 @@ func TestQuerierOOOQuery(t *testing.T) { }, "integer histogram counter resets": { // Adding counter reset to all histograms means each histogram will have its own chunk. - appendFunc: func(app storage.Appender, ts int64, _ bool) (storage.SeriesRef, error) { + appendFunc: func(app storage.AppenderV2, ts int64, _ bool) (storage.SeriesRef, error) { h := tsdbutil.GenerateTestHistogram(ts) h.CounterResetHint = histogram.CounterReset // For this scenario, ignore the counterReset argument. - return app.AppendHistogram(0, labels.FromStrings("foo", "bar1"), ts, h, nil) + return app.Append(0, labels.FromStrings("foo", "bar1"), 0, ts, 0, h, nil, storage.AOptions{}) }, sampleFunc: func(ts int64) chunks.Sample { return sample{t: ts, h: tsdbutil.GenerateTestHistogram(ts)} @@ -5561,13 +4333,13 @@ func TestQuerierOOOQuery(t *testing.T) { for name, scenario := range scenarios { t.Run(name, func(t *testing.T) { - testQuerierOOOQuery(t, scenario.appendFunc, scenario.sampleFunc) + testQuerierOOOQueryAppendV2(t, scenario.appendFunc, scenario.sampleFunc) }) } } -func testQuerierOOOQuery(t *testing.T, - appendFunc func(app storage.Appender, ts int64, counterReset bool) (storage.SeriesRef, error), +func testQuerierOOOQueryAppendV2(t *testing.T, + appendFunc func(app storage.AppenderV2, ts int64, counterReset bool) (storage.SeriesRef, error), sampleFunc func(ts int64) chunks.Sample, ) { opts := DefaultOptions() @@ -5580,7 +4352,7 @@ func testQuerierOOOQuery(t *testing.T, minutes := func(m int64) int64 { return m * time.Minute.Milliseconds() } addSample := func(db *DB, fromMins, toMins, queryMinT, queryMaxT int64, expSamples []chunks.Sample, filter filterFunc, counterReset bool) ([]chunks.Sample, int) { - app := db.Appender(context.Background()) + app := db.AppenderV2(context.Background()) totalAppended := 0 for m := fromMins; m <= toMins; m += time.Minute.Milliseconds() { if !filter(m / time.Minute.Milliseconds()) { @@ -5803,7 +4575,7 @@ func testQuerierOOOQuery(t *testing.T, } } -func TestChunkQuerierOOOQuery(t *testing.T) { +func TestChunkQuerierOOOQuery_AppendV2(t *testing.T) { nBucketHistogram := func(n int64) *histogram.Histogram { h := &histogram.Histogram{ Count: uint64(n), @@ -5821,37 +4593,37 @@ func TestChunkQuerierOOOQuery(t *testing.T) { } scenarios := map[string]struct { - appendFunc func(app storage.Appender, ts int64, counterReset bool) (storage.SeriesRef, error) + appendFunc func(app storage.AppenderV2, ts int64, counterReset bool) (storage.SeriesRef, error) sampleFunc func(ts int64) chunks.Sample checkInUseBucket bool }{ "float": { - appendFunc: func(app storage.Appender, ts int64, _ bool) (storage.SeriesRef, error) { - return app.Append(0, labels.FromStrings("foo", "bar1"), ts, float64(ts)) + appendFunc: func(app storage.AppenderV2, ts int64, _ bool) (storage.SeriesRef, error) { + return app.Append(0, labels.FromStrings("foo", "bar1"), 0, ts, float64(ts), nil, nil, storage.AOptions{}) }, sampleFunc: func(ts int64) chunks.Sample { return sample{t: ts, f: float64(ts)} }, }, "integer histogram": { - appendFunc: func(app storage.Appender, ts int64, counterReset bool) (storage.SeriesRef, error) { + appendFunc: func(app storage.AppenderV2, ts int64, counterReset bool) (storage.SeriesRef, error) { h := tsdbutil.GenerateTestHistogram(ts) if counterReset { h.CounterResetHint = histogram.CounterReset } - return app.AppendHistogram(0, labels.FromStrings("foo", "bar1"), ts, h, nil) + return app.Append(0, labels.FromStrings("foo", "bar1"), 0, ts, 0, h, nil, storage.AOptions{}) }, sampleFunc: func(ts int64) chunks.Sample { return sample{t: ts, h: tsdbutil.GenerateTestHistogram(ts)} }, }, "float histogram": { - appendFunc: func(app storage.Appender, ts int64, counterReset bool) (storage.SeriesRef, error) { + appendFunc: func(app storage.AppenderV2, ts int64, counterReset bool) (storage.SeriesRef, error) { fh := tsdbutil.GenerateTestFloatHistogram(ts) if counterReset { fh.CounterResetHint = histogram.CounterReset } - return app.AppendHistogram(0, labels.FromStrings("foo", "bar1"), ts, nil, fh) + return app.Append(0, labels.FromStrings("foo", "bar1"), 0, ts, 0, nil, fh, storage.AOptions{}) }, sampleFunc: func(ts int64) chunks.Sample { return sample{t: ts, fh: tsdbutil.GenerateTestFloatHistogram(ts)} @@ -5859,10 +4631,10 @@ func TestChunkQuerierOOOQuery(t *testing.T) { }, "integer histogram counter resets": { // Adding counter reset to all histograms means each histogram will have its own chunk. - appendFunc: func(app storage.Appender, ts int64, _ bool) (storage.SeriesRef, error) { + appendFunc: func(app storage.AppenderV2, ts int64, _ bool) (storage.SeriesRef, error) { h := tsdbutil.GenerateTestHistogram(ts) h.CounterResetHint = histogram.CounterReset // For this scenario, ignore the counterReset argument. - return app.AppendHistogram(0, labels.FromStrings("foo", "bar1"), ts, h, nil) + return app.Append(0, labels.FromStrings("foo", "bar1"), 0, ts, 0, h, nil, storage.AOptions{}) }, sampleFunc: func(ts int64) chunks.Sample { return sample{t: ts, h: tsdbutil.GenerateTestHistogram(ts)} @@ -5870,9 +4642,9 @@ func TestChunkQuerierOOOQuery(t *testing.T) { }, "integer histogram with recode": { // Histograms have increasing number of buckets so their chunks are recoded. - appendFunc: func(app storage.Appender, ts int64, _ bool) (storage.SeriesRef, error) { + appendFunc: func(app storage.AppenderV2, ts int64, _ bool) (storage.SeriesRef, error) { n := ts / time.Minute.Milliseconds() - return app.AppendHistogram(0, labels.FromStrings("foo", "bar1"), ts, nBucketHistogram(n), nil) + return app.Append(0, labels.FromStrings("foo", "bar1"), 0, ts, 0, nBucketHistogram(n), nil, storage.AOptions{}) }, sampleFunc: func(ts int64) chunks.Sample { n := ts / time.Minute.Milliseconds() @@ -5885,13 +4657,13 @@ func TestChunkQuerierOOOQuery(t *testing.T) { } for name, scenario := range scenarios { t.Run(name, func(t *testing.T) { - testChunkQuerierOOOQuery(t, scenario.appendFunc, scenario.sampleFunc, scenario.checkInUseBucket) + testChunkQuerierOOOQueryAppendV2(t, scenario.appendFunc, scenario.sampleFunc, scenario.checkInUseBucket) }) } } -func testChunkQuerierOOOQuery(t *testing.T, - appendFunc func(app storage.Appender, ts int64, counterReset bool) (storage.SeriesRef, error), +func testChunkQuerierOOOQueryAppendV2(t *testing.T, + appendFunc func(app storage.AppenderV2, ts int64, counterReset bool) (storage.SeriesRef, error), sampleFunc func(ts int64) chunks.Sample, checkInUseBuckets bool, ) { @@ -5906,7 +4678,7 @@ func testChunkQuerierOOOQuery(t *testing.T, minutes := func(m int64) int64 { return m * time.Minute.Milliseconds() } addSample := func(db *DB, fromMins, toMins, queryMinT, queryMaxT int64, expSamples []chunks.Sample, filter filterFunc, counterReset bool) ([]chunks.Sample, int) { - app := db.Appender(context.Background()) + app := db.AppenderV2(context.Background()) totalAppended := 0 for m := fromMins; m <= toMins; m += time.Minute.Milliseconds() { if !filter(m / time.Minute.Milliseconds()) { @@ -6161,17 +4933,17 @@ func testChunkQuerierOOOQuery(t *testing.T, // could potentially come in that would change the status of the header. In this case, the UnknownCounterReset // headers would be re-checked at query time and updated as needed. However, this test is checking the counter // reset headers at the time of storage. -func TestOOONativeHistogramsWithCounterResets(t *testing.T) { +func TestOOONativeHistogramsWithCounterResets_AppendV2(t *testing.T) { for name, scenario := range sampleTypeScenarios { t.Run(name, func(t *testing.T) { if name == intHistogram || name == floatHistogram { - testOOONativeHistogramsWithCounterResets(t, scenario) + testOOONativeHistogramsWithCounterResetsAppendV2(t, scenario) } }) } } -func testOOONativeHistogramsWithCounterResets(t *testing.T, scenario sampleTypeScenario) { +func testOOONativeHistogramsWithCounterResetsAppendV2(t *testing.T, scenario sampleTypeScenario) { opts := DefaultOptions() opts.OutOfOrderCapMax = 30 opts.OutOfOrderTimeWindow = 24 * time.Hour.Milliseconds() @@ -6278,7 +5050,7 @@ func testOOONativeHistogramsWithCounterResets(t *testing.T, scenario sampleTypeS db := newTestDB(t, withOpts(opts)) db.DisableCompactions() - app := db.Appender(context.Background()) + app := db.AppenderV2(context.Background()) expSamples := make(map[string][]chunks.Sample) @@ -6290,7 +5062,7 @@ func testOOONativeHistogramsWithCounterResets(t *testing.T, scenario sampleTypeS if resetCount { j = 0 } - _, s, err := scenario.appendFunc(app, lbls, minutes(i), j) + _, s, err := scenario.appendFunc(storage.AppenderV2AsLimitedV1(app), lbls, minutes(i), j) require.NoError(t, err) if s.Type() == chunkenc.ValHistogram { s.H().CounterResetHint = batch.expCounterResetHints[smplIdx] @@ -6324,16 +5096,16 @@ func testOOONativeHistogramsWithCounterResets(t *testing.T, scenario sampleTypeS } } -func TestOOOInterleavedImplicitCounterResets(t *testing.T) { +func TestOOOInterleavedImplicitCounterResets_AppendV2(t *testing.T) { for name, scenario := range sampleTypeScenarios { t.Run(name, func(t *testing.T) { - testOOOInterleavedImplicitCounterResets(t, name, scenario) + testOOOInterleavedImplicitCounterResetsV2(t, name, scenario) }) } } -func testOOOInterleavedImplicitCounterResets(t *testing.T, name string, scenario sampleTypeScenario) { - var appendFunc func(app storage.Appender, ts, v int64) error +func testOOOInterleavedImplicitCounterResetsV2(t *testing.T, name string, scenario sampleTypeScenario) { + var appendFunc func(app storage.AppenderV2, ts, v int64) error if scenario.sampleType != sampleMetricTypeHistogram { return @@ -6341,29 +5113,29 @@ func testOOOInterleavedImplicitCounterResets(t *testing.T, name string, scenario switch name { case intHistogram: - appendFunc = func(app storage.Appender, ts, v int64) error { + appendFunc = func(app storage.AppenderV2, ts, v int64) error { h := &histogram.Histogram{ Count: uint64(v), Sum: float64(v), PositiveSpans: []histogram.Span{{Offset: 0, Length: 1}}, PositiveBuckets: []int64{v}, } - _, err := app.AppendHistogram(0, labels.FromStrings("foo", "bar1"), ts, h, nil) + _, err := app.Append(0, labels.FromStrings("foo", "bar1"), 0, ts, 0, h, nil, storage.AOptions{}) return err } case floatHistogram: - appendFunc = func(app storage.Appender, ts, v int64) error { + appendFunc = func(app storage.AppenderV2, ts, v int64) error { fh := &histogram.FloatHistogram{ Count: float64(v), Sum: float64(v), PositiveSpans: []histogram.Span{{Offset: 0, Length: 1}}, PositiveBuckets: []float64{float64(v)}, } - _, err := app.AppendHistogram(0, labels.FromStrings("foo", "bar1"), ts, nil, fh) + _, err := app.Append(0, labels.FromStrings("foo", "bar1"), 0, ts, 0, nil, fh, storage.AOptions{}) return err } case customBucketsIntHistogram: - appendFunc = func(app storage.Appender, ts, v int64) error { + appendFunc = func(app storage.AppenderV2, ts, v int64) error { h := &histogram.Histogram{ Schema: -53, Count: uint64(v), @@ -6372,11 +5144,11 @@ func testOOOInterleavedImplicitCounterResets(t *testing.T, name string, scenario PositiveBuckets: []int64{v}, CustomValues: []float64{float64(1), float64(2), float64(3)}, } - _, err := app.AppendHistogram(0, labels.FromStrings("foo", "bar1"), ts, h, nil) + _, err := app.Append(0, labels.FromStrings("foo", "bar1"), 0, ts, 0, h, nil, storage.AOptions{}) return err } case customBucketsFloatHistogram: - appendFunc = func(app storage.Appender, ts, v int64) error { + appendFunc = func(app storage.AppenderV2, ts, v int64) error { fh := &histogram.FloatHistogram{ Schema: -53, Count: float64(v), @@ -6385,7 +5157,7 @@ func testOOOInterleavedImplicitCounterResets(t *testing.T, name string, scenario PositiveBuckets: []float64{float64(v)}, CustomValues: []float64{float64(1), float64(2), float64(3)}, } - _, err := app.AppendHistogram(0, labels.FromStrings("foo", "bar1"), ts, nil, fh) + _, err := app.Append(0, labels.FromStrings("foo", "bar1"), 0, ts, 0, nil, fh, storage.AOptions{}) return err } case gaugeIntHistogram, gaugeFloatHistogram: @@ -6512,7 +5284,7 @@ func testOOOInterleavedImplicitCounterResets(t *testing.T, name string, scenario db := newTestDB(t, withOpts(opts)) db.DisableCompactions() - app := db.Appender(context.Background()) + app := db.AppenderV2(context.Background()) for _, s := range tc.samples { require.NoError(t, appendFunc(app, s.ts, s.v)) } @@ -6594,15 +5366,15 @@ func testOOOInterleavedImplicitCounterResets(t *testing.T, name string, scenario } } -func TestOOOAppendAndQuery(t *testing.T) { +func TestOOOAppendAndQuery_AppendV2(t *testing.T) { for name, scenario := range sampleTypeScenarios { t.Run(name, func(t *testing.T) { - testOOOAppendAndQuery(t, scenario) + testOOOAppendAndQueryAppendV2(t, scenario) }) } } -func testOOOAppendAndQuery(t *testing.T, scenario sampleTypeScenario) { +func testOOOAppendAndQueryAppendV2(t *testing.T, scenario sampleTypeScenario) { opts := DefaultOptions() opts.OutOfOrderCapMax = 30 opts.OutOfOrderTimeWindow = 4 * time.Hour.Milliseconds() @@ -6617,12 +5389,12 @@ func testOOOAppendAndQuery(t *testing.T, scenario sampleTypeScenario) { appendedSamples := make(map[string][]chunks.Sample) totalSamples := 0 addSample := func(lbls labels.Labels, fromMins, toMins int64, faceError bool) { - app := db.Appender(context.Background()) + app := db.AppenderV2(context.Background()) key := lbls.String() from, to := minutes(fromMins), minutes(toMins) for m := from; m <= to; m += time.Minute.Milliseconds() { val := rand.Intn(1000) - _, s, err := scenario.appendFunc(app, lbls, m, int64(val)) + _, s, err := scenario.appendFunc(storage.AppenderV2AsLimitedV1(app), lbls, m, int64(val)) if faceError { require.Error(t, err) } else { @@ -6724,15 +5496,15 @@ func testOOOAppendAndQuery(t *testing.T, scenario sampleTypeScenario) { testQuery(math.MinInt64, math.MaxInt64) } -func TestOOODisabled(t *testing.T) { +func TestOOODisabled_AppendV2(t *testing.T) { for name, scenario := range sampleTypeScenarios { t.Run(name, func(t *testing.T) { - testOOODisabled(t, scenario) + testOOODisabledAppendV2(t, scenario) }) } } -func testOOODisabled(t *testing.T, scenario sampleTypeScenario) { +func testOOODisabledAppendV2(t *testing.T, scenario sampleTypeScenario) { opts := DefaultOptions() opts.OutOfOrderTimeWindow = 0 db := newTestDB(t, withOpts(opts)) @@ -6745,11 +5517,11 @@ func testOOODisabled(t *testing.T, scenario sampleTypeScenario) { failedSamples := 0 addSample := func(db *DB, lbls labels.Labels, fromMins, toMins int64, faceError bool) { - app := db.Appender(context.Background()) + app := db.AppenderV2(context.Background()) key := lbls.String() from, to := minutes(fromMins), minutes(toMins) for m := from; m <= to; m += time.Minute.Milliseconds() { - _, _, err := scenario.appendFunc(app, lbls, m, m) + _, _, err := scenario.appendFunc(storage.AppenderV2AsLimitedV1(app), lbls, m, m) if faceError { require.Error(t, err) failedSamples++ @@ -6794,15 +5566,15 @@ func testOOODisabled(t *testing.T, scenario sampleTypeScenario) { require.Nil(t, ms.ooo) } -func TestWBLAndMmapReplay(t *testing.T) { +func TestWBLAndMmapReplay_AppendV2(t *testing.T) { for name, scenario := range sampleTypeScenarios { t.Run(name, func(t *testing.T) { - testWBLAndMmapReplay(t, scenario) + testWBLAndMmapReplayAppendV2(t, scenario) }) } } -func testWBLAndMmapReplay(t *testing.T, scenario sampleTypeScenario) { +func testWBLAndMmapReplayAppendV2(t *testing.T, scenario sampleTypeScenario) { opts := DefaultOptions() opts.OutOfOrderCapMax = 30 opts.OutOfOrderTimeWindow = 4 * time.Hour.Milliseconds() @@ -6816,12 +5588,12 @@ func testWBLAndMmapReplay(t *testing.T, scenario sampleTypeScenario) { expSamples := make(map[string][]chunks.Sample) totalSamples := 0 addSample := func(lbls labels.Labels, fromMins, toMins int64) { - app := db.Appender(context.Background()) + app := db.AppenderV2(context.Background()) key := lbls.String() from, to := minutes(fromMins), minutes(toMins) for m := from; m <= to; m += time.Minute.Milliseconds() { val := rand.Intn(1000) - _, s, err := scenario.appendFunc(app, lbls, m, int64(val)) + _, s, err := scenario.appendFunc(storage.AppenderV2AsLimitedV1(app), lbls, m, int64(val)) require.NoError(t, err) expSamples[key] = append(expSamples[key], s) totalSamples++ @@ -6975,7 +5747,7 @@ func testWBLAndMmapReplay(t *testing.T, scenario sampleTypeScenario) { }) } -func TestOOOHistogramCompactionWithCounterResets(t *testing.T) { +func TestOOOHistogramCompactionWithCounterResets_AppendV2(t *testing.T) { for _, floatHistogram := range []bool{false, true} { ctx := context.Background() @@ -6992,12 +5764,12 @@ func TestOOOHistogramCompactionWithCounterResets(t *testing.T) { var series1ExpSamplesPreCompact, series2ExpSamplesPreCompact, series1ExpSamplesPostCompact, series2ExpSamplesPostCompact []chunks.Sample addSample := func(ts int64, l labels.Labels, val int, hint histogram.CounterResetHint) sample { - app := db.Appender(context.Background()) + app := db.AppenderV2(context.Background()) tsMs := ts * time.Minute.Milliseconds() if floatHistogram { h := tsdbutil.GenerateTestFloatHistogram(int64(val)) h.CounterResetHint = hint - _, err := app.AppendHistogram(0, l, tsMs, nil, h) + _, err := app.Append(0, l, 0, tsMs, 0, nil, h, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) return sample{t: tsMs, fh: h.Copy()} @@ -7005,7 +5777,7 @@ func TestOOOHistogramCompactionWithCounterResets(t *testing.T) { h := tsdbutil.GenerateTestHistogram(int64(val)) h.CounterResetHint = hint - _, err := app.AppendHistogram(0, l, tsMs, h, nil) + _, err := app.Append(0, l, 0, tsMs, 0, h, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) return sample{t: tsMs, h: h.Copy()} @@ -7330,7 +6102,7 @@ func TestOOOHistogramCompactionWithCounterResets(t *testing.T) { } } -func TestInterleavedInOrderAndOOOHistogramCompactionWithCounterResets(t *testing.T) { +func TestInterleavedInOrderAndOOOHistogramCompactionWithCounterResets_AppendV2(t *testing.T) { for _, floatHistogram := range []bool{false, true} { ctx := context.Background() @@ -7344,18 +6116,18 @@ func TestInterleavedInOrderAndOOOHistogramCompactionWithCounterResets(t *testing series1 := labels.FromStrings("foo", "bar1") addSample := func(ts int64, l labels.Labels, val int) sample { - app := db.Appender(context.Background()) + app := db.AppenderV2(context.Background()) tsMs := ts if floatHistogram { h := tsdbutil.GenerateTestFloatHistogram(int64(val)) - _, err := app.AppendHistogram(0, l, tsMs, nil, h) + _, err := app.Append(0, l, 0, tsMs, 0, nil, h, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) return sample{t: tsMs, fh: h.Copy()} } h := tsdbutil.GenerateTestHistogram(int64(val)) - _, err := app.AppendHistogram(0, l, tsMs, h, nil) + _, err := app.Append(0, l, 0, tsMs, 0, h, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) return sample{t: tsMs, h: h.Copy()} @@ -7403,7 +6175,8 @@ func TestInterleavedInOrderAndOOOHistogramCompactionWithCounterResets(t *testing // Compact the in-order head and expect another block. // Since this is a forced compaction, this block is not aligned with 2h. - require.NoError(t, db.CompactHead(NewRangeHead(db.head, 0, 3))) + err := db.CompactHead(NewRangeHead(db.head, 0, 3)) + require.NoError(t, err) require.Len(t, db.Blocks(), 2) // Blocks created out of normal and OOO head now. But not merged. @@ -7419,28 +6192,16 @@ func TestInterleavedInOrderAndOOOHistogramCompactionWithCounterResets(t *testing } } -func copyWithCounterReset(s sample, hint histogram.CounterResetHint) sample { - if s.h != nil { - h := s.h.Copy() - h.CounterResetHint = hint - return sample{t: s.t, h: h} - } - - h := s.fh.Copy() - h.CounterResetHint = hint - return sample{t: s.t, fh: h} -} - -func TestOOOCompactionFailure(t *testing.T) { +func TestOOOCompactionFailure_AppendV2(t *testing.T) { t.Parallel() for name, scenario := range sampleTypeScenarios { t.Run(name, func(t *testing.T) { - testOOOCompactionFailure(t, scenario) + testOOOCompactionFailureAppendV2(t, scenario) }) } } -func testOOOCompactionFailure(t *testing.T, scenario sampleTypeScenario) { +func testOOOCompactionFailureAppendV2(t *testing.T, scenario sampleTypeScenario) { ctx := context.Background() opts := DefaultOptions() @@ -7448,14 +6209,17 @@ func testOOOCompactionFailure(t *testing.T, scenario sampleTypeScenario) { opts.OutOfOrderTimeWindow = 300 * time.Minute.Milliseconds() db := newTestDB(t, withOpts(opts)) db.DisableCompactions() // We want to manually call it. + t.Cleanup(func() { + require.NoError(t, db.Close()) + }) series1 := labels.FromStrings("foo", "bar1") addSample := func(fromMins, toMins int64) { - app := db.Appender(context.Background()) + app := db.AppenderV2(context.Background()) for m := fromMins; m <= toMins; m++ { ts := m * time.Minute.Milliseconds() - _, _, err := scenario.appendFunc(app, series1, ts, ts) + _, _, err := scenario.appendFunc(storage.AppenderV2AsLimitedV1(app), series1, ts, ts) require.NoError(t, err) } require.NoError(t, app.Commit()) @@ -7572,7 +6336,7 @@ func testOOOCompactionFailure(t *testing.T, scenario sampleTypeScenario) { verifyMmapFiles("000001") } -func TestWBLCorruption(t *testing.T) { +func TestWBLCorruption_AppendV2(t *testing.T) { opts := DefaultOptions() opts.OutOfOrderCapMax = 30 opts.OutOfOrderTimeWindow = 300 * time.Minute.Milliseconds() @@ -7582,10 +6346,10 @@ func TestWBLCorruption(t *testing.T) { series1 := labels.FromStrings("foo", "bar1") var allSamples, expAfterRestart []chunks.Sample addSamples := func(fromMins, toMins int64, afterRestart bool) { - app := db.Appender(context.Background()) + app := db.AppenderV2(context.Background()) for m := fromMins; m <= toMins; m++ { ts := m * time.Minute.Milliseconds() - _, err := app.Append(0, series1, ts, float64(ts)) + _, err := app.Append(0, series1, 0, ts, float64(ts), nil, nil, storage.AOptions{}) require.NoError(t, err) allSamples = append(allSamples, sample{t: ts, f: float64(ts)}) if afterRestart { @@ -7711,15 +6475,15 @@ func TestWBLCorruption(t *testing.T) { verifySamples(expAfterRestart) } -func TestOOOMmapCorruption(t *testing.T) { +func TestOOOMmapCorruption_AppendV2(t *testing.T) { for name, scenario := range sampleTypeScenarios { t.Run(name, func(t *testing.T) { - testOOOMmapCorruption(t, scenario) + testOOOMmapCorruptionAppendV2(t, scenario) }) } } -func testOOOMmapCorruption(t *testing.T, scenario sampleTypeScenario) { +func testOOOMmapCorruptionAppendV2(t *testing.T, scenario sampleTypeScenario) { opts := DefaultOptions() opts.OutOfOrderCapMax = 10 opts.OutOfOrderTimeWindow = 300 * time.Minute.Milliseconds() @@ -7729,10 +6493,10 @@ func testOOOMmapCorruption(t *testing.T, scenario sampleTypeScenario) { series1 := labels.FromStrings("foo", "bar1") var allSamples, expInMmapChunks []chunks.Sample addSamples := func(fromMins, toMins int64, inMmapAfterCorruption bool) { - app := db.Appender(context.Background()) + app := db.AppenderV2(context.Background()) for m := fromMins; m <= toMins; m++ { ts := m * time.Minute.Milliseconds() - _, s, err := scenario.appendFunc(app, series1, ts, ts) + _, s, err := scenario.appendFunc(storage.AppenderV2AsLimitedV1(app), series1, ts, ts) require.NoError(t, err) allSamples = append(allSamples, s) if inMmapAfterCorruption { @@ -7835,16 +6599,16 @@ func testOOOMmapCorruption(t *testing.T, scenario sampleTypeScenario) { verifySamples(allSamples) } -func TestOutOfOrderRuntimeConfig(t *testing.T) { +func TestOutOfOrderRuntimeConfig_AppendV2(t *testing.T) { t.Parallel() for name, scenario := range sampleTypeScenarios { t.Run(name, func(t *testing.T) { - testOutOfOrderRuntimeConfig(t, scenario) + testOutOfOrderRuntimeConfigAppendV2(t, scenario) }) } } -func testOutOfOrderRuntimeConfig(t *testing.T, scenario sampleTypeScenario) { +func testOutOfOrderRuntimeConfigAppendV2(t *testing.T, scenario sampleTypeScenario) { ctx := context.Background() getDB := func(oooTimeWindow int64) *DB { @@ -7867,10 +6631,10 @@ func testOutOfOrderRuntimeConfig(t *testing.T, scenario sampleTypeScenario) { series1 := labels.FromStrings("foo", "bar1") addSamples := func(t *testing.T, db *DB, fromMins, toMins int64, success bool, allSamples []chunks.Sample) []chunks.Sample { - app := db.Appender(context.Background()) + app := db.AppenderV2(context.Background()) for m := fromMins; m <= toMins; m++ { ts := m * time.Minute.Milliseconds() - _, s, err := scenario.appendFunc(app, series1, ts, ts) + _, s, err := scenario.appendFunc(storage.AppenderV2AsLimitedV1(app), series1, ts, ts) if success { require.NoError(t, err) allSamples = append(allSamples, s) @@ -8067,22 +6831,22 @@ func testOutOfOrderRuntimeConfig(t *testing.T, scenario sampleTypeScenario) { }) } -func TestNoGapAfterRestartWithOOO(t *testing.T) { +func TestNoGapAfterRestartWithOOO_AppendV2(t *testing.T) { t.Parallel() for name, scenario := range sampleTypeScenarios { t.Run(name, func(t *testing.T) { - testNoGapAfterRestartWithOOO(t, scenario) + testNoGapAfterRestartWithOOOAppendV2(t, scenario) }) } } -func testNoGapAfterRestartWithOOO(t *testing.T, scenario sampleTypeScenario) { +func testNoGapAfterRestartWithOOOAppendV2(t *testing.T, scenario sampleTypeScenario) { series1 := labels.FromStrings("foo", "bar1") addSamples := func(t *testing.T, db *DB, fromMins, toMins int64, success bool) { - app := db.Appender(context.Background()) + app := db.AppenderV2(context.Background()) for m := fromMins; m <= toMins; m++ { ts := m * time.Minute.Milliseconds() - _, _, err := scenario.appendFunc(app, series1, ts, ts) + _, _, err := scenario.appendFunc(storage.AppenderV2AsLimitedV1(app), series1, ts, ts) if success { require.NoError(t, err) } else { @@ -8176,15 +6940,15 @@ func testNoGapAfterRestartWithOOO(t *testing.T, scenario sampleTypeScenario) { } } -func TestWblReplayAfterOOODisableAndRestart(t *testing.T) { +func TestWblReplayAfterOOODisableAndRestart_AppendV2(t *testing.T) { for name, scenario := range sampleTypeScenarios { t.Run(name, func(t *testing.T) { - testWblReplayAfterOOODisableAndRestart(t, scenario) + testWblReplayAfterOOODisableAndRestartAppendV2(t, scenario) }) } } -func testWblReplayAfterOOODisableAndRestart(t *testing.T, scenario sampleTypeScenario) { +func testWblReplayAfterOOODisableAndRestartAppendV2(t *testing.T, scenario sampleTypeScenario) { opts := DefaultOptions() opts.OutOfOrderTimeWindow = 60 * time.Minute.Milliseconds() @@ -8193,10 +6957,10 @@ func testWblReplayAfterOOODisableAndRestart(t *testing.T, scenario sampleTypeSce series1 := labels.FromStrings("foo", "bar1") var allSamples []chunks.Sample addSamples := func(fromMins, toMins int64) { - app := db.Appender(context.Background()) + app := db.AppenderV2(context.Background()) for m := fromMins; m <= toMins; m++ { ts := m * time.Minute.Milliseconds() - _, s, err := scenario.appendFunc(app, series1, ts, ts) + _, s, err := scenario.appendFunc(storage.AppenderV2AsLimitedV1(app), series1, ts, ts) require.NoError(t, err) allSamples = append(allSamples, s) } @@ -8236,15 +7000,15 @@ func testWblReplayAfterOOODisableAndRestart(t *testing.T, scenario sampleTypeSce verifySamples(allSamples) } -func TestPanicOnApplyConfig(t *testing.T) { +func TestPanicOnApplyConfig_AppendV2(t *testing.T) { for name, scenario := range sampleTypeScenarios { t.Run(name, func(t *testing.T) { - testPanicOnApplyConfig(t, scenario) + testPanicOnApplyConfigAppendV2(t, scenario) }) } } -func testPanicOnApplyConfig(t *testing.T, scenario sampleTypeScenario) { +func testPanicOnApplyConfigAppendV2(t *testing.T, scenario sampleTypeScenario) { opts := DefaultOptions() opts.OutOfOrderTimeWindow = 60 * time.Minute.Milliseconds() @@ -8253,10 +7017,10 @@ func testPanicOnApplyConfig(t *testing.T, scenario sampleTypeScenario) { series1 := labels.FromStrings("foo", "bar1") var allSamples []chunks.Sample addSamples := func(fromMins, toMins int64) { - app := db.Appender(context.Background()) + app := db.AppenderV2(context.Background()) for m := fromMins; m <= toMins; m++ { ts := m * time.Minute.Milliseconds() - _, s, err := scenario.appendFunc(app, series1, ts, ts) + _, s, err := scenario.appendFunc(storage.AppenderV2AsLimitedV1(app), series1, ts, ts) require.NoError(t, err) allSamples = append(allSamples, s) } @@ -8285,16 +7049,16 @@ func testPanicOnApplyConfig(t *testing.T, scenario sampleTypeScenario) { require.NoError(t, err) } -func TestDiskFillingUpAfterDisablingOOO(t *testing.T) { +func TestDiskFillingUpAfterDisablingOOO_AppendV2(t *testing.T) { t.Parallel() for name, scenario := range sampleTypeScenarios { t.Run(name, func(t *testing.T) { - testDiskFillingUpAfterDisablingOOO(t, scenario) + testDiskFillingUpAfterDisablingOOOAppenderV2(t, scenario) }) } } -func testDiskFillingUpAfterDisablingOOO(t *testing.T, scenario sampleTypeScenario) { +func testDiskFillingUpAfterDisablingOOOAppenderV2(t *testing.T, scenario sampleTypeScenario) { t.Parallel() ctx := context.Background() @@ -8307,10 +7071,10 @@ func testDiskFillingUpAfterDisablingOOO(t *testing.T, scenario sampleTypeScenari series1 := labels.FromStrings("foo", "bar1") var allSamples []chunks.Sample addSamples := func(fromMins, toMins int64) { - app := db.Appender(context.Background()) + app := db.AppenderV2(context.Background()) for m := fromMins; m <= toMins; m++ { ts := m * time.Minute.Milliseconds() - _, s, err := scenario.appendFunc(app, series1, ts, ts) + _, s, err := scenario.appendFunc(storage.AppenderV2AsLimitedV1(app), series1, ts, ts) require.NoError(t, err) allSamples = append(allSamples, s) } @@ -8376,19 +7140,22 @@ func testDiskFillingUpAfterDisablingOOO(t *testing.T, scenario sampleTypeScenari require.Equal(t, int64(0), finfo.Size()) } -func TestHistogramAppendAndQuery(t *testing.T) { +func TestHistogramAppendAndQuery_AppendV2(t *testing.T) { t.Run("integer histograms", func(t *testing.T) { - testHistogramAppendAndQueryHelper(t, false) + testHistogramAppendAndQueryHelperAppendV2(t, false) }) t.Run("float histograms", func(t *testing.T) { - testHistogramAppendAndQueryHelper(t, true) + testHistogramAppendAndQueryHelperAppendV2(t, true) }) } -func testHistogramAppendAndQueryHelper(t *testing.T, floatHistogram bool) { +func testHistogramAppendAndQueryHelperAppendV2(t *testing.T, floatHistogram bool) { t.Helper() db := newTestDB(t) minute := func(m int) int64 { return int64(m) * time.Minute.Milliseconds() } + t.Cleanup(func() { + require.NoError(t, db.Close()) + }) ctx := context.Background() appendHistogram := func(t *testing.T, @@ -8397,14 +7164,14 @@ func testHistogramAppendAndQueryHelper(t *testing.T, floatHistogram bool) { ) { t.Helper() var err error - app := db.Appender(ctx) + app := db.AppenderV2(ctx) if floatHistogram { - _, err = app.AppendHistogram(0, lbls, minute(tsMinute), nil, h.ToFloat(nil)) + _, err = app.Append(0, lbls, 0, minute(tsMinute), 0, nil, h.ToFloat(nil), storage.AOptions{}) efh := h.ToFloat(nil) efh.CounterResetHint = expCRH *exp = append(*exp, sample{t: minute(tsMinute), fh: efh}) } else { - _, err = app.AppendHistogram(0, lbls, minute(tsMinute), h.Copy(), nil) + _, err = app.Append(0, lbls, 0, minute(tsMinute), 0, h.Copy(), nil, storage.AOptions{}) eh := h.Copy() eh.CounterResetHint = expCRH *exp = append(*exp, sample{t: minute(tsMinute), h: eh}) @@ -8414,8 +7181,8 @@ func testHistogramAppendAndQueryHelper(t *testing.T, floatHistogram bool) { } appendFloat := func(t *testing.T, lbls labels.Labels, tsMinute int, val float64, exp *[]chunks.Sample) { t.Helper() - app := db.Appender(ctx) - _, err := app.Append(0, lbls, minute(tsMinute), val) + app := db.AppenderV2(ctx) + _, err := app.Append(0, lbls, 0, minute(tsMinute), val, nil, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) *exp = append(*exp, sample{t: minute(tsMinute), f: val}) @@ -8647,139 +7414,7 @@ func testHistogramAppendAndQueryHelper(t *testing.T, floatHistogram bool) { }) } -func TestQueryHistogramFromBlocksWithCompaction(t *testing.T) { - t.Parallel() - minute := func(m int) int64 { return int64(m) * time.Minute.Milliseconds() } - - testBlockQuerying := func(t *testing.T, blockSeries ...[]storage.Series) { - t.Helper() - - opts := DefaultOptions() - db := newTestDB(t, withOpts(opts)) - - var it chunkenc.Iterator - exp := make(map[string][]chunks.Sample) - for _, series := range blockSeries { - createBlock(t, db.Dir(), series) - - for _, s := range series { - lbls := s.Labels().String() - slice := exp[lbls] - it = s.Iterator(it) - smpls, err := storage.ExpandSamples(it, nil) - require.NoError(t, err) - slice = append(slice, smpls...) - sort.Slice(slice, func(i, j int) bool { - return slice[i].T() < slice[j].T() - }) - exp[lbls] = slice - } - } - - require.Empty(t, db.Blocks()) - require.NoError(t, db.reload()) - require.Len(t, db.Blocks(), len(blockSeries)) - - q, err := db.Querier(math.MinInt64, math.MaxInt64) - require.NoError(t, err) - res := query(t, q, labels.MustNewMatcher(labels.MatchRegexp, "__name__", ".*")) - compareSeries(t, exp, res) - - // Compact all the blocks together and query again. - blocks := db.Blocks() - blockDirs := make([]string, 0, len(blocks)) - for _, b := range blocks { - blockDirs = append(blockDirs, b.Dir()) - } - ids, err := db.compactor.Compact(db.Dir(), blockDirs, blocks) - require.NoError(t, err) - require.Len(t, ids, 1) - require.NoError(t, db.reload()) - require.Len(t, db.Blocks(), 1) - - q, err = db.Querier(math.MinInt64, math.MaxInt64) - require.NoError(t, err) - res = query(t, q, labels.MustNewMatcher(labels.MatchRegexp, "__name__", ".*")) - - // After compaction, we do not require "unknown" counter resets - // due to origin from different overlapping chunks anymore. - for _, ss := range exp { - for i, s := range ss[1:] { - if s.Type() == chunkenc.ValHistogram && ss[i].Type() == chunkenc.ValHistogram && s.H().CounterResetHint == histogram.UnknownCounterReset { - s.H().CounterResetHint = histogram.NotCounterReset - } - if s.Type() == chunkenc.ValFloatHistogram && ss[i].Type() == chunkenc.ValFloatHistogram && s.FH().CounterResetHint == histogram.UnknownCounterReset { - s.FH().CounterResetHint = histogram.NotCounterReset - } - } - } - compareSeries(t, exp, res) - } - - for _, floatHistogram := range []bool{false, true} { - t.Run(fmt.Sprintf("floatHistogram=%t", floatHistogram), func(t *testing.T) { - t.Run("serial blocks with only histograms", func(t *testing.T) { - testBlockQuerying(t, - genHistogramSeries(10, 5, minute(0), minute(119), minute(1), floatHistogram), - genHistogramSeries(10, 5, minute(120), minute(239), minute(1), floatHistogram), - genHistogramSeries(10, 5, minute(240), minute(359), minute(1), floatHistogram), - ) - }) - - t.Run("serial blocks with either histograms or floats in a block and not both", func(t *testing.T) { - testBlockQuerying(t, - genHistogramSeries(10, 5, minute(0), minute(119), minute(1), floatHistogram), - genSeriesFromSampleGenerator(10, 5, minute(120), minute(239), minute(1), func(ts int64) chunks.Sample { - return sample{t: ts, f: rand.Float64()} - }), - genHistogramSeries(10, 5, minute(240), minute(359), minute(1), floatHistogram), - ) - }) - - t.Run("serial blocks with mix of histograms and float64", func(t *testing.T) { - testBlockQuerying(t, - genHistogramAndFloatSeries(10, 5, minute(0), minute(60), minute(1), floatHistogram), - genHistogramSeries(10, 5, minute(61), minute(120), minute(1), floatHistogram), - genHistogramAndFloatSeries(10, 5, minute(121), minute(180), minute(1), floatHistogram), - genSeriesFromSampleGenerator(10, 5, minute(181), minute(240), minute(1), func(ts int64) chunks.Sample { - return sample{t: ts, f: rand.Float64()} - }), - ) - }) - - t.Run("overlapping blocks with only histograms", func(t *testing.T) { - testBlockQuerying(t, - genHistogramSeries(10, 5, minute(0), minute(120), minute(3), floatHistogram), - genHistogramSeries(10, 5, minute(1), minute(120), minute(3), floatHistogram), - genHistogramSeries(10, 5, minute(2), minute(120), minute(3), floatHistogram), - ) - }) - - t.Run("overlapping blocks with only histograms and only float in a series", func(t *testing.T) { - testBlockQuerying(t, - genHistogramSeries(10, 5, minute(0), minute(120), minute(3), floatHistogram), - genSeriesFromSampleGenerator(10, 5, minute(1), minute(120), minute(3), func(ts int64) chunks.Sample { - return sample{t: ts, f: rand.Float64()} - }), - genHistogramSeries(10, 5, minute(2), minute(120), minute(3), floatHistogram), - ) - }) - - t.Run("overlapping blocks with mix of histograms and float64", func(t *testing.T) { - testBlockQuerying(t, - genHistogramAndFloatSeries(10, 5, minute(0), minute(60), minute(3), floatHistogram), - genHistogramSeries(10, 5, minute(46), minute(100), minute(3), floatHistogram), - genHistogramAndFloatSeries(10, 5, minute(89), minute(140), minute(3), floatHistogram), - genSeriesFromSampleGenerator(10, 5, minute(126), minute(200), minute(3), func(ts int64) chunks.Sample { - return sample{t: ts, f: rand.Float64()} - }), - ) - }) - }) - } -} - -func TestOOONativeHistogramsSettings(t *testing.T) { +func TestOOONativeHistogramsSettings_AppendV2(t *testing.T) { h := &histogram.Histogram{ Count: 9, ZeroCount: 4, @@ -8800,11 +7435,11 @@ func TestOOONativeHistogramsSettings(t *testing.T) { opts.OutOfOrderTimeWindow = 0 db := newTestDB(t, withOpts(opts), withRngs(100)) - app := db.Appender(context.Background()) - _, err := app.AppendHistogram(0, l, 100, h, nil) + app := db.AppenderV2(context.Background()) + _, err := app.Append(0, l, 0, 100, 0, h, nil, storage.AOptions{}) require.NoError(t, err) - _, err = app.AppendHistogram(0, l, 50, h, nil) + _, err = app.Append(0, l, 0, 50, 0, h, nil, storage.AOptions{}) require.NoError(t, err) // The OOO sample is not detected until it is committed, so no error is returned require.NoError(t, app.Commit()) @@ -8822,14 +7457,14 @@ func TestOOONativeHistogramsSettings(t *testing.T) { db := newTestDB(t, withOpts(opts), withRngs(100)) // Add in-order samples - app := db.Appender(context.Background()) - _, err := app.AppendHistogram(0, l, 200, h, nil) + app := db.AppenderV2(context.Background()) + _, err := app.Append(0, l, 0, 200, 0, h, nil, storage.AOptions{}) require.NoError(t, err) // Add OOO samples - _, err = app.AppendHistogram(0, l, 100, h, nil) + _, err = app.Append(0, l, 0, 100, 0, h, nil, storage.AOptions{}) require.NoError(t, err) - _, err = app.AppendHistogram(0, l, 150, h, nil) + _, err = app.Append(0, l, 0, 150, 0, h, nil, storage.AOptions{}) require.NoError(t, err) require.NoError(t, app.Commit()) @@ -8843,71 +7478,11 @@ func TestOOONativeHistogramsSettings(t *testing.T) { }) } -// compareSeries essentially replaces `require.Equal(t, expected, actual)` in -// situations where the actual series might contain more counter reset hints -// "unknown" than the expected series. This can easily happen for long series -// that trigger new chunks. This function therefore tolerates counter reset -// hints "CounterReset" and "NotCounterReset" in an expected series where the -// actual series contains a counter reset hint "UnknownCounterReset". -// "GaugeType" hints are still strictly checked, and any "UnknownCounterReset" -// in an expected series has to be matched precisely by the actual series. -func compareSeries(t require.TestingT, expected, actual map[string][]chunks.Sample) { - if len(expected) != len(actual) { - // The reason for the difference is not the counter reset hints - // (alone), so let's use the pretty diffing by the require - // package. - require.Equal(t, expected, actual, "number of series differs") - } - for key, expSamples := range expected { - actSamples, ok := actual[key] - if !ok { - require.Equal(t, expected, actual, "expected series %q not found", key) - } - if len(expSamples) != len(actSamples) { - require.Equal(t, expSamples, actSamples, "number of samples for series %q differs", key) - } - - for i, eS := range expSamples { - aS := actSamples[i] - - // Must use the interface as Equal does not work when actual types differ - // not only does the type differ, but chunk.Sample.FH() interface may auto convert from chunk.Sample.H()! - require.Equal(t, eS.T(), aS.T(), "timestamp of sample %d in series %q differs", i, key) - - require.Equal(t, eS.Type(), aS.Type(), "type of sample %d in series %q differs", i, key) - - switch eS.Type() { - case chunkenc.ValFloat: - require.Equal(t, eS.F(), aS.F(), "sample %d in series %q differs", i, key) - case chunkenc.ValHistogram: - eH, aH := eS.H(), aS.H() - if aH.CounterResetHint == histogram.UnknownCounterReset { - eH = eH.Copy() - // It is always safe to set the counter reset hint to UnknownCounterReset - eH.CounterResetHint = histogram.UnknownCounterReset - eS = sample{t: eS.T(), h: eH} - } - require.Equal(t, eH, aH, "histogram sample %d in series %q differs", i, key) - - case chunkenc.ValFloatHistogram: - eFH, aFH := eS.FH(), aS.FH() - if aFH.CounterResetHint == histogram.UnknownCounterReset { - eFH = eFH.Copy() - // It is always safe to set the counter reset hint to UnknownCounterReset - eFH.CounterResetHint = histogram.UnknownCounterReset - eS = sample{t: eS.T(), fh: eFH} - } - require.Equal(t, eFH, aFH, "float histogram sample %d in series %q differs", i, key) - } - } - } -} - // TestChunkQuerierReadWriteRace looks for any possible race between appending // samples and reading chunks because the head chunk that is being appended to // can be read in parallel and we should be able to make a copy of the chunk without // worrying about the parallel write. -func TestChunkQuerierReadWriteRace(t *testing.T) { +func TestChunkQuerierReadWriteRace_AppendV2(t *testing.T) { t.Parallel() db := newTestDB(t) @@ -8917,10 +7492,10 @@ func TestChunkQuerierReadWriteRace(t *testing.T) { <-time.After(5 * time.Millisecond) // Initial pause while readers start. ts := 0 for range 500 { - app := db.Appender(context.Background()) + app := db.AppenderV2(context.Background()) for range 10 { ts++ - _, err := app.Append(0, lbls, int64(ts), float64(ts*100)) + _, err := app.Append(0, lbls, 0, int64(ts), float64(ts*100), nil, nil, storage.AOptions{}) if err != nil { return err } @@ -8974,26 +7549,8 @@ Outer: require.NoError(t, writerErr) } -type mockCompactorFn struct { - planFn func() ([]string, error) - compactFn func() ([]ulid.ULID, error) - writeFn func() ([]ulid.ULID, error) -} - -func (c *mockCompactorFn) Plan(string) ([]string, error) { - return c.planFn() -} - -func (c *mockCompactorFn) Compact(string, []string, []*Block) ([]ulid.ULID, error) { - return c.compactFn() -} - -func (c *mockCompactorFn) Write(string, BlockReader, int64, int64, *BlockMeta) ([]ulid.ULID, error) { - return c.writeFn() -} - // Regression test for https://github.com/prometheus/prometheus/pull/13754 -func TestAbortBlockCompactions(t *testing.T) { +func TestAbortBlockCompactions_AppendV2(t *testing.T) { // Create a test DB db := newTestDB(t) // It should NOT be compactable at the beginning of the test @@ -9031,7 +7588,7 @@ func TestAbortBlockCompactions(t *testing.T) { require.Equal(t, 4, compactions, "expected 4 compactions to be completed") } -func TestNewCompactorFunc(t *testing.T) { +func TestNewCompactorFunc_AppendV2(t *testing.T) { opts := DefaultOptions() block1 := ulid.MustNew(1, nil) block2 := ulid.MustNew(2, nil) @@ -9062,225 +7619,3 @@ func TestNewCompactorFunc(t *testing.T) { require.Len(t, ulids, 1) require.Equal(t, block2, ulids[0]) } - -func TestBlockQuerierAndBlockChunkQuerier(t *testing.T) { - opts := DefaultOptions() - opts.BlockQuerierFunc = func(b BlockReader, mint, maxt int64) (storage.Querier, error) { - // Only block with hints can be queried. - if len(b.Meta().Compaction.Hints) > 0 { - return NewBlockQuerier(b, mint, maxt) - } - return storage.NoopQuerier(), nil - } - opts.BlockChunkQuerierFunc = func(b BlockReader, mint, maxt int64) (storage.ChunkQuerier, error) { - // Only level 4 compaction block can be queried. - if b.Meta().Compaction.Level == 4 { - return NewBlockChunkQuerier(b, mint, maxt) - } - return storage.NoopChunkedQuerier(), nil - } - - db := newTestDB(t, withOpts(opts)) - - metas := []BlockMeta{ - {Compaction: BlockMetaCompaction{Hints: []string{"test-hint"}}}, - {Compaction: BlockMetaCompaction{Level: 4}}, - } - for i := range metas { - // Include blockID into series to identify which block got touched. - serieses := []storage.Series{storage.NewListSeries(labels.FromMap(map[string]string{"block": fmt.Sprintf("block-%d", i), labels.MetricName: "test_metric"}), []chunks.Sample{sample{t: 0, f: 1}})} - blockDir := createBlock(t, db.Dir(), serieses) - b, err := OpenBlock(db.logger, blockDir, db.chunkPool, nil) - require.NoError(t, err) - - // Overwrite meta.json with compaction section for testing purpose. - b.meta.Compaction = metas[i].Compaction - _, err = writeMetaFile(db.logger, blockDir, &b.meta) - require.NoError(t, err) - require.NoError(t, b.Close()) - } - require.NoError(t, db.reloadBlocks()) - require.Len(t, db.Blocks(), 2) - - querier, err := db.Querier(0, 500) - require.NoError(t, err) - defer querier.Close() - matcher := labels.MustNewMatcher(labels.MatchEqual, labels.MetricName, "test_metric") - seriesSet := querier.Select(context.Background(), false, nil, matcher) - count := 0 - var lbls labels.Labels - for seriesSet.Next() { - count++ - lbls = seriesSet.At().Labels() - } - require.NoError(t, seriesSet.Err()) - require.Equal(t, 1, count) - // Make sure only block-0 is queried. - require.Equal(t, "block-0", lbls.Get("block")) - - chunkQuerier, err := db.ChunkQuerier(0, 500) - require.NoError(t, err) - defer chunkQuerier.Close() - css := chunkQuerier.Select(context.Background(), false, nil, matcher) - count = 0 - // Reset lbls variable. - lbls = labels.EmptyLabels() - for css.Next() { - count++ - lbls = css.At().Labels() - } - require.NoError(t, css.Err()) - require.Equal(t, 1, count) - // Make sure only block-1 is queried. - require.Equal(t, "block-1", lbls.Get("block")) -} - -func TestGenerateCompactionDelay(t *testing.T) { - assertDelay := func(delay time.Duration, expectedMaxPercentDelay int) { - t.Helper() - require.GreaterOrEqual(t, delay, time.Duration(0)) - // Expect to generate a delay up to MaxPercentDelay of the head chunk range - require.LessOrEqual(t, delay, (time.Duration(60000*expectedMaxPercentDelay/100) * time.Millisecond)) - } - - opts := DefaultOptions() - cases := []struct { - compactionDelayPercent int - }{ - { - compactionDelayPercent: 1, - }, - { - compactionDelayPercent: 10, - }, - { - compactionDelayPercent: 60, - }, - { - compactionDelayPercent: 100, - }, - } - - opts.EnableDelayedCompaction = true - - for _, c := range cases { - opts.CompactionDelayMaxPercent = c.compactionDelayPercent - db := newTestDB(t, withOpts(opts), withRngs(60000)) - - // The offset is generated and changed while opening. - assertDelay(db.opts.CompactionDelay, c.compactionDelayPercent) - - for range 1000 { - assertDelay(db.generateCompactionDelay(), c.compactionDelayPercent) - } - } -} - -type blockedResponseRecorder struct { - r *httptest.ResponseRecorder - - // writeBlocked is used to block writing until the test wants it to resume. - writeBlocked chan struct{} - // writeStarted is closed by blockedResponseRecorder to signal that writing has started. - writeStarted chan struct{} -} - -func (br *blockedResponseRecorder) Write(buf []byte) (int, error) { - select { - case <-br.writeStarted: - default: - close(br.writeStarted) - } - - <-br.writeBlocked - return br.r.Write(buf) -} - -func (br *blockedResponseRecorder) Header() http.Header { return br.r.Header() } - -func (br *blockedResponseRecorder) WriteHeader(code int) { br.r.WriteHeader(code) } - -func (br *blockedResponseRecorder) Flush() { br.r.Flush() } - -// TestBlockClosingBlockedDuringRemoteRead ensures that a TSDB Block is not closed while it is being queried -// through remote read. This is a regression test for https://github.com/prometheus/prometheus/issues/14422. -// TODO: Ideally, this should reside in storage/remote/read_handler_test.go once the necessary TSDB utils are accessible there. -func TestBlockClosingBlockedDuringRemoteRead(t *testing.T) { - dir := t.TempDir() - - createBlock(t, dir, genSeries(2, 1, 0, 10)) - - // Not using newTestDB as db.Close is expected to return error. - db, err := Open(dir, nil, nil, nil, nil) - require.NoError(t, err) - defer db.Close() - - readAPI := remote.NewReadHandler( - nil, nil, db, - func() config.Config { - return config.Config{} - }, 0, 1, 0, - ) - - matcher, err := labels.NewMatcher(labels.MatchRegexp, "__name__", ".*") - require.NoError(t, err) - - query, err := remote.ToQuery(0, 10, []*labels.Matcher{matcher}, nil) - require.NoError(t, err) - - req := &prompb.ReadRequest{ - Queries: []*prompb.Query{query}, - AcceptedResponseTypes: []prompb.ReadRequest_ResponseType{prompb.ReadRequest_STREAMED_XOR_CHUNKS}, - } - data, err := proto.Marshal(req) - require.NoError(t, err) - - request, err := http.NewRequest(http.MethodPost, "", bytes.NewBuffer(snappy.Encode(nil, data))) - require.NoError(t, err) - - blockedRecorder := &blockedResponseRecorder{ - r: httptest.NewRecorder(), - writeBlocked: make(chan struct{}), - writeStarted: make(chan struct{}), - } - - readDone := make(chan struct{}) - go func() { - readAPI.ServeHTTP(blockedRecorder, request) - require.Equal(t, http.StatusOK, blockedRecorder.r.Code) - close(readDone) - }() - - // Wait for the read API to start streaming data. - <-blockedRecorder.writeStarted - - // Try to close the queried block. - blockClosed := make(chan struct{}) - go func() { - for _, block := range db.Blocks() { - block.Close() - } - close(blockClosed) - }() - - // Closing the queried block should block. - // Wait a little bit to make sure of that. - select { - case <-time.After(100 * time.Millisecond): - case <-readDone: - require.Fail(t, "read API should still be streaming data.") - case <-blockClosed: - require.Fail(t, "Block shouldn't get closed while being queried.") - } - - // Resume the read API data streaming. - close(blockedRecorder.writeBlocked) - <-readDone - - // The block should be no longer needed and closing it should end. - select { - case <-time.After(10 * time.Millisecond): - require.Fail(t, "Closing the block timed out.") - case <-blockClosed: - } -}