From aa144b7263564b51117e702a0419ac16322dd9cc Mon Sep 17 00:00:00 2001 From: Carrie Edwards Date: Wed, 16 Oct 2024 09:18:23 -0700 Subject: [PATCH 01/40] Handle custom buckets in WAL and WBL --- tsdb/encoding/encoding.go | 8 +++ tsdb/head.go | 2 + tsdb/head_append.go | 36 ++++++++++++- tsdb/head_wal.go | 107 ++++++++++++++++++++++++++++++++++++- tsdb/record/record.go | 67 ++++++++++++++++++++++- tsdb/record/record_test.go | 29 ++++++++++ tsdb/wlog/checkpoint.go | 13 +++++ 7 files changed, 259 insertions(+), 3 deletions(-) diff --git a/tsdb/encoding/encoding.go b/tsdb/encoding/encoding.go index cc7d0990f6..c339a9a5bb 100644 --- a/tsdb/encoding/encoding.go +++ b/tsdb/encoding/encoding.go @@ -104,6 +104,14 @@ func (e *Encbuf) PutHashSum(h hash.Hash) { e.B = h.Sum(e.B) } +// IsWholeWhenMultiplied checks to see if the number when multiplied by 1000 can +// be converted into an integer without losing precision. +func IsWholeWhenMultiplied(in float64) bool { + i := uint(math.Round(in * 1000)) + out := float64(i) / 1000 + return in == out +} + // Decbuf provides safe methods to extract data from a byte slice. It does all // necessary bounds checking and advancing of the byte slice. // Several datums can be extracted without checking for errors. However, before using diff --git a/tsdb/head.go b/tsdb/head.go index c67c438e52..b7a358a6a6 100644 --- a/tsdb/head.go +++ b/tsdb/head.go @@ -87,6 +87,7 @@ type Head struct { logger *slog.Logger appendPool zeropool.Pool[[]record.RefSample] exemplarsPool zeropool.Pool[[]exemplarWithSeriesRef] + customValuesPool zeropool.Pool[[]record.RefCustomValues] histogramsPool zeropool.Pool[[]record.RefHistogramSample] floatHistogramsPool zeropool.Pool[[]record.RefFloatHistogramSample] metadataPool zeropool.Pool[[]record.RefMetadata] @@ -2134,6 +2135,7 @@ type memSeries struct { // We keep the last histogram value here (in addition to appending it to the chunk) so we can check for duplicates. lastHistogramValue *histogram.Histogram lastFloatHistogramValue *histogram.FloatHistogram + customValues []float64 // Current appender for the head chunk. Set when a new head chunk is cut. // It is nil only if headChunks is nil. E.g. if there was an appender that created a new series, but rolled back the commit diff --git a/tsdb/head_append.go b/tsdb/head_append.go index ea2a163f26..fbed0ee7eb 100644 --- a/tsdb/head_append.go +++ b/tsdb/head_append.go @@ -181,6 +181,7 @@ func (h *Head) appender() *headAppender { samples: h.getAppendBuffer(), sampleSeries: h.getSeriesBuffer(), exemplars: exemplarsBuf, + customValues: h.getCustomValuesBuffer(), histograms: h.getHistogramBuffer(), floatHistograms: h.getFloatHistogramBuffer(), metadata: h.getMetadataBuffer(), @@ -244,6 +245,18 @@ func (h *Head) putExemplarBuffer(b []exemplarWithSeriesRef) { h.exemplarsPool.Put(b[:0]) } +func (h *Head) getCustomValuesBuffer() []record.RefCustomValues { + b := h.customValuesPool.Get() + if b == nil { + return make([]record.RefCustomValues, 0, 512) + } + return b +} + +func (h *Head) putCustomValuesBuffer(b []record.RefCustomValues) { + h.customValuesPool.Put(b[:0]) +} + func (h *Head) getHistogramBuffer() []record.RefHistogramSample { b := h.histogramsPool.Get() if b == nil { @@ -326,6 +339,7 @@ type headAppender struct { 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). + customValues []record.RefCustomValues // Custom values for histograms that use custom buckets held by this appender. 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. @@ -690,7 +704,12 @@ func (a *headAppender) AppendHistogram(ref storage.SeriesRef, lset labels.Labels // This whole "if" should be removed. if created && s.lastHistogramValue == nil && s.lastFloatHistogramValue == nil { s.lastHistogramValue = &histogram.Histogram{} - } + if histogram.IsCustomBucketsSchema(h.Schema) { + a.customValues = append(a.customValues, record.RefCustomValues{ + Ref: s.ref, + CustomValues: h.CustomValues, + }) + } // 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. @@ -727,6 +746,12 @@ func (a *headAppender) AppendHistogram(ref storage.SeriesRef, lset labels.Labels // This whole "if" should be removed. if created && s.lastHistogramValue == nil && s.lastFloatHistogramValue == nil { s.lastFloatHistogramValue = &histogram.FloatHistogram{} + if histogram.IsCustomBucketsSchema(fh.Schema) { + a.customValues = append(a.customValues, record.RefCustomValues{ + Ref: s.ref, + CustomValues: fh.CustomValues, + }) + } } // TODO(codesome): If we definitely know at this point that the sample is ooo, then optimise @@ -942,6 +967,13 @@ func (a *headAppender) log() error { return fmt.Errorf("log samples: %w", err) } } + if len(a.customValues) > 0 { + rec = enc.CustomValues(a.customValues, buf) + buf = rec[:0] + if err := a.head.wal.Log(rec); err != nil { + return fmt.Errorf("log custom values: %w", err) + } + } if len(a.histograms) > 0 { rec = enc.HistogramSamples(a.histograms, buf) buf = rec[:0] @@ -1428,6 +1460,7 @@ func (a *headAppender) Commit() (err error) { defer a.head.putAppendBuffer(a.samples) defer a.head.putSeriesBuffer(a.sampleSeries) defer a.head.putExemplarBuffer(a.exemplars) + defer a.head.putCustomValuesBuffer(a.customValues) defer a.head.putHistogramBuffer(a.histograms) defer a.head.putFloatHistogramBuffer(a.floatHistograms) defer a.head.putMetadataBuffer(a.metadata) @@ -1949,6 +1982,7 @@ func (a *headAppender) Rollback() (err error) { } a.head.putAppendBuffer(a.samples) a.head.putExemplarBuffer(a.exemplars) + a.head.putCustomValuesBuffer(a.customValues) a.head.putHistogramBuffer(a.histograms) a.head.putFloatHistogramBuffer(a.floatHistograms) a.head.putMetadataBuffer(a.metadata) diff --git a/tsdb/head_wal.go b/tsdb/head_wal.go index 6744d582ae..637929428d 100644 --- a/tsdb/head_wal.go +++ b/tsdb/head_wal.go @@ -58,6 +58,7 @@ func (h *Head) loadWAL(r *wlog.Reader, syms *labels.SymbolTable, multiRef map[ch var unknownExemplarRefs atomic.Uint64 var unknownHistogramRefs atomic.Uint64 var unknownMetadataRefs atomic.Uint64 + var unknownCustomValuesRefs atomic.Uint64 // Track number of series records that had overlapping m-map chunks. var mmapOverlappingChunks atomic.Uint64 @@ -81,6 +82,7 @@ func (h *Head) loadWAL(r *wlog.Reader, syms *labels.SymbolTable, multiRef map[ch histogramsPool zeropool.Pool[[]record.RefHistogramSample] floatHistogramsPool zeropool.Pool[[]record.RefFloatHistogramSample] metadataPool zeropool.Pool[[]record.RefMetadata] + customValuesPool zeropool.Pool[[]record.RefCustomValues] ) defer func() { @@ -223,6 +225,18 @@ func (h *Head) loadWAL(r *wlog.Reader, syms *labels.SymbolTable, multiRef map[ch return } decoded <- meta + case record.CustomValues: + customVals := customValuesPool.Get()[:0] + customVals, err := dec.CustomValues(rec, customVals) + if err != nil { + decodeErr = &wlog.CorruptionErr{ + Err: fmt.Errorf("decode custom values: %w", err), + Segment: r.Segment(), + Offset: r.Offset(), + } + return + } + decoded <- customVals default: // Noop. } @@ -331,6 +345,19 @@ Outer: if r, ok := multiRef[sam.Ref]; ok { sam.Ref = r } + if histogram.IsCustomBucketsSchema(sam.H.Schema) { + ms := h.series.getByID(sam.Ref) + if ms == nil { + unknownHistogramRefs.Inc() + continue + } + + if ms.lastFloatHistogramValue != nil { + sam.H.CustomValues = ms.lastFloatHistogramValue.CustomValues + } else { + sam.H.CustomValues = ms.customValues + } + } mod := uint64(sam.Ref) % uint64(concurrency) histogramShards[mod] = append(histogramShards[mod], histogramRecord{ref: sam.Ref, t: sam.T, h: sam.H}) } @@ -367,6 +394,14 @@ Outer: if r, ok := multiRef[sam.Ref]; ok { sam.Ref = r } + if histogram.IsCustomBucketsSchema(sam.FH.Schema) { + ms := h.series.getByID(sam.Ref) + if ms == nil { + unknownHistogramRefs.Inc() + continue + } + sam.FH.CustomValues = ms.customValues + } mod := uint64(sam.Ref) % uint64(concurrency) histogramShards[mod] = append(histogramShards[mod], histogramRecord{ref: sam.Ref, t: sam.T, fh: sam.FH}) } @@ -393,6 +428,29 @@ Outer: } } metadataPool.Put(v) + case []record.RefCustomValues: + for _, cv := range v { + s := h.series.getByID(cv.Ref) + if s == nil { + unknownCustomValuesRefs.Inc() + continue + } + //TODO: do we actually want to check lastFloatHistogramValue? + if s.lastHistogramValue == nil && s.lastFloatHistogramValue == nil { + s.customValues = cv.CustomValues + } + } + customValuesPool.Put(v) + // iterate over custom value records and do same things as for series/samples - put them in correct processor + // processor depends on series ref + // something like this: + // idx := uint64(mSeries.ref) % uint64(concurrency) + // processors[idx].input <- walSubsetProcessorInputItem{customValues: } + //for _, cv := range v { + // idx := uint64(cv.Ref) % uint64(concurrency) + // processors[idx].input <- walSubsetProcessorInputItem{customValues: cv} + //} + //customValuesPool.Put(v) default: panic(fmt.Errorf("unexpected decoded type: %T", d)) } @@ -616,10 +674,25 @@ func (wp *walSubsetProcessor) processWALSamples(h *Head, mmappedChunks, oooMmapp if s.t <= ms.mmMaxTime { continue } + var chunkCreated bool if s.h != nil { + //if histogram.IsCustomBucketsSchema(s.h.Schema) { + // if ms.lastHistogramValue != nil { + // + // } + //} _, chunkCreated = ms.appendHistogram(s.t, s.h, 0, appendChunkOpts) } else { + //if histogram.IsCustomBucketsSchema(s.fh.Schema) { + // if ms.lastFloatHistogramValue != nil { + // s.h.CustomValues = ms.lastFloatHistogramValue.CustomValues + // } else { + // s.h.CustomValues = ms.customValues + // } + // //customVals := h. + // //s.h.CustomValues = + //} _, chunkCreated = ms.appendFloatHistogram(s.t, s.fh, 0, appendChunkOpts) } if chunkCreated { @@ -647,7 +720,7 @@ func (wp *walSubsetProcessor) processWALSamples(h *Head, mmappedChunks, oooMmapp func (h *Head) loadWBL(r *wlog.Reader, syms *labels.SymbolTable, multiRef map[chunks.HeadSeriesRef]chunks.HeadSeriesRef, lastMmapRef chunks.ChunkDiskMapperRef) (err error) { // Track number of samples, histogram samples, m-map markers, that referenced a series we don't know about // for error reporting. - var unknownRefs, unknownHistogramRefs, mmapMarkerUnknownRefs atomic.Uint64 + var unknownRefs, unknownHistogramRefs, unknownCustomValuesRefs, mmapMarkerUnknownRefs atomic.Uint64 lastSeq, lastOff := lastMmapRef.Unpack() // Start workers that each process samples for a partition of the series ID space. @@ -747,6 +820,18 @@ func (h *Head) loadWBL(r *wlog.Reader, syms *labels.SymbolTable, multiRef map[ch return } decodedCh <- hists + case record.CustomValues: + customVals := customValuesPool.Get().([]record.RefCustomValues)[:0] + customVals, err := dec.CustomValues(rec, customVals) + if err != nil { + decodeErr = &wlog.CorruptionErr{ + Err: fmt.Errorf("decode custom values: %w", err), + Segment: r.Segment(), + Offset: r.Offset(), + } + return + } + decodedCh <- customVals default: // Noop. } @@ -831,6 +916,18 @@ func (h *Head) loadWBL(r *wlog.Reader, syms *labels.SymbolTable, multiRef map[ch if r, ok := multiRef[sam.Ref]; ok { sam.Ref = r } + if histogram.IsCustomBucketsSchema(sam.H.Schema) { + ms := h.series.getByID(sam.Ref) + if ms == nil { + unknownHistogramRefs.Inc() + continue + } + if ms.lastFloatHistogramValue != nil { + sam.H.CustomValues = ms.lastFloatHistogramValue.CustomValues + } else { + sam.H.CustomValues = ms.customValues + } + } mod := uint64(sam.Ref) % uint64(concurrency) histogramShards[mod] = append(histogramShards[mod], histogramRecord{ref: sam.Ref, t: sam.T, h: sam.H}) } @@ -863,6 +960,14 @@ func (h *Head) loadWBL(r *wlog.Reader, syms *labels.SymbolTable, multiRef map[ch if r, ok := multiRef[sam.Ref]; ok { sam.Ref = r } + if histogram.IsCustomBucketsSchema(sam.FH.Schema) { + ms := h.series.getByID(sam.Ref) + if ms == nil { + unknownHistogramRefs.Inc() + continue + } + sam.FH.CustomValues = ms.customValues + } mod := uint64(sam.Ref) % uint64(concurrency) histogramShards[mod] = append(histogramShards[mod], histogramRecord{ref: sam.Ref, t: sam.T, fh: sam.FH}) } diff --git a/tsdb/record/record.go b/tsdb/record/record.go index 784d0b23d7..6f5fb24384 100644 --- a/tsdb/record/record.go +++ b/tsdb/record/record.go @@ -52,6 +52,7 @@ const ( HistogramSamples Type = 7 // FloatHistogramSamples is used to match WAL records of type Float Histograms. FloatHistogramSamples Type = 8 + CustomValues Type = 9 ) func (rt Type) String() string { @@ -72,6 +73,8 @@ func (rt Type) String() string { return "mmapmarkers" case Metadata: return "metadata" + case CustomValues: + return "custom_values" default: return "unknown" } @@ -147,6 +150,11 @@ type RefSeries struct { Labels labels.Labels } +type RefCustomValues struct { + Ref chunks.HeadSeriesRef + CustomValues []float64 +} + // RefSample is a timestamp/value pair associated with a reference to a series. // TODO(beorn7): Perhaps make this "polymorphic", including histogram and float-histogram pointers? Then get rid of RefHistogramSample. type RefSample struct { @@ -207,7 +215,7 @@ func (d *Decoder) Type(rec []byte) Type { return Unknown } switch t := Type(rec[0]); t { - case Series, Samples, Tombstones, Exemplars, MmapMarkers, Metadata, HistogramSamples, FloatHistogramSamples: + case Series, Samples, Tombstones, Exemplars, MmapMarkers, Metadata, HistogramSamples, FloatHistogramSamples, CustomValues: return t } return Unknown @@ -589,6 +597,39 @@ func DecodeFloatHistogram(buf *encoding.Decbuf, fh *histogram.FloatHistogram) { } } +// TODO: optimize +func (d *Decoder) CustomValues(rec []byte, customValues []RefCustomValues) ([]RefCustomValues, error) { + dec := encoding.Decbuf{B: rec} + + if Type(dec.Byte()) != CustomValues { + return nil, errors.New("invalid record type") + } + if dec.Len() == 0 { + return customValues, nil + } + for len(dec.B) > 0 && dec.Err() == nil { + ref := storage.SeriesRef(dec.Be64()) + l := dec.Uvarint() + if l > 0 { + vals := make([]float64, l) + for i := range vals { + vals[i] = dec.Be64Float64() + } + customValues = append(customValues, RefCustomValues{ + Ref: chunks.HeadSeriesRef(ref), + CustomValues: vals, + }) + } + } + if dec.Err() != nil { + return nil, dec.Err() + } + if len(dec.B) > 0 { + return nil, fmt.Errorf("unexpected %d bytes left in entry", len(dec.B)) + } + return customValues, nil +} + // Encoder encodes series, sample, and tombstones records. // The zero value is ready to use. type Encoder struct{} @@ -831,3 +872,27 @@ func EncodeFloatHistogram(buf *encoding.Encbuf, h *histogram.FloatHistogram) { buf.PutBEFloat64(b) } } + +func (e *Encoder) CustomValues(customValues []RefCustomValues, b []byte) []byte { + buf := encoding.Encbuf{B: b} + buf.PutByte(byte(CustomValues)) + + if len(customValues) == 0 { + return buf.Get() + } + + for _, v := range customValues { + buf.PutBE64(uint64(v.Ref)) + EncodeCustomValues(&buf, v.CustomValues) + } + + return buf.Get() +} + +// TODO: optimize +func EncodeCustomValues(buf *encoding.Encbuf, values []float64) { + buf.PutUvarint(len(values)) + for _, v := range values { + buf.PutBEFloat64(v) + } +} diff --git a/tsdb/record/record_test.go b/tsdb/record/record_test.go index f3a657aecb..c5f9f09f26 100644 --- a/tsdb/record/record_test.go +++ b/tsdb/record/record_test.go @@ -181,6 +181,22 @@ func TestRecord_EncodeDecode(t *testing.T) { decFloatHistograms, err = dec.FloatHistogramSamples(enc.FloatHistogramSamples(floatHistograms, nil), nil) require.NoError(t, err) require.Equal(t, floatHistograms, decFloatHistograms) + + // Custom values for histograms + customValues := []RefCustomValues{ + { + Ref: 56, + CustomValues: []float64{0, 1, 2, 3, 4}, + }, + { + Ref: 42, + CustomValues: []float64{5, 10, 15, 20, 25}, + }, + } + + decCustomValues, err := dec.CustomValues(enc.CustomValues(customValues, nil), nil) + require.NoError(t, err) + require.Equal(t, customValues, decCustomValues) } // TestRecord_Corrupted ensures that corrupted records return the correct error. @@ -269,6 +285,15 @@ func TestRecord_Corrupted(t *testing.T) { _, err := dec.HistogramSamples(corrupted, nil) require.ErrorIs(t, err, encoding.ErrInvalidSize) }) + + t.Run("Test corrupted customValues record", func(t *testing.T) { + customValues := []RefCustomValues{ + {Ref: 56, CustomValues: []float64{0, 1, 2, 3, 4}}, + } + corrupted := enc.CustomValues(customValues, nil)[:8] + _, err := dec.CustomValues(corrupted, nil) + require.ErrorIs(t, err, encoding.ErrInvalidSize) + }) } func TestRecord_Type(t *testing.T) { @@ -312,6 +337,10 @@ func TestRecord_Type(t *testing.T) { recordType = dec.Type(enc.HistogramSamples(histograms, nil)) require.Equal(t, HistogramSamples, recordType) + customValues := []RefCustomValues{{Ref: 56, CustomValues: []float64{0, 1, 2, 3, 4}}} + recordType = dec.Type(enc.CustomValues(customValues, nil)) + require.Equal(t, CustomValues, recordType) + recordType = dec.Type(nil) require.Equal(t, Unknown, recordType) diff --git a/tsdb/wlog/checkpoint.go b/tsdb/wlog/checkpoint.go index 58e11c770e..a96d142495 100644 --- a/tsdb/wlog/checkpoint.go +++ b/tsdb/wlog/checkpoint.go @@ -151,6 +151,7 @@ func Checkpoint(logger *slog.Logger, w *WL, from, to int, keep func(id chunks.He samples []record.RefSample histogramSamples []record.RefHistogramSample floatHistogramSamples []record.RefFloatHistogramSample + customValues []record.RefCustomValues tstones []tombstones.Stone exemplars []record.RefExemplar metadata []record.RefMetadata @@ -299,6 +300,18 @@ func Checkpoint(logger *slog.Logger, w *WL, from, to int, keep func(id chunks.He } stats.TotalMetadata += len(metadata) stats.DroppedMetadata += len(metadata) - repl + case record.CustomValues: + customValues, err = dec.CustomValues(rec, customValues) + if err != nil { + return nil, fmt.Errorf("decode custom values: %w", err) + } + repl := customValues[:0] + for _, v := range customValues { + repl = append(repl, v) + } + if len(repl) > 0 { + buf = enc.CustomValues(repl, buf) + } default: // Unknown record type, probably from a future Prometheus version. continue From 6d413fad361914372ccea6c99c74fb49fe1048d9 Mon Sep 17 00:00:00 2001 From: Carrie Edwards Date: Mon, 28 Oct 2024 08:43:00 -0700 Subject: [PATCH 02/40] Use histogram records for custom value handling --- tsdb/head_append.go | 37 +---------------- tsdb/head_wal.go | 72 +-------------------------------- tsdb/record/record.go | 81 +++++++++++++++----------------------- tsdb/record/record_test.go | 29 -------------- tsdb/wlog/checkpoint.go | 13 ------ 5 files changed, 34 insertions(+), 198 deletions(-) diff --git a/tsdb/head_append.go b/tsdb/head_append.go index fbed0ee7eb..7601f7847b 100644 --- a/tsdb/head_append.go +++ b/tsdb/head_append.go @@ -181,7 +181,6 @@ func (h *Head) appender() *headAppender { samples: h.getAppendBuffer(), sampleSeries: h.getSeriesBuffer(), exemplars: exemplarsBuf, - customValues: h.getCustomValuesBuffer(), histograms: h.getHistogramBuffer(), floatHistograms: h.getFloatHistogramBuffer(), metadata: h.getMetadataBuffer(), @@ -245,18 +244,6 @@ func (h *Head) putExemplarBuffer(b []exemplarWithSeriesRef) { h.exemplarsPool.Put(b[:0]) } -func (h *Head) getCustomValuesBuffer() []record.RefCustomValues { - b := h.customValuesPool.Get() - if b == nil { - return make([]record.RefCustomValues, 0, 512) - } - return b -} - -func (h *Head) putCustomValuesBuffer(b []record.RefCustomValues) { - h.customValuesPool.Put(b[:0]) -} - func (h *Head) getHistogramBuffer() []record.RefHistogramSample { b := h.histogramsPool.Get() if b == nil { @@ -339,7 +326,6 @@ type headAppender struct { 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). - customValues []record.RefCustomValues // Custom values for histograms that use custom buckets held by this appender. 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. @@ -704,13 +690,7 @@ func (a *headAppender) AppendHistogram(ref storage.SeriesRef, lset labels.Labels // This whole "if" should be removed. if created && s.lastHistogramValue == nil && s.lastFloatHistogramValue == nil { s.lastHistogramValue = &histogram.Histogram{} - if histogram.IsCustomBucketsSchema(h.Schema) { - a.customValues = append(a.customValues, record.RefCustomValues{ - Ref: s.ref, - CustomValues: h.CustomValues, - }) - } - + } // 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, a.head.opts.EnableOOONativeHistograms.Load()) @@ -746,12 +726,6 @@ func (a *headAppender) AppendHistogram(ref storage.SeriesRef, lset labels.Labels // This whole "if" should be removed. if created && s.lastHistogramValue == nil && s.lastFloatHistogramValue == nil { s.lastFloatHistogramValue = &histogram.FloatHistogram{} - if histogram.IsCustomBucketsSchema(fh.Schema) { - a.customValues = append(a.customValues, record.RefCustomValues{ - Ref: s.ref, - CustomValues: fh.CustomValues, - }) - } } // TODO(codesome): If we definitely know at this point that the sample is ooo, then optimise @@ -967,13 +941,6 @@ func (a *headAppender) log() error { return fmt.Errorf("log samples: %w", err) } } - if len(a.customValues) > 0 { - rec = enc.CustomValues(a.customValues, buf) - buf = rec[:0] - if err := a.head.wal.Log(rec); err != nil { - return fmt.Errorf("log custom values: %w", err) - } - } if len(a.histograms) > 0 { rec = enc.HistogramSamples(a.histograms, buf) buf = rec[:0] @@ -1460,7 +1427,6 @@ func (a *headAppender) Commit() (err error) { defer a.head.putAppendBuffer(a.samples) defer a.head.putSeriesBuffer(a.sampleSeries) defer a.head.putExemplarBuffer(a.exemplars) - defer a.head.putCustomValuesBuffer(a.customValues) defer a.head.putHistogramBuffer(a.histograms) defer a.head.putFloatHistogramBuffer(a.floatHistograms) defer a.head.putMetadataBuffer(a.metadata) @@ -1982,7 +1948,6 @@ func (a *headAppender) Rollback() (err error) { } a.head.putAppendBuffer(a.samples) a.head.putExemplarBuffer(a.exemplars) - a.head.putCustomValuesBuffer(a.customValues) a.head.putHistogramBuffer(a.histograms) a.head.putFloatHistogramBuffer(a.floatHistograms) a.head.putMetadataBuffer(a.metadata) diff --git a/tsdb/head_wal.go b/tsdb/head_wal.go index 637929428d..885d14a08b 100644 --- a/tsdb/head_wal.go +++ b/tsdb/head_wal.go @@ -58,7 +58,7 @@ func (h *Head) loadWAL(r *wlog.Reader, syms *labels.SymbolTable, multiRef map[ch var unknownExemplarRefs atomic.Uint64 var unknownHistogramRefs atomic.Uint64 var unknownMetadataRefs atomic.Uint64 - var unknownCustomValuesRefs atomic.Uint64 + // Track number of series records that had overlapping m-map chunks. var mmapOverlappingChunks atomic.Uint64 @@ -82,7 +82,6 @@ func (h *Head) loadWAL(r *wlog.Reader, syms *labels.SymbolTable, multiRef map[ch histogramsPool zeropool.Pool[[]record.RefHistogramSample] floatHistogramsPool zeropool.Pool[[]record.RefFloatHistogramSample] metadataPool zeropool.Pool[[]record.RefMetadata] - customValuesPool zeropool.Pool[[]record.RefCustomValues] ) defer func() { @@ -225,18 +224,6 @@ func (h *Head) loadWAL(r *wlog.Reader, syms *labels.SymbolTable, multiRef map[ch return } decoded <- meta - case record.CustomValues: - customVals := customValuesPool.Get()[:0] - customVals, err := dec.CustomValues(rec, customVals) - if err != nil { - decodeErr = &wlog.CorruptionErr{ - Err: fmt.Errorf("decode custom values: %w", err), - Segment: r.Segment(), - Offset: r.Offset(), - } - return - } - decoded <- customVals default: // Noop. } @@ -428,29 +415,6 @@ Outer: } } metadataPool.Put(v) - case []record.RefCustomValues: - for _, cv := range v { - s := h.series.getByID(cv.Ref) - if s == nil { - unknownCustomValuesRefs.Inc() - continue - } - //TODO: do we actually want to check lastFloatHistogramValue? - if s.lastHistogramValue == nil && s.lastFloatHistogramValue == nil { - s.customValues = cv.CustomValues - } - } - customValuesPool.Put(v) - // iterate over custom value records and do same things as for series/samples - put them in correct processor - // processor depends on series ref - // something like this: - // idx := uint64(mSeries.ref) % uint64(concurrency) - // processors[idx].input <- walSubsetProcessorInputItem{customValues: } - //for _, cv := range v { - // idx := uint64(cv.Ref) % uint64(concurrency) - // processors[idx].input <- walSubsetProcessorInputItem{customValues: cv} - //} - //customValuesPool.Put(v) default: panic(fmt.Errorf("unexpected decoded type: %T", d)) } @@ -720,7 +684,7 @@ func (wp *walSubsetProcessor) processWALSamples(h *Head, mmappedChunks, oooMmapp func (h *Head) loadWBL(r *wlog.Reader, syms *labels.SymbolTable, multiRef map[chunks.HeadSeriesRef]chunks.HeadSeriesRef, lastMmapRef chunks.ChunkDiskMapperRef) (err error) { // Track number of samples, histogram samples, m-map markers, that referenced a series we don't know about // for error reporting. - var unknownRefs, unknownHistogramRefs, unknownCustomValuesRefs, mmapMarkerUnknownRefs atomic.Uint64 + var unknownRefs, unknownHistogramRefs, mmapMarkerUnknownRefs atomic.Uint64 lastSeq, lastOff := lastMmapRef.Unpack() // Start workers that each process samples for a partition of the series ID space. @@ -820,18 +784,6 @@ func (h *Head) loadWBL(r *wlog.Reader, syms *labels.SymbolTable, multiRef map[ch return } decodedCh <- hists - case record.CustomValues: - customVals := customValuesPool.Get().([]record.RefCustomValues)[:0] - customVals, err := dec.CustomValues(rec, customVals) - if err != nil { - decodeErr = &wlog.CorruptionErr{ - Err: fmt.Errorf("decode custom values: %w", err), - Segment: r.Segment(), - Offset: r.Offset(), - } - return - } - decodedCh <- customVals default: // Noop. } @@ -916,18 +868,6 @@ func (h *Head) loadWBL(r *wlog.Reader, syms *labels.SymbolTable, multiRef map[ch if r, ok := multiRef[sam.Ref]; ok { sam.Ref = r } - if histogram.IsCustomBucketsSchema(sam.H.Schema) { - ms := h.series.getByID(sam.Ref) - if ms == nil { - unknownHistogramRefs.Inc() - continue - } - if ms.lastFloatHistogramValue != nil { - sam.H.CustomValues = ms.lastFloatHistogramValue.CustomValues - } else { - sam.H.CustomValues = ms.customValues - } - } mod := uint64(sam.Ref) % uint64(concurrency) histogramShards[mod] = append(histogramShards[mod], histogramRecord{ref: sam.Ref, t: sam.T, h: sam.H}) } @@ -960,14 +900,6 @@ func (h *Head) loadWBL(r *wlog.Reader, syms *labels.SymbolTable, multiRef map[ch if r, ok := multiRef[sam.Ref]; ok { sam.Ref = r } - if histogram.IsCustomBucketsSchema(sam.FH.Schema) { - ms := h.series.getByID(sam.Ref) - if ms == nil { - unknownHistogramRefs.Inc() - continue - } - sam.FH.CustomValues = ms.customValues - } mod := uint64(sam.Ref) % uint64(concurrency) histogramShards[mod] = append(histogramShards[mod], histogramRecord{ref: sam.Ref, t: sam.T, fh: sam.FH}) } diff --git a/tsdb/record/record.go b/tsdb/record/record.go index 6f5fb24384..d759c18551 100644 --- a/tsdb/record/record.go +++ b/tsdb/record/record.go @@ -513,6 +513,18 @@ func DecodeHistogram(buf *encoding.Decbuf, h *histogram.Histogram) { for i := range h.NegativeBuckets { h.NegativeBuckets[i] = buf.Varint64() } + + if histogram.IsCustomBucketsSchema(h.Schema) { + l = buf.Uvarint() + if l > 0 { + if l > 0 { + h.CustomValues = make([]float64, l) + } + for i := range h.CustomValues { + h.CustomValues[i] = buf.Be64Float64() + } + } + } } func (d *Decoder) FloatHistogramSamples(rec []byte, histograms []RefFloatHistogramSample) ([]RefFloatHistogramSample, error) { @@ -595,39 +607,18 @@ func DecodeFloatHistogram(buf *encoding.Decbuf, fh *histogram.FloatHistogram) { for i := range fh.NegativeBuckets { fh.NegativeBuckets[i] = buf.Be64Float64() } -} -// TODO: optimize -func (d *Decoder) CustomValues(rec []byte, customValues []RefCustomValues) ([]RefCustomValues, error) { - dec := encoding.Decbuf{B: rec} - - if Type(dec.Byte()) != CustomValues { - return nil, errors.New("invalid record type") - } - if dec.Len() == 0 { - return customValues, nil - } - for len(dec.B) > 0 && dec.Err() == nil { - ref := storage.SeriesRef(dec.Be64()) - l := dec.Uvarint() + if histogram.IsCustomBucketsSchema(fh.Schema) { + l = buf.Uvarint() if l > 0 { - vals := make([]float64, l) - for i := range vals { - vals[i] = dec.Be64Float64() + if l > 0 { + fh.CustomValues = make([]float64, l) + } + for i := range fh.CustomValues { + fh.CustomValues[i] = buf.Be64Float64() } - customValues = append(customValues, RefCustomValues{ - Ref: chunks.HeadSeriesRef(ref), - CustomValues: vals, - }) } } - if dec.Err() != nil { - return nil, dec.Err() - } - if len(dec.B) > 0 { - return nil, fmt.Errorf("unexpected %d bytes left in entry", len(dec.B)) - } - return customValues, nil } // Encoder encodes series, sample, and tombstones records. @@ -813,6 +804,13 @@ func EncodeHistogram(buf *encoding.Encbuf, h *histogram.Histogram) { for _, b := range h.NegativeBuckets { buf.PutVarint64(b) } + + if histogram.IsCustomBucketsSchema(h.Schema) { + buf.PutUvarint(len(h.CustomValues)) + for _, v := range h.CustomValues { + buf.PutBEFloat64(v) + } + } } func (e *Encoder) FloatHistogramSamples(histograms []RefFloatHistogramSample, b []byte) []byte { @@ -871,28 +869,11 @@ func EncodeFloatHistogram(buf *encoding.Encbuf, h *histogram.FloatHistogram) { for _, b := range h.NegativeBuckets { buf.PutBEFloat64(b) } -} - -func (e *Encoder) CustomValues(customValues []RefCustomValues, b []byte) []byte { - buf := encoding.Encbuf{B: b} - buf.PutByte(byte(CustomValues)) - - if len(customValues) == 0 { - return buf.Get() - } - - for _, v := range customValues { - buf.PutBE64(uint64(v.Ref)) - EncodeCustomValues(&buf, v.CustomValues) - } - return buf.Get() -} - -// TODO: optimize -func EncodeCustomValues(buf *encoding.Encbuf, values []float64) { - buf.PutUvarint(len(values)) - for _, v := range values { - buf.PutBEFloat64(v) + if histogram.IsCustomBucketsSchema(h.Schema) { + buf.PutUvarint(len(h.CustomValues)) + for _, v := range h.CustomValues { + buf.PutBEFloat64(v) + } } } diff --git a/tsdb/record/record_test.go b/tsdb/record/record_test.go index c5f9f09f26..f3a657aecb 100644 --- a/tsdb/record/record_test.go +++ b/tsdb/record/record_test.go @@ -181,22 +181,6 @@ func TestRecord_EncodeDecode(t *testing.T) { decFloatHistograms, err = dec.FloatHistogramSamples(enc.FloatHistogramSamples(floatHistograms, nil), nil) require.NoError(t, err) require.Equal(t, floatHistograms, decFloatHistograms) - - // Custom values for histograms - customValues := []RefCustomValues{ - { - Ref: 56, - CustomValues: []float64{0, 1, 2, 3, 4}, - }, - { - Ref: 42, - CustomValues: []float64{5, 10, 15, 20, 25}, - }, - } - - decCustomValues, err := dec.CustomValues(enc.CustomValues(customValues, nil), nil) - require.NoError(t, err) - require.Equal(t, customValues, decCustomValues) } // TestRecord_Corrupted ensures that corrupted records return the correct error. @@ -285,15 +269,6 @@ func TestRecord_Corrupted(t *testing.T) { _, err := dec.HistogramSamples(corrupted, nil) require.ErrorIs(t, err, encoding.ErrInvalidSize) }) - - t.Run("Test corrupted customValues record", func(t *testing.T) { - customValues := []RefCustomValues{ - {Ref: 56, CustomValues: []float64{0, 1, 2, 3, 4}}, - } - corrupted := enc.CustomValues(customValues, nil)[:8] - _, err := dec.CustomValues(corrupted, nil) - require.ErrorIs(t, err, encoding.ErrInvalidSize) - }) } func TestRecord_Type(t *testing.T) { @@ -337,10 +312,6 @@ func TestRecord_Type(t *testing.T) { recordType = dec.Type(enc.HistogramSamples(histograms, nil)) require.Equal(t, HistogramSamples, recordType) - customValues := []RefCustomValues{{Ref: 56, CustomValues: []float64{0, 1, 2, 3, 4}}} - recordType = dec.Type(enc.CustomValues(customValues, nil)) - require.Equal(t, CustomValues, recordType) - recordType = dec.Type(nil) require.Equal(t, Unknown, recordType) diff --git a/tsdb/wlog/checkpoint.go b/tsdb/wlog/checkpoint.go index a96d142495..58e11c770e 100644 --- a/tsdb/wlog/checkpoint.go +++ b/tsdb/wlog/checkpoint.go @@ -151,7 +151,6 @@ func Checkpoint(logger *slog.Logger, w *WL, from, to int, keep func(id chunks.He samples []record.RefSample histogramSamples []record.RefHistogramSample floatHistogramSamples []record.RefFloatHistogramSample - customValues []record.RefCustomValues tstones []tombstones.Stone exemplars []record.RefExemplar metadata []record.RefMetadata @@ -300,18 +299,6 @@ func Checkpoint(logger *slog.Logger, w *WL, from, to int, keep func(id chunks.He } stats.TotalMetadata += len(metadata) stats.DroppedMetadata += len(metadata) - repl - case record.CustomValues: - customValues, err = dec.CustomValues(rec, customValues) - if err != nil { - return nil, fmt.Errorf("decode custom values: %w", err) - } - repl := customValues[:0] - for _, v := range customValues { - repl = append(repl, v) - } - if len(repl) > 0 { - buf = enc.CustomValues(repl, buf) - } default: // Unknown record type, probably from a future Prometheus version. continue From cfcd51538dede02375a02aaf478e7fc7ac299890 Mon Sep 17 00:00:00 2001 From: Carrie Edwards Date: Mon, 28 Oct 2024 09:49:08 -0700 Subject: [PATCH 03/40] Remove references to custom values record --- tsdb/head.go | 2 -- tsdb/head_append.go | 11 ----------- tsdb/head_wal.go | 35 ----------------------------------- tsdb/record/record.go | 10 +--------- 4 files changed, 1 insertion(+), 57 deletions(-) diff --git a/tsdb/head.go b/tsdb/head.go index b7a358a6a6..c67c438e52 100644 --- a/tsdb/head.go +++ b/tsdb/head.go @@ -87,7 +87,6 @@ type Head struct { logger *slog.Logger appendPool zeropool.Pool[[]record.RefSample] exemplarsPool zeropool.Pool[[]exemplarWithSeriesRef] - customValuesPool zeropool.Pool[[]record.RefCustomValues] histogramsPool zeropool.Pool[[]record.RefHistogramSample] floatHistogramsPool zeropool.Pool[[]record.RefFloatHistogramSample] metadataPool zeropool.Pool[[]record.RefMetadata] @@ -2135,7 +2134,6 @@ type memSeries struct { // We keep the last histogram value here (in addition to appending it to the chunk) so we can check for duplicates. lastHistogramValue *histogram.Histogram lastFloatHistogramValue *histogram.FloatHistogram - customValues []float64 // Current appender for the head chunk. Set when a new head chunk is cut. // It is nil only if headChunks is nil. E.g. if there was an appender that created a new series, but rolled back the commit diff --git a/tsdb/head_append.go b/tsdb/head_append.go index 7601f7847b..7dacb9037b 100644 --- a/tsdb/head_append.go +++ b/tsdb/head_append.go @@ -1458,17 +1458,6 @@ func (a *headAppender) Commit() (err error) { a.commitFloatHistograms(acc) a.commitMetadata() - a.head.metrics.outOfOrderSamples.WithLabelValues(sampleMetricTypeFloat).Add(float64(acc.floatOOORejected)) - a.head.metrics.outOfOrderSamples.WithLabelValues(sampleMetricTypeHistogram).Add(float64(acc.histoOOORejected)) - a.head.metrics.outOfBoundSamples.WithLabelValues(sampleMetricTypeFloat).Add(float64(acc.floatOOBRejected)) - a.head.metrics.tooOldSamples.WithLabelValues(sampleMetricTypeFloat).Add(float64(acc.floatTooOldRejected)) - a.head.metrics.samplesAppended.WithLabelValues(sampleMetricTypeFloat).Add(float64(acc.floatsAppended)) - a.head.metrics.samplesAppended.WithLabelValues(sampleMetricTypeHistogram).Add(float64(acc.histogramsAppended)) - a.head.metrics.outOfOrderSamplesAppended.WithLabelValues(sampleMetricTypeFloat).Add(float64(acc.oooFloatsAccepted)) - a.head.metrics.outOfOrderSamplesAppended.WithLabelValues(sampleMetricTypeHistogram).Add(float64(acc.oooHistogramAccepted)) - a.head.updateMinMaxTime(acc.inOrderMint, acc.inOrderMaxt) - a.head.updateMinOOOMaxOOOTime(acc.oooMinT, acc.oooMaxT) - acc.collectOOORecords(a) if a.head.wbl != nil { if err := a.head.wbl.Log(acc.oooRecords...); err != nil { diff --git a/tsdb/head_wal.go b/tsdb/head_wal.go index 885d14a08b..dd4f4f8b17 100644 --- a/tsdb/head_wal.go +++ b/tsdb/head_wal.go @@ -332,19 +332,6 @@ Outer: if r, ok := multiRef[sam.Ref]; ok { sam.Ref = r } - if histogram.IsCustomBucketsSchema(sam.H.Schema) { - ms := h.series.getByID(sam.Ref) - if ms == nil { - unknownHistogramRefs.Inc() - continue - } - - if ms.lastFloatHistogramValue != nil { - sam.H.CustomValues = ms.lastFloatHistogramValue.CustomValues - } else { - sam.H.CustomValues = ms.customValues - } - } mod := uint64(sam.Ref) % uint64(concurrency) histogramShards[mod] = append(histogramShards[mod], histogramRecord{ref: sam.Ref, t: sam.T, h: sam.H}) } @@ -381,14 +368,6 @@ Outer: if r, ok := multiRef[sam.Ref]; ok { sam.Ref = r } - if histogram.IsCustomBucketsSchema(sam.FH.Schema) { - ms := h.series.getByID(sam.Ref) - if ms == nil { - unknownHistogramRefs.Inc() - continue - } - sam.FH.CustomValues = ms.customValues - } mod := uint64(sam.Ref) % uint64(concurrency) histogramShards[mod] = append(histogramShards[mod], histogramRecord{ref: sam.Ref, t: sam.T, fh: sam.FH}) } @@ -641,22 +620,8 @@ func (wp *walSubsetProcessor) processWALSamples(h *Head, mmappedChunks, oooMmapp var chunkCreated bool if s.h != nil { - //if histogram.IsCustomBucketsSchema(s.h.Schema) { - // if ms.lastHistogramValue != nil { - // - // } - //} _, chunkCreated = ms.appendHistogram(s.t, s.h, 0, appendChunkOpts) } else { - //if histogram.IsCustomBucketsSchema(s.fh.Schema) { - // if ms.lastFloatHistogramValue != nil { - // s.h.CustomValues = ms.lastFloatHistogramValue.CustomValues - // } else { - // s.h.CustomValues = ms.customValues - // } - // //customVals := h. - // //s.h.CustomValues = - //} _, chunkCreated = ms.appendFloatHistogram(s.t, s.fh, 0, appendChunkOpts) } if chunkCreated { diff --git a/tsdb/record/record.go b/tsdb/record/record.go index d759c18551..83adecdbb4 100644 --- a/tsdb/record/record.go +++ b/tsdb/record/record.go @@ -52,7 +52,6 @@ const ( HistogramSamples Type = 7 // FloatHistogramSamples is used to match WAL records of type Float Histograms. FloatHistogramSamples Type = 8 - CustomValues Type = 9 ) func (rt Type) String() string { @@ -73,8 +72,6 @@ func (rt Type) String() string { return "mmapmarkers" case Metadata: return "metadata" - case CustomValues: - return "custom_values" default: return "unknown" } @@ -150,11 +147,6 @@ type RefSeries struct { Labels labels.Labels } -type RefCustomValues struct { - Ref chunks.HeadSeriesRef - CustomValues []float64 -} - // RefSample is a timestamp/value pair associated with a reference to a series. // TODO(beorn7): Perhaps make this "polymorphic", including histogram and float-histogram pointers? Then get rid of RefHistogramSample. type RefSample struct { @@ -215,7 +207,7 @@ func (d *Decoder) Type(rec []byte) Type { return Unknown } switch t := Type(rec[0]); t { - case Series, Samples, Tombstones, Exemplars, MmapMarkers, Metadata, HistogramSamples, FloatHistogramSamples, CustomValues: + case Series, Samples, Tombstones, Exemplars, MmapMarkers, Metadata, HistogramSamples, FloatHistogramSamples: return t } return Unknown From 37df50adb9365b415e224128e02b2f5f606a71c2 Mon Sep 17 00:00:00 2001 From: Carrie Edwards Date: Wed, 13 Nov 2024 14:20:11 -0800 Subject: [PATCH 04/40] Attempt for record type --- tsdb/agent/db.go | 41 +++-- tsdb/agent/db_test.go | 7 +- tsdb/docs/format/wal.md | 26 +++ tsdb/encoding/encoding.go | 8 - tsdb/head_append.go | 46 ++++- tsdb/head_test.go | 313 +++++++++++++++++++++++++++++++++++ tsdb/head_wal.go | 7 +- tsdb/record/record.go | 108 +++++++++--- tsdb/record/record_test.go | 71 +++++++- tsdb/testutil.go | 34 +++- tsdb/tsdbutil/histogram.go | 23 +++ tsdb/wlog/checkpoint.go | 40 ++++- tsdb/wlog/checkpoint_test.go | 10 +- tsdb/wlog/watcher_test.go | 6 +- 14 files changed, 658 insertions(+), 82 deletions(-) diff --git a/tsdb/agent/db.go b/tsdb/agent/db.go index 3863e6cd99..5067edc3ae 100644 --- a/tsdb/agent/db.go +++ b/tsdb/agent/db.go @@ -463,7 +463,7 @@ func (db *DB) loadWAL(r *wlog.Reader, multiRef map[chunks.HeadSeriesRef]chunks.H return } decoded <- samples - case record.HistogramSamples: + case record.HistogramSamples, record.CustomBucketHistogramSamples: histograms := histogramsPool.Get()[:0] histograms, err = dec.HistogramSamples(rec, histograms) if err != nil { @@ -475,7 +475,7 @@ func (db *DB) loadWAL(r *wlog.Reader, multiRef map[chunks.HeadSeriesRef]chunks.H return } decoded <- histograms - case record.FloatHistogramSamples: + case record.FloatHistogramSamples, record.CustomBucketFloatHistogramSamples: floatHistograms := floatHistogramsPool.Get()[:0] floatHistograms, err = dec.FloatHistogramSamples(rec, floatHistograms) if err != nil { @@ -1154,19 +1154,40 @@ func (a *appender) log() error { } if len(a.pendingHistograms) > 0 { - buf = encoder.HistogramSamples(a.pendingHistograms, buf) - if err := a.wal.Log(buf); err != nil { - return err + buf1, buf2 := encoder.HistogramSamples(a.pendingHistograms, buf) + //buf = append(buf1, buf2...) + //if err := a.wal.Log(buf); err != nil { + // return err + //} + if len(buf1) > 0 { + buf = buf1[:0] + if err := a.wal.Log(buf1); err != nil { + return err + } + } + if len(buf2) > 0 { + buf = buf2[:0] + if err := a.wal.Log(buf2); err != nil { + return err + } } - buf = buf[:0] } if len(a.pendingFloatHistograms) > 0 { - buf = encoder.FloatHistogramSamples(a.pendingFloatHistograms, buf) - if err := a.wal.Log(buf); err != nil { - return err + buf1, buf2 := encoder.FloatHistogramSamples(a.pendingFloatHistograms, buf) + if len(buf1) > 0 { + buf = buf1[:0] + if err := a.wal.Log(buf1); err != nil { + return err + } } - buf = buf[:0] + if len(buf2) > 0 { + buf = buf2[:0] + if err := a.wal.Log(buf2); err != nil { + return err + } + } + //buf = buf[:0] } if len(a.pendingExamplars) > 0 { diff --git a/tsdb/agent/db_test.go b/tsdb/agent/db_test.go index b28c29095c..5332c61cdb 100644 --- a/tsdb/agent/db_test.go +++ b/tsdb/agent/db_test.go @@ -193,7 +193,8 @@ func TestCommit(t *testing.T) { ) for r.Next() { rec := r.Record() - switch dec.Type(rec) { + recType := dec.Type(rec) + switch recType { case record.Series: var series []record.RefSeries series, err = dec.Series(rec, series) @@ -206,13 +207,13 @@ func TestCommit(t *testing.T) { require.NoError(t, err) walSamplesCount += len(samples) - case record.HistogramSamples: + case record.HistogramSamples, record.CustomBucketHistogramSamples: var histograms []record.RefHistogramSample histograms, err = dec.HistogramSamples(rec, histograms) require.NoError(t, err) walHistogramCount += len(histograms) - case record.FloatHistogramSamples: + case record.FloatHistogramSamples, record.CustomBucketFloatHistogramSamples: var floatHistograms []record.RefFloatHistogramSample floatHistograms, err = dec.FloatHistogramSamples(rec, floatHistograms) require.NoError(t, err) diff --git a/tsdb/docs/format/wal.md b/tsdb/docs/format/wal.md index db1ce97a8b..835ede4113 100644 --- a/tsdb/docs/format/wal.md +++ b/tsdb/docs/format/wal.md @@ -79,6 +79,32 @@ The first sample record begins at the second row. └──────────────────────────────────────────────────────────────────┘ ``` +### Native histogram records + +Native histogram records are encoded as + +``` +┌──────────────────────────────────────────────────────────────────┐ +│ type = 2 <1b> │ +├──────────────────────────────────────────────────────────────────┤ +│ ┌────────────────────┬───────────────────────────┐ │ +│ │ id <8b> │ timestamp <8b> │ │ +│ └────────────────────┴───────────────────────────┘ │ +│ ┌────────────────────┬───────────────────────────┬ │ +│ │ id_delta │ timestamp_delta │ │ +│ ├────────────────────┴───────────────────────────┴─────────────┤ │ +│ │ n = len(labels) │ │ +│ ├──────────────────────┬───────────────────────────────────────┤ │ +│ │ len(str_1) │ str_1 │ │ +│ ├──────────────────────┴───────────────────────────────────────┤ │ +│ │ ... │ │ +│ ├───────────────────────┬──────────────────────────────────────┤ │ +│ │ len(str_2n) │ str_2n │ │ │ +│ └───────────────────────┴────────────────┴─────────────────────┘ │ +│ . . . │ +└──────────────────────────────────────────────────────────────────┘ +``` + ### Tombstone records Tombstone records encode tombstones as a list of triples `(series_id, min_time, max_time)` diff --git a/tsdb/encoding/encoding.go b/tsdb/encoding/encoding.go index c339a9a5bb..cc7d0990f6 100644 --- a/tsdb/encoding/encoding.go +++ b/tsdb/encoding/encoding.go @@ -104,14 +104,6 @@ func (e *Encbuf) PutHashSum(h hash.Hash) { e.B = h.Sum(e.B) } -// IsWholeWhenMultiplied checks to see if the number when multiplied by 1000 can -// be converted into an integer without losing precision. -func IsWholeWhenMultiplied(in float64) bool { - i := uint(math.Round(in * 1000)) - out := float64(i) / 1000 - return in == out -} - // Decbuf provides safe methods to extract data from a byte slice. It does all // necessary bounds checking and advancing of the byte slice. // Several datums can be extracted without checking for errors. However, before using diff --git a/tsdb/head_append.go b/tsdb/head_append.go index 7dacb9037b..3701a57135 100644 --- a/tsdb/head_append.go +++ b/tsdb/head_append.go @@ -942,17 +942,47 @@ func (a *headAppender) log() error { } } if len(a.histograms) > 0 { - rec = enc.HistogramSamples(a.histograms, buf) - buf = rec[:0] - if err := a.head.wal.Log(rec); err != nil { - return fmt.Errorf("log histograms: %w", err) + rec1, rec2 := enc.HistogramSamples(a.histograms, buf) + //rec = append(rec1, rec2...) + // + //buf = rec[:0] + // + //if err := a.head.wal.Log(rec); err != nil { + // return fmt.Errorf("log samples: %w", err) + //} + if len(rec1) != 0 { + buf = rec1[:0] + if err := a.head.wal.Log(rec1); err != nil { + return fmt.Errorf("log histograms: %w", err) + } + } + if len(rec2) != 0 { + buf = rec2[:0] + if err := a.head.wal.Log(rec2); err != nil { + return fmt.Errorf("log custom bucket histograms: %w", err) + } } } if len(a.floatHistograms) > 0 { - rec = enc.FloatHistogramSamples(a.floatHistograms, buf) - buf = rec[:0] - if err := a.head.wal.Log(rec); err != nil { - return fmt.Errorf("log float histograms: %w", err) + rec1, rec2 := enc.FloatHistogramSamples(a.floatHistograms, buf) + //rec = append(rec1, rec2...) + // + //buf = rec[:0] + // + //if err := a.head.wal.Log(rec); err != nil { + // return fmt.Errorf("log samples: %w", err) + //} + if len(rec1) != 0 { + buf = rec1[:0] + if err := a.head.wal.Log(rec1); err != nil { + return fmt.Errorf("log float histograms: %w", err) + } + } + if len(rec2) != 0 { + buf = rec2[:0] + if err := a.head.wal.Log(rec2); err != nil { + return fmt.Errorf("log custom bucket float histograms: %w", err) + } } } // Exemplars should be logged after samples (float/native histogram/etc), diff --git a/tsdb/head_test.go b/tsdb/head_test.go index 2ca3aeffc7..527476e113 100644 --- a/tsdb/head_test.go +++ b/tsdb/head_test.go @@ -740,6 +740,89 @@ func TestHead_ReadWAL(t *testing.T) { } } +func TestHead_ReadWAL2(t *testing.T) { + for _, compress := range []wlog.CompressionType{wlog.CompressionNone, wlog.CompressionSnappy, wlog.CompressionZstd} { + t.Run(fmt.Sprintf("compress=%s", compress), func(t *testing.T) { + entries := []interface{}{ + []record.RefSeries{ + {Ref: 10, Labels: labels.FromStrings("a", "1")}, + {Ref: 11, Labels: labels.FromStrings("a", "2")}, + {Ref: 100, Labels: labels.FromStrings("a", "3")}, + }, + []record.RefHistogramSample{ + {Ref: 0, T: 99, H: tsdbutil.GenerateTestHistogram(1)}, + {Ref: 10, T: 100, H: tsdbutil.GenerateTestCustomBucketsHistogram(2)}, + {Ref: 100, T: 100, H: tsdbutil.GenerateTestHistogram(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.RefHistogramSample{ + {Ref: 10, T: 101, H: tsdbutil.GenerateTestHistogram(5)}, + {Ref: 50, T: 101, H: tsdbutil.GenerateTestHistogram(6)}, + {Ref: 101, T: 101, H: tsdbutil.GenerateTestCustomBucketsHistogram(7)}, + }, + []tombstones.Stone{ + {Ref: 0, Intervals: []tombstones.Interval{{Mint: 99, Maxt: 101}}}, + }, + []record.RefExemplar{ + {Ref: 10, T: 100, V: 1, Labels: labels.FromStrings("trace_id", "asdf")}, + }, + } + + head, w := newTestHead(t, 1000, compress, false) + defer func() { + require.NoError(t, head.Close()) + }() + + populateTestWL(t, w, entries) + + 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) + + 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) + + expandChunk := func(c chunkenc.Iterator) (x []sample) { + for c.Next() == chunkenc.ValHistogram { + t, v := c.AtHistogram(nil) + //t, v := c.At() + x = append(x, sample{t: t, h: v}) + } + require.NoError(t, c.Err()) + return x + } + + c, _, _, err := s10.chunk(0, head.chunkDiskMapper, &head.memChunkPool) + require.NoError(t, err) + require.Equal(t, []sample{{100, 0, tsdbutil.GenerateTestCustomBucketsHistogram(2), nil}, {101, 0, tsdbutil.GenerateTestCustomBucketsHistogram(5), nil}}, expandChunk(c.chunk.Iterator(nil))) + c, _, _, err = s50.chunk(0, head.chunkDiskMapper, &head.memChunkPool) + require.NoError(t, err) + require.Equal(t, []sample{{101, 0, tsdbutil.GenerateTestHistogram(6), nil}}, expandChunk(c.chunk.Iterator(nil))) + // 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, 0, tsdbutil.GenerateTestCustomBucketsHistogram(7), 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.True(t, exemplar.Exemplar{Ts: 100, Value: 1, Labels: labels.FromStrings("trace_id", "asdf")}.Equals(e[0].Exemplars[0])) + }) + } +} + func TestHead_WALMultiRef(t *testing.T) { head, w := newTestHead(t, 1000, wlog.CompressionNone, false) @@ -3953,6 +4036,194 @@ func TestHistogramInWALAndMmapChunk(t *testing.T) { testQuery() } +func TestHistogramInWALAndMmapChunk2(t *testing.T) { + head, _ := newTestHead(t, 3000, wlog.CompressionNone, 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 _, custom := range []bool{true, false} { + app = head.Appender(context.Background()) + var hists []*histogram.Histogram + if custom { + hists = tsdbutil.GenerateTestCustomBucketsHistograms(numHistograms) + } else { + hists = tsdbutil.GenerateTestHistograms(numHistograms) + } + for _, h := range hists { + if !custom { + 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 _, custom := range []bool{true, false} { + app = head.Appender(context.Background()) + var hists []*histogram.FloatHistogram + if custom { + hists = tsdbutil.GenerateTestCustomBucketsFloatHistograms(numHistograms) + } else { + hists = tsdbutil.GenerateTestFloatHistograms(numHistograms) + } + for _, h := range hists { + if !custom { + 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, 19) + 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 _, custom := range []bool{true, false} { + app = head.Appender(context.Background()) + var hists []*histogram.Histogram + if custom { + hists = tsdbutil.GenerateTestCustomBucketsHistograms(100) + } else { + hists = tsdbutil.GenerateTestHistograms(100) + } + for _, h := range hists { + ts++ + if !custom { + h.NegativeSpans = h.PositiveSpans + h.NegativeBuckets = h.PositiveBuckets + } + _, err := app.AppendHistogram(0, s2, ts, h, nil) + require.NoError(t, err) + eh := h.Copy() + if 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 i := 0; i < 10; i++ { + 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 _, custom := range []bool{true, false} { + app = head.Appender(context.Background()) + var hists []*histogram.FloatHistogram + if custom { + hists = tsdbutil.GenerateTestCustomBucketsFloatHistograms(100) + } else { + hists = tsdbutil.GenerateTestFloatHistograms(100) + } + for _, h := range hists { + ts++ + if !custom { + h.NegativeSpans = h.PositiveSpans + h.NegativeBuckets = h.PositiveBuckets + } + _, err := app.AppendHistogram(0, s2, ts, nil, h) + require.NoError(t, err) + eh := h.Copy() + if 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 i := 0; i < 10; i++ { + 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, wlog.CompressionNone) + 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, wlog.CompressionNone, false) defer func() { @@ -5089,6 +5360,48 @@ func TestChunkSnapshotTakenAfterIncompleteSnapshot(t *testing.T) { require.Positive(t, offset) } +func TestHistogramWALANDWBLReplay(t *testing.T) { + dir := t.TempDir() + wal, err := wlog.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, wlog.CompressionSnappy) + require.NoError(t, err) + oooWlog, err := wlog.NewSize(nil, nil, filepath.Join(dir, wlog.WblDirName), 32768, wlog.CompressionSnappy) + require.NoError(t, err) + + opts := DefaultHeadOptions() + opts.ChunkRange = 1000 + opts.ChunkDirRoot = dir + opts.OutOfOrderTimeWindow.Store(30 * time.Minute.Milliseconds()) + opts.EnableNativeHistograms.Store(true) + opts.EnableOOONativeHistograms.Store(true) + + 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, val float64, isOOO bool, isCustomBucketHistogram bool) { + app := h.Appender(context.Background()) + var s sample + if isCustomBucketHistogram { + s = sample{t: mins * time.Minute.Milliseconds(), h: tsdbutil.GenerateTestCustomBucketsHistogram(int(val))} + } else { + s = sample{t: mins * time.Minute.Milliseconds(), h: tsdbutil.GenerateTestHistogram(int(val))} + } + _, err := app.AppendHistogram(0, l, mins*time.Minute.Milliseconds(), s.h, nil) + require.NoError(t, err) + require.NoError(t, app.Commit()) + + if isOOO { + expOOOSamples = append(expOOOSamples, s) + } + } + + // In-order histogram samples. + appendSample(60, 60, false, false) + +} + // TestWBLReplay checks the replay at a low level. func TestWBLReplay(t *testing.T) { for name, scenario := range sampleTypeScenarios { diff --git a/tsdb/head_wal.go b/tsdb/head_wal.go index dd4f4f8b17..9d1e24b706 100644 --- a/tsdb/head_wal.go +++ b/tsdb/head_wal.go @@ -139,7 +139,8 @@ func (h *Head) loadWAL(r *wlog.Reader, syms *labels.SymbolTable, multiRef map[ch dec := record.NewDecoder(syms) for r.Next() { rec := r.Record() - switch dec.Type(rec) { + recType := dec.Type(rec) + switch recType { case record.Series: series := seriesPool.Get()[:0] series, err = dec.Series(rec, series) @@ -188,7 +189,7 @@ func (h *Head) loadWAL(r *wlog.Reader, syms *labels.SymbolTable, multiRef map[ch return } decoded <- exemplars - case record.HistogramSamples: + case record.HistogramSamples, record.CustomBucketHistogramSamples: hists := histogramsPool.Get()[:0] hists, err = dec.HistogramSamples(rec, hists) if err != nil { @@ -200,7 +201,7 @@ func (h *Head) loadWAL(r *wlog.Reader, syms *labels.SymbolTable, multiRef map[ch return } decoded <- hists - case record.FloatHistogramSamples: + case record.FloatHistogramSamples, record.CustomBucketFloatHistogramSamples: hists := floatHistogramsPool.Get()[:0] hists, err = dec.FloatHistogramSamples(rec, hists) if err != nil { diff --git a/tsdb/record/record.go b/tsdb/record/record.go index 83adecdbb4..15f5053d53 100644 --- a/tsdb/record/record.go +++ b/tsdb/record/record.go @@ -52,6 +52,10 @@ const ( HistogramSamples Type = 7 // FloatHistogramSamples is used to match WAL records of type Float Histograms. FloatHistogramSamples Type = 8 + // CustomBucketHistogramSamples is used to match WAL records of type Histogram with custom buckets. + CustomBucketHistogramSamples Type = 9 + // CustomBucketFloatHistogramSamples is used to match WAL records of type Float Histogram with custom buckets. + CustomBucketFloatHistogramSamples Type = 10 ) func (rt Type) String() string { @@ -68,6 +72,10 @@ func (rt Type) String() string { return "histogram_samples" case FloatHistogramSamples: return "float_histogram_samples" + case CustomBucketHistogramSamples: + return "custom_bucket_histogram_samples" + case CustomBucketFloatHistogramSamples: + return "custom_bucket_float_histogram_samples" case MmapMarkers: return "mmapmarkers" case Metadata: @@ -185,6 +193,10 @@ type RefFloatHistogramSample struct { FH *histogram.FloatHistogram } +type RefCustomBucketHistogramSample struct { + RefHistogramSample +} + // RefMmapMarker marks that the all the samples of the given series until now have been m-mapped to disk. type RefMmapMarker struct { Ref chunks.HeadSeriesRef @@ -207,7 +219,7 @@ func (d *Decoder) Type(rec []byte) Type { return Unknown } switch t := Type(rec[0]); t { - case Series, Samples, Tombstones, Exemplars, MmapMarkers, Metadata, HistogramSamples, FloatHistogramSamples: + case Series, Samples, Tombstones, Exemplars, MmapMarkers, Metadata, HistogramSamples, FloatHistogramSamples, CustomBucketHistogramSamples, CustomBucketFloatHistogramSamples: return t } return Unknown @@ -428,7 +440,7 @@ func (d *Decoder) MmapMarkers(rec []byte, markers []RefMmapMarker) ([]RefMmapMar func (d *Decoder) HistogramSamples(rec []byte, histograms []RefHistogramSample) ([]RefHistogramSample, error) { dec := encoding.Decbuf{B: rec} t := Type(dec.Byte()) - if t != HistogramSamples { + if t != HistogramSamples && t != CustomBucketHistogramSamples { return nil, errors.New("invalid record type") } if dec.Len() == 0 { @@ -509,12 +521,10 @@ func DecodeHistogram(buf *encoding.Decbuf, h *histogram.Histogram) { if histogram.IsCustomBucketsSchema(h.Schema) { l = buf.Uvarint() if l > 0 { - if l > 0 { - h.CustomValues = make([]float64, l) - } - for i := range h.CustomValues { - h.CustomValues[i] = buf.Be64Float64() - } + h.CustomValues = make([]float64, l) + } + for i := range h.CustomValues { + h.CustomValues[i] = buf.Be64Float64() } } } @@ -522,7 +532,7 @@ func DecodeHistogram(buf *encoding.Decbuf, h *histogram.Histogram) { func (d *Decoder) FloatHistogramSamples(rec []byte, histograms []RefFloatHistogramSample) ([]RefFloatHistogramSample, error) { dec := encoding.Decbuf{B: rec} t := Type(dec.Byte()) - if t != FloatHistogramSamples { + if t != FloatHistogramSamples && t != CustomBucketFloatHistogramSamples { return nil, errors.New("invalid record type") } if dec.Len() == 0 { @@ -603,12 +613,10 @@ func DecodeFloatHistogram(buf *encoding.Decbuf, fh *histogram.FloatHistogram) { if histogram.IsCustomBucketsSchema(fh.Schema) { l = buf.Uvarint() if l > 0 { - if l > 0 { - fh.CustomValues = make([]float64, l) - } - for i := range fh.CustomValues { - fh.CustomValues[i] = buf.Be64Float64() - } + fh.CustomValues = make([]float64, l) + } + for i := range fh.CustomValues { + fh.CustomValues[i] = buf.Be64Float64() } } } @@ -740,12 +748,15 @@ func (e *Encoder) MmapMarkers(markers []RefMmapMarker, b []byte) []byte { return buf.Get() } -func (e *Encoder) HistogramSamples(histograms []RefHistogramSample, b []byte) []byte { +func (e *Encoder) HistogramSamples(histograms []RefHistogramSample, b []byte) ([]byte, []byte) { buf := encoding.Encbuf{B: b} buf.PutByte(byte(HistogramSamples)) + customBucketHistBuf := encoding.Encbuf{B: b} + customBucketHistBuf.PutByte(byte(CustomBucketHistogramSamples)) + if len(histograms) == 0 { - return buf.Get() + return buf.Get(), customBucketHistBuf.Get() } // Store base timestamp and base reference number of first histogram. @@ -754,14 +765,34 @@ func (e *Encoder) HistogramSamples(histograms []RefHistogramSample, b []byte) [] buf.PutBE64(uint64(first.Ref)) buf.PutBE64int64(first.T) + customBucketHistBuf.PutBE64(uint64(first.Ref)) + customBucketHistBuf.PutBE64int64(first.T) + + histsAdded := 0 + customBucketHistsAdded := 0 for _, h := range histograms { - buf.PutVarint64(int64(h.Ref) - int64(first.Ref)) - buf.PutVarint64(h.T - first.T) + if h.H.UsesCustomBuckets() { + customBucketHistBuf.PutVarint64(int64(h.Ref) - int64(first.Ref)) + customBucketHistBuf.PutVarint64(h.T - first.T) + + EncodeHistogram(&customBucketHistBuf, h.H) + customBucketHistsAdded++ + } else { + buf.PutVarint64(int64(h.Ref) - int64(first.Ref)) + buf.PutVarint64(h.T - first.T) + + EncodeHistogram(&buf, h.H) + histsAdded++ + } + } - EncodeHistogram(&buf, h.H) + if customBucketHistsAdded == 0 { + customBucketHistBuf.Reset() + } else if histsAdded == 0 { + buf.Reset() } - return buf.Get() + return buf.Get(), customBucketHistBuf.Get() } // EncodeHistogram encodes a Histogram into a byte slice. @@ -805,12 +836,15 @@ func EncodeHistogram(buf *encoding.Encbuf, h *histogram.Histogram) { } } -func (e *Encoder) FloatHistogramSamples(histograms []RefFloatHistogramSample, b []byte) []byte { +func (e *Encoder) FloatHistogramSamples(histograms []RefFloatHistogramSample, b []byte) ([]byte, []byte) { buf := encoding.Encbuf{B: b} buf.PutByte(byte(FloatHistogramSamples)) + customBucketHistBuf := encoding.Encbuf{B: b} + customBucketHistBuf.PutByte(byte(CustomBucketFloatHistogramSamples)) + if len(histograms) == 0 { - return buf.Get() + return buf.Get(), customBucketHistBuf.Get() } // Store base timestamp and base reference number of first histogram. @@ -819,14 +853,34 @@ func (e *Encoder) FloatHistogramSamples(histograms []RefFloatHistogramSample, b buf.PutBE64(uint64(first.Ref)) buf.PutBE64int64(first.T) + customBucketHistBuf.PutBE64(uint64(first.Ref)) + customBucketHistBuf.PutBE64int64(first.T) + + histsAdded := 0 + customBucketHistsAdded := 0 for _, h := range histograms { - buf.PutVarint64(int64(h.Ref) - int64(first.Ref)) - buf.PutVarint64(h.T - first.T) + if h.FH.UsesCustomBuckets() { + customBucketHistBuf.PutVarint64(int64(h.Ref) - int64(first.Ref)) + customBucketHistBuf.PutVarint64(h.T - first.T) + + EncodeFloatHistogram(&customBucketHistBuf, h.FH) + customBucketHistsAdded++ + } else { + buf.PutVarint64(int64(h.Ref) - int64(first.Ref)) + buf.PutVarint64(h.T - first.T) + + EncodeFloatHistogram(&buf, h.FH) + histsAdded++ + } + } - EncodeFloatHistogram(&buf, h.FH) + if customBucketHistsAdded == 0 { + customBucketHistBuf.Reset() + } else if histsAdded == 0 { + buf.Reset() } - return buf.Get() + return buf.Get(), customBucketHistBuf.Get() } // EncodeFloatHistogram encodes the Float Histogram into a byte slice. diff --git a/tsdb/record/record_test.go b/tsdb/record/record_test.go index f3a657aecb..67c7eab970 100644 --- a/tsdb/record/record_test.go +++ b/tsdb/record/record_test.go @@ -148,10 +148,30 @@ func TestRecord_EncodeDecode(t *testing.T) { NegativeBuckets: []int64{1, 2, -1}, }, }, + { + Ref: 67, + T: 5678, + H: &histogram.Histogram{ + Count: 8, + ZeroThreshold: 0.001, + Sum: 35.5, + Schema: -53, + PositiveSpans: []histogram.Span{ + {Offset: 0, Length: 2}, + {Offset: 2, Length: 2}, + }, + PositiveBuckets: []int64{2, -1, 2, 0}, + CustomValues: []float64{0, 2, 4, 6, 8}, + }, + }, } - decHistograms, err := dec.HistogramSamples(enc.HistogramSamples(histograms, nil), nil) + histSamples, customBucketHistSamples := enc.HistogramSamples(histograms, nil) + decHistograms, err := dec.HistogramSamples(histSamples, nil) require.NoError(t, err) + decCustomBucketHistograms, err := dec.HistogramSamples(customBucketHistSamples, nil) + require.NoError(t, err) + decHistograms = append(decHistograms, decCustomBucketHistograms...) require.Equal(t, histograms, decHistograms) floatHistograms := make([]RefFloatHistogramSample, len(histograms)) @@ -162,24 +182,36 @@ func TestRecord_EncodeDecode(t *testing.T) { FH: h.H.ToFloat(nil), } } - decFloatHistograms, err := dec.FloatHistogramSamples(enc.FloatHistogramSamples(floatHistograms, nil), nil) + histSamples, customBucketFloatHistSamples := enc.FloatHistogramSamples(floatHistograms, nil) + decFloatHistograms, err := dec.FloatHistogramSamples(histSamples, nil) require.NoError(t, err) + decCustomBucketFloatHistograms, err := dec.FloatHistogramSamples(customBucketFloatHistSamples, nil) + decFloatHistograms = append(decFloatHistograms, decCustomBucketFloatHistograms...) require.Equal(t, floatHistograms, decFloatHistograms) // Gauge integer histograms. for i := range histograms { histograms[i].H.CounterResetHint = histogram.GaugeType } - decHistograms, err = dec.HistogramSamples(enc.HistogramSamples(histograms, nil), nil) + + gaugeHistSamples, customBucketGaugeHistSamples := enc.HistogramSamples(histograms, nil) + decGaugeHistograms, err := dec.HistogramSamples(gaugeHistSamples, nil) require.NoError(t, err) - require.Equal(t, histograms, decHistograms) + decCustomBucketGaugeHistograms, err := dec.HistogramSamples(customBucketGaugeHistSamples, nil) + require.NoError(t, err) + decGaugeHistograms = append(decGaugeHistograms, decCustomBucketGaugeHistograms...) + require.Equal(t, histograms, decGaugeHistograms) // Gauge float histograms. for i := range floatHistograms { floatHistograms[i].FH.CounterResetHint = histogram.GaugeType } - decFloatHistograms, err = dec.FloatHistogramSamples(enc.FloatHistogramSamples(floatHistograms, nil), nil) + + gaugeHistSamples, customBucketGaugeFloatHistSamples := enc.FloatHistogramSamples(floatHistograms, nil) + decGaugeFloatHistograms, err := dec.FloatHistogramSamples(gaugeHistSamples, nil) require.NoError(t, err) + decCustomBucketGaugeFloatHistograms, err := dec.FloatHistogramSamples(customBucketGaugeFloatHistSamples, nil) + decFloatHistograms = append(decGaugeFloatHistograms, decCustomBucketGaugeFloatHistograms...) require.Equal(t, floatHistograms, decFloatHistograms) } @@ -265,8 +297,12 @@ func TestRecord_Corrupted(t *testing.T) { }, } - corrupted := enc.HistogramSamples(histograms, nil)[:8] - _, err := dec.HistogramSamples(corrupted, nil) + corruptedHists, corruptedCustomBucketHists := enc.HistogramSamples(histograms, nil) + corruptedHists = corruptedHists[:8] + corruptedCustomBucketHists = corruptedCustomBucketHists[:8] + _, err := dec.HistogramSamples(corruptedHists, nil) + require.ErrorIs(t, err, encoding.ErrInvalidSize) + _, err = dec.HistogramSamples(corruptedCustomBucketHists, nil) require.ErrorIs(t, err, encoding.ErrInvalidSize) }) } @@ -308,9 +344,28 @@ func TestRecord_Type(t *testing.T) { PositiveBuckets: []int64{1, 1, -1, 0}, }, }, + { + Ref: 67, + T: 5678, + H: &histogram.Histogram{ + Count: 8, + ZeroThreshold: 0.001, + Sum: 35.5, + Schema: -53, + PositiveSpans: []histogram.Span{ + {Offset: 0, Length: 2}, + {Offset: 2, Length: 2}, + }, + PositiveBuckets: []int64{2, -1, 2, 0}, + CustomValues: []float64{0, 2, 4, 6, 8}, + }, + }, } - recordType = dec.Type(enc.HistogramSamples(histograms, nil)) + hists, customBucketHists := enc.HistogramSamples(histograms, nil) + recordType = dec.Type(hists) require.Equal(t, HistogramSamples, recordType) + recordType = dec.Type(customBucketHists) + require.Equal(t, CustomBucketHistogramSamples, recordType) recordType = dec.Type(nil) require.Equal(t, Unknown, recordType) diff --git a/tsdb/testutil.go b/tsdb/testutil.go index c39eb133c7..a13d89186e 100644 --- a/tsdb/testutil.go +++ b/tsdb/testutil.go @@ -29,11 +29,13 @@ import ( ) const ( - float = "float" - intHistogram = "integer histogram" - floatHistogram = "float histogram" - gaugeIntHistogram = "gauge int histogram" - gaugeFloatHistogram = "gauge float histogram" + float = "float" + intHistogram = "integer histogram" + floatHistogram = "float histogram" + customBucketIntHistogram = "custom bucket int histogram" + customBucketFloatHistogram = "custom bucket float histogram" + gaugeIntHistogram = "gauge int histogram" + gaugeFloatHistogram = "gauge float histogram" ) type testValue struct { @@ -82,6 +84,28 @@ var sampleTypeScenarios = map[string]sampleTypeScenario{ return sample{t: ts, fh: tsdbutil.GenerateTestFloatHistogram(int(value))} }, }, + customBucketIntHistogram: { + sampleType: sampleMetricTypeHistogram, + appendFunc: func(appender storage.Appender, lbls labels.Labels, ts, value int64) (storage.SeriesRef, sample, error) { + s := sample{t: ts, h: tsdbutil.GenerateTestCustomBucketsHistogram(int(value))} + ref, err := appender.AppendHistogram(0, lbls, ts, s.h, nil) + return ref, s, err + }, + sampleFunc: func(ts, value int64) sample { + return sample{t: ts, h: tsdbutil.GenerateTestCustomBucketsHistogram(int(value))} + }, + }, + customBucketFloatHistogram: { + sampleType: sampleMetricTypeHistogram, + appendFunc: func(appender storage.Appender, lbls labels.Labels, ts, value int64) (storage.SeriesRef, sample, error) { + s := sample{t: ts, fh: tsdbutil.GenerateTestCustomBucketsFloatHistogram(int(value))} + ref, err := appender.AppendHistogram(0, lbls, ts, nil, s.fh) + return ref, s, err + }, + sampleFunc: func(ts, value int64) sample { + return sample{t: ts, fh: tsdbutil.GenerateTestCustomBucketsFloatHistogram(int(value))} + }, + }, gaugeIntHistogram: { sampleType: sampleMetricTypeHistogram, appendFunc: func(appender storage.Appender, lbls labels.Labels, ts, value int64) (storage.SeriesRef, sample, error) { diff --git a/tsdb/tsdbutil/histogram.go b/tsdb/tsdbutil/histogram.go index ce934a638d..4b6cebd579 100644 --- a/tsdb/tsdbutil/histogram.go +++ b/tsdb/tsdbutil/histogram.go @@ -57,6 +57,18 @@ func GenerateTestHistogram(i int) *histogram.Histogram { } } +func GenerateTestCustomBucketsHistograms(n int) (r []*histogram.Histogram) { + for i := 0; i < n; i++ { + h := GenerateTestCustomBucketsHistogram(i) + if i > 0 { + h.CounterResetHint = histogram.NotCounterReset + } + r = append(r, h) + + } + return r +} + func GenerateTestCustomBucketsHistogram(i int) *histogram.Histogram { return &histogram.Histogram{ Count: 5 + uint64(i*4), @@ -117,6 +129,17 @@ func GenerateTestFloatHistogram(i int) *histogram.FloatHistogram { } } +func GenerateTestCustomBucketsFloatHistograms(n int) (r []*histogram.FloatHistogram) { + for i := 0; i < n; i++ { + h := GenerateTestCustomBucketsFloatHistogram(i) + if i > 0 { + h.CounterResetHint = histogram.NotCounterReset + } + r = append(r, h) + } + return r +} + func GenerateTestCustomBucketsFloatHistogram(i int) *histogram.FloatHistogram { return &histogram.FloatHistogram{ Count: 5 + float64(i*4), diff --git a/tsdb/wlog/checkpoint.go b/tsdb/wlog/checkpoint.go index 58e11c770e..cd82676da9 100644 --- a/tsdb/wlog/checkpoint.go +++ b/tsdb/wlog/checkpoint.go @@ -221,11 +221,27 @@ func Checkpoint(logger *slog.Logger, w *WL, from, to int, keep func(id chunks.He } } if len(repl) > 0 { - buf = enc.HistogramSamples(repl, buf) + buf, _ = enc.HistogramSamples(repl, buf) + } + stats.TotalSamples += len(histogramSamples) + stats.DroppedSamples += len(histogramSamples) - len(repl) + case record.CustomBucketHistogramSamples: + histogramSamples, err = dec.HistogramSamples(rec, histogramSamples) + if err != nil { + return nil, fmt.Errorf("decode histogram samples: %w", err) + } + // Drop irrelevant histogramSamples in place. + repl := histogramSamples[:0] + for _, h := range histogramSamples { + if h.T >= mint { + repl = append(repl, h) + } + } + if len(repl) > 0 { + _, buf = enc.HistogramSamples(repl, buf) } stats.TotalSamples += len(histogramSamples) stats.DroppedSamples += len(histogramSamples) - len(repl) - case record.FloatHistogramSamples: floatHistogramSamples, err = dec.FloatHistogramSamples(rec, floatHistogramSamples) if err != nil { @@ -239,11 +255,27 @@ func Checkpoint(logger *slog.Logger, w *WL, from, to int, keep func(id chunks.He } } if len(repl) > 0 { - buf = enc.FloatHistogramSamples(repl, buf) + buf, _ = enc.FloatHistogramSamples(repl, buf) + } + stats.TotalSamples += len(floatHistogramSamples) + stats.DroppedSamples += len(floatHistogramSamples) - len(repl) + case record.CustomBucketFloatHistogramSamples: + floatHistogramSamples, err = dec.FloatHistogramSamples(rec, floatHistogramSamples) + if err != nil { + return nil, fmt.Errorf("decode float histogram samples: %w", err) + } + // Drop irrelevant floatHistogramSamples in place. + repl := floatHistogramSamples[:0] + for _, fh := range floatHistogramSamples { + if fh.T >= mint { + repl = append(repl, fh) + } + } + if len(repl) > 0 { + _, buf = enc.FloatHistogramSamples(repl, buf) } stats.TotalSamples += len(floatHistogramSamples) stats.DroppedSamples += len(floatHistogramSamples) - len(repl) - case record.Tombstones: tstones, err = dec.Tombstones(rec, tstones) if err != nil { diff --git a/tsdb/wlog/checkpoint_test.go b/tsdb/wlog/checkpoint_test.go index 8ee193f5ac..a5692a9aa4 100644 --- a/tsdb/wlog/checkpoint_test.go +++ b/tsdb/wlog/checkpoint_test.go @@ -208,22 +208,24 @@ func TestCheckpoint(t *testing.T) { require.NoError(t, w.Log(b)) samplesInWAL += 4 h := makeHistogram(i) - b = enc.HistogramSamples([]record.RefHistogramSample{ + b1, b2 := enc.HistogramSamples([]record.RefHistogramSample{ {Ref: 0, T: last, H: h}, {Ref: 1, T: last + 10000, H: h}, {Ref: 2, T: last + 20000, H: h}, {Ref: 3, T: last + 30000, H: h}, }, nil) - require.NoError(t, w.Log(b)) + require.NoError(t, w.Log(b1)) + require.NoError(t, w.Log(b2)) histogramsInWAL += 4 fh := makeFloatHistogram(i) - b = enc.FloatHistogramSamples([]record.RefFloatHistogramSample{ + b1, b2 = enc.FloatHistogramSamples([]record.RefFloatHistogramSample{ {Ref: 0, T: last, FH: fh}, {Ref: 1, T: last + 10000, FH: fh}, {Ref: 2, T: last + 20000, FH: fh}, {Ref: 3, T: last + 30000, FH: fh}, }, nil) - require.NoError(t, w.Log(b)) + require.NoError(t, w.Log(b1)) + require.NoError(t, w.Log(b2)) floatHistogramsInWAL += 4 b = enc.Exemplars([]record.RefExemplar{ diff --git a/tsdb/wlog/watcher_test.go b/tsdb/wlog/watcher_test.go index 398b0f4414..c2499a7cec 100644 --- a/tsdb/wlog/watcher_test.go +++ b/tsdb/wlog/watcher_test.go @@ -209,19 +209,21 @@ func TestTailSamples(t *testing.T) { NegativeBuckets: []int64{int64(-i) - 1}, } - histogram := enc.HistogramSamples([]record.RefHistogramSample{{ + histogram, customBucketHistogram := enc.HistogramSamples([]record.RefHistogramSample{{ Ref: chunks.HeadSeriesRef(inner), T: now.UnixNano() + 1, H: hist, }}, nil) require.NoError(t, w.Log(histogram)) + require.NoError(t, w.Log(customBucketHistogram)) - floatHistogram := enc.FloatHistogramSamples([]record.RefFloatHistogramSample{{ + floatHistogram, floatCustomBucketHistogram := enc.FloatHistogramSamples([]record.RefFloatHistogramSample{{ Ref: chunks.HeadSeriesRef(inner), T: now.UnixNano() + 1, FH: hist.ToFloat(nil), }}, nil) require.NoError(t, w.Log(floatHistogram)) + require.NoError(t, w.Log(floatCustomBucketHistogram)) } } From 454f6d39ca5748aed08b110ad3f2eb027bff8969 Mon Sep 17 00:00:00 2001 From: Carrie Edwards Date: Thu, 14 Nov 2024 10:00:55 -0800 Subject: [PATCH 05/40] Add separate handling for histograms and custom bucket histograms --- tsdb/agent/db.go | 41 ++++++------- tsdb/agent/db_test.go | 30 ++++++++- tsdb/head_append.go | 49 ++++++--------- tsdb/record/record.go | 114 ++++++++++++++++++++--------------- tsdb/record/record_test.go | 45 ++++++++++---- tsdb/wlog/checkpoint.go | 4 +- tsdb/wlog/checkpoint_test.go | 60 +++++++++++++++--- tsdb/wlog/watcher.go | 4 +- tsdb/wlog/watcher_test.go | 36 ++++++++--- 9 files changed, 245 insertions(+), 138 deletions(-) diff --git a/tsdb/agent/db.go b/tsdb/agent/db.go index 5067edc3ae..bcfc7be129 100644 --- a/tsdb/agent/db.go +++ b/tsdb/agent/db.go @@ -1154,40 +1154,35 @@ func (a *appender) log() error { } if len(a.pendingHistograms) > 0 { - buf1, buf2 := encoder.HistogramSamples(a.pendingHistograms, buf) - //buf = append(buf1, buf2...) - //if err := a.wal.Log(buf); err != nil { - // return err - //} - if len(buf1) > 0 { - buf = buf1[:0] - if err := a.wal.Log(buf1); err != nil { - return err - } + var customBucketsExist bool + buf, customBucketsExist = encoder.HistogramSamples(a.pendingHistograms, buf) + if err := a.wal.Log(buf); err != nil { + return err } - if len(buf2) > 0 { - buf = buf2[:0] - if err := a.wal.Log(buf2); err != nil { + buf = buf[:0] + if customBucketsExist { + buf = encoder.CustomBucketHistogramSamples(a.pendingHistograms, buf) + if err := a.wal.Log(buf); err != nil { return err } + buf = buf[:0] } } if len(a.pendingFloatHistograms) > 0 { - buf1, buf2 := encoder.FloatHistogramSamples(a.pendingFloatHistograms, buf) - if len(buf1) > 0 { - buf = buf1[:0] - if err := a.wal.Log(buf1); err != nil { - return err - } + var customBucketsExist bool + buf, customBucketsExist = encoder.FloatHistogramSamples(a.pendingFloatHistograms, buf) + if err := a.wal.Log(buf); err != nil { + return err } - if len(buf2) > 0 { - buf = buf2[:0] - if err := a.wal.Log(buf2); err != nil { + buf = buf[:0] + if customBucketsExist { + buf = encoder.CustomBucketFloatHistogramSamples(a.pendingFloatHistograms, buf) + if err := a.wal.Log(buf); err != nil { return err } + buf = buf[:0] } - //buf = buf[:0] } if len(a.pendingExamplars) > 0 { diff --git a/tsdb/agent/db_test.go b/tsdb/agent/db_test.go index 5332c61cdb..6b5d9ece05 100644 --- a/tsdb/agent/db_test.go +++ b/tsdb/agent/db_test.go @@ -163,6 +163,18 @@ func TestCommit(t *testing.T) { } } + lbls = labelsForTest(t.Name()+"_custom_bucket_histogram", numSeries) + for _, l := range lbls { + lset := labels.New(l...) + + customBucketHistograms := tsdbutil.GenerateTestCustomBucketsHistograms(numHistograms) + + for i := 0; i < numHistograms; i++ { + _, err := app.AppendHistogram(0, lset, int64(i), customBucketHistograms[i], nil) + require.NoError(t, err) + } + } + lbls = labelsForTest(t.Name()+"_float_histogram", numSeries) for _, l := range lbls { lset := labels.New(l...) @@ -175,6 +187,18 @@ func TestCommit(t *testing.T) { } } + lbls = labelsForTest(t.Name()+"custom_bucket_float_histogram", numSeries) + for _, l := range lbls { + lset := labels.New(l...) + + customBucketFloatHistograms := tsdbutil.GenerateTestCustomBucketsFloatHistograms(numHistograms) + + for i := 0; i < numHistograms; i++ { + _, err := app.AppendHistogram(0, lset, int64(i), nil, customBucketFloatHistograms[i]) + require.NoError(t, err) + } + } + require.NoError(t, app.Commit()) require.NoError(t, s.Close()) @@ -230,11 +254,11 @@ func TestCommit(t *testing.T) { } // Check that the WAL contained the same number of committed series/samples/exemplars. - require.Equal(t, numSeries*3, walSeriesCount, "unexpected number of series") + require.Equal(t, numSeries*5, walSeriesCount, "unexpected number of series") require.Equal(t, numSeries*numDatapoints, walSamplesCount, "unexpected number of samples") require.Equal(t, numSeries*numDatapoints, walExemplarsCount, "unexpected number of exemplars") - require.Equal(t, numSeries*numHistograms, walHistogramCount, "unexpected number of histograms") - require.Equal(t, numSeries*numHistograms, walFloatHistogramCount, "unexpected number of float histograms") + require.Equal(t, numSeries*numHistograms*2, walHistogramCount, "unexpected number of histograms") + require.Equal(t, numSeries*numHistograms*2, walFloatHistogramCount, "unexpected number of float histograms") } func TestRollback(t *testing.T) { diff --git a/tsdb/head_append.go b/tsdb/head_append.go index 3701a57135..78b256fee3 100644 --- a/tsdb/head_append.go +++ b/tsdb/head_append.go @@ -942,45 +942,30 @@ func (a *headAppender) log() error { } } if len(a.histograms) > 0 { - rec1, rec2 := enc.HistogramSamples(a.histograms, buf) - //rec = append(rec1, rec2...) - // - //buf = rec[:0] - // - //if err := a.head.wal.Log(rec); err != nil { - // return fmt.Errorf("log samples: %w", err) - //} - if len(rec1) != 0 { - buf = rec1[:0] - if err := a.head.wal.Log(rec1); err != nil { - return fmt.Errorf("log histograms: %w", err) - } + rec, customBucketsExist := enc.HistogramSamples(a.histograms, buf) + buf = rec[:0] + if err := a.head.wal.Log(rec); err != nil { + return fmt.Errorf("log histograms: %w", err) } - if len(rec2) != 0 { - buf = rec2[:0] - if err := a.head.wal.Log(rec2); err != nil { + + if customBucketsExist { + enc.CustomBucketHistogramSamples(a.histograms, buf) + buf = rec[:0] + if err := a.head.wal.Log(rec); err != nil { return fmt.Errorf("log custom bucket histograms: %w", err) } } } if len(a.floatHistograms) > 0 { - rec1, rec2 := enc.FloatHistogramSamples(a.floatHistograms, buf) - //rec = append(rec1, rec2...) - // - //buf = rec[:0] - // - //if err := a.head.wal.Log(rec); err != nil { - // return fmt.Errorf("log samples: %w", err) - //} - if len(rec1) != 0 { - buf = rec1[:0] - if err := a.head.wal.Log(rec1); err != nil { - return fmt.Errorf("log float histograms: %w", err) - } + rec, customBucketsExist := enc.FloatHistogramSamples(a.floatHistograms, buf) + buf = rec[:0] + if err := a.head.wal.Log(rec); err != nil { + return fmt.Errorf("log float histograms: %w", err) } - if len(rec2) != 0 { - buf = rec2[:0] - if err := a.head.wal.Log(rec2); err != nil { + + if customBucketsExist { + buf = rec[:0] + if err := a.head.wal.Log(rec); err != nil { return fmt.Errorf("log custom bucket float histograms: %w", err) } } diff --git a/tsdb/record/record.go b/tsdb/record/record.go index 15f5053d53..2b19cdbb6f 100644 --- a/tsdb/record/record.go +++ b/tsdb/record/record.go @@ -193,10 +193,6 @@ type RefFloatHistogramSample struct { FH *histogram.FloatHistogram } -type RefCustomBucketHistogramSample struct { - RefHistogramSample -} - // RefMmapMarker marks that the all the samples of the given series until now have been m-mapped to disk. type RefMmapMarker struct { Ref chunks.HeadSeriesRef @@ -748,15 +744,12 @@ func (e *Encoder) MmapMarkers(markers []RefMmapMarker, b []byte) []byte { return buf.Get() } -func (e *Encoder) HistogramSamples(histograms []RefHistogramSample, b []byte) ([]byte, []byte) { +func (e *Encoder) HistogramSamples(histograms []RefHistogramSample, b []byte) ([]byte, bool) { buf := encoding.Encbuf{B: b} buf.PutByte(byte(HistogramSamples)) - customBucketHistBuf := encoding.Encbuf{B: b} - customBucketHistBuf.PutByte(byte(CustomBucketHistogramSamples)) - if len(histograms) == 0 { - return buf.Get(), customBucketHistBuf.Get() + return buf.Get(), false } // Store base timestamp and base reference number of first histogram. @@ -765,34 +758,46 @@ func (e *Encoder) HistogramSamples(histograms []RefHistogramSample, b []byte) ([ buf.PutBE64(uint64(first.Ref)) buf.PutBE64int64(first.T) - customBucketHistBuf.PutBE64(uint64(first.Ref)) - customBucketHistBuf.PutBE64int64(first.T) - - histsAdded := 0 - customBucketHistsAdded := 0 + customBucketSamplesExist := false for _, h := range histograms { if h.H.UsesCustomBuckets() { - customBucketHistBuf.PutVarint64(int64(h.Ref) - int64(first.Ref)) - customBucketHistBuf.PutVarint64(h.T - first.T) + customBucketSamplesExist = true + continue + } + + buf.PutVarint64(int64(h.Ref) - int64(first.Ref)) + buf.PutVarint64(h.T - first.T) + + EncodeHistogram(&buf, h.H) + } + + return buf.Get(), customBucketSamplesExist +} + +func (e *Encoder) CustomBucketHistogramSamples(histograms []RefHistogramSample, b []byte) []byte { + buf := encoding.Encbuf{B: b} + buf.PutByte(byte(CustomBucketHistogramSamples)) + + if len(histograms) == 0 { + return buf.Get() + } - EncodeHistogram(&customBucketHistBuf, h.H) - customBucketHistsAdded++ - } else { + // Store base timestamp and base reference number of first histogram. + // All histograms encode their timestamp and ref as delta to those. + first := histograms[0] + buf.PutBE64(uint64(first.Ref)) + buf.PutBE64int64(first.T) + + for _, h := range histograms { + if h.H.UsesCustomBuckets() { buf.PutVarint64(int64(h.Ref) - int64(first.Ref)) buf.PutVarint64(h.T - first.T) EncodeHistogram(&buf, h.H) - histsAdded++ } } - if customBucketHistsAdded == 0 { - customBucketHistBuf.Reset() - } else if histsAdded == 0 { - buf.Reset() - } - - return buf.Get(), customBucketHistBuf.Get() + return buf.Get() } // EncodeHistogram encodes a Histogram into a byte slice. @@ -836,15 +841,12 @@ func EncodeHistogram(buf *encoding.Encbuf, h *histogram.Histogram) { } } -func (e *Encoder) FloatHistogramSamples(histograms []RefFloatHistogramSample, b []byte) ([]byte, []byte) { +func (e *Encoder) FloatHistogramSamples(histograms []RefFloatHistogramSample, b []byte) ([]byte, bool) { buf := encoding.Encbuf{B: b} buf.PutByte(byte(FloatHistogramSamples)) - customBucketHistBuf := encoding.Encbuf{B: b} - customBucketHistBuf.PutByte(byte(CustomBucketFloatHistogramSamples)) - if len(histograms) == 0 { - return buf.Get(), customBucketHistBuf.Get() + return buf.Get(), false } // Store base timestamp and base reference number of first histogram. @@ -853,34 +855,46 @@ func (e *Encoder) FloatHistogramSamples(histograms []RefFloatHistogramSample, b buf.PutBE64(uint64(first.Ref)) buf.PutBE64int64(first.T) - customBucketHistBuf.PutBE64(uint64(first.Ref)) - customBucketHistBuf.PutBE64int64(first.T) - - histsAdded := 0 - customBucketHistsAdded := 0 + customBucketsExist := false for _, h := range histograms { if h.FH.UsesCustomBuckets() { - customBucketHistBuf.PutVarint64(int64(h.Ref) - int64(first.Ref)) - customBucketHistBuf.PutVarint64(h.T - first.T) + customBucketsExist = true + continue + } + + buf.PutVarint64(int64(h.Ref) - int64(first.Ref)) + buf.PutVarint64(h.T - first.T) + + EncodeFloatHistogram(&buf, h.FH) + } + + return buf.Get(), customBucketsExist +} + +func (e *Encoder) CustomBucketFloatHistogramSamples(histograms []RefFloatHistogramSample, b []byte) []byte { + buf := encoding.Encbuf{B: b} + buf.PutByte(byte(CustomBucketFloatHistogramSamples)) + + if len(histograms) == 0 { + return buf.Get() + } - EncodeFloatHistogram(&customBucketHistBuf, h.FH) - customBucketHistsAdded++ - } else { + // Store base timestamp and base reference number of first histogram. + // All histograms encode their timestamp and ref as delta to those. + first := histograms[0] + buf.PutBE64(uint64(first.Ref)) + buf.PutBE64int64(first.T) + + for _, h := range histograms { + if h.FH.UsesCustomBuckets() { buf.PutVarint64(int64(h.Ref) - int64(first.Ref)) buf.PutVarint64(h.T - first.T) EncodeFloatHistogram(&buf, h.FH) - histsAdded++ } } - if customBucketHistsAdded == 0 { - customBucketHistBuf.Reset() - } else if histsAdded == 0 { - buf.Reset() - } - - return buf.Get(), customBucketHistBuf.Get() + return buf.Get() } // EncodeFloatHistogram encodes the Float Histogram into a byte slice. diff --git a/tsdb/record/record_test.go b/tsdb/record/record_test.go index 67c7eab970..af94f2b207 100644 --- a/tsdb/record/record_test.go +++ b/tsdb/record/record_test.go @@ -166,12 +166,12 @@ func TestRecord_EncodeDecode(t *testing.T) { }, } - histSamples, customBucketHistSamples := enc.HistogramSamples(histograms, nil) + histSamples, _ := enc.HistogramSamples(histograms, nil) + customBucketHistSamples := enc.CustomBucketHistogramSamples(histograms, nil) decHistograms, err := dec.HistogramSamples(histSamples, nil) require.NoError(t, err) - decCustomBucketHistograms, err := dec.HistogramSamples(customBucketHistSamples, nil) - require.NoError(t, err) - decHistograms = append(decHistograms, decCustomBucketHistograms...) + decCustomBucketHistSamples, err := dec.HistogramSamples(customBucketHistSamples, nil) + decHistograms = append(decHistograms, decCustomBucketHistSamples...) require.Equal(t, histograms, decHistograms) floatHistograms := make([]RefFloatHistogramSample, len(histograms)) @@ -182,10 +182,12 @@ func TestRecord_EncodeDecode(t *testing.T) { FH: h.H.ToFloat(nil), } } - histSamples, customBucketFloatHistSamples := enc.FloatHistogramSamples(floatHistograms, nil) - decFloatHistograms, err := dec.FloatHistogramSamples(histSamples, nil) + floatHistSamples, _ := enc.FloatHistogramSamples(floatHistograms, nil) + customBucketFloatHistSamples := enc.CustomBucketFloatHistogramSamples(floatHistograms, nil) + decFloatHistograms, err := dec.FloatHistogramSamples(floatHistSamples, nil) require.NoError(t, err) decCustomBucketFloatHistograms, err := dec.FloatHistogramSamples(customBucketFloatHistSamples, nil) + require.NoError(t, err) decFloatHistograms = append(decFloatHistograms, decCustomBucketFloatHistograms...) require.Equal(t, floatHistograms, decFloatHistograms) @@ -194,7 +196,8 @@ func TestRecord_EncodeDecode(t *testing.T) { histograms[i].H.CounterResetHint = histogram.GaugeType } - gaugeHistSamples, customBucketGaugeHistSamples := enc.HistogramSamples(histograms, nil) + gaugeHistSamples, _ := enc.HistogramSamples(histograms, nil) + customBucketGaugeHistSamples := enc.CustomBucketHistogramSamples(histograms, nil) decGaugeHistograms, err := dec.HistogramSamples(gaugeHistSamples, nil) require.NoError(t, err) decCustomBucketGaugeHistograms, err := dec.HistogramSamples(customBucketGaugeHistSamples, nil) @@ -207,10 +210,12 @@ func TestRecord_EncodeDecode(t *testing.T) { floatHistograms[i].FH.CounterResetHint = histogram.GaugeType } - gaugeHistSamples, customBucketGaugeFloatHistSamples := enc.FloatHistogramSamples(floatHistograms, nil) - decGaugeFloatHistograms, err := dec.FloatHistogramSamples(gaugeHistSamples, nil) + gaugeFloatHistSamples, _ := enc.FloatHistogramSamples(floatHistograms, nil) + customBucketGaugeFloatHistSamples := enc.CustomBucketFloatHistogramSamples(floatHistograms, nil) + decGaugeFloatHistograms, err := dec.FloatHistogramSamples(gaugeFloatHistSamples, nil) require.NoError(t, err) decCustomBucketGaugeFloatHistograms, err := dec.FloatHistogramSamples(customBucketGaugeFloatHistSamples, nil) + require.NoError(t, err) decFloatHistograms = append(decGaugeFloatHistograms, decCustomBucketGaugeFloatHistograms...) require.Equal(t, floatHistograms, decFloatHistograms) } @@ -295,10 +300,27 @@ func TestRecord_Corrupted(t *testing.T) { PositiveBuckets: []int64{1, 1, -1, 0}, }, }, + { + Ref: 67, + T: 5678, + H: &histogram.Histogram{ + Count: 8, + ZeroThreshold: 0.001, + Sum: 35.5, + Schema: -53, + PositiveSpans: []histogram.Span{ + {Offset: 0, Length: 2}, + {Offset: 2, Length: 2}, + }, + PositiveBuckets: []int64{2, -1, 2, 0}, + CustomValues: []float64{0, 2, 4, 6, 8}, + }, + }, } - corruptedHists, corruptedCustomBucketHists := enc.HistogramSamples(histograms, nil) + corruptedHists, _ := enc.HistogramSamples(histograms, nil) corruptedHists = corruptedHists[:8] + corruptedCustomBucketHists := enc.CustomBucketHistogramSamples(histograms, nil) corruptedCustomBucketHists = corruptedCustomBucketHists[:8] _, err := dec.HistogramSamples(corruptedHists, nil) require.ErrorIs(t, err, encoding.ErrInvalidSize) @@ -361,9 +383,10 @@ func TestRecord_Type(t *testing.T) { }, }, } - hists, customBucketHists := enc.HistogramSamples(histograms, nil) + hists, _ := enc.HistogramSamples(histograms, nil) recordType = dec.Type(hists) require.Equal(t, HistogramSamples, recordType) + customBucketHists := enc.CustomBucketHistogramSamples(histograms, nil) recordType = dec.Type(customBucketHists) require.Equal(t, CustomBucketHistogramSamples, recordType) diff --git a/tsdb/wlog/checkpoint.go b/tsdb/wlog/checkpoint.go index cd82676da9..5bb79595b9 100644 --- a/tsdb/wlog/checkpoint.go +++ b/tsdb/wlog/checkpoint.go @@ -238,7 +238,7 @@ func Checkpoint(logger *slog.Logger, w *WL, from, to int, keep func(id chunks.He } } if len(repl) > 0 { - _, buf = enc.HistogramSamples(repl, buf) + buf = enc.CustomBucketHistogramSamples(repl, buf) } stats.TotalSamples += len(histogramSamples) stats.DroppedSamples += len(histogramSamples) - len(repl) @@ -272,7 +272,7 @@ func Checkpoint(logger *slog.Logger, w *WL, from, to int, keep func(id chunks.He } } if len(repl) > 0 { - _, buf = enc.FloatHistogramSamples(repl, buf) + buf = enc.CustomBucketFloatHistogramSamples(repl, buf) } stats.TotalSamples += len(floatHistogramSamples) stats.DroppedSamples += len(floatHistogramSamples) - len(repl) diff --git a/tsdb/wlog/checkpoint_test.go b/tsdb/wlog/checkpoint_test.go index a5692a9aa4..f947f28095 100644 --- a/tsdb/wlog/checkpoint_test.go +++ b/tsdb/wlog/checkpoint_test.go @@ -127,6 +127,20 @@ func TestCheckpoint(t *testing.T) { PositiveBuckets: []int64{int64(i + 1), 1, -1, 0}, } } + makeCustomBucketHistogram := func(i int) *histogram.Histogram { + return &histogram.Histogram{ + Count: 5 + uint64(i*4), + ZeroCount: 2 + uint64(i), + ZeroThreshold: 0.001, + Sum: 18.4 * float64(i+1), + Schema: -53, + PositiveSpans: []histogram.Span{ + {Offset: 0, Length: 2}, + {Offset: 1, Length: 2}, + }, + CustomValues: []float64{0, 1, 2, 3, 4}, + } + } makeFloatHistogram := func(i int) *histogram.FloatHistogram { return &histogram.FloatHistogram{ Count: 5 + float64(i*4), @@ -141,6 +155,20 @@ func TestCheckpoint(t *testing.T) { PositiveBuckets: []float64{float64(i + 1), 1, -1, 0}, } } + makeCustomBucketFloatHistogram := func(i int) *histogram.FloatHistogram { + return &histogram.FloatHistogram{ + Count: 5 + float64(i*4), + ZeroCount: 2 + float64(i), + ZeroThreshold: 0.001, + Sum: 18.4 * float64(i+1), + Schema: -53, + PositiveSpans: []histogram.Span{ + {Offset: 0, Length: 2}, + {Offset: 1, Length: 2}, + }, + CustomValues: []float64{0, 1, 2, 3, 4}, + } + } for _, compress := range []CompressionType{CompressionNone, CompressionSnappy, CompressionZstd} { t.Run(fmt.Sprintf("compress=%s", compress), func(t *testing.T) { @@ -208,24 +236,40 @@ func TestCheckpoint(t *testing.T) { require.NoError(t, w.Log(b)) samplesInWAL += 4 h := makeHistogram(i) - b1, b2 := enc.HistogramSamples([]record.RefHistogramSample{ + b, _ = enc.HistogramSamples([]record.RefHistogramSample{ {Ref: 0, T: last, H: h}, {Ref: 1, T: last + 10000, H: h}, {Ref: 2, T: last + 20000, H: h}, {Ref: 3, T: last + 30000, H: h}, }, nil) - require.NoError(t, w.Log(b1)) - require.NoError(t, w.Log(b2)) + require.NoError(t, w.Log(b)) + histogramsInWAL += 4 + cbh := makeCustomBucketHistogram(i) + b = enc.CustomBucketHistogramSamples([]record.RefHistogramSample{ + {Ref: 0, T: last, H: cbh}, + {Ref: 1, T: last + 10000, H: cbh}, + {Ref: 2, T: last + 20000, H: cbh}, + {Ref: 3, T: last + 30000, H: cbh}, + }, nil) + require.NoError(t, w.Log(b)) histogramsInWAL += 4 fh := makeFloatHistogram(i) - b1, b2 = enc.FloatHistogramSamples([]record.RefFloatHistogramSample{ + b, _ = enc.FloatHistogramSamples([]record.RefFloatHistogramSample{ {Ref: 0, T: last, FH: fh}, {Ref: 1, T: last + 10000, FH: fh}, {Ref: 2, T: last + 20000, FH: fh}, {Ref: 3, T: last + 30000, FH: fh}, }, nil) - require.NoError(t, w.Log(b1)) - require.NoError(t, w.Log(b2)) + require.NoError(t, w.Log(b)) + floatHistogramsInWAL += 4 + cbfh := makeCustomBucketFloatHistogram(i) + b = enc.CustomBucketFloatHistogramSamples([]record.RefFloatHistogramSample{ + {Ref: 0, T: last, FH: cbfh}, + {Ref: 1, T: last + 10000, FH: cbfh}, + {Ref: 2, T: last + 20000, FH: cbfh}, + {Ref: 3, T: last + 30000, FH: cbfh}, + }, nil) + require.NoError(t, w.Log(b)) floatHistogramsInWAL += 4 b = enc.Exemplars([]record.RefExemplar{ @@ -286,14 +330,14 @@ func TestCheckpoint(t *testing.T) { require.GreaterOrEqual(t, s.T, last/2, "sample with wrong timestamp") } samplesInCheckpoint += len(samples) - case record.HistogramSamples: + case record.HistogramSamples, record.CustomBucketHistogramSamples: histograms, err := dec.HistogramSamples(rec, nil) require.NoError(t, err) for _, h := range histograms { require.GreaterOrEqual(t, h.T, last/2, "histogram with wrong timestamp") } histogramsInCheckpoint += len(histograms) - case record.FloatHistogramSamples: + case record.FloatHistogramSamples, record.CustomBucketFloatHistogramSamples: floatHistograms, err := dec.FloatHistogramSamples(rec, nil) require.NoError(t, err) for _, h := range floatHistograms { diff --git a/tsdb/wlog/watcher.go b/tsdb/wlog/watcher.go index 89db5d2dd7..169bd296fe 100644 --- a/tsdb/wlog/watcher.go +++ b/tsdb/wlog/watcher.go @@ -546,7 +546,7 @@ func (w *Watcher) readSegment(r *LiveReader, segmentNum int, tail bool) error { } w.writer.AppendExemplars(exemplars) - case record.HistogramSamples: + case record.HistogramSamples, record.CustomBucketHistogramSamples: // Skip if experimental "histograms over remote write" is not enabled. if !w.sendHistograms { break @@ -574,7 +574,7 @@ func (w *Watcher) readSegment(r *LiveReader, segmentNum int, tail bool) error { histogramsToSend = histogramsToSend[:0] } - case record.FloatHistogramSamples: + case record.FloatHistogramSamples, record.CustomBucketFloatHistogramSamples: // Skip if experimental "histograms over remote write" is not enabled. if !w.sendHistograms { break diff --git a/tsdb/wlog/watcher_test.go b/tsdb/wlog/watcher_test.go index c2499a7cec..5ff70bb215 100644 --- a/tsdb/wlog/watcher_test.go +++ b/tsdb/wlog/watcher_test.go @@ -209,21 +209,43 @@ func TestTailSamples(t *testing.T) { NegativeBuckets: []int64{int64(-i) - 1}, } - histogram, customBucketHistogram := enc.HistogramSamples([]record.RefHistogramSample{{ + histograms, _ := enc.HistogramSamples([]record.RefHistogramSample{{ Ref: chunks.HeadSeriesRef(inner), T: now.UnixNano() + 1, H: hist, }}, nil) - require.NoError(t, w.Log(histogram)) - require.NoError(t, w.Log(customBucketHistogram)) + require.NoError(t, w.Log(histograms)) + + customBucketHist := &histogram.Histogram{ + Schema: -53, + ZeroThreshold: 1e-128, + ZeroCount: 0, + Count: 2, + Sum: 0, + PositiveSpans: []histogram.Span{{Offset: 0, Length: 1}}, + CustomValues: []float64{float64(i) + 2}, + } + + customBucketHistograms := enc.CustomBucketHistogramSamples([]record.RefHistogramSample{{ + Ref: chunks.HeadSeriesRef(inner), + T: now.UnixNano() + 1, + H: customBucketHist, + }}, nil) + require.NoError(t, w.Log(customBucketHistograms)) - floatHistogram, floatCustomBucketHistogram := enc.FloatHistogramSamples([]record.RefFloatHistogramSample{{ + floatHistograms, _ := enc.FloatHistogramSamples([]record.RefFloatHistogramSample{{ Ref: chunks.HeadSeriesRef(inner), T: now.UnixNano() + 1, FH: hist.ToFloat(nil), }}, nil) - require.NoError(t, w.Log(floatHistogram)) - require.NoError(t, w.Log(floatCustomBucketHistogram)) + require.NoError(t, w.Log(floatHistograms)) + + customBucketFloatHistograms := enc.CustomBucketFloatHistogramSamples([]record.RefFloatHistogramSample{{ + Ref: chunks.HeadSeriesRef(inner), + T: now.UnixNano() + 1, + FH: customBucketHist.ToFloat(nil), + }}, nil) + require.NoError(t, w.Log(customBucketFloatHistograms)) } } @@ -250,7 +272,7 @@ func TestTailSamples(t *testing.T) { expectedSeries := seriesCount expectedSamples := seriesCount * samplesCount expectedExemplars := seriesCount * exemplarsCount - expectedHistograms := seriesCount * histogramsCount + expectedHistograms := seriesCount * histogramsCount * 2 retry(t, defaultRetryInterval, defaultRetries, func() bool { return wt.checkNumSeries() >= expectedSeries }) From 6684344026c9395df1db1f92114009cac50803f5 Mon Sep 17 00:00:00 2001 From: Carrie Edwards Date: Thu, 21 Nov 2024 10:50:18 -0800 Subject: [PATCH 06/40] Rename old histogram record type, use old names for new records --- tsdb/agent/db.go | 24 +-- tsdb/agent/db_test.go | 66 +++++++- tsdb/db_test.go | 186 +++++++++++++++++++- tsdb/head_append.go | 19 +-- tsdb/head_test.go | 317 +---------------------------------- tsdb/head_wal.go | 8 +- tsdb/ooo_head_read_test.go | 2 +- tsdb/record/record.go | 104 +++--------- tsdb/record/record_test.go | 36 +--- tsdb/tsdbutil/histogram.go | 1 - tsdb/wlog/checkpoint.go | 42 +---- tsdb/wlog/checkpoint_test.go | 12 +- tsdb/wlog/watcher.go | 4 +- tsdb/wlog/watcher_test.go | 8 +- 14 files changed, 298 insertions(+), 531 deletions(-) diff --git a/tsdb/agent/db.go b/tsdb/agent/db.go index bcfc7be129..5cf56d5871 100644 --- a/tsdb/agent/db.go +++ b/tsdb/agent/db.go @@ -463,7 +463,7 @@ func (db *DB) loadWAL(r *wlog.Reader, multiRef map[chunks.HeadSeriesRef]chunks.H return } decoded <- samples - case record.HistogramSamples, record.CustomBucketHistogramSamples: + case record.HistogramSamples, record.HistogramSamplesLegacy: histograms := histogramsPool.Get()[:0] histograms, err = dec.HistogramSamples(rec, histograms) if err != nil { @@ -475,7 +475,7 @@ func (db *DB) loadWAL(r *wlog.Reader, multiRef map[chunks.HeadSeriesRef]chunks.H return } decoded <- histograms - case record.FloatHistogramSamples, record.CustomBucketFloatHistogramSamples: + case record.FloatHistogramSamples, record.FloatHistogramSamplesLegacy: floatHistograms := floatHistogramsPool.Get()[:0] floatHistograms, err = dec.FloatHistogramSamples(rec, floatHistograms) if err != nil { @@ -1154,35 +1154,19 @@ func (a *appender) log() error { } if len(a.pendingHistograms) > 0 { - var customBucketsExist bool - buf, customBucketsExist = encoder.HistogramSamples(a.pendingHistograms, buf) + buf = encoder.HistogramSamples(a.pendingHistograms, buf) if err := a.wal.Log(buf); err != nil { return err } buf = buf[:0] - if customBucketsExist { - buf = encoder.CustomBucketHistogramSamples(a.pendingHistograms, buf) - if err := a.wal.Log(buf); err != nil { - return err - } - buf = buf[:0] - } } if len(a.pendingFloatHistograms) > 0 { - var customBucketsExist bool - buf, customBucketsExist = encoder.FloatHistogramSamples(a.pendingFloatHistograms, buf) + buf = encoder.FloatHistogramSamples(a.pendingFloatHistograms, buf) if err := a.wal.Log(buf); err != nil { return err } buf = buf[:0] - if customBucketsExist { - buf = encoder.CustomBucketFloatHistogramSamples(a.pendingFloatHistograms, buf) - if err := a.wal.Log(buf); err != nil { - return err - } - buf = buf[:0] - } } if len(a.pendingExamplars) > 0 { diff --git a/tsdb/agent/db_test.go b/tsdb/agent/db_test.go index 6b5d9ece05..8bcb71c86a 100644 --- a/tsdb/agent/db_test.go +++ b/tsdb/agent/db_test.go @@ -163,7 +163,7 @@ func TestCommit(t *testing.T) { } } - lbls = labelsForTest(t.Name()+"_custom_bucket_histogram", numSeries) + lbls = labelsForTest(t.Name()+"_custom_buckets_histogram", numSeries) for _, l := range lbls { lset := labels.New(l...) @@ -187,7 +187,7 @@ func TestCommit(t *testing.T) { } } - lbls = labelsForTest(t.Name()+"custom_bucket_float_histogram", numSeries) + lbls = labelsForTest(t.Name()+"_custom_buckets_float_histogram", numSeries) for _, l := range lbls { lset := labels.New(l...) @@ -231,13 +231,13 @@ func TestCommit(t *testing.T) { require.NoError(t, err) walSamplesCount += len(samples) - case record.HistogramSamples, record.CustomBucketHistogramSamples: + case record.HistogramSamples, record.HistogramSamplesLegacy: var histograms []record.RefHistogramSample histograms, err = dec.HistogramSamples(rec, histograms) require.NoError(t, err) walHistogramCount += len(histograms) - case record.FloatHistogramSamples, record.CustomBucketFloatHistogramSamples: + case record.FloatHistogramSamples, record.FloatHistogramSamplesLegacy: var floatHistograms []record.RefFloatHistogramSample floatHistograms, err = dec.FloatHistogramSamples(rec, floatHistograms) require.NoError(t, err) @@ -294,6 +294,18 @@ func TestRollback(t *testing.T) { } } + lbls = labelsForTest(t.Name()+"_custom_buckets_histogram", numSeries) + for _, l := range lbls { + lset := labels.New(l...) + + histograms := tsdbutil.GenerateTestCustomBucketsHistograms(numHistograms) + + for i := 0; i < numHistograms; i++ { + _, err := app.AppendHistogram(0, lset, int64(i), histograms[i], nil) + require.NoError(t, err) + } + } + lbls = labelsForTest(t.Name()+"_float_histogram", numSeries) for _, l := range lbls { lset := labels.New(l...) @@ -306,6 +318,18 @@ func TestRollback(t *testing.T) { } } + lbls = labelsForTest(t.Name()+"_custom_buckets_float_histogram", numSeries) + for _, l := range lbls { + lset := labels.New(l...) + + floatHistograms := tsdbutil.GenerateTestCustomBucketsFloatHistograms(numHistograms) + + for i := 0; i < numHistograms; i++ { + _, err := app.AppendHistogram(0, lset, int64(i), nil, floatHistograms[i]) + require.NoError(t, err) + } + } + // Do a rollback, which should clear uncommitted data. A followup call to // commit should persist nothing to the WAL. require.NoError(t, app.Rollback()) @@ -346,13 +370,13 @@ func TestRollback(t *testing.T) { require.NoError(t, err) walExemplarsCount += len(exemplars) - case record.HistogramSamples: + case record.HistogramSamples, record.HistogramSamplesLegacy: var histograms []record.RefHistogramSample histograms, err = dec.HistogramSamples(rec, histograms) require.NoError(t, err) walHistogramCount += len(histograms) - case record.FloatHistogramSamples: + case record.FloatHistogramSamples, record.FloatHistogramSamplesLegacy: var floatHistograms []record.RefFloatHistogramSample floatHistograms, err = dec.FloatHistogramSamples(rec, floatHistograms) require.NoError(t, err) @@ -363,7 +387,7 @@ func TestRollback(t *testing.T) { } // Check that only series get stored after calling Rollback. - require.Equal(t, numSeries*3, walSeriesCount, "series should have been written to WAL") + require.Equal(t, numSeries*5, 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") @@ -412,6 +436,19 @@ func TestFullTruncateWAL(t *testing.T) { require.NoError(t, app.Commit()) } + lbls = labelsForTest(t.Name()+"_custom_buckets_histogram", numSeries) + for _, l := range lbls { + lset := labels.New(l...) + + histograms := tsdbutil.GenerateTestCustomBucketsHistograms(numHistograms) + + for i := 0; i < numHistograms; i++ { + _, err := app.AppendHistogram(0, lset, int64(lastTs), histograms[i], nil) + require.NoError(t, err) + } + require.NoError(t, app.Commit()) + } + lbls = labelsForTest(t.Name()+"_float_histogram", numSeries) for _, l := range lbls { lset := labels.New(l...) @@ -425,11 +462,24 @@ func TestFullTruncateWAL(t *testing.T) { require.NoError(t, app.Commit()) } + lbls = labelsForTest(t.Name()+"_custom_buckets_float_histogram", numSeries) + for _, l := range lbls { + lset := labels.New(l...) + + floatHistograms := tsdbutil.GenerateTestCustomBucketsFloatHistograms(numHistograms) + + for i := 0; i < numHistograms; i++ { + _, err := app.AppendHistogram(0, lset, int64(lastTs), nil, floatHistograms[i]) + require.NoError(t, err) + } + require.NoError(t, app.Commit()) + } + // Truncate WAL with mint to GC all the samples. s.truncate(lastTs + 1) m := gatherFamily(t, reg, "prometheus_agent_deleted_series") - require.Equal(t, float64(numSeries*3), m.Metric[0].Gauge.GetValue(), "agent wal truncate mismatch of deleted series count") + require.Equal(t, float64(numSeries*5), m.Metric[0].Gauge.GetValue(), "agent wal truncate mismatch of deleted series count") } func TestPartialTruncateWAL(t *testing.T) { diff --git a/tsdb/db_test.go b/tsdb/db_test.go index 306dc4579e..4bbf4b4656 100644 --- a/tsdb/db_test.go +++ b/tsdb/db_test.go @@ -4281,6 +4281,188 @@ 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(int(mins)), nil) + require.NoError(t, err) + return seriesRef, nil + }, + expectedOOORecords: []interface{}{ + // 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: []interface{}{ + []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(int(mins))) + require.NoError(t, err) + return seriesRef, nil + }, + expectedOOORecords: []interface{}{ + // 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: []interface{}{ + []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) { @@ -4374,11 +4556,11 @@ func testOOOWALWrite(t *testing.T, markers, err := dec.MmapMarkers(rec, nil) require.NoError(t, err) records = append(records, markers) - case record.HistogramSamples: + case record.HistogramSamples, record.HistogramSamplesLegacy: histogramSamples, err := dec.HistogramSamples(rec, nil) require.NoError(t, err) records = append(records, histogramSamples) - case record.FloatHistogramSamples: + case record.FloatHistogramSamples, record.FloatHistogramSamplesLegacy: floatHistogramSamples, err := dec.FloatHistogramSamples(rec, nil) require.NoError(t, err) records = append(records, floatHistogramSamples) diff --git a/tsdb/head_append.go b/tsdb/head_append.go index 78b256fee3..7dacb9037b 100644 --- a/tsdb/head_append.go +++ b/tsdb/head_append.go @@ -942,33 +942,18 @@ func (a *headAppender) log() error { } } if len(a.histograms) > 0 { - rec, customBucketsExist := enc.HistogramSamples(a.histograms, buf) + rec = enc.HistogramSamples(a.histograms, buf) buf = rec[:0] if err := a.head.wal.Log(rec); err != nil { return fmt.Errorf("log histograms: %w", err) } - - if customBucketsExist { - enc.CustomBucketHistogramSamples(a.histograms, buf) - buf = rec[:0] - if err := a.head.wal.Log(rec); err != nil { - return fmt.Errorf("log custom bucket histograms: %w", err) - } - } } if len(a.floatHistograms) > 0 { - rec, customBucketsExist := enc.FloatHistogramSamples(a.floatHistograms, buf) + rec = enc.FloatHistogramSamples(a.floatHistograms, buf) buf = rec[:0] if err := a.head.wal.Log(rec); err != nil { return fmt.Errorf("log float histograms: %w", err) } - - if customBucketsExist { - buf = rec[:0] - if err := a.head.wal.Log(rec); err != nil { - return fmt.Errorf("log custom bucket 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 diff --git a/tsdb/head_test.go b/tsdb/head_test.go index 527476e113..c3377fecff 100644 --- a/tsdb/head_test.go +++ b/tsdb/head_test.go @@ -187,11 +187,11 @@ func readTestWAL(t testing.TB, dir string) (recs []interface{}) { samples, err := dec.Samples(rec, nil) require.NoError(t, err) recs = append(recs, samples) - case record.HistogramSamples: + case record.HistogramSamples, record.HistogramSamplesLegacy: samples, err := dec.HistogramSamples(rec, nil) require.NoError(t, err) recs = append(recs, samples) - case record.FloatHistogramSamples: + case record.FloatHistogramSamples, record.FloatHistogramSamplesLegacy: samples, err := dec.FloatHistogramSamples(rec, nil) require.NoError(t, err) recs = append(recs, samples) @@ -740,89 +740,6 @@ func TestHead_ReadWAL(t *testing.T) { } } -func TestHead_ReadWAL2(t *testing.T) { - for _, compress := range []wlog.CompressionType{wlog.CompressionNone, wlog.CompressionSnappy, wlog.CompressionZstd} { - t.Run(fmt.Sprintf("compress=%s", compress), func(t *testing.T) { - entries := []interface{}{ - []record.RefSeries{ - {Ref: 10, Labels: labels.FromStrings("a", "1")}, - {Ref: 11, Labels: labels.FromStrings("a", "2")}, - {Ref: 100, Labels: labels.FromStrings("a", "3")}, - }, - []record.RefHistogramSample{ - {Ref: 0, T: 99, H: tsdbutil.GenerateTestHistogram(1)}, - {Ref: 10, T: 100, H: tsdbutil.GenerateTestCustomBucketsHistogram(2)}, - {Ref: 100, T: 100, H: tsdbutil.GenerateTestHistogram(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.RefHistogramSample{ - {Ref: 10, T: 101, H: tsdbutil.GenerateTestHistogram(5)}, - {Ref: 50, T: 101, H: tsdbutil.GenerateTestHistogram(6)}, - {Ref: 101, T: 101, H: tsdbutil.GenerateTestCustomBucketsHistogram(7)}, - }, - []tombstones.Stone{ - {Ref: 0, Intervals: []tombstones.Interval{{Mint: 99, Maxt: 101}}}, - }, - []record.RefExemplar{ - {Ref: 10, T: 100, V: 1, Labels: labels.FromStrings("trace_id", "asdf")}, - }, - } - - head, w := newTestHead(t, 1000, compress, false) - defer func() { - require.NoError(t, head.Close()) - }() - - populateTestWL(t, w, entries) - - 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) - - 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) - - expandChunk := func(c chunkenc.Iterator) (x []sample) { - for c.Next() == chunkenc.ValHistogram { - t, v := c.AtHistogram(nil) - //t, v := c.At() - x = append(x, sample{t: t, h: v}) - } - require.NoError(t, c.Err()) - return x - } - - c, _, _, err := s10.chunk(0, head.chunkDiskMapper, &head.memChunkPool) - require.NoError(t, err) - require.Equal(t, []sample{{100, 0, tsdbutil.GenerateTestCustomBucketsHistogram(2), nil}, {101, 0, tsdbutil.GenerateTestCustomBucketsHistogram(5), nil}}, expandChunk(c.chunk.Iterator(nil))) - c, _, _, err = s50.chunk(0, head.chunkDiskMapper, &head.memChunkPool) - require.NoError(t, err) - require.Equal(t, []sample{{101, 0, tsdbutil.GenerateTestHistogram(6), nil}}, expandChunk(c.chunk.Iterator(nil))) - // 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, 0, tsdbutil.GenerateTestCustomBucketsHistogram(7), 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.True(t, exemplar.Exemplar{Ts: 100, Value: 1, Labels: labels.FromStrings("trace_id", "asdf")}.Equals(e[0].Exemplars[0])) - }) - } -} - func TestHead_WALMultiRef(t *testing.T) { head, w := newTestHead(t, 1000, wlog.CompressionNone, false) @@ -4036,194 +3953,6 @@ func TestHistogramInWALAndMmapChunk(t *testing.T) { testQuery() } -func TestHistogramInWALAndMmapChunk2(t *testing.T) { - head, _ := newTestHead(t, 3000, wlog.CompressionNone, 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 _, custom := range []bool{true, false} { - app = head.Appender(context.Background()) - var hists []*histogram.Histogram - if custom { - hists = tsdbutil.GenerateTestCustomBucketsHistograms(numHistograms) - } else { - hists = tsdbutil.GenerateTestHistograms(numHistograms) - } - for _, h := range hists { - if !custom { - 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 _, custom := range []bool{true, false} { - app = head.Appender(context.Background()) - var hists []*histogram.FloatHistogram - if custom { - hists = tsdbutil.GenerateTestCustomBucketsFloatHistograms(numHistograms) - } else { - hists = tsdbutil.GenerateTestFloatHistograms(numHistograms) - } - for _, h := range hists { - if !custom { - 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, 19) - 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 _, custom := range []bool{true, false} { - app = head.Appender(context.Background()) - var hists []*histogram.Histogram - if custom { - hists = tsdbutil.GenerateTestCustomBucketsHistograms(100) - } else { - hists = tsdbutil.GenerateTestHistograms(100) - } - for _, h := range hists { - ts++ - if !custom { - h.NegativeSpans = h.PositiveSpans - h.NegativeBuckets = h.PositiveBuckets - } - _, err := app.AppendHistogram(0, s2, ts, h, nil) - require.NoError(t, err) - eh := h.Copy() - if 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 i := 0; i < 10; i++ { - 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 _, custom := range []bool{true, false} { - app = head.Appender(context.Background()) - var hists []*histogram.FloatHistogram - if custom { - hists = tsdbutil.GenerateTestCustomBucketsFloatHistograms(100) - } else { - hists = tsdbutil.GenerateTestFloatHistograms(100) - } - for _, h := range hists { - ts++ - if !custom { - h.NegativeSpans = h.PositiveSpans - h.NegativeBuckets = h.PositiveBuckets - } - _, err := app.AppendHistogram(0, s2, ts, nil, h) - require.NoError(t, err) - eh := h.Copy() - if 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 i := 0; i < 10; i++ { - 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, wlog.CompressionNone) - 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, wlog.CompressionNone, false) defer func() { @@ -5360,48 +5089,6 @@ func TestChunkSnapshotTakenAfterIncompleteSnapshot(t *testing.T) { require.Positive(t, offset) } -func TestHistogramWALANDWBLReplay(t *testing.T) { - dir := t.TempDir() - wal, err := wlog.NewSize(nil, nil, filepath.Join(dir, "wal"), 32768, wlog.CompressionSnappy) - require.NoError(t, err) - oooWlog, err := wlog.NewSize(nil, nil, filepath.Join(dir, wlog.WblDirName), 32768, wlog.CompressionSnappy) - require.NoError(t, err) - - opts := DefaultHeadOptions() - opts.ChunkRange = 1000 - opts.ChunkDirRoot = dir - opts.OutOfOrderTimeWindow.Store(30 * time.Minute.Milliseconds()) - opts.EnableNativeHistograms.Store(true) - opts.EnableOOONativeHistograms.Store(true) - - 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, val float64, isOOO bool, isCustomBucketHistogram bool) { - app := h.Appender(context.Background()) - var s sample - if isCustomBucketHistogram { - s = sample{t: mins * time.Minute.Milliseconds(), h: tsdbutil.GenerateTestCustomBucketsHistogram(int(val))} - } else { - s = sample{t: mins * time.Minute.Milliseconds(), h: tsdbutil.GenerateTestHistogram(int(val))} - } - _, err := app.AppendHistogram(0, l, mins*time.Minute.Milliseconds(), s.h, nil) - require.NoError(t, err) - require.NoError(t, app.Commit()) - - if isOOO { - expOOOSamples = append(expOOOSamples, s) - } - } - - // In-order histogram samples. - appendSample(60, 60, false, false) - -} - // TestWBLReplay checks the replay at a low level. func TestWBLReplay(t *testing.T) { for name, scenario := range sampleTypeScenarios { diff --git a/tsdb/head_wal.go b/tsdb/head_wal.go index 9d1e24b706..458162522b 100644 --- a/tsdb/head_wal.go +++ b/tsdb/head_wal.go @@ -189,7 +189,7 @@ func (h *Head) loadWAL(r *wlog.Reader, syms *labels.SymbolTable, multiRef map[ch return } decoded <- exemplars - case record.HistogramSamples, record.CustomBucketHistogramSamples: + case record.HistogramSamples, record.HistogramSamplesLegacy: hists := histogramsPool.Get()[:0] hists, err = dec.HistogramSamples(rec, hists) if err != nil { @@ -201,7 +201,7 @@ func (h *Head) loadWAL(r *wlog.Reader, syms *labels.SymbolTable, multiRef map[ch return } decoded <- hists - case record.FloatHistogramSamples, record.CustomBucketFloatHistogramSamples: + case record.FloatHistogramSamples, record.FloatHistogramSamplesLegacy: hists := floatHistogramsPool.Get()[:0] hists, err = dec.FloatHistogramSamples(rec, hists) if err != nil { @@ -726,7 +726,7 @@ func (h *Head) loadWBL(r *wlog.Reader, syms *labels.SymbolTable, multiRef map[ch return } decodedCh <- markers - case record.HistogramSamples: + case record.HistogramSamples, record.HistogramSamplesLegacy: hists := histogramSamplesPool.Get()[:0] hists, err = dec.HistogramSamples(rec, hists) if err != nil { @@ -738,7 +738,7 @@ func (h *Head) loadWBL(r *wlog.Reader, syms *labels.SymbolTable, multiRef map[ch return } decodedCh <- hists - case record.FloatHistogramSamples: + case record.FloatHistogramSamples, record.FloatHistogramSamplesLegacy: hists := floatHistogramSamplesPool.Get()[:0] hists, err = dec.FloatHistogramSamples(rec, hists) if err != nil { diff --git a/tsdb/ooo_head_read_test.go b/tsdb/ooo_head_read_test.go index bc1cb67d1e..adbd3278ba 100644 --- a/tsdb/ooo_head_read_test.go +++ b/tsdb/ooo_head_read_test.go @@ -963,7 +963,7 @@ func testOOOHeadChunkReader_Chunk_ConsistentQueryResponseDespiteOfHeadExpanding( }, }, { - name: "After Series() prev head gets mmapped after getting samples, new head gets new samples also overlapping, none of these should appear in response.", + name: "After Series() prev head mmapped after getting samples, new head gets new samples also overlapping, none should appear in response.", queryMinT: minutes(0), queryMaxT: minutes(100), firstInOrderSampleAt: minutes(120), diff --git a/tsdb/record/record.go b/tsdb/record/record.go index 2b19cdbb6f..2dd7ffe027 100644 --- a/tsdb/record/record.go +++ b/tsdb/record/record.go @@ -48,14 +48,14 @@ const ( MmapMarkers Type = 5 // Metadata is used to match WAL records of type Metadata. Metadata Type = 6 - // HistogramSamples is used to match WAL records of type Histograms. - HistogramSamples Type = 7 - // FloatHistogramSamples is used to match WAL records of type Float Histograms. - FloatHistogramSamples Type = 8 - // CustomBucketHistogramSamples is used to match WAL records of type Histogram with custom buckets. - CustomBucketHistogramSamples Type = 9 - // CustomBucketFloatHistogramSamples is used to match WAL records of type Float Histogram with custom buckets. - CustomBucketFloatHistogramSamples Type = 10 + // HistogramSamplesLegacy is used to match WAL records of type Histograms prior to intrdocuing support of custom buckets, to maintain backwards compatibility. + HistogramSamplesLegacy Type = 7 + // FloatHistogramSamplesLegacy is used to match WAL records of type Float Histograms proior to introducing support of custom buckets, to maintain backwards compatibility. + FloatHistogramSamplesLegacy Type = 8 + // HistogramSamples is used to match WAL records of type Histogram, and supports custom buckets. + HistogramSamples Type = 9 + // FloatHistogramSamples is used to match WAL records of type Float Histogram, and supports custom buckets. + FloatHistogramSamples Type = 10 ) func (rt Type) String() string { @@ -68,14 +68,14 @@ func (rt Type) String() string { return "tombstones" case Exemplars: return "exemplars" + case HistogramSamplesLegacy: + return "histogram_samples_legacy" + case FloatHistogramSamplesLegacy: + return "float_histogram_samples_legacy" case HistogramSamples: - return "histogram_samples" + return "histogram_sample" case FloatHistogramSamples: return "float_histogram_samples" - case CustomBucketHistogramSamples: - return "custom_bucket_histogram_samples" - case CustomBucketFloatHistogramSamples: - return "custom_bucket_float_histogram_samples" case MmapMarkers: return "mmapmarkers" case Metadata: @@ -215,7 +215,7 @@ func (d *Decoder) Type(rec []byte) Type { return Unknown } switch t := Type(rec[0]); t { - case Series, Samples, Tombstones, Exemplars, MmapMarkers, Metadata, HistogramSamples, FloatHistogramSamples, CustomBucketHistogramSamples, CustomBucketFloatHistogramSamples: + case Series, Samples, Tombstones, Exemplars, MmapMarkers, Metadata, HistogramSamplesLegacy, FloatHistogramSamplesLegacy, HistogramSamples, FloatHistogramSamples: return t } return Unknown @@ -436,7 +436,7 @@ func (d *Decoder) MmapMarkers(rec []byte, markers []RefMmapMarker) ([]RefMmapMar func (d *Decoder) HistogramSamples(rec []byte, histograms []RefHistogramSample) ([]RefHistogramSample, error) { dec := encoding.Decbuf{B: rec} t := Type(dec.Byte()) - if t != HistogramSamples && t != CustomBucketHistogramSamples { + if t != HistogramSamples && t != HistogramSamplesLegacy { return nil, errors.New("invalid record type") } if dec.Len() == 0 { @@ -528,7 +528,7 @@ func DecodeHistogram(buf *encoding.Decbuf, h *histogram.Histogram) { func (d *Decoder) FloatHistogramSamples(rec []byte, histograms []RefFloatHistogramSample) ([]RefFloatHistogramSample, error) { dec := encoding.Decbuf{B: rec} t := Type(dec.Byte()) - if t != FloatHistogramSamples && t != CustomBucketFloatHistogramSamples { + if t != FloatHistogramSamples && t != FloatHistogramSamplesLegacy { return nil, errors.New("invalid record type") } if dec.Len() == 0 { @@ -744,12 +744,12 @@ func (e *Encoder) MmapMarkers(markers []RefMmapMarker, b []byte) []byte { return buf.Get() } -func (e *Encoder) HistogramSamples(histograms []RefHistogramSample, b []byte) ([]byte, bool) { +func (e *Encoder) HistogramSamples(histograms []RefHistogramSample, b []byte) []byte { buf := encoding.Encbuf{B: b} buf.PutByte(byte(HistogramSamples)) if len(histograms) == 0 { - return buf.Get(), false + return buf.Get() } // Store base timestamp and base reference number of first histogram. @@ -758,45 +758,13 @@ func (e *Encoder) HistogramSamples(histograms []RefHistogramSample, b []byte) ([ buf.PutBE64(uint64(first.Ref)) buf.PutBE64int64(first.T) - customBucketSamplesExist := false for _, h := range histograms { - if h.H.UsesCustomBuckets() { - customBucketSamplesExist = true - continue - } - buf.PutVarint64(int64(h.Ref) - int64(first.Ref)) buf.PutVarint64(h.T - first.T) EncodeHistogram(&buf, h.H) } - return buf.Get(), customBucketSamplesExist -} - -func (e *Encoder) CustomBucketHistogramSamples(histograms []RefHistogramSample, b []byte) []byte { - buf := encoding.Encbuf{B: b} - buf.PutByte(byte(CustomBucketHistogramSamples)) - - if len(histograms) == 0 { - return buf.Get() - } - - // Store base timestamp and base reference number of first histogram. - // All histograms encode their timestamp and ref as delta to those. - first := histograms[0] - buf.PutBE64(uint64(first.Ref)) - buf.PutBE64int64(first.T) - - for _, h := range histograms { - if h.H.UsesCustomBuckets() { - buf.PutVarint64(int64(h.Ref) - int64(first.Ref)) - buf.PutVarint64(h.T - first.T) - - EncodeHistogram(&buf, h.H) - } - } - return buf.Get() } @@ -841,12 +809,12 @@ func EncodeHistogram(buf *encoding.Encbuf, h *histogram.Histogram) { } } -func (e *Encoder) FloatHistogramSamples(histograms []RefFloatHistogramSample, b []byte) ([]byte, bool) { +func (e *Encoder) FloatHistogramSamples(histograms []RefFloatHistogramSample, b []byte) []byte { buf := encoding.Encbuf{B: b} buf.PutByte(byte(FloatHistogramSamples)) if len(histograms) == 0 { - return buf.Get(), false + return buf.Get() } // Store base timestamp and base reference number of first histogram. @@ -855,45 +823,13 @@ func (e *Encoder) FloatHistogramSamples(histograms []RefFloatHistogramSample, b buf.PutBE64(uint64(first.Ref)) buf.PutBE64int64(first.T) - customBucketsExist := false for _, h := range histograms { - if h.FH.UsesCustomBuckets() { - customBucketsExist = true - continue - } - buf.PutVarint64(int64(h.Ref) - int64(first.Ref)) buf.PutVarint64(h.T - first.T) EncodeFloatHistogram(&buf, h.FH) } - return buf.Get(), customBucketsExist -} - -func (e *Encoder) CustomBucketFloatHistogramSamples(histograms []RefFloatHistogramSample, b []byte) []byte { - buf := encoding.Encbuf{B: b} - buf.PutByte(byte(CustomBucketFloatHistogramSamples)) - - if len(histograms) == 0 { - return buf.Get() - } - - // Store base timestamp and base reference number of first histogram. - // All histograms encode their timestamp and ref as delta to those. - first := histograms[0] - buf.PutBE64(uint64(first.Ref)) - buf.PutBE64int64(first.T) - - for _, h := range histograms { - if h.FH.UsesCustomBuckets() { - buf.PutVarint64(int64(h.Ref) - int64(first.Ref)) - buf.PutVarint64(h.T - first.T) - - EncodeFloatHistogram(&buf, h.FH) - } - } - return buf.Get() } diff --git a/tsdb/record/record_test.go b/tsdb/record/record_test.go index af94f2b207..901fe2e9f6 100644 --- a/tsdb/record/record_test.go +++ b/tsdb/record/record_test.go @@ -166,12 +166,9 @@ func TestRecord_EncodeDecode(t *testing.T) { }, } - histSamples, _ := enc.HistogramSamples(histograms, nil) - customBucketHistSamples := enc.CustomBucketHistogramSamples(histograms, nil) + histSamples := enc.HistogramSamples(histograms, nil) decHistograms, err := dec.HistogramSamples(histSamples, nil) require.NoError(t, err) - decCustomBucketHistSamples, err := dec.HistogramSamples(customBucketHistSamples, nil) - decHistograms = append(decHistograms, decCustomBucketHistSamples...) require.Equal(t, histograms, decHistograms) floatHistograms := make([]RefFloatHistogramSample, len(histograms)) @@ -182,13 +179,9 @@ func TestRecord_EncodeDecode(t *testing.T) { FH: h.H.ToFloat(nil), } } - floatHistSamples, _ := enc.FloatHistogramSamples(floatHistograms, nil) - customBucketFloatHistSamples := enc.CustomBucketFloatHistogramSamples(floatHistograms, nil) + floatHistSamples := enc.FloatHistogramSamples(floatHistograms, nil) decFloatHistograms, err := dec.FloatHistogramSamples(floatHistSamples, nil) require.NoError(t, err) - decCustomBucketFloatHistograms, err := dec.FloatHistogramSamples(customBucketFloatHistSamples, nil) - require.NoError(t, err) - decFloatHistograms = append(decFloatHistograms, decCustomBucketFloatHistograms...) require.Equal(t, floatHistograms, decFloatHistograms) // Gauge integer histograms. @@ -196,13 +189,9 @@ func TestRecord_EncodeDecode(t *testing.T) { histograms[i].H.CounterResetHint = histogram.GaugeType } - gaugeHistSamples, _ := enc.HistogramSamples(histograms, nil) - customBucketGaugeHistSamples := enc.CustomBucketHistogramSamples(histograms, nil) + gaugeHistSamples := enc.HistogramSamples(histograms, nil) decGaugeHistograms, err := dec.HistogramSamples(gaugeHistSamples, nil) require.NoError(t, err) - decCustomBucketGaugeHistograms, err := dec.HistogramSamples(customBucketGaugeHistSamples, nil) - require.NoError(t, err) - decGaugeHistograms = append(decGaugeHistograms, decCustomBucketGaugeHistograms...) require.Equal(t, histograms, decGaugeHistograms) // Gauge float histograms. @@ -210,14 +199,10 @@ func TestRecord_EncodeDecode(t *testing.T) { floatHistograms[i].FH.CounterResetHint = histogram.GaugeType } - gaugeFloatHistSamples, _ := enc.FloatHistogramSamples(floatHistograms, nil) - customBucketGaugeFloatHistSamples := enc.CustomBucketFloatHistogramSamples(floatHistograms, nil) + gaugeFloatHistSamples := enc.FloatHistogramSamples(floatHistograms, nil) decGaugeFloatHistograms, err := dec.FloatHistogramSamples(gaugeFloatHistSamples, nil) require.NoError(t, err) - decCustomBucketGaugeFloatHistograms, err := dec.FloatHistogramSamples(customBucketGaugeFloatHistSamples, nil) - require.NoError(t, err) - decFloatHistograms = append(decGaugeFloatHistograms, decCustomBucketGaugeFloatHistograms...) - require.Equal(t, floatHistograms, decFloatHistograms) + require.Equal(t, floatHistograms, decGaugeFloatHistograms) } // TestRecord_Corrupted ensures that corrupted records return the correct error. @@ -318,14 +303,10 @@ func TestRecord_Corrupted(t *testing.T) { }, } - corruptedHists, _ := enc.HistogramSamples(histograms, nil) + corruptedHists := enc.HistogramSamples(histograms, nil) corruptedHists = corruptedHists[:8] - corruptedCustomBucketHists := enc.CustomBucketHistogramSamples(histograms, nil) - corruptedCustomBucketHists = corruptedCustomBucketHists[:8] _, err := dec.HistogramSamples(corruptedHists, nil) require.ErrorIs(t, err, encoding.ErrInvalidSize) - _, err = dec.HistogramSamples(corruptedCustomBucketHists, nil) - require.ErrorIs(t, err, encoding.ErrInvalidSize) }) } @@ -383,12 +364,9 @@ func TestRecord_Type(t *testing.T) { }, }, } - hists, _ := enc.HistogramSamples(histograms, nil) + hists := enc.HistogramSamples(histograms, nil) recordType = dec.Type(hists) require.Equal(t, HistogramSamples, recordType) - customBucketHists := enc.CustomBucketHistogramSamples(histograms, nil) - recordType = dec.Type(customBucketHists) - require.Equal(t, CustomBucketHistogramSamples, recordType) recordType = dec.Type(nil) require.Equal(t, Unknown, recordType) diff --git a/tsdb/tsdbutil/histogram.go b/tsdb/tsdbutil/histogram.go index 4b6cebd579..9230b68376 100644 --- a/tsdb/tsdbutil/histogram.go +++ b/tsdb/tsdbutil/histogram.go @@ -64,7 +64,6 @@ func GenerateTestCustomBucketsHistograms(n int) (r []*histogram.Histogram) { h.CounterResetHint = histogram.NotCounterReset } r = append(r, h) - } return r } diff --git a/tsdb/wlog/checkpoint.go b/tsdb/wlog/checkpoint.go index 5bb79595b9..ffb96dbe22 100644 --- a/tsdb/wlog/checkpoint.go +++ b/tsdb/wlog/checkpoint.go @@ -208,7 +208,7 @@ func Checkpoint(logger *slog.Logger, w *WL, from, to int, keep func(id chunks.He stats.TotalSamples += len(samples) stats.DroppedSamples += len(samples) - len(repl) - case record.HistogramSamples: + case record.HistogramSamples, record.HistogramSamplesLegacy: histogramSamples, err = dec.HistogramSamples(rec, histogramSamples) if err != nil { return nil, fmt.Errorf("decode histogram samples: %w", err) @@ -221,45 +221,11 @@ func Checkpoint(logger *slog.Logger, w *WL, from, to int, keep func(id chunks.He } } if len(repl) > 0 { - buf, _ = enc.HistogramSamples(repl, buf) + buf = enc.HistogramSamples(repl, buf) } stats.TotalSamples += len(histogramSamples) stats.DroppedSamples += len(histogramSamples) - len(repl) - case record.CustomBucketHistogramSamples: - histogramSamples, err = dec.HistogramSamples(rec, histogramSamples) - if err != nil { - return nil, fmt.Errorf("decode histogram samples: %w", err) - } - // Drop irrelevant histogramSamples in place. - repl := histogramSamples[:0] - for _, h := range histogramSamples { - if h.T >= mint { - repl = append(repl, h) - } - } - if len(repl) > 0 { - buf = enc.CustomBucketHistogramSamples(repl, buf) - } - stats.TotalSamples += len(histogramSamples) - stats.DroppedSamples += len(histogramSamples) - len(repl) - case record.FloatHistogramSamples: - floatHistogramSamples, err = dec.FloatHistogramSamples(rec, floatHistogramSamples) - if err != nil { - return nil, fmt.Errorf("decode float histogram samples: %w", err) - } - // Drop irrelevant floatHistogramSamples in place. - repl := floatHistogramSamples[:0] - for _, fh := range floatHistogramSamples { - if fh.T >= mint { - repl = append(repl, fh) - } - } - if len(repl) > 0 { - buf, _ = enc.FloatHistogramSamples(repl, buf) - } - stats.TotalSamples += len(floatHistogramSamples) - stats.DroppedSamples += len(floatHistogramSamples) - len(repl) - case record.CustomBucketFloatHistogramSamples: + case record.FloatHistogramSamples, record.FloatHistogramSamplesLegacy: floatHistogramSamples, err = dec.FloatHistogramSamples(rec, floatHistogramSamples) if err != nil { return nil, fmt.Errorf("decode float histogram samples: %w", err) @@ -272,7 +238,7 @@ func Checkpoint(logger *slog.Logger, w *WL, from, to int, keep func(id chunks.He } } if len(repl) > 0 { - buf = enc.CustomBucketFloatHistogramSamples(repl, buf) + buf = enc.FloatHistogramSamples(repl, buf) } stats.TotalSamples += len(floatHistogramSamples) stats.DroppedSamples += len(floatHistogramSamples) - len(repl) diff --git a/tsdb/wlog/checkpoint_test.go b/tsdb/wlog/checkpoint_test.go index f947f28095..b2c603f134 100644 --- a/tsdb/wlog/checkpoint_test.go +++ b/tsdb/wlog/checkpoint_test.go @@ -236,7 +236,7 @@ func TestCheckpoint(t *testing.T) { require.NoError(t, w.Log(b)) samplesInWAL += 4 h := makeHistogram(i) - b, _ = enc.HistogramSamples([]record.RefHistogramSample{ + b = enc.HistogramSamples([]record.RefHistogramSample{ {Ref: 0, T: last, H: h}, {Ref: 1, T: last + 10000, H: h}, {Ref: 2, T: last + 20000, H: h}, @@ -245,7 +245,7 @@ func TestCheckpoint(t *testing.T) { require.NoError(t, w.Log(b)) histogramsInWAL += 4 cbh := makeCustomBucketHistogram(i) - b = enc.CustomBucketHistogramSamples([]record.RefHistogramSample{ + b = enc.HistogramSamples([]record.RefHistogramSample{ {Ref: 0, T: last, H: cbh}, {Ref: 1, T: last + 10000, H: cbh}, {Ref: 2, T: last + 20000, H: cbh}, @@ -254,7 +254,7 @@ func TestCheckpoint(t *testing.T) { require.NoError(t, w.Log(b)) histogramsInWAL += 4 fh := makeFloatHistogram(i) - b, _ = enc.FloatHistogramSamples([]record.RefFloatHistogramSample{ + b = enc.FloatHistogramSamples([]record.RefFloatHistogramSample{ {Ref: 0, T: last, FH: fh}, {Ref: 1, T: last + 10000, FH: fh}, {Ref: 2, T: last + 20000, FH: fh}, @@ -263,7 +263,7 @@ func TestCheckpoint(t *testing.T) { require.NoError(t, w.Log(b)) floatHistogramsInWAL += 4 cbfh := makeCustomBucketFloatHistogram(i) - b = enc.CustomBucketFloatHistogramSamples([]record.RefFloatHistogramSample{ + b = enc.FloatHistogramSamples([]record.RefFloatHistogramSample{ {Ref: 0, T: last, FH: cbfh}, {Ref: 1, T: last + 10000, FH: cbfh}, {Ref: 2, T: last + 20000, FH: cbfh}, @@ -330,14 +330,14 @@ func TestCheckpoint(t *testing.T) { require.GreaterOrEqual(t, s.T, last/2, "sample with wrong timestamp") } samplesInCheckpoint += len(samples) - case record.HistogramSamples, record.CustomBucketHistogramSamples: + case record.HistogramSamples, record.HistogramSamplesLegacy: histograms, err := dec.HistogramSamples(rec, nil) require.NoError(t, err) for _, h := range histograms { require.GreaterOrEqual(t, h.T, last/2, "histogram with wrong timestamp") } histogramsInCheckpoint += len(histograms) - case record.FloatHistogramSamples, record.CustomBucketFloatHistogramSamples: + case record.FloatHistogramSamples, record.FloatHistogramSamplesLegacy: floatHistograms, err := dec.FloatHistogramSamples(rec, nil) require.NoError(t, err) for _, h := range floatHistograms { diff --git a/tsdb/wlog/watcher.go b/tsdb/wlog/watcher.go index 169bd296fe..07f881eeaf 100644 --- a/tsdb/wlog/watcher.go +++ b/tsdb/wlog/watcher.go @@ -546,7 +546,7 @@ func (w *Watcher) readSegment(r *LiveReader, segmentNum int, tail bool) error { } w.writer.AppendExemplars(exemplars) - case record.HistogramSamples, record.CustomBucketHistogramSamples: + case record.HistogramSamples, record.HistogramSamplesLegacy: // Skip if experimental "histograms over remote write" is not enabled. if !w.sendHistograms { break @@ -574,7 +574,7 @@ func (w *Watcher) readSegment(r *LiveReader, segmentNum int, tail bool) error { histogramsToSend = histogramsToSend[:0] } - case record.FloatHistogramSamples, record.CustomBucketFloatHistogramSamples: + case record.FloatHistogramSamples, record.FloatHistogramSamplesLegacy: // Skip if experimental "histograms over remote write" is not enabled. if !w.sendHistograms { break diff --git a/tsdb/wlog/watcher_test.go b/tsdb/wlog/watcher_test.go index 5ff70bb215..21490154d9 100644 --- a/tsdb/wlog/watcher_test.go +++ b/tsdb/wlog/watcher_test.go @@ -209,7 +209,7 @@ func TestTailSamples(t *testing.T) { NegativeBuckets: []int64{int64(-i) - 1}, } - histograms, _ := enc.HistogramSamples([]record.RefHistogramSample{{ + histograms := enc.HistogramSamples([]record.RefHistogramSample{{ Ref: chunks.HeadSeriesRef(inner), T: now.UnixNano() + 1, H: hist, @@ -226,21 +226,21 @@ func TestTailSamples(t *testing.T) { CustomValues: []float64{float64(i) + 2}, } - customBucketHistograms := enc.CustomBucketHistogramSamples([]record.RefHistogramSample{{ + customBucketHistograms := enc.HistogramSamples([]record.RefHistogramSample{{ Ref: chunks.HeadSeriesRef(inner), T: now.UnixNano() + 1, H: customBucketHist, }}, nil) require.NoError(t, w.Log(customBucketHistograms)) - floatHistograms, _ := enc.FloatHistogramSamples([]record.RefFloatHistogramSample{{ + floatHistograms := enc.FloatHistogramSamples([]record.RefFloatHistogramSample{{ Ref: chunks.HeadSeriesRef(inner), T: now.UnixNano() + 1, FH: hist.ToFloat(nil), }}, nil) require.NoError(t, w.Log(floatHistograms)) - customBucketFloatHistograms := enc.CustomBucketFloatHistogramSamples([]record.RefFloatHistogramSample{{ + customBucketFloatHistograms := enc.FloatHistogramSamples([]record.RefFloatHistogramSample{{ Ref: chunks.HeadSeriesRef(inner), T: now.UnixNano() + 1, FH: customBucketHist.ToFloat(nil), From f8a39767a43eee03396912a3b6028bd4b9a88284 Mon Sep 17 00:00:00 2001 From: Carrie Edwards Date: Tue, 26 Nov 2024 11:14:09 -0800 Subject: [PATCH 07/40] Update WAL doc to include native histogram encodings --- tsdb/docs/format/wal.md | 216 +++++++++++++++++++++++++++++++++++----- tsdb/head_append.go | 1 + tsdb/head_wal.go | 5 +- 3 files changed, 192 insertions(+), 30 deletions(-) diff --git a/tsdb/docs/format/wal.md b/tsdb/docs/format/wal.md index 835ede4113..092999a53a 100644 --- a/tsdb/docs/format/wal.md +++ b/tsdb/docs/format/wal.md @@ -79,32 +79,6 @@ The first sample record begins at the second row. └──────────────────────────────────────────────────────────────────┘ ``` -### Native histogram records - -Native histogram records are encoded as - -``` -┌──────────────────────────────────────────────────────────────────┐ -│ type = 2 <1b> │ -├──────────────────────────────────────────────────────────────────┤ -│ ┌────────────────────┬───────────────────────────┐ │ -│ │ id <8b> │ timestamp <8b> │ │ -│ └────────────────────┴───────────────────────────┘ │ -│ ┌────────────────────┬───────────────────────────┬ │ -│ │ id_delta │ timestamp_delta │ │ -│ ├────────────────────┴───────────────────────────┴─────────────┤ │ -│ │ n = len(labels) │ │ -│ ├──────────────────────┬───────────────────────────────────────┤ │ -│ │ len(str_1) │ str_1 │ │ -│ ├──────────────────────┴───────────────────────────────────────┤ │ -│ │ ... │ │ -│ ├───────────────────────┬──────────────────────────────────────┤ │ -│ │ len(str_2n) │ str_2n │ │ │ -│ └───────────────────────┴────────────────┴─────────────────────┘ │ -│ . . . │ -└──────────────────────────────────────────────────────────────────┘ -``` - ### Tombstone records Tombstone records encode tombstones as a list of triples `(series_id, min_time, max_time)` @@ -182,3 +156,193 @@ Metadata records encode the metadata updates associated with a series. └────────────────────────────────────────────┘ ``` +### Native histogram records + +Native histogram records are encoded as a list of histogram samples. +Series reference and timestamp are encoded as deltas w.r.t the first histogram sample. +The first row stores the starting id and the starting timestamp. +The first native histogram sample record begins at the second row. + +There are several different types of native histogram samples. + +Integer histogram encoding: + +``` +┌──────────────────────────────────────────────────────────────────┐ +│ type = 9 <1b> │ +├──────────────────────────────────────────────────────────────────┤ +│ ┌────────────────────┬───────────────────────────┐ │ +│ │ id <8b> │ timestamp <8b> │ │ +│ └────────────────────┴───────────────────────────┘ │ +│ ┌────────────────────┬───────────────────────────┬ │ +│ │ id_delta │ timestamp_delta │ │ +│ ├────────────────────┴────┬──────────────────────┤ │ +│ │ counter_reset_hint <8b> │ schema │ │ +│ ├──────────────────────┬──┴──────────────────────┤ │ +│ │ zero_threshold <8b> │ zero_count │ │ +│ ├──────────────────────┴┬────────────────────────┤ │ +│ │ count │ sum │ │ +│ ├───────────────────────┴────────┬───────────────┴─────────────┐ │ +│ │ len(positive_spans) │ offset │ │ +│ ├────────────────────────────────┼─────────────────────────────┤ │ +│ │ │ length │ │ +│ │ ├─────────────────────────────┤ │ +│ │ │ . . . │ │ +│ ├────────────────────────────────┼─────────────────────────────┤ │ +│ │ len(negative_spans) │ offset │ │ +│ ├────────────────────────────────┼─────────────────────────────┤ │ +│ │ │ length │ │ +│ │ ├─────────────────────────────┤ │ +│ │ │ . . . │ │ +│ ├────────────────────────────────┼─────────────────────────────┤ │ +│ │ len(positive_buckets) │ bucket_count │ │ +│ ├────────────────────────────────┼─────────────────────────────┤ │ +│ │ │ . . . │ │ +│ ├────────────────────────────────┼─────────────────────────────┤ │ +│ │ len(negative_buckets) │ bucket_count │ │ +│ ├────────────────────────────────┼─────────────────────────────┤ │ +│ │ │ . . . │ │ +│ └────────────────────────────────┴─────────────────────────────┘ │ +│ . . . │ +└──────────────────────────────────────────────────────────────────┘ +``` + +There are also integer histograms that have custom buckets, which will always +have a schem of 053. Custom bucket native histograms additionally encode +a field that specifies the custom values: + +``` +┌──────────────────────────────────────────────────────────────────┐ +│ type = 9 <1b> │ +├──────────────────────────────────────────────────────────────────┤ +│ ┌────────────────────┬───────────────────────────┐ │ +│ │ id <8b> │ timestamp <8b> │ │ +│ └────────────────────┴───────────────────────────┘ │ +│ ┌────────────────────┬───────────────────────────┬ │ +│ │ id_delta │ timestamp_delta │ │ +│ ├────────────────────┴────┬──────────────────────┤ │ +│ │ counter_reset_hint <8b> │ schema │ │ +│ ├──────────────────────┬──┴──────────────────────┤ │ +│ │ zero_threshold <8b> │ zero_count │ │ +│ ├──────────────────────┴┬────────────────────────┤ │ +│ │ count │ sum │ │ +│ ├───────────────────────┴────────┬───────────────┴─────────────┐ │ +│ │ len(positive_spans) │ offset │ │ +│ ├────────────────────────────────┼─────────────────────────────┤ │ +│ │ │ length │ │ +│ │ ├─────────────────────────────┤ │ +│ │ │ . . . │ │ +│ ├────────────────────────────────┼─────────────────────────────┤ │ +│ │ len(negative_spans) │ offset │ │ +│ ├────────────────────────────────┼─────────────────────────────┤ │ +│ │ │ length │ │ +│ │ ├─────────────────────────────┤ │ +│ │ │ . . . │ │ +│ ├────────────────────────────────┼─────────────────────────────┤ │ +│ │ len(positive_buckets) │ bucket_count │ │ +│ ├────────────────────────────────┼─────────────────────────────┤ │ +│ │ │ . . . │ │ +│ ├────────────────────────────────┼─────────────────────────────┤ │ +│ │ len(negative_buckets) │ bucket_count │ │ +│ ├────────────────────────────────┼─────────────────────────────┤ │ +│ │ │ . . . │ │ +│ ├────────────────────────────────┼─────────────────────────────┤ │ +│ │ len(custom_values) │ value <8b> │ │ +│ ├────────────────────────────────┼─────────────────────────────┤ │ +│ │ │ . . . │ │ +│ └────────────────────────────────┴─────────────────────────────┘ │ +│ . . . │ +└──────────────────────────────────────────────────────────────────┘ +``` + +(Note: negative spans and negative buckets will be empty for custom bucket native histograms.) + +Float histogram encoding: + +``` +┌──────────────────────────────────────────────────────────────────┐ +│ type = 10 <1b> │ +├──────────────────────────────────────────────────────────────────┤ +│ ┌────────────────────┬───────────────────────────┐ │ +│ │ id <8b> │ timestamp <8b> │ │ +│ └────────────────────┴───────────────────────────┘ │ +│ ┌────────────────────┬───────────────────────────┐ │ +│ │ id_delta │ timestamp_delta │ │ +│ ├────────────────────┴────┬──────────────────────┤ │ +│ │ counter_reset_hint <8b> │ schema │ │ +│ ├──────────────────────┬──┴──────────────────────┤ │ +│ │ zero_threshold <8b> │ zero_count <8b> │ │ +│ ├──────────────────────┴┬────────────────────────┤ │ +│ │ count <8b> │ sum <8b> │ │ +│ ├───────────────────────┴────────┬───────────────┴─────────────┐ │ +│ │ len(positive_spans) │ offset │ │ +│ ├────────────────────────────────┼─────────────────────────────┤ │ +│ │ │ length │ │ +│ │ ├─────────────────────────────┤ │ +│ │ │ . . . │ │ +│ ├────────────────────────────────┼─────────────────────────────┤ │ +│ │ len(negative_spans) │ offset │ │ +│ ├────────────────────────────────┼─────────────────────────────┤ │ +│ │ │ length │ │ +│ │ ├─────────────────────────────┤ │ +│ │ │ . . . │ │ +│ ├────────────────────────────────┼─────────────────────────────┤ │ +│ │ len(positive_buckets) │ bucket_count <8b> │ │ +│ ├────────────────────────────────┼─────────────────────────────┤ │ +│ │ │ . . . │ │ +│ ├────────────────────────────────┼─────────────────────────────┤ │ +│ │ len(negative_buckets) │ bucket_count <8b> │ │ +│ ├────────────────────────────────┼─────────────────────────────┤ │ +│ │ │ . . . │ │ +│ └────────────────────────────────┴─────────────────────────────┘ │ +│ . . . │ +└──────────────────────────────────────────────────────────────────┘ +``` + +There are also float histograms with custom buckets. + +``` +┌──────────────────────────────────────────────────────────────────┐ +│ type = 10 <1b> │ +├──────────────────────────────────────────────────────────────────┤ +│ ┌────────────────────┬───────────────────────────┐ │ +│ │ id <8b> │ timestamp <8b> │ │ +│ └────────────────────┴───────────────────────────┘ │ +│ ┌────────────────────┬───────────────────────────┐ │ +│ │ id_delta │ timestamp_delta │ │ +│ ├────────────────────┴────┬──────────────────────┤ │ +│ │ counter_reset_hint <8b> │ schema │ │ +│ ├──────────────────────┬──┴──────────────────────┤ │ +│ │ zero_threshold <8b> │ zero_count <8b> │ │ +│ ├──────────────────────┴┬────────────────────────┤ │ +│ │ count <8b> │ sum <8b> │ │ +│ ├───────────────────────┴────────┬───────────────┴─────────────┐ │ +│ │ len(positive_spans) │ offset │ │ +│ ├────────────────────────────────┼─────────────────────────────┤ │ +│ │ │ length │ │ +│ │ ├─────────────────────────────┤ │ +│ │ │ . . . │ │ +│ ├────────────────────────────────┼─────────────────────────────┤ │ +│ │ len(negative_spans) │ offset │ │ +│ ├────────────────────────────────┼─────────────────────────────┤ │ +│ │ │ length │ │ +│ │ ├─────────────────────────────┤ │ +│ │ │ . . . │ │ +│ ├────────────────────────────────┼─────────────────────────────┤ │ +│ │ len(positive_buckets) │ bucket_count <8b> │ │ +│ ├────────────────────────────────┼─────────────────────────────┤ │ +│ │ │ . . . │ │ +│ ├────────────────────────────────┼─────────────────────────────┤ │ +│ │ len(negative_buckets) │ bucket_count <8b> │ │ +│ ├────────────────────────────────┼─────────────────────────────┤ │ +│ │ │ . . . │ │ +│ ├────────────────────────────────┼─────────────────────────────┤ │ +│ │ len(custom_values) │ value <8b> │ │ +│ ├────────────────────────────────┼─────────────────────────────┤ │ +│ │ │ . . . │ │ +│ └────────────────────────────────┴─────────────────────────────┘ │ +│ . . . │ +└──────────────────────────────────────────────────────────────────┘ +``` + +(Note: negative spans and negative buckets will also be empty for custom bucket float native histograms.) diff --git a/tsdb/head_append.go b/tsdb/head_append.go index 7dacb9037b..1cac44e160 100644 --- a/tsdb/head_append.go +++ b/tsdb/head_append.go @@ -691,6 +691,7 @@ func (a *headAppender) AppendHistogram(ref storage.SeriesRef, lset labels.Labels if created && s.lastHistogramValue == nil && s.lastFloatHistogramValue == nil { s.lastHistogramValue = &histogram.Histogram{} } + // 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, a.head.opts.EnableOOONativeHistograms.Load()) diff --git a/tsdb/head_wal.go b/tsdb/head_wal.go index 458162522b..d71dc9d33d 100644 --- a/tsdb/head_wal.go +++ b/tsdb/head_wal.go @@ -58,7 +58,6 @@ func (h *Head) loadWAL(r *wlog.Reader, syms *labels.SymbolTable, multiRef map[ch var unknownExemplarRefs atomic.Uint64 var unknownHistogramRefs atomic.Uint64 var unknownMetadataRefs atomic.Uint64 - // Track number of series records that had overlapping m-map chunks. var mmapOverlappingChunks atomic.Uint64 @@ -139,8 +138,7 @@ func (h *Head) loadWAL(r *wlog.Reader, syms *labels.SymbolTable, multiRef map[ch dec := record.NewDecoder(syms) for r.Next() { rec := r.Record() - recType := dec.Type(rec) - switch recType { + switch dec.Type(rec) { case record.Series: series := seriesPool.Get()[:0] series, err = dec.Series(rec, series) @@ -618,7 +616,6 @@ func (wp *walSubsetProcessor) processWALSamples(h *Head, mmappedChunks, oooMmapp if s.t <= ms.mmMaxTime { continue } - var chunkCreated bool if s.h != nil { _, chunkCreated = ms.appendHistogram(s.t, s.h, 0, appendChunkOpts) From 6b44c1437f296569254c166e3c3ff0dc949f0e35 Mon Sep 17 00:00:00 2001 From: Carrie Edwards Date: Wed, 27 Nov 2024 09:24:59 -0800 Subject: [PATCH 08/40] Fix comment and histogram record string --- tsdb/record/record.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tsdb/record/record.go b/tsdb/record/record.go index 2dd7ffe027..0707ed54fe 100644 --- a/tsdb/record/record.go +++ b/tsdb/record/record.go @@ -48,9 +48,9 @@ const ( MmapMarkers Type = 5 // Metadata is used to match WAL records of type Metadata. Metadata Type = 6 - // HistogramSamplesLegacy is used to match WAL records of type Histograms prior to intrdocuing support of custom buckets, to maintain backwards compatibility. + // HistogramSamplesLegacy is used to match WAL records of type Histograms prior to introducing support of custom buckets, for backwards compatibility. HistogramSamplesLegacy Type = 7 - // FloatHistogramSamplesLegacy is used to match WAL records of type Float Histograms proior to introducing support of custom buckets, to maintain backwards compatibility. + // FloatHistogramSamplesLegacy is used to match WAL records of type Float Histograms prior to introducing support of custom buckets, for backwards compatibility. FloatHistogramSamplesLegacy Type = 8 // HistogramSamples is used to match WAL records of type Histogram, and supports custom buckets. HistogramSamples Type = 9 @@ -73,7 +73,7 @@ func (rt Type) String() string { case FloatHistogramSamplesLegacy: return "float_histogram_samples_legacy" case HistogramSamples: - return "histogram_sample" + return "histogram_samples" case FloatHistogramSamples: return "float_histogram_samples" case MmapMarkers: From 45944c1847d453c41870d6664c352a44d02682e3 Mon Sep 17 00:00:00 2001 From: Carrie Edwards Date: Wed, 27 Nov 2024 09:26:46 -0800 Subject: [PATCH 09/40] Extend tsdb agent tests with custom bucket histograms --- tsdb/agent/db_test.go | 132 ++++++++++++++++++++++++++++++++++++++++-- 1 file changed, 128 insertions(+), 4 deletions(-) diff --git a/tsdb/agent/db_test.go b/tsdb/agent/db_test.go index 8bcb71c86a..cc5e0d34cb 100644 --- a/tsdb/agent/db_test.go +++ b/tsdb/agent/db_test.go @@ -524,6 +524,19 @@ func TestPartialTruncateWAL(t *testing.T) { require.NoError(t, app.Commit()) } + lbls = labelsForTest(t.Name()+"_custom_buckets_histogram_batch-1", numSeries) + for _, l := range lbls { + lset := labels.New(l...) + + histograms := tsdbutil.GenerateTestCustomBucketsHistograms(numDatapoints) + + for i := 0; i < numDatapoints; i++ { + _, err := app.AppendHistogram(0, lset, lastTs, histograms[i], nil) + require.NoError(t, err) + } + require.NoError(t, app.Commit()) + } + lbls = labelsForTest(t.Name()+"_float_histogram_batch-1", numSeries) for _, l := range lbls { lset := labels.New(l...) @@ -537,6 +550,19 @@ func TestPartialTruncateWAL(t *testing.T) { require.NoError(t, app.Commit()) } + lbls = labelsForTest(t.Name()+"_custom_buckets_float_histogram_batch-1", numSeries) + for _, l := range lbls { + lset := labels.New(l...) + + floatHistograms := tsdbutil.GenerateTestCustomBucketsFloatHistograms(numDatapoints) + + for i := 0; i < numDatapoints; i++ { + _, err := app.AppendHistogram(0, lset, lastTs, nil, floatHistograms[i]) + require.NoError(t, err) + } + require.NoError(t, app.Commit()) + } + // Create second batch of 800 series with 1000 data-points with a fixed lastTs as 600. lastTs = 600 lbls = labelsForTest(t.Name()+"batch-2", numSeries) @@ -563,6 +589,19 @@ func TestPartialTruncateWAL(t *testing.T) { require.NoError(t, app.Commit()) } + lbls = labelsForTest(t.Name()+"_custom_buckets_histogram_batch-2", numSeries) + for _, l := range lbls { + lset := labels.New(l...) + + histograms := tsdbutil.GenerateTestCustomBucketsHistograms(numDatapoints) + + for i := 0; i < numDatapoints; i++ { + _, err := app.AppendHistogram(0, lset, lastTs, histograms[i], nil) + require.NoError(t, err) + } + require.NoError(t, app.Commit()) + } + lbls = labelsForTest(t.Name()+"_float_histogram_batch-2", numSeries) for _, l := range lbls { lset := labels.New(l...) @@ -576,11 +615,24 @@ func TestPartialTruncateWAL(t *testing.T) { require.NoError(t, app.Commit()) } + lbls = labelsForTest(t.Name()+"_custom_buckets_float_histogram_batch-2", numSeries) + for _, l := range lbls { + lset := labels.New(l...) + + floatHistograms := tsdbutil.GenerateTestCustomBucketsFloatHistograms(numDatapoints) + + for i := 0; i < numDatapoints; i++ { + _, err := app.AppendHistogram(0, lset, lastTs, nil, floatHistograms[i]) + require.NoError(t, err) + } + require.NoError(t, app.Commit()) + } + // Truncate WAL with mint to GC only the first batch of 800 series and retaining 2nd batch of 800 series. s.truncate(lastTs - 1) m := gatherFamily(t, reg, "prometheus_agent_deleted_series") - require.Equal(t, float64(numSeries*3), m.Metric[0].Gauge.GetValue(), "agent wal truncate mismatch of deleted series count") + require.Equal(t, float64(numSeries*5), m.Metric[0].Gauge.GetValue(), "agent wal truncate mismatch of deleted series count") } func TestWALReplay(t *testing.T) { @@ -616,6 +668,18 @@ func TestWALReplay(t *testing.T) { } } + lbls = labelsForTest(t.Name()+"_custom_buckets_histogram", numSeries) + for _, l := range lbls { + lset := labels.New(l...) + + histograms := tsdbutil.GenerateTestCustomBucketsHistograms(numHistograms) + + for i := 0; i < numHistograms; i++ { + _, err := app.AppendHistogram(0, lset, lastTs, histograms[i], nil) + require.NoError(t, err) + } + } + lbls = labelsForTest(t.Name()+"_float_histogram", numSeries) for _, l := range lbls { lset := labels.New(l...) @@ -628,6 +692,18 @@ func TestWALReplay(t *testing.T) { } } + lbls = labelsForTest(t.Name()+"_custom_buckets_float_histogram", numSeries) + for _, l := range lbls { + lset := labels.New(l...) + + floatHistograms := tsdbutil.GenerateTestCustomBucketsFloatHistograms(numHistograms) + + for i := 0; i < numHistograms; i++ { + _, err := app.AppendHistogram(0, lset, lastTs, nil, floatHistograms[i]) + require.NoError(t, err) + } + } + require.NoError(t, app.Commit()) require.NoError(t, s.Close()) @@ -646,7 +722,7 @@ func TestWALReplay(t *testing.T) { // Check if all the series are retrieved back from the WAL. m := gatherFamily(t, reg, "prometheus_agent_active_series") - require.Equal(t, float64(numSeries*3), m.Metric[0].Gauge.GetValue(), "agent wal replay mismatch of active series count") + require.Equal(t, float64(numSeries*5), m.Metric[0].Gauge.GetValue(), "agent wal replay mismatch of active series count") // Check if lastTs of the samples retrieved from the WAL is retained. metrics := replayStorage.series.series @@ -878,6 +954,18 @@ func TestDBAllowOOOSamples(t *testing.T) { } } + lbls = labelsForTest(t.Name()+"_custom_buckets_histogram", numSeries) + for _, l := range lbls { + lset := labels.New(l...) + + histograms := tsdbutil.GenerateTestCustomBucketsHistograms(numHistograms) + + for i := offset; i < numDatapoints+offset; i++ { + _, err := app.AppendHistogram(0, lset, int64(i), histograms[i-offset], nil) + require.NoError(t, err) + } + } + lbls = labelsForTest(t.Name()+"_float_histogram", numSeries) for _, l := range lbls { lset := labels.New(l...) @@ -890,10 +978,22 @@ func TestDBAllowOOOSamples(t *testing.T) { } } + lbls = labelsForTest(t.Name()+"_custom_buckets_float_histogram", numSeries) + for _, l := range lbls { + lset := labels.New(l...) + + floatHistograms := tsdbutil.GenerateTestCustomBucketsFloatHistograms(numHistograms) + + for i := offset; i < numDatapoints+offset; i++ { + _, err := app.AppendHistogram(0, lset, int64(i), nil, floatHistograms[i-offset]) + require.NoError(t, err) + } + } + require.NoError(t, app.Commit()) m := gatherFamily(t, reg, "prometheus_agent_samples_appended_total") require.Equal(t, float64(20), m.Metric[0].Counter.GetValue(), "agent wal mismatch of total appended samples") - require.Equal(t, float64(40), m.Metric[1].Counter.GetValue(), "agent wal mismatch of total appended histograms") + require.Equal(t, float64(80), m.Metric[1].Counter.GetValue(), "agent wal mismatch of total appended histograms") require.NoError(t, s.Close()) // Hack: s.wal.Dir() is the /wal subdirectory of the original storage path. @@ -942,6 +1042,18 @@ func TestDBAllowOOOSamples(t *testing.T) { } } + lbls = labelsForTest(t.Name()+"_custom_buckets_histogram", numSeries*2) + for _, l := range lbls { + lset := labels.New(l...) + + histograms := tsdbutil.GenerateTestCustomBucketsHistograms(numHistograms) + + for i := 0; i < numDatapoints; i++ { + _, err := app.AppendHistogram(0, lset, int64(i), histograms[i], nil) + require.NoError(t, err) + } + } + lbls = labelsForTest(t.Name()+"_float_histogram", numSeries*2) for _, l := range lbls { lset := labels.New(l...) @@ -954,10 +1066,22 @@ func TestDBAllowOOOSamples(t *testing.T) { } } + lbls = labelsForTest(t.Name()+"_custom_buckets_float_histogram", numSeries*2) + for _, l := range lbls { + lset := labels.New(l...) + + floatHistograms := tsdbutil.GenerateTestCustomBucketsFloatHistograms(numHistograms) + + for i := 0; i < numDatapoints; i++ { + _, err := app.AppendHistogram(0, lset, int64(i), nil, floatHistograms[i]) + require.NoError(t, err) + } + } + require.NoError(t, app.Commit()) m = gatherFamily(t, reg2, "prometheus_agent_samples_appended_total") require.Equal(t, float64(40), m.Metric[0].Counter.GetValue(), "agent wal mismatch of total appended samples") - require.Equal(t, float64(80), m.Metric[1].Counter.GetValue(), "agent wal mismatch of total appended histograms") + require.Equal(t, float64(160), m.Metric[1].Counter.GetValue(), "agent wal mismatch of total appended histograms") require.NoError(t, db.Close()) } From a046417bc0fd9f741e42d785392a19d8b86547c7 Mon Sep 17 00:00:00 2001 From: Carrie Edwards Date: Fri, 6 Dec 2024 13:46:20 -0800 Subject: [PATCH 10/40] Use new record type only for NHCB --- tsdb/agent/db.go | 40 +++++++--- tsdb/agent/db_test.go | 11 ++- tsdb/db_test.go | 148 ++++++++++++++++++++++------------- tsdb/head_append.go | 63 ++++++++++++--- tsdb/head_test.go | 4 +- tsdb/head_wal.go | 8 +- tsdb/record/record.go | 103 ++++++++++++++++++++---- tsdb/record/record_test.go | 34 ++++++-- tsdb/testutil.go | 18 ++--- tsdb/wlog/checkpoint.go | 39 ++++++++- tsdb/wlog/checkpoint_test.go | 12 +-- tsdb/wlog/watcher.go | 4 +- tsdb/wlog/watcher_test.go | 8 +- 13 files changed, 357 insertions(+), 135 deletions(-) diff --git a/tsdb/agent/db.go b/tsdb/agent/db.go index 5cf56d5871..0bcef8e7bc 100644 --- a/tsdb/agent/db.go +++ b/tsdb/agent/db.go @@ -463,7 +463,7 @@ func (db *DB) loadWAL(r *wlog.Reader, multiRef map[chunks.HeadSeriesRef]chunks.H return } decoded <- samples - case record.HistogramSamples, record.HistogramSamplesLegacy: + case record.HistogramSamples, record.CustomBucketsHistogramSamples: histograms := histogramsPool.Get()[:0] histograms, err = dec.HistogramSamples(rec, histograms) if err != nil { @@ -475,7 +475,7 @@ func (db *DB) loadWAL(r *wlog.Reader, multiRef map[chunks.HeadSeriesRef]chunks.H return } decoded <- histograms - case record.FloatHistogramSamples, record.FloatHistogramSamplesLegacy: + case record.FloatHistogramSamples, record.CustomBucketsFloatHistogramSamples: floatHistograms := floatHistogramsPool.Get()[:0] floatHistograms, err = dec.FloatHistogramSamples(rec, floatHistograms) if err != nil { @@ -1154,19 +1154,39 @@ func (a *appender) log() error { } if len(a.pendingHistograms) > 0 { - buf = encoder.HistogramSamples(a.pendingHistograms, buf) - if err := a.wal.Log(buf); err != nil { - return err + var customBucketsHistograms []record.RefHistogramSample + buf, customBucketsHistograms = encoder.HistogramSamples(a.pendingHistograms, buf) + if len(buf) > 0 { + if err := a.wal.Log(buf); err != nil { + return err + } + buf = buf[:0] + } + if len(customBucketsHistograms) > 0 { + buf = encoder.CustomBucketsHistogramSamples(customBucketsHistograms, nil) + if err := a.wal.Log(buf); err != nil { + return err + } + buf = buf[:0] } - buf = buf[:0] } if len(a.pendingFloatHistograms) > 0 { - buf = encoder.FloatHistogramSamples(a.pendingFloatHistograms, buf) - if err := a.wal.Log(buf); err != nil { - return err + var customBucketsFloatHistograms []record.RefFloatHistogramSample + buf, customBucketsFloatHistograms = encoder.FloatHistogramSamples(a.pendingFloatHistograms, buf) + if len(buf) > 0 { + if err := a.wal.Log(buf); err != nil { + return err + } + buf = buf[:0] + } + if len(customBucketsFloatHistograms) > 0 { + buf = encoder.CustomBucketsFloatHistogramSamples(customBucketsFloatHistograms, nil) + if err := a.wal.Log(buf); err != nil { + return err + } + buf = buf[:0] } - buf = buf[:0] } if len(a.pendingExamplars) > 0 { diff --git a/tsdb/agent/db_test.go b/tsdb/agent/db_test.go index cc5e0d34cb..c81c63f739 100644 --- a/tsdb/agent/db_test.go +++ b/tsdb/agent/db_test.go @@ -217,8 +217,7 @@ func TestCommit(t *testing.T) { ) for r.Next() { rec := r.Record() - recType := dec.Type(rec) - switch recType { + switch dec.Type(rec) { case record.Series: var series []record.RefSeries series, err = dec.Series(rec, series) @@ -231,13 +230,13 @@ func TestCommit(t *testing.T) { require.NoError(t, err) walSamplesCount += len(samples) - case record.HistogramSamples, record.HistogramSamplesLegacy: + 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.FloatHistogramSamplesLegacy: + case record.FloatHistogramSamples, record.CustomBucketsFloatHistogramSamples: var floatHistograms []record.RefFloatHistogramSample floatHistograms, err = dec.FloatHistogramSamples(rec, floatHistograms) require.NoError(t, err) @@ -370,13 +369,13 @@ func TestRollback(t *testing.T) { require.NoError(t, err) walExemplarsCount += len(exemplars) - case record.HistogramSamples, record.HistogramSamplesLegacy: + 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.FloatHistogramSamplesLegacy: + case record.FloatHistogramSamples, record.CustomBucketsFloatHistogramSamples: var floatHistograms []record.RefFloatHistogramSample floatHistograms, err = dec.FloatHistogramSamples(rec, floatHistograms) require.NoError(t, err) diff --git a/tsdb/db_test.go b/tsdb/db_test.go index 4bbf4b4656..5024a0cfbb 100644 --- a/tsdb/db_test.go +++ b/tsdb/db_test.go @@ -4556,11 +4556,11 @@ func testOOOWALWrite(t *testing.T, markers, err := dec.MmapMarkers(rec, nil) require.NoError(t, err) records = append(records, markers) - case record.HistogramSamples, record.HistogramSamplesLegacy: + case record.HistogramSamples, record.CustomBucketsHistogramSamples: histogramSamples, err := dec.HistogramSamples(rec, nil) require.NoError(t, err) records = append(records, histogramSamples) - case record.FloatHistogramSamples, record.FloatHistogramSamplesLegacy: + case record.FloatHistogramSamples, record.CustomBucketsFloatHistogramSamples: floatHistogramSamples, err := dec.FloatHistogramSamples(rec, nil) require.NoError(t, err) records = append(records, floatHistogramSamples) @@ -6461,6 +6461,32 @@ func testOOOInterleavedImplicitCounterResets(t *testing.T, name string, scenario _, 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(v)}, + } + _, 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(v)}, + } + _, err := app.AppendHistogram(0, labels.FromStrings("foo", "bar1"), ts, nil, fh) + return err + } case gaugeIntHistogram, gaugeFloatHistogram: return } @@ -6491,29 +6517,29 @@ func testOOOInterleavedImplicitCounterResets(t *testing.T, name string, scenario // 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-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. @@ -6544,36 +6570,36 @@ func testOOOInterleavedImplicitCounterResets(t *testing.T, name string, scenario {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. - }, - }, + //"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 { @@ -6617,6 +6643,12 @@ func testOOOInterleavedImplicitCounterResets(t *testing.T, name string, scenario 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) } @@ -6648,6 +6680,12 @@ func testOOOInterleavedImplicitCounterResets(t *testing.T, name string, scenario 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) } diff --git a/tsdb/head_append.go b/tsdb/head_append.go index 1cac44e160..c94c42bc53 100644 --- a/tsdb/head_append.go +++ b/tsdb/head_append.go @@ -943,17 +943,37 @@ func (a *headAppender) log() error { } } if len(a.histograms) > 0 { - rec = enc.HistogramSamples(a.histograms, buf) + var customBucketsHistograms []record.RefHistogramSample + rec, customBucketsHistograms = enc.HistogramSamples(a.histograms, buf) buf = rec[:0] - if err := a.head.wal.Log(rec); err != nil { - return fmt.Errorf("log histograms: %w", err) + 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(a.floatHistograms) > 0 { - rec = enc.FloatHistogramSamples(a.floatHistograms, buf) + var customBucketsFloatHistograms []record.RefFloatHistogramSample + rec, customBucketsFloatHistograms = enc.FloatHistogramSamples(a.floatHistograms, buf) buf = rec[:0] - if err := a.head.wal.Log(rec); err != nil { - return fmt.Errorf("log float histograms: %w", err) + 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), @@ -1070,12 +1090,24 @@ func (acc *appenderCommitContext) collectOOORecords(a *headAppender) { acc.oooRecords = append(acc.oooRecords, r) } if len(acc.wblHistograms) > 0 { - r := acc.enc.HistogramSamples(acc.wblHistograms, a.head.getBytesBuffer()) - acc.oooRecords = append(acc.oooRecords, r) + 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 := acc.enc.FloatHistogramSamples(acc.wblFloatHistograms, a.head.getBytesBuffer()) - acc.oooRecords = append(acc.oooRecords, r) + 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 @@ -1459,6 +1491,17 @@ func (a *headAppender) Commit() (err error) { a.commitFloatHistograms(acc) a.commitMetadata() + a.head.metrics.outOfOrderSamples.WithLabelValues(sampleMetricTypeFloat).Add(float64(acc.floatOOORejected)) + a.head.metrics.outOfOrderSamples.WithLabelValues(sampleMetricTypeHistogram).Add(float64(acc.histoOOORejected)) + a.head.metrics.outOfBoundSamples.WithLabelValues(sampleMetricTypeFloat).Add(float64(acc.floatOOBRejected)) + a.head.metrics.tooOldSamples.WithLabelValues(sampleMetricTypeFloat).Add(float64(acc.floatTooOldRejected)) + a.head.metrics.samplesAppended.WithLabelValues(sampleMetricTypeFloat).Add(float64(acc.floatsAppended)) + a.head.metrics.samplesAppended.WithLabelValues(sampleMetricTypeHistogram).Add(float64(acc.histogramsAppended)) + a.head.metrics.outOfOrderSamplesAppended.WithLabelValues(sampleMetricTypeFloat).Add(float64(acc.oooFloatsAccepted)) + a.head.metrics.outOfOrderSamplesAppended.WithLabelValues(sampleMetricTypeHistogram).Add(float64(acc.oooHistogramAccepted)) + a.head.updateMinMaxTime(acc.inOrderMint, acc.inOrderMaxt) + a.head.updateMinOOOMaxOOOTime(acc.oooMinT, acc.oooMaxT) + acc.collectOOORecords(a) if a.head.wbl != nil { if err := a.head.wbl.Log(acc.oooRecords...); err != nil { diff --git a/tsdb/head_test.go b/tsdb/head_test.go index c3377fecff..b77da3e0a4 100644 --- a/tsdb/head_test.go +++ b/tsdb/head_test.go @@ -187,11 +187,11 @@ func readTestWAL(t testing.TB, dir string) (recs []interface{}) { samples, err := dec.Samples(rec, nil) require.NoError(t, err) recs = append(recs, samples) - case record.HistogramSamples, record.HistogramSamplesLegacy: + case record.HistogramSamples, record.CustomBucketsHistogramSamples: samples, err := dec.HistogramSamples(rec, nil) require.NoError(t, err) recs = append(recs, samples) - case record.FloatHistogramSamples, record.FloatHistogramSamplesLegacy: + case record.FloatHistogramSamples, record.CustomBucketsFloatHistogramSamples: samples, err := dec.FloatHistogramSamples(rec, nil) require.NoError(t, err) recs = append(recs, samples) diff --git a/tsdb/head_wal.go b/tsdb/head_wal.go index d71dc9d33d..e9557c59f6 100644 --- a/tsdb/head_wal.go +++ b/tsdb/head_wal.go @@ -187,7 +187,7 @@ func (h *Head) loadWAL(r *wlog.Reader, syms *labels.SymbolTable, multiRef map[ch return } decoded <- exemplars - case record.HistogramSamples, record.HistogramSamplesLegacy: + case record.HistogramSamples, record.CustomBucketsHistogramSamples: hists := histogramsPool.Get()[:0] hists, err = dec.HistogramSamples(rec, hists) if err != nil { @@ -199,7 +199,7 @@ func (h *Head) loadWAL(r *wlog.Reader, syms *labels.SymbolTable, multiRef map[ch return } decoded <- hists - case record.FloatHistogramSamples, record.FloatHistogramSamplesLegacy: + case record.FloatHistogramSamples, record.CustomBucketsFloatHistogramSamples: hists := floatHistogramsPool.Get()[:0] hists, err = dec.FloatHistogramSamples(rec, hists) if err != nil { @@ -723,7 +723,7 @@ func (h *Head) loadWBL(r *wlog.Reader, syms *labels.SymbolTable, multiRef map[ch return } decodedCh <- markers - case record.HistogramSamples, record.HistogramSamplesLegacy: + case record.HistogramSamples, record.CustomBucketsHistogramSamples: hists := histogramSamplesPool.Get()[:0] hists, err = dec.HistogramSamples(rec, hists) if err != nil { @@ -735,7 +735,7 @@ func (h *Head) loadWBL(r *wlog.Reader, syms *labels.SymbolTable, multiRef map[ch return } decodedCh <- hists - case record.FloatHistogramSamples, record.FloatHistogramSamplesLegacy: + case record.FloatHistogramSamples, record.CustomBucketsFloatHistogramSamples: hists := floatHistogramSamplesPool.Get()[:0] hists, err = dec.FloatHistogramSamples(rec, hists) if err != nil { diff --git a/tsdb/record/record.go b/tsdb/record/record.go index 0707ed54fe..ccfbbfcef9 100644 --- a/tsdb/record/record.go +++ b/tsdb/record/record.go @@ -48,14 +48,14 @@ const ( MmapMarkers Type = 5 // Metadata is used to match WAL records of type Metadata. Metadata Type = 6 - // HistogramSamplesLegacy is used to match WAL records of type Histograms prior to introducing support of custom buckets, for backwards compatibility. - HistogramSamplesLegacy Type = 7 - // FloatHistogramSamplesLegacy is used to match WAL records of type Float Histograms prior to introducing support of custom buckets, for backwards compatibility. - FloatHistogramSamplesLegacy Type = 8 - // HistogramSamples is used to match WAL records of type Histogram, and supports custom buckets. - HistogramSamples Type = 9 - // FloatHistogramSamples is used to match WAL records of type Float Histogram, and supports custom buckets. - FloatHistogramSamples Type = 10 + // HistogramSamples is used to match WAL records of type Histograms. + HistogramSamples Type = 7 + // FloatHistogramSamples is used to match WAL records of type Float Histograms. + FloatHistogramSamples Type = 8 + // CustomBucketsHistogramSamples is used to match WAL records of type Histogram with custom buckets. + CustomBucketsHistogramSamples Type = 9 + // CustomBucketsFloatHistogramSamples is used to match WAL records of type Float Histogram with custom buckets. + CustomBucketsFloatHistogramSamples Type = 10 ) func (rt Type) String() string { @@ -68,14 +68,14 @@ func (rt Type) String() string { return "tombstones" case Exemplars: return "exemplars" - case HistogramSamplesLegacy: - return "histogram_samples_legacy" - case FloatHistogramSamplesLegacy: - return "float_histogram_samples_legacy" case HistogramSamples: return "histogram_samples" case FloatHistogramSamples: return "float_histogram_samples" + case CustomBucketsHistogramSamples: + return "custom_buckets_histogram_samples" + case CustomBucketsFloatHistogramSamples: + return "custom_buckets_float_histogram_samples" case MmapMarkers: return "mmapmarkers" case Metadata: @@ -215,7 +215,7 @@ func (d *Decoder) Type(rec []byte) Type { return Unknown } switch t := Type(rec[0]); t { - case Series, Samples, Tombstones, Exemplars, MmapMarkers, Metadata, HistogramSamplesLegacy, FloatHistogramSamplesLegacy, HistogramSamples, FloatHistogramSamples: + case Series, Samples, Tombstones, Exemplars, MmapMarkers, Metadata, HistogramSamples, FloatHistogramSamples, CustomBucketsHistogramSamples, CustomBucketsFloatHistogramSamples: return t } return Unknown @@ -436,7 +436,7 @@ func (d *Decoder) MmapMarkers(rec []byte, markers []RefMmapMarker) ([]RefMmapMar func (d *Decoder) HistogramSamples(rec []byte, histograms []RefHistogramSample) ([]RefHistogramSample, error) { dec := encoding.Decbuf{B: rec} t := Type(dec.Byte()) - if t != HistogramSamples && t != HistogramSamplesLegacy { + if t != HistogramSamples && t != CustomBucketsHistogramSamples { return nil, errors.New("invalid record type") } if dec.Len() == 0 { @@ -528,7 +528,7 @@ func DecodeHistogram(buf *encoding.Decbuf, h *histogram.Histogram) { func (d *Decoder) FloatHistogramSamples(rec []byte, histograms []RefFloatHistogramSample) ([]RefFloatHistogramSample, error) { dec := encoding.Decbuf{B: rec} t := Type(dec.Byte()) - if t != FloatHistogramSamples && t != FloatHistogramSamplesLegacy { + if t != FloatHistogramSamples && t != CustomBucketsFloatHistogramSamples { return nil, errors.New("invalid record type") } if dec.Len() == 0 { @@ -744,10 +744,44 @@ func (e *Encoder) MmapMarkers(markers []RefMmapMarker, b []byte) []byte { return buf.Get() } -func (e *Encoder) HistogramSamples(histograms []RefHistogramSample, b []byte) []byte { +func (e *Encoder) HistogramSamples(histograms []RefHistogramSample, b []byte) ([]byte, []RefHistogramSample) { buf := encoding.Encbuf{B: b} buf.PutByte(byte(HistogramSamples)) + if len(histograms) == 0 { + return buf.Get(), nil + } + var customBucketHistograms []RefHistogramSample + + // Store base timestamp and base reference number of first histogram. + // All histograms encode their timestamp and ref as delta to those. + first := histograms[0] + buf.PutBE64(uint64(first.Ref)) + buf.PutBE64int64(first.T) + + for _, h := range histograms { + if h.H.UsesCustomBuckets() { + customBucketHistograms = append(customBucketHistograms, h) + continue + } + buf.PutVarint64(int64(h.Ref) - int64(first.Ref)) + buf.PutVarint64(h.T - first.T) + + EncodeHistogram(&buf, h.H) + } + + // Reset buffer if only custom bucket histograms existed in list of histogram samples + if len(histograms) == len(customBucketHistograms) { + buf.Reset() + } + + return buf.Get(), customBucketHistograms +} + +func (e *Encoder) CustomBucketsHistogramSamples(histograms []RefHistogramSample, b []byte) []byte { + buf := encoding.Encbuf{B: b} + buf.PutByte(byte(CustomBucketsHistogramSamples)) + if len(histograms) == 0 { return buf.Get() } @@ -809,10 +843,45 @@ func EncodeHistogram(buf *encoding.Encbuf, h *histogram.Histogram) { } } -func (e *Encoder) FloatHistogramSamples(histograms []RefFloatHistogramSample, b []byte) []byte { +func (e *Encoder) FloatHistogramSamples(histograms []RefFloatHistogramSample, b []byte) ([]byte, []RefFloatHistogramSample) { buf := encoding.Encbuf{B: b} buf.PutByte(byte(FloatHistogramSamples)) + if len(histograms) == 0 { + return buf.Get(), nil + } + + var customBucketsFloatHistograms []RefFloatHistogramSample + + // Store base timestamp and base reference number of first histogram. + // All histograms encode their timestamp and ref as delta to those. + first := histograms[0] + buf.PutBE64(uint64(first.Ref)) + buf.PutBE64int64(first.T) + + for _, h := range histograms { + if h.FH.UsesCustomBuckets() { + customBucketsFloatHistograms = append(customBucketsFloatHistograms, h) + continue + } + buf.PutVarint64(int64(h.Ref) - int64(first.Ref)) + buf.PutVarint64(h.T - first.T) + + EncodeFloatHistogram(&buf, h.FH) + } + + // Reset buffer if only custom bucket histograms existed in list of histogram samples + if len(histograms) == len(customBucketsFloatHistograms) { + buf.Reset() + } + + return buf.Get(), customBucketsFloatHistograms +} + +func (e *Encoder) CustomBucketsFloatHistogramSamples(histograms []RefFloatHistogramSample, b []byte) []byte { + buf := encoding.Encbuf{B: b} + buf.PutByte(byte(CustomBucketsFloatHistogramSamples)) + if len(histograms) == 0 { return buf.Get() } diff --git a/tsdb/record/record_test.go b/tsdb/record/record_test.go index 901fe2e9f6..030b7e2bc7 100644 --- a/tsdb/record/record_test.go +++ b/tsdb/record/record_test.go @@ -166,9 +166,13 @@ func TestRecord_EncodeDecode(t *testing.T) { }, } - histSamples := enc.HistogramSamples(histograms, nil) + histSamples, customBucketsHistograms := enc.HistogramSamples(histograms, nil) + customBucketsHistSamples := enc.CustomBucketsHistogramSamples(customBucketsHistograms, nil) decHistograms, err := dec.HistogramSamples(histSamples, nil) require.NoError(t, err) + decCustomBucketsHistograms, err := dec.HistogramSamples(customBucketsHistSamples, nil) + require.NoError(t, err) + decHistograms = append(decHistograms, decCustomBucketsHistograms...) require.Equal(t, histograms, decHistograms) floatHistograms := make([]RefFloatHistogramSample, len(histograms)) @@ -179,9 +183,13 @@ func TestRecord_EncodeDecode(t *testing.T) { FH: h.H.ToFloat(nil), } } - floatHistSamples := enc.FloatHistogramSamples(floatHistograms, nil) + floatHistSamples, customBucketsFloatHistograms := enc.FloatHistogramSamples(floatHistograms, nil) + customBucketsFloatHistSamples := enc.CustomBucketsFloatHistogramSamples(customBucketsFloatHistograms, nil) decFloatHistograms, err := dec.FloatHistogramSamples(floatHistSamples, nil) require.NoError(t, err) + decCustomBucketsFloatHistograms, err := dec.FloatHistogramSamples(customBucketsFloatHistSamples, nil) + require.NoError(t, err) + decFloatHistograms = append(decFloatHistograms, decCustomBucketsFloatHistograms...) require.Equal(t, floatHistograms, decFloatHistograms) // Gauge integer histograms. @@ -189,9 +197,13 @@ func TestRecord_EncodeDecode(t *testing.T) { histograms[i].H.CounterResetHint = histogram.GaugeType } - gaugeHistSamples := enc.HistogramSamples(histograms, nil) + gaugeHistSamples, customBucketsGaugeHistograms := enc.HistogramSamples(histograms, nil) + customBucketsGaugeHistSamples := enc.CustomBucketsHistogramSamples(customBucketsGaugeHistograms, nil) decGaugeHistograms, err := dec.HistogramSamples(gaugeHistSamples, nil) require.NoError(t, err) + decCustomBucketsGaugeHistograms, err := dec.HistogramSamples(customBucketsGaugeHistSamples, nil) + require.NoError(t, err) + decGaugeHistograms = append(decGaugeHistograms, decCustomBucketsGaugeHistograms...) require.Equal(t, histograms, decGaugeHistograms) // Gauge float histograms. @@ -199,9 +211,12 @@ func TestRecord_EncodeDecode(t *testing.T) { floatHistograms[i].FH.CounterResetHint = histogram.GaugeType } - gaugeFloatHistSamples := enc.FloatHistogramSamples(floatHistograms, nil) + gaugeFloatHistSamples, customBucketsGaugeFloatHistograms := enc.FloatHistogramSamples(floatHistograms, nil) + customBucketsGaugeFloatHistSamples := enc.CustomBucketsFloatHistogramSamples(customBucketsGaugeFloatHistograms, nil) decGaugeFloatHistograms, err := dec.FloatHistogramSamples(gaugeFloatHistSamples, nil) require.NoError(t, err) + decCustomBucketsGaugeFloatHistograms, err := dec.FloatHistogramSamples(customBucketsGaugeFloatHistSamples, nil) + decGaugeFloatHistograms = append(decGaugeFloatHistograms, decCustomBucketsGaugeFloatHistograms...) require.Equal(t, floatHistograms, decGaugeFloatHistograms) } @@ -303,10 +318,14 @@ func TestRecord_Corrupted(t *testing.T) { }, } - corruptedHists := enc.HistogramSamples(histograms, nil) + corruptedHists, customBucketsHists := enc.HistogramSamples(histograms, nil) corruptedHists = corruptedHists[:8] + corruptedCustomBucketsHists := enc.CustomBucketsHistogramSamples(customBucketsHists, nil) + corruptedCustomBucketsHists = corruptedCustomBucketsHists[:8] _, err := dec.HistogramSamples(corruptedHists, nil) require.ErrorIs(t, err, encoding.ErrInvalidSize) + _, err = dec.HistogramSamples(corruptedCustomBucketsHists, nil) + require.ErrorIs(t, err, encoding.ErrInvalidSize) }) } @@ -364,9 +383,12 @@ func TestRecord_Type(t *testing.T) { }, }, } - hists := enc.HistogramSamples(histograms, nil) + hists, customBucketsHistograms := enc.HistogramSamples(histograms, nil) recordType = dec.Type(hists) require.Equal(t, HistogramSamples, recordType) + customBucketsHists := enc.CustomBucketsHistogramSamples(customBucketsHistograms, nil) + recordType = dec.Type(customBucketsHists) + require.Equal(t, CustomBucketsHistogramSamples, recordType) recordType = dec.Type(nil) require.Equal(t, Unknown, recordType) diff --git a/tsdb/testutil.go b/tsdb/testutil.go index a13d89186e..ccfee182c6 100644 --- a/tsdb/testutil.go +++ b/tsdb/testutil.go @@ -29,13 +29,13 @@ import ( ) const ( - float = "float" - intHistogram = "integer histogram" - floatHistogram = "float histogram" - customBucketIntHistogram = "custom bucket int histogram" - customBucketFloatHistogram = "custom bucket float histogram" - gaugeIntHistogram = "gauge int histogram" - gaugeFloatHistogram = "gauge float histogram" + float = "float" + intHistogram = "integer histogram" + floatHistogram = "float histogram" + customBucketsIntHistogram = "custom buckets int histogram" + customBucketsFloatHistogram = "custom buckets float histogram" + gaugeIntHistogram = "gauge int histogram" + gaugeFloatHistogram = "gauge float histogram" ) type testValue struct { @@ -84,7 +84,7 @@ var sampleTypeScenarios = map[string]sampleTypeScenario{ return sample{t: ts, fh: tsdbutil.GenerateTestFloatHistogram(int(value))} }, }, - customBucketIntHistogram: { + customBucketsIntHistogram: { sampleType: sampleMetricTypeHistogram, appendFunc: func(appender storage.Appender, lbls labels.Labels, ts, value int64) (storage.SeriesRef, sample, error) { s := sample{t: ts, h: tsdbutil.GenerateTestCustomBucketsHistogram(int(value))} @@ -95,7 +95,7 @@ var sampleTypeScenarios = map[string]sampleTypeScenario{ return sample{t: ts, h: tsdbutil.GenerateTestCustomBucketsHistogram(int(value))} }, }, - customBucketFloatHistogram: { + customBucketsFloatHistogram: { sampleType: sampleMetricTypeHistogram, appendFunc: func(appender storage.Appender, lbls labels.Labels, ts, value int64) (storage.SeriesRef, sample, error) { s := sample{t: ts, fh: tsdbutil.GenerateTestCustomBucketsFloatHistogram(int(value))} diff --git a/tsdb/wlog/checkpoint.go b/tsdb/wlog/checkpoint.go index ffb96dbe22..63a7737b3a 100644 --- a/tsdb/wlog/checkpoint.go +++ b/tsdb/wlog/checkpoint.go @@ -208,7 +208,7 @@ func Checkpoint(logger *slog.Logger, w *WL, from, to int, keep func(id chunks.He stats.TotalSamples += len(samples) stats.DroppedSamples += len(samples) - len(repl) - case record.HistogramSamples, record.HistogramSamplesLegacy: + case record.HistogramSamples: histogramSamples, err = dec.HistogramSamples(rec, histogramSamples) if err != nil { return nil, fmt.Errorf("decode histogram samples: %w", err) @@ -221,11 +221,42 @@ func Checkpoint(logger *slog.Logger, w *WL, from, to int, keep func(id chunks.He } } if len(repl) > 0 { - buf = enc.HistogramSamples(repl, buf) + buf, _ = enc.HistogramSamples(repl, buf) } stats.TotalSamples += len(histogramSamples) stats.DroppedSamples += len(histogramSamples) - len(repl) - case record.FloatHistogramSamples, record.FloatHistogramSamplesLegacy: + case record.CustomBucketsHistogramSamples: + histogramSamples, err = dec.HistogramSamples(rec, histogramSamples) + // Drop irrelevant histogramSamples in place. + repl := histogramSamples[:0] + for _, h := range histogramSamples { + if h.T >= mint { + repl = append(repl, h) + } + } + if len(repl) > 0 { + buf = enc.CustomBucketsHistogramSamples(repl, buf) + } + stats.TotalSamples += len(histogramSamples) + stats.DroppedSamples += len(histogramSamples) - len(repl) + case record.FloatHistogramSamples: + floatHistogramSamples, err = dec.FloatHistogramSamples(rec, floatHistogramSamples) + if err != nil { + return nil, fmt.Errorf("decode float histogram samples: %w", err) + } + // Drop irrelevant floatHistogramSamples in place. + repl := floatHistogramSamples[:0] + for _, fh := range floatHistogramSamples { + if fh.T >= mint { + repl = append(repl, fh) + } + } + if len(repl) > 0 { + buf, _ = enc.FloatHistogramSamples(repl, buf) + } + stats.TotalSamples += len(floatHistogramSamples) + stats.DroppedSamples += len(floatHistogramSamples) - len(repl) + case record.CustomBucketsFloatHistogramSamples: floatHistogramSamples, err = dec.FloatHistogramSamples(rec, floatHistogramSamples) if err != nil { return nil, fmt.Errorf("decode float histogram samples: %w", err) @@ -238,7 +269,7 @@ func Checkpoint(logger *slog.Logger, w *WL, from, to int, keep func(id chunks.He } } if len(repl) > 0 { - buf = enc.FloatHistogramSamples(repl, buf) + buf = enc.CustomBucketsFloatHistogramSamples(repl, buf) } stats.TotalSamples += len(floatHistogramSamples) stats.DroppedSamples += len(floatHistogramSamples) - len(repl) diff --git a/tsdb/wlog/checkpoint_test.go b/tsdb/wlog/checkpoint_test.go index b2c603f134..873513c4ec 100644 --- a/tsdb/wlog/checkpoint_test.go +++ b/tsdb/wlog/checkpoint_test.go @@ -236,7 +236,7 @@ func TestCheckpoint(t *testing.T) { require.NoError(t, w.Log(b)) samplesInWAL += 4 h := makeHistogram(i) - b = enc.HistogramSamples([]record.RefHistogramSample{ + b, _ = enc.HistogramSamples([]record.RefHistogramSample{ {Ref: 0, T: last, H: h}, {Ref: 1, T: last + 10000, H: h}, {Ref: 2, T: last + 20000, H: h}, @@ -245,7 +245,7 @@ func TestCheckpoint(t *testing.T) { require.NoError(t, w.Log(b)) histogramsInWAL += 4 cbh := makeCustomBucketHistogram(i) - b = enc.HistogramSamples([]record.RefHistogramSample{ + b = enc.CustomBucketsHistogramSamples([]record.RefHistogramSample{ {Ref: 0, T: last, H: cbh}, {Ref: 1, T: last + 10000, H: cbh}, {Ref: 2, T: last + 20000, H: cbh}, @@ -254,7 +254,7 @@ func TestCheckpoint(t *testing.T) { require.NoError(t, w.Log(b)) histogramsInWAL += 4 fh := makeFloatHistogram(i) - b = enc.FloatHistogramSamples([]record.RefFloatHistogramSample{ + b, _ = enc.FloatHistogramSamples([]record.RefFloatHistogramSample{ {Ref: 0, T: last, FH: fh}, {Ref: 1, T: last + 10000, FH: fh}, {Ref: 2, T: last + 20000, FH: fh}, @@ -263,7 +263,7 @@ func TestCheckpoint(t *testing.T) { require.NoError(t, w.Log(b)) floatHistogramsInWAL += 4 cbfh := makeCustomBucketFloatHistogram(i) - b = enc.FloatHistogramSamples([]record.RefFloatHistogramSample{ + b = enc.CustomBucketsFloatHistogramSamples([]record.RefFloatHistogramSample{ {Ref: 0, T: last, FH: cbfh}, {Ref: 1, T: last + 10000, FH: cbfh}, {Ref: 2, T: last + 20000, FH: cbfh}, @@ -330,14 +330,14 @@ func TestCheckpoint(t *testing.T) { require.GreaterOrEqual(t, s.T, last/2, "sample with wrong timestamp") } samplesInCheckpoint += len(samples) - case record.HistogramSamples, record.HistogramSamplesLegacy: + case record.HistogramSamples, record.CustomBucketsHistogramSamples: histograms, err := dec.HistogramSamples(rec, nil) require.NoError(t, err) for _, h := range histograms { require.GreaterOrEqual(t, h.T, last/2, "histogram with wrong timestamp") } histogramsInCheckpoint += len(histograms) - case record.FloatHistogramSamples, record.FloatHistogramSamplesLegacy: + case record.FloatHistogramSamples, record.CustomBucketsFloatHistogramSamples: floatHistograms, err := dec.FloatHistogramSamples(rec, nil) require.NoError(t, err) for _, h := range floatHistograms { diff --git a/tsdb/wlog/watcher.go b/tsdb/wlog/watcher.go index 07f881eeaf..6f1bc1df35 100644 --- a/tsdb/wlog/watcher.go +++ b/tsdb/wlog/watcher.go @@ -546,7 +546,7 @@ func (w *Watcher) readSegment(r *LiveReader, segmentNum int, tail bool) error { } w.writer.AppendExemplars(exemplars) - case record.HistogramSamples, record.HistogramSamplesLegacy: + case record.HistogramSamples, record.CustomBucketsHistogramSamples: // Skip if experimental "histograms over remote write" is not enabled. if !w.sendHistograms { break @@ -574,7 +574,7 @@ func (w *Watcher) readSegment(r *LiveReader, segmentNum int, tail bool) error { histogramsToSend = histogramsToSend[:0] } - case record.FloatHistogramSamples, record.FloatHistogramSamplesLegacy: + case record.FloatHistogramSamples, record.CustomBucketsFloatHistogramSamples: // Skip if experimental "histograms over remote write" is not enabled. if !w.sendHistograms { break diff --git a/tsdb/wlog/watcher_test.go b/tsdb/wlog/watcher_test.go index 21490154d9..a793c90a95 100644 --- a/tsdb/wlog/watcher_test.go +++ b/tsdb/wlog/watcher_test.go @@ -209,7 +209,7 @@ func TestTailSamples(t *testing.T) { NegativeBuckets: []int64{int64(-i) - 1}, } - histograms := enc.HistogramSamples([]record.RefHistogramSample{{ + histograms, _ := enc.HistogramSamples([]record.RefHistogramSample{{ Ref: chunks.HeadSeriesRef(inner), T: now.UnixNano() + 1, H: hist, @@ -226,21 +226,21 @@ func TestTailSamples(t *testing.T) { CustomValues: []float64{float64(i) + 2}, } - customBucketHistograms := enc.HistogramSamples([]record.RefHistogramSample{{ + customBucketHistograms := enc.CustomBucketsHistogramSamples([]record.RefHistogramSample{{ Ref: chunks.HeadSeriesRef(inner), T: now.UnixNano() + 1, H: customBucketHist, }}, nil) require.NoError(t, w.Log(customBucketHistograms)) - floatHistograms := enc.FloatHistogramSamples([]record.RefFloatHistogramSample{{ + floatHistograms, _ := enc.FloatHistogramSamples([]record.RefFloatHistogramSample{{ Ref: chunks.HeadSeriesRef(inner), T: now.UnixNano() + 1, FH: hist.ToFloat(nil), }}, nil) require.NoError(t, w.Log(floatHistograms)) - customBucketFloatHistograms := enc.FloatHistogramSamples([]record.RefFloatHistogramSample{{ + customBucketFloatHistograms := enc.CustomBucketsFloatHistogramSamples([]record.RefFloatHistogramSample{{ Ref: chunks.HeadSeriesRef(inner), T: now.UnixNano() + 1, FH: customBucketHist.ToFloat(nil), From 1933ccc9be1fcd9ca14bfa653733dde21a59af40 Mon Sep 17 00:00:00 2001 From: Carrie Edwards Date: Fri, 6 Dec 2024 14:55:19 -0800 Subject: [PATCH 11/40] Fix test --- tsdb/db_test.go | 110 ++++++++++++++++++++++++------------------------ 1 file changed, 55 insertions(+), 55 deletions(-) diff --git a/tsdb/db_test.go b/tsdb/db_test.go index 5024a0cfbb..e67f4821a4 100644 --- a/tsdb/db_test.go +++ b/tsdb/db_test.go @@ -6469,7 +6469,7 @@ func testOOOInterleavedImplicitCounterResets(t *testing.T, name string, scenario Sum: float64(v), PositiveSpans: []histogram.Span{{Offset: 0, Length: 1}}, PositiveBuckets: []int64{v}, - CustomValues: []float64{float64(v)}, + CustomValues: []float64{float64(1), float64(2), float64(3)}, } _, err := app.AppendHistogram(0, labels.FromStrings("foo", "bar1"), ts, h, nil) return err @@ -6482,7 +6482,7 @@ func testOOOInterleavedImplicitCounterResets(t *testing.T, name string, scenario Sum: float64(v), PositiveSpans: []histogram.Span{{Offset: 0, Length: 1}}, PositiveBuckets: []float64{float64(v)}, - CustomValues: []float64{float64(v)}, + CustomValues: []float64{float64(1), float64(2), float64(3)}, } _, err := app.AppendHistogram(0, labels.FromStrings("foo", "bar1"), ts, nil, fh) return err @@ -6517,29 +6517,29 @@ func testOOOInterleavedImplicitCounterResets(t *testing.T, name string, scenario // 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-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. @@ -6570,36 +6570,36 @@ func testOOOInterleavedImplicitCounterResets(t *testing.T, name string, scenario {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. - // }, - //}, + "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 { From b94c87bea627613d60fdf50e0084737abab4c5e8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gy=C3=B6rgy=20Krajcsovits?= Date: Tue, 10 Dec 2024 16:16:46 +0100 Subject: [PATCH 12/40] fix(test): TestCheckpoint segment size too low MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The segment size was too low for the additional NHCB data, thus it created more segments then expected. This meant that less were in the lower numbered segments, which meant more was kept. FAIL: TestCheckpoint (4.05s) FAIL: TestCheckpoint/compress=none (0.22s) checkpoint_test.go:361: Error Trace: /home/krajo/go/github.com/prometheus/prometheus/tsdb/wlog/checkpoint_test.go:361 Error: "0.8586956521739131" is not less than "0.8" Test: TestCheckpoint/compress=none Signed-off-by: György Krajcsovits --- tsdb/wlog/checkpoint_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tsdb/wlog/checkpoint_test.go b/tsdb/wlog/checkpoint_test.go index 873513c4ec..a052de9258 100644 --- a/tsdb/wlog/checkpoint_test.go +++ b/tsdb/wlog/checkpoint_test.go @@ -195,7 +195,7 @@ func TestCheckpoint(t *testing.T) { require.NoError(t, w.Close()) // Start a WAL and write records to it as usual. - w, err = NewSize(nil, nil, dir, 64*1024, compress) + w, err = NewSize(nil, nil, dir, 128*1024, compress) require.NoError(t, err) samplesInWAL, histogramsInWAL, floatHistogramsInWAL := 0, 0, 0 From 8f572fe905c5fa9fcae8edad9b5300e18592887a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gy=C3=B6rgy=20Krajcsovits?= Date: Tue, 10 Dec 2024 16:25:20 +0100 Subject: [PATCH 13/40] fix(lint): linter errors MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: György Krajcsovits --- tsdb/record/record_test.go | 1 + tsdb/wlog/checkpoint.go | 3 +++ 2 files changed, 4 insertions(+) diff --git a/tsdb/record/record_test.go b/tsdb/record/record_test.go index 030b7e2bc7..e26f964072 100644 --- a/tsdb/record/record_test.go +++ b/tsdb/record/record_test.go @@ -216,6 +216,7 @@ func TestRecord_EncodeDecode(t *testing.T) { decGaugeFloatHistograms, err := dec.FloatHistogramSamples(gaugeFloatHistSamples, nil) require.NoError(t, err) decCustomBucketsGaugeFloatHistograms, err := dec.FloatHistogramSamples(customBucketsGaugeFloatHistSamples, nil) + require.NoError(t, err) decGaugeFloatHistograms = append(decGaugeFloatHistograms, decCustomBucketsGaugeFloatHistograms...) require.Equal(t, floatHistograms, decGaugeFloatHistograms) } diff --git a/tsdb/wlog/checkpoint.go b/tsdb/wlog/checkpoint.go index 63a7737b3a..45c506e802 100644 --- a/tsdb/wlog/checkpoint.go +++ b/tsdb/wlog/checkpoint.go @@ -227,6 +227,9 @@ func Checkpoint(logger *slog.Logger, w *WL, from, to int, keep func(id chunks.He stats.DroppedSamples += len(histogramSamples) - len(repl) case record.CustomBucketsHistogramSamples: histogramSamples, err = dec.HistogramSamples(rec, histogramSamples) + if err != nil { + return nil, fmt.Errorf("decode histogram samples: %w", err) + } // Drop irrelevant histogramSamples in place. repl := histogramSamples[:0] for _, h := range histogramSamples { From 07276aeece7b19ec637a5109754d01d318941bd4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gy=C3=B6rgy=20Krajcsovits?= Date: Tue, 10 Dec 2024 16:25:50 +0100 Subject: [PATCH 14/40] fix(test): if we are dereferencing a slice we should check its len MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: György Krajcsovits --- tsdb/agent/db_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/tsdb/agent/db_test.go b/tsdb/agent/db_test.go index c81c63f739..0840cebe5c 100644 --- a/tsdb/agent/db_test.go +++ b/tsdb/agent/db_test.go @@ -631,6 +631,7 @@ func TestPartialTruncateWAL(t *testing.T) { s.truncate(lastTs - 1) m := gatherFamily(t, reg, "prometheus_agent_deleted_series") + require.Len(t, m.Metric, 1) require.Equal(t, float64(numSeries*5), m.Metric[0].Gauge.GetValue(), "agent wal truncate mismatch of deleted series count") } From a325ff142cd5822c3835c202e300a093d50768f1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gy=C3=B6rgy=20Krajcsovits?= Date: Tue, 10 Dec 2024 17:30:46 +0100 Subject: [PATCH 15/40] fix(test): do not run automatic WAL truncate during test MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Remove the 2 minute timeout as the default is 2 hours and wouldn't interfere. With the test. Otherwise the extra samples combined with race detection can push the test over 2 minutes and make it fail. Signed-off-by: György Krajcsovits --- tsdb/agent/db_test.go | 1 - 1 file changed, 1 deletion(-) diff --git a/tsdb/agent/db_test.go b/tsdb/agent/db_test.go index 0840cebe5c..0238a8e140 100644 --- a/tsdb/agent/db_test.go +++ b/tsdb/agent/db_test.go @@ -488,7 +488,6 @@ func TestPartialTruncateWAL(t *testing.T) { ) opts := DefaultOptions() - opts.TruncateFrequency = time.Minute * 2 reg := prometheus.NewRegistry() s := createTestAgentDB(t, reg, opts) From d64d1c4c0a1a2c2fb4d07ba6e8244e36d827214a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gy=C3=B6rgy=20Krajcsovits?= Date: Thu, 12 Dec 2024 10:59:06 +0100 Subject: [PATCH 16/40] Benchmark encoding classic and nhcb MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: György Krajcsovits --- tsdb/record/record_test.go | 140 +++++++++++++++++++++++++++++++++++++ 1 file changed, 140 insertions(+) diff --git a/tsdb/record/record_test.go b/tsdb/record/record_test.go index e26f964072..e64182c6fb 100644 --- a/tsdb/record/record_test.go +++ b/tsdb/record/record_test.go @@ -15,13 +15,16 @@ package record import ( + "fmt" "math/rand" "testing" "github.com/stretchr/testify/require" + "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/histogram" "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/encoding" "github.com/prometheus/prometheus/tsdb/tombstones" "github.com/prometheus/prometheus/util/testutil" @@ -464,3 +467,140 @@ func TestRecord_MetadataDecodeUnknownExtraFields(t *testing.T) { require.NoError(t, err) require.Equal(t, expectedMetadata, decMetadata) } + +type recordsMaker struct { + name string + init func(int, int, int) +} + +// BenchmarkWAL_HistogramLog measures efficiency of encoding classic +// histograms and native historgrams with custom buckets (NHCB). +func BenchmarkWAL_HistogramEncoding(b *testing.B) { + // Cache for the refs. + var series []RefSeries + var samples []RefSample + var nhcbs []RefHistogramSample + + resetCache := func() { + series = nil + samples = nil + nhcbs = nil + } + + initClassicRefs := func(labelCount, histograms, buckets int) { + ref := chunks.HeadSeriesRef(0) + lbls := map[string]string{} + for i := range labelCount { + lbls[fmt.Sprintf("l%d", i)] = fmt.Sprintf("v%d", i) + } + for i := range histograms { + lbls[model.MetricNameLabel] = fmt.Sprintf("series_%d_count", i) + series = append(series, RefSeries{ + Ref: ref, + Labels: labels.FromMap(lbls), + }) + samples = append(samples, RefSample{ + Ref: ref, + T: 100, + V: float64(i), + }) + ref++ + + lbls[model.MetricNameLabel] = fmt.Sprintf("series_%d_sum", i) + series = append(series, RefSeries{ + Ref: ref, + Labels: labels.FromMap(lbls), + }) + samples = append(samples, RefSample{ + Ref: ref, + T: 100, + V: float64(i), + }) + ref++ + + if buckets == 0 { + continue + } + lbls[model.MetricNameLabel] = fmt.Sprintf("series_%d_bucket", i) + for j := range buckets { + lbls[model.BucketLabel] = fmt.Sprintf("%d", j) + series = append(series, RefSeries{ + Ref: ref, + Labels: labels.FromMap(lbls), + }) + samples = append(samples, RefSample{ + Ref: ref, + T: 100, + V: float64(i + j), + }) + ref++ + } + delete(lbls, model.BucketLabel) + } + } + + initNHCBRefs := func(labelCount, histograms, buckets int) { + ref := chunks.HeadSeriesRef(0) + lbls := map[string]string{} + for i := range labelCount { + lbls[fmt.Sprintf("l%d", i)] = fmt.Sprintf("v%d", i) + } + for i := range histograms { + lbls[model.MetricNameLabel] = fmt.Sprintf("series_%d", i) + series = append(series, RefSeries{ + Ref: ref, + Labels: labels.FromMap(lbls), + }) + h := &histogram.Histogram{ + Schema: histogram.CustomBucketsSchema, + Count: uint64(i), + Sum: float64(i), + PositiveSpans: []histogram.Span{{Length: uint32(buckets)}}, + PositiveBuckets: make([]int64, buckets+1), + CustomValues: make([]float64, buckets), + } + for j := range buckets { + h.PositiveBuckets[j] = int64(i + j) + } + nhcbs = append(nhcbs, RefHistogramSample{ + Ref: ref, + T: 100, + H: h, + }) + ref++ + } + } + + for _, maker := range []recordsMaker{ + { + name: "classic", + init: initClassicRefs, + }, + { + name: "nhcb", + init: initNHCBRefs, + }, + } { + for _, labelCount := range []int{0, 10, 50} { + for _, histograms := range []int{10, 100, 1000} { + for _, buckets := range []int{0, 1, 10, 100} { + b.Run(fmt.Sprintf("%s labels=%d histograms=%d buckets=%d", maker.name, labelCount, histograms, buckets), func(b *testing.B) { + resetCache() + maker.init(labelCount, histograms, buckets) + enc := Encoder{} + for range b.N { + var buf []byte + enc.Series(series, buf) + enc.Samples(samples, buf) + var leftOver []RefHistogramSample + _, leftOver = enc.HistogramSamples(nhcbs, buf) + if len(leftOver) > 0 { + enc.CustomBucketsHistogramSamples(leftOver, buf) + } + } + }) + } + } + } + } +} From fdb1516af1c671457bc88687370b95911d4915a2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gy=C3=B6rgy=20Krajcsovits?= Date: Thu, 12 Dec 2024 12:47:43 +0100 Subject: [PATCH 17/40] Fix lint errors MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: György Krajcsovits --- tsdb/record/record_test.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tsdb/record/record_test.go b/tsdb/record/record_test.go index e64182c6fb..a035a45fc2 100644 --- a/tsdb/record/record_test.go +++ b/tsdb/record/record_test.go @@ -17,11 +17,13 @@ package record import ( "fmt" "math/rand" + "strconv" "testing" "github.com/stretchr/testify/require" "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/histogram" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/tsdb/chunks" @@ -523,7 +525,7 @@ func BenchmarkWAL_HistogramEncoding(b *testing.B) { } lbls[model.MetricNameLabel] = fmt.Sprintf("series_%d_bucket", i) for j := range buckets { - lbls[model.BucketLabel] = fmt.Sprintf("%d", j) + lbls[model.BucketLabel] = fmt.Sprintf("%g", j) series = append(series, RefSeries{ Ref: ref, Labels: labels.FromMap(lbls), From cf36792e14736ecdc0b9f81635c18b54c331e306 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gy=C3=B6rgy=20Krajcsovits?= Date: Thu, 12 Dec 2024 12:49:28 +0100 Subject: [PATCH 18/40] Fix unused import MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: György Krajcsovits --- tsdb/record/record_test.go | 1 - 1 file changed, 1 deletion(-) diff --git a/tsdb/record/record_test.go b/tsdb/record/record_test.go index a035a45fc2..7cab35bd43 100644 --- a/tsdb/record/record_test.go +++ b/tsdb/record/record_test.go @@ -17,7 +17,6 @@ package record import ( "fmt" "math/rand" - "strconv" "testing" "github.com/stretchr/testify/require" From df88de5800b57f615cdbf7010d02bc4cae4098e2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gy=C3=B6rgy=20Krajcsovits?= Date: Thu, 12 Dec 2024 12:52:01 +0100 Subject: [PATCH 19/40] Fix lint for real MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: György Krajcsovits --- tsdb/record/record_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tsdb/record/record_test.go b/tsdb/record/record_test.go index 7cab35bd43..6e9c6e483c 100644 --- a/tsdb/record/record_test.go +++ b/tsdb/record/record_test.go @@ -524,7 +524,7 @@ func BenchmarkWAL_HistogramEncoding(b *testing.B) { } lbls[model.MetricNameLabel] = fmt.Sprintf("series_%d_bucket", i) for j := range buckets { - lbls[model.BucketLabel] = fmt.Sprintf("%g", j) + lbls[model.BucketLabel] = fmt.Sprintf("%d.0", j) series = append(series, RefSeries{ Ref: ref, Labels: labels.FromMap(lbls), From e630ffdbedaf276da8c7e1a015af893ddc16d77a Mon Sep 17 00:00:00 2001 From: Bryan Boreham Date: Fri, 26 Jul 2024 18:17:35 +0100 Subject: [PATCH 20/40] TSDB: extend BenchmarkMemPostings_PostingsForLabelMatching to check merge speed We need to create more postings entries so the merger has some work to do. Not material for the regexp ones as they match so few series. Signed-off-by: Bryan Boreham --- tsdb/index/postings_test.go | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/tsdb/index/postings_test.go b/tsdb/index/postings_test.go index 6dd9f25bc0..c4fb1f12f4 100644 --- a/tsdb/index/postings_test.go +++ b/tsdb/index/postings_test.go @@ -1410,12 +1410,15 @@ func BenchmarkMemPostings_PostingsForLabelMatching(b *testing.B) { slowRegexp := "^" + slowRegexpString() + "$" b.Logf("Slow regexp length = %d", len(slowRegexp)) slow := regexp.MustCompile(slowRegexp) + const seriesPerLabel = 10 for _, labelValueCount := range []int{1_000, 10_000, 100_000} { b.Run(fmt.Sprintf("labels=%d", labelValueCount), func(b *testing.B) { mp := NewMemPostings() for i := 0; i < labelValueCount; i++ { - mp.Add(storage.SeriesRef(i), labels.FromStrings("label", strconv.Itoa(i))) + for j := 0; j < seriesPerLabel; j++ { + mp.Add(storage.SeriesRef(i*seriesPerLabel+j), labels.FromStrings("__name__", strconv.Itoa(j), "label", strconv.Itoa(i))) + } } fp, err := ExpandPostings(mp.PostingsForLabelMatching(context.Background(), "label", fast.MatchString)) @@ -1435,6 +1438,18 @@ func BenchmarkMemPostings_PostingsForLabelMatching(b *testing.B) { mp.PostingsForLabelMatching(context.Background(), "label", slow.MatchString).Next() } }) + + b.Run("matcher=all", func(b *testing.B) { + for i := 0; i < b.N; i++ { + // Match everything. + p := mp.PostingsForLabelMatching(context.Background(), "label", func(_ string) bool { return true }) + var sum storage.SeriesRef + // Iterate through all results to exercise merge function. + for p.Next() { + sum += p.At() + } + } + }) }) } } From 1b22242024d09a287a30ad30560dfc58febee966 Mon Sep 17 00:00:00 2001 From: Bryan Boreham Date: Fri, 26 Jul 2024 20:00:03 +0100 Subject: [PATCH 21/40] TSDB BenchmarkMerge: run fewer sizes As long as we run small and big sizes, we don't need all the sizes inbetween. Signed-off-by: Bryan Boreham --- tsdb/index/postings_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tsdb/index/postings_test.go b/tsdb/index/postings_test.go index c4fb1f12f4..cd7fa9d3c6 100644 --- a/tsdb/index/postings_test.go +++ b/tsdb/index/postings_test.go @@ -393,7 +393,7 @@ func BenchmarkMerge(t *testing.B) { } its := make([]Postings, len(refs)) - for _, nSeries := range []int{1, 10, 100, 1000, 10000, 100000} { + for _, nSeries := range []int{1, 10, 10000, 100000} { t.Run(strconv.Itoa(nSeries), func(bench *testing.B) { ctx := context.Background() for i := 0; i < bench.N; i++ { From 0a8779f46dace4d24dd9c14e81cba065c23e2a88 Mon Sep 17 00:00:00 2001 From: Bryan Boreham Date: Fri, 26 Jul 2024 19:35:58 +0100 Subject: [PATCH 22/40] TSDB: Make mergedPostings generic Now we can call it with more specific types which is more efficient than making everything go through the `Postings` interface. Benchmark the concrete type. Signed-off-by: Bryan Boreham --- tsdb/index/postings.go | 20 ++++++++++---------- tsdb/index/postings_test.go | 2 +- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/tsdb/index/postings.go b/tsdb/index/postings.go index a2c5a82239..5384133832 100644 --- a/tsdb/index/postings.go +++ b/tsdb/index/postings.go @@ -660,7 +660,7 @@ func (it *intersectPostings) Err() error { } // Merge returns a new iterator over the union of the input iterators. -func Merge(_ context.Context, its ...Postings) Postings { +func Merge[T Postings](_ context.Context, its ...T) Postings { if len(its) == 0 { return EmptyPostings() } @@ -675,19 +675,19 @@ func Merge(_ context.Context, its ...Postings) Postings { return p } -type mergedPostings struct { - p []Postings - h *loser.Tree[storage.SeriesRef, Postings] +type mergedPostings[T Postings] struct { + p []T + h *loser.Tree[storage.SeriesRef, T] cur storage.SeriesRef } -func newMergedPostings(p []Postings) (m *mergedPostings, nonEmpty bool) { +func newMergedPostings[T Postings](p []T) (m *mergedPostings[T], nonEmpty bool) { const maxVal = storage.SeriesRef(math.MaxUint64) // This value must be higher than all real values used in the tree. lt := loser.New(p, maxVal) - return &mergedPostings{p: p, h: lt}, true + return &mergedPostings[T]{p: p, h: lt}, true } -func (it *mergedPostings) Next() bool { +func (it *mergedPostings[T]) Next() bool { for { if !it.h.Next() { return false @@ -701,7 +701,7 @@ func (it *mergedPostings) Next() bool { } } -func (it *mergedPostings) Seek(id storage.SeriesRef) bool { +func (it *mergedPostings[T]) Seek(id storage.SeriesRef) bool { for !it.h.IsEmpty() && it.h.At() < id { finished := !it.h.Winner().Seek(id) it.h.Fix(finished) @@ -713,11 +713,11 @@ func (it *mergedPostings) Seek(id storage.SeriesRef) bool { return true } -func (it mergedPostings) At() storage.SeriesRef { +func (it mergedPostings[T]) At() storage.SeriesRef { return it.cur } -func (it mergedPostings) Err() error { +func (it mergedPostings[T]) Err() error { for _, p := range it.p { if err := p.Err(); err != nil { return err diff --git a/tsdb/index/postings_test.go b/tsdb/index/postings_test.go index cd7fa9d3c6..77d59ec995 100644 --- a/tsdb/index/postings_test.go +++ b/tsdb/index/postings_test.go @@ -392,7 +392,7 @@ func BenchmarkMerge(t *testing.B) { refs = append(refs, temp) } - its := make([]Postings, len(refs)) + its := make([]*ListPostings, len(refs)) for _, nSeries := range []int{1, 10, 10000, 100000} { t.Run(strconv.Itoa(nSeries), func(bench *testing.B) { ctx := context.Background() From cfa32f3d2847eb9d40ebc16ce9bb8ebfeb0705a1 Mon Sep 17 00:00:00 2001 From: Bryan Boreham Date: Fri, 26 Jul 2024 20:08:51 +0100 Subject: [PATCH 23/40] TSDB: Move merge of head postings into index This enables it to take advantage of a more compact data structure since all postings are known to be `*ListPostings`. Remove the `Get` member which was not used for anything else, and fix up tests. Signed-off-by: Bryan Boreham --- tsdb/head_read.go | 15 +-------------- tsdb/head_test.go | 12 ++++++------ tsdb/index/postings.go | 38 ++++++++++++++++++------------------- tsdb/index/postings_test.go | 8 ++++---- 4 files changed, 30 insertions(+), 43 deletions(-) diff --git a/tsdb/head_read.go b/tsdb/head_read.go index 79ed0f0240..b95257c28a 100644 --- a/tsdb/head_read.go +++ b/tsdb/head_read.go @@ -103,20 +103,7 @@ func (h *headIndexReader) LabelNames(ctx context.Context, matchers ...*labels.Ma // Postings returns the postings list iterator for the label pairs. func (h *headIndexReader) Postings(ctx context.Context, name string, values ...string) (index.Postings, error) { - switch len(values) { - case 0: - return index.EmptyPostings(), nil - case 1: - return h.head.postings.Get(name, values[0]), nil - default: - res := make([]index.Postings, 0, len(values)) - for _, value := range values { - if p := h.head.postings.Get(name, value); !index.IsEmptyPostingsType(p) { - res = append(res, p) - } - } - return index.Merge(ctx, res...), nil - } + return h.head.postings.Postings(ctx, name, values...), nil } func (h *headIndexReader) PostingsForLabelMatching(ctx context.Context, name string, match func(string) bool) index.Postings { diff --git a/tsdb/head_test.go b/tsdb/head_test.go index fb158b593c..e3742cbe9c 100644 --- a/tsdb/head_test.go +++ b/tsdb/head_test.go @@ -962,12 +962,12 @@ func TestHead_Truncate(t *testing.T) { require.Nil(t, h.series.getByID(s3.ref)) require.Nil(t, h.series.getByID(s4.ref)) - postingsA1, _ := index.ExpandPostings(h.postings.Get("a", "1")) - postingsA2, _ := index.ExpandPostings(h.postings.Get("a", "2")) - postingsB1, _ := index.ExpandPostings(h.postings.Get("b", "1")) - postingsB2, _ := index.ExpandPostings(h.postings.Get("b", "2")) - postingsC1, _ := index.ExpandPostings(h.postings.Get("c", "1")) - postingsAll, _ := index.ExpandPostings(h.postings.Get("", "")) + 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) diff --git a/tsdb/index/postings.go b/tsdb/index/postings.go index 5384133832..03e3f7a239 100644 --- a/tsdb/index/postings.go +++ b/tsdb/index/postings.go @@ -235,25 +235,9 @@ func (p *MemPostings) Stats(label string, limit int, labelSizeFunc func(string, } } -// Get returns a postings list for the given label pair. -func (p *MemPostings) Get(name, value string) Postings { - var lp []storage.SeriesRef - p.mtx.RLock() - l := p.m[name] - if l != nil { - lp = l[value] - } - p.mtx.RUnlock() - - if lp == nil { - return EmptyPostings() - } - return newListPostings(lp...) -} - // All returns a postings list over all documents ever added. func (p *MemPostings) All() Postings { - return p.Get(AllPostingsKey()) + return p.Postings(context.Background(), allPostingsKey.Name, allPostingsKey.Value) } // EnsureOrder ensures that all postings lists are sorted. After it returns all further @@ -490,7 +474,7 @@ func (p *MemPostings) PostingsForLabelMatching(ctx context.Context, name string, } // Now `vals` only contains the values that matched, get their postings. - its := make([]Postings, 0, len(vals)) + its := make([]*ListPostings, 0, len(vals)) lps := make([]ListPostings, len(vals)) p.mtx.RLock() e := p.m[name] @@ -510,11 +494,27 @@ func (p *MemPostings) PostingsForLabelMatching(ctx context.Context, name string, return Merge(ctx, its...) } +// Postings returns a postings iterator for the given label values. +func (p *MemPostings) Postings(ctx context.Context, name string, values ...string) Postings { + res := make([]*ListPostings, 0, len(values)) + lps := make([]ListPostings, len(values)) + p.mtx.RLock() + postingsMapForName := p.m[name] + for i, value := range values { + if lp := postingsMapForName[value]; lp != nil { + lps[i] = ListPostings{list: lp} + res = append(res, &lps[i]) + } + } + p.mtx.RUnlock() + return Merge(ctx, res...) +} + func (p *MemPostings) PostingsForAllLabelValues(ctx context.Context, name string) Postings { p.mtx.RLock() e := p.m[name] - its := make([]Postings, 0, len(e)) + its := make([]*ListPostings, 0, len(e)) lps := make([]ListPostings, len(e)) i := 0 for _, refs := range e { diff --git a/tsdb/index/postings_test.go b/tsdb/index/postings_test.go index 77d59ec995..cf5ab6c0f8 100644 --- a/tsdb/index/postings_test.go +++ b/tsdb/index/postings_test.go @@ -979,7 +979,7 @@ func TestMemPostings_Delete(t *testing.T) { p.Add(2, labels.FromStrings("lbl1", "b")) p.Add(3, labels.FromStrings("lbl2", "a")) - before := p.Get(allPostingsKey.Name, allPostingsKey.Value) + before := p.Postings(context.Background(), allPostingsKey.Name, allPostingsKey.Value) deletedRefs := map[storage.SeriesRef]struct{}{ 2: {}, } @@ -987,7 +987,7 @@ func TestMemPostings_Delete(t *testing.T) { {Name: "lbl1", Value: "b"}: {}, } p.Delete(deletedRefs, affectedLabels) - after := p.Get(allPostingsKey.Name, allPostingsKey.Value) + after := p.Postings(context.Background(), allPostingsKey.Name, allPostingsKey.Value) // Make sure postings gotten before the delete have the old data when // iterated over. @@ -1001,7 +1001,7 @@ func TestMemPostings_Delete(t *testing.T) { require.NoError(t, err) require.Equal(t, []storage.SeriesRef{1, 3}, expanded) - deleted := p.Get("lbl1", "b") + deleted := p.Postings(context.Background(), "lbl1", "b") expanded, err = ExpandPostings(deleted) require.NoError(t, err) require.Empty(t, expanded, "expected empty postings, got %v", expanded) @@ -1073,7 +1073,7 @@ func BenchmarkMemPostings_Delete(b *testing.B) { return default: // Get a random value of this label. - p.Get(lbl, itoa(rand.Intn(10000))).Next() + p.Postings(context.Background(), lbl, itoa(rand.Intn(10000))).Next() } } }(i) From 1508149184faaa46acd7ac22aa3116f8f8ccfdd3 Mon Sep 17 00:00:00 2001 From: Carrie Edwards Date: Fri, 27 Dec 2024 09:09:13 -0800 Subject: [PATCH 24/40] Update benchmark test and comment --- tsdb/record/record.go | 2 +- tsdb/record/record_test.go | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/tsdb/record/record.go b/tsdb/record/record.go index ccfbbfcef9..4d2a52b9af 100644 --- a/tsdb/record/record.go +++ b/tsdb/record/record.go @@ -770,7 +770,7 @@ func (e *Encoder) HistogramSamples(histograms []RefHistogramSample, b []byte) ([ EncodeHistogram(&buf, h.H) } - // Reset buffer if only custom bucket histograms existed in list of histogram samples + // Reset buffer if only custom bucket histograms existed in list of histogram samples. if len(histograms) == len(customBucketHistograms) { buf.Reset() } diff --git a/tsdb/record/record_test.go b/tsdb/record/record_test.go index 6e9c6e483c..f615a334ea 100644 --- a/tsdb/record/record_test.go +++ b/tsdb/record/record_test.go @@ -585,7 +585,7 @@ func BenchmarkWAL_HistogramEncoding(b *testing.B) { for _, labelCount := range []int{0, 10, 50} { for _, histograms := range []int{10, 100, 1000} { for _, buckets := range []int{0, 1, 10, 100} { - b.Run(fmt.Sprintf("%s labels=%d histograms=%d buckets=%d", maker.name, labelCount, histograms, buckets), func(b *testing.B) { + b.Run(fmt.Sprintf("type=%s/labels=%d/histograms=%d/buckets=%d", maker.name, labelCount, histograms, buckets), func(b *testing.B) { resetCache() maker.init(labelCount, histograms, buckets) enc := Encoder{} @@ -598,6 +598,7 @@ func BenchmarkWAL_HistogramEncoding(b *testing.B) { if len(leftOver) > 0 { enc.CustomBucketsHistogramSamples(leftOver, buf) } + b.ReportMetric(float64(len(buf)), "recordBytes/ops") } }) } From 4cbd9ffb91ce9c64db4b29591475a811b0c7c95e Mon Sep 17 00:00:00 2001 From: TJ Hoplock <33664289+tjhop@users.noreply.github.com> Date: Mon, 30 Dec 2024 03:46:17 -0500 Subject: [PATCH 25/40] docs: update required go version in readme to 1.22 (#15447) It was bumped during 3.0 with the adoption of log/slog and other dep updates. ``` ~/go/src/github.com/prometheus/prometheus (main [ ]) -> grep '^go' go.mod go 1.22.0 ``` Signed-off-by: TJ Hoplock --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 63e5b13ba1..658cee4640 100644 --- a/README.md +++ b/README.md @@ -67,7 +67,7 @@ Prometheus will now be reachable at . To build Prometheus from source code, You need: -* Go [version 1.17 or greater](https://golang.org/doc/install). +* Go [version 1.22 or greater](https://golang.org/doc/install). * NodeJS [version 16 or greater](https://nodejs.org/). * npm [version 7 or greater](https://www.npmjs.com/). From f37b5adfef76f884be9c79d9e121f0e828eebf9b Mon Sep 17 00:00:00 2001 From: Arve Knudsen Date: Tue, 31 Dec 2024 11:12:21 +0100 Subject: [PATCH 26/40] OTLP receiver: Optimize by initializing regexps at program start (#15733) Signed-off-by: Arve Knudsen --- .../otlptranslator/prometheus/normalize_name.go | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/storage/remote/otlptranslator/prometheus/normalize_name.go b/storage/remote/otlptranslator/prometheus/normalize_name.go index 0a48e28219..580c72b548 100644 --- a/storage/remote/otlptranslator/prometheus/normalize_name.go +++ b/storage/remote/otlptranslator/prometheus/normalize_name.go @@ -95,9 +95,6 @@ func BuildCompliantName(metric pmetric.Metric, namespace string, addMetricSuffix var metricName string if !allowUTF8 { - // Regexp for metric name characters that should be replaced with _. - invalidMetricCharRE := regexp.MustCompile(`[^a-zA-Z0-9:_]`) - // Simple case (no full normalization, no units, etc.). metricName = strings.Join(strings.FieldsFunc(metric.Name(), func(r rune) bool { return invalidMetricCharRE.MatchString(string(r)) @@ -119,7 +116,12 @@ func BuildCompliantName(metric pmetric.Metric, namespace string, addMetricSuffix return metricName } -var nonMetricNameCharRE = regexp.MustCompile(`[^a-zA-Z0-9:]`) +var ( + nonMetricNameCharRE = regexp.MustCompile(`[^a-zA-Z0-9:]`) + // Regexp for metric name characters that should be replaced with _. + invalidMetricCharRE = regexp.MustCompile(`[^a-zA-Z0-9:_]`) + multipleUnderscoresRE = regexp.MustCompile(`__+`) +) // Build a normalized name for the specified metric. func normalizeName(metric pmetric.Metric, namespace string, allowUTF8 bool) string { @@ -227,7 +229,6 @@ func normalizeName(metric pmetric.Metric, namespace string, allowUTF8 bool) stri func cleanUpUnit(unit string) string { // Multiple consecutive underscores are replaced with a single underscore. // This is part of the OTel to Prometheus specification: https://github.com/open-telemetry/opentelemetry-specification/blob/v1.38.0/specification/compatibility/prometheus_and_openmetrics.md#otlp-metric-points-to-prometheus. - multipleUnderscoresRE := regexp.MustCompile(`__+`) return strings.TrimPrefix(multipleUnderscoresRE.ReplaceAllString( nonMetricNameCharRE.ReplaceAllString(unit, "_"), "_", From a7ccc8e091fd5a91df345ecba3cef940ef47f0ba Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gy=C3=B6rgy=20Krajcsovits?= Date: Thu, 2 Jan 2025 12:45:20 +0100 Subject: [PATCH 27/40] record_test.go: avoid captures, simply return test refs MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: György Krajcsovits --- tsdb/record/record_test.go | 36 ++++++++++++++---------------------- 1 file changed, 14 insertions(+), 22 deletions(-) diff --git a/tsdb/record/record_test.go b/tsdb/record/record_test.go index f615a334ea..dc625f0830 100644 --- a/tsdb/record/record_test.go +++ b/tsdb/record/record_test.go @@ -469,26 +469,17 @@ func TestRecord_MetadataDecodeUnknownExtraFields(t *testing.T) { require.Equal(t, expectedMetadata, decMetadata) } +type refsCreateFn func(labelCount, histograms, buckets int) ([]RefSeries, []RefSample, []RefHistogramSample) + type recordsMaker struct { name string - init func(int, int, int) + make refsCreateFn } // BenchmarkWAL_HistogramLog measures efficiency of encoding classic // histograms and native historgrams with custom buckets (NHCB). func BenchmarkWAL_HistogramEncoding(b *testing.B) { - // Cache for the refs. - var series []RefSeries - var samples []RefSample - var nhcbs []RefHistogramSample - - resetCache := func() { - series = nil - samples = nil - nhcbs = nil - } - - initClassicRefs := func(labelCount, histograms, buckets int) { + initClassicRefs := func(labelCount, histograms, buckets int) (series []RefSeries, floatSamples []RefSample, histSamples []RefHistogramSample) { ref := chunks.HeadSeriesRef(0) lbls := map[string]string{} for i := range labelCount { @@ -500,7 +491,7 @@ func BenchmarkWAL_HistogramEncoding(b *testing.B) { Ref: ref, Labels: labels.FromMap(lbls), }) - samples = append(samples, RefSample{ + floatSamples = append(floatSamples, RefSample{ Ref: ref, T: 100, V: float64(i), @@ -512,7 +503,7 @@ func BenchmarkWAL_HistogramEncoding(b *testing.B) { Ref: ref, Labels: labels.FromMap(lbls), }) - samples = append(samples, RefSample{ + floatSamples = append(floatSamples, RefSample{ Ref: ref, T: 100, V: float64(i), @@ -529,7 +520,7 @@ func BenchmarkWAL_HistogramEncoding(b *testing.B) { Ref: ref, Labels: labels.FromMap(lbls), }) - samples = append(samples, RefSample{ + floatSamples = append(floatSamples, RefSample{ Ref: ref, T: 100, V: float64(i + j), @@ -538,9 +529,10 @@ func BenchmarkWAL_HistogramEncoding(b *testing.B) { } delete(lbls, model.BucketLabel) } + return } - initNHCBRefs := func(labelCount, histograms, buckets int) { + initNHCBRefs := func(labelCount, histograms, buckets int) (series []RefSeries, floatSamples []RefSample, histSamples []RefHistogramSample) { ref := chunks.HeadSeriesRef(0) lbls := map[string]string{} for i := range labelCount { @@ -563,31 +555,31 @@ func BenchmarkWAL_HistogramEncoding(b *testing.B) { for j := range buckets { h.PositiveBuckets[j] = int64(i + j) } - nhcbs = append(nhcbs, RefHistogramSample{ + histSamples = append(histSamples, RefHistogramSample{ Ref: ref, T: 100, H: h, }) ref++ } + return } for _, maker := range []recordsMaker{ { name: "classic", - init: initClassicRefs, + make: initClassicRefs, }, { name: "nhcb", - init: initNHCBRefs, + make: initNHCBRefs, }, } { for _, labelCount := range []int{0, 10, 50} { for _, histograms := range []int{10, 100, 1000} { for _, buckets := range []int{0, 1, 10, 100} { b.Run(fmt.Sprintf("type=%s/labels=%d/histograms=%d/buckets=%d", maker.name, labelCount, histograms, buckets), func(b *testing.B) { - resetCache() - maker.init(labelCount, histograms, buckets) + series, samples, nhcbs := maker.make(labelCount, histograms, buckets) enc := Encoder{} for range b.N { var buf []byte From cfcb00a716f2dfa286a8f1a479ea7d2790277acd Mon Sep 17 00:00:00 2001 From: George Krajcsovits Date: Thu, 2 Jan 2025 15:51:52 +0100 Subject: [PATCH 28/40] perf(nhcbparse): unroll recursion (#15776) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit https://github.com/prometheus/prometheus/pull/15467#issuecomment-2563585979 Signed-off-by: György Krajcsovits --- model/textparse/nhcbparse.go | 90 ++++++++++++++++++------------------ 1 file changed, 46 insertions(+), 44 deletions(-) diff --git a/model/textparse/nhcbparse.go b/model/textparse/nhcbparse.go index ff756965f4..83e381539f 100644 --- a/model/textparse/nhcbparse.go +++ b/model/textparse/nhcbparse.go @@ -177,61 +177,63 @@ func (p *NHCBParser) CreatedTimestamp() *int64 { } func (p *NHCBParser) Next() (Entry, error) { - if p.state == stateEmitting { - p.state = stateStart - if p.entry == EntrySeries { - isNHCB := p.handleClassicHistogramSeries(p.lset) - if isNHCB && !p.keepClassicHistograms { - // Do not return the classic histogram series if it was converted to NHCB and we are not keeping classic histograms. - return p.Next() + for { + if p.state == stateEmitting { + p.state = stateStart + if p.entry == EntrySeries { + isNHCB := p.handleClassicHistogramSeries(p.lset) + if isNHCB && !p.keepClassicHistograms { + // Do not return the classic histogram series if it was converted to NHCB and we are not keeping classic histograms. + continue + } } + return p.entry, p.err } - return p.entry, p.err - } - p.entry, p.err = p.parser.Next() - if p.err != nil { - if errors.Is(p.err, io.EOF) && p.processNHCB() { - return EntryHistogram, nil - } - return EntryInvalid, p.err - } - switch p.entry { - case EntrySeries: - p.bytes, p.ts, p.value = p.parser.Series() - p.metricString = p.parser.Metric(&p.lset) - // Check the label set to see if we can continue or need to emit the NHCB. - var isNHCB bool - if p.compareLabels() { - // Labels differ. Check if we can emit the NHCB. - if p.processNHCB() { + p.entry, p.err = p.parser.Next() + if p.err != nil { + if errors.Is(p.err, io.EOF) && p.processNHCB() { return EntryHistogram, nil } - isNHCB = p.handleClassicHistogramSeries(p.lset) - } else { - // Labels are the same. Check if after an exponential histogram. - if p.lastHistogramExponential { - isNHCB = false - } else { + return EntryInvalid, p.err + } + switch p.entry { + case EntrySeries: + p.bytes, p.ts, p.value = p.parser.Series() + p.metricString = p.parser.Metric(&p.lset) + // Check the label set to see if we can continue or need to emit the NHCB. + var isNHCB bool + if p.compareLabels() { + // Labels differ. Check if we can emit the NHCB. + if p.processNHCB() { + return EntryHistogram, nil + } isNHCB = p.handleClassicHistogramSeries(p.lset) + } else { + // Labels are the same. Check if after an exponential histogram. + if p.lastHistogramExponential { + isNHCB = false + } else { + isNHCB = p.handleClassicHistogramSeries(p.lset) + } + } + if isNHCB && !p.keepClassicHistograms { + // Do not return the classic histogram series if it was converted to NHCB and we are not keeping classic histograms. + continue } + return p.entry, p.err + case EntryHistogram: + p.bytes, p.ts, p.h, p.fh = p.parser.Histogram() + p.metricString = p.parser.Metric(&p.lset) + p.storeExponentialLabels() + case EntryType: + p.bName, p.typ = p.parser.Type() } - if isNHCB && !p.keepClassicHistograms { - // Do not return the classic histogram series if it was converted to NHCB and we are not keeping classic histograms. - return p.Next() + if p.processNHCB() { + return EntryHistogram, nil } return p.entry, p.err - case EntryHistogram: - p.bytes, p.ts, p.h, p.fh = p.parser.Histogram() - p.metricString = p.parser.Metric(&p.lset) - p.storeExponentialLabels() - case EntryType: - p.bName, p.typ = p.parser.Type() - } - if p.processNHCB() { - return EntryHistogram, nil } - return p.entry, p.err } // Return true if labels have changed and we should emit the NHCB. From a6947a03692f825fcf907be3ef8e3dfb8f47d7b2 Mon Sep 17 00:00:00 2001 From: Bryan Boreham Date: Fri, 3 Jan 2025 14:28:51 +0000 Subject: [PATCH 29/40] Merge 3.1 into main (#15775) Signed-off-by: Bryan Boreham --- CHANGELOG.md | 54 ++++++++++++++++++-- VERSION | 2 +- web/ui/mantine-ui/package.json | 4 +- web/ui/module/codemirror-promql/package.json | 4 +- web/ui/module/lezer-promql/package.json | 2 +- web/ui/package-lock.json | 14 ++--- web/ui/package.json | 2 +- 7 files changed, 63 insertions(+), 19 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index a75e163ec0..d0a7ef6611 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,9 +2,53 @@ ## unreleased -* [CHANGE] Notifier: Increment the prometheus_notifications_errors_total metric by the number of affected alerts rather than by one per batch of affected alerts. #15428 -* [ENHANCEMENT] OTLP receiver: Convert also metric metadata. #15416 -* [BUGFIX] OTLP receiver: Allow colons in non-standard units. #15710 +## 3.1.0 / 2025-01-02 + + * [SECURITY] upgrade golang.org/x/crypto to address reported CVE-2024-45337. #15691 + * [CHANGE] Notifier: Increment prometheus_notifications_errors_total by the number of affected alerts rather than per batch. #15428 + * [CHANGE] API: list rules field "groupNextToken:omitempty" renamed to "groupNextToken". #15400 + * [ENHANCEMENT] OTLP translate: keep identifying attributes in target_info. #15448 + * [ENHANCEMENT] Paginate rule groups, add infinite scroll to rules within groups. #15677 + * [ENHANCEMENT] TSDB: Improve calculation of space used by labels. #13880 + * [ENHANCEMENT] Rules: new metric rule_group_last_rule_duration_sum_seconds. #15672 + * [ENHANCEMENT] Observability: Export 'go_sync_mutex_wait_total_seconds_total' metric. #15339 + * [ENHANCEMEN] Remote-Write: optionally use a DNS resolver that picks a random IP. #15329 + * [PERF] Optimize `l=~".+"` matcher. #15474, #15684 + * [PERF] TSDB: Cache all symbols for compaction . #15455 + * [PERF] TSDB: MemPostings: keep a map of label values slices. #15426 + * [PERF] Remote-Write: Remove interning hook. #15456 + * [PERF] Scrape: optimize string manipulation for experimental native histograms with custom buckets. #15453 + * [PERF] TSDB: reduce memory allocations. #15465, #15427 + * [PERF] Storage: Implement limit in mergeGenericQuerier. #14489 + * [PERF] TSDB: Optimize inverse matching. #14144 + * [PERF] Regex: use stack memory for lowercase copy of string. #15210 + * [PERF] TSDB: When deleting from postings index, pause to unlock and let readers read. #15242 + * [BUGFIX] Main: Avoid possible segfault at exit. (#15724) + * [BUGFIX] Rules: Do not run rules concurrently if uncertain about dependencies. #15560 + * [BUGFIX] PromQL: Adds test for `absent`, `absent_over_time` and `deriv` func with histograms. #15667 + * [BUGFIX] PromQL: Fix various bugs related to quoting UTF-8 characters. #15531 + * [BUGFIX] Scrape: fix nil panic after scrape loop reload. #15563 + * [BUGFIX] Remote-write: fix panic on repeated log message. #15562 + * [BUGFIX] Scrape: reload would ignore always_scrape_classic_histograms and convert_classic_histograms_to_nhcb configs. #15489 + * [BUGFIX] TSDB: fix data corruption in experimental native histograms. #15482 + * [BUGFIX] PromQL: Ignore histograms in all time related functions. #15479 + * [BUGFIX] OTLP receiver: Convert metric metadata. #15416 + * [BUGFIX] PromQL: Fix `resets` function for histograms. #15527 + * [BUGFIX] PromQL: Fix behaviour of `changes()` for mix of histograms and floats. #15469 + * [BUGFIX] PromQL: Fix behaviour of some aggregations with histograms. #15432 + * [BUGFIX] allow quoted exemplar keys in openmetrics text format. #15260 + * [BUGFIX] TSDB: fixes for rare conditions when loading write-behind-log (WBL). #15380 + * [BUGFIX] `round()` function did not remove `__name__` label. #15250 + * [BUGFIX] Promtool: analyze block shows metric name with 0 cardinality. #15438 + * [BUGFIX] PromQL: Fix `count_values` for histograms. #15422 + * [BUGFIX] PromQL: fix issues with comparison binary operations with `bool` modifier and native histograms. #15413 + * [BUGFIX] PromQL: fix incorrect "native histogram ignored in aggregation" annotations. #15414 + * [BUGFIX] PromQL: Corrects the behaviour of some operator and aggregators with Native Histograms. #15245 + * [BUGFIX] TSDB: Always return unknown hint for first sample in non-gauge histogram chunk. #15343 + * [BUGFIX] PromQL: Clamp functions: Ignore any points with native histograms. #15169 + * [BUGFIX] TSDB: Fix race on stale values in headAppender. #15322 + * [BUGFIX] UI: Fix selector / series formatting for empty metric names. #15340 + * [BUGFIX] OTLP receiver: Allow colons in non-standard units. #15710 ## 3.0.1 / 2024-11-28 @@ -37,14 +81,14 @@ This release includes new features such as a brand new UI and UTF-8 support enab * [CHANGE] PromQL: Range selectors and the lookback delta are now left-open, i.e. a sample coinciding with the lower time limit is excluded rather than included. #13904 * [CHANGE] Kubernetes SD: Remove support for `discovery.k8s.io/v1beta1` API version of EndpointSlice. This version is no longer served as of Kubernetes v1.25. #14365 * [CHANGE] Kubernetes SD: Remove support for `networking.k8s.io/v1beta1` API version of Ingress. This version is no longer served as of Kubernetes v1.22. #14365 -* [CHANGE] UTF-8: Enable UTF-8 support by default. Prometheus now allows all UTF-8 characters in metric and label names. The corresponding `utf8-name` feature flag has been removed. #14705 +* [CHANGE] UTF-8: Enable UTF-8 support by default. Prometheus now allows all UTF-8 characters in metric and label names. The corresponding `utf8-name` feature flag has been removed. #14705, #15258 * [CHANGE] Console: Remove example files for the console feature. Users can continue using the console feature by supplying their own JavaScript and templates. #14807 * [CHANGE] SD: Enable the new service discovery manager by default. This SD manager does not restart unchanged discoveries upon reloading. This makes reloads faster and reduces pressure on service discoveries' sources. The corresponding `new-service-discovery-manager` feature flag has been removed. #14770 * [CHANGE] Agent mode has been promoted to stable. The feature flag `agent` has been removed. To run Prometheus in Agent mode, use the new `--agent` cmdline arg instead. #14747 * [CHANGE] Remove deprecated `remote-write-receiver`,`promql-at-modifier`, and `promql-negative-offset` feature flags. #13456, #14526 * [CHANGE] Remove deprecated `storage.tsdb.allow-overlapping-blocks`, `alertmanager.timeout`, and `storage.tsdb.retention` flags. #14640, #14643 * [FEATURE] OTLP receiver: Ability to skip UTF-8 normalization using `otlp.translation_strategy = NoUTF8EscapingWithSuffixes` configuration option. #15384 -* [FEATURE] Support config reload automatically - feature flag `auto-reload-config`. #14769 +* [FEATURE] Support config reload automatically - feature flag `auto-reload-config`. #14769, #15011 * [ENHANCEMENT] Scraping, rules: handle targets reappearing, or rules moving group, when out-of-order is enabled. #14710 * [ENHANCEMENT] Tools: add debug printouts to promtool rules unit testing #15196 * [ENHANCEMENT] Scraping: support Created-Timestamp feature on native histograms. #14694 diff --git a/VERSION b/VERSION index cb2b00e4f7..fd2a01863f 100644 --- a/VERSION +++ b/VERSION @@ -1 +1 @@ -3.0.1 +3.1.0 diff --git a/web/ui/mantine-ui/package.json b/web/ui/mantine-ui/package.json index a8c7ebd417..c3f1e0fbfd 100644 --- a/web/ui/mantine-ui/package.json +++ b/web/ui/mantine-ui/package.json @@ -1,7 +1,7 @@ { "name": "@prometheus-io/mantine-ui", "private": true, - "version": "0.300.1", + "version": "0.301.0", "type": "module", "scripts": { "start": "vite", @@ -28,7 +28,7 @@ "@microsoft/fetch-event-source": "^2.0.1", "@nexucis/fuzzy": "^0.5.1", "@nexucis/kvsearch": "^0.9.1", - "@prometheus-io/codemirror-promql": "0.300.1", + "@prometheus-io/codemirror-promql": "0.301.0", "@reduxjs/toolkit": "^2.5.0", "@tabler/icons-react": "^3.24.0", "@tanstack/react-query": "^5.62.7", diff --git a/web/ui/module/codemirror-promql/package.json b/web/ui/module/codemirror-promql/package.json index c10ebf6115..d4e95c7ee8 100644 --- a/web/ui/module/codemirror-promql/package.json +++ b/web/ui/module/codemirror-promql/package.json @@ -1,6 +1,6 @@ { "name": "@prometheus-io/codemirror-promql", - "version": "0.300.1", + "version": "0.301.0", "description": "a CodeMirror mode for the PromQL language", "types": "dist/esm/index.d.ts", "module": "dist/esm/index.js", @@ -29,7 +29,7 @@ }, "homepage": "https://github.com/prometheus/prometheus/blob/main/web/ui/module/codemirror-promql/README.md", "dependencies": { - "@prometheus-io/lezer-promql": "0.300.1", + "@prometheus-io/lezer-promql": "0.301.0", "lru-cache": "^11.0.2" }, "devDependencies": { diff --git a/web/ui/module/lezer-promql/package.json b/web/ui/module/lezer-promql/package.json index bb7af294d9..282ab5ab62 100644 --- a/web/ui/module/lezer-promql/package.json +++ b/web/ui/module/lezer-promql/package.json @@ -1,6 +1,6 @@ { "name": "@prometheus-io/lezer-promql", - "version": "0.300.1", + "version": "0.301.0", "description": "lezer-based PromQL grammar", "main": "dist/index.cjs", "type": "module", diff --git a/web/ui/package-lock.json b/web/ui/package-lock.json index a73f55a49f..6db3035c27 100644 --- a/web/ui/package-lock.json +++ b/web/ui/package-lock.json @@ -1,12 +1,12 @@ { "name": "prometheus-io", - "version": "0.300.1", + "version": "0.301.0", "lockfileVersion": 3, "requires": true, "packages": { "": { "name": "prometheus-io", - "version": "0.300.1", + "version": "0.301.0", "workspaces": [ "mantine-ui", "module/*" @@ -24,7 +24,7 @@ }, "mantine-ui": { "name": "@prometheus-io/mantine-ui", - "version": "0.300.1", + "version": "0.301.0", "dependencies": { "@codemirror/autocomplete": "^6.18.3", "@codemirror/language": "^6.10.6", @@ -42,7 +42,7 @@ "@microsoft/fetch-event-source": "^2.0.1", "@nexucis/fuzzy": "^0.5.1", "@nexucis/kvsearch": "^0.9.1", - "@prometheus-io/codemirror-promql": "0.300.1", + "@prometheus-io/codemirror-promql": "0.301.0", "@reduxjs/toolkit": "^2.5.0", "@tabler/icons-react": "^3.24.0", "@tanstack/react-query": "^5.62.7", @@ -147,10 +147,10 @@ }, "module/codemirror-promql": { "name": "@prometheus-io/codemirror-promql", - "version": "0.300.1", + "version": "0.301.0", "license": "Apache-2.0", "dependencies": { - "@prometheus-io/lezer-promql": "0.300.1", + "@prometheus-io/lezer-promql": "0.301.0", "lru-cache": "^11.0.2" }, "devDependencies": { @@ -180,7 +180,7 @@ }, "module/lezer-promql": { "name": "@prometheus-io/lezer-promql", - "version": "0.300.1", + "version": "0.301.0", "license": "Apache-2.0", "devDependencies": { "@lezer/generator": "^1.7.2", diff --git a/web/ui/package.json b/web/ui/package.json index bfebd64bd5..62b3f26246 100644 --- a/web/ui/package.json +++ b/web/ui/package.json @@ -1,7 +1,7 @@ { "name": "prometheus-io", "description": "Monorepo for the Prometheus UI", - "version": "0.300.1", + "version": "0.301.0", "private": true, "scripts": { "build": "bash build_ui.sh --all", From 4f67a38a390224b4f501c2f139daabeeaa49792b Mon Sep 17 00:00:00 2001 From: Arve Knudsen Date: Fri, 3 Jan 2025 17:58:02 +0100 Subject: [PATCH 30/40] template: Use cases.Title instead of deprecated strings.Title (#15721) Signed-off-by: Arve Knudsen --- docs/configuration/template_reference.md | 2 +- template/template.go | 4 +++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/configuration/template_reference.md b/docs/configuration/template_reference.md index 47df9d1e09..ec4b31376c 100644 --- a/docs/configuration/template_reference.md +++ b/docs/configuration/template_reference.md @@ -68,7 +68,7 @@ versions. | Name | Arguments | Returns | Notes | | ------------- | ------------- | ------- | ----------- | -| title | string | string | [strings.Title](https://golang.org/pkg/strings/#Title), capitalises first character of each word.| +| title | string | string | [cases.Title](https://pkg.go.dev/golang.org/x/text/cases#Title), capitalises first character of each word.| | toUpper | string | string | [strings.ToUpper](https://golang.org/pkg/strings/#ToUpper), converts all characters to upper case.| | toLower | string | string | [strings.ToLower](https://golang.org/pkg/strings/#ToLower), converts all characters to lower case.| | stripPort | string | string | [net.SplitHostPort](https://pkg.go.dev/net#SplitHostPort), splits string into host and port, then returns only host.| diff --git a/template/template.go b/template/template.go index 0698c6c8ac..25b65eb577 100644 --- a/template/template.go +++ b/template/template.go @@ -30,6 +30,8 @@ import ( "github.com/grafana/regexp" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/common/model" + "golang.org/x/text/cases" + "golang.org/x/text/language" common_templates "github.com/prometheus/common/helpers/templates" @@ -166,7 +168,7 @@ func NewTemplateExpander( return html_template.HTML(text) }, "match": regexp.MatchString, - "title": strings.Title, //nolint:staticcheck // TODO(beorn7): Need to come up with a replacement using the cases package. + "title": cases.Title(language.AmericanEnglish, cases.NoLower).String, "toUpper": strings.ToUpper, "toLower": strings.ToLower, "graphLink": strutil.GraphLinkForExpression, From 56094197b598b56434c5d8c32b176f115821cc3b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?H=C3=A9lia=20Barroso?= <66432275+heliapb@users.noreply.github.com> Date: Mon, 6 Jan 2025 14:13:17 +0000 Subject: [PATCH 31/40] [Docs] Note that scrape_timeout cannot be greater than scrape_interval (#15786) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Hélia Barroso --- docs/configuration/configuration.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/configuration/configuration.md b/docs/configuration/configuration.md index 57f4013936..168c99d3ca 100644 --- a/docs/configuration/configuration.md +++ b/docs/configuration/configuration.md @@ -59,6 +59,7 @@ global: [ scrape_interval: | default = 1m ] # How long until a scrape request times out. + # It cannot be greater than the scrape interval. [ scrape_timeout: | default = 10s ] # The protocols to negotiate during a scrape with the client. @@ -221,6 +222,7 @@ job_name: [ scrape_interval: | default = ] # Per-scrape timeout when scraping this job. +# It cannot be greater than the scrape interval. [ scrape_timeout: | default = ] # The protocols to negotiate during a scrape with the client. From 5fdec3140188808bb9c2bf48e94b8533dbf75ac5 Mon Sep 17 00:00:00 2001 From: Arthur Silva Sens Date: Mon, 6 Jan 2025 11:30:39 -0300 Subject: [PATCH 32/40] otlp/translator: Use separate function for metric names with UTF8 characters (#15664) BuildCompliantName was renamed to BuildCompliantMetricName, and it no longer takes UTF8 support into consideration. It focuses on building a metric name that follows Prometheus conventions. A new function, BuildMetricName, was added to optionally add unit and type suffixes to OTLP metric names without translating any characters to underscores(_). --- .../prometheus/helpers_from_stdlib.go | 106 -------- ...rmalize_name.go => metric_name_builder.go} | 211 ++++++++------ .../prometheus/metric_name_builder_test.go | 257 ++++++++++++++++++ .../prometheus/normalize_name_test.go | 210 -------------- .../prometheusremotewrite/histograms_test.go | 2 +- .../prometheusremotewrite/metrics_to_prw.go | 7 +- .../metrics_to_prw_test.go | 2 +- 7 files changed, 391 insertions(+), 404 deletions(-) delete mode 100644 storage/remote/otlptranslator/prometheus/helpers_from_stdlib.go rename storage/remote/otlptranslator/prometheus/{normalize_name.go => metric_name_builder.go} (56%) create mode 100644 storage/remote/otlptranslator/prometheus/metric_name_builder_test.go delete mode 100644 storage/remote/otlptranslator/prometheus/normalize_name_test.go diff --git a/storage/remote/otlptranslator/prometheus/helpers_from_stdlib.go b/storage/remote/otlptranslator/prometheus/helpers_from_stdlib.go deleted file mode 100644 index cb9257d073..0000000000 --- a/storage/remote/otlptranslator/prometheus/helpers_from_stdlib.go +++ /dev/null @@ -1,106 +0,0 @@ -// Copyright 2024 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. -// Provenance-includes-location: https://github.com/golang/go/blob/f2d118fd5f7e872804a5825ce29797f81a28b0fa/src/strings/strings.go -// Provenance-includes-license: BSD-3-Clause -// Provenance-includes-copyright: Copyright The Go Authors. - -package prometheus - -import "strings" - -// fieldsFunc is a copy of strings.FieldsFunc from the Go standard library, -// but it also returns the separators as part of the result. -func fieldsFunc(s string, f func(rune) bool) ([]string, []string) { - // A span is used to record a slice of s of the form s[start:end]. - // The start index is inclusive and the end index is exclusive. - type span struct { - start int - end int - } - spans := make([]span, 0, 32) - separators := make([]string, 0, 32) - - // Find the field start and end indices. - // Doing this in a separate pass (rather than slicing the string s - // and collecting the result substrings right away) is significantly - // more efficient, possibly due to cache effects. - start := -1 // valid span start if >= 0 - for end, rune := range s { - if f(rune) { - if start >= 0 { - spans = append(spans, span{start, end}) - // Set start to a negative value. - // Note: using -1 here consistently and reproducibly - // slows down this code by a several percent on amd64. - start = ^start - separators = append(separators, string(s[end])) - } - } else { - if start < 0 { - start = end - } - } - } - - // Last field might end at EOF. - if start >= 0 { - spans = append(spans, span{start, len(s)}) - } - - // Create strings from recorded field indices. - a := make([]string, len(spans)) - for i, span := range spans { - a[i] = s[span.start:span.end] - } - - return a, separators -} - -// join is a copy of strings.Join from the Go standard library, -// but it also accepts a slice of separators to join the elements with. -// If the slice of separators is shorter than the slice of elements, use a default value. -// We also don't check for integer overflow. -func join(elems []string, separators []string, def string) string { - switch len(elems) { - case 0: - return "" - case 1: - return elems[0] - } - - var n int - var sep string - sepLen := len(separators) - for i, elem := range elems { - if i >= sepLen { - sep = def - } else { - sep = separators[i] - } - n += len(sep) + len(elem) - } - - var b strings.Builder - b.Grow(n) - b.WriteString(elems[0]) - for i, s := range elems[1:] { - if i >= sepLen { - sep = def - } else { - sep = separators[i] - } - b.WriteString(sep) - b.WriteString(s) - } - return b.String() -} diff --git a/storage/remote/otlptranslator/prometheus/normalize_name.go b/storage/remote/otlptranslator/prometheus/metric_name_builder.go similarity index 56% rename from storage/remote/otlptranslator/prometheus/normalize_name.go rename to storage/remote/otlptranslator/prometheus/metric_name_builder.go index 580c72b548..8b5ea2a046 100644 --- a/storage/remote/otlptranslator/prometheus/normalize_name.go +++ b/storage/remote/otlptranslator/prometheus/metric_name_builder.go @@ -78,7 +78,7 @@ var perUnitMap = map[string]string{ "y": "year", } -// BuildCompliantName builds a Prometheus-compliant metric name for the specified metric. +// BuildCompliantMetricName builds a Prometheus-compliant metric name for the specified metric. // // Metric name is prefixed with specified namespace and underscore (if any). // Namespace is not cleaned up. Make sure specified namespace follows Prometheus @@ -87,29 +87,24 @@ var perUnitMap = map[string]string{ // See rules at https://prometheus.io/docs/concepts/data_model/#metric-names-and-labels, // https://prometheus.io/docs/practices/naming/#metric-and-label-naming // and https://github.com/open-telemetry/opentelemetry-specification/blob/v1.38.0/specification/compatibility/prometheus_and_openmetrics.md#otlp-metric-points-to-prometheus. -func BuildCompliantName(metric pmetric.Metric, namespace string, addMetricSuffixes, allowUTF8 bool) string { +func BuildCompliantMetricName(metric pmetric.Metric, namespace string, addMetricSuffixes bool) string { // Full normalization following standard Prometheus naming conventions if addMetricSuffixes { - return normalizeName(metric, namespace, allowUTF8) + return normalizeName(metric, namespace) } - var metricName string - if !allowUTF8 { - // Simple case (no full normalization, no units, etc.). - metricName = strings.Join(strings.FieldsFunc(metric.Name(), func(r rune) bool { - return invalidMetricCharRE.MatchString(string(r)) - }), "_") - } else { - metricName = metric.Name() - } + // Simple case (no full normalization, no units, etc.). + metricName := strings.Join(strings.FieldsFunc(metric.Name(), func(r rune) bool { + return invalidMetricCharRE.MatchString(string(r)) + }), "_") // Namespace? if namespace != "" { return namespace + "_" + metricName } - // Metric name starts with a digit and utf8 not allowed? Prefix it with an underscore. - if metricName != "" && unicode.IsDigit(rune(metricName[0])) && !allowUTF8 { + // Metric name starts with a digit? Prefix it with an underscore. + if metricName != "" && unicode.IsDigit(rune(metricName[0])) { metricName = "_" + metricName } @@ -124,70 +119,17 @@ var ( ) // Build a normalized name for the specified metric. -func normalizeName(metric pmetric.Metric, namespace string, allowUTF8 bool) string { - var nameTokens []string - var separators []string - if !allowUTF8 { - // Split metric name into "tokens" (of supported metric name runes). - // Note that this has the side effect of replacing multiple consecutive underscores with a single underscore. - // This is part of the OTel to Prometheus specification: https://github.com/open-telemetry/opentelemetry-specification/blob/v1.38.0/specification/compatibility/prometheus_and_openmetrics.md#otlp-metric-points-to-prometheus. - nameTokens = strings.FieldsFunc( - metric.Name(), - func(r rune) bool { return nonMetricNameCharRE.MatchString(string(r)) }, - ) - } else { - translationFunc := func(r rune) bool { return !unicode.IsLetter(r) && !unicode.IsDigit(r) && r != ':' } - // Split metric name into "tokens" (of supported metric name runes). - nameTokens, separators = fieldsFunc(metric.Name(), translationFunc) - } - - // Split unit at the '/' if any - unitTokens := strings.SplitN(metric.Unit(), "/", 2) - - // Main unit - // Append if not blank, doesn't contain '{}', and is not present in metric name already - if len(unitTokens) > 0 { - var mainUnitProm, perUnitProm string - mainUnitOTel := strings.TrimSpace(unitTokens[0]) - if mainUnitOTel != "" && !strings.ContainsAny(mainUnitOTel, "{}") { - mainUnitProm = unitMapGetOrDefault(mainUnitOTel) - if !allowUTF8 { - mainUnitProm = cleanUpUnit(mainUnitProm) - } - if slices.Contains(nameTokens, mainUnitProm) { - mainUnitProm = "" - } - } - - // Per unit - // Append if not blank, doesn't contain '{}', and is not present in metric name already - if len(unitTokens) > 1 && unitTokens[1] != "" { - perUnitOTel := strings.TrimSpace(unitTokens[1]) - if perUnitOTel != "" && !strings.ContainsAny(perUnitOTel, "{}") { - perUnitProm = perUnitMapGetOrDefault(perUnitOTel) - if !allowUTF8 { - perUnitProm = cleanUpUnit(perUnitProm) - } - } - if perUnitProm != "" { - perUnitProm = "per_" + perUnitProm - if slices.Contains(nameTokens, perUnitProm) { - perUnitProm = "" - } - } - } - - if perUnitProm != "" { - mainUnitProm = strings.TrimSuffix(mainUnitProm, "_") - } +func normalizeName(metric pmetric.Metric, namespace string) string { + // Split metric name into "tokens" (of supported metric name runes). + // Note that this has the side effect of replacing multiple consecutive underscores with a single underscore. + // This is part of the OTel to Prometheus specification: https://github.com/open-telemetry/opentelemetry-specification/blob/v1.38.0/specification/compatibility/prometheus_and_openmetrics.md#otlp-metric-points-to-prometheus. + nameTokens := strings.FieldsFunc( + metric.Name(), + func(r rune) bool { return nonMetricNameCharRE.MatchString(string(r)) }, + ) - if mainUnitProm != "" { - nameTokens = append(nameTokens, mainUnitProm) - } - if perUnitProm != "" { - nameTokens = append(nameTokens, perUnitProm) - } - } + mainUnitSuffix, perUnitSuffix := buildUnitSuffixes(metric.Unit()) + nameTokens = addUnitTokens(nameTokens, cleanUpUnit(mainUnitSuffix), cleanUpUnit(perUnitSuffix)) // Append _total for Counters if metric.Type() == pmetric.MetricTypeSum && metric.Sum().IsMonotonic() { @@ -208,14 +150,8 @@ func normalizeName(metric pmetric.Metric, namespace string, allowUTF8 bool) stri nameTokens = append([]string{namespace}, nameTokens...) } - var normalizedName string - if !allowUTF8 { - // Build the string from the tokens, separated with underscores - normalizedName = strings.Join(nameTokens, "_") - } else { - // Build the string from the tokens + separators. - normalizedName = join(nameTokens, separators, "_") - } + // Build the string from the tokens, separated with underscores + normalizedName := strings.Join(nameTokens, "_") // Metric name cannot start with a digit, so prefix it with "_" in this case if normalizedName != "" && unicode.IsDigit(rune(normalizedName[0])) { @@ -225,6 +161,39 @@ func normalizeName(metric pmetric.Metric, namespace string, allowUTF8 bool) stri return normalizedName } +// addUnitTokens will add the suffixes to the nameTokens if they are not already present. +// It will also remove trailing underscores from the main suffix to avoid double underscores +// when joining the tokens. +// +// If the 'per' unit ends with underscore, the underscore will be removed. If the per unit is just +// 'per_', it will be entirely removed. +func addUnitTokens(nameTokens []string, mainUnitSuffix, perUnitSuffix string) []string { + if slices.Contains(nameTokens, mainUnitSuffix) { + mainUnitSuffix = "" + } + + if perUnitSuffix == "per_" { + perUnitSuffix = "" + } else { + perUnitSuffix = strings.TrimSuffix(perUnitSuffix, "_") + if slices.Contains(nameTokens, perUnitSuffix) { + perUnitSuffix = "" + } + } + + if perUnitSuffix != "" { + mainUnitSuffix = strings.TrimSuffix(mainUnitSuffix, "_") + } + + if mainUnitSuffix != "" { + nameTokens = append(nameTokens, mainUnitSuffix) + } + if perUnitSuffix != "" { + nameTokens = append(nameTokens, perUnitSuffix) + } + return nameTokens +} + // cleanUpUnit cleans up unit so it matches model.LabelNameRE. func cleanUpUnit(unit string) string { // Multiple consecutive underscores are replaced with a single underscore. @@ -263,3 +232,75 @@ func removeItem(slice []string, value string) []string { } return newSlice } + +// BuildMetricName builds a valid metric name but without following Prometheus naming conventions. +// It doesn't do any character transformation, it only prefixes the metric name with the namespace, if any, +// and adds metric type suffixes, e.g. "_total" for counters and unit suffixes. +// +// Differently from BuildCompliantMetricName, it doesn't check for the presence of unit and type suffixes. +// If "addMetricSuffixes" is true, it will add them anyway. +// +// Please use BuildCompliantMetricName for a metric name that follows Prometheus naming conventions. +func BuildMetricName(metric pmetric.Metric, namespace string, addMetricSuffixes bool) string { + metricName := metric.Name() + + if namespace != "" { + metricName = namespace + "_" + metricName + } + + if addMetricSuffixes { + mainUnitSuffix, perUnitSuffix := buildUnitSuffixes(metric.Unit()) + if mainUnitSuffix != "" { + metricName = metricName + "_" + mainUnitSuffix + } + if perUnitSuffix != "" { + metricName = metricName + "_" + perUnitSuffix + } + + // Append _total for Counters + if metric.Type() == pmetric.MetricTypeSum && metric.Sum().IsMonotonic() { + metricName = metricName + "_total" + } + + // Append _ratio for metrics with unit "1" + // Some OTel receivers improperly use unit "1" for counters of objects + // See https://github.com/open-telemetry/opentelemetry-collector-contrib/issues?q=is%3Aissue+some+metric+units+don%27t+follow+otel+semantic+conventions + // Until these issues have been fixed, we're appending `_ratio` for gauges ONLY + // Theoretically, counters could be ratios as well, but it's absurd (for mathematical reasons) + if metric.Unit() == "1" && metric.Type() == pmetric.MetricTypeGauge { + metricName = metricName + "_ratio" + } + } + return metricName +} + +// buildUnitSuffixes builds the main and per unit suffixes for the specified unit +// but doesn't do any special character transformation to accommodate Prometheus naming conventions. +// Removing trailing underscores or appending suffixes is done in the caller. +func buildUnitSuffixes(unit string) (mainUnitSuffix, perUnitSuffix string) { + // Split unit at the '/' if any + unitTokens := strings.SplitN(unit, "/", 2) + + if len(unitTokens) > 0 { + // Main unit + // Update if not blank and doesn't contain '{}' + mainUnitOTel := strings.TrimSpace(unitTokens[0]) + if mainUnitOTel != "" && !strings.ContainsAny(mainUnitOTel, "{}") { + mainUnitSuffix = unitMapGetOrDefault(mainUnitOTel) + } + + // Per unit + // Update if not blank and doesn't contain '{}' + if len(unitTokens) > 1 && unitTokens[1] != "" { + perUnitOTel := strings.TrimSpace(unitTokens[1]) + if perUnitOTel != "" && !strings.ContainsAny(perUnitOTel, "{}") { + perUnitSuffix = perUnitMapGetOrDefault(perUnitOTel) + } + if perUnitSuffix != "" { + perUnitSuffix = "per_" + perUnitSuffix + } + } + } + + return mainUnitSuffix, perUnitSuffix +} diff --git a/storage/remote/otlptranslator/prometheus/metric_name_builder_test.go b/storage/remote/otlptranslator/prometheus/metric_name_builder_test.go new file mode 100644 index 0000000000..1c4a6124c4 --- /dev/null +++ b/storage/remote/otlptranslator/prometheus/metric_name_builder_test.go @@ -0,0 +1,257 @@ +// Copyright 2024 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. +// Provenance-includes-location: https://github.com/open-telemetry/opentelemetry-collector-contrib/blob/95e8f8fdc2a9dc87230406c9a3cf02be4fd68bea/pkg/translator/prometheus/normalize_name_test.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: Copyright The OpenTelemetry Authors. + +package prometheus + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestByte(t *testing.T) { + require.Equal(t, "system_filesystem_usage_bytes", normalizeName(createGauge("system.filesystem.usage", "By"), "")) +} + +func TestByteCounter(t *testing.T) { + require.Equal(t, "system_io_bytes_total", normalizeName(createCounter("system.io", "By"), "")) + require.Equal(t, "network_transmitted_bytes_total", normalizeName(createCounter("network_transmitted_bytes_total", "By"), "")) +} + +func TestWhiteSpaces(t *testing.T) { + require.Equal(t, "system_filesystem_usage_bytes", normalizeName(createGauge("\t system.filesystem.usage ", " By\t"), "")) +} + +func TestNonStandardUnit(t *testing.T) { + require.Equal(t, "system_network_dropped", normalizeName(createGauge("system.network.dropped", "{packets}"), "")) + // The normal metric name character set is allowed in non-standard units. + require.Equal(t, "system_network_dropped_nonstandard:_1", normalizeName(createGauge("system.network.dropped", "nonstandard:_1"), "")) +} + +func TestNonStandardUnitCounter(t *testing.T) { + require.Equal(t, "system_network_dropped_total", normalizeName(createCounter("system.network.dropped", "{packets}"), "")) +} + +func TestBrokenUnit(t *testing.T) { + require.Equal(t, "system_network_dropped_packets", normalizeName(createGauge("system.network.dropped", "packets"), "")) + require.Equal(t, "system_network_packets_dropped", normalizeName(createGauge("system.network.packets.dropped", "packets"), "")) + require.Equal(t, "system_network_packets", normalizeName(createGauge("system.network.packets", "packets"), "")) +} + +func TestBrokenUnitCounter(t *testing.T) { + require.Equal(t, "system_network_dropped_packets_total", normalizeName(createCounter("system.network.dropped", "packets"), "")) + require.Equal(t, "system_network_packets_dropped_total", normalizeName(createCounter("system.network.packets.dropped", "packets"), "")) + require.Equal(t, "system_network_packets_total", normalizeName(createCounter("system.network.packets", "packets"), "")) +} + +func TestRatio(t *testing.T) { + require.Equal(t, "hw_gpu_memory_utilization_ratio", normalizeName(createGauge("hw.gpu.memory.utilization", "1"), "")) + require.Equal(t, "hw_fan_speed_ratio", normalizeName(createGauge("hw.fan.speed_ratio", "1"), "")) + require.Equal(t, "objects_total", normalizeName(createCounter("objects", "1"), "")) +} + +func TestHertz(t *testing.T) { + require.Equal(t, "hw_cpu_speed_limit_hertz", normalizeName(createGauge("hw.cpu.speed_limit", "Hz"), "")) +} + +func TestPer(t *testing.T) { + require.Equal(t, "broken_metric_speed_km_per_hour", normalizeName(createGauge("broken.metric.speed", "km/h"), "")) + require.Equal(t, "astro_light_speed_limit_meters_per_second", normalizeName(createGauge("astro.light.speed_limit", "m/s"), "")) + // The normal metric name character set is allowed in non-standard units. + require.Equal(t, "system_network_dropped_non_per_standard:_1", normalizeName(createGauge("system.network.dropped", "non/standard:_1"), "")) + + t.Run("invalid per unit", func(t *testing.T) { + require.Equal(t, "broken_metric_speed_km", normalizeName(createGauge("broken.metric.speed", "km/°"), "")) + }) +} + +func TestPercent(t *testing.T) { + require.Equal(t, "broken_metric_success_ratio_percent", normalizeName(createGauge("broken.metric.success_ratio", "%"), "")) + require.Equal(t, "broken_metric_success_percent", normalizeName(createGauge("broken.metric.success_percent", "%"), "")) +} + +func TestEmpty(t *testing.T) { + require.Equal(t, "test_metric_no_unit", normalizeName(createGauge("test.metric.no_unit", ""), "")) + require.Equal(t, "test_metric_spaces", normalizeName(createGauge("test.metric.spaces", " \t "), "")) +} + +func TestOTelReceivers(t *testing.T) { + require.Equal(t, "active_directory_ds_replication_network_io_bytes_total", normalizeName(createCounter("active_directory.ds.replication.network.io", "By"), "")) + require.Equal(t, "active_directory_ds_replication_sync_object_pending_total", normalizeName(createCounter("active_directory.ds.replication.sync.object.pending", "{objects}"), "")) + require.Equal(t, "active_directory_ds_replication_object_rate_per_second", normalizeName(createGauge("active_directory.ds.replication.object.rate", "{objects}/s"), "")) + require.Equal(t, "active_directory_ds_name_cache_hit_rate_percent", normalizeName(createGauge("active_directory.ds.name_cache.hit_rate", "%"), "")) + require.Equal(t, "active_directory_ds_ldap_bind_last_successful_time_milliseconds", normalizeName(createGauge("active_directory.ds.ldap.bind.last_successful.time", "ms"), "")) + require.Equal(t, "apache_current_connections", normalizeName(createGauge("apache.current_connections", "connections"), "")) + require.Equal(t, "apache_workers_connections", normalizeName(createGauge("apache.workers", "connections"), "")) + require.Equal(t, "apache_requests_total", normalizeName(createCounter("apache.requests", "1"), "")) + require.Equal(t, "bigip_virtual_server_request_count_total", normalizeName(createCounter("bigip.virtual_server.request.count", "{requests}"), "")) + require.Equal(t, "system_cpu_utilization_ratio", normalizeName(createGauge("system.cpu.utilization", "1"), "")) + require.Equal(t, "system_disk_operation_time_seconds_total", normalizeName(createCounter("system.disk.operation_time", "s"), "")) + require.Equal(t, "system_cpu_load_average_15m_ratio", normalizeName(createGauge("system.cpu.load_average.15m", "1"), "")) + require.Equal(t, "memcached_operation_hit_ratio_percent", normalizeName(createGauge("memcached.operation_hit_ratio", "%"), "")) + require.Equal(t, "mongodbatlas_process_asserts_per_second", normalizeName(createGauge("mongodbatlas.process.asserts", "{assertions}/s"), "")) + require.Equal(t, "mongodbatlas_process_journaling_data_files_mebibytes", normalizeName(createGauge("mongodbatlas.process.journaling.data_files", "MiBy"), "")) + require.Equal(t, "mongodbatlas_process_network_io_bytes_per_second", normalizeName(createGauge("mongodbatlas.process.network.io", "By/s"), "")) + require.Equal(t, "mongodbatlas_process_oplog_rate_gibibytes_per_hour", normalizeName(createGauge("mongodbatlas.process.oplog.rate", "GiBy/h"), "")) + require.Equal(t, "mongodbatlas_process_db_query_targeting_scanned_per_returned", normalizeName(createGauge("mongodbatlas.process.db.query_targeting.scanned_per_returned", "{scanned}/{returned}"), "")) + require.Equal(t, "nginx_requests", normalizeName(createGauge("nginx.requests", "requests"), "")) + require.Equal(t, "nginx_connections_accepted", normalizeName(createGauge("nginx.connections_accepted", "connections"), "")) + require.Equal(t, "nsxt_node_memory_usage_kilobytes", normalizeName(createGauge("nsxt.node.memory.usage", "KBy"), "")) + require.Equal(t, "redis_latest_fork_microseconds", normalizeName(createGauge("redis.latest_fork", "us"), "")) +} + +func TestNamespace(t *testing.T) { + require.Equal(t, "space_test", normalizeName(createGauge("test", ""), "space")) + require.Equal(t, "space_test", normalizeName(createGauge("#test", ""), "space")) +} + +func TestCleanUpUnit(t *testing.T) { + require.Equal(t, "", cleanUpUnit("")) + require.Equal(t, "a_b", cleanUpUnit("a b")) + require.Equal(t, "hello_world", cleanUpUnit("hello, world")) + require.Equal(t, "hello_you_2", cleanUpUnit("hello you 2")) + require.Equal(t, "1000", cleanUpUnit("$1000")) + require.Equal(t, "", cleanUpUnit("*+$^=)")) +} + +func TestUnitMapGetOrDefault(t *testing.T) { + require.Equal(t, "", unitMapGetOrDefault("")) + require.Equal(t, "seconds", unitMapGetOrDefault("s")) + require.Equal(t, "invalid", unitMapGetOrDefault("invalid")) +} + +func TestPerUnitMapGetOrDefault(t *testing.T) { + require.Equal(t, "", perUnitMapGetOrDefault("")) + require.Equal(t, "second", perUnitMapGetOrDefault("s")) + require.Equal(t, "invalid", perUnitMapGetOrDefault("invalid")) +} + +func TestBuildUnitSuffixes(t *testing.T) { + tests := []struct { + unit string + expectedMain string + expectedPer string + }{ + {"", "", ""}, + {"s", "seconds", ""}, + {"By/s", "bytes", "per_second"}, + {"requests/m", "requests", "per_minute"}, + {"{invalid}/second", "", "per_second"}, + {"bytes/{invalid}", "bytes", ""}, + } + + for _, test := range tests { + mainUnitSuffix, perUnitSuffix := buildUnitSuffixes(test.unit) + require.Equal(t, test.expectedMain, mainUnitSuffix) + require.Equal(t, test.expectedPer, perUnitSuffix) + } +} + +func TestAddUnitTokens(t *testing.T) { + tests := []struct { + nameTokens []string + mainUnitSuffix string + perUnitSuffix string + expected []string + }{ + {[]string{}, "", "", []string{}}, + {[]string{"token1"}, "main", "", []string{"token1", "main"}}, + {[]string{"token1"}, "", "per", []string{"token1", "per"}}, + {[]string{"token1"}, "main", "per", []string{"token1", "main", "per"}}, + {[]string{"token1", "per"}, "main", "per", []string{"token1", "per", "main"}}, + {[]string{"token1", "main"}, "main", "per", []string{"token1", "main", "per"}}, + {[]string{"token1"}, "main_", "per", []string{"token1", "main", "per"}}, + {[]string{"token1"}, "main_unit", "per_seconds_", []string{"token1", "main_unit", "per_seconds"}}, // trailing underscores are removed + {[]string{"token1"}, "main_unit", "per_", []string{"token1", "main_unit"}}, // 'per_' is removed entirely + } + + for _, test := range tests { + result := addUnitTokens(test.nameTokens, test.mainUnitSuffix, test.perUnitSuffix) + require.Equal(t, test.expected, result) + } +} + +func TestRemoveItem(t *testing.T) { + require.Equal(t, []string{}, removeItem([]string{}, "test")) + require.Equal(t, []string{}, removeItem([]string{}, "")) + require.Equal(t, []string{"a", "b", "c"}, removeItem([]string{"a", "b", "c"}, "d")) + require.Equal(t, []string{"a", "b", "c"}, removeItem([]string{"a", "b", "c"}, "")) + require.Equal(t, []string{"a", "b"}, removeItem([]string{"a", "b", "c"}, "c")) + require.Equal(t, []string{"a", "c"}, removeItem([]string{"a", "b", "c"}, "b")) + require.Equal(t, []string{"b", "c"}, removeItem([]string{"a", "b", "c"}, "a")) +} + +func TestBuildCompliantMetricNameWithSuffixes(t *testing.T) { + require.Equal(t, "system_io_bytes_total", BuildCompliantMetricName(createCounter("system.io", "By"), "", true)) + require.Equal(t, "system_network_io_bytes_total", BuildCompliantMetricName(createCounter("network.io", "By"), "system", true)) + require.Equal(t, "_3_14_digits", BuildCompliantMetricName(createGauge("3.14 digits", ""), "", true)) + require.Equal(t, "envoy_rule_engine_zlib_buf_error", BuildCompliantMetricName(createGauge("envoy__rule_engine_zlib_buf_error", ""), "", true)) + require.Equal(t, ":foo::bar", BuildCompliantMetricName(createGauge(":foo::bar", ""), "", true)) + require.Equal(t, ":foo::bar_total", BuildCompliantMetricName(createCounter(":foo::bar", ""), "", true)) + // Gauges with unit 1 are considered ratios. + require.Equal(t, "foo_bar_ratio", BuildCompliantMetricName(createGauge("foo.bar", "1"), "", true)) + // Slashes in units are converted. + require.Equal(t, "system_io_foo_per_bar_total", BuildCompliantMetricName(createCounter("system.io", "foo/bar"), "", true)) + require.Equal(t, "metric_with_foreign_characters_total", BuildCompliantMetricName(createCounter("metric_with_字符_foreign_characters", ""), "", true)) + // Removes non aplhanumerical characters from units, but leaves colons. + require.Equal(t, "temperature_:C", BuildCompliantMetricName(createGauge("temperature", "%*()°:C"), "", true)) +} + +func TestBuildCompliantMetricNameWithoutSuffixes(t *testing.T) { + require.Equal(t, "system_io", BuildCompliantMetricName(createCounter("system.io", "By"), "", false)) + require.Equal(t, "system_network_io", BuildCompliantMetricName(createCounter("network.io", "By"), "system", false)) + require.Equal(t, "system_network_I_O", BuildCompliantMetricName(createCounter("network (I/O)", "By"), "system", false)) + require.Equal(t, "_3_14_digits", BuildCompliantMetricName(createGauge("3.14 digits", "By"), "", false)) + require.Equal(t, "envoy__rule_engine_zlib_buf_error", BuildCompliantMetricName(createGauge("envoy__rule_engine_zlib_buf_error", ""), "", false)) + require.Equal(t, ":foo::bar", BuildCompliantMetricName(createGauge(":foo::bar", ""), "", false)) + require.Equal(t, ":foo::bar", BuildCompliantMetricName(createCounter(":foo::bar", ""), "", false)) + require.Equal(t, "foo_bar", BuildCompliantMetricName(createGauge("foo.bar", "1"), "", false)) + require.Equal(t, "system_io", BuildCompliantMetricName(createCounter("system.io", "foo/bar"), "", false)) + require.Equal(t, "metric_with___foreign_characters", BuildCompliantMetricName(createCounter("metric_with_字符_foreign_characters", ""), "", false)) +} + +func TestBuildMetricNameWithSuffixes(t *testing.T) { + require.Equal(t, "system.io_bytes_total", BuildMetricName(createCounter("system.io", "By"), "", true)) + require.Equal(t, "system_network.io_bytes_total", BuildMetricName(createCounter("network.io", "By"), "system", true)) + require.Equal(t, "3.14 digits", BuildMetricName(createGauge("3.14 digits", ""), "", true)) + require.Equal(t, "envoy__rule_engine_zlib_buf_error", BuildMetricName(createGauge("envoy__rule_engine_zlib_buf_error", ""), "", true)) + require.Equal(t, ":foo::bar", BuildMetricName(createGauge(":foo::bar", ""), "", true)) + require.Equal(t, ":foo::bar_total", BuildMetricName(createCounter(":foo::bar", ""), "", true)) + // Gauges with unit 1 are considered ratios. + require.Equal(t, "foo.bar_ratio", BuildMetricName(createGauge("foo.bar", "1"), "", true)) + // Slashes in units are converted. + require.Equal(t, "system.io_foo_per_bar_total", BuildMetricName(createCounter("system.io", "foo/bar"), "", true)) + require.Equal(t, "metric_with_字符_foreign_characters_total", BuildMetricName(createCounter("metric_with_字符_foreign_characters", ""), "", true)) + require.Equal(t, "temperature_%*()°C", BuildMetricName(createGauge("temperature", "%*()°C"), "", true)) // Keeps the all characters in unit + // Tests below show weird interactions that users can have with the metric names. + // With BuildMetricName we don't check if units/type suffixes are already present in the metric name, we always add them. + require.Equal(t, "system_io_seconds_seconds", BuildMetricName(createGauge("system_io_seconds", "s"), "", true)) + require.Equal(t, "system_io_total_total", BuildMetricName(createCounter("system_io_total", ""), "", true)) +} + +func TestBuildMetricNameWithoutSuffixes(t *testing.T) { + require.Equal(t, "system.io", BuildMetricName(createCounter("system.io", "By"), "", false)) + require.Equal(t, "system_network.io", BuildMetricName(createCounter("network.io", "By"), "system", false)) + require.Equal(t, "3.14 digits", BuildMetricName(createGauge("3.14 digits", ""), "", false)) + require.Equal(t, "envoy__rule_engine_zlib_buf_error", BuildMetricName(createGauge("envoy__rule_engine_zlib_buf_error", ""), "", false)) + require.Equal(t, ":foo::bar", BuildMetricName(createGauge(":foo::bar", ""), "", false)) + require.Equal(t, ":foo::bar", BuildMetricName(createCounter(":foo::bar", ""), "", false)) + // Gauges with unit 1 are considered ratios. + require.Equal(t, "foo.bar", BuildMetricName(createGauge("foo.bar", "1"), "", false)) + require.Equal(t, "metric_with_字符_foreign_characters", BuildMetricName(createCounter("metric_with_字符_foreign_characters", ""), "", false)) + require.Equal(t, "system_io_seconds", BuildMetricName(createGauge("system_io_seconds", "s"), "", false)) + require.Equal(t, "system_io_total", BuildMetricName(createCounter("system_io_total", ""), "", false)) +} diff --git a/storage/remote/otlptranslator/prometheus/normalize_name_test.go b/storage/remote/otlptranslator/prometheus/normalize_name_test.go deleted file mode 100644 index 0473f6cbe1..0000000000 --- a/storage/remote/otlptranslator/prometheus/normalize_name_test.go +++ /dev/null @@ -1,210 +0,0 @@ -// Copyright 2024 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. -// Provenance-includes-location: https://github.com/open-telemetry/opentelemetry-collector-contrib/blob/95e8f8fdc2a9dc87230406c9a3cf02be4fd68bea/pkg/translator/prometheus/normalize_name_test.go -// Provenance-includes-license: Apache-2.0 -// Provenance-includes-copyright: Copyright The OpenTelemetry Authors. - -package prometheus - -import ( - "testing" - - "github.com/stretchr/testify/require" -) - -func TestByte(t *testing.T) { - require.Equal(t, "system_filesystem_usage_bytes", normalizeName(createGauge("system.filesystem.usage", "By"), "", false)) -} - -func TestByteCounter(t *testing.T) { - require.Equal(t, "system_io_bytes_total", normalizeName(createCounter("system.io", "By"), "", false)) - require.Equal(t, "network_transmitted_bytes_total", normalizeName(createCounter("network_transmitted_bytes_total", "By"), "", false)) -} - -func TestWhiteSpaces(t *testing.T) { - require.Equal(t, "system_filesystem_usage_bytes", normalizeName(createGauge("\t system.filesystem.usage ", " By\t"), "", false)) -} - -func TestNonStandardUnit(t *testing.T) { - require.Equal(t, "system_network_dropped", normalizeName(createGauge("system.network.dropped", "{packets}"), "", false)) - // The normal metric name character set is allowed in non-standard units. - require.Equal(t, "system_network_dropped_nonstandard:_1", normalizeName(createGauge("system.network.dropped", "nonstandard:_1"), "", false)) -} - -func TestNonStandardUnitCounter(t *testing.T) { - require.Equal(t, "system_network_dropped_total", normalizeName(createCounter("system.network.dropped", "{packets}"), "", false)) -} - -func TestBrokenUnit(t *testing.T) { - require.Equal(t, "system_network_dropped_packets", normalizeName(createGauge("system.network.dropped", "packets"), "", false)) - require.Equal(t, "system_network_packets_dropped", normalizeName(createGauge("system.network.packets.dropped", "packets"), "", false)) - require.Equal(t, "system_network_packets", normalizeName(createGauge("system.network.packets", "packets"), "", false)) -} - -func TestBrokenUnitCounter(t *testing.T) { - require.Equal(t, "system_network_dropped_packets_total", normalizeName(createCounter("system.network.dropped", "packets"), "", false)) - require.Equal(t, "system_network_packets_dropped_total", normalizeName(createCounter("system.network.packets.dropped", "packets"), "", false)) - require.Equal(t, "system_network_packets_total", normalizeName(createCounter("system.network.packets", "packets"), "", false)) -} - -func TestRatio(t *testing.T) { - require.Equal(t, "hw_gpu_memory_utilization_ratio", normalizeName(createGauge("hw.gpu.memory.utilization", "1"), "", false)) - require.Equal(t, "hw_fan_speed_ratio", normalizeName(createGauge("hw.fan.speed_ratio", "1"), "", false)) - require.Equal(t, "objects_total", normalizeName(createCounter("objects", "1"), "", false)) -} - -func TestHertz(t *testing.T) { - require.Equal(t, "hw_cpu_speed_limit_hertz", normalizeName(createGauge("hw.cpu.speed_limit", "Hz"), "", false)) -} - -func TestPer(t *testing.T) { - require.Equal(t, "broken_metric_speed_km_per_hour", normalizeName(createGauge("broken.metric.speed", "km/h"), "", false)) - require.Equal(t, "astro_light_speed_limit_meters_per_second", normalizeName(createGauge("astro.light.speed_limit", "m/s"), "", false)) - // The normal metric name character set is allowed in non-standard units. - require.Equal(t, "system_network_dropped_non_per_standard:_1", normalizeName(createGauge("system.network.dropped", "non/standard:_1"), "", false)) - - t.Run("invalid per unit", func(t *testing.T) { - require.Equal(t, "broken_metric_speed_km", normalizeName(createGauge("broken.metric.speed", "km/°"), "", false)) - }) -} - -func TestPercent(t *testing.T) { - require.Equal(t, "broken_metric_success_ratio_percent", normalizeName(createGauge("broken.metric.success_ratio", "%"), "", false)) - require.Equal(t, "broken_metric_success_percent", normalizeName(createGauge("broken.metric.success_percent", "%"), "", false)) -} - -func TestEmpty(t *testing.T) { - require.Equal(t, "test_metric_no_unit", normalizeName(createGauge("test.metric.no_unit", ""), "", false)) - require.Equal(t, "test_metric_spaces", normalizeName(createGauge("test.metric.spaces", " \t "), "", false)) -} - -func TestAllowUTF8(t *testing.T) { - t.Run("allow UTF8", func(t *testing.T) { - require.Equal(t, "unsupported.metric.temperature_°F", normalizeName(createGauge("unsupported.metric.temperature", "°F"), "", true)) - require.Equal(t, "unsupported.metric.weird_+=.:,!* & #", normalizeName(createGauge("unsupported.metric.weird", "+=.:,!* & #"), "", true)) - require.Equal(t, "unsupported.metric.redundant___test $_per_°C", normalizeName(createGauge("unsupported.metric.redundant", "__test $/°C"), "", true)) - require.Equal(t, "metric_with_字符_foreign_characters_ど", normalizeName(createGauge("metric_with_字符_foreign_characters", "ど"), "", true)) - }) - t.Run("disallow UTF8", func(t *testing.T) { - require.Equal(t, "unsupported_metric_temperature_F", normalizeName(createGauge("unsupported.metric.temperature", "°F"), "", false)) - require.Equal(t, "unsupported_metric_weird", normalizeName(createGauge("unsupported.metric.weird", "+=.,!* & #"), "", false)) - require.Equal(t, "unsupported_metric_redundant_test_per_C", normalizeName(createGauge("unsupported.metric.redundant", "__test $/°C"), "", false)) - require.Equal(t, "metric_with_foreign_characters", normalizeName(createGauge("metric_with_字符_foreign_characters", "ど"), "", false)) - }) -} - -func TestAllowUTF8KnownBugs(t *testing.T) { - // Due to historical reasons, the translator code was copied from OpenTelemetry collector codebase. - // Over there, they tried to provide means to translate metric names following Prometheus conventions that are documented here: - // https://prometheus.io/docs/practices/naming/ - // - // Althogh not explicitly said, it was implied that words should be separated by a single underscore and the codebase was written - // with that in mind. - // - // Now that we're allowing OTel users to have their original names stored in prometheus without any transformation, we're facing problems - // where two (or more) UTF-8 characters are being used to separate words. - // TODO(arthursens): Fix it! - - // We're asserting on 'NotEqual', which proves the bug. - require.NotEqual(t, "metric....split_=+by_//utf8characters", normalizeName(createGauge("metric....split_=+by_//utf8characters", ""), "", true)) - // Here we're asserting on 'Equal', showing the current behavior. - require.Equal(t, "metric.split_by_utf8characters", normalizeName(createGauge("metric....split_=+by_//utf8characters", ""), "", true)) -} - -func TestOTelReceivers(t *testing.T) { - require.Equal(t, "active_directory_ds_replication_network_io_bytes_total", normalizeName(createCounter("active_directory.ds.replication.network.io", "By"), "", false)) - require.Equal(t, "active_directory_ds_replication_sync_object_pending_total", normalizeName(createCounter("active_directory.ds.replication.sync.object.pending", "{objects}"), "", false)) - require.Equal(t, "active_directory_ds_replication_object_rate_per_second", normalizeName(createGauge("active_directory.ds.replication.object.rate", "{objects}/s"), "", false)) - require.Equal(t, "active_directory_ds_name_cache_hit_rate_percent", normalizeName(createGauge("active_directory.ds.name_cache.hit_rate", "%"), "", false)) - require.Equal(t, "active_directory_ds_ldap_bind_last_successful_time_milliseconds", normalizeName(createGauge("active_directory.ds.ldap.bind.last_successful.time", "ms"), "", false)) - require.Equal(t, "apache_current_connections", normalizeName(createGauge("apache.current_connections", "connections"), "", false)) - require.Equal(t, "apache_workers_connections", normalizeName(createGauge("apache.workers", "connections"), "", false)) - require.Equal(t, "apache_requests_total", normalizeName(createCounter("apache.requests", "1"), "", false)) - require.Equal(t, "bigip_virtual_server_request_count_total", normalizeName(createCounter("bigip.virtual_server.request.count", "{requests}"), "", false)) - require.Equal(t, "system_cpu_utilization_ratio", normalizeName(createGauge("system.cpu.utilization", "1"), "", false)) - require.Equal(t, "system_disk_operation_time_seconds_total", normalizeName(createCounter("system.disk.operation_time", "s"), "", false)) - require.Equal(t, "system_cpu_load_average_15m_ratio", normalizeName(createGauge("system.cpu.load_average.15m", "1"), "", false)) - require.Equal(t, "memcached_operation_hit_ratio_percent", normalizeName(createGauge("memcached.operation_hit_ratio", "%"), "", false)) - require.Equal(t, "mongodbatlas_process_asserts_per_second", normalizeName(createGauge("mongodbatlas.process.asserts", "{assertions}/s"), "", false)) - require.Equal(t, "mongodbatlas_process_journaling_data_files_mebibytes", normalizeName(createGauge("mongodbatlas.process.journaling.data_files", "MiBy"), "", false)) - require.Equal(t, "mongodbatlas_process_network_io_bytes_per_second", normalizeName(createGauge("mongodbatlas.process.network.io", "By/s"), "", false)) - require.Equal(t, "mongodbatlas_process_oplog_rate_gibibytes_per_hour", normalizeName(createGauge("mongodbatlas.process.oplog.rate", "GiBy/h"), "", false)) - require.Equal(t, "mongodbatlas_process_db_query_targeting_scanned_per_returned", normalizeName(createGauge("mongodbatlas.process.db.query_targeting.scanned_per_returned", "{scanned}/{returned}"), "", false)) - require.Equal(t, "nginx_requests", normalizeName(createGauge("nginx.requests", "requests"), "", false)) - require.Equal(t, "nginx_connections_accepted", normalizeName(createGauge("nginx.connections_accepted", "connections"), "", false)) - require.Equal(t, "nsxt_node_memory_usage_kilobytes", normalizeName(createGauge("nsxt.node.memory.usage", "KBy"), "", false)) - require.Equal(t, "redis_latest_fork_microseconds", normalizeName(createGauge("redis.latest_fork", "us"), "", false)) -} - -func TestNamespace(t *testing.T) { - require.Equal(t, "space_test", normalizeName(createGauge("test", ""), "space", false)) - require.Equal(t, "space_test", normalizeName(createGauge("#test", ""), "space", false)) -} - -func TestCleanUpUnit(t *testing.T) { - require.Equal(t, "", cleanUpUnit("")) - require.Equal(t, "a_b", cleanUpUnit("a b")) - require.Equal(t, "hello_world", cleanUpUnit("hello, world")) - require.Equal(t, "hello_you_2", cleanUpUnit("hello you 2")) - require.Equal(t, "1000", cleanUpUnit("$1000")) - require.Equal(t, "", cleanUpUnit("*+$^=)")) -} - -func TestUnitMapGetOrDefault(t *testing.T) { - require.Equal(t, "", unitMapGetOrDefault("")) - require.Equal(t, "seconds", unitMapGetOrDefault("s")) - require.Equal(t, "invalid", unitMapGetOrDefault("invalid")) -} - -func TestPerUnitMapGetOrDefault(t *testing.T) { - require.Equal(t, "", perUnitMapGetOrDefault("")) - require.Equal(t, "second", perUnitMapGetOrDefault("s")) - require.Equal(t, "invalid", perUnitMapGetOrDefault("invalid")) -} - -func TestRemoveItem(t *testing.T) { - require.Equal(t, []string{}, removeItem([]string{}, "test")) - require.Equal(t, []string{}, removeItem([]string{}, "")) - require.Equal(t, []string{"a", "b", "c"}, removeItem([]string{"a", "b", "c"}, "d")) - require.Equal(t, []string{"a", "b", "c"}, removeItem([]string{"a", "b", "c"}, "")) - require.Equal(t, []string{"a", "b"}, removeItem([]string{"a", "b", "c"}, "c")) - require.Equal(t, []string{"a", "c"}, removeItem([]string{"a", "b", "c"}, "b")) - require.Equal(t, []string{"b", "c"}, removeItem([]string{"a", "b", "c"}, "a")) -} - -func TestBuildCompliantNameWithSuffixes(t *testing.T) { - require.Equal(t, "system_io_bytes_total", BuildCompliantName(createCounter("system.io", "By"), "", true, false)) - require.Equal(t, "system_network_io_bytes_total", BuildCompliantName(createCounter("network.io", "By"), "system", true, false)) - require.Equal(t, "_3_14_digits", BuildCompliantName(createGauge("3.14 digits", ""), "", true, false)) - require.Equal(t, "envoy_rule_engine_zlib_buf_error", BuildCompliantName(createGauge("envoy__rule_engine_zlib_buf_error", ""), "", true, false)) - require.Equal(t, ":foo::bar", BuildCompliantName(createGauge(":foo::bar", ""), "", true, false)) - require.Equal(t, ":foo::bar_total", BuildCompliantName(createCounter(":foo::bar", ""), "", true, false)) - // Gauges with unit 1 are considered ratios. - require.Equal(t, "foo_bar_ratio", BuildCompliantName(createGauge("foo.bar", "1"), "", true, false)) - // Slashes in units are converted. - require.Equal(t, "system_io_foo_per_bar_total", BuildCompliantName(createCounter("system.io", "foo/bar"), "", true, false)) - require.Equal(t, "metric_with_foreign_characters_total", BuildCompliantName(createCounter("metric_with_字符_foreign_characters", ""), "", true, false)) -} - -func TestBuildCompliantNameWithoutSuffixes(t *testing.T) { - require.Equal(t, "system_io", BuildCompliantName(createCounter("system.io", "By"), "", false, false)) - require.Equal(t, "system_network_io", BuildCompliantName(createCounter("network.io", "By"), "system", false, false)) - require.Equal(t, "system_network_I_O", BuildCompliantName(createCounter("network (I/O)", "By"), "system", false, false)) - require.Equal(t, "_3_14_digits", BuildCompliantName(createGauge("3.14 digits", "By"), "", false, false)) - require.Equal(t, "envoy__rule_engine_zlib_buf_error", BuildCompliantName(createGauge("envoy__rule_engine_zlib_buf_error", ""), "", false, false)) - require.Equal(t, ":foo::bar", BuildCompliantName(createGauge(":foo::bar", ""), "", false, false)) - require.Equal(t, ":foo::bar", BuildCompliantName(createCounter(":foo::bar", ""), "", false, false)) - require.Equal(t, "foo_bar", BuildCompliantName(createGauge("foo.bar", "1"), "", false, false)) - require.Equal(t, "system_io", BuildCompliantName(createCounter("system.io", "foo/bar"), "", false, false)) - require.Equal(t, "metric_with___foreign_characters", BuildCompliantName(createCounter("metric_with_字符_foreign_characters", ""), "", false, false)) -} diff --git a/storage/remote/otlptranslator/prometheusremotewrite/histograms_test.go b/storage/remote/otlptranslator/prometheusremotewrite/histograms_test.go index dcd83b7f93..520d571b65 100644 --- a/storage/remote/otlptranslator/prometheusremotewrite/histograms_test.go +++ b/storage/remote/otlptranslator/prometheusremotewrite/histograms_test.go @@ -762,7 +762,7 @@ func TestPrometheusConverter_addExponentialHistogramDataPoints(t *testing.T) { Settings{ ExportCreatedMetric: true, }, - prometheustranslator.BuildCompliantName(metric, "", true, true), + prometheustranslator.BuildCompliantMetricName(metric, "", true), ) require.NoError(t, err) require.Empty(t, annots) diff --git a/storage/remote/otlptranslator/prometheusremotewrite/metrics_to_prw.go b/storage/remote/otlptranslator/prometheusremotewrite/metrics_to_prw.go index 6779c9ed80..1545accf2f 100644 --- a/storage/remote/otlptranslator/prometheusremotewrite/metrics_to_prw.go +++ b/storage/remote/otlptranslator/prometheusremotewrite/metrics_to_prw.go @@ -96,7 +96,12 @@ func (c *PrometheusConverter) FromMetrics(ctx context.Context, md pmetric.Metric continue } - promName := prometheustranslator.BuildCompliantName(metric, settings.Namespace, settings.AddMetricSuffixes, settings.AllowUTF8) + var promName string + if settings.AllowUTF8 { + promName = prometheustranslator.BuildMetricName(metric, settings.Namespace, settings.AddMetricSuffixes) + } else { + promName = prometheustranslator.BuildCompliantMetricName(metric, settings.Namespace, settings.AddMetricSuffixes) + } c.metadata = append(c.metadata, prompb.MetricMetadata{ Type: otelMetricTypeToPromMetricType(metric), MetricFamilyName: promName, diff --git a/storage/remote/otlptranslator/prometheusremotewrite/metrics_to_prw_test.go b/storage/remote/otlptranslator/prometheusremotewrite/metrics_to_prw_test.go index 05abc7743f..a3b4b08df4 100644 --- a/storage/remote/otlptranslator/prometheusremotewrite/metrics_to_prw_test.go +++ b/storage/remote/otlptranslator/prometheusremotewrite/metrics_to_prw_test.go @@ -46,7 +46,7 @@ func TestFromMetrics(t *testing.T) { metricSlice := scopeMetricsSlice.At(j).Metrics() for k := 0; k < metricSlice.Len(); k++ { metric := metricSlice.At(k) - promName := prometheustranslator.BuildCompliantName(metric, "", false, false) + promName := prometheustranslator.BuildCompliantMetricName(metric, "", false) expMetadata = append(expMetadata, prompb.MetricMetadata{ Type: otelMetricTypeToPromMetricType(metric), MetricFamilyName: promName, From 8067f2797198d7f43006c5ed5a3d60900d355cfa Mon Sep 17 00:00:00 2001 From: Julien Duchesne Date: Mon, 6 Jan 2025 13:51:19 -0500 Subject: [PATCH 33/40] `RuleConcurrencyController`: Add `SplitGroupIntoBatches` method (#15681) * `RuleConcurrencyController`: Add `SplitGroupIntoBatches` method The concurrency implementation can now return a slice of concurrent rule batches This allows for additional concurrency as opposed to the current interface which is limited by the order in which the rules have been loaded Also, I removed the `concurrencyController` attribute from the group. That information is duplicated in the opts.RuleConcurrencyController` attribute, leading to some confusing behavior, especially in tests. Signed-off-by: Julien Duchesne * Address PR comments Signed-off-by: Julien Duchesne * Apply suggestions from code review Co-authored-by: gotjosh Signed-off-by: Julien Duchesne --------- Signed-off-by: Julien Duchesne Signed-off-by: Julien Duchesne Co-authored-by: gotjosh --- rules/fixtures/rules_chain.yaml | 22 ++++ .../rules_multiple_dependents_on_base.yaml | 21 ++++ rules/fixtures/rules_multiple_groups.yaml | 4 + rules/group.go | 80 ++++++------ rules/manager.go | 59 +++++++-- rules/manager_test.go | 114 ++++++++++++++++++ 6 files changed, 253 insertions(+), 47 deletions(-) create mode 100644 rules/fixtures/rules_chain.yaml create mode 100644 rules/fixtures/rules_multiple_dependents_on_base.yaml diff --git a/rules/fixtures/rules_chain.yaml b/rules/fixtures/rules_chain.yaml new file mode 100644 index 0000000000..00043b8d6f --- /dev/null +++ b/rules/fixtures/rules_chain.yaml @@ -0,0 +1,22 @@ +groups: + - name: chain + rules: + # Evaluated concurrently, no dependencies + - record: job:http_requests:rate1m + expr: sum by (job)(rate(http_requests_total[1m])) + - record: job:http_requests:rate5m + expr: sum by (job)(rate(http_requests_total[1m])) + + # Evaluated sequentially, dependents and dependencies + - record: job1:http_requests:rate1m + expr: job:http_requests:rate1m{job="job1"} + - record: job1_cluster1:http_requests:rate1m + expr: job1:http_requests:rate1m{cluster="cluster1"} + + # Evaluated concurrently, no dependents + - record: job1_cluster2:http_requests:rate1m + expr: job1:http_requests:rate1m{cluster="cluster2"} + - record: job1_cluster1_namespace1:http_requests:rate1m + expr: job1_cluster1:http_requests:rate1m{namespace="namespace1"} + - record: job1_cluster1_namespace2:http_requests:rate1m + expr: job1_cluster1:http_requests:rate1m{namespace="namespace2"} diff --git a/rules/fixtures/rules_multiple_dependents_on_base.yaml b/rules/fixtures/rules_multiple_dependents_on_base.yaml new file mode 100644 index 0000000000..40ef14de8c --- /dev/null +++ b/rules/fixtures/rules_multiple_dependents_on_base.yaml @@ -0,0 +1,21 @@ +groups: + - name: concurrent_dependents + rules: + # 3 dependents on the same base + - record: job:http_requests:rate1m + expr: sum by (job)(rate(http_requests_total[1m])) + - record: job1:http_requests:rate1m + expr: job:http_requests:rate1m{job="job1"} + - record: job2:http_requests:rate1m + expr: job:http_requests:rate1m{job="job2"} + - record: job3:http_requests:rate1m + expr: job:http_requests:rate1m{job="job3"} + # another 3 dependents on the same base + - record: job:http_requests:rate5m + expr: sum by (job)(rate(http_requests_total[5m])) + - record: job1:http_requests:rate5m + expr: job:http_requests:rate5m{job="job1"} + - record: job2:http_requests:rate5m + expr: job:http_requests:rate5m{job="job2"} + - record: job3:http_requests:rate5m + expr: job:http_requests:rate5m{job="job3"} diff --git a/rules/fixtures/rules_multiple_groups.yaml b/rules/fixtures/rules_multiple_groups.yaml index 87f31a6ca5..592219e981 100644 --- a/rules/fixtures/rules_multiple_groups.yaml +++ b/rules/fixtures/rules_multiple_groups.yaml @@ -6,6 +6,8 @@ groups: expr: sum by (job)(rate(http_requests_total[1m])) - record: job:http_requests:rate5m expr: sum by (job)(rate(http_requests_total[5m])) + - record: job:http_requests:rate10m + expr: sum by (job)(rate(http_requests_total[10m])) # dependents - record: job:http_requests:rate15m @@ -20,6 +22,8 @@ groups: expr: sum by (job)(rate(grpc_requests_total[1m])) - record: job:grpc_requests:rate5m expr: sum by (job)(rate(grpc_requests_total[5m])) + - record: job:grpc_requests:rate10m + expr: sum by (job)(rate(grpc_requests_total[10m])) # dependents - record: job:grpc_requests:rate15m diff --git a/rules/group.go b/rules/group.go index ecc96d0a12..cabb45abbb 100644 --- a/rules/group.go +++ b/rules/group.go @@ -74,9 +74,7 @@ type Group struct { // defaults to DefaultEvalIterationFunc. evalIterationFunc GroupEvalIterationFunc - // concurrencyController controls the rules evaluation concurrency. - concurrencyController RuleConcurrencyController - appOpts *storage.AppendOptions + appOpts *storage.AppendOptions } // GroupEvalIterationFunc is used to implement and extend rule group @@ -126,33 +124,27 @@ func NewGroup(o GroupOptions) *Group { evalIterationFunc = DefaultEvalIterationFunc } - concurrencyController := opts.RuleConcurrencyController - if concurrencyController == nil { - concurrencyController = sequentialRuleEvalController{} - } - if opts.Logger == nil { opts.Logger = promslog.NewNopLogger() } return &Group{ - name: o.Name, - file: o.File, - interval: o.Interval, - queryOffset: o.QueryOffset, - limit: o.Limit, - rules: o.Rules, - shouldRestore: o.ShouldRestore, - opts: opts, - seriesInPreviousEval: make([]map[string]labels.Labels, len(o.Rules)), - done: make(chan struct{}), - managerDone: o.done, - terminated: make(chan struct{}), - logger: opts.Logger.With("file", o.File, "group", o.Name), - metrics: metrics, - evalIterationFunc: evalIterationFunc, - concurrencyController: concurrencyController, - appOpts: &storage.AppendOptions{DiscardOutOfOrder: true}, + name: o.Name, + file: o.File, + interval: o.Interval, + queryOffset: o.QueryOffset, + limit: o.Limit, + rules: o.Rules, + shouldRestore: o.ShouldRestore, + opts: opts, + seriesInPreviousEval: make([]map[string]labels.Labels, len(o.Rules)), + done: make(chan struct{}), + managerDone: o.done, + terminated: make(chan struct{}), + logger: opts.Logger.With("file", o.File, "group", o.Name), + metrics: metrics, + evalIterationFunc: evalIterationFunc, + appOpts: &storage.AppendOptions{DiscardOutOfOrder: true}, } } @@ -647,25 +639,33 @@ func (g *Group) Eval(ctx context.Context, ts time.Time) { } var wg sync.WaitGroup - for i, rule := range g.rules { - select { - case <-g.done: - return - default: - } + ctrl := g.opts.RuleConcurrencyController + if ctrl == nil { + ctrl = sequentialRuleEvalController{} + } + for _, batch := range ctrl.SplitGroupIntoBatches(ctx, g) { + for _, ruleIndex := range batch { + select { + case <-g.done: + return + default: + } - if ctrl := g.concurrencyController; ctrl.Allow(ctx, g, rule) { - wg.Add(1) + rule := g.rules[ruleIndex] + if len(batch) > 1 && ctrl.Allow(ctx, g, rule) { + wg.Add(1) - go eval(i, rule, func() { - wg.Done() - ctrl.Done(ctx) - }) - } else { - eval(i, rule, nil) + go eval(ruleIndex, rule, func() { + wg.Done() + ctrl.Done(ctx) + }) + } else { + eval(ruleIndex, rule, nil) + } } + // It is important that we finish processing any rules in this current batch - before we move into the next one. + wg.Wait() } - wg.Wait() g.metrics.GroupSamples.WithLabelValues(GroupKey(g.File(), g.Name())).Set(samplesTotal.Load()) g.cleanupStaleSeries(ctx, ts) diff --git a/rules/manager.go b/rules/manager.go index edc67a832b..390742ce50 100644 --- a/rules/manager.go +++ b/rules/manager.go @@ -465,10 +465,17 @@ func (c ruleDependencyController) AnalyseRules(rules []Rule) { } } +// ConcurrentRules represents a slice of indexes of rules that can be evaluated concurrently. +type ConcurrentRules []int + // RuleConcurrencyController controls concurrency for rules that are safe to be evaluated concurrently. // Its purpose is to bound the amount of concurrency in rule evaluations to avoid overwhelming the Prometheus // server with additional query load. Concurrency is controlled globally, not on a per-group basis. type RuleConcurrencyController interface { + // SplitGroupIntoBatches returns an ordered slice of of ConcurrentRules, which are batches of rules that can be evaluated concurrently. + // The rules are represented by their index from the input rule group. + SplitGroupIntoBatches(ctx context.Context, group *Group) []ConcurrentRules + // Allow determines if the given rule is allowed to be evaluated concurrently. // If Allow() returns true, then Done() must be called to release the acquired slot and corresponding cleanup is done. // It is important that both *Group and Rule are not retained and only be used for the duration of the call. @@ -490,21 +497,51 @@ func newRuleConcurrencyController(maxConcurrency int64) RuleConcurrencyControlle } func (c *concurrentRuleEvalController) Allow(_ context.Context, _ *Group, rule Rule) bool { - // To allow a rule to be executed concurrently, we need 3 conditions: - // 1. The rule must not have any rules that depend on it. - // 2. The rule itself must not depend on any other rules. - // 3. If 1 & 2 are true, then and only then we should try to acquire the concurrency slot. - if rule.NoDependentRules() && rule.NoDependencyRules() { - return c.sema.TryAcquire(1) + return c.sema.TryAcquire(1) +} + +func (c *concurrentRuleEvalController) SplitGroupIntoBatches(_ context.Context, g *Group) []ConcurrentRules { + // Using the rule dependency controller information (rules being identified as having no dependencies or no dependants), + // we can safely run the following concurrent groups: + // 1. Concurrently, all rules that have no dependencies + // 2. Sequentially, all rules that have both dependencies and dependants + // 3. Concurrently, all rules that have no dependants + + var noDependencies []int + var dependenciesAndDependants []int + var noDependants []int + + for i, r := range g.rules { + switch { + case r.NoDependencyRules(): + noDependencies = append(noDependencies, i) + case !r.NoDependentRules() && !r.NoDependencyRules(): + dependenciesAndDependants = append(dependenciesAndDependants, i) + case r.NoDependentRules(): + noDependants = append(noDependants, i) + } } - return false + var order []ConcurrentRules + if len(noDependencies) > 0 { + order = append(order, noDependencies) + } + for _, r := range dependenciesAndDependants { + order = append(order, []int{r}) + } + if len(noDependants) > 0 { + order = append(order, noDependants) + } + + return order } func (c *concurrentRuleEvalController) Done(_ context.Context) { c.sema.Release(1) } +var _ RuleConcurrencyController = &sequentialRuleEvalController{} + // sequentialRuleEvalController is a RuleConcurrencyController that runs every rule sequentially. type sequentialRuleEvalController struct{} @@ -512,6 +549,14 @@ func (c sequentialRuleEvalController) Allow(_ context.Context, _ *Group, _ Rule) return false } +func (c sequentialRuleEvalController) SplitGroupIntoBatches(_ context.Context, g *Group) []ConcurrentRules { + order := make([]ConcurrentRules, len(g.rules)) + for i := range g.rules { + order[i] = []int{i} + } + return order +} + func (c sequentialRuleEvalController) Done(_ context.Context) {} // FromMaps returns new sorted Labels from the given maps, overriding each other in order. diff --git a/rules/manager_test.go b/rules/manager_test.go index 94ee1e8b8b..df6f5fd1b4 100644 --- a/rules/manager_test.go +++ b/rules/manager_test.go @@ -1987,6 +1987,15 @@ func TestAsyncRuleEvaluation(t *testing.T) { start := time.Now() DefaultEvalIterationFunc(ctx, group, start) + // Expected evaluation order + order := group.opts.RuleConcurrencyController.SplitGroupIntoBatches(ctx, group) + require.Equal(t, []ConcurrentRules{ + {0}, + {1}, + {2}, + {3}, + }, order) + // Never expect more than 1 inflight query at a time. require.EqualValues(t, 1, maxInflight.Load()) // Each rule should take at least 1 second to execute sequentially. @@ -2065,6 +2074,12 @@ func TestAsyncRuleEvaluation(t *testing.T) { start := time.Now() DefaultEvalIterationFunc(ctx, group, start) + // Expected evaluation order (isn't affected by concurrency settings) + order := group.opts.RuleConcurrencyController.SplitGroupIntoBatches(ctx, group) + require.Equal(t, []ConcurrentRules{ + {0, 1, 2, 3, 4, 5}, + }, order) + // Max inflight can be 1 synchronous eval and up to MaxConcurrentEvals concurrent evals. require.EqualValues(t, opts.MaxConcurrentEvals+1, maxInflight.Load()) // Some rules should execute concurrently so should complete quicker. @@ -2104,6 +2119,12 @@ func TestAsyncRuleEvaluation(t *testing.T) { DefaultEvalIterationFunc(ctx, group, start) + // Expected evaluation order + order := group.opts.RuleConcurrencyController.SplitGroupIntoBatches(ctx, group) + require.Equal(t, []ConcurrentRules{ + {0, 1, 2, 3, 4, 5}, + }, order) + // Max inflight can be up to MaxConcurrentEvals concurrent evals, since there is sufficient concurrency to run all rules at once. require.LessOrEqual(t, int64(maxInflight.Load()), opts.MaxConcurrentEvals) // Some rules should execute concurrently so should complete quicker. @@ -2153,6 +2174,99 @@ func TestAsyncRuleEvaluation(t *testing.T) { require.EqualValues(t, ruleCount, testutil.ToFloat64(group.metrics.GroupSamples)) } }) + + t.Run("asynchronous evaluation of rules that benefit from reordering", func(t *testing.T) { + t.Parallel() + storage := teststorage.New(t) + t.Cleanup(func() { storage.Close() }) + inflightQueries := atomic.Int32{} + maxInflight := atomic.Int32{} + + ctx, cancel := context.WithCancel(context.Background()) + t.Cleanup(cancel) + + ruleCount := 8 + opts := optsFactory(storage, &maxInflight, &inflightQueries, 0) + + // Configure concurrency settings. + opts.ConcurrentEvalsEnabled = true + opts.MaxConcurrentEvals = int64(ruleCount) * 2 + opts.RuleConcurrencyController = nil + ruleManager := NewManager(opts) + + groups, errs := ruleManager.LoadGroups(time.Second, labels.EmptyLabels(), "", nil, []string{"fixtures/rules_multiple_dependents_on_base.yaml"}...) + require.Empty(t, errs) + require.Len(t, groups, 1) + var group *Group + for _, g := range groups { + group = g + } + + start := time.Now() + + // Expected evaluation order + order := group.opts.RuleConcurrencyController.SplitGroupIntoBatches(ctx, group) + require.Equal(t, []ConcurrentRules{ + {0, 4}, + {1, 2, 3, 5, 6, 7}, + }, order) + + group.Eval(ctx, start) + + // Inflight queries should be equal to 6. This is the size of the second batch of rules that can be executed concurrently. + require.EqualValues(t, 6, maxInflight.Load()) + // Some rules should execute concurrently so should complete quicker. + require.Less(t, time.Since(start).Seconds(), (time.Duration(ruleCount) * artificialDelay).Seconds()) + // Each rule produces one vector. + require.EqualValues(t, ruleCount, testutil.ToFloat64(group.metrics.GroupSamples)) + }) + + t.Run("attempted asynchronous evaluation of chained rules", func(t *testing.T) { + t.Parallel() + storage := teststorage.New(t) + t.Cleanup(func() { storage.Close() }) + inflightQueries := atomic.Int32{} + maxInflight := atomic.Int32{} + + ctx, cancel := context.WithCancel(context.Background()) + t.Cleanup(cancel) + + ruleCount := 7 + opts := optsFactory(storage, &maxInflight, &inflightQueries, 0) + + // Configure concurrency settings. + opts.ConcurrentEvalsEnabled = true + opts.MaxConcurrentEvals = int64(ruleCount) * 2 + opts.RuleConcurrencyController = nil + ruleManager := NewManager(opts) + + groups, errs := ruleManager.LoadGroups(time.Second, labels.EmptyLabels(), "", nil, []string{"fixtures/rules_chain.yaml"}...) + require.Empty(t, errs) + require.Len(t, groups, 1) + var group *Group + for _, g := range groups { + group = g + } + + start := time.Now() + + // Expected evaluation order + order := group.opts.RuleConcurrencyController.SplitGroupIntoBatches(ctx, group) + require.Equal(t, []ConcurrentRules{ + {0, 1}, + {2}, + {3}, + {4, 5, 6}, + }, order) + + group.Eval(ctx, start) + + require.EqualValues(t, 3, maxInflight.Load()) + // Some rules should execute concurrently so should complete quicker. + require.Less(t, time.Since(start).Seconds(), (time.Duration(ruleCount) * artificialDelay).Seconds()) + // Each rule produces one vector. + require.EqualValues(t, ruleCount, testutil.ToFloat64(group.metrics.GroupSamples)) + }) } func TestBoundedRuleEvalConcurrency(t *testing.T) { From 1a27ab29b8a33592e50daef4c59f6fc7b9a5b1f9 Mon Sep 17 00:00:00 2001 From: Julien Duchesne Date: Mon, 6 Jan 2025 15:48:38 -0500 Subject: [PATCH 34/40] Rules: Store dependencies instead of boolean (#15689) * Rules: Store dependencies instead of boolean To improve https://github.com/prometheus/prometheus/pull/15681 further, we'll need to store the dependencies and dependents of each Right now, if a rule has both (at least 1) dependents and dependencies, it is not possible to determine the order to run the rules and they must all run sequentially This PR only changes the dependents and dependencies attributes of rules, it does not implement a new topological sort algorithm Signed-off-by: Julien Duchesne * Store a slice of Rule instead Signed-off-by: Julien Duchesne * Add `BenchmarkRuleDependencyController_AnalyseRules` for future reference Signed-off-by: Julien Duchesne --------- Signed-off-by: Julien Duchesne --- rules/alerting.go | 53 ++++++++++++++++++++++++++++++++-------- rules/alerting_test.go | 20 ++++++++++----- rules/group.go | 26 +++++++++----------- rules/manager.go | 8 +++--- rules/manager_test.go | 35 +++++++++++++++++++++----- rules/origin_test.go | 14 ++++++----- rules/recording.go | 54 +++++++++++++++++++++++++++++++++-------- rules/recording_test.go | 20 ++++++++++----- rules/rule.go | 14 ++++++++--- 9 files changed, 178 insertions(+), 66 deletions(-) diff --git a/rules/alerting.go b/rules/alerting.go index e7f15baefe..77d53395e0 100644 --- a/rules/alerting.go +++ b/rules/alerting.go @@ -143,8 +143,9 @@ type AlertingRule struct { logger *slog.Logger - noDependentRules *atomic.Bool - noDependencyRules *atomic.Bool + dependenciesMutex sync.RWMutex + dependentRules []Rule + dependencyRules []Rule } // NewAlertingRule constructs a new AlertingRule. @@ -171,8 +172,6 @@ func NewAlertingRule( evaluationTimestamp: atomic.NewTime(time.Time{}), evaluationDuration: atomic.NewDuration(0), lastError: atomic.NewError(nil), - noDependentRules: atomic.NewBool(false), - noDependencyRules: atomic.NewBool(false), } } @@ -316,20 +315,54 @@ func (r *AlertingRule) Restored() bool { return r.restored.Load() } -func (r *AlertingRule) SetNoDependentRules(noDependentRules bool) { - r.noDependentRules.Store(noDependentRules) +func (r *AlertingRule) SetDependentRules(dependents []Rule) { + r.dependenciesMutex.Lock() + defer r.dependenciesMutex.Unlock() + + r.dependentRules = make([]Rule, len(dependents)) + copy(r.dependentRules, dependents) } func (r *AlertingRule) NoDependentRules() bool { - return r.noDependentRules.Load() + r.dependenciesMutex.RLock() + defer r.dependenciesMutex.RUnlock() + + if r.dependentRules == nil { + return false // We don't know if there are dependent rules. + } + + return len(r.dependentRules) == 0 +} + +func (r *AlertingRule) DependentRules() []Rule { + r.dependenciesMutex.RLock() + defer r.dependenciesMutex.RUnlock() + return r.dependentRules } -func (r *AlertingRule) SetNoDependencyRules(noDependencyRules bool) { - r.noDependencyRules.Store(noDependencyRules) +func (r *AlertingRule) SetDependencyRules(dependencies []Rule) { + r.dependenciesMutex.Lock() + defer r.dependenciesMutex.Unlock() + + r.dependencyRules = make([]Rule, len(dependencies)) + copy(r.dependencyRules, dependencies) } func (r *AlertingRule) NoDependencyRules() bool { - return r.noDependencyRules.Load() + r.dependenciesMutex.RLock() + defer r.dependenciesMutex.RUnlock() + + if r.dependencyRules == nil { + return false // We don't know if there are dependency rules. + } + + return len(r.dependencyRules) == 0 +} + +func (r *AlertingRule) DependencyRules() []Rule { + r.dependenciesMutex.RLock() + defer r.dependenciesMutex.RUnlock() + return r.dependencyRules } // resolvedRetention is the duration for which a resolved alert instance diff --git a/rules/alerting_test.go b/rules/alerting_test.go index f0aa339cc7..f7bdf4a955 100644 --- a/rules/alerting_test.go +++ b/rules/alerting_test.go @@ -998,7 +998,9 @@ func TestAlertingEvalWithOrigin(t *testing.T) { require.Equal(t, detail, NewRuleDetail(rule)) } -func TestAlertingRule_SetNoDependentRules(t *testing.T) { +func TestAlertingRule_SetDependentRules(t *testing.T) { + dependentRule := NewRecordingRule("test1", nil, labels.EmptyLabels()) + rule := NewAlertingRule( "test", &parser.NumberLiteral{Val: 1}, @@ -1012,14 +1014,18 @@ func TestAlertingRule_SetNoDependentRules(t *testing.T) { ) require.False(t, rule.NoDependentRules()) - rule.SetNoDependentRules(false) + rule.SetDependentRules([]Rule{dependentRule}) require.False(t, rule.NoDependentRules()) + require.Equal(t, []Rule{dependentRule}, rule.DependentRules()) - rule.SetNoDependentRules(true) + rule.SetDependentRules([]Rule{}) require.True(t, rule.NoDependentRules()) + require.Empty(t, rule.DependentRules()) } -func TestAlertingRule_SetNoDependencyRules(t *testing.T) { +func TestAlertingRule_SetDependencyRules(t *testing.T) { + dependencyRule := NewRecordingRule("test1", nil, labels.EmptyLabels()) + rule := NewAlertingRule( "test", &parser.NumberLiteral{Val: 1}, @@ -1033,11 +1039,13 @@ func TestAlertingRule_SetNoDependencyRules(t *testing.T) { ) require.False(t, rule.NoDependencyRules()) - rule.SetNoDependencyRules(false) + rule.SetDependencyRules([]Rule{dependencyRule}) require.False(t, rule.NoDependencyRules()) + require.Equal(t, []Rule{dependencyRule}, rule.DependencyRules()) - rule.SetNoDependencyRules(true) + rule.SetDependencyRules([]Rule{}) require.True(t, rule.NoDependencyRules()) + require.Empty(t, rule.DependencyRules()) } func TestAlertingRule_ActiveAlertsCount(t *testing.T) { diff --git a/rules/group.go b/rules/group.go index cabb45abbb..724b926d4f 100644 --- a/rules/group.go +++ b/rules/group.go @@ -1034,27 +1034,25 @@ func NewGroupMetrics(reg prometheus.Registerer) *Metrics { // output metric produced by another rule in its expression (i.e. as its "input"). type dependencyMap map[Rule][]Rule -// dependents returns the count of rules which use the output of the given rule as one of their inputs. -func (m dependencyMap) dependents(r Rule) int { - return len(m[r]) +// dependents returns the rules which use the output of the given rule as one of their inputs. +func (m dependencyMap) dependents(r Rule) []Rule { + return m[r] } -// dependencies returns the count of rules on which the given rule is dependent for input. -func (m dependencyMap) dependencies(r Rule) int { +// dependencies returns the rules on which the given rule is dependent for input. +func (m dependencyMap) dependencies(r Rule) []Rule { if len(m) == 0 { - return 0 + return []Rule{} } - var count int - for _, children := range m { - for _, child := range children { - if child == r { - count++ - } + var dependencies []Rule + for rule, dependents := range m { + if slices.Contains(dependents, r) { + dependencies = append(dependencies, rule) } } - return count + return dependencies } // isIndependent determines whether the given rule is not dependent on another rule for its input, nor is any other rule @@ -1064,7 +1062,7 @@ func (m dependencyMap) isIndependent(r Rule) bool { return false } - return m.dependents(r)+m.dependencies(r) == 0 + return len(m.dependents(r)) == 0 && len(m.dependencies(r)) == 0 } // buildDependencyMap builds a data-structure which contains the relationships between rules within a group. diff --git a/rules/manager.go b/rules/manager.go index 390742ce50..c4c0f8a1ef 100644 --- a/rules/manager.go +++ b/rules/manager.go @@ -444,8 +444,8 @@ func SendAlerts(s Sender, externalURL string) NotifyFunc { // RuleDependencyController controls whether a set of rules have dependencies between each other. type RuleDependencyController interface { // AnalyseRules analyses dependencies between the input rules. For each rule that it's guaranteed - // not having any dependants and/or dependency, this function should call Rule.SetNoDependentRules(true) - // and/or Rule.SetNoDependencyRules(true). + // not having any dependants and/or dependency, this function should call Rule.SetDependentRules(...) + // and/or Rule.SetDependencyRules(...). AnalyseRules(rules []Rule) } @@ -460,8 +460,8 @@ func (c ruleDependencyController) AnalyseRules(rules []Rule) { } for _, r := range rules { - r.SetNoDependentRules(depMap.dependents(r) == 0) - r.SetNoDependencyRules(depMap.dependencies(r) == 0) + r.SetDependentRules(depMap.dependents(r)) + r.SetDependencyRules(depMap.dependencies(r)) } } diff --git a/rules/manager_test.go b/rules/manager_test.go index df6f5fd1b4..defa93a68c 100644 --- a/rules/manager_test.go +++ b/rules/manager_test.go @@ -1423,8 +1423,6 @@ func TestRuleGroupEvalIterationFunc(t *testing.T) { evaluationTimestamp: atomic.NewTime(time.Time{}), evaluationDuration: atomic.NewDuration(0), lastError: atomic.NewError(nil), - noDependentRules: atomic.NewBool(false), - noDependencyRules: atomic.NewBool(false), } group := NewGroup(GroupOptions{ @@ -1613,11 +1611,12 @@ func TestDependencyMap(t *testing.T) { depMap := buildDependencyMap(group.rules) require.Zero(t, depMap.dependencies(rule)) - require.Equal(t, 2, depMap.dependents(rule)) + require.Equal(t, []Rule{rule2, rule4}, depMap.dependents(rule)) + require.Len(t, depMap.dependents(rule), 2) require.False(t, depMap.isIndependent(rule)) require.Zero(t, depMap.dependents(rule2)) - require.Equal(t, 1, depMap.dependencies(rule2)) + require.Equal(t, []Rule{rule}, depMap.dependencies(rule2)) require.False(t, depMap.isIndependent(rule2)) require.Zero(t, depMap.dependents(rule3)) @@ -1625,7 +1624,7 @@ func TestDependencyMap(t *testing.T) { require.True(t, depMap.isIndependent(rule3)) require.Zero(t, depMap.dependents(rule4)) - require.Equal(t, 1, depMap.dependencies(rule4)) + require.Equal(t, []Rule{rule}, depMap.dependencies(rule4)) require.False(t, depMap.isIndependent(rule4)) } @@ -1958,7 +1957,8 @@ func TestDependencyMapUpdatesOnGroupUpdate(t *testing.T) { require.NotEqual(t, orig[h], depMap) // We expect there to be some dependencies since the new rule group contains a dependency. require.NotEmpty(t, depMap) - require.Equal(t, 1, depMap.dependents(rr)) + require.Len(t, depMap.dependents(rr), 1) + require.Equal(t, "HighRequestRate", depMap.dependents(rr)[0].Name()) require.Zero(t, depMap.dependencies(rr)) } } @@ -2508,3 +2508,26 @@ func TestRuleDependencyController_AnalyseRules(t *testing.T) { }) } } + +func BenchmarkRuleDependencyController_AnalyseRules(b *testing.B) { + storage := teststorage.New(b) + b.Cleanup(func() { storage.Close() }) + + ruleManager := NewManager(&ManagerOptions{ + Context: context.Background(), + Logger: promslog.NewNopLogger(), + Appendable: storage, + QueryFunc: func(ctx context.Context, q string, ts time.Time) (promql.Vector, error) { return nil, nil }, + }) + + groups, errs := ruleManager.LoadGroups(time.Second, labels.EmptyLabels(), "", nil, "fixtures/rules_multiple.yaml") + require.Empty(b, errs) + require.Len(b, groups, 1) + + b.ResetTimer() + for i := 0; i < b.N; i++ { + for _, g := range groups { + ruleManager.opts.RuleDependencyController.AnalyseRules(g.rules) + } + } +} diff --git a/rules/origin_test.go b/rules/origin_test.go index 0bf428f3c1..b38f5d99b2 100644 --- a/rules/origin_test.go +++ b/rules/origin_test.go @@ -45,10 +45,12 @@ func (u unknownRule) SetEvaluationDuration(time.Duration) {} func (u unknownRule) GetEvaluationDuration() time.Duration { return 0 } func (u unknownRule) SetEvaluationTimestamp(time.Time) {} func (u unknownRule) GetEvaluationTimestamp() time.Time { return time.Time{} } -func (u unknownRule) SetNoDependentRules(bool) {} +func (u unknownRule) SetDependentRules([]Rule) {} func (u unknownRule) NoDependentRules() bool { return false } -func (u unknownRule) SetNoDependencyRules(bool) {} +func (u unknownRule) DependentRules() []Rule { return nil } +func (u unknownRule) SetDependencyRules([]Rule) {} func (u unknownRule) NoDependencyRules() bool { return false } +func (u unknownRule) DependencyRules() []Rule { return nil } func TestNewRuleDetailPanics(t *testing.T) { require.PanicsWithValue(t, `unknown rule type "rules.unknownRule"`, func() { @@ -76,12 +78,12 @@ func TestNewRuleDetail(t *testing.T) { require.False(t, detail.NoDependentRules) require.False(t, detail.NoDependencyRules) - rule.SetNoDependentRules(true) + rule.SetDependentRules([]Rule{}) detail = NewRuleDetail(rule) require.True(t, detail.NoDependentRules) require.False(t, detail.NoDependencyRules) - rule.SetNoDependencyRules(true) + rule.SetDependencyRules([]Rule{}) detail = NewRuleDetail(rule) require.True(t, detail.NoDependentRules) require.True(t, detail.NoDependencyRules) @@ -104,12 +106,12 @@ func TestNewRuleDetail(t *testing.T) { require.False(t, detail.NoDependentRules) require.False(t, detail.NoDependencyRules) - rule.SetNoDependentRules(true) + rule.SetDependentRules([]Rule{}) detail = NewRuleDetail(rule) require.True(t, detail.NoDependentRules) require.False(t, detail.NoDependencyRules) - rule.SetNoDependencyRules(true) + rule.SetDependencyRules([]Rule{}) detail = NewRuleDetail(rule) require.True(t, detail.NoDependentRules) require.True(t, detail.NoDependencyRules) diff --git a/rules/recording.go b/rules/recording.go index 52c2a875ab..3b6db210af 100644 --- a/rules/recording.go +++ b/rules/recording.go @@ -18,6 +18,7 @@ import ( "errors" "fmt" "net/url" + "sync" "time" "go.uber.org/atomic" @@ -43,8 +44,9 @@ type RecordingRule struct { // Duration of how long it took to evaluate the recording rule. evaluationDuration *atomic.Duration - noDependentRules *atomic.Bool - noDependencyRules *atomic.Bool + dependenciesMutex sync.RWMutex + dependentRules []Rule + dependencyRules []Rule } // NewRecordingRule returns a new recording rule. @@ -57,8 +59,6 @@ func NewRecordingRule(name string, vector parser.Expr, lset labels.Labels) *Reco evaluationTimestamp: atomic.NewTime(time.Time{}), evaluationDuration: atomic.NewDuration(0), lastError: atomic.NewError(nil), - noDependentRules: atomic.NewBool(false), - noDependencyRules: atomic.NewBool(false), } } @@ -172,18 +172,52 @@ func (rule *RecordingRule) GetEvaluationTimestamp() time.Time { return rule.evaluationTimestamp.Load() } -func (rule *RecordingRule) SetNoDependentRules(noDependentRules bool) { - rule.noDependentRules.Store(noDependentRules) +func (rule *RecordingRule) SetDependentRules(dependents []Rule) { + rule.dependenciesMutex.Lock() + defer rule.dependenciesMutex.Unlock() + + rule.dependentRules = make([]Rule, len(dependents)) + copy(rule.dependentRules, dependents) } func (rule *RecordingRule) NoDependentRules() bool { - return rule.noDependentRules.Load() + rule.dependenciesMutex.RLock() + defer rule.dependenciesMutex.RUnlock() + + if rule.dependentRules == nil { + return false // We don't know if there are dependent rules. + } + + return len(rule.dependentRules) == 0 +} + +func (rule *RecordingRule) DependentRules() []Rule { + rule.dependenciesMutex.RLock() + defer rule.dependenciesMutex.RUnlock() + return rule.dependentRules } -func (rule *RecordingRule) SetNoDependencyRules(noDependencyRules bool) { - rule.noDependencyRules.Store(noDependencyRules) +func (rule *RecordingRule) SetDependencyRules(dependencies []Rule) { + rule.dependenciesMutex.Lock() + defer rule.dependenciesMutex.Unlock() + + rule.dependencyRules = make([]Rule, len(dependencies)) + copy(rule.dependencyRules, dependencies) } func (rule *RecordingRule) NoDependencyRules() bool { - return rule.noDependencyRules.Load() + rule.dependenciesMutex.RLock() + defer rule.dependenciesMutex.RUnlock() + + if rule.dependencyRules == nil { + return false // We don't know if there are dependency rules. + } + + return len(rule.dependencyRules) == 0 +} + +func (rule *RecordingRule) DependencyRules() []Rule { + rule.dependenciesMutex.RLock() + defer rule.dependenciesMutex.RUnlock() + return rule.dependencyRules } diff --git a/rules/recording_test.go b/rules/recording_test.go index 72c0764f9b..3fbf11c435 100644 --- a/rules/recording_test.go +++ b/rules/recording_test.go @@ -255,24 +255,32 @@ func TestRecordingEvalWithOrigin(t *testing.T) { require.Equal(t, detail, NewRuleDetail(rule)) } -func TestRecordingRule_SetNoDependentRules(t *testing.T) { +func TestRecordingRule_SetDependentRules(t *testing.T) { + dependentRule := NewRecordingRule("test1", nil, labels.EmptyLabels()) + rule := NewRecordingRule("1", &parser.NumberLiteral{Val: 1}, labels.EmptyLabels()) require.False(t, rule.NoDependentRules()) - rule.SetNoDependentRules(false) + rule.SetDependentRules([]Rule{dependentRule}) require.False(t, rule.NoDependentRules()) + require.Equal(t, []Rule{dependentRule}, rule.DependentRules()) - rule.SetNoDependentRules(true) + rule.SetDependentRules([]Rule{}) require.True(t, rule.NoDependentRules()) + require.Empty(t, rule.DependentRules()) } -func TestRecordingRule_SetNoDependencyRules(t *testing.T) { +func TestRecordingRule_SetDependencyRules(t *testing.T) { + dependencyRule := NewRecordingRule("test1", nil, labels.EmptyLabels()) + rule := NewRecordingRule("1", &parser.NumberLiteral{Val: 1}, labels.EmptyLabels()) require.False(t, rule.NoDependencyRules()) - rule.SetNoDependencyRules(false) + rule.SetDependencyRules([]Rule{dependencyRule}) require.False(t, rule.NoDependencyRules()) + require.Equal(t, []Rule{dependencyRule}, rule.DependencyRules()) - rule.SetNoDependencyRules(true) + rule.SetDependencyRules([]Rule{}) require.True(t, rule.NoDependencyRules()) + require.Empty(t, rule.DependencyRules()) } diff --git a/rules/rule.go b/rules/rule.go index 687c03d000..33f1755ac5 100644 --- a/rules/rule.go +++ b/rules/rule.go @@ -62,19 +62,25 @@ type Rule interface { // NOTE: Used dynamically by rules.html template. GetEvaluationTimestamp() time.Time - // SetNoDependentRules sets whether there's no other rule in the rule group that depends on this rule. - SetNoDependentRules(bool) + // SetDependentRules sets rules which depend on the output of this rule. + SetDependentRules(rules []Rule) // NoDependentRules returns true if it's guaranteed that in the rule group there's no other rule // which depends on this one. In case this function returns false there's no such guarantee, which // means there may or may not be other rules depending on this one. NoDependentRules() bool - // SetNoDependencyRules sets whether this rule doesn't depend on the output of any rule in the rule group. - SetNoDependencyRules(bool) + // DependentRules returns the rules which depend on the output of this rule. + DependentRules() []Rule + + // SetDependencyRules sets rules on which this rule depends. + SetDependencyRules(rules []Rule) // NoDependencyRules returns true if it's guaranteed that this rule doesn't depend on the output of // any other rule in the group. In case this function returns false there's no such guarantee, which // means the rule may or may not depend on other rules. NoDependencyRules() bool + + // DependencyRules returns the rules on which this rule depends. + DependencyRules() []Rule } From 73a3438c1b6da19adcb099b62ae5376dee95b352 Mon Sep 17 00:00:00 2001 From: sujal shah Date: Sun, 5 Jan 2025 23:45:47 +0530 Subject: [PATCH 35/40] api: Add two new fields Node and ServerTime. This commit introduced two field in `/status` endpoint: - The node currently serving the request. - The current server time for debugging time drift issues. fixes #15394. Signed-off-by: sujal shah --- docs/querying/api.md | 2 ++ web/api/v1/api.go | 2 ++ web/ui/mantine-ui/src/pages/StatusPage.tsx | 6 ++++++ web/web.go | 7 +++++++ 4 files changed, 17 insertions(+) diff --git a/docs/querying/api.md b/docs/querying/api.md index 87de463288..f1e7129303 100644 --- a/docs/querying/api.md +++ b/docs/querying/api.md @@ -1158,6 +1158,8 @@ $ curl http://localhost:9090/api/v1/status/runtimeinfo "data": { "startTime": "2019-11-02T17:23:59.301361365+01:00", "CWD": "/", + "hostname" : "DESKTOP-717H17Q", + "serverTime": "2025-01-05T18:27:33Z", "reloadConfigSuccess": true, "lastConfigTime": "2019-11-02T17:23:59+01:00", "timeSeriesCount": 873, diff --git a/web/api/v1/api.go b/web/api/v1/api.go index 392dfc6aab..49112c7888 100644 --- a/web/api/v1/api.go +++ b/web/api/v1/api.go @@ -144,6 +144,8 @@ type PrometheusVersion struct { type RuntimeInfo struct { StartTime time.Time `json:"startTime"` CWD string `json:"CWD"` + Hostname string `json:"hostname"` + ServerTime time.Time `json:"serverTime"` ReloadConfigSuccess bool `json:"reloadConfigSuccess"` LastConfigTime time.Time `json:"lastConfigTime"` CorruptionCount int64 `json:"corruptionCount"` diff --git a/web/ui/mantine-ui/src/pages/StatusPage.tsx b/web/ui/mantine-ui/src/pages/StatusPage.tsx index 71dc476a2d..c968f1e866 100644 --- a/web/ui/mantine-ui/src/pages/StatusPage.tsx +++ b/web/ui/mantine-ui/src/pages/StatusPage.tsx @@ -29,6 +29,12 @@ export default function StatusPage() { formatTimestamp(new Date(v as string).valueOf() / 1000, useLocalTime), }, CWD: { title: "Working directory" }, + hostname: { title: "Hostname" }, + serverTime: { + title: "Server Time", + formatValue: (v: string | boolean) => + formatTimestamp(new Date(v as string).valueOf() / 1000, useLocalTime), + }, reloadConfigSuccess: { title: "Configuration reload", formatValue: (v: string | boolean) => (v ? "Successful" : "Unsuccessful"), diff --git a/web/web.go b/web/web.go index 08c683bae8..9ce66b7ff5 100644 --- a/web/web.go +++ b/web/web.go @@ -804,6 +804,13 @@ func (h *Handler) runtimeInfo() (api_v1.RuntimeInfo, error) { GODEBUG: os.Getenv("GODEBUG"), } + hostname, err := os.Hostname() + if err != nil { + return status, fmt.Errorf("Error getting hostname: %w", err) + } + status.Hostname = hostname + status.ServerTime = time.Now().UTC() + if h.options.TSDBRetentionDuration != 0 { status.StorageRetention = h.options.TSDBRetentionDuration.String() } From df55e536b801b8512ce425501791bf3b6727a4a5 Mon Sep 17 00:00:00 2001 From: beorn7 Date: Tue, 7 Jan 2025 17:51:57 +0100 Subject: [PATCH 36/40] docs: fix spelling Signed-off-by: beorn7 --- docs/http_sd.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/http_sd.md b/docs/http_sd.md index 884deb9f3c..aadc488738 100644 --- a/docs/http_sd.md +++ b/docs/http_sd.md @@ -8,7 +8,7 @@ sort_rank: 7 Prometheus provides a generic [HTTP Service Discovery](https://prometheus.io/docs/prometheus/latest/configuration/configuration/#http_sd_config), that enables it to discover targets over an HTTP endpoint. -The HTTP Service Discovery is complimentary to the supported service +The HTTP Service Discovery is complementary to the supported service discovery mechanisms, and is an alternative to [File-based Service Discovery](https://prometheus.io/docs/guides/file-sd/#use-file-based-service-discovery-to-discover-scrape-targets). ## Comparison between File-Based SD and HTTP SD From 919a5b657e1d6a23d6ba6c853ee63da5dbc352d2 Mon Sep 17 00:00:00 2001 From: Ben Ye Date: Tue, 7 Jan 2025 08:58:26 -0800 Subject: [PATCH 37/40] Expose ListPostings Length via Len() method (#15678) tsdb: expose remaining ListPostings Length Signed-off-by: Ben Ye --------- Signed-off-by: Ben Ye --- tsdb/index/postings.go | 5 +++++ tsdb/index/postings_test.go | 22 ++++++++++++++++++++++ 2 files changed, 27 insertions(+) diff --git a/tsdb/index/postings.go b/tsdb/index/postings.go index 03e3f7a239..e3ba5d64b4 100644 --- a/tsdb/index/postings.go +++ b/tsdb/index/postings.go @@ -863,6 +863,11 @@ func (it *ListPostings) Err() error { return nil } +// Len returns the remaining number of postings in the list. +func (it *ListPostings) Len() int { + return len(it.list) +} + // bigEndianPostings implements the Postings interface over a byte stream of // big endian numbers. type bigEndianPostings struct { diff --git a/tsdb/index/postings_test.go b/tsdb/index/postings_test.go index cf5ab6c0f8..feaba90e52 100644 --- a/tsdb/index/postings_test.go +++ b/tsdb/index/postings_test.go @@ -1244,63 +1244,78 @@ func TestPostingsWithIndexHeap(t *testing.T) { func TestListPostings(t *testing.T) { t.Run("empty list", func(t *testing.T) { p := NewListPostings(nil) + require.Equal(t, 0, p.(*ListPostings).Len()) require.False(t, p.Next()) require.False(t, p.Seek(10)) require.False(t, p.Next()) require.NoError(t, p.Err()) + require.Equal(t, 0, p.(*ListPostings).Len()) }) t.Run("one posting", func(t *testing.T) { t.Run("next", func(t *testing.T) { p := NewListPostings([]storage.SeriesRef{10}) + require.Equal(t, 1, p.(*ListPostings).Len()) require.True(t, p.Next()) require.Equal(t, storage.SeriesRef(10), p.At()) require.False(t, p.Next()) require.NoError(t, p.Err()) + require.Equal(t, 0, p.(*ListPostings).Len()) }) t.Run("seek less", func(t *testing.T) { p := NewListPostings([]storage.SeriesRef{10}) + require.Equal(t, 1, p.(*ListPostings).Len()) require.True(t, p.Seek(5)) require.Equal(t, storage.SeriesRef(10), p.At()) require.True(t, p.Seek(5)) require.Equal(t, storage.SeriesRef(10), p.At()) require.False(t, p.Next()) require.NoError(t, p.Err()) + require.Equal(t, 0, p.(*ListPostings).Len()) }) t.Run("seek equal", func(t *testing.T) { p := NewListPostings([]storage.SeriesRef{10}) + require.Equal(t, 1, p.(*ListPostings).Len()) require.True(t, p.Seek(10)) require.Equal(t, storage.SeriesRef(10), p.At()) require.False(t, p.Next()) require.NoError(t, p.Err()) + require.Equal(t, 0, p.(*ListPostings).Len()) }) t.Run("seek more", func(t *testing.T) { p := NewListPostings([]storage.SeriesRef{10}) + require.Equal(t, 1, p.(*ListPostings).Len()) require.False(t, p.Seek(15)) require.False(t, p.Next()) require.NoError(t, p.Err()) + require.Equal(t, 0, p.(*ListPostings).Len()) }) t.Run("seek after next", func(t *testing.T) { p := NewListPostings([]storage.SeriesRef{10}) + require.Equal(t, 1, p.(*ListPostings).Len()) require.True(t, p.Next()) require.False(t, p.Seek(15)) require.False(t, p.Next()) require.NoError(t, p.Err()) + require.Equal(t, 0, p.(*ListPostings).Len()) }) }) t.Run("multiple postings", func(t *testing.T) { t.Run("next", func(t *testing.T) { p := NewListPostings([]storage.SeriesRef{10, 20}) + require.Equal(t, 2, p.(*ListPostings).Len()) require.True(t, p.Next()) require.Equal(t, storage.SeriesRef(10), p.At()) require.True(t, p.Next()) require.Equal(t, storage.SeriesRef(20), p.At()) require.False(t, p.Next()) require.NoError(t, p.Err()) + require.Equal(t, 0, p.(*ListPostings).Len()) }) t.Run("seek", func(t *testing.T) { p := NewListPostings([]storage.SeriesRef{10, 20}) + require.Equal(t, 2, p.(*ListPostings).Len()) require.True(t, p.Seek(5)) require.Equal(t, storage.SeriesRef(10), p.At()) require.True(t, p.Seek(5)) @@ -1315,23 +1330,30 @@ func TestListPostings(t *testing.T) { require.Equal(t, storage.SeriesRef(20), p.At()) require.False(t, p.Next()) require.NoError(t, p.Err()) + require.Equal(t, 0, p.(*ListPostings).Len()) }) t.Run("seek lest than last", func(t *testing.T) { p := NewListPostings([]storage.SeriesRef{10, 20, 30, 40, 50}) + require.Equal(t, 5, p.(*ListPostings).Len()) require.True(t, p.Seek(45)) require.Equal(t, storage.SeriesRef(50), p.At()) require.False(t, p.Next()) + require.Equal(t, 0, p.(*ListPostings).Len()) }) t.Run("seek exactly last", func(t *testing.T) { p := NewListPostings([]storage.SeriesRef{10, 20, 30, 40, 50}) + require.Equal(t, 5, p.(*ListPostings).Len()) require.True(t, p.Seek(50)) require.Equal(t, storage.SeriesRef(50), p.At()) require.False(t, p.Next()) + require.Equal(t, 0, p.(*ListPostings).Len()) }) t.Run("seek more than last", func(t *testing.T) { p := NewListPostings([]storage.SeriesRef{10, 20, 30, 40, 50}) + require.Equal(t, 5, p.(*ListPostings).Len()) require.False(t, p.Seek(60)) require.False(t, p.Next()) + require.Equal(t, 0, p.(*ListPostings).Len()) }) }) From 7687661453fbf9ba3e9cfeb1f16e5c4ac66b8fae Mon Sep 17 00:00:00 2001 From: beorn7 Date: Wed, 8 Jan 2025 12:55:27 +0100 Subject: [PATCH 38/40] promqltest: make eval_ordered ignore annotations Besides making eval_ordered ignore annotations, this does the following: - Adds a test to verify that eval_ordered indeed ignores an info annotations now, while eval complains about it, eval_info recognizes it and, eval_warn flags the missing of the warn annotation. - Refactors the annotation check into its own method. - Moves closing of the query to the appropriate place where it wasn't so far. Signed-off-by: beorn7 --- promql/promqltest/test.go | 47 ++++++++++++++++++---------------- promql/promqltest/test_test.go | 38 +++++++++++++++++++++++++++ 2 files changed, 63 insertions(+), 22 deletions(-) diff --git a/promql/promqltest/test.go b/promql/promqltest/test.go index efa2136f10..518164827a 100644 --- a/promql/promqltest/test.go +++ b/promql/promqltest/test.go @@ -39,6 +39,7 @@ import ( "github.com/prometheus/prometheus/promql/parser/posrange" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/util/almost" + "github.com/prometheus/prometheus/util/annotations" "github.com/prometheus/prometheus/util/convertnhcb" "github.com/prometheus/prometheus/util/teststorage" "github.com/prometheus/prometheus/util/testutil" @@ -692,6 +693,24 @@ func (ev *evalCmd) expectMetric(pos int, m labels.Labels, vals ...parser.Sequenc ev.expected[h] = entry{pos: pos, vals: vals} } +// checkAnnotations asserts if the annotations match the expectations. +func (ev *evalCmd) checkAnnotations(expr string, annos annotations.Annotations) error { + countWarnings, countInfo := annos.CountWarningsAndInfo() + switch { + case ev.ordered: + // Ignore annotations if testing for order. + case !ev.warn && countWarnings > 0: + return fmt.Errorf("unexpected warnings evaluating query %q (line %d): %v", expr, ev.line, annos.AsErrors()) + case ev.warn && countWarnings == 0: + return fmt.Errorf("expected warnings evaluating query %q (line %d) but got none", expr, ev.line) + case !ev.info && countInfo > 0: + return fmt.Errorf("unexpected info annotations evaluating query %q (line %d): %v", expr, ev.line, annos.AsErrors()) + case ev.info && countInfo == 0: + return fmt.Errorf("expected info annotations evaluating query %q (line %d) but got none", expr, ev.line) + } + return nil +} + // compareResult compares the result value with the defined expectation. func (ev *evalCmd) compareResult(result parser.Value) error { switch val := result.(type) { @@ -1131,6 +1150,7 @@ func (t *test) execRangeEval(cmd *evalCmd, engine promql.QueryEngine) error { if err != nil { return fmt.Errorf("error creating range query for %q (line %d): %w", cmd.expr, cmd.line, err) } + defer q.Close() res := q.Exec(t.context) if res.Err != nil { if cmd.fail { @@ -1142,18 +1162,9 @@ func (t *test) execRangeEval(cmd *evalCmd, engine promql.QueryEngine) error { if res.Err == nil && cmd.fail { return fmt.Errorf("expected error evaluating query %q (line %d) but got none", cmd.expr, cmd.line) } - countWarnings, countInfo := res.Warnings.CountWarningsAndInfo() - switch { - case !cmd.warn && countWarnings > 0: - return fmt.Errorf("unexpected warnings evaluating query %q (line %d): %v", cmd.expr, cmd.line, res.Warnings) - case cmd.warn && countWarnings == 0: - return fmt.Errorf("expected warnings evaluating query %q (line %d) but got none", cmd.expr, cmd.line) - case !cmd.info && countInfo > 0: - return fmt.Errorf("unexpected info annotations evaluating query %q (line %d): %v", cmd.expr, cmd.line, res.Warnings) - case cmd.info && countInfo == 0: - return fmt.Errorf("expected info annotations evaluating query %q (line %d) but got none", cmd.expr, cmd.line) + if err := cmd.checkAnnotations(cmd.expr, res.Warnings); err != nil { + return err } - defer q.Close() if err := cmd.compareResult(res.Value); err != nil { return fmt.Errorf("error in %s %s (line %d): %w", cmd, cmd.expr, cmd.line, err) @@ -1196,16 +1207,8 @@ func (t *test) runInstantQuery(iq atModifierTestCase, cmd *evalCmd, engine promq if res.Err == nil && cmd.fail { return fmt.Errorf("expected error evaluating query %q (line %d) but got none", iq.expr, cmd.line) } - countWarnings, countInfo := res.Warnings.CountWarningsAndInfo() - switch { - case !cmd.warn && countWarnings > 0: - return fmt.Errorf("unexpected warnings evaluating query %q (line %d): %v", iq.expr, cmd.line, res.Warnings) - case cmd.warn && countWarnings == 0: - return fmt.Errorf("expected warnings evaluating query %q (line %d) but got none", iq.expr, cmd.line) - case !cmd.info && countInfo > 0: - return fmt.Errorf("unexpected info annotations evaluating query %q (line %d): %v", iq.expr, cmd.line, res.Warnings) - case cmd.info && countInfo == 0: - return fmt.Errorf("expected info annotations evaluating query %q (line %d) but got none", iq.expr, cmd.line) + if err := cmd.checkAnnotations(iq.expr, res.Warnings); err != nil { + return err } err = cmd.compareResult(res.Value) if err != nil { @@ -1218,11 +1221,11 @@ func (t *test) runInstantQuery(iq atModifierTestCase, cmd *evalCmd, engine promq if err != nil { return fmt.Errorf("error creating range query for %q (line %d): %w", cmd.expr, cmd.line, err) } + defer q.Close() rangeRes := q.Exec(t.context) if rangeRes.Err != nil { return fmt.Errorf("error evaluating query %q (line %d) in range mode: %w", iq.expr, cmd.line, rangeRes.Err) } - defer q.Close() if cmd.ordered { // Range queries are always sorted by labels, so skip this test case that expects results in a particular order. return nil diff --git a/promql/promqltest/test_test.go b/promql/promqltest/test_test.go index 327dcd78fe..96499e869d 100644 --- a/promql/promqltest/test_test.go +++ b/promql/promqltest/test_test.go @@ -353,6 +353,44 @@ eval_ordered instant at 50m sort(http_requests) `, expectedError: `error in eval sort(http_requests) (line 10): unexpected metric {__name__="http_requests", group="canary", instance="1", job="api-server"} in result, has value 400`, }, + "instant query with results expected to match provided order, result is in expected order and info annotation is ignored": { + input: testData + ` +eval_ordered instant at 50m sort(rate(http_requests[10m])) + {group="production", instance="0", job="api-server"} 0.03333333333333333 + {group="production", instance="1", job="api-server"} 0.06666666666666667 + {group="canary", instance="0", job="api-server"} 0.1 + {group="canary", instance="1", job="api-server"} 0.13333333333333333 +`, + }, + "instant query with expected info annotation": { + input: testData + ` +eval_info instant at 50m sort(rate(http_requests[10m])) + {group="production", instance="0", job="api-server"} 0.03333333333333333 + {group="production", instance="1", job="api-server"} 0.06666666666666667 + {group="canary", instance="0", job="api-server"} 0.1 + {group="canary", instance="1", job="api-server"} 0.13333333333333333 +`, + }, + "instant query with unexpected info annotation": { + input: testData + ` +eval instant at 50m sort(rate(http_requests[10m])) + {group="production", instance="0", job="api-server"} 0.03333333333333333 + {group="production", instance="1", job="api-server"} 0.06666666666666667 + {group="canary", instance="0", job="api-server"} 0.1 + {group="canary", instance="1", job="api-server"} 0.13333333333333333 +`, + expectedError: `unexpected info annotations evaluating query "sort(rate(http_requests[10m]))" (line 10): [PromQL info: metric might not be a counter, name does not end in _total/_sum/_count/_bucket: "http_requests"]`, + }, + "instant query with unexpectedly missing warn annotation": { + input: testData + ` +eval_warn instant at 50m sort(rate(http_requests[10m])) + {group="production", instance="0", job="api-server"} 0.03333333333333333 + {group="production", instance="1", job="api-server"} 0.06666666666666667 + {group="canary", instance="0", job="api-server"} 0.1 + {group="canary", instance="1", job="api-server"} 0.13333333333333333 +`, + expectedError: `expected warnings evaluating query "sort(rate(http_requests[10m]))" (line 10) but got none`, + }, "instant query with invalid timestamp": { input: `eval instant at abc123 vector(0)`, expectedError: `error in eval vector(0) (line 1): invalid timestamp definition "abc123": not a valid duration string: "abc123"`, From d9a80a91e3c7d8f98e72924b55bc6da3c4311b51 Mon Sep 17 00:00:00 2001 From: beorn7 Date: Wed, 8 Jan 2025 13:57:13 +0100 Subject: [PATCH 39/40] docs: Document eval_warn and eval_info MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This also improves the documentation in the following ways: - Clarifies that `eval` requires no annotations. - Clarifies that `eval_ordered` ignores annotations. - Clarifies that `eval_ordered` does not work with matrix returns (which could very well be created by instant queries). - Clarifies that there are more `eval` commands than just `eval`. - Improves wording for `eval_ordered`. - Replaces `...` by the typographical correct `…`. - Fixes a numerical error in an example. Signed-off-by: beorn7 --- promql/promqltest/README.md | 22 +++++++++++++++------- 1 file changed, 15 insertions(+), 7 deletions(-) diff --git a/promql/promqltest/README.md b/promql/promqltest/README.md index af34354241..25c2653ab3 100644 --- a/promql/promqltest/README.md +++ b/promql/promqltest/README.md @@ -22,7 +22,7 @@ Each test file contains a series of commands. There are three kinds of commands: * `load` * `clear` -* `eval` +* `eval` (including the variants `eval_fail`, `eval_warn`, `eval_info`, and `eval_ordered`) Each command is executed in the order given in the file. @@ -50,12 +50,12 @@ load 1m my_metric{env="prod"} 5 2+3x2 _ stale {{schema:1 sum:3 count:22 buckets:[5 10 7]}} ``` -...will create a single series with labels `my_metric{env="prod"}`, with the following points: +… will create a single series with labels `my_metric{env="prod"}`, with the following points: * t=0: value is 5 * t=1m: value is 2 * t=2m: value is 5 -* t=3m: value is 7 +* t=3m: value is 8 * t=4m: no point * t=5m: stale marker * t=6m: native histogram with schema 1, sum -3, count 22 and bucket counts 5, 10 and 7 @@ -74,6 +74,7 @@ When loading a batch of classic histogram float series, you can optionally appen ## `eval` command `eval` runs a query against the test environment and asserts that the result is as expected. +It requires the query to succeed without any (info or warn) annotations. Both instant and range queries are supported. @@ -110,11 +111,18 @@ eval range from 0 to 3m step 1m sum by (env) (my_metric) {env="test"} 10 20 30 45 ``` -Instant queries also support asserting that the series are returned in exactly the order specified: use `eval_ordered instant ...` instead of `eval instant ...`. -This is not supported for range queries. +To assert that a query succeeds with an info or warn annotation, use the +`eval_info` or `eval_warn` commands, respectively. -It is also possible to test that queries fail: use `eval_fail instant ...` or `eval_fail range ...`. -`eval_fail` optionally takes an expected error message string or regexp to assert that the error message is as expected. +Instant queries also support asserting that the series are returned in exactly +the order specified: use `eval_ordered instant ...` instead of `eval instant +...`. `eval_ordered` ignores any annotations. The assertion always fails for +matrix results. + +To assert that a query fails, use the `eval_fail` command. `eval_fail` does not +expect any result lines. Instead, it optionally accepts an expected error +message string or regular expression to assert that the error message is as +expected. For example: From a768a3b95e65efd0338cb2db598191ecd117f362 Mon Sep 17 00:00:00 2001 From: Julien Duchesne Date: Wed, 8 Jan 2025 11:32:48 -0500 Subject: [PATCH 40/40] Rule Concurrency: Test safe abort of rule evaluations (#15797) This test was added in the Grafana fork a while ago: https://github.com/grafana/mimir-prometheus/pull/714 and has been helpful to make sure we can safely terminate rule evaluations early The new rule evaluation logic (done here: https://github.com/prometheus/prometheus/pull/15681) does not have the bug, but the test was useful to verify that Signed-off-by: Julien Duchesne --- rules/manager_test.go | 35 +++++++++++++++++++++++++++++++++++ 1 file changed, 35 insertions(+) diff --git a/rules/manager_test.go b/rules/manager_test.go index defa93a68c..5c3fcc96a8 100644 --- a/rules/manager_test.go +++ b/rules/manager_test.go @@ -2326,6 +2326,41 @@ func TestUpdateWhenStopped(t *testing.T) { require.NoError(t, err) } +func TestGroup_Eval_RaceConditionOnStoppingGroupEvaluationWhileRulesAreEvaluatedConcurrently(t *testing.T) { + storage := teststorage.New(t) + t.Cleanup(func() { storage.Close() }) + + var ( + inflightQueries atomic.Int32 + maxInflight atomic.Int32 + maxConcurrency int64 = 10 + ) + + files := []string{"fixtures/rules_multiple_groups.yaml"} + files2 := []string{"fixtures/rules.yaml"} + + ruleManager := NewManager(optsFactory(storage, &maxInflight, &inflightQueries, maxConcurrency)) + go func() { + ruleManager.Run() + }() + <-ruleManager.block + + // Update the group a decent number of times to simulate start and stopping in the middle of an evaluation. + for i := 0; i < 10; i++ { + err := ruleManager.Update(time.Second, files, labels.EmptyLabels(), "", nil) + require.NoError(t, err) + + // Wait half of the query execution duration and then change the rule groups loaded by the manager + // so that the previous rule group will be interrupted while the query is executing. + time.Sleep(artificialDelay / 2) + + err = ruleManager.Update(time.Second, files2, labels.EmptyLabels(), "", nil) + require.NoError(t, err) + } + + ruleManager.Stop() +} + const artificialDelay = 250 * time.Millisecond func optsFactory(storage storage.Storage, maxInflight, inflightQueries *atomic.Int32, maxConcurrent int64) *ManagerOptions {