From aa144b7263564b51117e702a0419ac16322dd9cc Mon Sep 17 00:00:00 2001 From: Carrie Edwards Date: Wed, 16 Oct 2024 09:18:23 -0700 Subject: [PATCH 001/110] 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 002/110] 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 003/110] 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 004/110] 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 | 112 +++++++++---- 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, 660 insertions(+), 84 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) - for _, h := range histograms { - buf.PutVarint64(int64(h.Ref) - int64(first.Ref)) - buf.PutVarint64(h.T - first.T) + customBucketHistBuf.PutBE64(uint64(first.Ref)) + customBucketHistBuf.PutBE64int64(first.T) - EncodeHistogram(&buf, h.H) + histsAdded := 0 + customBucketHistsAdded := 0 + for _, h := range histograms { + 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++ + } } - return buf.Get() + if customBucketHistsAdded == 0 { + customBucketHistBuf.Reset() + } else if histsAdded == 0 { + buf.Reset() + } + + 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) - for _, h := range histograms { - buf.PutVarint64(int64(h.Ref) - int64(first.Ref)) - buf.PutVarint64(h.T - first.T) + customBucketHistBuf.PutBE64(uint64(first.Ref)) + customBucketHistBuf.PutBE64int64(first.T) - EncodeFloatHistogram(&buf, h.FH) + histsAdded := 0 + customBucketHistsAdded := 0 + for _, h := range histograms { + 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++ + } } - return buf.Get() + if customBucketHistsAdded == 0 { + customBucketHistBuf.Reset() + } else if histsAdded == 0 { + buf.Reset() + } + + 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 005/110] Add separate handling for histograms and custom bucket histograms --- tsdb/agent/db.go | 43 ++++++------- 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, 246 insertions(+), 139 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 { + var customBucketsExist bool + buf, customBucketsExist = 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(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 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 + } - EncodeHistogram(&customBucketHistBuf, h.H) - customBucketHistsAdded++ - } else { + 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) - 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 + } - EncodeFloatHistogram(&customBucketHistBuf, h.FH) - customBucketHistsAdded++ - } else { + 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) - 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)) - floatHistogram, floatCustomBucketHistogram := enc.FloatHistogramSamples([]record.RefFloatHistogramSample{{ + 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)) + + 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 006/110] 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 | 112 +++---------- 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, 302 insertions(+), 535 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,40 +744,10 @@ 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 - } - - // 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) - - 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() } @@ -789,12 +759,10 @@ func (e *Encoder) CustomBucketHistogramSamples(histograms []RefHistogramSample, 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) + buf.PutVarint64(int64(h.Ref) - int64(first.Ref)) + buf.PutVarint64(h.T - first.T) - EncodeHistogram(&buf, h.H) - } + EncodeHistogram(&buf, h.H) } return buf.Get() @@ -841,40 +809,10 @@ 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 - } - - // 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) - - 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() } @@ -886,12 +824,10 @@ func (e *Encoder) CustomBucketFloatHistogramSamples(histograms []RefFloatHistogr 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) + buf.PutVarint64(int64(h.Ref) - int64(first.Ref)) + buf.PutVarint64(h.T - first.T) - EncodeFloatHistogram(&buf, h.FH) - } + 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,28 +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: + case record.FloatHistogramSamples, record.FloatHistogramSamplesLegacy: floatHistogramSamples, err = dec.FloatHistogramSamples(rec, floatHistogramSamples) if err != nil { return nil, fmt.Errorf("decode float histogram samples: %w", err) @@ -255,24 +238,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) - } - 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.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 007/110] 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 008/110] 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 009/110] 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 7aa1f4571b09936e67997d25a610c2974dace810 Mon Sep 17 00:00:00 2001 From: machine424 Date: Thu, 5 Dec 2024 13:28:15 +0100 Subject: [PATCH 010/110] fix(remote): rename some remote read related metrics for better clarity. From the remote read handler side: prometheus_api_remote_read_queries -> prometheus_remote_read_handler_queries From the remote read client side: prometheus_remote_storage_read_queries_total -> prometheus_remote_read_client_queries_total prometheus_remote_storage_remote_read_queries -> prometheus_remote_read_client_queries prometheus_remote_storage_read_request_duration_seconds -> prometheus_remote_read_client_request_duration_seconds Signed-off-by: machine424 --- storage/remote/client.go | 12 ++++++------ storage/remote/read_handler.go | 8 ++++---- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/storage/remote/client.go b/storage/remote/client.go index ad766be9bf..2538ee90a0 100644 --- a/storage/remote/client.go +++ b/storage/remote/client.go @@ -81,8 +81,8 @@ var ( remoteReadQueriesTotal = prometheus.NewCounterVec( prometheus.CounterOpts{ Namespace: namespace, - Subsystem: subsystem, - Name: "read_queries_total", + Subsystem: "remote_read_client", + Name: "queries_total", Help: "The total number of remote read queries.", }, []string{remoteName, endpoint, "response_type", "code"}, @@ -90,8 +90,8 @@ var ( remoteReadQueries = prometheus.NewGaugeVec( prometheus.GaugeOpts{ Namespace: namespace, - Subsystem: subsystem, - Name: "remote_read_queries", + Subsystem: "remote_read_client", + Name: "queries", Help: "The number of in-flight remote read queries.", }, []string{remoteName, endpoint}, @@ -99,8 +99,8 @@ var ( remoteReadQueryDuration = prometheus.NewHistogramVec( prometheus.HistogramOpts{ Namespace: namespace, - Subsystem: subsystem, - Name: "read_request_duration_seconds", + Subsystem: "remote_read_client", + Name: "request_duration_seconds", Help: "Histogram of the latency for remote read requests. Note that for streamed responses this is only the duration of the initial call and does not include the processing of the stream.", Buckets: append(prometheus.DefBuckets, 25, 60), NativeHistogramBucketFactor: 1.1, diff --git a/storage/remote/read_handler.go b/storage/remote/read_handler.go index 8f2945f974..3e315a6157 100644 --- a/storage/remote/read_handler.go +++ b/storage/remote/read_handler.go @@ -56,10 +56,10 @@ func NewReadHandler(logger *slog.Logger, r prometheus.Registerer, queryable stor marshalPool: &sync.Pool{}, queries: prometheus.NewGauge(prometheus.GaugeOpts{ - Namespace: "prometheus", - Subsystem: "api", // TODO: changes to storage in Prometheus 3.0. - Name: "remote_read_queries", - Help: "The current number of remote read queries being executed or waiting.", + Namespace: namespace, + Subsystem: "remote_read_handler", + Name: "queries", + Help: "The current number of remote read queries that are either in execution or queued on the handler.", }), } if r != nil { From a046417bc0fd9f741e42d785392a19d8b86547c7 Mon Sep 17 00:00:00 2001 From: Carrie Edwards Date: Fri, 6 Dec 2024 13:46:20 -0800 Subject: [PATCH 011/110] 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,25 @@ 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) @@ -238,7 +252,24 @@ 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.CustomBucketsFloatHistogramSamples: + 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.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 012/110] 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 013/110] 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 014/110] 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 015/110] 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 016/110] 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 017/110] 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 018/110] 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 019/110] 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 020/110] 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 ceb2f653ba550f1d3dae4ec03f786391ff96769e Mon Sep 17 00:00:00 2001 From: Anand Rajagopal Date: Fri, 13 Dec 2024 01:19:40 +0000 Subject: [PATCH 021/110] Add an option to restore new rule groups added to existing rule manager Signed-off-by: Anand Rajagopal --- rules/fixtures/alert_rule.yaml | 6 ++ rules/fixtures/alert_rule1.yaml | 6 ++ rules/manager.go | 32 ++++---- rules/manager_test.go | 133 ++++++++++++++++++++++++++++++++ 4 files changed, 164 insertions(+), 13 deletions(-) create mode 100644 rules/fixtures/alert_rule.yaml create mode 100644 rules/fixtures/alert_rule1.yaml diff --git a/rules/fixtures/alert_rule.yaml b/rules/fixtures/alert_rule.yaml new file mode 100644 index 0000000000..0b6d69dafe --- /dev/null +++ b/rules/fixtures/alert_rule.yaml @@ -0,0 +1,6 @@ +groups: + - name: test + interval: 1s + rules: + - alert: rule1 + expr: 1 < bool 2 diff --git a/rules/fixtures/alert_rule1.yaml b/rules/fixtures/alert_rule1.yaml new file mode 100644 index 0000000000..306ff41b6e --- /dev/null +++ b/rules/fixtures/alert_rule1.yaml @@ -0,0 +1,6 @@ +groups: + - name: test2 + interval: 1s + rules: + - alert: rule2 + expr: 1 < bool 2 diff --git a/rules/manager.go b/rules/manager.go index 6e9bf64691..9bdb66f1b0 100644 --- a/rules/manager.go +++ b/rules/manager.go @@ -89,12 +89,13 @@ func DefaultEvalIterationFunc(ctx context.Context, g *Group, evalTimestamp time. // The Manager manages recording and alerting rules. type Manager struct { - opts *ManagerOptions - groups map[string]*Group - mtx sync.RWMutex - block chan struct{} - done chan struct{} - restored bool + opts *ManagerOptions + groups map[string]*Group + mtx sync.RWMutex + block chan struct{} + done chan struct{} + restored bool + restoreNewRuleGroups bool logger *slog.Logger } @@ -121,6 +122,10 @@ type ManagerOptions struct { ConcurrentEvalsEnabled bool RuleConcurrencyController RuleConcurrencyController RuleDependencyController RuleDependencyController + // At present, manager only restores `for` state when manager is newly created which happens + // during restarts. This flag provides an option to restore the `for` state when new rule groups are + // added to an existing manager + RestoreNewRuleGroups bool Metrics *Metrics } @@ -153,11 +158,12 @@ func NewManager(o *ManagerOptions) *Manager { } m := &Manager{ - groups: map[string]*Group{}, - opts: o, - block: make(chan struct{}), - done: make(chan struct{}), - logger: o.Logger, + groups: map[string]*Group{}, + opts: o, + block: make(chan struct{}), + done: make(chan struct{}), + logger: o.Logger, + restoreNewRuleGroups: o.RestoreNewRuleGroups, } return m @@ -295,7 +301,7 @@ func (m *Manager) LoadGroups( ) (map[string]*Group, []error) { groups := make(map[string]*Group) - shouldRestore := !m.restored + shouldRestore := !m.restored || m.restoreNewRuleGroups for _, fn := range filenames { rgs, errs := m.opts.GroupLoader.Load(fn) @@ -328,7 +334,7 @@ func (m *Manager) LoadGroups( labels.FromMap(r.Annotations), externalLabels, externalURL, - m.restored, + !shouldRestore, m.logger.With("alert", r.Alert), )) continue diff --git a/rules/manager_test.go b/rules/manager_test.go index 6afac993d8..bbc0a6023e 100644 --- a/rules/manager_test.go +++ b/rules/manager_test.go @@ -2112,6 +2112,139 @@ func TestAsyncRuleEvaluation(t *testing.T) { }) } +func TestNewRuleGroupRestoration(t *testing.T) { + store := teststorage.New(t) + t.Cleanup(func() { store.Close() }) + var ( + inflightQueries atomic.Int32 + maxInflight atomic.Int32 + maxConcurrency int64 + interval = 60 * time.Second + ) + + waitForEvaluations := func(t *testing.T, ch <-chan int32, targetCount int32) { + for { + select { + case cnt := <-ch: + if cnt == targetCount { + return + } + case <-time.After(5 * time.Second): + return + } + } + } + + files := []string{"fixtures/alert_rule.yaml"} + + option := optsFactory(store, &maxInflight, &inflightQueries, maxConcurrency) + option.Queryable = store + option.Appendable = store + option.NotifyFunc = func(ctx context.Context, expr string, alerts ...*Alert) {} + + var evalCount atomic.Int32 + ch := make(chan int32) + noopEvalIterFunc := func(ctx context.Context, g *Group, evalTimestamp time.Time) { + evalCount.Inc() + ch <- evalCount.Load() + } + + ruleManager := NewManager(option) + go ruleManager.Run() + err := ruleManager.Update(interval, files, labels.EmptyLabels(), "", noopEvalIterFunc) + require.NoError(t, err) + + waitForEvaluations(t, ch, 3) + require.Equal(t, int32(3), evalCount.Load()) + ruleGroups := make(map[string]struct{}) + for _, group := range ruleManager.groups { + ruleGroups[group.Name()] = struct{}{} + require.False(t, group.shouldRestore) + for _, rule := range group.rules { + require.True(t, rule.(*AlertingRule).restored.Load()) + } + } + + files = append(files, "fixtures/alert_rule1.yaml") + err = ruleManager.Update(interval, files, labels.EmptyLabels(), "", nil) + require.NoError(t, err) + ruleManager.Stop() + for _, group := range ruleManager.groups { + // new rule groups added to existing manager will not be restored + require.False(t, group.shouldRestore) + } +} + +func TestNewRuleGroupRestorationWithRestoreNewGroupOption(t *testing.T) { + store := teststorage.New(t) + t.Cleanup(func() { store.Close() }) + var ( + inflightQueries atomic.Int32 + maxInflight atomic.Int32 + maxConcurrency int64 + interval = 60 * time.Second + ) + + waitForEvaluations := func(t *testing.T, ch <-chan int32, targetCount int32) { + for { + select { + case cnt := <-ch: + if cnt == targetCount { + return + } + case <-time.After(5 * time.Second): + return + } + } + } + + files := []string{"fixtures/alert_rule.yaml"} + + option := optsFactory(store, &maxInflight, &inflightQueries, maxConcurrency) + option.Queryable = store + option.Appendable = store + option.RestoreNewRuleGroups = true + option.NotifyFunc = func(ctx context.Context, expr string, alerts ...*Alert) {} + + var evalCount atomic.Int32 + ch := make(chan int32) + noopEvalIterFunc := func(ctx context.Context, g *Group, evalTimestamp time.Time) { + evalCount.Inc() + ch <- evalCount.Load() + } + + ruleManager := NewManager(option) + go ruleManager.Run() + err := ruleManager.Update(interval, files, labels.EmptyLabels(), "", noopEvalIterFunc) + require.NoError(t, err) + + waitForEvaluations(t, ch, 3) + require.Equal(t, int32(3), evalCount.Load()) + ruleGroups := make(map[string]struct{}) + for _, group := range ruleManager.groups { + ruleGroups[group.Name()] = struct{}{} + require.False(t, group.shouldRestore) + for _, rule := range group.rules { + require.True(t, rule.(*AlertingRule).restored.Load()) + } + } + + files = append(files, "fixtures/alert_rule1.yaml") + err = ruleManager.Update(interval, files, labels.EmptyLabels(), "", nil) + require.NoError(t, err) + // stop eval + ruleManager.Stop() + for _, group := range ruleManager.groups { + if _, OK := ruleGroups[group.Name()]; OK { + // already restored + require.False(t, group.shouldRestore) + continue + } + // new rule groups added to existing manager will be restored + require.True(t, group.shouldRestore) + } +} + func TestBoundedRuleEvalConcurrency(t *testing.T) { storage := teststorage.New(t) t.Cleanup(func() { storage.Close() }) From afde4707c52ccce477c5dd10cdd2af05c641369c Mon Sep 17 00:00:00 2001 From: Jan Horstmann Date: Mon, 4 Nov 2024 09:06:52 +0100 Subject: [PATCH 022/110] Update mixin dashboard Update and rewrite the mixin dashboards to use the grafonnet ([1]) library. Grafana has deprecated angular plugins ([2]) as used by grafonnet-lib ([3]) with removal pending for grafana version 12. Additionally grafonnet-lib is deprecated/unmaintained in favor of grafonnet. Therefore the mixin dashboards have been updated to use grafonnet. Closes: https://github.com/prometheus/prometheus/issues/14404 [1] https://github.com/grafana/grafonnet [2] https://grafana.com/docs/grafana/latest/developers/angular_deprecation/ [3] https://github.com/grafana/grafonnet-lib Signed-off-by: Jan Horstmann --- .../prometheus-mixin/dashboards.libsonnet | 1185 +++++++++++------ .../prometheus-mixin/jsonnetfile.json | 15 +- 2 files changed, 789 insertions(+), 411 deletions(-) diff --git a/documentation/prometheus-mixin/dashboards.libsonnet b/documentation/prometheus-mixin/dashboards.libsonnet index 2bdd168cc9..22b8c92e6e 100644 --- a/documentation/prometheus-mixin/dashboards.libsonnet +++ b/documentation/prometheus-mixin/dashboards.libsonnet @@ -1,438 +1,825 @@ -local grafana = import 'github.com/grafana/grafonnet-lib/grafonnet/grafana.libsonnet'; -local g = import 'github.com/grafana/jsonnet-libs/grafana-builder/grafana.libsonnet'; +local grafana = import 'github.com/grafana/grafonnet/gen/grafonnet-latest/main.libsonnet'; local dashboard = grafana.dashboard; -local row = grafana.row; -local singlestat = grafana.singlestat; -local prometheus = grafana.prometheus; -local graphPanel = grafana.graphPanel; -local tablePanel = grafana.tablePanel; -local template = grafana.template; +local prometheus = grafana.query.prometheus; +local variable = dashboard.variable; +local panel = grafana.panel; +local row = panel.row; + { grafanaDashboards+:: { + + local panelTimeSeriesStdOptions = + {} + + panel.timeSeries.queryOptions.withDatasource('prometheus', '$datasource') + + panel.timeSeries.fieldConfig.defaults.custom.withFillOpacity(10) + + panel.timeSeries.fieldConfig.defaults.custom.withShowPoints('never') + + panel.timeSeries.options.tooltip.withMode('multi') + , + + local panelTimeSeriesStacking = + {} + + panel.timeSeries.fieldConfig.defaults.custom.withFillOpacity(100) + + panel.timeSeries.fieldConfig.defaults.custom.withLineWidth(0) + + panel.timeSeries.fieldConfig.defaults.custom.stacking.withMode('normal') + , + 'prometheus.json': + local showMultiCluster = $._config.showMultiCluster; - local dashboard = g.dashboard( - '%(prefix)sOverview' % $._config.grafanaPrometheus - ); - local templatedDashboard = if showMultiCluster then - dashboard - .addMultiTemplate('cluster', 'prometheus_build_info{%(prometheusSelector)s}' % $._config, $._config.clusterLabel) - .addMultiTemplate('job', 'prometheus_build_info{cluster=~"$cluster"}', 'job') - .addMultiTemplate('instance', 'prometheus_build_info{cluster=~"$cluster", job=~"$job"}', 'instance') - else - dashboard - .addMultiTemplate('job', 'prometheus_build_info{%(prometheusSelector)s}' % $._config, 'job') - .addMultiTemplate('instance', 'prometheus_build_info{job=~"$job"}', 'instance'); - templatedDashboard - .addRow( - g.row('Prometheus Stats') - .addPanel( - g.panel('Prometheus Stats') + - g.tablePanel(if showMultiCluster then [ - 'count by (cluster, job, instance, version) (prometheus_build_info{cluster=~"$cluster", job=~"$job", instance=~"$instance"})', - 'max by (cluster, job, instance) (time() - process_start_time_seconds{cluster=~"$cluster", job=~"$job", instance=~"$instance"})', - ] else [ - 'count by (job, instance, version) (prometheus_build_info{job=~"$job", instance=~"$instance"})', - 'max by (job, instance) (time() - process_start_time_seconds{job=~"$job", instance=~"$instance"})', - ], { - cluster: { alias: if showMultiCluster then 'Cluster' else '' }, - job: { alias: 'Job' }, - instance: { alias: 'Instance' }, - version: { alias: 'Version' }, - 'Value #A': { alias: 'Count', type: 'hidden' }, - 'Value #B': { alias: 'Uptime', type: 'number', unit: 's' }, - }) - ) - ) - .addRow( - g.row('Discovery') - .addPanel( - g.panel('Target Sync') + - g.queryPanel(if showMultiCluster then 'sum(rate(prometheus_target_sync_length_seconds_sum{cluster=~"$cluster",job=~"$job",instance=~"$instance"}[5m])) by (cluster, job, scrape_job, instance) * 1e3' - else 'sum(rate(prometheus_target_sync_length_seconds_sum{job=~"$job",instance=~"$instance"}[5m])) by (scrape_job) * 1e3', - if showMultiCluster then '{{cluster}}:{{job}}:{{instance}}:{{scrape_job}}' - else '{{scrape_job}}') + - { yaxes: g.yaxes('ms') } - ) - .addPanel( - g.panel('Targets') + - g.queryPanel(if showMultiCluster then 'sum by (cluster, job, instance) (prometheus_sd_discovered_targets{cluster=~"$cluster", job=~"$job",instance=~"$instance"})' - else 'sum(prometheus_sd_discovered_targets{job=~"$job",instance=~"$instance"})', - if showMultiCluster then '{{cluster}}:{{job}}:{{instance}}' - else 'Targets') + - g.stack - ) - ) - .addRow( - g.row('Retrieval') - .addPanel( - g.panel('Average Scrape Interval Duration') + - g.queryPanel(if showMultiCluster then 'rate(prometheus_target_interval_length_seconds_sum{cluster=~"$cluster", job=~"$job",instance=~"$instance"}[5m]) / rate(prometheus_target_interval_length_seconds_count{cluster=~"$cluster", job=~"$job",instance=~"$instance"}[5m]) * 1e3' - else 'rate(prometheus_target_interval_length_seconds_sum{job=~"$job",instance=~"$instance"}[5m]) / rate(prometheus_target_interval_length_seconds_count{job=~"$job",instance=~"$instance"}[5m]) * 1e3', - if showMultiCluster then '{{cluster}}:{{job}}:{{instance}} {{interval}} configured' - else '{{interval}} configured') + - { yaxes: g.yaxes('ms') } - ) - .addPanel( - g.panel('Scrape failures') + - g.queryPanel(if showMultiCluster then [ - 'sum by (cluster, job, instance) (rate(prometheus_target_scrapes_exceeded_body_size_limit_total{cluster=~"$cluster",job=~"$job",instance=~"$instance"}[1m]))', - 'sum by (cluster, job, instance) (rate(prometheus_target_scrapes_exceeded_sample_limit_total{cluster=~"$cluster",job=~"$job",instance=~"$instance"}[1m]))', - 'sum by (cluster, job, instance) (rate(prometheus_target_scrapes_sample_duplicate_timestamp_total{cluster=~"$cluster",job=~"$job",instance=~"$instance"}[1m]))', - 'sum by (cluster, job, instance) (rate(prometheus_target_scrapes_sample_out_of_bounds_total{cluster=~"$cluster",job=~"$job",instance=~"$instance"}[1m]))', - 'sum by (cluster, job, instance) (rate(prometheus_target_scrapes_sample_out_of_order_total{cluster=~"$cluster",job=~"$job",instance=~"$instance"}[1m]))', - ] else [ - 'sum by (job) (rate(prometheus_target_scrapes_exceeded_body_size_limit_total[1m]))', - 'sum by (job) (rate(prometheus_target_scrapes_exceeded_sample_limit_total[1m]))', - 'sum by (job) (rate(prometheus_target_scrapes_sample_duplicate_timestamp_total[1m]))', - 'sum by (job) (rate(prometheus_target_scrapes_sample_out_of_bounds_total[1m]))', - 'sum by (job) (rate(prometheus_target_scrapes_sample_out_of_order_total[1m]))', - ], if showMultiCluster then [ - 'exceeded body size limit: {{cluster}} {{job}} {{instance}}', - 'exceeded sample limit: {{cluster}} {{job}} {{instance}}', - 'duplicate timestamp: {{cluster}} {{job}} {{instance}}', - 'out of bounds: {{cluster}} {{job}} {{instance}}', - 'out of order: {{cluster}} {{job}} {{instance}}', - ] else [ - 'exceeded body size limit: {{job}}', - 'exceeded sample limit: {{job}}', - 'duplicate timestamp: {{job}}', - 'out of bounds: {{job}}', - 'out of order: {{job}}', - ]) + - g.stack - ) - .addPanel( - g.panel('Appended Samples') + - g.queryPanel(if showMultiCluster then 'rate(prometheus_tsdb_head_samples_appended_total{cluster=~"$cluster", job=~"$job",instance=~"$instance"}[5m])' - else 'rate(prometheus_tsdb_head_samples_appended_total{job=~"$job",instance=~"$instance"}[5m])', - if showMultiCluster then '{{cluster}} {{job}} {{instance}}' - else '{{job}} {{instance}}') + - g.stack - ) - ) - .addRow( - g.row('Storage') - .addPanel( - g.panel('Head Series') + - g.queryPanel(if showMultiCluster then 'prometheus_tsdb_head_series{cluster=~"$cluster",job=~"$job",instance=~"$instance"}' - else 'prometheus_tsdb_head_series{job=~"$job",instance=~"$instance"}', - if showMultiCluster then '{{cluster}} {{job}} {{instance}} head series' - else '{{job}} {{instance}} head series') + - g.stack - ) - .addPanel( - g.panel('Head Chunks') + - g.queryPanel(if showMultiCluster then 'prometheus_tsdb_head_chunks{cluster=~"$cluster",job=~"$job",instance=~"$instance"}' - else 'prometheus_tsdb_head_chunks{job=~"$job",instance=~"$instance"}', - if showMultiCluster then '{{cluster}} {{job}} {{instance}} head chunks' - else '{{job}} {{instance}} head chunks') + - g.stack - ) - ) - .addRow( - g.row('Query') - .addPanel( - g.panel('Query Rate') + - g.queryPanel(if showMultiCluster then 'rate(prometheus_engine_query_duration_seconds_count{cluster=~"$cluster",job=~"$job",instance=~"$instance",slice="inner_eval"}[5m])' - else 'rate(prometheus_engine_query_duration_seconds_count{job=~"$job",instance=~"$instance",slice="inner_eval"}[5m])', - if showMultiCluster then '{{cluster}} {{job}} {{instance}}' - else '{{job}} {{instance}}') + - g.stack, - ) - .addPanel( - g.panel('Stage Duration') + - g.queryPanel(if showMultiCluster then 'max by (slice) (prometheus_engine_query_duration_seconds{quantile="0.9",cluster=~"$cluster", job=~"$job",instance=~"$instance"}) * 1e3' - else 'max by (slice) (prometheus_engine_query_duration_seconds{quantile="0.9",job=~"$job",instance=~"$instance"}) * 1e3', - if showMultiCluster then '{{slice}}' - else '{{slice}}') + - { yaxes: g.yaxes('ms') } + - g.stack, - ) - ) + { - tags: $._config.grafanaPrometheus.tags, - refresh: $._config.grafanaPrometheus.refresh, - }, + + local datasourceVariable = + variable.datasource.new('datasource', 'prometheus') + + variable.datasource.generalOptions.withLabel('Data source') + + variable.datasource.generalOptions.withCurrent('default') + + variable.datasource.generalOptions.showOnDashboard.withLabelAndValue() + ; + + local clusterVariable = + variable.query.new('cluster') + + variable.query.generalOptions.withLabel('cluster') + + variable.query.withDatasourceFromVariable(datasourceVariable) + + variable.query.refresh.onTime() + + variable.query.withSort(type='alphabetical', asc=false) + + variable.query.selectionOptions.withIncludeAll(true, '.+') + + variable.query.selectionOptions.withMulti(true) + + variable.query.generalOptions.withCurrent('$__all') + + variable.query.queryTypes.withLabelValues($._config.clusterLabel, metric='prometheus_build_info{%(prometheusSelector)s}' % $._config) + + variable.datasource.generalOptions.showOnDashboard.withLabelAndValue() + ; + + local jobVariable = + variable.query.new('job') + + variable.query.generalOptions.withLabel('job') + + variable.query.withDatasourceFromVariable(datasourceVariable) + + variable.query.refresh.onTime() + + variable.query.withSort(type='alphabetical', asc=false) + + variable.query.selectionOptions.withIncludeAll(true, '.+') + + variable.query.selectionOptions.withMulti(true) + + if showMultiCluster then + variable.query.queryTypes.withLabelValues('job', metric='prometheus_build_info{cluster=~"$cluster"}') + else + variable.query.queryTypes.withLabelValues('job', metric='prometheus_build_info{%(prometheusSelector)s}' % $._config) + ; + + local instanceVariable = + variable.query.new('instance') + + variable.query.generalOptions.withLabel('instance') + + variable.query.withDatasourceFromVariable(datasourceVariable) + + variable.query.refresh.onTime() + + variable.query.withSort(type='alphabetical', asc=false) + + variable.query.selectionOptions.withIncludeAll(true, '.+') + + variable.query.selectionOptions.withMulti(true) + + if showMultiCluster then + variable.query.queryTypes.withLabelValues('instance', metric='prometheus_build_info{cluster=~"$cluster", job=~"$job"}') + else + variable.query.queryTypes.withLabelValues('instance', metric='prometheus_build_info{job=~"$job"}') + ; + + local prometheusStats = + panel.table.new('Prometheus Stats') + + panel.table.queryOptions.withDatasource('prometheus', '$datasource') + + panel.table.standardOptions.withUnit('short') + + panel.table.standardOptions.withDecimals(2) + + panel.table.standardOptions.withDisplayName('') + + panel.table.standardOptions.withOverrides([ + panel.table.standardOptions.override.byName.new('Time') + + panel.table.standardOptions.override.byName.withProperty('displayName', 'Time') + + panel.table.standardOptions.override.byName.withProperty('custom.align', null) + + panel.table.standardOptions.override.byName.withProperty('custom.hidden', 'true'), + panel.table.standardOptions.override.byName.new('cluster') + + panel.table.standardOptions.override.byName.withProperty('custom.align', null) + + panel.table.standardOptions.override.byName.withProperty('unit', 'short') + + panel.table.standardOptions.override.byName.withProperty('decimals', 2) + + if showMultiCluster then panel.table.standardOptions.override.byName.withProperty('displayName', 'Cluster') else {}, + panel.table.standardOptions.override.byName.new('job') + + panel.table.standardOptions.override.byName.withProperty('custom.align', null) + + panel.table.standardOptions.override.byName.withProperty('unit', 'short') + + panel.table.standardOptions.override.byName.withProperty('decimals', 2) + + panel.table.standardOptions.override.byName.withProperty('displayName', 'Job'), + panel.table.standardOptions.override.byName.new('instance') + + panel.table.standardOptions.override.byName.withProperty('displayName', 'Instance') + + panel.table.standardOptions.override.byName.withProperty('custom.align', null) + + panel.table.standardOptions.override.byName.withProperty('unit', 'short') + + panel.table.standardOptions.override.byName.withProperty('decimals', 2), + panel.table.standardOptions.override.byName.new('version') + + panel.table.standardOptions.override.byName.withProperty('displayName', 'Version') + + panel.table.standardOptions.override.byName.withProperty('custom.align', null) + + panel.table.standardOptions.override.byName.withProperty('unit', 'short') + + panel.table.standardOptions.override.byName.withProperty('decimals', 2), + panel.table.standardOptions.override.byName.new('Value #A') + + panel.table.standardOptions.override.byName.withProperty('displayName', 'Count') + + panel.table.standardOptions.override.byName.withProperty('custom.align', null) + + panel.table.standardOptions.override.byName.withProperty('unit', 'short') + + panel.table.standardOptions.override.byName.withProperty('decimals', 2) + + panel.table.standardOptions.override.byName.withProperty('custom.hidden', 'true'), + panel.table.standardOptions.override.byName.new('Value #B') + + panel.table.standardOptions.override.byName.withProperty('displayName', 'Uptime') + + panel.table.standardOptions.override.byName.withProperty('custom.align', null) + + panel.table.standardOptions.override.byName.withProperty('unit', 's'), + ]) + + if showMultiCluster then + panel.table.queryOptions.withTargets([ + prometheus.new( + '$datasource', + 'count by (cluster, job, instance, version) (prometheus_build_info{cluster=~"$cluster", job=~"$job", instance=~"$instance"})' + ) + + prometheus.withFormat('table') + + prometheus.withInstant(true) + + prometheus.withLegendFormat(''), + prometheus.new( + '$datasource', + 'max by (cluster, job, instance) (time() - process_start_time_seconds{cluster=~"$cluster", job=~"$job", instance=~"$instance"})' + ) + + prometheus.withFormat('table') + + prometheus.withInstant(true) + + prometheus.withLegendFormat(''), + ]) + else + panel.table.queryOptions.withTargets([ + prometheus.new( + '$datasource', + 'count by (job, instance, version) (prometheus_build_info{job=~"$job", instance=~"$instance"})' + ) + + prometheus.withFormat('table') + + prometheus.withInstant(true) + + prometheus.withLegendFormat(''), + prometheus.new( + '$datasource', + 'max by (job, instance) (time() - process_start_time_seconds{job=~"$job", instance=~"$instance"})' + ) + + prometheus.withFormat('table') + + prometheus.withInstant(true) + + prometheus.withLegendFormat(''), + ]) + ; + + local targetSync = + panel.timeSeries.new('Target Sync') + + panelTimeSeriesStdOptions + + panel.timeSeries.options.tooltip.withSort('desc') + + panel.timeSeries.standardOptions.withMin(0) + + panel.timeSeries.standardOptions.withUnit('ms') + + if showMultiCluster then + panel.timeSeries.queryOptions.withTargets([ + prometheus.new( + '$datasource', + 'sum(rate(prometheus_target_sync_length_seconds_sum{cluster=~"$cluster",job=~"$job",instance=~"$instance"}[5m])) by (cluster, job, scrape_job, instance) * 1e3' + ) + + prometheus.withFormat('time_series') + + prometheus.withLegendFormat('{{cluster}}:{{job}}:{{instance}}:{{scrape_job}}'), + ]) + else + panel.timeSeries.queryOptions.withTargets([ + prometheus.new( + '$datasource', + 'sum(rate(prometheus_target_sync_length_seconds_sum{job=~"$job",instance=~"$instance"}[5m])) by (scrape_job) * 1e3' + ) + + prometheus.withFormat('time_series') + + prometheus.withLegendFormat('{{scrape_job}}'), + ]) + ; + + local targets = + panel.timeSeries.new('Targets') + + panelTimeSeriesStdOptions + + panel.timeSeries.options.tooltip.withSort('desc') + + panel.timeSeries.standardOptions.withMin(0) + + panelTimeSeriesStacking + + panel.timeSeries.standardOptions.withUnit('short') + + if showMultiCluster then + panel.timeSeries.queryOptions.withTargets([ + prometheus.new( + '$datasource', + 'sum by (cluster, job, instance) (prometheus_sd_discovered_targets{cluster=~"$cluster", job=~"$job",instance=~"$instance"})' + ) + + prometheus.withFormat('time_series') + + prometheus.withLegendFormat('{{cluster}}:{{job}}:{{instance}}'), + ]) + else + panel.timeSeries.queryOptions.withTargets([ + prometheus.new( + '$datasource', + 'sum(prometheus_sd_discovered_targets{job=~"$job",instance=~"$instance"})' + ) + + prometheus.withFormat('time_series') + + prometheus.withLegendFormat('Targets'), + ]) + ; + + local averageScrapeIntervalDuration = + panel.timeSeries.new('Average Scrape Interval Duration') + + panelTimeSeriesStdOptions + + panel.timeSeries.options.tooltip.withSort('desc') + + panel.timeSeries.standardOptions.withMin(0) + + panel.timeSeries.standardOptions.withUnit('ms') + + if showMultiCluster then + panel.timeSeries.queryOptions.withTargets([ + prometheus.new( + '$datasource', + 'rate(prometheus_target_interval_length_seconds_sum{cluster=~"$cluster", job=~"$job",instance=~"$instance"}[5m]) / rate(prometheus_target_interval_length_seconds_count{cluster=~"$cluster", job=~"$job",instance=~"$instance"}[5m]) * 1e3' + ) + + prometheus.withFormat('time_series') + + prometheus.withLegendFormat('{{cluster}}:{{job}}:{{instance}} {{interval}} configured'), + ]) + else + panel.timeSeries.queryOptions.withTargets([ + prometheus.new( + '$datasource', + 'rate(prometheus_target_interval_length_seconds_sum{job=~"$job",instance=~"$instance"}[5m]) / rate(prometheus_target_interval_length_seconds_count{job=~"$job",instance=~"$instance"}[5m]) * 1e3' + ) + + prometheus.withFormat('time_series') + + prometheus.withLegendFormat('{{interval}} configured'), + ]) + ; + + local scrapeFailures = + panel.timeSeries.new('Scrape failures') + + panelTimeSeriesStdOptions + + panel.timeSeries.options.tooltip.withSort('desc') + + panel.timeSeries.standardOptions.withMin(0) + + panelTimeSeriesStacking + + panel.timeSeries.standardOptions.withUnit('ms') + + if showMultiCluster then + panel.timeSeries.queryOptions.withTargets([ + prometheus.new( + '$datasource', + 'sum by (cluster, job, instance) (rate(prometheus_target_scrapes_exceeded_body_size_limit_total{cluster=~"$cluster",job=~"$job",instance=~"$instance"}[1m]))' + ) + + prometheus.withFormat('time_series') + + prometheus.withLegendFormat('exceeded body size limit: {{cluster}} {{job}} {{instance}}'), + prometheus.new( + '$datasource', + 'sum by (cluster, job, instance) (rate(prometheus_target_scrapes_exceeded_sample_limit_total{cluster=~"$cluster",job=~"$job",instance=~"$instance"}[1m]))' + ) + + prometheus.withFormat('time_series') + + prometheus.withLegendFormat('exceeded sample limit: {{cluster}} {{job}} {{instance}}'), + prometheus.new( + '$datasource', + 'sum by (cluster, job, instance) (rate(prometheus_target_scrapes_sample_duplicate_timestamp_total{cluster=~"$cluster",job=~"$job",instance=~"$instance"}[1m]))' + ) + + prometheus.withFormat('time_series') + + prometheus.withLegendFormat('duplicate timestamp: {{cluster}} {{job}} {{instance}}'), + prometheus.new( + '$datasource', + 'sum by (cluster, job, instance) (rate(prometheus_target_scrapes_sample_out_of_bounds_total{cluster=~"$cluster",job=~"$job",instance=~"$instance"}[1m]))' + ) + + prometheus.withFormat('time_series') + + prometheus.withLegendFormat('out of bounds: {{cluster}} {{job}} {{instance}}'), + prometheus.new( + '$datasource', + 'sum by (cluster, job, instance) (rate(prometheus_target_scrapes_sample_out_of_order_total{cluster=~"$cluster",job=~"$job",instance=~"$instance"}[1m]))' + ) + + prometheus.withFormat('time_series') + + prometheus.withLegendFormat('out of order: {{cluster}} {{job}} {{instance}}'), + ]) + else + panel.timeSeries.queryOptions.withTargets([ + prometheus.new( + '$datasource', + 'sum by (job) (rate(prometheus_target_scrapes_exceeded_body_size_limit_total[1m]))' + ) + + prometheus.withFormat('time_series') + + prometheus.withLegendFormat('exceeded body size limit: {{job}}'), + prometheus.new( + '$datasource', + 'sum by (job) (rate(prometheus_target_scrapes_exceeded_sample_limit_total[1m]))' + ) + + prometheus.withFormat('time_series') + + prometheus.withLegendFormat('exceeded sample limit: {{job}}'), + prometheus.new( + '$datasource', + 'sum by (job) (rate(prometheus_target_scrapes_sample_duplicate_timestamp_total[1m]))' + ) + + prometheus.withFormat('time_series') + + prometheus.withLegendFormat('duplicate timestamp: {{job}}'), + prometheus.new( + '$datasource', + 'sum by (job) (rate(prometheus_target_scrapes_sample_out_of_bounds_total[1m]))' + ) + + prometheus.withFormat('time_series') + + prometheus.withLegendFormat('out of bounds: {{job}}'), + prometheus.new( + '$datasource', + 'sum by (job) (rate(prometheus_target_scrapes_sample_out_of_order_total[1m]))' + ) + + prometheus.withFormat('time_series') + + prometheus.withLegendFormat('out of order: {{job}}'), + ]) + ; + + local appendedSamples = + panel.timeSeries.new('Appended Samples') + + panelTimeSeriesStdOptions + + panel.timeSeries.options.tooltip.withSort('desc') + + panel.timeSeries.standardOptions.withMin(0) + + panelTimeSeriesStacking + + panel.timeSeries.standardOptions.withUnit('short') + + if showMultiCluster then + panel.timeSeries.queryOptions.withTargets([ + prometheus.new( + '$datasource', + 'rate(prometheus_tsdb_head_samples_appended_total{cluster=~"$cluster", job=~"$job",instance=~"$instance"}[5m])' + ) + + prometheus.withFormat('time_series') + + prometheus.withLegendFormat('{{cluster}} {{job}} {{instance}}'), + ]) + else + panel.timeSeries.queryOptions.withTargets([ + prometheus.new( + '$datasource', + 'rate(prometheus_tsdb_head_samples_appended_total{job=~"$job",instance=~"$instance"}[5m])' + ) + + prometheus.withFormat('time_series') + + prometheus.withLegendFormat('{{job}} {{instance}}'), + ]) + ; + + local headSeries = + panel.timeSeries.new('Head Series') + + panelTimeSeriesStdOptions + + panel.timeSeries.options.tooltip.withSort('desc') + + panel.timeSeries.standardOptions.withMin(0) + + panelTimeSeriesStacking + + panel.timeSeries.standardOptions.withUnit('short') + + if showMultiCluster then + panel.timeSeries.queryOptions.withTargets([ + prometheus.new( + '$datasource', + 'prometheus_tsdb_head_series{cluster=~"$cluster",job=~"$job",instance=~"$instance"}' + ) + + prometheus.withFormat('time_series') + + prometheus.withLegendFormat('{{cluster}} {{job}} {{instance}} head series'), + ]) + else + panel.timeSeries.queryOptions.withTargets([ + prometheus.new( + '$datasource', + 'prometheus_tsdb_head_series{job=~"$job",instance=~"$instance"}' + ) + + prometheus.withFormat('time_series') + + prometheus.withLegendFormat('{{job}} {{instance}} head series'), + ]) + ; + + local headChunks = + panel.timeSeries.new('Head Chunks') + + panelTimeSeriesStdOptions + + panel.timeSeries.options.tooltip.withSort('desc') + + panel.timeSeries.standardOptions.withMin(0) + + panelTimeSeriesStacking + + panel.timeSeries.standardOptions.withUnit('short') + + if showMultiCluster then + panel.timeSeries.queryOptions.withTargets([ + prometheus.new( + '$datasource', + 'prometheus_tsdb_head_chunks{cluster=~"$cluster",job=~"$job",instance=~"$instance"}' + ) + + prometheus.withFormat('time_series') + + prometheus.withLegendFormat('{{cluster}} {{job}} {{instance}} head chunks'), + ]) + else + panel.timeSeries.queryOptions.withTargets([ + prometheus.new( + '$datasource', + 'prometheus_tsdb_head_chunks{job=~"$job",instance=~"$instance"}' + ) + + prometheus.withFormat('time_series') + + prometheus.withLegendFormat('{{job}} {{instance}} head chunks'), + ]) + ; + + local queryRate = + panel.timeSeries.new('Query Rate') + + panelTimeSeriesStdOptions + + panel.timeSeries.options.tooltip.withSort('desc') + + panel.timeSeries.standardOptions.withMin(0) + + panelTimeSeriesStacking + + panel.timeSeries.standardOptions.withUnit('short') + + if showMultiCluster then + panel.timeSeries.queryOptions.withTargets([ + prometheus.new( + '$datasource', + 'rate(prometheus_engine_query_duration_seconds_count{cluster=~"$cluster",job=~"$job",instance=~"$instance",slice="inner_eval"}[5m])' + ) + + prometheus.withFormat('time_series') + + prometheus.withLegendFormat('{{cluster}} {{job}} {{instance}}'), + ]) + else + panel.timeSeries.queryOptions.withTargets([ + prometheus.new( + '$datasource', + 'rate(prometheus_engine_query_duration_seconds_count{job=~"$job",instance=~"$instance",slice="inner_eval"}[5m])' + ) + + prometheus.withFormat('time_series') + + prometheus.withLegendFormat('{{job}} {{instance}}'), + ]) + ; + + local stageDuration = + panel.timeSeries.new('Stage Duration') + + panelTimeSeriesStdOptions + + panel.timeSeries.options.tooltip.withSort('desc') + + panel.timeSeries.standardOptions.withMin(0) + + panelTimeSeriesStacking + + panel.timeSeries.standardOptions.withUnit('ms') + + if showMultiCluster then + panel.timeSeries.queryOptions.withTargets([ + prometheus.new( + '$datasource', + 'max by (slice) (prometheus_engine_query_duration_seconds{quantile="0.9",cluster=~"$cluster", job=~"$job",instance=~"$instance"}) * 1e3' + ) + + prometheus.withFormat('time_series') + + prometheus.withLegendFormat('{{slice}}'), + ]) + else + panel.timeSeries.queryOptions.withTargets([ + prometheus.new( + '$datasource', + 'max by (slice) (prometheus_engine_query_duration_seconds{quantile="0.9",job=~"$job",instance=~"$instance"}) * 1e3' + ) + + prometheus.withFormat('time_series') + + prometheus.withLegendFormat('{{slice}}'), + ]) + ; + + dashboard.new('%(prefix)sOverview' % $._config.grafanaPrometheus) + + dashboard.time.withFrom('now-1h') + + dashboard.withTags($._config.grafanaPrometheus.tags) + + dashboard.timepicker.withRefreshIntervals($._config.grafanaPrometheus.refresh) + + dashboard.withVariables(std.prune([ + datasourceVariable, + if showMultiCluster then clusterVariable, + jobVariable, + instanceVariable, + ])) + + dashboard.withPanels( + grafana.util.grid.makeGrid([ + row.new('Prometheus Stats') + + row.withPanels([ + prometheusStats, + ]), + ], panelWidth=24, panelHeight=7) + + + grafana.util.grid.makeGrid([ + row.new('Discovery') + + row.withPanels([ + targetSync, + targets, + ]), + ], panelWidth=12, panelHeight=7, startY=8) + + + grafana.util.grid.makeGrid([ + row.new('Retrieval') + + row.withPanels([ + averageScrapeIntervalDuration, + scrapeFailures, + appendedSamples, + ]), + ], panelWidth=8, panelHeight=7, startY=16) + + + grafana.util.grid.makeGrid([ + row.new('Storage') + + row.withPanels([ + headSeries, + headChunks, + ]), + row.new('Query') + + row.withPanels([ + queryRate, + stageDuration, + ]), + ], panelWidth=12, panelHeight=7, startY=24) + ), // Remote write specific dashboard. 'prometheus-remote-write.json': + + local datasourceVariable = + variable.datasource.new('datasource', 'prometheus') + + variable.datasource.generalOptions.withCurrent('default') + + variable.datasource.generalOptions.showOnDashboard.withLabelAndValue() + ; + + local clusterVariable = + variable.query.new('cluster') + + variable.query.withDatasourceFromVariable(datasourceVariable) + + variable.query.refresh.onTime() + + variable.query.selectionOptions.withIncludeAll(true) + + variable.query.generalOptions.withCurrent('$__all') + + variable.query.queryTypes.withLabelValues($._config.clusterLabel, metric='prometheus_build_info') + + variable.datasource.generalOptions.showOnDashboard.withLabelAndValue() + ; + + local instanceVariable = + variable.query.new('instance') + + variable.query.withDatasourceFromVariable(datasourceVariable) + + variable.query.refresh.onTime() + + variable.query.selectionOptions.withIncludeAll(true) + + variable.query.queryTypes.withLabelValues('instance', metric='prometheus_build_info{cluster=~"$cluster"}') + ; + + local urlVariable = + variable.query.new('url') + + variable.query.withDatasourceFromVariable(datasourceVariable) + + variable.query.refresh.onTime() + + variable.query.selectionOptions.withIncludeAll(true) + + variable.query.queryTypes.withLabelValues('url', metric='prometheus_remote_storage_shards{cluster=~"$cluster", instance=~"$instance"}') + ; + local timestampComparison = - graphPanel.new( - 'Highest Timestamp In vs. Highest Timestamp Sent', - datasource='$datasource', - span=6, - ) - .addTarget(prometheus.target( - ||| - ( - prometheus_remote_storage_highest_timestamp_in_seconds{cluster=~"$cluster", instance=~"$instance"} - - - ignoring(remote_name, url) group_right(instance) (prometheus_remote_storage_queue_highest_sent_timestamp_seconds{cluster=~"$cluster", instance=~"$instance", url=~"$url"} != 0) - ) - |||, - legendFormat='{{cluster}}:{{instance}} {{remote_name}}:{{url}}', - )); + panel.timeSeries.new('Highest Timestamp In vs. Highest Timestamp Sent') + + panelTimeSeriesStdOptions + + panel.timeSeries.standardOptions.withUnit('short') + + panel.timeSeries.queryOptions.withTargets([ + prometheus.new( + '$datasource', + ||| + ( + prometheus_remote_storage_highest_timestamp_in_seconds{cluster=~"$cluster", instance=~"$instance"} + - + ignoring(remote_name, url) group_right(instance) (prometheus_remote_storage_queue_highest_sent_timestamp_seconds{cluster=~"$cluster", instance=~"$instance", url=~"$url"} != 0) + ) + ||| + ) + + prometheus.withFormat('time_series') + + prometheus.withIntervalFactor(2) + + prometheus.withLegendFormat('{{cluster}}:{{instance}} {{remote_name}}:{{url}}'), + ]); local timestampComparisonRate = - graphPanel.new( - 'Rate[5m]', - datasource='$datasource', - span=6, - ) - .addTarget(prometheus.target( - ||| - clamp_min( - rate(prometheus_remote_storage_highest_timestamp_in_seconds{cluster=~"$cluster", instance=~"$instance"}[5m]) - - - ignoring (remote_name, url) group_right(instance) rate(prometheus_remote_storage_queue_highest_sent_timestamp_seconds{cluster=~"$cluster", instance=~"$instance", url=~"$url"}[5m]) - , 0) - |||, - legendFormat='{{cluster}}:{{instance}} {{remote_name}}:{{url}}', - )); + panel.timeSeries.new('Rate[5m]') + + panelTimeSeriesStdOptions + + panel.timeSeries.standardOptions.withUnit('short') + + panel.timeSeries.queryOptions.withTargets([ + prometheus.new( + '$datasource', + ||| + clamp_min( + rate(prometheus_remote_storage_highest_timestamp_in_seconds{cluster=~"$cluster", instance=~"$instance"}[5m]) + - + ignoring (remote_name, url) group_right(instance) rate(prometheus_remote_storage_queue_highest_sent_timestamp_seconds{cluster=~"$cluster", instance=~"$instance", url=~"$url"}[5m]) + , 0) + ||| + ) + + prometheus.withFormat('time_series') + + prometheus.withIntervalFactor(2) + + prometheus.withLegendFormat('{{cluster}}:{{instance}} {{remote_name}}:{{url}}'), + ]); local samplesRate = - graphPanel.new( - 'Rate, in vs. succeeded or dropped [5m]', - datasource='$datasource', - span=12, - ) - .addTarget(prometheus.target( - ||| - rate( - prometheus_remote_storage_samples_in_total{cluster=~"$cluster", instance=~"$instance"}[5m]) - - - ignoring(remote_name, url) group_right(instance) (rate(prometheus_remote_storage_succeeded_samples_total{cluster=~"$cluster", instance=~"$instance", url=~"$url"}[5m]) or rate(prometheus_remote_storage_samples_total{cluster=~"$cluster", instance=~"$instance", url=~"$url"}[5m])) - - - (rate(prometheus_remote_storage_dropped_samples_total{cluster=~"$cluster", instance=~"$instance", url=~"$url"}[5m]) or rate(prometheus_remote_storage_samples_dropped_total{cluster=~"$cluster", instance=~"$instance", url=~"$url"}[5m])) - |||, - legendFormat='{{cluster}}:{{instance}} {{remote_name}}:{{url}}' - )); + panel.timeSeries.new('Rate, in vs. succeeded or dropped [5m]') + + panelTimeSeriesStdOptions + + panel.timeSeries.standardOptions.withUnit('short') + + panel.timeSeries.queryOptions.withTargets([ + prometheus.new( + '$datasource', + ||| + rate( + prometheus_remote_storage_samples_in_total{cluster=~"$cluster", instance=~"$instance"}[5m]) + - + ignoring(remote_name, url) group_right(instance) (rate(prometheus_remote_storage_succeeded_samples_total{cluster=~"$cluster", instance=~"$instance", url=~"$url"}[5m]) or rate(prometheus_remote_storage_samples_total{cluster=~"$cluster", instance=~"$instance", url=~"$url"}[5m])) + - + (rate(prometheus_remote_storage_dropped_samples_total{cluster=~"$cluster", instance=~"$instance", url=~"$url"}[5m]) or rate(prometheus_remote_storage_samples_dropped_total{cluster=~"$cluster", instance=~"$instance", url=~"$url"}[5m])) + ||| + ) + + prometheus.withFormat('time_series') + + prometheus.withIntervalFactor(2) + + prometheus.withLegendFormat('{{cluster}}:{{instance}} {{remote_name}}:{{url}}'), + ]); local currentShards = - graphPanel.new( - 'Current Shards', - datasource='$datasource', - span=12, - min_span=6, - ) - .addTarget(prometheus.target( - 'prometheus_remote_storage_shards{cluster=~"$cluster", instance=~"$instance", url=~"$url"}', - legendFormat='{{cluster}}:{{instance}} {{remote_name}}:{{url}}' - )); + panel.timeSeries.new('Current Shards') + + panelTimeSeriesStdOptions + + panel.timeSeries.standardOptions.withUnit('short') + + panel.timeSeries.queryOptions.withTargets([ + prometheus.new( + '$datasource', + 'prometheus_remote_storage_shards{cluster=~"$cluster", instance=~"$instance", url=~"$url"}' + ) + + prometheus.withFormat('time_series') + + prometheus.withIntervalFactor(2) + + prometheus.withLegendFormat('{{cluster}}:{{instance}} {{remote_name}}:{{url}}'), + ]); local maxShards = - graphPanel.new( - 'Max Shards', - datasource='$datasource', - span=4, - ) - .addTarget(prometheus.target( - 'prometheus_remote_storage_shards_max{cluster=~"$cluster", instance=~"$instance", url=~"$url"}', - legendFormat='{{cluster}}:{{instance}} {{remote_name}}:{{url}}' - )); + panel.timeSeries.new('Max Shards') + + panelTimeSeriesStdOptions + + panel.timeSeries.standardOptions.withUnit('short') + + panel.timeSeries.queryOptions.withTargets([ + prometheus.new( + '$datasource', + 'prometheus_remote_storage_shards_max{cluster=~"$cluster", instance=~"$instance", url=~"$url"}' + ) + + prometheus.withFormat('time_series') + + prometheus.withIntervalFactor(2) + + prometheus.withLegendFormat('{{cluster}}:{{instance}} {{remote_name}}:{{url}}'), + ]); local minShards = - graphPanel.new( - 'Min Shards', - datasource='$datasource', - span=4, - ) - .addTarget(prometheus.target( - 'prometheus_remote_storage_shards_min{cluster=~"$cluster", instance=~"$instance", url=~"$url"}', - legendFormat='{{cluster}}:{{instance}} {{remote_name}}:{{url}}' - )); + panel.timeSeries.new('Min Shards') + + panelTimeSeriesStdOptions + + panel.timeSeries.standardOptions.withUnit('short') + + panel.timeSeries.queryOptions.withTargets([ + prometheus.new( + '$datasource', + 'prometheus_remote_storage_shards_min{cluster=~"$cluster", instance=~"$instance", url=~"$url"}' + ) + + prometheus.withFormat('time_series') + + prometheus.withIntervalFactor(2) + + prometheus.withLegendFormat('{{cluster}}:{{instance}} {{remote_name}}:{{url}}'), + ]); local desiredShards = - graphPanel.new( - 'Desired Shards', - datasource='$datasource', - span=4, - ) - .addTarget(prometheus.target( - 'prometheus_remote_storage_shards_desired{cluster=~"$cluster", instance=~"$instance", url=~"$url"}', - legendFormat='{{cluster}}:{{instance}} {{remote_name}}:{{url}}' - )); + panel.timeSeries.new('Desired Shards') + + panelTimeSeriesStdOptions + + panel.timeSeries.standardOptions.withUnit('short') + + panel.timeSeries.queryOptions.withTargets([ + prometheus.new( + '$datasource', + 'prometheus_remote_storage_shards_desired{cluster=~"$cluster", instance=~"$instance", url=~"$url"}' + ) + + prometheus.withFormat('time_series') + + prometheus.withIntervalFactor(2) + + prometheus.withLegendFormat('{{cluster}}:{{instance}} {{remote_name}}:{{url}}'), + ]); local shardsCapacity = - graphPanel.new( - 'Shard Capacity', - datasource='$datasource', - span=6, - ) - .addTarget(prometheus.target( - 'prometheus_remote_storage_shard_capacity{cluster=~"$cluster", instance=~"$instance", url=~"$url"}', - legendFormat='{{cluster}}:{{instance}} {{remote_name}}:{{url}}' - )); - + panel.timeSeries.new('Shard Capacity') + + panelTimeSeriesStdOptions + + panel.timeSeries.standardOptions.withUnit('short') + + panel.timeSeries.queryOptions.withTargets([ + prometheus.new( + '$datasource', + 'prometheus_remote_storage_shard_capacity{cluster=~"$cluster", instance=~"$instance", url=~"$url"}' + ) + + prometheus.withFormat('time_series') + + prometheus.withIntervalFactor(2) + + prometheus.withLegendFormat('{{cluster}}:{{instance}} {{remote_name}}:{{url}}'), + ]); local pendingSamples = - graphPanel.new( - 'Pending Samples', - datasource='$datasource', - span=6, - ) - .addTarget(prometheus.target( - 'prometheus_remote_storage_pending_samples{cluster=~"$cluster", instance=~"$instance", url=~"$url"} or prometheus_remote_storage_samples_pending{cluster=~"$cluster", instance=~"$instance", url=~"$url"}', - legendFormat='{{cluster}}:{{instance}} {{remote_name}}:{{url}}' - )); + panel.timeSeries.new('Pending Samples') + + panelTimeSeriesStdOptions + + panel.timeSeries.standardOptions.withUnit('short') + + panel.timeSeries.queryOptions.withTargets([ + prometheus.new( + '$datasource', + 'prometheus_remote_storage_pending_samples{cluster=~"$cluster", instance=~"$instance", url=~"$url"} or prometheus_remote_storage_samples_pending{cluster=~"$cluster", instance=~"$instance", url=~"$url"}' + ) + + prometheus.withFormat('time_series') + + prometheus.withIntervalFactor(2) + + prometheus.withLegendFormat('{{cluster}}:{{instance}} {{remote_name}}:{{url}}'), + ]); local walSegment = - graphPanel.new( - 'TSDB Current Segment', - datasource='$datasource', - span=6, - formatY1='none', - ) - .addTarget(prometheus.target( - 'prometheus_tsdb_wal_segment_current{cluster=~"$cluster", instance=~"$instance"}', - legendFormat='{{cluster}}:{{instance}}' - )); + panel.timeSeries.new('TSDB Current Segment') + + panelTimeSeriesStdOptions + + panel.timeSeries.options.tooltip.withMode('single') + + panel.timeSeries.fieldConfig.defaults.custom.withFillOpacity(0) + + panel.timeSeries.standardOptions.withUnit('none') + + panel.timeSeries.queryOptions.withTargets([ + prometheus.new( + '$datasource', + 'prometheus_tsdb_wal_segment_current{cluster=~"$cluster", instance=~"$instance"}' + ) + + prometheus.withFormat('time_series') + + prometheus.withIntervalFactor(2) + + prometheus.withLegendFormat('{{cluster}}:{{instance}}'), + ]); local queueSegment = - graphPanel.new( - 'Remote Write Current Segment', - datasource='$datasource', - span=6, - formatY1='none', - ) - .addTarget(prometheus.target( - 'prometheus_wal_watcher_current_segment{cluster=~"$cluster", instance=~"$instance"}', - legendFormat='{{cluster}}:{{instance}} {{consumer}}' - )); + panel.timeSeries.new('Remote Write Current Segment') + + panelTimeSeriesStdOptions + + panel.timeSeries.options.tooltip.withMode('single') + + panel.timeSeries.fieldConfig.defaults.custom.withFillOpacity(0) + + panel.timeSeries.standardOptions.withUnit('none') + + panel.timeSeries.queryOptions.withTargets([ + prometheus.new( + '$datasource', + 'prometheus_wal_watcher_current_segment{cluster=~"$cluster", instance=~"$instance"}' + ) + + prometheus.withFormat('time_series') + + prometheus.withIntervalFactor(2) + + prometheus.withLegendFormat('{{cluster}}:{{instance}} {{consumer}}'), + ]); local droppedSamples = - graphPanel.new( - 'Dropped Samples', - datasource='$datasource', - span=3, - ) - .addTarget(prometheus.target( - 'rate(prometheus_remote_storage_dropped_samples_total{cluster=~"$cluster", instance=~"$instance", url=~"$url"}[5m]) or rate(prometheus_remote_storage_samples_dropped_total{cluster=~"$cluster", instance=~"$instance", url=~"$url"}[5m])', - legendFormat='{{cluster}}:{{instance}} {{remote_name}}:{{url}}' - )); + panel.timeSeries.new('Dropped Samples') + + panelTimeSeriesStdOptions + + panel.timeSeries.options.tooltip.withMode('single') + + panel.timeSeries.fieldConfig.defaults.custom.withFillOpacity(0) + + panel.timeSeries.queryOptions.withTargets([ + prometheus.new( + '$datasource', + 'rate(prometheus_remote_storage_dropped_samples_total{cluster=~"$cluster", instance=~"$instance", url=~"$url"}[5m]) or rate(prometheus_remote_storage_samples_dropped_total{cluster=~"$cluster", instance=~"$instance", url=~"$url"}[5m])' + ) + + prometheus.withFormat('time_series') + + prometheus.withIntervalFactor(2) + + prometheus.withLegendFormat('{{cluster}}:{{instance}} {{remote_name}}:{{url}}'), + ]); local failedSamples = - graphPanel.new( - 'Failed Samples', - datasource='$datasource', - span=3, - ) - .addTarget(prometheus.target( - 'rate(prometheus_remote_storage_failed_samples_total{cluster=~"$cluster", instance=~"$instance", url=~"$url"}[5m]) or rate(prometheus_remote_storage_samples_failed_total{cluster=~"$cluster", instance=~"$instance", url=~"$url"}[5m])', - legendFormat='{{cluster}}:{{instance}} {{remote_name}}:{{url}}' - )); + panel.timeSeries.new('Failed Samples') + + panelTimeSeriesStdOptions + + panel.timeSeries.options.tooltip.withMode('single') + + panel.timeSeries.fieldConfig.defaults.custom.withFillOpacity(0) + + panel.timeSeries.queryOptions.withTargets([ + prometheus.new( + '$datasource', + 'rate(prometheus_remote_storage_failed_samples_total{cluster=~"$cluster", instance=~"$instance", url=~"$url"}[5m]) or rate(prometheus_remote_storage_samples_failed_total{cluster=~"$cluster", instance=~"$instance", url=~"$url"}[5m])' + ) + + prometheus.withFormat('time_series') + + prometheus.withIntervalFactor(2) + + prometheus.withLegendFormat('{{cluster}}:{{instance}} {{remote_name}}:{{url}}'), + ]); local retriedSamples = - graphPanel.new( - 'Retried Samples', - datasource='$datasource', - span=3, - ) - .addTarget(prometheus.target( - 'rate(prometheus_remote_storage_retried_samples_total{cluster=~"$cluster", instance=~"$instance", url=~"$url"}[5m]) or rate(prometheus_remote_storage_samples_retried_total{cluster=~"$cluster", instance=~"$instance", url=~"$url"}[5m])', - legendFormat='{{cluster}}:{{instance}} {{remote_name}}:{{url}}' - )); + panel.timeSeries.new('Retried Samples') + + panelTimeSeriesStdOptions + + panel.timeSeries.options.tooltip.withMode('single') + + panel.timeSeries.fieldConfig.defaults.custom.withFillOpacity(0) + + panel.timeSeries.queryOptions.withTargets([ + prometheus.new( + '$datasource', + 'rate(prometheus_remote_storage_retried_samples_total{cluster=~"$cluster", instance=~"$instance", url=~"$url"}[5m]) or rate(prometheus_remote_storage_samples_retried_total{cluster=~"$cluster", instance=~"$instance", url=~"$url"}[5m])' + ) + + prometheus.withFormat('time_series') + + prometheus.withIntervalFactor(2) + + prometheus.withLegendFormat('{{cluster}}:{{instance}} {{remote_name}}:{{url}}'), + ]); local enqueueRetries = - graphPanel.new( - 'Enqueue Retries', - datasource='$datasource', - span=3, - ) - .addTarget(prometheus.target( - 'rate(prometheus_remote_storage_enqueue_retries_total{cluster=~"$cluster", instance=~"$instance", url=~"$url"}[5m])', - legendFormat='{{cluster}}:{{instance}} {{remote_name}}:{{url}}' - )); + panel.timeSeries.new('Enqueue Retries') + + panelTimeSeriesStdOptions + + panel.timeSeries.options.tooltip.withMode('single') + + panel.timeSeries.fieldConfig.defaults.custom.withFillOpacity(0) + + panel.timeSeries.queryOptions.withTargets([ + prometheus.new( + '$datasource', + 'rate(prometheus_remote_storage_enqueue_retries_total{cluster=~"$cluster", instance=~"$instance", url=~"$url"}[5m])' + ) + + prometheus.withFormat('time_series') + + prometheus.withIntervalFactor(2) + + prometheus.withLegendFormat('{{cluster}}:{{instance}} {{remote_name}}:{{url}}'), + ]); - dashboard.new( - title='%(prefix)sRemote Write' % $._config.grafanaPrometheus, - editable=true - ) - .addTemplate( - { - hide: 0, - label: null, - name: 'datasource', - options: [], - query: 'prometheus', - refresh: 1, - regex: '', - type: 'datasource', - }, - ) - .addTemplate( - template.new( - 'cluster', - '$datasource', - 'label_values(prometheus_build_info, cluster)' % $._config, - refresh='time', - current={ - selected: true, - text: 'All', - value: '$__all', - }, - includeAll=true, - ) - ) - .addTemplate( - template.new( - 'instance', - '$datasource', - 'label_values(prometheus_build_info{cluster=~"$cluster"}, instance)' % $._config, - refresh='time', - current={ - selected: true, - text: 'All', - value: '$__all', - }, - includeAll=true, - ) - ) - .addTemplate( - template.new( - 'url', - '$datasource', - 'label_values(prometheus_remote_storage_shards{cluster=~"$cluster", instance=~"$instance"}, url)' % $._config, - refresh='time', - includeAll=true, - ) - ) - .addRow( - row.new('Timestamps') - .addPanel(timestampComparison) - .addPanel(timestampComparisonRate) - ) - .addRow( - row.new('Samples') - .addPanel(samplesRate) - ) - .addRow( - row.new( - 'Shards' - ) - .addPanel(currentShards) - .addPanel(maxShards) - .addPanel(minShards) - .addPanel(desiredShards) - ) - .addRow( - row.new('Shard Details') - .addPanel(shardsCapacity) - .addPanel(pendingSamples) - ) - .addRow( - row.new('Segments') - .addPanel(walSegment) - .addPanel(queueSegment) - ) - .addRow( - row.new('Misc. Rates') - .addPanel(droppedSamples) - .addPanel(failedSamples) - .addPanel(retriedSamples) - .addPanel(enqueueRetries) - ) + { - tags: $._config.grafanaPrometheus.tags, - refresh: $._config.grafanaPrometheus.refresh, - }, + dashboard.new('%(prefix)sRemote Write' % $._config.grafanaPrometheus) + + dashboard.time.withFrom('now-1h') + + dashboard.withTags($._config.grafanaPrometheus.tags) + + dashboard.timepicker.withRefreshIntervals($._config.grafanaPrometheus.refresh) + + dashboard.withVariables([ + datasourceVariable, + clusterVariable, + instanceVariable, + urlVariable, + ]) + + dashboard.withPanels( + grafana.util.grid.makeGrid([ + row.new('Timestamps') + + row.withPanels([ + timestampComparison, + timestampComparisonRate, + ]), + ], panelWidth=12, panelHeight=7) + + + grafana.util.grid.makeGrid([ + row.new('Samples') + + row.withPanels([ + samplesRate + + panel.timeSeries.gridPos.withW(24), + ]), + row.new('Shards'), + ], panelWidth=24, panelHeight=7, startY=8) + + + grafana.util.grid.wrapPanels([ + currentShards + + panel.timeSeries.gridPos.withW(24), + maxShards, + minShards, + desiredShards, + ], panelWidth=8, panelHeight=7, startY=16) + + + grafana.util.grid.makeGrid([ + row.new('Shard Details') + + row.withPanels([ + shardsCapacity, + pendingSamples, + ]), + row.new('Segments') + + row.withPanels([ + walSegment, + queueSegment, + ]), + ], panelWidth=12, panelHeight=7, startY=24) + + + grafana.util.grid.makeGrid([ + row.new('Misc. Rates') + + row.withPanels([ + droppedSamples, + failedSamples, + retriedSamples, + enqueueRetries, + ]), + ], panelWidth=6, panelHeight=7, startY=40) + ), }, } diff --git a/documentation/prometheus-mixin/jsonnetfile.json b/documentation/prometheus-mixin/jsonnetfile.json index 1c64fd0151..2d56d91245 100644 --- a/documentation/prometheus-mixin/jsonnetfile.json +++ b/documentation/prometheus-mixin/jsonnetfile.json @@ -4,20 +4,11 @@ { "source": { "git": { - "remote": "https://github.com/grafana/grafonnet-lib.git", - "subdir": "grafonnet" + "remote": "https://github.com/grafana/grafonnet.git", + "subdir": "gen/grafonnet-latest" } }, - "version": "master" - }, - { - "source": { - "git": { - "remote": "https://github.com/grafana/jsonnet-libs.git", - "subdir": "grafana-builder" - } - }, - "version": "master" + "version": "main" } ], "legacyImports": false From 1508149184faaa46acd7ac22aa3116f8f8ccfdd3 Mon Sep 17 00:00:00 2001 From: Carrie Edwards Date: Fri, 27 Dec 2024 09:09:13 -0800 Subject: [PATCH 023/110] 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 8f4557b0b14382674d706cc38b673be9e6f1bbd8 Mon Sep 17 00:00:00 2001 From: Bryan Boreham Date: Tue, 20 Aug 2024 20:11:21 +0100 Subject: [PATCH 024/110] Scraping benchmark: more realistic test Don't repeat type and help text. Signed-off-by: Bryan Boreham --- scrape/scrape_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/scrape/scrape_test.go b/scrape/scrape_test.go index f9164ea7ac..1f0291ae00 100644 --- a/scrape/scrape_test.go +++ b/scrape/scrape_test.go @@ -1256,9 +1256,9 @@ func TestScrapeLoopFailLegacyUnderUTF8(t *testing.T) { func makeTestMetrics(n int) []byte { // Construct a metrics string to parse sb := bytes.Buffer{} + fmt.Fprintf(&sb, "# TYPE metric_a gauge\n") + fmt.Fprintf(&sb, "# HELP metric_a help text\n") for i := 0; i < n; i++ { - fmt.Fprintf(&sb, "# TYPE metric_a gauge\n") - fmt.Fprintf(&sb, "# HELP metric_a help text\n") fmt.Fprintf(&sb, "metric_a{foo=\"%d\",bar=\"%d\"} 1\n", i, i*100) } fmt.Fprintf(&sb, "# EOF\n") From b4ef38cfc8b4ff100783fcc26e909c77993c26f7 Mon Sep 17 00:00:00 2001 From: Bryan Boreham Date: Tue, 20 Aug 2024 20:12:02 +0100 Subject: [PATCH 025/110] Scraping: Add benchmark for protobuf format Extract helper function textToProto(). Signed-off-by: Bryan Boreham --- scrape/scrape_test.go | 72 +++++++++++++++++++++++++++++++++++-------- 1 file changed, 60 insertions(+), 12 deletions(-) diff --git a/scrape/scrape_test.go b/scrape/scrape_test.go index 1f0291ae00..fa11ad6dcf 100644 --- a/scrape/scrape_test.go +++ b/scrape/scrape_test.go @@ -1295,6 +1295,45 @@ func BenchmarkScrapeLoopAppendOM(b *testing.B) { } } +func BenchmarkScrapeLoopAppendProto(b *testing.B) { + ctx, sl := simpleTestScrapeLoop(b) + + slApp := sl.appender(ctx) + + // Construct a metrics string to parse + sb := bytes.Buffer{} + fmt.Fprintf(&sb, "type: GAUGE\n") + fmt.Fprintf(&sb, "help: \"metric_a help text\"\n") + fmt.Fprintf(&sb, "name: \"metric_a\"\n") + for i := 0; i < 100; i++ { + fmt.Fprintf(&sb, `metric: < + label: < + name: "foo" + value: "%d" + > + label: < + name: "bar" + value: "%d" + > + gauge: < + value: 1 + > +> +`, i, i*100) + } + // From text to proto message. + pb := bytes.Buffer{} + require.NoError(b, textToProto(sb.String(), &pb)) + ts := time.Time{} + + b.ResetTimer() + + for i := 0; i < b.N; i++ { + ts = ts.Add(time.Second) + _, _, _, _ = sl.append(slApp, pb.Bytes(), "application/vnd.google.protobuf", ts) + } +} + func TestSetOptionsHandlingStaleness(t *testing.T) { s := teststorage.New(t, 600000) defer s.Close() @@ -2454,18 +2493,7 @@ metric: < buf := &bytes.Buffer{} if test.contentType == "application/vnd.google.protobuf" { - // In case of protobuf, we have to create the binary representation. - pb := &dto.MetricFamily{} - // From text to proto message. - require.NoError(t, proto.UnmarshalText(test.scrapeText, pb)) - // From proto message to binary protobuf. - protoBuf, err := proto.Marshal(pb) - require.NoError(t, err) - - // Write first length, then binary protobuf. - varintBuf := binary.AppendUvarint(nil, uint64(len(protoBuf))) - buf.Write(varintBuf) - buf.Write(protoBuf) + require.NoError(t, textToProto(test.scrapeText, buf)) } else { buf.WriteString(test.scrapeText) } @@ -2480,6 +2508,26 @@ metric: < } } +func textToProto(text string, buf *bytes.Buffer) error { + // In case of protobuf, we have to create the binary representation. + pb := &dto.MetricFamily{} + // From text to proto message. + err := proto.UnmarshalText(text, pb) + if err != nil { + return err + } + // From proto message to binary protobuf. + protoBuf, err := proto.Marshal(pb) + if err != nil { + return err + } + // Write first length, then binary protobuf. + varintBuf := binary.AppendUvarint(nil, uint64(len(protoBuf))) + buf.Write(varintBuf) + buf.Write(protoBuf) + return nil +} + func TestScrapeLoopAppendExemplarSeries(t *testing.T) { scrapeText := []string{`metric_total{n="1"} 1 # {t="1"} 1.0 10000 # EOF`, `metric_total{n="1"} 2 # {t="2"} 2.0 20000 From bc9210e393d520d460dd197bf640578f2daca90a Mon Sep 17 00:00:00 2001 From: Bryan Boreham Date: Fri, 23 Aug 2024 09:12:02 +0100 Subject: [PATCH 026/110] [TESTS] Scrape: make caching work in benchmark Returning 0 from Append means 'unknown', so the series is never cached. Return arbitrary numbers instead. Signed-off-by: Bryan Boreham --- scrape/helpers_test.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/scrape/helpers_test.go b/scrape/helpers_test.go index 12a56d7071..7bc9e3f7d4 100644 --- a/scrape/helpers_test.go +++ b/scrape/helpers_test.go @@ -46,15 +46,15 @@ type nopAppender struct{} func (a nopAppender) SetOptions(opts *storage.AppendOptions) {} func (a nopAppender) Append(storage.SeriesRef, labels.Labels, int64, float64) (storage.SeriesRef, error) { - return 0, nil + return 1, nil } func (a nopAppender) AppendExemplar(storage.SeriesRef, labels.Labels, exemplar.Exemplar) (storage.SeriesRef, error) { - return 0, nil + return 2, nil } func (a nopAppender) AppendHistogram(storage.SeriesRef, labels.Labels, int64, *histogram.Histogram, *histogram.FloatHistogram) (storage.SeriesRef, error) { - return 0, nil + return 3, nil } func (a nopAppender) AppendHistogramCTZeroSample(ref storage.SeriesRef, l labels.Labels, t, ct int64, h *histogram.Histogram, fh *histogram.FloatHistogram) (storage.SeriesRef, error) { @@ -62,11 +62,11 @@ func (a nopAppender) AppendHistogramCTZeroSample(ref storage.SeriesRef, l labels } func (a nopAppender) UpdateMetadata(storage.SeriesRef, labels.Labels, metadata.Metadata) (storage.SeriesRef, error) { - return 0, nil + return 4, nil } func (a nopAppender) AppendCTZeroSample(storage.SeriesRef, labels.Labels, int64, int64) (storage.SeriesRef, error) { - return 0, nil + return 5, nil } func (a nopAppender) Commit() error { return nil } From 281306765e3ddd2b82c0849a9fe443ff2cda88a2 Mon Sep 17 00:00:00 2001 From: bwplotka Date: Sun, 29 Dec 2024 15:10:39 +0000 Subject: [PATCH 027/110] scrape: Unified scrape loop benchmark. Signed-off-by: bwplotka --- scrape/scrape_test.go | 115 ++++++++++++++++++++---------------------- 1 file changed, 54 insertions(+), 61 deletions(-) diff --git a/scrape/scrape_test.go b/scrape/scrape_test.go index fa11ad6dcf..a67d52e5cc 100644 --- a/scrape/scrape_test.go +++ b/scrape/scrape_test.go @@ -39,6 +39,7 @@ import ( prom_testutil "github.com/prometheus/client_golang/prometheus/testutil" dto "github.com/prometheus/client_model/go" config_util "github.com/prometheus/common/config" + "github.com/prometheus/common/expfmt" "github.com/prometheus/common/model" "github.com/prometheus/common/promslog" "github.com/stretchr/testify/require" @@ -1265,72 +1266,64 @@ func makeTestMetrics(n int) []byte { return sb.Bytes() } +func promTextToProto(tb testing.TB, text []byte) []byte { + tb.Helper() + + d := expfmt.NewDecoder(bytes.NewReader(text), expfmt.TextVersion) + + pb := &dto.MetricFamily{} + if err := d.Decode(pb); err != nil { + tb.Fatal(err) + } + o, err := proto.Marshal(pb) + if err != nil { + tb.Fatal(err) + } + buf := bytes.Buffer{} + // Write first length, then binary protobuf. + varintBuf := binary.AppendUvarint(nil, uint64(len(o))) + buf.Write(varintBuf) + buf.Write(o) + return buf.Bytes() +} + +/* + export bench=scrape-loop-v1 && go test \ + -run '^$' -bench '^BenchmarkScrapeLoopAppend' \ + -benchtime 5s -count 6 -cpu 2 -timeout 999m \ + | tee ${bench}.txt +*/ func BenchmarkScrapeLoopAppend(b *testing.B) { - ctx, sl := simpleTestScrapeLoop(b) + metricsText := makeTestMetrics(100) - slApp := sl.appender(ctx) - metrics := makeTestMetrics(100) - ts := time.Time{} + // Create proto representation. + metricsProto := promTextToProto(b, metricsText) - b.ResetTimer() + for _, bcase := range []struct { + name string + contentType string + parsable []byte + }{ + {name: "PromText", contentType: "text/plain", parsable: metricsText}, + {name: "OMText", contentType: "application/openmetrics-text", parsable: metricsText}, + {name: "PromProto", contentType: "application/vnd.google.protobuf", parsable: metricsProto}, + } { + b.Run(fmt.Sprintf("fmt=%v", bcase.name), func(b *testing.B) { + ctx, sl := simpleTestScrapeLoop(b) - for i := 0; i < b.N; i++ { - ts = ts.Add(time.Second) - _, _, _, _ = sl.append(slApp, metrics, "text/plain", ts) - } -} + slApp := sl.appender(ctx) + ts := time.Time{} -func BenchmarkScrapeLoopAppendOM(b *testing.B) { - ctx, sl := simpleTestScrapeLoop(b) - - slApp := sl.appender(ctx) - metrics := makeTestMetrics(100) - ts := time.Time{} - - b.ResetTimer() - - for i := 0; i < b.N; i++ { - ts = ts.Add(time.Second) - _, _, _, _ = sl.append(slApp, metrics, "application/openmetrics-text", ts) - } -} - -func BenchmarkScrapeLoopAppendProto(b *testing.B) { - ctx, sl := simpleTestScrapeLoop(b) - - slApp := sl.appender(ctx) - - // Construct a metrics string to parse - sb := bytes.Buffer{} - fmt.Fprintf(&sb, "type: GAUGE\n") - fmt.Fprintf(&sb, "help: \"metric_a help text\"\n") - fmt.Fprintf(&sb, "name: \"metric_a\"\n") - for i := 0; i < 100; i++ { - fmt.Fprintf(&sb, `metric: < - label: < - name: "foo" - value: "%d" - > - label: < - name: "bar" - value: "%d" - > - gauge: < - value: 1 - > -> -`, i, i*100) - } - // From text to proto message. - pb := bytes.Buffer{} - require.NoError(b, textToProto(sb.String(), &pb)) - ts := time.Time{} - - b.ResetTimer() - - for i := 0; i < b.N; i++ { - ts = ts.Add(time.Second) - _, _, _, _ = sl.append(slApp, pb.Bytes(), "application/vnd.google.protobuf", ts) + b.ReportAllocs() + b.ResetTimer() + for i := 0; i < b.N; i++ { + ts = ts.Add(time.Second) + _, _, _, err := sl.append(slApp, bcase.parsable, bcase.contentType, ts) + if err != nil { + b.Fatal(err) + } + } + }) } } 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 028/110] 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 029/110] 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 | 94 ++++++++++++++++++------------------ 1 file changed, 48 insertions(+), 46 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 { - isNHCB = p.handleClassicHistogramSeries(p.lset) - } + return EntryInvalid, p.err } - 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() + 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 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 030/110] 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 031/110] 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 032/110] [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 033/110] 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) - } +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)) }, + ) - // 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, "_") - } - - 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 034/110] `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 | 86 ++++++------- rules/manager.go | 59 +++++++-- rules/manager_test.go | 114 ++++++++++++++++++ 6 files changed, 256 insertions(+), 50 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: - } - - if ctrl := g.concurrencyController; ctrl.Allow(ctx, g, rule) { - wg.Add(1) - - go eval(i, rule, func() { - wg.Done() - ctrl.Done(ctx) - }) - } else { - eval(i, rule, nil) - } + 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: + } + + rule := g.rules[ruleIndex] + if len(batch) > 1 && ctrl.Allow(ctx, g, rule) { + wg.Add(1) + + 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 035/110] 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) SetNoDependencyRules(noDependencyRules bool) { - r.noDependencyRules.Store(noDependencyRules) +func (r *AlertingRule) DependentRules() []Rule { + r.dependenciesMutex.RLock() + defer r.dependenciesMutex.RUnlock() + return r.dependentRules +} + +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) SetNoDependencyRules(noDependencyRules bool) { - rule.noDependencyRules.Store(noDependencyRules) +func (rule *RecordingRule) DependentRules() []Rule { + rule.dependenciesMutex.RLock() + defer rule.dependenciesMutex.RUnlock() + return rule.dependentRules +} + +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 036/110] 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 037/110] 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 038/110] 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 039/110] 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 040/110] 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 041/110] 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 { From 9d6f88cb7300eb8c006942f8ce2560eca4e553c7 Mon Sep 17 00:00:00 2001 From: Fiona Liao Date: Thu, 9 Jan 2025 09:29:57 +0000 Subject: [PATCH 042/110] Add additional tests for operators over incompatible nhcb (#15787) * Add additional tests for operators over incompatible nhcb Signed-off-by: Fiona Liao --- .../testdata/native_histograms.test | 33 +++++++++++++++++++ 1 file changed, 33 insertions(+) diff --git a/promql/promqltest/testdata/native_histograms.test b/promql/promqltest/testdata/native_histograms.test index 6be298cf7d..414619d5cd 100644 --- a/promql/promqltest/testdata/native_histograms.test +++ b/promql/promqltest/testdata/native_histograms.test @@ -1128,6 +1128,39 @@ eval_warn range from 0 to 12m step 6m sum(metric) eval_warn range from 0 to 12m step 6m avg(metric) {} _ {{schema:-53 sum:1 count:1 custom_values:[5 10] buckets:[1]}} _ +# Test incompatible schemas with additional aggregation operators +eval range from 0 to 12m step 6m count(metric) + {} 2 2 3 + +eval range from 0 to 12m step 6m group(metric) + {} 1 1 1 + +eval range from 0 to 12m step 6m count(limitk(1, metric)) + {} 1 1 1 + +eval range from 0 to 12m step 6m limitk(3, metric) + metric{series="1"} _ {{schema:-53 sum:1 count:1 custom_values:[5 10] buckets:[1]}} {{schema:-53 sum:1 count:1 custom_values:[5 10] buckets:[1]}} + metric{series="2"} {{schema:-53 sum:1 count:1 custom_values:[2] buckets:[1]}} _ {{schema:-53 sum:1 count:1 custom_values:[2] buckets:[1]}} + metric{series="3"} {{schema:-53 sum:1 count:1 custom_values:[5 10] buckets:[1]}} {{schema:-53 sum:1 count:1 custom_values:[5 10] buckets:[1]}} {{schema:-53 sum:1 count:1 custom_values:[5 10] buckets:[1]}} + +eval range from 0 to 12m step 6m limit_ratio(1, metric) + metric{series="1"} _ {{schema:-53 sum:1 count:1 custom_values:[5 10] buckets:[1]}} {{schema:-53 sum:1 count:1 custom_values:[5 10] buckets:[1]}} + metric{series="2"} {{schema:-53 sum:1 count:1 custom_values:[2] buckets:[1]}} _ {{schema:-53 sum:1 count:1 custom_values:[2] buckets:[1]}} + metric{series="3"} {{schema:-53 sum:1 count:1 custom_values:[5 10] buckets:[1]}} {{schema:-53 sum:1 count:1 custom_values:[5 10] buckets:[1]}} {{schema:-53 sum:1 count:1 custom_values:[5 10] buckets:[1]}} + +# Test incompatible schemas with and/or +eval range from 0 to 12m step 6m metric{series="1"} and ignoring(series) metric{series="2"} + metric{series="1"} _ _ {{schema:-53 sum:1 count:1 custom_values:[5 10] buckets:[1]}} + +eval range from 0 to 12m step 6m metric{series="1"} or ignoring(series) metric{series="2"} + metric{series="1"} _ {{schema:-53 sum:1 count:1 custom_values:[5 10] buckets:[1]}} {{schema:-53 sum:1 count:1 custom_values:[5 10] buckets:[1]}} + metric{series="2"} {{schema:-53 sum:1 count:1 custom_values:[2] buckets:[1]}} _ _ + +# Test incompatible schemas with arithmetic binary operators +eval_warn range from 0 to 12m step 6m metric{series="2"} + ignoring (series) metric{series="3"} + +eval_warn range from 0 to 12m step 6m metric{series="2"} - ignoring (series) metric{series="3"} + clear load 1m From b3e30d52cecd2cb6b34c9ca74633fa6c13da0d24 Mon Sep 17 00:00:00 2001 From: Neeraj Gartia <80708727+NeerajGartia21@users.noreply.github.com> Date: Thu, 9 Jan 2025 21:08:42 +0530 Subject: [PATCH 043/110] [BUGFIX] PromQL: Fix `` functions with histograms (#15711) fix aggr_over_time with histograms Signed-off-by: Neeraj Gartia --------- Signed-off-by: Neeraj Gartia --- promql/functions.go | 75 +++++++++++++--------- promql/promqltest/testdata/functions.test | 78 +++++++++++++++++++---- 2 files changed, 111 insertions(+), 42 deletions(-) diff --git a/promql/functions.go b/promql/functions.go index 5f31a3db18..2d809571d4 100644 --- a/promql/functions.go +++ b/promql/functions.go @@ -691,9 +691,15 @@ func funcLastOverTime(vals []parser.Value, args parser.Expressions, enh *EvalNod // === mad_over_time(Matrix parser.ValueTypeMatrix) (Vector, Annotations) === func funcMadOverTime(vals []parser.Value, args parser.Expressions, enh *EvalNodeHelper) (Vector, annotations.Annotations) { - if len(vals[0].(Matrix)[0].Floats) == 0 { + samples := vals[0].(Matrix)[0] + var annos annotations.Annotations + if len(samples.Floats) == 0 { return enh.Out, nil } + if len(samples.Histograms) > 0 { + metricName := samples.Metric.Get(labels.MetricName) + annos.Add(annotations.NewHistogramIgnoredInMixedRangeInfo(metricName, args[0].PositionRange())) + } return aggrOverTime(vals, enh, func(s Series) float64 { values := make(vectorByValueHeap, 0, len(s.Floats)) for _, f := range s.Floats { @@ -705,18 +711,20 @@ func funcMadOverTime(vals []parser.Value, args parser.Expressions, enh *EvalNode values = append(values, Sample{F: math.Abs(f.F - median)}) } return quantile(0.5, values) - }), nil + }), annos } // === max_over_time(Matrix parser.ValueTypeMatrix) (Vector, Annotations) === func funcMaxOverTime(vals []parser.Value, args parser.Expressions, enh *EvalNodeHelper) (Vector, annotations.Annotations) { - if len(vals[0].(Matrix)[0].Floats) == 0 { - // TODO(beorn7): The passed values only contain - // histograms. max_over_time ignores histograms for now. If - // there are only histograms, we have to return without adding - // anything to enh.Out. + samples := vals[0].(Matrix)[0] + var annos annotations.Annotations + if len(samples.Floats) == 0 { return enh.Out, nil } + if len(samples.Histograms) > 0 { + metricName := samples.Metric.Get(labels.MetricName) + annos.Add(annotations.NewHistogramIgnoredInMixedRangeInfo(metricName, args[0].PositionRange())) + } return aggrOverTime(vals, enh, func(s Series) float64 { maxVal := s.Floats[0].F for _, f := range s.Floats { @@ -725,18 +733,20 @@ func funcMaxOverTime(vals []parser.Value, args parser.Expressions, enh *EvalNode } } return maxVal - }), nil + }), annos } // === min_over_time(Matrix parser.ValueTypeMatrix) (Vector, Annotations) === func funcMinOverTime(vals []parser.Value, args parser.Expressions, enh *EvalNodeHelper) (Vector, annotations.Annotations) { - if len(vals[0].(Matrix)[0].Floats) == 0 { - // TODO(beorn7): The passed values only contain - // histograms. min_over_time ignores histograms for now. If - // there are only histograms, we have to return without adding - // anything to enh.Out. + samples := vals[0].(Matrix)[0] + var annos annotations.Annotations + if len(samples.Floats) == 0 { return enh.Out, nil } + if len(samples.Histograms) > 0 { + metricName := samples.Metric.Get(labels.MetricName) + annos.Add(annotations.NewHistogramIgnoredInMixedRangeInfo(metricName, args[0].PositionRange())) + } return aggrOverTime(vals, enh, func(s Series) float64 { minVal := s.Floats[0].F for _, f := range s.Floats { @@ -745,7 +755,7 @@ func funcMinOverTime(vals []parser.Value, args parser.Expressions, enh *EvalNode } } return minVal - }), nil + }), annos } // === sum_over_time(Matrix parser.ValueTypeMatrix) (Vector, Annotations) === @@ -794,10 +804,6 @@ func funcQuantileOverTime(vals []parser.Value, args parser.Expressions, enh *Eva q := vals[0].(Vector)[0].F el := vals[1].(Matrix)[0] if len(el.Floats) == 0 { - // TODO(beorn7): The passed values only contain - // histograms. quantile_over_time ignores histograms for now. If - // there are only histograms, we have to return without adding - // anything to enh.Out. return enh.Out, nil } @@ -805,7 +811,10 @@ func funcQuantileOverTime(vals []parser.Value, args parser.Expressions, enh *Eva if math.IsNaN(q) || q < 0 || q > 1 { annos.Add(annotations.NewInvalidQuantileWarning(q, args[0].PositionRange())) } - + if len(el.Histograms) > 0 { + metricName := el.Metric.Get(labels.MetricName) + annos.Add(annotations.NewHistogramIgnoredInAggregationInfo(metricName, args[0].PositionRange())) + } values := make(vectorByValueHeap, 0, len(el.Floats)) for _, f := range el.Floats { values = append(values, Sample{F: f.F}) @@ -815,13 +824,15 @@ func funcQuantileOverTime(vals []parser.Value, args parser.Expressions, enh *Eva // === stddev_over_time(Matrix parser.ValueTypeMatrix) (Vector, Annotations) === func funcStddevOverTime(vals []parser.Value, args parser.Expressions, enh *EvalNodeHelper) (Vector, annotations.Annotations) { - if len(vals[0].(Matrix)[0].Floats) == 0 { - // TODO(beorn7): The passed values only contain - // histograms. stddev_over_time ignores histograms for now. If - // there are only histograms, we have to return without adding - // anything to enh.Out. + samples := vals[0].(Matrix)[0] + var annos annotations.Annotations + if len(samples.Floats) == 0 { return enh.Out, nil } + if len(samples.Histograms) > 0 { + metricName := samples.Metric.Get(labels.MetricName) + annos.Add(annotations.NewHistogramIgnoredInMixedRangeInfo(metricName, args[0].PositionRange())) + } return aggrOverTime(vals, enh, func(s Series) float64 { var count float64 var mean, cMean float64 @@ -833,18 +844,20 @@ func funcStddevOverTime(vals []parser.Value, args parser.Expressions, enh *EvalN aux, cAux = kahanSumInc(delta*(f.F-(mean+cMean)), aux, cAux) } return math.Sqrt((aux + cAux) / count) - }), nil + }), annos } // === stdvar_over_time(Matrix parser.ValueTypeMatrix) (Vector, Annotations) === func funcStdvarOverTime(vals []parser.Value, args parser.Expressions, enh *EvalNodeHelper) (Vector, annotations.Annotations) { - if len(vals[0].(Matrix)[0].Floats) == 0 { - // TODO(beorn7): The passed values only contain - // histograms. stdvar_over_time ignores histograms for now. If - // there are only histograms, we have to return without adding - // anything to enh.Out. + samples := vals[0].(Matrix)[0] + var annos annotations.Annotations + if len(samples.Floats) == 0 { return enh.Out, nil } + if len(samples.Histograms) > 0 { + metricName := samples.Metric.Get(labels.MetricName) + annos.Add(annotations.NewHistogramIgnoredInMixedRangeInfo(metricName, args[0].PositionRange())) + } return aggrOverTime(vals, enh, func(s Series) float64 { var count float64 var mean, cMean float64 @@ -856,7 +869,7 @@ func funcStdvarOverTime(vals []parser.Value, args parser.Expressions, enh *EvalN aux, cAux = kahanSumInc(delta*(f.F-(mean+cMean)), aux, cAux) } return (aux + cAux) / count - }), nil + }), annos } // === absent(Vector parser.ValueTypeVector) (Vector, Annotations) === diff --git a/promql/promqltest/testdata/functions.test b/promql/promqltest/testdata/functions.test index 6d2ade3abc..7fc636450f 100644 --- a/promql/promqltest/testdata/functions.test +++ b/promql/promqltest/testdata/functions.test @@ -929,35 +929,58 @@ eval instant at 1m avg_over_time(metric[2m]) # Tests for stddev_over_time and stdvar_over_time. clear load 10s - metric 0 8 8 2 3 + metric 0 8 8 2 3 + metric_histogram{type="only_histogram"} {{schema:1 sum:2 count:3}}x5 + metric_histogram{type="mix"} 1 1 1 {{schema:1 sum:2 count:3}} {{schema:1 sum:2 count:3}} eval instant at 1m stdvar_over_time(metric[2m]) - {} 10.56 + {} 10.56 eval instant at 1m stddev_over_time(metric[2m]) - {} 3.249615 + {} 3.249615 eval instant at 1m stddev_over_time((metric[2m])) - {} 3.249615 + {} 3.249615 + +# Tests for stddev_over_time and stdvar_over_time with histograms. +eval instant at 1m stddev_over_time(metric_histogram{type="only_histogram"}[2m]) + #empty + +eval_info instant at 1m stddev_over_time(metric_histogram{type="mix"}[2m]) + {type="mix"} 0 + +eval instant at 1m stdvar_over_time(metric_histogram{type="only_histogram"}[2m]) + #empty + +eval_info instant at 1m stdvar_over_time(metric_histogram{type="mix"}[2m]) + {type="mix"} 0 # Tests for stddev_over_time and stdvar_over_time #4927. clear load 10s - metric 1.5990505637277868 1.5990505637277868 1.5990505637277868 + metric 1.5990505637277868 1.5990505637277868 1.5990505637277868 eval instant at 1m stdvar_over_time(metric[1m]) - {} 0 + {} 0 eval instant at 1m stddev_over_time(metric[1m]) - {} 0 + {} 0 # Tests for mad_over_time. clear load 10s - metric 4 6 2 1 999 1 2 + metric 4 6 2 1 999 1 2 + metric_histogram{type="only_histogram"} {{schema:1 sum:2 count:3}}x5 + metric_histogram{type="mix"} 1 1 1 {{schema:1 sum:2 count:3}} {{schema:1 sum:2 count:3}} eval instant at 70s mad_over_time(metric[70s]) - {} 1 + {} 1 + +eval instant at 70s mad_over_time(metric_histogram{type="only_histogram"}[70s]) + #empty + +eval_info instant at 70s mad_over_time(metric_histogram{type="mix"}[70s]) + {type="mix"} 0 # Tests for quantile_over_time clear @@ -966,6 +989,8 @@ load 10s data{test="two samples"} 0 1 data{test="three samples"} 0 1 2 data{test="uneven samples"} 0 1 4 + data_histogram{test="only histogram samples"} {{schema:0 sum:1 count:2}}x4 + data_histogram{test="mix samples"} 0 1 2 {{schema:0 sum:1 count:2}}x2 eval instant at 1m quantile_over_time(0, data[2m]) {test="two samples"} 0 @@ -1007,6 +1032,12 @@ eval_warn instant at 1m (quantile_over_time(2, (data[2m]))) {test="three samples"} +Inf {test="uneven samples"} +Inf +eval instant at 1m quantile_over_time(0.5, data_histogram{test="only histogram samples"}[2m]) + #empty + +eval_info instant at 1m quantile_over_time(0.5, data_histogram{test="mix samples"}[2m]) + {test="mix samples"} 1 + clear # Test time-related functions. @@ -1120,15 +1151,17 @@ load 5m eval_fail instant at 0m changes({__name__=~'testmetric1|testmetric2'}[5m]) -# Tests for *_over_time clear +# Tests for *_over_time load 10s data{type="numbers"} 2 0 3 data{type="some_nan"} 2 0 NaN data{type="some_nan2"} 2 NaN 1 data{type="some_nan3"} NaN 0 1 data{type="only_nan"} NaN NaN NaN + data_histogram{type="only_histogram"} {{schema:0 sum:1 count:2}} {{schema:0 sum:2 count:3}} {{schema:0 sum:3 count:4}} + data_histogram{type="mix_samples"} 0 1 {{schema:0 sum:1 count:2}} {{schema:0 sum:2 count:3}} eval instant at 1m min_over_time(data[2m]) {type="numbers"} 0 @@ -1137,6 +1170,12 @@ eval instant at 1m min_over_time(data[2m]) {type="some_nan3"} 0 {type="only_nan"} NaN +eval instant at 1m min_over_time(data_histogram{type="only_histogram"}[2m]) + #empty + +eval_info instant at 1m min_over_time(data_histogram{type="mix_samples"}[2m]) + {type="mix_samples"} 0 + eval instant at 1m max_over_time(data[2m]) {type="numbers"} 3 {type="some_nan"} 2 @@ -1144,12 +1183,29 @@ eval instant at 1m max_over_time(data[2m]) {type="some_nan3"} 1 {type="only_nan"} NaN -eval instant at 1m last_over_time(data[2m]) +eval instant at 1m max_over_time(data_histogram{type="only_histogram"}[2m]) + #empty + +eval_info instant at 1m max_over_time(data_histogram{type="mix_samples"}[2m]) + {type="mix_samples"} 1 + +eval instant at 1m last_over_time({__name__=~"data(_histogram)?"}[2m]) data{type="numbers"} 3 data{type="some_nan"} NaN data{type="some_nan2"} 1 data{type="some_nan3"} 1 data{type="only_nan"} NaN + data_histogram{type="only_histogram"} {{schema:0 sum:3 count:4}} + data_histogram{type="mix_samples"} {{schema:0 sum:2 count:3}} + +eval instant at 1m count_over_time({__name__=~"data(_histogram)?"}[2m]) + {type="numbers"} 3 + {type="some_nan"} 3 + {type="some_nan2"} 3 + {type="some_nan3"} 3 + {type="only_nan"} 3 + {type="only_histogram"} 3 + {type="mix_samples"} 4 clear From 6339989e25102f37a57030f4338b4850c8c5b30e Mon Sep 17 00:00:00 2001 From: Vandit Singh <107131545+Vandit1604@users.noreply.github.com> Date: Thu, 9 Jan 2025 21:57:39 +0530 Subject: [PATCH 044/110] web/api: Add a limit parameter to /query and /query_range (#15552) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit add limit param to query and rangeQuery --------- Signed-off-by: Vandit Singh Signed-off-by: Vandit Singh <107131545+Vandit1604@users.noreply.github.com> Co-authored-by: Björn Rabenstein --- docs/querying/api.md | 2 + web/api/v1/api.go | 53 +++++++++- web/api/v1/api_test.go | 228 ++++++++++++++++++++++++++++++++++++++++- 3 files changed, 280 insertions(+), 3 deletions(-) diff --git a/docs/querying/api.md b/docs/querying/api.md index f1e7129303..e3f97886dc 100644 --- a/docs/querying/api.md +++ b/docs/querying/api.md @@ -86,6 +86,7 @@ URL query parameters: - `time=`: Evaluation timestamp. Optional. - `timeout=`: Evaluation timeout. Optional. Defaults to and is capped by the value of the `-query.timeout` flag. +- `limit=`: Maximum number of returned series. Doesn’t affect scalars or strings but truncates the number of series for matrices and vectors. Optional. 0 means disabled. The current server time is used if the `time` parameter is omitted. @@ -154,6 +155,7 @@ URL query parameters: - `step=`: Query resolution step width in `duration` format or float number of seconds. - `timeout=`: Evaluation timeout. Optional. Defaults to and is capped by the value of the `-query.timeout` flag. +- `limit=`: Maximum number of returned series. Optional. 0 means disabled. You can URL-encode these parameters directly in the request body by using the `POST` method and `Content-Type: application/x-www-form-urlencoded` header. This is useful when specifying a large diff --git a/web/api/v1/api.go b/web/api/v1/api.go index 6e9c589087..4903f925cc 100644 --- a/web/api/v1/api.go +++ b/web/api/v1/api.go @@ -438,6 +438,10 @@ func (api *API) options(*http.Request) apiFuncResult { } func (api *API) query(r *http.Request) (result apiFuncResult) { + limit, err := parseLimitParam(r.FormValue("limit")) + if err != nil { + return invalidParamError(err, "limit") + } ts, err := parseTimeParam(r, "time", api.now()) if err != nil { return invalidParamError(err, "time") @@ -479,6 +483,15 @@ func (api *API) query(r *http.Request) (result apiFuncResult) { return apiFuncResult{nil, returnAPIError(res.Err), res.Warnings, qry.Close} } + warnings := res.Warnings + if limit > 0 { + var isTruncated bool + + res, isTruncated = truncateResults(res, limit) + if isTruncated { + warnings = warnings.Add(errors.New("results truncated due to limit")) + } + } // Optional stats field in response if parameter "stats" is not empty. sr := api.statsRenderer if sr == nil { @@ -490,7 +503,7 @@ func (api *API) query(r *http.Request) (result apiFuncResult) { ResultType: res.Value.Type(), Result: res.Value, Stats: qs, - }, nil, res.Warnings, qry.Close} + }, nil, warnings, qry.Close} } func (api *API) formatQuery(r *http.Request) (result apiFuncResult) { @@ -526,6 +539,10 @@ func extractQueryOpts(r *http.Request) (promql.QueryOpts, error) { } func (api *API) queryRange(r *http.Request) (result apiFuncResult) { + limit, err := parseLimitParam(r.FormValue("limit")) + if err != nil { + return invalidParamError(err, "limit") + } start, err := parseTime(r.FormValue("start")) if err != nil { return invalidParamError(err, "start") @@ -590,6 +607,16 @@ func (api *API) queryRange(r *http.Request) (result apiFuncResult) { return apiFuncResult{nil, returnAPIError(res.Err), res.Warnings, qry.Close} } + warnings := res.Warnings + if limit > 0 { + var isTruncated bool + + res, isTruncated = truncateResults(res, limit) + if isTruncated { + warnings = warnings.Add(errors.New("results truncated due to limit")) + } + } + // Optional stats field in response if parameter "stats" is not empty. sr := api.statsRenderer if sr == nil { @@ -601,7 +628,7 @@ func (api *API) queryRange(r *http.Request) (result apiFuncResult) { ResultType: res.Value.Type(), Result: res.Value, Stats: qs, - }, nil, res.Warnings, qry.Close} + }, nil, warnings, qry.Close} } func (api *API) queryExemplars(r *http.Request) apiFuncResult { @@ -2102,3 +2129,25 @@ func toHintLimit(limit int) int { } return limit } + +// truncateResults truncates result for queryRange() and query(). +// No truncation for other types(Scalars or Strings). +func truncateResults(result *promql.Result, limit int) (*promql.Result, bool) { + isTruncated := false + + switch v := result.Value.(type) { + case promql.Matrix: + if len(v) > limit { + result.Value = v[:limit] + isTruncated = true + } + case promql.Vector: + if len(v) > limit { + result.Value = v[:limit] + isTruncated = true + } + } + + // Return the modified result. Unchanged for other types. + return result, isTruncated +} diff --git a/web/api/v1/api_test.go b/web/api/v1/api_test.go index 175ed2e0f0..e6ca43508b 100644 --- a/web/api/v1/api_test.go +++ b/web/api/v1/api_test.go @@ -1164,6 +1164,49 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E }, }, }, + // Only matrix and vector responses are limited/truncated. String and scalar responses aren't truncated. + { + endpoint: api.query, + query: url.Values{ + "query": []string{"2"}, + "time": []string{"123.4"}, + "limit": []string{"1"}, + }, + response: &QueryData{ + ResultType: parser.ValueTypeScalar, + Result: promql.Scalar{ + V: 2, + T: timestamp.FromTime(start.Add(123*time.Second + 400*time.Millisecond)), + }, + }, + warningsCount: 0, + }, + // When limit = 0, limit is disabled. + { + endpoint: api.query, + query: url.Values{ + "query": []string{"2"}, + "time": []string{"123.4"}, + "limit": []string{"0"}, + }, + response: &QueryData{ + ResultType: parser.ValueTypeScalar, + Result: promql.Scalar{ + V: 2, + T: timestamp.FromTime(start.Add(123*time.Second + 400*time.Millisecond)), + }, + }, + warningsCount: 0, + }, + { + endpoint: api.query, + query: url.Values{ + "query": []string{"2"}, + "time": []string{"123.4"}, + "limit": []string{"-1"}, + }, + errType: errorBadData, + }, { endpoint: api.query, query: url.Values{ @@ -1205,6 +1248,179 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E }, }, }, + { + endpoint: api.query, + query: url.Values{ + "query": []string{ + `label_replace(vector(42), "foo", "bar", "", "") or label_replace(vector(3.1415), "dings", "bums", "", "")`, + }, + "time": []string{"123.4"}, + "limit": []string{"2"}, + }, + warningsCount: 0, + responseAsJSON: `{ + "resultType": "vector", + "result": [ + { + "metric": { + "foo": "bar" + }, + "value": [123.4, "42"] + }, + { + "metric": { + "dings": "bums" + }, + "value": [123.4, "3.1415"] + } + ] + }`, + }, + { + endpoint: api.query, + query: url.Values{ + "query": []string{ + `label_replace(vector(42), "foo", "bar", "", "") or label_replace(vector(3.1415), "dings", "bums", "", "")`, + }, + "time": []string{"123.4"}, + "limit": []string{"1"}, + }, + warningsCount: 1, + responseAsJSON: `{ + "resultType": "vector", + "result": [ + { + "metric": { + "foo": "bar" + }, + "value": [123.4, "42"] + } + ] + }`, + }, + { + endpoint: api.query, + query: url.Values{ + "query": []string{ + `label_replace(vector(42), "foo", "bar", "", "") or label_replace(vector(3.1415), "dings", "bums", "", "")`, + }, + "time": []string{"123.4"}, + "limit": []string{"0"}, + }, + responseAsJSON: `{ + "resultType": "vector", + "result": [ + { + "metric": { + "foo": "bar" + }, + "value": [123.4, "42"] + }, + { + "metric": { + "dings": "bums" + }, + "value": [123.4, "3.1415"] + } + ] + }`, + warningsCount: 0, + }, + // limit=0 means no limit. + { + endpoint: api.queryRange, + query: url.Values{ + "query": []string{ + `label_replace(vector(42), "foo", "bar", "", "") or label_replace(vector(3.1415), "dings", "bums", "", "")`, + }, + "start": []string{"0"}, + "end": []string{"2"}, + "step": []string{"1"}, + "limit": []string{"0"}, + }, + response: &QueryData{ + ResultType: parser.ValueTypeMatrix, + Result: promql.Matrix{ + promql.Series{ + Metric: labels.FromMap(map[string]string{"dings": "bums"}), + Floats: []promql.FPoint{ + {F: 3.1415, T: timestamp.FromTime(start)}, + {F: 3.1415, T: timestamp.FromTime(start.Add(1 * time.Second))}, + {F: 3.1415, T: timestamp.FromTime(start.Add(2 * time.Second))}, + }, + }, + promql.Series{ + Metric: labels.FromMap(map[string]string{"foo": "bar"}), + Floats: []promql.FPoint{ + {F: 42, T: timestamp.FromTime(start)}, + {F: 42, T: timestamp.FromTime(start.Add(1 * time.Second))}, + {F: 42, T: timestamp.FromTime(start.Add(2 * time.Second))}, + }, + }, + }, + }, + warningsCount: 0, + }, + { + endpoint: api.queryRange, + query: url.Values{ + "query": []string{ + `label_replace(vector(42), "foo", "bar", "", "") or label_replace(vector(3.1415), "dings", "bums", "", "")`, + }, + "start": []string{"0"}, + "end": []string{"2"}, + "step": []string{"1"}, + "limit": []string{"1"}, + }, + response: &QueryData{ + ResultType: parser.ValueTypeMatrix, + Result: promql.Matrix{ + promql.Series{ + Metric: labels.FromMap(map[string]string{"dings": "bums"}), + Floats: []promql.FPoint{ + {F: 3.1415, T: timestamp.FromTime(start)}, + {F: 3.1415, T: timestamp.FromTime(start.Add(1 * time.Second))}, + {F: 3.1415, T: timestamp.FromTime(start.Add(2 * time.Second))}, + }, + }, + }, + }, + warningsCount: 1, + }, + { + endpoint: api.queryRange, + query: url.Values{ + "query": []string{ + `label_replace(vector(42), "foo", "bar", "", "") or label_replace(vector(3.1415), "dings", "bums", "", "")`, + }, + "start": []string{"0"}, + "end": []string{"2"}, + "step": []string{"1"}, + "limit": []string{"2"}, + }, + response: &QueryData{ + ResultType: parser.ValueTypeMatrix, + Result: promql.Matrix{ + promql.Series{ + Metric: labels.FromMap(map[string]string{"dings": "bums"}), + Floats: []promql.FPoint{ + {F: 3.1415, T: timestamp.FromTime(start)}, + {F: 3.1415, T: timestamp.FromTime(start.Add(1 * time.Second))}, + {F: 3.1415, T: timestamp.FromTime(start.Add(2 * time.Second))}, + }, + }, + promql.Series{ + Metric: labels.FromMap(map[string]string{"foo": "bar"}), + Floats: []promql.FPoint{ + {F: 42, T: timestamp.FromTime(start)}, + {F: 42, T: timestamp.FromTime(start.Add(1 * time.Second))}, + {F: 42, T: timestamp.FromTime(start.Add(2 * time.Second))}, + }, + }, + }, + }, + warningsCount: 0, + }, { endpoint: api.queryRange, query: url.Values{ @@ -1222,7 +1438,6 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E {F: 1, T: timestamp.FromTime(start.Add(1 * time.Second))}, {F: 2, T: timestamp.FromTime(start.Add(2 * time.Second))}, }, - // No Metric returned - use zero value for comparison. }, }, }, @@ -1235,6 +1450,17 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E }, responseAsJSON: `{"resultType":"vector","result":[]}`, }, + { + endpoint: api.queryRange, + query: url.Values{ + "query": []string{"bottomk(2, notExists)"}, + "start": []string{"0"}, + "end": []string{"2"}, + "step": []string{"1"}, + "limit": []string{"-1"}, + }, + errType: errorBadData, + }, // Test empty matrix result { endpoint: api.queryRange, From f030894c2cd124923030bd2cfa8a7b91a00544d6 Mon Sep 17 00:00:00 2001 From: Arve Knudsen Date: Thu, 9 Jan 2025 17:51:26 +0100 Subject: [PATCH 045/110] Fix issues raised by staticcheck (#15722) Fix issues raised by staticcheck We are not enabling staticcheck explicitly, though, because it has too many false positives. --------- Signed-off-by: Arve Knudsen --- cmd/prometheus/main.go | 6 +++--- cmd/promtool/main.go | 9 ++++----- promql/promqltest/test.go | 4 ++-- scrape/target.go | 4 ++-- storage/remote/metadata_watcher.go | 2 +- storage/remote/queue_manager.go | 2 +- tsdb/wlog/watcher.go | 2 +- web/api/v1/api.go | 2 +- web/api/v1/api_test.go | 2 +- 9 files changed, 16 insertions(+), 17 deletions(-) diff --git a/cmd/prometheus/main.go b/cmd/prometheus/main.go index 06f46f8d72..03c20dc52d 100644 --- a/cmd/prometheus/main.go +++ b/cmd/prometheus/main.go @@ -534,7 +534,7 @@ func main() { _, err := a.Parse(os.Args[1:]) if err != nil { - fmt.Fprintln(os.Stderr, fmt.Errorf("Error parsing command line arguments: %w", err)) + fmt.Fprintf(os.Stderr, "Error parsing command line arguments: %s\n", err) a.Usage(os.Args[1:]) os.Exit(2) } @@ -548,7 +548,7 @@ func main() { notifs.AddNotification(notifications.StartingUp) if err := cfg.setFeatureListOptions(logger); err != nil { - fmt.Fprintln(os.Stderr, fmt.Errorf("Error parsing feature list: %w", err)) + fmt.Fprintf(os.Stderr, "Error parsing feature list: %s\n", err) os.Exit(1) } @@ -1742,7 +1742,7 @@ func (s *readyStorage) WALReplayStatus() (tsdb.WALReplayStatus, error) { } // ErrNotReady is returned if the underlying scrape manager is not ready yet. -var ErrNotReady = errors.New("Scrape manager not ready") +var ErrNotReady = errors.New("scrape manager not ready") // ReadyScrapeManager allows a scrape manager to be retrieved. Even if it's set at a later point in time. type readyScrapeManager struct { diff --git a/cmd/promtool/main.go b/cmd/promtool/main.go index b52fe7cdbb..62a1d4f906 100644 --- a/cmd/promtool/main.go +++ b/cmd/promtool/main.go @@ -36,7 +36,7 @@ import ( "github.com/prometheus/client_golang/api" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/testutil/promlint" - config_util "github.com/prometheus/common/config" + "github.com/prometheus/common/expfmt" "github.com/prometheus/common/model" "github.com/prometheus/common/promslog" "github.com/prometheus/common/version" @@ -45,7 +45,6 @@ import ( dto "github.com/prometheus/client_model/go" promconfig "github.com/prometheus/common/config" - "github.com/prometheus/common/expfmt" "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/discovery" @@ -312,12 +311,12 @@ func main() { kingpin.Fatalf("Cannot set base auth in the server URL and use a http.config.file at the same time") } var err error - httpConfig, _, err := config_util.LoadHTTPConfigFile(httpConfigFilePath) + httpConfig, _, err := promconfig.LoadHTTPConfigFile(httpConfigFilePath) if err != nil { kingpin.Fatalf("Failed to load HTTP config file: %v", err) } - httpRoundTripper, err = promconfig.NewRoundTripperFromConfig(*httpConfig, "promtool", config_util.WithUserAgent("promtool/"+version.Version)) + httpRoundTripper, err = promconfig.NewRoundTripperFromConfig(*httpConfig, "promtool", promconfig.WithUserAgent("promtool/"+version.Version)) if err != nil { kingpin.Fatalf("Failed to create a new HTTP round tripper: %v", err) } @@ -702,7 +701,7 @@ func checkConfig(agentMode bool, filename string, checkSyntaxOnly bool) ([]strin return ruleFiles, nil } -func checkTLSConfig(tlsConfig config_util.TLSConfig, checkSyntaxOnly bool) error { +func checkTLSConfig(tlsConfig promconfig.TLSConfig, checkSyntaxOnly bool) error { if len(tlsConfig.CertFile) > 0 && len(tlsConfig.KeyFile) == 0 { return fmt.Errorf("client cert file %q specified without client key file", tlsConfig.CertFile) } diff --git a/promql/promqltest/test.go b/promql/promqltest/test.go index 518164827a..5e0d9083cb 100644 --- a/promql/promqltest/test.go +++ b/promql/promqltest/test.go @@ -1419,8 +1419,8 @@ func (ll *LazyLoader) appendTill(ts int64) error { // WithSamplesTill loads the samples till given timestamp and executes the given function. func (ll *LazyLoader) WithSamplesTill(ts time.Time, fn func(error)) { - tsMilli := ts.Sub(time.Unix(0, 0).UTC()) / time.Millisecond - fn(ll.appendTill(int64(tsMilli))) + till := ts.Sub(time.Unix(0, 0).UTC()) / time.Millisecond + fn(ll.appendTill(int64(till))) } // QueryEngine returns the LazyLoader's query engine. diff --git a/scrape/target.go b/scrape/target.go index d05866f863..22cde01c05 100644 --- a/scrape/target.go +++ b/scrape/target.go @@ -295,12 +295,12 @@ func (t *Target) intervalAndTimeout(defaultInterval, defaultDuration time.Durati intervalLabel := t.labels.Get(model.ScrapeIntervalLabel) interval, err := model.ParseDuration(intervalLabel) if err != nil { - return defaultInterval, defaultDuration, fmt.Errorf("Error parsing interval label %q: %w", intervalLabel, err) + return defaultInterval, defaultDuration, fmt.Errorf("error parsing interval label %q: %w", intervalLabel, err) } timeoutLabel := t.labels.Get(model.ScrapeTimeoutLabel) timeout, err := model.ParseDuration(timeoutLabel) if err != nil { - return defaultInterval, defaultDuration, fmt.Errorf("Error parsing timeout label %q: %w", timeoutLabel, err) + return defaultInterval, defaultDuration, fmt.Errorf("error parsing timeout label %q: %w", timeoutLabel, err) } return time.Duration(interval), time.Duration(timeout), nil diff --git a/storage/remote/metadata_watcher.go b/storage/remote/metadata_watcher.go index 9306dcb4c2..d7f376c96a 100644 --- a/storage/remote/metadata_watcher.go +++ b/storage/remote/metadata_watcher.go @@ -38,7 +38,7 @@ type Watchable interface { type noopScrapeManager struct{} func (noop *noopScrapeManager) Get() (*scrape.Manager, error) { - return nil, errors.New("Scrape manager not ready") + return nil, errors.New("scrape manager not ready") } // MetadataWatcher watches the Scrape Manager for a given WriteMetadataTo. diff --git a/storage/remote/queue_manager.go b/storage/remote/queue_manager.go index 475c126eff..4b966059f6 100644 --- a/storage/remote/queue_manager.go +++ b/storage/remote/queue_manager.go @@ -2119,7 +2119,7 @@ func compressPayload(tmpbuf *[]byte, inp []byte, enc Compression) (compressed [] } return compressed, nil default: - return compressed, fmt.Errorf("Unknown compression scheme [%v]", enc) + return compressed, fmt.Errorf("unknown compression scheme [%v]", enc) } } diff --git a/tsdb/wlog/watcher.go b/tsdb/wlog/watcher.go index 6f1bc1df35..ca74a9ceaf 100644 --- a/tsdb/wlog/watcher.go +++ b/tsdb/wlog/watcher.go @@ -679,7 +679,7 @@ func (w *Watcher) readCheckpoint(checkpointDir string, readFn segmentReadFn) err // Ensure we read the whole contents of every segment in the checkpoint dir. segs, err := listSegments(checkpointDir) if err != nil { - return fmt.Errorf("Unable to get segments checkpoint dir: %w", err) + return fmt.Errorf("unable to get segments checkpoint dir: %w", err) } for _, segRef := range segs { size, err := getSegmentSize(checkpointDir, segRef.index) diff --git a/web/api/v1/api.go b/web/api/v1/api.go index 4903f925cc..ea7d5c5fe4 100644 --- a/web/api/v1/api.go +++ b/web/api/v1/api.go @@ -2043,7 +2043,7 @@ func parseTimeParam(r *http.Request, paramName string, defaultValue time.Time) ( } result, err := parseTime(val) if err != nil { - return time.Time{}, fmt.Errorf("Invalid time value for '%s': %w", paramName, err) + return time.Time{}, fmt.Errorf("invalid time value for '%s': %w", paramName, err) } return result, nil } diff --git a/web/api/v1/api_test.go b/web/api/v1/api_test.go index e6ca43508b..37227d849d 100644 --- a/web/api/v1/api_test.go +++ b/web/api/v1/api_test.go @@ -4186,7 +4186,7 @@ func TestParseTimeParam(t *testing.T) { asTime: time.Time{}, asError: func() error { _, err := parseTime("baz") - return fmt.Errorf("Invalid time value for '%s': %w", "foo", err) + return fmt.Errorf("invalid time value for '%s': %w", "foo", err) }, }, }, From 0a19f1268e2dbb9bae9568f007a9809e8fba0402 Mon Sep 17 00:00:00 2001 From: Julien Duchesne Date: Thu, 9 Jan 2025 18:14:21 -0500 Subject: [PATCH 046/110] Rule Concurrency: Simpler loop for sequential (default) executions (#15801) --- rules/group.go | 50 +++++++++++++++++++++++++++++-------------- rules/manager.go | 6 +----- rules/manager_test.go | 7 +----- 3 files changed, 36 insertions(+), 27 deletions(-) diff --git a/rules/group.go b/rules/group.go index 724b926d4f..4398d9211d 100644 --- a/rules/group.go +++ b/rules/group.go @@ -643,28 +643,46 @@ func (g *Group) Eval(ctx context.Context, ts time.Time) { if ctrl == nil { ctrl = sequentialRuleEvalController{} } - for _, batch := range ctrl.SplitGroupIntoBatches(ctx, g) { - for _, ruleIndex := range batch { + + batches := ctrl.SplitGroupIntoBatches(ctx, g) + if len(batches) == 0 { + // Sequential evaluation when batches aren't set. + // This is the behaviour without a defined RuleConcurrencyController + for i, rule := range g.rules { + // Check if the group has been stopped. select { case <-g.done: return default: } - - rule := g.rules[ruleIndex] - if len(batch) > 1 && ctrl.Allow(ctx, g, rule) { - wg.Add(1) - - go eval(ruleIndex, rule, func() { - wg.Done() - ctrl.Done(ctx) - }) - } else { - eval(ruleIndex, rule, nil) - } + eval(i, rule, nil) + } + } else { + // Concurrent evaluation. + for _, batch := range batches { + for _, ruleIndex := range batch { + // Check if the group has been stopped. + select { + case <-g.done: + wg.Wait() + return + default: + } + rule := g.rules[ruleIndex] + if len(batch) > 1 && ctrl.Allow(ctx, g, rule) { + wg.Add(1) + + 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() } - // It is important that we finish processing any rules in this current batch - before we move into the next one. - wg.Wait() } g.metrics.GroupSamples.WithLabelValues(GroupKey(g.File(), g.Name())).Set(samplesTotal.Load()) diff --git a/rules/manager.go b/rules/manager.go index c4c0f8a1ef..a3ae716e2b 100644 --- a/rules/manager.go +++ b/rules/manager.go @@ -550,11 +550,7 @@ func (c sequentialRuleEvalController) Allow(_ context.Context, _ *Group, _ Rule) } 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 + return nil } func (c sequentialRuleEvalController) Done(_ context.Context) {} diff --git a/rules/manager_test.go b/rules/manager_test.go index 5c3fcc96a8..45da7a44b0 100644 --- a/rules/manager_test.go +++ b/rules/manager_test.go @@ -1989,12 +1989,7 @@ func TestAsyncRuleEvaluation(t *testing.T) { // Expected evaluation order order := group.opts.RuleConcurrencyController.SplitGroupIntoBatches(ctx, group) - require.Equal(t, []ConcurrentRules{ - {0}, - {1}, - {2}, - {3}, - }, order) + require.Nil(t, order) // Never expect more than 1 inflight query at a time. require.EqualValues(t, 1, maxInflight.Load()) From 616914abe22ae745349b1891ba8065c2fd4abf4b Mon Sep 17 00:00:00 2001 From: crystalstall Date: Mon, 6 Jan 2025 16:13:18 +0800 Subject: [PATCH 047/110] Signed-off-by: crystalstall refactor: using slices.Contains to simplify the code Signed-off-by: crystalstall --- discovery/consul/consul.go | 8 ++------ tsdb/block.go | 13 ++----------- util/testutil/port.go | 8 ++------ 3 files changed, 6 insertions(+), 23 deletions(-) diff --git a/discovery/consul/consul.go b/discovery/consul/consul.go index 33b82d23a4..4c8de6e291 100644 --- a/discovery/consul/consul.go +++ b/discovery/consul/consul.go @@ -19,6 +19,7 @@ import ( "fmt" "log/slog" "net" + "slices" "strconv" "strings" "time" @@ -248,12 +249,7 @@ func (d *Discovery) shouldWatchFromName(name string) bool { return true } - for _, sn := range d.watchedServices { - if sn == name { - return true - } - } - return false + return slices.Contains(d.watchedServices, name) } // shouldWatchFromTags returns whether the service of the given name should be watched based on its tags. diff --git a/tsdb/block.go b/tsdb/block.go index 6483f8d8bb..4ffd2463c3 100644 --- a/tsdb/block.go +++ b/tsdb/block.go @@ -221,7 +221,7 @@ type BlockMetaCompaction struct { } func (bm *BlockMetaCompaction) SetOutOfOrder() { - if bm.containsHint(CompactionHintFromOutOfOrder) { + if bm.FromOutOfOrder() { return } bm.Hints = append(bm.Hints, CompactionHintFromOutOfOrder) @@ -229,16 +229,7 @@ func (bm *BlockMetaCompaction) SetOutOfOrder() { } func (bm *BlockMetaCompaction) FromOutOfOrder() bool { - return bm.containsHint(CompactionHintFromOutOfOrder) -} - -func (bm *BlockMetaCompaction) containsHint(hint string) bool { - for _, h := range bm.Hints { - if h == hint { - return true - } - } - return false + return slices.Contains(bm.Hints, CompactionHintFromOutOfOrder) } const ( diff --git a/util/testutil/port.go b/util/testutil/port.go index 7cf4cf1ccc..91c1291749 100644 --- a/util/testutil/port.go +++ b/util/testutil/port.go @@ -15,6 +15,7 @@ package testutil import ( "net" + "slices" "sync" "testing" ) @@ -48,12 +49,7 @@ func RandomUnprivilegedPort(t *testing.T) int { } func portWasUsed(port int) bool { - for _, usedPort := range usedPorts { - if port == usedPort { - return true - } - } - return false + return slices.Contains(usedPorts, port) } func getPort() (int, error) { From 47563d942eb459071b91050be2d1d1dcca73d64b Mon Sep 17 00:00:00 2001 From: Owen Williams Date: Fri, 10 Jan 2025 15:19:45 -0500 Subject: [PATCH 048/110] parser: fix misleading error message in grouping processing Signed-off-by: Owen Williams --- promql/parser/generated_parser.y | 9 +++++---- promql/parser/generated_parser.y.go | 9 +++++---- 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/promql/parser/generated_parser.y b/promql/parser/generated_parser.y index 3865dc6548..ca710b1ab0 100644 --- a/promql/parser/generated_parser.y +++ b/promql/parser/generated_parser.y @@ -363,17 +363,18 @@ grouping_label_list: grouping_label : maybe_label { if !model.LabelName($1.Val).IsValid() { - yylex.(*parser).unexpected("grouping opts", "label") + yylex.(*parser).addParseErrf($1.PositionRange(),"invalid label name for grouping: %q", $1.Val) } $$ = $1 } | STRING { - if !model.LabelName(yylex.(*parser).unquoteString($1.Val)).IsValid() { - yylex.(*parser).unexpected("grouping opts", "label") + unquoted := yylex.(*parser).unquoteString($1.Val) + if !model.LabelName(unquoted).IsValid() { + yylex.(*parser).addParseErrf($1.PositionRange(),"invalid label name for grouping: %q", unquoted) } $$ = $1 $$.Pos++ - $$.Val = yylex.(*parser).unquoteString($$.Val) + $$.Val = unquoted } | error { yylex.(*parser).unexpected("grouping opts", "label"); $$ = Item{} } diff --git a/promql/parser/generated_parser.y.go b/promql/parser/generated_parser.y.go index 7ff8591169..04bc081f2f 100644 --- a/promql/parser/generated_parser.y.go +++ b/promql/parser/generated_parser.y.go @@ -1259,19 +1259,20 @@ yydefault: yyDollar = yyS[yypt-1 : yypt+1] { if !model.LabelName(yyDollar[1].item.Val).IsValid() { - yylex.(*parser).unexpected("grouping opts", "label") + yylex.(*parser).addParseErrf(yyDollar[1].item.PositionRange(), "invalid label name for grouping: %q", yyDollar[1].item.Val) } yyVAL.item = yyDollar[1].item } case 59: yyDollar = yyS[yypt-1 : yypt+1] { - if !model.LabelName(yylex.(*parser).unquoteString(yyDollar[1].item.Val)).IsValid() { - yylex.(*parser).unexpected("grouping opts", "label") + unquoted := yylex.(*parser).unquoteString(yyDollar[1].item.Val) + if !model.LabelName(unquoted).IsValid() { + yylex.(*parser).addParseErrf(yyDollar[1].item.PositionRange(), "invalid label name for grouping: %q", unquoted) } yyVAL.item = yyDollar[1].item yyVAL.item.Pos++ - yyVAL.item.Val = yylex.(*parser).unquoteString(yyVAL.item.Val) + yyVAL.item.Val = unquoted } case 60: yyDollar = yyS[yypt-1 : yypt+1] From 6ccd9add1e023f84589d1ca2e18b8c5256bf47a0 Mon Sep 17 00:00:00 2001 From: leonnicolas Date: Fri, 10 Jan 2025 23:32:05 +0100 Subject: [PATCH 049/110] Make "hide empty rules persistent" It can be a bit annoying to always press "hide empty rules". This commit uses the session storage of the browser to make it persistent. Signed-off-by: leonnicolas --- .../src/components/SettingsMenu.tsx | 54 +++++++++------ web/ui/mantine-ui/src/pages/AlertsPage.tsx | 69 +++++++++++-------- .../src/state/localStorageMiddleware.ts | 5 +- web/ui/mantine-ui/src/state/settingsSlice.ts | 14 ++-- 4 files changed, 89 insertions(+), 53 deletions(-) diff --git a/web/ui/mantine-ui/src/components/SettingsMenu.tsx b/web/ui/mantine-ui/src/components/SettingsMenu.tsx index aae38909d0..79d9c201cc 100644 --- a/web/ui/mantine-ui/src/components/SettingsMenu.tsx +++ b/web/ui/mantine-ui/src/components/SettingsMenu.tsx @@ -5,7 +5,7 @@ import { Checkbox, Stack, Group, - NumberInput, + NumberInput } from "@mantine/core"; import { IconSettings } from "@tabler/icons-react"; import { FC } from "react"; @@ -21,8 +21,9 @@ const SettingsMenu: FC = () => { enableSyntaxHighlighting, enableLinter, showAnnotations, + hideEmptyGroups, ruleGroupsPerPage, - alertGroupsPerPage, + alertGroupsPerPage } = useSettings(); const dispatch = useAppDispatch(); @@ -48,7 +49,7 @@ const SettingsMenu: FC = () => { onChange={(event) => dispatch( updateSettings({ - useLocalTime: event.currentTarget.checked, + useLocalTime: event.currentTarget.checked }) ) } @@ -63,7 +64,7 @@ const SettingsMenu: FC = () => { onChange={(event) => dispatch( updateSettings({ - enableQueryHistory: event.currentTarget.checked, + enableQueryHistory: event.currentTarget.checked }) ) } @@ -74,7 +75,7 @@ const SettingsMenu: FC = () => { onChange={(event) => dispatch( updateSettings({ - enableAutocomplete: event.currentTarget.checked, + enableAutocomplete: event.currentTarget.checked }) ) } @@ -85,7 +86,7 @@ const SettingsMenu: FC = () => { onChange={(event) => dispatch( updateSettings({ - enableSyntaxHighlighting: event.currentTarget.checked, + enableSyntaxHighlighting: event.currentTarget.checked }) ) } @@ -96,7 +97,7 @@ const SettingsMenu: FC = () => { onChange={(event) => dispatch( updateSettings({ - enableLinter: event.currentTarget.checked, + enableLinter: event.currentTarget.checked }) ) } @@ -107,17 +108,30 @@ const SettingsMenu: FC = () => {
- - dispatch( - updateSettings({ - showAnnotations: event.currentTarget.checked, - }) - ) - } - /> + + + dispatch( + updateSettings({ + showAnnotations: event.currentTarget.checked + }) + ) + } + /> + + dispatch( + updateSettings({ + hideEmptyGroups: event.currentTarget.checked + }) + ) + } + /> +
{ dispatch( updateSettings({ - alertGroupsPerPage: value, + alertGroupsPerPage: value }) ); }} @@ -151,7 +165,7 @@ const SettingsMenu: FC = () => { dispatch( updateSettings({ - ruleGroupsPerPage: value, + ruleGroupsPerPage: value }) ); }} diff --git a/web/ui/mantine-ui/src/pages/AlertsPage.tsx b/web/ui/mantine-ui/src/pages/AlertsPage.tsx index 3143f0b41d..33e170556d 100644 --- a/web/ui/mantine-ui/src/pages/AlertsPage.tsx +++ b/web/ui/mantine-ui/src/pages/AlertsPage.tsx @@ -11,7 +11,7 @@ import { Alert, TextInput, Anchor, - Pagination, + Pagination } from "@mantine/core"; import { useSuspenseAPIQuery } from "../api/api"; import { AlertingRule, AlertingRulesResult } from "../api/responseTypes/rules"; @@ -23,14 +23,14 @@ import { Fragment, useEffect, useMemo } from "react"; import { StateMultiSelect } from "../components/StateMultiSelect"; import { IconInfoCircle, IconSearch } from "@tabler/icons-react"; import { LabelBadges } from "../components/LabelBadges"; -import { useSettings } from "../state/settingsSlice"; +import { useSettings, updateSettings } from "../state/settingsSlice"; +import { useAppDispatch } from "../state/hooks"; import { ArrayParam, - BooleanParam, NumberParam, StringParam, useQueryParam, - withDefault, + withDefault } from "use-query-params"; import { useDebouncedValue } from "@mantine/hooks"; import { KVSearch } from "@nexucis/kvsearch"; @@ -67,7 +67,7 @@ type AlertsPageData = { const kvSearch = new KVSearch({ shouldSort: true, - indexedKeys: ["name", "labels", ["labels", /.*/]], + indexedKeys: ["name", "labels", ["labels", /.*/]] }); const buildAlertsPageData = ( @@ -79,9 +79,9 @@ const buildAlertsPageData = ( globalCounts: { inactive: 0, pending: 0, - firing: 0, + firing: 0 }, - groups: [], + groups: [] }; for (const group of data.groups) { @@ -89,7 +89,7 @@ const buildAlertsPageData = ( total: 0, inactive: 0, pending: 0, - firing: 0, + firing: 0 }; for (const r of group.rules) { @@ -126,9 +126,9 @@ const buildAlertsPageData = ( rule: r, counts: { firing: r.alerts.filter((a) => a.state === "firing").length, - pending: r.alerts.filter((a) => a.state === "pending").length, - }, - })), + pending: r.alerts.filter((a) => a.state === "pending").length + } + })) }); } @@ -146,11 +146,12 @@ export default function AlertsPage() { const { data } = useSuspenseAPIQuery({ path: `/rules`, params: { - type: "alert", - }, + type: "alert" + } }); - const { showAnnotations } = useSettings(); + const { hideEmptyGroups, showAnnotations } = useSettings(); + const dispatch = useAppDispatch(); // Define URL query params. const [stateFilter, setStateFilter] = useQueryParam( @@ -162,10 +163,6 @@ export default function AlertsPage() { withDefault(StringParam, "") ); const [debouncedSearch] = useDebouncedValue(searchFilter.trim(), 250); - const [showEmptyGroups, setShowEmptyGroups] = useQueryParam( - "showEmptyGroups", - withDefault(BooleanParam, true) - ); const { alertGroupsPerPage } = useSettings(); const [activePage, setActivePage] = useQueryParam( @@ -181,10 +178,10 @@ export default function AlertsPage() { const shownGroups = useMemo( () => - showEmptyGroups + !hideEmptyGroups ? alertsPageData.groups : alertsPageData.groups.filter((g) => g.rules.length > 0), - [alertsPageData.groups, showEmptyGroups] + [alertsPageData.groups, hideEmptyGroups] ); // If we were e.g. on page 10 and the number of total pages decreases to 5 (due to filtering @@ -255,7 +252,13 @@ export default function AlertsPage() { setShowEmptyGroups(false)} + onClick={() => { + dispatch( + updateSettings({ + hideEmptyGroups: true + }) + ); + }} > Hide empty groups @@ -267,7 +270,13 @@ export default function AlertsPage() { setShowEmptyGroups(false)} + onClick={() => { + dispatch( + updateSettings({ + hideEmptyGroups: true + }) + ); + }} > Hide empty groups @@ -286,8 +295,8 @@ export default function AlertsPage() { // have a different background color than their surrounding group card in dark mode, // but it would be better to use CSS to override the light/dark colors for // collapsed/expanded accordion items. - backgroundColor: "#c0c0c015", - }, + backgroundColor: "#c0c0c015" + } }} key={j} value={j.toString()} @@ -403,7 +412,7 @@ export default function AlertsPage() { )} )), - [currentPageGroups, showAnnotations, setShowEmptyGroups] + [currentPageGroups, showAnnotations, dispatch] ); return ( @@ -442,7 +451,7 @@ export default function AlertsPage() { No rules found. ) : ( - !showEmptyGroups && + hideEmptyGroups && alertsPageData.groups.length !== shownGroups.length && ( Hiding {alertsPageData.groups.length - shownGroups.length} empty groups due to filters or no rules. - setShowEmptyGroups(true)}> + + dispatch(updateSettings({ hideEmptyGroups: false })) + } + > Show empty groups diff --git a/web/ui/mantine-ui/src/state/localStorageMiddleware.ts b/web/ui/mantine-ui/src/state/localStorageMiddleware.ts index 79baa5ac64..b3dc70662b 100644 --- a/web/ui/mantine-ui/src/state/localStorageMiddleware.ts +++ b/web/ui/mantine-ui/src/state/localStorageMiddleware.ts @@ -37,7 +37,7 @@ startAppListening({ effect: ({ payload }) => { persistToLocalStorage( localStorageKeyServiceDiscoveryPageCollapsedPools, - payload + payload, ); }, }); @@ -47,7 +47,7 @@ startAppListening({ effect: (_, { getState }) => { persistToLocalStorage( localStorageKeyQueryHistory, - getState().queryPage.queryHistory + getState().queryPage.queryHistory, ); }, }); @@ -62,6 +62,7 @@ startAppListening({ case "enableAutocomplete": case "enableSyntaxHighlighting": case "enableLinter": + case "hideEmptyGroups": case "showAnnotations": case "ruleGroupsPerPage": return persistToLocalStorage(`settings.${key}`, value); diff --git a/web/ui/mantine-ui/src/state/settingsSlice.ts b/web/ui/mantine-ui/src/state/settingsSlice.ts index c4154b7253..da1c1546bb 100644 --- a/web/ui/mantine-ui/src/state/settingsSlice.ts +++ b/web/ui/mantine-ui/src/state/settingsSlice.ts @@ -13,6 +13,7 @@ interface Settings { enableAutocomplete: boolean; enableSyntaxHighlighting: boolean; enableLinter: boolean; + hideEmptyGroups: boolean; showAnnotations: boolean; ruleGroupsPerPage: number; alertGroupsPerPage: number; @@ -30,6 +31,7 @@ export const localStorageKeyEnableAutocomplete = "settings.enableAutocomplete"; export const localStorageKeyEnableSyntaxHighlighting = "settings.enableSyntaxHighlighting"; export const localStorageKeyEnableLinter = "settings.enableLinter"; +export const localStorageKeyHideEmptyGroups = "settings.hideEmptyGroups"; export const localStorageKeyShowAnnotations = "settings.showAnnotations"; export const localStorageKeyRuleGroupsPerPage = "settings.ruleGroupsPerPage"; export const localStorageKeyAlertGroupsPerPage = "settings.alertGroupsPerPage"; @@ -53,7 +55,7 @@ const getPathPrefix = (path: string) => { "/flags", "/config", "/alertmanager-discovery", - "/agent", + "/agent" ]; const pagePath = pagePaths.find((p) => path.endsWith(p)); @@ -95,6 +97,10 @@ export const initialState: Settings = { localStorageKeyEnableLinter, true ), + hideEmptyGroups: initializeFromLocalStorage( + localStorageKeyHideEmptyGroups, + false + ), showAnnotations: initializeFromLocalStorage( localStorageKeyShowAnnotations, true @@ -106,7 +112,7 @@ export const initialState: Settings = { alertGroupsPerPage: initializeFromLocalStorage( localStorageKeyAlertGroupsPerPage, 10 - ), + ) }; export const settingsSlice = createSlice({ @@ -115,8 +121,8 @@ export const settingsSlice = createSlice({ reducers: { updateSettings: (state, { payload }: PayloadAction>) => { Object.assign(state, payload); - }, - }, + } + } }); export const { updateSettings } = settingsSlice.actions; From b3531a12f333cdf0b290f60898447eb8681f151a Mon Sep 17 00:00:00 2001 From: leonnicolas Date: Mon, 13 Jan 2025 09:37:44 +0100 Subject: [PATCH 050/110] Make "show empty pools" setting persistent Just like for showing empty groups on the Alerts page, also make the setting for showing empty pools on the Targets page persistent. Signed-off-by: leonnicolas --- .../src/components/SettingsMenu.tsx | 36 +++++++++++++------ web/ui/mantine-ui/src/pages/AlertsPage.tsx | 30 ++++++++-------- .../src/pages/targets/ScrapePoolsList.tsx | 21 ++++++----- .../src/state/localStorageMiddleware.ts | 5 +-- web/ui/mantine-ui/src/state/settingsSlice.ts | 14 +++++--- 5 files changed, 66 insertions(+), 40 deletions(-) diff --git a/web/ui/mantine-ui/src/components/SettingsMenu.tsx b/web/ui/mantine-ui/src/components/SettingsMenu.tsx index 79d9c201cc..d7fe458ca5 100644 --- a/web/ui/mantine-ui/src/components/SettingsMenu.tsx +++ b/web/ui/mantine-ui/src/components/SettingsMenu.tsx @@ -5,7 +5,7 @@ import { Checkbox, Stack, Group, - NumberInput + NumberInput, } from "@mantine/core"; import { IconSettings } from "@tabler/icons-react"; import { FC } from "react"; @@ -23,7 +23,8 @@ const SettingsMenu: FC = () => { showAnnotations, hideEmptyGroups, ruleGroupsPerPage, - alertGroupsPerPage + alertGroupsPerPage, + showEmptyPools, } = useSettings(); const dispatch = useAppDispatch(); @@ -49,7 +50,7 @@ const SettingsMenu: FC = () => { onChange={(event) => dispatch( updateSettings({ - useLocalTime: event.currentTarget.checked + useLocalTime: event.currentTarget.checked, }) ) } @@ -64,7 +65,7 @@ const SettingsMenu: FC = () => { onChange={(event) => dispatch( updateSettings({ - enableQueryHistory: event.currentTarget.checked + enableQueryHistory: event.currentTarget.checked, }) ) } @@ -75,7 +76,7 @@ const SettingsMenu: FC = () => { onChange={(event) => dispatch( updateSettings({ - enableAutocomplete: event.currentTarget.checked + enableAutocomplete: event.currentTarget.checked, }) ) } @@ -86,7 +87,7 @@ const SettingsMenu: FC = () => { onChange={(event) => dispatch( updateSettings({ - enableSyntaxHighlighting: event.currentTarget.checked + enableSyntaxHighlighting: event.currentTarget.checked, }) ) } @@ -97,13 +98,26 @@ const SettingsMenu: FC = () => { onChange={(event) => dispatch( updateSettings({ - enableLinter: event.currentTarget.checked + enableLinter: event.currentTarget.checked, }) ) } />
+
+ + dispatch( + updateSettings({ + showEmptyPools: event.currentTarget.checked, + }) + ) + } + /> +
@@ -115,7 +129,7 @@ const SettingsMenu: FC = () => { onChange={(event) => dispatch( updateSettings({ - showAnnotations: event.currentTarget.checked + showAnnotations: event.currentTarget.checked, }) ) } @@ -126,7 +140,7 @@ const SettingsMenu: FC = () => { onChange={(event) => dispatch( updateSettings({ - hideEmptyGroups: event.currentTarget.checked + hideEmptyGroups: event.currentTarget.checked, }) ) } @@ -146,7 +160,7 @@ const SettingsMenu: FC = () => { dispatch( updateSettings({ - alertGroupsPerPage: value + alertGroupsPerPage: value, }) ); }} @@ -165,7 +179,7 @@ const SettingsMenu: FC = () => { dispatch( updateSettings({ - ruleGroupsPerPage: value + ruleGroupsPerPage: value, }) ); }} diff --git a/web/ui/mantine-ui/src/pages/AlertsPage.tsx b/web/ui/mantine-ui/src/pages/AlertsPage.tsx index 33e170556d..69c4d7f130 100644 --- a/web/ui/mantine-ui/src/pages/AlertsPage.tsx +++ b/web/ui/mantine-ui/src/pages/AlertsPage.tsx @@ -11,7 +11,7 @@ import { Alert, TextInput, Anchor, - Pagination + Pagination, } from "@mantine/core"; import { useSuspenseAPIQuery } from "../api/api"; import { AlertingRule, AlertingRulesResult } from "../api/responseTypes/rules"; @@ -30,7 +30,7 @@ import { NumberParam, StringParam, useQueryParam, - withDefault + withDefault, } from "use-query-params"; import { useDebouncedValue } from "@mantine/hooks"; import { KVSearch } from "@nexucis/kvsearch"; @@ -67,7 +67,7 @@ type AlertsPageData = { const kvSearch = new KVSearch({ shouldSort: true, - indexedKeys: ["name", "labels", ["labels", /.*/]] + indexedKeys: ["name", "labels", ["labels", /.*/]], }); const buildAlertsPageData = ( @@ -79,9 +79,9 @@ const buildAlertsPageData = ( globalCounts: { inactive: 0, pending: 0, - firing: 0 + firing: 0, }, - groups: [] + groups: [], }; for (const group of data.groups) { @@ -89,7 +89,7 @@ const buildAlertsPageData = ( total: 0, inactive: 0, pending: 0, - firing: 0 + firing: 0, }; for (const r of group.rules) { @@ -126,9 +126,9 @@ const buildAlertsPageData = ( rule: r, counts: { firing: r.alerts.filter((a) => a.state === "firing").length, - pending: r.alerts.filter((a) => a.state === "pending").length - } - })) + pending: r.alerts.filter((a) => a.state === "pending").length, + }, + })), }); } @@ -146,8 +146,8 @@ export default function AlertsPage() { const { data } = useSuspenseAPIQuery({ path: `/rules`, params: { - type: "alert" - } + type: "alert", + }, }); const { hideEmptyGroups, showAnnotations } = useSettings(); @@ -255,7 +255,7 @@ export default function AlertsPage() { onClick={() => { dispatch( updateSettings({ - hideEmptyGroups: true + hideEmptyGroups: true, }) ); }} @@ -273,7 +273,7 @@ export default function AlertsPage() { onClick={() => { dispatch( updateSettings({ - hideEmptyGroups: true + hideEmptyGroups: true, }) ); }} @@ -295,8 +295,8 @@ export default function AlertsPage() { // have a different background color than their surrounding group card in dark mode, // but it would be better to use CSS to override the light/dark colors for // collapsed/expanded accordion items. - backgroundColor: "#c0c0c015" - } + backgroundColor: "#c0c0c015", + }, }} key={j} value={j.toString()} diff --git a/web/ui/mantine-ui/src/pages/targets/ScrapePoolsList.tsx b/web/ui/mantine-ui/src/pages/targets/ScrapePoolsList.tsx index be91b149bb..68c1db184b 100644 --- a/web/ui/mantine-ui/src/pages/targets/ScrapePoolsList.tsx +++ b/web/ui/mantine-ui/src/pages/targets/ScrapePoolsList.tsx @@ -30,6 +30,7 @@ import { setCollapsedPools, setShowLimitAlert, } from "../../state/targetsPageSlice"; +import { useSettings, updateSettings } from "../../state/settingsSlice"; import EndpointLink from "../../components/EndpointLink"; import CustomInfiniteScroll from "../../components/CustomInfiniteScroll"; @@ -38,7 +39,6 @@ import panelClasses from "../../Panel.module.css"; import TargetLabels from "./TargetLabels"; import { useDebouncedValue } from "@mantine/hooks"; import { targetPoolDisplayLimit } from "./TargetsPage"; -import { BooleanParam, useQueryParam, withDefault } from "use-query-params"; import { badgeIconStyle } from "../../styles"; type ScrapePool = { @@ -164,11 +164,8 @@ const ScrapePoolList: FC = ({ }, }); + const { showEmptyPools } = useSettings(); const dispatch = useAppDispatch(); - const [showEmptyPools, setShowEmptyPools] = useQueryParam( - "showEmptyPools", - withDefault(BooleanParam, true) - ); const { collapsedPools, showLimitAlert } = useAppSelector( (state) => state.targetsPage @@ -207,7 +204,11 @@ const ScrapePoolList: FC = ({ > Hiding {allPoolNames.length - shownPoolNames.length} empty pools due to filters or no targets. - setShowEmptyPools(true)}> + dispatch(updateSettings({ showEmptyPools: true }))} + > Show empty pools @@ -281,7 +282,9 @@ const ScrapePoolList: FC = ({ setShowEmptyPools(false)} + onClick={() => + dispatch(updateSettings({ showEmptyPools: false })) + } > Hide empty pools @@ -293,7 +296,9 @@ const ScrapePoolList: FC = ({ setShowEmptyPools(false)} + onClick={() => + dispatch(updateSettings({ showEmptyPools: false })) + } > Hide empty pools diff --git a/web/ui/mantine-ui/src/state/localStorageMiddleware.ts b/web/ui/mantine-ui/src/state/localStorageMiddleware.ts index b3dc70662b..e27e858444 100644 --- a/web/ui/mantine-ui/src/state/localStorageMiddleware.ts +++ b/web/ui/mantine-ui/src/state/localStorageMiddleware.ts @@ -37,7 +37,7 @@ startAppListening({ effect: ({ payload }) => { persistToLocalStorage( localStorageKeyServiceDiscoveryPageCollapsedPools, - payload, + payload ); }, }); @@ -47,7 +47,7 @@ startAppListening({ effect: (_, { getState }) => { persistToLocalStorage( localStorageKeyQueryHistory, - getState().queryPage.queryHistory, + getState().queryPage.queryHistory ); }, }); @@ -65,6 +65,7 @@ startAppListening({ case "hideEmptyGroups": case "showAnnotations": case "ruleGroupsPerPage": + case "showEmptyPools": return persistToLocalStorage(`settings.${key}`, value); } }); diff --git a/web/ui/mantine-ui/src/state/settingsSlice.ts b/web/ui/mantine-ui/src/state/settingsSlice.ts index da1c1546bb..06b4c06ca0 100644 --- a/web/ui/mantine-ui/src/state/settingsSlice.ts +++ b/web/ui/mantine-ui/src/state/settingsSlice.ts @@ -17,6 +17,7 @@ interface Settings { showAnnotations: boolean; ruleGroupsPerPage: number; alertGroupsPerPage: number; + showEmptyPools: boolean; } // Declared/defined in public/index.html, value replaced by Prometheus when serving bundle. @@ -35,6 +36,7 @@ export const localStorageKeyHideEmptyGroups = "settings.hideEmptyGroups"; export const localStorageKeyShowAnnotations = "settings.showAnnotations"; export const localStorageKeyRuleGroupsPerPage = "settings.ruleGroupsPerPage"; export const localStorageKeyAlertGroupsPerPage = "settings.alertGroupsPerPage"; +export const localStorageKeyShowEmptyPools = "settings.showEmptyPools"; // This dynamically/generically determines the pathPrefix by stripping the first known // endpoint suffix from the window location path. It works out of the box for both direct @@ -55,7 +57,7 @@ const getPathPrefix = (path: string) => { "/flags", "/config", "/alertmanager-discovery", - "/agent" + "/agent", ]; const pagePath = pagePaths.find((p) => path.endsWith(p)); @@ -112,7 +114,11 @@ export const initialState: Settings = { alertGroupsPerPage: initializeFromLocalStorage( localStorageKeyAlertGroupsPerPage, 10 - ) + ), + showEmptyPools: initializeFromLocalStorage( + localStorageKeyShowEmptyPools, + true + ), }; export const settingsSlice = createSlice({ @@ -121,8 +127,8 @@ export const settingsSlice = createSlice({ reducers: { updateSettings: (state, { payload }: PayloadAction>) => { Object.assign(state, payload); - } - } + }, + }, }); export const { updateSettings } = settingsSlice.actions; From 60b71108e5d4f9ebb763aaf3a30f86718b4323f2 Mon Sep 17 00:00:00 2001 From: Julius Volz Date: Mon, 13 Jan 2025 16:23:51 +0100 Subject: [PATCH 051/110] Merge two accidental "Alerts page settings" sections into one Signed-off-by: Julius Volz --- .../src/components/SettingsMenu.tsx | 56 +++++++++---------- 1 file changed, 28 insertions(+), 28 deletions(-) diff --git a/web/ui/mantine-ui/src/components/SettingsMenu.tsx b/web/ui/mantine-ui/src/components/SettingsMenu.tsx index aae38909d0..e5376eef2c 100644 --- a/web/ui/mantine-ui/src/components/SettingsMenu.tsx +++ b/web/ui/mantine-ui/src/components/SettingsMenu.tsx @@ -107,36 +107,36 @@ const SettingsMenu: FC = () => {
- - dispatch( - updateSettings({ - showAnnotations: event.currentTarget.checked, - }) - ) - } - /> -
-
- { - if (typeof value !== "number") { - return; + + + dispatch( + updateSettings({ + showAnnotations: event.currentTarget.checked, + }) + ) } + /> + { + if (typeof value !== "number") { + return; + } - dispatch( - updateSettings({ - alertGroupsPerPage: value, - }) - ); - }} - /> + dispatch( + updateSettings({ + alertGroupsPerPage: value, + }) + ); + }} + /> +
Date: Mon, 13 Jan 2025 17:56:11 +0100 Subject: [PATCH 052/110] Remove settings from setting menu Signed-off-by: leonnicolas --- .../src/components/SettingsMenu.tsx | 50 ++++--------------- web/ui/mantine-ui/src/pages/AlertsPage.tsx | 43 ++++++---------- .../src/pages/targets/ScrapePoolsList.tsx | 21 +++----- .../src/state/localStorageMiddleware.ts | 2 - web/ui/mantine-ui/src/state/settingsSlice.ts | 12 ----- 5 files changed, 33 insertions(+), 95 deletions(-) diff --git a/web/ui/mantine-ui/src/components/SettingsMenu.tsx b/web/ui/mantine-ui/src/components/SettingsMenu.tsx index d7fe458ca5..aae38909d0 100644 --- a/web/ui/mantine-ui/src/components/SettingsMenu.tsx +++ b/web/ui/mantine-ui/src/components/SettingsMenu.tsx @@ -21,10 +21,8 @@ const SettingsMenu: FC = () => { enableSyntaxHighlighting, enableLinter, showAnnotations, - hideEmptyGroups, ruleGroupsPerPage, alertGroupsPerPage, - showEmptyPools, } = useSettings(); const dispatch = useAppDispatch(); @@ -105,47 +103,21 @@ const SettingsMenu: FC = () => { />
-
- - dispatch( - updateSettings({ - showEmptyPools: event.currentTarget.checked, - }) - ) - } - /> -
- - - dispatch( - updateSettings({ - showAnnotations: event.currentTarget.checked, - }) - ) - } - /> - - dispatch( - updateSettings({ - hideEmptyGroups: event.currentTarget.checked, - }) - ) - } - /> - + + dispatch( + updateSettings({ + showAnnotations: event.currentTarget.checked, + }) + ) + } + />
(searchFilter.trim(), 250); + const [showEmptyGroups, setShowEmptyGroups] = useLocalStorage({ + key: "alerts-page-show-empty-groups", + defaultValue: true, + }); const { alertGroupsPerPage } = useSettings(); const [activePage, setActivePage] = useQueryParam( @@ -178,10 +181,10 @@ export default function AlertsPage() { const shownGroups = useMemo( () => - !hideEmptyGroups + showEmptyGroups ? alertsPageData.groups : alertsPageData.groups.filter((g) => g.rules.length > 0), - [alertsPageData.groups, hideEmptyGroups] + [alertsPageData.groups, showEmptyGroups] ); // If we were e.g. on page 10 and the number of total pages decreases to 5 (due to filtering @@ -252,13 +255,7 @@ export default function AlertsPage() { { - dispatch( - updateSettings({ - hideEmptyGroups: true, - }) - ); - }} + onClick={() => setShowEmptyGroups(false)} > Hide empty groups @@ -270,13 +267,7 @@ export default function AlertsPage() { { - dispatch( - updateSettings({ - hideEmptyGroups: true, - }) - ); - }} + onClick={() => setShowEmptyGroups(false)} > Hide empty groups @@ -412,7 +403,7 @@ export default function AlertsPage() { )} )), - [currentPageGroups, showAnnotations, dispatch] + [currentPageGroups, showAnnotations, setShowEmptyGroups] ); return ( @@ -451,7 +442,7 @@ export default function AlertsPage() { No rules found. ) : ( - hideEmptyGroups && + !showEmptyGroups && alertsPageData.groups.length !== shownGroups.length && ( Hiding {alertsPageData.groups.length - shownGroups.length} empty groups due to filters or no rules. - - dispatch(updateSettings({ hideEmptyGroups: false })) - } - > + setShowEmptyGroups(true)}> Show empty groups diff --git a/web/ui/mantine-ui/src/pages/targets/ScrapePoolsList.tsx b/web/ui/mantine-ui/src/pages/targets/ScrapePoolsList.tsx index 68c1db184b..f674a1f4d3 100644 --- a/web/ui/mantine-ui/src/pages/targets/ScrapePoolsList.tsx +++ b/web/ui/mantine-ui/src/pages/targets/ScrapePoolsList.tsx @@ -25,12 +25,12 @@ import { humanizeDuration, now, } from "../../lib/formatTime"; +import { useLocalStorage } from "@mantine/hooks"; import { useAppDispatch, useAppSelector } from "../../state/hooks"; import { setCollapsedPools, setShowLimitAlert, } from "../../state/targetsPageSlice"; -import { useSettings, updateSettings } from "../../state/settingsSlice"; import EndpointLink from "../../components/EndpointLink"; import CustomInfiniteScroll from "../../components/CustomInfiniteScroll"; @@ -164,8 +164,11 @@ const ScrapePoolList: FC = ({ }, }); - const { showEmptyPools } = useSettings(); const dispatch = useAppDispatch(); + const [showEmptyPools, setShowEmptyPools] = useLocalStorage({ + key: "targets-page-show-empty-pools", + defaultValue: true, + }); const { collapsedPools, showLimitAlert } = useAppSelector( (state) => state.targetsPage @@ -204,11 +207,7 @@ const ScrapePoolList: FC = ({ > Hiding {allPoolNames.length - shownPoolNames.length} empty pools due to filters or no targets. - dispatch(updateSettings({ showEmptyPools: true }))} - > + setShowEmptyPools(true)}> Show empty pools @@ -282,9 +281,7 @@ const ScrapePoolList: FC = ({ - dispatch(updateSettings({ showEmptyPools: false })) - } + onClick={() => setShowEmptyPools(false)} > Hide empty pools @@ -296,9 +293,7 @@ const ScrapePoolList: FC = ({ - dispatch(updateSettings({ showEmptyPools: false })) - } + onClick={() => setShowEmptyPools(false)} > Hide empty pools diff --git a/web/ui/mantine-ui/src/state/localStorageMiddleware.ts b/web/ui/mantine-ui/src/state/localStorageMiddleware.ts index e27e858444..79baa5ac64 100644 --- a/web/ui/mantine-ui/src/state/localStorageMiddleware.ts +++ b/web/ui/mantine-ui/src/state/localStorageMiddleware.ts @@ -62,10 +62,8 @@ startAppListening({ case "enableAutocomplete": case "enableSyntaxHighlighting": case "enableLinter": - case "hideEmptyGroups": case "showAnnotations": case "ruleGroupsPerPage": - case "showEmptyPools": return persistToLocalStorage(`settings.${key}`, value); } }); diff --git a/web/ui/mantine-ui/src/state/settingsSlice.ts b/web/ui/mantine-ui/src/state/settingsSlice.ts index 06b4c06ca0..c4154b7253 100644 --- a/web/ui/mantine-ui/src/state/settingsSlice.ts +++ b/web/ui/mantine-ui/src/state/settingsSlice.ts @@ -13,11 +13,9 @@ interface Settings { enableAutocomplete: boolean; enableSyntaxHighlighting: boolean; enableLinter: boolean; - hideEmptyGroups: boolean; showAnnotations: boolean; ruleGroupsPerPage: number; alertGroupsPerPage: number; - showEmptyPools: boolean; } // Declared/defined in public/index.html, value replaced by Prometheus when serving bundle. @@ -32,11 +30,9 @@ export const localStorageKeyEnableAutocomplete = "settings.enableAutocomplete"; export const localStorageKeyEnableSyntaxHighlighting = "settings.enableSyntaxHighlighting"; export const localStorageKeyEnableLinter = "settings.enableLinter"; -export const localStorageKeyHideEmptyGroups = "settings.hideEmptyGroups"; export const localStorageKeyShowAnnotations = "settings.showAnnotations"; export const localStorageKeyRuleGroupsPerPage = "settings.ruleGroupsPerPage"; export const localStorageKeyAlertGroupsPerPage = "settings.alertGroupsPerPage"; -export const localStorageKeyShowEmptyPools = "settings.showEmptyPools"; // This dynamically/generically determines the pathPrefix by stripping the first known // endpoint suffix from the window location path. It works out of the box for both direct @@ -99,10 +95,6 @@ export const initialState: Settings = { localStorageKeyEnableLinter, true ), - hideEmptyGroups: initializeFromLocalStorage( - localStorageKeyHideEmptyGroups, - false - ), showAnnotations: initializeFromLocalStorage( localStorageKeyShowAnnotations, true @@ -115,10 +107,6 @@ export const initialState: Settings = { localStorageKeyAlertGroupsPerPage, 10 ), - showEmptyPools: initializeFromLocalStorage( - localStorageKeyShowEmptyPools, - true - ), }; export const settingsSlice = createSlice({ From 7e27fd7417805c267a030dc12cde12174736b89b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Jan 2025 19:06:49 +0100 Subject: [PATCH 053/110] chore(deps-dev): bump vite from 6.0.3 to 6.0.7 in /web/ui (#15781) Bumps [vite](https://github.com/vitejs/vite/tree/HEAD/packages/vite) from 6.0.3 to 6.0.7. - [Release notes](https://github.com/vitejs/vite/releases) - [Changelog](https://github.com/vitejs/vite/blob/main/packages/vite/CHANGELOG.md) - [Commits](https://github.com/vitejs/vite/commits/v6.0.7/packages/vite) --- updated-dependencies: - dependency-name: vite dependency-type: direct:development update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- web/ui/mantine-ui/package.json | 2 +- web/ui/package-lock.json | 253 ++++++++++++++++----------------- web/ui/package.json | 2 +- 3 files changed, 124 insertions(+), 133 deletions(-) diff --git a/web/ui/mantine-ui/package.json b/web/ui/mantine-ui/package.json index c3f1e0fbfd..5f3d95a2e3 100644 --- a/web/ui/mantine-ui/package.json +++ b/web/ui/mantine-ui/package.json @@ -67,7 +67,7 @@ "postcss": "^8.4.47", "postcss-preset-mantine": "^1.17.0", "postcss-simple-vars": "^7.0.1", - "vite": "^6.0.3", + "vite": "^6.0.7", "vitest": "^2.1.8" } } diff --git a/web/ui/package-lock.json b/web/ui/package-lock.json index 6db3035c27..a29e4a3d02 100644 --- a/web/ui/package-lock.json +++ b/web/ui/package-lock.json @@ -19,7 +19,7 @@ "prettier": "^3.4.2", "ts-jest": "^29.2.2", "typescript": "^5.7.2", - "vite": "^6.0.3" + "vite": "^6.0.7" } }, "mantine-ui": { @@ -81,7 +81,7 @@ "postcss": "^8.4.47", "postcss-preset-mantine": "^1.17.0", "postcss-simple-vars": "^7.0.1", - "vite": "^6.0.3", + "vite": "^6.0.7", "vitest": "^2.1.8" } }, @@ -876,14 +876,13 @@ } }, "node_modules/@esbuild/aix-ppc64": { - "version": "0.24.0", - "resolved": "https://registry.npmjs.org/@esbuild/aix-ppc64/-/aix-ppc64-0.24.0.tgz", - "integrity": "sha512-WtKdFM7ls47zkKHFVzMz8opM7LkcsIp9amDUBIAWirg70RM71WRSjdILPsY5Uv1D42ZpUfaPILDlfactHgsRkw==", + "version": "0.24.2", + "resolved": "https://registry.npmjs.org/@esbuild/aix-ppc64/-/aix-ppc64-0.24.2.tgz", + "integrity": "sha512-thpVCb/rhxE/BnMLQ7GReQLLN8q9qbHmI55F4489/ByVg2aQaQ6kbcLb6FHkocZzQhxc4gx0sCk0tJkKBFzDhA==", "cpu": [ "ppc64" ], "dev": true, - "license": "MIT", "optional": true, "os": [ "aix" @@ -893,14 +892,13 @@ } }, "node_modules/@esbuild/android-arm": { - "version": "0.24.0", - "resolved": "https://registry.npmjs.org/@esbuild/android-arm/-/android-arm-0.24.0.tgz", - "integrity": "sha512-arAtTPo76fJ/ICkXWetLCc9EwEHKaeya4vMrReVlEIUCAUncH7M4bhMQ+M9Vf+FFOZJdTNMXNBrWwW+OXWpSew==", + "version": "0.24.2", + "resolved": "https://registry.npmjs.org/@esbuild/android-arm/-/android-arm-0.24.2.tgz", + "integrity": "sha512-tmwl4hJkCfNHwFB3nBa8z1Uy3ypZpxqxfTQOcHX+xRByyYgunVbZ9MzUUfb0RxaHIMnbHagwAxuTL+tnNM+1/Q==", "cpu": [ "arm" ], "dev": true, - "license": "MIT", "optional": true, "os": [ "android" @@ -910,14 +908,13 @@ } }, "node_modules/@esbuild/android-arm64": { - "version": "0.24.0", - "resolved": "https://registry.npmjs.org/@esbuild/android-arm64/-/android-arm64-0.24.0.tgz", - "integrity": "sha512-Vsm497xFM7tTIPYK9bNTYJyF/lsP590Qc1WxJdlB6ljCbdZKU9SY8i7+Iin4kyhV/KV5J2rOKsBQbB77Ab7L/w==", + "version": "0.24.2", + "resolved": "https://registry.npmjs.org/@esbuild/android-arm64/-/android-arm64-0.24.2.tgz", + "integrity": "sha512-cNLgeqCqV8WxfcTIOeL4OAtSmL8JjcN6m09XIgro1Wi7cF4t/THaWEa7eL5CMoMBdjoHOTh/vwTO/o2TRXIyzg==", "cpu": [ "arm64" ], "dev": true, - "license": "MIT", "optional": true, "os": [ "android" @@ -927,14 +924,13 @@ } }, "node_modules/@esbuild/android-x64": { - "version": "0.24.0", - "resolved": "https://registry.npmjs.org/@esbuild/android-x64/-/android-x64-0.24.0.tgz", - "integrity": "sha512-t8GrvnFkiIY7pa7mMgJd7p8p8qqYIz1NYiAoKc75Zyv73L3DZW++oYMSHPRarcotTKuSs6m3hTOa5CKHaS02TQ==", + "version": "0.24.2", + "resolved": "https://registry.npmjs.org/@esbuild/android-x64/-/android-x64-0.24.2.tgz", + "integrity": "sha512-B6Q0YQDqMx9D7rvIcsXfmJfvUYLoP722bgfBlO5cGvNVb5V/+Y7nhBE3mHV9OpxBf4eAS2S68KZztiPaWq4XYw==", "cpu": [ "x64" ], "dev": true, - "license": "MIT", "optional": true, "os": [ "android" @@ -944,14 +940,13 @@ } }, "node_modules/@esbuild/darwin-arm64": { - "version": "0.24.0", - "resolved": "https://registry.npmjs.org/@esbuild/darwin-arm64/-/darwin-arm64-0.24.0.tgz", - "integrity": "sha512-CKyDpRbK1hXwv79soeTJNHb5EiG6ct3efd/FTPdzOWdbZZfGhpbcqIpiD0+vwmpu0wTIL97ZRPZu8vUt46nBSw==", + "version": "0.24.2", + "resolved": "https://registry.npmjs.org/@esbuild/darwin-arm64/-/darwin-arm64-0.24.2.tgz", + "integrity": "sha512-kj3AnYWc+CekmZnS5IPu9D+HWtUI49hbnyqk0FLEJDbzCIQt7hg7ucF1SQAilhtYpIujfaHr6O0UHlzzSPdOeA==", "cpu": [ "arm64" ], "dev": true, - "license": "MIT", "optional": true, "os": [ "darwin" @@ -961,14 +956,13 @@ } }, "node_modules/@esbuild/darwin-x64": { - "version": "0.24.0", - "resolved": "https://registry.npmjs.org/@esbuild/darwin-x64/-/darwin-x64-0.24.0.tgz", - "integrity": "sha512-rgtz6flkVkh58od4PwTRqxbKH9cOjaXCMZgWD905JOzjFKW+7EiUObfd/Kav+A6Gyud6WZk9w+xu6QLytdi2OA==", + "version": "0.24.2", + "resolved": "https://registry.npmjs.org/@esbuild/darwin-x64/-/darwin-x64-0.24.2.tgz", + "integrity": "sha512-WeSrmwwHaPkNR5H3yYfowhZcbriGqooyu3zI/3GGpF8AyUdsrrP0X6KumITGA9WOyiJavnGZUwPGvxvwfWPHIA==", "cpu": [ "x64" ], "dev": true, - "license": "MIT", "optional": true, "os": [ "darwin" @@ -978,14 +972,13 @@ } }, "node_modules/@esbuild/freebsd-arm64": { - "version": "0.24.0", - "resolved": "https://registry.npmjs.org/@esbuild/freebsd-arm64/-/freebsd-arm64-0.24.0.tgz", - "integrity": "sha512-6Mtdq5nHggwfDNLAHkPlyLBpE5L6hwsuXZX8XNmHno9JuL2+bg2BX5tRkwjyfn6sKbxZTq68suOjgWqCicvPXA==", + "version": "0.24.2", + "resolved": "https://registry.npmjs.org/@esbuild/freebsd-arm64/-/freebsd-arm64-0.24.2.tgz", + "integrity": "sha512-UN8HXjtJ0k/Mj6a9+5u6+2eZ2ERD7Edt1Q9IZiB5UZAIdPnVKDoG7mdTVGhHJIeEml60JteamR3qhsr1r8gXvg==", "cpu": [ "arm64" ], "dev": true, - "license": "MIT", "optional": true, "os": [ "freebsd" @@ -995,14 +988,13 @@ } }, "node_modules/@esbuild/freebsd-x64": { - "version": "0.24.0", - "resolved": "https://registry.npmjs.org/@esbuild/freebsd-x64/-/freebsd-x64-0.24.0.tgz", - "integrity": "sha512-D3H+xh3/zphoX8ck4S2RxKR6gHlHDXXzOf6f/9dbFt/NRBDIE33+cVa49Kil4WUjxMGW0ZIYBYtaGCa2+OsQwQ==", + "version": "0.24.2", + "resolved": "https://registry.npmjs.org/@esbuild/freebsd-x64/-/freebsd-x64-0.24.2.tgz", + "integrity": "sha512-TvW7wE/89PYW+IevEJXZ5sF6gJRDY/14hyIGFXdIucxCsbRmLUcjseQu1SyTko+2idmCw94TgyaEZi9HUSOe3Q==", "cpu": [ "x64" ], "dev": true, - "license": "MIT", "optional": true, "os": [ "freebsd" @@ -1012,14 +1004,13 @@ } }, "node_modules/@esbuild/linux-arm": { - "version": "0.24.0", - "resolved": "https://registry.npmjs.org/@esbuild/linux-arm/-/linux-arm-0.24.0.tgz", - "integrity": "sha512-gJKIi2IjRo5G6Glxb8d3DzYXlxdEj2NlkixPsqePSZMhLudqPhtZ4BUrpIuTjJYXxvF9njql+vRjB2oaC9XpBw==", + "version": "0.24.2", + "resolved": "https://registry.npmjs.org/@esbuild/linux-arm/-/linux-arm-0.24.2.tgz", + "integrity": "sha512-n0WRM/gWIdU29J57hJyUdIsk0WarGd6To0s+Y+LwvlC55wt+GT/OgkwoXCXvIue1i1sSNWblHEig00GBWiJgfA==", "cpu": [ "arm" ], "dev": true, - "license": "MIT", "optional": true, "os": [ "linux" @@ -1029,14 +1020,13 @@ } }, "node_modules/@esbuild/linux-arm64": { - "version": "0.24.0", - "resolved": "https://registry.npmjs.org/@esbuild/linux-arm64/-/linux-arm64-0.24.0.tgz", - "integrity": "sha512-TDijPXTOeE3eaMkRYpcy3LarIg13dS9wWHRdwYRnzlwlA370rNdZqbcp0WTyyV/k2zSxfko52+C7jU5F9Tfj1g==", + "version": "0.24.2", + "resolved": "https://registry.npmjs.org/@esbuild/linux-arm64/-/linux-arm64-0.24.2.tgz", + "integrity": "sha512-7HnAD6074BW43YvvUmE/35Id9/NB7BeX5EoNkK9obndmZBUk8xmJJeU7DwmUeN7tkysslb2eSl6CTrYz6oEMQg==", "cpu": [ "arm64" ], "dev": true, - "license": "MIT", "optional": true, "os": [ "linux" @@ -1046,14 +1036,13 @@ } }, "node_modules/@esbuild/linux-ia32": { - "version": "0.24.0", - "resolved": "https://registry.npmjs.org/@esbuild/linux-ia32/-/linux-ia32-0.24.0.tgz", - "integrity": "sha512-K40ip1LAcA0byL05TbCQ4yJ4swvnbzHscRmUilrmP9Am7//0UjPreh4lpYzvThT2Quw66MhjG//20mrufm40mA==", + "version": "0.24.2", + "resolved": "https://registry.npmjs.org/@esbuild/linux-ia32/-/linux-ia32-0.24.2.tgz", + "integrity": "sha512-sfv0tGPQhcZOgTKO3oBE9xpHuUqguHvSo4jl+wjnKwFpapx+vUDcawbwPNuBIAYdRAvIDBfZVvXprIj3HA+Ugw==", "cpu": [ "ia32" ], "dev": true, - "license": "MIT", "optional": true, "os": [ "linux" @@ -1063,14 +1052,13 @@ } }, "node_modules/@esbuild/linux-loong64": { - "version": "0.24.0", - "resolved": "https://registry.npmjs.org/@esbuild/linux-loong64/-/linux-loong64-0.24.0.tgz", - "integrity": "sha512-0mswrYP/9ai+CU0BzBfPMZ8RVm3RGAN/lmOMgW4aFUSOQBjA31UP8Mr6DDhWSuMwj7jaWOT0p0WoZ6jeHhrD7g==", + "version": "0.24.2", + "resolved": "https://registry.npmjs.org/@esbuild/linux-loong64/-/linux-loong64-0.24.2.tgz", + "integrity": "sha512-CN9AZr8kEndGooS35ntToZLTQLHEjtVB5n7dl8ZcTZMonJ7CCfStrYhrzF97eAecqVbVJ7APOEe18RPI4KLhwQ==", "cpu": [ "loong64" ], "dev": true, - "license": "MIT", "optional": true, "os": [ "linux" @@ -1080,14 +1068,13 @@ } }, "node_modules/@esbuild/linux-mips64el": { - "version": "0.24.0", - "resolved": "https://registry.npmjs.org/@esbuild/linux-mips64el/-/linux-mips64el-0.24.0.tgz", - "integrity": "sha512-hIKvXm0/3w/5+RDtCJeXqMZGkI2s4oMUGj3/jM0QzhgIASWrGO5/RlzAzm5nNh/awHE0A19h/CvHQe6FaBNrRA==", + "version": "0.24.2", + "resolved": "https://registry.npmjs.org/@esbuild/linux-mips64el/-/linux-mips64el-0.24.2.tgz", + "integrity": "sha512-iMkk7qr/wl3exJATwkISxI7kTcmHKE+BlymIAbHO8xanq/TjHaaVThFF6ipWzPHryoFsesNQJPE/3wFJw4+huw==", "cpu": [ "mips64el" ], "dev": true, - "license": "MIT", "optional": true, "os": [ "linux" @@ -1097,14 +1084,13 @@ } }, "node_modules/@esbuild/linux-ppc64": { - "version": "0.24.0", - "resolved": "https://registry.npmjs.org/@esbuild/linux-ppc64/-/linux-ppc64-0.24.0.tgz", - "integrity": "sha512-HcZh5BNq0aC52UoocJxaKORfFODWXZxtBaaZNuN3PUX3MoDsChsZqopzi5UupRhPHSEHotoiptqikjN/B77mYQ==", + "version": "0.24.2", + "resolved": "https://registry.npmjs.org/@esbuild/linux-ppc64/-/linux-ppc64-0.24.2.tgz", + "integrity": "sha512-shsVrgCZ57Vr2L8mm39kO5PPIb+843FStGt7sGGoqiiWYconSxwTiuswC1VJZLCjNiMLAMh34jg4VSEQb+iEbw==", "cpu": [ "ppc64" ], "dev": true, - "license": "MIT", "optional": true, "os": [ "linux" @@ -1114,14 +1100,13 @@ } }, "node_modules/@esbuild/linux-riscv64": { - "version": "0.24.0", - "resolved": "https://registry.npmjs.org/@esbuild/linux-riscv64/-/linux-riscv64-0.24.0.tgz", - "integrity": "sha512-bEh7dMn/h3QxeR2KTy1DUszQjUrIHPZKyO6aN1X4BCnhfYhuQqedHaa5MxSQA/06j3GpiIlFGSsy1c7Gf9padw==", + "version": "0.24.2", + "resolved": "https://registry.npmjs.org/@esbuild/linux-riscv64/-/linux-riscv64-0.24.2.tgz", + "integrity": "sha512-4eSFWnU9Hhd68fW16GD0TINewo1L6dRrB+oLNNbYyMUAeOD2yCK5KXGK1GH4qD/kT+bTEXjsyTCiJGHPZ3eM9Q==", "cpu": [ "riscv64" ], "dev": true, - "license": "MIT", "optional": true, "os": [ "linux" @@ -1131,14 +1116,13 @@ } }, "node_modules/@esbuild/linux-s390x": { - "version": "0.24.0", - "resolved": "https://registry.npmjs.org/@esbuild/linux-s390x/-/linux-s390x-0.24.0.tgz", - "integrity": "sha512-ZcQ6+qRkw1UcZGPyrCiHHkmBaj9SiCD8Oqd556HldP+QlpUIe2Wgn3ehQGVoPOvZvtHm8HPx+bH20c9pvbkX3g==", + "version": "0.24.2", + "resolved": "https://registry.npmjs.org/@esbuild/linux-s390x/-/linux-s390x-0.24.2.tgz", + "integrity": "sha512-S0Bh0A53b0YHL2XEXC20bHLuGMOhFDO6GN4b3YjRLK//Ep3ql3erpNcPlEFed93hsQAjAQDNsvcK+hV90FubSw==", "cpu": [ "s390x" ], "dev": true, - "license": "MIT", "optional": true, "os": [ "linux" @@ -1148,14 +1132,13 @@ } }, "node_modules/@esbuild/linux-x64": { - "version": "0.24.0", - "resolved": "https://registry.npmjs.org/@esbuild/linux-x64/-/linux-x64-0.24.0.tgz", - "integrity": "sha512-vbutsFqQ+foy3wSSbmjBXXIJ6PL3scghJoM8zCL142cGaZKAdCZHyf+Bpu/MmX9zT9Q0zFBVKb36Ma5Fzfa8xA==", + "version": "0.24.2", + "resolved": "https://registry.npmjs.org/@esbuild/linux-x64/-/linux-x64-0.24.2.tgz", + "integrity": "sha512-8Qi4nQcCTbLnK9WoMjdC9NiTG6/E38RNICU6sUNqK0QFxCYgoARqVqxdFmWkdonVsvGqWhmm7MO0jyTqLqwj0Q==", "cpu": [ "x64" ], "dev": true, - "license": "MIT", "optional": true, "os": [ "linux" @@ -1164,15 +1147,30 @@ "node": ">=18" } }, + "node_modules/@esbuild/netbsd-arm64": { + "version": "0.24.2", + "resolved": "https://registry.npmjs.org/@esbuild/netbsd-arm64/-/netbsd-arm64-0.24.2.tgz", + "integrity": "sha512-wuLK/VztRRpMt9zyHSazyCVdCXlpHkKm34WUyinD2lzK07FAHTq0KQvZZlXikNWkDGoT6x3TD51jKQ7gMVpopw==", + "cpu": [ + "arm64" + ], + "dev": true, + "optional": true, + "os": [ + "netbsd" + ], + "engines": { + "node": ">=18" + } + }, "node_modules/@esbuild/netbsd-x64": { - "version": "0.24.0", - "resolved": "https://registry.npmjs.org/@esbuild/netbsd-x64/-/netbsd-x64-0.24.0.tgz", - "integrity": "sha512-hjQ0R/ulkO8fCYFsG0FZoH+pWgTTDreqpqY7UnQntnaKv95uP5iW3+dChxnx7C3trQQU40S+OgWhUVwCjVFLvg==", + "version": "0.24.2", + "resolved": "https://registry.npmjs.org/@esbuild/netbsd-x64/-/netbsd-x64-0.24.2.tgz", + "integrity": "sha512-VefFaQUc4FMmJuAxmIHgUmfNiLXY438XrL4GDNV1Y1H/RW3qow68xTwjZKfj/+Plp9NANmzbH5R40Meudu8mmw==", "cpu": [ "x64" ], "dev": true, - "license": "MIT", "optional": true, "os": [ "netbsd" @@ -1182,14 +1180,13 @@ } }, "node_modules/@esbuild/openbsd-arm64": { - "version": "0.24.0", - "resolved": "https://registry.npmjs.org/@esbuild/openbsd-arm64/-/openbsd-arm64-0.24.0.tgz", - "integrity": "sha512-MD9uzzkPQbYehwcN583yx3Tu5M8EIoTD+tUgKF982WYL9Pf5rKy9ltgD0eUgs8pvKnmizxjXZyLt0z6DC3rRXg==", + "version": "0.24.2", + "resolved": "https://registry.npmjs.org/@esbuild/openbsd-arm64/-/openbsd-arm64-0.24.2.tgz", + "integrity": "sha512-YQbi46SBct6iKnszhSvdluqDmxCJA+Pu280Av9WICNwQmMxV7nLRHZfjQzwbPs3jeWnuAhE9Jy0NrnJ12Oz+0A==", "cpu": [ "arm64" ], "dev": true, - "license": "MIT", "optional": true, "os": [ "openbsd" @@ -1199,14 +1196,13 @@ } }, "node_modules/@esbuild/openbsd-x64": { - "version": "0.24.0", - "resolved": "https://registry.npmjs.org/@esbuild/openbsd-x64/-/openbsd-x64-0.24.0.tgz", - "integrity": "sha512-4ir0aY1NGUhIC1hdoCzr1+5b43mw99uNwVzhIq1OY3QcEwPDO3B7WNXBzaKY5Nsf1+N11i1eOfFcq+D/gOS15Q==", + "version": "0.24.2", + "resolved": "https://registry.npmjs.org/@esbuild/openbsd-x64/-/openbsd-x64-0.24.2.tgz", + "integrity": "sha512-+iDS6zpNM6EnJyWv0bMGLWSWeXGN/HTaF/LXHXHwejGsVi+ooqDfMCCTerNFxEkM3wYVcExkeGXNqshc9iMaOA==", "cpu": [ "x64" ], "dev": true, - "license": "MIT", "optional": true, "os": [ "openbsd" @@ -1216,14 +1212,13 @@ } }, "node_modules/@esbuild/sunos-x64": { - "version": "0.24.0", - "resolved": "https://registry.npmjs.org/@esbuild/sunos-x64/-/sunos-x64-0.24.0.tgz", - "integrity": "sha512-jVzdzsbM5xrotH+W5f1s+JtUy1UWgjU0Cf4wMvffTB8m6wP5/kx0KiaLHlbJO+dMgtxKV8RQ/JvtlFcdZ1zCPA==", + "version": "0.24.2", + "resolved": "https://registry.npmjs.org/@esbuild/sunos-x64/-/sunos-x64-0.24.2.tgz", + "integrity": "sha512-hTdsW27jcktEvpwNHJU4ZwWFGkz2zRJUz8pvddmXPtXDzVKTTINmlmga3ZzwcuMpUvLw7JkLy9QLKyGpD2Yxig==", "cpu": [ "x64" ], "dev": true, - "license": "MIT", "optional": true, "os": [ "sunos" @@ -1233,14 +1228,13 @@ } }, "node_modules/@esbuild/win32-arm64": { - "version": "0.24.0", - "resolved": "https://registry.npmjs.org/@esbuild/win32-arm64/-/win32-arm64-0.24.0.tgz", - "integrity": "sha512-iKc8GAslzRpBytO2/aN3d2yb2z8XTVfNV0PjGlCxKo5SgWmNXx82I/Q3aG1tFfS+A2igVCY97TJ8tnYwpUWLCA==", + "version": "0.24.2", + "resolved": "https://registry.npmjs.org/@esbuild/win32-arm64/-/win32-arm64-0.24.2.tgz", + "integrity": "sha512-LihEQ2BBKVFLOC9ZItT9iFprsE9tqjDjnbulhHoFxYQtQfai7qfluVODIYxt1PgdoyQkz23+01rzwNwYfutxUQ==", "cpu": [ "arm64" ], "dev": true, - "license": "MIT", "optional": true, "os": [ "win32" @@ -1250,14 +1244,13 @@ } }, "node_modules/@esbuild/win32-ia32": { - "version": "0.24.0", - "resolved": "https://registry.npmjs.org/@esbuild/win32-ia32/-/win32-ia32-0.24.0.tgz", - "integrity": "sha512-vQW36KZolfIudCcTnaTpmLQ24Ha1RjygBo39/aLkM2kmjkWmZGEJ5Gn9l5/7tzXA42QGIoWbICfg6KLLkIw6yw==", + "version": "0.24.2", + "resolved": "https://registry.npmjs.org/@esbuild/win32-ia32/-/win32-ia32-0.24.2.tgz", + "integrity": "sha512-q+iGUwfs8tncmFC9pcnD5IvRHAzmbwQ3GPS5/ceCyHdjXubwQWI12MKWSNSMYLJMq23/IUCvJMS76PDqXe1fxA==", "cpu": [ "ia32" ], "dev": true, - "license": "MIT", "optional": true, "os": [ "win32" @@ -1267,14 +1260,13 @@ } }, "node_modules/@esbuild/win32-x64": { - "version": "0.24.0", - "resolved": "https://registry.npmjs.org/@esbuild/win32-x64/-/win32-x64-0.24.0.tgz", - "integrity": "sha512-7IAFPrjSQIJrGsK6flwg7NFmwBoSTyF3rl7If0hNUFQU4ilTsEPL6GuMuU9BfIWVVGuRnuIidkSMC+c0Otu8IA==", + "version": "0.24.2", + "resolved": "https://registry.npmjs.org/@esbuild/win32-x64/-/win32-x64-0.24.2.tgz", + "integrity": "sha512-7VTgWzgMGvup6aSqDPLiW5zHaxYJGTO4OokMjIlrCtf+VpEL+cXKtCvg723iguPYI5oaUNdS+/V7OU2gvXVWEg==", "cpu": [ "x64" ], "dev": true, - "license": "MIT", "optional": true, "os": [ "win32" @@ -4474,12 +4466,11 @@ "license": "MIT" }, "node_modules/esbuild": { - "version": "0.24.0", - "resolved": "https://registry.npmjs.org/esbuild/-/esbuild-0.24.0.tgz", - "integrity": "sha512-FuLPevChGDshgSicjisSooU0cemp/sGXR841D5LHMB7mTVOmsEHcAxaH3irL53+8YDIeVNQEySh4DaYU/iuPqQ==", + "version": "0.24.2", + "resolved": "https://registry.npmjs.org/esbuild/-/esbuild-0.24.2.tgz", + "integrity": "sha512-+9egpBW8I3CD5XPe0n6BfT5fxLzxrlDzqydF3aviG+9ni1lDC/OvMHcxqEFV0+LANZG5R1bFMWfUrjVsdwxJvA==", "dev": true, "hasInstallScript": true, - "license": "MIT", "bin": { "esbuild": "bin/esbuild" }, @@ -4487,30 +4478,31 @@ "node": ">=18" }, "optionalDependencies": { - "@esbuild/aix-ppc64": "0.24.0", - "@esbuild/android-arm": "0.24.0", - "@esbuild/android-arm64": "0.24.0", - "@esbuild/android-x64": "0.24.0", - "@esbuild/darwin-arm64": "0.24.0", - "@esbuild/darwin-x64": "0.24.0", - "@esbuild/freebsd-arm64": "0.24.0", - "@esbuild/freebsd-x64": "0.24.0", - "@esbuild/linux-arm": "0.24.0", - "@esbuild/linux-arm64": "0.24.0", - "@esbuild/linux-ia32": "0.24.0", - "@esbuild/linux-loong64": "0.24.0", - "@esbuild/linux-mips64el": "0.24.0", - "@esbuild/linux-ppc64": "0.24.0", - "@esbuild/linux-riscv64": "0.24.0", - "@esbuild/linux-s390x": "0.24.0", - "@esbuild/linux-x64": "0.24.0", - "@esbuild/netbsd-x64": "0.24.0", - "@esbuild/openbsd-arm64": "0.24.0", - "@esbuild/openbsd-x64": "0.24.0", - "@esbuild/sunos-x64": "0.24.0", - "@esbuild/win32-arm64": "0.24.0", - "@esbuild/win32-ia32": "0.24.0", - "@esbuild/win32-x64": "0.24.0" + "@esbuild/aix-ppc64": "0.24.2", + "@esbuild/android-arm": "0.24.2", + "@esbuild/android-arm64": "0.24.2", + "@esbuild/android-x64": "0.24.2", + "@esbuild/darwin-arm64": "0.24.2", + "@esbuild/darwin-x64": "0.24.2", + "@esbuild/freebsd-arm64": "0.24.2", + "@esbuild/freebsd-x64": "0.24.2", + "@esbuild/linux-arm": "0.24.2", + "@esbuild/linux-arm64": "0.24.2", + "@esbuild/linux-ia32": "0.24.2", + "@esbuild/linux-loong64": "0.24.2", + "@esbuild/linux-mips64el": "0.24.2", + "@esbuild/linux-ppc64": "0.24.2", + "@esbuild/linux-riscv64": "0.24.2", + "@esbuild/linux-s390x": "0.24.2", + "@esbuild/linux-x64": "0.24.2", + "@esbuild/netbsd-arm64": "0.24.2", + "@esbuild/netbsd-x64": "0.24.2", + "@esbuild/openbsd-arm64": "0.24.2", + "@esbuild/openbsd-x64": "0.24.2", + "@esbuild/sunos-x64": "0.24.2", + "@esbuild/win32-arm64": "0.24.2", + "@esbuild/win32-ia32": "0.24.2", + "@esbuild/win32-x64": "0.24.2" } }, "node_modules/escalade": { @@ -9164,13 +9156,12 @@ } }, "node_modules/vite": { - "version": "6.0.3", - "resolved": "https://registry.npmjs.org/vite/-/vite-6.0.3.tgz", - "integrity": "sha512-Cmuo5P0ENTN6HxLSo6IHsjCLn/81Vgrp81oaiFFMRa8gGDj5xEjIcEpf2ZymZtZR8oU0P2JX5WuUp/rlXcHkAw==", + "version": "6.0.7", + "resolved": "https://registry.npmjs.org/vite/-/vite-6.0.7.tgz", + "integrity": "sha512-RDt8r/7qx9940f8FcOIAH9PTViRrghKaK2K1jY3RaAURrEUbm9Du1mJ72G+jlhtG3WwodnfzY8ORQZbBavZEAQ==", "dev": true, - "license": "MIT", "dependencies": { - "esbuild": "^0.24.0", + "esbuild": "^0.24.2", "postcss": "^8.4.49", "rollup": "^4.23.0" }, diff --git a/web/ui/package.json b/web/ui/package.json index 62b3f26246..10ba19538f 100644 --- a/web/ui/package.json +++ b/web/ui/package.json @@ -22,6 +22,6 @@ "prettier": "^3.4.2", "ts-jest": "^29.2.2", "typescript": "^5.7.2", - "vite": "^6.0.3" + "vite": "^6.0.7" } } From 1542e73b663be8507253dec920437647107de5fb Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Jan 2025 19:07:29 +0100 Subject: [PATCH 054/110] chore(deps): bump github.com/prometheus/sigv4 from 0.1.0 to 0.1.1 (#15764) Bumps [github.com/prometheus/sigv4](https://github.com/prometheus/sigv4) from 0.1.0 to 0.1.1. - [Release notes](https://github.com/prometheus/sigv4/releases) - [Changelog](https://github.com/prometheus/sigv4/blob/main/RELEASE.md) - [Commits](https://github.com/prometheus/sigv4/compare/v0.1.0...v0.1.1) --- updated-dependencies: - dependency-name: github.com/prometheus/sigv4 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index 01af5f1c8b..941e6439d4 100644 --- a/go.mod +++ b/go.mod @@ -55,7 +55,7 @@ require ( github.com/prometheus/common v0.61.0 github.com/prometheus/common/assets v0.2.0 github.com/prometheus/exporter-toolkit v0.13.2 - github.com/prometheus/sigv4 v0.1.0 + github.com/prometheus/sigv4 v0.1.1 github.com/scaleway/scaleway-sdk-go v1.0.0-beta.30 github.com/shurcooL/httpfs v0.0.0-20230704072500-f1e31cf0ba5c github.com/stretchr/testify v1.10.0 diff --git a/go.sum b/go.sum index 72abefc4c3..bc4fa905c8 100644 --- a/go.sum +++ b/go.sum @@ -437,8 +437,8 @@ github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsT github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= -github.com/prometheus/sigv4 v0.1.0 h1:FgxH+m1qf9dGQ4w8Dd6VkthmpFQfGTzUeavMoQeG1LA= -github.com/prometheus/sigv4 v0.1.0/go.mod h1:doosPW9dOitMzYe2I2BN0jZqUuBrGPbXrNsTScN18iU= +github.com/prometheus/sigv4 v0.1.1 h1:UJxjOqVcXctZlwDjpUpZ2OiMWJdFijgSofwLzO1Xk0Q= +github.com/prometheus/sigv4 v0.1.1/go.mod h1:RAmWVKqx0bwi0Qm4lrKMXFM0nhpesBcenfCtz9qRyH8= github.com/redis/go-redis/v9 v9.6.1 h1:HHDteefn6ZkTtY5fGUE8tj8uy85AHk6zP7CpzIAM0y4= github.com/redis/go-redis/v9 v9.6.1/go.mod h1:0C0c6ycQsdpVNQpxb1njEQIqkx5UcsM8FJCQLgE9+RA= github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ= From 042cea1e18ce6332a88a3146e483292bc5212420 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Jan 2025 19:08:03 +0100 Subject: [PATCH 055/110] chore(deps): bump google.golang.org/protobuf from 1.36.0 to 1.36.1 (#15767) Bumps google.golang.org/protobuf from 1.36.0 to 1.36.1. --- updated-dependencies: - dependency-name: google.golang.org/protobuf dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index 941e6439d4..8b1f8a1c83 100644 --- a/go.mod +++ b/go.mod @@ -82,7 +82,7 @@ require ( google.golang.org/api v0.213.0 google.golang.org/genproto/googleapis/api v0.0.0-20241216192217-9240e9c98484 google.golang.org/grpc v1.69.0 - google.golang.org/protobuf v1.36.0 + google.golang.org/protobuf v1.36.1 gopkg.in/yaml.v2 v2.4.0 gopkg.in/yaml.v3 v3.0.1 k8s.io/api v0.31.3 diff --git a/go.sum b/go.sum index bc4fa905c8..fa2eb22242 100644 --- a/go.sum +++ b/go.sum @@ -650,8 +650,8 @@ google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8 google.golang.org/grpc v1.33.1/go.mod h1:fr5YgcSWrqhRRxogOsw7RzIpsmvOZ6IcH4kBYTpR3n0= google.golang.org/grpc v1.69.0 h1:quSiOM1GJPmPH5XtU+BCoVXcDVJJAzNcoyfC2cCjGkI= google.golang.org/grpc v1.69.0/go.mod h1:vyjdE6jLBI76dgpDojsFGNaHlxdjXN9ghpnd2o7JGZ4= -google.golang.org/protobuf v1.36.0 h1:mjIs9gYtt56AzC4ZaffQuh88TZurBGhIJMBZGSxNerQ= -google.golang.org/protobuf v1.36.0/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= +google.golang.org/protobuf v1.36.1 h1:yBPeRvTftaleIgM3PZ/WBIZ7XM/eEYAaEyCwvyjq/gk= +google.golang.org/protobuf v1.36.1/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= From 337c7394181fe975c8359f7904b5f6fdec6c52fd Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Jan 2025 19:11:53 +0100 Subject: [PATCH 056/110] chore(deps): bump github.com/linode/linodego from 1.43.0 to 1.44.0 (#15770) Bumps [github.com/linode/linodego](https://github.com/linode/linodego) from 1.43.0 to 1.44.0. - [Release notes](https://github.com/linode/linodego/releases) - [Commits](https://github.com/linode/linodego/compare/v1.43.0...v1.44.0) --- updated-dependencies: - dependency-name: github.com/linode/linodego dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- go.mod | 4 ++-- go.sum | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/go.mod b/go.mod index 8b1f8a1c83..6ae5ba9675 100644 --- a/go.mod +++ b/go.mod @@ -41,7 +41,7 @@ require ( github.com/json-iterator/go v1.1.12 github.com/klauspost/compress v1.17.11 github.com/kolo/xmlrpc v0.0.0-20220921171641-a4b6fa1dd06b - github.com/linode/linodego v1.43.0 + github.com/linode/linodego v1.44.0 github.com/miekg/dns v1.1.62 github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f @@ -127,7 +127,7 @@ require ( github.com/go-openapi/spec v0.20.14 // indirect github.com/go-openapi/swag v0.23.0 // indirect github.com/go-openapi/validate v0.23.0 // indirect - github.com/go-resty/resty/v2 v2.15.3 // indirect + github.com/go-resty/resty/v2 v2.16.2 // indirect github.com/godbus/dbus/v5 v5.0.4 // indirect github.com/golang-jwt/jwt/v5 v5.2.1 // indirect github.com/golang/glog v1.2.2 // indirect diff --git a/go.sum b/go.sum index fa2eb22242..b264521043 100644 --- a/go.sum +++ b/go.sum @@ -160,8 +160,8 @@ github.com/go-openapi/swag v0.23.0 h1:vsEVJDUo2hPJ2tu0/Xc+4noaxyEffXNIs3cOULZ+Gr github.com/go-openapi/swag v0.23.0/go.mod h1:esZ8ITTYEsH1V2trKHjAN8Ai7xHb8RV+YSZ577vPjgQ= github.com/go-openapi/validate v0.23.0 h1:2l7PJLzCis4YUGEoW6eoQw3WhyM65WSIcjX6SQnlfDw= github.com/go-openapi/validate v0.23.0/go.mod h1:EeiAZ5bmpSIOJV1WLfyYF9qp/B1ZgSaEpHTJHtN5cbE= -github.com/go-resty/resty/v2 v2.15.3 h1:bqff+hcqAflpiF591hhJzNdkRsFhlB96CYfBwSFvql8= -github.com/go-resty/resty/v2 v2.15.3/go.mod h1:0fHAoK7JoBy/Ch36N8VFeMsK7xQOHhvWaC3iOktwmIU= +github.com/go-resty/resty/v2 v2.16.2 h1:CpRqTjIzq/rweXUt9+GxzzQdlkqMdt8Lm/fuK/CAbAg= +github.com/go-resty/resty/v2 v2.16.2/go.mod h1:0fHAoK7JoBy/Ch36N8VFeMsK7xQOHhvWaC3iOktwmIU= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/go-task/slim-sprig/v3 v3.0.0 h1:sUs3vkvUymDpBKi3qH1YSqBQk9+9D/8M2mN1vB6EwHI= github.com/go-task/slim-sprig/v3 v3.0.0/go.mod h1:W848ghGpv3Qj3dhTPRyJypKRiqCdHZiAzKg9hl15HA8= @@ -317,8 +317,8 @@ github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/kylelemons/godebug v1.1.0 h1:RPNrshWIDI6G2gRW9EHilWtl7Z6Sb1BR0xunSBf0SNc= github.com/kylelemons/godebug v1.1.0/go.mod h1:9/0rRGxNHcop5bhtWyNeEfOS8JIWk580+fNqagV/RAw= -github.com/linode/linodego v1.43.0 h1:sGeBB3caZt7vKBoPS5p4AVzmlG4JoqQOdigIibx3egk= -github.com/linode/linodego v1.43.0/go.mod h1:n4TMFu1UVNala+icHqrTEFFaicYSF74cSAUG5zkTwfA= +github.com/linode/linodego v1.44.0 h1:JZLLWzCAx3CmHSV9NmCoXisuqKtrmPhfY9MrgvaHMUY= +github.com/linode/linodego v1.44.0/go.mod h1:umdoNOmtbqAdGQbmQnPFZ2YS4US+/mU/1bA7MjoKAvg= github.com/mailru/easyjson v0.7.7 h1:UGYAvKxe3sBsEDzO8ZeWOSlIQfWFlxbzLZe7hwFURr0= github.com/mailru/easyjson v0.7.7/go.mod h1:xzfreul335JAWq5oZzymOObrkdz5UnU4kGfJJLY9Nlc= github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= From 3351e79a504724782168e61cc7ba84985670ddf1 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Jan 2025 19:12:25 +0100 Subject: [PATCH 057/110] chore(deps-dev): bump @types/node in /web/ui/react-app (#15768) Bumps [@types/node](https://github.com/DefinitelyTyped/DefinitelyTyped/tree/HEAD/types/node) from 22.10.2 to 22.10.3. - [Release notes](https://github.com/DefinitelyTyped/DefinitelyTyped/releases) - [Commits](https://github.com/DefinitelyTyped/DefinitelyTyped/commits/HEAD/types/node) --- updated-dependencies: - dependency-name: "@types/node" dependency-type: direct:development update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- web/ui/react-app/package-lock.json | 9 ++++----- web/ui/react-app/package.json | 2 +- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/web/ui/react-app/package-lock.json b/web/ui/react-app/package-lock.json index 6dec476163..f06b873077 100644 --- a/web/ui/react-app/package-lock.json +++ b/web/ui/react-app/package-lock.json @@ -53,7 +53,7 @@ "@types/flot": "0.0.36", "@types/jest": "^29.5.14", "@types/jquery": "^3.5.32", - "@types/node": "^22.10.2", + "@types/node": "^22.10.3", "@types/react": "^17.0.71", "@types/react-copy-to-clipboard": "^5.0.7", "@types/react-dom": "^17.0.25", @@ -5403,10 +5403,9 @@ "dev": true }, "node_modules/@types/node": { - "version": "22.10.2", - "resolved": "https://registry.npmjs.org/@types/node/-/node-22.10.2.tgz", - "integrity": "sha512-Xxr6BBRCAOQixvonOye19wnzyDiUtTeqldOOmj3CkeblonbccA12PFwlufvRdrpjXxqnmUaeiU5EOA+7s5diUQ==", - "license": "MIT", + "version": "22.10.3", + "resolved": "https://registry.npmjs.org/@types/node/-/node-22.10.3.tgz", + "integrity": "sha512-DifAyw4BkrufCILvD3ucnuN8eydUfc/C1GlyrnI+LK6543w5/L3VeVgf05o3B4fqSXP1dKYLOZsKfutpxPzZrw==", "dependencies": { "undici-types": "~6.20.0" } diff --git a/web/ui/react-app/package.json b/web/ui/react-app/package.json index a801be62d7..685db2592b 100644 --- a/web/ui/react-app/package.json +++ b/web/ui/react-app/package.json @@ -70,7 +70,7 @@ "@types/flot": "0.0.36", "@types/jest": "^29.5.14", "@types/jquery": "^3.5.32", - "@types/node": "^22.10.2", + "@types/node": "^22.10.3", "@types/react": "^17.0.71", "@types/react-copy-to-clipboard": "^5.0.7", "@types/react-dom": "^17.0.25", From 2a5cd8179dfa2e6871f314d423f5ba1bd4663ace Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Jan 2025 19:13:12 +0100 Subject: [PATCH 058/110] chore(deps): bump github/codeql-action from 3.27.7 to 3.28.0 (#15759) Bumps [github/codeql-action](https://github.com/github/codeql-action) from 3.27.7 to 3.28.0. - [Release notes](https://github.com/github/codeql-action/releases) - [Changelog](https://github.com/github/codeql-action/blob/main/CHANGELOG.md) - [Commits](https://github.com/github/codeql-action/compare/babb554ede22fd5605947329c4d04d8e7a0b8155...48ab28a6f5dbc2a99bf1e0131198dd8f1df78169) --- updated-dependencies: - dependency-name: github/codeql-action dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/codeql-analysis.yml | 6 +++--- .github/workflows/scorecards.yml | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/.github/workflows/codeql-analysis.yml b/.github/workflows/codeql-analysis.yml index 56839da9cb..9a522fb6c9 100644 --- a/.github/workflows/codeql-analysis.yml +++ b/.github/workflows/codeql-analysis.yml @@ -27,12 +27,12 @@ jobs: uses: actions/checkout@11bd71901bbe5b1630ceea73d27597364c9af683 # v4.2.2 - name: Initialize CodeQL - uses: github/codeql-action/init@babb554ede22fd5605947329c4d04d8e7a0b8155 # v3.27.7 + uses: github/codeql-action/init@48ab28a6f5dbc2a99bf1e0131198dd8f1df78169 # v3.28.0 with: languages: ${{ matrix.language }} - name: Autobuild - uses: github/codeql-action/autobuild@babb554ede22fd5605947329c4d04d8e7a0b8155 # v3.27.7 + uses: github/codeql-action/autobuild@48ab28a6f5dbc2a99bf1e0131198dd8f1df78169 # v3.28.0 - name: Perform CodeQL Analysis - uses: github/codeql-action/analyze@babb554ede22fd5605947329c4d04d8e7a0b8155 # v3.27.7 + uses: github/codeql-action/analyze@48ab28a6f5dbc2a99bf1e0131198dd8f1df78169 # v3.28.0 diff --git a/.github/workflows/scorecards.yml b/.github/workflows/scorecards.yml index edb347abea..212c2cd617 100644 --- a/.github/workflows/scorecards.yml +++ b/.github/workflows/scorecards.yml @@ -45,6 +45,6 @@ jobs: # Upload the results to GitHub's code scanning dashboard. - name: "Upload to code-scanning" - uses: github/codeql-action/upload-sarif@babb554ede22fd5605947329c4d04d8e7a0b8155 # tag=v3.27.7 + uses: github/codeql-action/upload-sarif@48ab28a6f5dbc2a99bf1e0131198dd8f1df78169 # tag=v3.28.0 with: sarif_file: results.sarif From ebb3c4e35f705f19aeeb2646cc81ba3328079667 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Jan 2025 19:13:29 +0100 Subject: [PATCH 059/110] chore(deps): bump actions/upload-artifact from 4.4.3 to 4.5.0 (#15757) Bumps [actions/upload-artifact](https://github.com/actions/upload-artifact) from 4.4.3 to 4.5.0. - [Release notes](https://github.com/actions/upload-artifact/releases) - [Commits](https://github.com/actions/upload-artifact/compare/b4b15b8c7c6ac21ea08fcf65892d2ee8f75cf882...6f51ac03b9356f520e9adb1b1b7802705f340c2b) --- updated-dependencies: - dependency-name: actions/upload-artifact dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/fuzzing.yml | 2 +- .github/workflows/scorecards.yml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/fuzzing.yml b/.github/workflows/fuzzing.yml index 5f1b0f25ce..9f5ef96296 100644 --- a/.github/workflows/fuzzing.yml +++ b/.github/workflows/fuzzing.yml @@ -21,7 +21,7 @@ jobs: fuzz-seconds: 600 dry-run: false - name: Upload Crash - uses: actions/upload-artifact@b4b15b8c7c6ac21ea08fcf65892d2ee8f75cf882 # v4.4.3 + uses: actions/upload-artifact@6f51ac03b9356f520e9adb1b1b7802705f340c2b # v4.5.0 if: failure() && steps.build.outcome == 'success' with: name: artifacts diff --git a/.github/workflows/scorecards.yml b/.github/workflows/scorecards.yml index 212c2cd617..1415fead8e 100644 --- a/.github/workflows/scorecards.yml +++ b/.github/workflows/scorecards.yml @@ -37,7 +37,7 @@ jobs: # Upload the results as artifacts (optional). Commenting out will disable uploads of run results in SARIF # format to the repository Actions tab. - name: "Upload artifact" - uses: actions/upload-artifact@b4b15b8c7c6ac21ea08fcf65892d2ee8f75cf882 # tag=v4.4.3 + uses: actions/upload-artifact@6f51ac03b9356f520e9adb1b1b7802705f340c2b # tag=v4.5.0 with: name: SARIF file path: results.sarif From cb0dc8bc555a1d10a1ece8175b4aa85b42437b11 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Jan 2025 19:13:47 +0100 Subject: [PATCH 060/110] chore(deps): bump bufbuild/buf-setup-action from 1.47.2 to 1.48.0 (#15756) Bumps [bufbuild/buf-setup-action](https://github.com/bufbuild/buf-setup-action) from 1.47.2 to 1.48.0. - [Release notes](https://github.com/bufbuild/buf-setup-action/releases) - [Commits](https://github.com/bufbuild/buf-setup-action/compare/9672cee01808979ea1249f81d6d321217b9a10f6...1115d0acd3d2a120b30023fac52abc46807c8fd6) --- updated-dependencies: - dependency-name: bufbuild/buf-setup-action dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/buf-lint.yml | 2 +- .github/workflows/buf.yml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/buf-lint.yml b/.github/workflows/buf-lint.yml index d03f190769..31e4d7ee2a 100644 --- a/.github/workflows/buf-lint.yml +++ b/.github/workflows/buf-lint.yml @@ -13,7 +13,7 @@ jobs: runs-on: ubuntu-latest steps: - uses: actions/checkout@11bd71901bbe5b1630ceea73d27597364c9af683 # v4.2.2 - - uses: bufbuild/buf-setup-action@9672cee01808979ea1249f81d6d321217b9a10f6 # v1.47.2 + - uses: bufbuild/buf-setup-action@1115d0acd3d2a120b30023fac52abc46807c8fd6 # v1.48.0 with: github_token: ${{ secrets.GITHUB_TOKEN }} - uses: bufbuild/buf-lint-action@06f9dd823d873146471cfaaf108a993fe00e5325 # v1.1.1 diff --git a/.github/workflows/buf.yml b/.github/workflows/buf.yml index bf8ae3f6a4..b72b927bb0 100644 --- a/.github/workflows/buf.yml +++ b/.github/workflows/buf.yml @@ -13,7 +13,7 @@ jobs: if: github.repository_owner == 'prometheus' steps: - uses: actions/checkout@11bd71901bbe5b1630ceea73d27597364c9af683 # v4.2.2 - - uses: bufbuild/buf-setup-action@9672cee01808979ea1249f81d6d321217b9a10f6 # v1.47.2 + - uses: bufbuild/buf-setup-action@1115d0acd3d2a120b30023fac52abc46807c8fd6 # v1.48.0 with: github_token: ${{ secrets.GITHUB_TOKEN }} - uses: bufbuild/buf-lint-action@06f9dd823d873146471cfaaf108a993fe00e5325 # v1.1.1 From f5976c0afb4f4aa5d3e878d05b43edcbea527198 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Jan 2025 19:14:43 +0100 Subject: [PATCH 061/110] chore(deps): bump sass from 1.82.0 to 1.83.0 in /web/ui/react-app (#15762) Bumps [sass](https://github.com/sass/dart-sass) from 1.82.0 to 1.83.0. - [Release notes](https://github.com/sass/dart-sass/releases) - [Changelog](https://github.com/sass/dart-sass/blob/main/CHANGELOG.md) - [Commits](https://github.com/sass/dart-sass/compare/1.82.0...1.83.0) --- updated-dependencies: - dependency-name: sass dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- web/ui/react-app/package-lock.json | 9 ++++----- web/ui/react-app/package.json | 2 +- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/web/ui/react-app/package-lock.json b/web/ui/react-app/package-lock.json index f06b873077..ece343d719 100644 --- a/web/ui/react-app/package-lock.json +++ b/web/ui/react-app/package-lock.json @@ -43,7 +43,7 @@ "react-test-renderer": "^17.0.2", "reactstrap": "^8.10.1", "sanitize-html": "^2.13.1", - "sass": "1.82.0", + "sass": "1.83.0", "tempusdominus-bootstrap-4": "^5.39.2", "tempusdominus-core": "^5.19.3" }, @@ -21202,10 +21202,9 @@ "dev": true }, "node_modules/sass": { - "version": "1.82.0", - "resolved": "https://registry.npmjs.org/sass/-/sass-1.82.0.tgz", - "integrity": "sha512-j4GMCTa8elGyN9A7x7bEglx0VgSpNUG4W4wNedQ33wSMdnkqQCT8HTwOaVSV4e6yQovcu/3Oc4coJP/l0xhL2Q==", - "license": "MIT", + "version": "1.83.0", + "resolved": "https://registry.npmjs.org/sass/-/sass-1.83.0.tgz", + "integrity": "sha512-qsSxlayzoOjdvXMVLkzF84DJFc2HZEL/rFyGIKbbilYtAvlCxyuzUeff9LawTn4btVnLKg75Z8MMr1lxU1lfGw==", "dependencies": { "chokidar": "^4.0.0", "immutable": "^5.0.2", diff --git a/web/ui/react-app/package.json b/web/ui/react-app/package.json index 685db2592b..4e699abfb9 100644 --- a/web/ui/react-app/package.json +++ b/web/ui/react-app/package.json @@ -38,7 +38,7 @@ "react-test-renderer": "^17.0.2", "reactstrap": "^8.10.1", "sanitize-html": "^2.13.1", - "sass": "1.82.0", + "sass": "1.83.0", "tempusdominus-bootstrap-4": "^5.39.2", "tempusdominus-core": "^5.19.3" }, From 0bc95c388f4b6c7d083611a02bbfe027af7100a7 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Jan 2025 19:46:39 +0100 Subject: [PATCH 062/110] chore(deps): bump google.golang.org/api from 0.213.0 to 0.216.0 (#15812) Bumps [google.golang.org/api](https://github.com/googleapis/google-api-go-client) from 0.213.0 to 0.216.0. - [Release notes](https://github.com/googleapis/google-api-go-client/releases) - [Changelog](https://github.com/googleapis/google-api-go-client/blob/main/CHANGES.md) - [Commits](https://github.com/googleapis/google-api-go-client/compare/v0.213.0...v0.216.0) --- updated-dependencies: - dependency-name: google.golang.org/api dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- go.mod | 14 +++++++------- go.sum | 28 ++++++++++++++-------------- 2 files changed, 21 insertions(+), 21 deletions(-) diff --git a/go.mod b/go.mod index 6ae5ba9675..ee88a0e5f9 100644 --- a/go.mod +++ b/go.mod @@ -74,14 +74,14 @@ require ( go.uber.org/automaxprocs v1.6.0 go.uber.org/goleak v1.3.0 go.uber.org/multierr v1.11.0 - golang.org/x/oauth2 v0.24.0 + golang.org/x/oauth2 v0.25.0 golang.org/x/sync v0.10.0 golang.org/x/sys v0.28.0 golang.org/x/text v0.21.0 golang.org/x/tools v0.28.0 - google.golang.org/api v0.213.0 + google.golang.org/api v0.216.0 google.golang.org/genproto/googleapis/api v0.0.0-20241216192217-9240e9c98484 - google.golang.org/grpc v1.69.0 + google.golang.org/grpc v1.69.2 google.golang.org/protobuf v1.36.1 gopkg.in/yaml.v2 v2.4.0 gopkg.in/yaml.v3 v3.0.1 @@ -137,7 +137,7 @@ require ( github.com/google/gofuzz v1.2.0 // indirect github.com/google/s2a-go v0.1.8 // indirect github.com/googleapis/enterprise-certificate-proxy v0.3.4 // indirect - github.com/googleapis/gax-go/v2 v2.14.0 // indirect + github.com/googleapis/gax-go/v2 v2.14.1 // indirect github.com/gorilla/websocket v1.5.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.24.0 // indirect github.com/hashicorp/cronexpr v1.1.2 // indirect @@ -189,10 +189,10 @@ require ( golang.org/x/crypto v0.31.0 // indirect golang.org/x/exp v0.0.0-20240119083558-1b970713d09a // indirect golang.org/x/mod v0.22.0 // indirect - golang.org/x/net v0.32.0 // indirect + golang.org/x/net v0.33.0 // indirect golang.org/x/term v0.27.0 // indirect - golang.org/x/time v0.8.0 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20241209162323-e6fa225c2576 // indirect + golang.org/x/time v0.9.0 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20250102185135-69823020774d // indirect gopkg.in/evanphx/json-patch.v4 v4.12.0 // indirect gopkg.in/inf.v0 v0.9.1 // indirect gopkg.in/ini.v1 v1.67.0 // indirect diff --git a/go.sum b/go.sum index b264521043..7ab4986307 100644 --- a/go.sum +++ b/go.sum @@ -213,8 +213,8 @@ github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/enterprise-certificate-proxy v0.3.4 h1:XYIDZApgAnrN1c855gTgghdIA6Stxb52D5RnLI1SLyw= github.com/googleapis/enterprise-certificate-proxy v0.3.4/go.mod h1:YKe7cfqYXjKGpGvmSg28/fFvhNzinZQm8DGnaburhGA= -github.com/googleapis/gax-go/v2 v2.14.0 h1:f+jMrjBPl+DL9nI4IQzLUxMq7XrAqFYB7hBPqMNIe8o= -github.com/googleapis/gax-go/v2 v2.14.0/go.mod h1:lhBCnjdLrWRaPvLWhmc8IS24m9mr07qSYnHncrgo+zk= +github.com/googleapis/gax-go/v2 v2.14.1 h1:hb0FFeiPaQskmvakKu5EbCbpntQn48jyHuvrkurSS/Q= +github.com/googleapis/gax-go/v2 v2.14.1/go.mod h1:Hb/NubMaVM88SrNkvl8X/o8XWwDJEPqouaLeN2IUxoA= github.com/gophercloud/gophercloud v1.14.1 h1:DTCNaTVGl8/cFu58O1JwWgis9gtISAFONqpMKNg/Vpw= github.com/gophercloud/gophercloud v1.14.1/go.mod h1:aAVqcocTSXh2vYFZ1JTvx4EQmfgzxRcNupUfxZbBNDM= github.com/gorilla/websocket v1.5.0 h1:PPwGk2jz7EePpoHN/+ClbZu8SPxiqlu12wZP/3sWmnc= @@ -558,12 +558,12 @@ golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwY golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210410081132-afb366fc7cd1/go.mod h1:9tjilg8BloeKEkVJvy7fQ90B1CfIiPueXVOjqfkSzI8= golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.32.0 h1:ZqPmj8Kzc+Y6e0+skZsuACbx+wzMgo5MQsJh9Qd6aYI= -golang.org/x/net v0.32.0/go.mod h1:CwU0IoeOlnQQWJ6ioyFrfRuomB8GKF6KbYXZVyeXNfs= +golang.org/x/net v0.33.0 h1:74SYHlV8BIgHIFC/LrYkOGIwL19eTYXQ5wc6TBuO36I= +golang.org/x/net v0.33.0/go.mod h1:HXLR5J+9DxmrqMwG9qjGCxZ+zKXxBru04zlTvWlWuN4= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/oauth2 v0.24.0 h1:KTBBxWqUa0ykRPLtV69rRto9TLXcqYkeswu48x/gvNE= -golang.org/x/oauth2 v0.24.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= +golang.org/x/oauth2 v0.25.0 h1:CY4y7XT9v0cRI9oupztF8AgiIu99L/ksR/Xp/6jrZ70= +golang.org/x/oauth2 v0.25.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -614,8 +614,8 @@ golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.21.0 h1:zyQAAkrwaneQ066sspRyJaG9VNi/YJ1NfzcGB3hZ/qo= golang.org/x/text v0.21.0/go.mod h1:4IBbMaMmOPCJ8SecivzSH54+73PCFmPWxNTLm+vZkEQ= -golang.org/x/time v0.8.0 h1:9i3RxcPv3PZnitoVGMPDKZSq1xW1gK1Xy3ArNOGZfEg= -golang.org/x/time v0.8.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= +golang.org/x/time v0.9.0 h1:EsRrnYcQiGH+5FfbgvV4AP7qEZstoyrHB0DzarOQ4ZY= +golang.org/x/time v0.9.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= @@ -632,8 +632,8 @@ golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8T golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -google.golang.org/api v0.213.0 h1:KmF6KaDyFqB417T68tMPbVmmwtIXs2VB60OJKIHB0xQ= -google.golang.org/api v0.213.0/go.mod h1:V0T5ZhNUUNpYAlL306gFZPFt5F5D/IeyLoktduYYnvQ= +google.golang.org/api v0.216.0 h1:xnEHy+xWFrtYInWPy8OdGFsyIfWJjtVnO39g7pz2BFY= +google.golang.org/api v0.216.0/go.mod h1:K9wzQMvWi47Z9IU7OgdOofvZuw75Ge3PPITImZR/UyI= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= @@ -641,15 +641,15 @@ google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98 google.golang.org/genproto v0.0.0-20200513103714-09dca8ec2884/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto/googleapis/api v0.0.0-20241216192217-9240e9c98484 h1:ChAdCYNQFDk5fYvFZMywKLIijG7TC2m1C2CMEu11G3o= google.golang.org/genproto/googleapis/api v0.0.0-20241216192217-9240e9c98484/go.mod h1:KRUmxRI4JmbpAm8gcZM4Jsffi859fo5LQjILwuqj9z8= -google.golang.org/genproto/googleapis/rpc v0.0.0-20241209162323-e6fa225c2576 h1:8ZmaLZE4XWrtU3MyClkYqqtl6Oegr3235h7jxsDyqCY= -google.golang.org/genproto/googleapis/rpc v0.0.0-20241209162323-e6fa225c2576/go.mod h1:5uTbfoYQed2U9p3KIj2/Zzm02PYhndfdmML0qC3q3FU= +google.golang.org/genproto/googleapis/rpc v0.0.0-20250102185135-69823020774d h1:xJJRGY7TJcvIlpSrN3K6LAWgNFUILlO+OMAqtg9aqnw= +google.golang.org/genproto/googleapis/rpc v0.0.0-20250102185135-69823020774d/go.mod h1:3ENsm/5D1mzDyhpzeRi1NR784I0BcofWBoSc5QqqMK4= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.33.1/go.mod h1:fr5YgcSWrqhRRxogOsw7RzIpsmvOZ6IcH4kBYTpR3n0= -google.golang.org/grpc v1.69.0 h1:quSiOM1GJPmPH5XtU+BCoVXcDVJJAzNcoyfC2cCjGkI= -google.golang.org/grpc v1.69.0/go.mod h1:vyjdE6jLBI76dgpDojsFGNaHlxdjXN9ghpnd2o7JGZ4= +google.golang.org/grpc v1.69.2 h1:U3S9QEtbXC0bYNvRtcoklF3xGtLViumSYxWykJS+7AU= +google.golang.org/grpc v1.69.2/go.mod h1:vyjdE6jLBI76dgpDojsFGNaHlxdjXN9ghpnd2o7JGZ4= google.golang.org/protobuf v1.36.1 h1:yBPeRvTftaleIgM3PZ/WBIZ7XM/eEYAaEyCwvyjq/gk= google.golang.org/protobuf v1.36.1/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= From 16420e859f38005c6f3c37428f4991780bc11e3c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Jan 2025 19:54:17 +0100 Subject: [PATCH 063/110] chore(deps): bump @codemirror/view in /web/ui/react-app (#15761) Bumps [@codemirror/view](https://github.com/codemirror/view) from 6.35.3 to 6.36.1. - [Changelog](https://github.com/codemirror/view/blob/main/CHANGELOG.md) - [Commits](https://github.com/codemirror/view/compare/6.35.3...6.36.1) --- updated-dependencies: - dependency-name: "@codemirror/view" dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- web/ui/react-app/package-lock.json | 9 ++++----- web/ui/react-app/package.json | 2 +- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/web/ui/react-app/package-lock.json b/web/ui/react-app/package-lock.json index ece343d719..0714720bb8 100644 --- a/web/ui/react-app/package-lock.json +++ b/web/ui/react-app/package-lock.json @@ -14,7 +14,7 @@ "@codemirror/lint": "^6.8.4", "@codemirror/search": "^6.5.8", "@codemirror/state": "^6.3.3", - "@codemirror/view": "^6.35.3", + "@codemirror/view": "^6.36.1", "@forevolve/bootstrap-dark": "^4.0.2", "@fortawesome/fontawesome-svg-core": "6.5.2", "@fortawesome/free-solid-svg-icons": "6.5.2", @@ -2312,10 +2312,9 @@ } }, "node_modules/@codemirror/view": { - "version": "6.35.3", - "resolved": "https://registry.npmjs.org/@codemirror/view/-/view-6.35.3.tgz", - "integrity": "sha512-ScY7L8+EGdPl4QtoBiOzE4FELp7JmNUsBvgBcCakXWM2uiv/K89VAzU3BMDscf0DsACLvTKePbd5+cFDTcei6g==", - "license": "MIT", + "version": "6.36.1", + "resolved": "https://registry.npmjs.org/@codemirror/view/-/view-6.36.1.tgz", + "integrity": "sha512-miD1nyT4m4uopZaDdO2uXU/LLHliKNYL9kB1C1wJHrunHLm/rpkb5QVSokqgw9hFqEZakrdlb/VGWX8aYZTslQ==", "dependencies": { "@codemirror/state": "^6.5.0", "style-mod": "^4.1.0", diff --git a/web/ui/react-app/package.json b/web/ui/react-app/package.json index 4e699abfb9..1ccbe38f67 100644 --- a/web/ui/react-app/package.json +++ b/web/ui/react-app/package.json @@ -9,7 +9,7 @@ "@codemirror/lint": "^6.8.4", "@codemirror/search": "^6.5.8", "@codemirror/state": "^6.3.3", - "@codemirror/view": "^6.35.3", + "@codemirror/view": "^6.36.1", "@forevolve/bootstrap-dark": "^4.0.2", "@fortawesome/fontawesome-svg-core": "6.5.2", "@fortawesome/free-solid-svg-icons": "6.5.2", From 2c2a1ac66d847cc4f5f3f35fd104aa52dfc0e58b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Jan 2025 19:55:02 +0100 Subject: [PATCH 064/110] chore(deps-dev): bump @eslint/js from 9.16.0 to 9.17.0 in /web/ui (#15753) Bumps [@eslint/js](https://github.com/eslint/eslint/tree/HEAD/packages/js) from 9.16.0 to 9.17.0. - [Release notes](https://github.com/eslint/eslint/releases) - [Changelog](https://github.com/eslint/eslint/blob/main/CHANGELOG.md) - [Commits](https://github.com/eslint/eslint/commits/v9.17.0/packages/js) --- updated-dependencies: - dependency-name: "@eslint/js" dependency-type: direct:development update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- web/ui/mantine-ui/package.json | 2 +- web/ui/package-lock.json | 17 +++++++++++++---- 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/web/ui/mantine-ui/package.json b/web/ui/mantine-ui/package.json index 5f3d95a2e3..89cb23f453 100644 --- a/web/ui/mantine-ui/package.json +++ b/web/ui/mantine-ui/package.json @@ -53,7 +53,7 @@ "devDependencies": { "@eslint/compat": "^1.2.4", "@eslint/eslintrc": "^3.2.0", - "@eslint/js": "^9.16.0", + "@eslint/js": "^9.17.0", "@types/react": "^18.3.5", "@types/react-dom": "^18.3.0", "@typescript-eslint/eslint-plugin": "^6.21.0", diff --git a/web/ui/package-lock.json b/web/ui/package-lock.json index a29e4a3d02..c1081e4a62 100644 --- a/web/ui/package-lock.json +++ b/web/ui/package-lock.json @@ -67,7 +67,7 @@ "devDependencies": { "@eslint/compat": "^1.2.4", "@eslint/eslintrc": "^3.2.0", - "@eslint/js": "^9.16.0", + "@eslint/js": "^9.17.0", "@types/react": "^18.3.5", "@types/react-dom": "^18.3.0", "@typescript-eslint/eslint-plugin": "^6.21.0", @@ -145,6 +145,15 @@ } } }, + "mantine-ui/node_modules/eslint/node_modules/@eslint/js": { + "version": "9.16.0", + "resolved": "https://registry.npmjs.org/@eslint/js/-/js-9.16.0.tgz", + "integrity": "sha512-tw2HxzQkrbeuvyj1tG2Yqq+0H9wGoI2IMk4EOsQeX+vmd75FtJAzf+gTA69WF+baUKRYQ3x2kbLE08js5OsTVg==", + "dev": true, + "engines": { + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" + } + }, "module/codemirror-promql": { "name": "@prometheus-io/codemirror-promql", "version": "0.301.0", @@ -1396,9 +1405,9 @@ } }, "node_modules/@eslint/js": { - "version": "9.16.0", - "resolved": "https://registry.npmjs.org/@eslint/js/-/js-9.16.0.tgz", - "integrity": "sha512-tw2HxzQkrbeuvyj1tG2Yqq+0H9wGoI2IMk4EOsQeX+vmd75FtJAzf+gTA69WF+baUKRYQ3x2kbLE08js5OsTVg==", + "version": "9.17.0", + "resolved": "https://registry.npmjs.org/@eslint/js/-/js-9.17.0.tgz", + "integrity": "sha512-Sxc4hqcs1kTu0iID3kcZDW3JHq2a77HO9P8CP6YEA/FpH3Ll8UXE2r/86Rz9YJLKme39S9vU5OWNjC6Xl0Cr3w==", "dev": true, "engines": { "node": "^18.18.0 || ^20.9.0 || >=21.1.0" From a59f23f55b2ef3741ed31589c91da8aa436d48ca Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Jan 2025 19:55:49 +0100 Subject: [PATCH 065/110] chore(deps): bump @uiw/react-codemirror from 4.23.6 to 4.23.7 in /web/ui (#15752) Bumps [@uiw/react-codemirror](https://github.com/uiwjs/react-codemirror) from 4.23.6 to 4.23.7. - [Release notes](https://github.com/uiwjs/react-codemirror/releases) - [Commits](https://github.com/uiwjs/react-codemirror/compare/v4.23.6...v4.23.7) --- updated-dependencies: - dependency-name: "@uiw/react-codemirror" dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- web/ui/mantine-ui/package.json | 2 +- web/ui/package-lock.json | 16 ++++++++-------- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/web/ui/mantine-ui/package.json b/web/ui/mantine-ui/package.json index 89cb23f453..d53c1fc841 100644 --- a/web/ui/mantine-ui/package.json +++ b/web/ui/mantine-ui/package.json @@ -36,7 +36,7 @@ "@testing-library/react": "^16.1.0", "@types/lodash": "^4.17.13", "@types/sanitize-html": "^2.13.0", - "@uiw/react-codemirror": "^4.23.6", + "@uiw/react-codemirror": "^4.23.7", "clsx": "^2.1.1", "dayjs": "^1.11.10", "lodash": "^4.17.21", diff --git a/web/ui/package-lock.json b/web/ui/package-lock.json index c1081e4a62..4c92167653 100644 --- a/web/ui/package-lock.json +++ b/web/ui/package-lock.json @@ -50,7 +50,7 @@ "@testing-library/react": "^16.1.0", "@types/lodash": "^4.17.13", "@types/sanitize-html": "^2.13.0", - "@uiw/react-codemirror": "^4.23.6", + "@uiw/react-codemirror": "^4.23.7", "clsx": "^2.1.1", "dayjs": "^1.11.10", "lodash": "^4.17.21", @@ -3270,9 +3270,9 @@ } }, "node_modules/@uiw/codemirror-extensions-basic-setup": { - "version": "4.23.6", - "resolved": "https://registry.npmjs.org/@uiw/codemirror-extensions-basic-setup/-/codemirror-extensions-basic-setup-4.23.6.tgz", - "integrity": "sha512-bvtq8IOvdkLJMhoJBRGPEzU51fMpPDwEhcAHp9xCR05MtbIokQgsnLXrmD1aZm6e7s/3q47H+qdSfAAkR5MkLA==", + "version": "4.23.7", + "resolved": "https://registry.npmjs.org/@uiw/codemirror-extensions-basic-setup/-/codemirror-extensions-basic-setup-4.23.7.tgz", + "integrity": "sha512-9/2EUa1Lck4kFKkR2BkxlZPpgD/EWuKHnOlysf1yHKZGraaZmZEaUw+utDK4QcuJc8Iz097vsLz4f4th5EU27g==", "dependencies": { "@codemirror/autocomplete": "^6.0.0", "@codemirror/commands": "^6.0.0", @@ -3296,15 +3296,15 @@ } }, "node_modules/@uiw/react-codemirror": { - "version": "4.23.6", - "resolved": "https://registry.npmjs.org/@uiw/react-codemirror/-/react-codemirror-4.23.6.tgz", - "integrity": "sha512-caYKGV6TfGLRV1HHD3p0G3FiVzKL1go7wes5XT2nWjB0+dTdyzyb81MKRSacptgZcotujfNO6QXn65uhETRAMw==", + "version": "4.23.7", + "resolved": "https://registry.npmjs.org/@uiw/react-codemirror/-/react-codemirror-4.23.7.tgz", + "integrity": "sha512-Nh/0P6W+kWta+ARp9YpnKPD9ick5teEnwmtNoPQnyd6NPv0EQP3Ui4YmRVNj1nkUEo+QjrAUaEfcejJ2up/HZA==", "dependencies": { "@babel/runtime": "^7.18.6", "@codemirror/commands": "^6.1.0", "@codemirror/state": "^6.1.1", "@codemirror/theme-one-dark": "^6.0.0", - "@uiw/codemirror-extensions-basic-setup": "4.23.6", + "@uiw/codemirror-extensions-basic-setup": "4.23.7", "codemirror": "^6.0.0" }, "funding": { From 23fe8bc1a304f1e585426cad0c44a6a4b9aa656c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Jan 2025 19:56:14 +0100 Subject: [PATCH 066/110] chore(deps-dev): bump globals from 15.13.0 to 15.14.0 in /web/ui (#15751) Bumps [globals](https://github.com/sindresorhus/globals) from 15.13.0 to 15.14.0. - [Release notes](https://github.com/sindresorhus/globals/releases) - [Commits](https://github.com/sindresorhus/globals/compare/v15.13.0...v15.14.0) --- updated-dependencies: - dependency-name: globals dependency-type: direct:development update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- web/ui/mantine-ui/package.json | 2 +- web/ui/package-lock.json | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/web/ui/mantine-ui/package.json b/web/ui/mantine-ui/package.json index d53c1fc841..e0200088e6 100644 --- a/web/ui/mantine-ui/package.json +++ b/web/ui/mantine-ui/package.json @@ -62,7 +62,7 @@ "eslint": "^9.16.0", "eslint-plugin-react-hooks": "^5.1.0", "eslint-plugin-react-refresh": "^0.4.16", - "globals": "^15.13.0", + "globals": "^15.14.0", "jsdom": "^25.0.1", "postcss": "^8.4.47", "postcss-preset-mantine": "^1.17.0", diff --git a/web/ui/package-lock.json b/web/ui/package-lock.json index 4c92167653..98c4c10984 100644 --- a/web/ui/package-lock.json +++ b/web/ui/package-lock.json @@ -76,7 +76,7 @@ "eslint": "^9.16.0", "eslint-plugin-react-hooks": "^5.1.0", "eslint-plugin-react-refresh": "^0.4.16", - "globals": "^15.13.0", + "globals": "^15.14.0", "jsdom": "^25.0.1", "postcss": "^8.4.47", "postcss-preset-mantine": "^1.17.0", @@ -5304,9 +5304,9 @@ } }, "node_modules/globals": { - "version": "15.13.0", - "resolved": "https://registry.npmjs.org/globals/-/globals-15.13.0.tgz", - "integrity": "sha512-49TewVEz0UxZjr1WYYsWpPrhyC/B/pA8Bq0fUmet2n+eR7yn0IvNzNaoBwnK6mdkzcN+se7Ez9zUgULTz2QH4g==", + "version": "15.14.0", + "resolved": "https://registry.npmjs.org/globals/-/globals-15.14.0.tgz", + "integrity": "sha512-OkToC372DtlQeje9/zHIo5CT8lRP/FUgEOKBEhU4e0abL7J7CD24fD9ohiLN5hagG/kWCYj4K5oaxxtj2Z0Dig==", "dev": true, "engines": { "node": ">=18" From e24375f1b33b92d1d2a1ca6b1fa84ae85b996528 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Jan 2025 19:56:42 +0100 Subject: [PATCH 067/110] chore(deps): bump @codemirror/language in /web/ui/react-app (#15754) Bumps [@codemirror/language](https://github.com/codemirror/language) from 6.10.6 to 6.10.8. - [Changelog](https://github.com/codemirror/language/blob/main/CHANGELOG.md) - [Commits](https://github.com/codemirror/language/compare/6.10.6...6.10.8) --- updated-dependencies: - dependency-name: "@codemirror/language" dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- web/ui/react-app/package-lock.json | 9 ++++----- web/ui/react-app/package.json | 2 +- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/web/ui/react-app/package-lock.json b/web/ui/react-app/package-lock.json index 0714720bb8..a74effa39c 100644 --- a/web/ui/react-app/package-lock.json +++ b/web/ui/react-app/package-lock.json @@ -10,7 +10,7 @@ "dependencies": { "@codemirror/autocomplete": "^6.18.3", "@codemirror/commands": "^6.7.1", - "@codemirror/language": "^6.10.6", + "@codemirror/language": "^6.10.8", "@codemirror/lint": "^6.8.4", "@codemirror/search": "^6.5.8", "@codemirror/state": "^6.3.3", @@ -2267,10 +2267,9 @@ } }, "node_modules/@codemirror/language": { - "version": "6.10.6", - "resolved": "https://registry.npmjs.org/@codemirror/language/-/language-6.10.6.tgz", - "integrity": "sha512-KrsbdCnxEztLVbB5PycWXFxas4EOyk/fPAfruSOnDDppevQgid2XZ+KbJ9u+fDikP/e7MW7HPBTvTb8JlZK9vA==", - "license": "MIT", + "version": "6.10.8", + "resolved": "https://registry.npmjs.org/@codemirror/language/-/language-6.10.8.tgz", + "integrity": "sha512-wcP8XPPhDH2vTqf181U8MbZnW+tDyPYy0UzVOa+oHORjyT+mhhom9vBd7dApJwoDz9Nb/a8kHjJIsuA/t8vNFw==", "dependencies": { "@codemirror/state": "^6.0.0", "@codemirror/view": "^6.23.0", diff --git a/web/ui/react-app/package.json b/web/ui/react-app/package.json index 1ccbe38f67..90fb4fb502 100644 --- a/web/ui/react-app/package.json +++ b/web/ui/react-app/package.json @@ -5,7 +5,7 @@ "dependencies": { "@codemirror/autocomplete": "^6.18.3", "@codemirror/commands": "^6.7.1", - "@codemirror/language": "^6.10.6", + "@codemirror/language": "^6.10.8", "@codemirror/lint": "^6.8.4", "@codemirror/search": "^6.5.8", "@codemirror/state": "^6.3.3", From 938bb65518c9f9afd0604ce3e6db7eafb0bfa90f Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Jan 2025 19:57:14 +0100 Subject: [PATCH 068/110] chore(deps): bump @mantine/code-highlight in /web/ui (#15749) Bumps [@mantine/code-highlight](https://github.com/mantinedev/mantine/tree/HEAD/packages/@mantine/code-highlight) from 7.15.0 to 7.15.2. - [Release notes](https://github.com/mantinedev/mantine/releases) - [Changelog](https://github.com/mantinedev/mantine/blob/master/CHANGELOG.md) - [Commits](https://github.com/mantinedev/mantine/commits/7.15.2/packages/@mantine/code-highlight) --- updated-dependencies: - dependency-name: "@mantine/code-highlight" dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- web/ui/mantine-ui/package.json | 2 +- web/ui/package-lock.json | 68 +++++++++++++++++----------------- 2 files changed, 36 insertions(+), 34 deletions(-) diff --git a/web/ui/mantine-ui/package.json b/web/ui/mantine-ui/package.json index e0200088e6..ad169cd582 100644 --- a/web/ui/mantine-ui/package.json +++ b/web/ui/mantine-ui/package.json @@ -20,7 +20,7 @@ "@floating-ui/dom": "^1.6.12", "@lezer/common": "^1.2.3", "@lezer/highlight": "^1.2.1", - "@mantine/code-highlight": "^7.15.0", + "@mantine/code-highlight": "^7.15.2", "@mantine/core": "^7.11.2", "@mantine/dates": "^7.15.0", "@mantine/hooks": "^7.11.2", diff --git a/web/ui/package-lock.json b/web/ui/package-lock.json index 98c4c10984..fc9a461621 100644 --- a/web/ui/package-lock.json +++ b/web/ui/package-lock.json @@ -34,7 +34,7 @@ "@floating-ui/dom": "^1.6.12", "@lezer/common": "^1.2.3", "@lezer/highlight": "^1.2.1", - "@mantine/code-highlight": "^7.15.0", + "@mantine/code-highlight": "^7.15.2", "@mantine/core": "^7.11.2", "@mantine/dates": "^7.15.0", "@mantine/hooks": "^7.11.2", @@ -2136,36 +2136,34 @@ } }, "node_modules/@mantine/code-highlight": { - "version": "7.15.0", - "resolved": "https://registry.npmjs.org/@mantine/code-highlight/-/code-highlight-7.15.0.tgz", - "integrity": "sha512-UJ1Qfjs7LigFQt/yuvjjWv7y6AGhfAc177tmlegs1E2OTVL8f11pM4PKGuoQxRD70Wwz+hG23ZNMumibEcRejg==", - "license": "MIT", + "version": "7.15.2", + "resolved": "https://registry.npmjs.org/@mantine/code-highlight/-/code-highlight-7.15.2.tgz", + "integrity": "sha512-vp4RLYtLQJ++W4TUVt5K78RAiwlqSTAM3N1yFdDjDF2d5sK3k7CEztYP2ahwroIuaMYgJy/iOIW7l5iMLBbl7w==", "dependencies": { "clsx": "^2.1.1", "highlight.js": "^11.10.0" }, "peerDependencies": { - "@mantine/core": "7.15.0", - "@mantine/hooks": "7.15.0", + "@mantine/core": "7.15.2", + "@mantine/hooks": "7.15.2", "react": "^18.x || ^19.x", "react-dom": "^18.x || ^19.x" } }, "node_modules/@mantine/core": { - "version": "7.15.0", - "resolved": "https://registry.npmjs.org/@mantine/core/-/core-7.15.0.tgz", - "integrity": "sha512-tZlRydfaEaaCNIi3tl1u+VtJxNyBoml2iCJAy6ZqcNNcjy/krJmta5lVtjUeApZfE33rkvr+3WVqGk0YjW6oSQ==", - "license": "MIT", + "version": "7.15.2", + "resolved": "https://registry.npmjs.org/@mantine/core/-/core-7.15.2.tgz", + "integrity": "sha512-640ns0L/HZAXYjz3+FRffr8UNcH1fU7ENUVxKLzqNA311Dcx0qS3byVKTY/IVJYln6AkjoEfIJMiixT9fCZBiQ==", "dependencies": { "@floating-ui/react": "^0.26.28", "clsx": "^2.1.1", "react-number-format": "^5.4.2", "react-remove-scroll": "^2.6.0", - "react-textarea-autosize": "8.5.5", + "react-textarea-autosize": "8.5.6", "type-fest": "^4.27.0" }, "peerDependencies": { - "@mantine/hooks": "7.15.0", + "@mantine/hooks": "7.15.2", "react": "^18.x || ^19.x", "react-dom": "^18.x || ^19.x" } @@ -2187,10 +2185,9 @@ } }, "node_modules/@mantine/hooks": { - "version": "7.15.0", - "resolved": "https://registry.npmjs.org/@mantine/hooks/-/hooks-7.15.0.tgz", - "integrity": "sha512-AV4ItRbVIWVDzpOPyj3ICrfQq7HEdKhO7IE7xxkdbJ4oj73DAq2jFsMoNdj3dN9u2tOn1bhPBIaP+8gKd0oAcw==", - "license": "MIT", + "version": "7.15.2", + "resolved": "https://registry.npmjs.org/@mantine/hooks/-/hooks-7.15.2.tgz", + "integrity": "sha512-p8dsW0fdJxzYhULbm1noFYRHuBvJHleYviC0BlwbkVySC8AsvFI8AmC3sMssWV3dQ3yQ/SidYo9U+K/czpDpZw==", "peerDependencies": { "react": "^18.x || ^19.x" } @@ -8091,9 +8088,9 @@ } }, "node_modules/react-textarea-autosize": { - "version": "8.5.5", - "resolved": "https://registry.npmjs.org/react-textarea-autosize/-/react-textarea-autosize-8.5.5.tgz", - "integrity": "sha512-CVA94zmfp8m4bSHtWwmANaBR8EPsKy2aZ7KwqhoS4Ftib87F9Kvi7XQhOixypPLMc6kVYgOXvKFuuzZDpHGRPg==", + "version": "8.5.6", + "resolved": "https://registry.npmjs.org/react-textarea-autosize/-/react-textarea-autosize-8.5.6.tgz", + "integrity": "sha512-aT3ioKXMa8f6zHYGebhbdMD2L00tKeRX1zuVuDx9YQK/JLLRSaSxq3ugECEmUB9z2kvk6bFSIoRHLkkUv0RJiw==", "dependencies": { "@babel/runtime": "^7.20.13", "use-composed-ref": "^1.3.0", @@ -8103,7 +8100,7 @@ "node": ">=10" }, "peerDependencies": { - "react": "^16.8.0 || ^17.0.0 || ^18.0.0" + "react": "^16.8.0 || ^17.0.0 || ^18.0.0 || ^19.0.0" } }, "node_modules/react-transition-group": { @@ -9052,19 +9049,24 @@ } }, "node_modules/use-composed-ref": { - "version": "1.3.0", - "resolved": "https://registry.npmjs.org/use-composed-ref/-/use-composed-ref-1.3.0.tgz", - "integrity": "sha512-GLMG0Jc/jiKov/3Ulid1wbv3r54K9HlMW29IWcDFPEqFkSO2nS0MuefWgMJpeHQ9YJeXDL3ZUF+P3jdXlZX/cQ==", + "version": "1.4.0", + "resolved": "https://registry.npmjs.org/use-composed-ref/-/use-composed-ref-1.4.0.tgz", + "integrity": "sha512-djviaxuOOh7wkj0paeO1Q/4wMZ8Zrnag5H6yBvzN7AKKe8beOaED9SF5/ByLqsku8NP4zQqsvM2u3ew/tJK8/w==", "peerDependencies": { - "react": "^16.8.0 || ^17.0.0 || ^18.0.0" + "react": "^16.8.0 || ^17.0.0 || ^18.0.0 || ^19.0.0" + }, + "peerDependenciesMeta": { + "@types/react": { + "optional": true + } } }, "node_modules/use-isomorphic-layout-effect": { - "version": "1.1.2", - "resolved": "https://registry.npmjs.org/use-isomorphic-layout-effect/-/use-isomorphic-layout-effect-1.1.2.tgz", - "integrity": "sha512-49L8yCO3iGT/ZF9QttjwLF/ZD9Iwto5LnH5LmEdk/6cFmXddqi2ulF0edxTwjj+7mqvpVVGQWvbXZdn32wRSHA==", + "version": "1.2.0", + "resolved": "https://registry.npmjs.org/use-isomorphic-layout-effect/-/use-isomorphic-layout-effect-1.2.0.tgz", + "integrity": "sha512-q6ayo8DWoPZT0VdG4u3D3uxcgONP3Mevx2i2b0434cwWBoL+aelL1DzkXI6w3PhTZzUeR2kaVlZn70iCiseP6w==", "peerDependencies": { - "react": "^16.8.0 || ^17.0.0 || ^18.0.0" + "react": "^16.8.0 || ^17.0.0 || ^18.0.0 || ^19.0.0" }, "peerDependenciesMeta": { "@types/react": { @@ -9073,14 +9075,14 @@ } }, "node_modules/use-latest": { - "version": "1.2.1", - "resolved": "https://registry.npmjs.org/use-latest/-/use-latest-1.2.1.tgz", - "integrity": "sha512-xA+AVm/Wlg3e2P/JiItTziwS7FK92LWrDB0p+hgXloIMuVCeJJ8v6f0eeHyPZaJrM+usM1FkFfbNCrJGs8A/zw==", + "version": "1.3.0", + "resolved": "https://registry.npmjs.org/use-latest/-/use-latest-1.3.0.tgz", + "integrity": "sha512-mhg3xdm9NaM8q+gLT8KryJPnRFOz1/5XPBhmDEVZK1webPzDjrPk7f/mbpeLqTgB9msytYWANxgALOCJKnLvcQ==", "dependencies": { "use-isomorphic-layout-effect": "^1.1.1" }, "peerDependencies": { - "react": "^16.8.0 || ^17.0.0 || ^18.0.0" + "react": "^16.8.0 || ^17.0.0 || ^18.0.0 || ^19.0.0" }, "peerDependenciesMeta": { "@types/react": { From 75d9c328d71be928f8aac0eefb4ae21b2aa4cacc Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Jan 2025 19:58:17 +0100 Subject: [PATCH 069/110] chore(deps): bump react-router-dom from 7.0.2 to 7.1.1 in /web/ui (#15743) Bumps [react-router-dom](https://github.com/remix-run/react-router/tree/HEAD/packages/react-router-dom) from 7.0.2 to 7.1.1. - [Release notes](https://github.com/remix-run/react-router/releases) - [Changelog](https://github.com/remix-run/react-router/blob/main/packages/react-router-dom/CHANGELOG.md) - [Commits](https://github.com/remix-run/react-router/commits/react-router-dom@7.1.1/packages/react-router-dom) --- updated-dependencies: - dependency-name: react-router-dom dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- web/ui/mantine-ui/package.json | 2 +- web/ui/package-lock.json | 28 +++++++++++----------------- 2 files changed, 12 insertions(+), 18 deletions(-) diff --git a/web/ui/mantine-ui/package.json b/web/ui/mantine-ui/package.json index ad169cd582..ff62f74a64 100644 --- a/web/ui/mantine-ui/package.json +++ b/web/ui/mantine-ui/package.json @@ -44,7 +44,7 @@ "react-dom": "^18.3.1", "react-infinite-scroll-component": "^6.1.0", "react-redux": "^9.2.0", - "react-router-dom": "^7.0.2", + "react-router-dom": "^7.1.1", "sanitize-html": "^2.13.1", "uplot": "^1.6.31", "uplot-react": "^1.2.2", diff --git a/web/ui/package-lock.json b/web/ui/package-lock.json index fc9a461621..e0f28f1e1d 100644 --- a/web/ui/package-lock.json +++ b/web/ui/package-lock.json @@ -58,7 +58,7 @@ "react-dom": "^18.3.1", "react-infinite-scroll-component": "^6.1.0", "react-redux": "^9.2.0", - "react-router-dom": "^7.0.2", + "react-router-dom": "^7.1.1", "sanitize-html": "^2.13.1", "uplot": "^1.6.31", "uplot-react": "^1.2.2", @@ -2839,8 +2839,7 @@ "node_modules/@types/cookie": { "version": "0.6.0", "resolved": "https://registry.npmjs.org/@types/cookie/-/cookie-0.6.0.tgz", - "integrity": "sha512-4Kh9a6B2bQciAhf7FSuMRRkUWecJgJu9nPnx3yzpsfXX/c50REIqpHY4C82bXP90qrLtXtkDxTZosYO3UpOwlA==", - "license": "MIT" + "integrity": "sha512-4Kh9a6B2bQciAhf7FSuMRRkUWecJgJu9nPnx3yzpsfXX/c50REIqpHY4C82bXP90qrLtXtkDxTZosYO3UpOwlA==" }, "node_modules/@types/estree": { "version": "1.0.6", @@ -4080,7 +4079,6 @@ "version": "1.0.2", "resolved": "https://registry.npmjs.org/cookie/-/cookie-1.0.2.tgz", "integrity": "sha512-9Kr/j4O16ISv8zBBhJoi4bXOYNTkFLOqSL3UDB0njXxCXNezjeyVrJyGOWtgfs/q2km1gwBcfH8q1yEGoMYunA==", - "license": "MIT", "engines": { "node": ">=18" } @@ -8026,10 +8024,9 @@ } }, "node_modules/react-router": { - "version": "7.0.2", - "resolved": "https://registry.npmjs.org/react-router/-/react-router-7.0.2.tgz", - "integrity": "sha512-m5AcPfTRUcjwmhBzOJGEl6Y7+Crqyju0+TgTQxoS4SO+BkWbhOrcfZNq6wSWdl2BBbJbsAoBUb8ZacOFT+/JlA==", - "license": "MIT", + "version": "7.1.1", + "resolved": "https://registry.npmjs.org/react-router/-/react-router-7.1.1.tgz", + "integrity": "sha512-39sXJkftkKWRZ2oJtHhCxmoCrBCULr/HAH4IT5DHlgu/Q0FCPV0S4Lx+abjDTx/74xoZzNYDYbOZWlJjruyuDQ==", "dependencies": { "@types/cookie": "^0.6.0", "cookie": "^1.0.1", @@ -8050,12 +8047,11 @@ } }, "node_modules/react-router-dom": { - "version": "7.0.2", - "resolved": "https://registry.npmjs.org/react-router-dom/-/react-router-dom-7.0.2.tgz", - "integrity": "sha512-VJOQ+CDWFDGaWdrG12Nl+d7yHtLaurNgAQZVgaIy7/Xd+DojgmYLosFfZdGz1wpxmjJIAkAMVTKWcvkx1oggAw==", - "license": "MIT", + "version": "7.1.1", + "resolved": "https://registry.npmjs.org/react-router-dom/-/react-router-dom-7.1.1.tgz", + "integrity": "sha512-vSrQHWlJ5DCfyrhgo0k6zViOe9ToK8uT5XGSmnuC2R3/g261IdIMpZVqfjD6vWSXdnf5Czs4VA/V60oVR6/jnA==", "dependencies": { - "react-router": "7.0.2" + "react-router": "7.1.1" }, "engines": { "node": ">=20.0.0" @@ -8395,8 +8391,7 @@ "node_modules/set-cookie-parser": { "version": "2.7.1", "resolved": "https://registry.npmjs.org/set-cookie-parser/-/set-cookie-parser-2.7.1.tgz", - "integrity": "sha512-IOc8uWeOZgnb3ptbCURJWNjWUPcO3ZnTTdzsurqERrP6nPyv+paC55vJM0LpOlT2ne+Ix+9+CRG1MNLlyZ4GjQ==", - "license": "MIT" + "integrity": "sha512-IOc8uWeOZgnb3ptbCURJWNjWUPcO3ZnTTdzsurqERrP6nPyv+paC55vJM0LpOlT2ne+Ix+9+CRG1MNLlyZ4GjQ==" }, "node_modules/shebang-command": { "version": "2.0.0", @@ -8911,8 +8906,7 @@ "node_modules/turbo-stream": { "version": "2.4.0", "resolved": "https://registry.npmjs.org/turbo-stream/-/turbo-stream-2.4.0.tgz", - "integrity": "sha512-FHncC10WpBd2eOmGwpmQsWLDoK4cqsA/UT/GqNoaKOQnT8uzhtCbg3EoUDMvqpOSAI0S26mr0rkjzbOO6S3v1g==", - "license": "ISC" + "integrity": "sha512-FHncC10WpBd2eOmGwpmQsWLDoK4cqsA/UT/GqNoaKOQnT8uzhtCbg3EoUDMvqpOSAI0S26mr0rkjzbOO6S3v1g==" }, "node_modules/type-check": { "version": "0.4.0", From c5ef7153e0ce22de376c6db2c27a334227777716 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Jan 2025 19:58:55 +0100 Subject: [PATCH 070/110] chore(deps): bump @tanstack/react-query in /web/ui (#15737) Bumps [@tanstack/react-query](https://github.com/TanStack/query/tree/HEAD/packages/react-query) from 5.62.7 to 5.62.11. - [Release notes](https://github.com/TanStack/query/releases) - [Commits](https://github.com/TanStack/query/commits/v5.62.11/packages/react-query) --- updated-dependencies: - dependency-name: "@tanstack/react-query" dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- web/ui/mantine-ui/package.json | 2 +- web/ui/package-lock.json | 18 ++++++++---------- 2 files changed, 9 insertions(+), 11 deletions(-) diff --git a/web/ui/mantine-ui/package.json b/web/ui/mantine-ui/package.json index ff62f74a64..f82c03262d 100644 --- a/web/ui/mantine-ui/package.json +++ b/web/ui/mantine-ui/package.json @@ -31,7 +31,7 @@ "@prometheus-io/codemirror-promql": "0.301.0", "@reduxjs/toolkit": "^2.5.0", "@tabler/icons-react": "^3.24.0", - "@tanstack/react-query": "^5.62.7", + "@tanstack/react-query": "^5.62.11", "@testing-library/jest-dom": "^6.6.3", "@testing-library/react": "^16.1.0", "@types/lodash": "^4.17.13", diff --git a/web/ui/package-lock.json b/web/ui/package-lock.json index e0f28f1e1d..e9955be556 100644 --- a/web/ui/package-lock.json +++ b/web/ui/package-lock.json @@ -45,7 +45,7 @@ "@prometheus-io/codemirror-promql": "0.301.0", "@reduxjs/toolkit": "^2.5.0", "@tabler/icons-react": "^3.24.0", - "@tanstack/react-query": "^5.62.7", + "@tanstack/react-query": "^5.62.11", "@testing-library/jest-dom": "^6.6.3", "@testing-library/react": "^16.1.0", "@types/lodash": "^4.17.13", @@ -2674,22 +2674,20 @@ } }, "node_modules/@tanstack/query-core": { - "version": "5.62.7", - "resolved": "https://registry.npmjs.org/@tanstack/query-core/-/query-core-5.62.7.tgz", - "integrity": "sha512-fgpfmwatsrUal6V+8EC2cxZIQVl9xvL7qYa03gsdsCy985UTUlS4N+/3hCzwR0PclYDqisca2AqR1BVgJGpUDA==", - "license": "MIT", + "version": "5.62.9", + "resolved": "https://registry.npmjs.org/@tanstack/query-core/-/query-core-5.62.9.tgz", + "integrity": "sha512-lwePd8hNYhyQ4nM/iRQ+Wz2cDtspGeZZHFZmCzHJ7mfKXt+9S301fULiY2IR2byJYY6Z03T427E5PoVfMexHjw==", "funding": { "type": "github", "url": "https://github.com/sponsors/tannerlinsley" } }, "node_modules/@tanstack/react-query": { - "version": "5.62.7", - "resolved": "https://registry.npmjs.org/@tanstack/react-query/-/react-query-5.62.7.tgz", - "integrity": "sha512-+xCtP4UAFDTlRTYyEjLx0sRtWyr5GIk7TZjZwBu4YaNahi3Rt2oMyRqfpfVrtwsqY2sayP4iXVCwmC+ZqqFmuw==", - "license": "MIT", + "version": "5.62.11", + "resolved": "https://registry.npmjs.org/@tanstack/react-query/-/react-query-5.62.11.tgz", + "integrity": "sha512-Xb1nw0cYMdtFmwkvH9+y5yYFhXvLRCnXoqlzSw7UkqtCVFq3cG8q+rHZ2Yz1XrC+/ysUaTqbLKJqk95mCgC1oQ==", "dependencies": { - "@tanstack/query-core": "5.62.7" + "@tanstack/query-core": "5.62.9" }, "funding": { "type": "github", From 16f3639521dde70e5189eb6394ba29daa0aec97d Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Jan 2025 19:59:32 +0100 Subject: [PATCH 071/110] chore(deps-dev): bump @rollup/plugin-node-resolve in /web/ui (#15740) Bumps [@rollup/plugin-node-resolve](https://github.com/rollup/plugins/tree/HEAD/packages/node-resolve) from 15.3.0 to 16.0.0. - [Changelog](https://github.com/rollup/plugins/blob/master/packages/node-resolve/CHANGELOG.md) - [Commits](https://github.com/rollup/plugins/commits/commonjs-v16.0.0/packages/node-resolve) --- updated-dependencies: - dependency-name: "@rollup/plugin-node-resolve" dependency-type: direct:development update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- web/ui/module/lezer-promql/package.json | 2 +- web/ui/package-lock.json | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/web/ui/module/lezer-promql/package.json b/web/ui/module/lezer-promql/package.json index 282ab5ab62..270e57797f 100644 --- a/web/ui/module/lezer-promql/package.json +++ b/web/ui/module/lezer-promql/package.json @@ -34,7 +34,7 @@ "@lezer/generator": "^1.7.2", "@lezer/highlight": "^1.2.0", "@lezer/lr": "^1.4.2", - "@rollup/plugin-node-resolve": "^15.3.0" + "@rollup/plugin-node-resolve": "^16.0.0" }, "peerDependencies": { "@lezer/highlight": "^1.1.2", diff --git a/web/ui/package-lock.json b/web/ui/package-lock.json index e9955be556..e7d0b000fa 100644 --- a/web/ui/package-lock.json +++ b/web/ui/package-lock.json @@ -195,7 +195,7 @@ "@lezer/generator": "^1.7.2", "@lezer/highlight": "^1.2.0", "@lezer/lr": "^1.4.2", - "@rollup/plugin-node-resolve": "^15.3.0" + "@rollup/plugin-node-resolve": "^16.0.0" }, "peerDependencies": { "@lezer/highlight": "^1.1.2", @@ -2331,9 +2331,9 @@ } }, "node_modules/@rollup/plugin-node-resolve": { - "version": "15.3.0", - "resolved": "https://registry.npmjs.org/@rollup/plugin-node-resolve/-/plugin-node-resolve-15.3.0.tgz", - "integrity": "sha512-9eO5McEICxMzJpDW9OnMYSv4Sta3hmt7VtBFz5zR9273suNOydOyq/FrGeGy+KsTRFm8w0SLVhzig2ILFT63Ag==", + "version": "16.0.0", + "resolved": "https://registry.npmjs.org/@rollup/plugin-node-resolve/-/plugin-node-resolve-16.0.0.tgz", + "integrity": "sha512-0FPvAeVUT/zdWoO0jnb/V5BlBsUSNfkIOtFHzMO4H9MOklrmQFY6FduVHKucNb/aTFxvnGhj4MNj/T1oNdDfNg==", "dev": true, "dependencies": { "@rollup/pluginutils": "^5.0.1", From 287e56058b3bdf3deaa48367ef5117805565e0a1 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Jan 2025 20:23:36 +0100 Subject: [PATCH 072/110] chore(deps): bump google.golang.org/grpc from 1.69.0 to 1.69.4 (#15811) Bumps [google.golang.org/grpc](https://github.com/grpc/grpc-go) from 1.69.0 to 1.69.4. - [Release notes](https://github.com/grpc/grpc-go/releases) - [Commits](https://github.com/grpc/grpc-go/compare/v1.69.0...v1.69.4) --- updated-dependencies: - dependency-name: google.golang.org/grpc dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index ee88a0e5f9..93cd89f469 100644 --- a/go.mod +++ b/go.mod @@ -81,7 +81,7 @@ require ( golang.org/x/tools v0.28.0 google.golang.org/api v0.216.0 google.golang.org/genproto/googleapis/api v0.0.0-20241216192217-9240e9c98484 - google.golang.org/grpc v1.69.2 + google.golang.org/grpc v1.69.4 google.golang.org/protobuf v1.36.1 gopkg.in/yaml.v2 v2.4.0 gopkg.in/yaml.v3 v3.0.1 diff --git a/go.sum b/go.sum index 7ab4986307..515fb9c053 100644 --- a/go.sum +++ b/go.sum @@ -648,8 +648,8 @@ google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyac google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.33.1/go.mod h1:fr5YgcSWrqhRRxogOsw7RzIpsmvOZ6IcH4kBYTpR3n0= -google.golang.org/grpc v1.69.2 h1:U3S9QEtbXC0bYNvRtcoklF3xGtLViumSYxWykJS+7AU= -google.golang.org/grpc v1.69.2/go.mod h1:vyjdE6jLBI76dgpDojsFGNaHlxdjXN9ghpnd2o7JGZ4= +google.golang.org/grpc v1.69.4 h1:MF5TftSMkd8GLw/m0KM6V8CMOCY6NZ1NQDPGFgbTt4A= +google.golang.org/grpc v1.69.4/go.mod h1:vyjdE6jLBI76dgpDojsFGNaHlxdjXN9ghpnd2o7JGZ4= google.golang.org/protobuf v1.36.1 h1:yBPeRvTftaleIgM3PZ/WBIZ7XM/eEYAaEyCwvyjq/gk= google.golang.org/protobuf v1.36.1/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= From fccd78d3cd40e5d05f69595109514276742cdcb8 Mon Sep 17 00:00:00 2001 From: leonnicolas Date: Mon, 13 Jan 2025 21:46:45 +0100 Subject: [PATCH 073/110] Update storage keys Signed-off-by: leonnicolas --- web/ui/mantine-ui/src/pages/AlertsPage.tsx | 2 +- web/ui/mantine-ui/src/pages/targets/ScrapePoolsList.tsx | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/web/ui/mantine-ui/src/pages/AlertsPage.tsx b/web/ui/mantine-ui/src/pages/AlertsPage.tsx index e107adfb23..4ce42d9059 100644 --- a/web/ui/mantine-ui/src/pages/AlertsPage.tsx +++ b/web/ui/mantine-ui/src/pages/AlertsPage.tsx @@ -163,7 +163,7 @@ export default function AlertsPage() { ); const [debouncedSearch] = useDebouncedValue(searchFilter.trim(), 250); const [showEmptyGroups, setShowEmptyGroups] = useLocalStorage({ - key: "alerts-page-show-empty-groups", + key: "alertsPage.showEmptyGroups", defaultValue: true, }); diff --git a/web/ui/mantine-ui/src/pages/targets/ScrapePoolsList.tsx b/web/ui/mantine-ui/src/pages/targets/ScrapePoolsList.tsx index f674a1f4d3..1e29c568e1 100644 --- a/web/ui/mantine-ui/src/pages/targets/ScrapePoolsList.tsx +++ b/web/ui/mantine-ui/src/pages/targets/ScrapePoolsList.tsx @@ -166,7 +166,7 @@ const ScrapePoolList: FC = ({ const dispatch = useAppDispatch(); const [showEmptyPools, setShowEmptyPools] = useLocalStorage({ - key: "targets-page-show-empty-pools", + key: "targetsPage.showEmptyPools", defaultValue: true, }); From e5415f8ecd62dc843e48147494392371185d004c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Jan 2025 21:54:09 +0100 Subject: [PATCH 074/110] chore(deps): bump sanitize-html from 2.13.1 to 2.14.0 in /web/ui (#15739) Bumps [sanitize-html](https://github.com/apostrophecms/sanitize-html) from 2.13.1 to 2.14.0. - [Changelog](https://github.com/apostrophecms/sanitize-html/blob/main/CHANGELOG.md) - [Commits](https://github.com/apostrophecms/sanitize-html/compare/2.13.1...2.14.0) --- updated-dependencies: - dependency-name: sanitize-html dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- web/ui/mantine-ui/package.json | 2 +- web/ui/package-lock.json | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/web/ui/mantine-ui/package.json b/web/ui/mantine-ui/package.json index f82c03262d..0ee8802e1b 100644 --- a/web/ui/mantine-ui/package.json +++ b/web/ui/mantine-ui/package.json @@ -45,7 +45,7 @@ "react-infinite-scroll-component": "^6.1.0", "react-redux": "^9.2.0", "react-router-dom": "^7.1.1", - "sanitize-html": "^2.13.1", + "sanitize-html": "^2.14.0", "uplot": "^1.6.31", "uplot-react": "^1.2.2", "use-query-params": "^2.2.1" diff --git a/web/ui/package-lock.json b/web/ui/package-lock.json index e7d0b000fa..f8f1ba90e6 100644 --- a/web/ui/package-lock.json +++ b/web/ui/package-lock.json @@ -59,7 +59,7 @@ "react-infinite-scroll-component": "^6.1.0", "react-redux": "^9.2.0", "react-router-dom": "^7.1.1", - "sanitize-html": "^2.13.1", + "sanitize-html": "^2.14.0", "uplot": "^1.6.31", "uplot-react": "^1.2.2", "use-query-params": "^2.2.1" @@ -8333,9 +8333,9 @@ "license": "MIT" }, "node_modules/sanitize-html": { - "version": "2.13.1", - "resolved": "https://registry.npmjs.org/sanitize-html/-/sanitize-html-2.13.1.tgz", - "integrity": "sha512-ZXtKq89oue4RP7abL9wp/9URJcqQNABB5GGJ2acW1sdO8JTVl92f4ygD7Yc9Ze09VAZhnt2zegeU0tbNsdcLYg==", + "version": "2.14.0", + "resolved": "https://registry.npmjs.org/sanitize-html/-/sanitize-html-2.14.0.tgz", + "integrity": "sha512-CafX+IUPxZshXqqRaG9ZClSlfPVjSxI0td7n07hk8QO2oO+9JDnlcL8iM8TWeOXOIBFgIOx6zioTzM53AOMn3g==", "dependencies": { "deepmerge": "^4.2.2", "escape-string-regexp": "^4.0.0", From 17ca5c8fa5fbe01cacb6191d418063cad74f1ea8 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Jan 2025 21:54:39 +0100 Subject: [PATCH 075/110] chore(deps): bump @codemirror/autocomplete in /web/ui (#15750) Bumps [@codemirror/autocomplete](https://github.com/codemirror/autocomplete) from 6.18.3 to 6.18.4. - [Changelog](https://github.com/codemirror/autocomplete/blob/main/CHANGELOG.md) - [Commits](https://github.com/codemirror/autocomplete/compare/6.18.3...6.18.4) --- updated-dependencies: - dependency-name: "@codemirror/autocomplete" dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- web/ui/mantine-ui/package.json | 2 +- web/ui/module/codemirror-promql/package.json | 2 +- web/ui/package-lock.json | 17 +++++------------ 3 files changed, 7 insertions(+), 14 deletions(-) diff --git a/web/ui/mantine-ui/package.json b/web/ui/mantine-ui/package.json index 0ee8802e1b..bd741503d9 100644 --- a/web/ui/mantine-ui/package.json +++ b/web/ui/mantine-ui/package.json @@ -12,7 +12,7 @@ "test": "vitest" }, "dependencies": { - "@codemirror/autocomplete": "^6.18.3", + "@codemirror/autocomplete": "^6.18.4", "@codemirror/language": "^6.10.6", "@codemirror/lint": "^6.8.4", "@codemirror/state": "^6.4.1", diff --git a/web/ui/module/codemirror-promql/package.json b/web/ui/module/codemirror-promql/package.json index d4e95c7ee8..aea4d22c62 100644 --- a/web/ui/module/codemirror-promql/package.json +++ b/web/ui/module/codemirror-promql/package.json @@ -33,7 +33,7 @@ "lru-cache": "^11.0.2" }, "devDependencies": { - "@codemirror/autocomplete": "^6.18.3", + "@codemirror/autocomplete": "^6.18.4", "@codemirror/language": "^6.10.6", "@codemirror/lint": "^6.8.4", "@codemirror/state": "^6.3.3", diff --git a/web/ui/package-lock.json b/web/ui/package-lock.json index f8f1ba90e6..74d898066e 100644 --- a/web/ui/package-lock.json +++ b/web/ui/package-lock.json @@ -26,7 +26,7 @@ "name": "@prometheus-io/mantine-ui", "version": "0.301.0", "dependencies": { - "@codemirror/autocomplete": "^6.18.3", + "@codemirror/autocomplete": "^6.18.4", "@codemirror/language": "^6.10.6", "@codemirror/lint": "^6.8.4", "@codemirror/state": "^6.4.1", @@ -163,7 +163,7 @@ "lru-cache": "^11.0.2" }, "devDependencies": { - "@codemirror/autocomplete": "^6.18.3", + "@codemirror/autocomplete": "^6.18.4", "@codemirror/language": "^6.10.6", "@codemirror/lint": "^6.8.4", "@codemirror/state": "^6.3.3", @@ -788,21 +788,14 @@ "peer": true }, "node_modules/@codemirror/autocomplete": { - "version": "6.18.3", - "resolved": "https://registry.npmjs.org/@codemirror/autocomplete/-/autocomplete-6.18.3.tgz", - "integrity": "sha512-1dNIOmiM0z4BIBwxmxEfA1yoxh1MF/6KPBbh20a5vphGV0ictKlgQsbJs6D6SkR6iJpGbpwRsa6PFMNlg9T9pQ==", - "license": "MIT", + "version": "6.18.4", + "resolved": "https://registry.npmjs.org/@codemirror/autocomplete/-/autocomplete-6.18.4.tgz", + "integrity": "sha512-sFAphGQIqyQZfP2ZBsSHV7xQvo9Py0rV0dW7W3IMRdS+zDuNb2l3no78CvUaWKGfzFjI4FTrLdUSj86IGb2hRA==", "dependencies": { "@codemirror/language": "^6.0.0", "@codemirror/state": "^6.0.0", "@codemirror/view": "^6.17.0", "@lezer/common": "^1.0.0" - }, - "peerDependencies": { - "@codemirror/language": "^6.0.0", - "@codemirror/state": "^6.0.0", - "@codemirror/view": "^6.0.0", - "@lezer/common": "^1.0.0" } }, "node_modules/@codemirror/commands": { From bde2d11ea42f820683b2c0498402e5bcae2544f9 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Jan 2025 21:55:14 +0100 Subject: [PATCH 076/110] chore(deps-dev): bump eslint from 9.16.0 to 9.18.0 in /web/ui (#15814) Bumps [eslint](https://github.com/eslint/eslint) from 9.16.0 to 9.18.0. - [Release notes](https://github.com/eslint/eslint/releases) - [Changelog](https://github.com/eslint/eslint/blob/main/CHANGELOG.md) - [Commits](https://github.com/eslint/eslint/compare/v9.16.0...v9.18.0) --- updated-dependencies: - dependency-name: eslint dependency-type: direct:development update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- web/ui/mantine-ui/package.json | 2 +- web/ui/package-lock.json | 81 +++++++++------------------------- 2 files changed, 21 insertions(+), 62 deletions(-) diff --git a/web/ui/mantine-ui/package.json b/web/ui/mantine-ui/package.json index bd741503d9..7ffe65c4c1 100644 --- a/web/ui/mantine-ui/package.json +++ b/web/ui/mantine-ui/package.json @@ -59,7 +59,7 @@ "@typescript-eslint/eslint-plugin": "^6.21.0", "@typescript-eslint/parser": "^6.21.0", "@vitejs/plugin-react": "^4.3.4", - "eslint": "^9.16.0", + "eslint": "^9.18.0", "eslint-plugin-react-hooks": "^5.1.0", "eslint-plugin-react-refresh": "^0.4.16", "globals": "^15.14.0", diff --git a/web/ui/package-lock.json b/web/ui/package-lock.json index 74d898066e..febeaa4dcd 100644 --- a/web/ui/package-lock.json +++ b/web/ui/package-lock.json @@ -73,7 +73,7 @@ "@typescript-eslint/eslint-plugin": "^6.21.0", "@typescript-eslint/parser": "^6.21.0", "@vitejs/plugin-react": "^4.3.4", - "eslint": "^9.16.0", + "eslint": "^9.18.0", "eslint-plugin-react-hooks": "^5.1.0", "eslint-plugin-react-refresh": "^0.4.16", "globals": "^15.14.0", @@ -86,19 +86,18 @@ } }, "mantine-ui/node_modules/eslint": { - "version": "9.16.0", - "resolved": "https://registry.npmjs.org/eslint/-/eslint-9.16.0.tgz", - "integrity": "sha512-whp8mSQI4C8VXd+fLgSM0lh3UlmcFtVwUQjyKCFfsp+2ItAIYhlq/hqGahGqHE6cv9unM41VlqKk2VtKYR2TaA==", + "version": "9.18.0", + "resolved": "https://registry.npmjs.org/eslint/-/eslint-9.18.0.tgz", + "integrity": "sha512-+waTfRWQlSbpt3KWE+CjrPPYnbq9kfZIYUqapc0uBXyjTp8aYXZDsUH16m39Ryq3NjAVP4tjuF7KaukeqoCoaA==", "dev": true, - "license": "MIT", "dependencies": { "@eslint-community/eslint-utils": "^4.2.0", "@eslint-community/regexpp": "^4.12.1", "@eslint/config-array": "^0.19.0", - "@eslint/core": "^0.9.0", + "@eslint/core": "^0.10.0", "@eslint/eslintrc": "^3.2.0", - "@eslint/js": "9.16.0", - "@eslint/plugin-kit": "^0.2.3", + "@eslint/js": "9.18.0", + "@eslint/plugin-kit": "^0.2.5", "@humanfs/node": "^0.16.6", "@humanwhocodes/module-importer": "^1.0.1", "@humanwhocodes/retry": "^0.4.1", @@ -106,7 +105,7 @@ "@types/json-schema": "^7.0.15", "ajv": "^6.12.4", "chalk": "^4.0.0", - "cross-spawn": "^7.0.5", + "cross-spawn": "^7.0.6", "debug": "^4.3.2", "escape-string-regexp": "^4.0.0", "eslint-scope": "^8.2.0", @@ -145,15 +144,6 @@ } } }, - "mantine-ui/node_modules/eslint/node_modules/@eslint/js": { - "version": "9.16.0", - "resolved": "https://registry.npmjs.org/@eslint/js/-/js-9.16.0.tgz", - "integrity": "sha512-tw2HxzQkrbeuvyj1tG2Yqq+0H9wGoI2IMk4EOsQeX+vmd75FtJAzf+gTA69WF+baUKRYQ3x2kbLE08js5OsTVg==", - "dev": true, - "engines": { - "node": "^18.18.0 || ^20.9.0 || >=21.1.0" - } - }, "module/codemirror-promql": { "name": "@prometheus-io/codemirror-promql", "version": "0.301.0", @@ -1338,7 +1328,6 @@ "resolved": "https://registry.npmjs.org/@eslint/config-array/-/config-array-0.19.1.tgz", "integrity": "sha512-fo6Mtm5mWyKjA/Chy1BYTdn5mGJoDNjC7C64ug20ADsRDGrA85bN3uK3MaKbeRkRuuIEAR5N33Jr1pbm411/PA==", "dev": true, - "license": "Apache-2.0", "dependencies": { "@eslint/object-schema": "^2.1.5", "debug": "^4.3.1", @@ -1349,11 +1338,10 @@ } }, "node_modules/@eslint/core": { - "version": "0.9.1", - "resolved": "https://registry.npmjs.org/@eslint/core/-/core-0.9.1.tgz", - "integrity": "sha512-GuUdqkyyzQI5RMIWkHhvTWLCyLo1jNK3vzkSyaExH5kHPDHcuL2VOpHjmMY+y3+NC69qAKToBqldTBgYeLSr9Q==", + "version": "0.10.0", + "resolved": "https://registry.npmjs.org/@eslint/core/-/core-0.10.0.tgz", + "integrity": "sha512-gFHJ+xBOo4G3WRlR1e/3G8A6/KZAH6zcE/hkLRCZTi/B9avAG365QhFA8uOGzTMqgTghpn7/fSnscW++dpMSAw==", "dev": true, - "license": "Apache-2.0", "dependencies": { "@types/json-schema": "^7.0.15" }, @@ -1398,9 +1386,9 @@ } }, "node_modules/@eslint/js": { - "version": "9.17.0", - "resolved": "https://registry.npmjs.org/@eslint/js/-/js-9.17.0.tgz", - "integrity": "sha512-Sxc4hqcs1kTu0iID3kcZDW3JHq2a77HO9P8CP6YEA/FpH3Ll8UXE2r/86Rz9YJLKme39S9vU5OWNjC6Xl0Cr3w==", + "version": "9.18.0", + "resolved": "https://registry.npmjs.org/@eslint/js/-/js-9.18.0.tgz", + "integrity": "sha512-fK6L7rxcq6/z+AaQMtiFTkvbHkBLNlwyRxHpKawP0x3u9+NC6MQTnFW+AdpwC6gfHTW0051cokQgtTN2FqlxQA==", "dev": true, "engines": { "node": "^18.18.0 || ^20.9.0 || >=21.1.0" @@ -1411,18 +1399,17 @@ "resolved": "https://registry.npmjs.org/@eslint/object-schema/-/object-schema-2.1.5.tgz", "integrity": "sha512-o0bhxnL89h5Bae5T318nFoFzGy+YE5i/gGkoPAgkmTVdRKTiv3p8JHevPiPaMwoloKfEiiaHlawCqaZMqRm+XQ==", "dev": true, - "license": "Apache-2.0", "engines": { "node": "^18.18.0 || ^20.9.0 || >=21.1.0" } }, "node_modules/@eslint/plugin-kit": { - "version": "0.2.4", - "resolved": "https://registry.npmjs.org/@eslint/plugin-kit/-/plugin-kit-0.2.4.tgz", - "integrity": "sha512-zSkKow6H5Kdm0ZUQUB2kV5JIXqoG0+uH5YADhaEHswm664N9Db8dXSi0nMJpacpMf+MyyglF1vnZohpEg5yUtg==", + "version": "0.2.5", + "resolved": "https://registry.npmjs.org/@eslint/plugin-kit/-/plugin-kit-0.2.5.tgz", + "integrity": "sha512-lB05FkqEdUg2AA0xEbUz0SnkXT1LcCTa438W4IWTUh4hdOnVbQyOJ81OrDXsJk/LSiJHubgGEFoR5EHq1NsH1A==", "dev": true, - "license": "Apache-2.0", "dependencies": { + "@eslint/core": "^0.10.0", "levn": "^0.4.1" }, "engines": { @@ -1484,7 +1471,6 @@ "resolved": "https://registry.npmjs.org/@humanfs/core/-/core-0.19.1.tgz", "integrity": "sha512-5DyQ4+1JEUzejeK1JGICcideyfUbGixgS9jNgex5nqkW+cY7WZhxBigmieN5Qnw9ZosSNVC9KQKyb+GUaGyKUA==", "dev": true, - "license": "Apache-2.0", "engines": { "node": ">=18.18.0" } @@ -1494,7 +1480,6 @@ "resolved": "https://registry.npmjs.org/@humanfs/node/-/node-0.16.6.tgz", "integrity": "sha512-YuI2ZHQL78Q5HbhDiBA1X4LmYdXCKCMQIfw0pw7piHJwyREFebJUvrQN4cMssyES6x+vfUbx1CIpaQUKYdQZOw==", "dev": true, - "license": "Apache-2.0", "dependencies": { "@humanfs/core": "^0.19.1", "@humanwhocodes/retry": "^0.3.0" @@ -1508,7 +1493,6 @@ "resolved": "https://registry.npmjs.org/@humanwhocodes/retry/-/retry-0.3.1.tgz", "integrity": "sha512-JBxkERygn7Bv/GbN5Rv8Ul6LVknS+5Bp6RgDC/O8gEBU/yeH5Ui5C/OlWrTb6qct7LjjfT6Re2NxB0ln0yYybA==", "dev": true, - "license": "Apache-2.0", "engines": { "node": ">=18.18" }, @@ -1523,7 +1507,6 @@ "integrity": "sha512-DZLEEqFWQFiyK6h5YIeynKx7JlvCYWL0cImfSRXZ9l4Sg2efkFGTuFf6vzXjK1cq6IYkU+Eg/JizXw+TD2vRNw==", "deprecated": "Use @eslint/config-array instead", "dev": true, - "license": "Apache-2.0", "peer": true, "dependencies": { "@humanwhocodes/object-schema": "^2.0.3", @@ -1554,7 +1537,6 @@ "integrity": "sha512-93zYdMES/c1D69yZiKDBj0V24vqNzB/koF26KPaagAfd3P/4gUlh3Dys5ogAK+Exi9QyzlD8x/08Zt7wIKcDcA==", "deprecated": "Use @eslint/object-schema instead", "dev": true, - "license": "BSD-3-Clause", "peer": true }, "node_modules/@humanwhocodes/retry": { @@ -1562,7 +1544,6 @@ "resolved": "https://registry.npmjs.org/@humanwhocodes/retry/-/retry-0.4.1.tgz", "integrity": "sha512-c7hNEllBlenFTHBky65mhq8WD2kbN9Q6gk0bTk8lSBvc554jpXSkST1iePudpt7+A/AQvuHs9EMqjHDXMY1lrA==", "dev": true, - "license": "Apache-2.0", "engines": { "node": ">=18.18" }, @@ -3312,7 +3293,6 @@ "resolved": "https://registry.npmjs.org/@ungap/structured-clone/-/structured-clone-1.2.1.tgz", "integrity": "sha512-fEzPV3hSkSMltkw152tJKNARhOupqbH96MZWyRjNaYZOMIzbrTeQDG+MTc6Mr2pgzFQzFxAfmhGDNP5QK++2ZA==", "dev": true, - "license": "ISC", "peer": true }, "node_modules/@vitejs/plugin-react": { @@ -4305,7 +4285,6 @@ "resolved": "https://registry.npmjs.org/doctrine/-/doctrine-3.0.0.tgz", "integrity": "sha512-yS+Q5i3hBf7GBkd4KG8a7eBNNWNGLTaEwwYWUijIYM7zrlYDM0BFXHjjPWlWZ1Rg7UaddZeIDmi9jF3HmqiQ2w==", "dev": true, - "license": "Apache-2.0", "peer": true, "dependencies": { "esutils": "^2.0.2" @@ -4528,7 +4507,6 @@ "integrity": "sha512-ypowyDxpVSYpkXr9WPv2PAZCtNip1Mv5KTW0SCurXv/9iOpcrH9PaqUElksqEB6pChqHGDRCFTyrZlGhnLNGiA==", "deprecated": "This version is no longer supported. Please see https://eslint.org/version-support for other options.", "dev": true, - "license": "MIT", "peer": true, "dependencies": { "@eslint-community/eslint-utils": "^4.2.0", @@ -4651,7 +4629,6 @@ "resolved": "https://registry.npmjs.org/eslint-scope/-/eslint-scope-8.2.0.tgz", "integrity": "sha512-PHlWUfG6lvPc3yvP5A4PNyBL1W8fkDUccmI21JUu/+GKZBoH/W5u6usENXUrWFRsyoW5ACUjFGgAFQp5gUlb/A==", "dev": true, - "license": "BSD-2-Clause", "dependencies": { "esrecurse": "^4.3.0", "estraverse": "^5.2.0" @@ -4681,7 +4658,6 @@ "resolved": "https://registry.npmjs.org/@eslint/eslintrc/-/eslintrc-2.1.4.tgz", "integrity": "sha512-269Z39MS6wVJtsoUl10L60WdkhJVdPG24Q4eZTH3nnF6lpvSShEK3wQjDX9JRWAUPvPh7COouPpU9IrqaZFvtQ==", "dev": true, - "license": "MIT", "peer": true, "dependencies": { "ajv": "^6.12.4", @@ -4706,7 +4682,6 @@ "resolved": "https://registry.npmjs.org/@eslint/js/-/js-8.57.1.tgz", "integrity": "sha512-d9zaMRSTIKDLhctzH12MtXvJKSSUhaHcjV+2Z+GK+EEY7XKpP5yR4x+N3TAcHTcu963nIr+TMcCb4DBCYX1z6Q==", "dev": true, - "license": "MIT", "peer": true, "engines": { "node": "^12.22.0 || ^14.17.0 || >=16.0.0" @@ -4717,7 +4692,6 @@ "resolved": "https://registry.npmjs.org/eslint-scope/-/eslint-scope-7.2.2.tgz", "integrity": "sha512-dOt21O7lTMhDM+X9mB4GX+DZrZtCUJPL/wlcTqxyrx5IvO0IYtILdtrQGQp+8n5S0gwSVmOf9NQrjMOgfQZlIg==", "dev": true, - "license": "BSD-2-Clause", "peer": true, "dependencies": { "esrecurse": "^4.3.0", @@ -4735,7 +4709,6 @@ "resolved": "https://registry.npmjs.org/eslint-visitor-keys/-/eslint-visitor-keys-3.4.3.tgz", "integrity": "sha512-wpc+LXeiyiisxPlEkUzU6svyS1frIO3Mgxj1fdy7Pm8Ygzguax2N3Fa/D/ag1WqbOprdI+uY6wMUl8/a2G+iag==", "dev": true, - "license": "Apache-2.0", "peer": true, "engines": { "node": "^12.22.0 || ^14.17.0 || >=16.0.0" @@ -4749,7 +4722,6 @@ "resolved": "https://registry.npmjs.org/espree/-/espree-9.6.1.tgz", "integrity": "sha512-oruZaFkjorTpF32kDSI5/75ViwGeZginGGy2NoOSg3Q9bnwlnmDm4HLnkl0RE3n+njDXR037aY1+x58Z/zFdwQ==", "dev": true, - "license": "BSD-2-Clause", "peer": true, "dependencies": { "acorn": "^8.9.0", @@ -4768,7 +4740,6 @@ "resolved": "https://registry.npmjs.org/file-entry-cache/-/file-entry-cache-6.0.1.tgz", "integrity": "sha512-7Gps/XWymbLk2QLYK4NzpMOrYjMhdIxXuIvy2QBsLE6ljuodKvdkWs/cpyJJ3CVIVpH0Oi1Hvg1ovbMzLdFBBg==", "dev": true, - "license": "MIT", "peer": true, "dependencies": { "flat-cache": "^3.0.4" @@ -4782,7 +4753,6 @@ "resolved": "https://registry.npmjs.org/flat-cache/-/flat-cache-3.2.0.tgz", "integrity": "sha512-CYcENa+FtcUKLmhhqyctpclsq7QF38pKjZHsGNiSQF5r4FtoKDWabFDl3hzaEQMvT1LHEysw5twgLvpYYb4vbw==", "dev": true, - "license": "MIT", "peer": true, "dependencies": { "flatted": "^3.2.9", @@ -4798,7 +4768,6 @@ "resolved": "https://registry.npmjs.org/globals/-/globals-13.24.0.tgz", "integrity": "sha512-AhO5QUcj8llrbG09iWhPU2B204J1xnPeL8kQmVorSsy+Sjj1sk8gIyh6cUocGmH4L0UuhAJy+hJMRA4mgA4mFQ==", "dev": true, - "license": "MIT", "peer": true, "dependencies": { "type-fest": "^0.20.2" @@ -4815,7 +4784,6 @@ "resolved": "https://registry.npmjs.org/type-fest/-/type-fest-0.20.2.tgz", "integrity": "sha512-Ne+eE4r0/iWnpAxD852z3A+N0Bt5RN//NjJwRd2VFHEmrywxf5vsZlh4R6lixl6B+wz/8d+maTSAkN1FIkI3LQ==", "dev": true, - "license": "(MIT OR CC0-1.0)", "peer": true, "engines": { "node": ">=10" @@ -4875,7 +4843,6 @@ "resolved": "https://registry.npmjs.org/esrecurse/-/esrecurse-4.3.0.tgz", "integrity": "sha512-KmfKL3b6G+RXvP8N1vr3Tq1kL/oCFgn2NYXEtqP8/L3pKapUA4G8cFVaoF3SU323CD4XypR/ffioHmkti6/Tag==", "dev": true, - "license": "BSD-2-Clause", "dependencies": { "estraverse": "^5.2.0" }, @@ -5059,7 +5026,6 @@ "resolved": "https://registry.npmjs.org/file-entry-cache/-/file-entry-cache-8.0.0.tgz", "integrity": "sha512-XXTUwCvisa5oacNGRP9SfNtYBNAMi+RPwBFmblZEF7N7swHYQS6/Zfk7SRwx4D5j3CH211YNRco1DEMNVfZCnQ==", "dev": true, - "license": "MIT", "dependencies": { "flat-cache": "^4.0.0" }, @@ -5135,7 +5101,6 @@ "resolved": "https://registry.npmjs.org/flat-cache/-/flat-cache-4.0.1.tgz", "integrity": "sha512-f7ccFPK3SXFHpx15UIGyRJ/FJQctuKZ0zVuN3frBo4HnK3cay9VEW0R6yPYFHC0AgqhukPzKjq22t5DmAyqGyw==", "dev": true, - "license": "MIT", "dependencies": { "flatted": "^3.2.9", "keyv": "^4.5.4" @@ -5148,8 +5113,7 @@ "version": "3.3.2", "resolved": "https://registry.npmjs.org/flatted/-/flatted-3.3.2.tgz", "integrity": "sha512-AiwGJM8YcNOaobumgtng+6NHuOqC3A7MixFeDafM3X9cIUM+xUXoS5Vfgf+OihAYe20fxqNM9yPBXJzRtZ/4eA==", - "dev": true, - "license": "ISC" + "dev": true }, "node_modules/form-data": { "version": "4.0.0", @@ -5656,7 +5620,6 @@ "resolved": "https://registry.npmjs.org/is-path-inside/-/is-path-inside-3.0.3.tgz", "integrity": "sha512-Fd4gABb+ycGAmKou8eMftCupSir5lRxqf4aD/vd0cD2qc4HL07OjCeuHMr8Ro4CoMaeCKDB0/ECBOVWjTwUvPQ==", "dev": true, - "license": "MIT", "peer": true, "engines": { "node": ">=8" @@ -6819,8 +6782,7 @@ "version": "3.0.1", "resolved": "https://registry.npmjs.org/json-buffer/-/json-buffer-3.0.1.tgz", "integrity": "sha512-4bV5BfR2mqfQTJm+V5tPPdf+ZpuhiIvTuAB5g8kcrXOZpTT/QwwVRWBywX1ozr6lEuPdbHxwaJlm9G6mI2sfSQ==", - "dev": true, - "license": "MIT" + "dev": true }, "node_modules/json-parse-even-better-errors": { "version": "2.3.1", @@ -6869,7 +6831,6 @@ "resolved": "https://registry.npmjs.org/keyv/-/keyv-4.5.4.tgz", "integrity": "sha512-oxVHkHR/EJf2CNXnWxRLW6mg7JyCCUcG0DtEGmL2ctUo1PNTin1PUil+r/+4r5MpVgC/fn1kjsx7mjSujKqIpw==", "dev": true, - "license": "MIT", "dependencies": { "json-buffer": "3.0.1" } @@ -8238,7 +8199,6 @@ "integrity": "sha512-JZkJMZkAGFFPP2YqXZXPbMlMBgsxzE8ILs4lMIX/2o0L9UBw9O/Y3o6wFw/i9YLapcUJWwqbi3kdxIPdC62TIA==", "deprecated": "Rimraf versions prior to v4 are no longer supported", "dev": true, - "license": "ISC", "peer": true, "dependencies": { "glob": "^7.1.3" @@ -8706,7 +8666,6 @@ "resolved": "https://registry.npmjs.org/text-table/-/text-table-0.2.0.tgz", "integrity": "sha512-N+8UisAXDGk8PFXP4HAzVR9nbfmVJ3zYLAWiTIoqC5v5isinhr+r5uaO8+7r3BMfuNIufIsA7RdpVgacC2cSpw==", "dev": true, - "license": "MIT", "peer": true }, "node_modules/throttle-debounce": { From 4c2a31642442661bb6fe78f229ec4d6afb5c3ac5 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Jan 2025 21:55:46 +0100 Subject: [PATCH 077/110] chore(deps): bump sanitize-html in /web/ui/react-app (#15758) Bumps [sanitize-html](https://github.com/apostrophecms/sanitize-html) from 2.13.1 to 2.14.0. - [Changelog](https://github.com/apostrophecms/sanitize-html/blob/main/CHANGELOG.md) - [Commits](https://github.com/apostrophecms/sanitize-html/compare/2.13.1...2.14.0) --- updated-dependencies: - dependency-name: sanitize-html dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- web/ui/react-app/package-lock.json | 9 ++++----- web/ui/react-app/package.json | 2 +- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/web/ui/react-app/package-lock.json b/web/ui/react-app/package-lock.json index a74effa39c..8a56e41e22 100644 --- a/web/ui/react-app/package-lock.json +++ b/web/ui/react-app/package-lock.json @@ -42,7 +42,7 @@ "react-router-dom": "^5.3.4", "react-test-renderer": "^17.0.2", "reactstrap": "^8.10.1", - "sanitize-html": "^2.13.1", + "sanitize-html": "^2.14.0", "sass": "1.83.0", "tempusdominus-bootstrap-4": "^5.39.2", "tempusdominus-core": "^5.19.3" @@ -21180,10 +21180,9 @@ "dev": true }, "node_modules/sanitize-html": { - "version": "2.13.1", - "resolved": "https://registry.npmjs.org/sanitize-html/-/sanitize-html-2.13.1.tgz", - "integrity": "sha512-ZXtKq89oue4RP7abL9wp/9URJcqQNABB5GGJ2acW1sdO8JTVl92f4ygD7Yc9Ze09VAZhnt2zegeU0tbNsdcLYg==", - "license": "MIT", + "version": "2.14.0", + "resolved": "https://registry.npmjs.org/sanitize-html/-/sanitize-html-2.14.0.tgz", + "integrity": "sha512-CafX+IUPxZshXqqRaG9ZClSlfPVjSxI0td7n07hk8QO2oO+9JDnlcL8iM8TWeOXOIBFgIOx6zioTzM53AOMn3g==", "dependencies": { "deepmerge": "^4.2.2", "escape-string-regexp": "^4.0.0", diff --git a/web/ui/react-app/package.json b/web/ui/react-app/package.json index 90fb4fb502..3bdb72251c 100644 --- a/web/ui/react-app/package.json +++ b/web/ui/react-app/package.json @@ -37,7 +37,7 @@ "react-router-dom": "^5.3.4", "react-test-renderer": "^17.0.2", "reactstrap": "^8.10.1", - "sanitize-html": "^2.13.1", + "sanitize-html": "^2.14.0", "sass": "1.83.0", "tempusdominus-bootstrap-4": "^5.39.2", "tempusdominus-core": "^5.19.3" From 16b5395bd77801d12bac9e39cf29ed8cc6b3841a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Jan 2025 21:56:24 +0100 Subject: [PATCH 078/110] chore(deps): bump @codemirror/autocomplete in /web/ui/react-app (#15760) Bumps [@codemirror/autocomplete](https://github.com/codemirror/autocomplete) from 6.18.3 to 6.18.4. - [Changelog](https://github.com/codemirror/autocomplete/blob/main/CHANGELOG.md) - [Commits](https://github.com/codemirror/autocomplete/compare/6.18.3...6.18.4) --- updated-dependencies: - dependency-name: "@codemirror/autocomplete" dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- web/ui/react-app/package-lock.json | 15 ++++----------- web/ui/react-app/package.json | 2 +- 2 files changed, 5 insertions(+), 12 deletions(-) diff --git a/web/ui/react-app/package-lock.json b/web/ui/react-app/package-lock.json index 8a56e41e22..ff74defe6e 100644 --- a/web/ui/react-app/package-lock.json +++ b/web/ui/react-app/package-lock.json @@ -8,7 +8,7 @@ "name": "@prometheus-io/app", "version": "0.300.1", "dependencies": { - "@codemirror/autocomplete": "^6.18.3", + "@codemirror/autocomplete": "^6.18.4", "@codemirror/commands": "^6.7.1", "@codemirror/language": "^6.10.8", "@codemirror/lint": "^6.8.4", @@ -2237,21 +2237,14 @@ "dev": true }, "node_modules/@codemirror/autocomplete": { - "version": "6.18.3", - "resolved": "https://registry.npmjs.org/@codemirror/autocomplete/-/autocomplete-6.18.3.tgz", - "integrity": "sha512-1dNIOmiM0z4BIBwxmxEfA1yoxh1MF/6KPBbh20a5vphGV0ictKlgQsbJs6D6SkR6iJpGbpwRsa6PFMNlg9T9pQ==", - "license": "MIT", + "version": "6.18.4", + "resolved": "https://registry.npmjs.org/@codemirror/autocomplete/-/autocomplete-6.18.4.tgz", + "integrity": "sha512-sFAphGQIqyQZfP2ZBsSHV7xQvo9Py0rV0dW7W3IMRdS+zDuNb2l3no78CvUaWKGfzFjI4FTrLdUSj86IGb2hRA==", "dependencies": { "@codemirror/language": "^6.0.0", "@codemirror/state": "^6.0.0", "@codemirror/view": "^6.17.0", "@lezer/common": "^1.0.0" - }, - "peerDependencies": { - "@codemirror/language": "^6.0.0", - "@codemirror/state": "^6.0.0", - "@codemirror/view": "^6.0.0", - "@lezer/common": "^1.0.0" } }, "node_modules/@codemirror/commands": { diff --git a/web/ui/react-app/package.json b/web/ui/react-app/package.json index 3bdb72251c..d126e85a79 100644 --- a/web/ui/react-app/package.json +++ b/web/ui/react-app/package.json @@ -3,7 +3,7 @@ "version": "0.300.1", "private": true, "dependencies": { - "@codemirror/autocomplete": "^6.18.3", + "@codemirror/autocomplete": "^6.18.4", "@codemirror/commands": "^6.7.1", "@codemirror/language": "^6.10.8", "@codemirror/lint": "^6.8.4", From 766d44d47b3a49882216c09e08f6d98af37ea377 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Jan 2025 21:56:56 +0100 Subject: [PATCH 079/110] chore(deps): bump @mantine/notifications in /web/ui (#15818) Bumps [@mantine/notifications](https://github.com/mantinedev/mantine/tree/HEAD/packages/@mantine/notifications) from 7.15.0 to 7.15.3. - [Release notes](https://github.com/mantinedev/mantine/releases) - [Changelog](https://github.com/mantinedev/mantine/blob/master/CHANGELOG.md) - [Commits](https://github.com/mantinedev/mantine/commits/7.15.3/packages/@mantine/notifications) --- updated-dependencies: - dependency-name: "@mantine/notifications" dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- web/ui/mantine-ui/package.json | 2 +- web/ui/package-lock.json | 115 +++++++++++++++------------------ 2 files changed, 53 insertions(+), 64 deletions(-) diff --git a/web/ui/mantine-ui/package.json b/web/ui/mantine-ui/package.json index 7ffe65c4c1..0c0d99027b 100644 --- a/web/ui/mantine-ui/package.json +++ b/web/ui/mantine-ui/package.json @@ -24,7 +24,7 @@ "@mantine/core": "^7.11.2", "@mantine/dates": "^7.15.0", "@mantine/hooks": "^7.11.2", - "@mantine/notifications": "^7.15.0", + "@mantine/notifications": "^7.15.3", "@microsoft/fetch-event-source": "^2.0.1", "@nexucis/fuzzy": "^0.5.1", "@nexucis/kvsearch": "^0.9.1", diff --git a/web/ui/package-lock.json b/web/ui/package-lock.json index febeaa4dcd..4592a18b3c 100644 --- a/web/ui/package-lock.json +++ b/web/ui/package-lock.json @@ -38,7 +38,7 @@ "@mantine/core": "^7.11.2", "@mantine/dates": "^7.15.0", "@mantine/hooks": "^7.11.2", - "@mantine/notifications": "^7.15.0", + "@mantine/notifications": "^7.15.3", "@microsoft/fetch-event-source": "^2.0.1", "@nexucis/fuzzy": "^0.5.1", "@nexucis/kvsearch": "^0.9.1", @@ -2125,19 +2125,19 @@ } }, "node_modules/@mantine/core": { - "version": "7.15.2", - "resolved": "https://registry.npmjs.org/@mantine/core/-/core-7.15.2.tgz", - "integrity": "sha512-640ns0L/HZAXYjz3+FRffr8UNcH1fU7ENUVxKLzqNA311Dcx0qS3byVKTY/IVJYln6AkjoEfIJMiixT9fCZBiQ==", + "version": "7.15.3", + "resolved": "https://registry.npmjs.org/@mantine/core/-/core-7.15.3.tgz", + "integrity": "sha512-8IMTq5xDJDjByDUYkDNKImikASStzrnPtVumKsrEnyEY0zhAWkAe/z/+PjTUMcN44ncJ/PrXQkJ6qMaVWzSZwA==", "dependencies": { "@floating-ui/react": "^0.26.28", "clsx": "^2.1.1", - "react-number-format": "^5.4.2", - "react-remove-scroll": "^2.6.0", + "react-number-format": "^5.4.3", + "react-remove-scroll": "^2.6.2", "react-textarea-autosize": "8.5.6", "type-fest": "^4.27.0" }, "peerDependencies": { - "@mantine/hooks": "7.15.2", + "@mantine/hooks": "7.15.3", "react": "^18.x || ^19.x", "react-dom": "^18.x || ^19.x" } @@ -2159,34 +2159,32 @@ } }, "node_modules/@mantine/hooks": { - "version": "7.15.2", - "resolved": "https://registry.npmjs.org/@mantine/hooks/-/hooks-7.15.2.tgz", - "integrity": "sha512-p8dsW0fdJxzYhULbm1noFYRHuBvJHleYviC0BlwbkVySC8AsvFI8AmC3sMssWV3dQ3yQ/SidYo9U+K/czpDpZw==", + "version": "7.15.3", + "resolved": "https://registry.npmjs.org/@mantine/hooks/-/hooks-7.15.3.tgz", + "integrity": "sha512-rZYObhrmww3OIb4O30pDox/rc+9k3AExO0FSw13t7cfz5/Di+Ho1cChswVFAshnp81ucGEod1fiDOfuyGW7JhA==", "peerDependencies": { "react": "^18.x || ^19.x" } }, "node_modules/@mantine/notifications": { - "version": "7.15.0", - "resolved": "https://registry.npmjs.org/@mantine/notifications/-/notifications-7.15.0.tgz", - "integrity": "sha512-F1g2mFRUTk++ATsbsi8T2WaTRhejB05FusvG3iHC4/a1+0K5Vjh2Armt3VNY0vsUR3V5RdAnP8uZDlMhM7YTQQ==", - "license": "MIT", + "version": "7.15.3", + "resolved": "https://registry.npmjs.org/@mantine/notifications/-/notifications-7.15.3.tgz", + "integrity": "sha512-C1obM5dQsSHIB3B3Kajk0TdLnBpLXFMOIy0otG5khoL/8c8qOU4U0kHxtPVFBFvU/hw4rx7/idiiJdjp8DepDQ==", "dependencies": { - "@mantine/store": "7.15.0", + "@mantine/store": "7.15.3", "react-transition-group": "4.4.5" }, "peerDependencies": { - "@mantine/core": "7.15.0", - "@mantine/hooks": "7.15.0", + "@mantine/core": "7.15.3", + "@mantine/hooks": "7.15.3", "react": "^18.x || ^19.x", "react-dom": "^18.x || ^19.x" } }, "node_modules/@mantine/store": { - "version": "7.15.0", - "resolved": "https://registry.npmjs.org/@mantine/store/-/store-7.15.0.tgz", - "integrity": "sha512-XzQcVFTkD0XspPNsB2NivzbAeZUrLFGO5j8hvKcmGGvUWYlR99GbL7q13ujwJQnNpElqAPSeuN161tnbCqB+Ng==", - "license": "MIT", + "version": "7.15.3", + "resolved": "https://registry.npmjs.org/@mantine/store/-/store-7.15.3.tgz", + "integrity": "sha512-E3pCEm5ozRF/iK/jM1liKntjqaKhotvPtNAqSBcx6AkWSJ8bt16JhNrmrs3J3RmWvfqzF+fftT8HI/3HYbgu9w==", "peerDependencies": { "react": "^18.x || ^19.x" } @@ -5521,14 +5519,6 @@ "license": "ISC", "peer": true }, - "node_modules/invariant": { - "version": "2.2.4", - "resolved": "https://registry.npmjs.org/invariant/-/invariant-2.2.4.tgz", - "integrity": "sha512-phJfQVBuaJM5raOpJjSfkiD6BpbCE4Ns//LaXl6wGYtUBY83nWS6Rf9tXm2e8VaK60JEjYldbPif/A2B1C2gNA==", - "dependencies": { - "loose-envify": "^1.0.0" - } - }, "node_modules/is-arrayish": { "version": "0.2.1", "resolved": "https://registry.npmjs.org/is-arrayish/-/is-arrayish-0.2.1.tgz", @@ -7889,12 +7879,12 @@ "peer": true }, "node_modules/react-number-format": { - "version": "5.4.2", - "resolved": "https://registry.npmjs.org/react-number-format/-/react-number-format-5.4.2.tgz", - "integrity": "sha512-cg//jVdS49PYDgmcYoBnMMHl4XNTMuV723ZnHD2aXYtWWWqbVF3hjQ8iB+UZEuXapLbeA8P8H+1o6ZB1lcw3vg==", + "version": "5.4.3", + "resolved": "https://registry.npmjs.org/react-number-format/-/react-number-format-5.4.3.tgz", + "integrity": "sha512-VCY5hFg/soBighAoGcdE+GagkJq0230qN6jcS5sp8wQX1qy1fYN/RX7/BXkrs0oyzzwqR8/+eSUrqXbGeywdUQ==", "peerDependencies": { - "react": "^0.14 || ^15.0.0 || ^16.0.0 || ^17.0.0 || ^18.0.0", - "react-dom": "^0.14 || ^15.0.0 || ^16.0.0 || ^17.0.0 || ^18.0.0" + "react": "^0.14 || ^15.0.0 || ^16.0.0 || ^17.0.0 || ^18.0.0 || ^19.0.0", + "react-dom": "^0.14 || ^15.0.0 || ^16.0.0 || ^17.0.0 || ^18.0.0 || ^19.0.0" } }, "node_modules/react-redux": { @@ -7931,22 +7921,22 @@ } }, "node_modules/react-remove-scroll": { - "version": "2.6.0", - "resolved": "https://registry.npmjs.org/react-remove-scroll/-/react-remove-scroll-2.6.0.tgz", - "integrity": "sha512-I2U4JVEsQenxDAKaVa3VZ/JeJZe0/2DxPWL8Tj8yLKctQJQiZM52pn/GWFpSp8dftjM3pSAHVJZscAnC/y+ySQ==", + "version": "2.6.2", + "resolved": "https://registry.npmjs.org/react-remove-scroll/-/react-remove-scroll-2.6.2.tgz", + "integrity": "sha512-KmONPx5fnlXYJQqC62Q+lwIeAk64ws/cUw6omIumRzMRPqgnYqhSSti99nbj0Ry13bv7dF+BKn7NB+OqkdZGTw==", "dependencies": { - "react-remove-scroll-bar": "^2.3.6", + "react-remove-scroll-bar": "^2.3.7", "react-style-singleton": "^2.2.1", "tslib": "^2.1.0", - "use-callback-ref": "^1.3.0", + "use-callback-ref": "^1.3.3", "use-sidecar": "^1.1.2" }, "engines": { "node": ">=10" }, "peerDependencies": { - "@types/react": "^16.8.0 || ^17.0.0 || ^18.0.0", - "react": "^16.8.0 || ^17.0.0 || ^18.0.0" + "@types/react": "*", + "react": "^16.8.0 || ^17.0.0 || ^18.0.0 || ^19.0.0 || ^19.0.0-rc" }, "peerDependenciesMeta": { "@types/react": { @@ -7955,19 +7945,19 @@ } }, "node_modules/react-remove-scroll-bar": { - "version": "2.3.6", - "resolved": "https://registry.npmjs.org/react-remove-scroll-bar/-/react-remove-scroll-bar-2.3.6.tgz", - "integrity": "sha512-DtSYaao4mBmX+HDo5YWYdBWQwYIQQshUV/dVxFxK+KM26Wjwp1gZ6rv6OC3oujI6Bfu6Xyg3TwK533AQutsn/g==", + "version": "2.3.8", + "resolved": "https://registry.npmjs.org/react-remove-scroll-bar/-/react-remove-scroll-bar-2.3.8.tgz", + "integrity": "sha512-9r+yi9+mgU33AKcj6IbT9oRCO78WriSj6t/cF8DWBZJ9aOGPOTEDvdUDz1FwKim7QXWwmHqtdHnRJfhAxEG46Q==", "dependencies": { - "react-style-singleton": "^2.2.1", + "react-style-singleton": "^2.2.2", "tslib": "^2.0.0" }, "engines": { "node": ">=10" }, "peerDependencies": { - "@types/react": "^16.8.0 || ^17.0.0 || ^18.0.0", - "react": "^16.8.0 || ^17.0.0 || ^18.0.0" + "@types/react": "*", + "react": "^16.8.0 || ^17.0.0 || ^18.0.0 || ^19.0.0" }, "peerDependenciesMeta": { "@types/react": { @@ -8014,20 +8004,19 @@ } }, "node_modules/react-style-singleton": { - "version": "2.2.1", - "resolved": "https://registry.npmjs.org/react-style-singleton/-/react-style-singleton-2.2.1.tgz", - "integrity": "sha512-ZWj0fHEMyWkHzKYUr2Bs/4zU6XLmq9HsgBURm7g5pAVfyn49DgUiNgY2d4lXRlYSiCif9YBGpQleewkcqddc7g==", + "version": "2.2.3", + "resolved": "https://registry.npmjs.org/react-style-singleton/-/react-style-singleton-2.2.3.tgz", + "integrity": "sha512-b6jSvxvVnyptAiLjbkWLE/lOnR4lfTtDAl+eUC7RZy+QQWc6wRzIV2CE6xBuMmDxc2qIihtDCZD5NPOFl7fRBQ==", "dependencies": { "get-nonce": "^1.0.0", - "invariant": "^2.2.4", "tslib": "^2.0.0" }, "engines": { "node": ">=10" }, "peerDependencies": { - "@types/react": "^16.8.0 || ^17.0.0 || ^18.0.0", - "react": "^16.8.0 || ^17.0.0 || ^18.0.0" + "@types/react": "*", + "react": "^16.8.0 || ^17.0.0 || ^18.0.0 || ^19.0.0 || ^19.0.0-rc" }, "peerDependenciesMeta": { "@types/react": { @@ -8973,9 +8962,9 @@ } }, "node_modules/use-callback-ref": { - "version": "1.3.2", - "resolved": "https://registry.npmjs.org/use-callback-ref/-/use-callback-ref-1.3.2.tgz", - "integrity": "sha512-elOQwe6Q8gqZgDA8mrh44qRTQqpIHDcZ3hXTLjBe1i4ph8XpNJnO+aQf3NaG+lriLopI4HMx9VjQLfPQ6vhnoA==", + "version": "1.3.3", + "resolved": "https://registry.npmjs.org/use-callback-ref/-/use-callback-ref-1.3.3.tgz", + "integrity": "sha512-jQL3lRnocaFtu3V00JToYz/4QkNWswxijDaCVNZRiRTO3HQDLsdu1ZtmIUvV4yPp+rvWm5j0y0TG/S61cuijTg==", "dependencies": { "tslib": "^2.0.0" }, @@ -8983,8 +8972,8 @@ "node": ">=10" }, "peerDependencies": { - "@types/react": "^16.8.0 || ^17.0.0 || ^18.0.0", - "react": "^16.8.0 || ^17.0.0 || ^18.0.0" + "@types/react": "*", + "react": "^16.8.0 || ^17.0.0 || ^18.0.0 || ^19.0.0 || ^19.0.0-rc" }, "peerDependenciesMeta": { "@types/react": { @@ -9058,9 +9047,9 @@ } }, "node_modules/use-sidecar": { - "version": "1.1.2", - "resolved": "https://registry.npmjs.org/use-sidecar/-/use-sidecar-1.1.2.tgz", - "integrity": "sha512-epTbsLuzZ7lPClpz2TyryBfztm7m+28DlEv2ZCQ3MDr5ssiwyOwGH/e5F9CkfWjJ1t4clvI58yF822/GUkjjhw==", + "version": "1.1.3", + "resolved": "https://registry.npmjs.org/use-sidecar/-/use-sidecar-1.1.3.tgz", + "integrity": "sha512-Fedw0aZvkhynoPYlA5WXrMCAMm+nSWdZt6lzJQ7Ok8S6Q+VsHmHpRWndVRJ8Be0ZbkfPc5LRYH+5XrzXcEeLRQ==", "dependencies": { "detect-node-es": "^1.1.0", "tslib": "^2.0.0" @@ -9069,8 +9058,8 @@ "node": ">=10" }, "peerDependencies": { - "@types/react": "^16.9.0 || ^17.0.0 || ^18.0.0", - "react": "^16.8.0 || ^17.0.0 || ^18.0.0" + "@types/react": "*", + "react": "^16.8.0 || ^17.0.0 || ^18.0.0 || ^19.0.0 || ^19.0.0-rc" }, "peerDependenciesMeta": { "@types/react": { From 9142f10fbbb37756c1c084d578964081f82feac4 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Jan 2025 21:57:28 +0100 Subject: [PATCH 080/110] chore(deps): bump @mantine/dates from 7.15.0 to 7.15.3 in /web/ui (#15819) Bumps [@mantine/dates](https://github.com/mantinedev/mantine/tree/HEAD/packages/@mantine/dates) from 7.15.0 to 7.15.3. - [Release notes](https://github.com/mantinedev/mantine/releases) - [Changelog](https://github.com/mantinedev/mantine/blob/master/CHANGELOG.md) - [Commits](https://github.com/mantinedev/mantine/commits/7.15.3/packages/@mantine/dates) --- updated-dependencies: - dependency-name: "@mantine/dates" dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- web/ui/mantine-ui/package.json | 2 +- web/ui/package-lock.json | 13 ++++++------- 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/web/ui/mantine-ui/package.json b/web/ui/mantine-ui/package.json index 0c0d99027b..b89ab83546 100644 --- a/web/ui/mantine-ui/package.json +++ b/web/ui/mantine-ui/package.json @@ -22,7 +22,7 @@ "@lezer/highlight": "^1.2.1", "@mantine/code-highlight": "^7.15.2", "@mantine/core": "^7.11.2", - "@mantine/dates": "^7.15.0", + "@mantine/dates": "^7.15.3", "@mantine/hooks": "^7.11.2", "@mantine/notifications": "^7.15.3", "@microsoft/fetch-event-source": "^2.0.1", diff --git a/web/ui/package-lock.json b/web/ui/package-lock.json index 4592a18b3c..ad4996bc9a 100644 --- a/web/ui/package-lock.json +++ b/web/ui/package-lock.json @@ -36,7 +36,7 @@ "@lezer/highlight": "^1.2.1", "@mantine/code-highlight": "^7.15.2", "@mantine/core": "^7.11.2", - "@mantine/dates": "^7.15.0", + "@mantine/dates": "^7.15.3", "@mantine/hooks": "^7.11.2", "@mantine/notifications": "^7.15.3", "@microsoft/fetch-event-source": "^2.0.1", @@ -2143,16 +2143,15 @@ } }, "node_modules/@mantine/dates": { - "version": "7.15.0", - "resolved": "https://registry.npmjs.org/@mantine/dates/-/dates-7.15.0.tgz", - "integrity": "sha512-EM1Tp29DH6i6XH+fPkpNsjUQA32u+4yqOggFtytvuHgDITOj82xh2NIZGSyGH/rzkcy/Bh/t65pK3SoZJGTY6Q==", - "license": "MIT", + "version": "7.15.3", + "resolved": "https://registry.npmjs.org/@mantine/dates/-/dates-7.15.3.tgz", + "integrity": "sha512-lv71dcfA8qB43v03cRELC2/G7FQXfAgj0tSMImj2p2FL3PSWWF4WRvW6byiB+hszk4lgooSo7kppzkSMVUlsdA==", "dependencies": { "clsx": "^2.1.1" }, "peerDependencies": { - "@mantine/core": "7.15.0", - "@mantine/hooks": "7.15.0", + "@mantine/core": "7.15.3", + "@mantine/hooks": "7.15.3", "dayjs": ">=1.0.0", "react": "^18.x || ^19.x", "react-dom": "^18.x || ^19.x" From f34f717b3d714a536fd32a4d25ac310072cf2e1c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Jan 2025 21:57:52 +0100 Subject: [PATCH 081/110] chore(deps): bump @tabler/icons-react from 3.24.0 to 3.28.1 in /web/ui (#15820) Bumps [@tabler/icons-react](https://github.com/tabler/tabler-icons/tree/HEAD/packages/icons-react) from 3.24.0 to 3.28.1. - [Release notes](https://github.com/tabler/tabler-icons/releases) - [Commits](https://github.com/tabler/tabler-icons/commits/v3.28.1/packages/icons-react) --- updated-dependencies: - dependency-name: "@tabler/icons-react" dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- web/ui/mantine-ui/package.json | 2 +- web/ui/package-lock.json | 18 ++++++++---------- 2 files changed, 9 insertions(+), 11 deletions(-) diff --git a/web/ui/mantine-ui/package.json b/web/ui/mantine-ui/package.json index b89ab83546..252df975f1 100644 --- a/web/ui/mantine-ui/package.json +++ b/web/ui/mantine-ui/package.json @@ -30,7 +30,7 @@ "@nexucis/kvsearch": "^0.9.1", "@prometheus-io/codemirror-promql": "0.301.0", "@reduxjs/toolkit": "^2.5.0", - "@tabler/icons-react": "^3.24.0", + "@tabler/icons-react": "^3.28.1", "@tanstack/react-query": "^5.62.11", "@testing-library/jest-dom": "^6.6.3", "@testing-library/react": "^16.1.0", diff --git a/web/ui/package-lock.json b/web/ui/package-lock.json index ad4996bc9a..4675460743 100644 --- a/web/ui/package-lock.json +++ b/web/ui/package-lock.json @@ -44,7 +44,7 @@ "@nexucis/kvsearch": "^0.9.1", "@prometheus-io/codemirror-promql": "0.301.0", "@reduxjs/toolkit": "^2.5.0", - "@tabler/icons-react": "^3.24.0", + "@tabler/icons-react": "^3.28.1", "@tanstack/react-query": "^5.62.11", "@testing-library/jest-dom": "^6.6.3", "@testing-library/react": "^16.1.0", @@ -2619,22 +2619,20 @@ } }, "node_modules/@tabler/icons": { - "version": "3.24.0", - "resolved": "https://registry.npmjs.org/@tabler/icons/-/icons-3.24.0.tgz", - "integrity": "sha512-qNis9e90QcdxAGV3wNIeX0Ba2R7ktm0cnqOToKHJfC2kj3fvJwEVLsw63K0/fm7NW8rSZjDSTQRmMnSg8g/wrg==", - "license": "MIT", + "version": "3.28.1", + "resolved": "https://registry.npmjs.org/@tabler/icons/-/icons-3.28.1.tgz", + "integrity": "sha512-h7nqKEvFooLtFxhMOC1/2eiV+KRXhBUuDUUJrJlt6Ft6tuMw2eU/9GLQgrTk41DNmIEzp/LI83K9J9UUU8YBYQ==", "funding": { "type": "github", "url": "https://github.com/sponsors/codecalm" } }, "node_modules/@tabler/icons-react": { - "version": "3.24.0", - "resolved": "https://registry.npmjs.org/@tabler/icons-react/-/icons-react-3.24.0.tgz", - "integrity": "sha512-m9c7TmlcDmKqvZAasG5rv1YvazZDrVEhNdNFa2d1Bzotc0dh+iceFdiZCEcYPDb5UcRyLAMvOaOC9y/5sfMMWw==", - "license": "MIT", + "version": "3.28.1", + "resolved": "https://registry.npmjs.org/@tabler/icons-react/-/icons-react-3.28.1.tgz", + "integrity": "sha512-KNBpA2kbxr3/2YK5swt7b/kd/xpDP1FHYZCxDFIw54tX8slELRFEf95VMxsccQHZeIcUbdoojmUUuYSbt/sM5Q==", "dependencies": { - "@tabler/icons": "3.24.0" + "@tabler/icons": "3.28.1" }, "funding": { "type": "github", From 93ca2e6113ab0b47ff782bc04ace63fed7303107 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Jan 2025 23:13:12 +0100 Subject: [PATCH 082/110] chore(deps): bump @codemirror/view from 6.35.3 to 6.36.2 in /web/ui (#15816) Bumps [@codemirror/view](https://github.com/codemirror/view) from 6.35.3 to 6.36.2. - [Changelog](https://github.com/codemirror/view/blob/main/CHANGELOG.md) - [Commits](https://github.com/codemirror/view/compare/6.35.3...6.36.2) --- updated-dependencies: - dependency-name: "@codemirror/view" dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- web/ui/mantine-ui/package.json | 2 +- web/ui/module/codemirror-promql/package.json | 2 +- web/ui/package-lock.json | 11 +++++------ 3 files changed, 7 insertions(+), 8 deletions(-) diff --git a/web/ui/mantine-ui/package.json b/web/ui/mantine-ui/package.json index 252df975f1..e14b945fd9 100644 --- a/web/ui/mantine-ui/package.json +++ b/web/ui/mantine-ui/package.json @@ -16,7 +16,7 @@ "@codemirror/language": "^6.10.6", "@codemirror/lint": "^6.8.4", "@codemirror/state": "^6.4.1", - "@codemirror/view": "^6.34.1", + "@codemirror/view": "^6.36.2", "@floating-ui/dom": "^1.6.12", "@lezer/common": "^1.2.3", "@lezer/highlight": "^1.2.1", diff --git a/web/ui/module/codemirror-promql/package.json b/web/ui/module/codemirror-promql/package.json index aea4d22c62..3f2c7f314d 100644 --- a/web/ui/module/codemirror-promql/package.json +++ b/web/ui/module/codemirror-promql/package.json @@ -37,7 +37,7 @@ "@codemirror/language": "^6.10.6", "@codemirror/lint": "^6.8.4", "@codemirror/state": "^6.3.3", - "@codemirror/view": "^6.34.1", + "@codemirror/view": "^6.36.2", "@lezer/common": "^1.2.3", "@lezer/highlight": "^1.2.0", "@lezer/lr": "^1.4.2", diff --git a/web/ui/package-lock.json b/web/ui/package-lock.json index 4675460743..06513f8ee3 100644 --- a/web/ui/package-lock.json +++ b/web/ui/package-lock.json @@ -30,7 +30,7 @@ "@codemirror/language": "^6.10.6", "@codemirror/lint": "^6.8.4", "@codemirror/state": "^6.4.1", - "@codemirror/view": "^6.34.1", + "@codemirror/view": "^6.36.2", "@floating-ui/dom": "^1.6.12", "@lezer/common": "^1.2.3", "@lezer/highlight": "^1.2.1", @@ -157,7 +157,7 @@ "@codemirror/language": "^6.10.6", "@codemirror/lint": "^6.8.4", "@codemirror/state": "^6.3.3", - "@codemirror/view": "^6.34.1", + "@codemirror/view": "^6.36.2", "@lezer/common": "^1.2.3", "@lezer/highlight": "^1.2.0", "@lezer/lr": "^1.4.2", @@ -857,10 +857,9 @@ } }, "node_modules/@codemirror/view": { - "version": "6.35.3", - "resolved": "https://registry.npmjs.org/@codemirror/view/-/view-6.35.3.tgz", - "integrity": "sha512-ScY7L8+EGdPl4QtoBiOzE4FELp7JmNUsBvgBcCakXWM2uiv/K89VAzU3BMDscf0DsACLvTKePbd5+cFDTcei6g==", - "license": "MIT", + "version": "6.36.2", + "resolved": "https://registry.npmjs.org/@codemirror/view/-/view-6.36.2.tgz", + "integrity": "sha512-DZ6ONbs8qdJK0fdN7AB82CgI6tYXf4HWk1wSVa0+9bhVznCuuvhQtX8bFBoy3dv8rZSQqUd8GvhVAcielcidrA==", "dependencies": { "@codemirror/state": "^6.5.0", "style-mod": "^4.1.0", From 1e81fd216631913104b37fc6a43312cb3af0eec4 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Jan 2025 23:14:05 +0100 Subject: [PATCH 083/110] chore(deps): bump @codemirror/language from 6.10.6 to 6.10.8 in /web/ui (#15742) Bumps [@codemirror/language](https://github.com/codemirror/language) from 6.10.6 to 6.10.8. - [Changelog](https://github.com/codemirror/language/blob/main/CHANGELOG.md) - [Commits](https://github.com/codemirror/language/compare/6.10.6...6.10.8) --- updated-dependencies: - dependency-name: "@codemirror/language" dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- web/ui/mantine-ui/package.json | 2 +- web/ui/module/codemirror-promql/package.json | 2 +- web/ui/package-lock.json | 10 +++++----- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/web/ui/mantine-ui/package.json b/web/ui/mantine-ui/package.json index e14b945fd9..9cdb67aea5 100644 --- a/web/ui/mantine-ui/package.json +++ b/web/ui/mantine-ui/package.json @@ -13,7 +13,7 @@ }, "dependencies": { "@codemirror/autocomplete": "^6.18.4", - "@codemirror/language": "^6.10.6", + "@codemirror/language": "^6.10.8", "@codemirror/lint": "^6.8.4", "@codemirror/state": "^6.4.1", "@codemirror/view": "^6.36.2", diff --git a/web/ui/module/codemirror-promql/package.json b/web/ui/module/codemirror-promql/package.json index 3f2c7f314d..270fa16b29 100644 --- a/web/ui/module/codemirror-promql/package.json +++ b/web/ui/module/codemirror-promql/package.json @@ -34,7 +34,7 @@ }, "devDependencies": { "@codemirror/autocomplete": "^6.18.4", - "@codemirror/language": "^6.10.6", + "@codemirror/language": "^6.10.8", "@codemirror/lint": "^6.8.4", "@codemirror/state": "^6.3.3", "@codemirror/view": "^6.36.2", diff --git a/web/ui/package-lock.json b/web/ui/package-lock.json index 06513f8ee3..f9beb96da0 100644 --- a/web/ui/package-lock.json +++ b/web/ui/package-lock.json @@ -27,7 +27,7 @@ "version": "0.301.0", "dependencies": { "@codemirror/autocomplete": "^6.18.4", - "@codemirror/language": "^6.10.6", + "@codemirror/language": "^6.10.8", "@codemirror/lint": "^6.8.4", "@codemirror/state": "^6.4.1", "@codemirror/view": "^6.36.2", @@ -154,7 +154,7 @@ }, "devDependencies": { "@codemirror/autocomplete": "^6.18.4", - "@codemirror/language": "^6.10.6", + "@codemirror/language": "^6.10.8", "@codemirror/lint": "^6.8.4", "@codemirror/state": "^6.3.3", "@codemirror/view": "^6.36.2", @@ -801,9 +801,9 @@ } }, "node_modules/@codemirror/language": { - "version": "6.10.6", - "resolved": "https://registry.npmjs.org/@codemirror/language/-/language-6.10.6.tgz", - "integrity": "sha512-KrsbdCnxEztLVbB5PycWXFxas4EOyk/fPAfruSOnDDppevQgid2XZ+KbJ9u+fDikP/e7MW7HPBTvTb8JlZK9vA==", + "version": "6.10.8", + "resolved": "https://registry.npmjs.org/@codemirror/language/-/language-6.10.8.tgz", + "integrity": "sha512-wcP8XPPhDH2vTqf181U8MbZnW+tDyPYy0UzVOa+oHORjyT+mhhom9vBd7dApJwoDz9Nb/a8kHjJIsuA/t8vNFw==", "dependencies": { "@codemirror/state": "^6.0.0", "@codemirror/view": "^6.23.0", From e1324112aa85facaca5f9ec9b5b7b05103c6804b Mon Sep 17 00:00:00 2001 From: Ben Ye Date: Mon, 13 Jan 2025 15:37:03 -0800 Subject: [PATCH 084/110] less context cancellation check for series API Signed-off-by: Ben Ye --- web/api/v1/api.go | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/web/api/v1/api.go b/web/api/v1/api.go index ea7d5c5fe4..fd53288717 100644 --- a/web/api/v1/api.go +++ b/web/api/v1/api.go @@ -68,6 +68,9 @@ const ( // Non-standard status code (originally introduced by nginx) for the case when a client closes // the connection while the server is still processing the request. statusClientClosedConnection = 499 + + // checkContextEveryNIterations is used in some tight loops to check if the context is done. + checkContextEveryNIterations = 128 ) type errorType string @@ -962,10 +965,15 @@ func (api *API) series(r *http.Request) (result apiFuncResult) { warnings := set.Warnings() + i := 1 for set.Next() { - if err := ctx.Err(); err != nil { - return apiFuncResult{nil, returnAPIError(err), warnings, closer} + if i%checkContextEveryNIterations == 0 { + if err := ctx.Err(); err != nil { + return apiFuncResult{nil, returnAPIError(err), warnings, closer} + } } + i++ + metrics = append(metrics, set.At().Labels()) if limit > 0 && len(metrics) > limit { From 5303e515afe40966ebac5451a5a27275a6cdfa1a Mon Sep 17 00:00:00 2001 From: sh0rez Date: Tue, 14 Jan 2025 15:33:31 +0100 Subject: [PATCH 085/110] remote/otlp: convert delta to cumulative (#15165) What Adds support for OTLP delta temporality to the OTLP endpoint. This is done by calling the deltatocumulative processor from the OpenTelemetry collector during OTLP conversion. Why Delta conversion is a naturally stateful process, which requires careful request routing when operated inside a collector. Prometheus is already stateful and doing the conversion in-server reduces the operational burden on the ingest architecture by only having one stateful component. How deltatocumulative is a OTel collector component that works as follows: * pmetric.Metrics come from a receiver or in this case from the HTTP client * It operates as an in-place update loop: * for each sample, if not delta, leave unmodified * if delta, do: * state += sample, where state is the in-memory sum of all previous samples * sample = state, sample value is now cumulative * this is supported for sums (counters), gauges, histograms (old histograms) and exponential histograms (native histograms) If a series receives no new samples for 5m, its state is removed from memory Performance Delta performance is a stateful operation and the OTel code is not highly optimized yet, e.g. it locks the entire processor for each request. Nonetheless, care has been taken to mitigate those effects: delta conversion is behind a feature flag. If disabled, no conversion code is ever invoked if enabled, conversion is not invoked if request not actually contains delta samples. This leads to no measureable performance difference between default-cumulative to convert-cumulative (only cumulative, feature on/off) Signed-off-by: sh0rez --- cmd/prometheus/main.go | 5 +- docs/command-line/prometheus.md | 2 +- docs/feature_flags.md | 22 ++ docs/querying/api.md | 9 + go.mod | 11 +- go.sum | 54 ++++- storage/remote/write_handler.go | 133 ++++++++--- storage/remote/write_test.go | 377 +++++++++++++++++++++++++++++++- web/api/v1/api.go | 4 +- web/api/v1/errors_test.go | 1 + web/web.go | 2 + 11 files changed, 586 insertions(+), 34 deletions(-) diff --git a/cmd/prometheus/main.go b/cmd/prometheus/main.go index 03c20dc52d..3c2b5ee0c1 100644 --- a/cmd/prometheus/main.go +++ b/cmd/prometheus/main.go @@ -275,6 +275,9 @@ func (c *flagConfig) setFeatureListOptions(logger *slog.Logger) error { case "old-ui": c.web.UseOldUI = true logger.Info("Serving previous version of the Prometheus web UI.") + case "otlp-deltatocumulative": + c.web.ConvertOTLPDelta = true + logger.Info("Converting delta OTLP metrics to cumulative") default: logger.Warn("Unknown option for --enable-feature", "option", o) } @@ -516,7 +519,7 @@ func main() { a.Flag("scrape.discovery-reload-interval", "Interval used by scrape manager to throttle target groups updates."). Hidden().Default("5s").SetValue(&cfg.scrape.DiscoveryReloadInterval) - a.Flag("enable-feature", "Comma separated feature names to enable. Valid options: exemplar-storage, expand-external-labels, memory-snapshot-on-shutdown, promql-per-step-stats, promql-experimental-functions, extra-scrape-metrics, auto-gomaxprocs, native-histograms, created-timestamp-zero-ingestion, concurrent-rule-eval, delayed-compaction, old-ui. See https://prometheus.io/docs/prometheus/latest/feature_flags/ for more details."). + a.Flag("enable-feature", "Comma separated feature names to enable. Valid options: exemplar-storage, expand-external-labels, memory-snapshot-on-shutdown, promql-per-step-stats, promql-experimental-functions, extra-scrape-metrics, auto-gomaxprocs, native-histograms, created-timestamp-zero-ingestion, concurrent-rule-eval, delayed-compaction, old-ui, otlp-deltatocumulative. See https://prometheus.io/docs/prometheus/latest/feature_flags/ for more details."). Default("").StringsVar(&cfg.featureList) a.Flag("agent", "Run Prometheus in 'Agent mode'.").BoolVar(&agentMode) diff --git a/docs/command-line/prometheus.md b/docs/command-line/prometheus.md index dd207dc382..9b4ec8b736 100644 --- a/docs/command-line/prometheus.md +++ b/docs/command-line/prometheus.md @@ -60,7 +60,7 @@ The Prometheus monitoring server | --query.timeout | Maximum time a query may take before being aborted. Use with server mode only. | `2m` | | --query.max-concurrency | Maximum number of queries executed concurrently. Use with server mode only. | `20` | | --query.max-samples | Maximum number of samples a single query can load into memory. Note that queries will fail if they try to load more samples than this into memory, so this also limits the number of samples a query can return. Use with server mode only. | `50000000` | -| --enable-feature ... | Comma separated feature names to enable. Valid options: exemplar-storage, expand-external-labels, memory-snapshot-on-shutdown, promql-per-step-stats, promql-experimental-functions, extra-scrape-metrics, auto-gomaxprocs, native-histograms, created-timestamp-zero-ingestion, concurrent-rule-eval, delayed-compaction, old-ui. See https://prometheus.io/docs/prometheus/latest/feature_flags/ for more details. | | +| --enable-feature ... | Comma separated feature names to enable. Valid options: exemplar-storage, expand-external-labels, memory-snapshot-on-shutdown, promql-per-step-stats, promql-experimental-functions, extra-scrape-metrics, auto-gomaxprocs, native-histograms, created-timestamp-zero-ingestion, concurrent-rule-eval, delayed-compaction, old-ui, otlp-deltatocumulative. See https://prometheus.io/docs/prometheus/latest/feature_flags/ for more details. | | | --agent | Run Prometheus in 'Agent mode'. | | | --log.level | Only log messages with the given severity or above. One of: [debug, info, warn, error] | `info` | | --log.format | Output format of log messages. One of: [logfmt, json] | `logfmt` | diff --git a/docs/feature_flags.md b/docs/feature_flags.md index 67f31c260f..0de3d2bf7d 100644 --- a/docs/feature_flags.md +++ b/docs/feature_flags.md @@ -151,3 +151,25 @@ Configuration reloads are triggered by detecting changes in the checksum of the main configuration file or any referenced files, such as rule and scrape configurations. To ensure consistency and avoid issues during reloads, it's recommended to update these files atomically. + +## OTLP Delta Conversion + +`--enable-feature=otlp-deltatocumulative` + +When enabled, Prometheus will convert OTLP metrics from delta temporality to their +cumulative equivalent, instead of dropping them. + +This uses +[deltatocumulative][d2c] +from the OTel collector, using its default settings. + +Delta conversion keeps in-memory state to aggregate delta changes per-series over time. +When Prometheus restarts, this state is lost, starting the aggregation from zero +again. This results in a counter reset in the cumulative series. + +This state is periodically ([`max_stale`][d2c]) cleared of inactive series. + +Enabling this _can_ have negative impact on performance, because the in-memory +state is mutex guarded. Cumulative-only OTLP requests are not affected. + +[d2c]: https://github.com/open-telemetry/opentelemetry-collector-contrib/tree/main/processor/deltatocumulativeprocessor diff --git a/docs/querying/api.md b/docs/querying/api.md index e3f97886dc..1366dc02c2 100644 --- a/docs/querying/api.md +++ b/docs/querying/api.md @@ -1424,6 +1424,15 @@ endpoint is `/api/v1/otlp/v1/metrics`. *New in v2.47* +### OTLP Delta + +Prometheus can convert incoming metrics from delta temporality to their cumulative equivalent. +This is done using [deltatocumulative](https://github.com/open-telemetry/opentelemetry-collector-contrib/tree/main/processor/deltatocumulativeprocessor) from the OpenTelemetry Collector. + +To enable, pass `--enable-feature=otlp-deltatocumulative`. + +*New in v3.2* + ## Notifications The following endpoints provide information about active status notifications concerning the Prometheus server itself. diff --git a/go.mod b/go.mod index 93cd89f469..98075a56fa 100644 --- a/go.mod +++ b/go.mod @@ -48,6 +48,7 @@ require ( github.com/nsf/jsondiff v0.0.0-20230430225905-43f6cf3098c1 github.com/oklog/run v1.1.0 github.com/oklog/ulid v1.3.1 + github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor v0.116.0 github.com/ovh/go-ovh v1.6.0 github.com/prometheus/alertmanager v0.27.0 github.com/prometheus/client_golang v1.20.5 @@ -60,7 +61,10 @@ require ( github.com/shurcooL/httpfs v0.0.0-20230704072500-f1e31cf0ba5c github.com/stretchr/testify v1.10.0 github.com/vultr/govultr/v2 v2.17.2 + go.opentelemetry.io/collector/component v0.116.0 + go.opentelemetry.io/collector/consumer v1.22.0 go.opentelemetry.io/collector/pdata v1.22.0 + go.opentelemetry.io/collector/processor v0.116.0 go.opentelemetry.io/collector/semconv v0.116.0 go.opentelemetry.io/contrib/instrumentation/net/http/httptrace/otelhttptrace v0.58.0 go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.58.0 @@ -68,6 +72,7 @@ require ( go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.33.0 go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.33.0 go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.33.0 + go.opentelemetry.io/otel/metric v1.33.0 go.opentelemetry.io/otel/sdk v1.33.0 go.opentelemetry.io/otel/trace v1.33.0 go.uber.org/atomic v1.11.0 @@ -168,6 +173,8 @@ require ( github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect github.com/modern-go/reflect2 v1.0.2 // indirect github.com/morikuni/aec v1.0.0 // indirect + github.com/open-telemetry/opentelemetry-collector-contrib/internal/exp/metrics v0.116.0 // indirect + github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatautil v0.116.0 // indirect github.com/opencontainers/go-digest v1.0.0 // indirect github.com/opencontainers/image-spec v1.0.2 // indirect github.com/opencontainers/runtime-spec v1.0.2 // indirect @@ -184,8 +191,10 @@ require ( github.com/xhit/go-str2duration/v2 v2.1.0 // indirect go.mongodb.org/mongo-driver v1.14.0 // indirect go.opentelemetry.io/auto/sdk v1.1.0 // indirect - go.opentelemetry.io/otel/metric v1.33.0 // indirect + go.opentelemetry.io/collector/config/configtelemetry v0.116.0 // indirect + go.opentelemetry.io/collector/pipeline v0.116.0 // indirect go.opentelemetry.io/proto/otlp v1.4.0 // indirect + go.uber.org/zap v1.27.0 // indirect golang.org/x/crypto v0.31.0 // indirect golang.org/x/exp v0.0.0-20240119083558-1b970713d09a // indirect golang.org/x/mod v0.22.0 // indirect diff --git a/go.sum b/go.sum index 515fb9c053..009b714060 100644 --- a/go.sum +++ b/go.sum @@ -165,6 +165,8 @@ github.com/go-resty/resty/v2 v2.16.2/go.mod h1:0fHAoK7JoBy/Ch36N8VFeMsK7xQOHhvWa github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/go-task/slim-sprig/v3 v3.0.0 h1:sUs3vkvUymDpBKi3qH1YSqBQk9+9D/8M2mN1vB6EwHI= github.com/go-task/slim-sprig/v3 v3.0.0/go.mod h1:W848ghGpv3Qj3dhTPRyJypKRiqCdHZiAzKg9hl15HA8= +github.com/go-viper/mapstructure/v2 v2.2.1 h1:ZAaOCxANMuZx5RCeg0mBdEZk7DZasvvZIxtHqx8aGss= +github.com/go-viper/mapstructure/v2 v2.2.1/go.mod h1:oJDH3BJKyqBA2TXFhDsKDGDTlndYOZ6rGS0BRZIxGhM= github.com/go-zookeeper/zk v1.0.4 h1:DPzxraQx7OrPyXq2phlGlNSIyWEsAox0RJmjTseMV6I= github.com/go-zookeeper/zk v1.0.4/go.mod h1:nOB03cncLtlp4t+UAkGSV+9beXP/akpekBwL+UX1Qcw= github.com/godbus/dbus/v5 v5.0.4 h1:9349emZab16e7zQvpmsbtjc18ykshndd8y2PG3sgJbA= @@ -304,6 +306,12 @@ github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/compress v1.17.11 h1:In6xLpyWOi1+C7tXUUWv2ot1QvBjxevKAaI6IXrJmUc= github.com/klauspost/compress v1.17.11/go.mod h1:pMDklpSncoRMuLFrf1W9Ss9KT+0rH90U12bZKk7uwG0= +github.com/knadh/koanf/maps v0.1.1 h1:G5TjmUh2D7G2YWf5SQQqSiHRJEjaicvU0KpypqB3NIs= +github.com/knadh/koanf/maps v0.1.1/go.mod h1:npD/QZY3V6ghQDdcQzl1W4ICNVTkohC8E73eI2xW4yI= +github.com/knadh/koanf/providers/confmap v0.1.0 h1:gOkxhHkemwG4LezxxN8DMOFopOPghxRVp7JbIvdvqzU= +github.com/knadh/koanf/providers/confmap v0.1.0/go.mod h1:2uLhxQzJnyHKfxG927awZC7+fyHFdQkd697K4MdLnIU= +github.com/knadh/koanf/v2 v2.1.2 h1:I2rtLRqXRy1p01m/utEtpZSSA6dcJbgGVuE27kW2PzQ= +github.com/knadh/koanf/v2 v2.1.2/go.mod h1:Gphfaen0q1Fc1HTgJgSTC4oRX9R2R5ErYMZJy8fLJBo= github.com/kolo/xmlrpc v0.0.0-20220921171641-a4b6fa1dd06b h1:udzkj9S/zlT5X367kqJis0QP7YMxobob6zhzq6Yre00= github.com/kolo/xmlrpc v0.0.0-20220921171641-a4b6fa1dd06b/go.mod h1:pcaDhQK0/NJZEvtCO0qQPPropqV0sJOJ6YW7X+9kRwM= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= @@ -348,6 +356,8 @@ github.com/miekg/dns v1.1.41/go.mod h1:p6aan82bvRIyn+zDIv9xYNUpwa73JcSh9BKwknJys github.com/miekg/dns v1.1.62 h1:cN8OuEF1/x5Rq6Np+h1epln8OiyPWV+lROx9LxcGgIQ= github.com/miekg/dns v1.1.62/go.mod h1:mvDlcItzm+br7MToIKqkglaGhlFMHJ9DTNNWONWXbNQ= github.com/mitchellh/cli v1.1.0/go.mod h1:xcISNoH86gajksDmfB23e/pu+B+GeFRMYmoHXxx3xhI= +github.com/mitchellh/copystructure v1.2.0 h1:vpKXTN4ewci03Vljg/q9QvCGUDttBOGBIa15WveJJGw= +github.com/mitchellh/copystructure v1.2.0/go.mod h1:qLl+cE2AmVv+CoeAwDPye/v+N2HKCj9FbZEVFJRxO9s= github.com/mitchellh/go-homedir v1.1.0 h1:lukF9ziXFxDFPkA1vsr5zpc1XuPDn/wFntq5mG+4E0Y= github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= github.com/mitchellh/go-testing-interface v1.14.1 h1:jrgshOhYAUVNMAJiKbEu7EqAwgJJ2JqpQmpLJOu07cU= @@ -355,6 +365,8 @@ github.com/mitchellh/go-testing-interface v1.14.1/go.mod h1:gfgS7OtZj6MA4U1UrDRp github.com/mitchellh/mapstructure v0.0.0-20160808181253-ca63d7c062ee/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= github.com/mitchellh/mapstructure v1.5.0 h1:jeMsZIYE/09sWLaz43PL7Gy6RuMjD2eJVyuac5Z2hdY= github.com/mitchellh/mapstructure v1.5.0/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo= +github.com/mitchellh/reflectwalk v1.0.2 h1:G2LzWKi524PWgd3mLHV8Y5k7s6XUvT0Gef6zxSIeXaQ= +github.com/mitchellh/reflectwalk v1.0.2/go.mod h1:mSTlrgnPZtwu0c4WaC2kGObEpuNDbx0jmZXqmk4esnw= github.com/moby/docker-image-spec v1.3.1 h1:jMKff3w6PgbfSa69GfNg+zN/XLhfXJGnEx3Nl2EsFP0= github.com/moby/docker-image-spec v1.3.1/go.mod h1:eKmb5VW8vQEh/BAr2yvVNvuiJuY6UIocYsFu/DxxRpo= github.com/moby/term v0.0.0-20210619224110-3f7ff695adc6 h1:dcztxKSvZ4Id8iPpHERQBbIJfabdt4wUm5qy3wOL2Zc= @@ -383,6 +395,14 @@ github.com/onsi/ginkgo/v2 v2.19.0 h1:9Cnnf7UHo57Hy3k6/m5k3dRfGTMXGvxhHFvkDTCTpvA github.com/onsi/ginkgo/v2 v2.19.0/go.mod h1:rlwLi9PilAFJ8jCg9UE1QP6VBpd6/xj3SRC0d6TU0To= github.com/onsi/gomega v1.19.0 h1:4ieX6qQjPP/BfC3mpsAtIGGlxTWPeA3Inl/7DtXw1tw= github.com/onsi/gomega v1.19.0/go.mod h1:LY+I3pBVzYsTBU1AnDwOSxaYi9WoWiqgwooUqq9yPro= +github.com/open-telemetry/opentelemetry-collector-contrib/internal/exp/metrics v0.116.0 h1:Kxk5Ral+Dc6VB9UmTketVjs+rbMZP8JxQ4SXDx4RivQ= +github.com/open-telemetry/opentelemetry-collector-contrib/internal/exp/metrics v0.116.0/go.mod h1:ctT6oQmGmWGGGgUIKyx2fDwqz77N9+04gqKkDyAzKCg= +github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatatest v0.116.0 h1:RlEK9MbxWyBHbLel8EJ1L7DbYVLai9dZL6Ljl2cBgyA= +github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatatest v0.116.0/go.mod h1:AVUEyIjPb+0ARr7mhIkZkdNg3fd0ZcRhzAi53oZhl1Q= +github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatautil v0.116.0 h1:jwnZYRBuPJnsKXE5H6ZvTEm91bXW5VP8+tLewzl54eg= +github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatautil v0.116.0/go.mod h1:NT3Ag+DdnIAZQfD7l7OHwlYqnaAJ19SoPZ0nhD9yx4s= +github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor v0.116.0 h1:ZBmLuipJv7BT9fho/2yAFsS8AtMsCOCe4ON8oqkX3n8= +github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor v0.116.0/go.mod h1:f0GdYWGxUunyRZ088gHnoX78pc/gZc3dQlRtidiGXzg= github.com/opencontainers/go-digest v1.0.0 h1:apOUWs51W5PlhuyGyz9FCeeBIOUDA/6nW8Oi/yOhh5U= github.com/opencontainers/go-digest v1.0.0/go.mod h1:0JzlMkj0TRzQZfJkVvzbP0HBR3IKzErnv2BNG4W4MAM= github.com/opencontainers/image-spec v1.0.2 h1:9yCKha/T5XdGtO0q9Q9a6T5NUCsTn/DrBg0D7ufOcFM= @@ -490,8 +510,36 @@ go.mongodb.org/mongo-driver v1.14.0 h1:P98w8egYRjYe3XDjxhYJagTokP/H6HzlsnojRgZRd go.mongodb.org/mongo-driver v1.14.0/go.mod h1:Vzb0Mk/pa7e6cWw85R4F/endUC3u0U9jGcNU603k65c= go.opentelemetry.io/auto/sdk v1.1.0 h1:cH53jehLUN6UFLY71z+NDOiNJqDdPRaXzTel0sJySYA= go.opentelemetry.io/auto/sdk v1.1.0/go.mod h1:3wSPjt5PWp2RhlCcmmOial7AvC4DQqZb7a7wCow3W8A= +go.opentelemetry.io/collector/component v0.116.0 h1:SQE1YeVfYCN7bw1n4hknUwJE5U/1qJL552sDhAdSlaA= +go.opentelemetry.io/collector/component v0.116.0/go.mod h1:MYgXFZWDTq0uPgF1mkLSFibtpNqksRVAOrmihckOQEs= +go.opentelemetry.io/collector/component/componentstatus v0.116.0 h1:wpgY0H2K9IPBzaNAvavKziK86VZ7TuNFQbS9OC4Z6Cs= +go.opentelemetry.io/collector/component/componentstatus v0.116.0/go.mod h1:ZRlVwHFMGNfcsAywEJqivOn5JzDZkpe3KZVSwMWu4tw= +go.opentelemetry.io/collector/component/componenttest v0.116.0 h1:UIcnx4Rrs/oDRYSAZNHRMUiYs2FBlwgV5Nc0oMYfR6A= +go.opentelemetry.io/collector/component/componenttest v0.116.0/go.mod h1:W40HaKPHdBFMVI7zzHE7dhdWC+CgAnAC9SmWetFBATY= +go.opentelemetry.io/collector/config/configtelemetry v0.116.0 h1:Vl49VCHQwBOeMswDpFwcl2HD8e9y94xlrfII3SR2VeQ= +go.opentelemetry.io/collector/config/configtelemetry v0.116.0/go.mod h1:SlBEwQg0qly75rXZ6W1Ig8jN25KBVBkFIIAUI1GiAAE= +go.opentelemetry.io/collector/confmap v1.22.0 h1:ZKQzRuj5lKu+seKArAAZ1yPRroDPricaIVIREm/jr3w= +go.opentelemetry.io/collector/confmap v1.22.0/go.mod h1:Rrhs+MWoaP6AswZp+ReQ2VO9dfOfcUjdjiSHBsG+nec= +go.opentelemetry.io/collector/consumer v1.22.0 h1:QmfnNizyNZFt0uK3GG/EoT5h6PvZJ0dgVTc5hFEc1l0= +go.opentelemetry.io/collector/consumer v1.22.0/go.mod h1:tiz2khNceFAPokxxfzAuFfIpShBasMT2AL2Sbc7+m0I= +go.opentelemetry.io/collector/consumer/consumertest v0.116.0 h1:pIVR7FtQMNAzfxBUSMEIC2dX5Lfo3O9ZBfx+sAwrrrM= +go.opentelemetry.io/collector/consumer/consumertest v0.116.0/go.mod h1:cV3cNDiPnls5JdhnOJJFVlclrClg9kPs04cXgYP9Gmk= +go.opentelemetry.io/collector/consumer/xconsumer v0.116.0 h1:ZrWvq7HumB0jRYmS2ztZ3hhXRNpUVBWPKMbPhsVGmZM= +go.opentelemetry.io/collector/consumer/xconsumer v0.116.0/go.mod h1:C+VFMk8vLzPun6XK8aMts6h4RaDjmzXHCPaiOxzRQzQ= go.opentelemetry.io/collector/pdata v1.22.0 h1:3yhjL46NLdTMoP8rkkcE9B0pzjf2973crn0KKhX5UrI= go.opentelemetry.io/collector/pdata v1.22.0/go.mod h1:nLLf6uDg8Kn5g3WNZwGyu8+kf77SwOqQvMTb5AXEbEY= +go.opentelemetry.io/collector/pdata/pprofile v0.116.0 h1:iE6lqkO7Hi6lTIIml1RI7yQ55CKqW12R2qHinwF5Zuk= +go.opentelemetry.io/collector/pdata/pprofile v0.116.0/go.mod h1:xQiPpjzIiXRFb+1fPxUy/3ygEZgo0Bu/xmLKOWu8vMQ= +go.opentelemetry.io/collector/pdata/testdata v0.116.0 h1:zmn1zpeX2BvzL6vt2dBF4OuAyFF2ml/OXcqflNgFiP0= +go.opentelemetry.io/collector/pdata/testdata v0.116.0/go.mod h1:ytWzICFN4XTDP6o65B4+Ed52JGdqgk9B8CpLHCeCpMo= +go.opentelemetry.io/collector/pipeline v0.116.0 h1:o8eKEuWEszmRpfShy7ElBoQ3Jo6kCi9ucm3yRgdNb9s= +go.opentelemetry.io/collector/pipeline v0.116.0/go.mod h1:qE3DmoB05AW0C3lmPvdxZqd/H4po84NPzd5MrqgtL74= +go.opentelemetry.io/collector/processor v0.116.0 h1:Kyu4tPzTdWNHtZjcxvI/bGNAgyv8L8Kem2r/Mk4IDAw= +go.opentelemetry.io/collector/processor v0.116.0/go.mod h1:+/Ugy48RAxlZEXmN2cw51W8t5wdHS9No+GAoP+moskk= +go.opentelemetry.io/collector/processor/processortest v0.116.0 h1:+IqNEVEE0E2MsO2g7+Y/9dz35sDuvAXRXrLts9NdXrA= +go.opentelemetry.io/collector/processor/processortest v0.116.0/go.mod h1:DLaQDBxzgeeaUO0ULMn/efos9PmHZkmYCHuxwCsiVHI= +go.opentelemetry.io/collector/processor/xprocessor v0.116.0 h1:iin/UwuWvSLB7ZNfINFUYbZ5lxIi1NjZ2brkyyFdiRA= +go.opentelemetry.io/collector/processor/xprocessor v0.116.0/go.mod h1:cnA43/XpKDbaOmd8buqKp/LGJ2l/OoCqbR//u5DMfn8= go.opentelemetry.io/collector/semconv v0.116.0 h1:63xCZomsKJAWmKGWD3lnORiE3WKW6AO4LjnzcHzGx3Y= go.opentelemetry.io/collector/semconv v0.116.0/go.mod h1:N6XE8Q0JKgBN2fAhkUQtqK9LT7rEGR6+Wu/Rtbal1iI= go.opentelemetry.io/contrib/instrumentation/net/http/httptrace/otelhttptrace v0.58.0 h1:xwH3QJv6zL4u+gkPUu59NeT1Gyw9nScWT8FQpKLUJJI= @@ -510,8 +558,8 @@ go.opentelemetry.io/otel/metric v1.33.0 h1:r+JOocAyeRVXD8lZpjdQjzMadVZp2M4WmQ+5W go.opentelemetry.io/otel/metric v1.33.0/go.mod h1:L9+Fyctbp6HFTddIxClbQkjtubW6O9QS3Ann/M82u6M= go.opentelemetry.io/otel/sdk v1.33.0 h1:iax7M131HuAm9QkZotNHEfstof92xM+N8sr3uHXc2IM= go.opentelemetry.io/otel/sdk v1.33.0/go.mod h1:A1Q5oi7/9XaMlIWzPSxLRWOI8nG3FnzHJNbiENQuihM= -go.opentelemetry.io/otel/sdk/metric v1.31.0 h1:i9hxxLJF/9kkvfHppyLL55aW7iIJz4JjxTeYusH7zMc= -go.opentelemetry.io/otel/sdk/metric v1.31.0/go.mod h1:CRInTMVvNhUKgSAMbKyTMxqOBC0zgyxzW55lZzX43Y8= +go.opentelemetry.io/otel/sdk/metric v1.32.0 h1:rZvFnvmvawYb0alrYkjraqJq0Z4ZUJAiyYCU9snn1CU= +go.opentelemetry.io/otel/sdk/metric v1.32.0/go.mod h1:PWeZlq0zt9YkYAp3gjKZ0eicRYvOh1Gd+X99x6GHpCQ= go.opentelemetry.io/otel/trace v1.33.0 h1:cCJuF7LRjUFso9LPnEAHJDB2pqzp+hbO8eu1qqW2d/s= go.opentelemetry.io/otel/trace v1.33.0/go.mod h1:uIcdVUZMpTAmz0tI1z04GoVSezK37CbGV4fr1f2nBck= go.opentelemetry.io/proto/otlp v1.4.0 h1:TA9WRvW6zMwP+Ssb6fLoUIuirti1gGbP28GcKG1jgeg= @@ -524,6 +572,8 @@ go.uber.org/goleak v1.3.0 h1:2K3zAYmnTNqV73imy9J1T3WC+gmCePx2hEGkimedGto= go.uber.org/goleak v1.3.0/go.mod h1:CoHD4mav9JJNrW/WLlf7HGZPjdw8EucARQHekz1X6bE= go.uber.org/multierr v1.11.0 h1:blXXJkSxSSfBVBlC76pxqeO+LN3aDfLQo+309xJstO0= go.uber.org/multierr v1.11.0/go.mod h1:20+QtiLqy0Nd6FdQB9TLXag12DsQkrbs3htMFfDN80Y= +go.uber.org/zap v1.27.0 h1:aJMhYGrd5QSmlpLMr2MftRKl7t8J8PTZPA732ud/XR8= +go.uber.org/zap v1.27.0/go.mod h1:GB2qFLM7cTU87MWRP2mPIjqfIDnGu+VIO4V/SdhGo2E= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190923035154-9ee001bba392/go.mod h1:/lpIB1dKB+9EgE3H3cr1v9wB50oz8l4C4h62xy7jSTY= diff --git a/storage/remote/write_handler.go b/storage/remote/write_handler.go index 89433ae6f2..02585539c0 100644 --- a/storage/remote/write_handler.go +++ b/storage/remote/write_handler.go @@ -38,6 +38,13 @@ import ( writev2 "github.com/prometheus/prometheus/prompb/io/prometheus/write/v2" "github.com/prometheus/prometheus/storage" otlptranslator "github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite" + + deltatocumulative "github.com/open-telemetry/opentelemetry-collector-contrib/processor/deltatocumulativeprocessor" + "go.opentelemetry.io/collector/component" + "go.opentelemetry.io/collector/consumer" + "go.opentelemetry.io/collector/pdata/pmetric" + "go.opentelemetry.io/collector/processor" + "go.opentelemetry.io/otel/metric/noop" ) type writeHandler struct { @@ -517,56 +524,107 @@ func (h *writeHandler) handleHistogramZeroSample(app storage.Appender, ref stora return ref, err } +type OTLPOptions struct { + // Convert delta samples to their cumulative equivalent by aggregating in-memory + ConvertDelta bool +} + // NewOTLPWriteHandler creates a http.Handler that accepts OTLP write requests and // writes them to the provided appendable. -func NewOTLPWriteHandler(logger *slog.Logger, appendable storage.Appendable, configFunc func() config.Config) http.Handler { - rwHandler := &writeHandler{ - logger: logger, - appendable: appendable, +func NewOTLPWriteHandler(logger *slog.Logger, reg prometheus.Registerer, appendable storage.Appendable, configFunc func() config.Config, opts OTLPOptions) http.Handler { + ex := &rwExporter{ + writeHandler: &writeHandler{ + logger: logger, + appendable: appendable, + }, + config: configFunc, } - return &otlpWriteHandler{ - logger: logger, - rwHandler: rwHandler, - configFunc: configFunc, + wh := &otlpWriteHandler{logger: logger, cumul: ex} + + if opts.ConvertDelta { + fac := deltatocumulative.NewFactory() + set := processor.Settings{TelemetrySettings: component.TelemetrySettings{MeterProvider: noop.NewMeterProvider()}} + d2c, err := fac.CreateMetrics(context.Background(), set, fac.CreateDefaultConfig(), wh.cumul) + if err != nil { + // fac.CreateMetrics directly calls [deltatocumulativeprocessor.createMetricsProcessor], + // which only errors if: + // - cfg.(type) != *Config + // - telemetry.New fails due to bad set.TelemetrySettings + // + // both cannot be the case, as we pass a valid *Config and valid TelemetrySettings. + // as such, we assume this error to never occur. + // if it is, our assumptions are broken in which case a panic seems acceptable. + panic(err) + } + if err := d2c.Start(context.Background(), nil); err != nil { + // deltatocumulative does not error on start. see above for panic reasoning + panic(err) + } + wh.delta = d2c } + + return wh } -type otlpWriteHandler struct { - logger *slog.Logger - rwHandler *writeHandler - configFunc func() config.Config +type rwExporter struct { + *writeHandler + config func() config.Config } -func (h *otlpWriteHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { - req, err := DecodeOTLPWriteRequest(r) - if err != nil { - h.logger.Error("Error decoding remote write request", "err", err.Error()) - http.Error(w, err.Error(), http.StatusBadRequest) - return - } - - otlpCfg := h.configFunc().OTLPConfig +func (rw *rwExporter) ConsumeMetrics(ctx context.Context, md pmetric.Metrics) error { + otlpCfg := rw.config().OTLPConfig converter := otlptranslator.NewPrometheusConverter() - annots, err := converter.FromMetrics(r.Context(), req.Metrics(), otlptranslator.Settings{ + annots, err := converter.FromMetrics(ctx, md, otlptranslator.Settings{ AddMetricSuffixes: true, AllowUTF8: otlpCfg.TranslationStrategy == config.NoUTF8EscapingWithSuffixes, PromoteResourceAttributes: otlpCfg.PromoteResourceAttributes, KeepIdentifyingResourceAttributes: otlpCfg.KeepIdentifyingResourceAttributes, }) if err != nil { - h.logger.Warn("Error translating OTLP metrics to Prometheus write request", "err", err) + rw.logger.Warn("Error translating OTLP metrics to Prometheus write request", "err", err) } ws, _ := annots.AsStrings("", 0, 0) if len(ws) > 0 { - h.logger.Warn("Warnings translating OTLP metrics to Prometheus write request", "warnings", ws) + rw.logger.Warn("Warnings translating OTLP metrics to Prometheus write request", "warnings", ws) } - err = h.rwHandler.write(r.Context(), &prompb.WriteRequest{ + err = rw.write(ctx, &prompb.WriteRequest{ Timeseries: converter.TimeSeries(), Metadata: converter.Metadata(), }) + return err +} + +func (rw *rwExporter) Capabilities() consumer.Capabilities { + return consumer.Capabilities{MutatesData: false} +} + +type otlpWriteHandler struct { + logger *slog.Logger + + cumul consumer.Metrics // only cumulative + delta consumer.Metrics // delta capable +} + +func (h *otlpWriteHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { + req, err := DecodeOTLPWriteRequest(r) + if err != nil { + h.logger.Error("Error decoding OTLP write request", "err", err.Error()) + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + + md := req.Metrics() + // if delta conversion enabled AND delta samples exist, use slower delta capable path + if h.delta != nil && hasDelta(md) { + err = h.delta.ConsumeMetrics(r.Context(), md) + } else { + // deltatocumulative currently holds a sync.Mutex when entering ConsumeMetrics. + // This is slow and not necessary when no delta samples exist anyways + err = h.cumul.ConsumeMetrics(r.Context(), md) + } switch { case err == nil: @@ -583,6 +641,31 @@ func (h *otlpWriteHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { w.WriteHeader(http.StatusOK) } +func hasDelta(md pmetric.Metrics) bool { + for i := range md.ResourceMetrics().Len() { + sms := md.ResourceMetrics().At(i).ScopeMetrics() + for i := range sms.Len() { + ms := sms.At(i).Metrics() + for i := range ms.Len() { + temporality := pmetric.AggregationTemporalityUnspecified + m := ms.At(i) + switch ms.At(i).Type() { + case pmetric.MetricTypeSum: + temporality = m.Sum().AggregationTemporality() + case pmetric.MetricTypeExponentialHistogram: + temporality = m.ExponentialHistogram().AggregationTemporality() + case pmetric.MetricTypeHistogram: + temporality = m.Histogram().AggregationTemporality() + } + if temporality == pmetric.AggregationTemporalityDelta { + return true + } + } + } + } + return false +} + type timeLimitAppender struct { storage.Appender diff --git a/storage/remote/write_test.go b/storage/remote/write_test.go index 83dfffbaef..8125da7f6e 100644 --- a/storage/remote/write_test.go +++ b/storage/remote/write_test.go @@ -15,13 +15,23 @@ package remote import ( "bytes" + "context" "errors" + "fmt" + "log/slog" + "math/rand/v2" "net/http" "net/http/httptest" "net/url" + "os" + "reflect" + "runtime" + "strconv" + "sync" "testing" "time" + "github.com/google/go-cmp/cmp" "github.com/prometheus/client_golang/prometheus" common_config "github.com/prometheus/common/config" "github.com/prometheus/common/model" @@ -31,8 +41,10 @@ import ( "go.opentelemetry.io/collector/pdata/pmetric/pmetricotlp" "github.com/prometheus/prometheus/config" + "github.com/prometheus/prometheus/model/histogram" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/model/relabel" + "github.com/prometheus/prometheus/storage" ) func testRemoteWriteConfig() *config.RemoteWriteConfig { @@ -379,11 +391,11 @@ func TestOTLPWriteHandler(t *testing.T) { req.Header.Set("Content-Type", "application/x-protobuf") appendable := &mockAppendable{} - handler := NewOTLPWriteHandler(nil, appendable, func() config.Config { + handler := NewOTLPWriteHandler(nil, nil, appendable, func() config.Config { return config.Config{ OTLPConfig: config.DefaultOTLPConfig, } - }) + }, OTLPOptions{}) recorder := httptest.NewRecorder() handler.ServeHTTP(recorder, req) @@ -476,3 +488,364 @@ func generateOTLPWriteRequest() pmetricotlp.ExportRequest { return pmetricotlp.NewExportRequestFromMetrics(d) } + +func TestOTLPDelta(t *testing.T) { + log := slog.New(slog.NewTextHandler(os.Stderr, &slog.HandlerOptions{Level: slog.LevelWarn})) + appendable := &mockAppendable{} + cfg := func() config.Config { + return config.Config{OTLPConfig: config.DefaultOTLPConfig} + } + handler := NewOTLPWriteHandler(log, nil, appendable, cfg, OTLPOptions{ConvertDelta: true}) + + md := pmetric.NewMetrics() + ms := md.ResourceMetrics().AppendEmpty().ScopeMetrics().AppendEmpty().Metrics() + + m := ms.AppendEmpty() + m.SetName("some.delta.total") + + sum := m.SetEmptySum() + sum.SetAggregationTemporality(pmetric.AggregationTemporalityDelta) + + ts := time.Date(2000, 1, 2, 3, 4, 0, 0, time.UTC) + for i := range 3 { + dp := sum.DataPoints().AppendEmpty() + dp.SetIntValue(int64(i)) + dp.SetTimestamp(pcommon.NewTimestampFromTime(ts.Add(time.Duration(i) * time.Second))) + } + + proto, err := pmetricotlp.NewExportRequestFromMetrics(md).MarshalProto() + require.NoError(t, err) + + req, err := http.NewRequest("", "", bytes.NewReader(proto)) + require.NoError(t, err) + req.Header.Set("Content-Type", "application/x-protobuf") + + rec := httptest.NewRecorder() + handler.ServeHTTP(rec, req) + require.Equal(t, http.StatusOK, rec.Result().StatusCode) + + ls := labels.FromStrings("__name__", "some_delta_total") + milli := func(sec int) int64 { + return time.Date(2000, 1, 2, 3, 4, sec, 0, time.UTC).UnixMilli() + } + + want := []mockSample{ + {t: milli(0), l: ls, v: 0}, // +0 + {t: milli(1), l: ls, v: 1}, // +1 + {t: milli(2), l: ls, v: 3}, // +2 + } + if diff := cmp.Diff(want, appendable.samples, cmp.Exporter(func(_ reflect.Type) bool { return true })); diff != "" { + t.Fatal(diff) + } +} + +func BenchmarkOTLP(b *testing.B) { + start := time.Date(2000, 1, 2, 3, 4, 5, 0, time.UTC) + + type Type struct { + name string + data func(mode pmetric.AggregationTemporality, dpc, epoch int) []pmetric.Metric + } + types := []Type{{ + name: "sum", + data: func() func(mode pmetric.AggregationTemporality, dpc, epoch int) []pmetric.Metric { + cumul := make(map[int]float64) + return func(mode pmetric.AggregationTemporality, dpc, epoch int) []pmetric.Metric { + m := pmetric.NewMetric() + sum := m.SetEmptySum() + sum.SetAggregationTemporality(mode) + dps := sum.DataPoints() + for id := range dpc { + dp := dps.AppendEmpty() + dp.SetStartTimestamp(pcommon.NewTimestampFromTime(start)) + dp.SetTimestamp(pcommon.NewTimestampFromTime(start.Add(time.Duration(epoch) * time.Minute))) + dp.Attributes().PutStr("id", strconv.Itoa(id)) + v := float64(rand.IntN(100)) / 10 + switch mode { + case pmetric.AggregationTemporalityDelta: + dp.SetDoubleValue(v) + case pmetric.AggregationTemporalityCumulative: + cumul[id] += v + dp.SetDoubleValue(cumul[id]) + } + } + return []pmetric.Metric{m} + } + }(), + }, { + name: "histogram", + data: func() func(mode pmetric.AggregationTemporality, dpc, epoch int) []pmetric.Metric { + bounds := [4]float64{1, 10, 100, 1000} + type state struct { + counts [4]uint64 + count uint64 + sum float64 + } + var cumul []state + return func(mode pmetric.AggregationTemporality, dpc, epoch int) []pmetric.Metric { + if cumul == nil { + cumul = make([]state, dpc) + } + m := pmetric.NewMetric() + hist := m.SetEmptyHistogram() + hist.SetAggregationTemporality(mode) + dps := hist.DataPoints() + for id := range dpc { + dp := dps.AppendEmpty() + dp.SetStartTimestamp(pcommon.NewTimestampFromTime(start)) + dp.SetTimestamp(pcommon.NewTimestampFromTime(start.Add(time.Duration(epoch) * time.Minute))) + dp.Attributes().PutStr("id", strconv.Itoa(id)) + dp.ExplicitBounds().FromRaw(bounds[:]) + + var obs *state + switch mode { + case pmetric.AggregationTemporalityDelta: + obs = new(state) + case pmetric.AggregationTemporalityCumulative: + obs = &cumul[id] + } + + for i := range obs.counts { + v := uint64(rand.IntN(10)) + obs.counts[i] += v + obs.count++ + obs.sum += float64(v) + } + + dp.SetCount(obs.count) + dp.SetSum(obs.sum) + dp.BucketCounts().FromRaw(obs.counts[:]) + } + return []pmetric.Metric{m} + } + }(), + }, { + name: "exponential", + data: func() func(mode pmetric.AggregationTemporality, dpc, epoch int) []pmetric.Metric { + type state struct { + counts [4]uint64 + count uint64 + sum float64 + } + var cumul []state + return func(mode pmetric.AggregationTemporality, dpc, epoch int) []pmetric.Metric { + if cumul == nil { + cumul = make([]state, dpc) + } + m := pmetric.NewMetric() + ex := m.SetEmptyExponentialHistogram() + ex.SetAggregationTemporality(mode) + dps := ex.DataPoints() + for id := range dpc { + dp := dps.AppendEmpty() + dp.SetStartTimestamp(pcommon.NewTimestampFromTime(start)) + dp.SetTimestamp(pcommon.NewTimestampFromTime(start.Add(time.Duration(epoch) * time.Minute))) + dp.Attributes().PutStr("id", strconv.Itoa(id)) + dp.SetScale(2) + + var obs *state + switch mode { + case pmetric.AggregationTemporalityDelta: + obs = new(state) + case pmetric.AggregationTemporalityCumulative: + obs = &cumul[id] + } + + for i := range obs.counts { + v := uint64(rand.IntN(10)) + obs.counts[i] += v + obs.count++ + obs.sum += float64(v) + } + + dp.Positive().BucketCounts().FromRaw(obs.counts[:]) + dp.SetCount(obs.count) + dp.SetSum(obs.sum) + } + + return []pmetric.Metric{m} + } + }(), + }} + + modes := []struct { + name string + data func(func(pmetric.AggregationTemporality, int, int) []pmetric.Metric, int) []pmetric.Metric + }{{ + name: "cumulative", + data: func(data func(pmetric.AggregationTemporality, int, int) []pmetric.Metric, epoch int) []pmetric.Metric { + return data(pmetric.AggregationTemporalityCumulative, 10, epoch) + }, + }, { + name: "delta", + data: func(data func(pmetric.AggregationTemporality, int, int) []pmetric.Metric, epoch int) []pmetric.Metric { + return data(pmetric.AggregationTemporalityDelta, 10, epoch) + }, + }, { + name: "mixed", + data: func(data func(pmetric.AggregationTemporality, int, int) []pmetric.Metric, epoch int) []pmetric.Metric { + cumul := data(pmetric.AggregationTemporalityCumulative, 5, epoch) + delta := data(pmetric.AggregationTemporalityDelta, 5, epoch) + out := append(cumul, delta...) + rand.Shuffle(len(out), func(i, j int) { out[i], out[j] = out[j], out[i] }) + return out + }, + }} + + configs := []struct { + name string + opts OTLPOptions + }{ + {name: "default"}, + {name: "convert", opts: OTLPOptions{ConvertDelta: true}}, + } + + Workers := runtime.GOMAXPROCS(0) + for _, cs := range types { + for _, mode := range modes { + for _, cfg := range configs { + b.Run(fmt.Sprintf("type=%s/temporality=%s/cfg=%s", cs.name, mode.name, cfg.name), func(b *testing.B) { + if !cfg.opts.ConvertDelta && (mode.name == "delta" || mode.name == "mixed") { + b.Skip("not possible") + } + + var total int + + // reqs is a [b.N]*http.Request, divided across the workers. + // deltatocumulative requires timestamps to be strictly in + // order on a per-series basis. to ensure this, each reqs[k] + // contains samples of differently named series, sorted + // strictly in time order + reqs := make([][]*http.Request, Workers) + for n := range b.N { + k := n % Workers + + md := pmetric.NewMetrics() + ms := md.ResourceMetrics().AppendEmpty(). + ScopeMetrics().AppendEmpty(). + Metrics() + + for i, m := range mode.data(cs.data, n) { + m.SetName(fmt.Sprintf("benchmark_%d_%d", k, i)) + m.MoveTo(ms.AppendEmpty()) + } + + total += sampleCount(md) + + ex := pmetricotlp.NewExportRequestFromMetrics(md) + data, err := ex.MarshalProto() + require.NoError(b, err) + + req, err := http.NewRequest("", "", bytes.NewReader(data)) + require.NoError(b, err) + req.Header.Set("Content-Type", "application/x-protobuf") + + reqs[k] = append(reqs[k], req) + } + + log := slog.New(slog.NewTextHandler(os.Stderr, &slog.HandlerOptions{Level: slog.LevelWarn})) + mock := new(mockAppendable) + appendable := syncAppendable{Appendable: mock, lock: new(sync.Mutex)} + cfgfn := func() config.Config { + return config.Config{OTLPConfig: config.DefaultOTLPConfig} + } + handler := NewOTLPWriteHandler(log, nil, appendable, cfgfn, cfg.opts) + + fail := make(chan struct{}) + done := make(chan struct{}) + + b.ResetTimer() + b.ReportAllocs() + + // we use multiple workers to mimic a real-world scenario + // where multiple OTel collectors are sending their + // time-series in parallel. + // this is necessary to exercise potential lock-contention + // in this benchmark + for k := range Workers { + go func() { + rec := httptest.NewRecorder() + for _, req := range reqs[k] { + handler.ServeHTTP(rec, req) + if rec.Result().StatusCode != http.StatusOK { + fail <- struct{}{} + return + } + } + done <- struct{}{} + }() + } + + for range Workers { + select { + case <-fail: + b.FailNow() + case <-done: + } + } + + require.Equal(b, total, len(mock.samples)+len(mock.histograms)) + }) + } + } + } +} + +func sampleCount(md pmetric.Metrics) int { + var total int + rms := md.ResourceMetrics() + for i := range rms.Len() { + sms := rms.At(i).ScopeMetrics() + for i := range sms.Len() { + ms := sms.At(i).Metrics() + for i := range ms.Len() { + m := ms.At(i) + switch m.Type() { + case pmetric.MetricTypeSum: + total += m.Sum().DataPoints().Len() + case pmetric.MetricTypeGauge: + total += m.Gauge().DataPoints().Len() + case pmetric.MetricTypeHistogram: + dps := m.Histogram().DataPoints() + for i := range dps.Len() { + total += dps.At(i).BucketCounts().Len() + total++ // le=+Inf series + total++ // _sum series + total++ // _count series + } + case pmetric.MetricTypeExponentialHistogram: + total += m.ExponentialHistogram().DataPoints().Len() + case pmetric.MetricTypeSummary: + total += m.Summary().DataPoints().Len() + } + } + } + } + return total +} + +type syncAppendable struct { + lock sync.Locker + storage.Appendable +} + +type syncAppender struct { + lock sync.Locker + storage.Appender +} + +func (s syncAppendable) Appender(ctx context.Context) storage.Appender { + return syncAppender{Appender: s.Appendable.Appender(ctx), lock: s.lock} +} + +func (s syncAppender) Append(ref storage.SeriesRef, l labels.Labels, t int64, v float64) (storage.SeriesRef, error) { + s.lock.Lock() + defer s.lock.Unlock() + return s.Appender.Append(ref, l, t, v) +} + +func (s syncAppender) AppendHistogram(ref storage.SeriesRef, l labels.Labels, t int64, h *histogram.Histogram, f *histogram.FloatHistogram) (storage.SeriesRef, error) { + s.lock.Lock() + defer s.lock.Unlock() + return s.Appender.AppendHistogram(ref, l, t, h, f) +} diff --git a/web/api/v1/api.go b/web/api/v1/api.go index ea7d5c5fe4..9821b26ec2 100644 --- a/web/api/v1/api.go +++ b/web/api/v1/api.go @@ -259,7 +259,7 @@ func NewAPI( statsRenderer StatsRenderer, rwEnabled bool, acceptRemoteWriteProtoMsgs []config.RemoteWriteProtoMsg, - otlpEnabled bool, + otlpEnabled, otlpDeltaToCumulative bool, ctZeroIngestionEnabled bool, ) *API { a := &API{ @@ -307,7 +307,7 @@ func NewAPI( a.remoteWriteHandler = remote.NewWriteHandler(logger, registerer, ap, acceptRemoteWriteProtoMsgs, ctZeroIngestionEnabled) } if otlpEnabled { - a.otlpWriteHandler = remote.NewOTLPWriteHandler(logger, ap, configFunc) + a.otlpWriteHandler = remote.NewOTLPWriteHandler(logger, registerer, ap, configFunc, remote.OTLPOptions{ConvertDelta: otlpDeltaToCumulative}) } return a diff --git a/web/api/v1/errors_test.go b/web/api/v1/errors_test.go index 0a5c76b48e..6ff0614c4c 100644 --- a/web/api/v1/errors_test.go +++ b/web/api/v1/errors_test.go @@ -143,6 +143,7 @@ func createPrometheusAPI(t *testing.T, q storage.SampleAndChunkQueryable) *route config.RemoteWriteProtoMsgs{config.RemoteWriteProtoMsgV1, config.RemoteWriteProtoMsgV2}, false, false, + false, ) promRouter := route.New().WithPrefix("/api/v1") diff --git a/web/web.go b/web/web.go index b3de603fb1..246108e5b1 100644 --- a/web/web.go +++ b/web/web.go @@ -289,6 +289,7 @@ type Options struct { RemoteReadBytesInFrame int EnableRemoteWriteReceiver bool EnableOTLPWriteReceiver bool + ConvertOTLPDelta bool IsAgent bool CTZeroIngestionEnabled bool AppName string @@ -387,6 +388,7 @@ func New(logger *slog.Logger, o *Options) *Handler { o.EnableRemoteWriteReceiver, o.AcceptRemoteWriteProtoMsgs, o.EnableOTLPWriteReceiver, + o.ConvertOTLPDelta, o.CTZeroIngestionEnabled, ) From 594b882e8b3aef70b6cc3ff44fc4ccdb8cf94c99 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gy=C3=B6rgy=20Krajcsovits?= Date: Tue, 14 Jan 2025 18:44:00 +0100 Subject: [PATCH 086/110] fix(remotewrite2): do not overwrite help text with unit in enqueue MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit While testing POC for https://github.com/grafana/mimir/issues/9072 I saw no unit or help metadata. Our test env: https://github.com/grafana/mimir/tree/main/development/mimir-monolithic-mode doesn't have units, so that was empty and cleared the help due to this bug. Signed-off-by: György Krajcsovits --- storage/remote/queue_manager.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/storage/remote/queue_manager.go b/storage/remote/queue_manager.go index 4b966059f6..f0ea33fb52 100644 --- a/storage/remote/queue_manager.go +++ b/storage/remote/queue_manager.go @@ -1923,7 +1923,7 @@ func populateV2TimeSeries(symbolTable *writev2.SymbolsTable, batch []timeSeries, if d.metadata != nil { pendingData[nPending].Metadata.Type = writev2.FromMetadataType(d.metadata.Type) pendingData[nPending].Metadata.HelpRef = symbolTable.Symbolize(d.metadata.Help) - pendingData[nPending].Metadata.HelpRef = symbolTable.Symbolize(d.metadata.Unit) + pendingData[nPending].Metadata.UnitRef = symbolTable.Symbolize(d.metadata.Unit) nPendingMetadata++ } From b3a9fa3df91b9663a26f6393b726d19a0c12a37f Mon Sep 17 00:00:00 2001 From: dongjiang Date: Wed, 15 Jan 2025 02:22:22 +0800 Subject: [PATCH 087/110] chore: Upgrade to golangci-lint v1.63.4 (#15799) update golangci-lint and enable more lntiers This updates golangci-lint to v1.63.4 and enables linters `nilnesserr` and `exptostd` Signed-off-by: dongjiang --------- Signed-off-by: dongjiang --- .github/workflows/ci.yml | 2 +- .golangci.yml | 7 +++++-- Makefile.common | 2 +- scripts/golangci-lint.yml | 2 +- 4 files changed, 8 insertions(+), 5 deletions(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index e96f77c114..3bf12b07c4 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -195,7 +195,7 @@ jobs: with: args: --verbose # Make sure to sync this with Makefile.common and scripts/golangci-lint.yml. - version: v1.62.0 + version: v1.63.4 fuzzing: uses: ./.github/workflows/fuzzing.yml if: github.event_name == 'pull_request' diff --git a/.golangci.yml b/.golangci.yml index dfc74139f9..f294a4d76e 100644 --- a/.golangci.yml +++ b/.golangci.yml @@ -5,25 +5,28 @@ output: sort-results: true linters: + # Keep this list sorted alphabetically enable: - depguard - errorlint + - exptostd - gocritic - godot - gofumpt - goimports + - loggercheck - misspell + - nilnesserr - nolintlint - perfsprint - predeclared - revive + - sloglint - testifylint - unconvert - unused - usestdlibvars - whitespace - - loggercheck - - sloglint issues: max-issues-per-linter: 0 diff --git a/Makefile.common b/Makefile.common index fc47bdbb21..d1576bb313 100644 --- a/Makefile.common +++ b/Makefile.common @@ -61,7 +61,7 @@ PROMU_URL := https://github.com/prometheus/promu/releases/download/v$(PROMU_ SKIP_GOLANGCI_LINT := GOLANGCI_LINT := GOLANGCI_LINT_OPTS ?= -GOLANGCI_LINT_VERSION ?= v1.62.0 +GOLANGCI_LINT_VERSION ?= v1.63.4 # golangci-lint only supports linux, darwin and windows platforms on i386/amd64/arm64. # windows isn't included here because of the path separator being different. ifeq ($(GOHOSTOS),$(filter $(GOHOSTOS),linux darwin)) diff --git a/scripts/golangci-lint.yml b/scripts/golangci-lint.yml index 01b943b9b5..0c00c410a4 100644 --- a/scripts/golangci-lint.yml +++ b/scripts/golangci-lint.yml @@ -36,4 +36,4 @@ jobs: uses: golangci/golangci-lint-action@971e284b6050e8a5849b72094c50ab08da042db8 # v6.1.1 with: args: --verbose - version: v1.62.0 + version: v1.63.4 From 5df6ea30420d94eec1f8f247367767053aabcb3c Mon Sep 17 00:00:00 2001 From: Arve Knudsen Date: Wed, 15 Jan 2025 08:45:05 +0100 Subject: [PATCH 088/110] promtool: Support linting of scrape interval (#15719) * PromTool: Support Scrape Interval Lint Checking --------- Signed-off-by: zhaowang Signed-off-by: Arve Knudsen Co-authored-by: zhaowang --- CHANGELOG.md | 2 + cmd/promtool/main.go | 150 ++++++++++++------ cmd/promtool/main_test.go | 61 +++++-- ...s-config.lint.too_long_scrape_interval.yml | 3 + docs/command-line/promtool.md | 9 +- 5 files changed, 167 insertions(+), 58 deletions(-) create mode 100644 cmd/promtool/testdata/prometheus-config.lint.too_long_scrape_interval.yml diff --git a/CHANGELOG.md b/CHANGELOG.md index d0a7ef6611..40be59f724 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,6 +2,8 @@ ## unreleased +* [ENHANCEMENT] promtool: Support linting of scrape interval, through lint option `too-long-scrape-interval`. #15719 + ## 3.1.0 / 2025-01-02 * [SECURITY] upgrade golang.org/x/crypto to address reported CVE-2024-45337. #15691 diff --git a/cmd/promtool/main.go b/cmd/promtool/main.go index 62a1d4f906..54292bbfe5 100644 --- a/cmd/promtool/main.go +++ b/cmd/promtool/main.go @@ -73,14 +73,19 @@ const ( // Exit code 3 is used for "one or more lint issues detected". lintErrExitCode = 3 - lintOptionAll = "all" - lintOptionDuplicateRules = "duplicate-rules" - lintOptionNone = "none" - checkHealth = "/-/healthy" - checkReadiness = "/-/ready" + lintOptionAll = "all" + lintOptionDuplicateRules = "duplicate-rules" + lintOptionTooLongScrapeInterval = "too-long-scrape-interval" + lintOptionNone = "none" + checkHealth = "/-/healthy" + checkReadiness = "/-/ready" ) -var lintOptions = []string{lintOptionAll, lintOptionDuplicateRules, lintOptionNone} +var ( + lintRulesOptions = []string{lintOptionAll, lintOptionDuplicateRules, lintOptionNone} + // Same as lintRulesOptions, but including scrape config linting options as well. + lintConfigOptions = append(append([]string{}, lintRulesOptions...), lintOptionTooLongScrapeInterval) +) func main() { var ( @@ -97,6 +102,10 @@ func main() { app.HelpFlag.Short('h') checkCmd := app.Command("check", "Check the resources for validity.") + checkLookbackDelta := checkCmd.Flag( + "query.lookback-delta", + "The server's maximum query lookback duration.", + ).Default("5m").Duration() experimental := app.Flag("experimental", "Enable experimental commands.").Bool() @@ -113,7 +122,7 @@ func main() { checkConfigSyntaxOnly := checkConfigCmd.Flag("syntax-only", "Only check the config file syntax, ignoring file and content validation referenced in the config").Bool() checkConfigLint := checkConfigCmd.Flag( "lint", - "Linting checks to apply to the rules specified in the config. Available options are: "+strings.Join(lintOptions, ", ")+". Use --lint=none to disable linting", + "Linting checks to apply to the rules/scrape configs specified in the config. Available options are: "+strings.Join(lintConfigOptions, ", ")+". Use --lint=none to disable linting", ).Default(lintOptionDuplicateRules).String() checkConfigLintFatal := checkConfigCmd.Flag( "lint-fatal", @@ -140,7 +149,7 @@ func main() { ).ExistingFiles() checkRulesLint := checkRulesCmd.Flag( "lint", - "Linting checks to apply. Available options are: "+strings.Join(lintOptions, ", ")+". Use --lint=none to disable linting", + "Linting checks to apply. Available options are: "+strings.Join(lintRulesOptions, ", ")+". Use --lint=none to disable linting", ).Default(lintOptionDuplicateRules).String() checkRulesLintFatal := checkRulesCmd.Flag( "lint-fatal", @@ -339,7 +348,7 @@ func main() { os.Exit(CheckSD(*sdConfigFile, *sdJobName, *sdTimeout, prometheus.DefaultRegisterer)) case checkConfigCmd.FullCommand(): - os.Exit(CheckConfig(*agentMode, *checkConfigSyntaxOnly, newLintConfig(*checkConfigLint, *checkConfigLintFatal), *configFiles...)) + os.Exit(CheckConfig(*agentMode, *checkConfigSyntaxOnly, newConfigLintConfig(*checkConfigLint, *checkConfigLintFatal, model.Duration(*checkLookbackDelta)), *configFiles...)) case checkServerHealthCmd.FullCommand(): os.Exit(checkErr(CheckServerStatus(serverURL, checkHealth, httpRoundTripper))) @@ -351,7 +360,7 @@ func main() { os.Exit(CheckWebConfig(*webConfigFiles...)) case checkRulesCmd.FullCommand(): - os.Exit(CheckRules(newLintConfig(*checkRulesLint, *checkRulesLintFatal), *ruleFiles...)) + os.Exit(CheckRules(newRulesLintConfig(*checkRulesLint, *checkRulesLintFatal), *ruleFiles...)) case checkMetricsCmd.FullCommand(): os.Exit(CheckMetrics(*checkMetricsExtended)) @@ -445,15 +454,15 @@ func checkExperimental(f bool) { var errLint = errors.New("lint error") -type lintConfig struct { +type rulesLintConfig struct { all bool duplicateRules bool fatal bool } -func newLintConfig(stringVal string, fatal bool) lintConfig { +func newRulesLintConfig(stringVal string, fatal bool) rulesLintConfig { items := strings.Split(stringVal, ",") - ls := lintConfig{ + ls := rulesLintConfig{ fatal: fatal, } for _, setting := range items { @@ -464,16 +473,57 @@ func newLintConfig(stringVal string, fatal bool) lintConfig { ls.duplicateRules = true case lintOptionNone: default: - fmt.Printf("WARNING: unknown lint option %s\n", setting) + fmt.Printf("WARNING: unknown lint option: %q\n", setting) } } return ls } -func (ls lintConfig) lintDuplicateRules() bool { +func (ls rulesLintConfig) lintDuplicateRules() bool { return ls.all || ls.duplicateRules } +type configLintConfig struct { + rulesLintConfig + + lookbackDelta model.Duration +} + +func newConfigLintConfig(optionsStr string, fatal bool, lookbackDelta model.Duration) configLintConfig { + c := configLintConfig{ + rulesLintConfig: rulesLintConfig{ + fatal: fatal, + }, + } + + lintNone := false + var rulesOptions []string + for _, option := range strings.Split(optionsStr, ",") { + switch option { + case lintOptionAll, lintOptionTooLongScrapeInterval: + c.lookbackDelta = lookbackDelta + if option == lintOptionAll { + rulesOptions = append(rulesOptions, lintOptionAll) + } + case lintOptionNone: + lintNone = true + default: + rulesOptions = append(rulesOptions, option) + } + } + + if lintNone { + c.lookbackDelta = 0 + rulesOptions = nil + } + + if len(rulesOptions) > 0 { + c.rulesLintConfig = newRulesLintConfig(strings.Join(rulesOptions, ","), fatal) + } + + return c +} + // CheckServerStatus - healthy & ready. func CheckServerStatus(serverURL *url.URL, checkEndpoint string, roundTripper http.RoundTripper) error { if serverURL.Scheme == "" { @@ -512,12 +562,12 @@ func CheckServerStatus(serverURL *url.URL, checkEndpoint string, roundTripper ht } // CheckConfig validates configuration files. -func CheckConfig(agentMode, checkSyntaxOnly bool, lintSettings lintConfig, files ...string) int { +func CheckConfig(agentMode, checkSyntaxOnly bool, lintSettings configLintConfig, files ...string) int { failed := false hasErrors := false for _, f := range files { - ruleFiles, err := checkConfig(agentMode, f, checkSyntaxOnly) + ruleFiles, scrapeConfigs, err := checkConfig(agentMode, f, checkSyntaxOnly) if err != nil { fmt.Fprintln(os.Stderr, " FAILED:", err) hasErrors = true @@ -530,12 +580,12 @@ func CheckConfig(agentMode, checkSyntaxOnly bool, lintSettings lintConfig, files } fmt.Println() - rulesFailed, rulesHasErrors := checkRules(ruleFiles, lintSettings) - if rulesFailed { - failed = rulesFailed - } - if rulesHasErrors { - hasErrors = rulesHasErrors + if !checkSyntaxOnly { + scrapeConfigsFailed := lintScrapeConfigs(scrapeConfigs, lintSettings) + failed = failed || scrapeConfigsFailed + rulesFailed, rulesHaveErrors := checkRules(ruleFiles, lintSettings.rulesLintConfig) + failed = failed || rulesFailed + hasErrors = hasErrors || rulesHaveErrors } } if failed && hasErrors { @@ -574,12 +624,12 @@ func checkFileExists(fn string) error { return err } -func checkConfig(agentMode bool, filename string, checkSyntaxOnly bool) ([]string, error) { +func checkConfig(agentMode bool, filename string, checkSyntaxOnly bool) ([]string, []*config.ScrapeConfig, error) { fmt.Println("Checking", filename) cfg, err := config.LoadFile(filename, agentMode, promslog.NewNopLogger()) if err != nil { - return nil, err + return nil, nil, err } var ruleFiles []string @@ -587,15 +637,15 @@ func checkConfig(agentMode bool, filename string, checkSyntaxOnly bool) ([]strin for _, rf := range cfg.RuleFiles { rfs, err := filepath.Glob(rf) if err != nil { - return nil, err + return nil, nil, err } // If an explicit file was given, error if it is not accessible. if !strings.Contains(rf, "*") { if len(rfs) == 0 { - return nil, fmt.Errorf("%q does not point to an existing file", rf) + return nil, nil, fmt.Errorf("%q does not point to an existing file", rf) } if err := checkFileExists(rfs[0]); err != nil { - return nil, fmt.Errorf("error checking rule file %q: %w", rfs[0], err) + return nil, nil, fmt.Errorf("error checking rule file %q: %w", rfs[0], err) } } ruleFiles = append(ruleFiles, rfs...) @@ -609,26 +659,26 @@ func checkConfig(agentMode bool, filename string, checkSyntaxOnly bool) ([]strin var err error scfgs, err = cfg.GetScrapeConfigs() if err != nil { - return nil, fmt.Errorf("error loading scrape configs: %w", err) + return nil, nil, fmt.Errorf("error loading scrape configs: %w", err) } } for _, scfg := range scfgs { if !checkSyntaxOnly && scfg.HTTPClientConfig.Authorization != nil { if err := checkFileExists(scfg.HTTPClientConfig.Authorization.CredentialsFile); err != nil { - return nil, fmt.Errorf("error checking authorization credentials or bearer token file %q: %w", scfg.HTTPClientConfig.Authorization.CredentialsFile, err) + return nil, nil, fmt.Errorf("error checking authorization credentials or bearer token file %q: %w", scfg.HTTPClientConfig.Authorization.CredentialsFile, err) } } if err := checkTLSConfig(scfg.HTTPClientConfig.TLSConfig, checkSyntaxOnly); err != nil { - return nil, err + return nil, nil, err } for _, c := range scfg.ServiceDiscoveryConfigs { switch c := c.(type) { case *kubernetes.SDConfig: if err := checkTLSConfig(c.HTTPClientConfig.TLSConfig, checkSyntaxOnly); err != nil { - return nil, err + return nil, nil, err } case *file.SDConfig: if checkSyntaxOnly { @@ -637,17 +687,17 @@ func checkConfig(agentMode bool, filename string, checkSyntaxOnly bool) ([]strin for _, file := range c.Files { files, err := filepath.Glob(file) if err != nil { - return nil, err + return nil, nil, err } if len(files) != 0 { for _, f := range files { var targetGroups []*targetgroup.Group targetGroups, err = checkSDFile(f) if err != nil { - return nil, fmt.Errorf("checking SD file %q: %w", file, err) + return nil, nil, fmt.Errorf("checking SD file %q: %w", file, err) } if err := checkTargetGroupsForScrapeConfig(targetGroups, scfg); err != nil { - return nil, err + return nil, nil, err } } continue @@ -656,7 +706,7 @@ func checkConfig(agentMode bool, filename string, checkSyntaxOnly bool) ([]strin } case discovery.StaticConfig: if err := checkTargetGroupsForScrapeConfig(c, scfg); err != nil { - return nil, err + return nil, nil, err } } } @@ -673,18 +723,18 @@ func checkConfig(agentMode bool, filename string, checkSyntaxOnly bool) ([]strin for _, file := range c.Files { files, err := filepath.Glob(file) if err != nil { - return nil, err + return nil, nil, err } if len(files) != 0 { for _, f := range files { var targetGroups []*targetgroup.Group targetGroups, err = checkSDFile(f) if err != nil { - return nil, fmt.Errorf("checking SD file %q: %w", file, err) + return nil, nil, fmt.Errorf("checking SD file %q: %w", file, err) } if err := checkTargetGroupsForAlertmanager(targetGroups, amcfg); err != nil { - return nil, err + return nil, nil, err } } continue @@ -693,12 +743,12 @@ func checkConfig(agentMode bool, filename string, checkSyntaxOnly bool) ([]strin } case discovery.StaticConfig: if err := checkTargetGroupsForAlertmanager(c, amcfg); err != nil { - return nil, err + return nil, nil, err } } } } - return ruleFiles, nil + return ruleFiles, scfgs, nil } func checkTLSConfig(tlsConfig promconfig.TLSConfig, checkSyntaxOnly bool) error { @@ -760,7 +810,7 @@ func checkSDFile(filename string) ([]*targetgroup.Group, error) { } // CheckRules validates rule files. -func CheckRules(ls lintConfig, files ...string) int { +func CheckRules(ls rulesLintConfig, files ...string) int { failed := false hasErrors := false if len(files) == 0 { @@ -780,7 +830,7 @@ func CheckRules(ls lintConfig, files ...string) int { } // checkRulesFromStdin validates rule from stdin. -func checkRulesFromStdin(ls lintConfig) (bool, bool) { +func checkRulesFromStdin(ls rulesLintConfig) (bool, bool) { failed := false hasErrors := false fmt.Println("Checking standard input") @@ -818,7 +868,7 @@ func checkRulesFromStdin(ls lintConfig) (bool, bool) { } // checkRules validates rule files. -func checkRules(files []string, ls lintConfig) (bool, bool) { +func checkRules(files []string, ls rulesLintConfig) (bool, bool) { failed := false hasErrors := false for _, f := range files { @@ -852,7 +902,7 @@ func checkRules(files []string, ls lintConfig) (bool, bool) { return failed, hasErrors } -func checkRuleGroups(rgs *rulefmt.RuleGroups, lintSettings lintConfig) (int, []error) { +func checkRuleGroups(rgs *rulefmt.RuleGroups, lintSettings rulesLintConfig) (int, []error) { numRules := 0 for _, rg := range rgs.Groups { numRules += len(rg.Rules) @@ -876,6 +926,16 @@ func checkRuleGroups(rgs *rulefmt.RuleGroups, lintSettings lintConfig) (int, []e return numRules, nil } +func lintScrapeConfigs(scrapeConfigs []*config.ScrapeConfig, lintSettings configLintConfig) bool { + for _, scfg := range scrapeConfigs { + if lintSettings.lookbackDelta > 0 && scfg.ScrapeInterval >= lintSettings.lookbackDelta { + fmt.Fprintf(os.Stderr, " FAILED: too long scrape interval found, data point will be marked as stale - job: %s, interval: %s\n", scfg.JobName, scfg.ScrapeInterval) + return true + } + } + return false +} + type compareRuleType struct { metric string label labels.Labels diff --git a/cmd/promtool/main_test.go b/cmd/promtool/main_test.go index 9a07269188..92e5ff9e67 100644 --- a/cmd/promtool/main_test.go +++ b/cmd/promtool/main_test.go @@ -234,7 +234,7 @@ func TestCheckTargetConfig(t *testing.T) { for _, test := range cases { t.Run(test.name, func(t *testing.T) { t.Parallel() - _, err := checkConfig(false, "testdata/"+test.file, false) + _, _, err := checkConfig(false, "testdata/"+test.file, false) if test.err != "" { require.EqualErrorf(t, err, test.err, "Expected error %q, got %q", test.err, err.Error()) return @@ -319,7 +319,7 @@ func TestCheckConfigSyntax(t *testing.T) { for _, test := range cases { t.Run(test.name, func(t *testing.T) { t.Parallel() - _, err := checkConfig(false, "testdata/"+test.file, test.syntaxOnly) + _, _, err := checkConfig(false, "testdata/"+test.file, test.syntaxOnly) expectedErrMsg := test.err if strings.Contains(runtime.GOOS, "windows") { expectedErrMsg = test.errWindows @@ -355,7 +355,7 @@ func TestAuthorizationConfig(t *testing.T) { for _, test := range cases { t.Run(test.name, func(t *testing.T) { t.Parallel() - _, err := checkConfig(false, "testdata/"+test.file, false) + _, _, err := checkConfig(false, "testdata/"+test.file, false) if test.err != "" { require.ErrorContains(t, err, test.err, "Expected error to contain %q, got %q", test.err, err.Error()) return @@ -508,7 +508,7 @@ func TestCheckRules(t *testing.T) { defer func(v *os.File) { os.Stdin = v }(os.Stdin) os.Stdin = r - exitCode := CheckRules(newLintConfig(lintOptionDuplicateRules, false)) + exitCode := CheckRules(newRulesLintConfig(lintOptionDuplicateRules, false)) require.Equal(t, successExitCode, exitCode, "") }) @@ -530,7 +530,7 @@ func TestCheckRules(t *testing.T) { defer func(v *os.File) { os.Stdin = v }(os.Stdin) os.Stdin = r - exitCode := CheckRules(newLintConfig(lintOptionDuplicateRules, false)) + exitCode := CheckRules(newRulesLintConfig(lintOptionDuplicateRules, false)) require.Equal(t, failureExitCode, exitCode, "") }) @@ -552,7 +552,7 @@ func TestCheckRules(t *testing.T) { defer func(v *os.File) { os.Stdin = v }(os.Stdin) os.Stdin = r - exitCode := CheckRules(newLintConfig(lintOptionDuplicateRules, true)) + exitCode := CheckRules(newRulesLintConfig(lintOptionDuplicateRules, true)) require.Equal(t, lintErrExitCode, exitCode, "") }) } @@ -560,23 +560,66 @@ func TestCheckRules(t *testing.T) { func TestCheckRulesWithRuleFiles(t *testing.T) { t.Run("rules-good", func(t *testing.T) { t.Parallel() - exitCode := CheckRules(newLintConfig(lintOptionDuplicateRules, false), "./testdata/rules.yml") + exitCode := CheckRules(newRulesLintConfig(lintOptionDuplicateRules, false), "./testdata/rules.yml") require.Equal(t, successExitCode, exitCode, "") }) t.Run("rules-bad", func(t *testing.T) { t.Parallel() - exitCode := CheckRules(newLintConfig(lintOptionDuplicateRules, false), "./testdata/rules-bad.yml") + exitCode := CheckRules(newRulesLintConfig(lintOptionDuplicateRules, false), "./testdata/rules-bad.yml") require.Equal(t, failureExitCode, exitCode, "") }) t.Run("rules-lint-fatal", func(t *testing.T) { t.Parallel() - exitCode := CheckRules(newLintConfig(lintOptionDuplicateRules, true), "./testdata/prometheus-rules.lint.yml") + exitCode := CheckRules(newRulesLintConfig(lintOptionDuplicateRules, true), "./testdata/prometheus-rules.lint.yml") require.Equal(t, lintErrExitCode, exitCode, "") }) } +func TestCheckScrapeConfigs(t *testing.T) { + for _, tc := range []struct { + name string + lookbackDelta model.Duration + expectError bool + }{ + { + name: "scrape interval less than lookback delta", + lookbackDelta: model.Duration(11 * time.Minute), + expectError: false, + }, + { + name: "scrape interval greater than lookback delta", + lookbackDelta: model.Duration(5 * time.Minute), + expectError: true, + }, + { + name: "scrape interval same as lookback delta", + lookbackDelta: model.Duration(10 * time.Minute), + expectError: true, + }, + } { + t.Run(tc.name, func(t *testing.T) { + // Non-fatal linting. + code := CheckConfig(false, false, newConfigLintConfig(lintOptionTooLongScrapeInterval, false, tc.lookbackDelta), "./testdata/prometheus-config.lint.too_long_scrape_interval.yml") + require.Equal(t, successExitCode, code, "Non-fatal linting should return success") + // Fatal linting. + code = CheckConfig(false, false, newConfigLintConfig(lintOptionTooLongScrapeInterval, true, tc.lookbackDelta), "./testdata/prometheus-config.lint.too_long_scrape_interval.yml") + if tc.expectError { + require.Equal(t, lintErrExitCode, code, "Fatal linting should return error") + } else { + require.Equal(t, successExitCode, code, "Fatal linting should return success when there are no problems") + } + // Check syntax only, no linting. + code = CheckConfig(false, true, newConfigLintConfig(lintOptionTooLongScrapeInterval, true, tc.lookbackDelta), "./testdata/prometheus-config.lint.too_long_scrape_interval.yml") + require.Equal(t, successExitCode, code, "Fatal linting should return success when checking syntax only") + // Lint option "none" should disable linting. + code = CheckConfig(false, false, newConfigLintConfig(lintOptionNone+","+lintOptionTooLongScrapeInterval, true, tc.lookbackDelta), "./testdata/prometheus-config.lint.too_long_scrape_interval.yml") + require.Equal(t, successExitCode, code, `Fatal linting should return success when lint option "none" is specified`) + }) + } +} + func TestTSDBDumpCommand(t *testing.T) { if testing.Short() { t.Skip("skipping test in short mode.") diff --git a/cmd/promtool/testdata/prometheus-config.lint.too_long_scrape_interval.yml b/cmd/promtool/testdata/prometheus-config.lint.too_long_scrape_interval.yml new file mode 100644 index 0000000000..0c85d13f31 --- /dev/null +++ b/cmd/promtool/testdata/prometheus-config.lint.too_long_scrape_interval.yml @@ -0,0 +1,3 @@ +scrape_configs: + - job_name: too_long_scrape_interval_test + scrape_interval: 10m diff --git a/docs/command-line/promtool.md b/docs/command-line/promtool.md index 5e2a8f6bb1..09800af748 100644 --- a/docs/command-line/promtool.md +++ b/docs/command-line/promtool.md @@ -59,9 +59,10 @@ Check the resources for validity. #### Flags -| Flag | Description | -| --- | --- | -| --extended | Print extended information related to the cardinality of the metrics. | +| Flag | Description | Default | +| --- | --- | --- | +| --query.lookback-delta | The server's maximum query lookback duration. | `5m` | +| --extended | Print extended information related to the cardinality of the metrics. | | @@ -102,7 +103,7 @@ Check if the config files are valid or not. | Flag | Description | Default | | --- | --- | --- | | --syntax-only | Only check the config file syntax, ignoring file and content validation referenced in the config | | -| --lint | Linting checks to apply to the rules specified in the config. Available options are: all, duplicate-rules, none. Use --lint=none to disable linting | `duplicate-rules` | +| --lint | Linting checks to apply to the rules/scrape configs specified in the config. Available options are: all, duplicate-rules, none, too-long-scrape-interval. Use --lint=none to disable linting | `duplicate-rules` | | --lint-fatal | Make lint errors exit with exit code 3. | `false` | | --agent | Check config file for Prometheus in Agent mode. | | From 92218ecb9b5c6e3c54abc909215d1da4cc8c9b75 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Giedrius=20Statkevi=C4=8Dius?= Date: Fri, 20 Dec 2024 14:56:51 +0200 Subject: [PATCH 089/110] promtool: add --ignore-unknown-fields MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Add --ignore-unknown-fields that ignores unknown fields in rule group files. There are lots of tools in the ecosystem that "like" to extend the rule group file structure but they are currently unreadable by promtool if there's anything extra. The purpose of this flag is so that we could use the "vanilla" promtool instead of rolling our own. Some examples of tools/code: https://github.com/grafana/mimir/blob/main/pkg/mimirtool/rules/rwrulefmt/rulefmt.go https://github.com/grafana/cortex-tools/blob/8898eb3cc5355b917634c9da476045df7b0426af/pkg/rules/rules.go#L18-L25 Signed-off-by: Giedrius Statkevičius --- cmd/promtool/main.go | 28 +++++++++------- cmd/promtool/main_test.go | 24 +++++++------- cmd/promtool/rules.go | 2 +- cmd/promtool/testdata/rules_extrafields.yml | 33 +++++++++++++++++++ .../testdata/rules_run_extrafields.yml | 21 ++++++++++++ cmd/promtool/unittest.go | 16 ++++----- cmd/promtool/unittest_test.go | 23 +++++++++---- docs/command-line/promtool.md | 3 ++ model/rulefmt/rulefmt.go | 10 +++--- model/rulefmt/rulefmt_test.go | 8 ++--- rules/manager.go | 12 +++---- rules/manager_test.go | 26 +++++++-------- 12 files changed, 140 insertions(+), 66 deletions(-) create mode 100644 cmd/promtool/testdata/rules_extrafields.yml create mode 100644 cmd/promtool/testdata/rules_run_extrafields.yml diff --git a/cmd/promtool/main.go b/cmd/promtool/main.go index 54292bbfe5..81ba93d2de 100644 --- a/cmd/promtool/main.go +++ b/cmd/promtool/main.go @@ -127,6 +127,7 @@ func main() { checkConfigLintFatal := checkConfigCmd.Flag( "lint-fatal", "Make lint errors exit with exit code 3.").Default("false").Bool() + checkConfigIgnoreUnknownFields := checkConfigCmd.Flag("ignore-unknown-fields", "Ignore unknown fields in the rule groups read by the config files. This is useful when you want to extend rule files with custom metadata. Ensure that those fields are removed before loading them into the Prometheus server as it performs strict checks by default.").Default("false").Bool() checkWebConfigCmd := checkCmd.Command("web-config", "Check if the web config files are valid or not.") webConfigFiles := checkWebConfigCmd.Arg( @@ -154,6 +155,7 @@ func main() { checkRulesLintFatal := checkRulesCmd.Flag( "lint-fatal", "Make lint errors exit with exit code 3.").Default("false").Bool() + checkRulesIgnoreUnknownFields := checkRulesCmd.Flag("ignore-unknown-fields", "Ignore unknown fields in the rule files. This is useful when you want to extend rule files with custom metadata. Ensure that those fields are removed before loading them into the Prometheus server as it performs strict checks by default.").Default("false").Bool() checkMetricsCmd := checkCmd.Command("metrics", checkMetricsUsage) checkMetricsExtended := checkCmd.Flag("extended", "Print extended information related to the cardinality of the metrics.").Bool() @@ -227,6 +229,7 @@ func main() { ).Required().ExistingFiles() testRulesDebug := testRulesCmd.Flag("debug", "Enable unit test debugging.").Default("false").Bool() testRulesDiff := testRulesCmd.Flag("diff", "[Experimental] Print colored differential output between expected & received output.").Default("false").Bool() + testRulesIgnoreUnknownFields := testRulesCmd.Flag("ignore-unknown-fields", "Ignore unknown fields in the test files. This is useful when you want to extend rule files with custom metadata. Ensure that those fields are removed before loading them into the Prometheus server as it performs strict checks by default.").Default("false").Bool() defaultDBPath := "data/" tsdbCmd := app.Command("tsdb", "Run tsdb commands.") @@ -348,7 +351,7 @@ func main() { os.Exit(CheckSD(*sdConfigFile, *sdJobName, *sdTimeout, prometheus.DefaultRegisterer)) case checkConfigCmd.FullCommand(): - os.Exit(CheckConfig(*agentMode, *checkConfigSyntaxOnly, newConfigLintConfig(*checkConfigLint, *checkConfigLintFatal, model.Duration(*checkLookbackDelta)), *configFiles...)) + os.Exit(CheckConfig(*agentMode, *checkConfigSyntaxOnly, newConfigLintConfig(*checkConfigLint, *checkConfigLintFatal, *checkConfigIgnoreUnknownFields, model.Duration(*checkLookbackDelta)), *configFiles...)) case checkServerHealthCmd.FullCommand(): os.Exit(checkErr(CheckServerStatus(serverURL, checkHealth, httpRoundTripper))) @@ -360,7 +363,7 @@ func main() { os.Exit(CheckWebConfig(*webConfigFiles...)) case checkRulesCmd.FullCommand(): - os.Exit(CheckRules(newRulesLintConfig(*checkRulesLint, *checkRulesLintFatal), *ruleFiles...)) + os.Exit(CheckRules(newRulesLintConfig(*checkRulesLint, *checkRulesLintFatal, *checkRulesIgnoreUnknownFields), *ruleFiles...)) case checkMetricsCmd.FullCommand(): os.Exit(CheckMetrics(*checkMetricsExtended)) @@ -402,6 +405,7 @@ func main() { *testRulesRun, *testRulesDiff, *testRulesDebug, + *testRulesIgnoreUnknownFields, *testRulesFiles...), ) @@ -455,15 +459,17 @@ func checkExperimental(f bool) { var errLint = errors.New("lint error") type rulesLintConfig struct { - all bool - duplicateRules bool - fatal bool + all bool + duplicateRules bool + fatal bool + ignoreUnknownFields bool } -func newRulesLintConfig(stringVal string, fatal bool) rulesLintConfig { +func newRulesLintConfig(stringVal string, fatal, ignoreUnknownFields bool) rulesLintConfig { items := strings.Split(stringVal, ",") ls := rulesLintConfig{ - fatal: fatal, + fatal: fatal, + ignoreUnknownFields: ignoreUnknownFields, } for _, setting := range items { switch setting { @@ -489,7 +495,7 @@ type configLintConfig struct { lookbackDelta model.Duration } -func newConfigLintConfig(optionsStr string, fatal bool, lookbackDelta model.Duration) configLintConfig { +func newConfigLintConfig(optionsStr string, fatal, ignoreUnknownFields bool, lookbackDelta model.Duration) configLintConfig { c := configLintConfig{ rulesLintConfig: rulesLintConfig{ fatal: fatal, @@ -518,7 +524,7 @@ func newConfigLintConfig(optionsStr string, fatal bool, lookbackDelta model.Dura } if len(rulesOptions) > 0 { - c.rulesLintConfig = newRulesLintConfig(strings.Join(rulesOptions, ","), fatal) + c.rulesLintConfig = newRulesLintConfig(strings.Join(rulesOptions, ","), fatal, ignoreUnknownFields) } return c @@ -839,7 +845,7 @@ func checkRulesFromStdin(ls rulesLintConfig) (bool, bool) { fmt.Fprintln(os.Stderr, " FAILED:", err) return true, true } - rgs, errs := rulefmt.Parse(data) + rgs, errs := rulefmt.Parse(data, ls.ignoreUnknownFields) if errs != nil { failed = true fmt.Fprintln(os.Stderr, " FAILED:") @@ -873,7 +879,7 @@ func checkRules(files []string, ls rulesLintConfig) (bool, bool) { hasErrors := false for _, f := range files { fmt.Println("Checking", f) - rgs, errs := rulefmt.ParseFile(f) + rgs, errs := rulefmt.ParseFile(f, ls.ignoreUnknownFields) if errs != nil { failed = true fmt.Fprintln(os.Stderr, " FAILED:") diff --git a/cmd/promtool/main_test.go b/cmd/promtool/main_test.go index 92e5ff9e67..48bed9a2df 100644 --- a/cmd/promtool/main_test.go +++ b/cmd/promtool/main_test.go @@ -185,7 +185,7 @@ func TestCheckDuplicates(t *testing.T) { c := test t.Run(c.name, func(t *testing.T) { t.Parallel() - rgs, err := rulefmt.ParseFile(c.ruleFile) + rgs, err := rulefmt.ParseFile(c.ruleFile, false) require.Empty(t, err) dups := checkDuplicates(rgs.Groups) require.Equal(t, c.expectedDups, dups) @@ -194,7 +194,7 @@ func TestCheckDuplicates(t *testing.T) { } func BenchmarkCheckDuplicates(b *testing.B) { - rgs, err := rulefmt.ParseFile("./testdata/rules_large.yml") + rgs, err := rulefmt.ParseFile("./testdata/rules_large.yml", false) require.Empty(b, err) b.ResetTimer() @@ -508,7 +508,7 @@ func TestCheckRules(t *testing.T) { defer func(v *os.File) { os.Stdin = v }(os.Stdin) os.Stdin = r - exitCode := CheckRules(newRulesLintConfig(lintOptionDuplicateRules, false)) + exitCode := CheckRules(newRulesLintConfig(lintOptionDuplicateRules, false, false)) require.Equal(t, successExitCode, exitCode, "") }) @@ -530,7 +530,7 @@ func TestCheckRules(t *testing.T) { defer func(v *os.File) { os.Stdin = v }(os.Stdin) os.Stdin = r - exitCode := CheckRules(newRulesLintConfig(lintOptionDuplicateRules, false)) + exitCode := CheckRules(newRulesLintConfig(lintOptionDuplicateRules, false, false)) require.Equal(t, failureExitCode, exitCode, "") }) @@ -552,7 +552,7 @@ func TestCheckRules(t *testing.T) { defer func(v *os.File) { os.Stdin = v }(os.Stdin) os.Stdin = r - exitCode := CheckRules(newRulesLintConfig(lintOptionDuplicateRules, true)) + exitCode := CheckRules(newRulesLintConfig(lintOptionDuplicateRules, true, false)) require.Equal(t, lintErrExitCode, exitCode, "") }) } @@ -560,19 +560,19 @@ func TestCheckRules(t *testing.T) { func TestCheckRulesWithRuleFiles(t *testing.T) { t.Run("rules-good", func(t *testing.T) { t.Parallel() - exitCode := CheckRules(newRulesLintConfig(lintOptionDuplicateRules, false), "./testdata/rules.yml") + exitCode := CheckRules(newRulesLintConfig(lintOptionDuplicateRules, false, false), "./testdata/rules.yml") require.Equal(t, successExitCode, exitCode, "") }) t.Run("rules-bad", func(t *testing.T) { t.Parallel() - exitCode := CheckRules(newRulesLintConfig(lintOptionDuplicateRules, false), "./testdata/rules-bad.yml") + exitCode := CheckRules(newRulesLintConfig(lintOptionDuplicateRules, false, false), "./testdata/rules-bad.yml") require.Equal(t, failureExitCode, exitCode, "") }) t.Run("rules-lint-fatal", func(t *testing.T) { t.Parallel() - exitCode := CheckRules(newRulesLintConfig(lintOptionDuplicateRules, true), "./testdata/prometheus-rules.lint.yml") + exitCode := CheckRules(newRulesLintConfig(lintOptionDuplicateRules, true, false), "./testdata/prometheus-rules.lint.yml") require.Equal(t, lintErrExitCode, exitCode, "") }) } @@ -601,20 +601,20 @@ func TestCheckScrapeConfigs(t *testing.T) { } { t.Run(tc.name, func(t *testing.T) { // Non-fatal linting. - code := CheckConfig(false, false, newConfigLintConfig(lintOptionTooLongScrapeInterval, false, tc.lookbackDelta), "./testdata/prometheus-config.lint.too_long_scrape_interval.yml") + code := CheckConfig(false, false, newConfigLintConfig(lintOptionTooLongScrapeInterval, false, false, tc.lookbackDelta), "./testdata/prometheus-config.lint.too_long_scrape_interval.yml") require.Equal(t, successExitCode, code, "Non-fatal linting should return success") // Fatal linting. - code = CheckConfig(false, false, newConfigLintConfig(lintOptionTooLongScrapeInterval, true, tc.lookbackDelta), "./testdata/prometheus-config.lint.too_long_scrape_interval.yml") + code = CheckConfig(false, false, newConfigLintConfig(lintOptionTooLongScrapeInterval, true, false, tc.lookbackDelta), "./testdata/prometheus-config.lint.too_long_scrape_interval.yml") if tc.expectError { require.Equal(t, lintErrExitCode, code, "Fatal linting should return error") } else { require.Equal(t, successExitCode, code, "Fatal linting should return success when there are no problems") } // Check syntax only, no linting. - code = CheckConfig(false, true, newConfigLintConfig(lintOptionTooLongScrapeInterval, true, tc.lookbackDelta), "./testdata/prometheus-config.lint.too_long_scrape_interval.yml") + code = CheckConfig(false, true, newConfigLintConfig(lintOptionTooLongScrapeInterval, true, false, tc.lookbackDelta), "./testdata/prometheus-config.lint.too_long_scrape_interval.yml") require.Equal(t, successExitCode, code, "Fatal linting should return success when checking syntax only") // Lint option "none" should disable linting. - code = CheckConfig(false, false, newConfigLintConfig(lintOptionNone+","+lintOptionTooLongScrapeInterval, true, tc.lookbackDelta), "./testdata/prometheus-config.lint.too_long_scrape_interval.yml") + code = CheckConfig(false, false, newConfigLintConfig(lintOptionNone+","+lintOptionTooLongScrapeInterval, true, false, tc.lookbackDelta), "./testdata/prometheus-config.lint.too_long_scrape_interval.yml") require.Equal(t, successExitCode, code, `Fatal linting should return success when lint option "none" is specified`) }) } diff --git a/cmd/promtool/rules.go b/cmd/promtool/rules.go index adb214b812..b2eb18ca8e 100644 --- a/cmd/promtool/rules.go +++ b/cmd/promtool/rules.go @@ -69,7 +69,7 @@ func newRuleImporter(logger *slog.Logger, config ruleImporterConfig, apiClient q // loadGroups parses groups from a list of recording rule files. func (importer *ruleImporter) loadGroups(_ context.Context, filenames []string) (errs []error) { - groups, errs := importer.ruleManager.LoadGroups(importer.config.evalInterval, labels.Labels{}, "", nil, filenames...) + groups, errs := importer.ruleManager.LoadGroups(importer.config.evalInterval, labels.Labels{}, "", nil, false, filenames...) if errs != nil { return errs } diff --git a/cmd/promtool/testdata/rules_extrafields.yml b/cmd/promtool/testdata/rules_extrafields.yml new file mode 100644 index 0000000000..85ef079bb8 --- /dev/null +++ b/cmd/promtool/testdata/rules_extrafields.yml @@ -0,0 +1,33 @@ +# This is the rules file. It has an extra "ownership" +# field in the second group. promtool should ignore this field +# and not return an error with --ignore-unknown-fields. + +groups: + - name: alerts + namespace: "foobar" + rules: + - alert: InstanceDown + expr: up == 0 + for: 5m + labels: + severity: page + annotations: + summary: "Instance {{ $labels.instance }} down" + description: "{{ $labels.instance }} of job {{ $labels.job }} has been down for more than 5 minutes." + - alert: AlwaysFiring + expr: 1 + + - name: rules + ownership: + service: "test" + rules: + - record: job:test:count_over_time1m + expr: sum without(instance) (count_over_time(test[1m])) + + # A recording rule that doesn't depend on input series. + - record: fixed_data + expr: 1 + + # Subquery with default resolution test. + - record: suquery_interval_test + expr: count_over_time(up[5m:]) diff --git a/cmd/promtool/testdata/rules_run_extrafields.yml b/cmd/promtool/testdata/rules_run_extrafields.yml new file mode 100644 index 0000000000..86879fc396 --- /dev/null +++ b/cmd/promtool/testdata/rules_run_extrafields.yml @@ -0,0 +1,21 @@ +# Minimal test case to see that --ignore-unknown-fields +# is working as expected. It should not return an error +# when any extra fields are present in the rules file. +rule_files: + - rules_extrafields.yml + +evaluation_interval: 1m + + +tests: + - name: extra ownership field test + input_series: + - series: test + values: 1 + + promql_expr_test: + - expr: test + eval_time: 0 + exp_samples: + - value: 1 + labels: test diff --git a/cmd/promtool/unittest.go b/cmd/promtool/unittest.go index 78dacdc569..7f494e27aa 100644 --- a/cmd/promtool/unittest.go +++ b/cmd/promtool/unittest.go @@ -46,11 +46,11 @@ import ( // RulesUnitTest does unit testing of rules based on the unit testing files provided. // More info about the file format can be found in the docs. -func RulesUnitTest(queryOpts promqltest.LazyLoaderOpts, runStrings []string, diffFlag, debug bool, files ...string) int { - return RulesUnitTestResult(io.Discard, queryOpts, runStrings, diffFlag, debug, files...) +func RulesUnitTest(queryOpts promqltest.LazyLoaderOpts, runStrings []string, diffFlag, debug, ignoreUnknownFields bool, files ...string) int { + return RulesUnitTestResult(io.Discard, queryOpts, runStrings, diffFlag, debug, ignoreUnknownFields, files...) } -func RulesUnitTestResult(results io.Writer, queryOpts promqltest.LazyLoaderOpts, runStrings []string, diffFlag, debug bool, files ...string) int { +func RulesUnitTestResult(results io.Writer, queryOpts promqltest.LazyLoaderOpts, runStrings []string, diffFlag, debug, ignoreUnknownFields bool, files ...string) int { failed := false junit := &junitxml.JUnitXML{} @@ -60,7 +60,7 @@ func RulesUnitTestResult(results io.Writer, queryOpts promqltest.LazyLoaderOpts, } for _, f := range files { - if errs := ruleUnitTest(f, queryOpts, run, diffFlag, debug, junit.Suite(f)); errs != nil { + if errs := ruleUnitTest(f, queryOpts, run, diffFlag, debug, ignoreUnknownFields, junit.Suite(f)); errs != nil { fmt.Fprintln(os.Stderr, " FAILED:") for _, e := range errs { fmt.Fprintln(os.Stderr, e.Error()) @@ -82,7 +82,7 @@ func RulesUnitTestResult(results io.Writer, queryOpts promqltest.LazyLoaderOpts, return successExitCode } -func ruleUnitTest(filename string, queryOpts promqltest.LazyLoaderOpts, run *regexp.Regexp, diffFlag, debug bool, ts *junitxml.TestSuite) []error { +func ruleUnitTest(filename string, queryOpts promqltest.LazyLoaderOpts, run *regexp.Regexp, diffFlag, debug, ignoreUnknownFields bool, ts *junitxml.TestSuite) []error { b, err := os.ReadFile(filename) if err != nil { ts.Abort(err) @@ -131,7 +131,7 @@ func ruleUnitTest(filename string, queryOpts promqltest.LazyLoaderOpts, run *reg if t.Interval == 0 { t.Interval = unitTestInp.EvaluationInterval } - ers := t.test(testname, evalInterval, groupOrderMap, queryOpts, diffFlag, debug, unitTestInp.RuleFiles...) + ers := t.test(testname, evalInterval, groupOrderMap, queryOpts, diffFlag, debug, ignoreUnknownFields, unitTestInp.RuleFiles...) if ers != nil { for _, e := range ers { tc.Fail(e.Error()) @@ -198,7 +198,7 @@ type testGroup struct { } // test performs the unit tests. -func (tg *testGroup) test(testname string, evalInterval time.Duration, groupOrderMap map[string]int, queryOpts promqltest.LazyLoaderOpts, diffFlag, debug bool, ruleFiles ...string) (outErr []error) { +func (tg *testGroup) test(testname string, evalInterval time.Duration, groupOrderMap map[string]int, queryOpts promqltest.LazyLoaderOpts, diffFlag, debug, ignoreUnknownFields bool, ruleFiles ...string) (outErr []error) { if debug { testStart := time.Now() fmt.Printf("DEBUG: Starting test %s\n", testname) @@ -228,7 +228,7 @@ func (tg *testGroup) test(testname string, evalInterval time.Duration, groupOrde Logger: promslog.NewNopLogger(), } m := rules.NewManager(opts) - groupsMap, ers := m.LoadGroups(time.Duration(tg.Interval), tg.ExternalLabels, tg.ExternalURL, nil, ruleFiles...) + groupsMap, ers := m.LoadGroups(time.Duration(tg.Interval), tg.ExternalLabels, tg.ExternalURL, nil, ignoreUnknownFields, ruleFiles...) if ers != nil { return ers } diff --git a/cmd/promtool/unittest_test.go b/cmd/promtool/unittest_test.go index ec34ad3185..7466b222ca 100644 --- a/cmd/promtool/unittest_test.go +++ b/cmd/promtool/unittest_test.go @@ -143,7 +143,7 @@ func TestRulesUnitTest(t *testing.T) { } t.Run(tt.name, func(t *testing.T) { t.Parallel() - if got := RulesUnitTest(tt.queryOpts, nil, false, false, tt.args.files...); got != tt.want { + if got := RulesUnitTest(tt.queryOpts, nil, false, false, false, tt.args.files...); got != tt.want { t.Errorf("RulesUnitTest() = %v, want %v", got, tt.want) } }) @@ -151,7 +151,7 @@ func TestRulesUnitTest(t *testing.T) { t.Run("Junit xml output ", func(t *testing.T) { t.Parallel() var buf bytes.Buffer - if got := RulesUnitTestResult(&buf, promqltest.LazyLoaderOpts{}, nil, false, false, reuseFiles...); got != 1 { + if got := RulesUnitTestResult(&buf, promqltest.LazyLoaderOpts{}, nil, false, false, false, reuseFiles...); got != 1 { t.Errorf("RulesUnitTestResults() = %v, want 1", got) } var test junitxml.JUnitXML @@ -194,10 +194,11 @@ func TestRulesUnitTestRun(t *testing.T) { files []string } tests := []struct { - name string - args args - queryOpts promqltest.LazyLoaderOpts - want int + name string + args args + queryOpts promqltest.LazyLoaderOpts + want int + ignoreUnknownFields bool }{ { name: "Test all without run arg", @@ -231,11 +232,19 @@ func TestRulesUnitTestRun(t *testing.T) { }, want: 1, }, + { + name: "Test all with extra fields", + args: args{ + files: []string{"./testdata/rules_run_extrafields.yml"}, + }, + ignoreUnknownFields: true, + want: 0, + }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { t.Parallel() - got := RulesUnitTest(tt.queryOpts, tt.args.run, false, false, tt.args.files...) + got := RulesUnitTest(tt.queryOpts, tt.args.run, false, false, tt.ignoreUnknownFields, tt.args.files...) require.Equal(t, tt.want, got) }) } diff --git a/docs/command-line/promtool.md b/docs/command-line/promtool.md index 09800af748..ab675e6345 100644 --- a/docs/command-line/promtool.md +++ b/docs/command-line/promtool.md @@ -105,6 +105,7 @@ Check if the config files are valid or not. | --syntax-only | Only check the config file syntax, ignoring file and content validation referenced in the config | | | --lint | Linting checks to apply to the rules/scrape configs specified in the config. Available options are: all, duplicate-rules, none, too-long-scrape-interval. Use --lint=none to disable linting | `duplicate-rules` | | --lint-fatal | Make lint errors exit with exit code 3. | `false` | +| --ignore-unknown-fields | Ignore unknown fields in the rule groups read by the config files. This is useful when you want to extend rule files with custom metadata. Ensure that those fields are removed before loading them into the Prometheus server as it performs strict checks by default. | `false` | | --agent | Check config file for Prometheus in Agent mode. | | @@ -178,6 +179,7 @@ Check if the rule files are valid or not. | --- | --- | --- | | --lint | Linting checks to apply. Available options are: all, duplicate-rules, none. Use --lint=none to disable linting | `duplicate-rules` | | --lint-fatal | Make lint errors exit with exit code 3. | `false` | +| --ignore-unknown-fields | Ignore unknown fields in the rule files. This is useful when you want to extend rule files with custom metadata. Ensure that those fields are removed before loading them into the Prometheus server as it performs strict checks by default. | `false` | @@ -465,6 +467,7 @@ Unit tests for rules. | --run ... | If set, will only run test groups whose names match the regular expression. Can be specified multiple times. | | | --debug | Enable unit test debugging. | `false` | | --diff | [Experimental] Print colored differential output between expected & received output. | `false` | +| --ignore-unknown-fields | Ignore unknown fields in the test files. This is useful when you want to extend rule files with custom metadata. Ensure that those fields are removed before loading them into the Prometheus server as it performs strict checks by default. | `false` | diff --git a/model/rulefmt/rulefmt.go b/model/rulefmt/rulefmt.go index bb36a21208..76b12f0e5c 100644 --- a/model/rulefmt/rulefmt.go +++ b/model/rulefmt/rulefmt.go @@ -314,7 +314,7 @@ func testTemplateParsing(rl *RuleNode) (errs []error) { } // Parse parses and validates a set of rules. -func Parse(content []byte) (*RuleGroups, []error) { +func Parse(content []byte, ignoreUnknownFields bool) (*RuleGroups, []error) { var ( groups RuleGroups node ruleGroups @@ -322,7 +322,9 @@ func Parse(content []byte) (*RuleGroups, []error) { ) decoder := yaml.NewDecoder(bytes.NewReader(content)) - decoder.KnownFields(true) + if !ignoreUnknownFields { + decoder.KnownFields(true) + } err := decoder.Decode(&groups) // Ignore io.EOF which happens with empty input. if err != nil && !errors.Is(err, io.EOF) { @@ -341,12 +343,12 @@ func Parse(content []byte) (*RuleGroups, []error) { } // ParseFile reads and parses rules from a file. -func ParseFile(file string) (*RuleGroups, []error) { +func ParseFile(file string, ignoreUnknownFields bool) (*RuleGroups, []error) { b, err := os.ReadFile(file) if err != nil { return nil, []error{fmt.Errorf("%s: %w", file, err)} } - rgs, errs := Parse(b) + rgs, errs := Parse(b, ignoreUnknownFields) for i := range errs { errs[i] = fmt.Errorf("%s: %w", file, errs[i]) } diff --git a/model/rulefmt/rulefmt_test.go b/model/rulefmt/rulefmt_test.go index 73ea174594..286e760a41 100644 --- a/model/rulefmt/rulefmt_test.go +++ b/model/rulefmt/rulefmt_test.go @@ -24,7 +24,7 @@ import ( ) func TestParseFileSuccess(t *testing.T) { - _, errs := ParseFile("testdata/test.yaml") + _, errs := ParseFile("testdata/test.yaml", false) require.Empty(t, errs, "unexpected errors parsing file") } @@ -84,7 +84,7 @@ func TestParseFileFailure(t *testing.T) { } for _, c := range table { - _, errs := ParseFile(filepath.Join("testdata", c.filename)) + _, errs := ParseFile(filepath.Join("testdata", c.filename), false) require.NotEmpty(t, errs, "Expected error parsing %s but got none", c.filename) require.ErrorContainsf(t, errs[0], c.errMsg, "Expected error for %s.", c.filename) } @@ -179,7 +179,7 @@ groups: } for _, tst := range tests { - rgs, errs := Parse([]byte(tst.ruleString)) + rgs, errs := Parse([]byte(tst.ruleString), false) require.NotNil(t, rgs, "Rule parsing, rule=\n"+tst.ruleString) passed := (tst.shouldPass && len(errs) == 0) || (!tst.shouldPass && len(errs) > 0) require.True(t, passed, "Rule validation failed, rule=\n"+tst.ruleString) @@ -206,7 +206,7 @@ groups: annotations: summary: "Instance {{ $labels.instance }} up" ` - _, errs := Parse([]byte(group)) + _, errs := Parse([]byte(group), false) require.Len(t, errs, 2, "Expected two errors") var err00 *Error require.ErrorAs(t, errs[0], &err00) diff --git a/rules/manager.go b/rules/manager.go index a3ae716e2b..2b06dee8b5 100644 --- a/rules/manager.go +++ b/rules/manager.go @@ -207,7 +207,7 @@ func (m *Manager) Update(interval time.Duration, files []string, externalLabels default: } - groups, errs := m.LoadGroups(interval, externalLabels, externalURL, groupEvalIterationFunc, files...) + groups, errs := m.LoadGroups(interval, externalLabels, externalURL, groupEvalIterationFunc, false, files...) if errs != nil { for _, e := range errs { @@ -276,7 +276,7 @@ func (m *Manager) Update(interval time.Duration, files []string, externalLabels // GroupLoader is responsible for loading rule groups from arbitrary sources and parsing them. type GroupLoader interface { - Load(identifier string) (*rulefmt.RuleGroups, []error) + Load(identifier string, ignoreUnknownFields bool) (*rulefmt.RuleGroups, []error) Parse(query string) (parser.Expr, error) } @@ -284,22 +284,22 @@ type GroupLoader interface { // and parser.ParseExpr. type FileLoader struct{} -func (FileLoader) Load(identifier string) (*rulefmt.RuleGroups, []error) { - return rulefmt.ParseFile(identifier) +func (FileLoader) Load(identifier string, ignoreUnknownFields bool) (*rulefmt.RuleGroups, []error) { + return rulefmt.ParseFile(identifier, ignoreUnknownFields) } func (FileLoader) Parse(query string) (parser.Expr, error) { return parser.ParseExpr(query) } // LoadGroups reads groups from a list of files. func (m *Manager) LoadGroups( - interval time.Duration, externalLabels labels.Labels, externalURL string, groupEvalIterationFunc GroupEvalIterationFunc, filenames ...string, + interval time.Duration, externalLabels labels.Labels, externalURL string, groupEvalIterationFunc GroupEvalIterationFunc, ignoreUnknownFields bool, filenames ...string, ) (map[string]*Group, []error) { groups := make(map[string]*Group) shouldRestore := !m.restored for _, fn := range filenames { - rgs, errs := m.opts.GroupLoader.Load(fn) + rgs, errs := m.opts.GroupLoader.Load(fn, ignoreUnknownFields) if errs != nil { return nil, errs } diff --git a/rules/manager_test.go b/rules/manager_test.go index 45da7a44b0..d7c767a3cd 100644 --- a/rules/manager_test.go +++ b/rules/manager_test.go @@ -808,7 +808,7 @@ func TestUpdate(t *testing.T) { } // Groups will be recreated if updated. - rgs, errs := rulefmt.ParseFile("fixtures/rules.yaml") + rgs, errs := rulefmt.ParseFile("fixtures/rules.yaml", false) require.Empty(t, errs, "file parsing failures") tmpFile, err := os.CreateTemp("", "rules.test.*.yaml") @@ -1532,7 +1532,7 @@ func TestManager_LoadGroups_ShouldCheckWhetherEachRuleHasDependentsAndDependenci }) t.Run("load a mix of dependent and independent rules", func(t *testing.T) { - groups, errs := ruleManager.LoadGroups(time.Second, labels.EmptyLabels(), "", nil, []string{"fixtures/rules_multiple.yaml"}...) + groups, errs := ruleManager.LoadGroups(time.Second, labels.EmptyLabels(), "", nil, false, []string{"fixtures/rules_multiple.yaml"}...) require.Empty(t, errs) require.Len(t, groups, 1) @@ -1567,7 +1567,7 @@ func TestManager_LoadGroups_ShouldCheckWhetherEachRuleHasDependentsAndDependenci }) t.Run("load only independent rules", func(t *testing.T) { - groups, errs := ruleManager.LoadGroups(time.Second, labels.EmptyLabels(), "", nil, []string{"fixtures/rules_multiple_independent.yaml"}...) + groups, errs := ruleManager.LoadGroups(time.Second, labels.EmptyLabels(), "", nil, false, []string{"fixtures/rules_multiple_independent.yaml"}...) require.Empty(t, errs) require.Len(t, groups, 1) @@ -1975,7 +1975,7 @@ func TestAsyncRuleEvaluation(t *testing.T) { t.Cleanup(cancel) ruleManager := NewManager(optsFactory(storage, &maxInflight, &inflightQueries, 0)) - groups, errs := ruleManager.LoadGroups(time.Second, labels.EmptyLabels(), "", nil, []string{"fixtures/rules_multiple.yaml"}...) + groups, errs := ruleManager.LoadGroups(time.Second, labels.EmptyLabels(), "", nil, false, []string{"fixtures/rules_multiple.yaml"}...) require.Empty(t, errs) require.Len(t, groups, 1) @@ -2021,7 +2021,7 @@ func TestAsyncRuleEvaluation(t *testing.T) { opts.RuleConcurrencyController = nil ruleManager := NewManager(opts) - groups, errs := ruleManager.LoadGroups(time.Second, labels.EmptyLabels(), "", nil, []string{"fixtures/rules_multiple.yaml"}...) + groups, errs := ruleManager.LoadGroups(time.Second, labels.EmptyLabels(), "", nil, false, []string{"fixtures/rules_multiple.yaml"}...) require.Empty(t, errs) require.Len(t, groups, 1) @@ -2059,7 +2059,7 @@ func TestAsyncRuleEvaluation(t *testing.T) { opts.RuleConcurrencyController = nil ruleManager := NewManager(opts) - groups, errs := ruleManager.LoadGroups(time.Second, labels.EmptyLabels(), "", nil, []string{"fixtures/rules_multiple_independent.yaml"}...) + groups, errs := ruleManager.LoadGroups(time.Second, labels.EmptyLabels(), "", nil, false, []string{"fixtures/rules_multiple_independent.yaml"}...) require.Empty(t, errs) require.Len(t, groups, 1) @@ -2103,7 +2103,7 @@ func TestAsyncRuleEvaluation(t *testing.T) { opts.RuleConcurrencyController = nil ruleManager := NewManager(opts) - groups, errs := ruleManager.LoadGroups(time.Second, labels.EmptyLabels(), "", nil, []string{"fixtures/rules_multiple_independent.yaml"}...) + groups, errs := ruleManager.LoadGroups(time.Second, labels.EmptyLabels(), "", nil, false, []string{"fixtures/rules_multiple_independent.yaml"}...) require.Empty(t, errs) require.Len(t, groups, 1) @@ -2150,7 +2150,7 @@ func TestAsyncRuleEvaluation(t *testing.T) { opts.RuleConcurrencyController = nil ruleManager := NewManager(opts) - groups, errs := ruleManager.LoadGroups(time.Second, labels.EmptyLabels(), "", nil, []string{"fixtures/rules_indeterminates.yaml"}...) + groups, errs := ruleManager.LoadGroups(time.Second, labels.EmptyLabels(), "", nil, false, []string{"fixtures/rules_indeterminates.yaml"}...) require.Empty(t, errs) require.Len(t, groups, 1) @@ -2189,7 +2189,7 @@ func TestAsyncRuleEvaluation(t *testing.T) { opts.RuleConcurrencyController = nil ruleManager := NewManager(opts) - groups, errs := ruleManager.LoadGroups(time.Second, labels.EmptyLabels(), "", nil, []string{"fixtures/rules_multiple_dependents_on_base.yaml"}...) + groups, errs := ruleManager.LoadGroups(time.Second, labels.EmptyLabels(), "", nil, false, []string{"fixtures/rules_multiple_dependents_on_base.yaml"}...) require.Empty(t, errs) require.Len(t, groups, 1) var group *Group @@ -2235,7 +2235,7 @@ func TestAsyncRuleEvaluation(t *testing.T) { opts.RuleConcurrencyController = nil ruleManager := NewManager(opts) - groups, errs := ruleManager.LoadGroups(time.Second, labels.EmptyLabels(), "", nil, []string{"fixtures/rules_chain.yaml"}...) + groups, errs := ruleManager.LoadGroups(time.Second, labels.EmptyLabels(), "", nil, false, []string{"fixtures/rules_chain.yaml"}...) require.Empty(t, errs) require.Len(t, groups, 1) var group *Group @@ -2279,7 +2279,7 @@ func TestBoundedRuleEvalConcurrency(t *testing.T) { ruleManager := NewManager(optsFactory(storage, &maxInflight, &inflightQueries, maxConcurrency)) - groups, errs := ruleManager.LoadGroups(time.Second, labels.EmptyLabels(), "", nil, files...) + groups, errs := ruleManager.LoadGroups(time.Second, labels.EmptyLabels(), "", nil, false, files...) require.Empty(t, errs) require.Len(t, groups, groupCount) @@ -2521,7 +2521,7 @@ func TestRuleDependencyController_AnalyseRules(t *testing.T) { 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, tc.ruleFile) + groups, errs := ruleManager.LoadGroups(time.Second, labels.EmptyLabels(), "", nil, false, tc.ruleFile) require.Empty(t, errs) require.Len(t, groups, 1) @@ -2550,7 +2550,7 @@ func BenchmarkRuleDependencyController_AnalyseRules(b *testing.B) { 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") + groups, errs := ruleManager.LoadGroups(time.Second, labels.EmptyLabels(), "", nil, false, "fixtures/rules_multiple.yaml") require.Empty(b, errs) require.Len(b, groups, 1) From 482a7fd89091fd23316c2eccda5c2bdb66bed95b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gy=C3=B6rgy=20Krajcsovits?= Date: Wed, 15 Jan 2025 12:42:52 +0100 Subject: [PATCH 090/110] fix(remotewrite2): do not send uninitialized garbage if there's no metadata MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Found during testing for https://github.com/grafana/mimir/issues/9072 Debug printout showed: KRAJO: seriesName=cortex_request_duration_seconds_bucket, metricFamily=cortex_request_duration_seconds_bucket, type=GAUGE, help=cortex_bucket_index_load_duration_seconds_sum, unit= which is nonsense. I can imagine more cases where this is the case and makes actual sense. Some targets might miss metadata and if there's a pipeline that loses it. Signed-off-by: György Krajcsovits --- storage/remote/queue_manager.go | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/storage/remote/queue_manager.go b/storage/remote/queue_manager.go index f0ea33fb52..731a0f405f 100644 --- a/storage/remote/queue_manager.go +++ b/storage/remote/queue_manager.go @@ -1919,12 +1919,17 @@ func populateV2TimeSeries(symbolTable *writev2.SymbolsTable, batch []timeSeries, var nPendingSamples, nPendingExemplars, nPendingHistograms, nPendingMetadata int for nPending, d := range batch { pendingData[nPending].Samples = pendingData[nPending].Samples[:0] - // todo: should we also safeguard against empty metadata here? if d.metadata != nil { pendingData[nPending].Metadata.Type = writev2.FromMetadataType(d.metadata.Type) pendingData[nPending].Metadata.HelpRef = symbolTable.Symbolize(d.metadata.Help) pendingData[nPending].Metadata.UnitRef = symbolTable.Symbolize(d.metadata.Unit) nPendingMetadata++ + } else { + // Safeguard against sending garbage in case of not having metadata + // for whatever reason. + pendingData[nPending].Metadata.Type = writev2.Metadata_METRIC_TYPE_UNSPECIFIED + pendingData[nPending].Metadata.HelpRef = symbolTable.Symbolize("") + pendingData[nPending].Metadata.UnitRef = symbolTable.Symbolize("") } if sendExemplars { From 9385f31147897efb147ee05f88a9aca7eecd527c Mon Sep 17 00:00:00 2001 From: bwplotka Date: Wed, 15 Jan 2025 11:33:42 +0000 Subject: [PATCH 091/110] scrape: Fix metadata in WAL not working for histograms and summaries. The was a bug (due to confusion?) on the local metadata cache that is cached by metric family not the series metric name. The fix is to NOT use that local cache at all (it's still needed for current metadata API implementation, added TODO on how we can get rid of it). I went ahead and also rename Metric field in metadata structs to MetricFamily to make clear it's not always __name__. Signed-off-by: bwplotka --- scrape/helpers_test.go | 27 ++- scrape/scrape.go | 207 ++++++++++++------- scrape/scrape_test.go | 129 +++++++++++- scrape/target.go | 16 +- storage/remote/metadata_watcher_test.go | 28 +-- storage/remote/queue_manager.go | 2 +- storage/remote/queue_manager_test.go | 10 +- web/api/v1/api.go | 28 +-- web/api/v1/api_test.go | 252 ++++++++++++------------ 9 files changed, 442 insertions(+), 257 deletions(-) diff --git a/scrape/helpers_test.go b/scrape/helpers_test.go index 7bc9e3f7d4..2719a467bc 100644 --- a/scrape/helpers_test.go +++ b/scrape/helpers_test.go @@ -90,6 +90,27 @@ type histogramSample struct { fh *histogram.FloatHistogram } +type metadataEntry struct { + m metadata.Metadata + metric labels.Labels +} + +func metadataEntryEqual(a, b metadataEntry) bool { + if !labels.Equal(a.metric, b.metric) { + return false + } + if a.m.Type != b.m.Type { + return false + } + if a.m.Unit != b.m.Unit { + return false + } + if a.m.Help != b.m.Help { + return false + } + return true +} + type collectResultAppendable struct { *collectResultAppender } @@ -112,8 +133,8 @@ type collectResultAppender struct { rolledbackHistograms []histogramSample resultExemplars []exemplar.Exemplar pendingExemplars []exemplar.Exemplar - resultMetadata []metadata.Metadata - pendingMetadata []metadata.Metadata + resultMetadata []metadataEntry + pendingMetadata []metadataEntry } func (a *collectResultAppender) SetOptions(opts *storage.AppendOptions) {} @@ -173,7 +194,7 @@ func (a *collectResultAppender) AppendHistogramCTZeroSample(ref storage.SeriesRe func (a *collectResultAppender) UpdateMetadata(ref storage.SeriesRef, l labels.Labels, m metadata.Metadata) (storage.SeriesRef, error) { a.mtx.Lock() defer a.mtx.Unlock() - a.pendingMetadata = append(a.pendingMetadata, m) + a.pendingMetadata = append(a.pendingMetadata, metadataEntry{metric: l, m: m}) if ref == 0 { ref = storage.SeriesRef(rand.Uint64()) } diff --git a/scrape/scrape.go b/scrape/scrape.go index 2da07d719e..1f41f50689 100644 --- a/scrape/scrape.go +++ b/scrape/scrape.go @@ -29,6 +29,7 @@ import ( "strings" "sync" "time" + "unsafe" "github.com/klauspost/compress/gzip" config_util "github.com/prometheus/common/config" @@ -931,6 +932,7 @@ type scrapeLoop struct { // scrapeCache tracks mappings of exposed metric strings to label sets and // storage references. Additionally, it tracks staleness of series between // scrapes. +// Cache is meant to be used per a single target. type scrapeCache struct { iter uint64 // Current scrape iteration. @@ -951,8 +953,10 @@ type scrapeCache struct { seriesCur map[uint64]labels.Labels seriesPrev map[uint64]labels.Labels - metaMtx sync.Mutex - metadata map[string]*metaEntry + // TODO(bwplotka): Consider moving Metadata API to use WAL instead of scrape loop to + // avoid locking (using metadata API can block scraping). + metaMtx sync.Mutex // Mutex is needed due to api touching it when metadata is queried. + metadata map[string]*metaEntry // metadata by metric family name. metrics *scrapeMetrics } @@ -1078,73 +1082,79 @@ func (c *scrapeCache) forEachStale(f func(labels.Labels) bool) { } } -func (c *scrapeCache) setType(metric []byte, t model.MetricType) { - c.metaMtx.Lock() +func yoloString(b []byte) string { + return unsafe.String(unsafe.SliceData(b), len(b)) +} - e, ok := c.metadata[string(metric)] +func (c *scrapeCache) setType(mfName []byte, t model.MetricType) ([]byte, *metaEntry) { + c.metaMtx.Lock() + defer c.metaMtx.Unlock() + + e, ok := c.metadata[yoloString(mfName)] if !ok { e = &metaEntry{Metadata: metadata.Metadata{Type: model.MetricTypeUnknown}} - c.metadata[string(metric)] = e + c.metadata[string(mfName)] = e } if e.Type != t { e.Type = t e.lastIterChange = c.iter } e.lastIter = c.iter - - c.metaMtx.Unlock() + return mfName, e } -func (c *scrapeCache) setHelp(metric, help []byte) { +func (c *scrapeCache) setHelp(mfName, help []byte) ([]byte, *metaEntry) { c.metaMtx.Lock() + defer c.metaMtx.Unlock() - e, ok := c.metadata[string(metric)] + e, ok := c.metadata[yoloString(mfName)] if !ok { e = &metaEntry{Metadata: metadata.Metadata{Type: model.MetricTypeUnknown}} - c.metadata[string(metric)] = e + c.metadata[string(mfName)] = e } if e.Help != string(help) { e.Help = string(help) e.lastIterChange = c.iter } e.lastIter = c.iter - - c.metaMtx.Unlock() + return mfName, e } -func (c *scrapeCache) setUnit(metric, unit []byte) { +func (c *scrapeCache) setUnit(mfName, unit []byte) ([]byte, *metaEntry) { c.metaMtx.Lock() + defer c.metaMtx.Unlock() - e, ok := c.metadata[string(metric)] + e, ok := c.metadata[yoloString(mfName)] if !ok { e = &metaEntry{Metadata: metadata.Metadata{Type: model.MetricTypeUnknown}} - c.metadata[string(metric)] = e + c.metadata[string(mfName)] = e } if e.Unit != string(unit) { e.Unit = string(unit) e.lastIterChange = c.iter } e.lastIter = c.iter - - c.metaMtx.Unlock() + return mfName, e } -func (c *scrapeCache) GetMetadata(metric string) (MetricMetadata, bool) { +// GetMetadata returns metadata given the metric family name. +func (c *scrapeCache) GetMetadata(mfName string) (MetricMetadata, bool) { c.metaMtx.Lock() defer c.metaMtx.Unlock() - m, ok := c.metadata[metric] + m, ok := c.metadata[mfName] if !ok { return MetricMetadata{}, false } return MetricMetadata{ - Metric: metric, - Type: m.Type, - Help: m.Help, - Unit: m.Unit, + MetricFamily: mfName, + Type: m.Type, + Help: m.Help, + Unit: m.Unit, }, true } +// ListMetadata lists metadata. func (c *scrapeCache) ListMetadata() []MetricMetadata { c.metaMtx.Lock() defer c.metaMtx.Unlock() @@ -1153,16 +1163,16 @@ func (c *scrapeCache) ListMetadata() []MetricMetadata { for m, e := range c.metadata { res = append(res, MetricMetadata{ - Metric: m, - Type: e.Type, - Help: e.Help, - Unit: e.Unit, + MetricFamily: m, + Type: e.Type, + Help: e.Help, + Unit: e.Unit, }) } return res } -// MetadataSize returns the size of the metadata cache. +// SizeMetadata returns the size of the metadata cache. func (c *scrapeCache) SizeMetadata() (s int) { c.metaMtx.Lock() defer c.metaMtx.Unlock() @@ -1173,7 +1183,7 @@ func (c *scrapeCache) SizeMetadata() (s int) { return s } -// MetadataLen returns the number of metadata entries in the cache. +// LengthMetadata returns the number of metadata entries in the cache. func (c *scrapeCache) LengthMetadata() int { c.metaMtx.Lock() defer c.metaMtx.Unlock() @@ -1607,39 +1617,17 @@ func (sl *scrapeLoop) append(app storage.Appender, b []byte, contentType string, ) } var ( - appErrs = appendErrors{} - sampleLimitErr error - bucketLimitErr error - lset labels.Labels // escapes to heap so hoisted out of loop - e exemplar.Exemplar // escapes to heap so hoisted out of loop - meta metadata.Metadata - metadataChanged bool + appErrs = appendErrors{} + sampleLimitErr error + bucketLimitErr error + lset labels.Labels // escapes to heap so hoisted out of loop + e exemplar.Exemplar // escapes to heap so hoisted out of loop + lastMeta *metaEntry + lastMFName []byte ) exemplars := make([]exemplar.Exemplar, 0, 1) - // updateMetadata updates the current iteration's metadata object and the - // metadataChanged value if we have metadata in the scrape cache AND the - // labelset is for a new series or the metadata for this series has just - // changed. It returns a boolean based on whether the metadata was updated. - updateMetadata := func(lset labels.Labels, isNewSeries bool) bool { - if !sl.appendMetadataToWAL { - return false - } - - sl.cache.metaMtx.Lock() - defer sl.cache.metaMtx.Unlock() - metaEntry, metaOk := sl.cache.metadata[lset.Get(labels.MetricName)] - if metaOk && (isNewSeries || metaEntry.lastIterChange == sl.cache.iter) { - metadataChanged = true - meta.Type = metaEntry.Type - meta.Unit = metaEntry.Unit - meta.Help = metaEntry.Help - return true - } - return false - } - // Take an appender with limits. app = appender(app, sl.sampleLimit, sl.bucketLimit, sl.maxSchema) @@ -1669,14 +1657,18 @@ loop: break } switch et { + // TODO(bwplotka): Consider changing parser to give metadata at once instead of type, help and unit in separation, ideally on `Series()/Histogram() + // otherwise we can expose metadata without series on metadata API. case textparse.EntryType: - sl.cache.setType(p.Type()) + // TODO(bwplotka): Build meta entry directly instead of locking and updating the map. This will + // allow to properly update metadata when e.g unit was added, then removed; + lastMFName, lastMeta = sl.cache.setType(p.Type()) continue case textparse.EntryHelp: - sl.cache.setHelp(p.Help()) + lastMFName, lastMeta = sl.cache.setHelp(p.Help()) continue case textparse.EntryUnit: - sl.cache.setUnit(p.Unit()) + lastMFName, lastMeta = sl.cache.setUnit(p.Unit()) continue case textparse.EntryComment: continue @@ -1699,10 +1691,6 @@ loop: t = *parsedTimestamp } - // Zero metadata out for current iteration until it's resolved. - meta = metadata.Metadata{} - metadataChanged = false - if sl.cache.getDropped(met) { continue } @@ -1716,9 +1704,6 @@ loop: ref = ce.ref lset = ce.lset hash = ce.hash - - // Update metadata only if it changed in the current iteration. - updateMetadata(lset, false) } else { p.Metric(&lset) hash = lset.Hash() @@ -1747,9 +1732,6 @@ loop: sl.metrics.targetScrapePoolExceededLabelLimits.Inc() break loop } - - // Append metadata for new series if they were present. - updateMetadata(lset, true) } if seriesAlreadyScraped && parsedTimestamp == nil { @@ -1857,10 +1839,18 @@ loop: sl.metrics.targetScrapeExemplarOutOfOrder.Add(float64(outOfOrderExemplars)) } - if sl.appendMetadataToWAL && metadataChanged { - if _, merr := app.UpdateMetadata(ref, lset, meta); merr != nil { - // No need to fail the scrape on errors appending metadata. - sl.l.Debug("Error when appending metadata in scrape loop", "ref", fmt.Sprintf("%d", ref), "metadata", fmt.Sprintf("%+v", meta), "err", merr) + if sl.appendMetadataToWAL && lastMeta != nil { + // Is it new series OR did metadata change for this family? + if !ok || lastMeta.lastIterChange == sl.cache.iter { + // In majority cases we can trust that the current series/histogram is matching the lastMeta and lastMFName. + // However, optional TYPE etc metadata and broken OM text can break this, detect those cases here. + // TODO(bwplotka): Consider moving this to parser as many parser users end up doing this (e.g. CT and NHCB parsing). + if isSeriesPartOfFamily(lset.Get(labels.MetricName), lastMFName, lastMeta.Type) { + if _, merr := app.UpdateMetadata(ref, lset, lastMeta.Metadata); merr != nil { + // No need to fail the scrape on errors appending metadata. + sl.l.Debug("Error when appending metadata in scrape loop", "ref", fmt.Sprintf("%d", ref), "metadata", fmt.Sprintf("%+v", lastMeta.Metadata), "err", merr) + } + } } } } @@ -1896,6 +1886,71 @@ loop: return } +func isSeriesPartOfFamily(mName string, mfName []byte, typ model.MetricType) bool { + mfNameStr := yoloString(mfName) + if !strings.HasPrefix(mName, mfNameStr) { // Fast path. + return false + } + + var ( + gotMFName string + ok bool + ) + switch typ { + case model.MetricTypeCounter: + // Prometheus allows _total, cut it from mf name to support this case. + mfNameStr, _ = strings.CutSuffix(mfNameStr, "_total") + + gotMFName, ok = strings.CutSuffix(mName, "_total") + if !ok { + gotMFName = mName + } + case model.MetricTypeHistogram: + gotMFName, ok = strings.CutSuffix(mName, "_bucket") + if !ok { + gotMFName, ok = strings.CutSuffix(mName, "_sum") + if !ok { + gotMFName, ok = strings.CutSuffix(mName, "_count") + if !ok { + gotMFName = mName + } + } + } + case model.MetricTypeGaugeHistogram: + gotMFName, ok = strings.CutSuffix(mName, "_bucket") + if !ok { + gotMFName, ok = strings.CutSuffix(mName, "_gsum") + if !ok { + gotMFName, ok = strings.CutSuffix(mName, "_gcount") + if !ok { + gotMFName = mName + } + } + } + case model.MetricTypeSummary: + gotMFName, ok = strings.CutSuffix(mName, "_sum") + if !ok { + gotMFName, ok = strings.CutSuffix(mName, "_count") + if !ok { + gotMFName = mName + } + } + case model.MetricTypeInfo: + // Technically prometheus text does not support info type, but we might + // accidentally allow info type in prom parse, so support metric family names + // with the _info explicitly too. + mfNameStr, _ = strings.CutSuffix(mfNameStr, "_info") + + gotMFName, ok = strings.CutSuffix(mName, "_info") + if !ok { + gotMFName = mName + } + default: + gotMFName = mName + } + return mfNameStr == gotMFName +} + // Adds samples to the appender, checking the error, and then returns the # of samples added, // whether the caller should continue to process more samples, and any sample or bucket limit errors. func (sl *scrapeLoop) checkAddError(met []byte, err error, sampleLimitErr, bucketLimitErr *error, appErrs *appendErrors) (bool, error) { diff --git a/scrape/scrape_test.go b/scrape/scrape_test.go index a67d52e5cc..c1fca54c6a 100644 --- a/scrape/scrape_test.go +++ b/scrape/scrape_test.go @@ -50,6 +50,7 @@ import ( "github.com/prometheus/prometheus/model/exemplar" "github.com/prometheus/prometheus/model/histogram" "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/model/metadata" "github.com/prometheus/prometheus/model/relabel" "github.com/prometheus/prometheus/model/textparse" "github.com/prometheus/prometheus/model/timestamp" @@ -96,7 +97,9 @@ func TestStorageHandlesOutOfOrderTimestamps(t *testing.T) { // Test with default OutOfOrderTimeWindow (0) t.Run("Out-Of-Order Sample Disabled", func(t *testing.T) { s := teststorage.New(t) - defer s.Close() + t.Cleanup(func() { + _ = s.Close() + }) runScrapeLoopTest(t, s, false) }) @@ -104,7 +107,9 @@ func TestStorageHandlesOutOfOrderTimestamps(t *testing.T) { // Test with specific OutOfOrderTimeWindow (600000) t.Run("Out-Of-Order Sample Enabled", func(t *testing.T) { s := teststorage.New(t, 600000) - defer s.Close() + t.Cleanup(func() { + _ = s.Close() + }) runScrapeLoopTest(t, s, true) }) @@ -126,13 +131,13 @@ func runScrapeLoopTest(t *testing.T, s *teststorage.TestStorage, expectOutOfOrde timestampInorder2 := now.Add(5 * time.Minute) slApp := sl.appender(context.Background()) - _, _, _, err := sl.append(slApp, []byte(`metric_a{a="1",b="1"} 1`), "text/plain", timestampInorder1) + _, _, _, err := sl.append(slApp, []byte(`metric_total{a="1",b="1"} 1`), "text/plain", timestampInorder1) require.NoError(t, err) - _, _, _, err = sl.append(slApp, []byte(`metric_a{a="1",b="1"} 2`), "text/plain", timestampOutOfOrder) + _, _, _, err = sl.append(slApp, []byte(`metric_total{a="1",b="1"} 2`), "text/plain", timestampOutOfOrder) require.NoError(t, err) - _, _, _, err = sl.append(slApp, []byte(`metric_a{a="1",b="1"} 3`), "text/plain", timestampInorder2) + _, _, _, err = sl.append(slApp, []byte(`metric_total{a="1",b="1"} 3`), "text/plain", timestampInorder2) require.NoError(t, err) require.NoError(t, slApp.Commit()) @@ -145,7 +150,7 @@ func runScrapeLoopTest(t *testing.T, s *teststorage.TestStorage, expectOutOfOrde defer q.Close() // Use a matcher to filter the metric name. - series := q.Select(ctx, false, nil, labels.MustNewMatcher(labels.MatchRegexp, "__name__", "metric_a")) + series := q.Select(ctx, false, nil, labels.MustNewMatcher(labels.MatchRegexp, "__name__", "metric_total")) var results []floatSample for series.Next() { @@ -165,12 +170,12 @@ func runScrapeLoopTest(t *testing.T, s *teststorage.TestStorage, expectOutOfOrde // Define the expected results want := []floatSample{ { - metric: labels.FromStrings("__name__", "metric_a", "a", "1", "b", "1"), + metric: labels.FromStrings("__name__", "metric_total", "a", "1", "b", "1"), t: timestamp.FromTime(timestampInorder1), f: 1, }, { - metric: labels.FromStrings("__name__", "metric_a", "a", "1", "b", "1"), + metric: labels.FromStrings("__name__", "metric_total", "a", "1", "b", "1"), t: timestamp.FromTime(timestampInorder2), f: 3, }, @@ -183,6 +188,110 @@ func runScrapeLoopTest(t *testing.T, s *teststorage.TestStorage, expectOutOfOrde } } +// Regression test against https://github.com/prometheus/prometheus/issues/15831. +func TestScrapeAppendMetadataUpdate(t *testing.T) { + const ( + scrape1 = `# TYPE test_metric counter +# HELP test_metric some help text +# UNIT test_metric metric +test_metric_total 1 +# TYPE test_metric2 gauge +# HELP test_metric2 other help text +test_metric2{foo="bar"} 2 +# TYPE test_metric3 gauge +# HELP test_metric3 this represents tricky case of "broken" text that is not trivial to detect +test_metric3_metric4{foo="bar"} 2 +# EOF` + scrape2 = `# TYPE test_metric counter +# HELP test_metric different help text +test_metric_total 11 +# TYPE test_metric2 gauge +# HELP test_metric2 other help text +# UNIT test_metric2 metric2 +test_metric2{foo="bar"} 22 +# EOF` + ) + + // Create an appender for adding samples to the storage. + capp := &collectResultAppender{next: nopAppender{}} + sl := newBasicScrapeLoop(t, context.Background(), nil, func(ctx context.Context) storage.Appender { return capp }, 0) + + now := time.Now() + slApp := sl.appender(context.Background()) + _, _, _, err := sl.append(slApp, []byte(scrape1), "application/openmetrics-text", now) + require.NoError(t, err) + require.NoError(t, slApp.Commit()) + testutil.RequireEqualWithOptions(t, []metadataEntry{ + {metric: labels.FromStrings("__name__", "test_metric_total"), m: metadata.Metadata{Type: "counter", Unit: "metric", Help: "some help text"}}, + {metric: labels.FromStrings("__name__", "test_metric2", "foo", "bar"), m: metadata.Metadata{Type: "gauge", Unit: "", Help: "other help text"}}, + }, capp.resultMetadata, []cmp.Option{cmp.Comparer(metadataEntryEqual)}) + capp.resultMetadata = nil + + // Next (the same) scrape should not add new metadata entries. + slApp = sl.appender(context.Background()) + _, _, _, err = sl.append(slApp, []byte(scrape1), "application/openmetrics-text", now.Add(15*time.Second)) + require.NoError(t, err) + require.NoError(t, slApp.Commit()) + testutil.RequireEqualWithOptions(t, []metadataEntry(nil), capp.resultMetadata, []cmp.Option{cmp.Comparer(metadataEntryEqual)}) + + slApp = sl.appender(context.Background()) + _, _, _, err = sl.append(slApp, []byte(scrape2), "application/openmetrics-text", now.Add(15*time.Second)) + require.NoError(t, err) + require.NoError(t, slApp.Commit()) + testutil.RequireEqualWithOptions(t, []metadataEntry{ + {metric: labels.FromStrings("__name__", "test_metric_total"), m: metadata.Metadata{Type: "counter", Unit: "metric", Help: "different help text"}}, // Here, technically we should have no unit, but it's a known limitation of the current implementation. + {metric: labels.FromStrings("__name__", "test_metric2", "foo", "bar"), m: metadata.Metadata{Type: "gauge", Unit: "metric2", Help: "other help text"}}, + }, capp.resultMetadata, []cmp.Option{cmp.Comparer(metadataEntryEqual)}) +} + +func TestIsSeriesPartOfFamily(t *testing.T) { + t.Run("counter", func(t *testing.T) { + require.True(t, isSeriesPartOfFamily("http_requests_total", []byte("http_requests_total"), model.MetricTypeCounter)) // Prometheus text style. + require.True(t, isSeriesPartOfFamily("http_requests_total", []byte("http_requests"), model.MetricTypeCounter)) // OM text style. + require.True(t, isSeriesPartOfFamily("http_requests_total", []byte("http_requests_total"), model.MetricTypeUnknown)) + + require.False(t, isSeriesPartOfFamily("http_requests_total", []byte("http_requests"), model.MetricTypeUnknown)) // We don't know. + require.False(t, isSeriesPartOfFamily("http_requests2_total", []byte("http_requests_total"), model.MetricTypeCounter)) + require.False(t, isSeriesPartOfFamily("http_requests_requests_total", []byte("http_requests"), model.MetricTypeCounter)) + }) + + t.Run("gauge", func(t *testing.T) { + require.True(t, isSeriesPartOfFamily("http_requests_count", []byte("http_requests_count"), model.MetricTypeGauge)) + require.True(t, isSeriesPartOfFamily("http_requests_count", []byte("http_requests_count"), model.MetricTypeUnknown)) + + require.False(t, isSeriesPartOfFamily("http_requests_count2", []byte("http_requests_count"), model.MetricTypeCounter)) + }) + + t.Run("histogram", func(t *testing.T) { + require.True(t, isSeriesPartOfFamily("http_requests_seconds_sum", []byte("http_requests_seconds"), model.MetricTypeHistogram)) + require.True(t, isSeriesPartOfFamily("http_requests_seconds_count", []byte("http_requests_seconds"), model.MetricTypeHistogram)) + require.True(t, isSeriesPartOfFamily("http_requests_seconds_bucket", []byte("http_requests_seconds"), model.MetricTypeHistogram)) + require.True(t, isSeriesPartOfFamily("http_requests_seconds", []byte("http_requests_seconds"), model.MetricTypeHistogram)) + + require.False(t, isSeriesPartOfFamily("http_requests_seconds_sum", []byte("http_requests_seconds"), model.MetricTypeUnknown)) // We don't know. + require.False(t, isSeriesPartOfFamily("http_requests_seconds2_sum", []byte("http_requests_seconds"), model.MetricTypeHistogram)) + }) + + t.Run("summary", func(t *testing.T) { + require.True(t, isSeriesPartOfFamily("http_requests_seconds_sum", []byte("http_requests_seconds"), model.MetricTypeSummary)) + require.True(t, isSeriesPartOfFamily("http_requests_seconds_count", []byte("http_requests_seconds"), model.MetricTypeSummary)) + require.True(t, isSeriesPartOfFamily("http_requests_seconds", []byte("http_requests_seconds"), model.MetricTypeSummary)) + + require.False(t, isSeriesPartOfFamily("http_requests_seconds_sum", []byte("http_requests_seconds"), model.MetricTypeUnknown)) // We don't know. + require.False(t, isSeriesPartOfFamily("http_requests_seconds2_sum", []byte("http_requests_seconds"), model.MetricTypeSummary)) + }) + + t.Run("info", func(t *testing.T) { + require.True(t, isSeriesPartOfFamily("go_build_info", []byte("go_build_info"), model.MetricTypeInfo)) // Prometheus text style. + require.True(t, isSeriesPartOfFamily("go_build_info", []byte("go_build"), model.MetricTypeInfo)) // OM text style. + require.True(t, isSeriesPartOfFamily("go_build_info", []byte("go_build_info"), model.MetricTypeUnknown)) + + require.False(t, isSeriesPartOfFamily("go_build_info", []byte("go_build"), model.MetricTypeUnknown)) // We don't know. + require.False(t, isSeriesPartOfFamily("go_build2_info", []byte("go_build_info"), model.MetricTypeInfo)) + require.False(t, isSeriesPartOfFamily("go_build_build_info", []byte("go_build_info"), model.MetricTypeInfo)) + }) +} + func TestDroppedTargetsList(t *testing.T) { var ( app = &nopAppendable{} @@ -824,7 +933,7 @@ func newBasicScrapeLoopWithFallback(t testing.TB, ctx context.Context, scraper s false, false, false, - false, + true, nil, false, newTestScrapeMetrics(t), @@ -1131,7 +1240,7 @@ func TestScrapeLoopMetadata(t *testing.T) { total, _, _, err := sl.append(slApp, []byte(`# TYPE test_metric counter # HELP test_metric some help text # UNIT test_metric metric -test_metric 1 +test_metric_total 1 # TYPE test_metric_no_help gauge # HELP test_metric_no_type other help text # EOF`), "application/openmetrics-text", time.Now()) diff --git a/scrape/target.go b/scrape/target.go index 22cde01c05..4f576504f0 100644 --- a/scrape/target.go +++ b/scrape/target.go @@ -78,17 +78,17 @@ func (t *Target) String() string { // MetricMetadataStore represents a storage for metadata. type MetricMetadataStore interface { ListMetadata() []MetricMetadata - GetMetadata(metric string) (MetricMetadata, bool) + GetMetadata(mfName string) (MetricMetadata, bool) SizeMetadata() int LengthMetadata() int } -// MetricMetadata is a piece of metadata for a metric. +// MetricMetadata is a piece of metadata for a metric family. type MetricMetadata struct { - Metric string - Type model.MetricType - Help string - Unit string + MetricFamily string + Type model.MetricType + Help string + Unit string } func (t *Target) ListMetadata() []MetricMetadata { @@ -124,14 +124,14 @@ func (t *Target) LengthMetadata() int { } // GetMetadata returns type and help metadata for the given metric. -func (t *Target) GetMetadata(metric string) (MetricMetadata, bool) { +func (t *Target) GetMetadata(mfName string) (MetricMetadata, bool) { t.mtx.RLock() defer t.mtx.RUnlock() if t.metadata == nil { return MetricMetadata{}, false } - return t.metadata.GetMetadata(metric) + return t.metadata.GetMetadata(mfName) } func (t *Target) SetMetadataStore(s MetricMetadataStore) { diff --git a/storage/remote/metadata_watcher_test.go b/storage/remote/metadata_watcher_test.go index ce9b9d022e..d939ef8efb 100644 --- a/storage/remote/metadata_watcher_test.go +++ b/storage/remote/metadata_watcher_test.go @@ -40,9 +40,9 @@ func (s *TestMetaStore) ListMetadata() []scrape.MetricMetadata { return s.Metadata } -func (s *TestMetaStore) GetMetadata(metric string) (scrape.MetricMetadata, bool) { +func (s *TestMetaStore) GetMetadata(mfName string) (scrape.MetricMetadata, bool) { for _, m := range s.Metadata { - if metric == m.Metric { + if mfName == m.MetricFamily { return m, true } } @@ -106,26 +106,26 @@ func TestWatchScrapeManager_ReadyForCollection(t *testing.T) { metadata := &TestMetaStore{ Metadata: []scrape.MetricMetadata{ { - Metric: "prometheus_tsdb_head_chunks_created_total", - Type: model.MetricTypeCounter, - Help: "Total number", - Unit: "", + MetricFamily: "prometheus_tsdb_head_chunks_created", + Type: model.MetricTypeCounter, + Help: "Total number", + Unit: "", }, { - Metric: "prometheus_remote_storage_retried_samples_total", - Type: model.MetricTypeCounter, - Help: "Total number", - Unit: "", + MetricFamily: "prometheus_remote_storage_retried_samples", + Type: model.MetricTypeCounter, + Help: "Total number", + Unit: "", }, }, } metadataDup := &TestMetaStore{ Metadata: []scrape.MetricMetadata{ { - Metric: "prometheus_tsdb_head_chunks_created_total", - Type: model.MetricTypeCounter, - Help: "Total number", - Unit: "", + MetricFamily: "prometheus_tsdb_head_chunks_created", + Type: model.MetricTypeCounter, + Help: "Total number", + Unit: "", }, }, } diff --git a/storage/remote/queue_manager.go b/storage/remote/queue_manager.go index f0ea33fb52..50f10eebe3 100644 --- a/storage/remote/queue_manager.go +++ b/storage/remote/queue_manager.go @@ -550,7 +550,7 @@ func (t *QueueManager) AppendWatcherMetadata(ctx context.Context, metadata []scr mm := make([]prompb.MetricMetadata, 0, len(metadata)) for _, entry := range metadata { mm = append(mm, prompb.MetricMetadata{ - MetricFamilyName: entry.Metric, + MetricFamilyName: entry.MetricFamily, Help: entry.Help, Type: prompb.FromMetadataType(entry.Type), Unit: entry.Unit, diff --git a/storage/remote/queue_manager_test.go b/storage/remote/queue_manager_test.go index 202c71c348..38eda81d97 100644 --- a/storage/remote/queue_manager_test.go +++ b/storage/remote/queue_manager_test.go @@ -342,10 +342,10 @@ func TestMetadataDelivery(t *testing.T) { numMetadata := 1532 for i := 0; i < numMetadata; i++ { metadata = append(metadata, scrape.MetricMetadata{ - Metric: "prometheus_remote_storage_sent_metadata_bytes_total_" + strconv.Itoa(i), - Type: model.MetricTypeCounter, - Help: "a nice help text", - Unit: "", + MetricFamily: "prometheus_remote_storage_sent_metadata_bytes_" + strconv.Itoa(i), + Type: model.MetricTypeCounter, + Help: "a nice help text", + Unit: "", }) } @@ -357,7 +357,7 @@ func TestMetadataDelivery(t *testing.T) { // fit into MaxSamplesPerSend. require.Equal(t, numMetadata/config.DefaultMetadataConfig.MaxSamplesPerSend+1, c.writesReceived) // Make sure the last samples were sent. - require.Equal(t, c.receivedMetadata[metadata[len(metadata)-1].Metric][0].MetricFamilyName, metadata[len(metadata)-1].Metric) + require.Equal(t, c.receivedMetadata[metadata[len(metadata)-1].MetricFamily][0].MetricFamilyName, metadata[len(metadata)-1].MetricFamily) } func TestWALMetadataDelivery(t *testing.T) { diff --git a/web/api/v1/api.go b/web/api/v1/api.go index 9821b26ec2..7687264426 100644 --- a/web/api/v1/api.go +++ b/web/api/v1/api.go @@ -1228,11 +1228,11 @@ func (api *API) targetMetadata(r *http.Request) apiFuncResult { if metric == "" { for _, md := range t.ListMetadata() { res = append(res, metricMetadata{ - Target: targetLabels, - Metric: md.Metric, - Type: md.Type, - Help: md.Help, - Unit: md.Unit, + Target: targetLabels, + MetricFamily: md.MetricFamily, + Type: md.Type, + Help: md.Help, + Unit: md.Unit, }) } continue @@ -1253,11 +1253,11 @@ func (api *API) targetMetadata(r *http.Request) apiFuncResult { } type metricMetadata struct { - Target labels.Labels `json:"target"` - Metric string `json:"metric,omitempty"` - Type model.MetricType `json:"type"` - Help string `json:"help"` - Unit string `json:"unit"` + Target labels.Labels `json:"target"` + MetricFamily string `json:"metric,omitempty"` + Type model.MetricType `json:"type"` + Help string `json:"help"` + Unit string `json:"unit"` } // AlertmanagerDiscovery has all the active Alertmanagers. @@ -1357,7 +1357,7 @@ func (api *API) metricMetadata(r *http.Request) apiFuncResult { if metric == "" { for _, mm := range t.ListMetadata() { m := metadata.Metadata{Type: mm.Type, Help: mm.Help, Unit: mm.Unit} - ms, ok := metrics[mm.Metric] + ms, ok := metrics[mm.MetricFamily] if limitPerMetric > 0 && len(ms) >= limitPerMetric { continue @@ -1365,7 +1365,7 @@ func (api *API) metricMetadata(r *http.Request) apiFuncResult { if !ok { ms = map[metadata.Metadata]struct{}{} - metrics[mm.Metric] = ms + metrics[mm.MetricFamily] = ms } ms[m] = struct{}{} } @@ -1374,7 +1374,7 @@ func (api *API) metricMetadata(r *http.Request) apiFuncResult { if md, ok := t.GetMetadata(metric); ok { m := metadata.Metadata{Type: md.Type, Help: md.Help, Unit: md.Unit} - ms, ok := metrics[md.Metric] + ms, ok := metrics[md.MetricFamily] if limitPerMetric > 0 && len(ms) >= limitPerMetric { continue @@ -1382,7 +1382,7 @@ func (api *API) metricMetadata(r *http.Request) apiFuncResult { if !ok { ms = map[metadata.Metadata]struct{}{} - metrics[md.Metric] = ms + metrics[md.MetricFamily] = ms } ms[m] = struct{}{} } diff --git a/web/api/v1/api_test.go b/web/api/v1/api_test.go index 37227d849d..f9bdbe3947 100644 --- a/web/api/v1/api_test.go +++ b/web/api/v1/api_test.go @@ -84,7 +84,7 @@ func (s *testMetaStore) ListMetadata() []scrape.MetricMetadata { func (s *testMetaStore) GetMetadata(metric string) (scrape.MetricMetadata, bool) { for _, m := range s.Metadata { - if metric == m.Metric { + if metric == m.MetricFamily { return m, true } } @@ -1891,10 +1891,10 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "test", metadata: []scrape.MetricMetadata{ { - Metric: "go_threads", - Type: model.MetricTypeGauge, - Help: "Number of OS threads created.", - Unit: "", + MetricFamily: "go_threads", + Type: model.MetricTypeGauge, + Help: "Number of OS threads created.", + Unit: "", }, }, }, @@ -1921,10 +1921,10 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "blackbox", metadata: []scrape.MetricMetadata{ { - Metric: "prometheus_tsdb_storage_blocks_bytes", - Type: model.MetricTypeGauge, - Help: "The number of bytes that are currently used for local storage by all blocks.", - Unit: "", + MetricFamily: "prometheus_tsdb_storage_blocks_bytes", + Type: model.MetricTypeGauge, + Help: "The number of bytes that are currently used for local storage by all blocks.", + Unit: "", }, }, }, @@ -1934,10 +1934,10 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E Target: labels.FromMap(map[string]string{ "job": "blackbox", }), - Metric: "prometheus_tsdb_storage_blocks_bytes", - Help: "The number of bytes that are currently used for local storage by all blocks.", - Type: model.MetricTypeGauge, - Unit: "", + MetricFamily: "prometheus_tsdb_storage_blocks_bytes", + Help: "The number of bytes that are currently used for local storage by all blocks.", + Type: model.MetricTypeGauge, + Unit: "", }, }, }, @@ -1949,10 +1949,10 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "test", metadata: []scrape.MetricMetadata{ { - Metric: "go_threads", - Type: model.MetricTypeGauge, - Help: "Number of OS threads created.", - Unit: "", + MetricFamily: "go_threads", + Type: model.MetricTypeGauge, + Help: "Number of OS threads created.", + Unit: "", }, }, }, @@ -1960,10 +1960,10 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "blackbox", metadata: []scrape.MetricMetadata{ { - Metric: "prometheus_tsdb_storage_blocks_bytes", - Type: model.MetricTypeGauge, - Help: "The number of bytes that are currently used for local storage by all blocks.", - Unit: "", + MetricFamily: "prometheus_tsdb_storage_blocks_bytes", + Type: model.MetricTypeGauge, + Help: "The number of bytes that are currently used for local storage by all blocks.", + Unit: "", }, }, }, @@ -1973,25 +1973,25 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E Target: labels.FromMap(map[string]string{ "job": "test", }), - Metric: "go_threads", - Help: "Number of OS threads created.", - Type: model.MetricTypeGauge, - Unit: "", + MetricFamily: "go_threads", + Help: "Number of OS threads created.", + Type: model.MetricTypeGauge, + Unit: "", }, { Target: labels.FromMap(map[string]string{ "job": "blackbox", }), - Metric: "prometheus_tsdb_storage_blocks_bytes", - Help: "The number of bytes that are currently used for local storage by all blocks.", - Type: model.MetricTypeGauge, - Unit: "", + MetricFamily: "prometheus_tsdb_storage_blocks_bytes", + Help: "The number of bytes that are currently used for local storage by all blocks.", + Type: model.MetricTypeGauge, + Unit: "", }, }, sorter: func(m interface{}) { sort.Slice(m.([]metricMetadata), func(i, j int) bool { s := m.([]metricMetadata) - return s[i].Metric < s[j].Metric + return s[i].MetricFamily < s[j].MetricFamily }) }, }, @@ -2026,16 +2026,16 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "test", metadata: []scrape.MetricMetadata{ { - Metric: "prometheus_engine_query_duration_seconds", - Type: model.MetricTypeSummary, - Help: "Query timings", - Unit: "", + MetricFamily: "prometheus_engine_query_duration_seconds", + Type: model.MetricTypeSummary, + Help: "Query timings", + Unit: "", }, { - Metric: "go_info", - Type: model.MetricTypeGauge, - Help: "Information about the Go environment.", - Unit: "", + MetricFamily: "go_info", + Type: model.MetricTypeGauge, + Help: "Information about the Go environment.", + Unit: "", }, }, }, @@ -2056,10 +2056,10 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "test", metadata: []scrape.MetricMetadata{ { - Metric: "go_threads", - Type: model.MetricTypeGauge, - Help: "Number of OS threads created", - Unit: "", + MetricFamily: "go_threads", + Type: model.MetricTypeGauge, + Help: "Number of OS threads created", + Unit: "", }, }, }, @@ -2067,10 +2067,10 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "blackbox", metadata: []scrape.MetricMetadata{ { - Metric: "go_threads", - Type: model.MetricTypeGauge, - Help: "Number of OS threads created", - Unit: "", + MetricFamily: "go_threads", + Type: model.MetricTypeGauge, + Help: "Number of OS threads created", + Unit: "", }, }, }, @@ -2089,10 +2089,10 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "test", metadata: []scrape.MetricMetadata{ { - Metric: "go_threads", - Type: model.MetricTypeGauge, - Help: "Number of OS threads created", - Unit: "", + MetricFamily: "go_threads", + Type: model.MetricTypeGauge, + Help: "Number of OS threads created", + Unit: "", }, }, }, @@ -2100,10 +2100,10 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "blackbox", metadata: []scrape.MetricMetadata{ { - Metric: "go_threads", - Type: model.MetricTypeGauge, - Help: "Number of OS threads that were created.", - Unit: "", + MetricFamily: "go_threads", + Type: model.MetricTypeGauge, + Help: "Number of OS threads that were created.", + Unit: "", }, }, }, @@ -2136,16 +2136,16 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "test", metadata: []scrape.MetricMetadata{ { - Metric: "go_threads", - Type: model.MetricTypeGauge, - Help: "Number of OS threads created", - Unit: "", + MetricFamily: "go_threads", + Type: model.MetricTypeGauge, + Help: "Number of OS threads created", + Unit: "", }, { - Metric: "prometheus_engine_query_duration_seconds", - Type: model.MetricTypeSummary, - Help: "Query Timings.", - Unit: "", + MetricFamily: "prometheus_engine_query_duration_seconds", + Type: model.MetricTypeSummary, + Help: "Query Timings.", + Unit: "", }, }, }, @@ -2153,10 +2153,10 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "blackbox", metadata: []scrape.MetricMetadata{ { - Metric: "go_gc_duration_seconds", - Type: model.MetricTypeSummary, - Help: "A summary of the GC invocation durations.", - Unit: "", + MetricFamily: "go_gc_duration_seconds", + Type: model.MetricTypeSummary, + Help: "A summary of the GC invocation durations.", + Unit: "", }, }, }, @@ -2172,22 +2172,22 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "test", metadata: []scrape.MetricMetadata{ { - Metric: "go_threads", - Type: model.MetricTypeGauge, - Help: "Number of OS threads created", - Unit: "", + MetricFamily: "go_threads", + Type: model.MetricTypeGauge, + Help: "Number of OS threads created", + Unit: "", }, { - Metric: "go_threads", - Type: model.MetricTypeGauge, - Help: "Repeated metadata", - Unit: "", + MetricFamily: "go_threads", + Type: model.MetricTypeGauge, + Help: "Repeated metadata", + Unit: "", }, { - Metric: "go_gc_duration_seconds", - Type: model.MetricTypeSummary, - Help: "A summary of the GC invocation durations.", - Unit: "", + MetricFamily: "go_gc_duration_seconds", + Type: model.MetricTypeSummary, + Help: "A summary of the GC invocation durations.", + Unit: "", }, }, }, @@ -2211,22 +2211,22 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "test", metadata: []scrape.MetricMetadata{ { - Metric: "go_threads", - Type: model.MetricTypeGauge, - Help: "Number of OS threads created", - Unit: "", + MetricFamily: "go_threads", + Type: model.MetricTypeGauge, + Help: "Number of OS threads created", + Unit: "", }, { - Metric: "go_threads", - Type: model.MetricTypeGauge, - Help: "Repeated metadata", - Unit: "", + MetricFamily: "go_threads", + Type: model.MetricTypeGauge, + Help: "Repeated metadata", + Unit: "", }, { - Metric: "go_gc_duration_seconds", - Type: model.MetricTypeSummary, - Help: "A summary of the GC invocation durations.", - Unit: "", + MetricFamily: "go_gc_duration_seconds", + Type: model.MetricTypeSummary, + Help: "A summary of the GC invocation durations.", + Unit: "", }, }, }, @@ -2244,22 +2244,22 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "test", metadata: []scrape.MetricMetadata{ { - Metric: "go_threads", - Type: model.MetricTypeGauge, - Help: "Number of OS threads created", - Unit: "", + MetricFamily: "go_threads", + Type: model.MetricTypeGauge, + Help: "Number of OS threads created", + Unit: "", }, { - Metric: "go_threads", - Type: model.MetricTypeGauge, - Help: "Repeated metadata", - Unit: "", + MetricFamily: "go_threads", + Type: model.MetricTypeGauge, + Help: "Repeated metadata", + Unit: "", }, { - Metric: "go_gc_duration_seconds", - Type: model.MetricTypeSummary, - Help: "A summary of the GC invocation durations.", - Unit: "", + MetricFamily: "go_gc_duration_seconds", + Type: model.MetricTypeSummary, + Help: "A summary of the GC invocation durations.", + Unit: "", }, }, }, @@ -2267,16 +2267,16 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "secondTarget", metadata: []scrape.MetricMetadata{ { - Metric: "go_threads", - Type: model.MetricTypeGauge, - Help: "Number of OS threads created, but from a different target", - Unit: "", + MetricFamily: "go_threads", + Type: model.MetricTypeGauge, + Help: "Number of OS threads created, but from a different target", + Unit: "", }, { - Metric: "go_gc_duration_seconds", - Type: model.MetricTypeSummary, - Help: "A summary of the GC invocation durations, but from a different target.", - Unit: "", + MetricFamily: "go_gc_duration_seconds", + Type: model.MetricTypeSummary, + Help: "A summary of the GC invocation durations, but from a different target.", + Unit: "", }, }, }, @@ -2293,10 +2293,10 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "test", metadata: []scrape.MetricMetadata{ { - Metric: "go_threads", - Type: model.MetricTypeGauge, - Help: "Number of OS threads created", - Unit: "", + MetricFamily: "go_threads", + Type: model.MetricTypeGauge, + Help: "Number of OS threads created", + Unit: "", }, }, }, @@ -2304,16 +2304,16 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "blackbox", metadata: []scrape.MetricMetadata{ { - Metric: "go_gc_duration_seconds", - Type: model.MetricTypeSummary, - Help: "A summary of the GC invocation durations.", - Unit: "", + MetricFamily: "go_gc_duration_seconds", + Type: model.MetricTypeSummary, + Help: "A summary of the GC invocation durations.", + Unit: "", }, { - Metric: "go_threads", - Type: model.MetricTypeGauge, - Help: "Number of OS threads that were created.", - Unit: "", + MetricFamily: "go_threads", + Type: model.MetricTypeGauge, + Help: "Number of OS threads that were created.", + Unit: "", }, }, }, @@ -2342,10 +2342,10 @@ func testEndpoints(t *testing.T, api *API, tr *testTargetRetriever, es storage.E identifier: "test", metadata: []scrape.MetricMetadata{ { - Metric: "go_threads", - Type: model.MetricTypeGauge, - Help: "Number of OS threads created", - Unit: "", + MetricFamily: "go_threads", + Type: model.MetricTypeGauge, + Help: "Number of OS threads created", + Unit: "", }, }, }, From 5ed0edd3abc6d3799794f77c803289e755b81427 Mon Sep 17 00:00:00 2001 From: George Krajcsovits Date: Thu, 16 Jan 2025 08:12:33 +0100 Subject: [PATCH 092/110] Update storage/remote/queue_manager.go Co-authored-by: Bartlomiej Plotka Signed-off-by: George Krajcsovits --- storage/remote/queue_manager.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/storage/remote/queue_manager.go b/storage/remote/queue_manager.go index 731a0f405f..7bf9600aff 100644 --- a/storage/remote/queue_manager.go +++ b/storage/remote/queue_manager.go @@ -1928,8 +1928,8 @@ func populateV2TimeSeries(symbolTable *writev2.SymbolsTable, batch []timeSeries, // Safeguard against sending garbage in case of not having metadata // for whatever reason. pendingData[nPending].Metadata.Type = writev2.Metadata_METRIC_TYPE_UNSPECIFIED - pendingData[nPending].Metadata.HelpRef = symbolTable.Symbolize("") - pendingData[nPending].Metadata.UnitRef = symbolTable.Symbolize("") + pendingData[nPending].Metadata.HelpRef = 0 + pendingData[nPending].Metadata.UnitRef = 0 } if sendExemplars { From af928a1ba42fd0e99b4c920142399b4da47c3d1f Mon Sep 17 00:00:00 2001 From: bwplotka Date: Thu, 16 Jan 2025 13:15:56 +0000 Subject: [PATCH 093/110] Addressed Krajo's comment. Signed-off-by: bwplotka --- scrape/scrape.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/scrape/scrape.go b/scrape/scrape.go index 1f41f50689..f9178c507b 100644 --- a/scrape/scrape.go +++ b/scrape/scrape.go @@ -1694,13 +1694,13 @@ loop: if sl.cache.getDropped(met) { continue } - ce, ok, seriesAlreadyScraped := sl.cache.get(met) + ce, seriesCached, seriesAlreadyScraped := sl.cache.get(met) var ( ref storage.SeriesRef hash uint64 ) - if ok { + if seriesCached { ref = ce.ref lset = ce.lset hash = ce.hash @@ -1781,7 +1781,7 @@ loop: break loop } - if !ok { + if !seriesCached { if parsedTimestamp == nil || sl.trackTimestampsStaleness { // Bypass staleness logic if there is an explicit timestamp. sl.cache.trackStaleness(hash, lset) @@ -1841,7 +1841,7 @@ loop: if sl.appendMetadataToWAL && lastMeta != nil { // Is it new series OR did metadata change for this family? - if !ok || lastMeta.lastIterChange == sl.cache.iter { + if !seriesCached || lastMeta.lastIterChange == sl.cache.iter { // In majority cases we can trust that the current series/histogram is matching the lastMeta and lastMFName. // However, optional TYPE etc metadata and broken OM text can break this, detect those cases here. // TODO(bwplotka): Consider moving this to parser as many parser users end up doing this (e.g. CT and NHCB parsing). From 3119567d5bd32591bbb7db927cf21a2f79a44a16 Mon Sep 17 00:00:00 2001 From: bwplotka Date: Thu, 16 Jan 2025 19:47:32 +0000 Subject: [PATCH 094/110] scrape: Add metadata for automatic metrics. Signed-off-by: bwplotka --- scrape/scrape.go | 125 ++++++++++++++++++++++++++++++++---------- scrape/scrape_test.go | 24 ++++++++ 2 files changed, 121 insertions(+), 28 deletions(-) diff --git a/scrape/scrape.go b/scrape/scrape.go index f9178c507b..cf330fb75d 100644 --- a/scrape/scrape.go +++ b/scrape/scrape.go @@ -1989,17 +1989,80 @@ func (sl *scrapeLoop) checkAddError(met []byte, err error, sampleLimitErr, bucke } } +// reportSample represents automatically generated timeseries documented in +// https://prometheus.io/docs/concepts/jobs_instances/#automatically-generated-labels-and-time-series +type reportSample struct { + metadata.Metadata + name []byte +} + // The constants are suffixed with the invalid \xff unicode rune to avoid collisions // with scraped metrics in the cache. var ( - scrapeHealthMetricName = []byte("up" + "\xff") - scrapeDurationMetricName = []byte("scrape_duration_seconds" + "\xff") - scrapeSamplesMetricName = []byte("scrape_samples_scraped" + "\xff") - samplesPostRelabelMetricName = []byte("scrape_samples_post_metric_relabeling" + "\xff") - scrapeSeriesAddedMetricName = []byte("scrape_series_added" + "\xff") - scrapeTimeoutMetricName = []byte("scrape_timeout_seconds" + "\xff") - scrapeSampleLimitMetricName = []byte("scrape_sample_limit" + "\xff") - scrapeBodySizeBytesMetricName = []byte("scrape_body_size_bytes" + "\xff") + scrapeHealthMetric = reportSample{ + name: []byte("up" + "\xff"), + Metadata: metadata.Metadata{ + Type: model.MetricTypeGauge, + Help: "Health of the scrape target. 1 means the target is healthy, 0 if the scrape failed.", + Unit: "targets", + }, + } + scrapeDurationMetric = reportSample{ + name: []byte("scrape_duration_seconds" + "\xff"), + Metadata: metadata.Metadata{ + Type: model.MetricTypeGauge, + Help: "Duration of the last scrape in seconds.", + Unit: "seconds", + }, + } + scrapeSamplesMetric = reportSample{ + name: []byte("scrape_samples_scraped" + "\xff"), + Metadata: metadata.Metadata{ + Type: model.MetricTypeGauge, + Help: "Number of samples last scraped.", + Unit: "samples", + }, + } + samplesPostRelabelMetric = reportSample{ + name: []byte("scrape_samples_post_metric_relabeling" + "\xff"), + Metadata: metadata.Metadata{ + Type: model.MetricTypeGauge, + Help: "Number of samples remaining after metric relabeling was applied.", + Unit: "samples", + }, + } + scrapeSeriesAddedMetric = reportSample{ + name: []byte("scrape_series_added" + "\xff"), + Metadata: metadata.Metadata{ + Type: model.MetricTypeGauge, + Help: "Number of series in the last scrape.", + Unit: "series", + }, + } + scrapeTimeoutMetric = reportSample{ + name: []byte("scrape_timeout_seconds" + "\xff"), + Metadata: metadata.Metadata{ + Type: model.MetricTypeGauge, + Help: "The configured scrape timeout for a target.", + Unit: "seconds", + }, + } + scrapeSampleLimitMetric = reportSample{ + name: []byte("scrape_sample_limit" + "\xff"), + Metadata: metadata.Metadata{ + Type: model.MetricTypeGauge, + Help: "The configured sample limit for a target. Returns zero if there is no limit configured.", + Unit: "samples", + }, + } + scrapeBodySizeBytesMetric = reportSample{ + name: []byte("scrape_body_size_bytes" + "\xff"), + Metadata: metadata.Metadata{ + Type: model.MetricTypeGauge, + Help: " The uncompressed size of the last scrape response, if successful. Scrapes failing because body_size_limit is exceeded report -1, other scrape failures report 0.", + Unit: "bytes", + }, + } ) func (sl *scrapeLoop) report(app storage.Appender, start time.Time, duration time.Duration, scraped, added, seriesAdded, bytes int, scrapeErr error) (err error) { @@ -2013,29 +2076,29 @@ func (sl *scrapeLoop) report(app storage.Appender, start time.Time, duration tim } b := labels.NewBuilderWithSymbolTable(sl.symbolTable) - if err = sl.addReportSample(app, scrapeHealthMetricName, ts, health, b); err != nil { + if err = sl.addReportSample(app, scrapeHealthMetric, ts, health, b); err != nil { return } - if err = sl.addReportSample(app, scrapeDurationMetricName, ts, duration.Seconds(), b); err != nil { + if err = sl.addReportSample(app, scrapeDurationMetric, ts, duration.Seconds(), b); err != nil { return } - if err = sl.addReportSample(app, scrapeSamplesMetricName, ts, float64(scraped), b); err != nil { + if err = sl.addReportSample(app, scrapeSamplesMetric, ts, float64(scraped), b); err != nil { return } - if err = sl.addReportSample(app, samplesPostRelabelMetricName, ts, float64(added), b); err != nil { + if err = sl.addReportSample(app, samplesPostRelabelMetric, ts, float64(added), b); err != nil { return } - if err = sl.addReportSample(app, scrapeSeriesAddedMetricName, ts, float64(seriesAdded), b); err != nil { + if err = sl.addReportSample(app, scrapeSeriesAddedMetric, ts, float64(seriesAdded), b); err != nil { return } if sl.reportExtraMetrics { - if err = sl.addReportSample(app, scrapeTimeoutMetricName, ts, sl.timeout.Seconds(), b); err != nil { + if err = sl.addReportSample(app, scrapeTimeoutMetric, ts, sl.timeout.Seconds(), b); err != nil { return } - if err = sl.addReportSample(app, scrapeSampleLimitMetricName, ts, float64(sl.sampleLimit), b); err != nil { + if err = sl.addReportSample(app, scrapeSampleLimitMetric, ts, float64(sl.sampleLimit), b); err != nil { return } - if err = sl.addReportSample(app, scrapeBodySizeBytesMetricName, ts, float64(bytes), b); err != nil { + if err = sl.addReportSample(app, scrapeBodySizeBytesMetric, ts, float64(bytes), b); err != nil { return } } @@ -2048,37 +2111,37 @@ func (sl *scrapeLoop) reportStale(app storage.Appender, start time.Time) (err er stale := math.Float64frombits(value.StaleNaN) b := labels.NewBuilder(labels.EmptyLabels()) - if err = sl.addReportSample(app, scrapeHealthMetricName, ts, stale, b); err != nil { + if err = sl.addReportSample(app, scrapeHealthMetric, ts, stale, b); err != nil { return } - if err = sl.addReportSample(app, scrapeDurationMetricName, ts, stale, b); err != nil { + if err = sl.addReportSample(app, scrapeDurationMetric, ts, stale, b); err != nil { return } - if err = sl.addReportSample(app, scrapeSamplesMetricName, ts, stale, b); err != nil { + if err = sl.addReportSample(app, scrapeSamplesMetric, ts, stale, b); err != nil { return } - if err = sl.addReportSample(app, samplesPostRelabelMetricName, ts, stale, b); err != nil { + if err = sl.addReportSample(app, samplesPostRelabelMetric, ts, stale, b); err != nil { return } - if err = sl.addReportSample(app, scrapeSeriesAddedMetricName, ts, stale, b); err != nil { + if err = sl.addReportSample(app, scrapeSeriesAddedMetric, ts, stale, b); err != nil { return } if sl.reportExtraMetrics { - if err = sl.addReportSample(app, scrapeTimeoutMetricName, ts, stale, b); err != nil { + if err = sl.addReportSample(app, scrapeTimeoutMetric, ts, stale, b); err != nil { return } - if err = sl.addReportSample(app, scrapeSampleLimitMetricName, ts, stale, b); err != nil { + if err = sl.addReportSample(app, scrapeSampleLimitMetric, ts, stale, b); err != nil { return } - if err = sl.addReportSample(app, scrapeBodySizeBytesMetricName, ts, stale, b); err != nil { + if err = sl.addReportSample(app, scrapeBodySizeBytesMetric, ts, stale, b); err != nil { return } } return } -func (sl *scrapeLoop) addReportSample(app storage.Appender, s []byte, t int64, v float64, b *labels.Builder) error { - ce, ok, _ := sl.cache.get(s) +func (sl *scrapeLoop) addReportSample(app storage.Appender, s reportSample, t int64, v float64, b *labels.Builder) error { + ce, ok, _ := sl.cache.get(s.name) var ref storage.SeriesRef var lset labels.Labels if ok { @@ -2089,7 +2152,7 @@ func (sl *scrapeLoop) addReportSample(app storage.Appender, s []byte, t int64, v // with scraped metrics in the cache. // We have to drop it when building the actual metric. b.Reset(labels.EmptyLabels()) - b.Set(labels.MetricName, string(s[:len(s)-1])) + b.Set(labels.MetricName, string(s.name[:len(s.name)-1])) lset = sl.reportSampleMutator(b.Labels()) } @@ -2097,7 +2160,13 @@ func (sl *scrapeLoop) addReportSample(app storage.Appender, s []byte, t int64, v switch { case err == nil: if !ok { - sl.cache.addRef(s, ref, lset, lset.Hash()) + sl.cache.addRef(s.name, ref, lset, lset.Hash()) + // We only need to add metadata once a scrape target appears. + if sl.appendMetadataToWAL { + if _, merr := app.UpdateMetadata(ref, lset, s.Metadata); merr != nil { + sl.l.Debug("Error when appending metadata in addReportSample", "ref", fmt.Sprintf("%d", ref), "metadata", fmt.Sprintf("%+v", s.Metadata), "err", merr) + } + } } return nil case errors.Is(err, storage.ErrOutOfOrderSample), errors.Is(err, storage.ErrDuplicateSampleForTimestamp): diff --git a/scrape/scrape_test.go b/scrape/scrape_test.go index c1fca54c6a..2bb9c7247d 100644 --- a/scrape/scrape_test.go +++ b/scrape/scrape_test.go @@ -244,6 +244,30 @@ test_metric2{foo="bar"} 22 }, capp.resultMetadata, []cmp.Option{cmp.Comparer(metadataEntryEqual)}) } +type nopScraper struct { + scraper +} + +func (n nopScraper) Report(start time.Time, dur time.Duration, err error) {} + +func TestScrapeReportMetadataUpdate(t *testing.T) { + // Create an appender for adding samples to the storage. + capp := &collectResultAppender{next: nopAppender{}} + sl := newBasicScrapeLoop(t, context.Background(), nopScraper{}, func(ctx context.Context) storage.Appender { return capp }, 0) + now := time.Now() + slApp := sl.appender(context.Background()) + + require.NoError(t, sl.report(slApp, now, 2*time.Second, 1, 1, 1, 512, nil)) + require.NoError(t, slApp.Commit()) + testutil.RequireEqualWithOptions(t, []metadataEntry{ + {metric: labels.FromStrings("__name__", "up"), m: scrapeHealthMetric.Metadata}, + {metric: labels.FromStrings("__name__", "scrape_duration_seconds"), m: scrapeDurationMetric.Metadata}, + {metric: labels.FromStrings("__name__", "scrape_samples_scraped"), m: scrapeSamplesMetric.Metadata}, + {metric: labels.FromStrings("__name__", "scrape_samples_post_metric_relabeling"), m: samplesPostRelabelMetric.Metadata}, + {metric: labels.FromStrings("__name__", "scrape_series_added"), m: scrapeSeriesAddedMetric.Metadata}, + }, capp.resultMetadata, []cmp.Option{cmp.Comparer(metadataEntryEqual)}) +} + func TestIsSeriesPartOfFamily(t *testing.T) { t.Run("counter", func(t *testing.T) { require.True(t, isSeriesPartOfFamily("http_requests_total", []byte("http_requests_total"), model.MetricTypeCounter)) // Prometheus text style. From 6f1ad64e272eb434018647f3f3e4e5cf2f716481 Mon Sep 17 00:00:00 2001 From: Bartlomiej Plotka Date: Fri, 17 Jan 2025 09:09:28 +0000 Subject: [PATCH 095/110] Update scrape/scrape.go Co-authored-by: Julius Volz Signed-off-by: Bartlomiej Plotka --- scrape/scrape.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scrape/scrape.go b/scrape/scrape.go index cf330fb75d..85eb07a1cc 100644 --- a/scrape/scrape.go +++ b/scrape/scrape.go @@ -2059,7 +2059,7 @@ var ( name: []byte("scrape_body_size_bytes" + "\xff"), Metadata: metadata.Metadata{ Type: model.MetricTypeGauge, - Help: " The uncompressed size of the last scrape response, if successful. Scrapes failing because body_size_limit is exceeded report -1, other scrape failures report 0.", + Help: "The uncompressed size of the last scrape response, if successful. Scrapes failing because body_size_limit is exceeded report -1, other scrape failures report 0.", Unit: "bytes", }, } From f46b984dd12fb69b239af0a8385aaf61efd5600b Mon Sep 17 00:00:00 2001 From: Fiona Liao Date: Fri, 17 Jan 2025 16:17:11 +0000 Subject: [PATCH 096/110] Add additional incompatible nhcb schemas tests for functions and comparison operators (#15813) promql: Add additional incompatible nhcb schemas tests for functions and comparison operators * Add agg_over_time tests for nhcb with incompatible schemas * Add more function and comparison operator tests --------- Signed-off-by: Fiona Liao --- .../testdata/native_histograms.test | 45 +++++++++++++++++++ 1 file changed, 45 insertions(+) diff --git a/promql/promqltest/testdata/native_histograms.test b/promql/promqltest/testdata/native_histograms.test index 414619d5cd..f03b39a9f6 100644 --- a/promql/promqltest/testdata/native_histograms.test +++ b/promql/promqltest/testdata/native_histograms.test @@ -1163,6 +1163,51 @@ eval_warn range from 0 to 12m step 6m metric{series="2"} - ignoring (series) met clear +# Test incompatible schemas with comparison binary operators +load 6m + metric1 {{schema:-53 sum:1 count:1 custom_values:[2] buckets:[1]}} {{schema:-53 sum:1 count:1 custom_values:[5 10] buckets:[1]}} + metric2 {{schema:-53 sum:1 count:1 custom_values:[5 10] buckets:[1]}} {{schema:-53 sum:1 count:1 custom_values:[5 10] buckets:[1]}} + +eval range from 0 to 6m step 6m metric1 == metric2 +metric1{} _ {{schema:-53 count:1 sum:1 custom_values:[5 10] buckets:[1]}} + +eval range from 0 to 6m step 6m metric1 != metric2 +metric1{} {{schema:-53 sum:1 count:1 custom_values:[2] buckets:[1]}} _ + +eval_info range from 0 to 6m step 6m metric2 > metric2 + +clear + +load 6m + nhcb_metric {{schema:-53 sum:1 count:1 custom_values:[2] buckets:[1]}} {{schema:-53 sum:1 count:1 custom_values:[5 10] buckets:[1]}} {{schema:-53 sum:1 count:1 custom_values:[5 10] buckets:[1]}} + +eval_warn instant at 12m sum_over_time(nhcb_metric[13m]) + +eval_warn instant at 12m avg_over_time(nhcb_metric[13m]) + +eval instant at 12m last_over_time(nhcb_metric[13m]) +nhcb_metric{} {{schema:-53 sum:1 count:1 custom_values:[5 10] buckets:[1]}} + +eval instant at 12m count_over_time(nhcb_metric[13m]) +{} 3 + +eval instant at 12m present_over_time(nhcb_metric[13m]) +{} 1 + +eval instant at 12m changes(nhcb_metric[13m]) +{} 1 + +eval_warn instant at 12m delta(nhcb_metric[13m]) + +eval_warn instant at 12m increase(nhcb_metric[13m]) + +eval_warn instant at 12m rate(nhcb_metric[13m]) + +eval instant at 12m resets(nhcb_metric[13m]) +{} 1 + +clear + load 1m metric{group="just-floats", series="1"} 2 metric{group="just-floats", series="2"} 3 From 2a17b99ee2b9ead6729e3c3aed179792399e4ea0 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 17 Jan 2025 16:25:23 +0000 Subject: [PATCH 097/110] chore(deps): bump nanoid from 3.3.7 to 3.3.8 in /web/ui/react-app (#15842) Bumps [nanoid](https://github.com/ai/nanoid) from 3.3.7 to 3.3.8. - [Release notes](https://github.com/ai/nanoid/releases) - [Changelog](https://github.com/ai/nanoid/blob/main/CHANGELOG.md) - [Commits](https://github.com/ai/nanoid/compare/3.3.7...3.3.8) --- updated-dependencies: - dependency-name: nanoid dependency-type: indirect ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- web/ui/react-app/package-lock.json | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/web/ui/react-app/package-lock.json b/web/ui/react-app/package-lock.json index ff74defe6e..77283e7817 100644 --- a/web/ui/react-app/package-lock.json +++ b/web/ui/react-app/package-lock.json @@ -16413,9 +16413,9 @@ } }, "node_modules/nanoid": { - "version": "3.3.7", - "resolved": "https://registry.npmjs.org/nanoid/-/nanoid-3.3.7.tgz", - "integrity": "sha512-eSRppjcPIatRIMC1U6UngP8XFcz8MQWGQdt1MTBQ7NaAmvXDfvNxbvWV3x2y6CdEUciCSsDHDQZbhYaB8QEo2g==", + "version": "3.3.8", + "resolved": "https://registry.npmjs.org/nanoid/-/nanoid-3.3.8.tgz", + "integrity": "sha512-WNLf5Sd8oZxOm+TzppcYk8gVOgP+l58xNy58D0nbUnOxOWRWvlcCV4kUF7ltmI6PsrLl/BgKEyS4mqsGChFN0w==", "funding": [ { "type": "github", From 6f1669e017583a899675d11fefaca443c88f28cb Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 17 Jan 2025 16:29:16 +0000 Subject: [PATCH 098/110] chore(deps): bump golang.org/x/crypto (#15840) Bumps [golang.org/x/crypto](https://github.com/golang/crypto) from 0.30.0 to 0.31.0. - [Commits](https://github.com/golang/crypto/compare/v0.30.0...v0.31.0) --- updated-dependencies: - dependency-name: golang.org/x/crypto dependency-type: indirect ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- documentation/examples/remote_storage/go.mod | 2 +- documentation/examples/remote_storage/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/documentation/examples/remote_storage/go.mod b/documentation/examples/remote_storage/go.mod index d2b15dbe52..c1fb2e3564 100644 --- a/documentation/examples/remote_storage/go.mod +++ b/documentation/examples/remote_storage/go.mod @@ -55,7 +55,7 @@ require ( go.opentelemetry.io/otel/trace v1.27.0 // indirect go.uber.org/atomic v1.11.0 // indirect go.uber.org/multierr v1.11.0 // indirect - golang.org/x/crypto v0.30.0 // indirect + golang.org/x/crypto v0.31.0 // indirect golang.org/x/net v0.32.0 // indirect golang.org/x/oauth2 v0.24.0 // indirect golang.org/x/sys v0.28.0 // indirect diff --git a/documentation/examples/remote_storage/go.sum b/documentation/examples/remote_storage/go.sum index 860b9a944b..c360e28e76 100644 --- a/documentation/examples/remote_storage/go.sum +++ b/documentation/examples/remote_storage/go.sum @@ -323,8 +323,8 @@ golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnf golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/crypto v0.30.0 h1:RwoQn3GkWiMkzlX562cLB7OxWvjH1L8xutO2WoJcRoY= -golang.org/x/crypto v0.30.0/go.mod h1:kDsLvtWBEx7MV9tJOj9bnXsPbxwJQ6csT/x4KIN4Ssk= +golang.org/x/crypto v0.31.0 h1:ihbySMvVjLAeSH1IbfcRTkD/iNscyz8rGzjF/E5hV6U= +golang.org/x/crypto v0.31.0/go.mod h1:kDsLvtWBEx7MV9tJOj9bnXsPbxwJQ6csT/x4KIN4Ssk= golang.org/x/exp v0.0.0-20240119083558-1b970713d09a h1:Q8/wZp0KX97QFTc2ywcOE0YRjZPVIx+MXInMzdvQqcA= golang.org/x/exp v0.0.0-20240119083558-1b970713d09a/go.mod h1:idGWGoKP1toJGkd5/ig9ZLuPcZBC3ewk7SzmH0uou08= golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= From a75760c3d2d8384acc1edb017dbbb7f9a65b6366 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 17 Jan 2025 16:34:27 +0000 Subject: [PATCH 099/110] chore(deps-dev): bump webpack from 5.91.0 to 5.97.1 in /web/ui/react-app (#15843) Bumps [webpack](https://github.com/webpack/webpack) from 5.91.0 to 5.97.1. - [Release notes](https://github.com/webpack/webpack/releases) - [Commits](https://github.com/webpack/webpack/compare/v5.91.0...v5.97.1) --- updated-dependencies: - dependency-name: webpack dependency-type: indirect ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- web/ui/react-app/package-lock.json | 258 ++++++++++++++--------------- 1 file changed, 124 insertions(+), 134 deletions(-) diff --git a/web/ui/react-app/package-lock.json b/web/ui/react-app/package-lock.json index 77283e7817..c17bf60abf 100644 --- a/web/ui/react-app/package-lock.json +++ b/web/ui/react-app/package-lock.json @@ -5224,9 +5224,9 @@ } }, "node_modules/@types/estree": { - "version": "1.0.5", - "resolved": "https://registry.npmjs.org/@types/estree/-/estree-1.0.5.tgz", - "integrity": "sha512-/kYRxGDLWzHOB7q+wtSUQlFrtcdUccpfy+X+9iMBpHK8QLLhx2wIPYuS5DYtR9Wa/YlZAbIovy7qVdB1Aq6Lyw==", + "version": "1.0.6", + "resolved": "https://registry.npmjs.org/@types/estree/-/estree-1.0.6.tgz", + "integrity": "sha512-AYnb1nQyY49te+VRAVgmzfcgjYS91mY5P0TKUDCLEM+gNnA+3T6rWITXRLYCpahpqSQbN5cE+gHpnPyXjHWxcw==", "dev": true }, "node_modules/@types/express": { @@ -5866,148 +5866,148 @@ "dev": true }, "node_modules/@webassemblyjs/ast": { - "version": "1.12.1", - "resolved": "https://registry.npmjs.org/@webassemblyjs/ast/-/ast-1.12.1.tgz", - "integrity": "sha512-EKfMUOPRRUTy5UII4qJDGPpqfwjOmZ5jeGFwid9mnoqIFK+e0vqoi1qH56JpmZSzEL53jKnNzScdmftJyG5xWg==", + "version": "1.14.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/ast/-/ast-1.14.1.tgz", + "integrity": "sha512-nuBEDgQfm1ccRp/8bCQrx1frohyufl4JlbMMZ4P1wpeOfDhF6FQkxZJ1b/e+PLwr6X1Nhw6OLme5usuBWYBvuQ==", "dev": true, "dependencies": { - "@webassemblyjs/helper-numbers": "1.11.6", - "@webassemblyjs/helper-wasm-bytecode": "1.11.6" + "@webassemblyjs/helper-numbers": "1.13.2", + "@webassemblyjs/helper-wasm-bytecode": "1.13.2" } }, "node_modules/@webassemblyjs/floating-point-hex-parser": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/floating-point-hex-parser/-/floating-point-hex-parser-1.11.6.tgz", - "integrity": "sha512-ejAj9hfRJ2XMsNHk/v6Fu2dGS+i4UaXBXGemOfQ/JfQ6mdQg/WXtwleQRLLS4OvfDhv8rYnVwH27YJLMyYsxhw==", + "version": "1.13.2", + "resolved": "https://registry.npmjs.org/@webassemblyjs/floating-point-hex-parser/-/floating-point-hex-parser-1.13.2.tgz", + "integrity": "sha512-6oXyTOzbKxGH4steLbLNOu71Oj+C8Lg34n6CqRvqfS2O71BxY6ByfMDRhBytzknj9yGUPVJ1qIKhRlAwO1AovA==", "dev": true }, "node_modules/@webassemblyjs/helper-api-error": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-api-error/-/helper-api-error-1.11.6.tgz", - "integrity": "sha512-o0YkoP4pVu4rN8aTJgAyj9hC2Sv5UlkzCHhxqWj8butaLvnpdc2jOwh4ewE6CX0txSfLn/UYaV/pheS2Txg//Q==", + "version": "1.13.2", + "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-api-error/-/helper-api-error-1.13.2.tgz", + "integrity": "sha512-U56GMYxy4ZQCbDZd6JuvvNV/WFildOjsaWD3Tzzvmw/mas3cXzRJPMjP83JqEsgSbyrmaGjBfDtV7KDXV9UzFQ==", "dev": true }, "node_modules/@webassemblyjs/helper-buffer": { - "version": "1.12.1", - "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-buffer/-/helper-buffer-1.12.1.tgz", - "integrity": "sha512-nzJwQw99DNDKr9BVCOZcLuJJUlqkJh+kVzVl6Fmq/tI5ZtEyWT1KZMyOXltXLZJmDtvLCDgwsyrkohEtopTXCw==", + "version": "1.14.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-buffer/-/helper-buffer-1.14.1.tgz", + "integrity": "sha512-jyH7wtcHiKssDtFPRB+iQdxlDf96m0E39yb0k5uJVhFGleZFoNw1c4aeIcVUPPbXUVJ94wwnMOAqUHyzoEPVMA==", "dev": true }, "node_modules/@webassemblyjs/helper-numbers": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-numbers/-/helper-numbers-1.11.6.tgz", - "integrity": "sha512-vUIhZ8LZoIWHBohiEObxVm6hwP034jwmc9kuq5GdHZH0wiLVLIPcMCdpJzG4C11cHoQ25TFIQj9kaVADVX7N3g==", + "version": "1.13.2", + "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-numbers/-/helper-numbers-1.13.2.tgz", + "integrity": "sha512-FE8aCmS5Q6eQYcV3gI35O4J789wlQA+7JrqTTpJqn5emA4U2hvwJmvFRC0HODS+3Ye6WioDklgd6scJ3+PLnEA==", "dev": true, "dependencies": { - "@webassemblyjs/floating-point-hex-parser": "1.11.6", - "@webassemblyjs/helper-api-error": "1.11.6", + "@webassemblyjs/floating-point-hex-parser": "1.13.2", + "@webassemblyjs/helper-api-error": "1.13.2", "@xtuc/long": "4.2.2" } }, "node_modules/@webassemblyjs/helper-wasm-bytecode": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-wasm-bytecode/-/helper-wasm-bytecode-1.11.6.tgz", - "integrity": "sha512-sFFHKwcmBprO9e7Icf0+gddyWYDViL8bpPjJJl0WHxCdETktXdmtWLGVzoHbqUcY4Be1LkNfwTmXOJUFZYSJdA==", + "version": "1.13.2", + "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-wasm-bytecode/-/helper-wasm-bytecode-1.13.2.tgz", + "integrity": "sha512-3QbLKy93F0EAIXLh0ogEVR6rOubA9AoZ+WRYhNbFyuB70j3dRdwH9g+qXhLAO0kiYGlg3TxDV+I4rQTr/YNXkA==", "dev": true }, "node_modules/@webassemblyjs/helper-wasm-section": { - "version": "1.12.1", - "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-wasm-section/-/helper-wasm-section-1.12.1.tgz", - "integrity": "sha512-Jif4vfB6FJlUlSbgEMHUyk1j234GTNG9dBJ4XJdOySoj518Xj0oGsNi59cUQF4RRMS9ouBUxDDdyBVfPTypa5g==", + "version": "1.14.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-wasm-section/-/helper-wasm-section-1.14.1.tgz", + "integrity": "sha512-ds5mXEqTJ6oxRoqjhWDU83OgzAYjwsCV8Lo/N+oRsNDmx/ZDpqalmrtgOMkHwxsG0iI//3BwWAErYRHtgn0dZw==", "dev": true, "dependencies": { - "@webassemblyjs/ast": "1.12.1", - "@webassemblyjs/helper-buffer": "1.12.1", - "@webassemblyjs/helper-wasm-bytecode": "1.11.6", - "@webassemblyjs/wasm-gen": "1.12.1" + "@webassemblyjs/ast": "1.14.1", + "@webassemblyjs/helper-buffer": "1.14.1", + "@webassemblyjs/helper-wasm-bytecode": "1.13.2", + "@webassemblyjs/wasm-gen": "1.14.1" } }, "node_modules/@webassemblyjs/ieee754": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/ieee754/-/ieee754-1.11.6.tgz", - "integrity": "sha512-LM4p2csPNvbij6U1f19v6WR56QZ8JcHg3QIJTlSwzFcmx6WSORicYj6I63f9yU1kEUtrpG+kjkiIAkevHpDXrg==", + "version": "1.13.2", + "resolved": "https://registry.npmjs.org/@webassemblyjs/ieee754/-/ieee754-1.13.2.tgz", + "integrity": "sha512-4LtOzh58S/5lX4ITKxnAK2USuNEvpdVV9AlgGQb8rJDHaLeHciwG4zlGr0j/SNWlr7x3vO1lDEsuePvtcDNCkw==", "dev": true, "dependencies": { "@xtuc/ieee754": "^1.2.0" } }, "node_modules/@webassemblyjs/leb128": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/leb128/-/leb128-1.11.6.tgz", - "integrity": "sha512-m7a0FhE67DQXgouf1tbN5XQcdWoNgaAuoULHIfGFIEVKA6tu/edls6XnIlkmS6FrXAquJRPni3ZZKjw6FSPjPQ==", + "version": "1.13.2", + "resolved": "https://registry.npmjs.org/@webassemblyjs/leb128/-/leb128-1.13.2.tgz", + "integrity": "sha512-Lde1oNoIdzVzdkNEAWZ1dZ5orIbff80YPdHx20mrHwHrVNNTjNr8E3xz9BdpcGqRQbAEa+fkrCb+fRFTl/6sQw==", "dev": true, "dependencies": { "@xtuc/long": "4.2.2" } }, "node_modules/@webassemblyjs/utf8": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/utf8/-/utf8-1.11.6.tgz", - "integrity": "sha512-vtXf2wTQ3+up9Zsg8sa2yWiQpzSsMyXj0qViVP6xKGCUT8p8YJ6HqI7l5eCnWx1T/FYdsv07HQs2wTFbbof/RA==", + "version": "1.13.2", + "resolved": "https://registry.npmjs.org/@webassemblyjs/utf8/-/utf8-1.13.2.tgz", + "integrity": "sha512-3NQWGjKTASY1xV5m7Hr0iPeXD9+RDobLll3T9d2AO+g3my8xy5peVyjSag4I50mR1bBSN/Ct12lo+R9tJk0NZQ==", "dev": true }, "node_modules/@webassemblyjs/wasm-edit": { - "version": "1.12.1", - "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-edit/-/wasm-edit-1.12.1.tgz", - "integrity": "sha512-1DuwbVvADvS5mGnXbE+c9NfA8QRcZ6iKquqjjmR10k6o+zzsRVesil54DKexiowcFCPdr/Q0qaMgB01+SQ1u6g==", + "version": "1.14.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-edit/-/wasm-edit-1.14.1.tgz", + "integrity": "sha512-RNJUIQH/J8iA/1NzlE4N7KtyZNHi3w7at7hDjvRNm5rcUXa00z1vRz3glZoULfJ5mpvYhLybmVcwcjGrC1pRrQ==", "dev": true, "dependencies": { - "@webassemblyjs/ast": "1.12.1", - "@webassemblyjs/helper-buffer": "1.12.1", - "@webassemblyjs/helper-wasm-bytecode": "1.11.6", - "@webassemblyjs/helper-wasm-section": "1.12.1", - "@webassemblyjs/wasm-gen": "1.12.1", - "@webassemblyjs/wasm-opt": "1.12.1", - "@webassemblyjs/wasm-parser": "1.12.1", - "@webassemblyjs/wast-printer": "1.12.1" + "@webassemblyjs/ast": "1.14.1", + "@webassemblyjs/helper-buffer": "1.14.1", + "@webassemblyjs/helper-wasm-bytecode": "1.13.2", + "@webassemblyjs/helper-wasm-section": "1.14.1", + "@webassemblyjs/wasm-gen": "1.14.1", + "@webassemblyjs/wasm-opt": "1.14.1", + "@webassemblyjs/wasm-parser": "1.14.1", + "@webassemblyjs/wast-printer": "1.14.1" } }, "node_modules/@webassemblyjs/wasm-gen": { - "version": "1.12.1", - "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-gen/-/wasm-gen-1.12.1.tgz", - "integrity": "sha512-TDq4Ojh9fcohAw6OIMXqiIcTq5KUXTGRkVxbSo1hQnSy6lAM5GSdfwWeSxpAo0YzgsgF182E/U0mDNhuA0tW7w==", + "version": "1.14.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-gen/-/wasm-gen-1.14.1.tgz", + "integrity": "sha512-AmomSIjP8ZbfGQhumkNvgC33AY7qtMCXnN6bL2u2Js4gVCg8fp735aEiMSBbDR7UQIj90n4wKAFUSEd0QN2Ukg==", "dev": true, "dependencies": { - "@webassemblyjs/ast": "1.12.1", - "@webassemblyjs/helper-wasm-bytecode": "1.11.6", - "@webassemblyjs/ieee754": "1.11.6", - "@webassemblyjs/leb128": "1.11.6", - "@webassemblyjs/utf8": "1.11.6" + "@webassemblyjs/ast": "1.14.1", + "@webassemblyjs/helper-wasm-bytecode": "1.13.2", + "@webassemblyjs/ieee754": "1.13.2", + "@webassemblyjs/leb128": "1.13.2", + "@webassemblyjs/utf8": "1.13.2" } }, "node_modules/@webassemblyjs/wasm-opt": { - "version": "1.12.1", - "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-opt/-/wasm-opt-1.12.1.tgz", - "integrity": "sha512-Jg99j/2gG2iaz3hijw857AVYekZe2SAskcqlWIZXjji5WStnOpVoat3gQfT/Q5tb2djnCjBtMocY/Su1GfxPBg==", + "version": "1.14.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-opt/-/wasm-opt-1.14.1.tgz", + "integrity": "sha512-PTcKLUNvBqnY2U6E5bdOQcSM+oVP/PmrDY9NzowJjislEjwP/C4an2303MCVS2Mg9d3AJpIGdUFIQQWbPds0Sw==", "dev": true, "dependencies": { - "@webassemblyjs/ast": "1.12.1", - "@webassemblyjs/helper-buffer": "1.12.1", - "@webassemblyjs/wasm-gen": "1.12.1", - "@webassemblyjs/wasm-parser": "1.12.1" + "@webassemblyjs/ast": "1.14.1", + "@webassemblyjs/helper-buffer": "1.14.1", + "@webassemblyjs/wasm-gen": "1.14.1", + "@webassemblyjs/wasm-parser": "1.14.1" } }, "node_modules/@webassemblyjs/wasm-parser": { - "version": "1.12.1", - "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-parser/-/wasm-parser-1.12.1.tgz", - "integrity": "sha512-xikIi7c2FHXysxXe3COrVUPSheuBtpcfhbpFj4gmu7KRLYOzANztwUU0IbsqvMqzuNK2+glRGWCEqZo1WCLyAQ==", + "version": "1.14.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-parser/-/wasm-parser-1.14.1.tgz", + "integrity": "sha512-JLBl+KZ0R5qB7mCnud/yyX08jWFw5MsoalJ1pQ4EdFlgj9VdXKGuENGsiCIjegI1W7p91rUlcB/LB5yRJKNTcQ==", "dev": true, "dependencies": { - "@webassemblyjs/ast": "1.12.1", - "@webassemblyjs/helper-api-error": "1.11.6", - "@webassemblyjs/helper-wasm-bytecode": "1.11.6", - "@webassemblyjs/ieee754": "1.11.6", - "@webassemblyjs/leb128": "1.11.6", - "@webassemblyjs/utf8": "1.11.6" + "@webassemblyjs/ast": "1.14.1", + "@webassemblyjs/helper-api-error": "1.13.2", + "@webassemblyjs/helper-wasm-bytecode": "1.13.2", + "@webassemblyjs/ieee754": "1.13.2", + "@webassemblyjs/leb128": "1.13.2", + "@webassemblyjs/utf8": "1.13.2" } }, "node_modules/@webassemblyjs/wast-printer": { - "version": "1.12.1", - "resolved": "https://registry.npmjs.org/@webassemblyjs/wast-printer/-/wast-printer-1.12.1.tgz", - "integrity": "sha512-+X4WAlOisVWQMikjbcvY2e0rwPsKQ9F688lksZhBcPycBBuii3O7m8FACbDMWDojpAqvjIncrG8J0XHKyQfVeA==", + "version": "1.14.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wast-printer/-/wast-printer-1.14.1.tgz", + "integrity": "sha512-kPSSXE6De1XOR820C90RIo2ogvZG+c3KiHzqUoO/F34Y2shGzesfqv7o57xrxovZJH/MetF5UjroJ/R/3isoiw==", "dev": true, "dependencies": { - "@webassemblyjs/ast": "1.12.1", + "@webassemblyjs/ast": "1.14.1", "@xtuc/long": "4.2.2" } }, @@ -6084,9 +6084,9 @@ } }, "node_modules/acorn": { - "version": "8.11.3", - "resolved": "https://registry.npmjs.org/acorn/-/acorn-8.11.3.tgz", - "integrity": "sha512-Y9rRfJG5jcKOE0CLisYbojUjIrIEE7AGMzA/Sm4BslANhbS+cDMpgBdcPT91oJ7OuJ9hYJBx59RjbhxVnrF8Xg==", + "version": "8.14.0", + "resolved": "https://registry.npmjs.org/acorn/-/acorn-8.14.0.tgz", + "integrity": "sha512-cl669nCJTZBsL97OF4kUQm5g5hC2uihk0NxY3WENAC0TYdILVkAyHymAntgxGkl7K+t0cXIrH5siy5S4XkFycA==", "dev": true, "bin": { "acorn": "bin/acorn" @@ -6117,15 +6117,6 @@ "node": ">=0.4.0" } }, - "node_modules/acorn-import-assertions": { - "version": "1.9.0", - "resolved": "https://registry.npmjs.org/acorn-import-assertions/-/acorn-import-assertions-1.9.0.tgz", - "integrity": "sha512-cmMwop9x+8KFhxvKrKfPYmN6/pKTYYHBqLa0DfvVZcKMJWNyWLnaqND7dx/qn66R7ewM1UX5XMaDVP5wlVTaVA==", - "dev": true, - "peerDependencies": { - "acorn": "^8" - } - }, "node_modules/acorn-jsx": { "version": "5.3.2", "resolved": "https://registry.npmjs.org/acorn-jsx/-/acorn-jsx-5.3.2.tgz", @@ -7115,9 +7106,9 @@ "dev": true }, "node_modules/browserslist": { - "version": "4.23.0", - "resolved": "https://registry.npmjs.org/browserslist/-/browserslist-4.23.0.tgz", - "integrity": "sha512-QW8HiM1shhT2GuzkvklfjcKDiWFXHOeFCIA/huJPwHsslwcydgk7X+z2zXpEijP98UCY7HbubZt5J2Zgvf0CaQ==", + "version": "4.24.4", + "resolved": "https://registry.npmjs.org/browserslist/-/browserslist-4.24.4.tgz", + "integrity": "sha512-KDi1Ny1gSePi1vm0q4oxSF8b4DR44GF4BbmS2YdhPLOEqd8pDviZOGH/GsmRwoWJ2+5Lr085X7naowMwKHDG1A==", "dev": true, "funding": [ { @@ -7134,10 +7125,10 @@ } ], "dependencies": { - "caniuse-lite": "^1.0.30001587", - "electron-to-chromium": "^1.4.668", - "node-releases": "^2.0.14", - "update-browserslist-db": "^1.0.13" + "caniuse-lite": "^1.0.30001688", + "electron-to-chromium": "^1.5.73", + "node-releases": "^2.0.19", + "update-browserslist-db": "^1.1.1" }, "bin": { "browserslist": "cli.js" @@ -7265,9 +7256,9 @@ } }, "node_modules/caniuse-lite": { - "version": "1.0.30001610", - "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001610.tgz", - "integrity": "sha512-QFutAY4NgaelojVMjY63o6XlZyORPaLfyMnsl3HgnWdJUcX6K0oaJymHjH8PT5Gk7sTm8rvC/c5COUQKXqmOMA==", + "version": "1.0.30001692", + "resolved": "https://registry.npmjs.org/caniuse-lite/-/caniuse-lite-1.0.30001692.tgz", + "integrity": "sha512-A95VKan0kdtrsnMubMKxEKUKImOPSuCpYgxSQBo036P5YYgVIcOYJEgt/txJWqObiRQeISNCfef9nvlQ0vbV7A==", "dev": true, "funding": [ { @@ -8906,9 +8897,9 @@ } }, "node_modules/electron-to-chromium": { - "version": "1.4.736", - "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.4.736.tgz", - "integrity": "sha512-Rer6wc3ynLelKNM4lOCg7/zPQj8tPOCB2hzD32PX9wd3hgRRi9MxEbmkFCokzcEhRVMiOVLjnL9ig9cefJ+6+Q==", + "version": "1.5.83", + "resolved": "https://registry.npmjs.org/electron-to-chromium/-/electron-to-chromium-1.5.83.tgz", + "integrity": "sha512-LcUDPqSt+V0QmI47XLzZrz5OqILSMGsPFkDYus22rIbgorSvBYEFqq854ltTmUdHkY92FSdAAvsh4jWEULMdfQ==", "dev": true }, "node_modules/emittery": { @@ -8949,9 +8940,9 @@ } }, "node_modules/enhanced-resolve": { - "version": "5.16.0", - "resolved": "https://registry.npmjs.org/enhanced-resolve/-/enhanced-resolve-5.16.0.tgz", - "integrity": "sha512-O+QWCviPNSSLAD9Ucn8Awv+poAkqn3T1XY5/N7kR7rQO9yfSGWkYZDwpJ+iKF7B8rxaQKWngSqACpgzeapSyoA==", + "version": "5.18.0", + "resolved": "https://registry.npmjs.org/enhanced-resolve/-/enhanced-resolve-5.18.0.tgz", + "integrity": "sha512-0/r0MySGYG8YqlayBZ6MuCfECmHFdJ5qyPh8s8wa5Hnm6SaFLSK1VYCbj+NKp090Nm1caZhD+QTnmxO7esYGyQ==", "dev": true, "dependencies": { "graceful-fs": "^4.2.4", @@ -9228,9 +9219,9 @@ } }, "node_modules/escalade": { - "version": "3.1.2", - "resolved": "https://registry.npmjs.org/escalade/-/escalade-3.1.2.tgz", - "integrity": "sha512-ErCHMCae19vR8vQGe50xIsVomy19rg6gFu3+r3jkEO46suLMWBksvVyoGgQV+jOfl84ZSOSlmv6Gxa89PmTGmA==", + "version": "3.2.0", + "resolved": "https://registry.npmjs.org/escalade/-/escalade-3.2.0.tgz", + "integrity": "sha512-WUj2qlxaQtO4g6Pq5c29GTcWGDyd8itL8zTlipgECz3JesAiiOKotd8JU6otB3PACgG6xkJUyVhboMS+bje/jA==", "dev": true, "engines": { "node": ">=6" @@ -16555,9 +16546,9 @@ "dev": true }, "node_modules/node-releases": { - "version": "2.0.14", - "resolved": "https://registry.npmjs.org/node-releases/-/node-releases-2.0.14.tgz", - "integrity": "sha512-y10wOWt8yZpqXmOgRo77WaHEmhYQYGNA6y421PKsKYWEK8aW+cqAphborZDhqfyKrbZEN92CN1X2KbafY2s7Yw==", + "version": "2.0.19", + "resolved": "https://registry.npmjs.org/node-releases/-/node-releases-2.0.19.tgz", + "integrity": "sha512-xxOWJsBKtzAq7DY0J+DTzuz58K8e7sJbdgwkbMWQe8UYB6ekmsQ45q0M/tJDsGaZmbC+l7n57UV8Hl5tHxO9uw==", "dev": true }, "node_modules/normalize-path": { @@ -17096,9 +17087,9 @@ "dev": true }, "node_modules/picocolors": { - "version": "1.0.0", - "resolved": "https://registry.npmjs.org/picocolors/-/picocolors-1.0.0.tgz", - "integrity": "sha512-1fygroTLlHu66zi26VoTDv8yRgm0Fccecssto+MhsZ0D/DGW2sm8E8AjW7NU5VVTRt5GxbeZ5qBuJr+HyLYkjQ==" + "version": "1.1.1", + "resolved": "https://registry.npmjs.org/picocolors/-/picocolors-1.1.1.tgz", + "integrity": "sha512-xceH2snhtb5M9liqDsmEw56le376mTZkEX/jEb/RxNFyegNul7eNslCXP9FDj/Lcu0X8KEyMceP2ntpaHrDEVA==" }, "node_modules/picomatch": { "version": "2.3.1", @@ -23165,9 +23156,9 @@ } }, "node_modules/update-browserslist-db": { - "version": "1.0.13", - "resolved": "https://registry.npmjs.org/update-browserslist-db/-/update-browserslist-db-1.0.13.tgz", - "integrity": "sha512-xebP81SNcPuNpPP3uzeW1NYXxI3rxyJzF3pD6sH4jE7o/IX+WtSpwnVU+qIsDPyk0d3hmFQ7mjqc6AtV604hbg==", + "version": "1.1.2", + "resolved": "https://registry.npmjs.org/update-browserslist-db/-/update-browserslist-db-1.1.2.tgz", + "integrity": "sha512-PPypAm5qvlD7XMZC3BujecnaOxwhrtoFR+Dqkk5Aa/6DssiH0ibKoketaj9w8LP7Bont1rYeoV5plxD7RTEPRg==", "dev": true, "funding": [ { @@ -23184,8 +23175,8 @@ } ], "dependencies": { - "escalade": "^3.1.1", - "picocolors": "^1.0.0" + "escalade": "^3.2.0", + "picocolors": "^1.1.1" }, "bin": { "update-browserslist-db": "cli.js" @@ -23360,21 +23351,20 @@ "dev": true }, "node_modules/webpack": { - "version": "5.91.0", - "resolved": "https://registry.npmjs.org/webpack/-/webpack-5.91.0.tgz", - "integrity": "sha512-rzVwlLeBWHJbmgTC/8TvAcu5vpJNII+MelQpylD4jNERPwpBJOE2lEcko1zJX3QJeLjTTAnQxn/OJ8bjDzVQaw==", + "version": "5.97.1", + "resolved": "https://registry.npmjs.org/webpack/-/webpack-5.97.1.tgz", + "integrity": "sha512-EksG6gFY3L1eFMROS/7Wzgrii5mBAFe4rIr3r2BTfo7bcc+DWwFZ4OJ/miOuHJO/A85HwyI4eQ0F6IKXesO7Fg==", "dev": true, "dependencies": { - "@types/eslint-scope": "^3.7.3", - "@types/estree": "^1.0.5", - "@webassemblyjs/ast": "^1.12.1", - "@webassemblyjs/wasm-edit": "^1.12.1", - "@webassemblyjs/wasm-parser": "^1.12.1", - "acorn": "^8.7.1", - "acorn-import-assertions": "^1.9.0", - "browserslist": "^4.21.10", + "@types/eslint-scope": "^3.7.7", + "@types/estree": "^1.0.6", + "@webassemblyjs/ast": "^1.14.1", + "@webassemblyjs/wasm-edit": "^1.14.1", + "@webassemblyjs/wasm-parser": "^1.14.1", + "acorn": "^8.14.0", + "browserslist": "^4.24.0", "chrome-trace-event": "^1.0.2", - "enhanced-resolve": "^5.16.0", + "enhanced-resolve": "^5.17.1", "es-module-lexer": "^1.2.1", "eslint-scope": "5.1.1", "events": "^3.2.0", From 3a5bf37d4bfcb8f19f6b22e0ea89959300f7ac6c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 17 Jan 2025 16:51:12 +0000 Subject: [PATCH 100/110] chore(deps): bump github.com/envoyproxy/go-control-plane (#15773) Bumps [github.com/envoyproxy/go-control-plane](https://github.com/envoyproxy/go-control-plane) from 0.13.1 to 0.13.2. - [Release notes](https://github.com/envoyproxy/go-control-plane/releases) - [Changelog](https://github.com/envoyproxy/go-control-plane/blob/main/CHANGELOG.md) - [Commits](https://github.com/envoyproxy/go-control-plane/compare/v0.13.1...v0.13.2) --- updated-dependencies: - dependency-name: github.com/envoyproxy/go-control-plane dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Signed-off-by: Arve Knudsen Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index 98075a56fa..5065092f2f 100644 --- a/go.mod +++ b/go.mod @@ -20,7 +20,7 @@ require ( github.com/digitalocean/godo v1.132.0 github.com/docker/docker v27.4.1+incompatible github.com/edsrzf/mmap-go v1.2.0 - github.com/envoyproxy/go-control-plane v0.13.1 + github.com/envoyproxy/go-control-plane/envoy v1.32.2 github.com/envoyproxy/protoc-gen-validate v1.1.0 github.com/facette/natsort v0.0.0-20181210072756-2cd4dd1e2dcb github.com/fsnotify/fsnotify v1.8.0 diff --git a/go.sum b/go.sum index 009b714060..8dafac6b17 100644 --- a/go.sum +++ b/go.sum @@ -110,8 +110,8 @@ github.com/emicklei/go-restful/v3 v3.11.0/go.mod h1:6n3XBCmQQb25CM2LCACGz8ukIrRr github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= -github.com/envoyproxy/go-control-plane v0.13.1 h1:vPfJZCkob6yTMEgS+0TwfTUfbHjfy/6vOJ8hUWX/uXE= -github.com/envoyproxy/go-control-plane v0.13.1/go.mod h1:X45hY0mufo6Fd0KW3rqsGvQMw58jvjymeCzBU3mWyHw= +github.com/envoyproxy/go-control-plane/envoy v1.32.2 h1:zidqwmijfcbyKqVxjQDFx042PgX+p9U+/fu/f9VtSk8= +github.com/envoyproxy/go-control-plane/envoy v1.32.2/go.mod h1:eR2SOX2IedqlPvmiKjUH7Wu//S602JKI7HPC/L3SRq8= github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/envoyproxy/protoc-gen-validate v1.1.0 h1:tntQDh69XqOCOZsDz0lVJQez/2L6Uu2PdjCQwWCJ3bM= github.com/envoyproxy/protoc-gen-validate v1.1.0/go.mod h1:sXRDRVmzEbkM7CVcM06s9shE/m23dg3wzjl0UWqJ2q4= From 9e14669c49a05cc1c814c9a9535d0c451f88e963 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 17 Jan 2025 16:57:29 +0000 Subject: [PATCH 101/110] chore(deps): bump golang.org/x/net (#15841) Bumps [golang.org/x/net](https://github.com/golang/net) from 0.32.0 to 0.33.0. - [Commits](https://github.com/golang/net/compare/v0.32.0...v0.33.0) --- updated-dependencies: - dependency-name: golang.org/x/net dependency-type: indirect ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- documentation/examples/remote_storage/go.mod | 2 +- documentation/examples/remote_storage/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/documentation/examples/remote_storage/go.mod b/documentation/examples/remote_storage/go.mod index c1fb2e3564..643fbc901c 100644 --- a/documentation/examples/remote_storage/go.mod +++ b/documentation/examples/remote_storage/go.mod @@ -56,7 +56,7 @@ require ( go.uber.org/atomic v1.11.0 // indirect go.uber.org/multierr v1.11.0 // indirect golang.org/x/crypto v0.31.0 // indirect - golang.org/x/net v0.32.0 // indirect + golang.org/x/net v0.33.0 // indirect golang.org/x/oauth2 v0.24.0 // indirect golang.org/x/sys v0.28.0 // indirect golang.org/x/text v0.21.0 // indirect diff --git a/documentation/examples/remote_storage/go.sum b/documentation/examples/remote_storage/go.sum index c360e28e76..23ac21c12c 100644 --- a/documentation/examples/remote_storage/go.sum +++ b/documentation/examples/remote_storage/go.sum @@ -344,8 +344,8 @@ golang.org/x/net v0.0.0-20200822124328-c89045814202/go.mod h1:/O7V0waA8r7cgGh81R golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20210525063256-abc453219eb5/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.32.0 h1:ZqPmj8Kzc+Y6e0+skZsuACbx+wzMgo5MQsJh9Qd6aYI= -golang.org/x/net v0.32.0/go.mod h1:CwU0IoeOlnQQWJ6ioyFrfRuomB8GKF6KbYXZVyeXNfs= +golang.org/x/net v0.33.0 h1:74SYHlV8BIgHIFC/LrYkOGIwL19eTYXQ5wc6TBuO36I= +golang.org/x/net v0.33.0/go.mod h1:HXLR5J+9DxmrqMwG9qjGCxZ+zKXxBru04zlTvWlWuN4= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20210514164344-f6687ab2804c/go.mod h1:KelEdhl1UZF7XfJ4dDtk6s++YSgaE7mD/BuKKDLBl4A= golang.org/x/oauth2 v0.24.0 h1:KTBBxWqUa0ykRPLtV69rRto9TLXcqYkeswu48x/gvNE= From 5bf7c3718bd08a0b3598badd237b5b91949fc1ae Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 17 Jan 2025 16:59:06 +0000 Subject: [PATCH 102/110] chore(deps): bump github.com/hashicorp/consul/api from 1.30.0 to 1.31.0 (#15766) Bumps [github.com/hashicorp/consul/api](https://github.com/hashicorp/consul) from 1.30.0 to 1.31.0. - [Release notes](https://github.com/hashicorp/consul/releases) - [Changelog](https://github.com/hashicorp/consul/blob/main/CHANGELOG.md) - [Commits](https://github.com/hashicorp/consul/compare/api/v1.30.0...api/v1.31.0) --- updated-dependencies: - dependency-name: github.com/hashicorp/consul/api dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index 5065092f2f..3eaca8f35d 100644 --- a/go.mod +++ b/go.mod @@ -34,7 +34,7 @@ require ( github.com/gophercloud/gophercloud v1.14.1 github.com/grafana/regexp v0.0.0-20240518133315-a468a5bfb3bc github.com/grpc-ecosystem/grpc-gateway v1.16.0 - github.com/hashicorp/consul/api v1.30.0 + github.com/hashicorp/consul/api v1.31.0 github.com/hashicorp/nomad/api v0.0.0-20241218080744-e3ac00f30eec github.com/hetznercloud/hcloud-go/v2 v2.17.1 github.com/ionos-cloud/sdk-go/v6 v6.3.0 diff --git a/go.sum b/go.sum index 8dafac6b17..d26f4079c5 100644 --- a/go.sum +++ b/go.sum @@ -227,8 +227,8 @@ github.com/grpc-ecosystem/grpc-gateway v1.16.0 h1:gmcG1KaJ57LophUzW0Hy8NmPhnMZb4 github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw= github.com/grpc-ecosystem/grpc-gateway/v2 v2.24.0 h1:TmHmbvxPmaegwhDubVz0lICL0J5Ka2vwTzhoePEXsGE= github.com/grpc-ecosystem/grpc-gateway/v2 v2.24.0/go.mod h1:qztMSjm835F2bXf+5HKAPIS5qsmQDqZna/PgVt4rWtI= -github.com/hashicorp/consul/api v1.30.0 h1:ArHVMMILb1nQv8vZSGIwwQd2gtc+oSQZ6CalyiyH2XQ= -github.com/hashicorp/consul/api v1.30.0/go.mod h1:B2uGchvaXVW2JhFoS8nqTxMD5PBykr4ebY4JWHTTeLM= +github.com/hashicorp/consul/api v1.31.0 h1:32BUNLembeSRek0G/ZAM6WNfdEwYdYo8oQ4+JoqGkNQ= +github.com/hashicorp/consul/api v1.31.0/go.mod h1:2ZGIiXM3A610NmDULmCHd/aqBJj8CkMfOhswhOafxRg= github.com/hashicorp/consul/sdk v0.16.1 h1:V8TxTnImoPD5cj0U9Spl0TUxcytjcbbJeADFF07KdHg= github.com/hashicorp/consul/sdk v0.16.1/go.mod h1:fSXvwxB2hmh1FMZCNl6PwX0Q/1wdWtHJcZ7Ea5tns0s= github.com/hashicorp/cronexpr v1.1.2 h1:wG/ZYIKT+RT3QkOdgYc+xsKWVRgnxJ1OJtjjy84fJ9A= From f405c78e4abebda5e7a462754f9d0abc4b5e222d Mon Sep 17 00:00:00 2001 From: Owen Williams Date: Fri, 17 Jan 2025 12:00:30 -0500 Subject: [PATCH 103/110] docs: clarify PromQL interval changes (#15824) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit docs: Clarify PromQL interval changes --------- Signed-off-by: Owen Williams Signed-off-by: Owen Williams Co-authored-by: Björn Rabenstein --- docs/migration.md | 87 +++++++++++++++++++++++++++++++---------------- 1 file changed, 57 insertions(+), 30 deletions(-) diff --git a/docs/migration.md b/docs/migration.md index e2d53472f3..34dae93e85 100644 --- a/docs/migration.md +++ b/docs/migration.md @@ -60,36 +60,63 @@ This document offers guidance on migrating from Prometheus 2.x to Prometheus 3.0 ## PromQL -- The `.` pattern in regular expressions in PromQL matches newline characters. - With this change a regular expressions like `.*` matches strings that include - `\n`. This applies to matchers in queries and relabel configs. - - For example, the following regular expressions now match the accompanying - strings, whereas in Prometheus v2 these combinations didn't match. - - `.*` additionally matches `foo\n` and `Foo\nBar` - - `foo.?bar` additionally matches `foo\nbar` - - `foo.+bar` additionally matches `foo\nbar` - - If you want Prometheus v3 to behave like v2, you will have to change your - regular expressions by replacing all `.` patterns with `[^\n]`, e.g. - `foo[^\n]*`. -- Lookback and range selectors are left open and right closed (previously left - closed and right closed). This change affects queries when the evaluation time - perfectly aligns with the sample timestamps. For example assume querying a - timeseries with evenly spaced samples exactly 1 minute apart. Before Prometheus - v3, a range query with `5m` would usually return 5 samples. But if the query - evaluation aligns perfectly with a scrape, it would return 6 samples. In - Prometheus v3 queries like this will always return 5 samples. - This change has likely few effects for everyday use, except for some subquery - use cases. - Query front-ends that align queries usually align subqueries to multiples of - the step size. These subqueries will likely be affected. - Tests are more likely to affected. To fix those either adjust the expected - number of samples or extend the range by less than one sample interval. -- The `holt_winters` function has been renamed to `double_exponential_smoothing` - and is now guarded by the `promql-experimental-functions` feature flag. - If you want to keep using `holt_winters`, you have to do both of these things: - - Rename `holt_winters` to `double_exponential_smoothing` in your queries. - - Pass `--enable-feature=promql-experimental-functions` in your Prometheus - CLI invocation. +### Regular expressions match newlines + +The `.` pattern in regular expressions in PromQL matches newline characters. +With this change a regular expressions like `.*` matches strings that include +`\n`. This applies to matchers in queries and relabel configs. + +For example, the following regular expressions now match the accompanying +strings, whereas in Prometheus v2 these combinations didn't match. + - `.*` additionally matches `foo\n` and `Foo\nBar` + - `foo.?bar` additionally matches `foo\nbar` + - `foo.+bar` additionally matches `foo\nbar` + +If you want Prometheus v3 to behave like v2, you will have to change your +regular expressions by replacing all `.` patterns with `[^\n]`, e.g. +`foo[^\n]*`. + +### Range selectors and lookback exclude samples coinciding with the left boundary + +Lookback and range selectors are now left-open and right-closed (previously +left-closed and right-closed), which makes their behavior more consistent. This +change affects queries where the left boundary of a range or the lookback delta +coincides with the timestamp of one or more samples. + +For example, assume we are querying a timeseries with evenly spaced samples +exactly 1 minute apart. Before Prometheus v3, a range query with `5m` would +usually return 5 samples. But if the query evaluation aligns perfectly with a +scrape, it would return 6 samples. In Prometheus v3 queries like this will +always return 5 samples given even spacing. + +This change will typically affect subqueries because their evaluation timing is +naturally perfectly evenly spaced and aligned with timestamps that are multiples +of the subquery resolution. Furthermore, query frontends often align subqueries +to multiples of the step size. In combination, this easily creates a situation +of perfect mutual alignment, often unintended and unknown by the user, so that +the new behavior might come as a surprise. Before Prometheus V3, a subquery of +`foo[1m:1m]` on such a system might have always returned two points, allowing +for rate calculations. In Prometheus V3, however, such a subquery will only +return one point, which is insufficient for a rate or increase calculation, +resulting in No Data returned. + +Such queries will need to be rewritten to extend the window to properly cover +more than one point. In this example, `foo[2m:1m]` would always return two +points no matter the query alignment. The exact form of the rewritten query may +depend on the intended results and there is no universal drop-in replacement for +queries whose behavior has changed. + +Tests are similarly more likely to affected. To fix those either adjust the +expected number of samples or extend the range. + +### holt_winters function renamed + +The `holt_winters` function has been renamed to `double_exponential_smoothing` +and is now guarded by the `promql-experimental-functions` feature flag. +If you want to keep using `holt_winters`, you have to do both of these things: + - Rename `holt_winters` to `double_exponential_smoothing` in your queries. + - Pass `--enable-feature=promql-experimental-functions` in your Prometheus + CLI invocation. ## Scrape protocols Prometheus v3 is more strict concerning the Content-Type header received when From c47465c0c529ee8c2f6aac979ebd0b4f55b1a891 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 17 Jan 2025 17:24:26 +0000 Subject: [PATCH 104/110] chore(deps): bump nanoid from 3.3.7 to 3.3.8 in /web/ui (#15844) Bumps [nanoid](https://github.com/ai/nanoid) from 3.3.7 to 3.3.8. - [Release notes](https://github.com/ai/nanoid/releases) - [Changelog](https://github.com/ai/nanoid/blob/main/CHANGELOG.md) - [Commits](https://github.com/ai/nanoid/compare/3.3.7...3.3.8) --- updated-dependencies: - dependency-name: nanoid dependency-type: indirect ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- web/ui/package-lock.json | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/web/ui/package-lock.json b/web/ui/package-lock.json index f9beb96da0..07b35914d3 100644 --- a/web/ui/package-lock.json +++ b/web/ui/package-lock.json @@ -7080,16 +7080,15 @@ "license": "MIT" }, "node_modules/nanoid": { - "version": "3.3.7", - "resolved": "https://registry.npmjs.org/nanoid/-/nanoid-3.3.7.tgz", - "integrity": "sha512-eSRppjcPIatRIMC1U6UngP8XFcz8MQWGQdt1MTBQ7NaAmvXDfvNxbvWV3x2y6CdEUciCSsDHDQZbhYaB8QEo2g==", + "version": "3.3.8", + "resolved": "https://registry.npmjs.org/nanoid/-/nanoid-3.3.8.tgz", + "integrity": "sha512-WNLf5Sd8oZxOm+TzppcYk8gVOgP+l58xNy58D0nbUnOxOWRWvlcCV4kUF7ltmI6PsrLl/BgKEyS4mqsGChFN0w==", "funding": [ { "type": "github", "url": "https://github.com/sponsors/ai" } ], - "license": "MIT", "bin": { "nanoid": "bin/nanoid.cjs" }, From 5e2b75ee5e8da0029362faa4bead50f1fdd05a24 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 17 Jan 2025 18:16:30 +0000 Subject: [PATCH 105/110] chore(deps-dev): bump @typescript-eslint/parser from 6.21.0 to 8.20.0 in /web/ui (#15821) * chore(deps-dev): bump @typescript-eslint/parser in /web/ui Bumps [@typescript-eslint/parser](https://github.com/typescript-eslint/typescript-eslint/tree/HEAD/packages/parser) from 6.21.0 to 8.20.0. - [Release notes](https://github.com/typescript-eslint/typescript-eslint/releases) - [Changelog](https://github.com/typescript-eslint/typescript-eslint/blob/main/packages/parser/CHANGELOG.md) - [Commits](https://github.com/typescript-eslint/typescript-eslint/commits/v8.20.0/packages/parser) --- updated-dependencies: - dependency-name: "@typescript-eslint/parser" dependency-type: direct:development update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Signed-off-by: Arve Knudsen * Fix lint errors/warnings Signed-off-by: Arve Knudsen --------- Signed-off-by: dependabot[bot] Signed-off-by: Arve Knudsen Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Arve Knudsen --- web/ui/mantine-ui/package.json | 4 +- .../src/components/EndpointLink.tsx | 2 +- .../src/components/ReadinessWrapper.tsx | 2 +- web/ui/mantine-ui/src/pages/query/Graph.tsx | 4 +- .../mantine-ui/src/pages/query/TableTab.tsx | 4 +- .../mantine-ui/src/pages/query/TreeNode.tsx | 16 +- .../src/pages/query/uPlotStackHelpers.ts | 7 +- .../ServiceDiscoveryPage.tsx | 4 +- .../src/pages/targets/TargetsPage.tsx | 4 +- web/ui/package-lock.json | 306 +++++++----------- web/ui/package.json | 4 +- .../src/pages/graph/DataTable.test.tsx | 2 +- 12 files changed, 141 insertions(+), 218 deletions(-) diff --git a/web/ui/mantine-ui/package.json b/web/ui/mantine-ui/package.json index 9cdb67aea5..9d4afdfae5 100644 --- a/web/ui/mantine-ui/package.json +++ b/web/ui/mantine-ui/package.json @@ -56,8 +56,8 @@ "@eslint/js": "^9.17.0", "@types/react": "^18.3.5", "@types/react-dom": "^18.3.0", - "@typescript-eslint/eslint-plugin": "^6.21.0", - "@typescript-eslint/parser": "^6.21.0", + "@typescript-eslint/eslint-plugin": "^8.20.0", + "@typescript-eslint/parser": "^8.20.0", "@vitejs/plugin-react": "^4.3.4", "eslint": "^9.18.0", "eslint-plugin-react-hooks": "^5.1.0", diff --git a/web/ui/mantine-ui/src/components/EndpointLink.tsx b/web/ui/mantine-ui/src/components/EndpointLink.tsx index c9b6a8989c..7e56a71abe 100644 --- a/web/ui/mantine-ui/src/components/EndpointLink.tsx +++ b/web/ui/mantine-ui/src/components/EndpointLink.tsx @@ -12,7 +12,7 @@ const EndpointLink: FC = ({ endpoint, globalUrl }) => { let invalidURL = false; try { url = new URL(endpoint); - } catch (err: unknown) { + } catch (_: unknown) { // In cases of IPv6 addresses with a Zone ID, URL may not be parseable. // See https://github.com/prometheus/prometheus/issues/9760 // In this case, we attempt to prepare a synthetic URL with the diff --git a/web/ui/mantine-ui/src/components/ReadinessWrapper.tsx b/web/ui/mantine-ui/src/components/ReadinessWrapper.tsx index 2e471de5e3..3be002e258 100644 --- a/web/ui/mantine-ui/src/components/ReadinessWrapper.tsx +++ b/web/ui/mantine-ui/src/components/ReadinessWrapper.tsx @@ -48,7 +48,7 @@ const ReadinessLoader: FC = () => { default: throw new Error(res.statusText); } - } catch (error) { + } catch (_) { throw new Error("Unexpected error while fetching ready status"); } }, diff --git a/web/ui/mantine-ui/src/pages/query/Graph.tsx b/web/ui/mantine-ui/src/pages/query/Graph.tsx index c25d5a98eb..b461c4ff06 100644 --- a/web/ui/mantine-ui/src/pages/query/Graph.tsx +++ b/web/ui/mantine-ui/src/pages/query/Graph.tsx @@ -102,7 +102,9 @@ const Graph: FC = ({ // Re-execute the query when the user presses Enter (or hits the Execute button). useEffect(() => { - effectiveExpr !== "" && refetch(); + if (effectiveExpr !== "") { + refetch(); + } }, [retriggerIdx, refetch, effectiveExpr, endTime, range, resolution]); // The useElementSize hook above only gets a valid size on the second render, so this diff --git a/web/ui/mantine-ui/src/pages/query/TableTab.tsx b/web/ui/mantine-ui/src/pages/query/TableTab.tsx index cab93e229b..5e2f804243 100644 --- a/web/ui/mantine-ui/src/pages/query/TableTab.tsx +++ b/web/ui/mantine-ui/src/pages/query/TableTab.tsx @@ -40,7 +40,9 @@ const TableTab: FC = ({ panelIdx, retriggerIdx, expr }) => { }); useEffect(() => { - expr !== "" && refetch(); + if (expr !== "") { + refetch(); + } }, [retriggerIdx, refetch, expr, endTime]); useLayoutEffect(() => { diff --git a/web/ui/mantine-ui/src/pages/query/TreeNode.tsx b/web/ui/mantine-ui/src/pages/query/TreeNode.tsx index 515e9baadf..25ffdce650 100644 --- a/web/ui/mantine-ui/src/pages/query/TreeNode.tsx +++ b/web/ui/mantine-ui/src/pages/query/TreeNode.tsx @@ -150,20 +150,20 @@ const TreeNode: FC<{ }); useEffect(() => { - if (mergedChildState === "error") { - reportNodeState && reportNodeState(childIdx, "error"); + if (mergedChildState === "error" && reportNodeState) { + reportNodeState(childIdx, "error"); } }, [mergedChildState, reportNodeState, childIdx]); useEffect(() => { - if (error) { - reportNodeState && reportNodeState(childIdx, "error"); + if (error && reportNodeState) { + reportNodeState(childIdx, "error"); } }, [error, reportNodeState, childIdx]); useEffect(() => { - if (isFetching) { - reportNodeState && reportNodeState(childIdx, "running"); + if (isFetching && reportNodeState) { + reportNodeState(childIdx, "running"); } }, [isFetching, reportNodeState, childIdx]); @@ -219,7 +219,9 @@ const TreeNode: FC<{ return; } - reportNodeState && reportNodeState(childIdx, "success"); + if (reportNodeState) { + reportNodeState(childIdx, "success"); + } let resultSeries = 0; const labelValuesByName: Record> = {}; diff --git a/web/ui/mantine-ui/src/pages/query/uPlotStackHelpers.ts b/web/ui/mantine-ui/src/pages/query/uPlotStackHelpers.ts index ac5d9ce688..ba9379bf69 100644 --- a/web/ui/mantine-ui/src/pages/query/uPlotStackHelpers.ts +++ b/web/ui/mantine-ui/src/pages/query/uPlotStackHelpers.ts @@ -24,10 +24,11 @@ function stack( } for (let i = 1; i < data.length; i++) { - !omit(i) && + if (!omit(i)) { bands.push({ series: [data.findIndex((_s, j) => j > i && !omit(j)), i], }); + } } bands = bands.filter((b) => b.series[1] > -1); @@ -65,7 +66,9 @@ export function setStackedOpts(opts: uPlot.Options, data: uPlot.AlignedData) { if (show) { const pts: number[] = []; data[seriesIdx].forEach((v, i) => { - v != null && pts.push(i); + if (v != null) { + pts.push(i); + } }); return pts; } diff --git a/web/ui/mantine-ui/src/pages/service-discovery/ServiceDiscoveryPage.tsx b/web/ui/mantine-ui/src/pages/service-discovery/ServiceDiscoveryPage.tsx index c0dbd1ca11..4bef1c8d92 100644 --- a/web/ui/mantine-ui/src/pages/service-discovery/ServiceDiscoveryPage.tsx +++ b/web/ui/mantine-ui/src/pages/service-discovery/ServiceDiscoveryPage.tsx @@ -84,7 +84,9 @@ export default function ServiceDiscoveryPage() { value={(limited && scrapePools[0]) || scrapePool || null} onChange={(value) => { setScrapePool(value); - showLimitAlert && dispatch(setShowLimitAlert(false)); + if (showLimitAlert) { + dispatch(setShowLimitAlert(false)); + } }} searchable /> diff --git a/web/ui/mantine-ui/src/pages/targets/TargetsPage.tsx b/web/ui/mantine-ui/src/pages/targets/TargetsPage.tsx index 60dae60541..399d1a458d 100644 --- a/web/ui/mantine-ui/src/pages/targets/TargetsPage.tsx +++ b/web/ui/mantine-ui/src/pages/targets/TargetsPage.tsx @@ -83,7 +83,9 @@ export default function TargetsPage() { value={(limited && scrapePools[0]) || scrapePool || null} onChange={(value) => { setScrapePool(value); - showLimitAlert && dispatch(setShowLimitAlert(false)); + if (showLimitAlert) { + dispatch(setShowLimitAlert(false)); + } }} searchable /> diff --git a/web/ui/package-lock.json b/web/ui/package-lock.json index 07b35914d3..569866873d 100644 --- a/web/ui/package-lock.json +++ b/web/ui/package-lock.json @@ -13,8 +13,8 @@ ], "devDependencies": { "@types/jest": "^29.5.14", - "@typescript-eslint/eslint-plugin": "^6.21.0", - "@typescript-eslint/parser": "^6.21.0", + "@typescript-eslint/eslint-plugin": "^8.20.0", + "@typescript-eslint/parser": "^8.20.0", "eslint-config-prettier": "^9.1.0", "prettier": "^3.4.2", "ts-jest": "^29.2.2", @@ -70,8 +70,8 @@ "@eslint/js": "^9.17.0", "@types/react": "^18.3.5", "@types/react-dom": "^18.3.0", - "@typescript-eslint/eslint-plugin": "^6.21.0", - "@typescript-eslint/parser": "^6.21.0", + "@typescript-eslint/eslint-plugin": "^8.20.0", + "@typescript-eslint/parser": "^8.20.0", "@vitejs/plugin-react": "^4.3.4", "eslint": "^9.18.0", "eslint-plugin-react-hooks": "^5.1.0", @@ -2960,12 +2960,6 @@ "htmlparser2": "^8.0.0" } }, - "node_modules/@types/semver": { - "version": "7.5.8", - "resolved": "https://registry.npmjs.org/@types/semver/-/semver-7.5.8.tgz", - "integrity": "sha512-I8EUhyrgfLrcTkzV3TSsGyl1tSuPrEDzr0yd5m90UgNxQkyDXULk3b6MlQqTCpZpNtWe1K0hzclnZkTcLBe2UQ==", - "dev": true - }, "node_modules/@types/stack-utils": { "version": "2.0.3", "resolved": "https://registry.npmjs.org/@types/stack-utils/-/stack-utils-2.0.3.tgz", @@ -2997,148 +2991,109 @@ "license": "MIT" }, "node_modules/@typescript-eslint/eslint-plugin": { - "version": "6.21.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/eslint-plugin/-/eslint-plugin-6.21.0.tgz", - "integrity": "sha512-oy9+hTPCUFpngkEZUSzbf9MxI65wbKFoQYsgPdILTfbUldp5ovUuphZVe4i30emU9M/kP+T64Di0mxl7dSw3MA==", + "version": "8.20.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/eslint-plugin/-/eslint-plugin-8.20.0.tgz", + "integrity": "sha512-naduuphVw5StFfqp4Gq4WhIBE2gN1GEmMUExpJYknZJdRnc+2gDzB8Z3+5+/Kv33hPQRDGzQO/0opHE72lZZ6A==", "dev": true, "license": "MIT", "dependencies": { - "@eslint-community/regexpp": "^4.5.1", - "@typescript-eslint/scope-manager": "6.21.0", - "@typescript-eslint/type-utils": "6.21.0", - "@typescript-eslint/utils": "6.21.0", - "@typescript-eslint/visitor-keys": "6.21.0", - "debug": "^4.3.4", + "@eslint-community/regexpp": "^4.10.0", + "@typescript-eslint/scope-manager": "8.20.0", + "@typescript-eslint/type-utils": "8.20.0", + "@typescript-eslint/utils": "8.20.0", + "@typescript-eslint/visitor-keys": "8.20.0", "graphemer": "^1.4.0", - "ignore": "^5.2.4", + "ignore": "^5.3.1", "natural-compare": "^1.4.0", - "semver": "^7.5.4", - "ts-api-utils": "^1.0.1" + "ts-api-utils": "^2.0.0" }, "engines": { - "node": "^16.0.0 || >=18.0.0" + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" }, "funding": { "type": "opencollective", "url": "https://opencollective.com/typescript-eslint" }, "peerDependencies": { - "@typescript-eslint/parser": "^6.0.0 || ^6.0.0-alpha", - "eslint": "^7.0.0 || ^8.0.0" - }, - "peerDependenciesMeta": { - "typescript": { - "optional": true - } - } - }, - "node_modules/@typescript-eslint/eslint-plugin/node_modules/@typescript-eslint/type-utils": { - "version": "6.21.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/type-utils/-/type-utils-6.21.0.tgz", - "integrity": "sha512-rZQI7wHfao8qMX3Rd3xqeYSMCL3SoiSQLBATSiVKARdFGCYSRvmViieZjqc58jKgs8Y8i9YvVVhRbHSTA4VBag==", - "dev": true, - "dependencies": { - "@typescript-eslint/typescript-estree": "6.21.0", - "@typescript-eslint/utils": "6.21.0", - "debug": "^4.3.4", - "ts-api-utils": "^1.0.1" - }, - "engines": { - "node": "^16.0.0 || >=18.0.0" - }, - "funding": { - "type": "opencollective", - "url": "https://opencollective.com/typescript-eslint" - }, - "peerDependencies": { - "eslint": "^7.0.0 || ^8.0.0" - }, - "peerDependenciesMeta": { - "typescript": { - "optional": true - } - } - }, - "node_modules/@typescript-eslint/eslint-plugin/node_modules/@typescript-eslint/utils": { - "version": "6.21.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/utils/-/utils-6.21.0.tgz", - "integrity": "sha512-NfWVaC8HP9T8cbKQxHcsJBY5YE1O33+jpMwN45qzWWaPDZgLIbo12toGMWnmhvCpd3sIxkpDw3Wv1B3dYrbDQQ==", - "dev": true, - "dependencies": { - "@eslint-community/eslint-utils": "^4.4.0", - "@types/json-schema": "^7.0.12", - "@types/semver": "^7.5.0", - "@typescript-eslint/scope-manager": "6.21.0", - "@typescript-eslint/types": "6.21.0", - "@typescript-eslint/typescript-estree": "6.21.0", - "semver": "^7.5.4" - }, - "engines": { - "node": "^16.0.0 || >=18.0.0" - }, - "funding": { - "type": "opencollective", - "url": "https://opencollective.com/typescript-eslint" - }, - "peerDependencies": { - "eslint": "^7.0.0 || ^8.0.0" + "@typescript-eslint/parser": "^8.0.0 || ^8.0.0-alpha.0", + "eslint": "^8.57.0 || ^9.0.0", + "typescript": ">=4.8.4 <5.8.0" } }, "node_modules/@typescript-eslint/parser": { - "version": "6.21.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/parser/-/parser-6.21.0.tgz", - "integrity": "sha512-tbsV1jPne5CkFQCgPBcDOt30ItF7aJoZL997JSF7MhGQqOeT3svWRYxiqlfA5RUdlHN6Fi+EI9bxqbdyAUZjYQ==", + "version": "8.20.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/parser/-/parser-8.20.0.tgz", + "integrity": "sha512-gKXG7A5HMyjDIedBi6bUrDcun8GIjnI8qOwVLiY3rx6T/sHP/19XLJOnIq/FgQvWLHja5JN/LSE7eklNBr612g==", "dev": true, - "license": "BSD-2-Clause", "dependencies": { - "@typescript-eslint/scope-manager": "6.21.0", - "@typescript-eslint/types": "6.21.0", - "@typescript-eslint/typescript-estree": "6.21.0", - "@typescript-eslint/visitor-keys": "6.21.0", + "@typescript-eslint/scope-manager": "8.20.0", + "@typescript-eslint/types": "8.20.0", + "@typescript-eslint/typescript-estree": "8.20.0", + "@typescript-eslint/visitor-keys": "8.20.0", "debug": "^4.3.4" }, "engines": { - "node": "^16.0.0 || >=18.0.0" + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" }, "funding": { "type": "opencollective", "url": "https://opencollective.com/typescript-eslint" }, "peerDependencies": { - "eslint": "^7.0.0 || ^8.0.0" - }, - "peerDependenciesMeta": { - "typescript": { - "optional": true - } + "eslint": "^8.57.0 || ^9.0.0", + "typescript": ">=4.8.4 <5.8.0" } }, "node_modules/@typescript-eslint/scope-manager": { - "version": "6.21.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/scope-manager/-/scope-manager-6.21.0.tgz", - "integrity": "sha512-OwLUIWZJry80O99zvqXVEioyniJMa+d2GrqpUTqi5/v5D5rOrppJVBPa0yKCblcigC0/aYAzxxqQ1B+DS2RYsg==", + "version": "8.20.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/scope-manager/-/scope-manager-8.20.0.tgz", + "integrity": "sha512-J7+VkpeGzhOt3FeG1+SzhiMj9NzGD/M6KoGn9f4dbz3YzK9hvbhVTmLj/HiTp9DazIzJ8B4XcM80LrR9Dm1rJw==", "dev": true, "license": "MIT", "dependencies": { - "@typescript-eslint/types": "6.21.0", - "@typescript-eslint/visitor-keys": "6.21.0" + "@typescript-eslint/types": "8.20.0", + "@typescript-eslint/visitor-keys": "8.20.0" }, "engines": { - "node": "^16.0.0 || >=18.0.0" + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" }, "funding": { "type": "opencollective", "url": "https://opencollective.com/typescript-eslint" } }, + "node_modules/@typescript-eslint/type-utils": { + "version": "8.20.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/type-utils/-/type-utils-8.20.0.tgz", + "integrity": "sha512-bPC+j71GGvA7rVNAHAtOjbVXbLN5PkwqMvy1cwGeaxUoRQXVuKCebRoLzm+IPW/NtFFpstn1ummSIasD5t60GA==", + "dev": true, + "license": "MIT", + "dependencies": { + "@typescript-eslint/typescript-estree": "8.20.0", + "@typescript-eslint/utils": "8.20.0", + "debug": "^4.3.4", + "ts-api-utils": "^2.0.0" + }, + "engines": { + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" + }, + "funding": { + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" + }, + "peerDependencies": { + "eslint": "^8.57.0 || ^9.0.0", + "typescript": ">=4.8.4 <5.8.0" + } + }, "node_modules/@typescript-eslint/types": { - "version": "6.21.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/types/-/types-6.21.0.tgz", - "integrity": "sha512-1kFmZ1rOm5epu9NZEZm1kckCDGj5UJEf7P1kliH4LKu/RkwpsfqqGmY2OOcUs18lSlQBKLDYBOGxRVtrMN5lpg==", + "version": "8.20.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/types/-/types-8.20.0.tgz", + "integrity": "sha512-cqaMiY72CkP+2xZRrFt3ExRBu0WmVitN/rYPZErA80mHjHx/Svgp8yfbzkJmDoQ/whcytOPO9/IZXnOc+wigRA==", "dev": true, "license": "MIT", "engines": { - "node": "^16.0.0 || >=18.0.0" + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" }, "funding": { "type": "opencollective", @@ -3146,32 +3101,30 @@ } }, "node_modules/@typescript-eslint/typescript-estree": { - "version": "6.21.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/typescript-estree/-/typescript-estree-6.21.0.tgz", - "integrity": "sha512-6npJTkZcO+y2/kr+z0hc4HwNfrrP4kNYh57ek7yCNlrBjWQ1Y0OS7jiZTkgumrvkX5HkEKXFZkkdFNkaW2wmUQ==", + "version": "8.20.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/typescript-estree/-/typescript-estree-8.20.0.tgz", + "integrity": "sha512-Y7ncuy78bJqHI35NwzWol8E0X7XkRVS4K4P4TCyzWkOJih5NDvtoRDW4Ba9YJJoB2igm9yXDdYI/+fkiiAxPzA==", "dev": true, - "license": "BSD-2-Clause", + "license": "MIT", "dependencies": { - "@typescript-eslint/types": "6.21.0", - "@typescript-eslint/visitor-keys": "6.21.0", + "@typescript-eslint/types": "8.20.0", + "@typescript-eslint/visitor-keys": "8.20.0", "debug": "^4.3.4", - "globby": "^11.1.0", + "fast-glob": "^3.3.2", "is-glob": "^4.0.3", - "minimatch": "9.0.3", - "semver": "^7.5.4", - "ts-api-utils": "^1.0.1" + "minimatch": "^9.0.4", + "semver": "^7.6.0", + "ts-api-utils": "^2.0.0" }, "engines": { - "node": "^16.0.0 || >=18.0.0" + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" }, "funding": { "type": "opencollective", "url": "https://opencollective.com/typescript-eslint" }, - "peerDependenciesMeta": { - "typescript": { - "optional": true - } + "peerDependencies": { + "typescript": ">=4.8.4 <5.8.0" } }, "node_modules/@typescript-eslint/typescript-estree/node_modules/brace-expansion": { @@ -3185,9 +3138,9 @@ } }, "node_modules/@typescript-eslint/typescript-estree/node_modules/minimatch": { - "version": "9.0.3", - "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-9.0.3.tgz", - "integrity": "sha512-RHiac9mvaRw0x3AYRgDC1CxAP7HTcNrrECeA8YYJeWnpo+2Q5CegtZjaotWTWxDG3UeGA1coE05iH1mPjT/2mg==", + "version": "9.0.5", + "resolved": "https://registry.npmjs.org/minimatch/-/minimatch-9.0.5.tgz", + "integrity": "sha512-G6T0ZX48xgozx7587koeX9Ys2NYy6Gmv//P89sEte9V9whIapMNF4idKxnW2QtCcLiTWlb/wfCabAtAFWhhBow==", "dev": true, "license": "ISC", "dependencies": { @@ -3200,35 +3153,46 @@ "url": "https://github.com/sponsors/isaacs" } }, - "node_modules/@typescript-eslint/visitor-keys": { - "version": "6.21.0", - "resolved": "https://registry.npmjs.org/@typescript-eslint/visitor-keys/-/visitor-keys-6.21.0.tgz", - "integrity": "sha512-JJtkDduxLi9bivAB+cYOVMtbkqdPOhZ+ZI5LC47MIRrDV4Yn2o+ZnW10Nkmr28xRpSpdJ6Sm42Hjf2+REYXm0A==", + "node_modules/@typescript-eslint/utils": { + "version": "8.20.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/utils/-/utils-8.20.0.tgz", + "integrity": "sha512-dq70RUw6UK9ei7vxc4KQtBRk7qkHZv447OUZ6RPQMQl71I3NZxQJX/f32Smr+iqWrB02pHKn2yAdHBb0KNrRMA==", "dev": true, "license": "MIT", "dependencies": { - "@typescript-eslint/types": "6.21.0", - "eslint-visitor-keys": "^3.4.1" + "@eslint-community/eslint-utils": "^4.4.0", + "@typescript-eslint/scope-manager": "8.20.0", + "@typescript-eslint/types": "8.20.0", + "@typescript-eslint/typescript-estree": "8.20.0" }, "engines": { - "node": "^16.0.0 || >=18.0.0" + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" }, "funding": { "type": "opencollective", "url": "https://opencollective.com/typescript-eslint" + }, + "peerDependencies": { + "eslint": "^8.57.0 || ^9.0.0", + "typescript": ">=4.8.4 <5.8.0" } }, - "node_modules/@typescript-eslint/visitor-keys/node_modules/eslint-visitor-keys": { - "version": "3.4.3", - "resolved": "https://registry.npmjs.org/eslint-visitor-keys/-/eslint-visitor-keys-3.4.3.tgz", - "integrity": "sha512-wpc+LXeiyiisxPlEkUzU6svyS1frIO3Mgxj1fdy7Pm8Ygzguax2N3Fa/D/ag1WqbOprdI+uY6wMUl8/a2G+iag==", + "node_modules/@typescript-eslint/visitor-keys": { + "version": "8.20.0", + "resolved": "https://registry.npmjs.org/@typescript-eslint/visitor-keys/-/visitor-keys-8.20.0.tgz", + "integrity": "sha512-v/BpkeeYAsPkKCkR8BDwcno0llhzWVqPOamQrAEMdpZav2Y9OVjd9dwJyBLJWwf335B5DmlifECIkZRJCaGaHA==", "dev": true, - "license": "Apache-2.0", + "license": "MIT", + "dependencies": { + "@typescript-eslint/types": "8.20.0", + "eslint-visitor-keys": "^4.2.0" + }, "engines": { - "node": "^12.22.0 || ^14.17.0 || >=16.0.0" + "node": "^18.18.0 || ^20.9.0 || >=21.1.0" }, "funding": { - "url": "https://opencollective.com/eslint" + "type": "opencollective", + "url": "https://opencollective.com/typescript-eslint" } }, "node_modules/@uiw/codemirror-extensions-basic-setup": { @@ -3534,16 +3498,6 @@ "dequal": "^2.0.3" } }, - "node_modules/array-union": { - "version": "2.1.0", - "resolved": "https://registry.npmjs.org/array-union/-/array-union-2.1.0.tgz", - "integrity": "sha512-HGyxoOTYUyCM6stUe6EJgnd4EoewAI7zMdfqO+kGjnlZmBDz/cR5pf8r/cR4Wq60sL/p0IkcjUEEPwS3GFrIyw==", - "dev": true, - "license": "MIT", - "engines": { - "node": ">=8" - } - }, "node_modules/assertion-error": { "version": "2.0.1", "resolved": "https://registry.npmjs.org/assertion-error/-/assertion-error-2.0.1.tgz", @@ -4261,19 +4215,6 @@ "node": "^14.15.0 || ^16.10.0 || >=18.0.0" } }, - "node_modules/dir-glob": { - "version": "3.0.1", - "resolved": "https://registry.npmjs.org/dir-glob/-/dir-glob-3.0.1.tgz", - "integrity": "sha512-WkrWp9GR4KXfKGYzOLmTuGVi1UWFfws377n9cc55/tb6DuqyF6pcQ5AbiHEshaDpY9v6oaSr2XCDidGmMwdzIA==", - "dev": true, - "license": "MIT", - "dependencies": { - "path-type": "^4.0.0" - }, - "engines": { - "node": ">=8" - } - }, "node_modules/doctrine": { "version": "3.0.0", "resolved": "https://registry.npmjs.org/doctrine/-/doctrine-3.0.0.tgz", @@ -5259,27 +5200,6 @@ "url": "https://github.com/sponsors/sindresorhus" } }, - "node_modules/globby": { - "version": "11.1.0", - "resolved": "https://registry.npmjs.org/globby/-/globby-11.1.0.tgz", - "integrity": "sha512-jhIXaOzy1sb8IyocaruWSn1TjmnBVs8Ayhcy83rmxNJ8q2uWKCAj3CnJY+KpGSXCueAPc0i05kVvVKtP1t9S3g==", - "dev": true, - "license": "MIT", - "dependencies": { - "array-union": "^2.1.0", - "dir-glob": "^3.0.1", - "fast-glob": "^3.2.9", - "ignore": "^5.2.0", - "merge2": "^1.4.1", - "slash": "^3.0.0" - }, - "engines": { - "node": ">=10" - }, - "funding": { - "url": "https://github.com/sponsors/sindresorhus" - } - }, "node_modules/graceful-fs": { "version": "4.2.11", "resolved": "https://registry.npmjs.org/graceful-fs/-/graceful-fs-4.2.11.tgz", @@ -7398,16 +7318,6 @@ "dev": true, "license": "MIT" }, - "node_modules/path-type": { - "version": "4.0.0", - "resolved": "https://registry.npmjs.org/path-type/-/path-type-4.0.0.tgz", - "integrity": "sha512-gDKb8aZMDeD/tZWs9P6+q0J9Mwkdl6xMV8TjnGP3qJVJ06bdMgkbBlLU8IdfOsIsFz2BW1rNVT3XuNEl8zPAvw==", - "dev": true, - "license": "MIT", - "engines": { - "node": ">=8" - } - }, "node_modules/pathe": { "version": "1.1.2", "resolved": "https://registry.npmjs.org/pathe/-/pathe-1.1.2.tgz", @@ -8770,16 +8680,16 @@ } }, "node_modules/ts-api-utils": { - "version": "1.3.0", - "resolved": "https://registry.npmjs.org/ts-api-utils/-/ts-api-utils-1.3.0.tgz", - "integrity": "sha512-UQMIo7pb8WRomKR1/+MFVLTroIvDVtMX3K6OUir8ynLyzB8Jeriont2bTAtmNPa1ekAgN7YPDyf6V+ygrdU+eQ==", + "version": "2.0.0", + "resolved": "https://registry.npmjs.org/ts-api-utils/-/ts-api-utils-2.0.0.tgz", + "integrity": "sha512-xCt/TOAc+EOHS1XPnijD3/yzpH6qg2xppZO1YDqGoVsNXfQfzHpOdNuXwrwOU8u4ITXJyDCTyt8w5g1sZv9ynQ==", "dev": true, "license": "MIT", "engines": { - "node": ">=16" + "node": ">=18.12" }, "peerDependencies": { - "typescript": ">=4.2.0" + "typescript": ">=4.8.4" } }, "node_modules/ts-jest": { diff --git a/web/ui/package.json b/web/ui/package.json index 10ba19538f..8539057eeb 100644 --- a/web/ui/package.json +++ b/web/ui/package.json @@ -16,8 +16,8 @@ ], "devDependencies": { "@types/jest": "^29.5.14", - "@typescript-eslint/eslint-plugin": "^6.21.0", - "@typescript-eslint/parser": "^6.21.0", + "@typescript-eslint/eslint-plugin": "^8.20.0", + "@typescript-eslint/parser": "^8.20.0", "eslint-config-prettier": "^9.1.0", "prettier": "^3.4.2", "ts-jest": "^29.2.2", diff --git a/web/ui/react-app/src/pages/graph/DataTable.test.tsx b/web/ui/react-app/src/pages/graph/DataTable.test.tsx index dbc1b18b8e..cf9b3a5331 100755 --- a/web/ui/react-app/src/pages/graph/DataTable.test.tsx +++ b/web/ui/react-app/src/pages/graph/DataTable.test.tsx @@ -1,5 +1,5 @@ import * as React from 'react'; -import { mount, shallow } from 'enzyme'; +import { shallow } from 'enzyme'; import DataTable, { DataTableProps } from './DataTable'; import { Alert, Table } from 'reactstrap'; import SeriesName from './SeriesName'; From 2f42bf2376bbfaed5427eb0708d64f2b0e002f15 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 17 Jan 2025 18:57:39 +0000 Subject: [PATCH 106/110] chore(deps): bump react-dom and @types/react-dom in /web/ui (#15661) * chore(deps): bump react, react-dom and @types/react in /web/ui Bumps [react](https://github.com/facebook/react/tree/HEAD/packages/react), [react-dom](https://github.com/facebook/react/tree/HEAD/packages/react-dom) and [@types/react](https://github.com/DefinitelyTyped/DefinitelyTyped/tree/HEAD/types/react). These dependencies needed to be updated together. Updates `react` from 18.3.1 to 19.0.0 - [Release notes](https://github.com/facebook/react/releases) - [Changelog](https://github.com/facebook/react/blob/main/CHANGELOG.md) - [Commits](https://github.com/facebook/react/commits/v19.0.0/packages/react) Updates `react-dom` from 18.3.1 to 19.0.0 - [Release notes](https://github.com/facebook/react/releases) - [Changelog](https://github.com/facebook/react/blob/main/CHANGELOG.md) - [Commits](https://github.com/facebook/react/commits/v19.0.0/packages/react-dom) Updates `@types/react` from 18.3.5 to 19.0.6 - [Release notes](https://github.com/DefinitelyTyped/DefinitelyTyped/releases) - [Commits](https://github.com/DefinitelyTyped/DefinitelyTyped/commits/HEAD/types/react) --- updated-dependencies: - dependency-name: react dependency-type: direct:production update-type: version-update:semver-major - dependency-name: react-dom dependency-type: direct:production update-type: version-update:semver-major - dependency-name: "@types/react" dependency-type: direct:development update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Signed-off-by: Arve Knudsen * chore(deps): bump react-dom and @types/react-dom in /web/ui Bumps [react-dom](https://github.com/facebook/react/tree/HEAD/packages/react-dom) and [@types/react-dom](https://github.com/DefinitelyTyped/DefinitelyTyped/tree/HEAD/types/react-dom). These dependencies needed to be updated together. Updates `react-dom` from 18.3.1 to 19.0.0 - [Release notes](https://github.com/facebook/react/releases) - [Changelog](https://github.com/facebook/react/blob/main/CHANGELOG.md) - [Commits](https://github.com/facebook/react/commits/v19.0.0/packages/react-dom) Updates `@types/react-dom` from 18.3.0 to 19.0.2 - [Release notes](https://github.com/DefinitelyTyped/DefinitelyTyped/releases) - [Commits](https://github.com/DefinitelyTyped/DefinitelyTyped/commits/HEAD/types/react-dom) --- updated-dependencies: - dependency-name: react-dom dependency-type: direct:production update-type: version-update:semver-major - dependency-name: "@types/react-dom" dependency-type: direct:development update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] * Fix up Signed-off-by: Arve Knudsen --------- Signed-off-by: dependabot[bot] Signed-off-by: Arve Knudsen Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Arve Knudsen --- web/ui/mantine-ui/package.json | 8 +-- web/ui/mantine-ui/src/promql/format.tsx | 2 +- web/ui/package-lock.json | 73 ++++++++----------------- 3 files changed, 29 insertions(+), 54 deletions(-) diff --git a/web/ui/mantine-ui/package.json b/web/ui/mantine-ui/package.json index 9d4afdfae5..54546c76ad 100644 --- a/web/ui/mantine-ui/package.json +++ b/web/ui/mantine-ui/package.json @@ -40,8 +40,8 @@ "clsx": "^2.1.1", "dayjs": "^1.11.10", "lodash": "^4.17.21", - "react": "^18.3.1", - "react-dom": "^18.3.1", + "react": "^19.0.0", + "react-dom": "^19.0.0", "react-infinite-scroll-component": "^6.1.0", "react-redux": "^9.2.0", "react-router-dom": "^7.1.1", @@ -54,8 +54,8 @@ "@eslint/compat": "^1.2.4", "@eslint/eslintrc": "^3.2.0", "@eslint/js": "^9.17.0", - "@types/react": "^18.3.5", - "@types/react-dom": "^18.3.0", + "@types/react": "^19.0.6", + "@types/react-dom": "^19.0.0", "@typescript-eslint/eslint-plugin": "^8.20.0", "@typescript-eslint/parser": "^8.20.0", "@vitejs/plugin-react": "^4.3.4", diff --git a/web/ui/mantine-ui/src/promql/format.tsx b/web/ui/mantine-ui/src/promql/format.tsx index 3996444085..47cd259bd8 100644 --- a/web/ui/mantine-ui/src/promql/format.tsx +++ b/web/ui/mantine-ui/src/promql/format.tsx @@ -1,4 +1,4 @@ -import React, { ReactElement, ReactNode } from "react"; +import React, { ReactElement, ReactNode, JSX } from "react"; import ASTNode, { VectorSelector, matchType, diff --git a/web/ui/package-lock.json b/web/ui/package-lock.json index 569866873d..dd251c660e 100644 --- a/web/ui/package-lock.json +++ b/web/ui/package-lock.json @@ -54,8 +54,8 @@ "clsx": "^2.1.1", "dayjs": "^1.11.10", "lodash": "^4.17.21", - "react": "^18.3.1", - "react-dom": "^18.3.1", + "react": "^19.0.0", + "react-dom": "^19.0.0", "react-infinite-scroll-component": "^6.1.0", "react-redux": "^9.2.0", "react-router-dom": "^7.1.1", @@ -68,8 +68,8 @@ "@eslint/compat": "^1.2.4", "@eslint/eslintrc": "^3.2.0", "@eslint/js": "^9.17.0", - "@types/react": "^18.3.5", - "@types/react-dom": "^18.3.0", + "@types/react": "^19.0.6", + "@types/react-dom": "^19.0.0", "@typescript-eslint/eslint-plugin": "^8.20.0", "@typescript-eslint/parser": "^8.20.0", "@vitejs/plugin-react": "^4.3.4", @@ -2916,32 +2916,23 @@ "undici-types": "~6.19.2" } }, - "node_modules/@types/prop-types": { - "version": "15.7.12", - "resolved": "https://registry.npmjs.org/@types/prop-types/-/prop-types-15.7.12.tgz", - "integrity": "sha512-5zvhXYtRNRluoE/jAp4GVsSduVUzNWKkOZrCDBWYtE7biZywwdC2AcEzg+cSMLFRfVgeAFqpfNabiPjxFddV1Q==", - "devOptional": true, - "license": "MIT" - }, "node_modules/@types/react": { - "version": "18.3.5", - "resolved": "https://registry.npmjs.org/@types/react/-/react-18.3.5.tgz", - "integrity": "sha512-WeqMfGJLGuLCqHGYRGHxnKrXcTitc6L/nBUWfWPcTarG3t9PsquqUMuVeXZeca+mglY4Vo5GZjCi0A3Or2lnxA==", + "version": "19.0.6", + "resolved": "https://registry.npmjs.org/@types/react/-/react-19.0.6.tgz", + "integrity": "sha512-gIlMztcTeDgXCUj0vCBOqEuSEhX//63fW9SZtCJ+agxoQTOklwDfiEMlTWn4mR/C/UK5VHlpwsCsOyf7/hc4lw==", "devOptional": true, - "license": "MIT", "dependencies": { - "@types/prop-types": "*", "csstype": "^3.0.2" } }, "node_modules/@types/react-dom": { - "version": "18.3.0", - "resolved": "https://registry.npmjs.org/@types/react-dom/-/react-dom-18.3.0.tgz", - "integrity": "sha512-EhwApuTmMBmXuFOikhQLIBUn6uFg81SwLMOAUgodJF14SOBOCMdU04gDoYi0WOJJHD144TL32z4yDqCW3dnkQg==", + "version": "19.0.3", + "resolved": "https://registry.npmjs.org/@types/react-dom/-/react-dom-19.0.3.tgz", + "integrity": "sha512-0Knk+HJiMP/qOZgMyNFamlIjw9OFCsyC2ZbigmEEyXXixgre6IQpm/4V+r3qH4GC1JPvRJKInw+on2rV6YZLeA==", "devOptional": true, "license": "MIT", - "dependencies": { - "@types/react": "*" + "peerDependencies": { + "@types/react": "^19.0.0" } }, "node_modules/@types/resolve": { @@ -4239,7 +4230,6 @@ "version": "5.2.1", "resolved": "https://registry.npmjs.org/dom-helpers/-/dom-helpers-5.2.1.tgz", "integrity": "sha512-nRCa7CK3VTrM2NmGkIy4cbK7IZlgBE/PYMn55rrXefr5xXDP0LdtfPnblFDoVdcAfslJ7or6iqAUnx0CCGIWQA==", - "license": "MIT", "dependencies": { "@babel/runtime": "^7.8.7", "csstype": "^3.0.2" @@ -6825,7 +6815,6 @@ "version": "1.4.0", "resolved": "https://registry.npmjs.org/loose-envify/-/loose-envify-1.4.0.tgz", "integrity": "sha512-lyuxPGr/Wfhrlem2CL/UcnUc1zcqKAImBDzukY7Y5F/yQiNdko6+fRLevlw1HgMySw7f611UIY408EtxRSoK3Q==", - "license": "MIT", "dependencies": { "js-tokens": "^3.0.0 || ^4.0.0" }, @@ -7134,7 +7123,6 @@ "version": "4.1.1", "resolved": "https://registry.npmjs.org/object-assign/-/object-assign-4.1.1.tgz", "integrity": "sha512-rJgTQnkUnH1sFw8yT6VSU3zD3sWmu6sZhIseY8VX+GRu3P6F7Fu+JNDoXfklElbLJSnc3FUQHVe4cU5hj+BcUg==", - "license": "MIT", "engines": { "node": ">=0.10.0" } @@ -7667,7 +7655,6 @@ "version": "15.8.1", "resolved": "https://registry.npmjs.org/prop-types/-/prop-types-15.8.1.tgz", "integrity": "sha512-oj87CgZICdulUohogVAR7AjlC0327U4el4L6eAvOqCeudMDVU0NThNaV+b9Df4dXgSP1gXMTnPdhfe/2qDH5cg==", - "license": "MIT", "dependencies": { "loose-envify": "^1.4.0", "object-assign": "^4.1.1", @@ -7677,8 +7664,7 @@ "node_modules/prop-types/node_modules/react-is": { "version": "16.13.1", "resolved": "https://registry.npmjs.org/react-is/-/react-is-16.13.1.tgz", - "integrity": "sha512-24e6ynE2H+OKt4kqsOvNd8kBpV65zoxbA4BVsEOB3ARVWQki/DHzaUoC5KuON/BiccDaCCTZBuOcfZs70kR8bQ==", - "license": "MIT" + "integrity": "sha512-24e6ynE2H+OKt4kqsOvNd8kBpV65zoxbA4BVsEOB3ARVWQki/DHzaUoC5KuON/BiccDaCCTZBuOcfZs70kR8bQ==" }, "node_modules/propagate": { "version": "2.0.1", @@ -7740,28 +7726,22 @@ "license": "MIT" }, "node_modules/react": { - "version": "18.3.1", - "resolved": "https://registry.npmjs.org/react/-/react-18.3.1.tgz", - "integrity": "sha512-wS+hAgJShR0KhEvPJArfuPVN1+Hz1t0Y6n5jLrGQbkb4urgPE/0Rve+1kMB1v/oWgHgm4WIcV+i7F2pTVj+2iQ==", - "license": "MIT", - "dependencies": { - "loose-envify": "^1.1.0" - }, + "version": "19.0.0", + "resolved": "https://registry.npmjs.org/react/-/react-19.0.0.tgz", + "integrity": "sha512-V8AVnmPIICiWpGfm6GLzCR/W5FXLchHop40W4nXBmdlEceh16rCN8O8LNWm5bh5XUX91fh7KpA+W0TgMKmgTpQ==", "engines": { "node": ">=0.10.0" } }, "node_modules/react-dom": { - "version": "18.3.1", - "resolved": "https://registry.npmjs.org/react-dom/-/react-dom-18.3.1.tgz", - "integrity": "sha512-5m4nQKp+rZRb09LNH59GM4BxTh9251/ylbKIbpe7TpGxfJ+9kv6BLkLBXIjjspbgbnIBNqlI23tRnTWT0snUIw==", - "license": "MIT", + "version": "19.0.0", + "resolved": "https://registry.npmjs.org/react-dom/-/react-dom-19.0.0.tgz", + "integrity": "sha512-4GV5sHFG0e/0AD4X+ySy6UJd3jVl1iNsNHdpad0qhABJ11twS3TTBnseqsKurKcsNqCEFeGL3uLpVChpIO3QfQ==", "dependencies": { - "loose-envify": "^1.1.0", - "scheduler": "^0.23.2" + "scheduler": "^0.25.0" }, "peerDependencies": { - "react": "^18.3.1" + "react": "^19.0.0" } }, "node_modules/react-infinite-scroll-component": { @@ -7949,7 +7929,6 @@ "version": "4.4.5", "resolved": "https://registry.npmjs.org/react-transition-group/-/react-transition-group-4.4.5.tgz", "integrity": "sha512-pZcd1MCJoiKiBR2NRxeCRg13uCXbydPnmB4EOeRrY7480qNWO8IIgQG6zlDkm6uRMsURXPuKq0GWtiM59a5Q6g==", - "license": "BSD-3-Clause", "dependencies": { "@babel/runtime": "^7.5.5", "dom-helpers": "^5.0.1", @@ -8206,13 +8185,9 @@ } }, "node_modules/scheduler": { - "version": "0.23.2", - "resolved": "https://registry.npmjs.org/scheduler/-/scheduler-0.23.2.tgz", - "integrity": "sha512-UOShsPwz7NrMUqhR6t0hWjFduvOzbtv7toDH1/hIrfRNIDBnnBWd0CwJTGvTpngVlmwGCdP9/Zl/tVrDqcuYzQ==", - "license": "MIT", - "dependencies": { - "loose-envify": "^1.1.0" - } + "version": "0.25.0", + "resolved": "https://registry.npmjs.org/scheduler/-/scheduler-0.25.0.tgz", + "integrity": "sha512-xFVuu11jh+xcO7JOAGJNOXld8/TcEHK/4CituBUeUb5hqxJLj9YuemAEuvm9gQ/+pgXYfbQuqAkiYu+u7YEsNA==" }, "node_modules/semver": { "version": "7.6.3", From a82f2b8168d7670d13f209be09f50fe9f919414b Mon Sep 17 00:00:00 2001 From: piguagua Date: Sat, 18 Jan 2025 04:26:08 +0800 Subject: [PATCH 107/110] chore: fix function name and struct name in comment (#15827) Signed-off-by: piguagua --- tsdb/record/record_test.go | 2 +- web/api/v1/errors_test.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tsdb/record/record_test.go b/tsdb/record/record_test.go index dc625f0830..9b2eb89c5a 100644 --- a/tsdb/record/record_test.go +++ b/tsdb/record/record_test.go @@ -476,7 +476,7 @@ type recordsMaker struct { make refsCreateFn } -// BenchmarkWAL_HistogramLog measures efficiency of encoding classic +// BenchmarkWAL_HistogramEncoding measures efficiency of encoding classic // histograms and native historgrams with custom buckets (NHCB). func BenchmarkWAL_HistogramEncoding(b *testing.B) { initClassicRefs := func(labelCount, histograms, buckets int) (series []RefSeries, floatSamples []RefSample, histSamples []RefHistogramSample) { diff --git a/web/api/v1/errors_test.go b/web/api/v1/errors_test.go index 6ff0614c4c..1cc90a4b62 100644 --- a/web/api/v1/errors_test.go +++ b/web/api/v1/errors_test.go @@ -217,7 +217,7 @@ func (t errorTestSeriesSet) Warnings() annotations.Annotations { return nil } -// DummyTargetRetriever implements github.com/prometheus/prometheus/web/api/v1.ScrapePoolsRetriever. +// DummyScrapePoolsRetriever implements github.com/prometheus/prometheus/web/api/v1.ScrapePoolsRetriever. type DummyScrapePoolsRetriever struct{} func (DummyScrapePoolsRetriever) ScrapePools() []string { From 9258e40589d971c0a009dccaa13462b87fb9454a Mon Sep 17 00:00:00 2001 From: Mikel Olasagasti Uranga Date: Sat, 18 Jan 2025 13:21:51 +0100 Subject: [PATCH 108/110] parser: fix non-constant format string call (#15835) Go 1.24 enhanced vet's printf analyzer to report calls of the form fmt.Printf(s), where s is a non-constant format string, with no other arguments. This change makes parser tests to fail. Signed-off-by: Mikel Olasagasti Uranga --- promql/parser/generated_parser.y | 2 +- promql/parser/generated_parser.y.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/promql/parser/generated_parser.y b/promql/parser/generated_parser.y index ca710b1ab0..cdb4532d3b 100644 --- a/promql/parser/generated_parser.y +++ b/promql/parser/generated_parser.y @@ -488,7 +488,7 @@ matrix_selector : expr LEFT_BRACKET number_duration_literal RIGHT_BRACKET if errMsg != ""{ errRange := mergeRanges(&$2, &$4) - yylex.(*parser).addParseErrf(errRange, errMsg) + yylex.(*parser).addParseErrf(errRange, "%s", errMsg) } numLit, _ := $3.(*NumberLiteral) diff --git a/promql/parser/generated_parser.y.go b/promql/parser/generated_parser.y.go index 04bc081f2f..78d5e15245 100644 --- a/promql/parser/generated_parser.y.go +++ b/promql/parser/generated_parser.y.go @@ -1385,7 +1385,7 @@ yydefault: if errMsg != "" { errRange := mergeRanges(&yyDollar[2].item, &yyDollar[4].item) - yylex.(*parser).addParseErrf(errRange, errMsg) + yylex.(*parser).addParseErrf(errRange, "%s", errMsg) } numLit, _ := yyDollar[3].node.(*NumberLiteral) From 2a8ae586f4f17e8616b02b40909abc3dde3733b3 Mon Sep 17 00:00:00 2001 From: Dimitar Dimitrov Date: Mon, 20 Jan 2025 21:26:58 +0100 Subject: [PATCH 109/110] ruler: stop all rule groups asynchronously on shutdown (#15804) * ruler: stop all rule groups asynchronously on shutdown During shutdown of the rules manager some rule groups have already stopped and are missing evaluations while we're waiting for other groups to finish their evaluation. When there are many groups (in the thousands), the whole shutdown process can take up to 10 minutes, during which we get miss evaluations. Signed-off-by: Dimitar Dimitrov * Use wrappers in stop(); rename awaitStopped() Signed-off-by: Dimitar Dimitrov * Add comment Signed-off-by: Dimitar Dimitrov --------- Signed-off-by: Dimitar Dimitrov --- rules/group.go | 10 +++++++++- rules/manager.go | 8 +++++++- 2 files changed, 16 insertions(+), 2 deletions(-) diff --git a/rules/group.go b/rules/group.go index 4398d9211d..9ad9aab093 100644 --- a/rules/group.go +++ b/rules/group.go @@ -302,11 +302,19 @@ func (g *Group) run(ctx context.Context) { } } -func (g *Group) stop() { +func (g *Group) stopAsync() { close(g.done) +} + +func (g *Group) waitStopped() { <-g.terminated } +func (g *Group) stop() { + g.stopAsync() + g.waitStopped() +} + func (g *Group) hash() uint64 { l := labels.New( labels.Label{Name: "name", Value: g.name}, diff --git a/rules/manager.go b/rules/manager.go index 50b2a7e99d..b1d3e8e3d6 100644 --- a/rules/manager.go +++ b/rules/manager.go @@ -188,8 +188,14 @@ func (m *Manager) Stop() { m.logger.Info("Stopping rule manager...") + // Stop all groups asynchronously, then wait for them to finish. + // This is faster than stopping and waiting for each group in sequence. for _, eg := range m.groups { - eg.stop() + eg.stopAsync() + } + + for _, eg := range m.groups { + eg.waitStopped() } // Shut down the groups waiting multiple evaluation intervals to write From e8fab32ca289a6bbdafd7cc9739b349e565d613c Mon Sep 17 00:00:00 2001 From: Paulo Dias <44772900+paulojmdias@users.noreply.github.com> Date: Tue, 21 Jan 2025 10:40:15 +0000 Subject: [PATCH 110/110] discovery: move openstack floating ips function from deprecated Compute API /os-floating-ips to Network API /floatingips (#14367) --- discovery/openstack/instance.go | 56 +++- discovery/openstack/instance_test.go | 28 +- discovery/openstack/mock_test.go | 471 +++++++++++++++++++++------ 3 files changed, 446 insertions(+), 109 deletions(-) diff --git a/discovery/openstack/instance.go b/discovery/openstack/instance.go index 2a9e29f2ef..f25c19badb 100644 --- a/discovery/openstack/instance.go +++ b/discovery/openstack/instance.go @@ -22,8 +22,9 @@ import ( "github.com/gophercloud/gophercloud" "github.com/gophercloud/gophercloud/openstack" - "github.com/gophercloud/gophercloud/openstack/compute/v2/extensions/floatingips" "github.com/gophercloud/gophercloud/openstack/compute/v2/servers" + "github.com/gophercloud/gophercloud/openstack/networking/v2/extensions/layer3/floatingips" + "github.com/gophercloud/gophercloud/openstack/networking/v2/ports" "github.com/gophercloud/gophercloud/pagination" "github.com/prometheus/common/model" "github.com/prometheus/common/promslog" @@ -72,8 +73,8 @@ func newInstanceDiscovery(provider *gophercloud.ProviderClient, opts *gopherclou } type floatingIPKey struct { - id string - fixed string + deviceID string + fixed string } func (i *InstanceDiscovery) refresh(ctx context.Context) ([]*targetgroup.Group, error) { @@ -90,9 +91,33 @@ func (i *InstanceDiscovery) refresh(ctx context.Context) ([]*targetgroup.Group, return nil, fmt.Errorf("could not create OpenStack compute session: %w", err) } + networkClient, err := openstack.NewNetworkV2(i.provider, gophercloud.EndpointOpts{ + Region: i.region, Availability: i.availability, + }) + if err != nil { + return nil, fmt.Errorf("could not create OpenStack network session: %w", err) + } + // OpenStack API reference - // https://developer.openstack.org/api-ref/compute/#list-floating-ips - pagerFIP := floatingips.List(client) + // https://docs.openstack.org/api-ref/network/v2/index.html#list-ports + portPages, err := ports.List(networkClient, ports.ListOpts{}).AllPages() + if err != nil { + return nil, fmt.Errorf("failed to list all ports: %w", err) + } + + allPorts, err := ports.ExtractPorts(portPages) + if err != nil { + return nil, fmt.Errorf("failed to extract Ports: %w", err) + } + + portList := make(map[string]string) + for _, port := range allPorts { + portList[port.ID] = port.DeviceID + } + + // OpenStack API reference + // https://docs.openstack.org/api-ref/network/v2/index.html#list-floating-ips + pagerFIP := floatingips.List(networkClient, floatingips.ListOpts{}) floatingIPList := make(map[floatingIPKey]string) floatingIPPresent := make(map[string]struct{}) err = pagerFIP.EachPage(func(page pagination.Page) (bool, error) { @@ -102,11 +127,24 @@ func (i *InstanceDiscovery) refresh(ctx context.Context) ([]*targetgroup.Group, } for _, ip := range result { // Skip not associated ips - if ip.InstanceID == "" || ip.FixedIP == "" { + if ip.PortID == "" || ip.FixedIP == "" { continue } - floatingIPList[floatingIPKey{id: ip.InstanceID, fixed: ip.FixedIP}] = ip.IP - floatingIPPresent[ip.IP] = struct{}{} + + // Fetch deviceID from portList + deviceID, ok := portList[ip.PortID] + if !ok { + i.logger.Warn("Floating IP PortID not found in portList", "PortID", ip.PortID) + continue + } + + key := floatingIPKey{ + deviceID: deviceID, + fixed: ip.FixedIP, + } + + floatingIPList[key] = ip.FloatingIP + floatingIPPresent[ip.FloatingIP] = struct{}{} } return true, nil }) @@ -198,7 +236,7 @@ func (i *InstanceDiscovery) refresh(ctx context.Context) ([]*targetgroup.Group, } lbls[openstackLabelAddressPool] = model.LabelValue(pool) lbls[openstackLabelPrivateIP] = model.LabelValue(addr) - if val, ok := floatingIPList[floatingIPKey{id: s.ID, fixed: addr}]; ok { + if val, ok := floatingIPList[floatingIPKey{deviceID: s.ID, fixed: addr}]; ok { lbls[openstackLabelPublicIP] = model.LabelValue(val) } addr = net.JoinHostPort(addr, strconv.Itoa(i.port)) diff --git a/discovery/openstack/instance_test.go b/discovery/openstack/instance_test.go index 2617baa4e3..0933b57067 100644 --- a/discovery/openstack/instance_test.go +++ b/discovery/openstack/instance_test.go @@ -32,6 +32,7 @@ func (s *OpenstackSDInstanceTestSuite) SetupTest(t *testing.T) { s.Mock.HandleServerListSuccessfully() s.Mock.HandleFloatingIPListSuccessfully() + s.Mock.HandlePortsListSuccessfully() s.Mock.HandleVersionsSuccessfully() s.Mock.HandleAuthSuccessfully() @@ -66,7 +67,7 @@ func TestOpenstackSDInstanceRefresh(t *testing.T) { tg := tgs[0] require.NotNil(t, tg) require.NotNil(t, tg.Targets) - require.Len(t, tg.Targets, 4) + require.Len(t, tg.Targets, 6) for i, lbls := range []model.LabelSet{ { @@ -119,6 +120,31 @@ func TestOpenstackSDInstanceRefresh(t *testing.T) { "__meta_openstack_project_id": model.LabelValue("fcad67a6189847c4aecfa3c81a05783b"), "__meta_openstack_user_id": model.LabelValue("9349aff8be7545ac9d2f1d00999a23cd"), }, + { + "__address__": model.LabelValue("10.0.0.33:0"), + "__meta_openstack_instance_flavor": model.LabelValue("m1.small"), + "__meta_openstack_instance_id": model.LabelValue("87caf8ed-d92a-41f6-9dcd-d1399e39899f"), + "__meta_openstack_instance_status": model.LabelValue("ACTIVE"), + "__meta_openstack_instance_name": model.LabelValue("merp-project2"), + "__meta_openstack_private_ip": model.LabelValue("10.0.0.33"), + "__meta_openstack_address_pool": model.LabelValue("private"), + "__meta_openstack_tag_env": model.LabelValue("prod"), + "__meta_openstack_project_id": model.LabelValue("b78fef2305934dbbbeb9a10b4c326f7a"), + "__meta_openstack_user_id": model.LabelValue("9349aff8be7545ac9d2f1d00999a23cd"), + }, + { + "__address__": model.LabelValue("10.0.0.34:0"), + "__meta_openstack_instance_flavor": model.LabelValue("m1.small"), + "__meta_openstack_instance_id": model.LabelValue("87caf8ed-d92a-41f6-9dcd-d1399e39899f"), + "__meta_openstack_instance_status": model.LabelValue("ACTIVE"), + "__meta_openstack_instance_name": model.LabelValue("merp-project2"), + "__meta_openstack_private_ip": model.LabelValue("10.0.0.34"), + "__meta_openstack_address_pool": model.LabelValue("private"), + "__meta_openstack_tag_env": model.LabelValue("prod"), + "__meta_openstack_public_ip": model.LabelValue("10.10.10.24"), + "__meta_openstack_project_id": model.LabelValue("b78fef2305934dbbbeb9a10b4c326f7a"), + "__meta_openstack_user_id": model.LabelValue("9349aff8be7545ac9d2f1d00999a23cd"), + }, } { t.Run(fmt.Sprintf("item %d", i), func(t *testing.T) { require.Equal(t, lbls, tg.Targets[i]) diff --git a/discovery/openstack/mock_test.go b/discovery/openstack/mock_test.go index 4518f41166..2d12dbc0df 100644 --- a/discovery/openstack/mock_test.go +++ b/discovery/openstack/mock_test.go @@ -124,7 +124,7 @@ func (m *SDMock) HandleAuthSuccessfully() { "type": "identity", "name": "keystone" }, - { + { "endpoints": [ { "id": "e2ffee808abc4a60916715b1d4b489dd", @@ -136,8 +136,20 @@ func (m *SDMock) HandleAuthSuccessfully() { ], "id": "b7f2a5b1a019459cb956e43a8cb41e31", "type": "compute" - } - + }, + { + "endpoints": [ + { + "id": "5448e46679564d7d95466c2bef54c296", + "interface": "public", + "region": "RegionOne", + "region_id": "RegionOne", + "url": "%s" + } + ], + "id": "589f3d99a3d94f5f871e9f5cf206d2e8", + "type": "network" + } ], "expires_at": "2013-02-27T18:30:59.999999Z", "is_domain": false, @@ -174,7 +186,7 @@ func (m *SDMock) HandleAuthSuccessfully() { } } } - `, m.Endpoint()) + `, m.Endpoint(), m.Endpoint()) }) } @@ -461,82 +473,159 @@ const serverListBody = ` "metadata": {} }, { - "status": "ACTIVE", - "updated": "2014-09-25T13:04:49Z", - "hostId": "29d3c8c896a45aa4c34e52247875d7fefc3d94bbcc9f622b5d204362", - "OS-EXT-SRV-ATTR:host": "devstack", - "addresses": { - "private": [ - { - "version": 4, - "addr": "10.0.0.33", - "OS-EXT-IPS:type": "fixed" - }, - { - "version": 4, - "addr": "10.0.0.34", - "OS-EXT-IPS:type": "fixed" - }, - { - "version": 4, - "addr": "10.10.10.4", - "OS-EXT-IPS:type": "floating" - } - ] - }, - "links": [ - { - "href": "http://104.130.131.164:8774/v2/fcad67a6189847c4aecfa3c81a05783b/servers/9e5476bd-a4ec-4653-93d6-72c93aa682ba", - "rel": "self" + "status": "ACTIVE", + "updated": "2014-09-25T13:04:49Z", + "hostId": "29d3c8c896a45aa4c34e52247875d7fefc3d94bbcc9f622b5d204362", + "OS-EXT-SRV-ATTR:host": "devstack", + "addresses": { + "private": [ + { + "version": 4, + "addr": "10.0.0.33", + "OS-EXT-IPS:type": "fixed" + }, + { + "version": 4, + "addr": "10.0.0.34", + "OS-EXT-IPS:type": "fixed" + }, + { + "version": 4, + "addr": "10.10.10.4", + "OS-EXT-IPS:type": "floating" + } + ] }, - { - "href": "http://104.130.131.164:8774/fcad67a6189847c4aecfa3c81a05783b/servers/9e5476bd-a4ec-4653-93d6-72c93aa682ba", - "rel": "bookmark" - } - ], - "key_name": null, - "image": "", - "OS-EXT-STS:task_state": null, - "OS-EXT-STS:vm_state": "active", - "OS-EXT-SRV-ATTR:instance_name": "instance-0000001d", - "OS-SRV-USG:launched_at": "2014-09-25T13:04:49.000000", - "OS-EXT-SRV-ATTR:hypervisor_hostname": "devstack", - "flavor": { - "vcpus": 2, - "ram": 4096, - "disk": 0, - "ephemeral": 0, - "swap": 0, - "original_name": "m1.small", - "extra_specs": { - "aggregate_instance_extra_specs:general": "true", - "hw:mem_page_size": "large", - "hw:vif_multiqueue_enabled": "true" + "links": [ + { + "href": "http://104.130.131.164:8774/v2/fcad67a6189847c4aecfa3c81a05783b/servers/9e5476bd-a4ec-4653-93d6-72c93aa682ba", + "rel": "self" + }, + { + "href": "http://104.130.131.164:8774/fcad67a6189847c4aecfa3c81a05783b/servers/9e5476bd-a4ec-4653-93d6-72c93aa682ba", + "rel": "bookmark" + } + ], + "key_name": null, + "image": "", + "OS-EXT-STS:task_state": null, + "OS-EXT-STS:vm_state": "active", + "OS-EXT-SRV-ATTR:instance_name": "instance-0000001d", + "OS-SRV-USG:launched_at": "2014-09-25T13:04:49.000000", + "OS-EXT-SRV-ATTR:hypervisor_hostname": "devstack", + "flavor": { + "vcpus": 2, + "ram": 4096, + "disk": 0, + "ephemeral": 0, + "swap": 0, + "original_name": "m1.small", + "extra_specs": { + "aggregate_instance_extra_specs:general": "true", + "hw:mem_page_size": "large", + "hw:vif_multiqueue_enabled": "true" + } + }, + "id": "9e5476bd-a4ec-4653-93d6-72c93aa682bb", + "security_groups": [ + { + "name": "default" + } + ], + "OS-SRV-USG:terminated_at": null, + "OS-EXT-AZ:availability_zone": "nova", + "user_id": "9349aff8be7545ac9d2f1d00999a23cd", + "name": "merp", + "created": "2014-09-25T13:04:41Z", + "tenant_id": "fcad67a6189847c4aecfa3c81a05783b", + "OS-DCF:diskConfig": "MANUAL", + "os-extended-volumes:volumes_attached": [], + "accessIPv4": "", + "accessIPv6": "", + "progress": 0, + "OS-EXT-STS:power_state": 1, + "config_drive": "", + "metadata": { + "env": "prod" } }, - "id": "9e5476bd-a4ec-4653-93d6-72c93aa682bb", - "security_groups": [ - { - "name": "default" + { + "status": "ACTIVE", + "updated": "2014-09-25T13:04:49Z", + "hostId": "29d3c8c896a45aa4c34e52247875d7fefc3d94bbcc9f622b5d204362", + "OS-EXT-SRV-ATTR:host": "devstack", + "addresses": { + "private": [ + { + "version": 4, + "addr": "10.0.0.33", + "OS-EXT-IPS:type": "fixed" + }, + { + "version": 4, + "addr": "10.0.0.34", + "OS-EXT-IPS:type": "fixed" + }, + { + "version": 4, + "addr": "10.10.10.24", + "OS-EXT-IPS:type": "floating" + } + ] + }, + "links": [ + { + "href": "http://104.130.131.164:8774/v2/b78fef2305934dbbbeb9a10b4c326f7a/servers/9e5476bd-a4ec-4653-93d6-72c93aa682ba", + "rel": "self" + }, + { + "href": "http://104.130.131.164:8774/b78fef2305934dbbbeb9a10b4c326f7a/servers/9e5476bd-a4ec-4653-93d6-72c93aa682ba", + "rel": "bookmark" + } + ], + "key_name": null, + "image": "", + "OS-EXT-STS:task_state": null, + "OS-EXT-STS:vm_state": "active", + "OS-EXT-SRV-ATTR:instance_name": "instance-0000002d", + "OS-SRV-USG:launched_at": "2014-09-25T13:04:49.000000", + "OS-EXT-SRV-ATTR:hypervisor_hostname": "devstack", + "flavor": { + "vcpus": 2, + "ram": 4096, + "disk": 0, + "ephemeral": 0, + "swap": 0, + "original_name": "m1.small", + "extra_specs": { + "aggregate_instance_extra_specs:general": "true", + "hw:mem_page_size": "large", + "hw:vif_multiqueue_enabled": "true" + } + }, + "id": "87caf8ed-d92a-41f6-9dcd-d1399e39899f", + "security_groups": [ + { + "name": "default" + } + ], + "OS-SRV-USG:terminated_at": null, + "OS-EXT-AZ:availability_zone": "nova", + "user_id": "9349aff8be7545ac9d2f1d00999a23cd", + "name": "merp-project2", + "created": "2014-09-25T13:04:41Z", + "tenant_id": "b78fef2305934dbbbeb9a10b4c326f7a", + "OS-DCF:diskConfig": "MANUAL", + "os-extended-volumes:volumes_attached": [], + "accessIPv4": "", + "accessIPv6": "", + "progress": 0, + "OS-EXT-STS:power_state": 1, + "config_drive": "", + "metadata": { + "env": "prod" } - ], - "OS-SRV-USG:terminated_at": null, - "OS-EXT-AZ:availability_zone": "nova", - "user_id": "9349aff8be7545ac9d2f1d00999a23cd", - "name": "merp", - "created": "2014-09-25T13:04:41Z", - "tenant_id": "fcad67a6189847c4aecfa3c81a05783b", - "OS-DCF:diskConfig": "MANUAL", - "os-extended-volumes:volumes_attached": [], - "accessIPv4": "", - "accessIPv6": "", - "progress": 0, - "OS-EXT-STS:power_state": 1, - "config_drive": "", - "metadata": { - "env": "prod" } - } ] } ` @@ -554,35 +643,82 @@ func (m *SDMock) HandleServerListSuccessfully() { const listOutput = ` { - "floating_ips": [ - { - "fixed_ip": null, - "id": "1", - "instance_id": null, - "ip": "10.10.10.1", - "pool": "nova" - }, - { - "fixed_ip": "10.0.0.32", - "id": "2", - "instance_id": "ef079b0c-e610-4dfb-b1aa-b49f07ac48e5", - "ip": "10.10.10.2", - "pool": "nova" - }, - { - "fixed_ip": "10.0.0.34", - "id": "3", - "instance_id": "9e5476bd-a4ec-4653-93d6-72c93aa682bb", - "ip": "10.10.10.4", - "pool": "nova" - } - ] + "floatingips": [ + { + "id": "03a77860-ae03-46c4-b502-caea11467a79", + "tenant_id": "fcad67a6189847c4aecfa3c81a05783b", + "floating_ip_address": "10.10.10.1", + "floating_network_id": "d02c4f18-d606-4864-b12a-1c9b39a46be2", + "router_id": "f03af93b-4e8f-4f55-adcf-a0317782ede2", + "port_id": "d5597901-48c8-4a69-a041-cfc5be158a04", + "fixed_ip_address": null, + "status": "ACTIVE", + "description": "", + "dns_domain": "", + "dns_name": "", + "port_forwardings": [], + "tags": [], + "created_at": "2023-08-30T16:30:27Z", + "updated_at": "2023-08-30T16:30:28Z" + }, + { + "id": "03e28c79-5a4c-491e-a4fe-3ff6bba830c6", + "tenant_id": "fcad67a6189847c4aecfa3c81a05783b", + "floating_ip_address": "10.10.10.2", + "floating_network_id": "d02c4f18-d606-4864-b12a-1c9b39a46be2", + "router_id": "f03af93b-4e8f-4f55-adcf-a0317782ede2", + "port_id": "4a45b012-0478-484d-8cf3-c8abdb194d08", + "fixed_ip_address": "10.0.0.32", + "status": "ACTIVE", + "description": "", + "dns_domain": "", + "dns_name": "", + "port_forwardings": [], + "tags": [], + "created_at": "2023-09-06T15:45:36Z", + "updated_at": "2023-09-06T15:45:36Z" + }, + { + "id": "087fcdd2-1d13-4f72-9c0e-c759e796d558", + "tenant_id": "fcad67a6189847c4aecfa3c81a05783b", + "floating_ip_address": "10.10.10.4", + "floating_network_id": "d02c4f18-d606-4864-b12a-1c9b39a46be2", + "router_id": "f03af93b-4e8f-4f55-adcf-a0317782ede2", + "port_id": "a0e244e8-7910-4427-b8d1-20470cad4f8a", + "fixed_ip_address": "10.0.0.34", + "status": "ACTIVE", + "description": "", + "dns_domain": "", + "dns_name": "", + "port_forwardings": [], + "tags": [], + "created_at": "2024-01-24T13:30:50Z", + "updated_at": "2024-01-24T13:30:51Z" + }, + { + "id": "b23df91a-a74a-4f75-b252-750aff4a5a0c", + "tenant_id": "b78fef2305934dbbbeb9a10b4c326f7a", + "floating_ip_address": "10.10.10.24", + "floating_network_id": "b19ff5bc-a49a-46cc-8d14-ca5f1e94791f", + "router_id": "65a5e5af-17f0-4124-9a81-c08b44f5b8a7", + "port_id": "b926ab68-ec54-46d8-8c50-1c07aafd5ae9", + "fixed_ip_address": "10.0.0.34", + "status": "ACTIVE", + "description": "", + "dns_domain": "", + "dns_name": "", + "port_forwardings": [], + "tags": [], + "created_at": "2024-01-24T13:30:50Z", + "updated_at": "2024-01-24T13:30:51Z" + } + ] } ` // HandleFloatingIPListSuccessfully mocks floating ips call. func (m *SDMock) HandleFloatingIPListSuccessfully() { - m.Mux.HandleFunc("/os-floating-ips", func(w http.ResponseWriter, r *http.Request) { + m.Mux.HandleFunc("/v2.0/floatingips", func(w http.ResponseWriter, r *http.Request) { testMethod(m.t, r, http.MethodGet) testHeader(m.t, r, "X-Auth-Token", tokenID) @@ -590,3 +726,140 @@ func (m *SDMock) HandleFloatingIPListSuccessfully() { fmt.Fprint(w, listOutput) }) } + +const portsListBody = ` +{ + "ports": [ + { + "id": "d5597901-48c8-4a69-a041-cfc5be158a04", + "name": "", + "network_id": "d02c4f18-d606-4864-b12a-1c9b39a46be2", + "tenant_id": "fcad67a6189847c4aecfa3c81a05783b", + "mac_address": "", + "admin_state_up": true, + "status": "DOWN", + "device_id": "", + "device_owner": "", + "fixed_ips": [], + "allowed_address_pairs": [], + "extra_dhcp_opts": [], + "security_groups": [], + "description": "", + "binding:vnic_type": "normal", + "port_security_enabled": true, + "dns_name": "", + "dns_assignment": [], + "dns_domain": "", + "tags": [], + "created_at": "2023-08-30T16:30:27Z", + "updated_at": "2023-08-30T16:30:28Z", + "revision_number": 0, + "project_id": "fcad67a6189847c4aecfa3c81a05783b" + }, + { + "id": "4a45b012-0478-484d-8cf3-c8abdb194d08", + "name": "ovn-lb-vip-0980c8de-58c3-481d-89e3-ed81f44286c0", + "network_id": "03200a39-b399-44f3-a778-6dbb93343a31", + "tenant_id": "fcad67a6189847c4aecfa3c81a05783b", + "mac_address": "fa:16:3e:23:12:a3", + "admin_state_up": true, + "status": "ACTIVE", + "device_id": "ef079b0c-e610-4dfb-b1aa-b49f07ac48e5", + "device_owner": "", + "fixed_ips": [ + { + "subnet_id": "", + "ip_address": "10.10.10.2" + } + ], + "allowed_address_pairs": [], + "extra_dhcp_opts": [], + "security_groups": [], + "description": "", + "binding:vnic_type": "normal", + "port_security_enabled": true, + "dns_name": "", + "dns_assignment": [], + "dns_domain": "", + "tags": [], + "created_at": "2023-09-06T15:45:36Z", + "updated_at": "2023-09-06T15:45:36Z", + "revision_number": 0, + "project_id": "fcad67a6189847c4aecfa3c81a05783b" + }, + { + "id": "a0e244e8-7910-4427-b8d1-20470cad4f8a", + "name": "ovn-lb-vip-26c0ccb1-3036-4345-99e8-d8f34a8ba6b2", + "network_id": "03200a39-b399-44f3-a778-6dbb93343a31", + "tenant_id": "fcad67a6189847c4aecfa3c81a05783b", + "mac_address": "fa:16:3e:5f:43:10", + "admin_state_up": true, + "status": "ACTIVE", + "device_id": "9e5476bd-a4ec-4653-93d6-72c93aa682bb", + "device_owner": "", + "fixed_ips": [ + { + "subnet_id": "", + "ip_address": "10.10.10.4" + } + ], + "allowed_address_pairs": [], + "extra_dhcp_opts": [], + "security_groups": [], + "description": "", + "binding:vnic_type": "normal", + "port_security_enabled": true, + "dns_name": "", + "dns_assignment": [], + "dns_domain": "", + "tags": [], + "created_at": "2024-01-24T13:30:50Z", + "updated_at": "2024-01-24T13:30:51Z", + "revision_number": 0, + "project_id": "fcad67a6189847c4aecfa3c81a05783b" + }, + { + "id": "b926ab68-ec54-46d8-8c50-1c07aafd5ae9", + "name": "dummy-port", + "network_id": "03200a39-b399-44f3-a778-6dbb93343a31", + "tenant_id": "b78fef2305934dbbbeb9a10b4c326f7a", + "mac_address": "fa:16:3e:5f:12:10", + "admin_state_up": true, + "status": "ACTIVE", + "device_id": "87caf8ed-d92a-41f6-9dcd-d1399e39899f", + "device_owner": "", + "fixed_ips": [ + { + "subnet_id": "", + "ip_address": "10.10.10.24" + } + ], + "allowed_address_pairs": [], + "extra_dhcp_opts": [], + "security_groups": [], + "description": "", + "binding:vnic_type": "normal", + "port_security_enabled": true, + "dns_name": "", + "dns_assignment": [], + "dns_domain": "", + "tags": [], + "created_at": "2024-01-24T13:30:50Z", + "updated_at": "2024-01-24T13:30:51Z", + "revision_number": 0, + "project_id": "b78fef2305934dbbbeb9a10b4c326f7a" + } + ] +} +` + +// HandlePortsListSuccessfully mocks the ports list API. +func (m *SDMock) HandlePortsListSuccessfully() { + m.Mux.HandleFunc("/v2.0/ports", func(w http.ResponseWriter, r *http.Request) { + testMethod(m.t, r, http.MethodGet) + testHeader(m.t, r, "X-Auth-Token", tokenID) + + w.Header().Add("Content-Type", "application/json") + fmt.Fprint(w, portsListBody) + }) +}