mirror of
https://github.com/prometheus/prometheus.git
synced 2025-03-05 20:59:13 -08:00
Improve MetricsForLabelMatchers
WIP: This needs more tests. It now gets a from and through value, which it may opportunistically use to optimize the retrieval. With possible future range indices, this could be used in a very efficient way. This change merely applies some easy checks, which should nevertheless solve the use case of heavy rule evaluations on servers with a lot of series churn. Idea is the following: - Only archive series that are at least as old as the headChunkTimeout (which was already extremely unlikely to happen). - Then maintain a high watermark for the last archival, i.e. no archived series has a sample more recent than that watermark. - Any query that doesn't reach to a time before that watermark doesn't have to touch the archive index at all. (A production server at Soundcloud with the aforementioned series churn and heavy rule evaluations spends 50% of its CPU time in archive index lookups. Since rule evaluations usually only touch very recent values, most of those lookup should disappear with this change.) - Federation with a very broad label matcher will profit from this, too. As a byproduct, the un-needed MetricForFingerprint method was removed from the Storage interface.
This commit is contained in:
parent
d77d625ad3
commit
836f1db04c
|
@ -79,7 +79,10 @@ func (a *Analyzer) Analyze(ctx context.Context) error {
|
||||||
Inspect(a.Expr, func(node Node) bool {
|
Inspect(a.Expr, func(node Node) bool {
|
||||||
switch n := node.(type) {
|
switch n := node.(type) {
|
||||||
case *VectorSelector:
|
case *VectorSelector:
|
||||||
n.metrics = a.Storage.MetricsForLabelMatchers(n.LabelMatchers...)
|
n.metrics = a.Storage.MetricsForLabelMatchers(
|
||||||
|
a.Start.Add(-n.Offset-StalenessDelta), a.End.Add(-n.Offset),
|
||||||
|
n.LabelMatchers...,
|
||||||
|
)
|
||||||
n.iterators = make(map[model.Fingerprint]local.SeriesIterator, len(n.metrics))
|
n.iterators = make(map[model.Fingerprint]local.SeriesIterator, len(n.metrics))
|
||||||
|
|
||||||
pt := getPreloadTimes(n.Offset)
|
pt := getPreloadTimes(n.Offset)
|
||||||
|
@ -95,7 +98,10 @@ func (a *Analyzer) Analyze(ctx context.Context) error {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
case *MatrixSelector:
|
case *MatrixSelector:
|
||||||
n.metrics = a.Storage.MetricsForLabelMatchers(n.LabelMatchers...)
|
n.metrics = a.Storage.MetricsForLabelMatchers(
|
||||||
|
a.Start.Add(-n.Offset-n.Range), a.End.Add(-n.Offset),
|
||||||
|
n.LabelMatchers...,
|
||||||
|
)
|
||||||
n.iterators = make(map[model.Fingerprint]local.SeriesIterator, len(n.metrics))
|
n.iterators = make(map[model.Fingerprint]local.SeriesIterator, len(n.metrics))
|
||||||
|
|
||||||
pt := getPreloadTimes(n.Offset)
|
pt := getPreloadTimes(n.Offset)
|
||||||
|
|
|
@ -40,20 +40,22 @@ type Storage interface {
|
||||||
// NewPreloader returns a new Preloader which allows preloading and pinning
|
// NewPreloader returns a new Preloader which allows preloading and pinning
|
||||||
// series data into memory for use within a query.
|
// series data into memory for use within a query.
|
||||||
NewPreloader() Preloader
|
NewPreloader() Preloader
|
||||||
// MetricsForLabelMatchers returns the metrics from storage that satisfy the given
|
// MetricsForLabelMatchers returns the metrics from storage that satisfy
|
||||||
// label matchers. At least one label matcher must be specified that does not
|
// the given label matchers. At least one label matcher must be
|
||||||
// match the empty string.
|
// specified that does not match the empty string. The times from and
|
||||||
MetricsForLabelMatchers(...*metric.LabelMatcher) map[model.Fingerprint]metric.Metric
|
// through are hints for the storage to optimize the search. The storage
|
||||||
// LastSamplePairForFingerprint returns the last sample pair that has
|
// MAY exclude metrics that have no samples in the specified interval
|
||||||
// been ingested for the provided fingerprint. If this instance of the
|
// from the returned map. In doubt, specify model.Earliest for from and
|
||||||
|
// model.Latest for through.
|
||||||
|
MetricsForLabelMatchers(from, through model.Time, matchers ...*metric.LabelMatcher) map[model.Fingerprint]metric.Metric
|
||||||
|
// LastSampleForFingerprint returns the last sample that has been
|
||||||
|
// ingested for the provided fingerprint. If this instance of the
|
||||||
// Storage has never ingested a sample for the provided fingerprint (or
|
// Storage has never ingested a sample for the provided fingerprint (or
|
||||||
// the last ingestion is so long ago that the series has been archived),
|
// the last ingestion is so long ago that the series has been archived),
|
||||||
// ZeroSamplePair is returned.
|
// ZeroSample is returned.
|
||||||
LastSamplePairForFingerprint(model.Fingerprint) model.SamplePair
|
LastSampleForFingerprint(model.Fingerprint) model.Sample
|
||||||
// Get all of the label values that are associated with a given label name.
|
// Get all of the label values that are associated with a given label name.
|
||||||
LabelValuesForLabelName(model.LabelName) model.LabelValues
|
LabelValuesForLabelName(model.LabelName) model.LabelValues
|
||||||
// Get the metric associated with the provided fingerprint.
|
|
||||||
MetricForFingerprint(model.Fingerprint) metric.Metric
|
|
||||||
// Drop all time series associated with the given fingerprints.
|
// Drop all time series associated with the given fingerprints.
|
||||||
DropMetricsForFingerprints(...model.Fingerprint)
|
DropMetricsForFingerprints(...model.Fingerprint)
|
||||||
// Run the various maintenance loops in goroutines. Returns when the
|
// Run the various maintenance loops in goroutines. Returns when the
|
||||||
|
@ -89,7 +91,7 @@ type SeriesIterator interface {
|
||||||
type Preloader interface {
|
type Preloader interface {
|
||||||
PreloadRange(
|
PreloadRange(
|
||||||
fp model.Fingerprint,
|
fp model.Fingerprint,
|
||||||
from model.Time, through model.Time,
|
from, through model.Time,
|
||||||
) SeriesIterator
|
) SeriesIterator
|
||||||
PreloadInstant(
|
PreloadInstant(
|
||||||
fp model.Fingerprint,
|
fp model.Fingerprint,
|
||||||
|
@ -100,8 +102,15 @@ type Preloader interface {
|
||||||
}
|
}
|
||||||
|
|
||||||
// ZeroSamplePair is the pseudo zero-value of model.SamplePair used by the local
|
// ZeroSamplePair is the pseudo zero-value of model.SamplePair used by the local
|
||||||
// package to signal a non-existing sample. It is a SamplePair with timestamp
|
// package to signal a non-existing sample pair. It is a SamplePair with
|
||||||
// model.Earliest and value 0.0. Note that the natural zero value of SamplePair
|
// timestamp model.Earliest and value 0.0. Note that the natural zero value of
|
||||||
// has a timestamp of 0, which is possible to appear in a real SamplePair and
|
// SamplePair has a timestamp of 0, which is possible to appear in a real
|
||||||
// thus not suitable to signal a non-existing SamplePair.
|
// SamplePair and thus not suitable to signal a non-existing SamplePair.
|
||||||
var ZeroSamplePair = model.SamplePair{Timestamp: model.Earliest}
|
var ZeroSamplePair = model.SamplePair{Timestamp: model.Earliest}
|
||||||
|
|
||||||
|
// ZeroSample is the pseudo zero-value of model.Sample used by the local package
|
||||||
|
// to signal a non-existing sample. It is a Sample with timestamp
|
||||||
|
// model.Earliest, value 0.0, and metric nil. Note that the natural zero value
|
||||||
|
// of Sample has a timestamp of 0, which is possible to appear in a real
|
||||||
|
// Sample and thus not suitable to signal a non-existing Sample.
|
||||||
|
var ZeroSample = model.Sample{Timestamp: model.Earliest}
|
||||||
|
|
|
@ -1068,6 +1068,9 @@ func (p *persistence) fingerprintsModifiedBefore(beforeTime model.Time) ([]model
|
||||||
// method is goroutine-safe.
|
// method is goroutine-safe.
|
||||||
func (p *persistence) archivedMetric(fp model.Fingerprint) (model.Metric, error) {
|
func (p *persistence) archivedMetric(fp model.Fingerprint) (model.Metric, error) {
|
||||||
metric, _, err := p.archivedFingerprintToMetrics.Lookup(fp)
|
metric, _, err := p.archivedFingerprintToMetrics.Lookup(fp)
|
||||||
|
if err != nil {
|
||||||
|
p.setDirty(true, err)
|
||||||
|
}
|
||||||
return metric, err
|
return metric, err
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -129,12 +129,13 @@ const (
|
||||||
type syncStrategy func() bool
|
type syncStrategy func() bool
|
||||||
|
|
||||||
type memorySeriesStorage struct {
|
type memorySeriesStorage struct {
|
||||||
// numChunksToPersist has to be aligned for atomic operations.
|
// archiveHighWatermark and numChunksToPersist have to be aligned for atomic operations.
|
||||||
numChunksToPersist int64 // The number of chunks waiting for persistence.
|
archiveHighWatermark model.Time // No archived series has samples after this time.
|
||||||
maxChunksToPersist int // If numChunksToPersist reaches this threshold, ingestion will be throttled.
|
numChunksToPersist int64 // The number of chunks waiting for persistence.
|
||||||
rushed bool // Whether the storage is in rushed mode.
|
maxChunksToPersist int // If numChunksToPersist reaches this threshold, ingestion will be throttled.
|
||||||
rushedMtx sync.Mutex // Protects entering and exiting rushed mode.
|
rushed bool // Whether the storage is in rushed mode.
|
||||||
throttled chan struct{} // This chan is sent to whenever NeedsThrottling() returns true (for logging).
|
rushedMtx sync.Mutex // Protects entering and exiting rushed mode.
|
||||||
|
throttled chan struct{} // This chan is sent to whenever NeedsThrottling() returns true (for logging).
|
||||||
|
|
||||||
fpLocker *fingerprintLocker
|
fpLocker *fingerprintLocker
|
||||||
fpToSeries *seriesMap
|
fpToSeries *seriesMap
|
||||||
|
@ -201,6 +202,7 @@ func NewMemorySeriesStorage(o *MemorySeriesStorageOptions) Storage {
|
||||||
dropAfter: o.PersistenceRetentionPeriod,
|
dropAfter: o.PersistenceRetentionPeriod,
|
||||||
checkpointInterval: o.CheckpointInterval,
|
checkpointInterval: o.CheckpointInterval,
|
||||||
checkpointDirtySeriesLimit: o.CheckpointDirtySeriesLimit,
|
checkpointDirtySeriesLimit: o.CheckpointDirtySeriesLimit,
|
||||||
|
archiveHighWatermark: model.Now().Add(-headChunkTimeout),
|
||||||
|
|
||||||
maxChunksToPersist: o.MaxChunksToPersist,
|
maxChunksToPersist: o.MaxChunksToPersist,
|
||||||
|
|
||||||
|
@ -368,15 +370,20 @@ func (s *memorySeriesStorage) WaitForIndexing() {
|
||||||
}
|
}
|
||||||
|
|
||||||
// LastSampleForFingerprint implements Storage.
|
// LastSampleForFingerprint implements Storage.
|
||||||
func (s *memorySeriesStorage) LastSamplePairForFingerprint(fp model.Fingerprint) model.SamplePair {
|
func (s *memorySeriesStorage) LastSampleForFingerprint(fp model.Fingerprint) model.Sample {
|
||||||
s.fpLocker.Lock(fp)
|
s.fpLocker.Lock(fp)
|
||||||
defer s.fpLocker.Unlock(fp)
|
defer s.fpLocker.Unlock(fp)
|
||||||
|
|
||||||
series, ok := s.fpToSeries.get(fp)
|
series, ok := s.fpToSeries.get(fp)
|
||||||
if !ok {
|
if !ok {
|
||||||
return ZeroSamplePair
|
return ZeroSample
|
||||||
|
}
|
||||||
|
sp := series.lastSamplePair()
|
||||||
|
return model.Sample{
|
||||||
|
Metric: series.metric,
|
||||||
|
Value: sp.Value,
|
||||||
|
Timestamp: sp.Timestamp,
|
||||||
}
|
}
|
||||||
return series.lastSamplePair()
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// boundedIterator wraps a SeriesIterator and does not allow fetching
|
// boundedIterator wraps a SeriesIterator and does not allow fetching
|
||||||
|
@ -439,7 +446,10 @@ func (s *memorySeriesStorage) fingerprintsForLabelPairs(pairs ...model.LabelPair
|
||||||
}
|
}
|
||||||
|
|
||||||
// MetricsForLabelMatchers implements Storage.
|
// MetricsForLabelMatchers implements Storage.
|
||||||
func (s *memorySeriesStorage) MetricsForLabelMatchers(matchers ...*metric.LabelMatcher) map[model.Fingerprint]metric.Metric {
|
func (s *memorySeriesStorage) MetricsForLabelMatchers(
|
||||||
|
from, through model.Time,
|
||||||
|
matchers ...*metric.LabelMatcher,
|
||||||
|
) map[model.Fingerprint]metric.Metric {
|
||||||
var (
|
var (
|
||||||
equals []model.LabelPair
|
equals []model.LabelPair
|
||||||
filters []*metric.LabelMatcher
|
filters []*metric.LabelMatcher
|
||||||
|
@ -491,9 +501,11 @@ func (s *memorySeriesStorage) MetricsForLabelMatchers(matchers ...*metric.LabelM
|
||||||
filters = remaining
|
filters = remaining
|
||||||
}
|
}
|
||||||
|
|
||||||
result := make(map[model.Fingerprint]metric.Metric, len(resFPs))
|
result := map[model.Fingerprint]metric.Metric{}
|
||||||
for fp := range resFPs {
|
for fp := range resFPs {
|
||||||
result[fp] = s.MetricForFingerprint(fp)
|
if metric, ok := s.metricForFingerprint(fp, from, through); ok {
|
||||||
|
result[fp] = metric
|
||||||
|
}
|
||||||
}
|
}
|
||||||
for _, matcher := range filters {
|
for _, matcher := range filters {
|
||||||
for fp, met := range result {
|
for fp, met := range result {
|
||||||
|
@ -505,6 +517,58 @@ func (s *memorySeriesStorage) MetricsForLabelMatchers(matchers ...*metric.LabelM
|
||||||
return result
|
return result
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// metricForFingerprint returns the metric for the given fingerprint if the
|
||||||
|
// corresponding time series has samples between 'from' and 'through'.
|
||||||
|
func (s *memorySeriesStorage) metricForFingerprint(
|
||||||
|
fp model.Fingerprint,
|
||||||
|
from, through model.Time,
|
||||||
|
) (metric.Metric, bool) {
|
||||||
|
// Lock FP so that no (un-)archiving will happen during lookup.
|
||||||
|
s.fpLocker.Lock(fp)
|
||||||
|
defer s.fpLocker.Unlock(fp)
|
||||||
|
|
||||||
|
watermark := model.Time(atomic.LoadInt64((*int64)(&s.archiveHighWatermark)))
|
||||||
|
|
||||||
|
series, ok := s.fpToSeries.get(fp)
|
||||||
|
if ok {
|
||||||
|
if series.lastTime.Before(from) || series.savedFirstTime.After(through) {
|
||||||
|
return metric.Metric{}, false
|
||||||
|
}
|
||||||
|
// Wrap the returned metric in a copy-on-write (COW) metric here because
|
||||||
|
// the caller might mutate it.
|
||||||
|
return metric.Metric{
|
||||||
|
Metric: series.metric,
|
||||||
|
}, true
|
||||||
|
}
|
||||||
|
// From here on, we are only concerned with archived metrics.
|
||||||
|
// If the high watermark of archived series is before 'from', we are done.
|
||||||
|
if watermark < from {
|
||||||
|
return metric.Metric{}, false
|
||||||
|
}
|
||||||
|
if from.After(model.Earliest) || through.Before(model.Latest) {
|
||||||
|
// The range lookup is relatively cheap, so let's do it first.
|
||||||
|
ok, first, last, err := s.persistence.hasArchivedMetric(fp)
|
||||||
|
if err != nil {
|
||||||
|
log.Errorf("Error retrieving archived time range for fingerprint %v: %v", fp, err)
|
||||||
|
return metric.Metric{}, false
|
||||||
|
}
|
||||||
|
if !ok || first.After(through) || last.Before(from) {
|
||||||
|
return metric.Metric{}, false
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
met, err := s.persistence.archivedMetric(fp)
|
||||||
|
if err != nil {
|
||||||
|
log.Errorf("Error retrieving archived metric for fingerprint %v: %v", fp, err)
|
||||||
|
return metric.Metric{}, false
|
||||||
|
}
|
||||||
|
|
||||||
|
return metric.Metric{
|
||||||
|
Metric: met,
|
||||||
|
Copied: false,
|
||||||
|
}, true
|
||||||
|
}
|
||||||
|
|
||||||
// LabelValuesForLabelName implements Storage.
|
// LabelValuesForLabelName implements Storage.
|
||||||
func (s *memorySeriesStorage) LabelValuesForLabelName(labelName model.LabelName) model.LabelValues {
|
func (s *memorySeriesStorage) LabelValuesForLabelName(labelName model.LabelName) model.LabelValues {
|
||||||
lvs, err := s.persistence.labelValuesForLabelName(labelName)
|
lvs, err := s.persistence.labelValuesForLabelName(labelName)
|
||||||
|
@ -514,30 +578,6 @@ func (s *memorySeriesStorage) LabelValuesForLabelName(labelName model.LabelName)
|
||||||
return lvs
|
return lvs
|
||||||
}
|
}
|
||||||
|
|
||||||
// MetricForFingerprint implements Storage.
|
|
||||||
func (s *memorySeriesStorage) MetricForFingerprint(fp model.Fingerprint) metric.Metric {
|
|
||||||
s.fpLocker.Lock(fp)
|
|
||||||
defer s.fpLocker.Unlock(fp)
|
|
||||||
|
|
||||||
series, ok := s.fpToSeries.get(fp)
|
|
||||||
if ok {
|
|
||||||
// Wrap the returned metric in a copy-on-write (COW) metric here because
|
|
||||||
// the caller might mutate it.
|
|
||||||
return metric.Metric{
|
|
||||||
Metric: series.metric,
|
|
||||||
}
|
|
||||||
}
|
|
||||||
met, err := s.persistence.archivedMetric(fp)
|
|
||||||
if err != nil {
|
|
||||||
log.Errorf("Error retrieving archived metric for fingerprint %v: %v", fp, err)
|
|
||||||
}
|
|
||||||
|
|
||||||
return metric.Metric{
|
|
||||||
Metric: met,
|
|
||||||
Copied: false,
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// DropMetric implements Storage.
|
// DropMetric implements Storage.
|
||||||
func (s *memorySeriesStorage) DropMetricsForFingerprints(fps ...model.Fingerprint) {
|
func (s *memorySeriesStorage) DropMetricsForFingerprints(fps ...model.Fingerprint) {
|
||||||
for _, fp := range fps {
|
for _, fp := range fps {
|
||||||
|
@ -1077,8 +1117,9 @@ func (s *memorySeriesStorage) maintainMemorySeries(
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// Archive if all chunks are evicted.
|
// Archive if all chunks are evicted. Also make sure the last sample has
|
||||||
if iOldestNotEvicted == -1 {
|
// an age of at least headChunkTimeout (which is very likely anyway).
|
||||||
|
if iOldestNotEvicted == -1 && model.Now().Sub(series.lastTime) > headChunkTimeout {
|
||||||
s.fpToSeries.del(fp)
|
s.fpToSeries.del(fp)
|
||||||
s.numSeries.Dec()
|
s.numSeries.Dec()
|
||||||
if err := s.persistence.archiveMetric(
|
if err := s.persistence.archiveMetric(
|
||||||
|
@ -1088,6 +1129,15 @@ func (s *memorySeriesStorage) maintainMemorySeries(
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
s.seriesOps.WithLabelValues(archive).Inc()
|
s.seriesOps.WithLabelValues(archive).Inc()
|
||||||
|
oldWatermark := atomic.LoadInt64((*int64)(&s.archiveHighWatermark))
|
||||||
|
if oldWatermark < int64(series.lastTime) {
|
||||||
|
if !atomic.CompareAndSwapInt64(
|
||||||
|
(*int64)(&s.archiveHighWatermark),
|
||||||
|
oldWatermark, int64(series.lastTime),
|
||||||
|
) {
|
||||||
|
panic("s.archiveHighWatermark modified outside of maintainMemorySeries")
|
||||||
|
}
|
||||||
|
}
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
// If we are here, the series is not archived, so check for chunkDesc
|
// If we are here, the series is not archived, so check for chunkDesc
|
||||||
|
|
|
@ -178,7 +178,10 @@ func TestMatches(t *testing.T) {
|
||||||
}
|
}
|
||||||
|
|
||||||
for _, mt := range matcherTests {
|
for _, mt := range matcherTests {
|
||||||
res := storage.MetricsForLabelMatchers(mt.matchers...)
|
res := storage.MetricsForLabelMatchers(
|
||||||
|
model.Earliest, model.Latest,
|
||||||
|
mt.matchers...,
|
||||||
|
)
|
||||||
if len(mt.expected) != len(res) {
|
if len(mt.expected) != len(res) {
|
||||||
t.Fatalf("expected %d matches for %q, found %d", len(mt.expected), mt.matchers, len(res))
|
t.Fatalf("expected %d matches for %q, found %d", len(mt.expected), mt.matchers, len(res))
|
||||||
}
|
}
|
||||||
|
@ -362,7 +365,10 @@ func BenchmarkLabelMatching(b *testing.B) {
|
||||||
for i := 0; i < b.N; i++ {
|
for i := 0; i < b.N; i++ {
|
||||||
benchLabelMatchingRes = map[model.Fingerprint]metric.Metric{}
|
benchLabelMatchingRes = map[model.Fingerprint]metric.Metric{}
|
||||||
for _, mt := range matcherTests {
|
for _, mt := range matcherTests {
|
||||||
benchLabelMatchingRes = s.MetricsForLabelMatchers(mt...)
|
benchLabelMatchingRes = s.MetricsForLabelMatchers(
|
||||||
|
model.Earliest, model.Latest,
|
||||||
|
mt...,
|
||||||
|
)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
// Stop timer to not count the storage closing.
|
// Stop timer to not count the storage closing.
|
||||||
|
|
|
@ -226,7 +226,10 @@ func (api *API) series(r *http.Request) (interface{}, *apiError) {
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, &apiError{errorBadData, err}
|
return nil, &apiError{errorBadData, err}
|
||||||
}
|
}
|
||||||
for fp, met := range api.Storage.MetricsForLabelMatchers(matchers...) {
|
for fp, met := range api.Storage.MetricsForLabelMatchers(
|
||||||
|
model.Earliest, model.Latest, // Get every series.
|
||||||
|
matchers...,
|
||||||
|
) {
|
||||||
res[fp] = met
|
res[fp] = met
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -250,7 +253,10 @@ func (api *API) dropSeries(r *http.Request) (interface{}, *apiError) {
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, &apiError{errorBadData, err}
|
return nil, &apiError{errorBadData, err}
|
||||||
}
|
}
|
||||||
for fp := range api.Storage.MetricsForLabelMatchers(matchers...) {
|
for fp := range api.Storage.MetricsForLabelMatchers(
|
||||||
|
model.Earliest, model.Latest, // Get every series.
|
||||||
|
matchers...,
|
||||||
|
) {
|
||||||
fps[fp] = struct{}{}
|
fps[fp] = struct{}{}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,7 +19,6 @@ import (
|
||||||
"github.com/golang/protobuf/proto"
|
"github.com/golang/protobuf/proto"
|
||||||
|
|
||||||
"github.com/prometheus/prometheus/promql"
|
"github.com/prometheus/prometheus/promql"
|
||||||
"github.com/prometheus/prometheus/storage/metric"
|
|
||||||
|
|
||||||
"github.com/prometheus/common/expfmt"
|
"github.com/prometheus/common/expfmt"
|
||||||
"github.com/prometheus/common/model"
|
"github.com/prometheus/common/model"
|
||||||
|
@ -33,7 +32,7 @@ func (h *Handler) federation(w http.ResponseWriter, req *http.Request) {
|
||||||
|
|
||||||
req.ParseForm()
|
req.ParseForm()
|
||||||
|
|
||||||
metrics := map[model.Fingerprint]metric.Metric{}
|
fps := map[model.Fingerprint]struct{}{}
|
||||||
|
|
||||||
for _, s := range req.Form["match[]"] {
|
for _, s := range req.Form["match[]"] {
|
||||||
matchers, err := promql.ParseMetricSelector(s)
|
matchers, err := promql.ParseMetricSelector(s)
|
||||||
|
@ -41,8 +40,11 @@ func (h *Handler) federation(w http.ResponseWriter, req *http.Request) {
|
||||||
http.Error(w, err.Error(), http.StatusBadRequest)
|
http.Error(w, err.Error(), http.StatusBadRequest)
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
for fp, met := range h.storage.MetricsForLabelMatchers(matchers...) {
|
for fp := range h.storage.MetricsForLabelMatchers(
|
||||||
metrics[fp] = met
|
model.Now().Add(-promql.StalenessDelta), model.Latest,
|
||||||
|
matchers...,
|
||||||
|
) {
|
||||||
|
fps[fp] = struct{}{}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -62,19 +64,19 @@ func (h *Handler) federation(w http.ResponseWriter, req *http.Request) {
|
||||||
Type: dto.MetricType_UNTYPED.Enum(),
|
Type: dto.MetricType_UNTYPED.Enum(),
|
||||||
}
|
}
|
||||||
|
|
||||||
for fp, met := range metrics {
|
for fp := range fps {
|
||||||
globalUsed := map[model.LabelName]struct{}{}
|
globalUsed := map[model.LabelName]struct{}{}
|
||||||
|
|
||||||
sp := h.storage.LastSamplePairForFingerprint(fp)
|
s := h.storage.LastSampleForFingerprint(fp)
|
||||||
// Discard if sample does not exist or lays before the staleness interval.
|
// Discard if sample does not exist or lays before the staleness interval.
|
||||||
if sp.Timestamp.Before(minTimestamp) {
|
if s.Timestamp.Before(minTimestamp) {
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
|
|
||||||
// Reset label slice.
|
// Reset label slice.
|
||||||
protMetric.Label = protMetric.Label[:0]
|
protMetric.Label = protMetric.Label[:0]
|
||||||
|
|
||||||
for ln, lv := range met.Metric {
|
for ln, lv := range s.Metric {
|
||||||
if ln == model.MetricNameLabel {
|
if ln == model.MetricNameLabel {
|
||||||
protMetricFam.Name = proto.String(string(lv))
|
protMetricFam.Name = proto.String(string(lv))
|
||||||
continue
|
continue
|
||||||
|
@ -98,8 +100,8 @@ func (h *Handler) federation(w http.ResponseWriter, req *http.Request) {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
protMetric.TimestampMs = proto.Int64(int64(sp.Timestamp))
|
protMetric.TimestampMs = proto.Int64(int64(s.Timestamp))
|
||||||
protMetric.Untyped.Value = proto.Float64(float64(sp.Value))
|
protMetric.Untyped.Value = proto.Float64(float64(s.Value))
|
||||||
|
|
||||||
if err := enc.Encode(protMetricFam); err != nil {
|
if err := enc.Encode(protMetricFam); err != nil {
|
||||||
http.Error(w, err.Error(), http.StatusInternalServerError)
|
http.Error(w, err.Error(), http.StatusInternalServerError)
|
||||||
|
|
Loading…
Reference in a new issue