From 0edfd7d95d0206031df26fed13ba107b8b91dcef Mon Sep 17 00:00:00 2001 From: Justin Jung Date: Fri, 19 Jul 2024 17:23:18 -0700 Subject: [PATCH 01/32] Fix user queue in scheduler that was not thread-safe (#6077) --- CHANGELOG.md | 2 ++ pkg/scheduler/queue/user_queues.go | 13 +++++++++- pkg/scheduler/queue/user_queues_test.go | 34 +++++++++++++++++++++++++ 3 files changed, 48 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index ba6acb3175..b992fbf7e2 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -27,10 +27,12 @@ * [ENHANCEMENT] Ruler: Add support for filtering by `match` field on Rules API. #6083 * [ENHANCEMENT] Distributor: Reduce memory usage when error volume is high. #6095 * [ENHANCEMENT] Compactor: Add unique execution ID for each compaction cycle in log for easy debugging. #6097 +* [ENHANCEMENT] Ruler: Add support for filtering by `state` and `health` field on Rules API. #6040 * [BUGFIX] Configsdb: Fix endline issue in db password. #5920 * [BUGFIX] Ingester: Fix `user` and `type` labels for the `cortex_ingester_tsdb_head_samples_appended_total` TSDB metric. #5952 * [BUGFIX] Querier: Enforce max query length check for `/api/v1/series` API even though `ignoreMaxQueryLength` is set to true. #6018 * [BUGFIX] Ingester: Fix issue with the minimize token generator where it was not taking in consideration the current ownerhip of an instance when generating extra tokens. #6062 +* [BUGFIX] Scheduler: Fix user queue in scheduler that was not thread-safe. #6077 ## 1.17.1 2024-05-20 diff --git a/pkg/scheduler/queue/user_queues.go b/pkg/scheduler/queue/user_queues.go index 25f562ee02..159df7810b 100644 --- a/pkg/scheduler/queue/user_queues.go +++ b/pkg/scheduler/queue/user_queues.go @@ -3,6 +3,7 @@ package queue import ( "math/rand" "sort" + "sync" "time" "github.com/prometheus/client_golang/prometheus" @@ -37,7 +38,8 @@ type querier struct { // This struct holds user queues for pending requests. It also keeps track of connected queriers, // and mapping between users and queriers. type queues struct { - userQueues map[string]*userQueue + userQueues map[string]*userQueue + userQueuesMx sync.RWMutex // List of all users with queues, used for iteration when searching for next queue to handle. // Users removed from the middle are replaced with "". To avoid skipping users during iteration, we only shrink @@ -103,6 +105,9 @@ func (q *queues) len() int { } func (q *queues) deleteQueue(userID string) { + q.userQueuesMx.Lock() + defer q.userQueuesMx.Unlock() + uq := q.userQueues[userID] if uq == nil { return @@ -132,6 +137,9 @@ func (q *queues) getOrAddQueue(userID string, maxQueriers int) userRequestQueue maxQueriers = 0 } + q.userQueuesMx.Lock() + defer q.userQueuesMx.Unlock() + uq := q.userQueues[userID] priorityEnabled := q.limits.QueryPriority(userID).Enabled maxOutstanding := q.limits.MaxOutstandingPerTenant(userID) @@ -237,6 +245,9 @@ func (q *queues) getNextQueueForQuerier(lastUserIndex int, querierID string) (us continue } + q.userQueuesMx.RLock() + defer q.userQueuesMx.RUnlock() + uq := q.userQueues[u] if uq.queriers != nil { diff --git a/pkg/scheduler/queue/user_queues_test.go b/pkg/scheduler/queue/user_queues_test.go index ded597baa0..4e720de402 100644 --- a/pkg/scheduler/queue/user_queues_test.go +++ b/pkg/scheduler/queue/user_queues_test.go @@ -5,6 +5,7 @@ import ( "math" "math/rand" "sort" + "sync" "testing" "time" @@ -457,6 +458,39 @@ func TestGetOrAddQueueShouldUpdateProperties(t *testing.T) { } } +func TestQueueConcurrency(t *testing.T) { + const numGoRoutines = 30 + limits := MockLimits{ + MaxOutstanding: 50, + } + q := newUserQueues(0, 0, limits, nil) + q.addQuerierConnection("q-1") + q.addQuerierConnection("q-2") + q.addQuerierConnection("q-3") + q.addQuerierConnection("q-4") + q.addQuerierConnection("q-5") + + var wg sync.WaitGroup + wg.Add(numGoRoutines) + + for i := 0; i < numGoRoutines; i++ { + go func(cnt int) { + defer wg.Done() + queue := q.getOrAddQueue("userID", 2) + if cnt%2 == 0 { + queue.enqueueRequest(MockRequest{}) + q.getNextQueueForQuerier(0, "q-1") + } else if cnt%5 == 0 { + queue.dequeueRequest(0, false) + } else if cnt%7 == 0 { + q.deleteQueue("userID") + } + }(i) + } + + wg.Wait() +} + func generateTenant(r *rand.Rand) string { return fmt.Sprint("tenant-", r.Int()%5) } From ca541eb7db678fdcb07e0597dd59f3e5f563b1db Mon Sep 17 00:00:00 2001 From: Ben Ye Date: Fri, 19 Jul 2024 22:36:52 -0700 Subject: [PATCH 02/32] implement max bucket limit and automatic resolution reduction Signed-off-by: Ben Ye --- pkg/cortexpb/histograms.go | 5 ++ pkg/distributor/distributor.go | 10 ++- pkg/util/validation/errors.go | 18 +++++ pkg/util/validation/limits.go | 10 ++- pkg/util/validation/validate.go | 56 ++++++++++++++ pkg/util/validation/validate_test.go | 109 +++++++++++++++++++++++++++ 6 files changed, 204 insertions(+), 4 deletions(-) diff --git a/pkg/cortexpb/histograms.go b/pkg/cortexpb/histograms.go index 2e2afef457..129b25d300 100644 --- a/pkg/cortexpb/histograms.go +++ b/pkg/cortexpb/histograms.go @@ -15,6 +15,11 @@ package cortexpb import "github.com/prometheus/prometheus/model/histogram" +const ( + ExponentialSchemaMax int32 = 8 + ExponentialSchemaMin int32 = -4 +) + func (h Histogram) IsFloatHistogram() bool { _, ok := h.GetCount().(*Histogram_CountFloat) return ok diff --git a/pkg/distributor/distributor.go b/pkg/distributor/distributor.go index 055053b436..664a221b10 100644 --- a/pkg/distributor/distributor.go +++ b/pkg/distributor/distributor.go @@ -579,12 +579,16 @@ func (d *Distributor) validateSeries(ts cortexpb.PreallocTimeseries, userID stri if len(ts.Histograms) > 0 { // Only alloc when data present histograms = make([]cortexpb.Histogram, 0, len(ts.Histograms)) - for _, h := range ts.Histograms { - // TODO(yeya24): add other validations for native histogram. - // For example, Prometheus scrape has bucket limit and schema check. + for i, h := range ts.Histograms { if err := validation.ValidateSampleTimestamp(d.validateMetrics, limits, userID, ts.Labels, h.TimestampMs); err != nil { return emptyPreallocSeries, err } + // TODO(yeya24): add max schema validation for native histogram if needed. + convertedHistogram, err := validation.ValidateNativeHistogram(d.validateMetrics, limits, userID, ts.Labels, h) + if err != nil { + return emptyPreallocSeries, err + } + ts.Histograms[i] = convertedHistogram } histograms = append(histograms, ts.Histograms...) } diff --git a/pkg/util/validation/errors.go b/pkg/util/validation/errors.go index 483bf6acc5..29fa69b0c0 100644 --- a/pkg/util/validation/errors.go +++ b/pkg/util/validation/errors.go @@ -225,6 +225,24 @@ func newExemplarLabelLengthError(seriesLabels []cortexpb.LabelAdapter, exemplarL } } +// histogramBucketLimitExceededError is a ValidationError implementation for samples with native histogram +// exceeding max bucket limit and cannot reduce resolution further to be within the max bucket limit. +type histogramBucketLimitExceededError struct { + series []cortexpb.LabelAdapter + limit int +} + +func newHistogramBucketLimitExceededError(series []cortexpb.LabelAdapter, limit int) ValidationError { + return &histogramBucketLimitExceededError{ + series: series, + limit: limit, + } +} + +func (e *histogramBucketLimitExceededError) Error() string { + return fmt.Sprintf("native histogram bucket count exceeded for metric (limit: %d) metric: %.200q", e.limit, formatLabelSet(e.series)) +} + // formatLabelSet formats label adapters as a metric name with labels, while preserving // label order, and keeping duplicates. If there are multiple "__name__" labels, only // first one is used as metric name, other ones will be included as regular labels. diff --git a/pkg/util/validation/limits.go b/pkg/util/validation/limits.go index 4586a24622..280e797079 100644 --- a/pkg/util/validation/limits.go +++ b/pkg/util/validation/limits.go @@ -134,7 +134,7 @@ type Limits struct { EnforceMetricName bool `yaml:"enforce_metric_name" json:"enforce_metric_name"` IngestionTenantShardSize int `yaml:"ingestion_tenant_shard_size" json:"ingestion_tenant_shard_size"` MetricRelabelConfigs []*relabel.Config `yaml:"metric_relabel_configs,omitempty" json:"metric_relabel_configs,omitempty" doc:"nocli|description=List of metric relabel configurations. Note that in most situations, it is more effective to use metrics relabeling directly in the Prometheus server, e.g. remote_write.write_relabel_configs."` - MaxExemplars int `yaml:"max_exemplars" json:"max_exemplars"` + MaxNativeHistogramBuckets int `yaml:"max_native_histogram_buckets" json:"max_native_histogram_buckets"` // Ingester enforced limits. // Series @@ -151,6 +151,8 @@ type Limits struct { MaxGlobalMetadataPerMetric int `yaml:"max_global_metadata_per_metric" json:"max_global_metadata_per_metric"` // Out-of-order OutOfOrderTimeWindow model.Duration `yaml:"out_of_order_time_window" json:"out_of_order_time_window"` + // Exemplars + MaxExemplars int `yaml:"max_exemplars" json:"max_exemplars"` // Querier enforced limits. MaxChunksPerQuery int `yaml:"max_fetched_chunks_per_query" json:"max_fetched_chunks_per_query"` @@ -232,6 +234,7 @@ func (l *Limits) RegisterFlags(f *flag.FlagSet) { f.Var(&l.CreationGracePeriod, "validation.create-grace-period", "Duration which table will be created/deleted before/after it's needed; we won't accept sample from before this time.") f.BoolVar(&l.EnforceMetricName, "validation.enforce-metric-name", true, "Enforce every sample has a metric name.") f.BoolVar(&l.EnforceMetadataMetricName, "validation.enforce-metadata-metric-name", true, "Enforce every metadata has a metric name.") + f.IntVar(&l.MaxNativeHistogramBuckets, "validation.max-native-histogram-buckets", 0, ".") f.IntVar(&l.MaxLocalSeriesPerUser, "ingester.max-series-per-user", 5000000, "The maximum number of active series per user, per ingester. 0 to disable.") f.IntVar(&l.MaxLocalSeriesPerMetric, "ingester.max-series-per-metric", 50000, "The maximum number of active series per metric name, per ingester. 0 to disable.") @@ -722,6 +725,11 @@ func (o *Overrides) EnforceMetadataMetricName(userID string) bool { return o.GetOverridesForUser(userID).EnforceMetadataMetricName } +// MaxNativeHistogramBuckets. +func (o *Overrides) MaxNativeHistogramBuckets(userID string) int { + return o.GetOverridesForUser(userID).MaxNativeHistogramBuckets +} + // MaxLocalMetricsWithMetadataPerUser returns the maximum number of metrics with metadata a user is allowed to store in a single ingester. func (o *Overrides) MaxLocalMetricsWithMetadataPerUser(userID string) int { return o.GetOverridesForUser(userID).MaxLocalMetricsWithMetadataPerUser diff --git a/pkg/util/validation/validate.go b/pkg/util/validation/validate.go index f03b8b74b1..031d4ac9a5 100644 --- a/pkg/util/validation/validate.go +++ b/pkg/util/validation/validate.go @@ -52,6 +52,9 @@ const ( exemplarLabelsTooLong = "exemplar_labels_too_long" exemplarTimestampInvalid = "exemplar_timestamp_invalid" + // Native Histogram specific validation reasons + nativeHistogramBucketsExceeded = "native_histogram_buckets_exceeded" + // RateLimited is one of the values for the reason to discard samples. // Declared here to avoid duplication in ingester and distributor. RateLimited = "rate_limited" @@ -262,6 +265,59 @@ func ValidateMetadata(validateMetrics *ValidateMetrics, cfg *Limits, userID stri return nil } +func ValidateNativeHistogram(validateMetrics *ValidateMetrics, limits *Limits, userID string, ls []cortexpb.LabelAdapter, histogram cortexpb.Histogram) (cortexpb.Histogram, error) { + if limits.MaxNativeHistogramBuckets == 0 { + return histogram, nil + } + + var ( + exceedLimit bool + ) + if histogram.IsFloatHistogram() { + // Initial check to see if the bucket limit is exceeded or not. If not, we can avoid type casting. + exceedLimit = len(histogram.PositiveCounts)+len(histogram.NegativeCounts) > limits.MaxNativeHistogramBuckets + if !exceedLimit { + return histogram, nil + } + // Exceed limit. + if histogram.Schema <= cortexpb.ExponentialSchemaMin { + validateMetrics.DiscardedSamples.WithLabelValues(nativeHistogramBucketsExceeded, userID).Inc() + return cortexpb.Histogram{}, newHistogramBucketLimitExceededError(ls, limits.MaxNativeHistogramBuckets) + } + fh := cortexpb.FloatHistogramProtoToFloatHistogram(histogram) + for len(fh.PositiveBuckets)+len(fh.NegativeBuckets) > limits.MaxNativeHistogramBuckets { + if fh.Schema <= cortexpb.ExponentialSchemaMin { + validateMetrics.DiscardedSamples.WithLabelValues(nativeHistogramBucketsExceeded, userID).Inc() + return cortexpb.Histogram{}, newHistogramBucketLimitExceededError(ls, limits.MaxNativeHistogramBuckets) + } + fh = fh.ReduceResolution(fh.Schema - 1) + } + // If resolution reduced, convert new float histogram to protobuf type again. + return cortexpb.FloatHistogramToHistogramProto(histogram.TimestampMs, fh), nil + } + + // Initial check to see if bucket limit is exceeded or not. If not, we can avoid type casting. + exceedLimit = len(histogram.PositiveDeltas)+len(histogram.NegativeDeltas) > limits.MaxNativeHistogramBuckets + if !exceedLimit { + return histogram, nil + } + // Exceed limit. + if histogram.Schema <= cortexpb.ExponentialSchemaMin { + validateMetrics.DiscardedSamples.WithLabelValues(nativeHistogramBucketsExceeded, userID).Inc() + return cortexpb.Histogram{}, newHistogramBucketLimitExceededError(ls, limits.MaxNativeHistogramBuckets) + } + h := cortexpb.HistogramProtoToHistogram(histogram) + for len(h.PositiveBuckets)+len(h.NegativeBuckets) > limits.MaxNativeHistogramBuckets { + if h.Schema <= cortexpb.ExponentialSchemaMin { + validateMetrics.DiscardedSamples.WithLabelValues(nativeHistogramBucketsExceeded, userID).Inc() + return cortexpb.Histogram{}, newHistogramBucketLimitExceededError(ls, limits.MaxNativeHistogramBuckets) + } + h = h.ReduceResolution(h.Schema - 1) + } + // If resolution reduced, convert new histogram to protobuf type again. + return cortexpb.HistogramToHistogramProto(histogram.TimestampMs, h), nil +} + func DeletePerUserValidationMetrics(validateMetrics *ValidateMetrics, userID string, log log.Logger) { filter := map[string]string{"user": userID} diff --git a/pkg/util/validation/validate_test.go b/pkg/util/validation/validate_test.go index 2ecec1ee72..7bfbed58f7 100644 --- a/pkg/util/validation/validate_test.go +++ b/pkg/util/validation/validate_test.go @@ -8,6 +8,8 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/testutil" "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/tsdb/tsdbutil" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/weaveworks/common/httpgrpc" @@ -291,3 +293,110 @@ func TestValidateLabelDuplication(t *testing.T) { }, "a") assert.Equal(t, expected, actual) } + +func TestValidateNativeHistogram(t *testing.T) { + userID := "fake" + lbls := cortexpb.FromLabelsToLabelAdapters(labels.FromStrings("foo", "bar")) + + // Test histogram has 4 positive buckets and 4 negative buckets so 8 in total. Schema set to 1. + h := tsdbutil.GenerateTestHistogram(0) + fh := tsdbutil.GenerateTestFloatHistogram(0) + + histogramWithSchemaMin := tsdbutil.GenerateTestHistogram(0) + histogramWithSchemaMin.Schema = cortexpb.ExponentialSchemaMin + floatHistogramWithSchemaMin := tsdbutil.GenerateTestFloatHistogram(0) + floatHistogramWithSchemaMin.Schema = cortexpb.ExponentialSchemaMin + for _, tc := range []struct { + name string + bucketLimit int + histogram cortexpb.Histogram + expectedHistogram cortexpb.Histogram + expectedErr error + }{ + { + name: "no limit, histogram", + histogram: cortexpb.HistogramToHistogramProto(0, h.Copy()), + expectedHistogram: cortexpb.HistogramToHistogramProto(0, h.Copy()), + }, + { + name: "no limit, float histogram", + histogram: cortexpb.FloatHistogramToHistogramProto(0, fh.Copy()), + expectedHistogram: cortexpb.FloatHistogramToHistogramProto(0, fh.Copy()), + }, + { + name: "within limit, histogram", + bucketLimit: 8, + histogram: cortexpb.HistogramToHistogramProto(0, h.Copy()), + expectedHistogram: cortexpb.HistogramToHistogramProto(0, h.Copy()), + }, + { + name: "within limit, float histogram", + bucketLimit: 8, + histogram: cortexpb.FloatHistogramToHistogramProto(0, fh.Copy()), + expectedHistogram: cortexpb.FloatHistogramToHistogramProto(0, fh.Copy()), + }, + { + name: "exceed limit and reduce resolution for 1 level, histogram", + bucketLimit: 6, + histogram: cortexpb.HistogramToHistogramProto(0, h.Copy()), + expectedHistogram: cortexpb.HistogramToHistogramProto(0, h.Copy().ReduceResolution(0)), + }, + { + name: "exceed limit and reduce resolution for 1 level, float histogram", + bucketLimit: 6, + histogram: cortexpb.FloatHistogramToHistogramProto(0, fh.Copy()), + expectedHistogram: cortexpb.FloatHistogramToHistogramProto(0, fh.Copy().ReduceResolution(0)), + }, + { + name: "exceed limit and reduce resolution for 2 levels, histogram", + bucketLimit: 4, + histogram: cortexpb.HistogramToHistogramProto(0, h.Copy()), + expectedHistogram: cortexpb.HistogramToHistogramProto(0, h.Copy().ReduceResolution(-1)), + }, + { + name: "exceed limit and reduce resolution for 2 levels, float histogram", + bucketLimit: 4, + histogram: cortexpb.FloatHistogramToHistogramProto(0, fh.Copy()), + expectedHistogram: cortexpb.FloatHistogramToHistogramProto(0, fh.Copy().ReduceResolution(-1)), + }, + { + name: "exceed limit but cannot reduce resolution further, histogram", + bucketLimit: 1, + histogram: cortexpb.HistogramToHistogramProto(0, h.Copy()), + expectedErr: newHistogramBucketLimitExceededError(lbls, 1), + }, + { + name: "exceed limit but cannot reduce resolution further, float histogram", + bucketLimit: 1, + histogram: cortexpb.FloatHistogramToHistogramProto(0, fh.Copy()), + expectedErr: newHistogramBucketLimitExceededError(lbls, 1), + }, + { + name: "exceed limit but cannot reduce resolution further with min schema, histogram", + bucketLimit: 4, + histogram: cortexpb.HistogramToHistogramProto(0, histogramWithSchemaMin.Copy()), + expectedErr: newHistogramBucketLimitExceededError(lbls, 4), + }, + { + name: "exceed limit but cannot reduce resolution further with min schema, float histogram", + bucketLimit: 4, + histogram: cortexpb.FloatHistogramToHistogramProto(0, floatHistogramWithSchemaMin.Copy()), + expectedErr: newHistogramBucketLimitExceededError(lbls, 4), + }, + } { + t.Run(tc.name, func(t *testing.T) { + reg := prometheus.NewRegistry() + validateMetrics := NewValidateMetrics(reg) + limits := new(Limits) + limits.MaxNativeHistogramBuckets = tc.bucketLimit + actualHistogram, actualErr := ValidateNativeHistogram(validateMetrics, limits, userID, lbls, tc.histogram) + if tc.expectedErr != nil { + require.Equal(t, tc.expectedErr, actualErr) + require.Equal(t, float64(1), testutil.ToFloat64(validateMetrics.DiscardedSamples.WithLabelValues(nativeHistogramBucketsExceeded, userID))) + } else { + require.NoError(t, actualErr) + require.Equal(t, tc.expectedHistogram, actualHistogram) + } + }) + } +} From 5b6e1ebece32a0b37aea558194f6f5f3803feeb5 Mon Sep 17 00:00:00 2001 From: Ben Ye Date: Mon, 22 Jul 2024 21:31:57 +0000 Subject: [PATCH 03/32] update doc and changelog Signed-off-by: Ben Ye --- CHANGELOG.md | 1 + docs/configuration/config-file-reference.md | 18 +++++++++++++----- pkg/util/validation/limits.go | 2 +- 3 files changed, 15 insertions(+), 6 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index b992fbf7e2..efa0a139f7 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -9,6 +9,7 @@ * [FEATURE] Query Frontend: Added a query rejection mechanism to block resource-intensive queries. #6005 * [FEATURE] OTLP: Support ingesting OTLP exponential metrics as native histograms. #6071 * [FEATURE] Ingester: Add `ingester.instance-limits.max-inflight-query-requests` to allow limiting ingester concurrent queries. #6081 +* [FEATURE] Distributor: Add `validation.max-native-histogram-buckets` to limit max number of bucket count. Distributor will try to automatically reduce histogram resolution until it is within the bucket limit or resolution cannot be reduced anymore. #6104 * [ENHANCEMENT] rulers: Add support to persist tokens in rulers. #5987 * [ENHANCEMENT] Query Frontend/Querier: Added store gateway postings touched count and touched size in Querier stats and log in Query Frontend. #5892 * [ENHANCEMENT] Query Frontend/Querier: Returns `warnings` on prometheus query responses. #5916 diff --git a/docs/configuration/config-file-reference.md b/docs/configuration/config-file-reference.md index 06e34d991f..1e730d99e1 100644 --- a/docs/configuration/config-file-reference.md +++ b/docs/configuration/config-file-reference.md @@ -3159,11 +3159,13 @@ The `limits_config` configures default and per-tenant limits imposed by Cortex s # e.g. remote_write.write_relabel_configs. [metric_relabel_configs: | default = []] -# Enables support for exemplars in TSDB and sets the maximum number that will be -# stored. less than zero means disabled. If the value is set to zero, cortex -# will fallback to blocks-storage.tsdb.max-exemplars value. -# CLI flag: -ingester.max-exemplars -[max_exemplars: | default = 0] +# Limit on total number of positive and negative buckets allowed in a single +# native histogram. The resolution of a histogram with more buckets will be +# reduced until the number of buckets is within the limit. If the limit cannot +# be reached, the sample will be discarded. 0 means no limit. Enforced at +# Distributor. +# CLI flag: -validation.max-native-histogram-buckets +[max_native_histogram_buckets: | default = 0] # The maximum number of active series per user, per ingester. 0 to disable. # CLI flag: -ingester.max-series-per-user @@ -3213,6 +3215,12 @@ The `limits_config` configures default and per-tenant limits imposed by Cortex s # CLI flag: -ingester.out-of-order-time-window [out_of_order_time_window: | default = 0s] +# Enables support for exemplars in TSDB and sets the maximum number that will be +# stored. less than zero means disabled. If the value is set to zero, cortex +# will fallback to blocks-storage.tsdb.max-exemplars value. +# CLI flag: -ingester.max-exemplars +[max_exemplars: | default = 0] + # Maximum number of chunks that can be fetched in a single query from ingesters # and long-term storage. This limit is enforced in the querier, ruler and # store-gateway. 0 to disable. diff --git a/pkg/util/validation/limits.go b/pkg/util/validation/limits.go index 280e797079..c13abeec83 100644 --- a/pkg/util/validation/limits.go +++ b/pkg/util/validation/limits.go @@ -234,7 +234,7 @@ func (l *Limits) RegisterFlags(f *flag.FlagSet) { f.Var(&l.CreationGracePeriod, "validation.create-grace-period", "Duration which table will be created/deleted before/after it's needed; we won't accept sample from before this time.") f.BoolVar(&l.EnforceMetricName, "validation.enforce-metric-name", true, "Enforce every sample has a metric name.") f.BoolVar(&l.EnforceMetadataMetricName, "validation.enforce-metadata-metric-name", true, "Enforce every metadata has a metric name.") - f.IntVar(&l.MaxNativeHistogramBuckets, "validation.max-native-histogram-buckets", 0, ".") + f.IntVar(&l.MaxNativeHistogramBuckets, "validation.max-native-histogram-buckets", 0, "Limit on total number of positive and negative buckets allowed in a single native histogram. The resolution of a histogram with more buckets will be reduced until the number of buckets is within the limit. If the limit cannot be reached, the sample will be discarded. 0 means no limit. Enforced at Distributor.") f.IntVar(&l.MaxLocalSeriesPerUser, "ingester.max-series-per-user", 5000000, "The maximum number of active series per user, per ingester. 0 to disable.") f.IntVar(&l.MaxLocalSeriesPerMetric, "ingester.max-series-per-metric", 50000, "The maximum number of active series per metric name, per ingester. 0 to disable.") From a88d2370e1d35de4227cf7fdbfc60b5f8ae3ff4e Mon Sep 17 00:00:00 2001 From: Ben Ye Date: Mon, 22 Jul 2024 16:10:43 -0700 Subject: [PATCH 04/32] address comments Signed-off-by: Ben Ye --- pkg/util/validation/limits.go | 3 ++- pkg/util/validation/validate.go | 10 +++++----- pkg/util/validation/validate_test.go | 2 +- 3 files changed, 8 insertions(+), 7 deletions(-) diff --git a/pkg/util/validation/limits.go b/pkg/util/validation/limits.go index c13abeec83..fc9faab078 100644 --- a/pkg/util/validation/limits.go +++ b/pkg/util/validation/limits.go @@ -725,7 +725,8 @@ func (o *Overrides) EnforceMetadataMetricName(userID string) bool { return o.GetOverridesForUser(userID).EnforceMetadataMetricName } -// MaxNativeHistogramBuckets. +// MaxNativeHistogramBuckets returns the maximum total number of positive and negative buckets of a single native histogram +// a user is allowed to store. func (o *Overrides) MaxNativeHistogramBuckets(userID string) int { return o.GetOverridesForUser(userID).MaxNativeHistogramBuckets } diff --git a/pkg/util/validation/validate.go b/pkg/util/validation/validate.go index 031d4ac9a5..46554e2d5b 100644 --- a/pkg/util/validation/validate.go +++ b/pkg/util/validation/validate.go @@ -53,7 +53,7 @@ const ( exemplarTimestampInvalid = "exemplar_timestamp_invalid" // Native Histogram specific validation reasons - nativeHistogramBucketsExceeded = "native_histogram_buckets_exceeded" + nativeHistogramBucketCountLimitExceeded = "native_histogram_buckets_exceeded" // RateLimited is one of the values for the reason to discard samples. // Declared here to avoid duplication in ingester and distributor. @@ -281,13 +281,13 @@ func ValidateNativeHistogram(validateMetrics *ValidateMetrics, limits *Limits, u } // Exceed limit. if histogram.Schema <= cortexpb.ExponentialSchemaMin { - validateMetrics.DiscardedSamples.WithLabelValues(nativeHistogramBucketsExceeded, userID).Inc() + validateMetrics.DiscardedSamples.WithLabelValues(nativeHistogramBucketCountLimitExceeded, userID).Inc() return cortexpb.Histogram{}, newHistogramBucketLimitExceededError(ls, limits.MaxNativeHistogramBuckets) } fh := cortexpb.FloatHistogramProtoToFloatHistogram(histogram) for len(fh.PositiveBuckets)+len(fh.NegativeBuckets) > limits.MaxNativeHistogramBuckets { if fh.Schema <= cortexpb.ExponentialSchemaMin { - validateMetrics.DiscardedSamples.WithLabelValues(nativeHistogramBucketsExceeded, userID).Inc() + validateMetrics.DiscardedSamples.WithLabelValues(nativeHistogramBucketCountLimitExceeded, userID).Inc() return cortexpb.Histogram{}, newHistogramBucketLimitExceededError(ls, limits.MaxNativeHistogramBuckets) } fh = fh.ReduceResolution(fh.Schema - 1) @@ -303,13 +303,13 @@ func ValidateNativeHistogram(validateMetrics *ValidateMetrics, limits *Limits, u } // Exceed limit. if histogram.Schema <= cortexpb.ExponentialSchemaMin { - validateMetrics.DiscardedSamples.WithLabelValues(nativeHistogramBucketsExceeded, userID).Inc() + validateMetrics.DiscardedSamples.WithLabelValues(nativeHistogramBucketCountLimitExceeded, userID).Inc() return cortexpb.Histogram{}, newHistogramBucketLimitExceededError(ls, limits.MaxNativeHistogramBuckets) } h := cortexpb.HistogramProtoToHistogram(histogram) for len(h.PositiveBuckets)+len(h.NegativeBuckets) > limits.MaxNativeHistogramBuckets { if h.Schema <= cortexpb.ExponentialSchemaMin { - validateMetrics.DiscardedSamples.WithLabelValues(nativeHistogramBucketsExceeded, userID).Inc() + validateMetrics.DiscardedSamples.WithLabelValues(nativeHistogramBucketCountLimitExceeded, userID).Inc() return cortexpb.Histogram{}, newHistogramBucketLimitExceededError(ls, limits.MaxNativeHistogramBuckets) } h = h.ReduceResolution(h.Schema - 1) diff --git a/pkg/util/validation/validate_test.go b/pkg/util/validation/validate_test.go index 7bfbed58f7..93051a63b8 100644 --- a/pkg/util/validation/validate_test.go +++ b/pkg/util/validation/validate_test.go @@ -392,7 +392,7 @@ func TestValidateNativeHistogram(t *testing.T) { actualHistogram, actualErr := ValidateNativeHistogram(validateMetrics, limits, userID, lbls, tc.histogram) if tc.expectedErr != nil { require.Equal(t, tc.expectedErr, actualErr) - require.Equal(t, float64(1), testutil.ToFloat64(validateMetrics.DiscardedSamples.WithLabelValues(nativeHistogramBucketsExceeded, userID))) + require.Equal(t, float64(1), testutil.ToFloat64(validateMetrics.DiscardedSamples.WithLabelValues(nativeHistogramBucketCountLimitExceeded, userID))) } else { require.NoError(t, actualErr) require.Equal(t, tc.expectedHistogram, actualHistogram) From 52575030da1a94ae97911b85fedb0afe6296d3a4 Mon Sep 17 00:00:00 2001 From: Ben Ye Date: Tue, 23 Jul 2024 09:32:27 -0700 Subject: [PATCH 05/32] Ingester: compact ooo head during head compaction (#6108) Signed-off-by: Ben Ye --- pkg/ingester/ingester.go | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/pkg/ingester/ingester.go b/pkg/ingester/ingester.go index edcff187f7..ee82393e9f 100644 --- a/pkg/ingester/ingester.go +++ b/pkg/ingester/ingester.go @@ -360,7 +360,7 @@ func (u *userTSDB) casState(from, to tsdbState) bool { } // compactHead compacts the Head block at specified block durations avoiding a single huge block. -func (u *userTSDB) compactHead(blockDuration int64) error { +func (u *userTSDB) compactHead(ctx context.Context, blockDuration int64) error { if !u.casState(active, forceCompacting) { return errors.New("TSDB head cannot be compacted because it is not in active state (possibly being closed or blocks shipping in progress)") } @@ -388,7 +388,10 @@ func (u *userTSDB) compactHead(blockDuration int64) error { minTime, maxTime = h.MinTime(), h.MaxTime() } - return u.db.CompactHead(tsdb.NewRangeHead(h, minTime, maxTime)) + if err := u.db.CompactHead(tsdb.NewRangeHead(h, minTime, maxTime)); err != nil { + return err + } + return u.db.CompactOOOHead(ctx) } // PreCreation implements SeriesLifecycleCallback interface. @@ -2536,12 +2539,12 @@ func (i *Ingester) compactBlocks(ctx context.Context, force bool, allowed *util. switch { case force: reason = "forced" - err = userDB.compactHead(i.cfg.BlocksStorageConfig.TSDB.BlockRanges[0].Milliseconds()) + err = userDB.compactHead(ctx, i.cfg.BlocksStorageConfig.TSDB.BlockRanges[0].Milliseconds()) case i.TSDBState.compactionIdleTimeout > 0 && userDB.isIdle(time.Now(), i.TSDBState.compactionIdleTimeout): reason = "idle" level.Info(logutil.WithContext(ctx, i.logger)).Log("msg", "TSDB is idle, forcing compaction", "user", userID) - err = userDB.compactHead(i.cfg.BlocksStorageConfig.TSDB.BlockRanges[0].Milliseconds()) + err = userDB.compactHead(ctx, i.cfg.BlocksStorageConfig.TSDB.BlockRanges[0].Milliseconds()) default: reason = "regular" From 03d2255f55c3b4714a35e918573b90f6c9d6abad Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 23 Jul 2024 09:33:17 -0700 Subject: [PATCH 06/32] Bump go.etcd.io/etcd/client/v3 from 3.5.14 to 3.5.15 (#6103) Bumps [go.etcd.io/etcd/client/v3](https://github.com/etcd-io/etcd) from 3.5.14 to 3.5.15. - [Release notes](https://github.com/etcd-io/etcd/releases) - [Commits](https://github.com/etcd-io/etcd/compare/v3.5.14...v3.5.15) --- updated-dependencies: - dependency-name: go.etcd.io/etcd/client/v3 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 | 6 +-- go.sum | 12 ++--- .../go.etcd.io/etcd/api/v3/version/version.go | 2 +- .../etcd/client/pkg/v3/logutil/zap.go | 2 +- .../etcd/client/pkg/v3/transport/listener.go | 50 +++++++++++++++++-- vendor/go.etcd.io/etcd/client/v3/watch.go | 2 +- vendor/modules.txt | 6 +-- 7 files changed, 62 insertions(+), 18 deletions(-) diff --git a/go.mod b/go.mod index 7155619924..b694b65f2a 100644 --- a/go.mod +++ b/go.mod @@ -56,9 +56,9 @@ require ( github.com/thanos-io/thanos v0.36.0-rc.0 github.com/uber/jaeger-client-go v2.30.0+incompatible github.com/weaveworks/common v0.0.0-20230728070032-dd9e68f319d5 - go.etcd.io/etcd/api/v3 v3.5.14 - go.etcd.io/etcd/client/pkg/v3 v3.5.14 - go.etcd.io/etcd/client/v3 v3.5.14 + go.etcd.io/etcd/api/v3 v3.5.15 + go.etcd.io/etcd/client/pkg/v3 v3.5.15 + go.etcd.io/etcd/client/v3 v3.5.15 go.opentelemetry.io/contrib/propagators/aws v1.28.0 go.opentelemetry.io/otel v1.28.0 go.opentelemetry.io/otel/bridge/opentracing v1.28.0 diff --git a/go.sum b/go.sum index 5935d501bb..82a3a60c9b 100644 --- a/go.sum +++ b/go.sum @@ -1456,15 +1456,15 @@ github.com/zeebo/xxh3 v1.0.2/go.mod h1:5NWz9Sef7zIDm2JHfFlcQvNekmcEl9ekUZQQKCYaD github.com/zhangyunhao116/umap v0.0.0-20221211160557-cb7705fafa39 h1:D3ltj0b2c2FgUacKrB1pWGgwrUyCESY9W8XYYQ5sqY8= github.com/zhangyunhao116/umap v0.0.0-20221211160557-cb7705fafa39/go.mod h1:r86X1CnsDRrOeLtJlqRWdELPWpkcf933GTlojQlifQw= go.etcd.io/etcd/api/v3 v3.5.4/go.mod h1:5GB2vv4A4AOn3yk7MftYGHkUfGtDHnEraIjym4dYz5A= -go.etcd.io/etcd/api/v3 v3.5.14 h1:vHObSCxyB9zlF60w7qzAdTcGaglbJOpSj1Xj9+WGxq0= -go.etcd.io/etcd/api/v3 v3.5.14/go.mod h1:BmtWcRlQvwa1h3G2jvKYwIQy4PkHlDej5t7uLMUdJUU= +go.etcd.io/etcd/api/v3 v3.5.15 h1:3KpLJir1ZEBrYuV2v+Twaa/e2MdDCEZ/70H+lzEiwsk= +go.etcd.io/etcd/api/v3 v3.5.15/go.mod h1:N9EhGzXq58WuMllgH9ZvnEr7SI9pS0k0+DHZezGp7jM= go.etcd.io/etcd/client/pkg/v3 v3.5.4/go.mod h1:IJHfcCEKxYu1Os13ZdwCwIUTUVGYTSAM3YSwc9/Ac1g= -go.etcd.io/etcd/client/pkg/v3 v3.5.14 h1:SaNH6Y+rVEdxfpA2Jr5wkEvN6Zykme5+YnbCkxvuWxQ= -go.etcd.io/etcd/client/pkg/v3 v3.5.14/go.mod h1:8uMgAokyG1czCtIdsq+AGyYQMvpIKnSvPjFMunkgeZI= +go.etcd.io/etcd/client/pkg/v3 v3.5.15 h1:fo0HpWz/KlHGMCC+YejpiCmyWDEuIpnTDzpJLB5fWlA= +go.etcd.io/etcd/client/pkg/v3 v3.5.15/go.mod h1:mXDI4NAOwEiszrHCb0aqfAYNCrZP4e9hRca3d1YK8EU= go.etcd.io/etcd/client/v2 v2.305.4/go.mod h1:Ud+VUwIi9/uQHOMA+4ekToJ12lTxlv0zB/+DHwTGEbU= go.etcd.io/etcd/client/v3 v3.5.4/go.mod h1:ZaRkVgBZC+L+dLCjTcF1hRXpgZXQPOvnA/Ak/gq3kiY= -go.etcd.io/etcd/client/v3 v3.5.14 h1:CWfRs4FDaDoSz81giL7zPpZH2Z35tbOrAJkkjMqOupg= -go.etcd.io/etcd/client/v3 v3.5.14/go.mod h1:k3XfdV/VIHy/97rqWjoUzrj9tk7GgJGH9J8L4dNXmAk= +go.etcd.io/etcd/client/v3 v3.5.15 h1:23M0eY4Fd/inNv1ZfU3AxrbbOdW79r9V9Rl62Nm6ip4= +go.etcd.io/etcd/client/v3 v3.5.15/go.mod h1:CLSJxrYjvLtHsrPKsy7LmZEE+DK2ktfd2bN4RhBMwlU= go.mongodb.org/mongo-driver v1.14.0 h1:P98w8egYRjYe3XDjxhYJagTokP/H6HzlsnojRgZRd80= go.mongodb.org/mongo-driver v1.14.0/go.mod h1:Vzb0Mk/pa7e6cWw85R4F/endUC3u0U9jGcNU603k65c= go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= diff --git a/vendor/go.etcd.io/etcd/api/v3/version/version.go b/vendor/go.etcd.io/etcd/api/v3/version/version.go index 6b5177fc3c..21b9950425 100644 --- a/vendor/go.etcd.io/etcd/api/v3/version/version.go +++ b/vendor/go.etcd.io/etcd/api/v3/version/version.go @@ -26,7 +26,7 @@ import ( var ( // MinClusterVersion is the min cluster version this etcd binary is compatible with. MinClusterVersion = "3.0.0" - Version = "3.5.14" + Version = "3.5.15" APIVersion = "unknown" // Git SHA Value will be set during build diff --git a/vendor/go.etcd.io/etcd/client/pkg/v3/logutil/zap.go b/vendor/go.etcd.io/etcd/client/pkg/v3/logutil/zap.go index 34f35b9f28..f0f3739aad 100644 --- a/vendor/go.etcd.io/etcd/client/pkg/v3/logutil/zap.go +++ b/vendor/go.etcd.io/etcd/client/pkg/v3/logutil/zap.go @@ -58,7 +58,7 @@ var DefaultZapLoggerConfig = zap.Config{ // Custom EncodeTime function to ensure we match format and precision of historic capnslog timestamps EncodeTime: func(t time.Time, enc zapcore.PrimitiveArrayEncoder) { - enc.AppendString(t.Format("2006-01-02T15:04:05.999999Z0700")) + enc.AppendString(t.Format("2006-01-02T15:04:05.000000Z0700")) }, EncodeDuration: zapcore.StringDurationEncoder, diff --git a/vendor/go.etcd.io/etcd/client/pkg/v3/transport/listener.go b/vendor/go.etcd.io/etcd/client/pkg/v3/transport/listener.go index 150545d08d..a7d37688d9 100644 --- a/vendor/go.etcd.io/etcd/client/pkg/v3/transport/listener.go +++ b/vendor/go.etcd.io/etcd/client/pkg/v3/transport/listener.go @@ -180,12 +180,23 @@ type TLSInfo struct { parseFunc func([]byte, []byte) (tls.Certificate, error) // AllowedCN is a CN which must be provided by a client. + // + // Deprecated: use AllowedCNs instead. AllowedCN string // AllowedHostname is an IP address or hostname that must match the TLS // certificate provided by a client. + // + // Deprecated: use AllowedHostnames instead. AllowedHostname string + // AllowedCNs is a list of acceptable CNs which must be provided by a client. + AllowedCNs []string + + // AllowedHostnames is a list of acceptable IP addresses or hostnames that must match the + // TLS certificate provided by a client. + AllowedHostnames []string + // Logger logs TLS errors. // If nil, all logs are discarded. Logger *zap.Logger @@ -407,19 +418,52 @@ func (info TLSInfo) baseConfig() (*tls.Config, error) { // Client certificates may be verified by either an exact match on the CN, // or a more general check of the CN and SANs. var verifyCertificate func(*x509.Certificate) bool + + if info.AllowedCN != "" && len(info.AllowedCNs) > 0 { + return nil, fmt.Errorf("AllowedCN and AllowedCNs are mutually exclusive (cn=%q, cns=%q)", info.AllowedCN, info.AllowedCNs) + } + if info.AllowedHostname != "" && len(info.AllowedHostnames) > 0 { + return nil, fmt.Errorf("AllowedHostname and AllowedHostnames are mutually exclusive (hostname=%q, hostnames=%q)", info.AllowedHostname, info.AllowedHostnames) + } + if info.AllowedCN != "" && info.AllowedHostname != "" { + return nil, fmt.Errorf("AllowedCN and AllowedHostname are mutually exclusive (cn=%q, hostname=%q)", info.AllowedCN, info.AllowedHostname) + } + if len(info.AllowedCNs) > 0 && len(info.AllowedHostnames) > 0 { + return nil, fmt.Errorf("AllowedCNs and AllowedHostnames are mutually exclusive (cns=%q, hostnames=%q)", info.AllowedCNs, info.AllowedHostnames) + } + if info.AllowedCN != "" { - if info.AllowedHostname != "" { - return nil, fmt.Errorf("AllowedCN and AllowedHostname are mutually exclusive (cn=%q, hostname=%q)", info.AllowedCN, info.AllowedHostname) - } + info.Logger.Warn("AllowedCN is deprecated, use AllowedCNs instead") verifyCertificate = func(cert *x509.Certificate) bool { return info.AllowedCN == cert.Subject.CommonName } } if info.AllowedHostname != "" { + info.Logger.Warn("AllowedHostname is deprecated, use AllowedHostnames instead") verifyCertificate = func(cert *x509.Certificate) bool { return cert.VerifyHostname(info.AllowedHostname) == nil } } + if len(info.AllowedCNs) > 0 { + verifyCertificate = func(cert *x509.Certificate) bool { + for _, allowedCN := range info.AllowedCNs { + if allowedCN == cert.Subject.CommonName { + return true + } + } + return false + } + } + if len(info.AllowedHostnames) > 0 { + verifyCertificate = func(cert *x509.Certificate) bool { + for _, allowedHostname := range info.AllowedHostnames { + if cert.VerifyHostname(allowedHostname) == nil { + return true + } + } + return false + } + } if verifyCertificate != nil { cfg.VerifyPeerCertificate = func(rawCerts [][]byte, verifiedChains [][]*x509.Certificate) error { for _, chains := range verifiedChains { diff --git a/vendor/go.etcd.io/etcd/client/v3/watch.go b/vendor/go.etcd.io/etcd/client/v3/watch.go index 41a6ec9763..963f7e65c3 100644 --- a/vendor/go.etcd.io/etcd/client/v3/watch.go +++ b/vendor/go.etcd.io/etcd/client/v3/watch.go @@ -1036,7 +1036,7 @@ func (pr *progressRequest) toPB() *pb.WatchRequest { func streamKeyFromCtx(ctx context.Context) string { if md, ok := metadata.FromOutgoingContext(ctx); ok { - return fmt.Sprintf("%+v", md) + return fmt.Sprintf("%+v", map[string][]string(md)) } return "" } diff --git a/vendor/modules.txt b/vendor/modules.txt index be234374e0..4cc60b0488 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -1075,7 +1075,7 @@ github.com/yuin/gopher-lua/pm # github.com/zhangyunhao116/umap v0.0.0-20221211160557-cb7705fafa39 ## explicit; go 1.13 github.com/zhangyunhao116/umap -# go.etcd.io/etcd/api/v3 v3.5.14 +# go.etcd.io/etcd/api/v3 v3.5.15 ## explicit; go 1.21 go.etcd.io/etcd/api/v3/authpb go.etcd.io/etcd/api/v3/etcdserverpb @@ -1083,7 +1083,7 @@ go.etcd.io/etcd/api/v3/membershippb go.etcd.io/etcd/api/v3/mvccpb go.etcd.io/etcd/api/v3/v3rpc/rpctypes go.etcd.io/etcd/api/v3/version -# go.etcd.io/etcd/client/pkg/v3 v3.5.14 +# go.etcd.io/etcd/client/pkg/v3 v3.5.15 ## explicit; go 1.21 go.etcd.io/etcd/client/pkg/v3/fileutil go.etcd.io/etcd/client/pkg/v3/logutil @@ -1091,7 +1091,7 @@ go.etcd.io/etcd/client/pkg/v3/systemd go.etcd.io/etcd/client/pkg/v3/tlsutil go.etcd.io/etcd/client/pkg/v3/transport go.etcd.io/etcd/client/pkg/v3/types -# go.etcd.io/etcd/client/v3 v3.5.14 +# go.etcd.io/etcd/client/v3 v3.5.15 ## explicit; go 1.21 go.etcd.io/etcd/client/v3 go.etcd.io/etcd/client/v3/credentials From e68ad87b50706b193408b2eae5f243f75c660453 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 23 Jul 2024 09:33:29 -0700 Subject: [PATCH 07/32] Bump go.opentelemetry.io/collector/pdata from 1.11.0 to 1.12.0 (#6102) Bumps [go.opentelemetry.io/collector/pdata](https://github.com/open-telemetry/opentelemetry-collector) from 1.11.0 to 1.12.0. - [Release notes](https://github.com/open-telemetry/opentelemetry-collector/releases) - [Changelog](https://github.com/open-telemetry/opentelemetry-collector/blob/main/CHANGELOG-API.md) - [Commits](https://github.com/open-telemetry/opentelemetry-collector/compare/pdata/v1.11.0...pdata/v1.12.0) --- updated-dependencies: - dependency-name: go.opentelemetry.io/collector/pdata 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 ++-- vendor/modules.txt | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/go.mod b/go.mod index b694b65f2a..4dfbc966e5 100644 --- a/go.mod +++ b/go.mod @@ -81,7 +81,7 @@ require ( github.com/cespare/xxhash/v2 v2.3.0 github.com/google/go-cmp v0.6.0 github.com/sercand/kuberesolver/v4 v4.0.0 - go.opentelemetry.io/collector/pdata v1.11.0 + go.opentelemetry.io/collector/pdata v1.12.0 golang.org/x/exp v0.0.0-20240119083558-1b970713d09a google.golang.org/protobuf v1.34.2 ) diff --git a/go.sum b/go.sum index 82a3a60c9b..210b07f262 100644 --- a/go.sum +++ b/go.sum @@ -1476,8 +1476,8 @@ go.opencensus.io v0.22.5/go.mod h1:5pWMHQbX5EPX2/62yrJeAkowc+lfs/XD7Uxpq3pI6kk= go.opencensus.io v0.23.0/go.mod h1:XItmlyltB5F7CS4xOC1DcqMoFqwtC6OG2xF7mCv7P7E= go.opencensus.io v0.24.0 h1:y73uSU6J157QMP2kn2r30vwW1A2W2WFwSCGnAVxeaD0= go.opencensus.io v0.24.0/go.mod h1:vNK8G9p7aAivkbmorf4v+7Hgx+Zs0yY+0fOtgBfjQKo= -go.opentelemetry.io/collector/pdata v1.11.0 h1:rzYyV1zfTQQz1DI9hCiaKyyaczqawN75XO9mdXmR/hE= -go.opentelemetry.io/collector/pdata v1.11.0/go.mod h1:IHxHsp+Jq/xfjORQMDJjSH6jvedOSTOyu3nbxqhWSYE= +go.opentelemetry.io/collector/pdata v1.12.0 h1:Xx5VK1p4VO0md8MWm2icwC1MnJ7f8EimKItMWw46BmA= +go.opentelemetry.io/collector/pdata v1.12.0/go.mod h1:MYeB0MmMAxeM0hstCFrCqWLzdyeYySim2dG6pDT6nYI= go.opentelemetry.io/collector/semconv v0.101.0 h1:tOe9iTe9dDCnvz/bqgfNRr4w80kXG8505tQJ5h5v08Q= go.opentelemetry.io/collector/semconv v0.101.0/go.mod h1:8ElcRZ8Cdw5JnvhTOQOdYizkJaQ10Z2fS+R6djOnj6A= go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.49.0 h1:4Pp6oUg3+e/6M4C0A/3kJ2VYa++dsWVTtGgLVj5xtHg= diff --git a/vendor/modules.txt b/vendor/modules.txt index 4cc60b0488..ed6c882669 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -1125,7 +1125,7 @@ go.opencensus.io/trace go.opencensus.io/trace/internal go.opencensus.io/trace/propagation go.opencensus.io/trace/tracestate -# go.opentelemetry.io/collector/pdata v1.11.0 +# go.opentelemetry.io/collector/pdata v1.12.0 ## explicit; go 1.21.0 go.opentelemetry.io/collector/pdata/internal go.opentelemetry.io/collector/pdata/internal/data From 69996b6d9262935bc87446b21ebad37a1886de3d Mon Sep 17 00:00:00 2001 From: Doupamo Eradiri Date: Tue, 23 Jul 2024 14:02:29 -0400 Subject: [PATCH 08/32] fix(build): docker compose environments (#6099) Fixed cortex dockerfile and updated consul docker image tag so that docker compose is able to successfully pull consul and swift images. Removed invalid enable_alertmanager_v2 cortex config param from microservice mode config files to fix cortex start up. cleanup: removed 'version' compose tag to silence deprecation warning Signed-off-by: Doupamo Eradiri --- development/tsdb-blocks-storage-s3-gossip/config/cortex.yaml | 1 - development/tsdb-blocks-storage-s3-gossip/dev.dockerfile | 2 +- development/tsdb-blocks-storage-s3-gossip/docker-compose.yml | 2 +- .../tsdb-blocks-storage-s3-single-binary/docker-compose.yml | 2 +- development/tsdb-blocks-storage-s3/config/cortex.yaml | 1 - development/tsdb-blocks-storage-s3/dev.dockerfile | 2 +- development/tsdb-blocks-storage-s3/docker-compose.yml | 2 +- .../docker-compose.yml | 5 ++--- 8 files changed, 7 insertions(+), 10 deletions(-) diff --git a/development/tsdb-blocks-storage-s3-gossip/config/cortex.yaml b/development/tsdb-blocks-storage-s3-gossip/config/cortex.yaml index bc79dabb8c..47a5140406 100644 --- a/development/tsdb-blocks-storage-s3-gossip/config/cortex.yaml +++ b/development/tsdb-blocks-storage-s3-gossip/config/cortex.yaml @@ -90,7 +90,6 @@ ruler: store: memberlist alertmanager_url: http://alertmanager:8010/alertmanager - enable_alertmanager_v2: false ruler_storage: backend: s3 diff --git a/development/tsdb-blocks-storage-s3-gossip/dev.dockerfile b/development/tsdb-blocks-storage-s3-gossip/dev.dockerfile index f05d0c667e..acc34126d3 100644 --- a/development/tsdb-blocks-storage-s3-gossip/dev.dockerfile +++ b/development/tsdb-blocks-storage-s3-gossip/dev.dockerfile @@ -1,6 +1,6 @@ FROM golang:1.19 ENV CGO_ENABLED=0 -RUN go get github.com/go-delve/delve/cmd/dlv +RUN go install github.com/go-delve/delve/cmd/dlv@latest FROM alpine:3.18 diff --git a/development/tsdb-blocks-storage-s3-gossip/docker-compose.yml b/development/tsdb-blocks-storage-s3-gossip/docker-compose.yml index bfc5fbf7fb..455a3e1f7a 100644 --- a/development/tsdb-blocks-storage-s3-gossip/docker-compose.yml +++ b/development/tsdb-blocks-storage-s3-gossip/docker-compose.yml @@ -2,7 +2,7 @@ version: '3.4' services: consul: - image: consul + image: consul:1.15.4 command: [ "agent", "-dev" ,"-client=0.0.0.0", "-log-level=info" ] ports: - 8500:8500 diff --git a/development/tsdb-blocks-storage-s3-single-binary/docker-compose.yml b/development/tsdb-blocks-storage-s3-single-binary/docker-compose.yml index f65fb0feda..4e6b396d19 100644 --- a/development/tsdb-blocks-storage-s3-single-binary/docker-compose.yml +++ b/development/tsdb-blocks-storage-s3-single-binary/docker-compose.yml @@ -2,7 +2,7 @@ version: '3.4' services: consul: - image: consul + image: consul:1.15.4 command: [ "agent", "-dev" ,"-client=0.0.0.0", "-log-level=info" ] ports: - 8500:8500 diff --git a/development/tsdb-blocks-storage-s3/config/cortex.yaml b/development/tsdb-blocks-storage-s3/config/cortex.yaml index 0f35136841..32da3d72f1 100644 --- a/development/tsdb-blocks-storage-s3/config/cortex.yaml +++ b/development/tsdb-blocks-storage-s3/config/cortex.yaml @@ -80,7 +80,6 @@ ruler: host: consul:8500 alertmanager_url: http://alertmanager-1:8031/alertmanager,http://alertmanager-2:8032/alertmanager,http://alertmanager-3:8033/alertmanager - enable_alertmanager_v2: false ruler_storage: backend: s3 diff --git a/development/tsdb-blocks-storage-s3/dev.dockerfile b/development/tsdb-blocks-storage-s3/dev.dockerfile index 698ed36ea2..4566c8dafb 100644 --- a/development/tsdb-blocks-storage-s3/dev.dockerfile +++ b/development/tsdb-blocks-storage-s3/dev.dockerfile @@ -1,6 +1,6 @@ FROM golang:1.19 ENV CGO_ENABLED=0 -RUN go get github.com/go-delve/delve/cmd/dlv +RUN go install github.com/go-delve/delve/cmd/dlv@latest FROM alpine:3.19 diff --git a/development/tsdb-blocks-storage-s3/docker-compose.yml b/development/tsdb-blocks-storage-s3/docker-compose.yml index 7985479c6f..5bb3a9d370 100644 --- a/development/tsdb-blocks-storage-s3/docker-compose.yml +++ b/development/tsdb-blocks-storage-s3/docker-compose.yml @@ -2,7 +2,7 @@ version: '3.4' services: consul: - image: consul + image: consul:1.15.4 command: [ "agent", "-dev" ,"-client=0.0.0.0", "-log-level=info" ] ports: - 8500:8500 diff --git a/development/tsdb-blocks-storage-swift-single-binary/docker-compose.yml b/development/tsdb-blocks-storage-swift-single-binary/docker-compose.yml index ef280142f6..c9a57460f3 100644 --- a/development/tsdb-blocks-storage-swift-single-binary/docker-compose.yml +++ b/development/tsdb-blocks-storage-swift-single-binary/docker-compose.yml @@ -1,14 +1,13 @@ -version: '3.4' services: consul: - image: consul + image: consul:1.15.4 command: [ "agent", "-dev" ,"-client=0.0.0.0", "-log-level=info" ] ports: - 8500:8500 swift: - image: beaukode/docker-swift-onlyone-authv2-keystone + image: beaukode/docker-swift-onlyone-authv2-keystone:latest volumes: - .data-swift:/srv:delegated ports: From 75cfab4d4d99ec9a9d5ba83c08a6e1ee688d5c3e Mon Sep 17 00:00:00 2001 From: Harry John Date: Tue, 23 Jul 2024 11:45:40 -0700 Subject: [PATCH 09/32] Chore: Update prometheus, thanos and promql-engine (#6105) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Chore: Update prometheus, thanos and promql-engine Signed-off-by: 🌲 Harry 🌊 John 🏔 * Remove test files Signed-off-by: 🌲 Harry 🌊 John 🏔 --------- Signed-off-by: 🌲 Harry 🌊 John 🏔 --- go.mod | 40 +- go.sum | 80 +- integration/e2e/util.go | 5 +- pkg/api/handlers.go | 1 + pkg/ingester/ingester.go | 4 +- pkg/querier/blocks_store_queryable.go | 4 +- pkg/querier/blocks_store_queryable_test.go | 4 +- pkg/querier/distributor_queryable.go | 4 +- pkg/querier/distributor_queryable_test.go | 2 +- pkg/querier/error_translate_queryable.go | 16 +- pkg/querier/error_translate_queryable_test.go | 5 +- pkg/querier/lazyquery/lazyquery.go | 8 +- pkg/querier/querier.go | 12 +- pkg/querier/querier_test.go | 10 +- pkg/querier/remote_read_test.go | 4 +- .../tenantfederation/merge_queryable.go | 12 +- .../tenantfederation/merge_queryable_test.go | 10 +- .../queryrange/split_by_interval_test.go | 5 - pkg/ruler/ruler_test.go | 4 +- .../azure-sdk-for-go/sdk/azcore/CHANGELOG.md | 15 + .../azcore/internal/pollers/async/async.go | 2 +- .../sdk/azcore/internal/pollers/body/body.go | 2 +- .../sdk/azcore/internal/pollers/fake/fake.go | 2 +- .../sdk/azcore/internal/pollers/loc/loc.go | 2 +- .../sdk/azcore/internal/pollers/op/op.go | 11 +- .../sdk/azcore/internal/pollers/util.go | 14 +- .../sdk/azcore/internal/shared/constants.go | 2 +- .../sdk/azcore/runtime/pager.go | 13 +- .../sdk/azcore/runtime/request.go | 10 + .../sdk/azidentity/CHANGELOG.md | 24 + .../azure-sdk-for-go/sdk/azidentity/README.md | 1 + .../sdk/azidentity/TOKEN_CACHING.MD | 1 + .../sdk/azidentity/TROUBLESHOOTING.md | 10 + .../azidentity/azure_pipelines_credential.go | 50 +- .../azure-sdk-for-go/sdk/azidentity/errors.go | 2 + .../sdk/azidentity/version.go | 2 +- .../prometheus/prometheus/config/config.go | 53 +- .../model/histogram/float_histogram.go | 310 +- .../prometheus/model/histogram/generic.go | 105 +- .../prometheus/model/histogram/histogram.go | 214 +- .../model/labels/labels_dedupelabels.go | 89 +- .../prometheus/model/labels/regexp.go | 177 +- .../prometheus/model/relabel/relabel.go | 5 + .../prometheus/notifier/notifier.go | 139 +- .../prometheus/prometheus/prompb/codec.go | 201 + .../prometheus/prometheus/prompb/custom.go | 8 - .../prompb/io/prometheus/write/v2/codec.go | 216 ++ .../prompb/io/prometheus/write/v2/custom.go | 165 + .../prompb/io/prometheus/write/v2/symbols.go | 83 + .../prompb/io/prometheus/write/v2/types.pb.go | 3241 +++++++++++++++++ .../prompb/io/prometheus/write/v2/types.proto | 260 ++ .../prometheus/prometheus/promql/engine.go | 275 +- .../prometheus/prometheus/promql/functions.go | 77 +- .../promql/parser/generated_parser.y | 121 +- .../promql/parser/generated_parser.y.go | 1014 +++--- .../prometheus/promql/parser/lex.go | 123 +- .../prometheus/promql/parser/parse.go | 27 +- .../prometheus/prometheus/promql/quantile.go | 16 +- .../prometheus/prometheus/rules/group.go | 37 +- .../prometheus/prometheus/rules/manager.go | 12 +- .../prometheus/prometheus/scrape/manager.go | 8 +- .../prometheus/prometheus/scrape/metrics.go | 10 + .../prometheus/prometheus/scrape/scrape.go | 36 +- .../prometheus/prometheus/scrape/target.go | 23 +- .../prometheus/storage/interface.go | 18 +- .../prometheus/storage/memoized_iterator.go | 5 + .../prometheus/prometheus/storage/merge.go | 42 +- .../prometheus/prometheus/storage/noop.go | 8 +- .../prometheus/storage/remote/client.go | 57 +- .../prometheus/storage/remote/codec.go | 211 +- .../storage/remote/max_timestamp.go | 6 - .../storage/remote/metadata_watcher.go | 4 +- .../prometheus/normalize_name.go | 2 - .../otlptranslator/prometheus/unit_to_ucum.go | 1 - .../prometheusremotewrite/helper.go | 14 +- .../prometheusremotewrite/histograms.go | 19 +- .../storage/remote/queue_manager.go | 472 ++- .../prometheus/storage/remote/read.go | 4 +- .../prometheus/storage/remote/storage.go | 4 +- .../prometheus/storage/remote/write.go | 12 +- .../storage/remote/write_handler.go | 466 ++- .../prometheus/storage/secondary.go | 8 +- .../tsdb/chunkenc/float_histogram.go | 52 +- .../prometheus/tsdb/chunkenc/histogram.go | 34 +- .../tsdb/chunkenc/histogram_meta.go | 90 +- .../prometheus/tsdb/chunkenc/xor.go | 2 +- .../prometheus/tsdb/chunks/chunks.go | 9 - .../prometheus/prometheus/tsdb/db.go | 3 + .../prometheus/prometheus/tsdb/head.go | 24 +- .../prometheus/prometheus/tsdb/head_append.go | 113 +- .../prometheus/tsdb/head_dedupelabels.go | 95 + .../prometheus/prometheus/tsdb/head_other.go | 32 + .../prometheus/prometheus/tsdb/head_read.go | 150 +- .../prometheus/prometheus/tsdb/head_wal.go | 8 +- .../prometheus/tsdb/index/postings.go | 4 +- .../prometheus/prometheus/tsdb/ooo_head.go | 107 +- .../prometheus/tsdb/ooo_head_read.go | 78 +- .../prometheus/prometheus/tsdb/querier.go | 4 +- .../prometheus/tsdb/record/record.go | 2 +- .../prometheus/prometheus/tsdb/testutil.go | 176 + .../prometheus/tsdb/tsdbutil/histogram.go | 28 + .../prometheus/tsdb/wlog/watcher.go | 38 +- .../util/annotations/annotations.go | 99 +- .../prometheus/prometheus/web/api/v1/api.go | 55 +- .../execution/aggregate/accumulator.go | 124 +- .../execution/aggregate/count_values.go | 198 + .../execution/aggregate/hashaggregate.go | 39 +- .../execution/aggregate/scalar_table.go | 27 +- .../execution/aggregate/vector_table.go | 26 +- .../promql-engine/execution/binary/vector.go | 73 +- .../promql-engine/execution/execution.go | 14 +- .../execution/function/histogram.go | 10 +- .../execution/function/quantile.go | 21 +- .../promql-engine/execution/noop/operator.go | 29 +- .../execution/remote/operator.go | 4 +- .../promql-engine/execution/scan/functions.go | 215 +- .../promql-engine/execution/scan/subquery.go | 27 +- .../execution/warnings/context.go | 11 +- .../promql-engine/logicalplan/codec.go | 42 +- .../promql-engine/logicalplan/distribute.go | 8 +- .../promql-engine/logicalplan/exprutil.go | 17 + .../logicalplan/histogram_stats.go | 45 + .../logicalplan/logical_nodes.go | 4 + .../promql-engine/logicalplan/plan.go | 74 + .../storage/prometheus/histograms.go | 66 + .../storage/prometheus/matrix_selector.go | 6 +- .../storage/prometheus/scanners.go | 51 +- .../storage/prometheus/series_selector.go | 4 +- .../thanos/pkg/discovery/dns/resolver.go | 4 +- .../thanos-io/thanos/pkg/query/querier.go | 4 +- .../thanos-io/thanos/pkg/store/tsdb.go | 4 +- .../contrib/propagators/autoprop/doc.go | 13 +- .../propagators/autoprop/propagator.go | 17 +- .../contrib/propagators/autoprop/registry.go | 13 +- .../contrib/propagators/b3/b3_config.go | 13 +- .../contrib/propagators/b3/b3_propagator.go | 20 +- .../contrib/propagators/b3/context.go | 13 +- .../contrib/propagators/b3/doc.go | 13 +- .../contrib/propagators/b3/version.go | 19 +- .../contrib/propagators/jaeger/context.go | 13 +- .../contrib/propagators/jaeger/doc.go | 13 +- .../propagators/jaeger/jaeger_propagator.go | 13 +- .../contrib/propagators/jaeger/version.go | 19 +- .../contrib/propagators/ot/doc.go | 13 +- .../contrib/propagators/ot/ot_propagator.go | 16 +- .../contrib/propagators/ot/version.go | 19 +- vendor/golang.org/x/exp/slices/sort.go | 4 +- vendor/gonum.org/v1/gonum/AUTHORS | 11 + vendor/gonum.org/v1/gonum/CONTRIBUTORS | 11 + vendor/gonum.org/v1/gonum/floats/README.md | 5 +- vendor/k8s.io/klog/v2/klog.go | 76 +- vendor/modules.txt | 59 +- 152 files changed, 9223 insertions(+), 2311 deletions(-) create mode 100644 vendor/github.com/prometheus/prometheus/prompb/codec.go create mode 100644 vendor/github.com/prometheus/prometheus/prompb/io/prometheus/write/v2/codec.go create mode 100644 vendor/github.com/prometheus/prometheus/prompb/io/prometheus/write/v2/custom.go create mode 100644 vendor/github.com/prometheus/prometheus/prompb/io/prometheus/write/v2/symbols.go create mode 100644 vendor/github.com/prometheus/prometheus/prompb/io/prometheus/write/v2/types.pb.go create mode 100644 vendor/github.com/prometheus/prometheus/prompb/io/prometheus/write/v2/types.proto create mode 100644 vendor/github.com/prometheus/prometheus/tsdb/head_dedupelabels.go create mode 100644 vendor/github.com/prometheus/prometheus/tsdb/head_other.go create mode 100644 vendor/github.com/prometheus/prometheus/tsdb/testutil.go create mode 100644 vendor/github.com/thanos-io/promql-engine/execution/aggregate/count_values.go create mode 100644 vendor/github.com/thanos-io/promql-engine/logicalplan/histogram_stats.go create mode 100644 vendor/github.com/thanos-io/promql-engine/storage/prometheus/histograms.go diff --git a/go.mod b/go.mod index 4dfbc966e5..e2634ddf43 100644 --- a/go.mod +++ b/go.mod @@ -1,6 +1,8 @@ module github.com/cortexproject/cortex -go 1.22 +go 1.22.0 + +toolchain go1.22.5 require ( github.com/Masterminds/squirrel v1.5.4 @@ -10,7 +12,7 @@ require ( github.com/aws/aws-sdk-go v1.54.19 github.com/bradfitz/gomemcache v0.0.0-20230905024940-24af94b03874 github.com/cespare/xxhash v1.1.0 - github.com/cortexproject/promqlsmith v0.0.0-20240328172224-5e341f0dd08e + github.com/cortexproject/promqlsmith v0.0.0-20240506042652-6cfdd9739a5e github.com/dustin/go-humanize v1.0.1 github.com/efficientgo/core v1.0.0-rc.2 github.com/facette/natsort v0.0.0-20181210072756-2cd4dd1e2dcb @@ -46,14 +48,14 @@ require ( github.com/prometheus/client_model v0.6.1 github.com/prometheus/common v0.55.0 // Prometheus maps version 2.x.y to tags v0.x.y. - github.com/prometheus/prometheus v0.53.1-0.20240625160125-1abeebacb870 + github.com/prometheus/prometheus v0.53.2-0.20240718123124-e9dec5fc537b github.com/segmentio/fasthash v1.0.3 github.com/sony/gobreaker v1.0.0 github.com/spf13/afero v1.11.0 github.com/stretchr/testify v1.9.0 github.com/thanos-io/objstore v0.0.0-20240622095743-1afe5d4bc3cd - github.com/thanos-io/promql-engine v0.0.0-20240515161521-93aa311933cf - github.com/thanos-io/thanos v0.36.0-rc.0 + github.com/thanos-io/promql-engine v0.0.0-20240718195911-cdbd6dfed36b + github.com/thanos-io/thanos v0.35.2-0.20240722172812-990a60b72647 github.com/uber/jaeger-client-go v2.30.0+incompatible github.com/weaveworks/common v0.0.0-20230728070032-dd9e68f319d5 go.etcd.io/etcd/api/v3 v3.5.15 @@ -82,7 +84,7 @@ require ( github.com/google/go-cmp v0.6.0 github.com/sercand/kuberesolver/v4 v4.0.0 go.opentelemetry.io/collector/pdata v1.12.0 - golang.org/x/exp v0.0.0-20240119083558-1b970713d09a + golang.org/x/exp v0.0.0-20240613232115-7f521ea00fb8 google.golang.org/protobuf v1.34.2 ) @@ -93,9 +95,9 @@ require ( cloud.google.com/go/compute/metadata v0.3.0 // indirect cloud.google.com/go/iam v1.1.8 // indirect cloud.google.com/go/storage v1.40.0 // indirect - github.com/Azure/azure-sdk-for-go/sdk/azcore v1.11.1 // indirect - github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.6.0 // indirect - github.com/Azure/azure-sdk-for-go/sdk/internal v1.8.0 // indirect + github.com/Azure/azure-sdk-for-go/sdk/azcore v1.12.0 // indirect + github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.7.0 // indirect + github.com/Azure/azure-sdk-for-go/sdk/internal v1.9.0 // indirect github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.3.0 // indirect github.com/AzureAD/microsoft-authentication-library-for-go v1.2.2 // indirect github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751 // indirect @@ -209,13 +211,13 @@ require ( github.com/zhangyunhao116/umap v0.0.0-20221211160557-cb7705fafa39 // indirect go.mongodb.org/mongo-driver v1.14.0 // indirect go.opencensus.io v0.24.0 // indirect - go.opentelemetry.io/collector/semconv v0.101.0 // indirect + go.opentelemetry.io/collector/semconv v0.104.0 // indirect go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.49.0 // indirect go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.52.0 // indirect - go.opentelemetry.io/contrib/propagators/autoprop v0.38.0 // indirect - go.opentelemetry.io/contrib/propagators/b3 v1.13.0 // indirect - go.opentelemetry.io/contrib/propagators/jaeger v1.13.0 // indirect - go.opentelemetry.io/contrib/propagators/ot v1.13.0 // indirect + go.opentelemetry.io/contrib/propagators/autoprop v0.53.0 // indirect + go.opentelemetry.io/contrib/propagators/b3 v1.28.0 // indirect + go.opentelemetry.io/contrib/propagators/jaeger v1.28.0 // indirect + go.opentelemetry.io/contrib/propagators/ot v1.28.0 // indirect go.opentelemetry.io/otel/metric v1.28.0 // indirect go.opentelemetry.io/proto/otlp v1.3.1 // indirect go.uber.org/goleak v1.3.0 // indirect @@ -229,17 +231,17 @@ require ( golang.org/x/sys v0.22.0 // indirect golang.org/x/text v0.16.0 // indirect golang.org/x/tools v0.22.0 // indirect - gonum.org/v1/gonum v0.12.0 // indirect + gonum.org/v1/gonum v0.15.0 // indirect google.golang.org/api v0.183.0 // indirect google.golang.org/genproto v0.0.0-20240528184218-531527333157 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20240701130421-f6361c86f094 // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240701130421-f6361c86f094 // indirect gopkg.in/alecthomas/kingpin.v2 v2.2.6 // indirect gopkg.in/telebot.v3 v3.2.1 // indirect - k8s.io/apimachinery v0.29.3 // indirect - k8s.io/client-go v0.29.3 // indirect - k8s.io/klog/v2 v2.120.1 // indirect - k8s.io/utils v0.0.0-20230726121419-3b25d923346b // indirect + k8s.io/apimachinery v0.30.2 // indirect + k8s.io/client-go v0.30.2 // indirect + k8s.io/klog/v2 v2.130.1 // indirect + k8s.io/utils v0.0.0-20240502163921-fe8a2dddb1d0 // indirect ) // Override since git.apache.org is down. The docs say to fetch from github. diff --git a/go.sum b/go.sum index 210b07f262..ae117c8408 100644 --- a/go.sum +++ b/go.sum @@ -524,12 +524,12 @@ cloud.google.com/go/workflows v1.10.0/go.mod h1:fZ8LmRmZQWacon9UCX1r/g/DfAXx5VcP dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= gioui.org v0.0.0-20210308172011-57750fc8a0a6/go.mod h1:RSH6KIUZ0p2xy5zHDxgAM4zumjgTw83q2ge/PI+yyw8= git.sr.ht/~sbinet/gg v0.3.1/go.mod h1:KGYtlADtqsqANL9ueOFkWymvzUvLMQllU5Ixo+8v3pc= -github.com/Azure/azure-sdk-for-go/sdk/azcore v1.11.1 h1:E+OJmp2tPvt1W+amx48v1eqbjDYsgN+RzP4q16yV5eM= -github.com/Azure/azure-sdk-for-go/sdk/azcore v1.11.1/go.mod h1:a6xsAQUZg+VsS3TJ05SRp524Hs4pZ/AeFSr5ENf0Yjo= -github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.6.0 h1:U2rTu3Ef+7w9FHKIAXM6ZyqF3UOWJZ12zIm8zECAFfg= -github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.6.0/go.mod h1:9kIvujWAA58nmPmWB1m23fyWic1kYZMxD9CxaWn4Qpg= -github.com/Azure/azure-sdk-for-go/sdk/internal v1.8.0 h1:jBQA3cKT4L2rWMpgE7Yt3Hwh2aUj8KXjIGLxjHeYNNo= -github.com/Azure/azure-sdk-for-go/sdk/internal v1.8.0/go.mod h1:4OG6tQ9EOP/MT0NMjDlRzWoVFxfu9rN9B2X+tlSVktg= +github.com/Azure/azure-sdk-for-go/sdk/azcore v1.12.0 h1:1nGuui+4POelzDwI7RG56yfQJHCnKvwfMoU7VsEp+Zg= +github.com/Azure/azure-sdk-for-go/sdk/azcore v1.12.0/go.mod h1:99EvauvlcJ1U06amZiksfYz/3aFGyIhWGHVyiZXtBAI= +github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.7.0 h1:tfLQ34V6F7tVSwoTf/4lH5sE0o6eCJuNDTmH09nDpbc= +github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.7.0/go.mod h1:9kIvujWAA58nmPmWB1m23fyWic1kYZMxD9CxaWn4Qpg= +github.com/Azure/azure-sdk-for-go/sdk/internal v1.9.0 h1:H+U3Gk9zY56G3u872L82bk4thcsy2Gghb9ExT4Zvm1o= +github.com/Azure/azure-sdk-for-go/sdk/internal v1.9.0/go.mod h1:mgrmMSgaLp9hmax62XQTd0N4aAqSE5E0DulSpVYK7vc= github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/compute/armcompute/v5 v5.7.0 h1:LkHbJbgF3YyvC53aqYGR+wWQDn2Rdp9AQdGndf9QvY4= github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/compute/armcompute/v5 v5.7.0/go.mod h1:QyiQdW4f4/BIfB8ZutZ2s+28RAgfa/pT+zS++ZHyM1I= github.com/Azure/azure-sdk-for-go/sdk/resourcemanager/network/armnetwork/v4 v4.3.0 h1:bXwSugBiSbgtz7rOtbfGf+woewp4f06orW9OP5BjHLA= @@ -704,8 +704,8 @@ github.com/coreos/go-systemd/v22 v22.3.2/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSV github.com/coreos/go-systemd/v22 v22.4.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/coreos/go-systemd/v22 v22.5.0 h1:RrqgGjYQKalulkV8NGVIfkXQf6YYmOyiJKk8iXXhfZs= github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= -github.com/cortexproject/promqlsmith v0.0.0-20240328172224-5e341f0dd08e h1:NEErJR7YmpQ28oZezPCeOwMvW2qrZRdR+AAB6n5oHiE= -github.com/cortexproject/promqlsmith v0.0.0-20240328172224-5e341f0dd08e/go.mod h1:fcysbw4fOsOipXKeXPXWSh7tXrUQSUr5V4duojv0oCM= +github.com/cortexproject/promqlsmith v0.0.0-20240506042652-6cfdd9739a5e h1:nOWmgQD3L/Z0bmm29iDxB7nlqjMnh7yD/PNOx9rnZmA= +github.com/cortexproject/promqlsmith v0.0.0-20240506042652-6cfdd9739a5e/go.mod h1:+bSqRETXJ1uk2S93m//htzTVqu8DJPvlGEb3bSE9PzI= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= @@ -1068,8 +1068,8 @@ github.com/hashicorp/go-uuid v1.0.0/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/b github.com/hashicorp/go-uuid v1.0.1/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= github.com/hashicorp/go-uuid v1.0.3 h1:2gKiV6YVmrJ1i2CKKa9obLvRieoRGviZFL26PcT/Co8= github.com/hashicorp/go-uuid v1.0.3/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= -github.com/hashicorp/go-version v1.6.0 h1:feTTfFNnjP967rlCxM/I9g701jU+RN74YKx2mOkIeek= -github.com/hashicorp/go-version v1.6.0/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09ZGVZPK5anwXA= +github.com/hashicorp/go-version v1.7.0 h1:5tqGy27NaOTB8yJKUZELlFAS/LTKJkrmONwQKeRZfjY= +github.com/hashicorp/go-version v1.7.0/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09ZGVZPK5anwXA= github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hashicorp/golang-lru v0.5.4/go.mod h1:iADmTwqILo4mZ8BN3D2Q6+9jd8WM5uGBxy+E8yxSoD4= @@ -1344,8 +1344,8 @@ github.com/prometheus/procfs v0.8.0/go.mod h1:z7EfXMXOkbkqb9IINtpCn86r/to3BnA0ua github.com/prometheus/procfs v0.9.0/go.mod h1:+pB4zwohETzFnmlpe6yd2lSc+0/46IYZRB/chUwxUZY= 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/prometheus v0.53.1-0.20240625160125-1abeebacb870 h1:crWrpxOPZFIZ+M7AniW0Aps4TvRvMdr0HmtNFeEj79I= -github.com/prometheus/prometheus v0.53.1-0.20240625160125-1abeebacb870/go.mod h1:FcNs5wa7M9yV8IlxlB/05s5oy9vULUIlu/tZsviRIT8= +github.com/prometheus/prometheus v0.53.2-0.20240718123124-e9dec5fc537b h1:3XXVgSiLpgc9xXIsmBmgmd9I6bPaAaP/CBwwGV+y92U= +github.com/prometheus/prometheus v0.53.2-0.20240718123124-e9dec5fc537b/go.mod h1:TzWm3Q1bk8bzJ6t7IwnBfzcQvf4FZGUm/M5ynmaqfVI= github.com/redis/rueidis v1.0.14-go1.18 h1:dGir5z8w8X1ex7JWO/Zx2FMBrZgQ8Yjm+lw9fPLSNGw= github.com/redis/rueidis v1.0.14-go1.18/go.mod h1:HGekzV3HbmzFmRK6j0xic8Z9119+ECoGMjeN1TV1NYU= github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= @@ -1424,10 +1424,10 @@ github.com/thanos-community/galaxycache v0.0.0-20211122094458-3a32041a1f1e h1:f1 github.com/thanos-community/galaxycache v0.0.0-20211122094458-3a32041a1f1e/go.mod h1:jXcofnrSln/cLI6/dhlBxPQZEEQHVPCcFaH75M+nSzM= github.com/thanos-io/objstore v0.0.0-20240622095743-1afe5d4bc3cd h1:YBDmfk3k/eOYLfP4SR/vZdXi5/65pqWPmR9Do2WjkRM= github.com/thanos-io/objstore v0.0.0-20240622095743-1afe5d4bc3cd/go.mod h1:3ukSkG4rIRUGkKM4oIz+BSuUx2e3RlQVVv3Cc3W+Tv4= -github.com/thanos-io/promql-engine v0.0.0-20240515161521-93aa311933cf h1:R6of9adrCWXhETBstsFzNqrZou5UqeY3fh3k5yv5POY= -github.com/thanos-io/promql-engine v0.0.0-20240515161521-93aa311933cf/go.mod h1:FEPnabuTql1bDA4OUM41mwcZOJ20R436k8vq+xtGEG0= -github.com/thanos-io/thanos v0.36.0-rc.0 h1:R89Z+Ku3Izqu9kagjMZ3IpCR5ButX37uzliNeo63JX4= -github.com/thanos-io/thanos v0.36.0-rc.0/go.mod h1:f7LiW4+/xvV5+gkseMuVbQnrbFTFnCPv5+X1M6mXkn4= +github.com/thanos-io/promql-engine v0.0.0-20240718195911-cdbd6dfed36b h1:V06gjM1OFiJydoClwiGOMCpBWLSpxa5FZBvBc3coQg4= +github.com/thanos-io/promql-engine v0.0.0-20240718195911-cdbd6dfed36b/go.mod h1:Gtv7CJIxGyiGsT+bNDg4nOAsL/bVKLlpfOZUSLSyYfY= +github.com/thanos-io/thanos v0.35.2-0.20240722172812-990a60b72647 h1:0qjB7yYBB4LeGw+BWVrEsPMHabYgXjfh2pD2vkuRa9s= +github.com/thanos-io/thanos v0.35.2-0.20240722172812-990a60b72647/go.mod h1:4QL7wA5z+Uh4tE6fm4Ar+nqQKgAxWzdOWdcBBjABUvo= github.com/tv42/httpunix v0.0.0-20150427012821-b75d8614f926/go.mod h1:9ESjWnEqriFuLhtthL60Sar/7RFoluCcXsuvEwTV5KM= github.com/uber/jaeger-client-go v2.28.0+incompatible/go.mod h1:WVhlPFC8FDjOFMMWRy2pZqQJSXxYSwNYOkTr/Z6d3Kk= github.com/uber/jaeger-client-go v2.30.0+incompatible h1:D6wyKGCecFaSRUpo8lCVbaOOb6ThwMmTEbhRwtKR97o= @@ -1478,22 +1478,22 @@ go.opencensus.io v0.24.0 h1:y73uSU6J157QMP2kn2r30vwW1A2W2WFwSCGnAVxeaD0= go.opencensus.io v0.24.0/go.mod h1:vNK8G9p7aAivkbmorf4v+7Hgx+Zs0yY+0fOtgBfjQKo= go.opentelemetry.io/collector/pdata v1.12.0 h1:Xx5VK1p4VO0md8MWm2icwC1MnJ7f8EimKItMWw46BmA= go.opentelemetry.io/collector/pdata v1.12.0/go.mod h1:MYeB0MmMAxeM0hstCFrCqWLzdyeYySim2dG6pDT6nYI= -go.opentelemetry.io/collector/semconv v0.101.0 h1:tOe9iTe9dDCnvz/bqgfNRr4w80kXG8505tQJ5h5v08Q= -go.opentelemetry.io/collector/semconv v0.101.0/go.mod h1:8ElcRZ8Cdw5JnvhTOQOdYizkJaQ10Z2fS+R6djOnj6A= +go.opentelemetry.io/collector/semconv v0.104.0 h1:dUvajnh+AYJLEW/XOPk0T0BlwltSdi3vrjO7nSOos3k= +go.opentelemetry.io/collector/semconv v0.104.0/go.mod h1:yMVUCNoQPZVq/IPfrHrnntZTWsLf5YGZ7qwKulIl5hw= go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.49.0 h1:4Pp6oUg3+e/6M4C0A/3kJ2VYa++dsWVTtGgLVj5xtHg= go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.49.0/go.mod h1:Mjt1i1INqiaoZOMGR1RIUJN+i3ChKoFRqzrRQhlkbs0= go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.52.0 h1:9l89oX4ba9kHbBol3Xin3leYJ+252h0zszDtBwyKe2A= go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.52.0/go.mod h1:XLZfZboOJWHNKUv7eH0inh0E9VV6eWDFB/9yJyTLPp0= -go.opentelemetry.io/contrib/propagators/autoprop v0.38.0 h1:WZwiLCwOL0XW/6TVT7LTtdRDveoHZ6q3wL+0iYsBcdE= -go.opentelemetry.io/contrib/propagators/autoprop v0.38.0/go.mod h1:JBebP2d0HiffbfelbIEoBOCl4790g7Z8lD1scUd3Vd8= +go.opentelemetry.io/contrib/propagators/autoprop v0.53.0 h1:4zaVLcJ5mvYw0vlk63TX62qS4qty/4jAY1BKZ1usu18= +go.opentelemetry.io/contrib/propagators/autoprop v0.53.0/go.mod h1:RPlvYtxp5D8PKnRzyPM+rwMQrvzdlfA49Sgworkg7aQ= go.opentelemetry.io/contrib/propagators/aws v1.28.0 h1:acyTl4oyin/iLr5Nz3u7p/PKHUbLh42w/fqg9LblExk= go.opentelemetry.io/contrib/propagators/aws v1.28.0/go.mod h1:5WgIv6yG9DvLlSY2uIHrYSeVVwCDCqp4jhwinNNyeT4= -go.opentelemetry.io/contrib/propagators/b3 v1.13.0 h1:f17PBmZK60RoHvOpJVqEka8oS2EXjpjHquESD/8zZ50= -go.opentelemetry.io/contrib/propagators/b3 v1.13.0/go.mod h1:zy2hz1TpGUoJzSwlBchVGvVAFQS8s2pglKLbrAFZ+Sc= -go.opentelemetry.io/contrib/propagators/jaeger v1.13.0 h1:+tVlvpiQMOCzi4EYCaBjblibpyKfqoph0fcITmtXMws= -go.opentelemetry.io/contrib/propagators/jaeger v1.13.0/go.mod h1:Qf7eVCLYawiNIB+A81kk8aFDFwYqXSqmt0N2RcvkLLI= -go.opentelemetry.io/contrib/propagators/ot v1.13.0 h1:tHWNd0WRS6w9keZoZg9aF3zYohdaBacQfojPYZJgATQ= -go.opentelemetry.io/contrib/propagators/ot v1.13.0/go.mod h1:R6Op9T6LxNaMRVlGD0wVwz40LSsAq296CXiEydKLQBU= +go.opentelemetry.io/contrib/propagators/b3 v1.28.0 h1:XR6CFQrQ/ttAYmTBX2loUEFGdk1h17pxYI8828dk/1Y= +go.opentelemetry.io/contrib/propagators/b3 v1.28.0/go.mod h1:DWRkzJONLquRz7OJPh2rRbZ7MugQj62rk7g6HRnEqh0= +go.opentelemetry.io/contrib/propagators/jaeger v1.28.0 h1:xQ3ktSVS128JWIaN1DiPGIjcH+GsvkibIAVRWFjS9eM= +go.opentelemetry.io/contrib/propagators/jaeger v1.28.0/go.mod h1:O9HIyI2kVBrFoEwQZ0IN6PHXykGoit4mZV2aEjkTRH4= +go.opentelemetry.io/contrib/propagators/ot v1.28.0 h1:rmlG+2pc5k5M7Y7izDrxAHZUIwDERdGMTD9oMV7llMk= +go.opentelemetry.io/contrib/propagators/ot v1.28.0/go.mod h1:MNgXIn+UrMbNGpd7xyckyo2LCHIgCdmdjEE7YNZGG+w= go.opentelemetry.io/otel v1.28.0 h1:/SqNcYk+idO0CxKEUOtKQClMK/MimZihKYMruSMViUo= go.opentelemetry.io/otel v1.28.0/go.mod h1:q68ijF8Fc8CnMHKyzqL6akLO46ePnjkgfIMIjUIX9z4= go.opentelemetry.io/otel/bridge/opentracing v1.28.0 h1:erHvOxIUFnSXj/HuS5SqaKe2CbWSBskONXm2bEBxYgc= @@ -1562,8 +1562,8 @@ golang.org/x/exp v0.0.0-20200119233911-0405dc783f0a/go.mod h1:2RIsYlXP63K8oxa1u0 golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM= golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= golang.org/x/exp v0.0.0-20220827204233-334a2380cb91/go.mod h1:cyybsKvd6eL0RnXn6p/Grxp8F5bW7iYuBgsNCOHpMYE= -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/exp v0.0.0-20240613232115-7f521ea00fb8 h1:yixxcjnhBmY0nkL253HFVIm0JsFHwrHdT3Yh6szTnfY= +golang.org/x/exp v0.0.0-20240613232115-7f521ea00fb8/go.mod h1:jj3sYF3dwk5D+ghuXyeI3r5MFf+NT2An6/9dOA95KSI= golang.org/x/image v0.0.0-20180708004352-c73c2afc3b81/go.mod h1:ux5Hcp/YLpHSI86hEcLt0YII63i6oz57MZXIpbrjZUs= golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= @@ -1945,8 +1945,8 @@ gonum.org/v1/gonum v0.0.0-20180816165407-929014505bf4/go.mod h1:Y+Yx5eoAFn32cQvJ gonum.org/v1/gonum v0.8.2/go.mod h1:oe/vMfY3deqTw+1EZJhuvEW2iwGF1bW9wwu7XCu0+v0= gonum.org/v1/gonum v0.9.3/go.mod h1:TZumC3NeyVQskjXqmyWt4S3bINhy7B4eYwW69EbyX+0= gonum.org/v1/gonum v0.11.0/go.mod h1:fSG4YDCxxUZQJ7rKsQrj0gMOg00Il0Z96/qMA4bVQhA= -gonum.org/v1/gonum v0.12.0 h1:xKuo6hzt+gMav00meVPUlXwSdoEJP46BR+wdxQEFK2o= -gonum.org/v1/gonum v0.12.0/go.mod h1:73TDxJfAAHeA8Mk9mf8NlIppyhQNo5GLTcYeqgo2lvY= +gonum.org/v1/gonum v0.15.0 h1:2lYxjRbTYyxkJxlhC+LvJIx3SsANPdRybu1tGj9/OrQ= +gonum.org/v1/gonum v0.15.0/go.mod h1:xzZVBJBtS+Mz4q0Yl2LJTk+OxOg4jiXZ7qBoM0uISGo= gonum.org/v1/netlib v0.0.0-20190313105609-8cb42192e0e0/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw= gonum.org/v1/plot v0.0.0-20190515093506-e2840ee46a6b/go.mod h1:Wt8AAjI+ypCyYX3nZBvf6cAIx93T+c/OS2HFAYskSZc= gonum.org/v1/plot v0.9.0/go.mod h1:3Pcqqmp6RHvJI72kgb8fThyUnav364FOsdDo2aGW5lY= @@ -2253,18 +2253,18 @@ honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= honnef.co/go/tools v0.0.1-2020.1.4/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= honnef.co/go/tools v0.1.3/go.mod h1:NgwopIslSNH47DimFoV78dnkksY2EFtX0ajyb3K/las= -k8s.io/api v0.29.3 h1:2ORfZ7+bGC3YJqGpV0KSDDEVf8hdGQ6A03/50vj8pmw= -k8s.io/api v0.29.3/go.mod h1:y2yg2NTyHUUkIoTC+phinTnEa3KFM6RZ3szxt014a80= -k8s.io/apimachinery v0.29.3 h1:2tbx+5L7RNvqJjn7RIuIKu9XTsIZ9Z5wX2G22XAa5EU= -k8s.io/apimachinery v0.29.3/go.mod h1:hx/S4V2PNW4OMg3WizRrHutyB5la0iCUbZym+W0EQIU= -k8s.io/client-go v0.29.3 h1:R/zaZbEAxqComZ9FHeQwOh3Y1ZUs7FaHKZdQtIc2WZg= -k8s.io/client-go v0.29.3/go.mod h1:tkDisCvgPfiRpxGnOORfkljmS+UrW+WtXAy2fTvXJB0= -k8s.io/klog/v2 v2.120.1 h1:QXU6cPEOIslTGvZaXvFWiP9VKyeet3sawzTOvdXb4Vw= -k8s.io/klog/v2 v2.120.1/go.mod h1:3Jpz1GvMt720eyJH1ckRHK1EDfpxISzJ7I9OYgaDtPE= +k8s.io/api v0.30.2 h1:+ZhRj+28QT4UOH+BKznu4CBgPWgkXO7XAvMcMl0qKvI= +k8s.io/api v0.30.2/go.mod h1:ULg5g9JvOev2dG0u2hig4Z7tQ2hHIuS+m8MNZ+X6EmI= +k8s.io/apimachinery v0.30.2 h1:fEMcnBj6qkzzPGSVsAZtQThU62SmQ4ZymlXRC5yFSCg= +k8s.io/apimachinery v0.30.2/go.mod h1:iexa2somDaxdnj7bha06bhb43Zpa6eWH8N8dbqVjTUc= +k8s.io/client-go v0.30.2 h1:sBIVJdojUNPDU/jObC+18tXWcTJVcwyqS9diGdWHk50= +k8s.io/client-go v0.30.2/go.mod h1:JglKSWULm9xlJLx4KCkfLLQ7XwtlbflV6uFFSHTMgVs= +k8s.io/klog/v2 v2.130.1 h1:n9Xl7H1Xvksem4KFG4PYbdQCQxqc/tTUyrgXaOhHSzk= +k8s.io/klog/v2 v2.130.1/go.mod h1:3Jpz1GvMt720eyJH1ckRHK1EDfpxISzJ7I9OYgaDtPE= k8s.io/kube-openapi v0.0.0-20240228011516-70dd3763d340 h1:BZqlfIlq5YbRMFko6/PM7FjZpUb45WallggurYhKGag= k8s.io/kube-openapi v0.0.0-20240228011516-70dd3763d340/go.mod h1:yD4MZYeKMBwQKVht279WycxKyM84kkAx2DPrTXaeb98= -k8s.io/utils v0.0.0-20230726121419-3b25d923346b h1:sgn3ZU783SCgtaSJjpcVVlRqd6GSnlTLKgpAAttJvpI= -k8s.io/utils v0.0.0-20230726121419-3b25d923346b/go.mod h1:OLgZIPagt7ERELqWJFomSt595RzquPNLL48iOWgYOg0= +k8s.io/utils v0.0.0-20240502163921-fe8a2dddb1d0 h1:jgGTlFYnhF1PM1Ax/lAlxUPE+KfCIXHaathvJg1C3ak= +k8s.io/utils v0.0.0-20240502163921-fe8a2dddb1d0/go.mod h1:OLgZIPagt7ERELqWJFomSt595RzquPNLL48iOWgYOg0= lukechampine.com/uint128 v1.1.1/go.mod h1:c4eWIwlEGaxC/+H1VguhU4PHXNWDCDMUlWdIWl2j1gk= lukechampine.com/uint128 v1.2.0/go.mod h1:c4eWIwlEGaxC/+H1VguhU4PHXNWDCDMUlWdIWl2j1gk= modernc.org/cc/v3 v3.36.0/go.mod h1:NFUHyPn4ekoC/JHeZFfZurN6ixxawE1BnVonP/oahEI= diff --git a/integration/e2e/util.go b/integration/e2e/util.go index 68a25a3af3..141d043ab5 100644 --- a/integration/e2e/util.go +++ b/integration/e2e/util.go @@ -19,7 +19,6 @@ import ( "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/prompb" "github.com/prometheus/prometheus/storage" - "github.com/prometheus/prometheus/storage/remote" "github.com/prometheus/prometheus/tsdb" "github.com/prometheus/prometheus/tsdb/tsdbutil" "github.com/thanos-io/thanos/pkg/block/metadata" @@ -174,10 +173,10 @@ func GenerateHistogramSeries(name string, ts time.Time, i uint32, floatHistogram ) if floatHistogram { fh = tsdbutil.GenerateTestFloatHistogram(int(i)) - ph = remote.FloatHistogramToHistogramProto(tsMillis, fh) + ph = prompb.FromFloatHistogram(tsMillis, fh) } else { h = tsdbutil.GenerateTestHistogram(int(i)) - ph = remote.HistogramToHistogramProto(tsMillis, h) + ph = prompb.FromIntHistogram(tsMillis, h) } // Generate the series diff --git a/pkg/api/handlers.go b/pkg/api/handlers.go index 17bd42d7c4..84001dc8ca 100644 --- a/pkg/api/handlers.go +++ b/pkg/api/handlers.go @@ -227,6 +227,7 @@ func NewQuerierHandler( reg, nil, false, + nil, false, ) diff --git a/pkg/ingester/ingester.go b/pkg/ingester/ingester.go index ee82393e9f..e6cf5d9701 100644 --- a/pkg/ingester/ingester.go +++ b/pkg/ingester/ingester.go @@ -1534,7 +1534,7 @@ func (i *Ingester) labelsValuesCommon(ctx context.Context, req *client.LabelValu return nil, cleanup, err } defer c() - vals, _, err := q.LabelValues(ctx, labelName, matchers...) + vals, _, err := q.LabelValues(ctx, labelName, nil, matchers...) if err != nil { return nil, cleanup, err } @@ -1615,7 +1615,7 @@ func (i *Ingester) labelNamesCommon(ctx context.Context, req *client.LabelNamesR return nil, cleanup, err } defer c() - names, _, err := q.LabelNames(ctx) + names, _, err := q.LabelNames(ctx, nil) if err != nil { return nil, cleanup, err } diff --git a/pkg/querier/blocks_store_queryable.go b/pkg/querier/blocks_store_queryable.go index 3340e9ecb0..caf4de3d6b 100644 --- a/pkg/querier/blocks_store_queryable.go +++ b/pkg/querier/blocks_store_queryable.go @@ -336,7 +336,7 @@ func (q *blocksStoreQuerier) Select(ctx context.Context, _ bool, sp *storage.Sel return q.selectSorted(ctx, sp, matchers...) } -func (q *blocksStoreQuerier) LabelNames(ctx context.Context, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { +func (q *blocksStoreQuerier) LabelNames(ctx context.Context, hints *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { userID, err := tenant.TenantID(ctx) if err != nil { return nil, nil, err @@ -375,7 +375,7 @@ func (q *blocksStoreQuerier) LabelNames(ctx context.Context, matchers ...*labels return strutil.MergeSlices(resNameSets...), resWarnings, nil } -func (q *blocksStoreQuerier) LabelValues(ctx context.Context, name string, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { +func (q *blocksStoreQuerier) LabelValues(ctx context.Context, name string, hints *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { userID, err := tenant.TenantID(ctx) if err != nil { return nil, nil, err diff --git a/pkg/querier/blocks_store_queryable_test.go b/pkg/querier/blocks_store_queryable_test.go index 9d0d31f79b..d89aa96f43 100644 --- a/pkg/querier/blocks_store_queryable_test.go +++ b/pkg/querier/blocks_store_queryable_test.go @@ -2036,7 +2036,7 @@ func TestBlocksStoreQuerier_Labels(t *testing.T) { } if testFunc == "LabelNames" { - names, warnings, err := q.LabelNames(ctx) + names, warnings, err := q.LabelNames(ctx, nil) if testData.expectedErr != "" { require.Equal(t, testData.expectedErr, err.Error()) continue @@ -2053,7 +2053,7 @@ func TestBlocksStoreQuerier_Labels(t *testing.T) { } if testFunc == "LabelValues" { - values, warnings, err := q.LabelValues(ctx, labels.MetricName) + values, warnings, err := q.LabelValues(ctx, labels.MetricName, nil) if testData.expectedErr != "" { require.Equal(t, testData.expectedErr, err.Error()) continue diff --git a/pkg/querier/distributor_queryable.go b/pkg/querier/distributor_queryable.go index 8775a7ce56..5bd933c209 100644 --- a/pkg/querier/distributor_queryable.go +++ b/pkg/querier/distributor_queryable.go @@ -164,7 +164,7 @@ func (q *distributorQuerier) streamingSelect(ctx context.Context, sortSeries boo return series.NewConcreteSeriesSet(sortSeries, serieses) } -func (q *distributorQuerier) LabelValues(ctx context.Context, name string, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { +func (q *distributorQuerier) LabelValues(ctx context.Context, name string, hints *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { var ( lvs []string err error @@ -179,7 +179,7 @@ func (q *distributorQuerier) LabelValues(ctx context.Context, name string, match return lvs, nil, err } -func (q *distributorQuerier) LabelNames(ctx context.Context, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { +func (q *distributorQuerier) LabelNames(ctx context.Context, hints *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { if len(matchers) > 0 { return q.labelNamesWithMatchers(ctx, matchers...) } diff --git a/pkg/querier/distributor_queryable_test.go b/pkg/querier/distributor_queryable_test.go index 6740d1406b..fdef2c0aaf 100644 --- a/pkg/querier/distributor_queryable_test.go +++ b/pkg/querier/distributor_queryable_test.go @@ -223,7 +223,7 @@ func TestDistributorQuerier_LabelNames(t *testing.T) { require.NoError(t, err) ctx := context.Background() - names, warnings, err := querier.LabelNames(ctx, someMatchers...) + names, warnings, err := querier.LabelNames(ctx, nil, someMatchers...) require.NoError(t, err) assert.Empty(t, warnings) assert.Equal(t, labelNames, names) diff --git a/pkg/querier/error_translate_queryable.go b/pkg/querier/error_translate_queryable.go index ccf0cee8c7..0c55a15c58 100644 --- a/pkg/querier/error_translate_queryable.go +++ b/pkg/querier/error_translate_queryable.go @@ -123,13 +123,13 @@ type errorTranslateQuerier struct { fn ErrTranslateFn } -func (e errorTranslateQuerier) LabelValues(ctx context.Context, name string, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { - values, warnings, err := e.q.LabelValues(ctx, name, matchers...) +func (e errorTranslateQuerier) LabelValues(ctx context.Context, name string, hints *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { + values, warnings, err := e.q.LabelValues(ctx, name, hints, matchers...) return values, warnings, e.fn(err) } -func (e errorTranslateQuerier) LabelNames(ctx context.Context, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { - values, warnings, err := e.q.LabelNames(ctx, matchers...) +func (e errorTranslateQuerier) LabelNames(ctx context.Context, hints *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { + values, warnings, err := e.q.LabelNames(ctx, hints, matchers...) return values, warnings, e.fn(err) } @@ -147,13 +147,13 @@ type errorTranslateChunkQuerier struct { fn ErrTranslateFn } -func (e errorTranslateChunkQuerier) LabelValues(ctx context.Context, name string, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { - values, warnings, err := e.q.LabelValues(ctx, name, matchers...) +func (e errorTranslateChunkQuerier) LabelValues(ctx context.Context, name string, hints *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { + values, warnings, err := e.q.LabelValues(ctx, name, hints, matchers...) return values, warnings, e.fn(err) } -func (e errorTranslateChunkQuerier) LabelNames(ctx context.Context, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { - values, warnings, err := e.q.LabelNames(ctx, matchers...) +func (e errorTranslateChunkQuerier) LabelNames(ctx context.Context, hints *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { + values, warnings, err := e.q.LabelNames(ctx, hints, matchers...) return values, warnings, e.fn(err) } diff --git a/pkg/querier/error_translate_queryable_test.go b/pkg/querier/error_translate_queryable_test.go index e62cad6bb5..7317f9bcb5 100644 --- a/pkg/querier/error_translate_queryable_test.go +++ b/pkg/querier/error_translate_queryable_test.go @@ -163,6 +163,7 @@ func createPrometheusAPI(q storage.SampleAndChunkQueryable, engine promql.QueryE nil, nil, false, + nil, false, ) @@ -193,11 +194,11 @@ type errorTestQuerier struct { err error } -func (t errorTestQuerier) LabelValues(ctx context.Context, name string, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { +func (t errorTestQuerier) LabelValues(ctx context.Context, name string, _ *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { return nil, nil, t.err } -func (t errorTestQuerier) LabelNames(ctx context.Context, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { +func (t errorTestQuerier) LabelNames(ctx context.Context, _ *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { return nil, nil, t.err } diff --git a/pkg/querier/lazyquery/lazyquery.go b/pkg/querier/lazyquery/lazyquery.go index 93634db957..d00c974551 100644 --- a/pkg/querier/lazyquery/lazyquery.go +++ b/pkg/querier/lazyquery/lazyquery.go @@ -34,13 +34,13 @@ func (l LazyQuerier) Select(ctx context.Context, selectSorted bool, params *stor } // LabelValues implements Storage.Querier -func (l LazyQuerier) LabelValues(ctx context.Context, name string, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { - return l.next.LabelValues(ctx, name, matchers...) +func (l LazyQuerier) LabelValues(ctx context.Context, name string, hints *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { + return l.next.LabelValues(ctx, name, hints, matchers...) } // LabelNames implements Storage.Querier -func (l LazyQuerier) LabelNames(ctx context.Context, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { - return l.next.LabelNames(ctx, matchers...) +func (l LazyQuerier) LabelNames(ctx context.Context, hints *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { + return l.next.LabelNames(ctx, hints, matchers...) } // Close implements Storage.Querier diff --git a/pkg/querier/querier.go b/pkg/querier/querier.go index ac01c5e9ce..9b0f5d447f 100644 --- a/pkg/querier/querier.go +++ b/pkg/querier/querier.go @@ -422,7 +422,7 @@ func (q querier) Select(ctx context.Context, sortSeries bool, sp *storage.Select } // LabelValues implements storage.Querier. -func (q querier) LabelValues(ctx context.Context, name string, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { +func (q querier) LabelValues(ctx context.Context, name string, hints *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { ctx, stats, _, _, _, _, queriers, err := q.setupFromCtx(ctx) if err == errEmptyTimeRange { return nil, nil, nil @@ -435,7 +435,7 @@ func (q querier) LabelValues(ctx context.Context, name string, matchers ...*labe }() if len(queriers) == 1 { - return queriers[0].LabelValues(ctx, name, matchers...) + return queriers[0].LabelValues(ctx, name, hints, matchers...) } var ( @@ -451,7 +451,7 @@ func (q querier) LabelValues(ctx context.Context, name string, matchers ...*labe querier := querier g.Go(func() error { // NB: Values are sorted in Cortex already. - myValues, myWarnings, err := querier.LabelValues(ctx, name, matchers...) + myValues, myWarnings, err := querier.LabelValues(ctx, name, hints, matchers...) if err != nil { return err } @@ -472,7 +472,7 @@ func (q querier) LabelValues(ctx context.Context, name string, matchers ...*labe return strutil.MergeSlices(sets...), warnings, nil } -func (q querier) LabelNames(ctx context.Context, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { +func (q querier) LabelNames(ctx context.Context, hints *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { ctx, stats, _, _, _, _, queriers, err := q.setupFromCtx(ctx) if err == errEmptyTimeRange { return nil, nil, nil @@ -485,7 +485,7 @@ func (q querier) LabelNames(ctx context.Context, matchers ...*labels.Matcher) ([ }() if len(queriers) == 1 { - return queriers[0].LabelNames(ctx, matchers...) + return queriers[0].LabelNames(ctx, hints, matchers...) } var ( @@ -501,7 +501,7 @@ func (q querier) LabelNames(ctx context.Context, matchers ...*labels.Matcher) ([ querier := querier g.Go(func() error { // NB: Names are sorted in Cortex already. - myNames, myWarnings, err := querier.LabelNames(ctx, matchers...) + myNames, myWarnings, err := querier.LabelNames(ctx, hints, matchers...) if err != nil { return err } diff --git a/pkg/querier/querier_test.go b/pkg/querier/querier_test.go index 7b2eef63a8..5175af939b 100644 --- a/pkg/querier/querier_test.go +++ b/pkg/querier/querier_test.go @@ -1126,7 +1126,7 @@ func TestQuerier_ValidateQueryTimeRange_MaxQueryLookback(t *testing.T) { q, err := queryable.Querier(util.TimeToMillis(testData.queryStartTime), util.TimeToMillis(testData.queryEndTime)) require.NoError(t, err) - _, _, err = q.LabelNames(ctx) + _, _, err = q.LabelNames(ctx, nil) require.NoError(t, err) if !testData.expectedSkipped { @@ -1154,7 +1154,7 @@ func TestQuerier_ValidateQueryTimeRange_MaxQueryLookback(t *testing.T) { q, err := queryable.Querier(util.TimeToMillis(testData.queryStartTime), util.TimeToMillis(testData.queryEndTime)) require.NoError(t, err) - _, _, err = q.LabelNames(ctx, matchers...) + _, _, err = q.LabelNames(ctx, nil, matchers...) require.NoError(t, err) if !testData.expectedSkipped { @@ -1181,7 +1181,7 @@ func TestQuerier_ValidateQueryTimeRange_MaxQueryLookback(t *testing.T) { q, err := queryable.Querier(util.TimeToMillis(testData.queryStartTime), util.TimeToMillis(testData.queryEndTime)) require.NoError(t, err) - _, _, err = q.LabelValues(ctx, labels.MetricName) + _, _, err = q.LabelValues(ctx, labels.MetricName, nil) require.NoError(t, err) if !testData.expectedSkipped { @@ -1426,11 +1426,11 @@ func (q *mockStoreQuerier) Select(ctx context.Context, _ bool, sp *storage.Selec return partitionChunks(chunks, q.mint, q.maxt, q.chunkIteratorFunc) } -func (q *mockStoreQuerier) LabelValues(ctx context.Context, name string, labels ...*labels.Matcher) ([]string, annotations.Annotations, error) { +func (q *mockStoreQuerier) LabelValues(ctx context.Context, name string, _ *storage.LabelHints, labels ...*labels.Matcher) ([]string, annotations.Annotations, error) { return nil, nil, nil } -func (q *mockStoreQuerier) LabelNames(ctx context.Context, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { +func (q *mockStoreQuerier) LabelNames(ctx context.Context, _ *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { return nil, nil, nil } diff --git a/pkg/querier/remote_read_test.go b/pkg/querier/remote_read_test.go index 13e8881a2b..b3e7c0c28b 100644 --- a/pkg/querier/remote_read_test.go +++ b/pkg/querier/remote_read_test.go @@ -99,11 +99,11 @@ func (m mockQuerier) Select(ctx context.Context, sortSeries bool, sp *storage.Se return series.MatrixToSeriesSet(sortSeries, m.matrix) } -func (m mockQuerier) LabelValues(ctx context.Context, name string, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { +func (m mockQuerier) LabelValues(ctx context.Context, name string, _ *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { return nil, nil, nil } -func (m mockQuerier) LabelNames(ctx context.Context, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { +func (m mockQuerier) LabelNames(ctx context.Context, _ *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { return nil, nil, nil } diff --git a/pkg/querier/tenantfederation/merge_queryable.go b/pkg/querier/tenantfederation/merge_queryable.go index c172261d1e..92f4f47b6f 100644 --- a/pkg/querier/tenantfederation/merge_queryable.go +++ b/pkg/querier/tenantfederation/merge_queryable.go @@ -124,7 +124,7 @@ type mergeQuerier struct { // For the label `idLabelName` it will return all the underlying ids available. // For the label "original_" + `idLabelName it will return all the values // of the underlying queriers for `idLabelName`. -func (m *mergeQuerier) LabelValues(ctx context.Context, name string, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { +func (m *mergeQuerier) LabelValues(ctx context.Context, name string, hints *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { ids, queriers, err := m.callback(ctx, m.mint, m.maxt) if err != nil { return nil, nil, err @@ -132,7 +132,7 @@ func (m *mergeQuerier) LabelValues(ctx context.Context, name string, matchers .. // by pass when only single querier is returned if m.byPassWithSingleQuerier && len(queriers) == 1 { - return queriers[0].LabelValues(ctx, name, matchers...) + return queriers[0].LabelValues(ctx, name, hints, matchers...) } log, _ := spanlogger.New(ctx, "mergeQuerier.LabelValues") defer log.Span.Finish() @@ -156,14 +156,14 @@ func (m *mergeQuerier) LabelValues(ctx context.Context, name string, matchers .. } return m.mergeDistinctStringSliceWithTenants(ctx, func(ctx context.Context, q storage.Querier) ([]string, annotations.Annotations, error) { - return q.LabelValues(ctx, name, filteredMatchers...) + return q.LabelValues(ctx, name, hints, filteredMatchers...) }, matchedTenants, ids, queriers) } // LabelNames returns all the unique label names present in the underlying // queriers. It also adds the `idLabelName` and if present in the original // results the original `idLabelName`. -func (m *mergeQuerier) LabelNames(ctx context.Context, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { +func (m *mergeQuerier) LabelNames(ctx context.Context, hints *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { ids, queriers, err := m.callback(ctx, m.mint, m.maxt) if err != nil { return nil, nil, err @@ -171,7 +171,7 @@ func (m *mergeQuerier) LabelNames(ctx context.Context, matchers ...*labels.Match // by pass when only single querier is returned if m.byPassWithSingleQuerier && len(queriers) == 1 { - return queriers[0].LabelNames(ctx, matchers...) + return queriers[0].LabelNames(ctx, hints, matchers...) } log, _ := spanlogger.New(ctx, "mergeQuerier.LabelNames") defer log.Span.Finish() @@ -179,7 +179,7 @@ func (m *mergeQuerier) LabelNames(ctx context.Context, matchers ...*labels.Match matchedTenants, filteredMatchers := filterValuesByMatchers(m.idLabelName, ids, matchers...) labelNames, warnings, err := m.mergeDistinctStringSliceWithTenants(ctx, func(ctx context.Context, q storage.Querier) ([]string, annotations.Annotations, error) { - return q.LabelNames(ctx, filteredMatchers...) + return q.LabelNames(ctx, hints, filteredMatchers...) }, matchedTenants, ids, queriers) if err != nil { return nil, nil, err diff --git a/pkg/querier/tenantfederation/merge_queryable_test.go b/pkg/querier/tenantfederation/merge_queryable_test.go index f0b985a0a7..2c296673a9 100644 --- a/pkg/querier/tenantfederation/merge_queryable_test.go +++ b/pkg/querier/tenantfederation/merge_queryable_test.go @@ -185,7 +185,7 @@ func (m mockTenantQuerier) Select(ctx context.Context, _ bool, sp *storage.Selec // LabelValues implements the storage.LabelQuerier interface. // The mockTenantQuerier returns all a sorted slice of all label values and does not support reducing the result set with matchers. -func (m mockTenantQuerier) LabelValues(ctx context.Context, name string, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { +func (m mockTenantQuerier) LabelValues(ctx context.Context, name string, hints *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { tenantIDs, err := tenant.TenantIDs(ctx) if err != nil { return nil, nil, err @@ -233,7 +233,7 @@ func (m mockTenantQuerier) LabelValues(ctx context.Context, name string, matcher // It returns a sorted slice of all label names in the querier. // If only one matcher is provided with label Name=seriesWithLabelNames then the resulting set will have the values of that matchers pipe-split appended. // I.e. querying for {seriesWithLabelNames="foo|bar|baz"} will have as result [bar, baz, foo, ] -func (m mockTenantQuerier) LabelNames(ctx context.Context, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { +func (m mockTenantQuerier) LabelNames(ctx context.Context, hints *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { tenantIDs, err := tenant.TenantIDs(ctx) if err != nil { return nil, nil, err @@ -378,7 +378,7 @@ func TestMergeQueryable_Querier(t *testing.T) { querier, err := q.Querier(mint, maxt) require.NoError(t, err) - _, _, err = querier.LabelValues(context.Background(), "test") + _, _, err = querier.LabelValues(context.Background(), "test", nil) require.EqualError(t, err, user.ErrNoOrgID.Error()) }) } @@ -770,7 +770,7 @@ func TestMergeQueryable_LabelNames(t *testing.T) { t.Run(scenario.labelNamesTestCase.name, func(t *testing.T) { t.Parallel() - labelNames, warnings, err := querier.LabelNames(ctx, scenario.labelNamesTestCase.matchers...) + labelNames, warnings, err := querier.LabelNames(ctx, nil, scenario.labelNamesTestCase.matchers...) if scenario.labelNamesTestCase.expectedQueryErr != nil { require.EqualError(t, err, scenario.labelNamesTestCase.expectedQueryErr.Error()) } else { @@ -957,7 +957,7 @@ func TestMergeQueryable_LabelValues(t *testing.T) { tc := tc t.Run(tc.name, func(t *testing.T) { t.Parallel() - actLabelValues, warnings, err := querier.LabelValues(ctx, tc.labelName, tc.matchers...) + actLabelValues, warnings, err := querier.LabelValues(ctx, tc.labelName, nil, tc.matchers...) if tc.expectedQueryErr != nil { require.EqualError(t, err, tc.expectedQueryErr.Error()) } else { diff --git a/pkg/querier/tripperware/queryrange/split_by_interval_test.go b/pkg/querier/tripperware/queryrange/split_by_interval_test.go index 5004c60a80..e3feb76b26 100644 --- a/pkg/querier/tripperware/queryrange/split_by_interval_test.go +++ b/pkg/querier/tripperware/queryrange/split_by_interval_test.go @@ -378,11 +378,6 @@ func Test_evaluateAtModifier(t *testing.T) { [2m:]) [10m:])`, }, - { - // parse error: missing unit character in duration - in: "http_requests_total[5] @ 10.001", - expectedErrorCode: http.StatusBadRequest, - }, { // parse error: @ modifier must be preceded by an instant vector selector or range vector selector or a subquery in: "sum(http_requests_total[5m]) @ 10.001", diff --git a/pkg/ruler/ruler_test.go b/pkg/ruler/ruler_test.go index 7c8b7d9296..f66efede18 100644 --- a/pkg/ruler/ruler_test.go +++ b/pkg/ruler/ruler_test.go @@ -121,11 +121,11 @@ func newEmptyQueryable() storage.Queryable { type emptyQuerier struct { } -func (e emptyQuerier) LabelValues(ctx context.Context, name string, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { +func (e emptyQuerier) LabelValues(ctx context.Context, name string, _ *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { return nil, nil, nil } -func (e emptyQuerier) LabelNames(ctx context.Context, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { +func (e emptyQuerier) LabelNames(ctx context.Context, _ *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { return nil, nil, nil } diff --git a/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/CHANGELOG.md b/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/CHANGELOG.md index a6675492b1..af095f1da9 100644 --- a/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/CHANGELOG.md +++ b/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/CHANGELOG.md @@ -1,5 +1,20 @@ # Release History +## 1.12.0 (2024-06-06) + +### Features Added + +* Added field `StatusCodes` to `runtime.FetcherForNextLinkOptions` allowing for additional HTTP status codes indicating success. +* Added func `NewUUID` to the `runtime` package for generating UUIDs. + +### Bugs Fixed + +* Fixed an issue that prevented pollers using the `Operation-Location` strategy from unmarshaling the final result in some cases. + +### Other Changes + +* Updated dependencies. + ## 1.11.1 (2024-04-02) ### Bugs Fixed diff --git a/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/internal/pollers/async/async.go b/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/internal/pollers/async/async.go index ccd4794e9e..a534627605 100644 --- a/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/internal/pollers/async/async.go +++ b/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/internal/pollers/async/async.go @@ -155,5 +155,5 @@ func (p *Poller[T]) Result(ctx context.Context, out *T) error { p.resp = resp } - return pollers.ResultHelper(p.resp, poller.Failed(p.CurState), out) + return pollers.ResultHelper(p.resp, poller.Failed(p.CurState), "", out) } diff --git a/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/internal/pollers/body/body.go b/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/internal/pollers/body/body.go index 0d781b31d0..8751b05147 100644 --- a/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/internal/pollers/body/body.go +++ b/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/internal/pollers/body/body.go @@ -131,5 +131,5 @@ func (p *Poller[T]) Poll(ctx context.Context) (*http.Response, error) { } func (p *Poller[T]) Result(ctx context.Context, out *T) error { - return pollers.ResultHelper(p.resp, poller.Failed(p.CurState), out) + return pollers.ResultHelper(p.resp, poller.Failed(p.CurState), "", out) } diff --git a/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/internal/pollers/fake/fake.go b/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/internal/pollers/fake/fake.go index 51aede8a2b..7f8d11b8ba 100644 --- a/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/internal/pollers/fake/fake.go +++ b/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/internal/pollers/fake/fake.go @@ -124,7 +124,7 @@ func (p *Poller[T]) Result(ctx context.Context, out *T) error { return exported.NewResponseError(p.resp) } - return pollers.ResultHelper(p.resp, poller.Failed(p.FakeStatus), out) + return pollers.ResultHelper(p.resp, poller.Failed(p.FakeStatus), "", out) } // SanitizePollerPath removes any fake-appended suffix from a URL's path. diff --git a/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/internal/pollers/loc/loc.go b/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/internal/pollers/loc/loc.go index 7a56c5211b..048285275d 100644 --- a/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/internal/pollers/loc/loc.go +++ b/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/internal/pollers/loc/loc.go @@ -119,5 +119,5 @@ func (p *Poller[T]) Poll(ctx context.Context) (*http.Response, error) { } func (p *Poller[T]) Result(ctx context.Context, out *T) error { - return pollers.ResultHelper(p.resp, poller.Failed(p.CurState), out) + return pollers.ResultHelper(p.resp, poller.Failed(p.CurState), "", out) } diff --git a/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/internal/pollers/op/op.go b/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/internal/pollers/op/op.go index ac1c0efb5a..03699fd762 100644 --- a/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/internal/pollers/op/op.go +++ b/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/internal/pollers/op/op.go @@ -115,10 +115,13 @@ func (p *Poller[T]) Poll(ctx context.Context) (*http.Response, error) { func (p *Poller[T]) Result(ctx context.Context, out *T) error { var req *exported.Request var err error + + // when the payload is included with the status monitor on + // terminal success it's in the "result" JSON property + payloadPath := "result" + if p.FinalState == pollers.FinalStateViaLocation && p.LocURL != "" { req, err = exported.NewRequest(ctx, http.MethodGet, p.LocURL) - } else if p.FinalState == pollers.FinalStateViaOpLocation && p.Method == http.MethodPost { - // no final GET required, terminal response should have it } else if rl, rlErr := poller.GetResourceLocation(p.resp); rlErr != nil && !errors.Is(rlErr, poller.ErrNoBody) { return rlErr } else if rl != "" { @@ -134,6 +137,8 @@ func (p *Poller[T]) Result(ctx context.Context, out *T) error { // if a final GET request has been created, execute it if req != nil { + // no JSON path when making a final GET request + payloadPath = "" resp, err := p.pl.Do(req) if err != nil { return err @@ -141,5 +146,5 @@ func (p *Poller[T]) Result(ctx context.Context, out *T) error { p.resp = resp } - return pollers.ResultHelper(p.resp, poller.Failed(p.CurState), out) + return pollers.ResultHelper(p.resp, poller.Failed(p.CurState), payloadPath, out) } diff --git a/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/internal/pollers/util.go b/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/internal/pollers/util.go index eb3cf651db..6a7a32e034 100644 --- a/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/internal/pollers/util.go +++ b/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/internal/pollers/util.go @@ -159,7 +159,7 @@ func PollHelper(ctx context.Context, endpoint string, pl azexported.Pipeline, up // ResultHelper processes the response as success or failure. // In the success case, it unmarshals the payload into either a new instance of T or out. // In the failure case, it creates an *azcore.Response error from the response. -func ResultHelper[T any](resp *http.Response, failed bool, out *T) error { +func ResultHelper[T any](resp *http.Response, failed bool, jsonPath string, out *T) error { // short-circuit the simple success case with no response body to unmarshal if resp.StatusCode == http.StatusNoContent { return nil @@ -176,6 +176,18 @@ func ResultHelper[T any](resp *http.Response, failed bool, out *T) error { if err != nil { return err } + + if jsonPath != "" && len(payload) > 0 { + // extract the payload from the specified JSON path. + // do this before the zero-length check in case there + // is no payload. + jsonBody := map[string]json.RawMessage{} + if err = json.Unmarshal(payload, &jsonBody); err != nil { + return err + } + payload = jsonBody[jsonPath] + } + if len(payload) == 0 { return nil } diff --git a/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/internal/shared/constants.go b/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/internal/shared/constants.go index 03691cbf02..79651fd962 100644 --- a/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/internal/shared/constants.go +++ b/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/internal/shared/constants.go @@ -40,5 +40,5 @@ const ( Module = "azcore" // Version is the semantic version (see http://semver.org) of this module. - Version = "v1.11.1" + Version = "v1.12.0" ) diff --git a/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/runtime/pager.go b/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/runtime/pager.go index cffe692d7e..b960cff0b2 100644 --- a/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/runtime/pager.go +++ b/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/runtime/pager.go @@ -94,6 +94,10 @@ type FetcherForNextLinkOptions struct { // NextReq is the func to be called when requesting subsequent pages. // Used for paged operations that have a custom next link operation. NextReq func(context.Context, string) (*policy.Request, error) + + // StatusCodes contains additional HTTP status codes indicating success. + // The default value is http.StatusOK. + StatusCodes []int } // FetcherForNextLink is a helper containing boilerplate code to simplify creating a PagingHandler[T].Fetcher from a next link URL. @@ -105,10 +109,13 @@ type FetcherForNextLinkOptions struct { func FetcherForNextLink(ctx context.Context, pl Pipeline, nextLink string, firstReq func(context.Context) (*policy.Request, error), options *FetcherForNextLinkOptions) (*http.Response, error) { var req *policy.Request var err error + if options == nil { + options = &FetcherForNextLinkOptions{} + } if nextLink == "" { req, err = firstReq(ctx) } else if nextLink, err = EncodeQueryParams(nextLink); err == nil { - if options != nil && options.NextReq != nil { + if options.NextReq != nil { req, err = options.NextReq(ctx, nextLink) } else { req, err = NewRequest(ctx, http.MethodGet, nextLink) @@ -121,7 +128,9 @@ func FetcherForNextLink(ctx context.Context, pl Pipeline, nextLink string, first if err != nil { return nil, err } - if !HasStatusCode(resp, http.StatusOK) { + successCodes := []int{http.StatusOK} + successCodes = append(successCodes, options.StatusCodes...) + if !HasStatusCode(resp, successCodes...) { return nil, NewResponseError(resp) } return resp, nil diff --git a/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/runtime/request.go b/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/runtime/request.go index 06ac95b1b7..40ddc8d922 100644 --- a/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/runtime/request.go +++ b/vendor/github.com/Azure/azure-sdk-for-go/sdk/azcore/runtime/request.go @@ -24,6 +24,7 @@ import ( "github.com/Azure/azure-sdk-for-go/sdk/azcore/internal/shared" "github.com/Azure/azure-sdk-for-go/sdk/azcore/policy" "github.com/Azure/azure-sdk-for-go/sdk/azcore/streaming" + "github.com/Azure/azure-sdk-for-go/sdk/internal/uuid" ) // Base64Encoding is usesd to specify which base-64 encoder/decoder to use when @@ -263,3 +264,12 @@ func SkipBodyDownload(req *policy.Request) { // CtxAPINameKey is used as a context key for adding/retrieving the API name. type CtxAPINameKey = shared.CtxAPINameKey + +// NewUUID returns a new UUID using the RFC4122 algorithm. +func NewUUID() (string, error) { + u, err := uuid.New() + if err != nil { + return "", err + } + return u.String(), nil +} diff --git a/vendor/github.com/Azure/azure-sdk-for-go/sdk/azidentity/CHANGELOG.md b/vendor/github.com/Azure/azure-sdk-for-go/sdk/azidentity/CHANGELOG.md index 6d4b6feb86..a8c2feb6d4 100644 --- a/vendor/github.com/Azure/azure-sdk-for-go/sdk/azidentity/CHANGELOG.md +++ b/vendor/github.com/Azure/azure-sdk-for-go/sdk/azidentity/CHANGELOG.md @@ -1,5 +1,29 @@ # Release History +## 1.7.0 (2024-06-20) + +### Features Added +* `AzurePipelinesCredential` authenticates an Azure Pipelines service connection with + workload identity federation + +### Breaking Changes +> These changes affect only code written against a beta version such as v1.7.0-beta.1 +* Removed the persistent token caching API. It will return in v1.8.0-beta.1 + +## 1.7.0-beta.1 (2024-06-10) + +### Features Added +* Restored `AzurePipelinesCredential` and persistent token caching API + +## Breaking Changes +> These changes affect only code written against a beta version such as v1.6.0-beta.4 +* Values which `NewAzurePipelinesCredential` read from environment variables in + prior versions are now parameters +* Renamed `AzurePipelinesServiceConnectionCredentialOptions` to `AzurePipelinesCredentialOptions` + +### Bugs Fixed +* Managed identity bug fixes + ## 1.6.0 (2024-06-10) ### Features Added diff --git a/vendor/github.com/Azure/azure-sdk-for-go/sdk/azidentity/README.md b/vendor/github.com/Azure/azure-sdk-for-go/sdk/azidentity/README.md index b5acff0e63..7e201ea2fd 100644 --- a/vendor/github.com/Azure/azure-sdk-for-go/sdk/azidentity/README.md +++ b/vendor/github.com/Azure/azure-sdk-for-go/sdk/azidentity/README.md @@ -140,6 +140,7 @@ client := armresources.NewResourceGroupsClient("subscription ID", chain, nil) |Credential|Usage |-|- +|[AzurePipelinesCredential](https://pkg.go.dev/github.com/Azure/azure-sdk-for-go/sdk/azidentity#AzurePipelinesCredential)|Authenticate an Azure Pipelines [service connection](https://learn.microsoft.com/azure/devops/pipelines/library/service-endpoints?view=azure-devops&tabs=yaml) |[ClientAssertionCredential](https://pkg.go.dev/github.com/Azure/azure-sdk-for-go/sdk/azidentity#ClientAssertionCredential)|Authenticate a service principal with a signed client assertion |[ClientCertificateCredential](https://pkg.go.dev/github.com/Azure/azure-sdk-for-go/sdk/azidentity#ClientCertificateCredential)|Authenticate a service principal with a certificate |[ClientSecretCredential](https://pkg.go.dev/github.com/Azure/azure-sdk-for-go/sdk/azidentity#ClientSecretCredential)|Authenticate a service principal with a secret diff --git a/vendor/github.com/Azure/azure-sdk-for-go/sdk/azidentity/TOKEN_CACHING.MD b/vendor/github.com/Azure/azure-sdk-for-go/sdk/azidentity/TOKEN_CACHING.MD index f9cc489433..fbaa292204 100644 --- a/vendor/github.com/Azure/azure-sdk-for-go/sdk/azidentity/TOKEN_CACHING.MD +++ b/vendor/github.com/Azure/azure-sdk-for-go/sdk/azidentity/TOKEN_CACHING.MD @@ -57,6 +57,7 @@ The following table indicates the state of in-memory and persistent caching in e |--------------------------------|---------------------------------------------------------------------|--------------------------| | `AzureCLICredential` | Not Supported | Not Supported | | `AzureDeveloperCLICredential` | Not Supported | Not Supported | +| `AzurePipelinesCredential` | Supported | Supported | | `ClientAssertionCredential` | Supported | Supported | | `ClientCertificateCredential` | Supported | Supported | | `ClientSecretCredential` | Supported | Supported | diff --git a/vendor/github.com/Azure/azure-sdk-for-go/sdk/azidentity/TROUBLESHOOTING.md b/vendor/github.com/Azure/azure-sdk-for-go/sdk/azidentity/TROUBLESHOOTING.md index 3564e685e1..54016a0709 100644 --- a/vendor/github.com/Azure/azure-sdk-for-go/sdk/azidentity/TROUBLESHOOTING.md +++ b/vendor/github.com/Azure/azure-sdk-for-go/sdk/azidentity/TROUBLESHOOTING.md @@ -10,6 +10,7 @@ This troubleshooting guide covers failure investigation techniques, common error - [Enable and configure logging](#enable-and-configure-logging) - [Troubleshoot AzureCLICredential authentication issues](#troubleshoot-azureclicredential-authentication-issues) - [Troubleshoot AzureDeveloperCLICredential authentication issues](#troubleshoot-azuredeveloperclicredential-authentication-issues) +- [Troubleshoot AzurePipelinesCredential authentication issues](#troubleshoot-azurepipelinescredential-authentication-issues) - [Troubleshoot ClientCertificateCredential authentication issues](#troubleshoot-clientcertificatecredential-authentication-issues) - [Troubleshoot ClientSecretCredential authentication issues](#troubleshoot-clientsecretcredential-authentication-issues) - [Troubleshoot DefaultAzureCredential authentication issues](#troubleshoot-defaultazurecredential-authentication-issues) @@ -226,6 +227,15 @@ azd auth token --output json --scope https://management.core.windows.net/.defaul |---|---|---| |no client ID/tenant ID/token file specified|Incomplete configuration|In most cases these values are provided via environment variables set by Azure Workload Identity.
  • If your application runs on Azure Kubernetes Servide (AKS) or a cluster that has deployed the Azure Workload Identity admission webhook, check pod labels and service account configuration. See the [AKS documentation](https://learn.microsoft.com/azure/aks/workload-identity-deploy-cluster#disable-workload-identity) and [Azure Workload Identity troubleshooting guide](https://azure.github.io/azure-workload-identity/docs/troubleshooting.html) for more details.
  • If your application isn't running on AKS or your cluster hasn't deployed the Workload Identity admission webhook, set these values in `WorkloadIdentityCredentialOptions` + +## Troubleshoot AzurePipelinesCredential authentication issues + +| Error Message |Description| Mitigation | +|---|---|---| +| AADSTS900023: Specified tenant identifier 'some tenant ID' is neither a valid DNS name, nor a valid external domain.|The `tenantID` argument to `NewAzurePipelinesCredential` is incorrect| Verify the tenant ID. It must identify the tenant of the user-assigned managed identity or service principal configured for the service connection.| +| No service connection found with identifier |The `serviceConnectionID` argument to `NewAzurePipelinesCredential` is incorrect| Verify the service connection ID. This parameter refers to the `resourceId` of the Azure Service Connection. It can also be found in the query string of the service connection's configuration in Azure DevOps. [Azure Pipelines documentation](https://learn.microsoft.com/azure/devops/pipelines/library/service-endpoints?view=azure-devops&tabs=yaml) has more information about service connections.| +|302 (Found) response from OIDC endpoint|The `systemAccessToken` argument to `NewAzurePipelinesCredential` is incorrect|Check pipeline configuration. This value comes from the predefined variable `System.AccessToken` [as described in Azure Pipelines documentation](https://learn.microsoft.com/azure/devops/pipelines/build/variables?view=azure-devops&tabs=yaml#systemaccesstoken).| + ## Get additional help Additional information on ways to reach out for support can be found in [SUPPORT.md](https://github.com/Azure/azure-sdk-for-go/blob/main/SUPPORT.md). diff --git a/vendor/github.com/Azure/azure-sdk-for-go/sdk/azidentity/azure_pipelines_credential.go b/vendor/github.com/Azure/azure-sdk-for-go/sdk/azidentity/azure_pipelines_credential.go index 2655543aee..80c1806bb1 100644 --- a/vendor/github.com/Azure/azure-sdk-for-go/sdk/azidentity/azure_pipelines_credential.go +++ b/vendor/github.com/Azure/azure-sdk-for-go/sdk/azidentity/azure_pipelines_credential.go @@ -19,21 +19,20 @@ import ( const ( credNameAzurePipelines = "AzurePipelinesCredential" oidcAPIVersion = "7.1" - systemAccessToken = "SYSTEM_ACCESSTOKEN" systemOIDCRequestURI = "SYSTEM_OIDCREQUESTURI" ) -// azurePipelinesCredential authenticates with workload identity federation in an Azure Pipeline. See +// AzurePipelinesCredential authenticates with workload identity federation in an Azure Pipeline. See // [Azure Pipelines documentation] for more information. // // [Azure Pipelines documentation]: https://learn.microsoft.com/azure/devops/pipelines/library/connect-to-azure?view=azure-devops#create-an-azure-resource-manager-service-connection-that-uses-workload-identity-federation -type azurePipelinesCredential struct { +type AzurePipelinesCredential struct { connectionID, oidcURI, systemAccessToken string cred *ClientAssertionCredential } -// azurePipelinesCredentialOptions contains optional parameters for AzurePipelinesCredential. -type azurePipelinesCredentialOptions struct { +// AzurePipelinesCredentialOptions contains optional parameters for AzurePipelinesCredential. +type AzurePipelinesCredentialOptions struct { azcore.ClientOptions // AdditionallyAllowedTenants specifies additional tenants for which the credential may acquire tokens. @@ -48,28 +47,39 @@ type azurePipelinesCredentialOptions struct { DisableInstanceDiscovery bool } -// newAzurePipelinesCredential is the constructor for AzurePipelinesCredential. In addition to its required arguments, -// it reads a security token for the running build, which is required to authenticate the service connection, from the -// environment variable SYSTEM_ACCESSTOKEN. See the [Azure Pipelines documentation] for an example showing how to set -// this variable in build job YAML. +// NewAzurePipelinesCredential is the constructor for AzurePipelinesCredential. +// +// - tenantID: tenant ID of the service principal federated with the service connection +// - clientID: client ID of that service principal +// - serviceConnectionID: ID of the service connection to authenticate +// - systemAccessToken: security token for the running build. See [Azure Pipelines documentation] for +// an example showing how to get this value. // // [Azure Pipelines documentation]: https://learn.microsoft.com/azure/devops/pipelines/build/variables?view=azure-devops&tabs=yaml#systemaccesstoken -func newAzurePipelinesCredential(tenantID, clientID, serviceConnectionID string, options *azurePipelinesCredentialOptions) (*azurePipelinesCredential, error) { - if options == nil { - options = &azurePipelinesCredentialOptions{} +func NewAzurePipelinesCredential(tenantID, clientID, serviceConnectionID, systemAccessToken string, options *AzurePipelinesCredentialOptions) (*AzurePipelinesCredential, error) { + if !validTenantID(tenantID) { + return nil, errInvalidTenantID + } + if clientID == "" { + return nil, errors.New("no client ID specified") + } + if serviceConnectionID == "" { + return nil, errors.New("no service connection ID specified") + } + if systemAccessToken == "" { + return nil, errors.New("no system access token specified") } u := os.Getenv(systemOIDCRequestURI) if u == "" { return nil, fmt.Errorf("no value for environment variable %s. This should be set by Azure Pipelines", systemOIDCRequestURI) } - sat := os.Getenv(systemAccessToken) - if sat == "" { - return nil, errors.New("no value for environment variable " + systemAccessToken) - } - a := azurePipelinesCredential{ + a := AzurePipelinesCredential{ connectionID: serviceConnectionID, oidcURI: u, - systemAccessToken: sat, + systemAccessToken: systemAccessToken, + } + if options == nil { + options = &AzurePipelinesCredentialOptions{} } caco := ClientAssertionCredentialOptions{ AdditionallyAllowedTenants: options.AdditionallyAllowedTenants, @@ -86,7 +96,7 @@ func newAzurePipelinesCredential(tenantID, clientID, serviceConnectionID string, } // GetToken requests an access token from Microsoft Entra ID. Azure SDK clients call this method automatically. -func (a *azurePipelinesCredential) GetToken(ctx context.Context, opts policy.TokenRequestOptions) (azcore.AccessToken, error) { +func (a *AzurePipelinesCredential) GetToken(ctx context.Context, opts policy.TokenRequestOptions) (azcore.AccessToken, error) { var err error ctx, endSpan := runtime.StartSpan(ctx, credNameAzurePipelines+"."+traceOpGetToken, a.cred.client.azClient.Tracer(), nil) defer func() { endSpan(err) }() @@ -94,7 +104,7 @@ func (a *azurePipelinesCredential) GetToken(ctx context.Context, opts policy.Tok return tk, err } -func (a *azurePipelinesCredential) getAssertion(ctx context.Context) (string, error) { +func (a *AzurePipelinesCredential) getAssertion(ctx context.Context) (string, error) { url := a.oidcURI + "?api-version=" + oidcAPIVersion + "&serviceConnectionId=" + a.connectionID url, err := runtime.EncodeQueryParams(url) if err != nil { diff --git a/vendor/github.com/Azure/azure-sdk-for-go/sdk/azidentity/errors.go b/vendor/github.com/Azure/azure-sdk-for-go/sdk/azidentity/errors.go index 698650bbb6..35fa01d136 100644 --- a/vendor/github.com/Azure/azure-sdk-for-go/sdk/azidentity/errors.go +++ b/vendor/github.com/Azure/azure-sdk-for-go/sdk/azidentity/errors.go @@ -83,6 +83,8 @@ func (e *AuthenticationFailedError) Error() string { anchor = "azure-cli" case credNameAzureDeveloperCLI: anchor = "azd" + case credNameAzurePipelines: + anchor = "apc" case credNameCert: anchor = "client-cert" case credNameSecret: diff --git a/vendor/github.com/Azure/azure-sdk-for-go/sdk/azidentity/version.go b/vendor/github.com/Azure/azure-sdk-for-go/sdk/azidentity/version.go index 459ef64c6f..4305b5d3d8 100644 --- a/vendor/github.com/Azure/azure-sdk-for-go/sdk/azidentity/version.go +++ b/vendor/github.com/Azure/azure-sdk-for-go/sdk/azidentity/version.go @@ -14,5 +14,5 @@ const ( module = "github.com/Azure/azure-sdk-for-go/sdk/" + component // Version is the semantic version (see http://semver.org) of this module. - version = "v1.6.0" + version = "v1.7.0" ) diff --git a/vendor/github.com/prometheus/prometheus/config/config.go b/vendor/github.com/prometheus/prometheus/config/config.go index 9defa10d48..c924e30989 100644 --- a/vendor/github.com/prometheus/prometheus/config/config.go +++ b/vendor/github.com/prometheus/prometheus/config/config.go @@ -180,6 +180,7 @@ var ( // DefaultRemoteWriteConfig is the default remote write configuration. DefaultRemoteWriteConfig = RemoteWriteConfig{ RemoteTimeout: model.Duration(30 * time.Second), + ProtobufMessage: RemoteWriteProtoMsgV1, QueueConfig: DefaultQueueConfig, MetadataConfig: DefaultMetadataConfig, HTTPClientConfig: config.DefaultHTTPClientConfig, @@ -279,7 +280,7 @@ func (c *Config) GetScrapeConfigs() ([]*ScrapeConfig, error) { jobNames := map[string]string{} for i, scfg := range c.ScrapeConfigs { - // We do these checks for library users that would not call Validate in + // We do these checks for library users that would not call validate in // Unmarshal. if err := scfg.Validate(c.GlobalConfig); err != nil { return nil, err @@ -1055,6 +1056,49 @@ func CheckTargetAddress(address model.LabelValue) error { return nil } +// RemoteWriteProtoMsg represents the known protobuf message for the remote write +// 1.0 and 2.0 specs. +type RemoteWriteProtoMsg string + +// Validate returns error if the given reference for the protobuf message is not supported. +func (s RemoteWriteProtoMsg) Validate() error { + switch s { + case RemoteWriteProtoMsgV1, RemoteWriteProtoMsgV2: + return nil + default: + return fmt.Errorf("unknown remote write protobuf message %v, supported: %v", s, RemoteWriteProtoMsgs{RemoteWriteProtoMsgV1, RemoteWriteProtoMsgV2}.String()) + } +} + +type RemoteWriteProtoMsgs []RemoteWriteProtoMsg + +func (m RemoteWriteProtoMsgs) Strings() []string { + ret := make([]string, 0, len(m)) + for _, typ := range m { + ret = append(ret, string(typ)) + } + return ret +} + +func (m RemoteWriteProtoMsgs) String() string { + return strings.Join(m.Strings(), ", ") +} + +var ( + // RemoteWriteProtoMsgV1 represents the deprecated `prometheus.WriteRequest` protobuf + // message introduced in the https://prometheus.io/docs/specs/remote_write_spec/. + // + // NOTE: This string is used for both HTTP header values and config value, so don't change + // this reference. + RemoteWriteProtoMsgV1 RemoteWriteProtoMsg = "prometheus.WriteRequest" + // RemoteWriteProtoMsgV2 represents the `io.prometheus.write.v2.Request` protobuf + // message introduced in https://prometheus.io/docs/specs/remote_write_spec_2_0/ + // + // NOTE: This string is used for both HTTP header values and config value, so don't change + // this reference. + RemoteWriteProtoMsgV2 RemoteWriteProtoMsg = "io.prometheus.write.v2.Request" +) + // RemoteWriteConfig is the configuration for writing to remote storage. type RemoteWriteConfig struct { URL *config.URL `yaml:"url"` @@ -1064,6 +1108,9 @@ type RemoteWriteConfig struct { Name string `yaml:"name,omitempty"` SendExemplars bool `yaml:"send_exemplars,omitempty"` SendNativeHistograms bool `yaml:"send_native_histograms,omitempty"` + // ProtobufMessage specifies the protobuf message to use against the remote + // receiver as specified in https://prometheus.io/docs/specs/remote_write_spec_2_0/ + ProtobufMessage RemoteWriteProtoMsg `yaml:"protobuf_message,omitempty"` // We cannot do proper Go type embedding below as the parser will then parse // values arbitrarily into the overflow maps of further-down types. @@ -1098,6 +1145,10 @@ func (c *RemoteWriteConfig) UnmarshalYAML(unmarshal func(interface{}) error) err return err } + if err := c.ProtobufMessage.Validate(); err != nil { + return fmt.Errorf("invalid protobuf_message value: %w", err) + } + // The UnmarshalYAML method of HTTPClientConfig is not being called because it's not a pointer. // We cannot make it a pointer as the parser panics for inlined pointer structs. // Thus we just do its validation here. diff --git a/vendor/github.com/prometheus/prometheus/model/histogram/float_histogram.go b/vendor/github.com/prometheus/prometheus/model/histogram/float_histogram.go index 19a92b3d5a..2a37ea66d4 100644 --- a/vendor/github.com/prometheus/prometheus/model/histogram/float_histogram.go +++ b/vendor/github.com/prometheus/prometheus/model/histogram/float_histogram.go @@ -30,11 +30,12 @@ import ( type FloatHistogram struct { // Counter reset information. CounterResetHint CounterResetHint - // Currently valid schema numbers are -4 <= n <= 8. They are all for - // base-2 bucket schemas, where 1 is a bucket boundary in each case, and - // then each power of two is divided into 2^n logarithmic buckets. Or - // in other words, each bucket boundary is the previous boundary times - // 2^(2^-n). + // Currently valid schema numbers are -4 <= n <= 8 for exponential buckets. + // They are all for base-2 bucket schemas, where 1 is a bucket boundary in + // each case, and then each power of two is divided into 2^n logarithmic buckets. + // Or in other words, each bucket boundary is the previous boundary times + // 2^(2^-n). Another valid schema number is -53 for custom buckets, defined by + // the CustomValues field. Schema int32 // Width of the zero bucket. ZeroThreshold float64 @@ -49,6 +50,16 @@ type FloatHistogram struct { // Observation counts in buckets. Each represents an absolute count and // must be zero or positive. PositiveBuckets, NegativeBuckets []float64 + // Holds the custom (usually upper) bounds for bucket definitions, otherwise nil. + // This slice is interned, to be treated as immutable and copied by reference. + // These numbers should be strictly increasing. This field is only used when the + // schema is for custom buckets, and the ZeroThreshold, ZeroCount, NegativeSpans + // and NegativeBuckets fields are not used in that case. + CustomValues []float64 +} + +func (h *FloatHistogram) UsesCustomBuckets() bool { + return IsCustomBucketsSchema(h.Schema) } // Copy returns a deep copy of the Histogram. @@ -56,28 +67,37 @@ func (h *FloatHistogram) Copy() *FloatHistogram { c := FloatHistogram{ CounterResetHint: h.CounterResetHint, Schema: h.Schema, - ZeroThreshold: h.ZeroThreshold, - ZeroCount: h.ZeroCount, Count: h.Count, Sum: h.Sum, } + if h.UsesCustomBuckets() { + if len(h.CustomValues) != 0 { + c.CustomValues = make([]float64, len(h.CustomValues)) + copy(c.CustomValues, h.CustomValues) + } + } else { + c.ZeroThreshold = h.ZeroThreshold + c.ZeroCount = h.ZeroCount + + if len(h.NegativeSpans) != 0 { + c.NegativeSpans = make([]Span, len(h.NegativeSpans)) + copy(c.NegativeSpans, h.NegativeSpans) + } + if len(h.NegativeBuckets) != 0 { + c.NegativeBuckets = make([]float64, len(h.NegativeBuckets)) + copy(c.NegativeBuckets, h.NegativeBuckets) + } + } + if len(h.PositiveSpans) != 0 { c.PositiveSpans = make([]Span, len(h.PositiveSpans)) copy(c.PositiveSpans, h.PositiveSpans) } - if len(h.NegativeSpans) != 0 { - c.NegativeSpans = make([]Span, len(h.NegativeSpans)) - copy(c.NegativeSpans, h.NegativeSpans) - } if len(h.PositiveBuckets) != 0 { c.PositiveBuckets = make([]float64, len(h.PositiveBuckets)) copy(c.PositiveBuckets, h.PositiveBuckets) } - if len(h.NegativeBuckets) != 0 { - c.NegativeBuckets = make([]float64, len(h.NegativeBuckets)) - copy(c.NegativeBuckets, h.NegativeBuckets) - } return &c } @@ -87,32 +107,53 @@ func (h *FloatHistogram) Copy() *FloatHistogram { func (h *FloatHistogram) CopyTo(to *FloatHistogram) { to.CounterResetHint = h.CounterResetHint to.Schema = h.Schema - to.ZeroThreshold = h.ZeroThreshold - to.ZeroCount = h.ZeroCount to.Count = h.Count to.Sum = h.Sum + if h.UsesCustomBuckets() { + to.ZeroThreshold = 0 + to.ZeroCount = 0 + + to.NegativeSpans = clearIfNotNil(to.NegativeSpans) + to.NegativeBuckets = clearIfNotNil(to.NegativeBuckets) + + to.CustomValues = resize(to.CustomValues, len(h.CustomValues)) + copy(to.CustomValues, h.CustomValues) + } else { + to.ZeroThreshold = h.ZeroThreshold + to.ZeroCount = h.ZeroCount + + to.NegativeSpans = resize(to.NegativeSpans, len(h.NegativeSpans)) + copy(to.NegativeSpans, h.NegativeSpans) + + to.NegativeBuckets = resize(to.NegativeBuckets, len(h.NegativeBuckets)) + copy(to.NegativeBuckets, h.NegativeBuckets) + + to.CustomValues = clearIfNotNil(to.CustomValues) + } + to.PositiveSpans = resize(to.PositiveSpans, len(h.PositiveSpans)) copy(to.PositiveSpans, h.PositiveSpans) - to.NegativeSpans = resize(to.NegativeSpans, len(h.NegativeSpans)) - copy(to.NegativeSpans, h.NegativeSpans) - to.PositiveBuckets = resize(to.PositiveBuckets, len(h.PositiveBuckets)) copy(to.PositiveBuckets, h.PositiveBuckets) - - to.NegativeBuckets = resize(to.NegativeBuckets, len(h.NegativeBuckets)) - copy(to.NegativeBuckets, h.NegativeBuckets) } // CopyToSchema works like Copy, but the returned deep copy has the provided // target schema, which must be ≤ the original schema (i.e. it must have a lower -// resolution). +// resolution). This method panics if a custom buckets schema is used in the +// receiving FloatHistogram or as the provided targetSchema. func (h *FloatHistogram) CopyToSchema(targetSchema int32) *FloatHistogram { if targetSchema == h.Schema { // Fast path. return h.Copy() } + if h.UsesCustomBuckets() { + panic(fmt.Errorf("cannot reduce resolution to %d when there are custom buckets", targetSchema)) + } + if IsCustomBucketsSchema(targetSchema) { + panic("cannot reduce resolution to custom buckets schema") + } if targetSchema > h.Schema { panic(fmt.Errorf("cannot copy from schema %d to %d", h.Schema, targetSchema)) } @@ -185,6 +226,9 @@ func (h *FloatHistogram) TestExpression() string { if m.ZeroThreshold != 0 { res = append(res, fmt.Sprintf("z_bucket_w:%g", m.ZeroThreshold)) } + if m.UsesCustomBuckets() { + res = append(res, fmt.Sprintf("custom_values:%g", m.CustomValues)) + } addBuckets := func(kind, bucketsKey, offsetKey string, buckets []float64, spans []Span) []string { if len(spans) > 1 { @@ -210,14 +254,18 @@ func (h *FloatHistogram) TestExpression() string { return "{{" + strings.Join(res, " ") + "}}" } -// ZeroBucket returns the zero bucket. +// ZeroBucket returns the zero bucket. This method panics if the schema is for custom buckets. func (h *FloatHistogram) ZeroBucket() Bucket[float64] { + if h.UsesCustomBuckets() { + panic("histograms with custom buckets have no zero bucket") + } return Bucket[float64]{ Lower: -h.ZeroThreshold, Upper: h.ZeroThreshold, LowerInclusive: true, UpperInclusive: true, Count: h.ZeroCount, + // Index is irrelevant for the zero bucket. } } @@ -263,9 +311,18 @@ func (h *FloatHistogram) Div(scalar float64) *FloatHistogram { // // The method reconciles differences in the zero threshold and in the schema, and // changes them if needed. The other histogram will not be modified in any case. +// Adding is currently only supported between 2 exponential histograms, or between +// 2 custom buckets histograms with the exact same custom bounds. // // This method returns a pointer to the receiving histogram for convenience. -func (h *FloatHistogram) Add(other *FloatHistogram) *FloatHistogram { +func (h *FloatHistogram) Add(other *FloatHistogram) (*FloatHistogram, error) { + if h.UsesCustomBuckets() != other.UsesCustomBuckets() { + return nil, ErrHistogramsIncompatibleSchema + } + if h.UsesCustomBuckets() && !FloatBucketsMatch(h.CustomValues, other.CustomValues) { + return nil, ErrHistogramsIncompatibleBounds + } + switch { case other.CounterResetHint == h.CounterResetHint: // Adding apples to apples, all good. No need to change anything. @@ -290,19 +347,28 @@ func (h *FloatHistogram) Add(other *FloatHistogram) *FloatHistogram { // TODO(trevorwhitney): Actually issue the warning as soon as the plumbing for it is in place } - otherZeroCount := h.reconcileZeroBuckets(other) - h.ZeroCount += otherZeroCount + if !h.UsesCustomBuckets() { + otherZeroCount := h.reconcileZeroBuckets(other) + h.ZeroCount += otherZeroCount + } h.Count += other.Count h.Sum += other.Sum var ( - hPositiveSpans = h.PositiveSpans - hPositiveBuckets = h.PositiveBuckets - hNegativeSpans = h.NegativeSpans - hNegativeBuckets = h.NegativeBuckets - + hPositiveSpans = h.PositiveSpans + hPositiveBuckets = h.PositiveBuckets otherPositiveSpans = other.PositiveSpans otherPositiveBuckets = other.PositiveBuckets + ) + + if h.UsesCustomBuckets() { + h.PositiveSpans, h.PositiveBuckets = addBuckets(h.Schema, h.ZeroThreshold, false, hPositiveSpans, hPositiveBuckets, otherPositiveSpans, otherPositiveBuckets) + return h, nil + } + + var ( + hNegativeSpans = h.NegativeSpans + hNegativeBuckets = h.NegativeBuckets otherNegativeSpans = other.NegativeSpans otherNegativeBuckets = other.NegativeBuckets ) @@ -321,24 +387,40 @@ func (h *FloatHistogram) Add(other *FloatHistogram) *FloatHistogram { h.PositiveSpans, h.PositiveBuckets = addBuckets(h.Schema, h.ZeroThreshold, false, hPositiveSpans, hPositiveBuckets, otherPositiveSpans, otherPositiveBuckets) h.NegativeSpans, h.NegativeBuckets = addBuckets(h.Schema, h.ZeroThreshold, false, hNegativeSpans, hNegativeBuckets, otherNegativeSpans, otherNegativeBuckets) - return h + return h, nil } // Sub works like Add but subtracts the other histogram. -func (h *FloatHistogram) Sub(other *FloatHistogram) *FloatHistogram { - otherZeroCount := h.reconcileZeroBuckets(other) - h.ZeroCount -= otherZeroCount +func (h *FloatHistogram) Sub(other *FloatHistogram) (*FloatHistogram, error) { + if h.UsesCustomBuckets() != other.UsesCustomBuckets() { + return nil, ErrHistogramsIncompatibleSchema + } + if h.UsesCustomBuckets() && !FloatBucketsMatch(h.CustomValues, other.CustomValues) { + return nil, ErrHistogramsIncompatibleBounds + } + + if !h.UsesCustomBuckets() { + otherZeroCount := h.reconcileZeroBuckets(other) + h.ZeroCount -= otherZeroCount + } h.Count -= other.Count h.Sum -= other.Sum var ( - hPositiveSpans = h.PositiveSpans - hPositiveBuckets = h.PositiveBuckets - hNegativeSpans = h.NegativeSpans - hNegativeBuckets = h.NegativeBuckets - + hPositiveSpans = h.PositiveSpans + hPositiveBuckets = h.PositiveBuckets otherPositiveSpans = other.PositiveSpans otherPositiveBuckets = other.PositiveBuckets + ) + + if h.UsesCustomBuckets() { + h.PositiveSpans, h.PositiveBuckets = addBuckets(h.Schema, h.ZeroThreshold, true, hPositiveSpans, hPositiveBuckets, otherPositiveSpans, otherPositiveBuckets) + return h, nil + } + + var ( + hNegativeSpans = h.NegativeSpans + hNegativeBuckets = h.NegativeBuckets otherNegativeSpans = other.NegativeSpans otherNegativeBuckets = other.NegativeBuckets ) @@ -356,7 +438,7 @@ func (h *FloatHistogram) Sub(other *FloatHistogram) *FloatHistogram { h.PositiveSpans, h.PositiveBuckets = addBuckets(h.Schema, h.ZeroThreshold, true, hPositiveSpans, hPositiveBuckets, otherPositiveSpans, otherPositiveBuckets) h.NegativeSpans, h.NegativeBuckets = addBuckets(h.Schema, h.ZeroThreshold, true, hNegativeSpans, hNegativeBuckets, otherNegativeSpans, otherNegativeBuckets) - return h + return h, nil } // Equals returns true if the given float histogram matches exactly. @@ -365,29 +447,42 @@ func (h *FloatHistogram) Sub(other *FloatHistogram) *FloatHistogram { // but they must represent the same bucket layout to match. // Sum, Count, ZeroCount and bucket values are compared based on their bit patterns // because this method is about data equality rather than mathematical equality. +// We ignore fields that are not used based on the exponential / custom buckets schema, +// but check fields where differences may cause unintended behaviour even if they are not +// supposed to be used according to the schema. func (h *FloatHistogram) Equals(h2 *FloatHistogram) bool { if h2 == nil { return false } - if h.Schema != h2.Schema || h.ZeroThreshold != h2.ZeroThreshold || - math.Float64bits(h.ZeroCount) != math.Float64bits(h2.ZeroCount) || + if h.Schema != h2.Schema || math.Float64bits(h.Count) != math.Float64bits(h2.Count) || math.Float64bits(h.Sum) != math.Float64bits(h2.Sum) { return false } - if !spansMatch(h.PositiveSpans, h2.PositiveSpans) { + if h.UsesCustomBuckets() { + if !FloatBucketsMatch(h.CustomValues, h2.CustomValues) { + return false + } + } + + if h.ZeroThreshold != h2.ZeroThreshold || + math.Float64bits(h.ZeroCount) != math.Float64bits(h2.ZeroCount) { return false } + if !spansMatch(h.NegativeSpans, h2.NegativeSpans) { return false } + if !FloatBucketsMatch(h.NegativeBuckets, h2.NegativeBuckets) { + return false + } - if !floatBucketsMatch(h.PositiveBuckets, h2.PositiveBuckets) { + if !spansMatch(h.PositiveSpans, h2.PositiveSpans) { return false } - if !floatBucketsMatch(h.NegativeBuckets, h2.NegativeBuckets) { + if !FloatBucketsMatch(h.PositiveBuckets, h2.PositiveBuckets) { return false } @@ -403,6 +498,7 @@ func (h *FloatHistogram) Size() int { negSpanSize := len(h.NegativeSpans) * 8 // 8 bytes (int32 + uint32). posBucketSize := len(h.PositiveBuckets) * 8 // 8 bytes (float64). negBucketSize := len(h.NegativeBuckets) * 8 // 8 bytes (float64). + customBoundSize := len(h.CustomValues) * 8 // 8 bytes (float64). // Total size of the struct. @@ -417,9 +513,10 @@ func (h *FloatHistogram) Size() int { // fh.NegativeSpans is 24 bytes. // fh.PositiveBuckets is 24 bytes. // fh.NegativeBuckets is 24 bytes. - structSize := 144 + // fh.CustomValues is 24 bytes. + structSize := 168 - return structSize + posSpanSize + negSpanSize + posBucketSize + negBucketSize + return structSize + posSpanSize + negSpanSize + posBucketSize + negBucketSize + customBoundSize } // Compact eliminates empty buckets at the beginning and end of each span, then @@ -504,6 +601,12 @@ func (h *FloatHistogram) DetectReset(previous *FloatHistogram) bool { if h.Count < previous.Count { return true } + if h.UsesCustomBuckets() != previous.UsesCustomBuckets() || (h.UsesCustomBuckets() && !FloatBucketsMatch(h.CustomValues, previous.CustomValues)) { + // Mark that something has changed or that the application has been restarted. However, this does + // not matter so much since the change in schema will be handled directly in the chunks and PromQL + // functions. + return true + } if h.Schema > previous.Schema { return true } @@ -609,7 +712,7 @@ func (h *FloatHistogram) NegativeBucketIterator() BucketIterator[float64] { // positive buckets in descending order (starting at the highest bucket and // going down towards the zero bucket). func (h *FloatHistogram) PositiveReverseBucketIterator() BucketIterator[float64] { - it := newReverseFloatBucketIterator(h.PositiveSpans, h.PositiveBuckets, h.Schema, true) + it := newReverseFloatBucketIterator(h.PositiveSpans, h.PositiveBuckets, h.Schema, true, h.CustomValues) return &it } @@ -617,7 +720,7 @@ func (h *FloatHistogram) PositiveReverseBucketIterator() BucketIterator[float64] // negative buckets in ascending order (starting at the lowest bucket and going // up towards the zero bucket). func (h *FloatHistogram) NegativeReverseBucketIterator() BucketIterator[float64] { - it := newReverseFloatBucketIterator(h.NegativeSpans, h.NegativeBuckets, h.Schema, false) + it := newReverseFloatBucketIterator(h.NegativeSpans, h.NegativeBuckets, h.Schema, false, nil) return &it } @@ -629,7 +732,7 @@ func (h *FloatHistogram) NegativeReverseBucketIterator() BucketIterator[float64] func (h *FloatHistogram) AllBucketIterator() BucketIterator[float64] { return &allFloatBucketIterator{ h: h, - leftIter: newReverseFloatBucketIterator(h.NegativeSpans, h.NegativeBuckets, h.Schema, false), + leftIter: newReverseFloatBucketIterator(h.NegativeSpans, h.NegativeBuckets, h.Schema, false, nil), rightIter: h.floatBucketIterator(true, 0, h.Schema), state: -1, } @@ -643,30 +746,52 @@ func (h *FloatHistogram) AllBucketIterator() BucketIterator[float64] { func (h *FloatHistogram) AllReverseBucketIterator() BucketIterator[float64] { return &allFloatBucketIterator{ h: h, - leftIter: newReverseFloatBucketIterator(h.PositiveSpans, h.PositiveBuckets, h.Schema, true), + leftIter: newReverseFloatBucketIterator(h.PositiveSpans, h.PositiveBuckets, h.Schema, true, h.CustomValues), rightIter: h.floatBucketIterator(false, 0, h.Schema), state: -1, } } // Validate validates consistency between span and bucket slices. Also, buckets are checked -// against negative values. +// against negative values. We check to make sure there are no unexpected fields or field values +// based on the exponential / custom buckets schema. // We do not check for h.Count being at least as large as the sum of the // counts in the buckets because floating point precision issues can // create false positives here. func (h *FloatHistogram) Validate() error { - if err := checkHistogramSpans(h.NegativeSpans, len(h.NegativeBuckets)); err != nil { - return fmt.Errorf("negative side: %w", err) - } - if err := checkHistogramSpans(h.PositiveSpans, len(h.PositiveBuckets)); err != nil { - return fmt.Errorf("positive side: %w", err) - } var nCount, pCount float64 - err := checkHistogramBuckets(h.NegativeBuckets, &nCount, false) - if err != nil { - return fmt.Errorf("negative side: %w", err) + if h.UsesCustomBuckets() { + if err := checkHistogramCustomBounds(h.CustomValues, h.PositiveSpans, len(h.PositiveBuckets)); err != nil { + return fmt.Errorf("custom buckets: %w", err) + } + if h.ZeroCount != 0 { + return fmt.Errorf("custom buckets: must have zero count of 0") + } + if h.ZeroThreshold != 0 { + return fmt.Errorf("custom buckets: must have zero threshold of 0") + } + if len(h.NegativeSpans) > 0 { + return fmt.Errorf("custom buckets: must not have negative spans") + } + if len(h.NegativeBuckets) > 0 { + return fmt.Errorf("custom buckets: must not have negative buckets") + } + } else { + if err := checkHistogramSpans(h.PositiveSpans, len(h.PositiveBuckets)); err != nil { + return fmt.Errorf("positive side: %w", err) + } + if err := checkHistogramSpans(h.NegativeSpans, len(h.NegativeBuckets)); err != nil { + return fmt.Errorf("negative side: %w", err) + } + err := checkHistogramBuckets(h.NegativeBuckets, &nCount, false) + if err != nil { + return fmt.Errorf("negative side: %w", err) + } + if h.CustomValues != nil { + return fmt.Errorf("histogram with exponential schema must not have custom bounds") + } } - err = checkHistogramBuckets(h.PositiveBuckets, &pCount, false) + err := checkHistogramBuckets(h.PositiveBuckets, &pCount, false) if err != nil { return fmt.Errorf("positive side: %w", err) } @@ -790,17 +915,25 @@ func (h *FloatHistogram) reconcileZeroBuckets(other *FloatHistogram) float64 { // If positive is true, the returned iterator iterates through the positive // buckets, otherwise through the negative buckets. // -// If absoluteStartValue is < the lowest absolute value of any upper bucket -// boundary, the iterator starts with the first bucket. Otherwise, it will skip -// all buckets with an absolute value of their upper boundary ≤ -// absoluteStartValue. +// Only for exponential schemas, if absoluteStartValue is < the lowest absolute +// value of any upper bucket boundary, the iterator starts with the first bucket. +// Otherwise, it will skip all buckets with an absolute value of their upper boundary ≤ +// absoluteStartValue. For custom bucket schemas, absoluteStartValue is ignored and +// no buckets are skipped. // // targetSchema must be ≤ the schema of FloatHistogram (and of course within the // legal values for schemas in general). The buckets are merged to match the -// targetSchema prior to iterating (without mutating FloatHistogram). +// targetSchema prior to iterating (without mutating FloatHistogram), but custom buckets +// schemas cannot be merged with other schemas. func (h *FloatHistogram) floatBucketIterator( positive bool, absoluteStartValue float64, targetSchema int32, ) floatBucketIterator { + if h.UsesCustomBuckets() && targetSchema != h.Schema { + panic(fmt.Errorf("cannot merge from custom buckets schema to exponential schema")) + } + if !h.UsesCustomBuckets() && IsCustomBucketsSchema(targetSchema) { + panic(fmt.Errorf("cannot merge from exponential buckets schema to custom schema")) + } if targetSchema > h.Schema { panic(fmt.Errorf("cannot merge from schema %d to %d", h.Schema, targetSchema)) } @@ -816,6 +949,7 @@ func (h *FloatHistogram) floatBucketIterator( if positive { i.spans = h.PositiveSpans i.buckets = h.PositiveBuckets + i.customValues = h.CustomValues } else { i.spans = h.NegativeSpans i.buckets = h.NegativeBuckets @@ -825,14 +959,15 @@ func (h *FloatHistogram) floatBucketIterator( // reverseFloatBucketIterator is a low-level constructor for reverse bucket iterators. func newReverseFloatBucketIterator( - spans []Span, buckets []float64, schema int32, positive bool, + spans []Span, buckets []float64, schema int32, positive bool, customValues []float64, ) reverseFloatBucketIterator { r := reverseFloatBucketIterator{ baseBucketIterator: baseBucketIterator[float64, float64]{ - schema: schema, - spans: spans, - buckets: buckets, - positive: positive, + schema: schema, + spans: spans, + buckets: buckets, + positive: positive, + customValues: customValues, }, } @@ -946,9 +1081,9 @@ func (i *floatBucketIterator) Next() bool { } } - // Skip buckets before absoluteStartValue. + // Skip buckets before absoluteStartValue for exponential schemas. // TODO(beorn7): Maybe do something more efficient than this recursive call. - if !i.boundReachedStartValue && getBound(i.currIdx, i.targetSchema) <= i.absoluteStartValue { + if !i.boundReachedStartValue && IsExponentialSchema(i.targetSchema) && getBoundExponential(i.currIdx, i.targetSchema) <= i.absoluteStartValue { return i.Next() } i.boundReachedStartValue = true @@ -1010,14 +1145,7 @@ func (i *allFloatBucketIterator) Next() bool { case 0: i.state = 1 if i.h.ZeroCount > 0 { - i.currBucket = Bucket[float64]{ - Lower: -i.h.ZeroThreshold, - Upper: i.h.ZeroThreshold, - LowerInclusive: true, - UpperInclusive: true, - Count: i.h.ZeroCount, - // Index is irrelevant for the zero bucket. - } + i.currBucket = i.h.ZeroBucket() return true } return i.Next() @@ -1076,7 +1204,7 @@ func addBuckets( for _, spanB := range spansB { indexB += spanB.Offset for j := 0; j < int(spanB.Length); j++ { - if lowerThanThreshold && getBound(indexB, schema) <= threshold { + if lowerThanThreshold && IsExponentialSchema(schema) && getBoundExponential(indexB, schema) <= threshold { goto nextLoop } lowerThanThreshold = false @@ -1177,7 +1305,7 @@ func addBuckets( return spansA, bucketsA } -func floatBucketsMatch(b1, b2 []float64) bool { +func FloatBucketsMatch(b1, b2 []float64) bool { if len(b1) != len(b2) { return false } @@ -1191,7 +1319,15 @@ func floatBucketsMatch(b1, b2 []float64) bool { // ReduceResolution reduces the float histogram's spans, buckets into target schema. // The target schema must be smaller than the current float histogram's schema. +// This will panic if the histogram has custom buckets or if the target schema is +// a custom buckets schema. func (h *FloatHistogram) ReduceResolution(targetSchema int32) *FloatHistogram { + if h.UsesCustomBuckets() { + panic("cannot reduce resolution when there are custom buckets") + } + if IsCustomBucketsSchema(targetSchema) { + panic("cannot reduce resolution to custom buckets schema") + } if targetSchema >= h.Schema { panic(fmt.Errorf("cannot reduce resolution from schema %d to %d", h.Schema, targetSchema)) } diff --git a/vendor/github.com/prometheus/prometheus/model/histogram/generic.go b/vendor/github.com/prometheus/prometheus/model/histogram/generic.go index 67abe7b0aa..a36b58d069 100644 --- a/vendor/github.com/prometheus/prometheus/model/histogram/generic.go +++ b/vendor/github.com/prometheus/prometheus/model/histogram/generic.go @@ -20,14 +20,33 @@ import ( "strings" ) +const ( + ExponentialSchemaMax int32 = 8 + ExponentialSchemaMin int32 = -4 + CustomBucketsSchema int32 = -53 +) + var ( - ErrHistogramCountNotBigEnough = errors.New("histogram's observation count should be at least the number of observations found in the buckets") - ErrHistogramCountMismatch = errors.New("histogram's observation count should equal the number of observations found in the buckets (in absence of NaN)") - ErrHistogramNegativeBucketCount = errors.New("histogram has a bucket whose observation count is negative") - ErrHistogramSpanNegativeOffset = errors.New("histogram has a span whose offset is negative") - ErrHistogramSpansBucketsMismatch = errors.New("histogram spans specify different number of buckets than provided") + ErrHistogramCountNotBigEnough = errors.New("histogram's observation count should be at least the number of observations found in the buckets") + ErrHistogramCountMismatch = errors.New("histogram's observation count should equal the number of observations found in the buckets (in absence of NaN)") + ErrHistogramNegativeBucketCount = errors.New("histogram has a bucket whose observation count is negative") + ErrHistogramSpanNegativeOffset = errors.New("histogram has a span whose offset is negative") + ErrHistogramSpansBucketsMismatch = errors.New("histogram spans specify different number of buckets than provided") + ErrHistogramCustomBucketsMismatch = errors.New("histogram custom bounds are too few") + ErrHistogramCustomBucketsInvalid = errors.New("histogram custom bounds must be in strictly increasing order") + ErrHistogramCustomBucketsInfinite = errors.New("histogram custom bounds must be finite") + ErrHistogramsIncompatibleSchema = errors.New("cannot apply this operation on histograms with a mix of exponential and custom bucket schemas") + ErrHistogramsIncompatibleBounds = errors.New("cannot apply this operation on custom buckets histograms with different custom bounds") ) +func IsCustomBucketsSchema(s int32) bool { + return s == CustomBucketsSchema +} + +func IsExponentialSchema(s int32) bool { + return s >= ExponentialSchemaMin && s <= ExponentialSchemaMax +} + // BucketCount is a type constraint for the count in a bucket, which can be // float64 (for type FloatHistogram) or uint64 (for type Histogram). type BucketCount interface { @@ -115,6 +134,8 @@ type baseBucketIterator[BC BucketCount, IBC InternalBucketCount] struct { currCount IBC // Count in the current bucket. currIdx int32 // The actual bucket index. + + customValues []float64 // Bounds (usually upper) for histograms with custom buckets. } func (b *baseBucketIterator[BC, IBC]) At() Bucket[BC] { @@ -128,14 +149,19 @@ func (b *baseBucketIterator[BC, IBC]) at(schema int32) Bucket[BC] { Index: b.currIdx, } if b.positive { - bucket.Upper = getBound(b.currIdx, schema) - bucket.Lower = getBound(b.currIdx-1, schema) + bucket.Upper = getBound(b.currIdx, schema, b.customValues) + bucket.Lower = getBound(b.currIdx-1, schema, b.customValues) + } else { + bucket.Lower = -getBound(b.currIdx, schema, b.customValues) + bucket.Upper = -getBound(b.currIdx-1, schema, b.customValues) + } + if IsCustomBucketsSchema(schema) { + bucket.LowerInclusive = b.currIdx == 0 + bucket.UpperInclusive = true } else { - bucket.Lower = -getBound(b.currIdx, schema) - bucket.Upper = -getBound(b.currIdx-1, schema) + bucket.LowerInclusive = bucket.Lower < 0 + bucket.UpperInclusive = bucket.Upper > 0 } - bucket.LowerInclusive = bucket.Lower < 0 - bucket.UpperInclusive = bucket.Upper > 0 return bucket } @@ -393,7 +419,55 @@ func checkHistogramBuckets[BC BucketCount, IBC InternalBucketCount](buckets []IB return nil } -func getBound(idx, schema int32) float64 { +func checkHistogramCustomBounds(bounds []float64, spans []Span, numBuckets int) error { + prev := math.Inf(-1) + for _, curr := range bounds { + if curr <= prev { + return fmt.Errorf("previous bound is %f and current is %f: %w", prev, curr, ErrHistogramCustomBucketsInvalid) + } + prev = curr + } + if prev == math.Inf(1) { + return fmt.Errorf("last +Inf bound must not be explicitly defined: %w", ErrHistogramCustomBucketsInfinite) + } + + var spanBuckets int + var totalSpanLength int + for n, span := range spans { + if span.Offset < 0 { + return fmt.Errorf("span number %d with offset %d: %w", n+1, span.Offset, ErrHistogramSpanNegativeOffset) + } + spanBuckets += int(span.Length) + totalSpanLength += int(span.Length) + int(span.Offset) + } + if spanBuckets != numBuckets { + return fmt.Errorf("spans need %d buckets, have %d buckets: %w", spanBuckets, numBuckets, ErrHistogramSpansBucketsMismatch) + } + if (len(bounds) + 1) < totalSpanLength { + return fmt.Errorf("only %d custom bounds defined which is insufficient to cover total span length of %d: %w", len(bounds), totalSpanLength, ErrHistogramCustomBucketsMismatch) + } + + return nil +} + +func getBound(idx, schema int32, customValues []float64) float64 { + if IsCustomBucketsSchema(schema) { + length := int32(len(customValues)) + switch { + case idx > length || idx < -1: + panic(fmt.Errorf("index %d out of bounds for custom bounds of length %d", idx, length)) + case idx == length: + return math.Inf(1) + case idx == -1: + return math.Inf(-1) + default: + return customValues[idx] + } + } + return getBoundExponential(idx, schema) +} + +func getBoundExponential(idx, schema int32) float64 { // Here a bit of context about the behavior for the last bucket counting // regular numbers (called simply "last bucket" below) and the bucket // counting observations of ±Inf (called "inf bucket" below, with an idx @@ -703,3 +777,10 @@ func reduceResolution[IBC InternalBucketCount]( return targetSpans, targetBuckets } + +func clearIfNotNil[T any](items []T) []T { + if items == nil { + return nil + } + return items[:0] +} diff --git a/vendor/github.com/prometheus/prometheus/model/histogram/histogram.go b/vendor/github.com/prometheus/prometheus/model/histogram/histogram.go index d94cf98fa6..e4b99ec420 100644 --- a/vendor/github.com/prometheus/prometheus/model/histogram/histogram.go +++ b/vendor/github.com/prometheus/prometheus/model/histogram/histogram.go @@ -49,11 +49,12 @@ const ( type Histogram struct { // Counter reset information. CounterResetHint CounterResetHint - // Currently valid schema numbers are -4 <= n <= 8. They are all for - // base-2 bucket schemas, where 1 is a bucket boundary in each case, and - // then each power of two is divided into 2^n logarithmic buckets. Or - // in other words, each bucket boundary is the previous boundary times - // 2^(2^-n). + // Currently valid schema numbers are -4 <= n <= 8 for exponential buckets, + // They are all for base-2 bucket schemas, where 1 is a bucket boundary in + // each case, and then each power of two is divided into 2^n logarithmic buckets. + // Or in other words, each bucket boundary is the previous boundary times + // 2^(2^-n). Another valid schema number is -53 for custom buckets, defined by + // the CustomValues field. Schema int32 // Width of the zero bucket. ZeroThreshold float64 @@ -69,6 +70,12 @@ type Histogram struct { // count. All following ones are deltas relative to the previous // element. PositiveBuckets, NegativeBuckets []int64 + // Holds the custom (usually upper) bounds for bucket definitions, otherwise nil. + // This slice is interned, to be treated as immutable and copied by reference. + // These numbers should be strictly increasing. This field is only used when the + // schema is for custom buckets, and the ZeroThreshold, ZeroCount, NegativeSpans + // and NegativeBuckets fields are not used in that case. + CustomValues []float64 } // A Span defines a continuous sequence of buckets. @@ -80,33 +87,46 @@ type Span struct { Length uint32 } +func (h *Histogram) UsesCustomBuckets() bool { + return IsCustomBucketsSchema(h.Schema) +} + // Copy returns a deep copy of the Histogram. func (h *Histogram) Copy() *Histogram { c := Histogram{ CounterResetHint: h.CounterResetHint, Schema: h.Schema, - ZeroThreshold: h.ZeroThreshold, - ZeroCount: h.ZeroCount, Count: h.Count, Sum: h.Sum, } + if h.UsesCustomBuckets() { + if len(h.CustomValues) != 0 { + c.CustomValues = make([]float64, len(h.CustomValues)) + copy(c.CustomValues, h.CustomValues) + } + } else { + c.ZeroThreshold = h.ZeroThreshold + c.ZeroCount = h.ZeroCount + + if len(h.NegativeSpans) != 0 { + c.NegativeSpans = make([]Span, len(h.NegativeSpans)) + copy(c.NegativeSpans, h.NegativeSpans) + } + if len(h.NegativeBuckets) != 0 { + c.NegativeBuckets = make([]int64, len(h.NegativeBuckets)) + copy(c.NegativeBuckets, h.NegativeBuckets) + } + } + if len(h.PositiveSpans) != 0 { c.PositiveSpans = make([]Span, len(h.PositiveSpans)) copy(c.PositiveSpans, h.PositiveSpans) } - if len(h.NegativeSpans) != 0 { - c.NegativeSpans = make([]Span, len(h.NegativeSpans)) - copy(c.NegativeSpans, h.NegativeSpans) - } if len(h.PositiveBuckets) != 0 { c.PositiveBuckets = make([]int64, len(h.PositiveBuckets)) copy(c.PositiveBuckets, h.PositiveBuckets) } - if len(h.NegativeBuckets) != 0 { - c.NegativeBuckets = make([]int64, len(h.NegativeBuckets)) - copy(c.NegativeBuckets, h.NegativeBuckets) - } return &c } @@ -116,22 +136,36 @@ func (h *Histogram) Copy() *Histogram { func (h *Histogram) CopyTo(to *Histogram) { to.CounterResetHint = h.CounterResetHint to.Schema = h.Schema - to.ZeroThreshold = h.ZeroThreshold - to.ZeroCount = h.ZeroCount to.Count = h.Count to.Sum = h.Sum + if h.UsesCustomBuckets() { + to.ZeroThreshold = 0 + to.ZeroCount = 0 + + to.NegativeSpans = clearIfNotNil(to.NegativeSpans) + to.NegativeBuckets = clearIfNotNil(to.NegativeBuckets) + + to.CustomValues = resize(to.CustomValues, len(h.CustomValues)) + copy(to.CustomValues, h.CustomValues) + } else { + to.ZeroThreshold = h.ZeroThreshold + to.ZeroCount = h.ZeroCount + + to.NegativeSpans = resize(to.NegativeSpans, len(h.NegativeSpans)) + copy(to.NegativeSpans, h.NegativeSpans) + + to.NegativeBuckets = resize(to.NegativeBuckets, len(h.NegativeBuckets)) + copy(to.NegativeBuckets, h.NegativeBuckets) + + to.CustomValues = clearIfNotNil(to.CustomValues) + } + to.PositiveSpans = resize(to.PositiveSpans, len(h.PositiveSpans)) copy(to.PositiveSpans, h.PositiveSpans) - to.NegativeSpans = resize(to.NegativeSpans, len(h.NegativeSpans)) - copy(to.NegativeSpans, h.NegativeSpans) - to.PositiveBuckets = resize(to.PositiveBuckets, len(h.PositiveBuckets)) copy(to.PositiveBuckets, h.PositiveBuckets) - - to.NegativeBuckets = resize(to.NegativeBuckets, len(h.NegativeBuckets)) - copy(to.NegativeBuckets, h.NegativeBuckets) } // String returns a string representation of the Histogram. @@ -165,8 +199,11 @@ func (h *Histogram) String() string { return sb.String() } -// ZeroBucket returns the zero bucket. +// ZeroBucket returns the zero bucket. This method panics if the schema is for custom buckets. func (h *Histogram) ZeroBucket() Bucket[uint64] { + if h.UsesCustomBuckets() { + panic("histograms with custom buckets have no zero bucket") + } return Bucket[uint64]{ Lower: -h.ZeroThreshold, Upper: h.ZeroThreshold, @@ -179,14 +216,14 @@ func (h *Histogram) ZeroBucket() Bucket[uint64] { // PositiveBucketIterator returns a BucketIterator to iterate over all positive // buckets in ascending order (starting next to the zero bucket and going up). func (h *Histogram) PositiveBucketIterator() BucketIterator[uint64] { - it := newRegularBucketIterator(h.PositiveSpans, h.PositiveBuckets, h.Schema, true) + it := newRegularBucketIterator(h.PositiveSpans, h.PositiveBuckets, h.Schema, true, h.CustomValues) return &it } // NegativeBucketIterator returns a BucketIterator to iterate over all negative // buckets in descending order (starting next to the zero bucket and going down). func (h *Histogram) NegativeBucketIterator() BucketIterator[uint64] { - it := newRegularBucketIterator(h.NegativeSpans, h.NegativeBuckets, h.Schema, false) + it := newRegularBucketIterator(h.NegativeSpans, h.NegativeBuckets, h.Schema, false, nil) return &it } @@ -207,28 +244,40 @@ func (h *Histogram) CumulativeBucketIterator() BucketIterator[uint64] { // but they must represent the same bucket layout to match. // Sum is compared based on its bit pattern because this method // is about data equality rather than mathematical equality. +// We ignore fields that are not used based on the exponential / custom buckets schema, +// but check fields where differences may cause unintended behaviour even if they are not +// supposed to be used according to the schema. func (h *Histogram) Equals(h2 *Histogram) bool { if h2 == nil { return false } - if h.Schema != h2.Schema || h.ZeroThreshold != h2.ZeroThreshold || - h.ZeroCount != h2.ZeroCount || h.Count != h2.Count || + if h.Schema != h2.Schema || h.Count != h2.Count || math.Float64bits(h.Sum) != math.Float64bits(h2.Sum) { return false } - if !spansMatch(h.PositiveSpans, h2.PositiveSpans) { + if h.UsesCustomBuckets() { + if !FloatBucketsMatch(h.CustomValues, h2.CustomValues) { + return false + } + } + + if h.ZeroThreshold != h2.ZeroThreshold || h.ZeroCount != h2.ZeroCount { return false } + if !spansMatch(h.NegativeSpans, h2.NegativeSpans) { return false } + if !slices.Equal(h.NegativeBuckets, h2.NegativeBuckets) { + return false + } - if !slices.Equal(h.PositiveBuckets, h2.PositiveBuckets) { + if !spansMatch(h.PositiveSpans, h2.PositiveSpans) { return false } - if !slices.Equal(h.NegativeBuckets, h2.NegativeBuckets) { + if !slices.Equal(h.PositiveBuckets, h2.PositiveBuckets) { return false } @@ -321,17 +370,36 @@ func (h *Histogram) ToFloat(fh *FloatHistogram) *FloatHistogram { } fh.CounterResetHint = h.CounterResetHint fh.Schema = h.Schema - fh.ZeroThreshold = h.ZeroThreshold - fh.ZeroCount = float64(h.ZeroCount) fh.Count = float64(h.Count) fh.Sum = h.Sum + if h.UsesCustomBuckets() { + fh.ZeroThreshold = 0 + fh.ZeroCount = 0 + fh.NegativeSpans = clearIfNotNil(fh.NegativeSpans) + fh.NegativeBuckets = clearIfNotNil(fh.NegativeBuckets) + + fh.CustomValues = resize(fh.CustomValues, len(h.CustomValues)) + copy(fh.CustomValues, h.CustomValues) + } else { + fh.ZeroThreshold = h.ZeroThreshold + fh.ZeroCount = float64(h.ZeroCount) + + fh.NegativeSpans = resize(fh.NegativeSpans, len(h.NegativeSpans)) + copy(fh.NegativeSpans, h.NegativeSpans) + + fh.NegativeBuckets = resize(fh.NegativeBuckets, len(h.NegativeBuckets)) + var currentNegative float64 + for i, b := range h.NegativeBuckets { + currentNegative += float64(b) + fh.NegativeBuckets[i] = currentNegative + } + fh.CustomValues = clearIfNotNil(fh.CustomValues) + } + fh.PositiveSpans = resize(fh.PositiveSpans, len(h.PositiveSpans)) copy(fh.PositiveSpans, h.PositiveSpans) - fh.NegativeSpans = resize(fh.NegativeSpans, len(h.NegativeSpans)) - copy(fh.NegativeSpans, h.NegativeSpans) - fh.PositiveBuckets = resize(fh.PositiveBuckets, len(h.PositiveBuckets)) var currentPositive float64 for i, b := range h.PositiveBuckets { @@ -339,13 +407,6 @@ func (h *Histogram) ToFloat(fh *FloatHistogram) *FloatHistogram { fh.PositiveBuckets[i] = currentPositive } - fh.NegativeBuckets = resize(fh.NegativeBuckets, len(h.NegativeBuckets)) - var currentNegative float64 - for i, b := range h.NegativeBuckets { - currentNegative += float64(b) - fh.NegativeBuckets[i] = currentNegative - } - return fh } @@ -357,25 +418,47 @@ func resize[T any](items []T, n int) []T { } // Validate validates consistency between span and bucket slices. Also, buckets are checked -// against negative values. +// against negative values. We check to make sure there are no unexpected fields or field values +// based on the exponential / custom buckets schema. // For histograms that have not observed any NaN values (based on IsNaN(h.Sum) check), a // strict h.Count = nCount + pCount + h.ZeroCount check is performed. // Otherwise, only a lower bound check will be done (h.Count >= nCount + pCount + h.ZeroCount), // because NaN observations do not increment the values of buckets (but they do increment // the total h.Count). func (h *Histogram) Validate() error { - if err := checkHistogramSpans(h.NegativeSpans, len(h.NegativeBuckets)); err != nil { - return fmt.Errorf("negative side: %w", err) - } - if err := checkHistogramSpans(h.PositiveSpans, len(h.PositiveBuckets)); err != nil { - return fmt.Errorf("positive side: %w", err) - } var nCount, pCount uint64 - err := checkHistogramBuckets(h.NegativeBuckets, &nCount, true) - if err != nil { - return fmt.Errorf("negative side: %w", err) + if h.UsesCustomBuckets() { + if err := checkHistogramCustomBounds(h.CustomValues, h.PositiveSpans, len(h.PositiveBuckets)); err != nil { + return fmt.Errorf("custom buckets: %w", err) + } + if h.ZeroCount != 0 { + return fmt.Errorf("custom buckets: must have zero count of 0") + } + if h.ZeroThreshold != 0 { + return fmt.Errorf("custom buckets: must have zero threshold of 0") + } + if len(h.NegativeSpans) > 0 { + return fmt.Errorf("custom buckets: must not have negative spans") + } + if len(h.NegativeBuckets) > 0 { + return fmt.Errorf("custom buckets: must not have negative buckets") + } + } else { + if err := checkHistogramSpans(h.PositiveSpans, len(h.PositiveBuckets)); err != nil { + return fmt.Errorf("positive side: %w", err) + } + if err := checkHistogramSpans(h.NegativeSpans, len(h.NegativeBuckets)); err != nil { + return fmt.Errorf("negative side: %w", err) + } + err := checkHistogramBuckets(h.NegativeBuckets, &nCount, true) + if err != nil { + return fmt.Errorf("negative side: %w", err) + } + if h.CustomValues != nil { + return fmt.Errorf("histogram with exponential schema must not have custom bounds") + } } - err = checkHistogramBuckets(h.PositiveBuckets, &pCount, true) + err := checkHistogramBuckets(h.PositiveBuckets, &pCount, true) if err != nil { return fmt.Errorf("positive side: %w", err) } @@ -398,12 +481,13 @@ type regularBucketIterator struct { baseBucketIterator[uint64, int64] } -func newRegularBucketIterator(spans []Span, buckets []int64, schema int32, positive bool) regularBucketIterator { +func newRegularBucketIterator(spans []Span, buckets []int64, schema int32, positive bool, customValues []float64) regularBucketIterator { i := baseBucketIterator[uint64, int64]{ - schema: schema, - spans: spans, - buckets: buckets, - positive: positive, + schema: schema, + spans: spans, + buckets: buckets, + positive: positive, + customValues: customValues, } return regularBucketIterator{i} } @@ -477,7 +561,7 @@ func (c *cumulativeBucketIterator) Next() bool { if c.emptyBucketCount > 0 { // We are traversing through empty buckets at the moment. - c.currUpper = getBound(c.currIdx, c.h.Schema) + c.currUpper = getBound(c.currIdx, c.h.Schema, c.h.CustomValues) c.currIdx++ c.emptyBucketCount-- return true @@ -494,7 +578,7 @@ func (c *cumulativeBucketIterator) Next() bool { c.currCount += c.h.PositiveBuckets[c.posBucketsIdx] c.currCumulativeCount += uint64(c.currCount) - c.currUpper = getBound(c.currIdx, c.h.Schema) + c.currUpper = getBound(c.currIdx, c.h.Schema, c.h.CustomValues) c.posBucketsIdx++ c.idxInSpan++ @@ -524,7 +608,15 @@ func (c *cumulativeBucketIterator) At() Bucket[uint64] { // ReduceResolution reduces the histogram's spans, buckets into target schema. // The target schema must be smaller than the current histogram's schema. +// This will panic if the histogram has custom buckets or if the target schema is +// a custom buckets schema. func (h *Histogram) ReduceResolution(targetSchema int32) *Histogram { + if h.UsesCustomBuckets() { + panic("cannot reduce resolution when there are custom buckets") + } + if IsCustomBucketsSchema(targetSchema) { + panic("cannot reduce resolution to custom buckets schema") + } if targetSchema >= h.Schema { panic(fmt.Errorf("cannot reduce resolution from schema %d to %d", h.Schema, targetSchema)) } diff --git a/vendor/github.com/prometheus/prometheus/model/labels/labels_dedupelabels.go b/vendor/github.com/prometheus/prometheus/model/labels/labels_dedupelabels.go index 972f5dc164..0e5bb048be 100644 --- a/vendor/github.com/prometheus/prometheus/model/labels/labels_dedupelabels.go +++ b/vendor/github.com/prometheus/prometheus/model/labels/labels_dedupelabels.go @@ -104,30 +104,39 @@ func (t *nameTable) ToName(num int) string { return t.byNum[num] } +// "Varint" in this file is non-standard: we encode small numbers (up to 32767) in 2 bytes, +// because we expect most Prometheus to have more than 127 unique strings. +// And we don't encode numbers larger than 4 bytes because we don't expect more than 536,870,912 unique strings. func decodeVarint(data string, index int) (int, int) { - // Fast-path for common case of a single byte, value 0..127. - b := data[index] + b := int(data[index]) + int(data[index+1])<<8 + index += 2 + if b < 0x8000 { + return b, index + } + return decodeVarintRest(b, data, index) +} + +func decodeVarintRest(b int, data string, index int) (int, int) { + value := int(b & 0x7FFF) + b = int(data[index]) index++ if b < 0x80 { - return int(b), index - } - value := int(b & 0x7F) - for shift := uint(7); ; shift += 7 { - // Just panic if we go of the end of data, since all Labels strings are constructed internally and - // malformed data indicates a bug, or memory corruption. - b := data[index] - index++ - value |= int(b&0x7F) << shift - if b < 0x80 { - break - } + return value | (b << 15), index } - return value, index + + value |= (b & 0x7f) << 15 + b = int(data[index]) + index++ + return value | (b << 22), index } func decodeString(t *nameTable, data string, index int) (string, int) { - var num int - num, index = decodeVarint(data, index) + // Copy decodeVarint here, because the Go compiler says it's too big to inline. + num := int(data[index]) + int(data[index+1])<<8 + index += 2 + if num >= 0x8000 { + num, index = decodeVarintRest(num, data, index) + } return t.ToName(num), index } @@ -321,7 +330,12 @@ func (ls Labels) Get(name string) string { } else if lName[0] > name[0] { // Stop looking if we've gone past. break } - _, i = decodeVarint(ls.data, i) + // Copy decodeVarint here, because the Go compiler says it's too big to inline. + num := int(ls.data[i]) + int(ls.data[i+1])<<8 + i += 2 + if num >= 0x8000 { + _, i = decodeVarintRest(num, ls.data, i) + } } return "" } @@ -339,7 +353,12 @@ func (ls Labels) Has(name string) bool { } else if lName[0] > name[0] { // Stop looking if we've gone past. break } - _, i = decodeVarint(ls.data, i) + // Copy decodeVarint here, because the Go compiler says it's too big to inline. + num := int(ls.data[i]) + int(ls.data[i+1])<<8 + i += 2 + if num >= 0x8000 { + _, i = decodeVarintRest(num, ls.data, i) + } } return false } @@ -641,29 +660,24 @@ func marshalNumbersToSizedBuffer(nums []int, data []byte) int { func sizeVarint(x uint64) (n int) { // Most common case first - if x < 1<<7 { - return 1 + if x < 1<<15 { + return 2 } - if x >= 1<<56 { - return 9 + if x < 1<<22 { + return 3 } - if x >= 1<<28 { - x >>= 28 - n = 4 + if x >= 1<<29 { + panic("Number too large to represent") } - if x >= 1<<14 { - x >>= 14 - n += 2 - } - if x >= 1<<7 { - n++ - } - return n + 1 + return 4 } func encodeVarintSlow(data []byte, offset int, v uint64) int { offset -= sizeVarint(v) base := offset + data[offset] = uint8(v) + v >>= 8 + offset++ for v >= 1<<7 { data[offset] = uint8(v&0x7f | 0x80) v >>= 7 @@ -673,11 +687,12 @@ func encodeVarintSlow(data []byte, offset int, v uint64) int { return base } -// Special code for the common case that a value is less than 128 +// Special code for the common case that a value is less than 32768 func encodeVarint(data []byte, offset, v int) int { - if v < 1<<7 { - offset-- + if v < 1<<15 { + offset -= 2 data[offset] = uint8(v) + data[offset+1] = uint8(v >> 8) return offset } return encodeVarintSlow(data, offset, uint64(v)) diff --git a/vendor/github.com/prometheus/prometheus/model/labels/regexp.go b/vendor/github.com/prometheus/prometheus/model/labels/regexp.go index 1e9db882bf..d2151d83dd 100644 --- a/vendor/github.com/prometheus/prometheus/model/labels/regexp.go +++ b/vendor/github.com/prometheus/prometheus/model/labels/regexp.go @@ -28,7 +28,7 @@ const ( maxSetMatches = 256 // The minimum number of alternate values a regex should have to trigger - // the optimization done by optimizeEqualStringMatchers() and so use a map + // the optimization done by optimizeEqualOrPrefixStringMatchers() and so use a map // to match values instead of iterating over a list. This value has // been computed running BenchmarkOptimizeEqualStringMatchers. minEqualMultiStringMatcherMapThreshold = 16 @@ -337,7 +337,7 @@ func optimizeAlternatingLiterals(s string) (StringMatcher, []string) { return nil, nil } - multiMatcher := newEqualMultiStringMatcher(true, estimatedAlternates) + multiMatcher := newEqualMultiStringMatcher(true, estimatedAlternates, 0, 0) for end := strings.IndexByte(s, '|'); end > -1; end = strings.IndexByte(s, '|') { // Split the string into the next literal and the remainder @@ -412,7 +412,7 @@ func stringMatcherFromRegexp(re *syntax.Regexp) StringMatcher { clearBeginEndText(re) m := stringMatcherFromRegexpInternal(re) - m = optimizeEqualStringMatchers(m, minEqualMultiStringMatcherMapThreshold) + m = optimizeEqualOrPrefixStringMatchers(m, minEqualMultiStringMatcherMapThreshold) return m } @@ -549,11 +549,7 @@ func stringMatcherFromRegexpInternal(re *syntax.Regexp) StringMatcher { // Right matcher with 1 fixed set match. case left == nil && len(matches) == 1: - return &literalPrefixStringMatcher{ - prefix: matches[0], - prefixCaseSensitive: matchesCaseSensitive, - right: right, - } + return newLiteralPrefixStringMatcher(matches[0], matchesCaseSensitive, right) // Left matcher with 1 fixed set match. case right == nil && len(matches) == 1: @@ -631,21 +627,47 @@ func (m *containsStringMatcher) Matches(s string) bool { return false } -// literalPrefixStringMatcher matches a string with the given literal prefix and right side matcher. -type literalPrefixStringMatcher struct { - prefix string - prefixCaseSensitive bool +func newLiteralPrefixStringMatcher(prefix string, prefixCaseSensitive bool, right StringMatcher) StringMatcher { + if prefixCaseSensitive { + return &literalPrefixSensitiveStringMatcher{ + prefix: prefix, + right: right, + } + } + + return &literalPrefixInsensitiveStringMatcher{ + prefix: prefix, + right: right, + } +} + +// literalPrefixSensitiveStringMatcher matches a string with the given literal case-sensitive prefix and right side matcher. +type literalPrefixSensitiveStringMatcher struct { + prefix string // The matcher that must match the right side. Can be nil. right StringMatcher } -func (m *literalPrefixStringMatcher) Matches(s string) bool { - // Ensure the prefix matches. - if m.prefixCaseSensitive && !strings.HasPrefix(s, m.prefix) { +func (m *literalPrefixSensitiveStringMatcher) Matches(s string) bool { + if !strings.HasPrefix(s, m.prefix) { return false } - if !m.prefixCaseSensitive && !hasPrefixCaseInsensitive(s, m.prefix) { + + // Ensure the right side matches. + return m.right.Matches(s[len(m.prefix):]) +} + +// literalPrefixInsensitiveStringMatcher matches a string with the given literal case-insensitive prefix and right side matcher. +type literalPrefixInsensitiveStringMatcher struct { + prefix string + + // The matcher that must match the right side. Can be nil. + right StringMatcher +} + +func (m *literalPrefixInsensitiveStringMatcher) Matches(s string) bool { + if !hasPrefixCaseInsensitive(s, m.prefix) { return false } @@ -710,17 +732,20 @@ func (m *equalStringMatcher) Matches(s string) bool { type multiStringMatcherBuilder interface { StringMatcher add(s string) + addPrefix(prefix string, prefixCaseSensitive bool, matcher StringMatcher) setMatches() []string } -func newEqualMultiStringMatcher(caseSensitive bool, estimatedSize int) multiStringMatcherBuilder { +func newEqualMultiStringMatcher(caseSensitive bool, estimatedSize, estimatedPrefixes, minPrefixLength int) multiStringMatcherBuilder { // If the estimated size is low enough, it's faster to use a slice instead of a map. - if estimatedSize < minEqualMultiStringMatcherMapThreshold { + if estimatedSize < minEqualMultiStringMatcherMapThreshold && estimatedPrefixes == 0 { return &equalMultiStringSliceMatcher{caseSensitive: caseSensitive, values: make([]string, 0, estimatedSize)} } return &equalMultiStringMapMatcher{ values: make(map[string]struct{}, estimatedSize), + prefixes: make(map[string][]StringMatcher, estimatedPrefixes), + minPrefixLen: minPrefixLength, caseSensitive: caseSensitive, } } @@ -736,6 +761,10 @@ func (m *equalMultiStringSliceMatcher) add(s string) { m.values = append(m.values, s) } +func (m *equalMultiStringSliceMatcher) addPrefix(_ string, _ bool, _ StringMatcher) { + panic("not implemented") +} + func (m *equalMultiStringSliceMatcher) setMatches() []string { return m.values } @@ -757,12 +786,17 @@ func (m *equalMultiStringSliceMatcher) Matches(s string) bool { return false } -// equalMultiStringMapMatcher matches a string exactly against a map of valid values. +// equalMultiStringMapMatcher matches a string exactly against a map of valid values +// or against a set of prefix matchers. type equalMultiStringMapMatcher struct { // values contains values to match a string against. If the matching is case insensitive, // the values here must be lowercase. values map[string]struct{} - + // prefixes maps strings, all of length minPrefixLen, to sets of matchers to check the rest of the string. + // If the matching is case insensitive, prefixes are all lowercase. + prefixes map[string][]StringMatcher + // minPrefixLen can be zero, meaning there are no prefix matchers. + minPrefixLen int caseSensitive bool } @@ -774,8 +808,27 @@ func (m *equalMultiStringMapMatcher) add(s string) { m.values[s] = struct{}{} } +func (m *equalMultiStringMapMatcher) addPrefix(prefix string, prefixCaseSensitive bool, matcher StringMatcher) { + if m.minPrefixLen == 0 { + panic("addPrefix called when no prefix length defined") + } + if len(prefix) < m.minPrefixLen { + panic("addPrefix called with a too short prefix") + } + if m.caseSensitive != prefixCaseSensitive { + panic("addPrefix called with a prefix whose case sensitivity is different than the expected one") + } + + s := prefix[:m.minPrefixLen] + if !m.caseSensitive { + s = strings.ToLower(s) + } + + m.prefixes[s] = append(m.prefixes[s], matcher) +} + func (m *equalMultiStringMapMatcher) setMatches() []string { - if len(m.values) >= maxSetMatches { + if len(m.values) >= maxSetMatches || len(m.prefixes) > 0 { return nil } @@ -791,8 +844,17 @@ func (m *equalMultiStringMapMatcher) Matches(s string) bool { s = toNormalisedLower(s) } - _, ok := m.values[s] - return ok + if _, ok := m.values[s]; ok { + return true + } + if m.minPrefixLen > 0 && len(s) >= m.minPrefixLen { + for _, matcher := range m.prefixes[s[:m.minPrefixLen]] { + if matcher.Matches(s) { + return true + } + } + } + return false } // toNormalisedLower normalise the input string using "Unicode Normalization Form D" and then convert @@ -875,20 +937,24 @@ func (m trueMatcher) Matches(_ string) bool { return true } -// optimizeEqualStringMatchers optimize a specific case where all matchers are made by an -// alternation (orStringMatcher) of strings checked for equality (equalStringMatcher). In -// this specific case, when we have many strings to match against we can use a map instead +// optimizeEqualOrPrefixStringMatchers optimize a specific case where all matchers are made by an +// alternation (orStringMatcher) of strings checked for equality (equalStringMatcher) or +// with a literal prefix (literalPrefixSensitiveStringMatcher or literalPrefixInsensitiveStringMatcher). +// +// In this specific case, when we have many strings to match against we can use a map instead // of iterating over the list of strings. -func optimizeEqualStringMatchers(input StringMatcher, threshold int) StringMatcher { +func optimizeEqualOrPrefixStringMatchers(input StringMatcher, threshold int) StringMatcher { var ( caseSensitive bool caseSensitiveSet bool numValues int + numPrefixes int + minPrefixLength int ) // Analyse the input StringMatcher to count the number of occurrences // and ensure all of them have the same case sensitivity. - analyseCallback := func(matcher *equalStringMatcher) bool { + analyseEqualMatcherCallback := func(matcher *equalStringMatcher) bool { // Ensure we don't have mixed case sensitivity. if caseSensitiveSet && caseSensitive != matcher.caseSensitive { return false @@ -901,34 +967,55 @@ func optimizeEqualStringMatchers(input StringMatcher, threshold int) StringMatch return true } - if !findEqualStringMatchers(input, analyseCallback) { + analysePrefixMatcherCallback := func(prefix string, prefixCaseSensitive bool, matcher StringMatcher) bool { + // Ensure we don't have mixed case sensitivity. + if caseSensitiveSet && caseSensitive != prefixCaseSensitive { + return false + } else if !caseSensitiveSet { + caseSensitive = prefixCaseSensitive + caseSensitiveSet = true + } + if numPrefixes == 0 || len(prefix) < minPrefixLength { + minPrefixLength = len(prefix) + } + + numPrefixes++ + return true + } + + if !findEqualOrPrefixStringMatchers(input, analyseEqualMatcherCallback, analysePrefixMatcherCallback) { return input } - // If the number of values found is less than the threshold, then we should skip the optimization. - if numValues < threshold { + // If the number of values and prefixes found is less than the threshold, then we should skip the optimization. + if (numValues + numPrefixes) < threshold { return input } // Parse again the input StringMatcher to extract all values and storing them. // We can skip the case sensitivity check because we've already checked it and // if the code reach this point then it means all matchers have the same case sensitivity. - multiMatcher := newEqualMultiStringMatcher(caseSensitive, numValues) + multiMatcher := newEqualMultiStringMatcher(caseSensitive, numValues, numPrefixes, minPrefixLength) // Ignore the return value because we already iterated over the input StringMatcher // and it was all good. - findEqualStringMatchers(input, func(matcher *equalStringMatcher) bool { + findEqualOrPrefixStringMatchers(input, func(matcher *equalStringMatcher) bool { multiMatcher.add(matcher.s) return true + }, func(prefix string, prefixCaseSensitive bool, matcher StringMatcher) bool { + multiMatcher.addPrefix(prefix, caseSensitive, matcher) + return true }) return multiMatcher } -// findEqualStringMatchers analyze the input StringMatcher and calls the callback for each -// equalStringMatcher found. Returns true if and only if the input StringMatcher is *only* -// composed by an alternation of equalStringMatcher. -func findEqualStringMatchers(input StringMatcher, callback func(matcher *equalStringMatcher) bool) bool { +// findEqualOrPrefixStringMatchers analyze the input StringMatcher and calls the equalMatcherCallback for each +// equalStringMatcher found, and prefixMatcherCallback for each literalPrefixSensitiveStringMatcher and literalPrefixInsensitiveStringMatcher found. +// +// Returns true if and only if the input StringMatcher is *only* composed by an alternation of equalStringMatcher and/or +// literal prefix matcher. Returns false if prefixMatcherCallback is nil and a literal prefix matcher is encountered. +func findEqualOrPrefixStringMatchers(input StringMatcher, equalMatcherCallback func(matcher *equalStringMatcher) bool, prefixMatcherCallback func(prefix string, prefixCaseSensitive bool, matcher StringMatcher) bool) bool { orInput, ok := input.(orStringMatcher) if !ok { return false @@ -937,17 +1024,27 @@ func findEqualStringMatchers(input StringMatcher, callback func(matcher *equalSt for _, m := range orInput { switch casted := m.(type) { case orStringMatcher: - if !findEqualStringMatchers(m, callback) { + if !findEqualOrPrefixStringMatchers(m, equalMatcherCallback, prefixMatcherCallback) { return false } case *equalStringMatcher: - if !callback(casted) { + if !equalMatcherCallback(casted) { + return false + } + + case *literalPrefixSensitiveStringMatcher: + if prefixMatcherCallback == nil || !prefixMatcherCallback(casted.prefix, true, casted) { + return false + } + + case *literalPrefixInsensitiveStringMatcher: + if prefixMatcherCallback == nil || !prefixMatcherCallback(casted.prefix, false, casted) { return false } default: - // It's not an equal string matcher, so we have to stop searching + // It's not an equal or prefix string matcher, so we have to stop searching // cause this optimization can't be applied. return false } diff --git a/vendor/github.com/prometheus/prometheus/model/relabel/relabel.go b/vendor/github.com/prometheus/prometheus/model/relabel/relabel.go index db2e6ce2ff..4f33edda43 100644 --- a/vendor/github.com/prometheus/prometheus/model/relabel/relabel.go +++ b/vendor/github.com/prometheus/prometheus/model/relabel/relabel.go @@ -206,6 +206,11 @@ func (re Regexp) MarshalYAML() (interface{}, error) { return nil, nil } +// IsZero implements the yaml.IsZeroer interface. +func (re Regexp) IsZero() bool { + return re.Regexp == DefaultRelabelConfig.Regex.Regexp +} + // String returns the original string used to compile the regular expression. func (re Regexp) String() string { str := re.Regexp.String() diff --git a/vendor/github.com/prometheus/prometheus/notifier/notifier.go b/vendor/github.com/prometheus/prometheus/notifier/notifier.go index cd00a4507d..68b0d4961e 100644 --- a/vendor/github.com/prometheus/prometheus/notifier/notifier.go +++ b/vendor/github.com/prometheus/prometheus/notifier/notifier.go @@ -110,10 +110,11 @@ type Manager struct { metrics *alertMetrics - more chan struct{} - mtx sync.RWMutex - ctx context.Context - cancel func() + more chan struct{} + mtx sync.RWMutex + + stopOnce *sync.Once + stopRequested chan struct{} alertmanagers map[string]*alertmanagerSet logger log.Logger @@ -121,9 +122,10 @@ type Manager struct { // Options are the configurable parameters of a Handler. type Options struct { - QueueCapacity int - ExternalLabels labels.Labels - RelabelConfigs []*relabel.Config + QueueCapacity int + DrainOnShutdown bool + ExternalLabels labels.Labels + RelabelConfigs []*relabel.Config // Used for sending HTTP requests to the Alertmanager. Do func(ctx context.Context, client *http.Client, req *http.Request) (*http.Response, error) @@ -217,8 +219,6 @@ func do(ctx context.Context, client *http.Client, req *http.Request) (*http.Resp // NewManager is the manager constructor. func NewManager(o *Options, logger log.Logger) *Manager { - ctx, cancel := context.WithCancel(context.Background()) - if o.Do == nil { o.Do = do } @@ -227,12 +227,12 @@ func NewManager(o *Options, logger log.Logger) *Manager { } n := &Manager{ - queue: make([]*Alert, 0, o.QueueCapacity), - ctx: ctx, - cancel: cancel, - more: make(chan struct{}, 1), - opts: o, - logger: logger, + queue: make([]*Alert, 0, o.QueueCapacity), + more: make(chan struct{}, 1), + stopRequested: make(chan struct{}), + stopOnce: &sync.Once{}, + opts: o, + logger: logger, } queueLenFunc := func() float64 { return float64(n.queueLen()) } @@ -298,40 +298,98 @@ func (n *Manager) nextBatch() []*Alert { return alerts } +// Run dispatches notifications continuously, returning once Stop has been called and all +// pending notifications have been drained from the queue (if draining is enabled). +// +// Dispatching of notifications occurs in parallel to processing target updates to avoid one starving the other. +// Refer to https://github.com/prometheus/prometheus/issues/13676 for more details. +func (n *Manager) Run(tsets <-chan map[string][]*targetgroup.Group) { + wg := sync.WaitGroup{} + wg.Add(2) + + go func() { + defer wg.Done() + n.targetUpdateLoop(tsets) + }() + + go func() { + defer wg.Done() + n.sendLoop() + n.drainQueue() + }() + + wg.Wait() + level.Info(n.logger).Log("msg", "Notification manager stopped") +} + // sendLoop continuously consumes the notifications queue and sends alerts to // the configured Alertmanagers. func (n *Manager) sendLoop() { for { + // If we've been asked to stop, that takes priority over sending any further notifications. select { - case <-n.ctx.Done(): + case <-n.stopRequested: return - case <-n.more: - } - alerts := n.nextBatch() + default: + select { + case <-n.stopRequested: + return - if !n.sendAll(alerts...) { - n.metrics.dropped.Add(float64(len(alerts))) - } - // If the queue still has items left, kick off the next iteration. - if n.queueLen() > 0 { - n.setMore() + case <-n.more: + n.sendOneBatch() + + // If the queue still has items left, kick off the next iteration. + if n.queueLen() > 0 { + n.setMore() + } + } } } } -// Run receives updates of target groups and triggers a reload. -// The dispatching of notifications occurs in the background to prevent blocking the receipt of target updates. -// Refer to https://github.com/prometheus/prometheus/issues/13676 for more details. -func (n *Manager) Run(tsets <-chan map[string][]*targetgroup.Group) { - go n.sendLoop() +// targetUpdateLoop receives updates of target groups and triggers a reload. +func (n *Manager) targetUpdateLoop(tsets <-chan map[string][]*targetgroup.Group) { for { + // If we've been asked to stop, that takes priority over processing any further target group updates. select { - case <-n.ctx.Done(): + case <-n.stopRequested: return - case ts := <-tsets: - n.reload(ts) + default: + select { + case <-n.stopRequested: + return + case ts := <-tsets: + n.reload(ts) + } + } + } +} + +func (n *Manager) sendOneBatch() { + alerts := n.nextBatch() + + if !n.sendAll(alerts...) { + n.metrics.dropped.Add(float64(len(alerts))) + } +} + +func (n *Manager) drainQueue() { + if !n.opts.DrainOnShutdown { + if n.queueLen() > 0 { + level.Warn(n.logger).Log("msg", "Draining remaining notifications on shutdown is disabled, and some notifications have been dropped", "count", n.queueLen()) + n.metrics.dropped.Add(float64(n.queueLen())) } + + return + } + + level.Info(n.logger).Log("msg", "Draining any remaining notifications...") + + for n.queueLen() > 0 { + n.sendOneBatch() } + + level.Info(n.logger).Log("msg", "Remaining notifications drained") } func (n *Manager) reload(tgs map[string][]*targetgroup.Group) { @@ -546,7 +604,7 @@ func (n *Manager) sendAll(alerts ...*Alert) bool { for _, am := range ams.ams { wg.Add(1) - ctx, cancel := context.WithTimeout(n.ctx, time.Duration(ams.cfg.Timeout)) + ctx, cancel := context.WithTimeout(context.Background(), time.Duration(ams.cfg.Timeout)) defer cancel() go func(ctx context.Context, client *http.Client, url string, payload []byte, count int) { @@ -624,10 +682,19 @@ func (n *Manager) sendOne(ctx context.Context, c *http.Client, url string, b []b return nil } -// Stop shuts down the notification handler. +// Stop signals the notification manager to shut down and immediately returns. +// +// Run will return once the notification manager has successfully shut down. +// +// The manager will optionally drain any queued notifications before shutting down. +// +// Stop is safe to call multiple times. func (n *Manager) Stop() { level.Info(n.logger).Log("msg", "Stopping notification manager...") - n.cancel() + + n.stopOnce.Do(func() { + close(n.stopRequested) + }) } // Alertmanager holds Alertmanager endpoint information. diff --git a/vendor/github.com/prometheus/prometheus/prompb/codec.go b/vendor/github.com/prometheus/prometheus/prompb/codec.go new file mode 100644 index 0000000000..ad30cd5e7b --- /dev/null +++ b/vendor/github.com/prometheus/prometheus/prompb/codec.go @@ -0,0 +1,201 @@ +// Copyright 2024 Prometheus Team +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package prompb + +import ( + "strings" + + "github.com/prometheus/common/model" + + "github.com/prometheus/prometheus/model/exemplar" + "github.com/prometheus/prometheus/model/histogram" + "github.com/prometheus/prometheus/model/labels" +) + +// NOTE(bwplotka): This file's code is tested in /prompb/rwcommon. + +// ToLabels return model labels.Labels from timeseries' remote labels. +func (m TimeSeries) ToLabels(b *labels.ScratchBuilder, _ []string) labels.Labels { + return labelProtosToLabels(b, m.GetLabels()) +} + +// ToLabels return model labels.Labels from timeseries' remote labels. +func (m ChunkedSeries) ToLabels(b *labels.ScratchBuilder, _ []string) labels.Labels { + return labelProtosToLabels(b, m.GetLabels()) +} + +func labelProtosToLabels(b *labels.ScratchBuilder, labelPairs []Label) labels.Labels { + b.Reset() + for _, l := range labelPairs { + b.Add(l.Name, l.Value) + } + b.Sort() + return b.Labels() +} + +// FromLabels transforms labels into prompb labels. The buffer slice +// will be used to avoid allocations if it is big enough to store the labels. +func FromLabels(lbls labels.Labels, buf []Label) []Label { + result := buf[:0] + lbls.Range(func(l labels.Label) { + result = append(result, Label{ + Name: l.Name, + Value: l.Value, + }) + }) + return result +} + +// FromMetadataType transforms a Prometheus metricType into prompb metricType. Since the former is a string we need to transform it to an enum. +func FromMetadataType(t model.MetricType) MetricMetadata_MetricType { + mt := strings.ToUpper(string(t)) + v, ok := MetricMetadata_MetricType_value[mt] + if !ok { + return MetricMetadata_UNKNOWN + } + return MetricMetadata_MetricType(v) +} + +// IsFloatHistogram returns true if the histogram is float. +func (h Histogram) IsFloatHistogram() bool { + _, ok := h.GetCount().(*Histogram_CountFloat) + return ok +} + +// ToIntHistogram returns integer Prometheus histogram from the remote implementation +// of integer histogram. If it's a float histogram, the method returns nil. +func (h Histogram) ToIntHistogram() *histogram.Histogram { + if h.IsFloatHistogram() { + return nil + } + return &histogram.Histogram{ + CounterResetHint: histogram.CounterResetHint(h.ResetHint), + Schema: h.Schema, + ZeroThreshold: h.ZeroThreshold, + ZeroCount: h.GetZeroCountInt(), + Count: h.GetCountInt(), + Sum: h.Sum, + PositiveSpans: spansProtoToSpans(h.GetPositiveSpans()), + PositiveBuckets: h.GetPositiveDeltas(), + NegativeSpans: spansProtoToSpans(h.GetNegativeSpans()), + NegativeBuckets: h.GetNegativeDeltas(), + } +} + +// ToFloatHistogram returns float Prometheus histogram from the remote implementation +// of float histogram. If the underlying implementation is an integer histogram, a +// conversion is performed. +func (h Histogram) ToFloatHistogram() *histogram.FloatHistogram { + if h.IsFloatHistogram() { + return &histogram.FloatHistogram{ + CounterResetHint: histogram.CounterResetHint(h.ResetHint), + Schema: h.Schema, + ZeroThreshold: h.ZeroThreshold, + ZeroCount: h.GetZeroCountFloat(), + Count: h.GetCountFloat(), + Sum: h.Sum, + PositiveSpans: spansProtoToSpans(h.GetPositiveSpans()), + PositiveBuckets: h.GetPositiveCounts(), + NegativeSpans: spansProtoToSpans(h.GetNegativeSpans()), + NegativeBuckets: h.GetNegativeCounts(), + } + } + // Conversion from integer histogram. + return &histogram.FloatHistogram{ + CounterResetHint: histogram.CounterResetHint(h.ResetHint), + Schema: h.Schema, + ZeroThreshold: h.ZeroThreshold, + ZeroCount: float64(h.GetZeroCountInt()), + Count: float64(h.GetCountInt()), + Sum: h.Sum, + PositiveSpans: spansProtoToSpans(h.GetPositiveSpans()), + PositiveBuckets: deltasToCounts(h.GetPositiveDeltas()), + NegativeSpans: spansProtoToSpans(h.GetNegativeSpans()), + NegativeBuckets: deltasToCounts(h.GetNegativeDeltas()), + } +} + +func spansProtoToSpans(s []BucketSpan) []histogram.Span { + spans := make([]histogram.Span, len(s)) + for i := 0; i < len(s); i++ { + spans[i] = histogram.Span{Offset: s[i].Offset, Length: s[i].Length} + } + + return spans +} + +func deltasToCounts(deltas []int64) []float64 { + counts := make([]float64, len(deltas)) + var cur float64 + for i, d := range deltas { + cur += float64(d) + counts[i] = cur + } + return counts +} + +// FromIntHistogram returns remote Histogram from the integer Histogram. +func FromIntHistogram(timestamp int64, h *histogram.Histogram) Histogram { + return Histogram{ + Count: &Histogram_CountInt{CountInt: h.Count}, + Sum: h.Sum, + Schema: h.Schema, + ZeroThreshold: h.ZeroThreshold, + ZeroCount: &Histogram_ZeroCountInt{ZeroCountInt: h.ZeroCount}, + NegativeSpans: spansToSpansProto(h.NegativeSpans), + NegativeDeltas: h.NegativeBuckets, + PositiveSpans: spansToSpansProto(h.PositiveSpans), + PositiveDeltas: h.PositiveBuckets, + ResetHint: Histogram_ResetHint(h.CounterResetHint), + Timestamp: timestamp, + } +} + +// FromFloatHistogram returns remote Histogram from the float Histogram. +func FromFloatHistogram(timestamp int64, fh *histogram.FloatHistogram) Histogram { + return Histogram{ + Count: &Histogram_CountFloat{CountFloat: fh.Count}, + Sum: fh.Sum, + Schema: fh.Schema, + ZeroThreshold: fh.ZeroThreshold, + ZeroCount: &Histogram_ZeroCountFloat{ZeroCountFloat: fh.ZeroCount}, + NegativeSpans: spansToSpansProto(fh.NegativeSpans), + NegativeCounts: fh.NegativeBuckets, + PositiveSpans: spansToSpansProto(fh.PositiveSpans), + PositiveCounts: fh.PositiveBuckets, + ResetHint: Histogram_ResetHint(fh.CounterResetHint), + Timestamp: timestamp, + } +} + +func spansToSpansProto(s []histogram.Span) []BucketSpan { + spans := make([]BucketSpan, len(s)) + for i := 0; i < len(s); i++ { + spans[i] = BucketSpan{Offset: s[i].Offset, Length: s[i].Length} + } + + return spans +} + +// ToExemplar converts remote exemplar to model exemplar. +func (m Exemplar) ToExemplar(b *labels.ScratchBuilder, _ []string) exemplar.Exemplar { + timestamp := m.Timestamp + + return exemplar.Exemplar{ + Labels: labelProtosToLabels(b, m.GetLabels()), + Value: m.Value, + Ts: timestamp, + HasTs: timestamp != 0, + } +} diff --git a/vendor/github.com/prometheus/prometheus/prompb/custom.go b/vendor/github.com/prometheus/prometheus/prompb/custom.go index 13d6e0f0cd..f73ddd446b 100644 --- a/vendor/github.com/prometheus/prometheus/prompb/custom.go +++ b/vendor/github.com/prometheus/prometheus/prompb/custom.go @@ -17,14 +17,6 @@ import ( "sync" ) -func (m Sample) T() int64 { return m.Timestamp } -func (m Sample) V() float64 { return m.Value } - -func (h Histogram) IsFloatHistogram() bool { - _, ok := h.GetCount().(*Histogram_CountFloat) - return ok -} - func (r *ChunkedReadResponse) PooledMarshal(p *sync.Pool) ([]byte, error) { size := r.Size() data, ok := p.Get().(*[]byte) diff --git a/vendor/github.com/prometheus/prometheus/prompb/io/prometheus/write/v2/codec.go b/vendor/github.com/prometheus/prometheus/prompb/io/prometheus/write/v2/codec.go new file mode 100644 index 0000000000..25fa0d4035 --- /dev/null +++ b/vendor/github.com/prometheus/prometheus/prompb/io/prometheus/write/v2/codec.go @@ -0,0 +1,216 @@ +// Copyright 2024 Prometheus Team +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package writev2 + +import ( + "github.com/prometheus/common/model" + + "github.com/prometheus/prometheus/model/exemplar" + "github.com/prometheus/prometheus/model/histogram" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/model/metadata" +) + +// NOTE(bwplotka): This file's code is tested in /prompb/rwcommon. + +// ToLabels return model labels.Labels from timeseries' remote labels. +func (m TimeSeries) ToLabels(b *labels.ScratchBuilder, symbols []string) labels.Labels { + return desymbolizeLabels(b, m.GetLabelsRefs(), symbols) +} + +// ToMetadata return model metadata from timeseries' remote metadata. +func (m TimeSeries) ToMetadata(symbols []string) metadata.Metadata { + typ := model.MetricTypeUnknown + switch m.Metadata.Type { + case Metadata_METRIC_TYPE_COUNTER: + typ = model.MetricTypeCounter + case Metadata_METRIC_TYPE_GAUGE: + typ = model.MetricTypeGauge + case Metadata_METRIC_TYPE_HISTOGRAM: + typ = model.MetricTypeHistogram + case Metadata_METRIC_TYPE_GAUGEHISTOGRAM: + typ = model.MetricTypeGaugeHistogram + case Metadata_METRIC_TYPE_SUMMARY: + typ = model.MetricTypeSummary + case Metadata_METRIC_TYPE_INFO: + typ = model.MetricTypeInfo + case Metadata_METRIC_TYPE_STATESET: + typ = model.MetricTypeStateset + } + return metadata.Metadata{ + Type: typ, + Unit: symbols[m.Metadata.UnitRef], + Help: symbols[m.Metadata.HelpRef], + } +} + +// FromMetadataType transforms a Prometheus metricType into writev2 metricType. +// Since the former is a string we need to transform it to an enum. +func FromMetadataType(t model.MetricType) Metadata_MetricType { + switch t { + case model.MetricTypeCounter: + return Metadata_METRIC_TYPE_COUNTER + case model.MetricTypeGauge: + return Metadata_METRIC_TYPE_GAUGE + case model.MetricTypeHistogram: + return Metadata_METRIC_TYPE_HISTOGRAM + case model.MetricTypeGaugeHistogram: + return Metadata_METRIC_TYPE_GAUGEHISTOGRAM + case model.MetricTypeSummary: + return Metadata_METRIC_TYPE_SUMMARY + case model.MetricTypeInfo: + return Metadata_METRIC_TYPE_INFO + case model.MetricTypeStateset: + return Metadata_METRIC_TYPE_STATESET + default: + return Metadata_METRIC_TYPE_UNSPECIFIED + } +} + +// IsFloatHistogram returns true if the histogram is float. +func (h Histogram) IsFloatHistogram() bool { + _, ok := h.GetCount().(*Histogram_CountFloat) + return ok +} + +// ToIntHistogram returns integer Prometheus histogram from the remote implementation +// of integer histogram. If it's a float histogram, the method returns nil. +func (h Histogram) ToIntHistogram() *histogram.Histogram { + if h.IsFloatHistogram() { + return nil + } + return &histogram.Histogram{ + CounterResetHint: histogram.CounterResetHint(h.ResetHint), + Schema: h.Schema, + ZeroThreshold: h.ZeroThreshold, + ZeroCount: h.GetZeroCountInt(), + Count: h.GetCountInt(), + Sum: h.Sum, + PositiveSpans: spansProtoToSpans(h.GetPositiveSpans()), + PositiveBuckets: h.GetPositiveDeltas(), + NegativeSpans: spansProtoToSpans(h.GetNegativeSpans()), + NegativeBuckets: h.GetNegativeDeltas(), + CustomValues: h.GetCustomValues(), + } +} + +// ToFloatHistogram returns float Prometheus histogram from the remote implementation +// of float histogram. If the underlying implementation is an integer histogram, a +// conversion is performed. +func (h Histogram) ToFloatHistogram() *histogram.FloatHistogram { + if h.IsFloatHistogram() { + return &histogram.FloatHistogram{ + CounterResetHint: histogram.CounterResetHint(h.ResetHint), + Schema: h.Schema, + ZeroThreshold: h.ZeroThreshold, + ZeroCount: h.GetZeroCountFloat(), + Count: h.GetCountFloat(), + Sum: h.Sum, + PositiveSpans: spansProtoToSpans(h.GetPositiveSpans()), + PositiveBuckets: h.GetPositiveCounts(), + NegativeSpans: spansProtoToSpans(h.GetNegativeSpans()), + NegativeBuckets: h.GetNegativeCounts(), + CustomValues: h.GetCustomValues(), + } + } + // Conversion from integer histogram. + return &histogram.FloatHistogram{ + CounterResetHint: histogram.CounterResetHint(h.ResetHint), + Schema: h.Schema, + ZeroThreshold: h.ZeroThreshold, + ZeroCount: float64(h.GetZeroCountInt()), + Count: float64(h.GetCountInt()), + Sum: h.Sum, + PositiveSpans: spansProtoToSpans(h.GetPositiveSpans()), + PositiveBuckets: deltasToCounts(h.GetPositiveDeltas()), + NegativeSpans: spansProtoToSpans(h.GetNegativeSpans()), + NegativeBuckets: deltasToCounts(h.GetNegativeDeltas()), + CustomValues: h.GetCustomValues(), + } +} + +func spansProtoToSpans(s []BucketSpan) []histogram.Span { + spans := make([]histogram.Span, len(s)) + for i := 0; i < len(s); i++ { + spans[i] = histogram.Span{Offset: s[i].Offset, Length: s[i].Length} + } + + return spans +} + +func deltasToCounts(deltas []int64) []float64 { + counts := make([]float64, len(deltas)) + var cur float64 + for i, d := range deltas { + cur += float64(d) + counts[i] = cur + } + return counts +} + +// FromIntHistogram returns remote Histogram from the integer Histogram. +func FromIntHistogram(timestamp int64, h *histogram.Histogram) Histogram { + return Histogram{ + Count: &Histogram_CountInt{CountInt: h.Count}, + Sum: h.Sum, + Schema: h.Schema, + ZeroThreshold: h.ZeroThreshold, + ZeroCount: &Histogram_ZeroCountInt{ZeroCountInt: h.ZeroCount}, + NegativeSpans: spansToSpansProto(h.NegativeSpans), + NegativeDeltas: h.NegativeBuckets, + PositiveSpans: spansToSpansProto(h.PositiveSpans), + PositiveDeltas: h.PositiveBuckets, + ResetHint: Histogram_ResetHint(h.CounterResetHint), + CustomValues: h.CustomValues, + Timestamp: timestamp, + } +} + +// FromFloatHistogram returns remote Histogram from the float Histogram. +func FromFloatHistogram(timestamp int64, fh *histogram.FloatHistogram) Histogram { + return Histogram{ + Count: &Histogram_CountFloat{CountFloat: fh.Count}, + Sum: fh.Sum, + Schema: fh.Schema, + ZeroThreshold: fh.ZeroThreshold, + ZeroCount: &Histogram_ZeroCountFloat{ZeroCountFloat: fh.ZeroCount}, + NegativeSpans: spansToSpansProto(fh.NegativeSpans), + NegativeCounts: fh.NegativeBuckets, + PositiveSpans: spansToSpansProto(fh.PositiveSpans), + PositiveCounts: fh.PositiveBuckets, + ResetHint: Histogram_ResetHint(fh.CounterResetHint), + CustomValues: fh.CustomValues, + Timestamp: timestamp, + } +} + +func spansToSpansProto(s []histogram.Span) []BucketSpan { + spans := make([]BucketSpan, len(s)) + for i := 0; i < len(s); i++ { + spans[i] = BucketSpan{Offset: s[i].Offset, Length: s[i].Length} + } + + return spans +} + +func (m Exemplar) ToExemplar(b *labels.ScratchBuilder, symbols []string) exemplar.Exemplar { + timestamp := m.Timestamp + + return exemplar.Exemplar{ + Labels: desymbolizeLabels(b, m.LabelsRefs, symbols), + Value: m.Value, + Ts: timestamp, + HasTs: timestamp != 0, + } +} diff --git a/vendor/github.com/prometheus/prometheus/prompb/io/prometheus/write/v2/custom.go b/vendor/github.com/prometheus/prometheus/prompb/io/prometheus/write/v2/custom.go new file mode 100644 index 0000000000..3aa778eb60 --- /dev/null +++ b/vendor/github.com/prometheus/prometheus/prompb/io/prometheus/write/v2/custom.go @@ -0,0 +1,165 @@ +// 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. + +package writev2 + +import ( + "slices" +) + +func (m Sample) T() int64 { return m.Timestamp } +func (m Sample) V() float64 { return m.Value } + +func (m *Request) OptimizedMarshal(dst []byte) ([]byte, error) { + siz := m.Size() + if cap(dst) < siz { + dst = make([]byte, siz) + } + n, err := m.OptimizedMarshalToSizedBuffer(dst[:siz]) + if err != nil { + return nil, err + } + return dst[:n], nil +} + +// OptimizedMarshalToSizedBuffer is mostly a copy of the generated MarshalToSizedBuffer, +// but calls OptimizedMarshalToSizedBuffer on the timeseries. +func (m *Request) OptimizedMarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if len(m.Timeseries) > 0 { + for iNdEx := len(m.Timeseries) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Timeseries[iNdEx].OptimizedMarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x2a + } + } + if len(m.Symbols) > 0 { + for iNdEx := len(m.Symbols) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Symbols[iNdEx]) + copy(dAtA[i:], m.Symbols[iNdEx]) + i = encodeVarintTypes(dAtA, i, uint64(len(m.Symbols[iNdEx]))) + i-- + dAtA[i] = 0x22 + } + } + return len(dAtA) - i, nil +} + +// OptimizedMarshalToSizedBuffer is mostly a copy of the generated MarshalToSizedBuffer, +// but marshals m.LabelsRefs in place without extra allocations. +func (m *TimeSeries) OptimizedMarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if m.CreatedTimestamp != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.CreatedTimestamp)) + i-- + dAtA[i] = 0x30 + } + { + size, err := m.Metadata.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x2a + if len(m.Histograms) > 0 { + for iNdEx := len(m.Histograms) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Histograms[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + } + if len(m.Exemplars) > 0 { + for iNdEx := len(m.Exemplars) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Exemplars[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x22 + } + } + if len(m.Samples) > 0 { + for iNdEx := len(m.Samples) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Samples[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + } + + if len(m.LabelsRefs) > 0 { + // This is the trick: encode the varints in reverse order to make it easier + // to do it in place. Then reverse the whole thing. + var j10 int + start := i + for _, num := range m.LabelsRefs { + for num >= 1<<7 { + dAtA[i-1] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + i-- + j10++ + } + dAtA[i-1] = uint8(num) + i-- + j10++ + } + slices.Reverse(dAtA[i:start]) + // --- end of trick + + i = encodeVarintTypes(dAtA, i, uint64(j10)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} diff --git a/vendor/github.com/prometheus/prometheus/prompb/io/prometheus/write/v2/symbols.go b/vendor/github.com/prometheus/prometheus/prompb/io/prometheus/write/v2/symbols.go new file mode 100644 index 0000000000..f316a976f2 --- /dev/null +++ b/vendor/github.com/prometheus/prometheus/prompb/io/prometheus/write/v2/symbols.go @@ -0,0 +1,83 @@ +// Copyright 2024 Prometheus Team +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package writev2 + +import "github.com/prometheus/prometheus/model/labels" + +// SymbolsTable implements table for easy symbol use. +type SymbolsTable struct { + strings []string + symbolsMap map[string]uint32 +} + +// NewSymbolTable returns a symbol table. +func NewSymbolTable() SymbolsTable { + return SymbolsTable{ + // Empty string is required as a first element. + symbolsMap: map[string]uint32{"": 0}, + strings: []string{""}, + } +} + +// Symbolize adds (if not added before) a string to the symbols table, +// while returning its reference number. +func (t *SymbolsTable) Symbolize(str string) uint32 { + if ref, ok := t.symbolsMap[str]; ok { + return ref + } + ref := uint32(len(t.strings)) + t.strings = append(t.strings, str) + t.symbolsMap[str] = ref + return ref +} + +// SymbolizeLabels symbolize Prometheus labels. +func (t *SymbolsTable) SymbolizeLabels(lbls labels.Labels, buf []uint32) []uint32 { + result := buf[:0] + lbls.Range(func(l labels.Label) { + off := t.Symbolize(l.Name) + result = append(result, off) + off = t.Symbolize(l.Value) + result = append(result, off) + }) + return result +} + +// Symbols returns computes symbols table to put in e.g. Request.Symbols. +// As per spec, order does not matter. +func (t *SymbolsTable) Symbols() []string { + return t.strings +} + +// Reset clears symbols table. +func (t *SymbolsTable) Reset() { + // NOTE: Make sure to keep empty symbol. + t.strings = t.strings[:1] + for k := range t.symbolsMap { + if k == "" { + continue + } + delete(t.symbolsMap, k) + } +} + +// desymbolizeLabels decodes label references, with given symbols to labels. +func desymbolizeLabels(b *labels.ScratchBuilder, labelRefs []uint32, symbols []string) labels.Labels { + b.Reset() + for i := 0; i < len(labelRefs); i += 2 { + b.Add(symbols[labelRefs[i]], symbols[labelRefs[i+1]]) + } + b.Sort() + return b.Labels() +} diff --git a/vendor/github.com/prometheus/prometheus/prompb/io/prometheus/write/v2/types.pb.go b/vendor/github.com/prometheus/prometheus/prompb/io/prometheus/write/v2/types.pb.go new file mode 100644 index 0000000000..d6ea8398f7 --- /dev/null +++ b/vendor/github.com/prometheus/prometheus/prompb/io/prometheus/write/v2/types.pb.go @@ -0,0 +1,3241 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: io/prometheus/write/v2/types.proto + +package writev2 + +import ( + encoding_binary "encoding/binary" + fmt "fmt" + _ "github.com/gogo/protobuf/gogoproto" + proto "github.com/gogo/protobuf/proto" + io "io" + math "math" + math_bits "math/bits" +) + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package + +type Metadata_MetricType int32 + +const ( + Metadata_METRIC_TYPE_UNSPECIFIED Metadata_MetricType = 0 + Metadata_METRIC_TYPE_COUNTER Metadata_MetricType = 1 + Metadata_METRIC_TYPE_GAUGE Metadata_MetricType = 2 + Metadata_METRIC_TYPE_HISTOGRAM Metadata_MetricType = 3 + Metadata_METRIC_TYPE_GAUGEHISTOGRAM Metadata_MetricType = 4 + Metadata_METRIC_TYPE_SUMMARY Metadata_MetricType = 5 + Metadata_METRIC_TYPE_INFO Metadata_MetricType = 6 + Metadata_METRIC_TYPE_STATESET Metadata_MetricType = 7 +) + +var Metadata_MetricType_name = map[int32]string{ + 0: "METRIC_TYPE_UNSPECIFIED", + 1: "METRIC_TYPE_COUNTER", + 2: "METRIC_TYPE_GAUGE", + 3: "METRIC_TYPE_HISTOGRAM", + 4: "METRIC_TYPE_GAUGEHISTOGRAM", + 5: "METRIC_TYPE_SUMMARY", + 6: "METRIC_TYPE_INFO", + 7: "METRIC_TYPE_STATESET", +} + +var Metadata_MetricType_value = map[string]int32{ + "METRIC_TYPE_UNSPECIFIED": 0, + "METRIC_TYPE_COUNTER": 1, + "METRIC_TYPE_GAUGE": 2, + "METRIC_TYPE_HISTOGRAM": 3, + "METRIC_TYPE_GAUGEHISTOGRAM": 4, + "METRIC_TYPE_SUMMARY": 5, + "METRIC_TYPE_INFO": 6, + "METRIC_TYPE_STATESET": 7, +} + +func (x Metadata_MetricType) String() string { + return proto.EnumName(Metadata_MetricType_name, int32(x)) +} + +func (Metadata_MetricType) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_f139519efd9fa8d7, []int{4, 0} +} + +type Histogram_ResetHint int32 + +const ( + Histogram_RESET_HINT_UNSPECIFIED Histogram_ResetHint = 0 + Histogram_RESET_HINT_YES Histogram_ResetHint = 1 + Histogram_RESET_HINT_NO Histogram_ResetHint = 2 + Histogram_RESET_HINT_GAUGE Histogram_ResetHint = 3 +) + +var Histogram_ResetHint_name = map[int32]string{ + 0: "RESET_HINT_UNSPECIFIED", + 1: "RESET_HINT_YES", + 2: "RESET_HINT_NO", + 3: "RESET_HINT_GAUGE", +} + +var Histogram_ResetHint_value = map[string]int32{ + "RESET_HINT_UNSPECIFIED": 0, + "RESET_HINT_YES": 1, + "RESET_HINT_NO": 2, + "RESET_HINT_GAUGE": 3, +} + +func (x Histogram_ResetHint) String() string { + return proto.EnumName(Histogram_ResetHint_name, int32(x)) +} + +func (Histogram_ResetHint) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_f139519efd9fa8d7, []int{5, 0} +} + +// Request represents a request to write the given timeseries to a remote destination. +// This message was introduced in the Remote Write 2.0 specification: +// https://prometheus.io/docs/concepts/remote_write_spec_2_0/ +// +// The canonical Content-Type request header value for this message is +// "application/x-protobuf;proto=io.prometheus.write.v2.Request" +// +// NOTE: gogoproto options might change in future for this file, they +// are not part of the spec proto (they only modify the generated Go code, not +// the serialized message). See: https://github.com/prometheus/prometheus/issues/11908 +type Request struct { + // symbols contains a de-duplicated array of string elements used for various + // items in a Request message, like labels and metadata items. For the sender's convenience + // around empty values for optional fields like unit_ref, symbols array MUST start with + // empty string. + // + // To decode each of the symbolized strings, referenced, by "ref(s)" suffix, you + // need to lookup the actual string by index from symbols array. The order of + // strings is up to the sender. The receiver should not assume any particular encoding. + Symbols []string `protobuf:"bytes,4,rep,name=symbols,proto3" json:"symbols,omitempty"` + // timeseries represents an array of distinct series with 0 or more samples. + Timeseries []TimeSeries `protobuf:"bytes,5,rep,name=timeseries,proto3" json:"timeseries"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Request) Reset() { *m = Request{} } +func (m *Request) String() string { return proto.CompactTextString(m) } +func (*Request) ProtoMessage() {} +func (*Request) Descriptor() ([]byte, []int) { + return fileDescriptor_f139519efd9fa8d7, []int{0} +} +func (m *Request) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Request) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Request.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *Request) XXX_Merge(src proto.Message) { + xxx_messageInfo_Request.Merge(m, src) +} +func (m *Request) XXX_Size() int { + return m.Size() +} +func (m *Request) XXX_DiscardUnknown() { + xxx_messageInfo_Request.DiscardUnknown(m) +} + +var xxx_messageInfo_Request proto.InternalMessageInfo + +func (m *Request) GetSymbols() []string { + if m != nil { + return m.Symbols + } + return nil +} + +func (m *Request) GetTimeseries() []TimeSeries { + if m != nil { + return m.Timeseries + } + return nil +} + +// TimeSeries represents a single series. +type TimeSeries struct { + // labels_refs is a list of label name-value pair references, encoded + // as indices to the Request.symbols array. This list's length is always + // a multiple of two, and the underlying labels should be sorted lexicographically. + // + // Note that there might be multiple TimeSeries objects in the same + // Requests with the same labels e.g. for different exemplars, metadata + // or created timestamp. + LabelsRefs []uint32 `protobuf:"varint,1,rep,packed,name=labels_refs,json=labelsRefs,proto3" json:"labels_refs,omitempty"` + // Timeseries messages can either specify samples or (native) histogram samples + // (histogram field), but not both. For a typical sender (real-time metric + // streaming), in healthy cases, there will be only one sample or histogram. + // + // Samples and histograms are sorted by timestamp (older first). + Samples []Sample `protobuf:"bytes,2,rep,name=samples,proto3" json:"samples"` + Histograms []Histogram `protobuf:"bytes,3,rep,name=histograms,proto3" json:"histograms"` + // exemplars represents an optional set of exemplars attached to this series' samples. + Exemplars []Exemplar `protobuf:"bytes,4,rep,name=exemplars,proto3" json:"exemplars"` + // metadata represents the metadata associated with the given series' samples. + Metadata Metadata `protobuf:"bytes,5,opt,name=metadata,proto3" json:"metadata"` + // created_timestamp represents an optional created timestamp associated with + // this series' samples in ms format, typically for counter or histogram type + // metrics. Created timestamp represents the time when the counter started + // counting (sometimes referred to as start timestamp), which can increase + // the accuracy of query results. + // + // Note that some receivers might require this and in return fail to + // ingest such samples within the Request. + // + // For Go, see github.com/prometheus/prometheus/model/timestamp/timestamp.go + // for conversion from/to time.Time to Prometheus timestamp. + // + // Note that the "optional" keyword is omitted due to + // https://cloud.google.com/apis/design/design_patterns.md#optional_primitive_fields + // Zero value means value not set. If you need to use exactly zero value for + // the timestamp, use 1 millisecond before or after. + CreatedTimestamp int64 `protobuf:"varint,6,opt,name=created_timestamp,json=createdTimestamp,proto3" json:"created_timestamp,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *TimeSeries) Reset() { *m = TimeSeries{} } +func (m *TimeSeries) String() string { return proto.CompactTextString(m) } +func (*TimeSeries) ProtoMessage() {} +func (*TimeSeries) Descriptor() ([]byte, []int) { + return fileDescriptor_f139519efd9fa8d7, []int{1} +} +func (m *TimeSeries) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *TimeSeries) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_TimeSeries.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *TimeSeries) XXX_Merge(src proto.Message) { + xxx_messageInfo_TimeSeries.Merge(m, src) +} +func (m *TimeSeries) XXX_Size() int { + return m.Size() +} +func (m *TimeSeries) XXX_DiscardUnknown() { + xxx_messageInfo_TimeSeries.DiscardUnknown(m) +} + +var xxx_messageInfo_TimeSeries proto.InternalMessageInfo + +func (m *TimeSeries) GetLabelsRefs() []uint32 { + if m != nil { + return m.LabelsRefs + } + return nil +} + +func (m *TimeSeries) GetSamples() []Sample { + if m != nil { + return m.Samples + } + return nil +} + +func (m *TimeSeries) GetHistograms() []Histogram { + if m != nil { + return m.Histograms + } + return nil +} + +func (m *TimeSeries) GetExemplars() []Exemplar { + if m != nil { + return m.Exemplars + } + return nil +} + +func (m *TimeSeries) GetMetadata() Metadata { + if m != nil { + return m.Metadata + } + return Metadata{} +} + +func (m *TimeSeries) GetCreatedTimestamp() int64 { + if m != nil { + return m.CreatedTimestamp + } + return 0 +} + +// Exemplar is an additional information attached to some series' samples. +// It is typically used to attach an example trace or request ID associated with +// the metric changes. +type Exemplar struct { + // labels_refs is an optional list of label name-value pair references, encoded + // as indices to the Request.symbols array. This list's len is always + // a multiple of 2, and the underlying labels should be sorted lexicographically. + // If the exemplar references a trace it should use the `trace_id` label name, as a best practice. + LabelsRefs []uint32 `protobuf:"varint,1,rep,packed,name=labels_refs,json=labelsRefs,proto3" json:"labels_refs,omitempty"` + // value represents an exact example value. This can be useful when the exemplar + // is attached to a histogram, which only gives an estimated value through buckets. + Value float64 `protobuf:"fixed64,2,opt,name=value,proto3" json:"value,omitempty"` + // timestamp represents an optional timestamp of the sample in ms. + // + // For Go, see github.com/prometheus/prometheus/model/timestamp/timestamp.go + // for conversion from/to time.Time to Prometheus timestamp. + // + // Note that the "optional" keyword is omitted due to + // https://cloud.google.com/apis/design/design_patterns.md#optional_primitive_fields + // Zero value means value not set. If you need to use exactly zero value for + // the timestamp, use 1 millisecond before or after. + Timestamp int64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Exemplar) Reset() { *m = Exemplar{} } +func (m *Exemplar) String() string { return proto.CompactTextString(m) } +func (*Exemplar) ProtoMessage() {} +func (*Exemplar) Descriptor() ([]byte, []int) { + return fileDescriptor_f139519efd9fa8d7, []int{2} +} +func (m *Exemplar) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Exemplar) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Exemplar.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *Exemplar) XXX_Merge(src proto.Message) { + xxx_messageInfo_Exemplar.Merge(m, src) +} +func (m *Exemplar) XXX_Size() int { + return m.Size() +} +func (m *Exemplar) XXX_DiscardUnknown() { + xxx_messageInfo_Exemplar.DiscardUnknown(m) +} + +var xxx_messageInfo_Exemplar proto.InternalMessageInfo + +func (m *Exemplar) GetLabelsRefs() []uint32 { + if m != nil { + return m.LabelsRefs + } + return nil +} + +func (m *Exemplar) GetValue() float64 { + if m != nil { + return m.Value + } + return 0 +} + +func (m *Exemplar) GetTimestamp() int64 { + if m != nil { + return m.Timestamp + } + return 0 +} + +// Sample represents series sample. +type Sample struct { + // value of the sample. + Value float64 `protobuf:"fixed64,1,opt,name=value,proto3" json:"value,omitempty"` + // timestamp represents timestamp of the sample in ms. + // + // For Go, see github.com/prometheus/prometheus/model/timestamp/timestamp.go + // for conversion from/to time.Time to Prometheus timestamp. + Timestamp int64 `protobuf:"varint,2,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Sample) Reset() { *m = Sample{} } +func (m *Sample) String() string { return proto.CompactTextString(m) } +func (*Sample) ProtoMessage() {} +func (*Sample) Descriptor() ([]byte, []int) { + return fileDescriptor_f139519efd9fa8d7, []int{3} +} +func (m *Sample) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Sample) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Sample.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *Sample) XXX_Merge(src proto.Message) { + xxx_messageInfo_Sample.Merge(m, src) +} +func (m *Sample) XXX_Size() int { + return m.Size() +} +func (m *Sample) XXX_DiscardUnknown() { + xxx_messageInfo_Sample.DiscardUnknown(m) +} + +var xxx_messageInfo_Sample proto.InternalMessageInfo + +func (m *Sample) GetValue() float64 { + if m != nil { + return m.Value + } + return 0 +} + +func (m *Sample) GetTimestamp() int64 { + if m != nil { + return m.Timestamp + } + return 0 +} + +// Metadata represents the metadata associated with the given series' samples. +type Metadata struct { + Type Metadata_MetricType `protobuf:"varint,1,opt,name=type,proto3,enum=io.prometheus.write.v2.Metadata_MetricType" json:"type,omitempty"` + // help_ref is a reference to the Request.symbols array representing help + // text for the metric. Help is optional, reference should point to an empty string in + // such a case. + HelpRef uint32 `protobuf:"varint,3,opt,name=help_ref,json=helpRef,proto3" json:"help_ref,omitempty"` + // unit_ref is a reference to the Request.symbols array representing a unit + // for the metric. Unit is optional, reference should point to an empty string in + // such a case. + UnitRef uint32 `protobuf:"varint,4,opt,name=unit_ref,json=unitRef,proto3" json:"unit_ref,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Metadata) Reset() { *m = Metadata{} } +func (m *Metadata) String() string { return proto.CompactTextString(m) } +func (*Metadata) ProtoMessage() {} +func (*Metadata) Descriptor() ([]byte, []int) { + return fileDescriptor_f139519efd9fa8d7, []int{4} +} +func (m *Metadata) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Metadata) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Metadata.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *Metadata) XXX_Merge(src proto.Message) { + xxx_messageInfo_Metadata.Merge(m, src) +} +func (m *Metadata) XXX_Size() int { + return m.Size() +} +func (m *Metadata) XXX_DiscardUnknown() { + xxx_messageInfo_Metadata.DiscardUnknown(m) +} + +var xxx_messageInfo_Metadata proto.InternalMessageInfo + +func (m *Metadata) GetType() Metadata_MetricType { + if m != nil { + return m.Type + } + return Metadata_METRIC_TYPE_UNSPECIFIED +} + +func (m *Metadata) GetHelpRef() uint32 { + if m != nil { + return m.HelpRef + } + return 0 +} + +func (m *Metadata) GetUnitRef() uint32 { + if m != nil { + return m.UnitRef + } + return 0 +} + +// A native histogram, also known as a sparse histogram. +// Original design doc: +// https://docs.google.com/document/d/1cLNv3aufPZb3fNfaJgdaRBZsInZKKIHo9E6HinJVbpM/edit +// The appendix of this design doc also explains the concept of float +// histograms. This Histogram message can represent both, the usual +// integer histogram as well as a float histogram. +type Histogram struct { + // Types that are valid to be assigned to Count: + // + // *Histogram_CountInt + // *Histogram_CountFloat + Count isHistogram_Count `protobuf_oneof:"count"` + Sum float64 `protobuf:"fixed64,3,opt,name=sum,proto3" json:"sum,omitempty"` + // The schema defines the bucket schema. Currently, valid numbers + // are -53 and numbers in range of -4 <= n <= 8. More valid numbers might be + // added in future for new bucketing layouts. + // + // The schema equal to -53 means custom buckets. See + // custom_values field description for more details. + // + // Values between -4 and 8 represent base-2 bucket schema, where 1 + // is a bucket boundary in each case, and then each power of two is + // divided into 2^n (n is schema value) logarithmic buckets. Or in other words, + // each bucket boundary is the previous boundary times 2^(2^-n). + Schema int32 `protobuf:"zigzag32,4,opt,name=schema,proto3" json:"schema,omitempty"` + ZeroThreshold float64 `protobuf:"fixed64,5,opt,name=zero_threshold,json=zeroThreshold,proto3" json:"zero_threshold,omitempty"` + // Types that are valid to be assigned to ZeroCount: + // + // *Histogram_ZeroCountInt + // *Histogram_ZeroCountFloat + ZeroCount isHistogram_ZeroCount `protobuf_oneof:"zero_count"` + // Negative Buckets. + NegativeSpans []BucketSpan `protobuf:"bytes,8,rep,name=negative_spans,json=negativeSpans,proto3" json:"negative_spans"` + // Use either "negative_deltas" or "negative_counts", the former for + // regular histograms with integer counts, the latter for + // float histograms. + NegativeDeltas []int64 `protobuf:"zigzag64,9,rep,packed,name=negative_deltas,json=negativeDeltas,proto3" json:"negative_deltas,omitempty"` + NegativeCounts []float64 `protobuf:"fixed64,10,rep,packed,name=negative_counts,json=negativeCounts,proto3" json:"negative_counts,omitempty"` + // Positive Buckets. + // + // In case of custom buckets (-53 schema value) the positive buckets are interpreted as follows: + // * The span offset+length points to an the index of the custom_values array + // or +Inf if pointing to the len of the array. + // * The counts and deltas have the same meaning as for exponential histograms. + PositiveSpans []BucketSpan `protobuf:"bytes,11,rep,name=positive_spans,json=positiveSpans,proto3" json:"positive_spans"` + // Use either "positive_deltas" or "positive_counts", the former for + // regular histograms with integer counts, the latter for + // float histograms. + PositiveDeltas []int64 `protobuf:"zigzag64,12,rep,packed,name=positive_deltas,json=positiveDeltas,proto3" json:"positive_deltas,omitempty"` + PositiveCounts []float64 `protobuf:"fixed64,13,rep,packed,name=positive_counts,json=positiveCounts,proto3" json:"positive_counts,omitempty"` + ResetHint Histogram_ResetHint `protobuf:"varint,14,opt,name=reset_hint,json=resetHint,proto3,enum=io.prometheus.write.v2.Histogram_ResetHint" json:"reset_hint,omitempty"` + // timestamp represents timestamp of the sample in ms. + // + // For Go, see github.com/prometheus/prometheus/model/timestamp/timestamp.go + // for conversion from/to time.Time to Prometheus timestamp. + Timestamp int64 `protobuf:"varint,15,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + // custom_values is an additional field used by non-exponential bucketing layouts. + // + // For custom buckets (-53 schema value) custom_values specify monotonically + // increasing upper inclusive boundaries for the bucket counts with arbitrary + // widths for this histogram. In other words, custom_values represents custom, + // explicit bucketing that could have been converted from the classic histograms. + // + // Those bounds are then referenced by spans in positive_spans with corresponding positive + // counts of deltas (refer to positive_spans for more details). This way we can + // have encode sparse histograms with custom bucketing (many buckets are often + // not used). + // + // Note that for custom bounds, even negative observations are placed in the positive + // counts to simplify the implementation and avoid ambiguity of where to place + // an underflow bucket, e.g. (-2, 1]. Therefore negative buckets and + // the zero bucket are unused, if the schema indicates custom bucketing. + // + // For each upper boundary the previous boundary represent the lower exclusive + // boundary for that bucket. The first element is the upper inclusive boundary + // for the first bucket, which implicitly has a lower inclusive bound of -Inf. + // This is similar to "le" label semantics on classic histograms. You may add a + // bucket with an upper bound of 0 to make sure that you really have no negative + // observations, but in practice, native histogram rendering will show both with + // or without first upper boundary 0 and no negative counts as the same case. + // + // The last element is not only the upper inclusive bound of the last regular + // bucket, but implicitly the lower exclusive bound of the +Inf bucket. + CustomValues []float64 `protobuf:"fixed64,16,rep,packed,name=custom_values,json=customValues,proto3" json:"custom_values,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Histogram) Reset() { *m = Histogram{} } +func (m *Histogram) String() string { return proto.CompactTextString(m) } +func (*Histogram) ProtoMessage() {} +func (*Histogram) Descriptor() ([]byte, []int) { + return fileDescriptor_f139519efd9fa8d7, []int{5} +} +func (m *Histogram) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *Histogram) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_Histogram.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *Histogram) XXX_Merge(src proto.Message) { + xxx_messageInfo_Histogram.Merge(m, src) +} +func (m *Histogram) XXX_Size() int { + return m.Size() +} +func (m *Histogram) XXX_DiscardUnknown() { + xxx_messageInfo_Histogram.DiscardUnknown(m) +} + +var xxx_messageInfo_Histogram proto.InternalMessageInfo + +type isHistogram_Count interface { + isHistogram_Count() + MarshalTo([]byte) (int, error) + Size() int +} +type isHistogram_ZeroCount interface { + isHistogram_ZeroCount() + MarshalTo([]byte) (int, error) + Size() int +} + +type Histogram_CountInt struct { + CountInt uint64 `protobuf:"varint,1,opt,name=count_int,json=countInt,proto3,oneof" json:"count_int,omitempty"` +} +type Histogram_CountFloat struct { + CountFloat float64 `protobuf:"fixed64,2,opt,name=count_float,json=countFloat,proto3,oneof" json:"count_float,omitempty"` +} +type Histogram_ZeroCountInt struct { + ZeroCountInt uint64 `protobuf:"varint,6,opt,name=zero_count_int,json=zeroCountInt,proto3,oneof" json:"zero_count_int,omitempty"` +} +type Histogram_ZeroCountFloat struct { + ZeroCountFloat float64 `protobuf:"fixed64,7,opt,name=zero_count_float,json=zeroCountFloat,proto3,oneof" json:"zero_count_float,omitempty"` +} + +func (*Histogram_CountInt) isHistogram_Count() {} +func (*Histogram_CountFloat) isHistogram_Count() {} +func (*Histogram_ZeroCountInt) isHistogram_ZeroCount() {} +func (*Histogram_ZeroCountFloat) isHistogram_ZeroCount() {} + +func (m *Histogram) GetCount() isHistogram_Count { + if m != nil { + return m.Count + } + return nil +} +func (m *Histogram) GetZeroCount() isHistogram_ZeroCount { + if m != nil { + return m.ZeroCount + } + return nil +} + +func (m *Histogram) GetCountInt() uint64 { + if x, ok := m.GetCount().(*Histogram_CountInt); ok { + return x.CountInt + } + return 0 +} + +func (m *Histogram) GetCountFloat() float64 { + if x, ok := m.GetCount().(*Histogram_CountFloat); ok { + return x.CountFloat + } + return 0 +} + +func (m *Histogram) GetSum() float64 { + if m != nil { + return m.Sum + } + return 0 +} + +func (m *Histogram) GetSchema() int32 { + if m != nil { + return m.Schema + } + return 0 +} + +func (m *Histogram) GetZeroThreshold() float64 { + if m != nil { + return m.ZeroThreshold + } + return 0 +} + +func (m *Histogram) GetZeroCountInt() uint64 { + if x, ok := m.GetZeroCount().(*Histogram_ZeroCountInt); ok { + return x.ZeroCountInt + } + return 0 +} + +func (m *Histogram) GetZeroCountFloat() float64 { + if x, ok := m.GetZeroCount().(*Histogram_ZeroCountFloat); ok { + return x.ZeroCountFloat + } + return 0 +} + +func (m *Histogram) GetNegativeSpans() []BucketSpan { + if m != nil { + return m.NegativeSpans + } + return nil +} + +func (m *Histogram) GetNegativeDeltas() []int64 { + if m != nil { + return m.NegativeDeltas + } + return nil +} + +func (m *Histogram) GetNegativeCounts() []float64 { + if m != nil { + return m.NegativeCounts + } + return nil +} + +func (m *Histogram) GetPositiveSpans() []BucketSpan { + if m != nil { + return m.PositiveSpans + } + return nil +} + +func (m *Histogram) GetPositiveDeltas() []int64 { + if m != nil { + return m.PositiveDeltas + } + return nil +} + +func (m *Histogram) GetPositiveCounts() []float64 { + if m != nil { + return m.PositiveCounts + } + return nil +} + +func (m *Histogram) GetResetHint() Histogram_ResetHint { + if m != nil { + return m.ResetHint + } + return Histogram_RESET_HINT_UNSPECIFIED +} + +func (m *Histogram) GetTimestamp() int64 { + if m != nil { + return m.Timestamp + } + return 0 +} + +func (m *Histogram) GetCustomValues() []float64 { + if m != nil { + return m.CustomValues + } + return nil +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*Histogram) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*Histogram_CountInt)(nil), + (*Histogram_CountFloat)(nil), + (*Histogram_ZeroCountInt)(nil), + (*Histogram_ZeroCountFloat)(nil), + } +} + +// A BucketSpan defines a number of consecutive buckets with their +// offset. Logically, it would be more straightforward to include the +// bucket counts in the Span. However, the protobuf representation is +// more compact in the way the data is structured here (with all the +// buckets in a single array separate from the Spans). +type BucketSpan struct { + Offset int32 `protobuf:"zigzag32,1,opt,name=offset,proto3" json:"offset,omitempty"` + Length uint32 `protobuf:"varint,2,opt,name=length,proto3" json:"length,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *BucketSpan) Reset() { *m = BucketSpan{} } +func (m *BucketSpan) String() string { return proto.CompactTextString(m) } +func (*BucketSpan) ProtoMessage() {} +func (*BucketSpan) Descriptor() ([]byte, []int) { + return fileDescriptor_f139519efd9fa8d7, []int{6} +} +func (m *BucketSpan) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *BucketSpan) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_BucketSpan.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *BucketSpan) XXX_Merge(src proto.Message) { + xxx_messageInfo_BucketSpan.Merge(m, src) +} +func (m *BucketSpan) XXX_Size() int { + return m.Size() +} +func (m *BucketSpan) XXX_DiscardUnknown() { + xxx_messageInfo_BucketSpan.DiscardUnknown(m) +} + +var xxx_messageInfo_BucketSpan proto.InternalMessageInfo + +func (m *BucketSpan) GetOffset() int32 { + if m != nil { + return m.Offset + } + return 0 +} + +func (m *BucketSpan) GetLength() uint32 { + if m != nil { + return m.Length + } + return 0 +} + +func init() { + proto.RegisterEnum("io.prometheus.write.v2.Metadata_MetricType", Metadata_MetricType_name, Metadata_MetricType_value) + proto.RegisterEnum("io.prometheus.write.v2.Histogram_ResetHint", Histogram_ResetHint_name, Histogram_ResetHint_value) + proto.RegisterType((*Request)(nil), "io.prometheus.write.v2.Request") + proto.RegisterType((*TimeSeries)(nil), "io.prometheus.write.v2.TimeSeries") + proto.RegisterType((*Exemplar)(nil), "io.prometheus.write.v2.Exemplar") + proto.RegisterType((*Sample)(nil), "io.prometheus.write.v2.Sample") + proto.RegisterType((*Metadata)(nil), "io.prometheus.write.v2.Metadata") + proto.RegisterType((*Histogram)(nil), "io.prometheus.write.v2.Histogram") + proto.RegisterType((*BucketSpan)(nil), "io.prometheus.write.v2.BucketSpan") +} + +func init() { + proto.RegisterFile("io/prometheus/write/v2/types.proto", fileDescriptor_f139519efd9fa8d7) +} + +var fileDescriptor_f139519efd9fa8d7 = []byte{ + // 926 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x55, 0x5d, 0x6f, 0xe3, 0x44, + 0x14, 0xed, 0xc4, 0x69, 0x3e, 0x6e, 0x9a, 0xac, 0x33, 0xb4, 0x5d, 0x6f, 0x81, 0x6c, 0xd6, 0x08, + 0x88, 0x58, 0x29, 0x91, 0xc2, 0xeb, 0x0a, 0xd4, 0xb4, 0x6e, 0x93, 0x95, 0x92, 0xac, 0x26, 0x2e, + 0x52, 0x79, 0xb1, 0xdc, 0x64, 0x92, 0x58, 0xd8, 0xb1, 0xf1, 0x4c, 0x02, 0xe5, 0xf7, 0xf1, 0xb0, + 0x8f, 0xfc, 0x01, 0x10, 0xf4, 0x9d, 0xff, 0x80, 0x66, 0xfc, 0xd9, 0x42, 0xbb, 0xe2, 0x6d, 0xe6, + 0xdc, 0x73, 0xee, 0x3d, 0xb9, 0xbe, 0x77, 0x02, 0xba, 0xe3, 0xf7, 0x82, 0xd0, 0xf7, 0x28, 0x5f, + 0xd3, 0x2d, 0xeb, 0xfd, 0x14, 0x3a, 0x9c, 0xf6, 0x76, 0xfd, 0x1e, 0xbf, 0x0d, 0x28, 0xeb, 0x06, + 0xa1, 0xcf, 0x7d, 0x7c, 0xec, 0xf8, 0xdd, 0x8c, 0xd3, 0x95, 0x9c, 0xee, 0xae, 0x7f, 0x72, 0xb8, + 0xf2, 0x57, 0xbe, 0xa4, 0xf4, 0xc4, 0x29, 0x62, 0xeb, 0x0c, 0xca, 0x84, 0xfe, 0xb8, 0xa5, 0x8c, + 0x63, 0x0d, 0xca, 0xec, 0xd6, 0xbb, 0xf1, 0x5d, 0xa6, 0x15, 0xdb, 0x4a, 0xa7, 0x4a, 0x92, 0x2b, + 0x1e, 0x02, 0x70, 0xc7, 0xa3, 0x8c, 0x86, 0x0e, 0x65, 0xda, 0x7e, 0x5b, 0xe9, 0xd4, 0xfa, 0x7a, + 0xf7, 0xbf, 0xeb, 0x74, 0x4d, 0xc7, 0xa3, 0x33, 0xc9, 0x1c, 0x14, 0xdf, 0xff, 0xf1, 0x72, 0x8f, + 0xe4, 0xb4, 0x6f, 0x8b, 0x15, 0xa4, 0x16, 0xf5, 0xbf, 0x0b, 0x00, 0x19, 0x0d, 0xbf, 0x84, 0x9a, + 0x6b, 0xdf, 0x50, 0x97, 0x59, 0x21, 0x5d, 0x32, 0x0d, 0xb5, 0x95, 0x4e, 0x9d, 0x40, 0x04, 0x11, + 0xba, 0x64, 0xf8, 0x1b, 0x28, 0x33, 0xdb, 0x0b, 0x5c, 0xca, 0xb4, 0x82, 0x2c, 0xde, 0x7a, 0xac, + 0xf8, 0x4c, 0xd2, 0xe2, 0xc2, 0x89, 0x08, 0x5f, 0x02, 0xac, 0x1d, 0xc6, 0xfd, 0x55, 0x68, 0x7b, + 0x4c, 0x53, 0x64, 0x8a, 0x57, 0x8f, 0xa5, 0x18, 0x26, 0xcc, 0xc4, 0x7e, 0x26, 0xc5, 0xe7, 0x50, + 0xa5, 0x3f, 0x53, 0x2f, 0x70, 0xed, 0x30, 0x6a, 0x52, 0xad, 0xdf, 0x7e, 0x2c, 0x8f, 0x11, 0x13, + 0xe3, 0x34, 0x99, 0x10, 0x0f, 0xa0, 0xe2, 0x51, 0x6e, 0x2f, 0x6c, 0x6e, 0x6b, 0xfb, 0x6d, 0xf4, + 0x54, 0x92, 0x71, 0xcc, 0x8b, 0x93, 0xa4, 0x3a, 0xfc, 0x1a, 0x9a, 0xf3, 0x90, 0xda, 0x9c, 0x2e, + 0x2c, 0xd9, 0x5e, 0x6e, 0x7b, 0x81, 0x56, 0x6a, 0xa3, 0x8e, 0x42, 0xd4, 0x38, 0x60, 0x26, 0xb8, + 0x6e, 0x41, 0x25, 0x71, 0xf3, 0xe1, 0x66, 0x1f, 0xc2, 0xfe, 0xce, 0x76, 0xb7, 0x54, 0x2b, 0xb4, + 0x51, 0x07, 0x91, 0xe8, 0x82, 0x3f, 0x81, 0x6a, 0x56, 0x47, 0x91, 0x75, 0x32, 0x40, 0x7f, 0x03, + 0xa5, 0xa8, 0xf3, 0x99, 0x1a, 0x3d, 0xaa, 0x2e, 0x3c, 0x54, 0xff, 0x55, 0x80, 0x4a, 0xf2, 0x43, + 0xf1, 0xb7, 0x50, 0x14, 0xd3, 0x2c, 0xf5, 0x8d, 0xfe, 0xeb, 0x0f, 0x35, 0x46, 0x1c, 0x42, 0x67, + 0x6e, 0xde, 0x06, 0x94, 0x48, 0x21, 0x7e, 0x01, 0x95, 0x35, 0x75, 0x03, 0xf1, 0xf3, 0xa4, 0xd1, + 0x3a, 0x29, 0x8b, 0x3b, 0xa1, 0x4b, 0x11, 0xda, 0x6e, 0x1c, 0x2e, 0x43, 0xc5, 0x28, 0x24, 0xee, + 0x84, 0x2e, 0xf5, 0xdf, 0x11, 0x40, 0x96, 0x0a, 0x7f, 0x0c, 0xcf, 0xc7, 0x86, 0x49, 0x46, 0x67, + 0x96, 0x79, 0xfd, 0xce, 0xb0, 0xae, 0x26, 0xb3, 0x77, 0xc6, 0xd9, 0xe8, 0x62, 0x64, 0x9c, 0xab, + 0x7b, 0xf8, 0x39, 0x7c, 0x94, 0x0f, 0x9e, 0x4d, 0xaf, 0x26, 0xa6, 0x41, 0x54, 0x84, 0x8f, 0xa0, + 0x99, 0x0f, 0x5c, 0x9e, 0x5e, 0x5d, 0x1a, 0x6a, 0x01, 0xbf, 0x80, 0xa3, 0x3c, 0x3c, 0x1c, 0xcd, + 0xcc, 0xe9, 0x25, 0x39, 0x1d, 0xab, 0x0a, 0x6e, 0xc1, 0xc9, 0xbf, 0x14, 0x59, 0xbc, 0xf8, 0xb0, + 0xd4, 0xec, 0x6a, 0x3c, 0x3e, 0x25, 0xd7, 0xea, 0x3e, 0x3e, 0x04, 0x35, 0x1f, 0x18, 0x4d, 0x2e, + 0xa6, 0x6a, 0x09, 0x6b, 0x70, 0x78, 0x8f, 0x6e, 0x9e, 0x9a, 0xc6, 0xcc, 0x30, 0xd5, 0xb2, 0xfe, + 0x6b, 0x09, 0xaa, 0xe9, 0x64, 0xe3, 0x4f, 0xa1, 0x3a, 0xf7, 0xb7, 0x1b, 0x6e, 0x39, 0x1b, 0x2e, + 0x3b, 0x5d, 0x1c, 0xee, 0x91, 0x8a, 0x84, 0x46, 0x1b, 0x8e, 0x5f, 0x41, 0x2d, 0x0a, 0x2f, 0x5d, + 0xdf, 0xe6, 0xd1, 0x20, 0x0c, 0xf7, 0x08, 0x48, 0xf0, 0x42, 0x60, 0x58, 0x05, 0x85, 0x6d, 0x3d, + 0xd9, 0x60, 0x44, 0xc4, 0x11, 0x1f, 0x43, 0x89, 0xcd, 0xd7, 0xd4, 0xb3, 0x65, 0x6b, 0x9b, 0x24, + 0xbe, 0xe1, 0xcf, 0xa1, 0xf1, 0x0b, 0x0d, 0x7d, 0x8b, 0xaf, 0x43, 0xca, 0xd6, 0xbe, 0xbb, 0x90, + 0x33, 0x8f, 0x48, 0x5d, 0xa0, 0x66, 0x02, 0xe2, 0x2f, 0x62, 0x5a, 0xe6, 0xab, 0x24, 0x7d, 0x21, + 0x72, 0x20, 0xf0, 0xb3, 0xc4, 0xdb, 0x57, 0xa0, 0xe6, 0x78, 0x91, 0xc1, 0xb2, 0x34, 0x88, 0x48, + 0x23, 0x65, 0x46, 0x26, 0xa7, 0xd0, 0xd8, 0xd0, 0x95, 0xcd, 0x9d, 0x1d, 0xb5, 0x58, 0x60, 0x6f, + 0x98, 0x56, 0x79, 0xfa, 0xed, 0x1a, 0x6c, 0xe7, 0x3f, 0x50, 0x3e, 0x0b, 0xec, 0x4d, 0xbc, 0x70, + 0xf5, 0x44, 0x2f, 0x30, 0x86, 0xbf, 0x84, 0x67, 0x69, 0xc2, 0x05, 0x75, 0xb9, 0xcd, 0xb4, 0x6a, + 0x5b, 0xe9, 0x60, 0x92, 0xd6, 0x39, 0x97, 0xe8, 0x3d, 0xa2, 0x74, 0xca, 0x34, 0x68, 0x2b, 0x1d, + 0x94, 0x11, 0xa5, 0x4d, 0x26, 0x2c, 0x06, 0x3e, 0x73, 0x72, 0x16, 0x6b, 0xff, 0xd7, 0x62, 0xa2, + 0x4f, 0x2d, 0xa6, 0x09, 0x63, 0x8b, 0x07, 0x91, 0xc5, 0x04, 0xce, 0x2c, 0xa6, 0xc4, 0xd8, 0x62, + 0x3d, 0xb2, 0x98, 0xc0, 0xb1, 0xc5, 0xb7, 0x00, 0x21, 0x65, 0x94, 0x5b, 0x6b, 0xf1, 0x55, 0x1a, + 0x4f, 0xef, 0x65, 0x3a, 0x63, 0x5d, 0x22, 0x34, 0x43, 0x67, 0xc3, 0x49, 0x35, 0x4c, 0x8e, 0xf7, + 0x1f, 0x82, 0x67, 0x0f, 0x1e, 0x02, 0xfc, 0x19, 0xd4, 0xe7, 0x5b, 0xc6, 0x7d, 0xcf, 0x92, 0xcf, + 0x06, 0xd3, 0x54, 0x69, 0xe8, 0x20, 0x02, 0xbf, 0x93, 0x98, 0xbe, 0x80, 0x6a, 0x9a, 0x1a, 0x9f, + 0xc0, 0x31, 0x11, 0x13, 0x6e, 0x0d, 0x47, 0x13, 0xf3, 0xc1, 0x9a, 0x62, 0x68, 0xe4, 0x62, 0xd7, + 0xc6, 0x4c, 0x45, 0xb8, 0x09, 0xf5, 0x1c, 0x36, 0x99, 0xaa, 0x05, 0xb1, 0x49, 0x39, 0x28, 0xda, + 0x59, 0x65, 0x50, 0x86, 0x7d, 0xd9, 0x94, 0xc1, 0x01, 0x40, 0x36, 0x6f, 0xfa, 0x1b, 0x80, 0xec, + 0x03, 0x88, 0x91, 0xf7, 0x97, 0x4b, 0x46, 0xa3, 0x1d, 0x6a, 0x92, 0xf8, 0x26, 0x70, 0x97, 0x6e, + 0x56, 0x7c, 0x2d, 0x57, 0xa7, 0x4e, 0xe2, 0xdb, 0xe0, 0xe8, 0xfd, 0x5d, 0x0b, 0xfd, 0x76, 0xd7, + 0x42, 0x7f, 0xde, 0xb5, 0xd0, 0xf7, 0x65, 0xd9, 0xb4, 0x5d, 0xff, 0xa6, 0x24, 0xff, 0x8a, 0xbf, + 0xfe, 0x27, 0x00, 0x00, 0xff, 0xff, 0x3e, 0xfc, 0x93, 0x1c, 0xde, 0x07, 0x00, 0x00, +} + +func (m *Request) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Request) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Request) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if len(m.Timeseries) > 0 { + for iNdEx := len(m.Timeseries) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Timeseries[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x2a + } + } + if len(m.Symbols) > 0 { + for iNdEx := len(m.Symbols) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.Symbols[iNdEx]) + copy(dAtA[i:], m.Symbols[iNdEx]) + i = encodeVarintTypes(dAtA, i, uint64(len(m.Symbols[iNdEx]))) + i-- + dAtA[i] = 0x22 + } + } + return len(dAtA) - i, nil +} + +func (m *TimeSeries) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *TimeSeries) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *TimeSeries) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if m.CreatedTimestamp != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.CreatedTimestamp)) + i-- + dAtA[i] = 0x30 + } + { + size, err := m.Metadata.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x2a + if len(m.Exemplars) > 0 { + for iNdEx := len(m.Exemplars) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Exemplars[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x22 + } + } + if len(m.Histograms) > 0 { + for iNdEx := len(m.Histograms) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Histograms[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + } + if len(m.Samples) > 0 { + for iNdEx := len(m.Samples) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Samples[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + } + if len(m.LabelsRefs) > 0 { + dAtA3 := make([]byte, len(m.LabelsRefs)*10) + var j2 int + for _, num := range m.LabelsRefs { + for num >= 1<<7 { + dAtA3[j2] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j2++ + } + dAtA3[j2] = uint8(num) + j2++ + } + i -= j2 + copy(dAtA[i:], dAtA3[:j2]) + i = encodeVarintTypes(dAtA, i, uint64(j2)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Exemplar) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Exemplar) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Exemplar) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if m.Timestamp != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.Timestamp)) + i-- + dAtA[i] = 0x18 + } + if m.Value != 0 { + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.Value)))) + i-- + dAtA[i] = 0x11 + } + if len(m.LabelsRefs) > 0 { + dAtA5 := make([]byte, len(m.LabelsRefs)*10) + var j4 int + for _, num := range m.LabelsRefs { + for num >= 1<<7 { + dAtA5[j4] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j4++ + } + dAtA5[j4] = uint8(num) + j4++ + } + i -= j4 + copy(dAtA[i:], dAtA5[:j4]) + i = encodeVarintTypes(dAtA, i, uint64(j4)) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + +func (m *Sample) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Sample) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Sample) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if m.Timestamp != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.Timestamp)) + i-- + dAtA[i] = 0x10 + } + if m.Value != 0 { + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.Value)))) + i-- + dAtA[i] = 0x9 + } + return len(dAtA) - i, nil +} + +func (m *Metadata) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Metadata) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Metadata) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if m.UnitRef != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.UnitRef)) + i-- + dAtA[i] = 0x20 + } + if m.HelpRef != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.HelpRef)) + i-- + dAtA[i] = 0x18 + } + if m.Type != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.Type)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *Histogram) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Histogram) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Histogram) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if len(m.CustomValues) > 0 { + for iNdEx := len(m.CustomValues) - 1; iNdEx >= 0; iNdEx-- { + f6 := math.Float64bits(float64(m.CustomValues[iNdEx])) + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(f6)) + } + i = encodeVarintTypes(dAtA, i, uint64(len(m.CustomValues)*8)) + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x82 + } + if m.Timestamp != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.Timestamp)) + i-- + dAtA[i] = 0x78 + } + if m.ResetHint != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.ResetHint)) + i-- + dAtA[i] = 0x70 + } + if len(m.PositiveCounts) > 0 { + for iNdEx := len(m.PositiveCounts) - 1; iNdEx >= 0; iNdEx-- { + f7 := math.Float64bits(float64(m.PositiveCounts[iNdEx])) + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(f7)) + } + i = encodeVarintTypes(dAtA, i, uint64(len(m.PositiveCounts)*8)) + i-- + dAtA[i] = 0x6a + } + if len(m.PositiveDeltas) > 0 { + var j8 int + dAtA10 := make([]byte, len(m.PositiveDeltas)*10) + for _, num := range m.PositiveDeltas { + x9 := (uint64(num) << 1) ^ uint64((num >> 63)) + for x9 >= 1<<7 { + dAtA10[j8] = uint8(uint64(x9)&0x7f | 0x80) + j8++ + x9 >>= 7 + } + dAtA10[j8] = uint8(x9) + j8++ + } + i -= j8 + copy(dAtA[i:], dAtA10[:j8]) + i = encodeVarintTypes(dAtA, i, uint64(j8)) + i-- + dAtA[i] = 0x62 + } + if len(m.PositiveSpans) > 0 { + for iNdEx := len(m.PositiveSpans) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.PositiveSpans[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x5a + } + } + if len(m.NegativeCounts) > 0 { + for iNdEx := len(m.NegativeCounts) - 1; iNdEx >= 0; iNdEx-- { + f11 := math.Float64bits(float64(m.NegativeCounts[iNdEx])) + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(f11)) + } + i = encodeVarintTypes(dAtA, i, uint64(len(m.NegativeCounts)*8)) + i-- + dAtA[i] = 0x52 + } + if len(m.NegativeDeltas) > 0 { + var j12 int + dAtA14 := make([]byte, len(m.NegativeDeltas)*10) + for _, num := range m.NegativeDeltas { + x13 := (uint64(num) << 1) ^ uint64((num >> 63)) + for x13 >= 1<<7 { + dAtA14[j12] = uint8(uint64(x13)&0x7f | 0x80) + j12++ + x13 >>= 7 + } + dAtA14[j12] = uint8(x13) + j12++ + } + i -= j12 + copy(dAtA[i:], dAtA14[:j12]) + i = encodeVarintTypes(dAtA, i, uint64(j12)) + i-- + dAtA[i] = 0x4a + } + if len(m.NegativeSpans) > 0 { + for iNdEx := len(m.NegativeSpans) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.NegativeSpans[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintTypes(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x42 + } + } + if m.ZeroCount != nil { + { + size := m.ZeroCount.Size() + i -= size + if _, err := m.ZeroCount.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } + } + if m.ZeroThreshold != 0 { + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.ZeroThreshold)))) + i-- + dAtA[i] = 0x29 + } + if m.Schema != 0 { + i = encodeVarintTypes(dAtA, i, uint64((uint32(m.Schema)<<1)^uint32((m.Schema>>31)))) + i-- + dAtA[i] = 0x20 + } + if m.Sum != 0 { + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.Sum)))) + i-- + dAtA[i] = 0x19 + } + if m.Count != nil { + { + size := m.Count.Size() + i -= size + if _, err := m.Count.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } + } + return len(dAtA) - i, nil +} + +func (m *Histogram_CountInt) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Histogram_CountInt) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i = encodeVarintTypes(dAtA, i, uint64(m.CountInt)) + i-- + dAtA[i] = 0x8 + return len(dAtA) - i, nil +} +func (m *Histogram_CountFloat) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Histogram_CountFloat) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.CountFloat)))) + i-- + dAtA[i] = 0x11 + return len(dAtA) - i, nil +} +func (m *Histogram_ZeroCountInt) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Histogram_ZeroCountInt) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i = encodeVarintTypes(dAtA, i, uint64(m.ZeroCountInt)) + i-- + dAtA[i] = 0x30 + return len(dAtA) - i, nil +} +func (m *Histogram_ZeroCountFloat) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *Histogram_ZeroCountFloat) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i -= 8 + encoding_binary.LittleEndian.PutUint64(dAtA[i:], uint64(math.Float64bits(float64(m.ZeroCountFloat)))) + i-- + dAtA[i] = 0x39 + return len(dAtA) - i, nil +} +func (m *BucketSpan) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *BucketSpan) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *BucketSpan) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if m.Length != 0 { + i = encodeVarintTypes(dAtA, i, uint64(m.Length)) + i-- + dAtA[i] = 0x10 + } + if m.Offset != 0 { + i = encodeVarintTypes(dAtA, i, uint64((uint32(m.Offset)<<1)^uint32((m.Offset>>31)))) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func encodeVarintTypes(dAtA []byte, offset int, v uint64) int { + offset -= sovTypes(v) + base := offset + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return base +} +func (m *Request) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.Symbols) > 0 { + for _, s := range m.Symbols { + l = len(s) + n += 1 + l + sovTypes(uint64(l)) + } + } + if len(m.Timeseries) > 0 { + for _, e := range m.Timeseries { + l = e.Size() + n += 1 + l + sovTypes(uint64(l)) + } + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *TimeSeries) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.LabelsRefs) > 0 { + l = 0 + for _, e := range m.LabelsRefs { + l += sovTypes(uint64(e)) + } + n += 1 + sovTypes(uint64(l)) + l + } + if len(m.Samples) > 0 { + for _, e := range m.Samples { + l = e.Size() + n += 1 + l + sovTypes(uint64(l)) + } + } + if len(m.Histograms) > 0 { + for _, e := range m.Histograms { + l = e.Size() + n += 1 + l + sovTypes(uint64(l)) + } + } + if len(m.Exemplars) > 0 { + for _, e := range m.Exemplars { + l = e.Size() + n += 1 + l + sovTypes(uint64(l)) + } + } + l = m.Metadata.Size() + n += 1 + l + sovTypes(uint64(l)) + if m.CreatedTimestamp != 0 { + n += 1 + sovTypes(uint64(m.CreatedTimestamp)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Exemplar) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if len(m.LabelsRefs) > 0 { + l = 0 + for _, e := range m.LabelsRefs { + l += sovTypes(uint64(e)) + } + n += 1 + sovTypes(uint64(l)) + l + } + if m.Value != 0 { + n += 9 + } + if m.Timestamp != 0 { + n += 1 + sovTypes(uint64(m.Timestamp)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Sample) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Value != 0 { + n += 9 + } + if m.Timestamp != 0 { + n += 1 + sovTypes(uint64(m.Timestamp)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Metadata) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Type != 0 { + n += 1 + sovTypes(uint64(m.Type)) + } + if m.HelpRef != 0 { + n += 1 + sovTypes(uint64(m.HelpRef)) + } + if m.UnitRef != 0 { + n += 1 + sovTypes(uint64(m.UnitRef)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Histogram) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Count != nil { + n += m.Count.Size() + } + if m.Sum != 0 { + n += 9 + } + if m.Schema != 0 { + n += 1 + sozTypes(uint64(m.Schema)) + } + if m.ZeroThreshold != 0 { + n += 9 + } + if m.ZeroCount != nil { + n += m.ZeroCount.Size() + } + if len(m.NegativeSpans) > 0 { + for _, e := range m.NegativeSpans { + l = e.Size() + n += 1 + l + sovTypes(uint64(l)) + } + } + if len(m.NegativeDeltas) > 0 { + l = 0 + for _, e := range m.NegativeDeltas { + l += sozTypes(uint64(e)) + } + n += 1 + sovTypes(uint64(l)) + l + } + if len(m.NegativeCounts) > 0 { + n += 1 + sovTypes(uint64(len(m.NegativeCounts)*8)) + len(m.NegativeCounts)*8 + } + if len(m.PositiveSpans) > 0 { + for _, e := range m.PositiveSpans { + l = e.Size() + n += 1 + l + sovTypes(uint64(l)) + } + } + if len(m.PositiveDeltas) > 0 { + l = 0 + for _, e := range m.PositiveDeltas { + l += sozTypes(uint64(e)) + } + n += 1 + sovTypes(uint64(l)) + l + } + if len(m.PositiveCounts) > 0 { + n += 1 + sovTypes(uint64(len(m.PositiveCounts)*8)) + len(m.PositiveCounts)*8 + } + if m.ResetHint != 0 { + n += 1 + sovTypes(uint64(m.ResetHint)) + } + if m.Timestamp != 0 { + n += 1 + sovTypes(uint64(m.Timestamp)) + } + if len(m.CustomValues) > 0 { + n += 2 + sovTypes(uint64(len(m.CustomValues)*8)) + len(m.CustomValues)*8 + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *Histogram_CountInt) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 1 + sovTypes(uint64(m.CountInt)) + return n +} +func (m *Histogram_CountFloat) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 9 + return n +} +func (m *Histogram_ZeroCountInt) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 1 + sovTypes(uint64(m.ZeroCountInt)) + return n +} +func (m *Histogram_ZeroCountFloat) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + n += 9 + return n +} +func (m *BucketSpan) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Offset != 0 { + n += 1 + sozTypes(uint64(m.Offset)) + } + if m.Length != 0 { + n += 1 + sovTypes(uint64(m.Length)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func sovTypes(x uint64) (n int) { + return (math_bits.Len64(x|1) + 6) / 7 +} +func sozTypes(x uint64) (n int) { + return sovTypes(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (m *Request) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Request: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Request: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Symbols", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Symbols = append(m.Symbols, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Timeseries", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Timeseries = append(m.Timeseries, TimeSeries{}) + if err := m.Timeseries[len(m.Timeseries)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipTypes(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *TimeSeries) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: TimeSeries: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: TimeSeries: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType == 0 { + var v uint32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.LabelsRefs = append(m.LabelsRefs, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.LabelsRefs) == 0 { + m.LabelsRefs = make([]uint32, 0, elementCount) + } + for iNdEx < postIndex { + var v uint32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.LabelsRefs = append(m.LabelsRefs, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field LabelsRefs", wireType) + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Samples", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Samples = append(m.Samples, Sample{}) + if err := m.Samples[len(m.Samples)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Histograms", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Histograms = append(m.Histograms, Histogram{}) + if err := m.Histograms[len(m.Histograms)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Exemplars", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Exemplars = append(m.Exemplars, Exemplar{}) + if err := m.Exemplars[len(m.Exemplars)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Metadata", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.Metadata.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field CreatedTimestamp", wireType) + } + m.CreatedTimestamp = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.CreatedTimestamp |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipTypes(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Exemplar) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Exemplar: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Exemplar: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType == 0 { + var v uint32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.LabelsRefs = append(m.LabelsRefs, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.LabelsRefs) == 0 { + m.LabelsRefs = make([]uint32, 0, elementCount) + } + for iNdEx < postIndex { + var v uint32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.LabelsRefs = append(m.LabelsRefs, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field LabelsRefs", wireType) + } + case 2: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) + } + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + m.Value = float64(math.Float64frombits(v)) + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType) + } + m.Timestamp = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Timestamp |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipTypes(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Sample) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Sample: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Sample: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field Value", wireType) + } + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + m.Value = float64(math.Float64frombits(v)) + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType) + } + m.Timestamp = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Timestamp |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipTypes(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Metadata) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Metadata: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Metadata: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Type", wireType) + } + m.Type = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Type |= Metadata_MetricType(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field HelpRef", wireType) + } + m.HelpRef = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.HelpRef |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field UnitRef", wireType) + } + m.UnitRef = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.UnitRef |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipTypes(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *Histogram) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Histogram: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Histogram: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field CountInt", wireType) + } + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Count = &Histogram_CountInt{v} + case 2: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field CountFloat", wireType) + } + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + m.Count = &Histogram_CountFloat{float64(math.Float64frombits(v))} + case 3: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field Sum", wireType) + } + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + m.Sum = float64(math.Float64frombits(v)) + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Schema", wireType) + } + var v int32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + v = int32((uint32(v) >> 1) ^ uint32(((v&1)<<31)>>31)) + m.Schema = v + case 5: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field ZeroThreshold", wireType) + } + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + m.ZeroThreshold = float64(math.Float64frombits(v)) + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ZeroCountInt", wireType) + } + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.ZeroCount = &Histogram_ZeroCountInt{v} + case 7: + if wireType != 1 { + return fmt.Errorf("proto: wrong wireType = %d for field ZeroCountFloat", wireType) + } + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + m.ZeroCount = &Histogram_ZeroCountFloat{float64(math.Float64frombits(v))} + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field NegativeSpans", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.NegativeSpans = append(m.NegativeSpans, BucketSpan{}) + if err := m.NegativeSpans[len(m.NegativeSpans)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 9: + if wireType == 0 { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + v = (v >> 1) ^ uint64((int64(v&1)<<63)>>63) + m.NegativeDeltas = append(m.NegativeDeltas, int64(v)) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.NegativeDeltas) == 0 { + m.NegativeDeltas = make([]int64, 0, elementCount) + } + for iNdEx < postIndex { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + v = (v >> 1) ^ uint64((int64(v&1)<<63)>>63) + m.NegativeDeltas = append(m.NegativeDeltas, int64(v)) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field NegativeDeltas", wireType) + } + case 10: + if wireType == 1 { + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + v2 := float64(math.Float64frombits(v)) + m.NegativeCounts = append(m.NegativeCounts, v2) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + elementCount = packedLen / 8 + if elementCount != 0 && len(m.NegativeCounts) == 0 { + m.NegativeCounts = make([]float64, 0, elementCount) + } + for iNdEx < postIndex { + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + v2 := float64(math.Float64frombits(v)) + m.NegativeCounts = append(m.NegativeCounts, v2) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field NegativeCounts", wireType) + } + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PositiveSpans", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.PositiveSpans = append(m.PositiveSpans, BucketSpan{}) + if err := m.PositiveSpans[len(m.PositiveSpans)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 12: + if wireType == 0 { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + v = (v >> 1) ^ uint64((int64(v&1)<<63)>>63) + m.PositiveDeltas = append(m.PositiveDeltas, int64(v)) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.PositiveDeltas) == 0 { + m.PositiveDeltas = make([]int64, 0, elementCount) + } + for iNdEx < postIndex { + var v uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + v = (v >> 1) ^ uint64((int64(v&1)<<63)>>63) + m.PositiveDeltas = append(m.PositiveDeltas, int64(v)) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field PositiveDeltas", wireType) + } + case 13: + if wireType == 1 { + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + v2 := float64(math.Float64frombits(v)) + m.PositiveCounts = append(m.PositiveCounts, v2) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + elementCount = packedLen / 8 + if elementCount != 0 && len(m.PositiveCounts) == 0 { + m.PositiveCounts = make([]float64, 0, elementCount) + } + for iNdEx < postIndex { + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + v2 := float64(math.Float64frombits(v)) + m.PositiveCounts = append(m.PositiveCounts, v2) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field PositiveCounts", wireType) + } + case 14: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ResetHint", wireType) + } + m.ResetHint = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.ResetHint |= Histogram_ResetHint(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 15: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType) + } + m.Timestamp = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Timestamp |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 16: + if wireType == 1 { + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + v2 := float64(math.Float64frombits(v)) + m.CustomValues = append(m.CustomValues, v2) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthTypes + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthTypes + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + elementCount = packedLen / 8 + if elementCount != 0 && len(m.CustomValues) == 0 { + m.CustomValues = make([]float64, 0, elementCount) + } + for iNdEx < postIndex { + var v uint64 + if (iNdEx + 8) > l { + return io.ErrUnexpectedEOF + } + v = uint64(encoding_binary.LittleEndian.Uint64(dAtA[iNdEx:])) + iNdEx += 8 + v2 := float64(math.Float64frombits(v)) + m.CustomValues = append(m.CustomValues, v2) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field CustomValues", wireType) + } + default: + iNdEx = preIndex + skippy, err := skipTypes(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *BucketSpan) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: BucketSpan: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: BucketSpan: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Offset", wireType) + } + var v int32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int32(b&0x7F) << shift + if b < 0x80 { + break + } + } + v = int32((uint32(v) >> 1) ^ uint32(((v&1)<<31)>>31)) + m.Offset = v + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Length", wireType) + } + m.Length = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowTypes + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Length |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipTypes(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthTypes + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func skipTypes(dAtA []byte) (n int, err error) { + l := len(dAtA) + iNdEx := 0 + depth := 0 + for iNdEx < l { + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowTypes + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + wireType := int(wire & 0x7) + switch wireType { + case 0: + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowTypes + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + iNdEx++ + if dAtA[iNdEx-1] < 0x80 { + break + } + } + case 1: + iNdEx += 8 + case 2: + var length int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowTypes + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + length |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if length < 0 { + return 0, ErrInvalidLengthTypes + } + iNdEx += length + case 3: + depth++ + case 4: + if depth == 0 { + return 0, ErrUnexpectedEndOfGroupTypes + } + depth-- + case 5: + iNdEx += 4 + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + if iNdEx < 0 { + return 0, ErrInvalidLengthTypes + } + if depth == 0 { + return iNdEx, nil + } + } + return 0, io.ErrUnexpectedEOF +} + +var ( + ErrInvalidLengthTypes = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowTypes = fmt.Errorf("proto: integer overflow") + ErrUnexpectedEndOfGroupTypes = fmt.Errorf("proto: unexpected end of group") +) diff --git a/vendor/github.com/prometheus/prometheus/prompb/io/prometheus/write/v2/types.proto b/vendor/github.com/prometheus/prometheus/prompb/io/prometheus/write/v2/types.proto new file mode 100644 index 0000000000..0cc7b8bc4a --- /dev/null +++ b/vendor/github.com/prometheus/prometheus/prompb/io/prometheus/write/v2/types.proto @@ -0,0 +1,260 @@ +// Copyright 2024 Prometheus Team +// 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. + +// NOTE: This file is also available on https://buf.build/prometheus/prometheus/docs/main:io.prometheus.write.v2 + +syntax = "proto3"; +package io.prometheus.write.v2; + +option go_package = "writev2"; + +import "gogoproto/gogo.proto"; + +// Request represents a request to write the given timeseries to a remote destination. +// This message was introduced in the Remote Write 2.0 specification: +// https://prometheus.io/docs/concepts/remote_write_spec_2_0/ +// +// The canonical Content-Type request header value for this message is +// "application/x-protobuf;proto=io.prometheus.write.v2.Request" +// +// NOTE: gogoproto options might change in future for this file, they +// are not part of the spec proto (they only modify the generated Go code, not +// the serialized message). See: https://github.com/prometheus/prometheus/issues/11908 +message Request { + // Since Request supersedes 1.0 spec's prometheus.WriteRequest, we reserve the top-down message + // for the deterministic interop between those two, see types_test.go for details. + // Generally it's not needed, because Receivers must use the Content-Type header, but we want to + // be sympathetic to adopters with mistaken implementations and have deterministic error (empty + // message if you use the wrong proto schema). + reserved 1 to 3; + + // symbols contains a de-duplicated array of string elements used for various + // items in a Request message, like labels and metadata items. For the sender's convenience + // around empty values for optional fields like unit_ref, symbols array MUST start with + // empty string. + // + // To decode each of the symbolized strings, referenced, by "ref(s)" suffix, you + // need to lookup the actual string by index from symbols array. The order of + // strings is up to the sender. The receiver should not assume any particular encoding. + repeated string symbols = 4; + // timeseries represents an array of distinct series with 0 or more samples. + repeated TimeSeries timeseries = 5 [(gogoproto.nullable) = false]; +} + +// TimeSeries represents a single series. +message TimeSeries { + // labels_refs is a list of label name-value pair references, encoded + // as indices to the Request.symbols array. This list's length is always + // a multiple of two, and the underlying labels should be sorted lexicographically. + // + // Note that there might be multiple TimeSeries objects in the same + // Requests with the same labels e.g. for different exemplars, metadata + // or created timestamp. + repeated uint32 labels_refs = 1; + + // Timeseries messages can either specify samples or (native) histogram samples + // (histogram field), but not both. For a typical sender (real-time metric + // streaming), in healthy cases, there will be only one sample or histogram. + // + // Samples and histograms are sorted by timestamp (older first). + repeated Sample samples = 2 [(gogoproto.nullable) = false]; + repeated Histogram histograms = 3 [(gogoproto.nullable) = false]; + + // exemplars represents an optional set of exemplars attached to this series' samples. + repeated Exemplar exemplars = 4 [(gogoproto.nullable) = false]; + + // metadata represents the metadata associated with the given series' samples. + Metadata metadata = 5 [(gogoproto.nullable) = false]; + + // created_timestamp represents an optional created timestamp associated with + // this series' samples in ms format, typically for counter or histogram type + // metrics. Created timestamp represents the time when the counter started + // counting (sometimes referred to as start timestamp), which can increase + // the accuracy of query results. + // + // Note that some receivers might require this and in return fail to + // ingest such samples within the Request. + // + // For Go, see github.com/prometheus/prometheus/model/timestamp/timestamp.go + // for conversion from/to time.Time to Prometheus timestamp. + // + // Note that the "optional" keyword is omitted due to + // https://cloud.google.com/apis/design/design_patterns.md#optional_primitive_fields + // Zero value means value not set. If you need to use exactly zero value for + // the timestamp, use 1 millisecond before or after. + int64 created_timestamp = 6; +} + +// Exemplar is an additional information attached to some series' samples. +// It is typically used to attach an example trace or request ID associated with +// the metric changes. +message Exemplar { + // labels_refs is an optional list of label name-value pair references, encoded + // as indices to the Request.symbols array. This list's len is always + // a multiple of 2, and the underlying labels should be sorted lexicographically. + // If the exemplar references a trace it should use the `trace_id` label name, as a best practice. + repeated uint32 labels_refs = 1; + // value represents an exact example value. This can be useful when the exemplar + // is attached to a histogram, which only gives an estimated value through buckets. + double value = 2; + // timestamp represents an optional timestamp of the sample in ms. + // + // For Go, see github.com/prometheus/prometheus/model/timestamp/timestamp.go + // for conversion from/to time.Time to Prometheus timestamp. + // + // Note that the "optional" keyword is omitted due to + // https://cloud.google.com/apis/design/design_patterns.md#optional_primitive_fields + // Zero value means value not set. If you need to use exactly zero value for + // the timestamp, use 1 millisecond before or after. + int64 timestamp = 3; +} + +// Sample represents series sample. +message Sample { + // value of the sample. + double value = 1; + // timestamp represents timestamp of the sample in ms. + // + // For Go, see github.com/prometheus/prometheus/model/timestamp/timestamp.go + // for conversion from/to time.Time to Prometheus timestamp. + int64 timestamp = 2; +} + +// Metadata represents the metadata associated with the given series' samples. +message Metadata { + enum MetricType { + METRIC_TYPE_UNSPECIFIED = 0; + METRIC_TYPE_COUNTER = 1; + METRIC_TYPE_GAUGE = 2; + METRIC_TYPE_HISTOGRAM = 3; + METRIC_TYPE_GAUGEHISTOGRAM = 4; + METRIC_TYPE_SUMMARY = 5; + METRIC_TYPE_INFO = 6; + METRIC_TYPE_STATESET = 7; + } + MetricType type = 1; + // help_ref is a reference to the Request.symbols array representing help + // text for the metric. Help is optional, reference should point to an empty string in + // such a case. + uint32 help_ref = 3; + // unit_ref is a reference to the Request.symbols array representing a unit + // for the metric. Unit is optional, reference should point to an empty string in + // such a case. + uint32 unit_ref = 4; +} + +// A native histogram, also known as a sparse histogram. +// Original design doc: +// https://docs.google.com/document/d/1cLNv3aufPZb3fNfaJgdaRBZsInZKKIHo9E6HinJVbpM/edit +// The appendix of this design doc also explains the concept of float +// histograms. This Histogram message can represent both, the usual +// integer histogram as well as a float histogram. +message Histogram { + enum ResetHint { + RESET_HINT_UNSPECIFIED = 0; // Need to test for a counter reset explicitly. + RESET_HINT_YES = 1; // This is the 1st histogram after a counter reset. + RESET_HINT_NO = 2; // There was no counter reset between this and the previous Histogram. + RESET_HINT_GAUGE = 3; // This is a gauge histogram where counter resets don't happen. + } + + oneof count { // Count of observations in the histogram. + uint64 count_int = 1; + double count_float = 2; + } + double sum = 3; // Sum of observations in the histogram. + + // The schema defines the bucket schema. Currently, valid numbers + // are -53 and numbers in range of -4 <= n <= 8. More valid numbers might be + // added in future for new bucketing layouts. + // + // The schema equal to -53 means custom buckets. See + // custom_values field description for more details. + // + // Values between -4 and 8 represent base-2 bucket schema, where 1 + // is a bucket boundary in each case, and then each power of two is + // divided into 2^n (n is schema value) logarithmic buckets. Or in other words, + // each bucket boundary is the previous boundary times 2^(2^-n). + sint32 schema = 4; + double zero_threshold = 5; // Breadth of the zero bucket. + oneof zero_count { // Count in zero bucket. + uint64 zero_count_int = 6; + double zero_count_float = 7; + } + + // Negative Buckets. + repeated BucketSpan negative_spans = 8 [(gogoproto.nullable) = false]; + // Use either "negative_deltas" or "negative_counts", the former for + // regular histograms with integer counts, the latter for + // float histograms. + repeated sint64 negative_deltas = 9; // Count delta of each bucket compared to previous one (or to zero for 1st bucket). + repeated double negative_counts = 10; // Absolute count of each bucket. + + // Positive Buckets. + // + // In case of custom buckets (-53 schema value) the positive buckets are interpreted as follows: + // * The span offset+length points to an the index of the custom_values array + // or +Inf if pointing to the len of the array. + // * The counts and deltas have the same meaning as for exponential histograms. + repeated BucketSpan positive_spans = 11 [(gogoproto.nullable) = false]; + // Use either "positive_deltas" or "positive_counts", the former for + // regular histograms with integer counts, the latter for + // float histograms. + repeated sint64 positive_deltas = 12; // Count delta of each bucket compared to previous one (or to zero for 1st bucket). + repeated double positive_counts = 13; // Absolute count of each bucket. + + ResetHint reset_hint = 14; + // timestamp represents timestamp of the sample in ms. + // + // For Go, see github.com/prometheus/prometheus/model/timestamp/timestamp.go + // for conversion from/to time.Time to Prometheus timestamp. + int64 timestamp = 15; + + // custom_values is an additional field used by non-exponential bucketing layouts. + // + // For custom buckets (-53 schema value) custom_values specify monotonically + // increasing upper inclusive boundaries for the bucket counts with arbitrary + // widths for this histogram. In other words, custom_values represents custom, + // explicit bucketing that could have been converted from the classic histograms. + // + // Those bounds are then referenced by spans in positive_spans with corresponding positive + // counts of deltas (refer to positive_spans for more details). This way we can + // have encode sparse histograms with custom bucketing (many buckets are often + // not used). + // + // Note that for custom bounds, even negative observations are placed in the positive + // counts to simplify the implementation and avoid ambiguity of where to place + // an underflow bucket, e.g. (-2, 1]. Therefore negative buckets and + // the zero bucket are unused, if the schema indicates custom bucketing. + // + // For each upper boundary the previous boundary represent the lower exclusive + // boundary for that bucket. The first element is the upper inclusive boundary + // for the first bucket, which implicitly has a lower inclusive bound of -Inf. + // This is similar to "le" label semantics on classic histograms. You may add a + // bucket with an upper bound of 0 to make sure that you really have no negative + // observations, but in practice, native histogram rendering will show both with + // or without first upper boundary 0 and no negative counts as the same case. + // + // The last element is not only the upper inclusive bound of the last regular + // bucket, but implicitly the lower exclusive bound of the +Inf bucket. + repeated double custom_values = 16; +} + +// A BucketSpan defines a number of consecutive buckets with their +// offset. Logically, it would be more straightforward to include the +// bucket counts in the Span. However, the protobuf representation is +// more compact in the way the data is structured here (with all the +// buckets in a single array separate from the Spans). +message BucketSpan { + sint32 offset = 1; // Gap to previous span, or starting point for 1st span (which can be negative). + uint32 length = 2; // Length of consecutive buckets. +} diff --git a/vendor/github.com/prometheus/prometheus/promql/engine.go b/vendor/github.com/prometheus/prometheus/promql/engine.go index 2a84871f00..25e67db633 100644 --- a/vendor/github.com/prometheus/prometheus/promql/engine.go +++ b/vendor/github.com/prometheus/prometheus/promql/engine.go @@ -1318,7 +1318,7 @@ func (ev *evaluator) rangeEvalAgg(aggExpr *parser.AggregateExpr, sortedGrouping index, ok := groupToResultIndex[groupingKey] // Add a new group if it doesn't exist. if !ok { - if aggExpr.Op != parser.TOPK && aggExpr.Op != parser.BOTTOMK { + if aggExpr.Op != parser.TOPK && aggExpr.Op != parser.BOTTOMK && aggExpr.Op != parser.LIMITK && aggExpr.Op != parser.LIMIT_RATIO { m := generateGroupingLabels(enh, series.Metric, aggExpr.Without, sortedGrouping) result = append(result, Series{Metric: m}) } @@ -1331,9 +1331,10 @@ func (ev *evaluator) rangeEvalAgg(aggExpr *parser.AggregateExpr, sortedGrouping groups := make([]groupedAggregation, groupCount) var k int + var ratio float64 var seriess map[uint64]Series switch aggExpr.Op { - case parser.TOPK, parser.BOTTOMK: + case parser.TOPK, parser.BOTTOMK, parser.LIMITK: if !convertibleToInt64(param) { ev.errorf("Scalar value %v overflows int64", param) } @@ -1345,6 +1346,23 @@ func (ev *evaluator) rangeEvalAgg(aggExpr *parser.AggregateExpr, sortedGrouping return nil, warnings } seriess = make(map[uint64]Series, len(inputMatrix)) // Output series by series hash. + case parser.LIMIT_RATIO: + if math.IsNaN(param) { + ev.errorf("Ratio value %v is NaN", param) + } + switch { + case param == 0: + return nil, warnings + case param < -1.0: + ratio = -1.0 + warnings.Add(annotations.NewInvalidRatioWarning(param, ratio, aggExpr.Param.PositionRange())) + case param > 1.0: + ratio = 1.0 + warnings.Add(annotations.NewInvalidRatioWarning(param, ratio, aggExpr.Param.PositionRange())) + default: + ratio = param + } + seriess = make(map[uint64]Series, len(inputMatrix)) // Output series by series hash. case parser.QUANTILE: if math.IsNaN(param) || param < 0 || param > 1 { warnings.Add(annotations.NewInvalidQuantileWarning(param, aggExpr.Param.PositionRange())) @@ -1362,11 +1380,12 @@ func (ev *evaluator) rangeEvalAgg(aggExpr *parser.AggregateExpr, sortedGrouping enh.Ts = ts var ws annotations.Annotations switch aggExpr.Op { - case parser.TOPK, parser.BOTTOMK: - result, ws = ev.aggregationK(aggExpr, k, inputMatrix, seriesToResult, groups, enh, seriess) + case parser.TOPK, parser.BOTTOMK, parser.LIMITK, parser.LIMIT_RATIO: + result, ws = ev.aggregationK(aggExpr, k, ratio, inputMatrix, seriesToResult, groups, enh, seriess) // If this could be an instant query, shortcut so as not to change sort order. if ev.endTimestamp == ev.startTimestamp { - return result, ws + warnings.Merge(ws) + return result, warnings } default: ws = ev.aggregation(aggExpr, param, inputMatrix, result, seriesToResult, groups, enh) @@ -1381,7 +1400,7 @@ func (ev *evaluator) rangeEvalAgg(aggExpr *parser.AggregateExpr, sortedGrouping // Assemble the output matrix. By the time we get here we know we don't have too many samples. switch aggExpr.Op { - case parser.TOPK, parser.BOTTOMK: + case parser.TOPK, parser.BOTTOMK, parser.LIMITK, parser.LIMIT_RATIO: result = make(Matrix, 0, len(seriess)) for _, ss := range seriess { result = append(result, ss) @@ -1793,18 +1812,21 @@ func (ev *evaluator) eval(expr parser.Expr) (parser.Value, annotations.Annotatio }, e.LHS, e.RHS) default: return ev.rangeEval(initSignatures, func(v []parser.Value, sh [][]EvalSeriesHelper, enh *EvalNodeHelper) (Vector, annotations.Annotations) { - return ev.VectorBinop(e.Op, v[0].(Vector), v[1].(Vector), e.VectorMatching, e.ReturnBool, sh[0], sh[1], enh), nil + vec, err := ev.VectorBinop(e.Op, v[0].(Vector), v[1].(Vector), e.VectorMatching, e.ReturnBool, sh[0], sh[1], enh) + return vec, handleVectorBinopError(err, e) }, e.LHS, e.RHS) } case lt == parser.ValueTypeVector && rt == parser.ValueTypeScalar: return ev.rangeEval(nil, func(v []parser.Value, _ [][]EvalSeriesHelper, enh *EvalNodeHelper) (Vector, annotations.Annotations) { - return ev.VectorscalarBinop(e.Op, v[0].(Vector), Scalar{V: v[1].(Vector)[0].F}, false, e.ReturnBool, enh), nil + vec, err := ev.VectorscalarBinop(e.Op, v[0].(Vector), Scalar{V: v[1].(Vector)[0].F}, false, e.ReturnBool, enh) + return vec, handleVectorBinopError(err, e) }, e.LHS, e.RHS) case lt == parser.ValueTypeScalar && rt == parser.ValueTypeVector: return ev.rangeEval(nil, func(v []parser.Value, _ [][]EvalSeriesHelper, enh *EvalNodeHelper) (Vector, annotations.Annotations) { - return ev.VectorscalarBinop(e.Op, v[1].(Vector), Scalar{V: v[0].(Vector)[0].F}, true, e.ReturnBool, enh), nil + vec, err := ev.VectorscalarBinop(e.Op, v[1].(Vector), Scalar{V: v[0].(Vector)[0].F}, true, e.ReturnBool, enh) + return vec, handleVectorBinopError(err, e) }, e.LHS, e.RHS) } @@ -2437,12 +2459,12 @@ func (ev *evaluator) VectorUnless(lhs, rhs Vector, matching *parser.VectorMatchi } // VectorBinop evaluates a binary operation between two Vectors, excluding set operators. -func (ev *evaluator) VectorBinop(op parser.ItemType, lhs, rhs Vector, matching *parser.VectorMatching, returnBool bool, lhsh, rhsh []EvalSeriesHelper, enh *EvalNodeHelper) Vector { +func (ev *evaluator) VectorBinop(op parser.ItemType, lhs, rhs Vector, matching *parser.VectorMatching, returnBool bool, lhsh, rhsh []EvalSeriesHelper, enh *EvalNodeHelper) (Vector, error) { if matching.Card == parser.CardManyToMany { panic("many-to-many only allowed for set operators") } if len(lhs) == 0 || len(rhs) == 0 { - return nil // Short-circuit: nothing is going to match. + return nil, nil // Short-circuit: nothing is going to match. } // The control flow below handles one-to-one or many-to-one matching. @@ -2495,6 +2517,7 @@ func (ev *evaluator) VectorBinop(op parser.ItemType, lhs, rhs Vector, matching * // For all lhs samples find a respective rhs sample and perform // the binary operation. + var lastErr error for i, ls := range lhs { sig := lhsh[i].signature @@ -2510,7 +2533,10 @@ func (ev *evaluator) VectorBinop(op parser.ItemType, lhs, rhs Vector, matching * fl, fr = fr, fl hl, hr = hr, hl } - floatValue, histogramValue, keep := vectorElemBinop(op, fl, fr, hl, hr) + floatValue, histogramValue, keep, err := vectorElemBinop(op, fl, fr, hl, hr) + if err != nil { + lastErr = err + } switch { case returnBool: if keep { @@ -2552,7 +2578,7 @@ func (ev *evaluator) VectorBinop(op parser.ItemType, lhs, rhs Vector, matching * H: histogramValue, }) } - return enh.Out + return enh.Out, lastErr } func signatureFunc(on bool, b []byte, names ...string) func(labels.Labels) string { @@ -2615,7 +2641,8 @@ func resultMetric(lhs, rhs labels.Labels, op parser.ItemType, matching *parser.V } // VectorscalarBinop evaluates a binary operation between a Vector and a Scalar. -func (ev *evaluator) VectorscalarBinop(op parser.ItemType, lhs Vector, rhs Scalar, swap, returnBool bool, enh *EvalNodeHelper) Vector { +func (ev *evaluator) VectorscalarBinop(op parser.ItemType, lhs Vector, rhs Scalar, swap, returnBool bool, enh *EvalNodeHelper) (Vector, error) { + var lastErr error for _, lhsSample := range lhs { lf, rf := lhsSample.F, rhs.V var rh *histogram.FloatHistogram @@ -2626,7 +2653,10 @@ func (ev *evaluator) VectorscalarBinop(op parser.ItemType, lhs Vector, rhs Scala lf, rf = rf, lf lh, rh = rh, lh } - float, histogram, keep := vectorElemBinop(op, lf, rf, lh, rh) + float, histogram, keep, err := vectorElemBinop(op, lf, rf, lh, rh) + if err != nil { + lastErr = err + } // Catch cases where the scalar is the LHS in a scalar-vector comparison operation. // We want to always keep the vector element value as the output value, even if it's on the RHS. if op.IsComparisonOperator() && swap { @@ -2650,7 +2680,7 @@ func (ev *evaluator) VectorscalarBinop(op parser.ItemType, lhs Vector, rhs Scala enh.Out = append(enh.Out, lhsSample) } } - return enh.Out + return enh.Out, lastErr } // scalarBinop evaluates a binary operation between two Scalars. @@ -2687,62 +2717,71 @@ func scalarBinop(op parser.ItemType, lhs, rhs float64) float64 { } // vectorElemBinop evaluates a binary operation between two Vector elements. -func vectorElemBinop(op parser.ItemType, lhs, rhs float64, hlhs, hrhs *histogram.FloatHistogram) (float64, *histogram.FloatHistogram, bool) { +func vectorElemBinop(op parser.ItemType, lhs, rhs float64, hlhs, hrhs *histogram.FloatHistogram) (float64, *histogram.FloatHistogram, bool, error) { switch op { case parser.ADD: if hlhs != nil && hrhs != nil { - return 0, hlhs.Copy().Add(hrhs).Compact(0), true + res, err := hlhs.Copy().Add(hrhs) + if err != nil { + return 0, nil, false, err + } + return 0, res.Compact(0), true, nil } - return lhs + rhs, nil, true + return lhs + rhs, nil, true, nil case parser.SUB: if hlhs != nil && hrhs != nil { - return 0, hlhs.Copy().Sub(hrhs).Compact(0), true + res, err := hlhs.Copy().Sub(hrhs) + if err != nil { + return 0, nil, false, err + } + return 0, res.Compact(0), true, nil } - return lhs - rhs, nil, true + return lhs - rhs, nil, true, nil case parser.MUL: if hlhs != nil && hrhs == nil { - return 0, hlhs.Copy().Mul(rhs), true + return 0, hlhs.Copy().Mul(rhs), true, nil } if hlhs == nil && hrhs != nil { - return 0, hrhs.Copy().Mul(lhs), true + return 0, hrhs.Copy().Mul(lhs), true, nil } - return lhs * rhs, nil, true + return lhs * rhs, nil, true, nil case parser.DIV: if hlhs != nil && hrhs == nil { - return 0, hlhs.Copy().Div(rhs), true + return 0, hlhs.Copy().Div(rhs), true, nil } - return lhs / rhs, nil, true + return lhs / rhs, nil, true, nil case parser.POW: - return math.Pow(lhs, rhs), nil, true + return math.Pow(lhs, rhs), nil, true, nil case parser.MOD: - return math.Mod(lhs, rhs), nil, true + return math.Mod(lhs, rhs), nil, true, nil case parser.EQLC: - return lhs, nil, lhs == rhs + return lhs, nil, lhs == rhs, nil case parser.NEQ: - return lhs, nil, lhs != rhs + return lhs, nil, lhs != rhs, nil case parser.GTR: - return lhs, nil, lhs > rhs + return lhs, nil, lhs > rhs, nil case parser.LSS: - return lhs, nil, lhs < rhs + return lhs, nil, lhs < rhs, nil case parser.GTE: - return lhs, nil, lhs >= rhs + return lhs, nil, lhs >= rhs, nil case parser.LTE: - return lhs, nil, lhs <= rhs + return lhs, nil, lhs <= rhs, nil case parser.ATAN2: - return math.Atan2(lhs, rhs), nil, true + return math.Atan2(lhs, rhs), nil, true, nil } panic(fmt.Errorf("operator %q not allowed for operations between Vectors", op)) } type groupedAggregation struct { - seen bool // Was this output groups seen in the input at this timestamp. - hasFloat bool // Has at least 1 float64 sample aggregated. - hasHistogram bool // Has at least 1 histogram sample aggregated. - floatValue float64 - histogramValue *histogram.FloatHistogram - floatMean float64 // Mean, or "compensating value" for Kahan summation. - groupCount int - heap vectorByValueHeap + seen bool // Was this output groups seen in the input at this timestamp. + hasFloat bool // Has at least 1 float64 sample aggregated. + hasHistogram bool // Has at least 1 histogram sample aggregated. + floatValue float64 + histogramValue *histogram.FloatHistogram + floatMean float64 // Mean, or "compensating value" for Kahan summation. + groupCount int + groupAggrComplete bool // Used by LIMITK to short-cut series loop when we've reached K elem on every group + heap vectorByValueHeap } // aggregation evaluates sum, avg, count, stdvar, stddev or quantile at one timestep on inputMatrix. @@ -2798,7 +2837,10 @@ func (ev *evaluator) aggregation(e *parser.AggregateExpr, q float64, inputMatrix if h != nil { group.hasHistogram = true if group.histogramValue != nil { - group.histogramValue.Add(h) + _, err := group.histogramValue.Add(h) + if err != nil { + handleAggregationError(err, e, inputMatrix[si].Metric.Get(model.MetricNameLabel), &annos) + } } // Otherwise the aggregation contained floats // previously and will be invalid anyway. No @@ -2815,8 +2857,14 @@ func (ev *evaluator) aggregation(e *parser.AggregateExpr, q float64, inputMatrix if group.histogramValue != nil { left := h.Copy().Div(float64(group.groupCount)) right := group.histogramValue.Copy().Div(float64(group.groupCount)) - toAdd := left.Sub(right) - group.histogramValue.Add(toAdd) + toAdd, err := left.Sub(right) + if err != nil { + handleAggregationError(err, e, inputMatrix[si].Metric.Get(model.MetricNameLabel), &annos) + } + _, err = group.histogramValue.Add(toAdd) + if err != nil { + handleAggregationError(err, e, inputMatrix[si].Metric.Get(model.MetricNameLabel), &annos) + } } // Otherwise the aggregation contained floats // previously and will be invalid anyway. No @@ -2930,19 +2978,22 @@ func (ev *evaluator) aggregation(e *parser.AggregateExpr, q float64, inputMatrix return annos } -// aggregationK evaluates topk or bottomk at one timestep on inputMatrix. +// aggregationK evaluates topk, bottomk, limitk, or limit_ratio at one timestep on inputMatrix. // Output that has the same labels as the input, but just k of them per group. // seriesToResult maps inputMatrix indexes to groups indexes. -// For an instant query, returns a Matrix in descending order for topk or ascending for bottomk. +// For an instant query, returns a Matrix in descending order for topk or ascending for bottomk, or without any order for limitk / limit_ratio. // For a range query, aggregates output in the seriess map. -func (ev *evaluator) aggregationK(e *parser.AggregateExpr, k int, inputMatrix Matrix, seriesToResult []int, groups []groupedAggregation, enh *EvalNodeHelper, seriess map[uint64]Series) (Matrix, annotations.Annotations) { +func (ev *evaluator) aggregationK(e *parser.AggregateExpr, k int, r float64, inputMatrix Matrix, seriesToResult []int, groups []groupedAggregation, enh *EvalNodeHelper, seriess map[uint64]Series) (Matrix, annotations.Annotations) { op := e.Op var s Sample var annos annotations.Annotations + // Used to short-cut the loop for LIMITK if we already collected k elements for every group + groupsRemaining := len(groups) for i := range groups { groups[i].seen = false } +seriesLoop: for si := range inputMatrix { f, _, ok := ev.nextValues(enh.Ts, &inputMatrix[si]) if !ok { @@ -2953,11 +3004,23 @@ func (ev *evaluator) aggregationK(e *parser.AggregateExpr, k int, inputMatrix Ma group := &groups[seriesToResult[si]] // Initialize this group if it's the first time we've seen it. if !group.seen { - *group = groupedAggregation{ - seen: true, - heap: make(vectorByValueHeap, 1, k), + // LIMIT_RATIO is a special case, as we may not add this very sample to the heap, + // while we also don't know the final size of it. + if op == parser.LIMIT_RATIO { + *group = groupedAggregation{ + seen: true, + heap: make(vectorByValueHeap, 0), + } + if ratiosampler.AddRatioSample(r, &s) { + heap.Push(&group.heap, &s) + } + } else { + *group = groupedAggregation{ + seen: true, + heap: make(vectorByValueHeap, 1, k), + } + group.heap[0] = s } - group.heap[0] = s continue } @@ -2988,6 +3051,26 @@ func (ev *evaluator) aggregationK(e *parser.AggregateExpr, k int, inputMatrix Ma } } + case parser.LIMITK: + if len(group.heap) < k { + heap.Push(&group.heap, &s) + } + // LIMITK optimization: early break if we've added K elem to _every_ group, + // especially useful for large timeseries where the user is exploring labels via e.g. + // limitk(10, my_metric) + if !group.groupAggrComplete && len(group.heap) == k { + group.groupAggrComplete = true + groupsRemaining-- + if groupsRemaining == 0 { + break seriesLoop + } + } + + case parser.LIMIT_RATIO: + if ratiosampler.AddRatioSample(r, &s) { + heap.Push(&group.heap, &s) + } + default: panic(fmt.Errorf("expected aggregation operator but got %q", op)) } @@ -3037,6 +3120,11 @@ func (ev *evaluator) aggregationK(e *parser.AggregateExpr, k int, inputMatrix Ma for _, v := range aggr.heap { add(v.Metric, v.F) } + + case parser.LIMITK, parser.LIMIT_RATIO: + for _, v := range aggr.heap { + add(v.Metric, v.F) + } } } @@ -3115,6 +3203,31 @@ func (ev *evaluator) nextValues(ts int64, series *Series) (f float64, h *histogr return f, h, true } +// handleAggregationError adds the appropriate annotation based on the aggregation error. +func handleAggregationError(err error, e *parser.AggregateExpr, metricName string, annos *annotations.Annotations) { + pos := e.Expr.PositionRange() + if errors.Is(err, histogram.ErrHistogramsIncompatibleSchema) { + annos.Add(annotations.NewMixedExponentialCustomHistogramsWarning(metricName, pos)) + } else if errors.Is(err, histogram.ErrHistogramsIncompatibleBounds) { + annos.Add(annotations.NewIncompatibleCustomBucketsHistogramsWarning(metricName, pos)) + } +} + +// handleVectorBinopError returns the appropriate annotation based on the vector binary operation error. +func handleVectorBinopError(err error, e *parser.BinaryExpr) annotations.Annotations { + if err == nil { + return nil + } + metricName := "" + pos := e.PositionRange() + if errors.Is(err, histogram.ErrHistogramsIncompatibleSchema) { + return annotations.New().Add(annotations.NewMixedExponentialCustomHistogramsWarning(metricName, pos)) + } else if errors.Is(err, histogram.ErrHistogramsIncompatibleBounds) { + return annotations.New().Add(annotations.NewIncompatibleCustomBucketsHistogramsWarning(metricName, pos)) + } + return nil +} + // groupingKey builds and returns the grouping key for the given metric and // grouping labels. func generateGroupingKey(metric labels.Labels, grouping []string, without bool, buf []byte) (uint64, []byte) { @@ -3337,6 +3450,12 @@ func setOffsetForAtModifier(evalTime int64, expr parser.Expr) { // required for correctness. func detectHistogramStatsDecoding(expr parser.Expr) { parser.Inspect(expr, func(node parser.Node, path []parser.Node) error { + if n, ok := node.(*parser.BinaryExpr); ok { + detectHistogramStatsDecoding(n.LHS) + detectHistogramStatsDecoding(n.RHS) + return fmt.Errorf("stop") + } + n, ok := (node).(*parser.VectorSelector) if !ok { return nil @@ -3366,6 +3485,56 @@ func makeInt64Pointer(val int64) *int64 { return valp } +// Add RatioSampler interface to allow unit-testing (previously: Randomizer). +type RatioSampler interface { + // Return this sample "offset" between [0.0, 1.0] + sampleOffset(ts int64, sample *Sample) float64 + AddRatioSample(r float64, sample *Sample) bool +} + +// Use Hash(labels.String()) / maxUint64 as a "deterministic" +// value in [0.0, 1.0]. +type HashRatioSampler struct{} + +var ratiosampler RatioSampler = NewHashRatioSampler() + +func NewHashRatioSampler() *HashRatioSampler { + return &HashRatioSampler{} +} + +func (s *HashRatioSampler) sampleOffset(ts int64, sample *Sample) float64 { + const ( + float64MaxUint64 = float64(math.MaxUint64) + ) + return float64(sample.Metric.Hash()) / float64MaxUint64 +} + +func (s *HashRatioSampler) AddRatioSample(ratioLimit float64, sample *Sample) bool { + // If ratioLimit >= 0: add sample if sampleOffset is lesser than ratioLimit + // + // 0.0 ratioLimit 1.0 + // [---------|--------------------------] + // [#########...........................] + // + // e.g.: + // sampleOffset==0.3 && ratioLimit==0.4 + // 0.3 < 0.4 ? --> add sample + // + // Else if ratioLimit < 0: add sample if rand() return the "complement" of ratioLimit>=0 case + // (loosely similar behavior to negative array index in other programming languages) + // + // 0.0 1+ratioLimit 1.0 + // [---------|--------------------------] + // [.........###########################] + // + // e.g.: + // sampleOffset==0.3 && ratioLimit==-0.6 + // 0.3 >= 0.4 ? --> don't add sample + sampleOffset := s.sampleOffset(sample.T, sample) + return (ratioLimit >= 0 && sampleOffset < ratioLimit) || + (ratioLimit < 0 && sampleOffset >= (1.0+ratioLimit)) +} + type histogramStatsSeries struct { storage.Series } diff --git a/vendor/github.com/prometheus/prometheus/promql/functions.go b/vendor/github.com/prometheus/prometheus/promql/functions.go index 9b3be22874..dcc2cd7590 100644 --- a/vendor/github.com/prometheus/prometheus/promql/functions.go +++ b/vendor/github.com/prometheus/prometheus/promql/functions.go @@ -14,6 +14,7 @@ package promql import ( + "errors" "fmt" "math" "slices" @@ -210,14 +211,28 @@ func histogramRate(points []HPoint, isCounter bool, metricName string, pos posra } h := last.CopyToSchema(minSchema) - h.Sub(prev) + _, err := h.Sub(prev) + if err != nil { + if errors.Is(err, histogram.ErrHistogramsIncompatibleSchema) { + return nil, annotations.New().Add(annotations.NewMixedExponentialCustomHistogramsWarning(metricName, pos)) + } else if errors.Is(err, histogram.ErrHistogramsIncompatibleBounds) { + return nil, annotations.New().Add(annotations.NewIncompatibleCustomBucketsHistogramsWarning(metricName, pos)) + } + } if isCounter { // Second iteration to deal with counter resets. for _, currPoint := range points[1:] { curr := currPoint.H if curr.DetectReset(prev) { - h.Add(prev) + _, err := h.Add(prev) + if err != nil { + if errors.Is(err, histogram.ErrHistogramsIncompatibleSchema) { + return nil, annotations.New().Add(annotations.NewMixedExponentialCustomHistogramsWarning(metricName, pos)) + } else if errors.Is(err, histogram.ErrHistogramsIncompatibleBounds) { + return nil, annotations.New().Add(annotations.NewIncompatibleCustomBucketsHistogramsWarning(metricName, pos)) + } + } } prev = curr } @@ -513,10 +528,11 @@ func aggrOverTime(vals []parser.Value, enh *EvalNodeHelper, aggrFn func(Series) return append(enh.Out, Sample{F: aggrFn(el)}) } -func aggrHistOverTime(vals []parser.Value, enh *EvalNodeHelper, aggrFn func(Series) *histogram.FloatHistogram) Vector { +func aggrHistOverTime(vals []parser.Value, enh *EvalNodeHelper, aggrFn func(Series) (*histogram.FloatHistogram, error)) (Vector, error) { el := vals[0].(Matrix)[0] + res, err := aggrFn(el) - return append(enh.Out, Sample{H: aggrFn(el)}) + return append(enh.Out, Sample{H: res}), err } // === avg_over_time(Matrix parser.ValueTypeMatrix) (Vector, Annotations) === @@ -528,18 +544,33 @@ func funcAvgOverTime(vals []parser.Value, args parser.Expressions, enh *EvalNode } if len(firstSeries.Floats) == 0 { // The passed values only contain histograms. - return aggrHistOverTime(vals, enh, func(s Series) *histogram.FloatHistogram { + vec, err := aggrHistOverTime(vals, enh, func(s Series) (*histogram.FloatHistogram, error) { count := 1 mean := s.Histograms[0].H.Copy() for _, h := range s.Histograms[1:] { count++ left := h.H.Copy().Div(float64(count)) right := mean.Copy().Div(float64(count)) - toAdd := left.Sub(right) - mean.Add(toAdd) + toAdd, err := left.Sub(right) + if err != nil { + return mean, err + } + _, err = mean.Add(toAdd) + if err != nil { + return mean, err + } } - return mean - }), nil + return mean, nil + }) + if err != nil { + metricName := firstSeries.Metric.Get(labels.MetricName) + if errors.Is(err, histogram.ErrHistogramsIncompatibleSchema) { + return enh.Out, annotations.New().Add(annotations.NewMixedExponentialCustomHistogramsWarning(metricName, args[0].PositionRange())) + } else if errors.Is(err, histogram.ErrHistogramsIncompatibleBounds) { + return enh.Out, annotations.New().Add(annotations.NewIncompatibleCustomBucketsHistogramsWarning(metricName, args[0].PositionRange())) + } + } + return vec, nil } return aggrOverTime(vals, enh, func(s Series) float64 { var mean, count, c float64 @@ -673,13 +704,25 @@ func funcSumOverTime(vals []parser.Value, args parser.Expressions, enh *EvalNode } if len(firstSeries.Floats) == 0 { // The passed values only contain histograms. - return aggrHistOverTime(vals, enh, func(s Series) *histogram.FloatHistogram { + vec, err := aggrHistOverTime(vals, enh, func(s Series) (*histogram.FloatHistogram, error) { sum := s.Histograms[0].H.Copy() for _, h := range s.Histograms[1:] { - sum.Add(h.H) + _, err := sum.Add(h.H) + if err != nil { + return sum, err + } } - return sum - }), nil + return sum, nil + }) + if err != nil { + metricName := firstSeries.Metric.Get(labels.MetricName) + if errors.Is(err, histogram.ErrHistogramsIncompatibleSchema) { + return enh.Out, annotations.New().Add(annotations.NewMixedExponentialCustomHistogramsWarning(metricName, args[0].PositionRange())) + } else if errors.Is(err, histogram.ErrHistogramsIncompatibleBounds) { + return enh.Out, annotations.New().Add(annotations.NewIncompatibleCustomBucketsHistogramsWarning(metricName, args[0].PositionRange())) + } + } + return vec, nil } return aggrOverTime(vals, enh, func(s Series) float64 { var sum, c float64 @@ -950,10 +993,14 @@ func funcTimestamp(vals []parser.Value, args parser.Expressions, enh *EvalNodeHe func kahanSumInc(inc, sum, c float64) (newSum, newC float64) { t := sum + inc + switch { + case math.IsInf(t, 0): + c = 0 + // Using Neumaier improvement, swap if next term larger than sum. - if math.Abs(sum) >= math.Abs(inc) { + case math.Abs(sum) >= math.Abs(inc): c += (sum - t) + inc - } else { + default: c += (inc - t) + sum } return t, c diff --git a/vendor/github.com/prometheus/prometheus/promql/parser/generated_parser.y b/vendor/github.com/prometheus/prometheus/promql/parser/generated_parser.y index 841bd31c19..b99e67424f 100644 --- a/vendor/github.com/prometheus/prometheus/promql/parser/generated_parser.y +++ b/vendor/github.com/prometheus/prometheus/promql/parser/generated_parser.y @@ -43,7 +43,6 @@ import ( int int64 uint uint64 float float64 - duration time.Duration } @@ -84,6 +83,7 @@ BUCKETS_DESC NEGATIVE_BUCKETS_DESC ZERO_BUCKET_DESC ZERO_BUCKET_WIDTH_DESC +CUSTOM_VALUES_DESC %token histogramDescEnd // Operators. @@ -125,6 +125,8 @@ STDDEV STDVAR SUM TOPK +LIMITK +LIMIT_RATIO %token aggregatorsEnd // Keywords. @@ -173,8 +175,7 @@ START_METRIC_SELECTOR %type int %type uint %type number series_value signed_number signed_or_unsigned_number -%type step_invariant_expr aggregate_expr aggregate_modifier bin_modifier binary_expr bool_modifier expr function_call function_call_args function_call_body group_modifiers label_matchers matrix_selector number_literal offset_expr on_or_ignoring paren_expr string_literal subquery_expr unary_expr vector_selector -%type duration maybe_duration +%type step_invariant_expr aggregate_expr aggregate_modifier bin_modifier binary_expr bool_modifier expr function_call function_call_args function_call_body group_modifiers label_matchers matrix_selector number_duration_literal offset_expr on_or_ignoring paren_expr string_literal subquery_expr unary_expr vector_selector %start start @@ -215,7 +216,7 @@ expr : | binary_expr | function_call | matrix_selector - | number_literal + | number_duration_literal | offset_expr | paren_expr | string_literal @@ -412,18 +413,22 @@ paren_expr : LEFT_PAREN expr RIGHT_PAREN * Offset modifiers. */ -offset_expr: expr OFFSET duration +offset_expr: expr OFFSET number_duration_literal { - yylex.(*parser).addOffset($1, $3) - $$ = $1 + numLit, _ := $3.(*NumberLiteral) + dur := time.Duration(numLit.Val * 1000) * time.Millisecond + yylex.(*parser).addOffset($1, dur) + $$ = $1 } - | expr OFFSET SUB duration + | expr OFFSET SUB number_duration_literal { - yylex.(*parser).addOffset($1, -$4) - $$ = $1 + numLit, _ := $4.(*NumberLiteral) + dur := time.Duration(numLit.Val * 1000) * time.Millisecond + yylex.(*parser).addOffset($1, -dur) + $$ = $1 } | expr OFFSET error - { yylex.(*parser).unexpected("offset", "duration"); $$ = $1 } + { yylex.(*parser).unexpected("offset", "number or duration"); $$ = $1 } ; /* * @ modifiers. @@ -449,7 +454,7 @@ at_modifier_preprocessors: START | END; * Subquery and range selectors. */ -matrix_selector : expr LEFT_BRACKET duration RIGHT_BRACKET +matrix_selector : expr LEFT_BRACKET number_duration_literal RIGHT_BRACKET { var errMsg string vs, ok := $1.(*VectorSelector) @@ -466,32 +471,44 @@ matrix_selector : expr LEFT_BRACKET duration RIGHT_BRACKET yylex.(*parser).addParseErrf(errRange, errMsg) } + numLit, _ := $3.(*NumberLiteral) $$ = &MatrixSelector{ VectorSelector: $1.(Expr), - Range: $3, + Range: time.Duration(numLit.Val * 1000) * time.Millisecond, EndPos: yylex.(*parser).lastClosing, } } ; -subquery_expr : expr LEFT_BRACKET duration COLON maybe_duration RIGHT_BRACKET +subquery_expr : expr LEFT_BRACKET number_duration_literal COLON number_duration_literal RIGHT_BRACKET { + numLitRange, _ := $3.(*NumberLiteral) + numLitStep, _ := $5.(*NumberLiteral) $$ = &SubqueryExpr{ Expr: $1.(Expr), - Range: $3, - Step: $5, - + Range: time.Duration(numLitRange.Val * 1000) * time.Millisecond, + Step: time.Duration(numLitStep.Val * 1000) * time.Millisecond, EndPos: $6.Pos + 1, } } - | expr LEFT_BRACKET duration COLON duration error + | expr LEFT_BRACKET number_duration_literal COLON RIGHT_BRACKET + { + numLitRange, _ := $3.(*NumberLiteral) + $$ = &SubqueryExpr{ + Expr: $1.(Expr), + Range: time.Duration(numLitRange.Val * 1000) * time.Millisecond, + Step: 0, + EndPos: $5.Pos + 1, + } + } + | expr LEFT_BRACKET number_duration_literal COLON number_duration_literal error { yylex.(*parser).unexpected("subquery selector", "\"]\""); $$ = $1 } - | expr LEFT_BRACKET duration COLON error - { yylex.(*parser).unexpected("subquery selector", "duration or \"]\""); $$ = $1 } - | expr LEFT_BRACKET duration error + | expr LEFT_BRACKET number_duration_literal COLON error + { yylex.(*parser).unexpected("subquery selector", "number or duration or \"]\""); $$ = $1 } + | expr LEFT_BRACKET number_duration_literal error { yylex.(*parser).unexpected("subquery or range", "\":\" or \"]\""); $$ = $1 } | expr LEFT_BRACKET error - { yylex.(*parser).unexpected("subquery selector", "duration"); $$ = $1 } + { yylex.(*parser).unexpected("subquery selector", "number or duration"); $$ = $1 } ; /* @@ -608,7 +625,7 @@ metric : metric_identifier label_set ; -metric_identifier: AVG | BOTTOMK | BY | COUNT | COUNT_VALUES | GROUP | IDENTIFIER | LAND | LOR | LUNLESS | MAX | METRIC_IDENTIFIER | MIN | OFFSET | QUANTILE | STDDEV | STDVAR | SUM | TOPK | WITHOUT | START | END; +metric_identifier: AVG | BOTTOMK | BY | COUNT | COUNT_VALUES | GROUP | IDENTIFIER | LAND | LOR | LUNLESS | MAX | METRIC_IDENTIFIER | MIN | OFFSET | QUANTILE | STDDEV | STDVAR | SUM | TOPK | WITHOUT | START | END | LIMITK | LIMIT_RATIO; label_set : LEFT_BRACE label_set_list RIGHT_BRACE { $$ = labels.New($2...) } @@ -797,6 +814,11 @@ histogram_desc_item $$ = yylex.(*parser).newMap() $$["z_bucket_w"] = $3 } + | CUSTOM_VALUES_DESC COLON bucket_set + { + $$ = yylex.(*parser).newMap() + $$["custom_values"] = $3 + } | BUCKETS_DESC COLON bucket_set { $$ = yylex.(*parser).newMap() @@ -845,10 +867,10 @@ bucket_set_list : bucket_set_list SPACE number * Keyword lists. */ -aggregate_op : AVG | BOTTOMK | COUNT | COUNT_VALUES | GROUP | MAX | MIN | QUANTILE | STDDEV | STDVAR | SUM | TOPK ; +aggregate_op : AVG | BOTTOMK | COUNT | COUNT_VALUES | GROUP | MAX | MIN | QUANTILE | STDDEV | STDVAR | SUM | TOPK | LIMITK | LIMIT_RATIO; // Inside of grouping options label names can be recognized as keywords by the lexer. This is a list of keywords that could also be a label name. -maybe_label : AVG | BOOL | BOTTOMK | BY | COUNT | COUNT_VALUES | GROUP | GROUP_LEFT | GROUP_RIGHT | IDENTIFIER | IGNORING | LAND | LOR | LUNLESS | MAX | METRIC_IDENTIFIER | MIN | OFFSET | ON | QUANTILE | STDDEV | STDVAR | SUM | TOPK | START | END | ATAN2; +maybe_label : AVG | BOOL | BOTTOMK | BY | COUNT | COUNT_VALUES | GROUP | GROUP_LEFT | GROUP_RIGHT | IDENTIFIER | IGNORING | LAND | LOR | LUNLESS | MAX | METRIC_IDENTIFIER | MIN | OFFSET | ON | QUANTILE | STDDEV | STDVAR | SUM | TOPK | START | END | ATAN2 | LIMITK | LIMIT_RATIO; unary_op : ADD | SUB; @@ -858,16 +880,43 @@ match_op : EQL | NEQ | EQL_REGEX | NEQ_REGEX ; * Literals. */ -number_literal : NUMBER +number_duration_literal : NUMBER { - $$ = &NumberLiteral{ + $$ = &NumberLiteral{ Val: yylex.(*parser).number($1.Val), PosRange: $1.PositionRange(), + } } + | DURATION + { + var err error + var dur time.Duration + dur, err = parseDuration($1.Val) + if err != nil { + yylex.(*parser).addParseErr($1.PositionRange(), err) + } + $$ = &NumberLiteral{ + Val: dur.Seconds(), + PosRange: $1.PositionRange(), + } } ; -number : NUMBER { $$ = yylex.(*parser).number($1.Val) } ; +number : NUMBER + { + $$ = yylex.(*parser).number($1.Val) + } + | DURATION + { + var err error + var dur time.Duration + dur, err = parseDuration($1.Val) + if err != nil { + yylex.(*parser).addParseErr($1.PositionRange(), err) + } + $$ = dur.Seconds() + } + ; signed_number : ADD number { $$ = $2 } | SUB number { $$ = -$2 } @@ -889,17 +938,6 @@ int : SUB uint { $$ = -int64($2) } | uint { $$ = int64($1) } ; -duration : DURATION - { - var err error - $$, err = parseDuration($1.Val) - if err != nil { - yylex.(*parser).addParseErr($1.PositionRange(), err) - } - } - ; - - string_literal : STRING { $$ = &StringLiteral{ @@ -923,11 +961,6 @@ string_identifier : STRING * Wrappers for optional arguments. */ -maybe_duration : /* empty */ - {$$ = 0} - | duration - ; - maybe_grouping_labels: /* empty */ { $$ = nil } | grouping_labels ; diff --git a/vendor/github.com/prometheus/prometheus/promql/parser/generated_parser.y.go b/vendor/github.com/prometheus/prometheus/promql/parser/generated_parser.y.go index 3075b9b1b1..423082dafe 100644 --- a/vendor/github.com/prometheus/prometheus/promql/parser/generated_parser.y.go +++ b/vendor/github.com/prometheus/prometheus/promql/parser/generated_parser.y.go @@ -31,7 +31,6 @@ type yySymType struct { int int64 uint uint64 float float64 - duration time.Duration } const EQL = 57346 @@ -67,62 +66,65 @@ const BUCKETS_DESC = 57375 const NEGATIVE_BUCKETS_DESC = 57376 const ZERO_BUCKET_DESC = 57377 const ZERO_BUCKET_WIDTH_DESC = 57378 -const histogramDescEnd = 57379 -const operatorsStart = 57380 -const ADD = 57381 -const DIV = 57382 -const EQLC = 57383 -const EQL_REGEX = 57384 -const GTE = 57385 -const GTR = 57386 -const LAND = 57387 -const LOR = 57388 -const LSS = 57389 -const LTE = 57390 -const LUNLESS = 57391 -const MOD = 57392 -const MUL = 57393 -const NEQ = 57394 -const NEQ_REGEX = 57395 -const POW = 57396 -const SUB = 57397 -const AT = 57398 -const ATAN2 = 57399 -const operatorsEnd = 57400 -const aggregatorsStart = 57401 -const AVG = 57402 -const BOTTOMK = 57403 -const COUNT = 57404 -const COUNT_VALUES = 57405 -const GROUP = 57406 -const MAX = 57407 -const MIN = 57408 -const QUANTILE = 57409 -const STDDEV = 57410 -const STDVAR = 57411 -const SUM = 57412 -const TOPK = 57413 -const aggregatorsEnd = 57414 -const keywordsStart = 57415 -const BOOL = 57416 -const BY = 57417 -const GROUP_LEFT = 57418 -const GROUP_RIGHT = 57419 -const IGNORING = 57420 -const OFFSET = 57421 -const ON = 57422 -const WITHOUT = 57423 -const keywordsEnd = 57424 -const preprocessorStart = 57425 -const START = 57426 -const END = 57427 -const preprocessorEnd = 57428 -const startSymbolsStart = 57429 -const START_METRIC = 57430 -const START_SERIES_DESCRIPTION = 57431 -const START_EXPRESSION = 57432 -const START_METRIC_SELECTOR = 57433 -const startSymbolsEnd = 57434 +const CUSTOM_VALUES_DESC = 57379 +const histogramDescEnd = 57380 +const operatorsStart = 57381 +const ADD = 57382 +const DIV = 57383 +const EQLC = 57384 +const EQL_REGEX = 57385 +const GTE = 57386 +const GTR = 57387 +const LAND = 57388 +const LOR = 57389 +const LSS = 57390 +const LTE = 57391 +const LUNLESS = 57392 +const MOD = 57393 +const MUL = 57394 +const NEQ = 57395 +const NEQ_REGEX = 57396 +const POW = 57397 +const SUB = 57398 +const AT = 57399 +const ATAN2 = 57400 +const operatorsEnd = 57401 +const aggregatorsStart = 57402 +const AVG = 57403 +const BOTTOMK = 57404 +const COUNT = 57405 +const COUNT_VALUES = 57406 +const GROUP = 57407 +const MAX = 57408 +const MIN = 57409 +const QUANTILE = 57410 +const STDDEV = 57411 +const STDVAR = 57412 +const SUM = 57413 +const TOPK = 57414 +const LIMITK = 57415 +const LIMIT_RATIO = 57416 +const aggregatorsEnd = 57417 +const keywordsStart = 57418 +const BOOL = 57419 +const BY = 57420 +const GROUP_LEFT = 57421 +const GROUP_RIGHT = 57422 +const IGNORING = 57423 +const OFFSET = 57424 +const ON = 57425 +const WITHOUT = 57426 +const keywordsEnd = 57427 +const preprocessorStart = 57428 +const START = 57429 +const END = 57430 +const preprocessorEnd = 57431 +const startSymbolsStart = 57432 +const START_METRIC = 57433 +const START_SERIES_DESCRIPTION = 57434 +const START_EXPRESSION = 57435 +const START_METRIC_SELECTOR = 57436 +const startSymbolsEnd = 57437 var yyToknames = [...]string{ "$end", @@ -161,6 +163,7 @@ var yyToknames = [...]string{ "NEGATIVE_BUCKETS_DESC", "ZERO_BUCKET_DESC", "ZERO_BUCKET_WIDTH_DESC", + "CUSTOM_VALUES_DESC", "histogramDescEnd", "operatorsStart", "ADD", @@ -196,6 +199,8 @@ var yyToknames = [...]string{ "STDVAR", "SUM", "TOPK", + "LIMITK", + "LIMIT_RATIO", "aggregatorsEnd", "keywordsStart", "BOOL", @@ -229,280 +234,291 @@ var yyExca = [...]int16{ -1, 1, 1, -1, -2, 0, - -1, 35, - 1, 134, - 10, 134, - 24, 134, + -1, 37, + 1, 137, + 10, 137, + 24, 137, -2, 0, - -1, 58, - 2, 171, - 15, 171, - 75, 171, - 81, 171, - -2, 100, - -1, 59, - 2, 172, - 15, 172, - 75, 172, - 81, 172, - -2, 101, - -1, 60, - 2, 173, - 15, 173, - 75, 173, - 81, 173, - -2, 103, -1, 61, - 2, 174, - 15, 174, - 75, 174, - 81, 174, - -2, 104, - -1, 62, 2, 175, 15, 175, - 75, 175, - 81, 175, - -2, 105, - -1, 63, + 78, 175, + 84, 175, + -2, 101, + -1, 62, 2, 176, 15, 176, - 75, 176, - 81, 176, - -2, 110, - -1, 64, + 78, 176, + 84, 176, + -2, 102, + -1, 63, 2, 177, 15, 177, - 75, 177, - 81, 177, - -2, 112, - -1, 65, + 78, 177, + 84, 177, + -2, 104, + -1, 64, 2, 178, 15, 178, - 75, 178, - 81, 178, - -2, 114, - -1, 66, + 78, 178, + 84, 178, + -2, 105, + -1, 65, 2, 179, 15, 179, - 75, 179, - 81, 179, - -2, 115, - -1, 67, + 78, 179, + 84, 179, + -2, 106, + -1, 66, 2, 180, 15, 180, - 75, 180, - 81, 180, - -2, 116, - -1, 68, + 78, 180, + 84, 180, + -2, 111, + -1, 67, 2, 181, 15, 181, - 75, 181, - 81, 181, - -2, 117, - -1, 69, + 78, 181, + 84, 181, + -2, 113, + -1, 68, 2, 182, 15, 182, - 75, 182, - 81, 182, + 78, 182, + 84, 182, + -2, 115, + -1, 69, + 2, 183, + 15, 183, + 78, 183, + 84, 183, + -2, 116, + -1, 70, + 2, 184, + 15, 184, + 78, 184, + 84, 184, + -2, 117, + -1, 71, + 2, 185, + 15, 185, + 78, 185, + 84, 185, -2, 118, - -1, 195, - 12, 230, - 13, 230, - 18, 230, - 19, 230, - 25, 230, - 39, 230, - 45, 230, - 46, 230, - 49, 230, - 55, 230, - 60, 230, - 61, 230, - 62, 230, - 63, 230, - 64, 230, - 65, 230, - 66, 230, - 67, 230, - 68, 230, - 69, 230, - 70, 230, - 71, 230, - 75, 230, - 79, 230, - 81, 230, - 84, 230, - 85, 230, - -2, 0, - -1, 196, - 12, 230, - 13, 230, - 18, 230, - 19, 230, - 25, 230, - 39, 230, - 45, 230, - 46, 230, - 49, 230, - 55, 230, - 60, 230, - 61, 230, - 62, 230, - 63, 230, - 64, 230, - 65, 230, - 66, 230, - 67, 230, - 68, 230, - 69, 230, - 70, 230, - 71, 230, - 75, 230, - 79, 230, - 81, 230, - 84, 230, - 85, 230, + -1, 72, + 2, 186, + 15, 186, + 78, 186, + 84, 186, + -2, 119, + -1, 73, + 2, 187, + 15, 187, + 78, 187, + 84, 187, + -2, 123, + -1, 74, + 2, 188, + 15, 188, + 78, 188, + 84, 188, + -2, 124, + -1, 200, + 9, 237, + 12, 237, + 13, 237, + 18, 237, + 19, 237, + 25, 237, + 40, 237, + 46, 237, + 47, 237, + 50, 237, + 56, 237, + 61, 237, + 62, 237, + 63, 237, + 64, 237, + 65, 237, + 66, 237, + 67, 237, + 68, 237, + 69, 237, + 70, 237, + 71, 237, + 72, 237, + 73, 237, + 74, 237, + 78, 237, + 82, 237, + 84, 237, + 87, 237, + 88, 237, -2, 0, - -1, 217, - 21, 228, - -2, 0, - -1, 285, - 21, 229, + -1, 201, + 9, 237, + 12, 237, + 13, 237, + 18, 237, + 19, 237, + 25, 237, + 40, 237, + 46, 237, + 47, 237, + 50, 237, + 56, 237, + 61, 237, + 62, 237, + 63, 237, + 64, 237, + 65, 237, + 66, 237, + 67, 237, + 68, 237, + 69, 237, + 70, 237, + 71, 237, + 72, 237, + 73, 237, + 74, 237, + 78, 237, + 82, 237, + 84, 237, + 87, 237, + 88, 237, -2, 0, } const yyPrivate = 57344 -const yyLast = 742 +const yyLast = 728 var yyAct = [...]int16{ - 151, 322, 320, 268, 327, 148, 221, 37, 187, 144, - 281, 280, 152, 113, 77, 173, 104, 102, 101, 6, - 128, 223, 105, 193, 155, 194, 195, 196, 339, 262, - 260, 233, 317, 316, 57, 100, 294, 239, 103, 146, - 300, 313, 263, 156, 156, 283, 147, 338, 259, 123, - 337, 106, 252, 311, 155, 299, 340, 301, 264, 157, - 157, 108, 298, 109, 235, 236, 292, 251, 237, 107, - 155, 292, 174, 191, 175, 96, 250, 99, 258, 224, - 226, 228, 229, 230, 238, 240, 243, 244, 245, 246, - 247, 110, 145, 225, 227, 231, 232, 234, 241, 242, - 98, 257, 321, 248, 249, 2, 3, 4, 5, 218, - 158, 104, 177, 217, 168, 162, 165, 105, 175, 160, - 164, 161, 176, 178, 189, 213, 106, 328, 216, 256, - 183, 179, 192, 163, 181, 100, 190, 197, 198, 199, - 200, 201, 202, 203, 204, 205, 206, 207, 208, 209, - 210, 211, 255, 182, 72, 212, 177, 214, 215, 33, - 82, 84, 85, 7, 86, 87, 176, 178, 90, 91, - 223, 93, 94, 95, 116, 96, 97, 99, 83, 147, - 233, 286, 289, 116, 114, 254, 239, 288, 147, 172, - 220, 124, 253, 114, 171, 310, 309, 117, 120, 261, - 98, 112, 287, 119, 278, 279, 117, 170, 282, 10, - 308, 159, 307, 235, 236, 312, 118, 237, 147, 74, - 306, 305, 304, 303, 302, 250, 81, 285, 224, 226, - 228, 229, 230, 238, 240, 243, 244, 245, 246, 247, - 79, 79, 225, 227, 231, 232, 234, 241, 242, 48, - 78, 78, 248, 249, 122, 73, 121, 150, 180, 76, - 290, 291, 293, 56, 295, 8, 9, 9, 34, 35, - 1, 284, 296, 297, 155, 129, 130, 131, 132, 133, - 134, 135, 136, 137, 138, 139, 140, 141, 142, 143, - 47, 46, 45, 44, 156, 314, 315, 127, 43, 42, - 41, 185, 319, 125, 166, 324, 325, 326, 188, 323, - 157, 329, 191, 331, 330, 155, 40, 126, 332, 333, - 100, 51, 72, 334, 53, 39, 38, 22, 52, 336, - 49, 167, 186, 335, 54, 156, 265, 80, 341, 153, - 154, 184, 219, 75, 115, 82, 84, 149, 70, 55, - 222, 157, 50, 111, 18, 19, 93, 94, 20, 0, - 96, 97, 99, 83, 71, 0, 0, 0, 0, 58, - 59, 60, 61, 62, 63, 64, 65, 66, 67, 68, - 69, 0, 0, 0, 13, 98, 0, 0, 24, 0, - 30, 0, 0, 31, 32, 36, 100, 51, 72, 0, - 53, 267, 0, 22, 52, 0, 0, 0, 266, 0, - 54, 0, 270, 271, 269, 275, 277, 274, 276, 272, - 273, 0, 84, 0, 70, 0, 0, 0, 0, 0, - 18, 19, 93, 94, 20, 0, 96, 0, 99, 83, - 71, 0, 0, 0, 0, 58, 59, 60, 61, 62, - 63, 64, 65, 66, 67, 68, 69, 0, 0, 0, - 13, 98, 0, 0, 24, 0, 30, 0, 0, 31, - 32, 51, 72, 0, 53, 318, 0, 22, 52, 0, - 0, 0, 0, 0, 54, 0, 270, 271, 269, 275, - 277, 274, 276, 272, 273, 0, 0, 0, 70, 0, - 0, 0, 0, 0, 18, 19, 0, 0, 20, 0, - 0, 0, 17, 72, 71, 0, 0, 0, 22, 58, - 59, 60, 61, 62, 63, 64, 65, 66, 67, 68, - 69, 0, 0, 0, 13, 0, 0, 0, 24, 0, - 30, 0, 0, 31, 32, 18, 19, 0, 0, 20, - 0, 0, 0, 17, 33, 0, 0, 0, 0, 22, - 11, 12, 14, 15, 16, 21, 23, 25, 26, 27, - 28, 29, 0, 0, 0, 13, 0, 0, 0, 24, - 0, 30, 0, 0, 31, 32, 18, 19, 0, 0, - 20, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 11, 12, 14, 15, 16, 21, 23, 25, 26, - 27, 28, 29, 100, 0, 0, 13, 0, 0, 0, - 24, 169, 30, 0, 0, 31, 32, 0, 0, 0, - 0, 0, 100, 0, 0, 0, 0, 0, 82, 84, - 85, 0, 86, 87, 88, 89, 90, 91, 92, 93, - 94, 95, 0, 96, 97, 99, 83, 82, 84, 85, - 0, 86, 87, 88, 89, 90, 91, 92, 93, 94, - 95, 0, 96, 97, 99, 83, 100, 0, 98, 0, + 155, 331, 329, 275, 336, 152, 226, 39, 192, 44, + 289, 288, 156, 118, 82, 178, 55, 106, 6, 53, + 77, 109, 56, 133, 108, 22, 54, 110, 107, 172, + 300, 198, 57, 199, 200, 201, 60, 111, 326, 151, + 325, 302, 321, 308, 266, 154, 55, 75, 128, 105, + 291, 300, 160, 18, 19, 309, 54, 20, 307, 218, + 105, 320, 159, 76, 113, 306, 114, 330, 61, 62, + 63, 64, 65, 66, 67, 68, 69, 70, 71, 72, + 73, 74, 112, 161, 180, 13, 87, 89, 265, 24, + 101, 30, 104, 150, 31, 32, 115, 98, 99, 162, + 109, 101, 102, 104, 88, 349, 110, 2, 3, 4, + 5, 264, 196, 149, 111, 163, 160, 103, 337, 173, + 167, 170, 84, 182, 348, 166, 159, 347, 103, 194, + 157, 158, 83, 181, 183, 165, 184, 197, 77, 186, + 185, 195, 202, 203, 204, 205, 206, 207, 208, 209, + 210, 211, 212, 213, 214, 215, 216, 129, 269, 263, + 217, 160, 219, 220, 55, 38, 35, 53, 77, 267, + 56, 159, 270, 22, 54, 121, 297, 188, 7, 259, + 57, 296, 262, 161, 319, 119, 318, 317, 271, 179, + 261, 180, 161, 260, 258, 75, 295, 84, 122, 162, + 187, 18, 19, 316, 268, 20, 315, 83, 162, 286, + 287, 76, 314, 290, 313, 81, 61, 62, 63, 64, + 65, 66, 67, 68, 69, 70, 71, 72, 73, 74, + 182, 86, 292, 13, 55, 10, 312, 24, 311, 30, + 181, 183, 31, 32, 54, 79, 134, 135, 136, 137, + 138, 139, 140, 141, 142, 143, 144, 145, 146, 147, + 148, 310, 127, 36, 126, 1, 121, 298, 299, 301, + 164, 303, 49, 48, 190, 294, 119, 55, 160, 304, + 305, 193, 55, 160, 117, 196, 223, 54, 159, 122, + 222, 228, 54, 159, 293, 350, 50, 47, 46, 169, + 132, 238, 78, 323, 324, 221, 45, 244, 43, 161, + 328, 322, 168, 333, 334, 335, 130, 332, 171, 177, + 339, 338, 341, 340, 176, 162, 125, 342, 343, 42, + 59, 124, 344, 9, 9, 240, 241, 175, 346, 242, + 131, 8, 41, 40, 123, 37, 51, 255, 351, 191, + 229, 231, 233, 234, 235, 243, 245, 248, 249, 250, + 251, 252, 256, 257, 345, 272, 230, 232, 236, 237, + 239, 246, 247, 85, 189, 55, 253, 254, 53, 77, + 224, 56, 80, 120, 22, 54, 153, 58, 227, 52, + 116, 57, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 228, 0, 0, 0, 0, 75, 0, 0, 0, + 0, 238, 18, 19, 0, 0, 20, 244, 0, 0, + 0, 225, 76, 0, 0, 0, 0, 61, 62, 63, + 64, 65, 66, 67, 68, 69, 70, 71, 72, 73, + 74, 0, 0, 0, 13, 240, 241, 0, 24, 242, + 30, 0, 0, 31, 32, 0, 0, 255, 105, 0, + 229, 231, 233, 234, 235, 243, 245, 248, 249, 250, + 251, 252, 256, 257, 0, 0, 230, 232, 236, 237, + 239, 246, 247, 17, 77, 89, 253, 254, 0, 22, + 0, 0, 327, 0, 0, 98, 99, 0, 0, 101, + 0, 104, 88, 277, 278, 276, 283, 285, 282, 284, + 279, 280, 281, 17, 35, 0, 0, 18, 19, 22, + 0, 20, 0, 0, 0, 0, 103, 0, 0, 0, + 0, 0, 11, 12, 14, 15, 16, 21, 23, 25, + 26, 27, 28, 29, 33, 34, 0, 18, 19, 13, + 0, 20, 0, 24, 0, 30, 0, 0, 31, 32, + 0, 0, 11, 12, 14, 15, 16, 21, 23, 25, + 26, 27, 28, 29, 33, 34, 105, 0, 0, 13, + 0, 0, 0, 24, 174, 30, 0, 0, 31, 32, + 0, 0, 0, 0, 0, 105, 0, 0, 0, 0, + 0, 0, 87, 89, 90, 0, 91, 92, 93, 94, + 95, 96, 97, 98, 99, 100, 0, 101, 102, 104, + 88, 87, 89, 90, 0, 91, 92, 93, 94, 95, + 96, 97, 98, 99, 100, 274, 101, 102, 104, 88, + 105, 0, 273, 0, 103, 0, 277, 278, 276, 283, + 285, 282, 284, 279, 280, 281, 0, 0, 0, 105, + 0, 0, 0, 103, 0, 0, 87, 89, 90, 0, + 91, 92, 93, 0, 95, 96, 97, 98, 99, 100, + 0, 101, 102, 104, 88, 87, 89, 90, 0, 91, + 92, 0, 0, 95, 96, 0, 98, 99, 100, 0, + 101, 102, 104, 88, 0, 0, 0, 0, 103, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 98, 0, 0, - 0, 82, 84, 85, 0, 86, 87, 88, 0, 90, - 91, 92, 93, 94, 95, 0, 96, 97, 99, 83, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 98, + 0, 0, 0, 0, 0, 0, 0, 103, } var yyPact = [...]int16{ - 17, 153, 541, 541, 385, 500, -1000, -1000, -1000, 146, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + 16, 168, 501, 501, 155, 471, -1000, -1000, -1000, 153, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, 239, -1000, 224, -1000, 618, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - 36, 111, -1000, 459, -1000, 459, 141, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, 195, -1000, 229, -1000, 581, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, 181, -1000, -1000, 196, -1000, -1000, 252, -1000, - 25, -1000, -54, -54, -54, -54, -54, -54, -54, -54, - -54, -54, -54, -54, -54, -54, -54, -54, 37, 255, - 209, 111, -59, -1000, 118, 118, 309, -1000, 599, 21, - -1000, 187, -1000, -1000, 70, 114, -1000, -1000, -1000, 238, - -1000, 128, -1000, 296, 459, -1000, -55, -50, -1000, 459, - 459, 459, 459, 459, 459, 459, 459, 459, 459, 459, - 459, 459, 459, 459, -1000, 170, -1000, -1000, -1000, 110, - -1000, -1000, -1000, -1000, -1000, -1000, 51, 51, 107, -1000, - -1000, -1000, -1000, 168, -1000, -1000, 45, -1000, 618, -1000, - -1000, 172, -1000, 127, -1000, -1000, -1000, -1000, -1000, 76, - -1000, -1000, -1000, -1000, -1000, 22, 4, 3, -1000, -1000, - -1000, 384, 382, 118, 118, 118, 118, 21, 21, 306, - 306, 306, 121, 662, 306, 306, 121, 21, 21, 306, - 21, 382, -1000, 23, -1000, -1000, -1000, 179, -1000, 180, + -1000, -1000, 22, 99, -1000, -1000, 366, -1000, 366, 125, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, 264, -1000, -1000, + 324, -1000, -1000, 260, -1000, 24, -1000, -54, -54, -54, + -54, -54, -54, -54, -54, -54, -54, -54, -54, -54, + -54, -54, -54, 37, 43, 268, 99, -57, -1000, 297, + 297, 7, -1000, 562, 35, -1000, 317, -1000, -1000, 187, + 80, -1000, -1000, -1000, 120, -1000, 175, -1000, 269, 366, + -1000, -50, -45, -1000, 366, 366, 366, 366, 366, 366, + 366, 366, 366, 366, 366, 366, 366, 366, 366, -1000, + 225, -1000, -1000, 44, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, 107, 107, 284, -1000, -1000, -1000, -1000, 399, -1000, + -1000, 172, -1000, 581, -1000, -1000, 173, -1000, 157, -1000, + -1000, -1000, -1000, -1000, 86, -1000, -1000, -1000, -1000, -1000, + 18, 143, 132, -1000, -1000, -1000, 618, 444, 297, 297, + 297, 297, 35, 35, 46, 46, 46, 645, 626, 46, + 46, 645, 35, 35, 46, 35, 444, -1000, 28, -1000, + -1000, -1000, 273, -1000, 174, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, 459, -1000, -1000, -1000, -1000, -1000, -1000, 52, - 52, 10, 52, 57, 57, 38, 40, -1000, -1000, 218, - 217, 216, 215, 214, 206, 204, 190, 189, -1000, -1000, - -1000, -1000, -1000, -1000, 32, 213, -1000, -1000, 19, -1000, - 618, -1000, -1000, -1000, 52, -1000, 7, 6, 458, -1000, - -1000, -1000, 47, 5, 51, 51, 51, 113, 47, 113, - 47, -1000, -1000, -1000, -1000, -1000, 52, 52, -1000, -1000, - -1000, 52, -1000, -1000, -1000, -1000, -1000, -1000, 51, -1000, - -1000, -1000, -1000, -1000, -1000, 26, -1000, 35, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 366, + -1000, -1000, -1000, -1000, -1000, -1000, 32, 32, 15, 32, + 96, 96, 41, 38, -1000, -1000, 255, 232, 230, 208, + 206, 200, 197, 181, 180, 178, -1000, -1000, -1000, -1000, + -1000, -1000, 40, -1000, -1000, -1000, 289, -1000, 581, -1000, + -1000, -1000, 32, -1000, 14, 12, 475, -1000, -1000, -1000, + 11, 152, 107, 107, 107, 104, 104, 11, 104, 11, + -1000, -1000, -1000, -1000, -1000, 32, 32, -1000, -1000, -1000, + 32, -1000, -1000, -1000, -1000, -1000, -1000, 107, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, 103, -1000, 274, -1000, -1000, -1000, -1000, } var yyPgo = [...]int16{ - 0, 353, 13, 352, 6, 15, 350, 263, 349, 347, - 344, 209, 265, 343, 14, 342, 10, 11, 341, 337, - 8, 336, 3, 4, 333, 2, 1, 0, 332, 12, - 5, 330, 326, 18, 191, 325, 317, 7, 316, 304, - 17, 303, 34, 300, 299, 298, 297, 293, 292, 291, - 290, 249, 9, 271, 270, 268, + 0, 390, 13, 389, 6, 15, 388, 330, 387, 386, + 383, 235, 341, 382, 14, 380, 10, 11, 374, 373, + 8, 365, 3, 4, 364, 2, 1, 0, 349, 12, + 5, 346, 343, 17, 157, 342, 340, 7, 329, 318, + 28, 316, 36, 308, 9, 306, 300, 298, 297, 273, + 272, 296, 265, 263, } var yyR1 = [...]int8{ - 0, 54, 54, 54, 54, 54, 54, 54, 37, 37, + 0, 52, 52, 52, 52, 52, 52, 52, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 32, 32, 32, 32, 33, 33, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, @@ -510,22 +526,22 @@ var yyR1 = [...]int8{ 41, 16, 16, 16, 16, 15, 15, 15, 4, 4, 38, 40, 40, 39, 39, 39, 47, 45, 45, 45, 31, 31, 31, 9, 9, 43, 49, 49, 49, 49, - 49, 50, 51, 51, 51, 42, 42, 42, 1, 1, - 1, 2, 2, 2, 2, 2, 2, 2, 12, 12, + 49, 49, 50, 51, 51, 51, 42, 42, 42, 1, + 1, 1, 2, 2, 2, 2, 2, 2, 2, 12, + 12, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, - 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, - 7, 7, 11, 11, 11, 11, 13, 13, 13, 14, - 14, 14, 14, 55, 19, 19, 19, 19, 18, 18, - 18, 18, 18, 18, 18, 18, 18, 28, 28, 28, - 20, 20, 20, 20, 21, 21, 21, 22, 22, 22, - 22, 22, 22, 22, 22, 22, 23, 23, 24, 24, - 24, 3, 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 6, 6, 6, 6, 6, 6, 6, + 7, 7, 7, 7, 7, 11, 11, 11, 11, 13, + 13, 13, 14, 14, 14, 14, 53, 19, 19, 19, + 19, 18, 18, 18, 18, 18, 18, 18, 18, 18, + 28, 28, 28, 20, 20, 20, 20, 21, 21, 21, + 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, + 23, 23, 24, 24, 24, 3, 3, 3, 3, 3, + 3, 3, 3, 3, 3, 3, 3, 3, 3, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, - 8, 8, 5, 5, 5, 5, 44, 27, 29, 29, - 30, 30, 26, 25, 25, 52, 48, 10, 53, 53, - 17, 17, + 6, 6, 6, 6, 6, 6, 6, 6, 8, 8, + 5, 5, 5, 5, 44, 44, 27, 27, 29, 29, + 30, 30, 26, 25, 25, 48, 10, 17, 17, } var yyR2 = [...]int8{ @@ -536,99 +552,101 @@ var yyR2 = [...]int8{ 4, 4, 1, 0, 1, 3, 3, 1, 1, 3, 3, 3, 4, 2, 1, 3, 1, 2, 1, 1, 2, 3, 2, 3, 1, 2, 3, 3, 4, 3, - 3, 5, 3, 1, 1, 4, 6, 6, 5, 4, - 3, 2, 2, 1, 1, 3, 4, 2, 3, 1, - 2, 3, 3, 1, 3, 3, 2, 1, 2, 1, + 3, 5, 3, 1, 1, 4, 6, 5, 6, 5, + 4, 3, 2, 2, 1, 1, 3, 4, 2, 3, + 1, 2, 3, 3, 1, 3, 3, 2, 1, 2, + 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, + 1, 1, 1, 1, 1, 3, 4, 2, 0, 3, + 1, 2, 3, 3, 2, 1, 2, 0, 3, 2, + 1, 1, 3, 1, 3, 4, 1, 3, 5, 5, + 1, 1, 1, 4, 3, 3, 2, 3, 1, 2, + 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, + 4, 3, 3, 1, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 3, 4, 2, 0, 3, 1, 2, 3, - 3, 2, 1, 2, 0, 3, 2, 1, 1, 3, - 1, 3, 4, 1, 3, 5, 5, 1, 1, 1, - 4, 3, 3, 2, 3, 1, 2, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 4, 3, 3, 1, - 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 2, 2, - 1, 1, 1, 2, 1, 1, 1, 1, 0, 1, - 0, 1, + 1, 1, 1, 2, 1, 1, 1, 0, 1, } var yyChk = [...]int16{ - -1000, -54, 88, 89, 90, 91, 2, 10, -12, -7, - -11, 60, 61, 75, 62, 63, 64, 12, 45, 46, - 49, 65, 18, 66, 79, 67, 68, 69, 70, 71, - 81, 84, 85, 13, -55, -12, 10, -37, -32, -35, - -38, -43, -44, -45, -47, -48, -49, -50, -51, -31, - -3, 12, 19, 15, 25, -8, -7, -42, 60, 61, - 62, 63, 64, 65, 66, 67, 68, 69, 70, 71, - 39, 55, 13, -51, -11, -13, 20, -14, 12, 2, - -19, 2, 39, 57, 40, 41, 43, 44, 45, 46, - 47, 48, 49, 50, 51, 52, 54, 55, 79, 56, - 14, -33, -40, 2, 75, 81, 15, -40, -37, -37, - -42, -1, 20, -2, 12, -10, 2, 25, 20, 7, - 2, 4, 2, 24, -34, -41, -36, -46, 74, -34, - -34, -34, -34, -34, -34, -34, -34, -34, -34, -34, - -34, -34, -34, -34, -52, 55, 2, 9, -30, -9, - 2, -27, -29, 84, 85, 19, 39, 55, -52, 2, - -40, -33, -16, 15, 2, -16, -39, 22, -37, 22, - 20, 7, 2, -5, 2, 4, 52, 42, 53, -5, - 20, -14, 25, 2, -18, 5, -28, -20, 12, -27, - -29, 16, -37, 78, 80, 76, 77, -37, -37, -37, - -37, -37, -37, -37, -37, -37, -37, -37, -37, -37, - -37, -37, -52, 15, -27, -27, 21, 6, 2, -15, - 22, -4, -6, 2, 60, 74, 61, 75, 62, 63, - 64, 76, 77, 12, 78, 45, 46, 49, 65, 18, - 66, 79, 80, 67, 68, 69, 70, 71, 84, 85, - 57, 22, 7, 20, -2, 25, 2, 25, 2, 26, - 26, -29, 26, 39, 55, -21, 24, 17, -22, 30, - 28, 29, 35, 36, 33, 31, 34, 32, -16, -16, - -17, -16, -17, 22, -53, -52, 2, 22, 7, 2, - -37, -26, 19, -26, 26, -26, -20, -20, 24, 17, - 2, 17, 6, 6, 6, 6, 6, 6, 6, 6, - 6, 21, 2, 22, -4, -26, 26, 26, 17, -22, - -25, 55, -26, -30, -27, -27, -27, -23, 14, -25, + -1000, -52, 91, 92, 93, 94, 2, 10, -12, -7, + -11, 61, 62, 78, 63, 64, 65, 12, 46, 47, + 50, 66, 18, 67, 82, 68, 69, 70, 71, 72, + 84, 87, 88, 73, 74, 13, -53, -12, 10, -37, + -32, -35, -38, -43, -44, -45, -47, -48, -49, -50, + -51, -31, -3, 12, 19, 9, 15, 25, -8, -7, + -42, 61, 62, 63, 64, 65, 66, 67, 68, 69, + 70, 71, 72, 73, 74, 40, 56, 13, -51, -11, + -13, 20, -14, 12, 2, -19, 2, 40, 58, 41, + 42, 44, 45, 46, 47, 48, 49, 50, 51, 52, + 53, 55, 56, 82, 57, 14, -33, -40, 2, 78, + 84, 15, -40, -37, -37, -42, -1, 20, -2, 12, + -10, 2, 25, 20, 7, 2, 4, 2, 24, -34, + -41, -36, -46, 77, -34, -34, -34, -34, -34, -34, + -34, -34, -34, -34, -34, -34, -34, -34, -34, -44, + 56, 2, -30, -9, 2, -27, -29, 87, 88, 19, + 9, 40, 56, -44, 2, -40, -33, -16, 15, 2, + -16, -39, 22, -37, 22, 20, 7, 2, -5, 2, + 4, 53, 43, 54, -5, 20, -14, 25, 2, -18, + 5, -28, -20, 12, -27, -29, 16, -37, 81, 83, + 79, 80, -37, -37, -37, -37, -37, -37, -37, -37, + -37, -37, -37, -37, -37, -37, -37, -44, 15, -27, + -27, 21, 6, 2, -15, 22, -4, -6, 2, 61, + 77, 62, 78, 63, 64, 65, 79, 80, 12, 81, + 46, 47, 50, 66, 18, 67, 82, 83, 68, 69, + 70, 71, 72, 87, 88, 58, 73, 74, 22, 7, + 20, -2, 25, 2, 25, 2, 26, 26, -29, 26, + 40, 56, -21, 24, 17, -22, 30, 28, 29, 35, + 36, 37, 33, 31, 34, 32, -16, -16, -17, -16, + -17, 22, -44, 21, 2, 22, 7, 2, -37, -26, + 19, -26, 26, -26, -20, -20, 24, 17, 2, 17, + 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, + 21, 2, 22, -4, -26, 26, 26, 17, -22, -25, + 56, -26, -30, -27, -27, -27, -23, 14, -23, -25, -23, -25, -26, -26, -26, -24, -27, 24, 21, 2, 21, -27, } var yyDef = [...]int16{ - 0, -2, 125, 125, 0, 0, 7, 6, 1, 125, - 99, 100, 101, 102, 103, 104, 105, 106, 107, 108, - 109, 110, 111, 112, 113, 114, 115, 116, 117, 118, - 119, 120, 121, 0, 2, -2, 3, 4, 8, 9, - 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, - 0, 106, 216, 0, 226, 0, 83, 84, -2, -2, - -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, - 210, 211, 0, 5, 98, 0, 124, 127, 0, 132, - 133, 137, 43, 43, 43, 43, 43, 43, 43, 43, - 43, 43, 43, 43, 43, 43, 43, 43, 0, 0, - 0, 0, 22, 23, 0, 0, 0, 60, 0, 81, - 82, 0, 87, 89, 0, 93, 97, 227, 122, 0, - 128, 0, 131, 136, 0, 42, 47, 48, 44, 0, + 0, -2, 128, 128, 0, 0, 7, 6, 1, 128, + 100, 101, 102, 103, 104, 105, 106, 107, 108, 109, + 110, 111, 112, 113, 114, 115, 116, 117, 118, 119, + 120, 121, 122, 123, 124, 0, 2, -2, 3, 4, + 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, + 18, 19, 0, 107, 224, 225, 0, 235, 0, 84, + 85, -2, -2, -2, -2, -2, -2, -2, -2, -2, + -2, -2, -2, -2, -2, 218, 219, 0, 5, 99, + 0, 127, 130, 0, 135, 136, 140, 43, 43, 43, + 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, + 43, 43, 43, 0, 0, 0, 0, 22, 23, 0, + 0, 0, 60, 0, 82, 83, 0, 88, 90, 0, + 94, 98, 236, 125, 0, 131, 0, 134, 139, 0, + 42, 47, 48, 44, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 67, + 0, 69, 70, 0, 72, 230, 231, 73, 74, 226, + 227, 0, 0, 0, 81, 20, 21, 24, 0, 54, + 25, 0, 62, 64, 66, 86, 0, 91, 0, 97, + 220, 221, 222, 223, 0, 126, 129, 132, 133, 138, + 141, 143, 146, 150, 151, 152, 0, 26, 0, 0, + -2, -2, 27, 28, 29, 30, 31, 32, 33, 34, + 35, 36, 37, 38, 39, 40, 41, 68, 0, 228, + 229, 75, 0, 80, 0, 53, 56, 58, 59, 189, + 190, 191, 192, 193, 194, 195, 196, 197, 198, 199, + 200, 201, 202, 203, 204, 205, 206, 207, 208, 209, + 210, 211, 212, 213, 214, 215, 216, 217, 61, 65, + 87, 89, 92, 96, 93, 95, 0, 0, 0, 0, + 0, 0, 0, 0, 156, 158, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 45, 46, 49, 238, + 50, 71, 0, 77, 79, 51, 0, 57, 63, 142, + 232, 144, 0, 147, 0, 0, 0, 154, 159, 155, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 67, 0, 69, 225, 70, 0, - 72, 220, 221, 73, 74, 217, 0, 0, 0, 80, - 20, 21, 24, 0, 54, 25, 0, 62, 64, 66, - 85, 0, 90, 0, 96, 212, 213, 214, 215, 0, - 123, 126, 129, 130, 135, 138, 140, 143, 147, 148, - 149, 0, 26, 0, 0, -2, -2, 27, 28, 29, - 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, - 40, 41, 68, 0, 218, 219, 75, -2, 79, 0, - 53, 56, 58, 59, 183, 184, 185, 186, 187, 188, - 189, 190, 191, 192, 193, 194, 195, 196, 197, 198, - 199, 200, 201, 202, 203, 204, 205, 206, 207, 208, - 209, 61, 65, 86, 88, 91, 95, 92, 94, 0, - 0, 0, 0, 0, 0, 0, 0, 153, 155, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 45, 46, - 49, 231, 50, 71, 0, -2, 78, 51, 0, 57, - 63, 139, 222, 141, 0, 144, 0, 0, 0, 151, - 156, 152, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 76, 77, 52, 55, 142, 0, 0, 150, 154, - 157, 0, 224, 158, 159, 160, 161, 162, 0, 163, - 164, 165, 145, 146, 223, 0, 169, 0, 167, 170, - 166, 168, + 76, 78, 52, 55, 145, 0, 0, 153, 157, 160, + 0, 234, 161, 162, 163, 164, 165, 0, 166, 167, + 168, 169, 148, 149, 233, 0, 173, 0, 171, 174, + 170, 172, } var yyTok1 = [...]int8{ @@ -645,7 +663,7 @@ var yyTok2 = [...]int8{ 62, 63, 64, 65, 66, 67, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, 78, 79, 80, 81, 82, 83, 84, 85, 86, 87, 88, 89, 90, 91, - 92, + 92, 93, 94, 95, } var yyTok3 = [...]int8{ @@ -1274,19 +1292,23 @@ yydefault: case 67: yyDollar = yyS[yypt-3 : yypt+1] { - yylex.(*parser).addOffset(yyDollar[1].node, yyDollar[3].duration) + numLit, _ := yyDollar[3].node.(*NumberLiteral) + dur := time.Duration(numLit.Val*1000) * time.Millisecond + yylex.(*parser).addOffset(yyDollar[1].node, dur) yyVAL.node = yyDollar[1].node } case 68: yyDollar = yyS[yypt-4 : yypt+1] { - yylex.(*parser).addOffset(yyDollar[1].node, -yyDollar[4].duration) + numLit, _ := yyDollar[4].node.(*NumberLiteral) + dur := time.Duration(numLit.Val*1000) * time.Millisecond + yylex.(*parser).addOffset(yyDollar[1].node, -dur) yyVAL.node = yyDollar[1].node } case 69: yyDollar = yyS[yypt-3 : yypt+1] { - yylex.(*parser).unexpected("offset", "duration") + yylex.(*parser).unexpected("offset", "number or duration") yyVAL.node = yyDollar[1].node } case 70: @@ -1325,48 +1347,61 @@ yydefault: yylex.(*parser).addParseErrf(errRange, errMsg) } + numLit, _ := yyDollar[3].node.(*NumberLiteral) yyVAL.node = &MatrixSelector{ VectorSelector: yyDollar[1].node.(Expr), - Range: yyDollar[3].duration, + Range: time.Duration(numLit.Val*1000) * time.Millisecond, EndPos: yylex.(*parser).lastClosing, } } case 76: yyDollar = yyS[yypt-6 : yypt+1] { + numLitRange, _ := yyDollar[3].node.(*NumberLiteral) + numLitStep, _ := yyDollar[5].node.(*NumberLiteral) yyVAL.node = &SubqueryExpr{ - Expr: yyDollar[1].node.(Expr), - Range: yyDollar[3].duration, - Step: yyDollar[5].duration, - + Expr: yyDollar[1].node.(Expr), + Range: time.Duration(numLitRange.Val*1000) * time.Millisecond, + Step: time.Duration(numLitStep.Val*1000) * time.Millisecond, EndPos: yyDollar[6].item.Pos + 1, } } case 77: + yyDollar = yyS[yypt-5 : yypt+1] + { + numLitRange, _ := yyDollar[3].node.(*NumberLiteral) + yyVAL.node = &SubqueryExpr{ + Expr: yyDollar[1].node.(Expr), + Range: time.Duration(numLitRange.Val*1000) * time.Millisecond, + Step: 0, + EndPos: yyDollar[5].item.Pos + 1, + } + } + case 78: yyDollar = yyS[yypt-6 : yypt+1] { yylex.(*parser).unexpected("subquery selector", "\"]\"") yyVAL.node = yyDollar[1].node } - case 78: + case 79: yyDollar = yyS[yypt-5 : yypt+1] { - yylex.(*parser).unexpected("subquery selector", "duration or \"]\"") + yylex.(*parser).unexpected("subquery selector", "number or duration or \"]\"") yyVAL.node = yyDollar[1].node } - case 79: + case 80: yyDollar = yyS[yypt-4 : yypt+1] { yylex.(*parser).unexpected("subquery or range", "\":\" or \"]\"") yyVAL.node = yyDollar[1].node } - case 80: + case 81: yyDollar = yyS[yypt-3 : yypt+1] { - yylex.(*parser).unexpected("subquery selector", "duration") + yylex.(*parser).unexpected("subquery selector", "number or duration") yyVAL.node = yyDollar[1].node } - case 81: + case 82: yyDollar = yyS[yypt-2 : yypt+1] { if nl, ok := yyDollar[2].node.(*NumberLiteral); ok { @@ -1379,7 +1414,7 @@ yydefault: yyVAL.node = &UnaryExpr{Op: yyDollar[1].item.Typ, Expr: yyDollar[2].node.(Expr), StartPos: yyDollar[1].item.Pos} } } - case 82: + case 83: yyDollar = yyS[yypt-2 : yypt+1] { vs := yyDollar[2].node.(*VectorSelector) @@ -1388,7 +1423,7 @@ yydefault: yylex.(*parser).assembleVectorSelector(vs) yyVAL.node = vs } - case 83: + case 84: yyDollar = yyS[yypt-1 : yypt+1] { vs := &VectorSelector{ @@ -1399,14 +1434,14 @@ yydefault: yylex.(*parser).assembleVectorSelector(vs) yyVAL.node = vs } - case 84: + case 85: yyDollar = yyS[yypt-1 : yypt+1] { vs := yyDollar[1].node.(*VectorSelector) yylex.(*parser).assembleVectorSelector(vs) yyVAL.node = vs } - case 85: + case 86: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.node = &VectorSelector{ @@ -1414,7 +1449,7 @@ yydefault: PosRange: mergeRanges(&yyDollar[1].item, &yyDollar[3].item), } } - case 86: + case 87: yyDollar = yyS[yypt-4 : yypt+1] { yyVAL.node = &VectorSelector{ @@ -1422,7 +1457,7 @@ yydefault: PosRange: mergeRanges(&yyDollar[1].item, &yyDollar[4].item), } } - case 87: + case 88: yyDollar = yyS[yypt-2 : yypt+1] { yyVAL.node = &VectorSelector{ @@ -1430,7 +1465,7 @@ yydefault: PosRange: mergeRanges(&yyDollar[1].item, &yyDollar[2].item), } } - case 88: + case 89: yyDollar = yyS[yypt-3 : yypt+1] { if yyDollar[1].matchers != nil { @@ -1439,128 +1474,128 @@ yydefault: yyVAL.matchers = yyDollar[1].matchers } } - case 89: + case 90: yyDollar = yyS[yypt-1 : yypt+1] { yyVAL.matchers = []*labels.Matcher{yyDollar[1].matcher} } - case 90: + case 91: yyDollar = yyS[yypt-2 : yypt+1] { yylex.(*parser).unexpected("label matching", "\",\" or \"}\"") yyVAL.matchers = yyDollar[1].matchers } - case 91: + case 92: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.matcher = yylex.(*parser).newLabelMatcher(yyDollar[1].item, yyDollar[2].item, yyDollar[3].item) } - case 92: + case 93: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.matcher = yylex.(*parser).newLabelMatcher(yyDollar[1].item, yyDollar[2].item, yyDollar[3].item) } - case 93: + case 94: yyDollar = yyS[yypt-1 : yypt+1] { yyVAL.matcher = yylex.(*parser).newMetricNameMatcher(yyDollar[1].item) } - case 94: + case 95: yyDollar = yyS[yypt-3 : yypt+1] { yylex.(*parser).unexpected("label matching", "string") yyVAL.matcher = nil } - case 95: + case 96: yyDollar = yyS[yypt-3 : yypt+1] { yylex.(*parser).unexpected("label matching", "string") yyVAL.matcher = nil } - case 96: + case 97: yyDollar = yyS[yypt-2 : yypt+1] { yylex.(*parser).unexpected("label matching", "label matching operator") yyVAL.matcher = nil } - case 97: + case 98: yyDollar = yyS[yypt-1 : yypt+1] { yylex.(*parser).unexpected("label matching", "identifier or \"}\"") yyVAL.matcher = nil } - case 98: + case 99: yyDollar = yyS[yypt-2 : yypt+1] { b := labels.NewBuilder(yyDollar[2].labels) b.Set(labels.MetricName, yyDollar[1].item.Val) yyVAL.labels = b.Labels() } - case 99: + case 100: yyDollar = yyS[yypt-1 : yypt+1] { yyVAL.labels = yyDollar[1].labels } - case 122: + case 125: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.labels = labels.New(yyDollar[2].lblList...) } - case 123: + case 126: yyDollar = yyS[yypt-4 : yypt+1] { yyVAL.labels = labels.New(yyDollar[2].lblList...) } - case 124: + case 127: yyDollar = yyS[yypt-2 : yypt+1] { yyVAL.labels = labels.New() } - case 125: + case 128: yyDollar = yyS[yypt-0 : yypt+1] { yyVAL.labels = labels.New() } - case 126: + case 129: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.lblList = append(yyDollar[1].lblList, yyDollar[3].label) } - case 127: + case 130: yyDollar = yyS[yypt-1 : yypt+1] { yyVAL.lblList = []labels.Label{yyDollar[1].label} } - case 128: + case 131: yyDollar = yyS[yypt-2 : yypt+1] { yylex.(*parser).unexpected("label set", "\",\" or \"}\"") yyVAL.lblList = yyDollar[1].lblList } - case 129: + case 132: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.label = labels.Label{Name: yyDollar[1].item.Val, Value: yylex.(*parser).unquoteString(yyDollar[3].item.Val)} } - case 130: + case 133: yyDollar = yyS[yypt-3 : yypt+1] { yylex.(*parser).unexpected("label set", "string") yyVAL.label = labels.Label{} } - case 131: + case 134: yyDollar = yyS[yypt-2 : yypt+1] { yylex.(*parser).unexpected("label set", "\"=\"") yyVAL.label = labels.Label{} } - case 132: + case 135: yyDollar = yyS[yypt-1 : yypt+1] { yylex.(*parser).unexpected("label set", "identifier or \"}\"") yyVAL.label = labels.Label{} } - case 133: + case 136: yyDollar = yyS[yypt-2 : yypt+1] { yylex.(*parser).generatedParserResult = &seriesDescription{ @@ -1568,33 +1603,33 @@ yydefault: values: yyDollar[2].series, } } - case 134: + case 137: yyDollar = yyS[yypt-0 : yypt+1] { yyVAL.series = []SequenceValue{} } - case 135: + case 138: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.series = append(yyDollar[1].series, yyDollar[3].series...) } - case 136: + case 139: yyDollar = yyS[yypt-2 : yypt+1] { yyVAL.series = yyDollar[1].series } - case 137: + case 140: yyDollar = yyS[yypt-1 : yypt+1] { yylex.(*parser).unexpected("series values", "") yyVAL.series = nil } - case 138: + case 141: yyDollar = yyS[yypt-1 : yypt+1] { yyVAL.series = []SequenceValue{{Omitted: true}} } - case 139: + case 142: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.series = []SequenceValue{} @@ -1602,12 +1637,12 @@ yydefault: yyVAL.series = append(yyVAL.series, SequenceValue{Omitted: true}) } } - case 140: + case 143: yyDollar = yyS[yypt-1 : yypt+1] { yyVAL.series = []SequenceValue{{Value: yyDollar[1].float}} } - case 141: + case 144: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.series = []SequenceValue{} @@ -1616,7 +1651,7 @@ yydefault: yyVAL.series = append(yyVAL.series, SequenceValue{Value: yyDollar[1].float}) } } - case 142: + case 145: yyDollar = yyS[yypt-4 : yypt+1] { yyVAL.series = []SequenceValue{} @@ -1626,12 +1661,12 @@ yydefault: yyDollar[1].float += yyDollar[2].float } } - case 143: + case 146: yyDollar = yyS[yypt-1 : yypt+1] { yyVAL.series = []SequenceValue{{Histogram: yyDollar[1].histogram}} } - case 144: + case 147: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.series = []SequenceValue{} @@ -1641,7 +1676,7 @@ yydefault: //$1 += $2 } } - case 145: + case 148: yyDollar = yyS[yypt-5 : yypt+1] { val, err := yylex.(*parser).histogramsIncreaseSeries(yyDollar[1].histogram, yyDollar[3].histogram, yyDollar[5].uint) @@ -1650,7 +1685,7 @@ yydefault: } yyVAL.series = val } - case 146: + case 149: yyDollar = yyS[yypt-5 : yypt+1] { val, err := yylex.(*parser).histogramsDecreaseSeries(yyDollar[1].histogram, yyDollar[3].histogram, yyDollar[5].uint) @@ -1659,7 +1694,7 @@ yydefault: } yyVAL.series = val } - case 147: + case 150: yyDollar = yyS[yypt-1 : yypt+1] { if yyDollar[1].item.Val != "stale" { @@ -1667,118 +1702,124 @@ yydefault: } yyVAL.float = math.Float64frombits(value.StaleNaN) } - case 150: + case 153: yyDollar = yyS[yypt-4 : yypt+1] { yyVAL.histogram = yylex.(*parser).buildHistogramFromMap(&yyDollar[2].descriptors) } - case 151: + case 154: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.histogram = yylex.(*parser).buildHistogramFromMap(&yyDollar[2].descriptors) } - case 152: + case 155: yyDollar = yyS[yypt-3 : yypt+1] { m := yylex.(*parser).newMap() yyVAL.histogram = yylex.(*parser).buildHistogramFromMap(&m) } - case 153: + case 156: yyDollar = yyS[yypt-2 : yypt+1] { m := yylex.(*parser).newMap() yyVAL.histogram = yylex.(*parser).buildHistogramFromMap(&m) } - case 154: + case 157: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.descriptors = *(yylex.(*parser).mergeMaps(&yyDollar[1].descriptors, &yyDollar[3].descriptors)) } - case 155: + case 158: yyDollar = yyS[yypt-1 : yypt+1] { yyVAL.descriptors = yyDollar[1].descriptors } - case 156: + case 159: yyDollar = yyS[yypt-2 : yypt+1] { yylex.(*parser).unexpected("histogram description", "histogram description key, e.g. buckets:[5 10 7]") } - case 157: + case 160: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.descriptors = yylex.(*parser).newMap() yyVAL.descriptors["schema"] = yyDollar[3].int } - case 158: + case 161: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.descriptors = yylex.(*parser).newMap() yyVAL.descriptors["sum"] = yyDollar[3].float } - case 159: + case 162: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.descriptors = yylex.(*parser).newMap() yyVAL.descriptors["count"] = yyDollar[3].float } - case 160: + case 163: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.descriptors = yylex.(*parser).newMap() yyVAL.descriptors["z_bucket"] = yyDollar[3].float } - case 161: + case 164: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.descriptors = yylex.(*parser).newMap() yyVAL.descriptors["z_bucket_w"] = yyDollar[3].float } - case 162: + case 165: + yyDollar = yyS[yypt-3 : yypt+1] + { + yyVAL.descriptors = yylex.(*parser).newMap() + yyVAL.descriptors["custom_values"] = yyDollar[3].bucket_set + } + case 166: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.descriptors = yylex.(*parser).newMap() yyVAL.descriptors["buckets"] = yyDollar[3].bucket_set } - case 163: + case 167: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.descriptors = yylex.(*parser).newMap() yyVAL.descriptors["offset"] = yyDollar[3].int } - case 164: + case 168: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.descriptors = yylex.(*parser).newMap() yyVAL.descriptors["n_buckets"] = yyDollar[3].bucket_set } - case 165: + case 169: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.descriptors = yylex.(*parser).newMap() yyVAL.descriptors["n_offset"] = yyDollar[3].int } - case 166: + case 170: yyDollar = yyS[yypt-4 : yypt+1] { yyVAL.bucket_set = yyDollar[2].bucket_set } - case 167: + case 171: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.bucket_set = yyDollar[2].bucket_set } - case 168: + case 172: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.bucket_set = append(yyDollar[1].bucket_set, yyDollar[3].float) } - case 169: + case 173: yyDollar = yyS[yypt-1 : yypt+1] { yyVAL.bucket_set = []float64{yyDollar[1].float} } - case 216: + case 224: yyDollar = yyS[yypt-1 : yypt+1] { yyVAL.node = &NumberLiteral{ @@ -1786,22 +1827,47 @@ yydefault: PosRange: yyDollar[1].item.PositionRange(), } } - case 217: + case 225: + yyDollar = yyS[yypt-1 : yypt+1] + { + var err error + var dur time.Duration + dur, err = parseDuration(yyDollar[1].item.Val) + if err != nil { + yylex.(*parser).addParseErr(yyDollar[1].item.PositionRange(), err) + } + yyVAL.node = &NumberLiteral{ + Val: dur.Seconds(), + PosRange: yyDollar[1].item.PositionRange(), + } + } + case 226: yyDollar = yyS[yypt-1 : yypt+1] { yyVAL.float = yylex.(*parser).number(yyDollar[1].item.Val) } - case 218: + case 227: + yyDollar = yyS[yypt-1 : yypt+1] + { + var err error + var dur time.Duration + dur, err = parseDuration(yyDollar[1].item.Val) + if err != nil { + yylex.(*parser).addParseErr(yyDollar[1].item.PositionRange(), err) + } + yyVAL.float = dur.Seconds() + } + case 228: yyDollar = yyS[yypt-2 : yypt+1] { yyVAL.float = yyDollar[2].float } - case 219: + case 229: yyDollar = yyS[yypt-2 : yypt+1] { yyVAL.float = -yyDollar[2].float } - case 222: + case 232: yyDollar = yyS[yypt-1 : yypt+1] { var err error @@ -1810,26 +1876,17 @@ yydefault: yylex.(*parser).addParseErrf(yyDollar[1].item.PositionRange(), "invalid repetition in series values: %s", err) } } - case 223: + case 233: yyDollar = yyS[yypt-2 : yypt+1] { yyVAL.int = -int64(yyDollar[2].uint) } - case 224: + case 234: yyDollar = yyS[yypt-1 : yypt+1] { yyVAL.int = int64(yyDollar[1].uint) } - case 225: - yyDollar = yyS[yypt-1 : yypt+1] - { - var err error - yyVAL.duration, err = parseDuration(yyDollar[1].item.Val) - if err != nil { - yylex.(*parser).addParseErr(yyDollar[1].item.PositionRange(), err) - } - } - case 226: + case 235: yyDollar = yyS[yypt-1 : yypt+1] { yyVAL.node = &StringLiteral{ @@ -1837,7 +1894,7 @@ yydefault: PosRange: yyDollar[1].item.PositionRange(), } } - case 227: + case 236: yyDollar = yyS[yypt-1 : yypt+1] { yyVAL.item = Item{ @@ -1846,12 +1903,7 @@ yydefault: Val: yylex.(*parser).unquoteString(yyDollar[1].item.Val), } } - case 228: - yyDollar = yyS[yypt-0 : yypt+1] - { - yyVAL.duration = 0 - } - case 230: + case 237: yyDollar = yyS[yypt-0 : yypt+1] { yyVAL.strings = nil diff --git a/vendor/github.com/prometheus/prometheus/promql/parser/lex.go b/vendor/github.com/prometheus/prometheus/promql/parser/lex.go index 4e3de2a668..18abd49ead 100644 --- a/vendor/github.com/prometheus/prometheus/promql/parser/lex.go +++ b/vendor/github.com/prometheus/prometheus/promql/parser/lex.go @@ -65,7 +65,7 @@ func (i ItemType) IsAggregator() bool { return i > aggregatorsStart && i < aggre // IsAggregatorWithParam returns true if the Item is an aggregator that takes a parameter. // Returns false otherwise. func (i ItemType) IsAggregatorWithParam() bool { - return i == TOPK || i == BOTTOMK || i == COUNT_VALUES || i == QUANTILE + return i == TOPK || i == BOTTOMK || i == COUNT_VALUES || i == QUANTILE || i == LIMITK || i == LIMIT_RATIO } // IsKeyword returns true if the Item corresponds to a keyword. @@ -118,6 +118,8 @@ var key = map[string]ItemType{ "bottomk": BOTTOMK, "count_values": COUNT_VALUES, "quantile": QUANTILE, + "limitk": LIMITK, + "limit_ratio": LIMIT_RATIO, // Keywords. "offset": OFFSET, @@ -135,15 +137,16 @@ var key = map[string]ItemType{ } var histogramDesc = map[string]ItemType{ - "sum": SUM_DESC, - "count": COUNT_DESC, - "schema": SCHEMA_DESC, - "offset": OFFSET_DESC, - "n_offset": NEGATIVE_OFFSET_DESC, - "buckets": BUCKETS_DESC, - "n_buckets": NEGATIVE_BUCKETS_DESC, - "z_bucket": ZERO_BUCKET_DESC, - "z_bucket_w": ZERO_BUCKET_WIDTH_DESC, + "sum": SUM_DESC, + "count": COUNT_DESC, + "schema": SCHEMA_DESC, + "offset": OFFSET_DESC, + "n_offset": NEGATIVE_OFFSET_DESC, + "buckets": BUCKETS_DESC, + "n_buckets": NEGATIVE_BUCKETS_DESC, + "z_bucket": ZERO_BUCKET_DESC, + "z_bucket_w": ZERO_BUCKET_WIDTH_DESC, + "custom_values": CUSTOM_VALUES_DESC, } // ItemTypeStr is the default string representations for common Items. It does not @@ -313,6 +316,11 @@ func (l *Lexer) accept(valid string) bool { return false } +// is peeks and returns true if the next rune is contained in the provided string. +func (l *Lexer) is(valid string) bool { + return strings.ContainsRune(valid, l.peek()) +} + // acceptRun consumes a run of runes from the valid set. func (l *Lexer) acceptRun(valid string) { for strings.ContainsRune(valid, l.next()) { @@ -470,7 +478,7 @@ func lexStatements(l *Lexer) stateFn { skipSpaces(l) } l.bracketOpen = true - return lexDuration + return lexNumberOrDuration case r == ']': if !l.bracketOpen { return l.errorf("unexpected right bracket %q", r) @@ -838,18 +846,6 @@ func lexLineComment(l *Lexer) stateFn { return lexStatements } -func lexDuration(l *Lexer) stateFn { - if l.scanNumber() { - return l.errorf("missing unit character in duration") - } - if !acceptRemainingDuration(l) { - return l.errorf("bad duration syntax: %q", l.input[l.start:l.pos]) - } - l.backup() - l.emit(DURATION) - return lexStatements -} - // lexNumber scans a number: decimal, hex, oct or float. func lexNumber(l *Lexer) stateFn { if !l.scanNumber() { @@ -901,18 +897,81 @@ func acceptRemainingDuration(l *Lexer) bool { // scanNumber scans numbers of different formats. The scanned Item is // not necessarily a valid number. This case is caught by the parser. func (l *Lexer) scanNumber() bool { - digits := "0123456789" + initialPos := l.pos + // Modify the digit pattern if the number is hexadecimal. + digitPattern := "0123456789" // Disallow hexadecimal in series descriptions as the syntax is ambiguous. - if !l.seriesDesc && l.accept("0") && l.accept("xX") { - digits = "0123456789abcdefABCDEF" + if !l.seriesDesc && + l.accept("0") && l.accept("xX") { + l.accept("_") // eg., 0X_1FFFP-16 == 0.1249847412109375 + digitPattern = "0123456789abcdefABCDEF" } - l.acceptRun(digits) - if l.accept(".") { - l.acceptRun(digits) + const ( + // Define dot, exponent, and underscore patterns. + dotPattern = "." + exponentPattern = "eE" + underscorePattern = "_" + // Anti-patterns are rune sets that cannot follow their respective rune. + dotAntiPattern = "_." + exponentAntiPattern = "._eE" // and EOL. + underscoreAntiPattern = "._eE" // and EOL. + ) + // All numbers follow the prefix: [.][d][d._eE]* + l.accept(dotPattern) + l.accept(digitPattern) + // [d._eE]* hereon. + dotConsumed := false + exponentConsumed := false + for l.is(digitPattern + dotPattern + underscorePattern + exponentPattern) { + // "." cannot repeat. + if l.is(dotPattern) { + if dotConsumed { + l.accept(dotPattern) + return false + } + } + // "eE" cannot repeat. + if l.is(exponentPattern) { + if exponentConsumed { + l.accept(exponentPattern) + return false + } + } + // Handle dots. + if l.accept(dotPattern) { + dotConsumed = true + if l.accept(dotAntiPattern) { + return false + } + // Fractional hexadecimal literals are not allowed. + if len(digitPattern) > 10 /* 0x[\da-fA-F].[\d]+p[\d] */ { + return false + } + continue + } + // Handle exponents. + if l.accept(exponentPattern) { + exponentConsumed = true + l.accept("+-") + if l.accept(exponentAntiPattern) || l.peek() == eof { + return false + } + continue + } + // Handle underscores. + if l.accept(underscorePattern) { + if l.accept(underscoreAntiPattern) || l.peek() == eof { + return false + } + + continue + } + // Handle digits at the end since we already consumed before this loop. + l.acceptRun(digitPattern) } - if l.accept("eE") { - l.accept("+-") - l.acceptRun("0123456789") + // Empty string is not a valid number. + if l.pos == initialPos { + return false } // Next thing must not be alphanumeric unless it's the times token // for series repetitions. diff --git a/vendor/github.com/prometheus/prometheus/promql/parser/parse.go b/vendor/github.com/prometheus/prometheus/promql/parser/parse.go index 2679336d6c..6f73e2427b 100644 --- a/vendor/github.com/prometheus/prometheus/promql/parser/parse.go +++ b/vendor/github.com/prometheus/prometheus/promql/parser/parse.go @@ -447,6 +447,10 @@ func (p *parser) newAggregateExpr(op Item, modifier, args Node) (ret *AggregateE desiredArgs := 1 if ret.Op.IsAggregatorWithParam() { + if !EnableExperimentalFunctions && (ret.Op == LIMITK || ret.Op == LIMIT_RATIO) { + p.addParseErrf(ret.PositionRange(), "limitk() and limit_ratio() are experimental and must be enabled with --enable-feature=promql-experimental-functions") + return + } desiredArgs = 2 ret.Param = arguments[0] @@ -481,19 +485,19 @@ func (p *parser) mergeMaps(left, right *map[string]interface{}) (ret *map[string } func (p *parser) histogramsIncreaseSeries(base, inc *histogram.FloatHistogram, times uint64) ([]SequenceValue, error) { - return p.histogramsSeries(base, inc, times, func(a, b *histogram.FloatHistogram) *histogram.FloatHistogram { + return p.histogramsSeries(base, inc, times, func(a, b *histogram.FloatHistogram) (*histogram.FloatHistogram, error) { return a.Add(b) }) } func (p *parser) histogramsDecreaseSeries(base, inc *histogram.FloatHistogram, times uint64) ([]SequenceValue, error) { - return p.histogramsSeries(base, inc, times, func(a, b *histogram.FloatHistogram) *histogram.FloatHistogram { + return p.histogramsSeries(base, inc, times, func(a, b *histogram.FloatHistogram) (*histogram.FloatHistogram, error) { return a.Sub(b) }) } func (p *parser) histogramsSeries(base, inc *histogram.FloatHistogram, times uint64, - combine func(*histogram.FloatHistogram, *histogram.FloatHistogram) *histogram.FloatHistogram, + combine func(*histogram.FloatHistogram, *histogram.FloatHistogram) (*histogram.FloatHistogram, error), ) ([]SequenceValue, error) { ret := make([]SequenceValue, times+1) // Add an additional value (the base) for time 0, which we ignore in tests. @@ -504,7 +508,11 @@ func (p *parser) histogramsSeries(base, inc *histogram.FloatHistogram, times uin return nil, fmt.Errorf("error combining histograms: cannot merge from schema %d to %d", inc.Schema, cur.Schema) } - cur = combine(cur.Copy(), inc) + var err error + cur, err = combine(cur.Copy(), inc) + if err != nil { + return ret, err + } ret[i] = SequenceValue{Histogram: cur} } @@ -562,6 +570,15 @@ func (p *parser) buildHistogramFromMap(desc *map[string]interface{}) *histogram. p.addParseErrf(p.yyParser.lval.item.PositionRange(), "error parsing z_bucket_w number: %v", val) } } + val, ok = (*desc)["custom_values"] + if ok { + customValues, ok := val.([]float64) + if ok { + output.CustomValues = customValues + } else { + p.addParseErrf(p.yyParser.lval.item.PositionRange(), "error parsing custom_values: %v", val) + } + } buckets, spans := p.buildHistogramBucketsAndSpans(desc, "buckets", "offset") output.PositiveBuckets = buckets @@ -659,7 +676,7 @@ func (p *parser) checkAST(node Node) (typ ValueType) { p.addParseErrf(n.PositionRange(), "aggregation operator expected in aggregation expression but got %q", n.Op) } p.expectType(n.Expr, ValueTypeVector, "aggregation expression") - if n.Op == TOPK || n.Op == BOTTOMK || n.Op == QUANTILE { + if n.Op == TOPK || n.Op == BOTTOMK || n.Op == QUANTILE || n.Op == LIMITK || n.Op == LIMIT_RATIO { p.expectType(n.Param, ValueTypeScalar, "aggregation parameter") } if n.Op == COUNT_VALUES { diff --git a/vendor/github.com/prometheus/prometheus/promql/quantile.go b/vendor/github.com/prometheus/prometheus/promql/quantile.go index d4bc9ee6e4..7ddb76acba 100644 --- a/vendor/github.com/prometheus/prometheus/promql/quantile.go +++ b/vendor/github.com/prometheus/prometheus/promql/quantile.go @@ -206,12 +206,15 @@ func histogramQuantile(q float64, h *histogram.FloatHistogram) float64 { for it.Next() { bucket = it.At() + if bucket.Count == 0 { + continue + } count += bucket.Count if count >= rank { break } } - if bucket.Lower < 0 && bucket.Upper > 0 { + if !h.UsesCustomBuckets() && bucket.Lower < 0 && bucket.Upper > 0 { switch { case len(h.NegativeBuckets) == 0 && len(h.PositiveBuckets) > 0: // The result is in the zero bucket and the histogram has only @@ -222,6 +225,17 @@ func histogramQuantile(q float64, h *histogram.FloatHistogram) float64 { // negative buckets. So we consider 0 to be the upper bound. bucket.Upper = 0 } + } else if h.UsesCustomBuckets() { + if bucket.Lower == math.Inf(-1) { + // first bucket, with lower bound -Inf + if bucket.Upper <= 0 { + return bucket.Upper + } + bucket.Lower = 0 + } else if bucket.Upper == math.Inf(1) { + // last bucket, with upper bound +Inf + return bucket.Lower + } } // Due to numerical inaccuracies, we could end up with a higher count // than h.Count. Thus, make sure count is never higher than h.Count. diff --git a/vendor/github.com/prometheus/prometheus/rules/group.go b/vendor/github.com/prometheus/prometheus/rules/group.go index c0ad18c187..0bc219a11b 100644 --- a/vendor/github.com/prometheus/prometheus/rules/group.go +++ b/vendor/github.com/prometheus/prometheus/rules/group.go @@ -151,7 +151,42 @@ func (g *Group) Name() string { return g.name } func (g *Group) File() string { return g.file } // Rules returns the group's rules. -func (g *Group) Rules() []Rule { return g.rules } +func (g *Group) Rules(matcherSets ...[]*labels.Matcher) []Rule { + if len(matcherSets) == 0 { + return g.rules + } + var rules []Rule + for _, rule := range g.rules { + if matchesMatcherSets(matcherSets, rule.Labels()) { + rules = append(rules, rule) + } + } + return rules +} + +func matches(lbls labels.Labels, matchers ...*labels.Matcher) bool { + for _, m := range matchers { + if v := lbls.Get(m.Name); !m.Matches(v) { + return false + } + } + return true +} + +// matchesMatcherSets ensures all matches in each matcher set are ANDed and the set of those is ORed. +func matchesMatcherSets(matcherSets [][]*labels.Matcher, lbls labels.Labels) bool { + if len(matcherSets) == 0 { + return true + } + + var ok bool + for _, matchers := range matcherSets { + if matches(lbls, matchers...) { + ok = true + } + } + return ok +} // Queryable returns the group's querable. func (g *Group) Queryable() storage.Queryable { return g.opts.Queryable } diff --git a/vendor/github.com/prometheus/prometheus/rules/manager.go b/vendor/github.com/prometheus/prometheus/rules/manager.go index 063189e0ab..ab33c3c7d8 100644 --- a/vendor/github.com/prometheus/prometheus/rules/manager.go +++ b/vendor/github.com/prometheus/prometheus/rules/manager.go @@ -190,10 +190,18 @@ func (m *Manager) Stop() { // Update the rule manager's state as the config requires. If // loading the new rules failed the old rule set is restored. +// This method will no-op in case the manager is already stopped. func (m *Manager) Update(interval time.Duration, files []string, externalLabels labels.Labels, externalURL string, groupEvalIterationFunc GroupEvalIterationFunc) error { m.mtx.Lock() defer m.mtx.Unlock() + // We cannot update a stopped manager + select { + case <-m.done: + return nil + default: + } + groups, errs := m.LoadGroups(interval, externalLabels, externalURL, groupEvalIterationFunc, files...) if errs != nil { @@ -372,13 +380,13 @@ func (m *Manager) RuleGroups() []*Group { } // Rules returns the list of the manager's rules. -func (m *Manager) Rules() []Rule { +func (m *Manager) Rules(matcherSets ...[]*labels.Matcher) []Rule { m.mtx.RLock() defer m.mtx.RUnlock() var rules []Rule for _, g := range m.groups { - rules = append(rules, g.rules...) + rules = append(rules, g.Rules(matcherSets...)...) } return rules diff --git a/vendor/github.com/prometheus/prometheus/scrape/manager.go b/vendor/github.com/prometheus/prometheus/scrape/manager.go index cb92db5a8c..156e949f83 100644 --- a/vendor/github.com/prometheus/prometheus/scrape/manager.go +++ b/vendor/github.com/prometheus/prometheus/scrape/manager.go @@ -73,9 +73,11 @@ type Options struct { // Option used by downstream scraper users like OpenTelemetry Collector // to help lookup metric metadata. Should be false for Prometheus. PassMetadataInContext bool - // Option to enable the experimental in-memory metadata storage and append - // metadata to the WAL. - EnableMetadataStorage bool + // Option to enable appending of scraped Metadata to the TSDB/other appenders. Individual appenders + // can decide what to do with metadata, but for practical purposes this flag exists so that metadata + // can be written to the WAL and thus read for remote write. + // TODO: implement some form of metadata storage + AppendMetadata bool // Option to increase the interval used by scrape manager to throttle target groups updates. DiscoveryReloadInterval model.Duration // Option to enable the ingestion of the created timestamp as a synthetic zero sample. diff --git a/vendor/github.com/prometheus/prometheus/scrape/metrics.go b/vendor/github.com/prometheus/prometheus/scrape/metrics.go index b67d0686b6..e7395c6191 100644 --- a/vendor/github.com/prometheus/prometheus/scrape/metrics.go +++ b/vendor/github.com/prometheus/prometheus/scrape/metrics.go @@ -34,6 +34,7 @@ type scrapeMetrics struct { targetScrapePoolExceededTargetLimit prometheus.Counter targetScrapePoolTargetLimit *prometheus.GaugeVec targetScrapePoolTargetsAdded *prometheus.GaugeVec + targetScrapePoolSymbolTableItems *prometheus.GaugeVec targetSyncIntervalLength *prometheus.SummaryVec targetSyncFailed *prometheus.CounterVec @@ -129,6 +130,13 @@ func newScrapeMetrics(reg prometheus.Registerer) (*scrapeMetrics, error) { }, []string{"scrape_job"}, ) + sm.targetScrapePoolSymbolTableItems = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Name: "prometheus_target_scrape_pool_symboltable_items", + Help: "Current number of symbols in table for this scrape pool.", + }, + []string{"scrape_job"}, + ) sm.targetScrapePoolSyncsCounter = prometheus.NewCounterVec( prometheus.CounterOpts{ Name: "prometheus_target_scrape_pool_sync_total", @@ -234,6 +242,7 @@ func newScrapeMetrics(reg prometheus.Registerer) (*scrapeMetrics, error) { sm.targetScrapePoolExceededTargetLimit, sm.targetScrapePoolTargetLimit, sm.targetScrapePoolTargetsAdded, + sm.targetScrapePoolSymbolTableItems, sm.targetSyncFailed, // Used by targetScraper. sm.targetScrapeExceededBodySizeLimit, @@ -274,6 +283,7 @@ func (sm *scrapeMetrics) Unregister() { sm.reg.Unregister(sm.targetScrapePoolExceededTargetLimit) sm.reg.Unregister(sm.targetScrapePoolTargetLimit) sm.reg.Unregister(sm.targetScrapePoolTargetsAdded) + sm.reg.Unregister(sm.targetScrapePoolSymbolTableItems) sm.reg.Unregister(sm.targetSyncFailed) sm.reg.Unregister(sm.targetScrapeExceededBodySizeLimit) sm.reg.Unregister(sm.targetScrapeCacheFlushForced) diff --git a/vendor/github.com/prometheus/prometheus/scrape/scrape.go b/vendor/github.com/prometheus/prometheus/scrape/scrape.go index c285f05e36..68411a62e0 100644 --- a/vendor/github.com/prometheus/prometheus/scrape/scrape.go +++ b/vendor/github.com/prometheus/prometheus/scrape/scrape.go @@ -181,7 +181,7 @@ func newScrapePool(cfg *config.ScrapeConfig, app storage.Appendable, offsetSeed options.EnableNativeHistogramsIngestion, options.EnableCreatedTimestampZeroIngestion, options.ExtraMetrics, - options.EnableMetadataStorage, + options.AppendMetadata, opts.target, options.PassMetadataInContext, metrics, @@ -246,6 +246,7 @@ func (sp *scrapePool) stop() { sp.metrics.targetScrapePoolSyncsCounter.DeleteLabelValues(sp.config.JobName) sp.metrics.targetScrapePoolTargetLimit.DeleteLabelValues(sp.config.JobName) sp.metrics.targetScrapePoolTargetsAdded.DeleteLabelValues(sp.config.JobName) + sp.metrics.targetScrapePoolSymbolTableItems.DeleteLabelValues(sp.config.JobName) sp.metrics.targetSyncIntervalLength.DeleteLabelValues(sp.config.JobName) sp.metrics.targetSyncFailed.DeleteLabelValues(sp.config.JobName) } @@ -273,6 +274,15 @@ func (sp *scrapePool) reload(cfg *config.ScrapeConfig) error { sp.metrics.targetScrapePoolTargetLimit.WithLabelValues(sp.config.JobName).Set(float64(sp.config.TargetLimit)) + sp.restartLoops(reuseCache) + oldClient.CloseIdleConnections() + sp.metrics.targetReloadIntervalLength.WithLabelValues(time.Duration(sp.config.ScrapeInterval).String()).Observe( + time.Since(start).Seconds(), + ) + return nil +} + +func (sp *scrapePool) restartLoops(reuseCache bool) { var ( wg sync.WaitGroup interval = time.Duration(sp.config.ScrapeInterval) @@ -313,7 +323,7 @@ func (sp *scrapePool) reload(cfg *config.ScrapeConfig) error { client: sp.client, timeout: timeout, bodySizeLimit: bodySizeLimit, - acceptHeader: acceptHeader(cfg.ScrapeProtocols), + acceptHeader: acceptHeader(sp.config.ScrapeProtocols), acceptEncodingHeader: acceptEncodingHeader(enableCompression), } newLoop = sp.newLoop(scrapeLoopOptions{ @@ -352,11 +362,10 @@ func (sp *scrapePool) reload(cfg *config.ScrapeConfig) error { sp.targetMtx.Unlock() wg.Wait() - oldClient.CloseIdleConnections() - sp.metrics.targetReloadIntervalLength.WithLabelValues(interval.String()).Observe( - time.Since(start).Seconds(), - ) +} +// Must be called with sp.mtx held. +func (sp *scrapePool) checkSymbolTable() { // Here we take steps to clear out the symbol table if it has grown a lot. // After waiting some time for things to settle, we take the size of the symbol-table. // If, after some more time, the table has grown to twice that size, we start a new one. @@ -367,11 +376,10 @@ func (sp *scrapePool) reload(cfg *config.ScrapeConfig) error { } else if sp.symbolTable.Len() > 2*sp.initialSymbolTableLen { sp.symbolTable = labels.NewSymbolTable() sp.initialSymbolTableLen = 0 + sp.restartLoops(false) // To drop all caches. } sp.lastSymbolTableCheck = time.Now() } - - return nil } // Sync converts target groups into actual scrape targets and synchronizes @@ -408,8 +416,10 @@ func (sp *scrapePool) Sync(tgs []*targetgroup.Group) { } } } + sp.metrics.targetScrapePoolSymbolTableItems.WithLabelValues(sp.config.JobName).Set(float64(sp.symbolTable.Len())) sp.targetMtx.Unlock() sp.sync(all) + sp.checkSymbolTable() sp.metrics.targetSyncIntervalLength.WithLabelValues(sp.config.JobName).Observe( time.Since(start).Seconds(), @@ -663,7 +673,7 @@ func appender(app storage.Appender, sampleLimit, bucketLimit int, maxSchema int3 } } - if maxSchema < nativeHistogramMaxSchema { + if maxSchema < histogram.ExponentialSchemaMax { app = &maxSchemaAppender{ Appender: app, maxSchema: maxSchema, @@ -1978,10 +1988,10 @@ func pickSchema(bucketFactor float64) int32 { } floor := math.Floor(-math.Log2(math.Log2(bucketFactor))) switch { - case floor >= float64(nativeHistogramMaxSchema): - return nativeHistogramMaxSchema - case floor <= float64(nativeHistogramMinSchema): - return nativeHistogramMinSchema + case floor >= float64(histogram.ExponentialSchemaMax): + return histogram.ExponentialSchemaMax + case floor <= float64(histogram.ExponentialSchemaMin): + return histogram.ExponentialSchemaMin default: return int32(floor) } diff --git a/vendor/github.com/prometheus/prometheus/scrape/target.go b/vendor/github.com/prometheus/prometheus/scrape/target.go index ad4b4f6857..9ef4471fbd 100644 --- a/vendor/github.com/prometheus/prometheus/scrape/target.go +++ b/vendor/github.com/prometheus/prometheus/scrape/target.go @@ -365,16 +365,26 @@ type bucketLimitAppender struct { func (app *bucketLimitAppender) AppendHistogram(ref storage.SeriesRef, lset labels.Labels, t int64, h *histogram.Histogram, fh *histogram.FloatHistogram) (storage.SeriesRef, error) { if h != nil { + // Return with an early error if the histogram has too many buckets and the + // schema is not exponential, in which case we can't reduce the resolution. + if len(h.PositiveBuckets)+len(h.NegativeBuckets) > app.limit && !histogram.IsExponentialSchema(h.Schema) { + return 0, errBucketLimit + } for len(h.PositiveBuckets)+len(h.NegativeBuckets) > app.limit { - if h.Schema == -4 { + if h.Schema <= histogram.ExponentialSchemaMin { return 0, errBucketLimit } h = h.ReduceResolution(h.Schema - 1) } } if fh != nil { + // Return with an early error if the histogram has too many buckets and the + // schema is not exponential, in which case we can't reduce the resolution. + if len(fh.PositiveBuckets)+len(fh.NegativeBuckets) > app.limit && !histogram.IsExponentialSchema(fh.Schema) { + return 0, errBucketLimit + } for len(fh.PositiveBuckets)+len(fh.NegativeBuckets) > app.limit { - if fh.Schema == -4 { + if fh.Schema <= histogram.ExponentialSchemaMin { return 0, errBucketLimit } fh = fh.ReduceResolution(fh.Schema - 1) @@ -387,11 +397,6 @@ func (app *bucketLimitAppender) AppendHistogram(ref storage.SeriesRef, lset labe return ref, nil } -const ( - nativeHistogramMaxSchema int32 = 8 - nativeHistogramMinSchema int32 = -4 -) - type maxSchemaAppender struct { storage.Appender @@ -400,12 +405,12 @@ type maxSchemaAppender struct { func (app *maxSchemaAppender) AppendHistogram(ref storage.SeriesRef, lset labels.Labels, t int64, h *histogram.Histogram, fh *histogram.FloatHistogram) (storage.SeriesRef, error) { if h != nil { - if h.Schema > app.maxSchema { + if histogram.IsExponentialSchema(h.Schema) && h.Schema > app.maxSchema { h = h.ReduceResolution(app.maxSchema) } } if fh != nil { - if fh.Schema > app.maxSchema { + if histogram.IsExponentialSchema(fh.Schema) && fh.Schema > app.maxSchema { fh = fh.ReduceResolution(app.maxSchema) } } diff --git a/vendor/github.com/prometheus/prometheus/storage/interface.go b/vendor/github.com/prometheus/prometheus/storage/interface.go index 493c2d6893..f85f985e9d 100644 --- a/vendor/github.com/prometheus/prometheus/storage/interface.go +++ b/vendor/github.com/prometheus/prometheus/storage/interface.go @@ -122,11 +122,11 @@ type MockQuerier struct { SelectMockFunction func(sortSeries bool, hints *SelectHints, matchers ...*labels.Matcher) SeriesSet } -func (q *MockQuerier) LabelValues(context.Context, string, ...*labels.Matcher) ([]string, annotations.Annotations, error) { +func (q *MockQuerier) LabelValues(context.Context, string, *LabelHints, ...*labels.Matcher) ([]string, annotations.Annotations, error) { return nil, nil, nil } -func (q *MockQuerier) LabelNames(context.Context, ...*labels.Matcher) ([]string, annotations.Annotations, error) { +func (q *MockQuerier) LabelNames(context.Context, *LabelHints, ...*labels.Matcher) ([]string, annotations.Annotations, error) { return nil, nil, nil } @@ -161,12 +161,12 @@ type LabelQuerier interface { // It is not safe to use the strings beyond the lifetime of the querier. // If matchers are specified the returned result set is reduced // to label values of metrics matching the matchers. - LabelValues(ctx context.Context, name string, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) + LabelValues(ctx context.Context, name string, hints *LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) // LabelNames returns all the unique label names present in the block in sorted order. // If matchers are specified the returned result set is reduced // to label names of metrics matching the matchers. - LabelNames(ctx context.Context, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) + LabelNames(ctx context.Context, hints *LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) // Close releases the resources of the Querier. Close() error @@ -190,6 +190,9 @@ type SelectHints struct { Start int64 // Start time in milliseconds for this select. End int64 // End time in milliseconds for this select. + // Maximum number of results returned. Use a value of 0 to disable. + Limit int + Step int64 // Query step size in milliseconds. Func string // String representation of surrounding function or aggregation. @@ -217,6 +220,13 @@ type SelectHints struct { DisableTrimming bool } +// LabelHints specifies hints passed for label reads. +// This is used only as an option for implementation to use. +type LabelHints struct { + // Maximum number of results returned. Use a value of 0 to disable. + Limit int +} + // TODO(bwplotka): Move to promql/engine_test.go? // QueryableFunc is an adapter to allow the use of ordinary functions as // Queryables. It follows the idea of http.HandlerFunc. diff --git a/vendor/github.com/prometheus/prometheus/storage/memoized_iterator.go b/vendor/github.com/prometheus/prometheus/storage/memoized_iterator.go index 4ab2aa5d78..273b3caa1d 100644 --- a/vendor/github.com/prometheus/prometheus/storage/memoized_iterator.go +++ b/vendor/github.com/prometheus/prometheus/storage/memoized_iterator.go @@ -136,6 +136,11 @@ func (b *MemoizedSeriesIterator) AtFloatHistogram() (int64, *histogram.FloatHist return b.it.AtFloatHistogram(nil) } +// AtT returns the timestamp of the current element of the iterator. +func (b *MemoizedSeriesIterator) AtT() int64 { + return b.it.AtT() +} + // Err returns the last encountered error. func (b *MemoizedSeriesIterator) Err() error { return b.it.Err() diff --git a/vendor/github.com/prometheus/prometheus/storage/merge.go b/vendor/github.com/prometheus/prometheus/storage/merge.go index 8855600220..194494b6a9 100644 --- a/vendor/github.com/prometheus/prometheus/storage/merge.go +++ b/vendor/github.com/prometheus/prometheus/storage/merge.go @@ -45,9 +45,15 @@ type mergeGenericQuerier struct { // // In case of overlaps between the data given by primaries' and secondaries' Selects, merge function will be used. func NewMergeQuerier(primaries, secondaries []Querier, mergeFn VerticalSeriesMergeFunc) Querier { - if len(primaries)+len(secondaries) == 0 { - return NoopQuerier() + switch { + case len(primaries)+len(secondaries) == 0: + return noopQuerier{} + case len(primaries) == 1 && len(secondaries) == 0: + return primaries[0] + case len(primaries) == 0 && len(secondaries) == 1: + return secondaries[0] } + queriers := make([]genericQuerier, 0, len(primaries)+len(secondaries)) for _, q := range primaries { if _, ok := q.(noopQuerier); !ok && q != nil { @@ -77,6 +83,15 @@ func NewMergeQuerier(primaries, secondaries []Querier, mergeFn VerticalSeriesMer // In case of overlaps between the data given by primaries' and secondaries' Selects, merge function will be used. // TODO(bwplotka): Currently merge will compact overlapping chunks with bigger chunk, without limit. Split it: https://github.com/prometheus/tsdb/issues/670 func NewMergeChunkQuerier(primaries, secondaries []ChunkQuerier, mergeFn VerticalChunkSeriesMergeFunc) ChunkQuerier { + switch { + case len(primaries) == 0 && len(secondaries) == 0: + return noopChunkQuerier{} + case len(primaries) == 1 && len(secondaries) == 0: + return primaries[0] + case len(primaries) == 0 && len(secondaries) == 1: + return secondaries[0] + } + queriers := make([]genericQuerier, 0, len(primaries)+len(secondaries)) for _, q := range primaries { if _, ok := q.(noopChunkQuerier); !ok && q != nil { @@ -102,13 +117,6 @@ func NewMergeChunkQuerier(primaries, secondaries []ChunkQuerier, mergeFn Vertica // Select returns a set of series that matches the given label matchers. func (q *mergeGenericQuerier) Select(ctx context.Context, sortSeries bool, hints *SelectHints, matchers ...*labels.Matcher) genericSeriesSet { - if len(q.queriers) == 0 { - return noopGenericSeriesSet{} - } - if len(q.queriers) == 1 { - return q.queriers[0].Select(ctx, sortSeries, hints, matchers...) - } - seriesSets := make([]genericSeriesSet, 0, len(q.queriers)) if !q.concurrentSelect { for _, querier := range q.queriers { @@ -161,8 +169,8 @@ func (l labelGenericQueriers) SplitByHalf() (labelGenericQueriers, labelGenericQ // LabelValues returns all potential values for a label name. // If matchers are specified the returned result set is reduced // to label values of metrics matching the matchers. -func (q *mergeGenericQuerier) LabelValues(ctx context.Context, name string, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { - res, ws, err := q.lvals(ctx, q.queriers, name, matchers...) +func (q *mergeGenericQuerier) LabelValues(ctx context.Context, name string, hints *LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { + res, ws, err := q.lvals(ctx, q.queriers, name, hints, matchers...) if err != nil { return nil, nil, fmt.Errorf("LabelValues() from merge generic querier for label %s: %w", name, err) } @@ -170,22 +178,22 @@ func (q *mergeGenericQuerier) LabelValues(ctx context.Context, name string, matc } // lvals performs merge sort for LabelValues from multiple queriers. -func (q *mergeGenericQuerier) lvals(ctx context.Context, lq labelGenericQueriers, n string, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { +func (q *mergeGenericQuerier) lvals(ctx context.Context, lq labelGenericQueriers, n string, hints *LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { if lq.Len() == 0 { return nil, nil, nil } if lq.Len() == 1 { - return lq.Get(0).LabelValues(ctx, n, matchers...) + return lq.Get(0).LabelValues(ctx, n, hints, matchers...) } a, b := lq.SplitByHalf() var ws annotations.Annotations - s1, w, err := q.lvals(ctx, a, n, matchers...) + s1, w, err := q.lvals(ctx, a, n, hints, matchers...) ws.Merge(w) if err != nil { return nil, ws, err } - s2, ws, err := q.lvals(ctx, b, n, matchers...) + s2, ws, err := q.lvals(ctx, b, n, hints, matchers...) ws.Merge(w) if err != nil { return nil, ws, err @@ -221,13 +229,13 @@ func mergeStrings(a, b []string) []string { } // LabelNames returns all the unique label names present in all queriers in sorted order. -func (q *mergeGenericQuerier) LabelNames(ctx context.Context, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { +func (q *mergeGenericQuerier) LabelNames(ctx context.Context, hints *LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { var ( labelNamesMap = make(map[string]struct{}) warnings annotations.Annotations ) for _, querier := range q.queriers { - names, wrn, err := querier.LabelNames(ctx, matchers...) + names, wrn, err := querier.LabelNames(ctx, hints, matchers...) if wrn != nil { // TODO(bwplotka): We could potentially wrap warnings. warnings.Merge(wrn) diff --git a/vendor/github.com/prometheus/prometheus/storage/noop.go b/vendor/github.com/prometheus/prometheus/storage/noop.go index be5741ddd8..f5092da7c7 100644 --- a/vendor/github.com/prometheus/prometheus/storage/noop.go +++ b/vendor/github.com/prometheus/prometheus/storage/noop.go @@ -31,11 +31,11 @@ func (noopQuerier) Select(context.Context, bool, *SelectHints, ...*labels.Matche return NoopSeriesSet() } -func (noopQuerier) LabelValues(context.Context, string, ...*labels.Matcher) ([]string, annotations.Annotations, error) { +func (noopQuerier) LabelValues(context.Context, string, *LabelHints, ...*labels.Matcher) ([]string, annotations.Annotations, error) { return nil, nil, nil } -func (noopQuerier) LabelNames(context.Context, ...*labels.Matcher) ([]string, annotations.Annotations, error) { +func (noopQuerier) LabelNames(context.Context, *LabelHints, ...*labels.Matcher) ([]string, annotations.Annotations, error) { return nil, nil, nil } @@ -54,11 +54,11 @@ func (noopChunkQuerier) Select(context.Context, bool, *SelectHints, ...*labels.M return NoopChunkedSeriesSet() } -func (noopChunkQuerier) LabelValues(context.Context, string, ...*labels.Matcher) ([]string, annotations.Annotations, error) { +func (noopChunkQuerier) LabelValues(context.Context, string, *LabelHints, ...*labels.Matcher) ([]string, annotations.Annotations, error) { return nil, nil, nil } -func (noopChunkQuerier) LabelNames(context.Context, ...*labels.Matcher) ([]string, annotations.Annotations, error) { +func (noopChunkQuerier) LabelNames(context.Context, *LabelHints, ...*labels.Matcher) ([]string, annotations.Annotations, error) { return nil, nil, nil } diff --git a/vendor/github.com/prometheus/prometheus/storage/remote/client.go b/vendor/github.com/prometheus/prometheus/storage/remote/client.go index e8791b643a..eff44c6060 100644 --- a/vendor/github.com/prometheus/prometheus/storage/remote/client.go +++ b/vendor/github.com/prometheus/prometheus/storage/remote/client.go @@ -35,13 +35,40 @@ import ( "go.opentelemetry.io/otel" "go.opentelemetry.io/otel/trace" + "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/prompb" "github.com/prometheus/prometheus/storage/remote/azuread" ) const maxErrMsgLen = 1024 -var UserAgent = fmt.Sprintf("Prometheus/%s", version.Version) +const ( + RemoteWriteVersionHeader = "X-Prometheus-Remote-Write-Version" + RemoteWriteVersion1HeaderValue = "0.1.0" + RemoteWriteVersion20HeaderValue = "2.0.0" + appProtoContentType = "application/x-protobuf" +) + +// Compression represents the encoding. Currently remote storage supports only +// one, but we experiment with more, thus leaving the compression scaffolding +// for now. +// NOTE(bwplotka): Keeping it public, as a non-stable help for importers to use. +type Compression string + +const ( + // SnappyBlockCompression represents https://github.com/google/snappy/blob/2c94e11145f0b7b184b831577c93e5a41c4c0346/format_description.txt + SnappyBlockCompression Compression = "snappy" +) + +var ( + // UserAgent represents Prometheus version to use for user agent header. + UserAgent = fmt.Sprintf("Prometheus/%s", version.Version) + + remoteWriteContentTypeHeaders = map[config.RemoteWriteProtoMsg]string{ + config.RemoteWriteProtoMsgV1: appProtoContentType, // Also application/x-protobuf;proto=prometheus.WriteRequest but simplified for compatibility with 1.x spec. + config.RemoteWriteProtoMsgV2: appProtoContentType + ";proto=io.prometheus.write.v2.Request", + } +) var ( remoteReadQueriesTotal = prometheus.NewCounterVec( @@ -93,6 +120,9 @@ type Client struct { readQueries prometheus.Gauge readQueriesTotal *prometheus.CounterVec readQueriesDuration prometheus.Observer + + writeProtoMsg config.RemoteWriteProtoMsg + writeCompression Compression // Not exposed by ClientConfig for now. } // ClientConfig configures a client. @@ -104,6 +134,7 @@ type ClientConfig struct { AzureADConfig *azuread.AzureADConfig Headers map[string]string RetryOnRateLimit bool + WriteProtoMsg config.RemoteWriteProtoMsg } // ReadClient uses the SAMPLES method of remote read to read series samples from remote server. @@ -162,14 +193,20 @@ func NewWriteClient(name string, conf *ClientConfig) (WriteClient, error) { } } - httpClient.Transport = otelhttp.NewTransport(t) + writeProtoMsg := config.RemoteWriteProtoMsgV1 + if conf.WriteProtoMsg != "" { + writeProtoMsg = conf.WriteProtoMsg + } + httpClient.Transport = otelhttp.NewTransport(t) return &Client{ remoteName: name, urlString: conf.URL.String(), Client: httpClient, retryOnRateLimit: conf.RetryOnRateLimit, timeout: time.Duration(conf.Timeout), + writeProtoMsg: writeProtoMsg, + writeCompression: SnappyBlockCompression, }, nil } @@ -206,10 +243,16 @@ func (c *Client) Store(ctx context.Context, req []byte, attempt int) error { return err } - httpReq.Header.Add("Content-Encoding", "snappy") - httpReq.Header.Set("Content-Type", "application/x-protobuf") + httpReq.Header.Add("Content-Encoding", string(c.writeCompression)) + httpReq.Header.Set("Content-Type", remoteWriteContentTypeHeaders[c.writeProtoMsg]) httpReq.Header.Set("User-Agent", UserAgent) - httpReq.Header.Set("X-Prometheus-Remote-Write-Version", "0.1.0") + if c.writeProtoMsg == config.RemoteWriteProtoMsgV1 { + // Compatibility mode for 1.0. + httpReq.Header.Set(RemoteWriteVersionHeader, RemoteWriteVersion1HeaderValue) + } else { + httpReq.Header.Set(RemoteWriteVersionHeader, RemoteWriteVersion20HeaderValue) + } + if attempt > 0 { httpReq.Header.Set("Retry-Attempt", strconv.Itoa(attempt)) } @@ -265,12 +308,12 @@ func retryAfterDuration(t string) model.Duration { } // Name uniquely identifies the client. -func (c Client) Name() string { +func (c *Client) Name() string { return c.remoteName } // Endpoint is the remote read or write endpoint. -func (c Client) Endpoint() string { +func (c *Client) Endpoint() string { return c.urlString } diff --git a/vendor/github.com/prometheus/prometheus/storage/remote/codec.go b/vendor/github.com/prometheus/prometheus/storage/remote/codec.go index 8c569ff038..c9220ca42d 100644 --- a/vendor/github.com/prometheus/prometheus/storage/remote/codec.go +++ b/vendor/github.com/prometheus/prometheus/storage/remote/codec.go @@ -22,7 +22,6 @@ import ( "net/http" "slices" "sort" - "strings" "sync" "github.com/gogo/protobuf/proto" @@ -30,10 +29,10 @@ import ( "github.com/prometheus/common/model" "go.opentelemetry.io/collector/pdata/pmetric/pmetricotlp" - "github.com/prometheus/prometheus/model/exemplar" "github.com/prometheus/prometheus/model/histogram" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/prompb" + writev2 "github.com/prometheus/prometheus/prompb/io/prometheus/write/v2" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/tsdb/chunks" @@ -153,10 +152,10 @@ func ToQueryResult(ss storage.SeriesSet, sampleLimit int) (*prompb.QueryResult, }) case chunkenc.ValHistogram: ts, h := iter.AtHistogram(nil) - histograms = append(histograms, HistogramToHistogramProto(ts, h)) + histograms = append(histograms, prompb.FromIntHistogram(ts, h)) case chunkenc.ValFloatHistogram: ts, fh := iter.AtFloatHistogram(nil) - histograms = append(histograms, FloatHistogramToHistogramProto(ts, fh)) + histograms = append(histograms, prompb.FromFloatHistogram(ts, fh)) default: return nil, ss.Warnings(), fmt.Errorf("unrecognized value type: %s", valType) } @@ -166,7 +165,7 @@ func ToQueryResult(ss storage.SeriesSet, sampleLimit int) (*prompb.QueryResult, } resp.Timeseries = append(resp.Timeseries, &prompb.TimeSeries{ - Labels: LabelsToLabelsProto(series.Labels(), nil), + Labels: prompb.FromLabels(series.Labels(), nil), Samples: samples, Histograms: histograms, }) @@ -182,7 +181,7 @@ func FromQueryResult(sortSeries bool, res *prompb.QueryResult) storage.SeriesSet if err := validateLabelsAndMetricName(ts.Labels); err != nil { return errSeriesSet{err: err} } - lbls := LabelProtosToLabels(&b, ts.Labels) + lbls := ts.ToLabels(&b, nil) series = append(series, &concreteSeries{labels: lbls, floats: ts.Samples, histograms: ts.Histograms}) } @@ -235,7 +234,7 @@ func StreamChunkedReadResponses( for ss.Next() { series := ss.At() iter = series.Iterator(iter) - lbls = MergeLabels(LabelsToLabelsProto(series.Labels(), lbls), sortedExternalLabels) + lbls = MergeLabels(prompb.FromLabels(series.Labels(), lbls), sortedExternalLabels) maxDataLength := maxBytesInFrame for _, lbl := range lbls { @@ -481,21 +480,16 @@ func (c *concreteSeriesIterator) AtHistogram(*histogram.Histogram) (int64, *hist panic("iterator is not on an integer histogram sample") } h := c.series.histograms[c.histogramsCur] - return h.Timestamp, HistogramProtoToHistogram(h) + return h.Timestamp, h.ToIntHistogram() } // AtFloatHistogram implements chunkenc.Iterator. func (c *concreteSeriesIterator) AtFloatHistogram(*histogram.FloatHistogram) (int64, *histogram.FloatHistogram) { - switch c.curValType { - case chunkenc.ValHistogram: - fh := c.series.histograms[c.histogramsCur] - return fh.Timestamp, HistogramProtoToFloatHistogram(fh) - case chunkenc.ValFloatHistogram: + if c.curValType == chunkenc.ValHistogram || c.curValType == chunkenc.ValFloatHistogram { fh := c.series.histograms[c.histogramsCur] - return fh.Timestamp, FloatHistogramProtoToFloatHistogram(fh) - default: - panic("iterator is not on a histogram sample") + return fh.Timestamp, fh.ToFloatHistogram() // integer will be auto-converted. } + panic("iterator is not on a histogram sample") } // AtT implements chunkenc.Iterator. @@ -618,141 +612,6 @@ func FromLabelMatchers(matchers []*prompb.LabelMatcher) ([]*labels.Matcher, erro return result, nil } -func exemplarProtoToExemplar(b *labels.ScratchBuilder, ep prompb.Exemplar) exemplar.Exemplar { - timestamp := ep.Timestamp - - return exemplar.Exemplar{ - Labels: LabelProtosToLabels(b, ep.Labels), - Value: ep.Value, - Ts: timestamp, - HasTs: timestamp != 0, - } -} - -// HistogramProtoToHistogram extracts a (normal integer) Histogram from the -// provided proto message. The caller has to make sure that the proto message -// represents an integer histogram and not a float histogram, or it panics. -func HistogramProtoToHistogram(hp prompb.Histogram) *histogram.Histogram { - if hp.IsFloatHistogram() { - panic("HistogramProtoToHistogram called with a float histogram") - } - return &histogram.Histogram{ - CounterResetHint: histogram.CounterResetHint(hp.ResetHint), - Schema: hp.Schema, - ZeroThreshold: hp.ZeroThreshold, - ZeroCount: hp.GetZeroCountInt(), - Count: hp.GetCountInt(), - Sum: hp.Sum, - PositiveSpans: spansProtoToSpans(hp.GetPositiveSpans()), - PositiveBuckets: hp.GetPositiveDeltas(), - NegativeSpans: spansProtoToSpans(hp.GetNegativeSpans()), - NegativeBuckets: hp.GetNegativeDeltas(), - } -} - -// FloatHistogramProtoToFloatHistogram extracts a float Histogram from the -// provided proto message to a Float Histogram. The caller has to make sure that -// the proto message represents a float histogram and not an integer histogram, -// or it panics. -func FloatHistogramProtoToFloatHistogram(hp prompb.Histogram) *histogram.FloatHistogram { - if !hp.IsFloatHistogram() { - panic("FloatHistogramProtoToFloatHistogram called with an integer histogram") - } - return &histogram.FloatHistogram{ - CounterResetHint: histogram.CounterResetHint(hp.ResetHint), - Schema: hp.Schema, - ZeroThreshold: hp.ZeroThreshold, - ZeroCount: hp.GetZeroCountFloat(), - Count: hp.GetCountFloat(), - Sum: hp.Sum, - PositiveSpans: spansProtoToSpans(hp.GetPositiveSpans()), - PositiveBuckets: hp.GetPositiveCounts(), - NegativeSpans: spansProtoToSpans(hp.GetNegativeSpans()), - NegativeBuckets: hp.GetNegativeCounts(), - } -} - -// HistogramProtoToFloatHistogram extracts and converts a (normal integer) histogram from the provided proto message -// to a float histogram. The caller has to make sure that the proto message represents an integer histogram and not a -// float histogram, or it panics. -func HistogramProtoToFloatHistogram(hp prompb.Histogram) *histogram.FloatHistogram { - if hp.IsFloatHistogram() { - panic("HistogramProtoToFloatHistogram called with a float histogram") - } - return &histogram.FloatHistogram{ - CounterResetHint: histogram.CounterResetHint(hp.ResetHint), - Schema: hp.Schema, - ZeroThreshold: hp.ZeroThreshold, - ZeroCount: float64(hp.GetZeroCountInt()), - Count: float64(hp.GetCountInt()), - Sum: hp.Sum, - PositiveSpans: spansProtoToSpans(hp.GetPositiveSpans()), - PositiveBuckets: deltasToCounts(hp.GetPositiveDeltas()), - NegativeSpans: spansProtoToSpans(hp.GetNegativeSpans()), - NegativeBuckets: deltasToCounts(hp.GetNegativeDeltas()), - } -} - -func spansProtoToSpans(s []prompb.BucketSpan) []histogram.Span { - spans := make([]histogram.Span, len(s)) - for i := 0; i < len(s); i++ { - spans[i] = histogram.Span{Offset: s[i].Offset, Length: s[i].Length} - } - - return spans -} - -func deltasToCounts(deltas []int64) []float64 { - counts := make([]float64, len(deltas)) - var cur float64 - for i, d := range deltas { - cur += float64(d) - counts[i] = cur - } - return counts -} - -func HistogramToHistogramProto(timestamp int64, h *histogram.Histogram) prompb.Histogram { - return prompb.Histogram{ - Count: &prompb.Histogram_CountInt{CountInt: h.Count}, - Sum: h.Sum, - Schema: h.Schema, - ZeroThreshold: h.ZeroThreshold, - ZeroCount: &prompb.Histogram_ZeroCountInt{ZeroCountInt: h.ZeroCount}, - NegativeSpans: spansToSpansProto(h.NegativeSpans), - NegativeDeltas: h.NegativeBuckets, - PositiveSpans: spansToSpansProto(h.PositiveSpans), - PositiveDeltas: h.PositiveBuckets, - ResetHint: prompb.Histogram_ResetHint(h.CounterResetHint), - Timestamp: timestamp, - } -} - -func FloatHistogramToHistogramProto(timestamp int64, fh *histogram.FloatHistogram) prompb.Histogram { - return prompb.Histogram{ - Count: &prompb.Histogram_CountFloat{CountFloat: fh.Count}, - Sum: fh.Sum, - Schema: fh.Schema, - ZeroThreshold: fh.ZeroThreshold, - ZeroCount: &prompb.Histogram_ZeroCountFloat{ZeroCountFloat: fh.ZeroCount}, - NegativeSpans: spansToSpansProto(fh.NegativeSpans), - NegativeCounts: fh.NegativeBuckets, - PositiveSpans: spansToSpansProto(fh.PositiveSpans), - PositiveCounts: fh.PositiveBuckets, - ResetHint: prompb.Histogram_ResetHint(fh.CounterResetHint), - Timestamp: timestamp, - } -} - -func spansToSpansProto(s []histogram.Span) []prompb.BucketSpan { - spans := make([]prompb.BucketSpan, len(s)) - for i := 0; i < len(s); i++ { - spans[i] = prompb.BucketSpan{Offset: s[i].Offset, Length: s[i].Length} - } - - return spans -} - // LabelProtosToMetric unpack a []*prompb.Label to a model.Metric. func LabelProtosToMetric(labelPairs []*prompb.Label) model.Metric { metric := make(model.Metric, len(labelPairs)) @@ -762,44 +621,32 @@ func LabelProtosToMetric(labelPairs []*prompb.Label) model.Metric { return metric } -// LabelProtosToLabels transforms prompb labels into labels. The labels builder -// will be used to build the returned labels. -func LabelProtosToLabels(b *labels.ScratchBuilder, labelPairs []prompb.Label) labels.Labels { - b.Reset() - for _, l := range labelPairs { - b.Add(l.Name, l.Value) +// DecodeWriteRequest from an io.Reader into a prompb.WriteRequest, handling +// snappy decompression. +// Used also by documentation/examples/remote_storage. +func DecodeWriteRequest(r io.Reader) (*prompb.WriteRequest, error) { + compressed, err := io.ReadAll(r) + if err != nil { + return nil, err } - b.Sort() - return b.Labels() -} -// LabelsToLabelsProto transforms labels into prompb labels. The buffer slice -// will be used to avoid allocations if it is big enough to store the labels. -func LabelsToLabelsProto(lbls labels.Labels, buf []prompb.Label) []prompb.Label { - result := buf[:0] - lbls.Range(func(l labels.Label) { - result = append(result, prompb.Label{ - Name: l.Name, - Value: l.Value, - }) - }) - return result -} + reqBuf, err := snappy.Decode(nil, compressed) + if err != nil { + return nil, err + } -// metricTypeToMetricTypeProto transforms a Prometheus metricType into prompb metricType. Since the former is a string we need to transform it to an enum. -func metricTypeToMetricTypeProto(t model.MetricType) prompb.MetricMetadata_MetricType { - mt := strings.ToUpper(string(t)) - v, ok := prompb.MetricMetadata_MetricType_value[mt] - if !ok { - return prompb.MetricMetadata_UNKNOWN + var req prompb.WriteRequest + if err := proto.Unmarshal(reqBuf, &req); err != nil { + return nil, err } - return prompb.MetricMetadata_MetricType(v) + return &req, nil } -// DecodeWriteRequest from an io.Reader into a prompb.WriteRequest, handling +// DecodeWriteV2Request from an io.Reader into a writev2.Request, handling // snappy decompression. -func DecodeWriteRequest(r io.Reader) (*prompb.WriteRequest, error) { +// Used also by documentation/examples/remote_storage. +func DecodeWriteV2Request(r io.Reader) (*writev2.Request, error) { compressed, err := io.ReadAll(r) if err != nil { return nil, err @@ -810,7 +657,7 @@ func DecodeWriteRequest(r io.Reader) (*prompb.WriteRequest, error) { return nil, err } - var req prompb.WriteRequest + var req writev2.Request if err := proto.Unmarshal(reqBuf, &req); err != nil { return nil, err } diff --git a/vendor/github.com/prometheus/prometheus/storage/remote/max_timestamp.go b/vendor/github.com/prometheus/prometheus/storage/remote/max_timestamp.go index 3a0a6d6fd4..bb67d9bb98 100644 --- a/vendor/github.com/prometheus/prometheus/storage/remote/max_timestamp.go +++ b/vendor/github.com/prometheus/prometheus/storage/remote/max_timestamp.go @@ -39,9 +39,3 @@ func (m *maxTimestamp) Get() float64 { defer m.mtx.Unlock() return m.value } - -func (m *maxTimestamp) Collect(c chan<- prometheus.Metric) { - if m.Get() > 0 { - m.Gauge.Collect(c) - } -} diff --git a/vendor/github.com/prometheus/prometheus/storage/remote/metadata_watcher.go b/vendor/github.com/prometheus/prometheus/storage/remote/metadata_watcher.go index abfea3c7b0..fdcd668f56 100644 --- a/vendor/github.com/prometheus/prometheus/storage/remote/metadata_watcher.go +++ b/vendor/github.com/prometheus/prometheus/storage/remote/metadata_watcher.go @@ -27,7 +27,7 @@ import ( // MetadataAppender is an interface used by the Metadata Watcher to send metadata, It is read from the scrape manager, on to somewhere else. type MetadataAppender interface { - AppendMetadata(context.Context, []scrape.MetricMetadata) + AppendWatcherMetadata(context.Context, []scrape.MetricMetadata) } // Watchable represents from where we fetch active targets for metadata. @@ -146,7 +146,7 @@ func (mw *MetadataWatcher) collect() { } // Blocks until the metadata is sent to the remote write endpoint or hardShutdownContext is expired. - mw.writer.AppendMetadata(mw.hardShutdownCtx, metadata) + mw.writer.AppendWatcherMetadata(mw.hardShutdownCtx, metadata) } func (mw *MetadataWatcher) ready() bool { diff --git a/vendor/github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheus/normalize_name.go b/vendor/github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheus/normalize_name.go index 4cf36671aa..71bba40e48 100644 --- a/vendor/github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheus/normalize_name.go +++ b/vendor/github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheus/normalize_name.go @@ -29,7 +29,6 @@ import ( // Prometheus best practices for units: https://prometheus.io/docs/practices/naming/#base-units // OpenMetrics specification for units: https://github.com/OpenObservability/OpenMetrics/blob/main/specification/OpenMetrics.md#units-and-base-units var unitMap = map[string]string{ - // Time "d": "days", "h": "hours", @@ -111,7 +110,6 @@ func BuildCompliantName(metric pmetric.Metric, namespace string, addMetricSuffix // Build a normalized name for the specified metric func normalizeName(metric pmetric.Metric, namespace string) string { - // Split metric name in "tokens" (remove all non-alphanumeric) nameTokens := strings.FieldsFunc( metric.Name(), diff --git a/vendor/github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheus/unit_to_ucum.go b/vendor/github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheus/unit_to_ucum.go index 1f8bf1a638..39a42734d7 100644 --- a/vendor/github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheus/unit_to_ucum.go +++ b/vendor/github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheus/unit_to_ucum.go @@ -19,7 +19,6 @@ package prometheus import "strings" var wordToUCUM = map[string]string{ - // Time "days": "d", "hours": "h", diff --git a/vendor/github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite/helper.go b/vendor/github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite/helper.go index 68be82e443..2571338532 100644 --- a/vendor/github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite/helper.go +++ b/vendor/github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite/helper.go @@ -182,12 +182,13 @@ func createAttributes(resource pcommon.Resource, attributes pcommon.Map, externa if i+1 >= len(extras) { break } - _, found := l[extras[i]] + + name := extras[i] + _, found := l[name] if found && logOnOverwrite { - log.Println("label " + extras[i] + " is overwritten. Check if Prometheus reserved labels are used.") + log.Println("label " + name + " is overwritten. Check if Prometheus reserved labels are used.") } // internal labels should be maintained - name := extras[i] if !(len(name) > 4 && name[:2] == "__" && name[len(name)-2:] == "__") { name = prometheustranslator.NormalizeLabel(name) } @@ -219,6 +220,13 @@ func isValidAggregationTemporality(metric pmetric.Metric) bool { return false } +// addHistogramDataPoints adds OTel histogram data points to the corresponding Prometheus time series +// as classical histogram samples. +// +// Note that we can't convert to native histograms, since these have exponential buckets and don't line up +// with the user defined bucket boundaries of non-exponential OTel histograms. +// However, work is under way to resolve this shortcoming through a feature called native histograms custom buckets: +// https://github.com/prometheus/prometheus/issues/13485. func (c *PrometheusConverter) addHistogramDataPoints(dataPoints pmetric.HistogramDataPointSlice, resource pcommon.Resource, settings Settings, baseName string) { for x := 0; x < dataPoints.Len(); x++ { diff --git a/vendor/github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite/histograms.go b/vendor/github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite/histograms.go index 31d343fe4d..21b3f5dd9f 100644 --- a/vendor/github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite/histograms.go +++ b/vendor/github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite/histograms.go @@ -30,10 +30,18 @@ import ( const defaultZeroThreshold = 1e-128 +// addExponentialHistogramDataPoints adds OTel exponential histogram data points to the corresponding time series +// as native histogram samples. func (c *PrometheusConverter) addExponentialHistogramDataPoints(dataPoints pmetric.ExponentialHistogramDataPointSlice, - resource pcommon.Resource, settings Settings, baseName string) error { + resource pcommon.Resource, settings Settings, promName string) error { for x := 0; x < dataPoints.Len(); x++ { pt := dataPoints.At(x) + + histogram, err := exponentialToNativeHistogram(pt) + if err != nil { + return err + } + lbls := createAttributes( resource, pt.Attributes(), @@ -41,14 +49,9 @@ func (c *PrometheusConverter) addExponentialHistogramDataPoints(dataPoints pmetr nil, true, model.MetricNameLabel, - baseName, + promName, ) ts, _ := c.getOrCreateTimeSeries(lbls) - - histogram, err := exponentialToNativeHistogram(pt) - if err != nil { - return err - } ts.Histograms = append(ts.Histograms, histogram) exemplars := getPromExemplars[pmetric.ExponentialHistogramDataPoint](pt) @@ -58,7 +61,7 @@ func (c *PrometheusConverter) addExponentialHistogramDataPoints(dataPoints pmetr return nil } -// exponentialToNativeHistogram translates OTel Exponential Histogram data point +// exponentialToNativeHistogram translates OTel Exponential Histogram data point // to Prometheus Native Histogram. func exponentialToNativeHistogram(p pmetric.ExponentialHistogramDataPoint) (prompb.Histogram, error) { scale := p.Scale() diff --git a/vendor/github.com/prometheus/prometheus/storage/remote/queue_manager.go b/vendor/github.com/prometheus/prometheus/storage/remote/queue_manager.go index dde78d35e5..5bafb9da20 100644 --- a/vendor/github.com/prometheus/prometheus/storage/remote/queue_manager.go +++ b/vendor/github.com/prometheus/prometheus/storage/remote/queue_manager.go @@ -36,9 +36,11 @@ import ( "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/model/histogram" "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/model/metadata" "github.com/prometheus/prometheus/model/relabel" "github.com/prometheus/prometheus/model/timestamp" "github.com/prometheus/prometheus/prompb" + writev2 "github.com/prometheus/prometheus/prompb/io/prometheus/write/v2" "github.com/prometheus/prometheus/scrape" "github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/record" @@ -230,7 +232,7 @@ func newQueueManagerMetrics(r prometheus.Registerer, rn, e string) *queueManager Namespace: namespace, Subsystem: subsystem, Name: "queue_highest_sent_timestamp_seconds", - Help: "Timestamp from a WAL sample, the highest timestamp successfully sent by this queue, in seconds since epoch.", + Help: "Timestamp from a WAL sample, the highest timestamp successfully sent by this queue, in seconds since epoch. Initialized to 0 when no data has been sent yet.", ConstLabels: constLabels, }), } @@ -389,7 +391,7 @@ func (m *queueManagerMetrics) unregister() { // external timeseries database. type WriteClient interface { // Store stores the given samples in the remote storage. - Store(context.Context, []byte, int) error + Store(ctx context.Context, req []byte, retryAttempt int) error // Name uniquely identifies the remote storage. Name() string // Endpoint is the remote read or write endpoint for the storage client. @@ -418,11 +420,14 @@ type QueueManager struct { clientMtx sync.RWMutex storeClient WriteClient + protoMsg config.RemoteWriteProtoMsg + enc Compression - seriesMtx sync.Mutex // Covers seriesLabels, droppedSeries and builder. - seriesLabels map[chunks.HeadSeriesRef]labels.Labels - droppedSeries map[chunks.HeadSeriesRef]struct{} - builder *labels.Builder + seriesMtx sync.Mutex // Covers seriesLabels, seriesMetadata, droppedSeries and builder. + seriesLabels map[chunks.HeadSeriesRef]labels.Labels + seriesMetadata map[chunks.HeadSeriesRef]*metadata.Metadata + droppedSeries map[chunks.HeadSeriesRef]struct{} + builder *labels.Builder seriesSegmentMtx sync.Mutex // Covers seriesSegmentIndexes - if you also lock seriesMtx, take seriesMtx first. seriesSegmentIndexes map[chunks.HeadSeriesRef]int @@ -463,6 +468,7 @@ func NewQueueManager( sm ReadyScrapeManager, enableExemplarRemoteWrite bool, enableNativeHistogramRemoteWrite bool, + protoMsg config.RemoteWriteProtoMsg, ) *QueueManager { if logger == nil { logger = log.NewNopLogger() @@ -487,6 +493,7 @@ func NewQueueManager( sendNativeHistograms: enableNativeHistogramRemoteWrite, seriesLabels: make(map[chunks.HeadSeriesRef]labels.Labels), + seriesMetadata: make(map[chunks.HeadSeriesRef]*metadata.Metadata), seriesSegmentIndexes: make(map[chunks.HeadSeriesRef]int), droppedSeries: make(map[chunks.HeadSeriesRef]struct{}), builder: labels.NewBuilder(labels.EmptyLabels()), @@ -503,9 +510,26 @@ func NewQueueManager( metrics: metrics, interner: interner, highestRecvTimestamp: highestRecvTimestamp, + + protoMsg: protoMsg, + enc: SnappyBlockCompression, // Hardcoded for now, but scaffolding exists for likely future use. + } + + walMetadata := false + if t.protoMsg != config.RemoteWriteProtoMsgV1 { + walMetadata = true + } + t.watcher = wlog.NewWatcher(watcherMetrics, readerMetrics, logger, client.Name(), t, dir, enableExemplarRemoteWrite, enableNativeHistogramRemoteWrite, walMetadata) + + // The current MetadataWatcher implementation is mutually exclusive + // with the new approach, which stores metadata as WAL records and + // ships them alongside series. If both mechanisms are set, the new one + // takes precedence by implicitly disabling the older one. + if t.mcfg.Send && t.protoMsg != config.RemoteWriteProtoMsgV1 { + level.Warn(logger).Log("msg", "usage of 'metadata_config.send' is redundant when using remote write v2 (or higher) as metadata will always be gathered from the WAL and included for every series within each write request") + t.mcfg.Send = false } - t.watcher = wlog.NewWatcher(watcherMetrics, readerMetrics, logger, client.Name(), t, dir, enableExemplarRemoteWrite, enableNativeHistogramRemoteWrite) if t.mcfg.Send { t.metadataWatcher = NewMetadataWatcher(logger, sm, client.Name(), t, t.mcfg.SendInterval, flushDeadline) } @@ -514,14 +538,21 @@ func NewQueueManager( return t } -// AppendMetadata sends metadata to the remote storage. Metadata is sent in batches, but is not parallelized. -func (t *QueueManager) AppendMetadata(ctx context.Context, metadata []scrape.MetricMetadata) { +// AppendWatcherMetadata sends metadata to the remote storage. Metadata is sent in batches, but is not parallelized. +// This is only used for the metadata_config.send setting and 1.x Remote Write. +func (t *QueueManager) AppendWatcherMetadata(ctx context.Context, metadata []scrape.MetricMetadata) { + // no op for any newer proto format, which will cache metadata sent to it from the WAL watcher. + if t.protoMsg != config.RemoteWriteProtoMsgV1 { + return + } + + // 1.X will still get metadata in batches. mm := make([]prompb.MetricMetadata, 0, len(metadata)) for _, entry := range metadata { mm = append(mm, prompb.MetricMetadata{ MetricFamilyName: entry.Metric, Help: entry.Help, - Type: metricTypeToMetricTypeProto(entry.Type), + Type: prompb.FromMetadataType(entry.Type), Unit: entry.Unit, }) } @@ -542,8 +573,8 @@ func (t *QueueManager) AppendMetadata(ctx context.Context, metadata []scrape.Met } func (t *QueueManager) sendMetadataWithBackoff(ctx context.Context, metadata []prompb.MetricMetadata, pBuf *proto.Buffer) error { - // Build the WriteRequest with no samples. - req, _, _, err := buildWriteRequest(t.logger, nil, metadata, pBuf, nil, nil) + // Build the WriteRequest with no samples (v1 flow). + req, _, _, err := buildWriteRequest(t.logger, nil, metadata, pBuf, nil, nil, t.enc) if err != nil { return err } @@ -629,6 +660,36 @@ func isTimeSeriesOldFilter(metrics *queueManagerMetrics, baseTime time.Time, sam } } +func isV2TimeSeriesOldFilter(metrics *queueManagerMetrics, baseTime time.Time, sampleAgeLimit time.Duration) func(ts writev2.TimeSeries) bool { + return func(ts writev2.TimeSeries) bool { + if sampleAgeLimit == 0 { + // If sampleAgeLimit is unset, then we never skip samples due to their age. + return false + } + switch { + // Only the first element should be set in the series, therefore we only check the first element. + case len(ts.Samples) > 0: + if isSampleOld(baseTime, sampleAgeLimit, ts.Samples[0].Timestamp) { + metrics.droppedSamplesTotal.WithLabelValues(reasonTooOld).Inc() + return true + } + case len(ts.Histograms) > 0: + if isSampleOld(baseTime, sampleAgeLimit, ts.Histograms[0].Timestamp) { + metrics.droppedHistogramsTotal.WithLabelValues(reasonTooOld).Inc() + return true + } + case len(ts.Exemplars) > 0: + if isSampleOld(baseTime, sampleAgeLimit, ts.Exemplars[0].Timestamp) { + metrics.droppedExemplarsTotal.WithLabelValues(reasonTooOld).Inc() + return true + } + default: + return false + } + return false + } +} + // Append queues a sample to be sent to the remote storage. Blocks until all samples are // enqueued on their shards or a shutdown signal is received. func (t *QueueManager) Append(samples []record.RefSample) bool { @@ -652,6 +713,9 @@ outer: t.seriesMtx.Unlock() continue } + // TODO(cstyan): Handle or at least log an error if no metadata is found. + // See https://github.com/prometheus/prometheus/issues/14405 + meta := t.seriesMetadata[s.Ref] t.seriesMtx.Unlock() // Start with a very small backoff. This should not be t.cfg.MinBackoff // as it can happen without errors, and we want to pickup work after @@ -666,6 +730,7 @@ outer: } if t.shards.enqueue(s.Ref, timeSeries{ seriesLabels: lbls, + metadata: meta, timestamp: s.T, value: s.V, sType: tSample, @@ -711,6 +776,7 @@ outer: t.seriesMtx.Unlock() continue } + meta := t.seriesMetadata[e.Ref] t.seriesMtx.Unlock() // This will only loop if the queues are being resharded. backoff := t.cfg.MinBackoff @@ -722,6 +788,7 @@ outer: } if t.shards.enqueue(e.Ref, timeSeries{ seriesLabels: lbls, + metadata: meta, timestamp: e.T, value: e.V, exemplarLabels: e.Labels, @@ -765,6 +832,7 @@ outer: t.seriesMtx.Unlock() continue } + meta := t.seriesMetadata[h.Ref] t.seriesMtx.Unlock() backoff := model.Duration(5 * time.Millisecond) @@ -776,6 +844,7 @@ outer: } if t.shards.enqueue(h.Ref, timeSeries{ seriesLabels: lbls, + metadata: meta, timestamp: h.T, histogram: h.H, sType: tHistogram, @@ -818,6 +887,7 @@ outer: t.seriesMtx.Unlock() continue } + meta := t.seriesMetadata[h.Ref] t.seriesMtx.Unlock() backoff := model.Duration(5 * time.Millisecond) @@ -829,6 +899,7 @@ outer: } if t.shards.enqueue(h.Ref, timeSeries{ seriesLabels: lbls, + metadata: meta, timestamp: h.T, floatHistogram: h.FH, sType: tFloatHistogram, @@ -925,6 +996,23 @@ func (t *QueueManager) StoreSeries(series []record.RefSeries, index int) { } } +// StoreMetadata keeps track of known series' metadata for lookups when sending samples to remote. +func (t *QueueManager) StoreMetadata(meta []record.RefMetadata) { + if t.protoMsg == config.RemoteWriteProtoMsgV1 { + return + } + + t.seriesMtx.Lock() + defer t.seriesMtx.Unlock() + for _, m := range meta { + t.seriesMetadata[m.Ref] = &metadata.Metadata{ + Type: record.ToMetricType(m.Type), + Unit: m.Unit, + Help: m.Help, + } + } +} + // UpdateSeriesSegment updates the segment number held against the series, // so we can trim older ones in SeriesReset. func (t *QueueManager) UpdateSeriesSegment(series []record.RefSeries, index int) { @@ -950,6 +1038,7 @@ func (t *QueueManager) SeriesReset(index int) { delete(t.seriesSegmentIndexes, k) t.releaseLabels(t.seriesLabels[k]) delete(t.seriesLabels, k) + delete(t.seriesMetadata, k) delete(t.droppedSeries, k) } } @@ -1165,6 +1254,7 @@ type shards struct { samplesDroppedOnHardShutdown atomic.Uint32 exemplarsDroppedOnHardShutdown atomic.Uint32 histogramsDroppedOnHardShutdown atomic.Uint32 + metadataDroppedOnHardShutdown atomic.Uint32 } // start the shards; must be called before any call to enqueue. @@ -1193,6 +1283,7 @@ func (s *shards) start(n int) { s.samplesDroppedOnHardShutdown.Store(0) s.exemplarsDroppedOnHardShutdown.Store(0) s.histogramsDroppedOnHardShutdown.Store(0) + s.metadataDroppedOnHardShutdown.Store(0) for i := 0; i < n; i++ { go s.runShard(hardShutdownCtx, i, newQueues[i]) } @@ -1245,7 +1336,6 @@ func (s *shards) stop() { func (s *shards) enqueue(ref chunks.HeadSeriesRef, data timeSeries) bool { s.mtx.RLock() defer s.mtx.RUnlock() - shard := uint64(ref) % uint64(len(s.queues)) select { case <-s.softShutdown: @@ -1288,6 +1378,7 @@ type timeSeries struct { value float64 histogram *histogram.Histogram floatHistogram *histogram.FloatHistogram + metadata *metadata.Metadata timestamp int64 exemplarLabels labels.Labels // The type of series: sample, exemplar, or histogram. @@ -1301,6 +1392,7 @@ const ( tExemplar tHistogram tFloatHistogram + tMetadata ) func newQueue(batchSize, capacity int) *queue { @@ -1324,6 +1416,10 @@ func newQueue(batchSize, capacity int) *queue { func (q *queue) Append(datum timeSeries) bool { q.batchMtx.Lock() defer q.batchMtx.Unlock() + // TODO(cstyan): Check if metadata now means we've reduced the total # of samples + // we can batch together here, and if so find a way to not include metadata + // in the batch size calculation. + // See https://github.com/prometheus/prometheus/issues/14405 q.batch = append(q.batch, datum) if len(q.batch) == cap(q.batch) { select { @@ -1347,7 +1443,6 @@ func (q *queue) Chan() <-chan []timeSeries { func (q *queue) Batch() []timeSeries { q.batchMtx.Lock() defer q.batchMtx.Unlock() - select { case batch := <-q.batchQueue: return batch @@ -1373,6 +1468,8 @@ func (q *queue) FlushAndShutdown(done <-chan struct{}) { for q.tryEnqueueingBatch(done) { time.Sleep(time.Second) } + q.batchMtx.Lock() + defer q.batchMtx.Unlock() q.batch = nil close(q.batchQueue) } @@ -1419,19 +1516,23 @@ func (s *shards) runShard(ctx context.Context, shardID int, queue *queue) { }() shardNum := strconv.Itoa(shardID) + symbolTable := writev2.NewSymbolTable() // Send batches of at most MaxSamplesPerSend samples to the remote storage. // If we have fewer samples than that, flush them out after a deadline anyways. var ( max = s.qm.cfg.MaxSamplesPerSend - pBuf = proto.NewBuffer(nil) - buf []byte + pBuf = proto.NewBuffer(nil) + pBufRaw []byte + buf []byte ) + // TODO(@tpaschalis) Should we also raise the max if we have WAL metadata? if s.qm.sendExemplars { max += int(float64(max) * 0.1) } + // TODO: Dry all of this, we should make an interface/generic for the timeseries type. batchQueue := queue.Chan() pendingData := make([]prompb.TimeSeries, max) for i := range pendingData { @@ -1440,6 +1541,10 @@ func (s *shards) runShard(ctx context.Context, shardID int, queue *queue) { pendingData[i].Exemplars = []prompb.Exemplar{{}} } } + pendingDataV2 := make([]writev2.TimeSeries, max) + for i := range pendingDataV2 { + pendingDataV2[i].Samples = []writev2.Sample{{}} + } timer := time.NewTimer(time.Duration(s.qm.cfg.BatchSendDeadline)) stop := func() { @@ -1452,6 +1557,24 @@ func (s *shards) runShard(ctx context.Context, shardID int, queue *queue) { } defer stop() + sendBatch := func(batch []timeSeries, protoMsg config.RemoteWriteProtoMsg, enc Compression, timer bool) { + switch protoMsg { + case config.RemoteWriteProtoMsgV1: + nPendingSamples, nPendingExemplars, nPendingHistograms := populateTimeSeries(batch, pendingData, s.qm.sendExemplars, s.qm.sendNativeHistograms) + n := nPendingSamples + nPendingExemplars + nPendingHistograms + if timer { + level.Debug(s.qm.logger).Log("msg", "runShard timer ticked, sending buffered data", "samples", nPendingSamples, + "exemplars", nPendingExemplars, "shard", shardNum, "histograms", nPendingHistograms) + } + _ = s.sendSamples(ctx, pendingData[:n], nPendingSamples, nPendingExemplars, nPendingHistograms, pBuf, &buf, enc) + case config.RemoteWriteProtoMsgV2: + nPendingSamples, nPendingExemplars, nPendingHistograms, nPendingMetadata := populateV2TimeSeries(&symbolTable, batch, pendingDataV2, s.qm.sendExemplars, s.qm.sendNativeHistograms) + n := nPendingSamples + nPendingExemplars + nPendingHistograms + _ = s.sendV2Samples(ctx, pendingDataV2[:n], symbolTable.Symbols(), nPendingSamples, nPendingExemplars, nPendingHistograms, nPendingMetadata, &pBufRaw, &buf, enc) + symbolTable.Reset() + } + } + for { select { case <-ctx.Done(): @@ -1475,10 +1598,11 @@ func (s *shards) runShard(ctx context.Context, shardID int, queue *queue) { if !ok { return } - nPendingSamples, nPendingExemplars, nPendingHistograms := s.populateTimeSeries(batch, pendingData) + + sendBatch(batch, s.qm.protoMsg, s.qm.enc, false) + // TODO(bwplotka): Previously the return was between popular and send. + // Consider this when DRY-ing https://github.com/prometheus/prometheus/issues/14409 queue.ReturnForReuse(batch) - n := nPendingSamples + nPendingExemplars + nPendingHistograms - s.sendSamples(ctx, pendingData[:n], nPendingSamples, nPendingExemplars, nPendingHistograms, pBuf, &buf) stop() timer.Reset(time.Duration(s.qm.cfg.BatchSendDeadline)) @@ -1486,11 +1610,7 @@ func (s *shards) runShard(ctx context.Context, shardID int, queue *queue) { case <-timer.C: batch := queue.Batch() if len(batch) > 0 { - nPendingSamples, nPendingExemplars, nPendingHistograms := s.populateTimeSeries(batch, pendingData) - n := nPendingSamples + nPendingExemplars + nPendingHistograms - level.Debug(s.qm.logger).Log("msg", "runShard timer ticked, sending buffered data", "samples", nPendingSamples, - "exemplars", nPendingExemplars, "shard", shardNum, "histograms", nPendingHistograms) - s.sendSamples(ctx, pendingData[:n], nPendingSamples, nPendingExemplars, nPendingHistograms, pBuf, &buf) + sendBatch(batch, s.qm.protoMsg, s.qm.enc, true) } queue.ReturnForReuse(batch) timer.Reset(time.Duration(s.qm.cfg.BatchSendDeadline)) @@ -1498,21 +1618,22 @@ func (s *shards) runShard(ctx context.Context, shardID int, queue *queue) { } } -func (s *shards) populateTimeSeries(batch []timeSeries, pendingData []prompb.TimeSeries) (int, int, int) { +func populateTimeSeries(batch []timeSeries, pendingData []prompb.TimeSeries, sendExemplars, sendNativeHistograms bool) (int, int, int) { var nPendingSamples, nPendingExemplars, nPendingHistograms int for nPending, d := range batch { pendingData[nPending].Samples = pendingData[nPending].Samples[:0] - if s.qm.sendExemplars { + if sendExemplars { pendingData[nPending].Exemplars = pendingData[nPending].Exemplars[:0] } - if s.qm.sendNativeHistograms { + if sendNativeHistograms { pendingData[nPending].Histograms = pendingData[nPending].Histograms[:0] } // Number of pending samples is limited by the fact that sendSamples (via sendSamplesWithBackoff) // retries endlessly, so once we reach max samples, if we can never send to the endpoint we'll // stop reading from the queue. This makes it safe to reference pendingSamples by index. - pendingData[nPending].Labels = LabelsToLabelsProto(d.seriesLabels, pendingData[nPending].Labels) + pendingData[nPending].Labels = prompb.FromLabels(d.seriesLabels, pendingData[nPending].Labels) + switch d.sType { case tSample: pendingData[nPending].Samples = append(pendingData[nPending].Samples, prompb.Sample{ @@ -1522,25 +1643,39 @@ func (s *shards) populateTimeSeries(batch []timeSeries, pendingData []prompb.Tim nPendingSamples++ case tExemplar: pendingData[nPending].Exemplars = append(pendingData[nPending].Exemplars, prompb.Exemplar{ - Labels: LabelsToLabelsProto(d.exemplarLabels, nil), + Labels: prompb.FromLabels(d.exemplarLabels, nil), Value: d.value, Timestamp: d.timestamp, }) nPendingExemplars++ case tHistogram: - pendingData[nPending].Histograms = append(pendingData[nPending].Histograms, HistogramToHistogramProto(d.timestamp, d.histogram)) + pendingData[nPending].Histograms = append(pendingData[nPending].Histograms, prompb.FromIntHistogram(d.timestamp, d.histogram)) nPendingHistograms++ case tFloatHistogram: - pendingData[nPending].Histograms = append(pendingData[nPending].Histograms, FloatHistogramToHistogramProto(d.timestamp, d.floatHistogram)) + pendingData[nPending].Histograms = append(pendingData[nPending].Histograms, prompb.FromFloatHistogram(d.timestamp, d.floatHistogram)) nPendingHistograms++ } } return nPendingSamples, nPendingExemplars, nPendingHistograms } -func (s *shards) sendSamples(ctx context.Context, samples []prompb.TimeSeries, sampleCount, exemplarCount, histogramCount int, pBuf *proto.Buffer, buf *[]byte) { +func (s *shards) sendSamples(ctx context.Context, samples []prompb.TimeSeries, sampleCount, exemplarCount, histogramCount int, pBuf *proto.Buffer, buf *[]byte, enc Compression) error { + begin := time.Now() + err := s.sendSamplesWithBackoff(ctx, samples, sampleCount, exemplarCount, histogramCount, 0, pBuf, buf, enc) + s.updateMetrics(ctx, err, sampleCount, exemplarCount, histogramCount, 0, time.Since(begin)) + return err +} + +// TODO(bwplotka): DRY this (have one logic for both v1 and v2). +// See https://github.com/prometheus/prometheus/issues/14409 +func (s *shards) sendV2Samples(ctx context.Context, samples []writev2.TimeSeries, labels []string, sampleCount, exemplarCount, histogramCount, metadataCount int, pBuf, buf *[]byte, enc Compression) error { begin := time.Now() - err := s.sendSamplesWithBackoff(ctx, samples, sampleCount, exemplarCount, histogramCount, pBuf, buf) + err := s.sendV2SamplesWithBackoff(ctx, samples, labels, sampleCount, exemplarCount, histogramCount, metadataCount, pBuf, buf, enc) + s.updateMetrics(ctx, err, sampleCount, exemplarCount, histogramCount, metadataCount, time.Since(begin)) + return err +} + +func (s *shards) updateMetrics(_ context.Context, err error, sampleCount, exemplarCount, histogramCount, metadataCount int, duration time.Duration) { if err != nil { level.Error(s.qm.logger).Log("msg", "non-recoverable error", "count", sampleCount, "exemplarCount", exemplarCount, "histogramCount", histogramCount, "err", err) s.qm.metrics.failedSamplesTotal.Add(float64(sampleCount)) @@ -1550,8 +1685,8 @@ func (s *shards) sendSamples(ctx context.Context, samples []prompb.TimeSeries, s // These counters are used to calculate the dynamic sharding, and as such // should be maintained irrespective of success or failure. - s.qm.dataOut.incr(int64(len(samples))) - s.qm.dataOutDuration.incr(int64(time.Since(begin))) + s.qm.dataOut.incr(int64(sampleCount + exemplarCount + histogramCount + metadataCount)) + s.qm.dataOutDuration.incr(int64(duration)) s.qm.lastSendTimestamp.Store(time.Now().Unix()) // Pending samples/exemplars/histograms also should be subtracted, as an error means // they will not be retried. @@ -1564,9 +1699,9 @@ func (s *shards) sendSamples(ctx context.Context, samples []prompb.TimeSeries, s } // sendSamples to the remote storage with backoff for recoverable errors. -func (s *shards) sendSamplesWithBackoff(ctx context.Context, samples []prompb.TimeSeries, sampleCount, exemplarCount, histogramCount int, pBuf *proto.Buffer, buf *[]byte) error { +func (s *shards) sendSamplesWithBackoff(ctx context.Context, samples []prompb.TimeSeries, sampleCount, exemplarCount, histogramCount, metadataCount int, pBuf *proto.Buffer, buf *[]byte, enc Compression) error { // Build the WriteRequest with no metadata. - req, highest, lowest, err := buildWriteRequest(s.qm.logger, samples, nil, pBuf, *buf, nil) + req, highest, lowest, err := buildWriteRequest(s.qm.logger, samples, nil, pBuf, buf, nil, enc) s.qm.buildRequestLimitTimestamp.Store(lowest) if err != nil { // Failing to build the write request is non-recoverable, since it will @@ -1590,8 +1725,9 @@ func (s *shards) sendSamplesWithBackoff(ctx context.Context, samples []prompb.Ti samples, nil, pBuf, - *buf, + buf, isTimeSeriesOldFilter(s.qm.metrics, currentTime, time.Duration(s.qm.cfg.SampleAgeLimit)), + enc, ) s.qm.buildRequestLimitTimestamp.Store(lowest) if err != nil { @@ -1622,6 +1758,7 @@ func (s *shards) sendSamplesWithBackoff(ctx context.Context, samples []prompb.Ti s.qm.metrics.samplesTotal.Add(float64(sampleCount)) s.qm.metrics.exemplarsTotal.Add(float64(exemplarCount)) s.qm.metrics.histogramsTotal.Add(float64(histogramCount)) + s.qm.metrics.metadataTotal.Add(float64(metadataCount)) err := s.qm.client().Store(ctx, *buf, try) s.qm.metrics.sentBatchDuration.Observe(time.Since(begin).Seconds()) @@ -1652,6 +1789,148 @@ func (s *shards) sendSamplesWithBackoff(ctx context.Context, samples []prompb.Ti return err } +// sendV2Samples to the remote storage with backoff for recoverable errors. +func (s *shards) sendV2SamplesWithBackoff(ctx context.Context, samples []writev2.TimeSeries, labels []string, sampleCount, exemplarCount, histogramCount, metadataCount int, pBuf, buf *[]byte, enc Compression) error { + // Build the WriteRequest with no metadata. + req, highest, lowest, err := buildV2WriteRequest(s.qm.logger, samples, labels, pBuf, buf, nil, enc) + s.qm.buildRequestLimitTimestamp.Store(lowest) + if err != nil { + // Failing to build the write request is non-recoverable, since it will + // only error if marshaling the proto to bytes fails. + return err + } + + reqSize := len(req) + *buf = req + + // An anonymous function allows us to defer the completion of our per-try spans + // without causing a memory leak, and it has the nice effect of not propagating any + // parameters for sendSamplesWithBackoff/3. + attemptStore := func(try int) error { + currentTime := time.Now() + lowest := s.qm.buildRequestLimitTimestamp.Load() + if isSampleOld(currentTime, time.Duration(s.qm.cfg.SampleAgeLimit), lowest) { + // This will filter out old samples during retries. + req, _, lowest, err := buildV2WriteRequest( + s.qm.logger, + samples, + labels, + pBuf, + buf, + isV2TimeSeriesOldFilter(s.qm.metrics, currentTime, time.Duration(s.qm.cfg.SampleAgeLimit)), + enc, + ) + s.qm.buildRequestLimitTimestamp.Store(lowest) + if err != nil { + return err + } + *buf = req + } + + ctx, span := otel.Tracer("").Start(ctx, "Remote Send Batch") + defer span.End() + + span.SetAttributes( + attribute.Int("request_size", reqSize), + attribute.Int("samples", sampleCount), + attribute.Int("try", try), + attribute.String("remote_name", s.qm.storeClient.Name()), + attribute.String("remote_url", s.qm.storeClient.Endpoint()), + ) + + if exemplarCount > 0 { + span.SetAttributes(attribute.Int("exemplars", exemplarCount)) + } + if histogramCount > 0 { + span.SetAttributes(attribute.Int("histograms", histogramCount)) + } + + begin := time.Now() + s.qm.metrics.samplesTotal.Add(float64(sampleCount)) + s.qm.metrics.exemplarsTotal.Add(float64(exemplarCount)) + s.qm.metrics.histogramsTotal.Add(float64(histogramCount)) + s.qm.metrics.metadataTotal.Add(float64(metadataCount)) + err := s.qm.client().Store(ctx, *buf, try) + s.qm.metrics.sentBatchDuration.Observe(time.Since(begin).Seconds()) + + if err != nil { + span.RecordError(err) + return err + } + + return nil + } + + onRetry := func() { + s.qm.metrics.retriedSamplesTotal.Add(float64(sampleCount)) + s.qm.metrics.retriedExemplarsTotal.Add(float64(exemplarCount)) + s.qm.metrics.retriedHistogramsTotal.Add(float64(histogramCount)) + } + + err = s.qm.sendWriteRequestWithBackoff(ctx, attemptStore, onRetry) + if errors.Is(err, context.Canceled) { + // When there is resharding, we cancel the context for this queue, which means the data is not sent. + // So we exit early to not update the metrics. + return err + } + + s.qm.metrics.sentBytesTotal.Add(float64(reqSize)) + s.qm.metrics.highestSentTimestamp.Set(float64(highest / 1000)) + + return err +} + +func populateV2TimeSeries(symbolTable *writev2.SymbolsTable, batch []timeSeries, pendingData []writev2.TimeSeries, sendExemplars, sendNativeHistograms bool) (int, int, int, int) { + 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.HelpRef = symbolTable.Symbolize(d.metadata.Unit) + nPendingMetadata++ + } + + if sendExemplars { + pendingData[nPending].Exemplars = pendingData[nPending].Exemplars[:0] + } + if sendNativeHistograms { + pendingData[nPending].Histograms = pendingData[nPending].Histograms[:0] + } + + // Number of pending samples is limited by the fact that sendSamples (via sendSamplesWithBackoff) + // retries endlessly, so once we reach max samples, if we can never send to the endpoint we'll + // stop reading from the queue. This makes it safe to reference pendingSamples by index. + pendingData[nPending].LabelsRefs = symbolTable.SymbolizeLabels(d.seriesLabels, pendingData[nPending].LabelsRefs) + switch d.sType { + case tSample: + pendingData[nPending].Samples = append(pendingData[nPending].Samples, writev2.Sample{ + Value: d.value, + Timestamp: d.timestamp, + }) + nPendingSamples++ + case tExemplar: + pendingData[nPending].Exemplars = append(pendingData[nPending].Exemplars, writev2.Exemplar{ + LabelsRefs: symbolTable.SymbolizeLabels(d.exemplarLabels, nil), // TODO: optimize, reuse slice + Value: d.value, + Timestamp: d.timestamp, + }) + nPendingExemplars++ + case tHistogram: + pendingData[nPending].Histograms = append(pendingData[nPending].Histograms, writev2.FromIntHistogram(d.timestamp, d.histogram)) + nPendingHistograms++ + case tFloatHistogram: + pendingData[nPending].Histograms = append(pendingData[nPending].Histograms, writev2.FromFloatHistogram(d.timestamp, d.floatHistogram)) + nPendingHistograms++ + case tMetadata: + // TODO: log or return an error? + // we shouldn't receive metadata type data here, it should already be inserted into the timeSeries + } + } + return nPendingSamples, nPendingExemplars, nPendingHistograms, nPendingMetadata +} + func (t *QueueManager) sendWriteRequestWithBackoff(ctx context.Context, attempt func(int) error, onRetry func()) error { backoff := t.cfg.MinBackoff sleepDuration := model.Duration(0) @@ -1795,7 +2074,21 @@ func buildTimeSeries(timeSeries []prompb.TimeSeries, filter func(prompb.TimeSeri return highest, lowest, timeSeries, droppedSamples, droppedExemplars, droppedHistograms } -func buildWriteRequest(logger log.Logger, timeSeries []prompb.TimeSeries, metadata []prompb.MetricMetadata, pBuf *proto.Buffer, buf []byte, filter func(prompb.TimeSeries) bool) ([]byte, int64, int64, error) { +func compressPayload(tmpbuf *[]byte, inp []byte, enc Compression) (compressed []byte, _ error) { + switch enc { + case SnappyBlockCompression: + compressed = snappy.Encode(*tmpbuf, inp) + if n := snappy.MaxEncodedLen(len(inp)); n > len(*tmpbuf) { + // grow the buffer for the next time + *tmpbuf = make([]byte, n) + } + return compressed, nil + default: + return compressed, fmt.Errorf("Unknown compression scheme [%v]", enc) + } +} + +func buildWriteRequest(logger log.Logger, timeSeries []prompb.TimeSeries, metadata []prompb.MetricMetadata, pBuf *proto.Buffer, buf *[]byte, filter func(prompb.TimeSeries) bool, enc Compression) (compressed []byte, highest, lowest int64, _ error) { highest, lowest, timeSeries, droppedSamples, droppedExemplars, droppedHistograms := buildTimeSeries(timeSeries, filter) @@ -1821,8 +2114,105 @@ func buildWriteRequest(logger log.Logger, timeSeries []prompb.TimeSeries, metada // snappy uses len() to see if it needs to allocate a new slice. Make the // buffer as long as possible. if buf != nil { - buf = buf[0:cap(buf)] + *buf = (*buf)[0:cap(*buf)] + } else { + buf = &[]byte{} + } + + compressed, err = compressPayload(buf, pBuf.Bytes(), enc) + if err != nil { + return nil, highest, lowest, err } - compressed := snappy.Encode(buf, pBuf.Bytes()) return compressed, highest, lowest, nil } + +func buildV2WriteRequest(logger log.Logger, samples []writev2.TimeSeries, labels []string, pBuf, buf *[]byte, filter func(writev2.TimeSeries) bool, enc Compression) (compressed []byte, highest, lowest int64, _ error) { + highest, lowest, timeSeries, droppedSamples, droppedExemplars, droppedHistograms := buildV2TimeSeries(samples, filter) + + if droppedSamples > 0 || droppedExemplars > 0 || droppedHistograms > 0 { + level.Debug(logger).Log("msg", "dropped data due to their age", "droppedSamples", droppedSamples, "droppedExemplars", droppedExemplars, "droppedHistograms", droppedHistograms) + } + + req := &writev2.Request{ + Symbols: labels, + Timeseries: timeSeries, + } + + if pBuf == nil { + pBuf = &[]byte{} // For convenience in tests. Not efficient. + } + + data, err := req.OptimizedMarshal(*pBuf) + if err != nil { + return nil, highest, lowest, err + } + *pBuf = data + + // snappy uses len() to see if it needs to allocate a new slice. Make the + // buffer as long as possible. + if buf != nil { + *buf = (*buf)[0:cap(*buf)] + } else { + buf = &[]byte{} + } + + compressed, err = compressPayload(buf, data, enc) + if err != nil { + return nil, highest, lowest, err + } + return compressed, highest, lowest, nil +} + +func buildV2TimeSeries(timeSeries []writev2.TimeSeries, filter func(writev2.TimeSeries) bool) (int64, int64, []writev2.TimeSeries, int, int, int) { + var highest int64 + var lowest int64 + var droppedSamples, droppedExemplars, droppedHistograms int + + keepIdx := 0 + lowest = math.MaxInt64 + for i, ts := range timeSeries { + if filter != nil && filter(ts) { + if len(ts.Samples) > 0 { + droppedSamples++ + } + if len(ts.Exemplars) > 0 { + droppedExemplars++ + } + if len(ts.Histograms) > 0 { + droppedHistograms++ + } + continue + } + + // At the moment we only ever append a TimeSeries with a single sample or exemplar in it. + if len(ts.Samples) > 0 && ts.Samples[0].Timestamp > highest { + highest = ts.Samples[0].Timestamp + } + if len(ts.Exemplars) > 0 && ts.Exemplars[0].Timestamp > highest { + highest = ts.Exemplars[0].Timestamp + } + if len(ts.Histograms) > 0 && ts.Histograms[0].Timestamp > highest { + highest = ts.Histograms[0].Timestamp + } + + // Get the lowest timestamp. + if len(ts.Samples) > 0 && ts.Samples[0].Timestamp < lowest { + lowest = ts.Samples[0].Timestamp + } + if len(ts.Exemplars) > 0 && ts.Exemplars[0].Timestamp < lowest { + lowest = ts.Exemplars[0].Timestamp + } + if len(ts.Histograms) > 0 && ts.Histograms[0].Timestamp < lowest { + lowest = ts.Histograms[0].Timestamp + } + if i != keepIdx { + // We have to swap the kept timeseries with the one which should be dropped. + // Copying any elements within timeSeries could cause data corruptions when reusing the slice in a next batch (shards.populateTimeSeries). + timeSeries[keepIdx], timeSeries[i] = timeSeries[i], timeSeries[keepIdx] + } + keepIdx++ + } + + timeSeries = timeSeries[:keepIdx] + return highest, lowest, timeSeries, droppedSamples, droppedExemplars, droppedHistograms +} diff --git a/vendor/github.com/prometheus/prometheus/storage/remote/read.go b/vendor/github.com/prometheus/prometheus/storage/remote/read.go index 723030091a..e54b14f1e3 100644 --- a/vendor/github.com/prometheus/prometheus/storage/remote/read.go +++ b/vendor/github.com/prometheus/prometheus/storage/remote/read.go @@ -210,13 +210,13 @@ func (q querier) addExternalLabels(ms []*labels.Matcher) ([]*labels.Matcher, []s } // LabelValues implements storage.Querier and is a noop. -func (q *querier) LabelValues(context.Context, string, ...*labels.Matcher) ([]string, annotations.Annotations, error) { +func (q *querier) LabelValues(context.Context, string, *storage.LabelHints, ...*labels.Matcher) ([]string, annotations.Annotations, error) { // TODO: Implement: https://github.com/prometheus/prometheus/issues/3351 return nil, nil, errors.New("not implemented") } // LabelNames implements storage.Querier and is a noop. -func (q *querier) LabelNames(context.Context, ...*labels.Matcher) ([]string, annotations.Annotations, error) { +func (q *querier) LabelNames(context.Context, *storage.LabelHints, ...*labels.Matcher) ([]string, annotations.Annotations, error) { // TODO: Implement: https://github.com/prometheus/prometheus/issues/3351 return nil, nil, errors.New("not implemented") } diff --git a/vendor/github.com/prometheus/prometheus/storage/remote/storage.go b/vendor/github.com/prometheus/prometheus/storage/remote/storage.go index 758ba3cc91..afa2d411a9 100644 --- a/vendor/github.com/prometheus/prometheus/storage/remote/storage.go +++ b/vendor/github.com/prometheus/prometheus/storage/remote/storage.go @@ -62,7 +62,7 @@ type Storage struct { } // NewStorage returns a remote.Storage. -func NewStorage(l log.Logger, reg prometheus.Registerer, stCallback startTimeCallback, walDir string, flushDeadline time.Duration, sm ReadyScrapeManager) *Storage { +func NewStorage(l log.Logger, reg prometheus.Registerer, stCallback startTimeCallback, walDir string, flushDeadline time.Duration, sm ReadyScrapeManager, metadataInWAL bool) *Storage { if l == nil { l = log.NewNopLogger() } @@ -72,7 +72,7 @@ func NewStorage(l log.Logger, reg prometheus.Registerer, stCallback startTimeCal logger: logger, localStartTimeCallback: stCallback, } - s.rws = NewWriteStorage(s.logger, reg, walDir, flushDeadline, sm) + s.rws = NewWriteStorage(s.logger, reg, walDir, flushDeadline, sm, metadataInWAL) return s } diff --git a/vendor/github.com/prometheus/prometheus/storage/remote/write.go b/vendor/github.com/prometheus/prometheus/storage/remote/write.go index 66455cb4dd..81902a8f1a 100644 --- a/vendor/github.com/prometheus/prometheus/storage/remote/write.go +++ b/vendor/github.com/prometheus/prometheus/storage/remote/write.go @@ -15,6 +15,7 @@ package remote import ( "context" + "errors" "fmt" "math" "sync" @@ -65,6 +66,7 @@ type WriteStorage struct { externalLabels labels.Labels dir string queues map[string]*QueueManager + metadataInWAL bool samplesIn *ewmaRate flushDeadline time.Duration interner *pool @@ -76,7 +78,7 @@ type WriteStorage struct { } // NewWriteStorage creates and runs a WriteStorage. -func NewWriteStorage(logger log.Logger, reg prometheus.Registerer, dir string, flushDeadline time.Duration, sm ReadyScrapeManager) *WriteStorage { +func NewWriteStorage(logger log.Logger, reg prometheus.Registerer, dir string, flushDeadline time.Duration, sm ReadyScrapeManager, metadataInWal bool) *WriteStorage { if logger == nil { logger = log.NewNopLogger() } @@ -92,12 +94,13 @@ func NewWriteStorage(logger log.Logger, reg prometheus.Registerer, dir string, f interner: newPool(), scraper: sm, quit: make(chan struct{}), + metadataInWAL: metadataInWal, highestTimestamp: &maxTimestamp{ Gauge: prometheus.NewGauge(prometheus.GaugeOpts{ Namespace: namespace, Subsystem: subsystem, Name: "highest_timestamp_in_seconds", - Help: "Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch.", + Help: "Highest timestamp that has come into the remote storage via the Appender interface, in seconds since epoch. Initialized to 0 when no data has been received yet.", }), }, } @@ -145,6 +148,9 @@ func (rws *WriteStorage) ApplyConfig(conf *config.Config) error { newQueues := make(map[string]*QueueManager) newHashes := []string{} for _, rwConf := range conf.RemoteWriteConfigs { + if rwConf.ProtobufMessage == config.RemoteWriteProtoMsgV2 && !rws.metadataInWAL { + return errors.New("invalid remote write configuration, if you are using remote write version 2.0 the `--enable-feature=metadata-wal-records` feature flag must be enabled") + } hash, err := toHash(rwConf) if err != nil { return err @@ -165,6 +171,7 @@ func (rws *WriteStorage) ApplyConfig(conf *config.Config) error { c, err := NewWriteClient(name, &ClientConfig{ URL: rwConf.URL, + WriteProtoMsg: rwConf.ProtobufMessage, Timeout: rwConf.RemoteTimeout, HTTPClientConfig: rwConf.HTTPClientConfig, SigV4Config: rwConf.SigV4Config, @@ -207,6 +214,7 @@ func (rws *WriteStorage) ApplyConfig(conf *config.Config) error { rws.scraper, rwConf.SendExemplars, rwConf.SendNativeHistograms, + rwConf.ProtobufMessage, ) // Keep track of which queues are new so we know which to start. newHashes = append(newHashes, hash) diff --git a/vendor/github.com/prometheus/prometheus/storage/remote/write_handler.go b/vendor/github.com/prometheus/prometheus/storage/remote/write_handler.go index 0832c65abe..d822373717 100644 --- a/vendor/github.com/prometheus/prometheus/storage/remote/write_handler.go +++ b/vendor/github.com/prometheus/prometheus/storage/remote/write_handler.go @@ -17,19 +17,26 @@ import ( "context" "errors" "fmt" + "io" "net/http" + "strconv" + "strings" "time" "github.com/go-kit/log" "github.com/go-kit/log/level" - + "github.com/gogo/protobuf/proto" + "github.com/golang/snappy" "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/model/exemplar" "github.com/prometheus/prometheus/model/histogram" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/model/timestamp" "github.com/prometheus/prometheus/prompb" + writev2 "github.com/prometheus/prometheus/prompb/io/prometheus/write/v2" "github.com/prometheus/prometheus/storage" otlptranslator "github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite" ) @@ -38,158 +45,447 @@ type writeHandler struct { logger log.Logger appendable storage.Appendable - samplesWithInvalidLabelsTotal prometheus.Counter + samplesWithInvalidLabelsTotal prometheus.Counter + samplesAppendedWithoutMetadata prometheus.Counter + + acceptedProtoMsgs map[config.RemoteWriteProtoMsg]struct{} } const maxAheadTime = 10 * time.Minute -// NewWriteHandler creates a http.Handler that accepts remote write requests and -// writes them to the provided appendable. -func NewWriteHandler(logger log.Logger, reg prometheus.Registerer, appendable storage.Appendable) http.Handler { +// NewWriteHandler creates a http.Handler that accepts remote write requests with +// the given message in acceptedProtoMsgs and writes them to the provided appendable. +// +// NOTE(bwplotka): When accepting v2 proto and spec, partial writes are possible +// as per https://prometheus.io/docs/specs/remote_write_spec_2_0/#partial-write. +func NewWriteHandler(logger log.Logger, reg prometheus.Registerer, appendable storage.Appendable, acceptedProtoMsgs []config.RemoteWriteProtoMsg) http.Handler { + protoMsgs := map[config.RemoteWriteProtoMsg]struct{}{} + for _, acc := range acceptedProtoMsgs { + protoMsgs[acc] = struct{}{} + } h := &writeHandler{ - logger: logger, - appendable: appendable, - - samplesWithInvalidLabelsTotal: prometheus.NewCounter(prometheus.CounterOpts{ + logger: logger, + appendable: appendable, + acceptedProtoMsgs: protoMsgs, + samplesWithInvalidLabelsTotal: promauto.With(reg).NewCounter(prometheus.CounterOpts{ Namespace: "prometheus", Subsystem: "api", Name: "remote_write_invalid_labels_samples_total", - Help: "The total number of remote write samples which contains invalid labels.", + Help: "The total number of received remote write samples and histogram samples which were rejected due to invalid labels.", + }), + samplesAppendedWithoutMetadata: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Namespace: "prometheus", + Subsystem: "api", + Name: "remote_write_without_metadata_appended_samples_total", + Help: "The total number of received remote write samples (and histogram samples) which were ingested without corresponding metadata.", }), - } - if reg != nil { - reg.MustRegister(h.samplesWithInvalidLabelsTotal) } return h } +func (h *writeHandler) parseProtoMsg(contentType string) (config.RemoteWriteProtoMsg, error) { + contentType = strings.TrimSpace(contentType) + + parts := strings.Split(contentType, ";") + if parts[0] != appProtoContentType { + return "", fmt.Errorf("expected %v as the first (media) part, got %v content-type", appProtoContentType, contentType) + } + // Parse potential https://www.rfc-editor.org/rfc/rfc9110#parameter + for _, p := range parts[1:] { + pair := strings.Split(p, "=") + if len(pair) != 2 { + return "", fmt.Errorf("as per https://www.rfc-editor.org/rfc/rfc9110#parameter expected parameters to be key-values, got %v in %v content-type", p, contentType) + } + if pair[0] == "proto" { + ret := config.RemoteWriteProtoMsg(pair[1]) + if err := ret.Validate(); err != nil { + return "", fmt.Errorf("got %v content type; %w", contentType, err) + } + return ret, nil + } + } + // No "proto=" parameter, assuming v1. + return config.RemoteWriteProtoMsgV1, nil +} + func (h *writeHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { - req, err := DecodeWriteRequest(r.Body) + contentType := r.Header.Get("Content-Type") + if contentType == "" { + // Don't break yolo 1.0 clients if not needed. This is similar to what we did + // before 2.0: https://github.com/prometheus/prometheus/blob/d78253319daa62c8f28ed47e40bafcad2dd8b586/storage/remote/write_handler.go#L62 + // We could give http.StatusUnsupportedMediaType, but let's assume 1.0 message by default. + contentType = appProtoContentType + } + + msgType, err := h.parseProtoMsg(contentType) + if err != nil { + level.Error(h.logger).Log("msg", "Error decoding remote write request", "err", err) + http.Error(w, err.Error(), http.StatusUnsupportedMediaType) + return + } + + if _, ok := h.acceptedProtoMsgs[msgType]; !ok { + err := fmt.Errorf("%v protobuf message is not accepted by this server; accepted %v", msgType, func() (ret []string) { + for k := range h.acceptedProtoMsgs { + ret = append(ret, string(k)) + } + return ret + }()) + level.Error(h.logger).Log("msg", "Error decoding remote write request", "err", err) + http.Error(w, err.Error(), http.StatusUnsupportedMediaType) + } + + enc := r.Header.Get("Content-Encoding") + if enc == "" { + // Don't break yolo 1.0 clients if not needed. This is similar to what we did + // before 2.0: https://github.com/prometheus/prometheus/blob/d78253319daa62c8f28ed47e40bafcad2dd8b586/storage/remote/write_handler.go#L62 + // We could give http.StatusUnsupportedMediaType, but let's assume snappy by default. + } else if enc != string(SnappyBlockCompression) { + err := fmt.Errorf("%v encoding (compression) is not accepted by this server; only %v is acceptable", enc, SnappyBlockCompression) + level.Error(h.logger).Log("msg", "Error decoding remote write request", "err", err) + http.Error(w, err.Error(), http.StatusUnsupportedMediaType) + } + + // Read the request body. + body, err := io.ReadAll(r.Body) if err != nil { level.Error(h.logger).Log("msg", "Error decoding remote write request", "err", err.Error()) http.Error(w, err.Error(), http.StatusBadRequest) return } - err = h.write(r.Context(), req) - switch { - case err == nil: - case errors.Is(err, storage.ErrOutOfOrderSample), errors.Is(err, storage.ErrOutOfBounds), errors.Is(err, storage.ErrDuplicateSampleForTimestamp), errors.Is(err, storage.ErrTooOldSample): - // Indicated an out of order sample is a bad request to prevent retries. + decompressed, err := snappy.Decode(nil, body) + if err != nil { + // TODO(bwplotka): Add more context to responded error? + level.Error(h.logger).Log("msg", "Error decompressing remote write request", "err", err.Error()) http.Error(w, err.Error(), http.StatusBadRequest) return - default: - level.Error(h.logger).Log("msg", "Error appending remote write", "err", err.Error()) - http.Error(w, err.Error(), http.StatusInternalServerError) - return } - w.WriteHeader(http.StatusNoContent) -} + // Now we have a decompressed buffer we can unmarshal it. -// checkAppendExemplarError modifies the AppendExemplar's returned error based on the error cause. -func (h *writeHandler) checkAppendExemplarError(err error, e exemplar.Exemplar, outOfOrderErrs *int) error { - unwrappedErr := errors.Unwrap(err) - if unwrappedErr == nil { - unwrappedErr = err + if msgType == config.RemoteWriteProtoMsgV1 { + // PRW 1.0 flow has different proto message and no partial write handling. + var req prompb.WriteRequest + if err := proto.Unmarshal(decompressed, &req); err != nil { + // TODO(bwplotka): Add more context to responded error? + level.Error(h.logger).Log("msg", "Error decoding v1 remote write request", "protobuf_message", msgType, "err", err.Error()) + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + if err = h.write(r.Context(), &req); err != nil { + switch { + case errors.Is(err, storage.ErrOutOfOrderSample), errors.Is(err, storage.ErrOutOfBounds), errors.Is(err, storage.ErrDuplicateSampleForTimestamp), errors.Is(err, storage.ErrTooOldSample): + // Indicated an out-of-order sample is a bad request to prevent retries. + http.Error(w, err.Error(), http.StatusBadRequest) + return + default: + level.Error(h.logger).Log("msg", "Error while remote writing the v1 request", "err", err.Error()) + http.Error(w, err.Error(), http.StatusInternalServerError) + return + } + } + w.WriteHeader(http.StatusNoContent) + return } - switch { - case errors.Is(unwrappedErr, storage.ErrNotFound): - return storage.ErrNotFound - case errors.Is(unwrappedErr, storage.ErrOutOfOrderExemplar): - *outOfOrderErrs++ - level.Debug(h.logger).Log("msg", "Out of order exemplar", "exemplar", fmt.Sprintf("%+v", e)) - return nil - default: - return err + + // Remote Write 2.x proto message handling. + var req writev2.Request + if err := proto.Unmarshal(decompressed, &req); err != nil { + // TODO(bwplotka): Add more context to responded error? + level.Error(h.logger).Log("msg", "Error decoding v2 remote write request", "protobuf_message", msgType, "err", err.Error()) + http.Error(w, err.Error(), http.StatusBadRequest) + return } + + respStats, errHTTPCode, err := h.writeV2(r.Context(), &req) + + // Set required X-Prometheus-Remote-Write-Written-* response headers, in all cases. + respStats.SetResponseHeaders(w.Header()) + + if err != nil { + if errHTTPCode/5 == 100 { // 5xx + level.Error(h.logger).Log("msg", "Error while remote writing the v2 request", "err", err.Error()) + } + http.Error(w, err.Error(), errHTTPCode) + return + } + w.WriteHeader(http.StatusNoContent) } func (h *writeHandler) write(ctx context.Context, req *prompb.WriteRequest) (err error) { outOfOrderExemplarErrs := 0 samplesWithInvalidLabels := 0 + samplesAppended := 0 - timeLimitApp := &timeLimitAppender{ + app := &timeLimitAppender{ Appender: h.appendable.Appender(ctx), maxTime: timestamp.FromTime(time.Now().Add(maxAheadTime)), } defer func() { if err != nil { - _ = timeLimitApp.Rollback() + _ = app.Rollback() return } - err = timeLimitApp.Commit() + err = app.Commit() + if err != nil { + h.samplesAppendedWithoutMetadata.Add(float64(samplesAppended)) + } }() b := labels.NewScratchBuilder(0) - var exemplarErr error - for _, ts := range req.Timeseries { - labels := LabelProtosToLabels(&b, ts.Labels) - if !labels.IsValid() { - level.Warn(h.logger).Log("msg", "Invalid metric names or labels", "got", labels.String()) + ls := ts.ToLabels(&b, nil) + if !ls.Has(labels.MetricName) || !ls.IsValid() { + level.Warn(h.logger).Log("msg", "Invalid metric names or labels", "got", ls.String()) samplesWithInvalidLabels++ + // TODO(bwplotka): Even as per 1.0 spec, this should be a 400 error, while other samples are + // potentially written. Perhaps unify with fixed writeV2 implementation a bit. continue } - var ref storage.SeriesRef - for _, s := range ts.Samples { - ref, err = timeLimitApp.Append(ref, labels, s.Timestamp, s.Value) - if err != nil { - unwrappedErr := errors.Unwrap(err) - if unwrappedErr == nil { - unwrappedErr = err - } - if errors.Is(err, storage.ErrOutOfOrderSample) || errors.Is(unwrappedErr, storage.ErrOutOfBounds) || errors.Is(unwrappedErr, storage.ErrDuplicateSampleForTimestamp) { - level.Error(h.logger).Log("msg", "Out of order sample from remote write", "err", err.Error(), "series", labels.String(), "timestamp", s.Timestamp) + + if err := h.appendV1Samples(app, ts.Samples, ls); err != nil { + return err + } + samplesAppended += len(ts.Samples) + + for _, ep := range ts.Exemplars { + e := ep.ToExemplar(&b, nil) + if _, err := app.AppendExemplar(0, ls, e); err != nil { + switch { + case errors.Is(err, storage.ErrOutOfOrderExemplar): + outOfOrderExemplarErrs++ + level.Debug(h.logger).Log("msg", "Out of order exemplar", "series", ls.String(), "exemplar", fmt.Sprintf("%+v", e)) + default: + // Since exemplar storage is still experimental, we don't fail the request on ingestion errors + level.Debug(h.logger).Log("msg", "Error while adding exemplar in AppendExemplar", "series", ls.String(), "exemplar", fmt.Sprintf("%+v", e), "err", err) } - return err } } - for _, ep := range ts.Exemplars { - e := exemplarProtoToExemplar(&b, ep) + if err = h.appendV1Histograms(app, ts.Histograms, ls); err != nil { + return err + } + samplesAppended += len(ts.Histograms) + } + + if outOfOrderExemplarErrs > 0 { + _ = level.Warn(h.logger).Log("msg", "Error on ingesting out-of-order exemplars", "num_dropped", outOfOrderExemplarErrs) + } + if samplesWithInvalidLabels > 0 { + h.samplesWithInvalidLabelsTotal.Add(float64(samplesWithInvalidLabels)) + } + return nil +} - _, exemplarErr = timeLimitApp.AppendExemplar(0, labels, e) - exemplarErr = h.checkAppendExemplarError(exemplarErr, e, &outOfOrderExemplarErrs) - if exemplarErr != nil { - // Since exemplar storage is still experimental, we don't fail the request on ingestion errors. - level.Debug(h.logger).Log("msg", "Error while adding exemplar in AddExemplar", "exemplar", fmt.Sprintf("%+v", e), "err", exemplarErr) +func (h *writeHandler) appendV1Samples(app storage.Appender, ss []prompb.Sample, labels labels.Labels) error { + var ref storage.SeriesRef + var err error + for _, s := range ss { + ref, err = app.Append(ref, labels, s.GetTimestamp(), s.GetValue()) + if err != nil { + if errors.Is(err, storage.ErrOutOfOrderSample) || + errors.Is(err, storage.ErrOutOfBounds) || + errors.Is(err, storage.ErrDuplicateSampleForTimestamp) { + level.Error(h.logger).Log("msg", "Out of order sample from remote write", "err", err.Error(), "series", labels.String(), "timestamp", s.Timestamp) + } + return err + } + } + return nil +} + +func (h *writeHandler) appendV1Histograms(app storage.Appender, hh []prompb.Histogram, labels labels.Labels) error { + var err error + for _, hp := range hh { + if hp.IsFloatHistogram() { + _, err = app.AppendHistogram(0, labels, hp.Timestamp, nil, hp.ToFloatHistogram()) + } else { + _, err = app.AppendHistogram(0, labels, hp.Timestamp, hp.ToIntHistogram(), nil) + } + if err != nil { + // Although AppendHistogram does not currently return ErrDuplicateSampleForTimestamp there is + // a note indicating its inclusion in the future. + if errors.Is(err, storage.ErrOutOfOrderSample) || + errors.Is(err, storage.ErrOutOfBounds) || + errors.Is(err, storage.ErrDuplicateSampleForTimestamp) { + level.Error(h.logger).Log("msg", "Out of order histogram from remote write", "err", err.Error(), "series", labels.String(), "timestamp", hp.Timestamp) } + return err } + } + return nil +} + +const ( + prw20WrittenSamplesHeader = "X-Prometheus-Remote-Write-Written-Samples" + rw20WrittenHistogramsHeader = "X-Prometheus-Remote-Write-Written-Histograms" + rw20WrittenExemplarsHeader = "X-Prometheus-Remote-Write-Written-Exemplars" +) + +type responseStats struct { + samples int + histograms int + exemplars int +} + +func (s responseStats) SetResponseHeaders(h http.Header) { + h.Set(prw20WrittenSamplesHeader, strconv.Itoa(s.samples)) + h.Set(rw20WrittenHistogramsHeader, strconv.Itoa(s.histograms)) + h.Set(rw20WrittenExemplarsHeader, strconv.Itoa(s.exemplars)) +} +// writeV2 is similar to write, but it works with v2 proto message, +// allows partial 4xx writes and gathers statistics. +// +// writeV2 returns the statistics. +// In error cases, writeV2, also returns statistics, but also the error that +// should be propagated to the remote write sender and httpCode to use for status. +// +// NOTE(bwplotka): TSDB storage is NOT idempotent, so we don't allow "partial retry-able" errors. +// Once we have 5xx type of error, we immediately stop and rollback all appends. +func (h *writeHandler) writeV2(ctx context.Context, req *writev2.Request) (_ responseStats, errHTTPCode int, _ error) { + app := &timeLimitAppender{ + Appender: h.appendable.Appender(ctx), + maxTime: timestamp.FromTime(time.Now().Add(maxAheadTime)), + } + + rs := responseStats{} + samplesWithoutMetadata, errHTTPCode, err := h.appendV2(app, req, &rs) + if err != nil { + if errHTTPCode/5 == 100 { + // On 5xx, we always rollback, because we expect + // sender to retry and TSDB is not idempotent. + if rerr := app.Rollback(); rerr != nil { + level.Error(h.logger).Log("msg", "writev2 rollback failed on retry-able error", "err", rerr) + } + return responseStats{}, errHTTPCode, err + } + + // Non-retriable (e.g. bad request error case). Can be partially written. + commitErr := app.Commit() + if commitErr != nil { + // Bad requests does not matter as we have internal error (retryable). + return responseStats{}, http.StatusInternalServerError, commitErr + } + // Bad request error happened, but rest of data (if any) was written. + h.samplesAppendedWithoutMetadata.Add(float64(samplesWithoutMetadata)) + return rs, errHTTPCode, err + } + + // All good just commit. + if err := app.Commit(); err != nil { + return responseStats{}, http.StatusInternalServerError, err + } + h.samplesAppendedWithoutMetadata.Add(float64(samplesWithoutMetadata)) + return rs, 0, nil +} + +func (h *writeHandler) appendV2(app storage.Appender, req *writev2.Request, rs *responseStats) (samplesWithoutMetadata, errHTTPCode int, err error) { + var ( + badRequestErrs []error + outOfOrderExemplarErrs, samplesWithInvalidLabels int + + b = labels.NewScratchBuilder(0) + ) + for _, ts := range req.Timeseries { + ls := ts.ToLabels(&b, req.Symbols) + // Validate series labels early. + // NOTE(bwplotka): While spec allows UTF-8, Prometheus Receiver may impose + // specific limits and follow https://prometheus.io/docs/specs/remote_write_spec_2_0/#invalid-samples case. + if !ls.Has(labels.MetricName) || !ls.IsValid() { + badRequestErrs = append(badRequestErrs, fmt.Errorf("invalid metric name or labels, got %v", ls.String())) + samplesWithInvalidLabels += len(ts.Samples) + len(ts.Histograms) + continue + } + + allSamplesSoFar := rs.samples + rs.histograms + var ref storage.SeriesRef + + // Samples. + for _, s := range ts.Samples { + ref, err = app.Append(ref, ls, s.GetTimestamp(), s.GetValue()) + if err == nil { + rs.samples++ + continue + } + // Handle append error. + if errors.Is(err, storage.ErrOutOfOrderSample) || + errors.Is(err, storage.ErrOutOfBounds) || + errors.Is(err, storage.ErrDuplicateSampleForTimestamp) || + errors.Is(err, storage.ErrTooOldSample) { + // TODO(bwplotka): Not too spammy log? + level.Error(h.logger).Log("msg", "Out of order sample from remote write", "err", err.Error(), "series", ls.String(), "timestamp", s.Timestamp) + badRequestErrs = append(badRequestErrs, fmt.Errorf("%w for series %v", err, ls.String())) + continue + } + return 0, http.StatusInternalServerError, err + } + + // Native Histograms. for _, hp := range ts.Histograms { if hp.IsFloatHistogram() { - fhs := FloatHistogramProtoToFloatHistogram(hp) - _, err = timeLimitApp.AppendHistogram(0, labels, hp.Timestamp, nil, fhs) + ref, err = app.AppendHistogram(ref, ls, hp.Timestamp, nil, hp.ToFloatHistogram()) } else { - hs := HistogramProtoToHistogram(hp) - _, err = timeLimitApp.AppendHistogram(0, labels, hp.Timestamp, hs, nil) + ref, err = app.AppendHistogram(ref, ls, hp.Timestamp, hp.ToIntHistogram(), nil) + } + if err == nil { + rs.histograms++ + continue } + // Handle append error. + // Although AppendHistogram does not currently return ErrDuplicateSampleForTimestamp there is + // a note indicating its inclusion in the future. + if errors.Is(err, storage.ErrOutOfOrderSample) || + errors.Is(err, storage.ErrOutOfBounds) || + errors.Is(err, storage.ErrDuplicateSampleForTimestamp) { + // TODO(bwplotka): Not too spammy log? + level.Error(h.logger).Log("msg", "Out of order histogram from remote write", "err", err.Error(), "series", ls.String(), "timestamp", hp.Timestamp) + badRequestErrs = append(badRequestErrs, fmt.Errorf("%w for series %v", err, ls.String())) + continue + } + return 0, http.StatusInternalServerError, err + } - if err != nil { - unwrappedErr := errors.Unwrap(err) - if unwrappedErr == nil { - unwrappedErr = err - } - // Although AppendHistogram does not currently return ErrDuplicateSampleForTimestamp there is - // a note indicating its inclusion in the future. - if errors.Is(unwrappedErr, storage.ErrOutOfOrderSample) || errors.Is(unwrappedErr, storage.ErrOutOfBounds) || errors.Is(unwrappedErr, storage.ErrDuplicateSampleForTimestamp) { - level.Error(h.logger).Log("msg", "Out of order histogram from remote write", "err", err.Error(), "series", labels.String(), "timestamp", hp.Timestamp) - } - return err + // Exemplars. + for _, ep := range ts.Exemplars { + e := ep.ToExemplar(&b, req.Symbols) + ref, err = app.AppendExemplar(ref, ls, e) + if err == nil { + rs.exemplars++ + continue + } + // Handle append error. + // TODO(bwplotka): I left the logic as in v1, but we might want to make it consistent with samples and histograms. + // Since exemplar storage is still experimental, we don't fail in anyway, the request on ingestion errors. + if errors.Is(err, storage.ErrOutOfOrderExemplar) { + outOfOrderExemplarErrs++ + level.Debug(h.logger).Log("msg", "Out of order exemplar", "err", err.Error(), "series", ls.String(), "exemplar", fmt.Sprintf("%+v", e)) + continue } + level.Debug(h.logger).Log("msg", "Error while adding exemplar in AppendExemplar", "series", ls.String(), "exemplar", fmt.Sprintf("%+v", e), "err", err) + } + + m := ts.ToMetadata(req.Symbols) + if _, err = app.UpdateMetadata(ref, ls, m); err != nil { + level.Debug(h.logger).Log("msg", "error while updating metadata from remote write", "err", err) + // Metadata is attached to each series, so since Prometheus does not reject sample without metadata information, + // we don't report remote write error either. We increment metric instead. + samplesWithoutMetadata += (rs.samples + rs.histograms) - allSamplesSoFar } } if outOfOrderExemplarErrs > 0 { - _ = level.Warn(h.logger).Log("msg", "Error on ingesting out-of-order exemplars", "num_dropped", outOfOrderExemplarErrs) - } - if samplesWithInvalidLabels > 0 { - h.samplesWithInvalidLabelsTotal.Add(float64(samplesWithInvalidLabels)) + level.Warn(h.logger).Log("msg", "Error on ingesting out-of-order exemplars", "num_dropped", outOfOrderExemplarErrs) } + h.samplesWithInvalidLabelsTotal.Add(float64(samplesWithInvalidLabels)) - return nil + if len(badRequestErrs) == 0 { + return samplesWithoutMetadata, 0, nil + } + // TODO(bwplotka): Better concat formatting? Perhaps add size limit? + return samplesWithoutMetadata, http.StatusBadRequest, errors.Join(badRequestErrs...) } // NewOTLPWriteHandler creates a http.Handler that accepts OTLP write requests and diff --git a/vendor/github.com/prometheus/prometheus/storage/secondary.go b/vendor/github.com/prometheus/prometheus/storage/secondary.go index 44d9781835..1cf8024b65 100644 --- a/vendor/github.com/prometheus/prometheus/storage/secondary.go +++ b/vendor/github.com/prometheus/prometheus/storage/secondary.go @@ -49,16 +49,16 @@ func newSecondaryQuerierFromChunk(cq ChunkQuerier) genericQuerier { return &secondaryQuerier{genericQuerier: newGenericQuerierFromChunk(cq)} } -func (s *secondaryQuerier) LabelValues(ctx context.Context, name string, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { - vals, w, err := s.genericQuerier.LabelValues(ctx, name, matchers...) +func (s *secondaryQuerier) LabelValues(ctx context.Context, name string, hints *LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { + vals, w, err := s.genericQuerier.LabelValues(ctx, name, hints, matchers...) if err != nil { return nil, w.Add(err), nil } return vals, w, nil } -func (s *secondaryQuerier) LabelNames(ctx context.Context, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { - names, w, err := s.genericQuerier.LabelNames(ctx, matchers...) +func (s *secondaryQuerier) LabelNames(ctx context.Context, hints *LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { + names, w, err := s.genericQuerier.LabelNames(ctx, hints, matchers...) if err != nil { return nil, w.Add(err), nil } diff --git a/vendor/github.com/prometheus/prometheus/tsdb/chunkenc/float_histogram.go b/vendor/github.com/prometheus/prometheus/tsdb/chunkenc/float_histogram.go index 1eed46ca87..a7c1fffb1e 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/chunkenc/float_histogram.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/chunkenc/float_histogram.go @@ -76,6 +76,7 @@ func (c *FloatHistogramChunk) NumSamples() int { func (c *FloatHistogramChunk) Layout() ( schema int32, zeroThreshold float64, negativeSpans, positiveSpans []histogram.Span, + customValues []float64, err error, ) { if c.NumSamples() == 0 { @@ -133,17 +134,18 @@ func (c *FloatHistogramChunk) Appender() (Appender, error) { a := &FloatHistogramAppender{ b: &c.b, - schema: it.schema, - zThreshold: it.zThreshold, - pSpans: it.pSpans, - nSpans: it.nSpans, - t: it.t, - tDelta: it.tDelta, - cnt: it.cnt, - zCnt: it.zCnt, - pBuckets: pBuckets, - nBuckets: nBuckets, - sum: it.sum, + schema: it.schema, + zThreshold: it.zThreshold, + pSpans: it.pSpans, + nSpans: it.nSpans, + customValues: it.customValues, + t: it.t, + tDelta: it.tDelta, + cnt: it.cnt, + zCnt: it.zCnt, + pBuckets: pBuckets, + nBuckets: nBuckets, + sum: it.sum, } if it.numTotal == 0 { a.sum.leading = 0xff @@ -191,6 +193,7 @@ type FloatHistogramAppender struct { schema int32 zThreshold float64 pSpans, nSpans []histogram.Span + customValues []float64 t, tDelta int64 sum, cnt, zCnt xorValue @@ -222,6 +225,7 @@ func (a *FloatHistogramAppender) Append(int64, float64) { // // The chunk is not appendable in the following cases: // - The schema has changed. +// - The custom bounds have changed if the current schema is custom buckets. // - The threshold for the zero bucket has changed. // - Any buckets have disappeared. // - There was a counter reset in the count of observations or in any bucket, including the zero bucket. @@ -263,6 +267,11 @@ func (a *FloatHistogramAppender) appendable(h *histogram.FloatHistogram) ( return } + if histogram.IsCustomBucketsSchema(h.Schema) && !histogram.FloatBucketsMatch(h.CustomValues, a.customValues) { + counterReset = true + return + } + if h.ZeroCount < a.zCnt.value { // There has been a counter reset since ZeroThreshold didn't change. counterReset = true @@ -303,6 +312,7 @@ func (a *FloatHistogramAppender) appendable(h *histogram.FloatHistogram) ( // // The chunk is not appendable in the following cases: // - The schema has changed. +// - The custom bounds have changed if the current schema is custom buckets. // - The threshold for the zero bucket has changed. // - The last sample in the chunk was stale while the current sample is not stale. func (a *FloatHistogramAppender) appendableGauge(h *histogram.FloatHistogram) ( @@ -329,6 +339,10 @@ func (a *FloatHistogramAppender) appendableGauge(h *histogram.FloatHistogram) ( return } + if histogram.IsCustomBucketsSchema(h.Schema) && !histogram.FloatBucketsMatch(h.CustomValues, a.customValues) { + return + } + positiveInserts, backwardPositiveInserts, positiveSpans = expandSpansBothWays(a.pSpans, h.PositiveSpans) negativeInserts, backwardNegativeInserts, negativeSpans = expandSpansBothWays(a.nSpans, h.NegativeSpans) okToAppend = true @@ -422,7 +436,7 @@ func (a *FloatHistogramAppender) appendFloatHistogram(t int64, h *histogram.Floa if num == 0 { // The first append gets the privilege to dictate the layout // but it's also responsible for encoding it into the chunk! - writeHistogramChunkLayout(a.b, h.Schema, h.ZeroThreshold, h.PositiveSpans, h.NegativeSpans) + writeHistogramChunkLayout(a.b, h.Schema, h.ZeroThreshold, h.PositiveSpans, h.NegativeSpans, h.CustomValues) a.schema = h.Schema a.zThreshold = h.ZeroThreshold @@ -438,6 +452,12 @@ func (a *FloatHistogramAppender) appendFloatHistogram(t int64, h *histogram.Floa } else { a.nSpans = nil } + if len(h.CustomValues) > 0 { + a.customValues = make([]float64, len(h.CustomValues)) + copy(a.customValues, h.CustomValues) + } else { + a.customValues = nil + } numPBuckets, numNBuckets := countSpans(h.PositiveSpans), countSpans(h.NegativeSpans) if numPBuckets > 0 { @@ -693,6 +713,7 @@ type floatHistogramIterator struct { schema int32 zThreshold float64 pSpans, nSpans []histogram.Span + customValues []float64 // For the fields that are tracked as deltas and ultimately dod's. t int64 @@ -753,6 +774,7 @@ func (it *floatHistogramIterator) AtFloatHistogram(fh *histogram.FloatHistogram) NegativeSpans: it.nSpans, PositiveBuckets: it.pBuckets, NegativeBuckets: it.nBuckets, + CustomValues: it.customValues, } } @@ -775,6 +797,9 @@ func (it *floatHistogramIterator) AtFloatHistogram(fh *histogram.FloatHistogram) fh.NegativeBuckets = resize(fh.NegativeBuckets, len(it.nBuckets)) copy(fh.NegativeBuckets, it.nBuckets) + fh.CustomValues = resize(fh.CustomValues, len(it.customValues)) + copy(fh.CustomValues, it.customValues) + return it.t, fh } @@ -819,7 +844,7 @@ func (it *floatHistogramIterator) Next() ValueType { // The first read is responsible for reading the chunk layout // and for initializing fields that depend on it. We give // counter reset info at chunk level, hence we discard it here. - schema, zeroThreshold, posSpans, negSpans, err := readHistogramChunkLayout(&it.br) + schema, zeroThreshold, posSpans, negSpans, customValues, err := readHistogramChunkLayout(&it.br) if err != nil { it.err = err return ValNone @@ -827,6 +852,7 @@ func (it *floatHistogramIterator) Next() ValueType { it.schema = schema it.zThreshold = zeroThreshold it.pSpans, it.nSpans = posSpans, negSpans + it.customValues = customValues numPBuckets, numNBuckets := countSpans(posSpans), countSpans(negSpans) // Allocate bucket slices as needed, recycling existing slices // in case this iterator was reset and already has slices of a diff --git a/vendor/github.com/prometheus/prometheus/tsdb/chunkenc/histogram.go b/vendor/github.com/prometheus/prometheus/tsdb/chunkenc/histogram.go index e12aec4dcd..aa74badd10 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/chunkenc/histogram.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/chunkenc/histogram.go @@ -69,6 +69,7 @@ func (c *HistogramChunk) NumSamples() int { func (c *HistogramChunk) Layout() ( schema int32, zeroThreshold float64, negativeSpans, positiveSpans []histogram.Span, + customValues []float64, err error, ) { if c.NumSamples() == 0 { @@ -131,6 +132,7 @@ func (c *HistogramChunk) Appender() (Appender, error) { zThreshold: it.zThreshold, pSpans: it.pSpans, nSpans: it.nSpans, + customValues: it.customValues, t: it.t, cnt: it.cnt, zCnt: it.zCnt, @@ -198,6 +200,7 @@ type HistogramAppender struct { schema int32 zThreshold float64 pSpans, nSpans []histogram.Span + customValues []float64 // Although we intend to start new chunks on counter resets, we still // have to handle negative deltas for gauge histograms. Therefore, even @@ -241,6 +244,7 @@ func (a *HistogramAppender) Append(int64, float64) { // The chunk is not appendable in the following cases: // // - The schema has changed. +// - The custom bounds have changed if the current schema is custom buckets. // - The threshold for the zero bucket has changed. // - Any buckets have disappeared. // - There was a counter reset in the count of observations or in any bucket, @@ -283,6 +287,11 @@ func (a *HistogramAppender) appendable(h *histogram.Histogram) ( return } + if histogram.IsCustomBucketsSchema(h.Schema) && !histogram.FloatBucketsMatch(h.CustomValues, a.customValues) { + counterReset = true + return + } + if h.ZeroCount < a.zCnt { // There has been a counter reset since ZeroThreshold didn't change. counterReset = true @@ -323,6 +332,7 @@ func (a *HistogramAppender) appendable(h *histogram.Histogram) ( // // The chunk is not appendable in the following cases: // - The schema has changed. +// - The custom bounds have changed if the current schema is custom buckets. // - The threshold for the zero bucket has changed. // - The last sample in the chunk was stale while the current sample is not stale. func (a *HistogramAppender) appendableGauge(h *histogram.Histogram) ( @@ -349,6 +359,10 @@ func (a *HistogramAppender) appendableGauge(h *histogram.Histogram) ( return } + if histogram.IsCustomBucketsSchema(h.Schema) && !histogram.FloatBucketsMatch(h.CustomValues, a.customValues) { + return + } + positiveInserts, backwardPositiveInserts, positiveSpans = expandSpansBothWays(a.pSpans, h.PositiveSpans) negativeInserts, backwardNegativeInserts, negativeSpans = expandSpansBothWays(a.nSpans, h.NegativeSpans) okToAppend = true @@ -442,7 +456,7 @@ func (a *HistogramAppender) appendHistogram(t int64, h *histogram.Histogram) { if num == 0 { // The first append gets the privilege to dictate the layout // but it's also responsible for encoding it into the chunk! - writeHistogramChunkLayout(a.b, h.Schema, h.ZeroThreshold, h.PositiveSpans, h.NegativeSpans) + writeHistogramChunkLayout(a.b, h.Schema, h.ZeroThreshold, h.PositiveSpans, h.NegativeSpans, h.CustomValues) a.schema = h.Schema a.zThreshold = h.ZeroThreshold @@ -458,6 +472,12 @@ func (a *HistogramAppender) appendHistogram(t int64, h *histogram.Histogram) { } else { a.nSpans = nil } + if len(h.CustomValues) > 0 { + a.customValues = make([]float64, len(h.CustomValues)) + copy(a.customValues, h.CustomValues) + } else { + a.customValues = nil + } numPBuckets, numNBuckets := countSpans(h.PositiveSpans), countSpans(h.NegativeSpans) if numPBuckets > 0 { @@ -741,6 +761,7 @@ type histogramIterator struct { schema int32 zThreshold float64 pSpans, nSpans []histogram.Span + customValues []float64 // For the fields that are tracked as deltas and ultimately dod's. t int64 @@ -797,6 +818,7 @@ func (it *histogramIterator) AtHistogram(h *histogram.Histogram) (int64, *histog NegativeSpans: it.nSpans, PositiveBuckets: it.pBuckets, NegativeBuckets: it.nBuckets, + CustomValues: it.customValues, } } @@ -819,6 +841,9 @@ func (it *histogramIterator) AtHistogram(h *histogram.Histogram) (int64, *histog h.NegativeBuckets = resize(h.NegativeBuckets, len(it.nBuckets)) copy(h.NegativeBuckets, it.nBuckets) + h.CustomValues = resize(h.CustomValues, len(it.customValues)) + copy(h.CustomValues, it.customValues) + return it.t, h } @@ -839,6 +864,7 @@ func (it *histogramIterator) AtFloatHistogram(fh *histogram.FloatHistogram) (int NegativeSpans: it.nSpans, PositiveBuckets: it.pFloatBuckets, NegativeBuckets: it.nFloatBuckets, + CustomValues: it.customValues, } } @@ -869,6 +895,9 @@ func (it *histogramIterator) AtFloatHistogram(fh *histogram.FloatHistogram) (int fh.NegativeBuckets[i] = currentNegative } + fh.CustomValues = resize(fh.CustomValues, len(it.customValues)) + copy(fh.CustomValues, it.customValues) + return it.t, fh } @@ -927,7 +956,7 @@ func (it *histogramIterator) Next() ValueType { // The first read is responsible for reading the chunk layout // and for initializing fields that depend on it. We give // counter reset info at chunk level, hence we discard it here. - schema, zeroThreshold, posSpans, negSpans, err := readHistogramChunkLayout(&it.br) + schema, zeroThreshold, posSpans, negSpans, customValues, err := readHistogramChunkLayout(&it.br) if err != nil { it.err = err return ValNone @@ -935,6 +964,7 @@ func (it *histogramIterator) Next() ValueType { it.schema = schema it.zThreshold = zeroThreshold it.pSpans, it.nSpans = posSpans, negSpans + it.customValues = customValues numPBuckets, numNBuckets := countSpans(posSpans), countSpans(negSpans) // The code below recycles existing slices in case this iterator // was reset and already has slices of a sufficient capacity. diff --git a/vendor/github.com/prometheus/prometheus/tsdb/chunkenc/histogram_meta.go b/vendor/github.com/prometheus/prometheus/tsdb/chunkenc/histogram_meta.go index 9aae485a83..c5381ba2fb 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/chunkenc/histogram_meta.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/chunkenc/histogram_meta.go @@ -21,17 +21,21 @@ import ( func writeHistogramChunkLayout( b *bstream, schema int32, zeroThreshold float64, - positiveSpans, negativeSpans []histogram.Span, + positiveSpans, negativeSpans []histogram.Span, customValues []float64, ) { putZeroThreshold(b, zeroThreshold) putVarbitInt(b, int64(schema)) putHistogramChunkLayoutSpans(b, positiveSpans) putHistogramChunkLayoutSpans(b, negativeSpans) + if histogram.IsCustomBucketsSchema(schema) { + putHistogramChunkLayoutCustomBounds(b, customValues) + } } func readHistogramChunkLayout(b *bstreamReader) ( schema int32, zeroThreshold float64, positiveSpans, negativeSpans []histogram.Span, + customValues []float64, err error, ) { zeroThreshold, err = readZeroThreshold(b) @@ -55,6 +59,13 @@ func readHistogramChunkLayout(b *bstreamReader) ( return } + if histogram.IsCustomBucketsSchema(schema) { + customValues, err = readHistogramChunkLayoutCustomBounds(b) + if err != nil { + return + } + } + return } @@ -91,6 +102,30 @@ func readHistogramChunkLayoutSpans(b *bstreamReader) ([]histogram.Span, error) { return spans, nil } +func putHistogramChunkLayoutCustomBounds(b *bstream, customValues []float64) { + putVarbitUint(b, uint64(len(customValues))) + for _, bound := range customValues { + putCustomBound(b, bound) + } +} + +func readHistogramChunkLayoutCustomBounds(b *bstreamReader) ([]float64, error) { + var customValues []float64 + num, err := readVarbitUint(b) + if err != nil { + return nil, err + } + for i := 0; i < int(num); i++ { + bound, err := readCustomBound(b) + if err != nil { + return nil, err + } + + customValues = append(customValues, bound) + } + return customValues, nil +} + // putZeroThreshold writes the zero threshold to the bstream. It stores typical // values in just one byte, but needs 9 bytes for other values. In detail: // - If the threshold is 0, store a single zero byte. @@ -139,6 +174,59 @@ func readZeroThreshold(br *bstreamReader) (float64, error) { } } +// 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 +} + +// putCustomBound writes a custom bound to the bstream. It stores values from +// 0 to 33554.430 (inclusive) that are multiples of 0.001 in unsigned varbit +// encoding of up to 4 bytes, but needs 1 bit + 8 bytes for other values like +// negative numbers, numbers greater than 33554.430, or numbers that are not +// a multiple of 0.001, on the assumption that they are less common. In detail: +// - Multiply the bound by 1000, without rounding. +// - If the multiplied bound is >= 0, <= 33554430 and a whole number, +// add 1 and store it in unsigned varbit encoding. All these numbers are +// greater than 0, so the leading bit of the varbit is always 1! +// - Otherwise, store a 0 bit, followed by the 8 bytes of the original +// bound as a float64. +// +// When reading the values, we can first decode a value as unsigned varbit, +// if it's 0, then we read the next 8 bytes as a float64, otherwise +// we can convert the value to a float64 by subtracting 1 and dividing by 1000. +func putCustomBound(b *bstream, f float64) { + tf := f * 1000 + // 33554431-1 comes from the maximum that can be stored in a varbit in 4 + // bytes, other values are stored in 8 bytes anyway. + if tf < 0 || tf > 33554430 || !isWholeWhenMultiplied(f) { + b.writeBit(zero) + b.writeBits(math.Float64bits(f), 64) + return + } + putVarbitUint(b, uint64(math.Round(tf))+1) +} + +// readCustomBound reads the custom bound written with putCustomBound. +func readCustomBound(br *bstreamReader) (float64, error) { + b, err := readVarbitUint(br) + if err != nil { + return 0, err + } + switch b { + case 0: + v, err := br.readBits(64) + if err != nil { + return 0, err + } + return math.Float64frombits(v), nil + default: + return float64(b-1) / 1000, nil + } +} + type bucketIterator struct { spans []histogram.Span span int // Span position of last yielded bucket. diff --git a/vendor/github.com/prometheus/prometheus/tsdb/chunkenc/xor.go b/vendor/github.com/prometheus/prometheus/tsdb/chunkenc/xor.go index 9430de3964..3177762f81 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/chunkenc/xor.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/chunkenc/xor.go @@ -60,7 +60,7 @@ type XORChunk struct { b bstream } -// NewXORChunk returns a new chunk with XOR encoding of the given size. +// NewXORChunk returns a new chunk with XOR encoding. func NewXORChunk() *XORChunk { b := make([]byte, 2, 128) return &XORChunk{b: bstream{stream: b, count: 0}} diff --git a/vendor/github.com/prometheus/prometheus/tsdb/chunks/chunks.go b/vendor/github.com/prometheus/prometheus/tsdb/chunks/chunks.go index e7df0eeed2..ec0f6d4036 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/chunks/chunks.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/chunks/chunks.go @@ -133,15 +133,6 @@ type Meta struct { // Time range the data covers. // When MaxTime == math.MaxInt64 the chunk is still open and being appended to. MinTime, MaxTime int64 - - // OOOLastRef, OOOLastMinTime and OOOLastMaxTime are kept as markers for - // overlapping chunks. - // These fields point to the last created out of order Chunk (the head) that existed - // when Series() was called and was overlapping. - // Series() and Chunk() method responses should be consistent for the same - // query even if new data is added in between the calls. - OOOLastRef ChunkRef - OOOLastMinTime, OOOLastMaxTime int64 } // ChunkFromSamples requires all samples to have the same type. diff --git a/vendor/github.com/prometheus/prometheus/tsdb/db.go b/vendor/github.com/prometheus/prometheus/tsdb/db.go index b2175d4758..090d6fcf0c 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/db.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/db.go @@ -1407,6 +1407,9 @@ func (db *DB) compactHead(head *RangeHead) error { if err = db.head.truncateMemory(head.BlockMaxTime()); err != nil { return fmt.Errorf("head memory truncate: %w", err) } + + db.head.RebuildSymbolTable(db.logger) + return nil } diff --git a/vendor/github.com/prometheus/prometheus/tsdb/head.go b/vendor/github.com/prometheus/prometheus/tsdb/head.go index 5972a9c5d6..b7bfaa0fda 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/head.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/head.go @@ -1759,12 +1759,12 @@ type seriesHashmap struct { func (m *seriesHashmap) get(hash uint64, lset labels.Labels) *memSeries { if s, found := m.unique[hash]; found { - if labels.Equal(s.lset, lset) { + if labels.Equal(s.labels(), lset) { return s } } for _, s := range m.conflicts[hash] { - if labels.Equal(s.lset, lset) { + if labels.Equal(s.labels(), lset) { return s } } @@ -1772,7 +1772,7 @@ func (m *seriesHashmap) get(hash uint64, lset labels.Labels) *memSeries { } func (m *seriesHashmap) set(hash uint64, s *memSeries) { - if existing, found := m.unique[hash]; !found || labels.Equal(existing.lset, s.lset) { + if existing, found := m.unique[hash]; !found || labels.Equal(existing.labels(), s.labels()) { m.unique[hash] = s return } @@ -1781,7 +1781,7 @@ func (m *seriesHashmap) set(hash uint64, s *memSeries) { } l := m.conflicts[hash] for i, prev := range l { - if labels.Equal(prev.lset, s.lset) { + if labels.Equal(prev.labels(), s.labels()) { l[i] = s return } @@ -1931,7 +1931,7 @@ func (s *stripeSeries) gc(mint int64, minOOOMmapRef chunks.ChunkDiskMapperRef) ( series.lset.Range(func(l labels.Label) { affected[l] = struct{}{} }) s.hashes[hashShard].del(hash, series.ref) delete(s.series[refShard], series.ref) - deletedForCallback[series.ref] = series.lset + deletedForCallback[series.ref] = series.lset // OK to access lset; series is locked at the top of this function. } s.iterForDeletion(check) @@ -2023,7 +2023,7 @@ func (s *stripeSeries) getOrSet(hash uint64, lset labels.Labels, createSeries fu } // Setting the series in the s.hashes marks the creation of series // as any further calls to this methods would return that series. - s.seriesLifecycleCallback.PostCreation(series.lset) + s.seriesLifecycleCallback.PostCreation(series.labels()) i = uint64(series.ref) & uint64(s.size-1) @@ -2064,16 +2064,19 @@ func (s sample) Type() chunkenc.ValueType { // memSeries is the in-memory representation of a series. None of its methods // are goroutine safe and it is the caller's responsibility to lock it. type memSeries struct { - sync.Mutex - + // Members up to the Mutex are not changed after construction, so can be accessed without a lock. ref chunks.HeadSeriesRef - lset labels.Labels meta *metadata.Metadata // Series labels hash to use for sharding purposes. The value is always 0 when sharding has not // been explicitly enabled in TSDB. shardHash uint64 + // Everything after here should only be accessed with the lock held. + sync.Mutex + + lset labels.Labels // Locking required with -tags dedupelabels, not otherwise. + // Immutable chunks on disk that have not yet gone into a block, in order of ascending time stamps. // When compaction runs, chunks get moved into a block and all pointers are shifted like so: // @@ -2096,6 +2099,7 @@ type memSeries struct { nextAt int64 // Timestamp at which to cut the next chunk. histogramChunkHasComputedEndTime bool // True if nextAt has been predicted for the current histograms chunk; false otherwise. + pendingCommit bool // Whether there are samples waiting to be committed to this series. // We keep the last value here (in addition to appending it to the chunk) so we can check for duplicates. lastValue float64 @@ -2111,8 +2115,6 @@ type memSeries struct { // txs is nil if isolation is disabled. txs *txRing - - pendingCommit bool // Whether there are samples waiting to be committed to this series. } // memSeriesOOOFields contains the fields required by memSeries diff --git a/vendor/github.com/prometheus/prometheus/tsdb/head_append.go b/vendor/github.com/prometheus/prometheus/tsdb/head_append.go index 62c3727e28..8d66d1e818 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/head_append.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/head_append.go @@ -554,7 +554,7 @@ func (a *headAppender) AppendExemplar(ref storage.SeriesRef, lset labels.Labels, // Ensure no empty labels have gotten through. e.Labels = e.Labels.WithoutEmpty() - err := a.head.exemplars.ValidateExemplar(s.lset, e) + err := a.head.exemplars.ValidateExemplar(s.labels(), e) if err != nil { if errors.Is(err, storage.ErrDuplicateExemplar) || errors.Is(err, storage.ErrExemplarsDisabled) { // Duplicate, don't return an error but don't accept the exemplar. @@ -708,7 +708,7 @@ func (a *headAppender) GetRef(lset labels.Labels, hash uint64) (storage.SeriesRe return 0, labels.EmptyLabels() } // returned labels must be suitable to pass to Append() - return storage.SeriesRef(s.ref), s.lset + return storage.SeriesRef(s.ref), s.labels() } // log writes all headAppender's data to the WAL. @@ -816,7 +816,7 @@ func (a *headAppender) Commit() (err error) { continue } // We don't instrument exemplar appends here, all is instrumented by storage. - if err := a.head.exemplars.AddExemplar(s.lset, e.exemplar); err != nil { + if err := a.head.exemplars.AddExemplar(s.labels(), e.exemplar); err != nil { if errors.Is(err, storage.ErrOutOfOrderExemplar) { continue } @@ -846,16 +846,17 @@ func (a *headAppender) Commit() (err error) { // number of samples rejected due to: out of bounds: with t < minValidTime (OOO support disabled) floatOOBRejected int - inOrderMint int64 = math.MaxInt64 - inOrderMaxt int64 = math.MinInt64 - ooomint int64 = math.MaxInt64 - ooomaxt int64 = math.MinInt64 - wblSamples []record.RefSample - oooMmapMarkers map[chunks.HeadSeriesRef]chunks.ChunkDiskMapperRef - oooRecords [][]byte - oooCapMax = a.head.opts.OutOfOrderCapMax.Load() - series *memSeries - appendChunkOpts = chunkOpts{ + inOrderMint int64 = math.MaxInt64 + inOrderMaxt int64 = math.MinInt64 + oooMinT int64 = math.MaxInt64 + oooMaxT int64 = math.MinInt64 + wblSamples []record.RefSample + oooMmapMarkers map[chunks.HeadSeriesRef][]chunks.ChunkDiskMapperRef + oooMmapMarkersCount int + oooRecords [][]byte + oooCapMax = a.head.opts.OutOfOrderCapMax.Load() + series *memSeries + appendChunkOpts = chunkOpts{ chunkDiskMapper: a.head.chunkDiskMapper, chunkRange: a.head.chunkRange.Load(), samplesPerChunk: a.head.opts.SamplesPerChunk, @@ -872,6 +873,7 @@ func (a *headAppender) Commit() (err error) { // WBL is not enabled. So no need to collect. wblSamples = nil oooMmapMarkers = nil + oooMmapMarkersCount = 0 return } // The m-map happens before adding a new sample. So we collect @@ -880,12 +882,14 @@ func (a *headAppender) Commit() (err error) { // WBL Before this Commit(): [old samples before this commit for chunk 1] // WBL After this Commit(): [old samples before this commit for chunk 1][new samples in this commit for chunk 1]mmapmarker1[samples for chunk 2]mmapmarker2[samples for chunk 3] if oooMmapMarkers != nil { - markers := make([]record.RefMmapMarker, 0, len(oooMmapMarkers)) - for ref, mmapRef := range oooMmapMarkers { - markers = append(markers, record.RefMmapMarker{ - Ref: ref, - MmapRef: mmapRef, - }) + markers := make([]record.RefMmapMarker, 0, oooMmapMarkersCount) + for ref, mmapRefs := range oooMmapMarkers { + for _, mmapRef := range mmapRefs { + markers = append(markers, record.RefMmapMarker{ + Ref: ref, + MmapRef: mmapRef, + }) + } } r := enc.MmapMarkers(markers, a.head.getBytesBuffer()) oooRecords = append(oooRecords, r) @@ -928,32 +932,39 @@ func (a *headAppender) Commit() (err error) { case oooSample: // Sample is OOO and OOO handling is enabled // and the delta is within the OOO tolerance. - var mmapRef chunks.ChunkDiskMapperRef - ok, chunkCreated, mmapRef = series.insert(s.T, s.V, a.head.chunkDiskMapper, oooCapMax) + var mmapRefs []chunks.ChunkDiskMapperRef + ok, chunkCreated, mmapRefs = series.insert(s.T, s.V, a.head.chunkDiskMapper, oooCapMax) if chunkCreated { r, ok := oooMmapMarkers[series.ref] - if !ok || r != 0 { + if !ok || r != nil { // !ok means there are no markers collected for these samples yet. So we first flush the samples // before setting this m-map marker. - // r != 0 means we have already m-mapped a chunk for this series in the same Commit(). + // r != nil means we have already m-mapped a chunk for this series in the same Commit(). // Hence, before we m-map again, we should add the samples and m-map markers // seen till now to the WBL records. collectOOORecords() } if oooMmapMarkers == nil { - oooMmapMarkers = make(map[chunks.HeadSeriesRef]chunks.ChunkDiskMapperRef) + oooMmapMarkers = make(map[chunks.HeadSeriesRef][]chunks.ChunkDiskMapperRef) + } + if len(mmapRefs) > 0 { + oooMmapMarkers[series.ref] = mmapRefs + oooMmapMarkersCount += len(mmapRefs) + } else { + // No chunk was written to disk, so we need to set an initial marker for this series. + oooMmapMarkers[series.ref] = []chunks.ChunkDiskMapperRef{0} + oooMmapMarkersCount++ } - oooMmapMarkers[series.ref] = mmapRef } if ok { wblSamples = append(wblSamples, s) - if s.T < ooomint { - ooomint = s.T + if s.T < oooMinT { + oooMinT = s.T } - if s.T > ooomaxt { - ooomaxt = s.T + if s.T > oooMaxT { + oooMaxT = s.T } floatOOOAccepted++ } else { @@ -1053,7 +1064,7 @@ func (a *headAppender) Commit() (err error) { a.head.metrics.samplesAppended.WithLabelValues(sampleMetricTypeHistogram).Add(float64(histogramsAppended)) a.head.metrics.outOfOrderSamplesAppended.WithLabelValues(sampleMetricTypeFloat).Add(float64(floatOOOAccepted)) a.head.updateMinMaxTime(inOrderMint, inOrderMaxt) - a.head.updateMinOOOMaxOOOTime(ooomint, ooomaxt) + a.head.updateMinOOOMaxOOOTime(oooMinT, oooMaxT) collectOOORecords() if a.head.wbl != nil { @@ -1069,14 +1080,14 @@ func (a *headAppender) Commit() (err error) { } // insert is like append, except it inserts. Used for OOO samples. -func (s *memSeries) insert(t int64, v float64, chunkDiskMapper *chunks.ChunkDiskMapper, oooCapMax int64) (inserted, chunkCreated bool, mmapRef chunks.ChunkDiskMapperRef) { +func (s *memSeries) insert(t int64, v float64, chunkDiskMapper *chunks.ChunkDiskMapper, oooCapMax int64) (inserted, chunkCreated bool, mmapRefs []chunks.ChunkDiskMapperRef) { if s.ooo == nil { s.ooo = &memSeriesOOOFields{} } c := s.ooo.oooHeadChunk if c == nil || c.chunk.NumSamples() == int(oooCapMax) { // Note: If no new samples come in then we rely on compaction to clean up stale in-memory OOO chunks. - c, mmapRef = s.cutNewOOOHeadChunk(t, chunkDiskMapper) + c, mmapRefs = s.cutNewOOOHeadChunk(t, chunkDiskMapper) chunkCreated = true } @@ -1089,7 +1100,7 @@ func (s *memSeries) insert(t int64, v float64, chunkDiskMapper *chunks.ChunkDisk c.maxTime = t } } - return ok, chunkCreated, mmapRef + return ok, chunkCreated, mmapRefs } // chunkOpts are chunk-level options that are passed when appending to a memSeries. @@ -1431,7 +1442,7 @@ func (s *memSeries) cutNewHeadChunk(mint int64, e chunkenc.Encoding, chunkRange // cutNewOOOHeadChunk cuts a new OOO chunk and m-maps the old chunk. // The caller must ensure that s.ooo is not nil. -func (s *memSeries) cutNewOOOHeadChunk(mint int64, chunkDiskMapper *chunks.ChunkDiskMapper) (*oooHeadChunk, chunks.ChunkDiskMapperRef) { +func (s *memSeries) cutNewOOOHeadChunk(mint int64, chunkDiskMapper *chunks.ChunkDiskMapper) (*oooHeadChunk, []chunks.ChunkDiskMapperRef) { ref := s.mmapCurrentOOOHeadChunk(chunkDiskMapper) s.ooo.oooHeadChunk = &oooHeadChunk{ @@ -1443,21 +1454,29 @@ func (s *memSeries) cutNewOOOHeadChunk(mint int64, chunkDiskMapper *chunks.Chunk return s.ooo.oooHeadChunk, ref } -func (s *memSeries) mmapCurrentOOOHeadChunk(chunkDiskMapper *chunks.ChunkDiskMapper) chunks.ChunkDiskMapperRef { +func (s *memSeries) mmapCurrentOOOHeadChunk(chunkDiskMapper *chunks.ChunkDiskMapper) []chunks.ChunkDiskMapperRef { if s.ooo == nil || s.ooo.oooHeadChunk == nil { - // There is no head chunk, so nothing to m-map here. - return 0 - } - xor, _ := s.ooo.oooHeadChunk.chunk.ToXOR() // Encode to XorChunk which is more compact and implements all of the needed functionality. - chunkRef := chunkDiskMapper.WriteChunk(s.ref, s.ooo.oooHeadChunk.minTime, s.ooo.oooHeadChunk.maxTime, xor, true, handleChunkWriteError) - s.ooo.oooMmappedChunks = append(s.ooo.oooMmappedChunks, &mmappedChunk{ - ref: chunkRef, - numSamples: uint16(xor.NumSamples()), - minTime: s.ooo.oooHeadChunk.minTime, - maxTime: s.ooo.oooHeadChunk.maxTime, - }) + // OOO is not enabled or there is no head chunk, so nothing to m-map here. + return nil + } + chks, err := s.ooo.oooHeadChunk.chunk.ToEncodedChunks(math.MinInt64, math.MaxInt64) + if err != nil { + handleChunkWriteError(err) + return nil + } + chunkRefs := make([]chunks.ChunkDiskMapperRef, 0, 1) + for _, memchunk := range chks { + chunkRef := chunkDiskMapper.WriteChunk(s.ref, s.ooo.oooHeadChunk.minTime, s.ooo.oooHeadChunk.maxTime, memchunk.chunk, true, handleChunkWriteError) + chunkRefs = append(chunkRefs, chunkRef) + s.ooo.oooMmappedChunks = append(s.ooo.oooMmappedChunks, &mmappedChunk{ + ref: chunkRef, + numSamples: uint16(memchunk.chunk.NumSamples()), + minTime: memchunk.minTime, + maxTime: memchunk.maxTime, + }) + } s.ooo.oooHeadChunk = nil - return chunkRef + return chunkRefs } // mmapChunks will m-map all but first chunk on s.headChunks list. diff --git a/vendor/github.com/prometheus/prometheus/tsdb/head_dedupelabels.go b/vendor/github.com/prometheus/prometheus/tsdb/head_dedupelabels.go new file mode 100644 index 0000000000..a16d907261 --- /dev/null +++ b/vendor/github.com/prometheus/prometheus/tsdb/head_dedupelabels.go @@ -0,0 +1,95 @@ +// 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. + +//go:build dedupelabels + +package tsdb + +import ( + "github.com/go-kit/log" + "github.com/go-kit/log/level" + + "github.com/prometheus/prometheus/model/labels" +) + +// Helper method to access labels under lock. +func (s *memSeries) labels() labels.Labels { + s.Lock() + defer s.Unlock() + return s.lset +} + +// RebuildSymbolTable goes through all the series in h, build a SymbolTable with all names and values, +// replace each series' Labels with one using that SymbolTable. +func (h *Head) RebuildSymbolTable(logger log.Logger) *labels.SymbolTable { + level.Info(logger).Log("msg", "RebuildSymbolTable starting") + st := labels.NewSymbolTable() + builder := labels.NewScratchBuilderWithSymbolTable(st, 0) + rebuildLabels := func(lbls labels.Labels) labels.Labels { + builder.Reset() + lbls.Range(func(l labels.Label) { + builder.Add(l.Name, l.Value) + }) + return builder.Labels() + } + + for i := 0; i < h.series.size; i++ { + h.series.locks[i].Lock() + + for _, s := range h.series.hashes[i].unique { + s.Lock() + s.lset = rebuildLabels(s.lset) + s.Unlock() + } + + for _, all := range h.series.hashes[i].conflicts { + for _, s := range all { + s.Lock() + s.lset = rebuildLabels(s.lset) + s.Unlock() + } + } + + h.series.locks[i].Unlock() + } + type withReset interface{ ResetSymbolTable(*labels.SymbolTable) } + if e, ok := h.exemplars.(withReset); ok { + e.ResetSymbolTable(st) + } + level.Info(logger).Log("msg", "RebuildSymbolTable finished", "size", st.Len()) + return st +} + +func (ce *CircularExemplarStorage) ResetSymbolTable(st *labels.SymbolTable) { + builder := labels.NewScratchBuilderWithSymbolTable(st, 0) + rebuildLabels := func(lbls labels.Labels) labels.Labels { + builder.Reset() + lbls.Range(func(l labels.Label) { + builder.Add(l.Name, l.Value) + }) + return builder.Labels() + } + + ce.lock.RLock() + defer ce.lock.RUnlock() + + for _, v := range ce.index { + v.seriesLabels = rebuildLabels(v.seriesLabels) + } + for i := range ce.exemplars { + if ce.exemplars[i].ref == nil { + continue + } + ce.exemplars[i].exemplar.Labels = rebuildLabels(ce.exemplars[i].exemplar.Labels) + } +} diff --git a/vendor/github.com/prometheus/prometheus/tsdb/head_other.go b/vendor/github.com/prometheus/prometheus/tsdb/head_other.go new file mode 100644 index 0000000000..eb1b93a3e5 --- /dev/null +++ b/vendor/github.com/prometheus/prometheus/tsdb/head_other.go @@ -0,0 +1,32 @@ +// 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. + +//go:build !dedupelabels + +package tsdb + +import ( + "github.com/go-kit/log" + + "github.com/prometheus/prometheus/model/labels" +) + +// Helper method to access labels; trivial when not using dedupelabels. +func (s *memSeries) labels() labels.Labels { + return s.lset +} + +// No-op when not using dedupelabels. +func (h *Head) RebuildSymbolTable(logger log.Logger) *labels.SymbolTable { + return nil +} diff --git a/vendor/github.com/prometheus/prometheus/tsdb/head_read.go b/vendor/github.com/prometheus/prometheus/tsdb/head_read.go index 689972f1b7..9ba8785ad2 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/head_read.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/head_read.go @@ -142,7 +142,7 @@ func (h *headIndexReader) SortedPostings(p index.Postings) index.Postings { } slices.SortFunc(series, func(a, b *memSeries) int { - return labels.Compare(a.lset, b.lset) + return labels.Compare(a.labels(), b.labels()) }) // Convert back to list. @@ -189,7 +189,7 @@ func (h *headIndexReader) Series(ref storage.SeriesRef, builder *labels.ScratchB h.head.metrics.seriesNotFound.Inc() return storage.ErrNotFound } - builder.Assign(s.lset) + builder.Assign(s.labels()) if chks == nil { return nil @@ -259,7 +259,7 @@ func (h *headIndexReader) LabelValueFor(_ context.Context, id storage.SeriesRef, return "", storage.ErrNotFound } - value := memSeries.lset.Get(label) + value := memSeries.labels().Get(label) if value == "" { return "", storage.ErrNotFound } @@ -283,7 +283,7 @@ func (h *headIndexReader) LabelNamesFor(ctx context.Context, series index.Postin // when series was garbage collected after the caller got the series IDs. continue } - memSeries.lset.Range(func(lbl labels.Label) { + memSeries.labels().Range(func(lbl labels.Label) { namesMap[lbl.Name] = struct{}{} }) } @@ -487,55 +487,24 @@ func (s *memSeries) oooMergedChunks(meta chunks.Meta, cdm *chunks.ChunkDiskMappe // We create a temporary slice of chunk metas to hold the information of all // possible chunks that may overlap with the requested chunk. - tmpChks := make([]chunkMetaAndChunkDiskMapperRef, 0, len(s.ooo.oooMmappedChunks)) - - oooHeadRef := chunks.ChunkRef(chunks.NewHeadChunkRef(s.ref, s.oooHeadChunkID(len(s.ooo.oooMmappedChunks)))) - if s.ooo.oooHeadChunk != nil && s.ooo.oooHeadChunk.OverlapsClosedInterval(mint, maxt) { - // We only want to append the head chunk if this chunk existed when - // Series() was called. This brings consistency in case new data - // is added in between Series() and Chunk() calls. - if oooHeadRef == meta.OOOLastRef { - tmpChks = append(tmpChks, chunkMetaAndChunkDiskMapperRef{ - meta: chunks.Meta{ - // Ignoring samples added before and after the last known min and max time for this chunk. - MinTime: meta.OOOLastMinTime, - MaxTime: meta.OOOLastMaxTime, - Ref: oooHeadRef, - }, - }) - } - } + tmpChks := make([]chunkMetaAndChunkDiskMapperRef, 0, len(s.ooo.oooMmappedChunks)+1) for i, c := range s.ooo.oooMmappedChunks { - chunkRef := chunks.ChunkRef(chunks.NewHeadChunkRef(s.ref, s.oooHeadChunkID(i))) - // We can skip chunks that came in later than the last known OOOLastRef. - if chunkRef > meta.OOOLastRef { - break - } - - switch { - case chunkRef == meta.OOOLastRef: - tmpChks = append(tmpChks, chunkMetaAndChunkDiskMapperRef{ - meta: chunks.Meta{ - MinTime: meta.OOOLastMinTime, - MaxTime: meta.OOOLastMaxTime, - Ref: chunkRef, - }, - ref: c.ref, - origMinT: c.minTime, - origMaxT: c.maxTime, - }) - case c.OverlapsClosedInterval(mint, maxt): + if c.OverlapsClosedInterval(mint, maxt) { tmpChks = append(tmpChks, chunkMetaAndChunkDiskMapperRef{ meta: chunks.Meta{ MinTime: c.minTime, MaxTime: c.maxTime, - Ref: chunkRef, + Ref: chunks.ChunkRef(chunks.NewHeadChunkRef(s.ref, s.oooHeadChunkID(i))), }, ref: c.ref, }) } } + // Add in data copied from the head OOO chunk. + if meta.Chunk != nil { + tmpChks = append(tmpChks, chunkMetaAndChunkDiskMapperRef{meta: meta}) + } // Next we want to sort all the collected chunks by min time so we can find // those that overlap and stop when we know the rest don't. @@ -548,22 +517,8 @@ func (s *memSeries) oooMergedChunks(meta chunks.Meta, cdm *chunks.ChunkDiskMappe continue } var iterable chunkenc.Iterable - if c.meta.Ref == oooHeadRef { - var xor *chunkenc.XORChunk - var err error - // If head chunk min and max time match the meta OOO markers - // that means that the chunk has not expanded so we can append - // it as it is. - if s.ooo.oooHeadChunk.minTime == meta.OOOLastMinTime && s.ooo.oooHeadChunk.maxTime == meta.OOOLastMaxTime { - xor, err = s.ooo.oooHeadChunk.chunk.ToXOR() // TODO(jesus.vazquez) (This is an optimization idea that has no priority and might not be that useful) See if we could use a copy of the underlying slice. That would leave the more expensive ToXOR() function only for the usecase where Bytes() is called. - } else { - // We need to remove samples that are outside of the markers - xor, err = s.ooo.oooHeadChunk.chunk.ToXORBetweenTimestamps(meta.OOOLastMinTime, meta.OOOLastMaxTime) - } - if err != nil { - return nil, fmt.Errorf("failed to convert ooo head chunk to xor chunk: %w", err) - } - iterable = xor + if c.meta.Chunk != nil { + iterable = c.meta.Chunk } else { chk, err := cdm.Chunk(c.ref) if err != nil { @@ -573,16 +528,7 @@ func (s *memSeries) oooMergedChunks(meta chunks.Meta, cdm *chunks.ChunkDiskMappe } return nil, err } - if c.meta.Ref == meta.OOOLastRef && - (c.origMinT != meta.OOOLastMinTime || c.origMaxT != meta.OOOLastMaxTime) { - // The head expanded and was memory mapped so now we need to - // wrap the chunk within a chunk that doesnt allows us to iterate - // through samples out of the OOOLastMinT and OOOLastMaxT - // markers. - iterable = boundedIterable{chk, meta.OOOLastMinTime, meta.OOOLastMaxTime} - } else { - iterable = chk - } + iterable = chk } mc.chunkIterables = append(mc.chunkIterables, iterable) if c.meta.MaxTime > absoluteMax { @@ -593,74 +539,6 @@ func (s *memSeries) oooMergedChunks(meta chunks.Meta, cdm *chunks.ChunkDiskMappe return mc, nil } -var _ chunkenc.Iterable = &boundedIterable{} - -// boundedIterable is an implementation of chunkenc.Iterable that uses a -// boundedIterator that only iterates through samples which timestamps are -// >= minT and <= maxT. -type boundedIterable struct { - chunk chunkenc.Chunk - minT int64 - maxT int64 -} - -func (b boundedIterable) Iterator(iterator chunkenc.Iterator) chunkenc.Iterator { - it := b.chunk.Iterator(iterator) - if it == nil { - panic("iterator shouldn't be nil") - } - return boundedIterator{it, b.minT, b.maxT} -} - -var _ chunkenc.Iterator = &boundedIterator{} - -// boundedIterator is an implementation of Iterator that only iterates through -// samples which timestamps are >= minT and <= maxT. -type boundedIterator struct { - chunkenc.Iterator - minT int64 - maxT int64 -} - -// Next the first time its called it will advance as many positions as necessary -// until its able to find a sample within the bounds minT and maxT. -// If there are samples within bounds it will advance one by one amongst them. -// If there are no samples within bounds it will return false. -func (b boundedIterator) Next() chunkenc.ValueType { - for b.Iterator.Next() == chunkenc.ValFloat { - t, _ := b.Iterator.At() - switch { - case t < b.minT: - continue - case t > b.maxT: - return chunkenc.ValNone - default: - return chunkenc.ValFloat - } - } - return chunkenc.ValNone -} - -func (b boundedIterator) Seek(t int64) chunkenc.ValueType { - if t < b.minT { - // We must seek at least up to b.minT if it is asked for something before that. - val := b.Iterator.Seek(b.minT) - if !(val == chunkenc.ValFloat) { - return chunkenc.ValNone - } - t, _ := b.Iterator.At() - if t <= b.maxT { - return chunkenc.ValFloat - } - } - if t > b.maxT { - // We seek anyway so that the subsequent Next() calls will also return false. - b.Iterator.Seek(t) - return chunkenc.ValNone - } - return b.Iterator.Seek(t) -} - // safeHeadChunk makes sure that the chunk can be accessed without a race condition. type safeHeadChunk struct { chunkenc.Chunk diff --git a/vendor/github.com/prometheus/prometheus/tsdb/head_wal.go b/vendor/github.com/prometheus/prometheus/tsdb/head_wal.go index 41f7dd46b2..787cb7c267 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/head_wal.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/head_wal.go @@ -126,7 +126,7 @@ func (h *Head) loadWAL(r *wlog.Reader, syms *labels.SymbolTable, multiRef map[ch } // At the moment the only possible error here is out of order exemplars, which we shouldn't see when // replaying the WAL, so lets just log the error if it's not that type. - err = h.exemplars.AddExemplar(ms.lset, exemplar.Exemplar{Ts: e.T, Value: e.V, Labels: e.Labels}) + err = h.exemplars.AddExemplar(ms.labels(), exemplar.Exemplar{Ts: e.T, Value: e.V, Labels: e.Labels}) if err != nil && errors.Is(err, storage.ErrOutOfOrderExemplar) { level.Warn(h.logger).Log("msg", "Unexpected error when replaying WAL on exemplar record", "err", err) } @@ -448,7 +448,7 @@ func (h *Head) resetSeriesWithMMappedChunks(mSeries *memSeries, mmc, oooMmc []*m ) { level.Debug(h.logger).Log( "msg", "M-mapped chunks overlap on a duplicate series record", - "series", mSeries.lset.String(), + "series", mSeries.labels().String(), "oldref", mSeries.ref, "oldmint", mSeries.mmappedChunks[0].minTime, "oldmaxt", mSeries.mmappedChunks[len(mSeries.mmappedChunks)-1].maxTime, @@ -932,7 +932,7 @@ func (s *memSeries) encodeToSnapshotRecord(b []byte) []byte { buf.PutByte(chunkSnapshotRecordTypeSeries) buf.PutBE64(uint64(s.ref)) - record.EncodeLabels(&buf, s.lset) + record.EncodeLabels(&buf, s.labels()) buf.PutBE64int64(0) // Backwards-compatibility; was chunkRange but now unused. s.Lock() @@ -1485,7 +1485,7 @@ Outer: continue } - if err := h.exemplars.AddExemplar(ms.lset, exemplar.Exemplar{ + if err := h.exemplars.AddExemplar(ms.labels(), exemplar.Exemplar{ Labels: e.Labels, Value: e.V, Ts: e.T, diff --git a/vendor/github.com/prometheus/prometheus/tsdb/index/postings.go b/vendor/github.com/prometheus/prometheus/tsdb/index/postings.go index d9b5b69de0..bfe74c323d 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/index/postings.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/index/postings.go @@ -755,9 +755,7 @@ func (it *ListPostings) Seek(x storage.SeriesRef) bool { } // Do binary search between current position and end. - i := sort.Search(len(it.list), func(i int) bool { - return it.list[i] >= x - }) + i, _ := slices.BinarySearch(it.list, x) if i < len(it.list) { it.cur = it.list[i] it.list = it.list[i+1:] diff --git a/vendor/github.com/prometheus/prometheus/tsdb/ooo_head.go b/vendor/github.com/prometheus/prometheus/tsdb/ooo_head.go index 7f2110fa65..b2556d62e9 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/ooo_head.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/ooo_head.go @@ -17,9 +17,10 @@ import ( "fmt" "sort" + "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/oklog/ulid" - "github.com/prometheus/prometheus/tsdb/chunkenc" "github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/tombstones" ) @@ -74,24 +75,22 @@ func (o *OOOChunk) NumSamples() int { return len(o.samples) } -func (o *OOOChunk) ToXOR() (*chunkenc.XORChunk, error) { - x := chunkenc.NewXORChunk() - app, err := x.Appender() - if err != nil { - return nil, err - } - for _, s := range o.samples { - app.Append(s.t, s.f) - } - return x, nil -} - -func (o *OOOChunk) ToXORBetweenTimestamps(mint, maxt int64) (*chunkenc.XORChunk, error) { - x := chunkenc.NewXORChunk() - app, err := x.Appender() - if err != nil { - return nil, err +// ToEncodedChunks returns chunks with the samples in the OOOChunk. +// +//nolint:revive // unexported-return. +func (o *OOOChunk) ToEncodedChunks(mint, maxt int64) (chks []memChunk, err error) { + if len(o.samples) == 0 { + return nil, nil } + // The most common case is that there will be a single chunk, with the same type of samples in it - this is always true for float samples. + chks = make([]memChunk, 0, 1) + var ( + cmint int64 + cmaxt int64 + chunk chunkenc.Chunk + app chunkenc.Appender + ) + prevEncoding := chunkenc.EncNone // Yes we could call the chunk for this, but this is more efficient. for _, s := range o.samples { if s.t < mint { continue @@ -99,9 +98,77 @@ func (o *OOOChunk) ToXORBetweenTimestamps(mint, maxt int64) (*chunkenc.XORChunk, if s.t > maxt { break } - app.Append(s.t, s.f) + encoding := chunkenc.EncXOR + if s.h != nil { + encoding = chunkenc.EncHistogram + } else if s.fh != nil { + encoding = chunkenc.EncFloatHistogram + } + + // prevApp is the appender for the previous sample. + prevApp := app + + if encoding != prevEncoding { // For the first sample, this will always be true as EncNone != EncXOR | EncHistogram | EncFloatHistogram + if prevEncoding != chunkenc.EncNone { + chks = append(chks, memChunk{chunk, cmint, cmaxt, nil}) + } + cmint = s.t + switch encoding { + case chunkenc.EncXOR: + chunk = chunkenc.NewXORChunk() + case chunkenc.EncHistogram: + chunk = chunkenc.NewHistogramChunk() + case chunkenc.EncFloatHistogram: + chunk = chunkenc.NewFloatHistogramChunk() + default: + chunk = chunkenc.NewXORChunk() + } + app, err = chunk.Appender() + if err != nil { + return + } + } + switch encoding { + case chunkenc.EncXOR: + app.Append(s.t, s.f) + case chunkenc.EncHistogram: + // Ignoring ok is ok, since we don't want to compare to the wrong previous appender anyway. + prevHApp, _ := prevApp.(*chunkenc.HistogramAppender) + var ( + newChunk chunkenc.Chunk + recoded bool + ) + newChunk, recoded, app, _ = app.AppendHistogram(prevHApp, s.t, s.h, false) + if newChunk != nil { // A new chunk was allocated. + if !recoded { + chks = append(chks, memChunk{chunk, cmint, cmaxt, nil}) + } + chunk = newChunk + cmint = s.t + } + case chunkenc.EncFloatHistogram: + // Ignoring ok is ok, since we don't want to compare to the wrong previous appender anyway. + prevHApp, _ := prevApp.(*chunkenc.FloatHistogramAppender) + var ( + newChunk chunkenc.Chunk + recoded bool + ) + newChunk, recoded, app, _ = app.AppendFloatHistogram(prevHApp, s.t, s.fh, false) + if newChunk != nil { // A new chunk was allocated. + if !recoded { + chks = append(chks, memChunk{chunk, cmint, cmaxt, nil}) + } + chunk = newChunk + cmint = s.t + } + } + cmaxt = s.t + prevEncoding = encoding + } + if prevEncoding != chunkenc.EncNone { + chks = append(chks, memChunk{chunk, cmint, cmaxt, nil}) } - return x, nil + return chks, nil } var _ BlockReader = &OOORangeHead{} diff --git a/vendor/github.com/prometheus/prometheus/tsdb/ooo_head_read.go b/vendor/github.com/prometheus/prometheus/tsdb/ooo_head_read.go index 3b5adf80c9..a35276af50 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/ooo_head_read.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/ooo_head_read.go @@ -78,7 +78,7 @@ func (oh *OOOHeadIndexReader) series(ref storage.SeriesRef, builder *labels.Scra oh.head.metrics.seriesNotFound.Inc() return storage.ErrNotFound } - builder.Assign(s.lset) + builder.Assign(s.labels()) if chks == nil { return nil @@ -94,48 +94,40 @@ func (oh *OOOHeadIndexReader) series(ref storage.SeriesRef, builder *labels.Scra tmpChks := make([]chunks.Meta, 0, len(s.ooo.oooMmappedChunks)) - // We define these markers to track the last chunk reference while we - // fill the chunk meta. - // These markers are useful to give consistent responses to repeated queries - // even if new chunks that might be overlapping or not are added afterwards. - // Also, lastMinT and lastMaxT are initialized to the max int as a sentinel - // value to know they are unset. - var lastChunkRef chunks.ChunkRef - lastMinT, lastMaxT := int64(math.MaxInt64), int64(math.MaxInt64) - - addChunk := func(minT, maxT int64, ref chunks.ChunkRef) { - // the first time we get called is for the last included chunk. - // set the markers accordingly - if lastMinT == int64(math.MaxInt64) { - lastChunkRef = ref - lastMinT = minT - lastMaxT = maxT - } - + addChunk := func(minT, maxT int64, ref chunks.ChunkRef, chunk chunkenc.Chunk) { tmpChks = append(tmpChks, chunks.Meta{ - MinTime: minT, - MaxTime: maxT, - Ref: ref, - OOOLastRef: lastChunkRef, - OOOLastMinTime: lastMinT, - OOOLastMaxTime: lastMaxT, + MinTime: minT, + MaxTime: maxT, + Ref: ref, + Chunk: chunk, }) } - // Collect all chunks that overlap the query range, in order from most recent to most old, - // so we can set the correct markers. + // Collect all chunks that overlap the query range. if s.ooo.oooHeadChunk != nil { c := s.ooo.oooHeadChunk if c.OverlapsClosedInterval(oh.mint, oh.maxt) && maxMmapRef == 0 { ref := chunks.ChunkRef(chunks.NewHeadChunkRef(s.ref, s.oooHeadChunkID(len(s.ooo.oooMmappedChunks)))) - addChunk(c.minTime, c.maxTime, ref) + if len(c.chunk.samples) > 0 { // Empty samples happens in tests, at least. + chks, err := s.ooo.oooHeadChunk.chunk.ToEncodedChunks(c.minTime, c.maxTime) + if err != nil { + handleChunkWriteError(err) + return nil + } + for _, chk := range chks { + addChunk(c.minTime, c.maxTime, ref, chk.chunk) + } + } else { + var emptyChunk chunkenc.Chunk + addChunk(c.minTime, c.maxTime, ref, emptyChunk) + } } } for i := len(s.ooo.oooMmappedChunks) - 1; i >= 0; i-- { c := s.ooo.oooMmappedChunks[i] if c.OverlapsClosedInterval(oh.mint, oh.maxt) && (maxMmapRef == 0 || maxMmapRef.GreaterThanOrEqualTo(c.ref)) && (lastGarbageCollectedMmapRef == 0 || c.ref.GreaterThan(lastGarbageCollectedMmapRef)) { ref := chunks.ChunkRef(chunks.NewHeadChunkRef(s.ref, s.oooHeadChunkID(i))) - addChunk(c.minTime, c.maxTime, ref) + addChunk(c.minTime, c.maxTime, ref, nil) } } @@ -163,6 +155,12 @@ func (oh *OOOHeadIndexReader) series(ref storage.SeriesRef, builder *labels.Scra case c.MaxTime > maxTime: maxTime = c.MaxTime (*chks)[len(*chks)-1].MaxTime = c.MaxTime + fallthrough + default: + // If the head OOO chunk is part of an output chunk, copy the chunk pointer. + if c.Chunk != nil { + (*chks)[len(*chks)-1].Chunk = c.Chunk + } } } @@ -185,10 +183,8 @@ func (oh *OOOHeadIndexReader) LabelValues(ctx context.Context, name string, matc } type chunkMetaAndChunkDiskMapperRef struct { - meta chunks.Meta - ref chunks.ChunkDiskMapperRef - origMinT int64 - origMaxT int64 + meta chunks.Meta + ref chunks.ChunkDiskMapperRef } func refLessByMinTimeAndMinRef(a, b chunkMetaAndChunkDiskMapperRef) int { @@ -353,14 +349,20 @@ func NewOOOCompactionHead(ctx context.Context, head *Head) (*OOOCompactionHead, continue } - mmapRef := ms.mmapCurrentOOOHeadChunk(head.chunkDiskMapper) - if mmapRef == 0 && len(ms.ooo.oooMmappedChunks) > 0 { + var lastMmapRef chunks.ChunkDiskMapperRef + mmapRefs := ms.mmapCurrentOOOHeadChunk(head.chunkDiskMapper) + if len(mmapRefs) == 0 && len(ms.ooo.oooMmappedChunks) > 0 { // Nothing was m-mapped. So take the mmapRef from the existing slice if it exists. - mmapRef = ms.ooo.oooMmappedChunks[len(ms.ooo.oooMmappedChunks)-1].ref + mmapRefs = []chunks.ChunkDiskMapperRef{ms.ooo.oooMmappedChunks[len(ms.ooo.oooMmappedChunks)-1].ref} + } + if len(mmapRefs) == 0 { + lastMmapRef = 0 + } else { + lastMmapRef = mmapRefs[len(mmapRefs)-1] } - seq, off := mmapRef.Unpack() + seq, off := lastMmapRef.Unpack() if seq > lastSeq || (seq == lastSeq && off > lastOff) { - ch.lastMmapRef, lastSeq, lastOff = mmapRef, seq, off + ch.lastMmapRef, lastSeq, lastOff = lastMmapRef, seq, off } if len(ms.ooo.oooMmappedChunks) > 0 { ch.postings = append(ch.postings, seriesRef) diff --git a/vendor/github.com/prometheus/prometheus/tsdb/querier.go b/vendor/github.com/prometheus/prometheus/tsdb/querier.go index fb4a87cc8c..910c2d7fc1 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/querier.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/querier.go @@ -77,12 +77,12 @@ func newBlockBaseQuerier(b BlockReader, mint, maxt int64) (*blockBaseQuerier, er }, nil } -func (q *blockBaseQuerier) LabelValues(ctx context.Context, name string, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { +func (q *blockBaseQuerier) LabelValues(ctx context.Context, name string, hints *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { res, err := q.index.SortedLabelValues(ctx, name, matchers...) return res, nil, err } -func (q *blockBaseQuerier) LabelNames(ctx context.Context, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { +func (q *blockBaseQuerier) LabelNames(ctx context.Context, hints *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { res, err := q.index.LabelNames(ctx, matchers...) return res, nil, err } diff --git a/vendor/github.com/prometheus/prometheus/tsdb/record/record.go b/vendor/github.com/prometheus/prometheus/tsdb/record/record.go index c95b25f06e..784d0b23d7 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/record/record.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/record/record.go @@ -543,7 +543,7 @@ func (d *Decoder) FloatHistogramSamples(rec []byte, histograms []RefFloatHistogr return histograms, nil } -// Decode decodes a Histogram from a byte slice. +// DecodeFloatHistogram decodes a Histogram from a byte slice. func DecodeFloatHistogram(buf *encoding.Decbuf, fh *histogram.FloatHistogram) { fh.CounterResetHint = histogram.CounterResetHint(buf.Byte()) diff --git a/vendor/github.com/prometheus/prometheus/tsdb/testutil.go b/vendor/github.com/prometheus/prometheus/tsdb/testutil.go new file mode 100644 index 0000000000..9730e47132 --- /dev/null +++ b/vendor/github.com/prometheus/prometheus/tsdb/testutil.go @@ -0,0 +1,176 @@ +// Copyright 2017 The Prometheus Authors +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package tsdb + +import ( + "testing" + + prom_testutil "github.com/prometheus/client_golang/prometheus/testutil" + + "github.com/stretchr/testify/require" + + "github.com/prometheus/prometheus/model/histogram" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/chunks" +) + +const ( + float = "float" +) + +type testValue struct { + Ts int64 + V int64 + CounterResetHeader histogram.CounterResetHint +} + +type sampleTypeScenario struct { + sampleType string + appendFunc func(appender storage.Appender, lbls labels.Labels, ts, value int64) (storage.SeriesRef, sample, error) + sampleFunc func(ts, value int64) sample +} + +// TODO: native histogram sample types will be added as part of out-of-order native histogram support; see #11220. +var sampleTypeScenarios = map[string]sampleTypeScenario{ + float: { + sampleType: sampleMetricTypeFloat, + appendFunc: func(appender storage.Appender, lbls labels.Labels, ts, value int64) (storage.SeriesRef, sample, error) { + s := sample{t: ts, f: float64(value)} + ref, err := appender.Append(0, lbls, ts, s.f) + return ref, s, err + }, + sampleFunc: func(ts, value int64) sample { + return sample{t: ts, f: float64(value)} + }, + }, + // intHistogram: { + // sampleType: sampleMetricTypeHistogram, + // appendFunc: func(appender storage.Appender, lbls labels.Labels, ts, value int64) (storage.SeriesRef, sample, error) { + // s := sample{t: ts, h: tsdbutil.GenerateTestHistogram(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.GenerateTestHistogram(int(value))} + // }, + // }, + // floatHistogram: { + // sampleType: sampleMetricTypeHistogram, + // appendFunc: func(appender storage.Appender, lbls labels.Labels, ts, value int64) (storage.SeriesRef, sample, error) { + // s := sample{t: ts, fh: tsdbutil.GenerateTestFloatHistogram(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.GenerateTestFloatHistogram(int(value))} + // }, + // }, + // gaugeIntHistogram: { + // sampleType: sampleMetricTypeHistogram, + // appendFunc: func(appender storage.Appender, lbls labels.Labels, ts, value int64) (storage.SeriesRef, sample, error) { + // s := sample{t: ts, h: tsdbutil.GenerateTestGaugeHistogram(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.GenerateTestGaugeHistogram(int(value))} + // }, + // }, + // gaugeFloatHistogram: { + // sampleType: sampleMetricTypeHistogram, + // appendFunc: func(appender storage.Appender, lbls labels.Labels, ts, value int64) (storage.SeriesRef, sample, error) { + // s := sample{t: ts, fh: tsdbutil.GenerateTestGaugeFloatHistogram(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.GenerateTestGaugeFloatHistogram(int(value))} + // }, + // }, +} + +// requireEqualSeries checks that the actual series are equal to the expected ones. It ignores the counter reset hints for histograms. +func requireEqualSeries(t *testing.T, expected, actual map[string][]chunks.Sample, ignoreCounterResets bool) { + for name, expectedItem := range expected { + actualItem, ok := actual[name] + require.True(t, ok, "Expected series %s not found", name) + requireEqualSamples(t, name, expectedItem, actualItem, ignoreCounterResets) + } + for name := range actual { + _, ok := expected[name] + require.True(t, ok, "Unexpected series %s", name) + } +} + +func requireEqualOOOSamples(t *testing.T, expectedSamples int, db *DB) { + require.Equal(t, float64(expectedSamples), + prom_testutil.ToFloat64(db.head.metrics.outOfOrderSamplesAppended.WithLabelValues(sampleMetricTypeFloat))+ + prom_testutil.ToFloat64(db.head.metrics.outOfOrderSamplesAppended.WithLabelValues(sampleMetricTypeHistogram)), + "number of ooo appended samples mismatch") +} + +func requireEqualSamples(t *testing.T, name string, expected, actual []chunks.Sample, ignoreCounterResets bool) { + require.Equal(t, len(expected), len(actual), "Length not equal to expected for %s", name) + for i, s := range expected { + expectedSample := s + actualSample := actual[i] + require.Equal(t, expectedSample.T(), actualSample.T(), "Different timestamps for %s[%d]", name, i) + require.Equal(t, expectedSample.Type().String(), actualSample.Type().String(), "Different types for %s[%d] at ts %d", name, i, expectedSample.T()) + switch { + case s.H() != nil: + { + expectedHist := expectedSample.H() + actualHist := actualSample.H() + if ignoreCounterResets && expectedHist.CounterResetHint != histogram.GaugeType { + expectedHist.CounterResetHint = histogram.UnknownCounterReset + actualHist.CounterResetHint = histogram.UnknownCounterReset + } else { + require.Equal(t, expectedHist.CounterResetHint, actualHist.CounterResetHint, "Sample header doesn't match for %s[%d] at ts %d, expected: %s, actual: %s", name, i, expectedSample.T(), counterResetAsString(expectedHist.CounterResetHint), counterResetAsString(actualHist.CounterResetHint)) + } + require.Equal(t, expectedHist, actualHist, "Sample doesn't match for %s[%d] at ts %d", name, i, expectedSample.T()) + } + case s.FH() != nil: + { + expectedHist := expectedSample.FH() + actualHist := actualSample.FH() + if ignoreCounterResets { + expectedHist.CounterResetHint = histogram.UnknownCounterReset + actualHist.CounterResetHint = histogram.UnknownCounterReset + } else { + require.Equal(t, expectedHist.CounterResetHint, actualHist.CounterResetHint, "Sample header doesn't match for %s[%d] at ts %d, expected: %s, actual: %s", name, i, expectedSample.T(), counterResetAsString(expectedHist.CounterResetHint), counterResetAsString(actualHist.CounterResetHint)) + } + require.Equal(t, expectedHist, actualHist, "Sample doesn't match for %s[%d] at ts %d", name, i, expectedSample.T()) + } + default: + expectedFloat := expectedSample.F() + actualFloat := actualSample.F() + require.Equal(t, expectedFloat, actualFloat, "Sample doesn't match for %s[%d] at ts %d", name, i, expectedSample.T()) + } + } +} + +func counterResetAsString(h histogram.CounterResetHint) string { + switch h { + case histogram.UnknownCounterReset: + return "UnknownCounterReset" + case histogram.CounterReset: + return "CounterReset" + case histogram.NotCounterReset: + return "NotCounterReset" + case histogram.GaugeType: + return "GaugeType" + } + panic("Unexpected counter reset type") +} diff --git a/vendor/github.com/prometheus/prometheus/tsdb/tsdbutil/histogram.go b/vendor/github.com/prometheus/prometheus/tsdb/tsdbutil/histogram.go index bb8d49b202..3c7349cf72 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/tsdbutil/histogram.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/tsdbutil/histogram.go @@ -59,6 +59,20 @@ func GenerateTestHistogram(i int) *histogram.Histogram { } } +func GenerateTestCustomBucketsHistogram(i int) *histogram.Histogram { + return &histogram.Histogram{ + Count: 5 + uint64(i*4), + Sum: 18.4 * float64(i+1), + Schema: histogram.CustomBucketsSchema, + PositiveSpans: []histogram.Span{ + {Offset: 0, Length: 2}, + {Offset: 1, Length: 2}, + }, + PositiveBuckets: []int64{int64(i + 1), 1, -1, 0}, + CustomValues: []float64{0, 1, 2, 3, 4}, + } +} + func GenerateTestGaugeHistograms(n int) (r []*histogram.Histogram) { for x := 0; x < n; x++ { i := int(math.Sin(float64(x))*100) + 100 @@ -105,6 +119,20 @@ func GenerateTestFloatHistogram(i int) *histogram.FloatHistogram { } } +func GenerateTestCustomBucketsFloatHistogram(i int) *histogram.FloatHistogram { + return &histogram.FloatHistogram{ + Count: 5 + float64(i*4), + Sum: 18.4 * float64(i+1), + Schema: histogram.CustomBucketsSchema, + PositiveSpans: []histogram.Span{ + {Offset: 0, Length: 2}, + {Offset: 1, Length: 2}, + }, + PositiveBuckets: []float64{float64(i + 1), float64(i + 2), float64(i + 1), float64(i + 1)}, + CustomValues: []float64{0, 1, 2, 3, 4}, + } +} + func GenerateTestGaugeFloatHistograms(n int) (r []*histogram.FloatHistogram) { for x := 0; x < n; x++ { i := int(math.Sin(float64(x))*100) + 100 diff --git a/vendor/github.com/prometheus/prometheus/tsdb/wlog/watcher.go b/vendor/github.com/prometheus/prometheus/tsdb/wlog/watcher.go index 8ebd9249aa..bc7a144e66 100644 --- a/vendor/github.com/prometheus/prometheus/tsdb/wlog/watcher.go +++ b/vendor/github.com/prometheus/prometheus/tsdb/wlog/watcher.go @@ -57,6 +57,7 @@ type WriteTo interface { AppendHistograms([]record.RefHistogramSample) bool AppendFloatHistograms([]record.RefFloatHistogramSample) bool StoreSeries([]record.RefSeries, int) + StoreMetadata([]record.RefMetadata) // Next two methods are intended for garbage-collection: first we call // UpdateSeriesSegment on all current series @@ -88,6 +89,7 @@ type Watcher struct { lastCheckpoint string sendExemplars bool sendHistograms bool + sendMetadata bool metrics *WatcherMetrics readerMetrics *LiveReaderMetrics @@ -170,7 +172,7 @@ func NewWatcherMetrics(reg prometheus.Registerer) *WatcherMetrics { } // NewWatcher creates a new WAL watcher for a given WriteTo. -func NewWatcher(metrics *WatcherMetrics, readerMetrics *LiveReaderMetrics, logger log.Logger, name string, writer WriteTo, dir string, sendExemplars, sendHistograms bool) *Watcher { +func NewWatcher(metrics *WatcherMetrics, readerMetrics *LiveReaderMetrics, logger log.Logger, name string, writer WriteTo, dir string, sendExemplars, sendHistograms, sendMetadata bool) *Watcher { if logger == nil { logger = log.NewNopLogger() } @@ -183,6 +185,7 @@ func NewWatcher(metrics *WatcherMetrics, readerMetrics *LiveReaderMetrics, logge name: name, sendExemplars: sendExemplars, sendHistograms: sendHistograms, + sendMetadata: sendMetadata, readNotify: make(chan struct{}), quit: make(chan struct{}), @@ -262,6 +265,11 @@ func (w *Watcher) loop() { // Run the watcher, which will tail the WAL until the quit channel is closed // or an error case is hit. func (w *Watcher) Run() error { + _, lastSegment, err := w.firstAndLast() + if err != nil { + return fmt.Errorf("wal.Segments: %w", err) + } + // We want to ensure this is false across iterations since // Run will be called again if there was a failure to read the WAL. w.sendSamples = false @@ -286,20 +294,14 @@ func (w *Watcher) Run() error { return err } - level.Debug(w.logger).Log("msg", "Tailing WAL", "lastCheckpoint", lastCheckpoint, "checkpointIndex", checkpointIndex, "currentSegment", currentSegment) + level.Debug(w.logger).Log("msg", "Tailing WAL", "lastCheckpoint", lastCheckpoint, "checkpointIndex", checkpointIndex, "currentSegment", currentSegment, "lastSegment", lastSegment) for !isClosed(w.quit) { w.currentSegmentMetric.Set(float64(currentSegment)) - // Re-check on each iteration in case a new segment was added, - // because watch() will wait for notifications on the last segment. - _, lastSegment, err := w.firstAndLast() - if err != nil { - return fmt.Errorf("wal.Segments: %w", err) - } - tail := currentSegment >= lastSegment - - level.Debug(w.logger).Log("msg", "Processing segment", "currentSegment", currentSegment, "lastSegment", lastSegment) - if err := w.watch(currentSegment, tail); err != nil && !errors.Is(err, ErrIgnorable) { + // On start, after reading the existing WAL for series records, we have a pointer to what is the latest segment. + // On subsequent calls to this function, currentSegment will have been incremented and we should open that segment. + level.Debug(w.logger).Log("msg", "Processing segment", "currentSegment", currentSegment) + if err := w.watch(currentSegment, currentSegment >= lastSegment); err != nil && !errors.Is(err, ErrIgnorable) { return err } @@ -541,6 +543,7 @@ func (w *Watcher) readSegment(r *LiveReader, segmentNum int, tail bool) error { histogramsToSend []record.RefHistogramSample floatHistograms []record.RefFloatHistogramSample floatHistogramsToSend []record.RefFloatHistogramSample + metadata []record.RefMetadata ) for r.Next() && !isClosed(w.quit) { rec := r.Record() @@ -652,6 +655,17 @@ func (w *Watcher) readSegment(r *LiveReader, segmentNum int, tail bool) error { w.writer.AppendFloatHistograms(floatHistogramsToSend) floatHistogramsToSend = floatHistogramsToSend[:0] } + + case record.Metadata: + if !w.sendMetadata || !tail { + break + } + meta, err := dec.Metadata(rec, metadata[:0]) + if err != nil { + w.recordDecodeFailsMetric.Inc() + return err + } + w.writer.StoreMetadata(meta) case record.Tombstones: default: diff --git a/vendor/github.com/prometheus/prometheus/util/annotations/annotations.go b/vendor/github.com/prometheus/prometheus/util/annotations/annotations.go index 16a920d57f..bc5d76db43 100644 --- a/vendor/github.com/prometheus/prometheus/util/annotations/annotations.go +++ b/vendor/github.com/prometheus/prometheus/util/annotations/annotations.go @@ -71,27 +71,58 @@ func (a Annotations) AsErrors() []error { return arr } -// AsStrings is a convenience function to return the annotations map as a slice -// of strings. The query string is used to get the line number and character offset -// positioning info of the elements which trigger an annotation. We limit the number -// of annotations returned here with maxAnnos (0 for no limit). -func (a Annotations) AsStrings(query string, maxAnnos int) []string { - arr := make([]string, 0, len(a)) +// AsStrings is a convenience function to return the annotations map as 2 slices +// of strings, separated into warnings and infos. The query string is used to get the +// line number and character offset positioning info of the elements which trigger an +// annotation. We limit the number of warnings and infos returned here with maxWarnings +// and maxInfos respectively (0 for no limit). +func (a Annotations) AsStrings(query string, maxWarnings, maxInfos int) (warnings, infos []string) { + warnings = make([]string, 0, maxWarnings+1) + infos = make([]string, 0, maxInfos+1) + warnSkipped := 0 + infoSkipped := 0 for _, err := range a { - if maxAnnos > 0 && len(arr) >= maxAnnos { - break - } var anErr annoErr if errors.As(err, &anErr) { anErr.Query = query err = anErr } - arr = append(arr, err.Error()) + switch { + case errors.Is(err, PromQLInfo): + if maxInfos == 0 || len(infos) < maxInfos { + infos = append(infos, err.Error()) + } else { + infoSkipped++ + } + default: + if maxWarnings == 0 || len(warnings) < maxWarnings { + warnings = append(warnings, err.Error()) + } else { + warnSkipped++ + } + } } - if maxAnnos > 0 && len(a) > maxAnnos { - arr = append(arr, fmt.Sprintf("%d more annotations omitted", len(a)-maxAnnos)) + if warnSkipped > 0 { + warnings = append(warnings, fmt.Sprintf("%d more warning annotations omitted", warnSkipped)) } - return arr + if infoSkipped > 0 { + infos = append(infos, fmt.Sprintf("%d more info annotations omitted", infoSkipped)) + } + return +} + +// CountWarningsAndInfo counts and returns the number of warnings and infos in the +// annotations wrapper. +func (a Annotations) CountWarningsAndInfo() (countWarnings, countInfo int) { + for _, err := range a { + if errors.Is(err, PromQLWarning) { + countWarnings++ + } + if errors.Is(err, PromQLInfo) { + countInfo++ + } + } + return } //nolint:revive // error-naming. @@ -103,12 +134,15 @@ var ( PromQLInfo = errors.New("PromQL info") PromQLWarning = errors.New("PromQL warning") - InvalidQuantileWarning = fmt.Errorf("%w: quantile value should be between 0 and 1", PromQLWarning) - BadBucketLabelWarning = fmt.Errorf("%w: bucket label %q is missing or has a malformed value", PromQLWarning, model.BucketLabel) - MixedFloatsHistogramsWarning = fmt.Errorf("%w: encountered a mix of histograms and floats for", PromQLWarning) - MixedClassicNativeHistogramsWarning = fmt.Errorf("%w: vector contains a mix of classic and native histograms for metric name", PromQLWarning) - NativeHistogramNotCounterWarning = fmt.Errorf("%w: this native histogram metric is not a counter:", PromQLWarning) - NativeHistogramNotGaugeWarning = fmt.Errorf("%w: this native histogram metric is not a gauge:", PromQLWarning) + InvalidRatioWarning = fmt.Errorf("%w: ratio value should be between -1 and 1", PromQLWarning) + InvalidQuantileWarning = fmt.Errorf("%w: quantile value should be between 0 and 1", PromQLWarning) + BadBucketLabelWarning = fmt.Errorf("%w: bucket label %q is missing or has a malformed value", PromQLWarning, model.BucketLabel) + MixedFloatsHistogramsWarning = fmt.Errorf("%w: encountered a mix of histograms and floats for", PromQLWarning) + MixedClassicNativeHistogramsWarning = fmt.Errorf("%w: vector contains a mix of classic and native histograms for metric name", PromQLWarning) + NativeHistogramNotCounterWarning = fmt.Errorf("%w: this native histogram metric is not a counter:", PromQLWarning) + NativeHistogramNotGaugeWarning = fmt.Errorf("%w: this native histogram metric is not a gauge:", PromQLWarning) + MixedExponentialCustomHistogramsWarning = fmt.Errorf("%w: vector contains a mix of histograms with exponential and custom buckets schemas for metric name", PromQLWarning) + IncompatibleCustomBucketsHistogramsWarning = fmt.Errorf("%w: vector contains histograms with incompatible custom buckets for metric name", PromQLWarning) PossibleNonCounterInfo = fmt.Errorf("%w: metric might not be a counter, name does not end in _total/_sum/_count/_bucket:", PromQLInfo) HistogramQuantileForcedMonotonicityInfo = fmt.Errorf("%w: input to histogram_quantile needed to be fixed for monotonicity (see https://prometheus.io/docs/prometheus/latest/querying/functions/#histogram_quantile) for metric name", PromQLInfo) @@ -140,6 +174,15 @@ func NewInvalidQuantileWarning(q float64, pos posrange.PositionRange) error { } } +// NewInvalidQuantileWarning is used when the user specifies an invalid ratio +// value, i.e. a float that is outside the range [-1, 1] or NaN. +func NewInvalidRatioWarning(q, to float64, pos posrange.PositionRange) error { + return annoErr{ + PositionRange: pos, + Err: fmt.Errorf("%w, got %g, capping to %g", InvalidRatioWarning, q, to), + } +} + // NewBadBucketLabelWarning is used when there is an error parsing the bucket label // of a classic histogram. func NewBadBucketLabelWarning(metricName, label string, pos posrange.PositionRange) error { @@ -195,6 +238,24 @@ func NewNativeHistogramNotGaugeWarning(metricName string, pos posrange.PositionR } } +// NewMixedExponentialCustomHistogramsWarning is used when the queried series includes +// histograms with both exponential and custom buckets schemas. +func NewMixedExponentialCustomHistogramsWarning(metricName string, pos posrange.PositionRange) error { + return annoErr{ + PositionRange: pos, + Err: fmt.Errorf("%w %q", MixedExponentialCustomHistogramsWarning, metricName), + } +} + +// NewIncompatibleCustomBucketsHistogramsWarning is used when the queried series includes +// custom buckets histograms with incompatible custom bounds. +func NewIncompatibleCustomBucketsHistogramsWarning(metricName string, pos posrange.PositionRange) error { + return annoErr{ + PositionRange: pos, + Err: fmt.Errorf("%w %q", IncompatibleCustomBucketsHistogramsWarning, metricName), + } +} + // NewPossibleNonCounterInfo is used when a named counter metric with only float samples does not // have the suffixes _total, _sum, _count, or _bucket. func NewPossibleNonCounterInfo(metricName string, pos posrange.PositionRange) error { diff --git a/vendor/github.com/prometheus/prometheus/web/api/v1/api.go b/vendor/github.com/prometheus/prometheus/web/api/v1/api.go index b95ff25cf9..03854787f8 100644 --- a/vendor/github.com/prometheus/prometheus/web/api/v1/api.go +++ b/vendor/github.com/prometheus/prometheus/web/api/v1/api.go @@ -159,6 +159,7 @@ type Response struct { ErrorType errorType `json:"errorType,omitempty"` Error string `json:"error,omitempty"` Warnings []string `json:"warnings,omitempty"` + Infos []string `json:"infos,omitempty"` } type apiFuncResult struct { @@ -248,6 +249,7 @@ func NewAPI( registerer prometheus.Registerer, statsRenderer StatsRenderer, rwEnabled bool, + acceptRemoteWriteProtoMsgs []config.RemoteWriteProtoMsg, otlpEnabled bool, ) *API { a := &API{ @@ -290,7 +292,7 @@ func NewAPI( } if rwEnabled { - a.remoteWriteHandler = remote.NewWriteHandler(logger, registerer, ap) + a.remoteWriteHandler = remote.NewWriteHandler(logger, registerer, ap, acceptRemoteWriteProtoMsgs) } if otlpEnabled { a.otlpWriteHandler = remote.NewOTLPWriteHandler(logger, ap) @@ -658,6 +660,10 @@ func (api *API) labelNames(r *http.Request) apiFuncResult { return apiFuncResult{nil, &apiError{errorBadData, err}, nil, nil} } + hints := &storage.LabelHints{ + Limit: toHintLimit(limit), + } + q, err := api.Queryable.Querier(timestamp.FromTime(start), timestamp.FromTime(end)) if err != nil { return apiFuncResult{nil, returnAPIError(err), nil, nil} @@ -672,7 +678,7 @@ func (api *API) labelNames(r *http.Request) apiFuncResult { labelNamesSet := make(map[string]struct{}) for _, matchers := range matcherSets { - vals, callWarnings, err := q.LabelNames(r.Context(), matchers...) + vals, callWarnings, err := q.LabelNames(r.Context(), hints, matchers...) if err != nil { return apiFuncResult{nil, returnAPIError(err), warnings, nil} } @@ -694,7 +700,7 @@ func (api *API) labelNames(r *http.Request) apiFuncResult { if len(matcherSets) == 1 { matchers = matcherSets[0] } - names, warnings, err = q.LabelNames(r.Context(), matchers...) + names, warnings, err = q.LabelNames(r.Context(), hints, matchers...) if err != nil { return apiFuncResult{nil, &apiError{errorExec, err}, warnings, nil} } @@ -704,7 +710,7 @@ func (api *API) labelNames(r *http.Request) apiFuncResult { names = []string{} } - if len(names) > limit { + if limit > 0 && len(names) > limit { names = names[:limit] warnings = warnings.Add(errors.New("results truncated due to limit")) } @@ -738,6 +744,10 @@ func (api *API) labelValues(r *http.Request) (result apiFuncResult) { return apiFuncResult{nil, &apiError{errorBadData, err}, nil, nil} } + hints := &storage.LabelHints{ + Limit: toHintLimit(limit), + } + q, err := api.Queryable.Querier(timestamp.FromTime(start), timestamp.FromTime(end)) if err != nil { return apiFuncResult{nil, &apiError{errorExec, err}, nil, nil} @@ -762,7 +772,7 @@ func (api *API) labelValues(r *http.Request) (result apiFuncResult) { var callWarnings annotations.Annotations labelValuesSet := make(map[string]struct{}) for _, matchers := range matcherSets { - vals, callWarnings, err = q.LabelValues(ctx, name, matchers...) + vals, callWarnings, err = q.LabelValues(ctx, name, hints, matchers...) if err != nil { return apiFuncResult{nil, &apiError{errorExec, err}, warnings, closer} } @@ -781,7 +791,7 @@ func (api *API) labelValues(r *http.Request) (result apiFuncResult) { if len(matcherSets) == 1 { matchers = matcherSets[0] } - vals, warnings, err = q.LabelValues(ctx, name, matchers...) + vals, warnings, err = q.LabelValues(ctx, name, hints, matchers...) if err != nil { return apiFuncResult{nil, &apiError{errorExec, err}, warnings, closer} } @@ -793,7 +803,7 @@ func (api *API) labelValues(r *http.Request) (result apiFuncResult) { slices.Sort(vals) - if len(vals) > limit { + if limit > 0 && len(vals) > limit { vals = vals[:limit] warnings = warnings.Add(errors.New("results truncated due to limit")) } @@ -863,6 +873,7 @@ func (api *API) series(r *http.Request) (result apiFuncResult) { Start: timestamp.FromTime(start), End: timestamp.FromTime(end), Func: "series", // There is no series function, this token is used for lookups that don't need samples. + Limit: toHintLimit(limit), } var set storage.SeriesSet @@ -889,7 +900,7 @@ func (api *API) series(r *http.Request) (result apiFuncResult) { } metrics = append(metrics, set.At().Labels()) - if len(metrics) > limit { + if limit > 0 && len(metrics) > limit { metrics = metrics[:limit] warnings.Add(errors.New("results truncated due to limit")) return apiFuncResult{metrics, nil, warnings, closer} @@ -1395,6 +1406,11 @@ func (api *API) rules(r *http.Request) apiFuncResult { rgSet := queryFormToSet(r.Form["rule_group[]"]) fSet := queryFormToSet(r.Form["file[]"]) + matcherSets, err := parseMatchersParam(r.Form["match[]"]) + if err != nil { + return apiFuncResult{nil, &apiError{errorBadData, err}, nil, nil} + } + ruleGroups := api.rulesRetriever(r.Context()).RuleGroups() res := &RuleDiscovery{RuleGroups: make([]*RuleGroup, 0, len(ruleGroups))} typ := strings.ToLower(r.URL.Query().Get("type")) @@ -1434,7 +1450,8 @@ func (api *API) rules(r *http.Request) apiFuncResult { EvaluationTime: grp.GetEvaluationTime().Seconds(), LastEvaluation: grp.GetLastEvaluation(), } - for _, rr := range grp.Rules() { + + for _, rr := range grp.Rules(matcherSets...) { var enrichedRule Rule if len(rnSet) > 0 { @@ -1746,11 +1763,13 @@ func (api *API) cleanTombstones(*http.Request) apiFuncResult { // can be empty if the position information isn't needed. func (api *API) respond(w http.ResponseWriter, req *http.Request, data interface{}, warnings annotations.Annotations, query string) { statusMessage := statusSuccess + warn, info := warnings.AsStrings(query, 10, 10) resp := &Response{ Status: statusMessage, Data: data, - Warnings: warnings.AsStrings(query, 10), + Warnings: warn, + Infos: info, } codec, err := api.negotiateCodec(req, resp) @@ -1898,8 +1917,8 @@ OUTER: return matcherSets, nil } +// parseLimitParam returning 0 means no limit is to be applied. func parseLimitParam(limitStr string) (limit int, err error) { - limit = math.MaxInt if limitStr == "" { return limit, nil } @@ -1908,9 +1927,19 @@ func parseLimitParam(limitStr string) (limit int, err error) { if err != nil { return limit, err } - if limit <= 0 { - return limit, errors.New("limit must be positive") + if limit < 0 { + return limit, errors.New("limit must be non-negative") } return limit, nil } + +// toHintLimit increases the API limit, as returned by parseLimitParam, by 1. +// This allows for emitting warnings when the results are truncated. +func toHintLimit(limit int) int { + // 0 means no limit and avoid int overflow + if limit > 0 && limit < math.MaxInt { + return limit + 1 + } + return limit +} diff --git a/vendor/github.com/thanos-io/promql-engine/execution/aggregate/accumulator.go b/vendor/github.com/thanos-io/promql-engine/execution/aggregate/accumulator.go index 89802be9a7..5ce536ce49 100644 --- a/vendor/github.com/thanos-io/promql-engine/execution/aggregate/accumulator.go +++ b/vendor/github.com/thanos-io/promql-engine/execution/aggregate/accumulator.go @@ -19,14 +19,14 @@ const ( ) type accumulator interface { - Add(v float64, h *histogram.FloatHistogram) + Add(v float64, h *histogram.FloatHistogram) error Value() (float64, *histogram.FloatHistogram) ValueType() ValueType Reset(float64) } type vectorAccumulator interface { - AddVector(vs []float64, hs []*histogram.FloatHistogram) + AddVector(vs []float64, hs []*histogram.FloatHistogram) error Value() (float64, *histogram.FloatHistogram) ValueType() ValueType Reset(float64) @@ -42,36 +42,44 @@ func newSumAcc() *sumAcc { return &sumAcc{} } -func (s *sumAcc) AddVector(float64s []float64, histograms []*histogram.FloatHistogram) { +func (s *sumAcc) AddVector(float64s []float64, histograms []*histogram.FloatHistogram) error { if len(float64s) > 0 { - s.value += floats.Sum(float64s) + s.value += SumCompensated(float64s) s.hasFloatVal = true } + var err error if len(histograms) > 0 { - s.histSum = histogramSum(s.histSum, histograms) + s.histSum, err = histogramSum(s.histSum, histograms) } + return err } -func (s *sumAcc) Add(v float64, h *histogram.FloatHistogram) { +func (s *sumAcc) Add(v float64, h *histogram.FloatHistogram) error { if h == nil { s.hasFloatVal = true s.value += v - return + return nil } if s.histSum == nil { s.histSum = h.Copy() - return + return nil } // The histogram being added must have an equal or larger schema. // https://github.com/prometheus/prometheus/blob/57bcbf18880f7554ae34c5b341d52fc53f059a97/promql/engine.go#L2448-L2456 + var err error if h.Schema >= s.histSum.Schema { - s.histSum = s.histSum.Add(h) + if s.histSum, err = s.histSum.Add(h); err != nil { + return err + } } else { t := h.Copy() - t.Add(s.histSum) + if _, err = t.Add(s.histSum); err != nil { + return err + } s.histSum = t } + return nil } func (s *sumAcc) Value() (float64, *histogram.FloatHistogram) { @@ -103,27 +111,30 @@ type maxAcc struct { hasValue bool } -func (c *maxAcc) AddVector(vs []float64, hs []*histogram.FloatHistogram) { +func (c *maxAcc) AddVector(vs []float64, hs []*histogram.FloatHistogram) error { if len(vs) == 0 { - return + return nil } fst, rem := vs[0], vs[1:] - c.Add(fst, nil) + if err := c.Add(fst, nil); err != nil { + return err + } if len(rem) == 0 { - return + return nil } - c.Add(floats.Max(rem), nil) + return c.Add(floats.Max(rem), nil) } -func (c *maxAcc) Add(v float64, h *histogram.FloatHistogram) { +func (c *maxAcc) Add(v float64, h *histogram.FloatHistogram) error { if !c.hasValue { c.value = v c.hasValue = true - return + return nil } if c.value < v || math.IsNaN(c.value) { c.value = v } + return nil } func (c *maxAcc) Value() (float64, *histogram.FloatHistogram) { @@ -152,27 +163,30 @@ type minAcc struct { hasValue bool } -func (c *minAcc) AddVector(vs []float64, hs []*histogram.FloatHistogram) { +func (c *minAcc) AddVector(vs []float64, hs []*histogram.FloatHistogram) error { if len(vs) == 0 { - return + return nil } fst, rem := vs[0], vs[1:] - c.Add(fst, nil) + if err := c.Add(fst, nil); err != nil { + return err + } if len(rem) == 0 { - return + return nil } - c.Add(floats.Min(rem), nil) + return c.Add(floats.Min(rem), nil) } -func (c *minAcc) Add(v float64, h *histogram.FloatHistogram) { +func (c *minAcc) Add(v float64, h *histogram.FloatHistogram) error { if !c.hasValue { c.value = v c.hasValue = true - return + return nil } if c.value > v || math.IsNaN(c.value) { c.value = v } + return nil } func (c *minAcc) Value() (float64, *histogram.FloatHistogram) { @@ -201,17 +215,19 @@ type groupAcc struct { hasValue bool } -func (c *groupAcc) AddVector(vs []float64, hs []*histogram.FloatHistogram) { +func (c *groupAcc) AddVector(vs []float64, hs []*histogram.FloatHistogram) error { if len(vs) == 0 && len(hs) == 0 { - return + return nil } c.hasValue = true c.value = 1 + return nil } -func (c *groupAcc) Add(v float64, h *histogram.FloatHistogram) { +func (c *groupAcc) Add(v float64, h *histogram.FloatHistogram) error { c.hasValue = true c.value = 1 + return nil } func (c *groupAcc) Value() (float64, *histogram.FloatHistogram) { @@ -240,16 +256,18 @@ func newCountAcc() *countAcc { return &countAcc{} } -func (c *countAcc) AddVector(vs []float64, hs []*histogram.FloatHistogram) { +func (c *countAcc) AddVector(vs []float64, hs []*histogram.FloatHistogram) error { if len(vs) > 0 || len(hs) > 0 { c.hasValue = true c.value += float64(len(vs)) + float64(len(hs)) } + return nil } -func (c *countAcc) Add(v float64, h *histogram.FloatHistogram) { +func (c *countAcc) Add(v float64, h *histogram.FloatHistogram) error { c.hasValue = true c.value += 1 + return nil } func (c *countAcc) Value() (float64, *histogram.FloatHistogram) { @@ -278,12 +296,12 @@ func newAvgAcc() *avgAcc { return &avgAcc{} } -func (a *avgAcc) Add(v float64, _ *histogram.FloatHistogram) { +func (a *avgAcc) Add(v float64, _ *histogram.FloatHistogram) error { a.count++ if !a.hasValue { a.hasValue = true a.avg = v - return + return nil } a.hasValue = true @@ -292,7 +310,7 @@ func (a *avgAcc) Add(v float64, _ *histogram.FloatHistogram) { // The `avg` and `v` values are `Inf` of the same sign. They // can't be subtracted, but the value of `avg` is correct // already. - return + return nil } if !math.IsInf(v, 0) && !math.IsNaN(v) { // At this stage, the avg is an infinite. If the added @@ -301,17 +319,21 @@ func (a *avgAcc) Add(v float64, _ *histogram.FloatHistogram) { // This is required because our calculation below removes // the avg value, which would look like Inf += x - Inf and // end up as a NaN. - return + return nil } } a.avg += v/float64(a.count) - a.avg/float64(a.count) + return nil } -func (a *avgAcc) AddVector(vs []float64, _ []*histogram.FloatHistogram) { +func (a *avgAcc) AddVector(vs []float64, _ []*histogram.FloatHistogram) error { for _, v := range vs { - a.Add(v, nil) + if err := a.Add(v, nil); err != nil { + return err + } } + return nil } func (a *avgAcc) Value() (float64, *histogram.FloatHistogram) { @@ -337,13 +359,14 @@ type statAcc struct { hasValue bool } -func (s *statAcc) Add(v float64, h *histogram.FloatHistogram) { +func (s *statAcc) Add(v float64, h *histogram.FloatHistogram) error { s.hasValue = true s.count++ delta := v - s.mean s.mean += delta / s.count s.value += delta * (v - s.mean) + return nil } func (s *statAcc) ValueType() ValueType { @@ -400,9 +423,10 @@ func newQuantileAcc() accumulator { return &quantileAcc{} } -func (q *quantileAcc) Add(v float64, h *histogram.FloatHistogram) { +func (q *quantileAcc) Add(v float64, h *histogram.FloatHistogram) error { q.hasValue = true q.points = append(q.points, v) + return nil } func (q *quantileAcc) Value() (float64, *histogram.FloatHistogram) { @@ -422,3 +446,29 @@ func (q *quantileAcc) Reset(f float64) { q.arg = f q.points = q.points[:0] } + +// SumCompensated returns the sum of the elements of the slice calculated with greater +// accuracy than Sum at the expense of additional computation. +func SumCompensated(s []float64) float64 { + // SumCompensated uses an improved version of Kahan's compensated + // summation algorithm proposed by Neumaier. + // See https://en.wikipedia.org/wiki/Kahan_summation_algorithm for details. + var sum, c float64 + for _, x := range s { + // This type conversion is here to prevent a sufficiently smart compiler + // from optimizing away these operations. + t := sum + x + switch { + case math.IsInf(t, 0): + c = 0 + + // Using Neumaier improvement, swap if next term larger than sum. + case math.Abs(sum) >= math.Abs(x): + c += (sum - t) + x + default: + c += (x - t) + sum + } + sum = t + } + return sum + c +} diff --git a/vendor/github.com/thanos-io/promql-engine/execution/aggregate/count_values.go b/vendor/github.com/thanos-io/promql-engine/execution/aggregate/count_values.go new file mode 100644 index 0000000000..cbafee09a7 --- /dev/null +++ b/vendor/github.com/thanos-io/promql-engine/execution/aggregate/count_values.go @@ -0,0 +1,198 @@ +// Copyright (c) The Thanos Community Authors. +// Licensed under the Apache License 2.0. + +package aggregate + +import ( + "context" + "fmt" + "slices" + "strconv" + "sync" + "time" + + "github.com/prometheus/prometheus/model/labels" + + "github.com/thanos-io/promql-engine/execution/model" + "github.com/thanos-io/promql-engine/query" +) + +type countValuesOperator struct { + model.OperatorTelemetry + + pool *model.VectorPool + next model.VectorOperator + param string + + by bool + grouping []string + + stepsBatch int + curStep int + + ts []int64 + counts []map[int]int + series []labels.Labels + + once sync.Once +} + +func NewCountValues(pool *model.VectorPool, next model.VectorOperator, param string, by bool, grouping []string, opts *query.Options) model.VectorOperator { + // Grouping labels need to be sorted in order for metric hashing to work. + // https://github.com/prometheus/prometheus/blob/8ed39fdab1ead382a354e45ded999eb3610f8d5f/model/labels/labels.go#L162-L181 + slices.Sort(grouping) + + op := &countValuesOperator{ + pool: pool, + next: next, + param: param, + stepsBatch: opts.StepsBatch, + by: by, + grouping: grouping, + } + op.OperatorTelemetry = model.NewTelemetry(op, opts.EnableAnalysis) + + return op +} + +func (c *countValuesOperator) Explain() []model.VectorOperator { + return []model.VectorOperator{c.next} +} + +func (c *countValuesOperator) GetPool() *model.VectorPool { + return c.pool +} + +func (c *countValuesOperator) String() string { + if c.by { + return fmt.Sprintf("[countValues] by (%v) - param (%v)", c.grouping, c.param) + } + return fmt.Sprintf("[countValues] without (%v) - param (%v)", c.grouping, c.param) +} + +func (c *countValuesOperator) Series(ctx context.Context) ([]labels.Labels, error) { + start := time.Now() + defer func() { c.AddExecutionTimeTaken(time.Since(start)) }() + + var err error + c.once.Do(func() { err = c.initSeriesOnce(ctx) }) + return c.series, err +} + +func (c *countValuesOperator) Next(ctx context.Context) ([]model.StepVector, error) { + start := time.Now() + defer func() { c.AddExecutionTimeTaken(time.Since(start)) }() + + select { + case <-ctx.Done(): + return nil, ctx.Err() + default: + } + + var err error + c.once.Do(func() { err = c.initSeriesOnce(ctx) }) + if err != nil { + return nil, err + } + + if c.curStep >= len(c.ts) { + return nil, nil + } + + batch := c.pool.GetVectorBatch() + for i := 0; i < c.stepsBatch; i++ { + if c.curStep >= len(c.ts) { + break + } + sv := c.pool.GetStepVector(c.ts[c.curStep]) + for i, v := range c.counts[c.curStep] { + sv.AppendSample(c.pool, uint64(i), float64(v)) + } + batch = append(batch, sv) + c.curStep++ + } + return batch, nil +} + +func (c *countValuesOperator) initSeriesOnce(ctx context.Context) error { + nextSeries, err := c.next.Series(ctx) + if err != nil { + return err + } + var ( + inputIdToHashBucket = make(map[int]uint64) + hashToBucketLabels = make(map[uint64]labels.Labels) + hashToOutputId = make(map[uint64]int) + + hashingBuf = make([]byte, 1024) + builder labels.ScratchBuilder + labelsMap = make(map[string]struct{}) + ) + for _, lblName := range c.grouping { + labelsMap[lblName] = struct{}{} + } + for i := 0; i < len(nextSeries); i++ { + hash, lbls := hashMetric(builder, nextSeries[i], !c.by, c.grouping, labelsMap, hashingBuf) + inputIdToHashBucket[i] = hash + if _, ok := hashToBucketLabels[hash]; !ok { + hashToBucketLabels[hash] = lbls + } + } + + ts := make([]int64, 0) + counts := make([]map[int]int, 0) + series := make([]labels.Labels, 0) + + b := labels.NewBuilder(labels.EmptyLabels()) + for { + select { + case <-ctx.Done(): + return ctx.Err() + default: + } + + in, err := c.next.Next(ctx) + if err != nil { + return err + } + if in == nil { + break + } + for i := range in { + ts = append(ts, in[i].T) + countPerHashbucket := make(map[uint64]map[float64]int, len(inputIdToHashBucket)) + for j := range in[i].Samples { + hash := inputIdToHashBucket[int(in[i].SampleIDs[j])] + if _, ok := countPerHashbucket[hash]; !ok { + countPerHashbucket[hash] = make(map[float64]int) + } + countPerHashbucket[hash][in[i].Samples[j]]++ + } + + countsPerOutputId := make(map[int]int) + for hash, counts := range countPerHashbucket { + b.Reset(hashToBucketLabels[hash]) + for f, count := range counts { + // TODO: Probably we should issue a warning if we override a label here + lbls := b.Set(c.param, strconv.FormatFloat(f, 'f', -1, 64)).Labels() + hash := lbls.Hash() + outputId, ok := hashToOutputId[hash] + if !ok { + series = append(series, lbls) + outputId = len(series) - 1 + hashToOutputId[hash] = outputId + } + countsPerOutputId[outputId] += count + } + } + counts = append(counts, countsPerOutputId) + } + c.next.GetPool().PutVectors(in) + } + + c.ts = ts + c.counts = counts + c.series = series + + return nil +} diff --git a/vendor/github.com/thanos-io/promql-engine/execution/aggregate/hashaggregate.go b/vendor/github.com/thanos-io/promql-engine/execution/aggregate/hashaggregate.go index 7ddf7b84ab..7c3129e985 100644 --- a/vendor/github.com/thanos-io/promql-engine/execution/aggregate/hashaggregate.go +++ b/vendor/github.com/thanos-io/promql-engine/execution/aggregate/hashaggregate.go @@ -11,6 +11,8 @@ import ( "time" "github.com/efficientgo/core/errors" + "github.com/prometheus/prometheus/promql/parser/posrange" + "github.com/prometheus/prometheus/util/annotations" "golang.org/x/exp/slices" "github.com/prometheus/prometheus/model/labels" @@ -18,6 +20,7 @@ import ( "github.com/thanos-io/promql-engine/execution/model" "github.com/thanos-io/promql-engine/execution/parse" + "github.com/thanos-io/promql-engine/execution/warnings" "github.com/thanos-io/promql-engine/query" ) @@ -125,21 +128,28 @@ func (a *aggregate) Next(ctx context.Context) ([]model.StepVector, error) { return nil, err } - args, err := a.paramOp.Next(ctx) - if err != nil { - return nil, err - } - for i := range args { - a.params[i] = args[i].Samples[0] - a.paramOp.GetPool().PutStepVector(args[i]) + if a.paramOp != nil { + args, err := a.paramOp.Next(ctx) + if err != nil { + return nil, err + } + for i := range args { + a.params[i] = args[i].Samples[0] + if sample := a.params[i]; math.IsNaN(sample) || sample < 0 || sample > 1 { + warnings.AddToContext(annotations.NewInvalidQuantileWarning(sample, posrange.PositionRange{}), ctx) + } + a.paramOp.GetPool().PutStepVector(args[i]) + } + a.paramOp.GetPool().PutVectors(args) } - a.paramOp.GetPool().PutVectors(args) for i, p := range a.params { a.tables[i].reset(p) } if a.lastBatch != nil { - a.aggregate(a.lastBatch) + if err := a.aggregate(a.lastBatch); err != nil { + return nil, err + } a.lastBatch = nil } for { @@ -153,7 +163,9 @@ func (a *aggregate) Next(ctx context.Context) ([]model.StepVector, error) { // Keep aggregating samples as long as timestamps of batches are equal. currentTs := a.tables[0].timestamp() if currentTs == math.MinInt64 || next[0].T == currentTs { - a.aggregate(next) + if err := a.aggregate(next); err != nil { + return nil, err + } continue } a.lastBatch = next @@ -174,12 +186,15 @@ func (a *aggregate) Next(ctx context.Context) ([]model.StepVector, error) { return result, nil } -func (a *aggregate) aggregate(in []model.StepVector) { +func (a *aggregate) aggregate(in []model.StepVector) error { for i, vector := range in { - a.tables[i].aggregate(vector) + if err := a.tables[i].aggregate(vector); err != nil { + return err + } a.next.GetPool().PutStepVector(vector) } a.next.GetPool().PutVectors(in) + return nil } func (a *aggregate) initializeTables(ctx context.Context) error { diff --git a/vendor/github.com/thanos-io/promql-engine/execution/aggregate/scalar_table.go b/vendor/github.com/thanos-io/promql-engine/execution/aggregate/scalar_table.go index 08c5e1c9f2..28e6d82e2a 100644 --- a/vendor/github.com/thanos-io/promql-engine/execution/aggregate/scalar_table.go +++ b/vendor/github.com/thanos-io/promql-engine/execution/aggregate/scalar_table.go @@ -12,11 +12,10 @@ import ( "github.com/efficientgo/core/errors" "github.com/prometheus/prometheus/model/histogram" "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/promql/parser" "github.com/prometheus/prometheus/promql/parser/posrange" "github.com/prometheus/prometheus/util/annotations" - "github.com/prometheus/prometheus/promql/parser" - "github.com/thanos-io/promql-engine/execution/model" "github.com/thanos-io/promql-engine/execution/parse" "github.com/thanos-io/promql-engine/execution/warnings" @@ -29,7 +28,7 @@ type aggregateTable interface { // If the table is empty, it returns math.MinInt64. timestamp() int64 // aggregate aggregates the given vector into the table. - aggregate(vector model.StepVector) + aggregate(vector model.StepVector) error // toVector writes out the accumulated result to the given vector and // resets the table. toVector(ctx context.Context, pool *model.VectorPool) model.StepVector @@ -78,29 +77,34 @@ func newScalarTable(inputSampleIDs []uint64, outputs []*model.Series, aggregatio }, nil } -func (t *scalarTable) aggregate(vector model.StepVector) { +func (t *scalarTable) aggregate(vector model.StepVector) error { t.ts = vector.T for i := range vector.Samples { - t.addSample(vector.SampleIDs[i], vector.Samples[i]) + if err := t.addSample(vector.SampleIDs[i], vector.Samples[i]); err != nil { + return err + } } for i := range vector.Histograms { - t.addHistogram(vector.HistogramIDs[i], vector.Histograms[i]) + if err := t.addHistogram(vector.HistogramIDs[i], vector.Histograms[i]); err != nil { + return err + } } + return nil } -func (t *scalarTable) addSample(sampleID uint64, sample float64) { +func (t *scalarTable) addSample(sampleID uint64, sample float64) error { outputSampleID := t.inputs[sampleID] output := t.outputs[outputSampleID] - t.accumulators[output.ID].Add(sample, nil) + return t.accumulators[output.ID].Add(sample, nil) } -func (t *scalarTable) addHistogram(sampleID uint64, h *histogram.FloatHistogram) { +func (t *scalarTable) addHistogram(sampleID uint64, h *histogram.FloatHistogram) error { outputSampleID := t.inputs[sampleID] output := t.outputs[outputSampleID] - t.accumulators[output.ID].Add(0, h) + return t.accumulators[output.ID].Add(0, h) } func (t *scalarTable) reset(arg float64) { @@ -124,8 +128,7 @@ func (t *scalarTable) toVector(ctx context.Context, pool *model.VectorPool) mode result.AppendHistogram(pool, v.ID, h) } case MixedTypeValue: - warn := annotations.New().Add(annotations.NewMixedFloatsHistogramsAggWarning(posrange.PositionRange{})) - warnings.AddToContext(warn, ctx) + warnings.AddToContext(annotations.NewMixedFloatsHistogramsAggWarning(posrange.PositionRange{}), ctx) } } return result diff --git a/vendor/github.com/thanos-io/promql-engine/execution/aggregate/vector_table.go b/vendor/github.com/thanos-io/promql-engine/execution/aggregate/vector_table.go index 5fad6cd9fb..f856188218 100644 --- a/vendor/github.com/thanos-io/promql-engine/execution/aggregate/vector_table.go +++ b/vendor/github.com/thanos-io/promql-engine/execution/aggregate/vector_table.go @@ -48,9 +48,9 @@ func (t *vectorTable) timestamp() int64 { return t.ts } -func (t *vectorTable) aggregate(vector model.StepVector) { +func (t *vectorTable) aggregate(vector model.StepVector) error { t.ts = vector.T - t.accumulator.AddVector(vector.Samples, vector.Histograms) + return t.accumulator.AddVector(vector.Samples, vector.Histograms) } func (t *vectorTable) toVector(ctx context.Context, pool *model.VectorPool) model.StepVector { @@ -66,8 +66,7 @@ func (t *vectorTable) toVector(ctx context.Context, pool *model.VectorPool) mode result.AppendHistogram(pool, 0, h) } case MixedTypeValue: - warn := annotations.New().Add(annotations.NewMixedFloatsHistogramsAggWarning(posrange.PositionRange{})) - warnings.AddToContext(warn, ctx) + warnings.AddToContext(annotations.NewMixedFloatsHistogramsAggWarning(posrange.PositionRange{}), ctx) } return result } @@ -97,12 +96,12 @@ func newVectorAccumulator(expr parser.ItemType) (vectorAccumulator, error) { return nil, errors.Wrap(parse.ErrNotSupportedExpr, msg) } -func histogramSum(current *histogram.FloatHistogram, histograms []*histogram.FloatHistogram) *histogram.FloatHistogram { +func histogramSum(current *histogram.FloatHistogram, histograms []*histogram.FloatHistogram) (*histogram.FloatHistogram, error) { if len(histograms) == 0 { - return current + return current, nil } if current == nil && len(histograms) == 1 { - return histograms[0].Copy() + return histograms[0].Copy(), nil } var histSum *histogram.FloatHistogram if current != nil { @@ -112,14 +111,19 @@ func histogramSum(current *histogram.FloatHistogram, histograms []*histogram.Flo histograms = histograms[1:] } + var err error for i := 0; i < len(histograms); i++ { if histograms[i].Schema >= histSum.Schema { - histSum = histSum.Add(histograms[i]) + histSum, err = histSum.Add(histograms[i]) + if err != nil { + return nil, err + } } else { t := histograms[i].Copy() - t.Add(histSum) - histSum = t + if histSum, err = t.Add(histSum); err != nil { + return nil, err + } } } - return histSum + return histSum, nil } diff --git a/vendor/github.com/thanos-io/promql-engine/execution/binary/vector.go b/vendor/github.com/thanos-io/promql-engine/execution/binary/vector.go index f284bd2e49..72c8081f15 100644 --- a/vendor/github.com/thanos-io/promql-engine/execution/binary/vector.go +++ b/vendor/github.com/thanos-io/promql-engine/execution/binary/vector.go @@ -268,14 +268,14 @@ func (o *vectorOperator) execBinaryUnless(lhs, rhs model.StepVector) (model.Step } // TODO: add support for histogram. -func (o *vectorOperator) computeBinaryPairing(hval, lval float64) (float64, bool) { +func (o *vectorOperator) computeBinaryPairing(hval, lval float64) (float64, bool, error) { // operand is not commutative so we need to address potential swapping if o.matching.Card == parser.CardOneToMany { - v, _, keep := vectorElemBinop(o.opType, lval, hval, nil, nil) - return v, keep + v, _, keep, err := vectorElemBinop(o.opType, lval, hval, nil, nil) + return v, keep, err } - v, _, keep := vectorElemBinop(o.opType, hval, lval, nil, nil) - return v, keep + v, _, keep, err := vectorElemBinop(o.opType, hval, lval, nil, nil) + return v, keep, err } func (o *vectorOperator) execBinaryArithmetic(lhs, rhs model.StepVector) (model.StepVector, error) { @@ -323,7 +323,10 @@ func (o *vectorOperator) execBinaryArithmetic(lhs, rhs model.StepVector) (model. } jp.bts = ts - val, keep := o.computeBinaryPairing(hcs.Samples[i], jp.val) + val, keep, err := o.computeBinaryPairing(hcs.Samples[i], jp.val) + if err != nil { + return model.StepVector{}, err + } if o.returnBool { val = 0 if keep { @@ -507,59 +510,75 @@ func signatureFunc(on bool, names ...string) func(labels.Labels) uint64 { // Lifted from: https://github.com/prometheus/prometheus/blob/a38179c4e183d9b50b271167bf90050eda8ec3d1/promql/engine.go#L2430. // TODO: call with histogram values in followup PR. // nolint: unparam -func vectorElemBinop(op parser.ItemType, lhs, rhs float64, hlhs, hrhs *histogram.FloatHistogram) (float64, *histogram.FloatHistogram, bool) { +func vectorElemBinop(op parser.ItemType, lhs, rhs float64, hlhs, hrhs *histogram.FloatHistogram) (float64, *histogram.FloatHistogram, bool, error) { switch op { case parser.ADD: if hlhs != nil && hrhs != nil { // The histogram being added must have the larger schema // code (i.e. the higher resolution). if hrhs.Schema >= hlhs.Schema { - return 0, hlhs.Copy().Add(hrhs).Compact(0), true + sum, err := hlhs.Copy().Add(hrhs) + if err != nil { + return 0, nil, false, err + } + return 0, sum.Compact(0), true, nil + } + sum, err := hrhs.Copy().Add(hlhs) + if err != nil { + return 0, nil, false, err } - return 0, hrhs.Copy().Add(hlhs).Compact(0), true + return 0, sum.Compact(0), true, nil } - return lhs + rhs, nil, true + return lhs + rhs, nil, true, nil case parser.SUB: if hlhs != nil && hrhs != nil { // The histogram being subtracted must have the larger schema // code (i.e. the higher resolution). if hrhs.Schema >= hlhs.Schema { - return 0, hlhs.Copy().Sub(hrhs).Compact(0), true + diff, err := hlhs.Copy().Sub(hrhs) + if err != nil { + return 0, nil, false, err + } + return 0, diff.Compact(0), true, nil + } + diff, err := hrhs.Copy().Mul(-1).Add(hlhs) + if err != nil { + return 0, nil, false, err } - return 0, hrhs.Copy().Mul(-1).Add(hlhs).Compact(0), true + return 0, diff.Compact(0), true, nil } - return lhs - rhs, nil, true + return lhs - rhs, nil, true, nil case parser.MUL: if hlhs != nil && hrhs == nil { - return 0, hlhs.Copy().Mul(rhs), true + return 0, hlhs.Copy().Mul(rhs), true, nil } if hlhs == nil && hrhs != nil { - return 0, hrhs.Copy().Mul(lhs), true + return 0, hrhs.Copy().Mul(lhs), true, nil } - return lhs * rhs, nil, true + return lhs * rhs, nil, true, nil case parser.DIV: if hlhs != nil && hrhs == nil { - return 0, hlhs.Copy().Div(rhs), true + return 0, hlhs.Copy().Div(rhs), true, nil } - return lhs / rhs, nil, true + return lhs / rhs, nil, true, nil case parser.POW: - return math.Pow(lhs, rhs), nil, true + return math.Pow(lhs, rhs), nil, true, nil case parser.MOD: - return math.Mod(lhs, rhs), nil, true + return math.Mod(lhs, rhs), nil, true, nil case parser.EQLC: - return lhs, nil, lhs == rhs + return lhs, nil, lhs == rhs, nil case parser.NEQ: - return lhs, nil, lhs != rhs + return lhs, nil, lhs != rhs, nil case parser.GTR: - return lhs, nil, lhs > rhs + return lhs, nil, lhs > rhs, nil case parser.LSS: - return lhs, nil, lhs < rhs + return lhs, nil, lhs < rhs, nil case parser.GTE: - return lhs, nil, lhs >= rhs + return lhs, nil, lhs >= rhs, nil case parser.LTE: - return lhs, nil, lhs <= rhs + return lhs, nil, lhs <= rhs, nil case parser.ATAN2: - return math.Atan2(lhs, rhs), nil, true + return math.Atan2(lhs, rhs), nil, true, nil } panic(errors.Newf("operator %q not allowed for operations between Vectors", op)) } diff --git a/vendor/github.com/thanos-io/promql-engine/execution/execution.go b/vendor/github.com/thanos-io/promql-engine/execution/execution.go index 919902d1ba..8746693b02 100644 --- a/vendor/github.com/thanos-io/promql-engine/execution/execution.go +++ b/vendor/github.com/thanos-io/promql-engine/execution/execution.go @@ -78,7 +78,7 @@ func newOperator(ctx context.Context, expr logicalplan.Node, storage storage.Sca case *logicalplan.CheckDuplicateLabels: return newDuplicateLabelCheck(ctx, e, storage, opts, hints) case logicalplan.Noop: - return noop.NewOperator(), nil + return noop.NewOperator(opts), nil case logicalplan.UserDefinedExpr: return e.MakeExecutionOperator(ctx, model.NewVectorPool(opts.StepsBatch), opts, hints) default: @@ -226,8 +226,6 @@ func newSubqueryFunction(ctx context.Context, e *logicalplan.FunctionCall, t *lo if err != nil { return nil, err } - default: - scalarArg = noop.NewOperator() } return scan.NewSubqueryOperator(model.NewVectorPool(opts.StepsBatch), inner, scalarArg, &outerOpts, e, t) @@ -259,21 +257,20 @@ func newAggregateExpression(ctx context.Context, e *logicalplan.Aggregation, sca if err != nil { return nil, err } + if e.Op == parser.COUNT_VALUES { + param := logicalplan.UnsafeUnwrapString(e.Param) + return aggregate.NewCountValues(model.NewVectorPool(opts.StepsBatch), next, param, !e.Without, e.Grouping, opts), nil + } // parameter is only required for count_values, quantile, topk and bottomk. var paramOp model.VectorOperator switch e.Op { - case parser.COUNT_VALUES: - return nil, parse.UnsupportedOperationErr(parser.COUNT_VALUES) case parser.QUANTILE, parser.TOPK, parser.BOTTOMK: paramOp, err = newOperator(ctx, e.Param, scanners, opts, hints) if err != nil { return nil, err } - default: - paramOp = noop.NewOperator() } - if e.Op == parser.TOPK || e.Op == parser.BOTTOMK { next, err = aggregate.NewKHashAggregate(model.NewVectorPool(opts.StepsBatch), next, paramOp, e.Op, !e.Without, e.Grouping, opts) } else { @@ -284,7 +281,6 @@ func newAggregateExpression(ctx context.Context, e *logicalplan.Aggregation, sca } return exchange.NewConcurrent(next, 2, opts), nil - } func newBinaryExpression(ctx context.Context, e *logicalplan.Binary, scanners storage.Scanners, opts *query.Options, hints promstorage.SelectHints) (model.VectorOperator, error) { diff --git a/vendor/github.com/thanos-io/promql-engine/execution/function/histogram.go b/vendor/github.com/thanos-io/promql-engine/execution/function/histogram.go index 1f89e00380..9adc4b656e 100644 --- a/vendor/github.com/thanos-io/promql-engine/execution/function/histogram.go +++ b/vendor/github.com/thanos-io/promql-engine/execution/function/histogram.go @@ -13,8 +13,11 @@ import ( "github.com/cespare/xxhash/v2" "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/promql/parser/posrange" + "github.com/prometheus/prometheus/util/annotations" "github.com/thanos-io/promql-engine/execution/model" + "github.com/thanos-io/promql-engine/execution/warnings" "github.com/thanos-io/promql-engine/extlabels" "github.com/thanos-io/promql-engine/logicalplan" "github.com/thanos-io/promql-engine/query" @@ -126,7 +129,11 @@ func (o *histogramOperator) Next(ctx context.Context) ([]model.StepVector, error o.scalarPoints = o.scalarPoints[:0] for _, scalar := range scalars { if len(scalar.Samples) > 0 { - o.scalarPoints = append(o.scalarPoints, scalar.Samples[0]) + sample := scalar.Samples[0] + if math.IsNaN(sample) || sample < 0 || sample > 1 { + warnings.AddToContext(annotations.NewInvalidQuantileWarning(sample, posrange.PositionRange{}), ctx) + } + o.scalarPoints = append(o.scalarPoints, sample) } o.scalarOp.GetPool().PutStepVector(scalar) } @@ -135,6 +142,7 @@ func (o *histogramOperator) Next(ctx context.Context) ([]model.StepVector, error return o.processInputSeries(vectors) } +// nolint: unparam func (o *histogramOperator) processInputSeries(vectors []model.StepVector) ([]model.StepVector, error) { out := o.pool.GetVectorBatch() for stepIndex, vector := range vectors { diff --git a/vendor/github.com/thanos-io/promql-engine/execution/function/quantile.go b/vendor/github.com/thanos-io/promql-engine/execution/function/quantile.go index 4223a71de3..4826feadd4 100644 --- a/vendor/github.com/thanos-io/promql-engine/execution/function/quantile.go +++ b/vendor/github.com/thanos-io/promql-engine/execution/function/quantile.go @@ -199,12 +199,15 @@ func histogramQuantile(q float64, h *histogram.FloatHistogram) float64 { for it.Next() { bucket = it.At() + if bucket.Count == 0 { + continue + } count += bucket.Count if count >= rank { break } } - if bucket.Lower < 0 && bucket.Upper > 0 { + if !h.UsesCustomBuckets() && bucket.Lower < 0 && bucket.Upper > 0 { switch { case len(h.NegativeBuckets) == 0 && len(h.PositiveBuckets) > 0: // The result is in the zero bucket and the histogram has only @@ -215,6 +218,17 @@ func histogramQuantile(q float64, h *histogram.FloatHistogram) float64 { // negative buckets. So we consider 0 to be the upper bound. bucket.Upper = 0 } + } else if h.UsesCustomBuckets() { + if bucket.Lower == math.Inf(-1) { + // first bucket, with lower bound -Inf + if bucket.Upper <= 0 { + return bucket.Upper + } + bucket.Lower = 0 + } else if bucket.Upper == math.Inf(1) { + // last bucket, with upper bound +Inf + return bucket.Lower + } } // Due to numerical inaccuracies, we could end up with a higher count // than h.Count. Thus, make sure count is never higher than h.Count. @@ -260,12 +274,13 @@ func histogramFraction(lower, upper float64, h *histogram.FloatHistogram) float6 for it.Next() { b := it.At() if b.Lower < 0 && b.Upper > 0 { - if len(h.NegativeBuckets) == 0 && len(h.PositiveBuckets) > 0 { + switch { + case len(h.NegativeBuckets) == 0 && len(h.PositiveBuckets) > 0: // This is the zero bucket and the histogram has only // positive buckets. So we consider 0 to be the lower // bound. b.Lower = 0 - } else if len(h.PositiveBuckets) == 0 && len(h.NegativeBuckets) > 0 { + case len(h.PositiveBuckets) == 0 && len(h.NegativeBuckets) > 0: // This is in the zero bucket and the histogram has only // negative buckets. So we consider 0 to be the upper // bound. diff --git a/vendor/github.com/thanos-io/promql-engine/execution/noop/operator.go b/vendor/github.com/thanos-io/promql-engine/execution/noop/operator.go index a7c6ebdb0f..5a02ee6f73 100644 --- a/vendor/github.com/thanos-io/promql-engine/execution/noop/operator.go +++ b/vendor/github.com/thanos-io/promql-engine/execution/noop/operator.go @@ -9,21 +9,30 @@ import ( "github.com/prometheus/prometheus/model/labels" "github.com/thanos-io/promql-engine/execution/model" + "github.com/thanos-io/promql-engine/query" + "github.com/thanos-io/promql-engine/storage/prometheus" ) type operator struct { - // this needs a pool so we dont panic if we want use "GetPool().PutVectors" - pool *model.VectorPool + model.VectorOperator } -func NewOperator() model.VectorOperator { return &operator{pool: model.NewVectorPool(0)} } - -func (o operator) String() string { return "[noop]" } - -func (o operator) Next(ctx context.Context) ([]model.StepVector, error) { return nil, nil } +func NewOperator(opts *query.Options) model.VectorOperator { + scanner := prometheus.NewVectorSelector( + model.NewVectorPool(0), + noopSelector{}, + opts, + 0, 0, false, 0, 1, + ) + return &operator{VectorOperator: scanner} +} -func (o operator) Series(ctx context.Context) ([]labels.Labels, error) { return nil, nil } +func (o operator) String() string { return "[noop]" } +func (o operator) Explain() (next []model.VectorOperator) { return nil } -func (o operator) GetPool() *model.VectorPool { return o.pool } +type noopSelector struct{} -func (o operator) Explain() (next []model.VectorOperator) { return nil } +func (n noopSelector) Matchers() []*labels.Matcher { return nil } +func (n noopSelector) GetSeries(ctx context.Context, shard, numShards int) ([]prometheus.SignedSeries, error) { + return nil, nil +} diff --git a/vendor/github.com/thanos-io/promql-engine/execution/remote/operator.go b/vendor/github.com/thanos-io/promql-engine/execution/remote/operator.go index f48363d5eb..bebe7612fa 100644 --- a/vendor/github.com/thanos-io/promql-engine/execution/remote/operator.go +++ b/vendor/github.com/thanos-io/promql-engine/execution/remote/operator.go @@ -116,7 +116,9 @@ func (s *storageAdapter) GetSeries(ctx context.Context, _, _ int) ([]promstorage func (s *storageAdapter) executeQuery(ctx context.Context) { result := s.query.Exec(ctx) - warnings.AddToContext(result.Warnings, ctx) + for _, w := range result.Warnings { + warnings.AddToContext(w, ctx) + } if result.Err != nil { s.err = result.Err return diff --git a/vendor/github.com/thanos-io/promql-engine/execution/scan/functions.go b/vendor/github.com/thanos-io/promql-engine/execution/scan/functions.go index 939ba9d884..5b430bef17 100644 --- a/vendor/github.com/thanos-io/promql-engine/execution/scan/functions.go +++ b/vendor/github.com/thanos-io/promql-engine/execution/scan/functions.go @@ -33,7 +33,7 @@ type FunctionArgs struct { ScalarPoint float64 } -type FunctionCall func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool) +type FunctionCall func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool, error) func instantValue(samples []ringbuffer.Sample[Value], isRate bool) (float64, bool) { lastSample := samples[len(samples)-1] @@ -62,170 +62,188 @@ func instantValue(samples []ringbuffer.Sample[Value], isRate bool) (float64, boo } var rangeVectorFuncs = map[string]FunctionCall{ - "sum_over_time": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool) { + "sum_over_time": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool, error) { if len(f.Samples) == 0 { - return 0., nil, false + return 0., nil, false, nil } - return sumOverTime(f.Samples), nil, true + return sumOverTime(f.Samples), nil, true, nil }, - "max_over_time": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool) { + "max_over_time": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool, error) { if len(f.Samples) == 0 { - return 0., nil, false + return 0., nil, false, nil } - return maxOverTime(f.Samples), nil, true + return maxOverTime(f.Samples), nil, true, nil }, - "min_over_time": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool) { + "min_over_time": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool, error) { if len(f.Samples) == 0 { - return 0., nil, false + return 0., nil, false, nil } - return minOverTime(f.Samples), nil, true + return minOverTime(f.Samples), nil, true, nil }, - "avg_over_time": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool) { + "avg_over_time": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool, error) { if len(f.Samples) == 0 { - return 0., nil, false + return 0., nil, false, nil } - return avgOverTime(f.Samples), nil, true + return avgOverTime(f.Samples), nil, true, nil }, - "stddev_over_time": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool) { + "stddev_over_time": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool, error) { if len(f.Samples) == 0 { - return 0., nil, false + return 0., nil, false, nil } - return stddevOverTime(f.Samples), nil, true + return stddevOverTime(f.Samples), nil, true, nil }, - "stdvar_over_time": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool) { + "stdvar_over_time": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool, error) { if len(f.Samples) == 0 { - return 0., nil, false + return 0., nil, false, nil } - return stdvarOverTime(f.Samples), nil, true + return stdvarOverTime(f.Samples), nil, true, nil }, - "count_over_time": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool) { + "count_over_time": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool, error) { if len(f.Samples) == 0 { - return 0., nil, false + return 0., nil, false, nil } - return countOverTime(f.Samples), nil, true + return countOverTime(f.Samples), nil, true, nil }, - "last_over_time": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool) { + "last_over_time": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool, error) { if len(f.Samples) == 0 { - return 0., nil, false + return 0., nil, false, nil } if f.Samples[0].V.H != nil { - return 0, f.Samples[len(f.Samples)-1].V.H.Copy(), true + return 0, f.Samples[len(f.Samples)-1].V.H.Copy(), true, nil } - return f.Samples[len(f.Samples)-1].V.F, nil, true + return f.Samples[len(f.Samples)-1].V.F, nil, true, nil }, - "present_over_time": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool) { + "present_over_time": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool, error) { if len(f.Samples) == 0 { - return 0., nil, false + return 0., nil, false, nil } - return 1., nil, true + return 1., nil, true, nil }, - "quantile_over_time": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool) { + "quantile_over_time": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool, error) { if len(f.Samples) == 0 { - return 0., nil, false + return 0., nil, false, nil } floats := make([]float64, len(f.Samples)) for i, sample := range f.Samples { floats[i] = sample.V.F } - return aggregate.Quantile(f.ScalarPoint, floats), nil, true + return aggregate.Quantile(f.ScalarPoint, floats), nil, true, nil }, - "changes": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool) { + "changes": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool, error) { if len(f.Samples) == 0 { - return 0., nil, false + return 0., nil, false, nil } - return changes(f.Samples), nil, true + return changes(f.Samples), nil, true, nil }, - "resets": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool) { + "resets": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool, error) { if len(f.Samples) == 0 { - return 0., nil, false + return 0., nil, false, nil } - return resets(f.Samples), nil, true + return resets(f.Samples), nil, true, nil }, - "deriv": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool) { + "deriv": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool, error) { if len(f.Samples) < 2 { - return 0., nil, false + return 0., nil, false, nil } - return deriv(f.Samples), nil, true + return deriv(f.Samples), nil, true, nil }, - "irate": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool) { + "irate": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool, error) { f.Samples = filterFloatOnlySamples(f.Samples) if len(f.Samples) < 2 { - return 0., nil, false + return 0., nil, false, nil } val, ok := instantValue(f.Samples, true) if !ok { - return 0., nil, false + return 0., nil, false, nil } - return val, nil, true + return val, nil, true, nil }, - "idelta": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool) { + "idelta": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool, error) { f.Samples = filterFloatOnlySamples(f.Samples) if len(f.Samples) < 2 { - return 0., nil, false + return 0., nil, false, nil } val, ok := instantValue(f.Samples, false) if !ok { - return 0., nil, false + return 0., nil, false, nil } - return val, nil, true + return val, nil, true, nil }, - "rate": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool) { + "rate": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool, error) { if len(f.Samples) < 2 { - return 0., nil, false + return 0., nil, false, nil } - v, h := extrapolatedRate(f.Samples, true, true, f.StepTime, f.SelectRange, f.Offset) - return v, h, true + v, h, err := extrapolatedRate(f.Samples, true, true, f.StepTime, f.SelectRange, f.Offset) + if err != nil { + return 0, nil, false, err + } + return v, h, true, nil }, - "delta": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool) { + "delta": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool, error) { if len(f.Samples) < 2 { - return 0., nil, false + return 0., nil, false, nil + } + v, h, err := extrapolatedRate(f.Samples, false, false, f.StepTime, f.SelectRange, f.Offset) + if err != nil { + return 0, nil, false, err } - v, h := extrapolatedRate(f.Samples, false, false, f.StepTime, f.SelectRange, f.Offset) - return v, h, true + return v, h, true, nil }, - "increase": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool) { + "increase": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool, error) { if len(f.Samples) < 2 { - return 0., nil, false + return 0., nil, false, nil + } + v, h, err := extrapolatedRate(f.Samples, true, false, f.StepTime, f.SelectRange, f.Offset) + if err != nil { + return 0, nil, false, err } - v, h := extrapolatedRate(f.Samples, true, false, f.StepTime, f.SelectRange, f.Offset) - return v, h, true + return v, h, true, nil }, - "xrate": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool) { + "xrate": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool, error) { if len(f.Samples) == 0 { - return 0., nil, false + return 0., nil, false, nil } if f.MetricAppearedTs == nil { panic("BUG: we got some Samples but metric still hasn't appeared") } - v, h := extendedRate(f.Samples, true, true, f.StepTime, f.SelectRange, f.Offset, *f.MetricAppearedTs) - return v, h, true + v, h, err := extendedRate(f.Samples, true, true, f.StepTime, f.SelectRange, f.Offset, *f.MetricAppearedTs) + if err != nil { + return 0, nil, false, err + } + return v, h, true, nil }, - "xdelta": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool) { + "xdelta": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool, error) { if len(f.Samples) == 0 { - return 0., nil, false + return 0., nil, false, nil } if f.MetricAppearedTs == nil { panic("BUG: we got some Samples but metric still hasn't appeared") } - v, h := extendedRate(f.Samples, false, false, f.StepTime, f.SelectRange, f.Offset, *f.MetricAppearedTs) - return v, h, true + v, h, err := extendedRate(f.Samples, false, false, f.StepTime, f.SelectRange, f.Offset, *f.MetricAppearedTs) + if err != nil { + return 0, nil, false, err + } + return v, h, true, nil }, - "xincrease": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool) { + "xincrease": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool, error) { if len(f.Samples) == 0 { - return 0., nil, false + return 0., nil, false, nil } if f.MetricAppearedTs == nil { panic("BUG: we got some Samples but metric still hasn't appeared") } - v, h := extendedRate(f.Samples, true, false, f.StepTime, f.SelectRange, f.Offset, *f.MetricAppearedTs) - return v, h, true + v, h, err := extendedRate(f.Samples, true, false, f.StepTime, f.SelectRange, f.Offset, *f.MetricAppearedTs) + if err != nil { + return 0, nil, false, err + } + return v, h, true, nil }, - "predict_linear": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool) { + "predict_linear": func(f FunctionArgs) (float64, *histogram.FloatHistogram, bool, error) { if len(f.Samples) < 2 { - return 0., nil, false + return 0., nil, false, nil } v := predictLinear(f.Samples, f.ScalarPoint, f.StepTime) - return v, nil, true + return v, nil, true, nil }, } @@ -241,7 +259,7 @@ func NewRangeVectorFunc(name string) (FunctionCall, error) { // It calculates the rate (allowing for counter resets if isCounter is true), // extrapolates if the first/last sample is close to the boundary, and returns // the result as either per-second (if isRate is true) or overall. -func extrapolatedRate(samples []ringbuffer.Sample[Value], isCounter, isRate bool, stepTime int64, selectRange int64, offset int64) (float64, *histogram.FloatHistogram) { +func extrapolatedRate(samples []ringbuffer.Sample[Value], isCounter, isRate bool, stepTime int64, selectRange int64, offset int64) (float64, *histogram.FloatHistogram, error) { var ( rangeStart = stepTime - (selectRange + offset) rangeEnd = stepTime - offset @@ -249,8 +267,12 @@ func extrapolatedRate(samples []ringbuffer.Sample[Value], isCounter, isRate bool resultHistogram *histogram.FloatHistogram ) + var err error if samples[0].V.H != nil { - resultHistogram = histogramRate(samples, isCounter) + resultHistogram, err = histogramRate(samples, isCounter) + if err != nil { + return 0, nil, err + } } else { resultValue = samples[len(samples)-1].V.F - samples[0].V.F if isCounter { @@ -313,14 +335,14 @@ func extrapolatedRate(samples []ringbuffer.Sample[Value], isCounter, isRate bool } - return resultValue, resultHistogram + return resultValue, resultHistogram, nil } // extendedRate is a utility function for xrate/xincrease/xdelta. // It calculates the rate (allowing for counter resets if isCounter is true), // taking into account the last sample before the range start, and returns // the result as either per-second (if isRate is true) or overall. -func extendedRate(samples []ringbuffer.Sample[Value], isCounter, isRate bool, stepTime int64, selectRange int64, offset int64, metricAppearedTs int64) (float64, *histogram.FloatHistogram) { +func extendedRate(samples []ringbuffer.Sample[Value], isCounter, isRate bool, stepTime int64, selectRange int64, offset int64, metricAppearedTs int64) (float64, *histogram.FloatHistogram, error) { var ( rangeStart = stepTime - (selectRange + offset) rangeEnd = stepTime - offset @@ -329,9 +351,14 @@ func extendedRate(samples []ringbuffer.Sample[Value], isCounter, isRate bool, st ) if samples[0].V.H != nil { + var err error // TODO - support extended rate for histograms - resultHistogram = histogramRate(samples, isCounter) - return resultValue, resultHistogram + resultHistogram, err = histogramRate(samples, isCounter) + if err != nil { + return 0, nil, err + } + + return resultValue, resultHistogram, nil } sameVals := true @@ -348,7 +375,7 @@ func extendedRate(samples []ringbuffer.Sample[Value], isCounter, isRate bool, st if isCounter && !isRate && sameVals { // Make sure we are not at the end of the range. if stepTime-offset <= until { - return samples[0].V.F, nil + return samples[0].V.F, nil, nil } } @@ -361,7 +388,7 @@ func extendedRate(samples []ringbuffer.Sample[Value], isCounter, isRate bool, st // If the point before the range is too far from rangeStart, drop it. if float64(rangeStart-samples[0].T) > averageDurationBetweenSamples { if len(samples) < 3 { - return resultValue, nil + return resultValue, nil, nil } firstPoint = 1 sampledInterval = float64(samples[len(samples)-1].T - samples[1].T) @@ -401,22 +428,22 @@ func extendedRate(samples []ringbuffer.Sample[Value], isCounter, isRate bool, st resultValue = resultValue / float64(selectRange/1000) } - return resultValue, nil + return resultValue, nil, nil } // histogramRate is a helper function for extrapolatedRate. It requires // points[0] to be a histogram. It returns nil if any other Point in points is // not a histogram. -func histogramRate(points []ringbuffer.Sample[Value], isCounter bool) *histogram.FloatHistogram { +func histogramRate(points []ringbuffer.Sample[Value], isCounter bool) (*histogram.FloatHistogram, error) { // Calculating a rate on a single sample is not defined. if len(points) < 2 { - return nil + return nil, nil } prev := points[0].V.H // We already know that this is a histogram. last := points[len(points)-1].V.H if last == nil { - return nil // Range contains a mix of histograms and floats. + return nil, nil // Range contains a mix of histograms and floats. } minSchema := prev.Schema if last.Schema < minSchema { @@ -431,7 +458,7 @@ func histogramRate(points []ringbuffer.Sample[Value], isCounter bool) *histogram for _, currPoint := range points[1 : len(points)-1] { curr := currPoint.V.H if curr == nil { - return nil // Range contains a mix of histograms and floats. + return nil, nil // Range contains a mix of histograms and floats. } if !isCounter { continue @@ -442,20 +469,24 @@ func histogramRate(points []ringbuffer.Sample[Value], isCounter bool) *histogram } h := last.CopyToSchema(minSchema) - h.Sub(prev) + if _, err := h.Sub(prev); err != nil { + return nil, err + } if isCounter { // Second iteration to deal with counter resets. for _, currPoint := range points[1:] { curr := currPoint.V.H if curr.DetectReset(prev) { - h.Add(prev) + if _, err := h.Add(prev); err != nil { + return nil, err + } } prev = curr } } h.CounterResetHint = histogram.GaugeType - return h.Compact(0) + return h.Compact(0), nil } func maxOverTime(points []ringbuffer.Sample[Value]) float64 { diff --git a/vendor/github.com/thanos-io/promql-engine/execution/scan/subquery.go b/vendor/github.com/thanos-io/promql-engine/execution/scan/subquery.go index b2b74727f0..11f60b89f5 100644 --- a/vendor/github.com/thanos-io/promql-engine/execution/scan/subquery.go +++ b/vendor/github.com/thanos-io/promql-engine/execution/scan/subquery.go @@ -6,6 +6,7 @@ package scan import ( "context" "fmt" + "math" "sync" "time" @@ -107,15 +108,20 @@ func (o *subqueryOperator) Next(ctx context.Context) ([]model.StepVector, error) return nil, err } - args, err := o.paramOp.Next(ctx) - if err != nil { - return nil, err - } - for i := range args { - o.params[i] = args[i].Samples[0] - o.paramOp.GetPool().PutStepVector(args[i]) + if o.paramOp != nil { + args, err := o.paramOp.Next(ctx) + if err != nil { + return nil, err + } + for i := range args { + o.params[i] = math.NaN() + if len(args[i].Samples) == 1 { + o.params[i] = args[i].Samples[0] + } + o.paramOp.GetPool().PutStepVector(args[i]) + } + o.paramOp.GetPool().PutVectors(args) } - o.paramOp.GetPool().PutVectors(args) res := o.pool.GetVectorBatch() for i := 0; o.currentStep <= o.maxt && i < o.stepsBatch; i++ { @@ -161,13 +167,16 @@ func (o *subqueryOperator) Next(ctx context.Context) ([]model.StepVector, error) sv := o.pool.GetStepVector(o.currentStep) for sampleId, rangeSamples := range o.buffers { - f, h, ok := o.call(FunctionArgs{ + f, h, ok, err := o.call(FunctionArgs{ ScalarPoint: o.params[i], Samples: rangeSamples.Samples(), StepTime: maxt + o.subQuery.Offset.Milliseconds(), SelectRange: o.subQuery.Range.Milliseconds(), Offset: o.subQuery.Offset.Milliseconds(), }) + if err != nil { + return nil, err + } if ok { if h != nil { sv.AppendHistogram(o.pool, uint64(sampleId), h) diff --git a/vendor/github.com/thanos-io/promql-engine/execution/warnings/context.go b/vendor/github.com/thanos-io/promql-engine/execution/warnings/context.go index 2472adab5e..1ac5830cbb 100644 --- a/vendor/github.com/thanos-io/promql-engine/execution/warnings/context.go +++ b/vendor/github.com/thanos-io/promql-engine/execution/warnings/context.go @@ -23,10 +23,10 @@ func newWarnings() *warnings { return &warnings{warns: annotations.Annotations{}} } -func (w *warnings) add(warns annotations.Annotations) { +func (w *warnings) add(warns error) { w.mu.Lock() defer w.mu.Unlock() - w.warns = w.warns.Merge(warns) + w.warns = w.warns.Add(warns) } func (w *warnings) get() annotations.Annotations { @@ -39,15 +39,12 @@ func NewContext(ctx context.Context) context.Context { return context.WithValue(ctx, key, newWarnings()) } -func AddToContext(warns annotations.Annotations, ctx context.Context) { - if len(warns) == 0 { - return - } +func AddToContext(warn error, ctx context.Context) { w, ok := ctx.Value(key).(*warnings) if !ok { return } - w.add(warns) + w.add(warn) } func FromContext(ctx context.Context) annotations.Annotations { diff --git a/vendor/github.com/thanos-io/promql-engine/logicalplan/codec.go b/vendor/github.com/thanos-io/promql-engine/logicalplan/codec.go index 0876dbef42..d57098443e 100644 --- a/vendor/github.com/thanos-io/promql-engine/logicalplan/codec.go +++ b/vendor/github.com/thanos-io/promql-engine/logicalplan/codec.go @@ -4,11 +4,19 @@ package logicalplan import ( + "bytes" "encoding/json" + "math" "github.com/prometheus/prometheus/model/labels" ) +const ( + nanVal = `"NaN"` + infVal = `"+Inf"` + negInfVal = `"-Inf"` +) + type jsonNode struct { Type NodeType `json:"type"` Data json.RawMessage `json:"data"` @@ -29,9 +37,25 @@ func marshalNode(node Node) ([]byte, error) { } children = append(children, childData) } - data, err := json.Marshal(node) - if err != nil { - return nil, err + var data json.RawMessage = nil + // Special handling for -Inf/+Inf values. + if n, ok := node.(*NumberLiteral); ok { + if math.IsInf(n.Val, 1) { + data = json.RawMessage(infVal) + } + if math.IsInf(n.Val, -1) { + data = json.RawMessage(negInfVal) + } + if math.IsNaN(n.Val) { + data = json.RawMessage(nanVal) + } + } + if data == nil { + var err error + data, err = json.Marshal(node) + if err != nil { + return nil, err + } } return json.Marshal(jsonNode{ Type: node.Type(), @@ -122,8 +146,16 @@ func unmarshalNode(data []byte) (Node, error) { return f, nil case NumberLiteralNode: n := &NumberLiteral{} - if err := json.Unmarshal(t.Data, n); err != nil { - return nil, err + if bytes.Equal(t.Data, []byte(infVal)) { + n.Val = math.Inf(1) + } else if bytes.Equal(t.Data, []byte(negInfVal)) { + n.Val = math.Inf(-1) + } else if bytes.Equal(t.Data, []byte(nanVal)) { + n.Val = math.NaN() + } else { + if err := json.Unmarshal(t.Data, n); err != nil { + return nil, err + } } return n, nil case StringLiteralNode: diff --git a/vendor/github.com/thanos-io/promql-engine/logicalplan/distribute.go b/vendor/github.com/thanos-io/promql-engine/logicalplan/distribute.go index a1e420f488..46a229faf9 100644 --- a/vendor/github.com/thanos-io/promql-engine/logicalplan/distribute.go +++ b/vendor/github.com/thanos-io/promql-engine/logicalplan/distribute.go @@ -463,7 +463,7 @@ func isDistributive(expr *Node, skipBinaryPushdown bool, engineLabels map[string case Deduplicate, RemoteExecution: return false case *Binary: - return isBinaryExpressionWithOneConstantSide(e) || (!skipBinaryPushdown && isBinaryExpressionWithDistributableMatching(e)) + return isBinaryExpressionWithOneScalarSide(e) || (!skipBinaryPushdown && isBinaryExpressionWithDistributableMatching(e)) case *Aggregation: // Certain aggregations are currently not supported. if _, ok := distributiveAggregations[e.Op]; !ok { @@ -482,9 +482,9 @@ func isDistributive(expr *Node, skipBinaryPushdown bool, engineLabels map[string return true } -func isBinaryExpressionWithOneConstantSide(expr *Binary) bool { - lhsConstant := IsConstantExpr(expr.LHS) - rhsConstant := IsConstantExpr(expr.RHS) +func isBinaryExpressionWithOneScalarSide(expr *Binary) bool { + lhsConstant := IsConstantScalarExpr(expr.LHS) + rhsConstant := IsConstantScalarExpr(expr.RHS) return lhsConstant || rhsConstant } diff --git a/vendor/github.com/thanos-io/promql-engine/logicalplan/exprutil.go b/vendor/github.com/thanos-io/promql-engine/logicalplan/exprutil.go index 4cd3f1b73b..7cb4a8d467 100644 --- a/vendor/github.com/thanos-io/promql-engine/logicalplan/exprutil.go +++ b/vendor/github.com/thanos-io/promql-engine/logicalplan/exprutil.go @@ -75,3 +75,20 @@ func IsConstantExpr(expr Node) bool { return false } } + +// IsConstantScalarExpr reports if the expression evaluates to a scalar. +func IsConstantScalarExpr(expr Node) bool { + // TODO: there are more possibilities for constant expressions + switch texpr := expr.(type) { + case *NumberLiteral, *StringLiteral: + return true + case *StepInvariantExpr: + return IsConstantScalarExpr(texpr.Expr) + case *Parens: + return IsConstantScalarExpr(texpr.Expr) + case *Binary: + return IsConstantScalarExpr(texpr.LHS) && IsConstantScalarExpr(texpr.RHS) + default: + return false + } +} diff --git a/vendor/github.com/thanos-io/promql-engine/logicalplan/histogram_stats.go b/vendor/github.com/thanos-io/promql-engine/logicalplan/histogram_stats.go new file mode 100644 index 0000000000..1cb1428d92 --- /dev/null +++ b/vendor/github.com/thanos-io/promql-engine/logicalplan/histogram_stats.go @@ -0,0 +1,45 @@ +// Copyright (c) The Thanos Community Authors. +// Licensed under the Apache License 2.0. + +package logicalplan + +import ( + "github.com/prometheus/prometheus/util/annotations" + + "github.com/thanos-io/promql-engine/query" +) + +type DetectHistogramStatsOptimizer struct{} + +func (d DetectHistogramStatsOptimizer) Optimize(plan Node, _ *query.Options) (Node, annotations.Annotations) { + return d.optimize(plan, false) +} + +func (d DetectHistogramStatsOptimizer) optimize(plan Node, decodeStats bool) (Node, annotations.Annotations) { + var stop bool + Traverse(&plan, func(node *Node) { + if stop { + return + } + switch n := (*node).(type) { + case *VectorSelector: + n.DecodeNativeHistogramStats = decodeStats + case *FunctionCall: + switch n.Func.Name { + case "histogram_count", "histogram_sum": + n.Args[0], _ = d.optimize(n.Args[0], true) + stop = true + return + case "histogram_quantile": + n.Args[1], _ = d.optimize(n.Args[1], false) + stop = true + return + case "histogram_fraction": + n.Args[2], _ = d.optimize(n.Args[2], false) + stop = true + return + } + } + }) + return plan, nil +} diff --git a/vendor/github.com/thanos-io/promql-engine/logicalplan/logical_nodes.go b/vendor/github.com/thanos-io/promql-engine/logicalplan/logical_nodes.go index 40acd97610..2fcb792d42 100644 --- a/vendor/github.com/thanos-io/promql-engine/logicalplan/logical_nodes.go +++ b/vendor/github.com/thanos-io/promql-engine/logicalplan/logical_nodes.go @@ -73,6 +73,10 @@ type VectorSelector struct { BatchSize int64 SelectTimestamp bool Projection Projection + // When set, histogram iterators can return objects which only have their + // CounterResetHint, Count and Sum values populated. Histogram buckets and spans + // will not be used during query evaluation. + DecodeNativeHistogramStats bool } func (f *VectorSelector) Clone() Node { diff --git a/vendor/github.com/thanos-io/promql-engine/logicalplan/plan.go b/vendor/github.com/thanos-io/promql-engine/logicalplan/plan.go index 5b15924db4..7c593a109c 100644 --- a/vendor/github.com/thanos-io/promql-engine/logicalplan/plan.go +++ b/vendor/github.com/thanos-io/promql-engine/logicalplan/plan.go @@ -23,6 +23,7 @@ var ( var DefaultOptimizers = []Optimizer{ SortMatchers{}, MergeSelectsOptimizer{}, + DetectHistogramStatsOptimizer{}, } type Plan interface { @@ -64,6 +65,13 @@ func NewFromAST(ast parser.Expr, queryOpts *query.Options, planOpts PlanOptions) // the engine handles sorting at the presentation layer expr = trimSorts(expr) + // best effort evaluate constant expressions + expr = reduceConstantExpressions(expr) + + // some parameters are implicitly step invariant, i.e. topk(scalar(SERIES), X) + // morally that should be done by PreprocessExpr but we can also fix it here + expr = preprocessAggregationParameters(expr) + return &plan{ expr: expr, opts: queryOpts, @@ -219,6 +227,18 @@ func replacePrometheusNodes(plan parser.Expr) Node { panic("Unrecognized AST node") } +func preprocessAggregationParameters(expr Node) Node { + Traverse(&expr, func(node *Node) { + switch t := (*node).(type) { + case *Aggregation: + if t.Param != nil { + t.Param = &StepInvariantExpr{Expr: t.Param} + } + } + }) + return expr +} + func trimSorts(expr Node) Node { canTrimSorts := true // We cannot trim inner sort if its an argument to a timestamp function. @@ -258,6 +278,60 @@ func trimSorts(expr Node) Node { return expr } +func reduceConstantExpressions(expr Node) Node { + TraverseBottomUp(nil, &expr, func(parent, current *Node) bool { + if current == nil || parent == nil { + return true + } + switch tparent := (*parent).(type) { + case *Parens: + num, err := UnwrapFloat(tparent.Expr) + if err != nil { + return false + } + *parent = &NumberLiteral{Val: num} + case *Unary: + num, err := UnwrapFloat(tparent.Expr) + if err != nil { + return false + } + switch tparent.Op { + case parser.ADD: + *parent = &NumberLiteral{Val: num} + case parser.SUB: + *parent = &NumberLiteral{Val: -num} + } + case *Binary: + lnum, err := UnwrapFloat(tparent.LHS) + if err != nil { + return false + } + rnum, err := UnwrapFloat(tparent.RHS) + if err != nil { + return false + } + switch tparent.Op { + case parser.ADD: + *parent = &NumberLiteral{Val: lnum + rnum} + case parser.SUB: + *parent = &NumberLiteral{Val: lnum - rnum} + case parser.MUL: + *parent = &NumberLiteral{Val: lnum * rnum} + case parser.DIV: + *parent = &NumberLiteral{Val: lnum / rnum} + case parser.POW: + *parent = &NumberLiteral{Val: math.Pow(lnum, rnum)} + case parser.MOD: + *parent = &NumberLiteral{Val: math.Mod(lnum, rnum)} + default: + return false + } + } + return false + }) + return expr +} + func trimParens(expr Node) Node { TraverseBottomUp(nil, &expr, func(parent, current *Node) bool { if current == nil || parent == nil { diff --git a/vendor/github.com/thanos-io/promql-engine/storage/prometheus/histograms.go b/vendor/github.com/thanos-io/promql-engine/storage/prometheus/histograms.go new file mode 100644 index 0000000000..769b7de59f --- /dev/null +++ b/vendor/github.com/thanos-io/promql-engine/storage/prometheus/histograms.go @@ -0,0 +1,66 @@ +// Copyright (c) The Thanos Community Authors. +// Licensed under the Apache License 2.0. + +package prometheus + +import ( + "github.com/prometheus/prometheus/model/histogram" + "github.com/prometheus/prometheus/model/value" + "github.com/prometheus/prometheus/tsdb/chunkenc" +) + +type histogramStatsIterator struct { + chunkenc.Iterator + hReader *histogram.Histogram + fhReader *histogram.FloatHistogram +} + +func NewHistogramStatsIterator(it chunkenc.Iterator) chunkenc.Iterator { + return histogramStatsIterator{ + Iterator: it, + hReader: &histogram.Histogram{}, + fhReader: &histogram.FloatHistogram{}, + } +} + +func (f histogramStatsIterator) AtHistogram(h *histogram.Histogram) (int64, *histogram.Histogram) { + var t int64 + t, f.hReader = f.Iterator.AtHistogram(f.hReader) + if value.IsStaleNaN(f.hReader.Sum) { + return t, &histogram.Histogram{Sum: f.hReader.Sum} + } + + if h == nil { + return t, &histogram.Histogram{ + CounterResetHint: f.hReader.CounterResetHint, + Count: f.hReader.Count, + Sum: f.hReader.Sum, + } + } + + h.CounterResetHint = f.hReader.CounterResetHint + h.Count = f.hReader.Count + h.Sum = f.hReader.Sum + return t, h +} + +func (f histogramStatsIterator) AtFloatHistogram(fh *histogram.FloatHistogram) (int64, *histogram.FloatHistogram) { + var t int64 + t, f.fhReader = f.Iterator.AtFloatHistogram(f.fhReader) + if value.IsStaleNaN(f.fhReader.Sum) { + return t, &histogram.FloatHistogram{Sum: f.fhReader.Sum} + } + + if fh == nil { + return t, &histogram.FloatHistogram{ + CounterResetHint: f.fhReader.CounterResetHint, + Count: f.fhReader.Count, + Sum: f.fhReader.Sum, + } + } + + fh.CounterResetHint = f.fhReader.CounterResetHint + fh.Count = f.fhReader.Count + fh.Sum = f.fhReader.Sum + return t, fh +} diff --git a/vendor/github.com/thanos-io/promql-engine/storage/prometheus/matrix_selector.go b/vendor/github.com/thanos-io/promql-engine/storage/prometheus/matrix_selector.go index 243e3d4570..72d0be728b 100644 --- a/vendor/github.com/thanos-io/promql-engine/storage/prometheus/matrix_selector.go +++ b/vendor/github.com/thanos-io/promql-engine/storage/prometheus/matrix_selector.go @@ -189,7 +189,7 @@ func (o *matrixSelector) Next(ctx context.Context) ([]model.StepVector, error) { // Also, allow operator to exist independently without being nested // under parser.Call by implementing new data model. // https://github.com/thanos-io/promql-engine/issues/39 - f, h, ok := o.call(scan.FunctionArgs{ + f, h, ok, err := o.call(scan.FunctionArgs{ Samples: series.buffer.Samples(), StepTime: seriesTs, SelectRange: o.selectRange, @@ -197,7 +197,9 @@ func (o *matrixSelector) Next(ctx context.Context) ([]model.StepVector, error) { ScalarPoint: o.scalarArg, MetricAppearedTs: series.metricAppearedTs, }) - + if err != nil { + return nil, err + } if ok { vectors[currStep].T = seriesTs if h != nil { diff --git a/vendor/github.com/thanos-io/promql-engine/storage/prometheus/scanners.go b/vendor/github.com/thanos-io/promql-engine/storage/prometheus/scanners.go index a9f88d4527..e4c718c530 100644 --- a/vendor/github.com/thanos-io/promql-engine/storage/prometheus/scanners.go +++ b/vendor/github.com/thanos-io/promql-engine/storage/prometheus/scanners.go @@ -5,13 +5,18 @@ package prometheus import ( "context" + "math" "github.com/efficientgo/core/errors" + "github.com/prometheus/prometheus/promql/parser/posrange" "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/prometheus/prometheus/util/annotations" "github.com/thanos-io/promql-engine/execution/exchange" "github.com/thanos-io/promql-engine/execution/model" "github.com/thanos-io/promql-engine/execution/parse" + "github.com/thanos-io/promql-engine/execution/warnings" "github.com/thanos-io/promql-engine/logicalplan" "github.com/thanos-io/promql-engine/query" ) @@ -31,6 +36,9 @@ func (p Scanners) NewVectorSelector( logicalNode logicalplan.VectorSelector, ) (model.VectorOperator, error) { selector := p.selectors.GetFilteredSelector(hints.Start, hints.End, opts.Step.Milliseconds(), logicalNode.VectorSelector.LabelMatchers, logicalNode.Filters, hints) + if logicalNode.DecodeNativeHistogramStats { + selector = newHistogramStatsSelector(selector) + } operators := make([]model.VectorOperator, 0, opts.DecodingConcurrency) for i := 0; i < opts.DecodingConcurrency; i++ { @@ -52,7 +60,7 @@ func (p Scanners) NewVectorSelector( } func (p Scanners) NewMatrixSelector( - _ context.Context, + ctx context.Context, opts *query.Options, hints storage.SelectHints, logicalNode logicalplan.MatrixSelector, @@ -66,6 +74,9 @@ func (p Scanners) NewMatrixSelector( return nil, errors.Wrapf(parse.ErrNotSupportedExpr, "quantile_over_time with expression as first argument is not supported") } arg = unwrap + if math.IsNaN(unwrap) || unwrap < 0 || unwrap > 1 { + warnings.AddToContext(annotations.NewInvalidQuantileWarning(unwrap, posrange.PositionRange{}), ctx) + } case "predict_linear": unwrap, err := logicalplan.UnwrapFloat(call.Args[1]) if err != nil { @@ -75,13 +86,16 @@ func (p Scanners) NewMatrixSelector( } vs := logicalNode.VectorSelector - filter := p.selectors.GetFilteredSelector(hints.Start, hints.End, opts.Step.Milliseconds(), vs.LabelMatchers, vs.Filters, hints) + selector := p.selectors.GetFilteredSelector(hints.Start, hints.End, opts.Step.Milliseconds(), vs.LabelMatchers, vs.Filters, hints) + if logicalNode.VectorSelector.DecodeNativeHistogramStats { + selector = newHistogramStatsSelector(selector) + } operators := make([]model.VectorOperator, 0, opts.DecodingConcurrency) for i := 0; i < opts.DecodingConcurrency; i++ { operator, err := NewMatrixSelector( model.NewVectorPool(opts.StepsBatch), - filter, + selector, call.Func.Name, arg, opts, @@ -99,3 +113,34 @@ func (p Scanners) NewMatrixSelector( return exchange.NewCoalesce(model.NewVectorPool(opts.StepsBatch), opts, vs.BatchSize*int64(opts.DecodingConcurrency), operators...), nil } + +type histogramStatsSelector struct { + SeriesSelector +} + +func newHistogramStatsSelector(seriesSelector SeriesSelector) histogramStatsSelector { + return histogramStatsSelector{SeriesSelector: seriesSelector} +} + +func (h histogramStatsSelector) GetSeries(ctx context.Context, shard, numShards int) ([]SignedSeries, error) { + series, err := h.SeriesSelector.GetSeries(ctx, shard, numShards) + if err != nil { + return nil, err + } + for i := range series { + series[i].Series = newHistogramStatsSeries(series[i].Series) + } + return series, nil +} + +type histogramStatsSeries struct { + storage.Series +} + +func newHistogramStatsSeries(series storage.Series) histogramStatsSeries { + return histogramStatsSeries{Series: series} +} + +func (h histogramStatsSeries) Iterator(it chunkenc.Iterator) chunkenc.Iterator { + return NewHistogramStatsIterator(h.Series.Iterator(it)) +} diff --git a/vendor/github.com/thanos-io/promql-engine/storage/prometheus/series_selector.go b/vendor/github.com/thanos-io/promql-engine/storage/prometheus/series_selector.go index d769969f98..cffad6c807 100644 --- a/vendor/github.com/thanos-io/promql-engine/storage/prometheus/series_selector.go +++ b/vendor/github.com/thanos-io/promql-engine/storage/prometheus/series_selector.go @@ -78,7 +78,9 @@ func (o *seriesSelector) loadSeries(ctx context.Context) error { i++ } - warnings.AddToContext(seriesSet.Warnings(), ctx) + for _, w := range seriesSet.Warnings() { + warnings.AddToContext(w, ctx) + } return seriesSet.Err() } diff --git a/vendor/github.com/thanos-io/thanos/pkg/discovery/dns/resolver.go b/vendor/github.com/thanos-io/thanos/pkg/discovery/dns/resolver.go index d0078aea57..0025178607 100644 --- a/vendor/github.com/thanos-io/thanos/pkg/discovery/dns/resolver.go +++ b/vendor/github.com/thanos-io/thanos/pkg/discovery/dns/resolver.go @@ -108,7 +108,9 @@ func (s *dnsSD) Resolve(ctx context.Context, name string, qtype QType) ([]string } if qtype == SRVNoA { - res = append(res, appendScheme(scheme, net.JoinHostPort(rec.Target, resPort))) + // Remove the final dot from rooted DNS names (this is for compatibility with Prometheus) + target := strings.TrimRight(rec.Target, ".") + res = append(res, appendScheme(scheme, net.JoinHostPort(target, resPort))) continue } // Do A lookup for the domain in SRV answer. diff --git a/vendor/github.com/thanos-io/thanos/pkg/query/querier.go b/vendor/github.com/thanos-io/thanos/pkg/query/querier.go index 9a1a311097..9fddae11a5 100644 --- a/vendor/github.com/thanos-io/thanos/pkg/query/querier.go +++ b/vendor/github.com/thanos-io/thanos/pkg/query/querier.go @@ -373,7 +373,7 @@ func (q *querier) selectFn(ctx context.Context, hints *storage.SelectHints, ms . } // LabelValues returns all potential values for a label name. -func (q *querier) LabelValues(ctx context.Context, name string, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { +func (q *querier) LabelValues(ctx context.Context, name string, _ *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { span, ctx := tracing.StartSpan(ctx, "querier_label_values") defer span.Finish() @@ -411,7 +411,7 @@ func (q *querier) LabelValues(ctx context.Context, name string, matchers ...*lab // LabelNames returns all the unique label names present in the block in sorted order constrained // by the given matchers. -func (q *querier) LabelNames(ctx context.Context, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { +func (q *querier) LabelNames(ctx context.Context, _ *storage.LabelHints, matchers ...*labels.Matcher) ([]string, annotations.Annotations, error) { span, ctx := tracing.StartSpan(ctx, "querier_label_names") defer span.Finish() diff --git a/vendor/github.com/thanos-io/thanos/pkg/store/tsdb.go b/vendor/github.com/thanos-io/thanos/pkg/store/tsdb.go index 7e2c472350..b19c131b54 100644 --- a/vendor/github.com/thanos-io/thanos/pkg/store/tsdb.go +++ b/vendor/github.com/thanos-io/thanos/pkg/store/tsdb.go @@ -328,7 +328,7 @@ func (s *TSDBStore) LabelNames(ctx context.Context, r *storepb.LabelNamesRequest } defer runutil.CloseWithLogOnErr(s.logger, q, "close tsdb querier label names") - res, _, err := q.LabelNames(ctx, matchers...) + res, _, err := q.LabelNames(ctx, nil, matchers...) if err != nil { return nil, status.Error(codes.Internal, err.Error()) } @@ -405,7 +405,7 @@ func (s *TSDBStore) LabelValues(ctx context.Context, r *storepb.LabelValuesReque return &storepb.LabelValuesResponse{}, nil } - res, _, err := q.LabelValues(ctx, r.Label, matchers...) + res, _, err := q.LabelValues(ctx, r.Label, nil, matchers...) if err != nil { return nil, status.Error(codes.Internal, err.Error()) } diff --git a/vendor/go.opentelemetry.io/contrib/propagators/autoprop/doc.go b/vendor/go.opentelemetry.io/contrib/propagators/autoprop/doc.go index 0f09e74574..999d78b6b9 100644 --- a/vendor/go.opentelemetry.io/contrib/propagators/autoprop/doc.go +++ b/vendor/go.opentelemetry.io/contrib/propagators/autoprop/doc.go @@ -1,16 +1,5 @@ // Copyright The OpenTelemetry 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. +// SPDX-License-Identifier: Apache-2.0 // Package autoprop provides an OpenTelemetry TextMapPropagator creation // function. The OpenTelemetry specification states that the default diff --git a/vendor/go.opentelemetry.io/contrib/propagators/autoprop/propagator.go b/vendor/go.opentelemetry.io/contrib/propagators/autoprop/propagator.go index 29ac8580a6..4452ce1122 100644 --- a/vendor/go.opentelemetry.io/contrib/propagators/autoprop/propagator.go +++ b/vendor/go.opentelemetry.io/contrib/propagators/autoprop/propagator.go @@ -1,16 +1,5 @@ // Copyright The OpenTelemetry 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. +// SPDX-License-Identifier: Apache-2.0 package autoprop // import "go.opentelemetry.io/contrib/propagators/autoprop" @@ -82,8 +71,8 @@ var errUnknownPropagator = errors.New("unknown propagator") // TextMapPropagator will be returned if "none" is defined anywhere in the // environment variable. func parseEnv() (propagation.TextMapPropagator, error) { - propStrs, defined := os.LookupEnv(otelPropagatorsEnvKey) - if !defined { + propStrs := os.Getenv(otelPropagatorsEnvKey) + if propStrs == "" { return nil, nil } return TextMapPropagator(strings.Split(propStrs, ",")...) diff --git a/vendor/go.opentelemetry.io/contrib/propagators/autoprop/registry.go b/vendor/go.opentelemetry.io/contrib/propagators/autoprop/registry.go index a831773f3d..2cb365aa05 100644 --- a/vendor/go.opentelemetry.io/contrib/propagators/autoprop/registry.go +++ b/vendor/go.opentelemetry.io/contrib/propagators/autoprop/registry.go @@ -1,16 +1,5 @@ // Copyright The OpenTelemetry 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. +// SPDX-License-Identifier: Apache-2.0 package autoprop // import "go.opentelemetry.io/contrib/propagators/autoprop" diff --git a/vendor/go.opentelemetry.io/contrib/propagators/b3/b3_config.go b/vendor/go.opentelemetry.io/contrib/propagators/b3/b3_config.go index 990d9a1251..15a5f7715d 100644 --- a/vendor/go.opentelemetry.io/contrib/propagators/b3/b3_config.go +++ b/vendor/go.opentelemetry.io/contrib/propagators/b3/b3_config.go @@ -1,16 +1,5 @@ // Copyright The OpenTelemetry 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. +// SPDX-License-Identifier: Apache-2.0 package b3 // import "go.opentelemetry.io/contrib/propagators/b3" diff --git a/vendor/go.opentelemetry.io/contrib/propagators/b3/b3_propagator.go b/vendor/go.opentelemetry.io/contrib/propagators/b3/b3_propagator.go index 52b6771034..45d0811a26 100644 --- a/vendor/go.opentelemetry.io/contrib/propagators/b3/b3_propagator.go +++ b/vendor/go.opentelemetry.io/contrib/propagators/b3/b3_propagator.go @@ -1,16 +1,5 @@ // Copyright The OpenTelemetry 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. +// SPDX-License-Identifier: Apache-2.0 package b3 // import "go.opentelemetry.io/contrib/propagators/b3" @@ -273,11 +262,11 @@ func extractSingle(ctx context.Context, contextHeader string) (context.Context, case string(contextHeader[traceID64BitsWidth]) == "-": // traceID must be 64 bits pos += traceID64BitsWidth // {traceID} - traceID = b3TraceIDPadding + string(contextHeader[0:pos]) + traceID = b3TraceIDPadding + contextHeader[0:pos] case string(contextHeader[32]) == "-": // traceID must be 128 bits pos += traceID128BitsWidth // {traceID} - traceID = string(contextHeader[0:pos]) + traceID = contextHeader[0:pos] default: return ctx, empty, errInvalidTraceIDValue } @@ -288,6 +277,9 @@ func extractSingle(ctx context.Context, contextHeader string) (context.Context, } pos += separatorWidth // {traceID}- + if headerLen < pos+spanIDWidth { + return ctx, empty, errInvalidSpanIDValue + } scc.SpanID, err = trace.SpanIDFromHex(contextHeader[pos : pos+spanIDWidth]) if err != nil { return ctx, empty, errInvalidSpanIDValue diff --git a/vendor/go.opentelemetry.io/contrib/propagators/b3/context.go b/vendor/go.opentelemetry.io/contrib/propagators/b3/context.go index 988cd2ca82..7d2ae0d703 100644 --- a/vendor/go.opentelemetry.io/contrib/propagators/b3/context.go +++ b/vendor/go.opentelemetry.io/contrib/propagators/b3/context.go @@ -1,16 +1,5 @@ // Copyright The OpenTelemetry 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. +// SPDX-License-Identifier: Apache-2.0 package b3 // import "go.opentelemetry.io/contrib/propagators/b3" diff --git a/vendor/go.opentelemetry.io/contrib/propagators/b3/doc.go b/vendor/go.opentelemetry.io/contrib/propagators/b3/doc.go index 21578cfbe6..8c3c42a981 100644 --- a/vendor/go.opentelemetry.io/contrib/propagators/b3/doc.go +++ b/vendor/go.opentelemetry.io/contrib/propagators/b3/doc.go @@ -1,16 +1,5 @@ // Copyright The OpenTelemetry 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. +// SPDX-License-Identifier: Apache-2.0 // Package b3 implements the B3 propagator specification as defined at // https://github.com/openzipkin/b3-propagation diff --git a/vendor/go.opentelemetry.io/contrib/propagators/b3/version.go b/vendor/go.opentelemetry.io/contrib/propagators/b3/version.go index d25a15a20b..88f1309f30 100644 --- a/vendor/go.opentelemetry.io/contrib/propagators/b3/version.go +++ b/vendor/go.opentelemetry.io/contrib/propagators/b3/version.go @@ -1,26 +1,17 @@ // Copyright The OpenTelemetry 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. +// SPDX-License-Identifier: Apache-2.0 package b3 // import "go.opentelemetry.io/contrib/propagators/b3" // Version is the current release version of the B3 propagator. func Version() string { - return "1.13.0" + return "1.28.0" // This string is updated by the pre_release.sh script during release } // SemVersion is the semantic version to be supplied to tracer/meter creation. +// +// Deprecated: Use [Version] instead. func SemVersion() string { - return "semver:" + Version() + return Version() } diff --git a/vendor/go.opentelemetry.io/contrib/propagators/jaeger/context.go b/vendor/go.opentelemetry.io/contrib/propagators/jaeger/context.go index 8a62451f74..85473c3bbe 100644 --- a/vendor/go.opentelemetry.io/contrib/propagators/jaeger/context.go +++ b/vendor/go.opentelemetry.io/contrib/propagators/jaeger/context.go @@ -1,16 +1,5 @@ // Copyright The OpenTelemetry 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. +// SPDX-License-Identifier: Apache-2.0 package jaeger // import "go.opentelemetry.io/contrib/propagators/jaeger" diff --git a/vendor/go.opentelemetry.io/contrib/propagators/jaeger/doc.go b/vendor/go.opentelemetry.io/contrib/propagators/jaeger/doc.go index 1d3fb9f89f..9419619842 100644 --- a/vendor/go.opentelemetry.io/contrib/propagators/jaeger/doc.go +++ b/vendor/go.opentelemetry.io/contrib/propagators/jaeger/doc.go @@ -1,16 +1,5 @@ // Copyright The OpenTelemetry 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. +// SPDX-License-Identifier: Apache-2.0 // Package jaeger implements the Jaeger propagator specification as defined at // https://www.jaegertracing.io/docs/1.18/client-libraries/#propagation-format diff --git a/vendor/go.opentelemetry.io/contrib/propagators/jaeger/jaeger_propagator.go b/vendor/go.opentelemetry.io/contrib/propagators/jaeger/jaeger_propagator.go index 08b5faa5d5..76e924f8a0 100644 --- a/vendor/go.opentelemetry.io/contrib/propagators/jaeger/jaeger_propagator.go +++ b/vendor/go.opentelemetry.io/contrib/propagators/jaeger/jaeger_propagator.go @@ -1,16 +1,5 @@ // Copyright The OpenTelemetry 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. +// SPDX-License-Identifier: Apache-2.0 package jaeger // import "go.opentelemetry.io/contrib/propagators/jaeger" diff --git a/vendor/go.opentelemetry.io/contrib/propagators/jaeger/version.go b/vendor/go.opentelemetry.io/contrib/propagators/jaeger/version.go index 401e37d89b..a64230ebaf 100644 --- a/vendor/go.opentelemetry.io/contrib/propagators/jaeger/version.go +++ b/vendor/go.opentelemetry.io/contrib/propagators/jaeger/version.go @@ -1,26 +1,17 @@ // Copyright The OpenTelemetry 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. +// SPDX-License-Identifier: Apache-2.0 package jaeger // import "go.opentelemetry.io/contrib/propagators/jaeger" // Version is the current release version of the Jaeger propagator. func Version() string { - return "1.13.0" + return "1.28.0" // This string is updated by the pre_release.sh script during release } // SemVersion is the semantic version to be supplied to tracer/meter creation. +// +// Deprecated: Use [Version] instead. func SemVersion() string { - return "semver:" + Version() + return Version() } diff --git a/vendor/go.opentelemetry.io/contrib/propagators/ot/doc.go b/vendor/go.opentelemetry.io/contrib/propagators/ot/doc.go index 9e524637c1..b63a1d3511 100644 --- a/vendor/go.opentelemetry.io/contrib/propagators/ot/doc.go +++ b/vendor/go.opentelemetry.io/contrib/propagators/ot/doc.go @@ -1,16 +1,5 @@ // Copyright The OpenTelemetry 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. +// SPDX-License-Identifier: Apache-2.0 // Package ot implements the ot-tracer-* propagator used by the default Tracer // implementation from the OpenTracing project. diff --git a/vendor/go.opentelemetry.io/contrib/propagators/ot/ot_propagator.go b/vendor/go.opentelemetry.io/contrib/propagators/ot/ot_propagator.go index 88366f3c26..765bf2a5bb 100644 --- a/vendor/go.opentelemetry.io/contrib/propagators/ot/ot_propagator.go +++ b/vendor/go.opentelemetry.io/contrib/propagators/ot/ot_propagator.go @@ -1,16 +1,5 @@ // Copyright The OpenTelemetry 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. +// SPDX-License-Identifier: Apache-2.0 package ot // import "go.opentelemetry.io/contrib/propagators/ot" @@ -49,8 +38,7 @@ var ( ) // OT propagator serializes SpanContext to/from ot-trace-* headers. -type OT struct { -} +type OT struct{} var _ propagation.TextMapPropagator = OT{} diff --git a/vendor/go.opentelemetry.io/contrib/propagators/ot/version.go b/vendor/go.opentelemetry.io/contrib/propagators/ot/version.go index e683ffc500..3c726b1896 100644 --- a/vendor/go.opentelemetry.io/contrib/propagators/ot/version.go +++ b/vendor/go.opentelemetry.io/contrib/propagators/ot/version.go @@ -1,26 +1,17 @@ // Copyright The OpenTelemetry 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. +// SPDX-License-Identifier: Apache-2.0 package ot // import "go.opentelemetry.io/contrib/propagators/ot" // Version is the current release version of the ot propagator. func Version() string { - return "1.13.0" + return "1.28.0" // This string is updated by the pre_release.sh script during release } // SemVersion is the semantic version to be supplied to tracer/meter creation. +// +// Deprecated: Use [Version] instead. func SemVersion() string { - return "semver:" + Version() + return Version() } diff --git a/vendor/golang.org/x/exp/slices/sort.go b/vendor/golang.org/x/exp/slices/sort.go index b67897f76b..f58bbc7ba4 100644 --- a/vendor/golang.org/x/exp/slices/sort.go +++ b/vendor/golang.org/x/exp/slices/sort.go @@ -22,10 +22,12 @@ func Sort[S ~[]E, E constraints.Ordered](x S) { // SortFunc sorts the slice x in ascending order as determined by the cmp // function. This sort is not guaranteed to be stable. // cmp(a, b) should return a negative number when a < b, a positive number when -// a > b and zero when a == b. +// a > b and zero when a == b or when a is not comparable to b in the sense +// of the formal definition of Strict Weak Ordering. // // SortFunc requires that cmp is a strict weak ordering. // See https://en.wikipedia.org/wiki/Weak_ordering#Strict_weak_orderings. +// To indicate 'uncomparable', return 0 from the function. func SortFunc[S ~[]E, E any](x S, cmp func(a, b E) int) { n := len(x) pdqsortCmpFunc(x, 0, n, bits.Len(uint(n)), cmp) diff --git a/vendor/gonum.org/v1/gonum/AUTHORS b/vendor/gonum.org/v1/gonum/AUTHORS index fbff74ec59..932c529804 100644 --- a/vendor/gonum.org/v1/gonum/AUTHORS +++ b/vendor/gonum.org/v1/gonum/AUTHORS @@ -10,12 +10,17 @@ Alexander Egurnov Andrei Blinnikov +antichris +Bailey Lissington Bill Gray Bill Noon Brendan Tracey Brent Pedersen +Bulat Khasanov Chad Kunde +Chan Kwan Yin Chih-Wei Chang +Chong-Yeol Nah Chris Tessum Christophe Meessen Christopher Waldon @@ -33,7 +38,9 @@ Dong-hee Na Dustin Spicuzza Egon Elbre Ekaterina Efimova +Eng Zer Jun Ethan Burns +Ethan Reesor Evert Lammerts Evgeny Savinov Fabian Wickborn @@ -52,13 +59,16 @@ James Bowman James Holmes <32bitkid@gmail.com> Janne Snabb Jeremy Atkinson +Jes Cok Jinesi Yelizati Jonas Kahler Jonas Schulze +Jonathan Bluett-Duncan Jonathan J Lawlor Jonathan Reiter Jonathan Schroeder Joost van Amersfoort +Jordan Stoker Joseph Watson Josh Wilson Julien Roland @@ -118,4 +128,5 @@ Will Tekulve Yasuhiro Matsumoto Yevgeniy Vahlis Yucheng Zhu +Yunomi Zoe Juozapaitis diff --git a/vendor/gonum.org/v1/gonum/CONTRIBUTORS b/vendor/gonum.org/v1/gonum/CONTRIBUTORS index baacd28c88..711ad614bd 100644 --- a/vendor/gonum.org/v1/gonum/CONTRIBUTORS +++ b/vendor/gonum.org/v1/gonum/CONTRIBUTORS @@ -18,12 +18,17 @@ Alexander Egurnov Andrei Blinnikov Andrew Brampton +antichris +Bailey Lissington Bill Gray Bill Noon Brendan Tracey Brent Pedersen +Bulat Khasanov Chad Kunde +Chan Kwan Yin Chih-Wei Chang +Chong-Yeol Nah Chris Tessum Christophe Meessen Christopher Waldon @@ -41,7 +46,9 @@ Dong-hee Na Dustin Spicuzza Egon Elbre Ekaterina Efimova +Eng Zer Jun Ethan Burns +Ethan Reesor Evert Lammerts Evgeny Savinov Fabian Wickborn @@ -59,14 +66,17 @@ James Bowman James Holmes <32bitkid@gmail.com> Janne Snabb Jeremy Atkinson +Jes Cok Jinesi Yelizati Jon Richards Jonas Kahler Jonas Schulze +Jonathan Bluett-Duncan Jonathan J Lawlor Jonathan Reiter Jonathan Schroeder Joost van Amersfoort +Jordan Stoker Joseph Watson Josh Wilson Julien Roland @@ -121,4 +131,5 @@ Will Tekulve Yasuhiro Matsumoto Yevgeniy Vahlis Yucheng Zhu +Yunomi Zoe Juozapaitis diff --git a/vendor/gonum.org/v1/gonum/floats/README.md b/vendor/gonum.org/v1/gonum/floats/README.md index ee867bb7bf..e8ef46d567 100644 --- a/vendor/gonum.org/v1/gonum/floats/README.md +++ b/vendor/gonum.org/v1/gonum/floats/README.md @@ -1,4 +1,7 @@ -# Gonum floats [![GoDoc](https://godoc.org/gonum.org/v1/gonum/floats?status.svg)](https://godoc.org/gonum.org/v1/gonum/floats) +# Gonum floats + +[![go.dev reference](https://pkg.go.dev/badge/gonum.org/v1/gonum/floats)](https://pkg.go.dev/gonum.org/v1/gonum/floats) +[![GoDoc](https://godocs.io/gonum.org/v1/gonum/floats?status.svg)](https://godocs.io/gonum.org/v1/gonum/floats) Package floats provides a set of helper routines for dealing with slices of float64. The functions avoid allocations to allow for use within tight loops without garbage collection overhead. diff --git a/vendor/k8s.io/klog/v2/klog.go b/vendor/k8s.io/klog/v2/klog.go index 026be9e3b1..47ec9466a6 100644 --- a/vendor/k8s.io/klog/v2/klog.go +++ b/vendor/k8s.io/klog/v2/klog.go @@ -404,13 +404,6 @@ func (t *traceLocation) Set(value string) error { return nil } -// flushSyncWriter is the interface satisfied by logging destinations. -type flushSyncWriter interface { - Flush() error - Sync() error - io.Writer -} - var logging loggingT var commandLine flag.FlagSet @@ -486,7 +479,7 @@ type settings struct { // Access to all of the following fields must be protected via a mutex. // file holds writer for each of the log types. - file [severity.NumSeverity]flushSyncWriter + file [severity.NumSeverity]io.Writer // flushInterval is the interval for periodic flushing. If zero, // the global default will be used. flushInterval time.Duration @@ -831,32 +824,12 @@ func (l *loggingT) printS(err error, s severity.Severity, depth int, msg string, buffer.PutBuffer(b) } -// redirectBuffer is used to set an alternate destination for the logs -type redirectBuffer struct { - w io.Writer -} - -func (rb *redirectBuffer) Sync() error { - return nil -} - -func (rb *redirectBuffer) Flush() error { - return nil -} - -func (rb *redirectBuffer) Write(bytes []byte) (n int, err error) { - return rb.w.Write(bytes) -} - // SetOutput sets the output destination for all severities func SetOutput(w io.Writer) { logging.mu.Lock() defer logging.mu.Unlock() for s := severity.FatalLog; s >= severity.InfoLog; s-- { - rb := &redirectBuffer{ - w: w, - } - logging.file[s] = rb + logging.file[s] = w } } @@ -868,10 +841,7 @@ func SetOutputBySeverity(name string, w io.Writer) { if !ok { panic(fmt.Sprintf("SetOutputBySeverity(%q): unrecognized severity name", name)) } - rb := &redirectBuffer{ - w: w, - } - logging.file[sev] = rb + logging.file[sev] = w } // LogToStderr sets whether to log exclusively to stderr, bypassing outputs @@ -1011,7 +981,8 @@ func (l *loggingT) exit(err error) { logExitFunc(err) return } - l.flushAll() + needToSync := l.flushAll() + l.syncAll(needToSync) OsExit(2) } @@ -1028,10 +999,6 @@ type syncBuffer struct { maxbytes uint64 // The max number of bytes this syncBuffer.file can hold before cleaning up. } -func (sb *syncBuffer) Sync() error { - return sb.file.Sync() -} - // CalculateMaxSize returns the real max size in bytes after considering the default max size and the flag options. func CalculateMaxSize() uint64 { if logging.logFile != "" { @@ -1223,24 +1190,45 @@ func StartFlushDaemon(interval time.Duration) { // lockAndFlushAll is like flushAll but locks l.mu first. func (l *loggingT) lockAndFlushAll() { l.mu.Lock() - l.flushAll() + needToSync := l.flushAll() l.mu.Unlock() + // Some environments are slow when syncing and holding the lock might cause contention. + l.syncAll(needToSync) } -// flushAll flushes all the logs and attempts to "sync" their data to disk. +// flushAll flushes all the logs // l.mu is held. -func (l *loggingT) flushAll() { +// +// The result is the number of files which need to be synced and the pointers to them. +func (l *loggingT) flushAll() fileArray { + var needToSync fileArray + // Flush from fatal down, in case there's trouble flushing. for s := severity.FatalLog; s >= severity.InfoLog; s-- { file := l.file[s] - if file != nil { - _ = file.Flush() // ignore error - _ = file.Sync() // ignore error + if sb, ok := file.(*syncBuffer); ok && sb.file != nil { + _ = sb.Flush() // ignore error + needToSync.files[needToSync.num] = sb.file + needToSync.num++ } } if logging.loggerOptions.flush != nil { logging.loggerOptions.flush() } + return needToSync +} + +type fileArray struct { + num int + files [severity.NumSeverity]*os.File +} + +// syncAll attempts to "sync" their data to disk. +func (l *loggingT) syncAll(needToSync fileArray) { + // Flush from fatal down, in case there's trouble flushing. + for i := 0; i < needToSync.num; i++ { + _ = needToSync.files[i].Sync() // ignore error + } } // CopyStandardLogTo arranges for messages written to the Go "log" package's diff --git a/vendor/modules.txt b/vendor/modules.txt index ed6c882669..16fe5ec6f8 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -36,7 +36,7 @@ cloud.google.com/go/storage cloud.google.com/go/storage/internal cloud.google.com/go/storage/internal/apiv2 cloud.google.com/go/storage/internal/apiv2/storagepb -# github.com/Azure/azure-sdk-for-go/sdk/azcore v1.11.1 +# github.com/Azure/azure-sdk-for-go/sdk/azcore v1.12.0 ## explicit; go 1.18 github.com/Azure/azure-sdk-for-go/sdk/azcore github.com/Azure/azure-sdk-for-go/sdk/azcore/arm/internal/resource @@ -58,11 +58,11 @@ github.com/Azure/azure-sdk-for-go/sdk/azcore/runtime github.com/Azure/azure-sdk-for-go/sdk/azcore/streaming github.com/Azure/azure-sdk-for-go/sdk/azcore/to github.com/Azure/azure-sdk-for-go/sdk/azcore/tracing -# github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.6.0 +# github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.7.0 ## explicit; go 1.18 github.com/Azure/azure-sdk-for-go/sdk/azidentity github.com/Azure/azure-sdk-for-go/sdk/azidentity/internal -# github.com/Azure/azure-sdk-for-go/sdk/internal v1.8.0 +# github.com/Azure/azure-sdk-for-go/sdk/internal v1.9.0 ## explicit; go 1.18 github.com/Azure/azure-sdk-for-go/sdk/internal/diag github.com/Azure/azure-sdk-for-go/sdk/internal/errorinfo @@ -302,7 +302,7 @@ github.com/coreos/go-semver/semver ## explicit; go 1.12 github.com/coreos/go-systemd/v22/activation github.com/coreos/go-systemd/v22/journal -# github.com/cortexproject/promqlsmith v0.0.0-20240328172224-5e341f0dd08e +# github.com/cortexproject/promqlsmith v0.0.0-20240506042652-6cfdd9739a5e ## explicit; go 1.20 github.com/cortexproject/promqlsmith # github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc @@ -824,8 +824,8 @@ github.com/prometheus/exporter-toolkit/web github.com/prometheus/procfs github.com/prometheus/procfs/internal/fs github.com/prometheus/procfs/internal/util -# github.com/prometheus/prometheus v0.53.1-0.20240625160125-1abeebacb870 -## explicit; go 1.21 +# github.com/prometheus/prometheus v0.53.2-0.20240718123124-e9dec5fc537b +## explicit; go 1.21.0 github.com/prometheus/prometheus/config github.com/prometheus/prometheus/discovery github.com/prometheus/prometheus/discovery/dns @@ -844,6 +844,7 @@ github.com/prometheus/prometheus/model/value github.com/prometheus/prometheus/notifier github.com/prometheus/prometheus/prompb github.com/prometheus/prometheus/prompb/io/prometheus/client +github.com/prometheus/prometheus/prompb/io/prometheus/write/v2 github.com/prometheus/prometheus/promql github.com/prometheus/prometheus/promql/parser github.com/prometheus/prometheus/promql/parser/posrange @@ -941,8 +942,8 @@ github.com/thanos-io/objstore/providers/gcs github.com/thanos-io/objstore/providers/s3 github.com/thanos-io/objstore/providers/swift github.com/thanos-io/objstore/tracing/opentracing -# github.com/thanos-io/promql-engine v0.0.0-20240515161521-93aa311933cf -## explicit; go 1.21 +# github.com/thanos-io/promql-engine v0.0.0-20240718195911-cdbd6dfed36b +## explicit; go 1.22.0 github.com/thanos-io/promql-engine/api github.com/thanos-io/promql-engine/engine github.com/thanos-io/promql-engine/execution @@ -964,8 +965,8 @@ github.com/thanos-io/promql-engine/query github.com/thanos-io/promql-engine/ringbuffer github.com/thanos-io/promql-engine/storage github.com/thanos-io/promql-engine/storage/prometheus -# github.com/thanos-io/thanos v0.36.0-rc.0 -## explicit; go 1.21 +# github.com/thanos-io/thanos v0.35.2-0.20240722172812-990a60b72647 +## explicit; go 1.22.0 github.com/thanos-io/thanos/pkg/api/query/querypb github.com/thanos-io/thanos/pkg/block github.com/thanos-io/thanos/pkg/block/indexheader @@ -1144,8 +1145,8 @@ go.opentelemetry.io/collector/pdata/internal/otlp go.opentelemetry.io/collector/pdata/pcommon go.opentelemetry.io/collector/pdata/pmetric go.opentelemetry.io/collector/pdata/pmetric/pmetricotlp -# go.opentelemetry.io/collector/semconv v0.101.0 -## explicit; go 1.21 +# go.opentelemetry.io/collector/semconv v0.104.0 +## explicit; go 1.21.0 go.opentelemetry.io/collector/semconv/v1.6.1 # go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.49.0 ## explicit; go 1.20 @@ -1156,20 +1157,20 @@ go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc/inte go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp/internal/semconv go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp/internal/semconvutil -# go.opentelemetry.io/contrib/propagators/autoprop v0.38.0 -## explicit; go 1.18 +# go.opentelemetry.io/contrib/propagators/autoprop v0.53.0 +## explicit; go 1.21 go.opentelemetry.io/contrib/propagators/autoprop # go.opentelemetry.io/contrib/propagators/aws v1.28.0 ## explicit; go 1.21 go.opentelemetry.io/contrib/propagators/aws/xray -# go.opentelemetry.io/contrib/propagators/b3 v1.13.0 -## explicit; go 1.18 +# go.opentelemetry.io/contrib/propagators/b3 v1.28.0 +## explicit; go 1.21 go.opentelemetry.io/contrib/propagators/b3 -# go.opentelemetry.io/contrib/propagators/jaeger v1.13.0 -## explicit; go 1.18 +# go.opentelemetry.io/contrib/propagators/jaeger v1.28.0 +## explicit; go 1.21 go.opentelemetry.io/contrib/propagators/jaeger -# go.opentelemetry.io/contrib/propagators/ot v1.13.0 -## explicit; go 1.18 +# go.opentelemetry.io/contrib/propagators/ot v1.28.0 +## explicit; go 1.21 go.opentelemetry.io/contrib/propagators/ot # go.opentelemetry.io/otel v1.28.0 ## explicit; go 1.21 @@ -1268,7 +1269,7 @@ golang.org/x/crypto/internal/alias golang.org/x/crypto/internal/poly1305 golang.org/x/crypto/pkcs12 golang.org/x/crypto/pkcs12/internal/rc2 -# golang.org/x/exp v0.0.0-20240119083558-1b970713d09a +# golang.org/x/exp v0.0.0-20240613232115-7f521ea00fb8 ## explicit; go 1.20 golang.org/x/exp/constraints golang.org/x/exp/maps @@ -1352,8 +1353,8 @@ golang.org/x/tools/internal/stdlib golang.org/x/tools/internal/tokeninternal golang.org/x/tools/internal/typesinternal golang.org/x/tools/internal/versions -# gonum.org/v1/gonum v0.12.0 -## explicit; go 1.18 +# gonum.org/v1/gonum v0.15.0 +## explicit; go 1.21 gonum.org/v1/gonum/floats gonum.org/v1/gonum/floats/scalar gonum.org/v1/gonum/internal/asm/f64 @@ -1515,14 +1516,14 @@ gopkg.in/yaml.v2 # gopkg.in/yaml.v3 v3.0.1 ## explicit gopkg.in/yaml.v3 -# k8s.io/apimachinery v0.29.3 -## explicit; go 1.21 +# k8s.io/apimachinery v0.30.2 +## explicit; go 1.22.0 k8s.io/apimachinery/pkg/util/runtime -# k8s.io/client-go v0.29.3 -## explicit; go 1.21 +# k8s.io/client-go v0.30.2 +## explicit; go 1.22.0 k8s.io/client-go/tools/metrics k8s.io/client-go/util/workqueue -# k8s.io/klog/v2 v2.120.1 +# k8s.io/klog/v2 v2.130.1 ## explicit; go 1.18 k8s.io/klog/v2 k8s.io/klog/v2/internal/buffer @@ -1531,7 +1532,7 @@ k8s.io/klog/v2/internal/dbg k8s.io/klog/v2/internal/serialize k8s.io/klog/v2/internal/severity k8s.io/klog/v2/internal/sloghandler -# k8s.io/utils v0.0.0-20230726121419-3b25d923346b +# k8s.io/utils v0.0.0-20240502163921-fe8a2dddb1d0 ## explicit; go 1.18 k8s.io/utils/clock # git.apache.org/thrift.git => github.com/apache/thrift v0.0.0-20180902110319-2566ecd5d999 From 3ae12a298dc52ce062e9505ab3b44f1f34e09530 Mon Sep 17 00:00:00 2001 From: Alex Le Date: Tue, 23 Jul 2024 11:54:18 -0700 Subject: [PATCH 10/32] Centralize metrics used by compactor and add user label to compactor metrics (#6096) * Centralize metrics used by compactor and add user label to compactor metrics. Signed-off-by: Alex Le * Updated CHANGELOG Signed-off-by: Alex Le * addressed comments Signed-off-by: Alex Le * Added back missing metric Signed-off-by: Alex Le --------- Signed-off-by: Alex Le --- CHANGELOG.md | 1 + pkg/compactor/blocks_cleaner.go | 25 +- pkg/compactor/blocks_cleaner_test.go | 47 +++- pkg/compactor/compactor.go | 173 +++++++------ pkg/compactor/compactor_metrics.go | 199 +++++++++++++++ pkg/compactor/compactor_metrics_test.go | 166 +++++++++++++ pkg/compactor/compactor_test.go | 180 +++----------- pkg/compactor/shuffle_sharding_grouper.go | 95 +++---- .../shuffle_sharding_grouper_test.go | 66 +++-- pkg/compactor/syncer_metrics.go | 124 ---------- pkg/compactor/syncer_metrics_test.go | 232 ------------------ 11 files changed, 596 insertions(+), 712 deletions(-) create mode 100644 pkg/compactor/compactor_metrics.go create mode 100644 pkg/compactor/compactor_metrics_test.go delete mode 100644 pkg/compactor/syncer_metrics.go delete mode 100644 pkg/compactor/syncer_metrics_test.go diff --git a/CHANGELOG.md b/CHANGELOG.md index efa0a139f7..1a9c050e98 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -27,6 +27,7 @@ * [ENHANCEMENT] Ruler: Add support for filtering by `state` and `health` field on Rules API. #6040 * [ENHANCEMENT] Ruler: Add support for filtering by `match` field on Rules API. #6083 * [ENHANCEMENT] Distributor: Reduce memory usage when error volume is high. #6095 +* [ENHANCEMENT] Compactor: Centralize metrics used by compactor and add user label to compactor metrics. #6096 * [ENHANCEMENT] Compactor: Add unique execution ID for each compaction cycle in log for easy debugging. #6097 * [ENHANCEMENT] Ruler: Add support for filtering by `state` and `health` field on Rules API. #6040 * [BUGFIX] Configsdb: Fix endline issue in db password. #5920 diff --git a/pkg/compactor/blocks_cleaner.go b/pkg/compactor/blocks_cleaner.go index d1a81f401c..a9c4fa7f3d 100644 --- a/pkg/compactor/blocks_cleaner.go +++ b/pkg/compactor/blocks_cleaner.go @@ -27,6 +27,7 @@ import ( const ( defaultDeleteBlocksConcurrency = 16 + reasonValueRetention = "retention" ) type BlocksCleanerConfig struct { @@ -56,7 +57,7 @@ type BlocksCleaner struct { runsLastSuccess prometheus.Gauge blocksCleanedTotal prometheus.Counter blocksFailedTotal prometheus.Counter - blocksMarkedForDeletion prometheus.Counter + blocksMarkedForDeletion *prometheus.CounterVec tenantBlocks *prometheus.GaugeVec tenantBlocksMarkedForDelete *prometheus.GaugeVec tenantBlocksMarkedForNoCompaction *prometheus.GaugeVec @@ -64,7 +65,15 @@ type BlocksCleaner struct { tenantBucketIndexLastUpdate *prometheus.GaugeVec } -func NewBlocksCleaner(cfg BlocksCleanerConfig, bucketClient objstore.InstrumentedBucket, usersScanner *cortex_tsdb.UsersScanner, cfgProvider ConfigProvider, logger log.Logger, reg prometheus.Registerer) *BlocksCleaner { +func NewBlocksCleaner( + cfg BlocksCleanerConfig, + bucketClient objstore.InstrumentedBucket, + usersScanner *cortex_tsdb.UsersScanner, + cfgProvider ConfigProvider, + logger log.Logger, + reg prometheus.Registerer, + blocksMarkedForDeletion *prometheus.CounterVec, +) *BlocksCleaner { c := &BlocksCleaner{ cfg: cfg, bucketClient: bucketClient, @@ -95,11 +104,7 @@ func NewBlocksCleaner(cfg BlocksCleanerConfig, bucketClient objstore.Instrumente Name: "cortex_compactor_block_cleanup_failures_total", Help: "Total number of blocks failed to be deleted.", }), - blocksMarkedForDeletion: promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Name: blocksMarkedForDeletionName, - Help: blocksMarkedForDeletionHelp, - ConstLabels: prometheus.Labels{"reason": "retention"}, - }), + blocksMarkedForDeletion: blocksMarkedForDeletion, // The following metrics don't have the "cortex_compactor" prefix because not strictly related to // the compactor. They're just tracked by the compactor because it's the most logical place where these @@ -374,7 +379,7 @@ func (c *BlocksCleaner) cleanUser(ctx context.Context, userID string, firstRun b // We do not want to stop the remaining work in the cleaner if an // error occurs here. Errors are logged in the function. retention := c.cfgProvider.CompactorBlocksRetentionPeriod(userID) - c.applyUserRetentionPeriod(ctx, idx, retention, userBucket, userLogger) + c.applyUserRetentionPeriod(ctx, idx, retention, userBucket, userLogger, userID) } // Generate an updated in-memory version of the bucket index. @@ -498,7 +503,7 @@ func (c *BlocksCleaner) cleanUserPartialBlocks(ctx context.Context, partials map } // applyUserRetentionPeriod marks blocks for deletion which have aged past the retention period. -func (c *BlocksCleaner) applyUserRetentionPeriod(ctx context.Context, idx *bucketindex.Index, retention time.Duration, userBucket objstore.Bucket, userLogger log.Logger) { +func (c *BlocksCleaner) applyUserRetentionPeriod(ctx context.Context, idx *bucketindex.Index, retention time.Duration, userBucket objstore.Bucket, userLogger log.Logger, userID string) { // The retention period of zero is a special value indicating to never delete. if retention <= 0 { return @@ -511,7 +516,7 @@ func (c *BlocksCleaner) applyUserRetentionPeriod(ctx context.Context, idx *bucke // the cleaner will retry applying the retention in its next cycle. for _, b := range blocks { level.Info(userLogger).Log("msg", "applied retention: marking block for deletion", "block", b.ID, "maxTime", b.MaxTime) - if err := block.MarkForDeletion(ctx, userLogger, userBucket, b.ID, fmt.Sprintf("block exceeding retention of %v", retention), c.blocksMarkedForDeletion); err != nil { + if err := block.MarkForDeletion(ctx, userLogger, userBucket, b.ID, fmt.Sprintf("block exceeding retention of %v", retention), c.blocksMarkedForDeletion.WithLabelValues(userID, reasonValueRetention)); err != nil { level.Warn(userLogger).Log("msg", "failed to mark block for deletion", "block", b.ID, "err", err) } } diff --git a/pkg/compactor/blocks_cleaner_test.go b/pkg/compactor/blocks_cleaner_test.go index 98f9565fd1..e33994df8b 100644 --- a/pkg/compactor/blocks_cleaner_test.go +++ b/pkg/compactor/blocks_cleaner_test.go @@ -12,6 +12,7 @@ import ( "github.com/go-kit/log" "github.com/oklog/ulid" "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" "github.com/prometheus/client_golang/prometheus/testutil" prom_testutil "github.com/prometheus/client_golang/prometheus/testutil" "github.com/stretchr/testify/assert" @@ -79,8 +80,12 @@ func TestBlockCleaner_KeyPermissionDenied(t *testing.T) { logger := log.NewNopLogger() scanner := tsdb.NewUsersScanner(mbucket, tsdb.AllUsers, logger) cfgProvider := newMockConfigProvider() + blocksMarkedForDeletion := prometheus.NewCounterVec(prometheus.CounterOpts{ + Name: blocksMarkedForDeletionName, + Help: blocksMarkedForDeletionHelp, + }, append(commonLabels, ReasonLabelName)) - cleaner := NewBlocksCleaner(cfg, mbucket, scanner, cfgProvider, logger, nil) + cleaner := NewBlocksCleaner(cfg, mbucket, scanner, cfgProvider, logger, nil, blocksMarkedForDeletion) // Clean User with no error cleaner.bucketClient = bkt @@ -176,8 +181,12 @@ func testBlocksCleanerWithOptions(t *testing.T, options testBlocksCleanerOptions logger := log.NewNopLogger() scanner := tsdb.NewUsersScanner(bucketClient, tsdb.AllUsers, logger) cfgProvider := newMockConfigProvider() + blocksMarkedForDeletion := promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Name: blocksMarkedForDeletionName, + Help: blocksMarkedForDeletionHelp, + }, append(commonLabels, ReasonLabelName)) - cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, reg) + cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, reg, blocksMarkedForDeletion) require.NoError(t, services.StartAndAwaitRunning(ctx, cleaner)) defer services.StopAndAwaitTerminated(ctx, cleaner) //nolint:errcheck @@ -333,8 +342,12 @@ func TestBlocksCleaner_ShouldContinueOnBlockDeletionFailure(t *testing.T) { logger := log.NewNopLogger() scanner := tsdb.NewUsersScanner(bucketClient, tsdb.AllUsers, logger) cfgProvider := newMockConfigProvider() + blocksMarkedForDeletion := prometheus.NewCounterVec(prometheus.CounterOpts{ + Name: blocksMarkedForDeletionName, + Help: blocksMarkedForDeletionHelp, + }, append(commonLabels, ReasonLabelName)) - cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, nil) + cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, nil, blocksMarkedForDeletion) require.NoError(t, services.StartAndAwaitRunning(ctx, cleaner)) defer services.StopAndAwaitTerminated(ctx, cleaner) //nolint:errcheck @@ -393,8 +406,12 @@ func TestBlocksCleaner_ShouldRebuildBucketIndexOnCorruptedOne(t *testing.T) { logger := log.NewNopLogger() scanner := tsdb.NewUsersScanner(bucketClient, tsdb.AllUsers, logger) cfgProvider := newMockConfigProvider() + blocksMarkedForDeletion := prometheus.NewCounterVec(prometheus.CounterOpts{ + Name: blocksMarkedForDeletionName, + Help: blocksMarkedForDeletionHelp, + }, append(commonLabels, ReasonLabelName)) - cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, nil) + cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, nil, blocksMarkedForDeletion) require.NoError(t, services.StartAndAwaitRunning(ctx, cleaner)) defer services.StopAndAwaitTerminated(ctx, cleaner) //nolint:errcheck @@ -447,8 +464,12 @@ func TestBlocksCleaner_ShouldRemoveMetricsForTenantsNotBelongingAnymoreToTheShar reg := prometheus.NewPedanticRegistry() scanner := tsdb.NewUsersScanner(bucketClient, tsdb.AllUsers, logger) cfgProvider := newMockConfigProvider() + blocksMarkedForDeletion := promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Name: blocksMarkedForDeletionName, + Help: blocksMarkedForDeletionHelp, + }, append(commonLabels, ReasonLabelName)) - cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, reg) + cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, reg, blocksMarkedForDeletion) require.NoError(t, cleaner.cleanUsers(ctx, true)) assert.NoError(t, prom_testutil.GatherAndCompare(reg, strings.NewReader(` @@ -578,8 +599,12 @@ func TestBlocksCleaner_ShouldRemoveBlocksOutsideRetentionPeriod(t *testing.T) { reg := prometheus.NewPedanticRegistry() scanner := tsdb.NewUsersScanner(bucketClient, tsdb.AllUsers, logger) cfgProvider := newMockConfigProvider() + blocksMarkedForDeletion := promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Name: blocksMarkedForDeletionName, + Help: blocksMarkedForDeletionHelp, + }, append(commonLabels, ReasonLabelName)) - cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, reg) + cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, reg, blocksMarkedForDeletion) assertBlockExists := func(user string, block ulid.ULID, expectExists bool) { exists, err := bucketClient.Exists(ctx, path.Join(user, block.String(), metadata.MetaFilename)) @@ -607,9 +632,6 @@ func TestBlocksCleaner_ShouldRemoveBlocksOutsideRetentionPeriod(t *testing.T) { # TYPE cortex_bucket_blocks_marked_for_deletion_count gauge cortex_bucket_blocks_marked_for_deletion_count{user="user-1"} 0 cortex_bucket_blocks_marked_for_deletion_count{user="user-2"} 0 - # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE cortex_compactor_blocks_marked_for_deletion_total counter - cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 `), "cortex_bucket_blocks_count", "cortex_bucket_blocks_marked_for_deletion_count", @@ -650,7 +672,7 @@ func TestBlocksCleaner_ShouldRemoveBlocksOutsideRetentionPeriod(t *testing.T) { cortex_bucket_blocks_marked_for_deletion_count{user="user-2"} 0 # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. # TYPE cortex_compactor_blocks_marked_for_deletion_total counter - cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 1 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention",user="user-1"} 1 `), "cortex_bucket_blocks_count", "cortex_bucket_blocks_marked_for_deletion_count", @@ -688,7 +710,7 @@ func TestBlocksCleaner_ShouldRemoveBlocksOutsideRetentionPeriod(t *testing.T) { cortex_bucket_blocks_marked_for_deletion_count{user="user-2"} 0 # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. # TYPE cortex_compactor_blocks_marked_for_deletion_total counter - cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 1 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention",user="user-1"} 1 `), "cortex_bucket_blocks_count", "cortex_bucket_blocks_marked_for_deletion_count", @@ -717,7 +739,8 @@ func TestBlocksCleaner_ShouldRemoveBlocksOutsideRetentionPeriod(t *testing.T) { cortex_bucket_blocks_marked_for_deletion_count{user="user-2"} 0 # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. # TYPE cortex_compactor_blocks_marked_for_deletion_total counter - cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 3 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention",user="user-1"} 1 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention",user="user-2"} 2 `), "cortex_bucket_blocks_count", "cortex_bucket_blocks_marked_for_deletion_count", diff --git a/pkg/compactor/compactor.go b/pkg/compactor/compactor.go index 3bbe673a0b..7eac246c4d 100644 --- a/pkg/compactor/compactor.go +++ b/pkg/compactor/compactor.go @@ -57,34 +57,36 @@ var ( errInvalidShardingStrategy = errors.New("invalid sharding strategy") errInvalidTenantShardSize = errors.New("invalid tenant shard size, the value must be greater than 0") - DefaultBlocksGrouperFactory = func(ctx context.Context, cfg Config, bkt objstore.InstrumentedBucket, logger log.Logger, reg prometheus.Registerer, blocksMarkedForDeletion, blocksMarkedForNoCompaction, garbageCollectedBlocks prometheus.Counter, _ prometheus.Gauge, _ prometheus.Counter, _ prometheus.Counter, _ *ring.Ring, _ *ring.Lifecycler, _ Limits, _ string, _ *compact.GatherNoCompactionMarkFilter) compact.Grouper { - return compact.NewDefaultGrouper( + DefaultBlocksGrouperFactory = func(ctx context.Context, cfg Config, bkt objstore.InstrumentedBucket, logger log.Logger, blocksMarkedForNoCompaction prometheus.Counter, _ prometheus.Counter, _ prometheus.Counter, syncerMetrics *compact.SyncerMetrics, compactorMetrics *compactorMetrics, _ *ring.Ring, _ *ring.Lifecycler, _ Limits, _ string, _ *compact.GatherNoCompactionMarkFilter) compact.Grouper { + return compact.NewDefaultGrouperWithMetrics( logger, bkt, cfg.AcceptMalformedIndex, true, // Enable vertical compaction - reg, - blocksMarkedForDeletion, - garbageCollectedBlocks, + compactorMetrics.compactions, + compactorMetrics.compactionRunsStarted, + compactorMetrics.compactionRunsCompleted, + compactorMetrics.compactionFailures, + compactorMetrics.verticalCompactions, + syncerMetrics.BlocksMarkedForDeletion, + syncerMetrics.GarbageCollectedBlocks, blocksMarkedForNoCompaction, metadata.NoneFunc, cfg.BlockFilesConcurrency, cfg.BlocksFetchConcurrency) } - ShuffleShardingGrouperFactory = func(ctx context.Context, cfg Config, bkt objstore.InstrumentedBucket, logger log.Logger, reg prometheus.Registerer, blocksMarkedForDeletion, blocksMarkedForNoCompaction, garbageCollectedBlocks prometheus.Counter, remainingPlannedCompactions prometheus.Gauge, blockVisitMarkerReadFailed prometheus.Counter, blockVisitMarkerWriteFailed prometheus.Counter, ring *ring.Ring, ringLifecycle *ring.Lifecycler, limits Limits, userID string, noCompactionMarkFilter *compact.GatherNoCompactionMarkFilter) compact.Grouper { + ShuffleShardingGrouperFactory = func(ctx context.Context, cfg Config, bkt objstore.InstrumentedBucket, logger log.Logger, blocksMarkedForNoCompaction prometheus.Counter, blockVisitMarkerReadFailed prometheus.Counter, blockVisitMarkerWriteFailed prometheus.Counter, syncerMetrics *compact.SyncerMetrics, compactorMetrics *compactorMetrics, ring *ring.Ring, ringLifecycle *ring.Lifecycler, limits Limits, userID string, noCompactionMarkFilter *compact.GatherNoCompactionMarkFilter) compact.Grouper { return NewShuffleShardingGrouper( ctx, logger, bkt, cfg.AcceptMalformedIndex, true, // Enable vertical compaction - reg, - blocksMarkedForDeletion, blocksMarkedForNoCompaction, - garbageCollectedBlocks, - remainingPlannedCompactions, metadata.NoneFunc, + syncerMetrics, + compactorMetrics, cfg, ring, ringLifecycle.Addr, @@ -106,7 +108,7 @@ var ( return nil, nil, err } - plannerFactory := func(ctx context.Context, bkt objstore.InstrumentedBucket, logger log.Logger, cfg Config, noCompactionMarkFilter *compact.GatherNoCompactionMarkFilter, ringLifecycle *ring.Lifecycler, _ prometheus.Counter, _ prometheus.Counter) compact.Planner { + plannerFactory := func(ctx context.Context, bkt objstore.InstrumentedBucket, logger log.Logger, cfg Config, noCompactionMarkFilter *compact.GatherNoCompactionMarkFilter, ringLifecycle *ring.Lifecycler, _ string, _ prometheus.Counter, _ prometheus.Counter, _ *compactorMetrics) compact.Planner { return compact.NewPlanner(logger, cfg.BlockRanges.ToMilliseconds(), noCompactionMarkFilter) } @@ -119,7 +121,7 @@ var ( return nil, nil, err } - plannerFactory := func(ctx context.Context, bkt objstore.InstrumentedBucket, logger log.Logger, cfg Config, noCompactionMarkFilter *compact.GatherNoCompactionMarkFilter, ringLifecycle *ring.Lifecycler, blockVisitMarkerReadFailed prometheus.Counter, blockVisitMarkerWriteFailed prometheus.Counter) compact.Planner { + plannerFactory := func(ctx context.Context, bkt objstore.InstrumentedBucket, logger log.Logger, cfg Config, noCompactionMarkFilter *compact.GatherNoCompactionMarkFilter, ringLifecycle *ring.Lifecycler, userID string, blockVisitMarkerReadFailed prometheus.Counter, blockVisitMarkerWriteFailed prometheus.Counter, compactorMetrics *compactorMetrics) compact.Planner { return NewShuffleShardingPlanner(ctx, bkt, logger, cfg.BlockRanges.ToMilliseconds(), noCompactionMarkFilter.NoCompactMarkedBlocks, ringLifecycle.ID, cfg.BlockVisitMarkerTimeout, cfg.BlockVisitMarkerFileUpdateInterval, blockVisitMarkerReadFailed, blockVisitMarkerWriteFailed) } @@ -133,13 +135,11 @@ type BlocksGrouperFactory func( cfg Config, bkt objstore.InstrumentedBucket, logger log.Logger, - reg prometheus.Registerer, - blocksMarkedForDeletion prometheus.Counter, blocksMarkedForNoCompact prometheus.Counter, - garbageCollectedBlocks prometheus.Counter, - remainingPlannedCompactions prometheus.Gauge, blockVisitMarkerReadFailed prometheus.Counter, blockVisitMarkerWriteFailed prometheus.Counter, + syncerMetrics *compact.SyncerMetrics, + compactorMetrics *compactorMetrics, ring *ring.Ring, ringLifecycler *ring.Lifecycler, limit Limits, @@ -162,8 +162,10 @@ type PlannerFactory func( cfg Config, noCompactionMarkFilter *compact.GatherNoCompactionMarkFilter, ringLifecycle *ring.Lifecycler, + userID string, blockVisitMarkerReadFailed prometheus.Counter, blockVisitMarkerWriteFailed prometheus.Counter, + compactorMetrics *compactorMetrics, ) compact.Planner // Limits defines limits used by the Compactor. @@ -330,25 +332,22 @@ type Compactor struct { // Metrics. CompactorStartDurationSeconds prometheus.Gauge - compactionRunsStarted prometheus.Counter - compactionRunsInterrupted prometheus.Counter - compactionRunsCompleted prometheus.Counter - compactionRunsFailed prometheus.Counter - compactionRunsLastSuccess prometheus.Gauge - compactionRunDiscoveredTenants prometheus.Gauge - compactionRunSkippedTenants prometheus.Gauge - compactionRunSucceededTenants prometheus.Gauge - compactionRunFailedTenants prometheus.Gauge - compactionRunInterval prometheus.Gauge - blocksMarkedForDeletion prometheus.Counter - blocksMarkedForNoCompaction prometheus.Counter - garbageCollectedBlocks prometheus.Counter - remainingPlannedCompactions prometheus.Gauge + CompactionRunsStarted prometheus.Counter + CompactionRunsInterrupted prometheus.Counter + CompactionRunsCompleted prometheus.Counter + CompactionRunsFailed prometheus.Counter + CompactionRunsLastSuccess prometheus.Gauge + CompactionRunDiscoveredTenants prometheus.Gauge + CompactionRunSkippedTenants prometheus.Gauge + CompactionRunSucceededTenants prometheus.Gauge + CompactionRunFailedTenants prometheus.Gauge + CompactionRunInterval prometheus.Gauge + BlocksMarkedForNoCompaction prometheus.Counter blockVisitMarkerReadFailed prometheus.Counter blockVisitMarkerWriteFailed prometheus.Counter - // TSDB syncer metrics - syncerMetrics *syncerMetrics + // Thanos compactor metrics per user + compactorMetrics *compactorMetrics } // NewCompactor makes a new Compactor. @@ -393,12 +392,11 @@ func newCompactor( blocksCompactorFactory BlocksCompactorFactory, limits *validation.Overrides, ) (*Compactor, error) { - var remainingPlannedCompactions prometheus.Gauge + var compactorMetrics *compactorMetrics if compactorCfg.ShardingStrategy == util.ShardingStrategyShuffle { - remainingPlannedCompactions = promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ - Name: "cortex_compactor_remaining_planned_compactions", - Help: "Total number of plans that remain to be compacted. Only available with shuffle-sharding strategy", - }) + compactorMetrics = newCompactorMetrics(registerer) + } else { + compactorMetrics = newDefaultCompactorMetrics(registerer) } c := &Compactor{ compactorCfg: compactorCfg, @@ -406,7 +404,6 @@ func newCompactor( parentLogger: logger, logger: log.With(logger, "component", "compactor"), registerer: registerer, - syncerMetrics: newSyncerMetrics(registerer), bucketClientFactory: bucketClientFactory, blocksGrouperFactory: blocksGrouperFactory, blocksCompactorFactory: blocksCompactorFactory, @@ -416,59 +413,50 @@ func newCompactor( Name: "cortex_compactor_start_duration_seconds", Help: "Time in seconds spent by compactor running start function", }), - compactionRunsStarted: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ + CompactionRunsStarted: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ Name: "cortex_compactor_runs_started_total", Help: "Total number of compaction runs started.", }), - compactionRunsInterrupted: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ + CompactionRunsInterrupted: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ Name: "cortex_compactor_runs_interrupted_total", Help: "Total number of compaction runs interrupted.", }), - compactionRunsCompleted: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ + CompactionRunsCompleted: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ Name: "cortex_compactor_runs_completed_total", Help: "Total number of compaction runs successfully completed.", }), - compactionRunsFailed: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ + CompactionRunsFailed: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ Name: "cortex_compactor_runs_failed_total", Help: "Total number of compaction runs failed.", }), - compactionRunsLastSuccess: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ + CompactionRunsLastSuccess: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ Name: "cortex_compactor_last_successful_run_timestamp_seconds", Help: "Unix timestamp of the last successful compaction run.", }), - compactionRunDiscoveredTenants: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ + CompactionRunDiscoveredTenants: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ Name: "cortex_compactor_tenants_discovered", Help: "Number of tenants discovered during the current compaction run. Reset to 0 when compactor is idle.", }), - compactionRunSkippedTenants: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ + CompactionRunSkippedTenants: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ Name: "cortex_compactor_tenants_skipped", Help: "Number of tenants skipped during the current compaction run. Reset to 0 when compactor is idle.", }), - compactionRunSucceededTenants: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ + CompactionRunSucceededTenants: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ Name: "cortex_compactor_tenants_processing_succeeded", Help: "Number of tenants successfully processed during the current compaction run. Reset to 0 when compactor is idle.", }), - compactionRunFailedTenants: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ + CompactionRunFailedTenants: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ Name: "cortex_compactor_tenants_processing_failed", Help: "Number of tenants failed processing during the current compaction run. Reset to 0 when compactor is idle.", }), - compactionRunInterval: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ + CompactionRunInterval: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ Name: "cortex_compactor_compaction_interval_seconds", Help: "The configured interval on which compaction is run in seconds. Useful when compared to the last successful run metric to accurately detect multiple failed compaction runs.", }), - blocksMarkedForDeletion: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ - Name: blocksMarkedForDeletionName, - Help: blocksMarkedForDeletionHelp, - ConstLabels: prometheus.Labels{"reason": "compaction"}, - }), - blocksMarkedForNoCompaction: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ + BlocksMarkedForNoCompaction: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ Name: "cortex_compactor_blocks_marked_for_no_compaction_total", Help: "Total number of blocks marked for no compact during a compaction run.", }), - garbageCollectedBlocks: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_garbage_collected_blocks_total", - Help: "Total number of blocks marked for deletion by compactor.", - }), blockVisitMarkerReadFailed: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ Name: "cortex_compactor_block_visit_marker_read_failed", Help: "Number of block visit marker file failed to be read.", @@ -477,8 +465,8 @@ func newCompactor( Name: "cortex_compactor_block_visit_marker_write_failed", Help: "Number of block visit marker file failed to be written.", }), - remainingPlannedCompactions: remainingPlannedCompactions, - limits: limits, + limits: limits, + compactorMetrics: compactorMetrics, } if len(compactorCfg.EnabledTenants) > 0 { @@ -490,8 +478,18 @@ func newCompactor( c.Service = services.NewBasicService(c.starting, c.running, c.stopping) + if c.registerer != nil { + // Copied from Thanos, pkg/block/fetcher.go + promauto.With(c.registerer).NewGaugeFunc(prometheus.GaugeOpts{ + Name: "cortex_compactor_meta_sync_consistency_delay_seconds", + Help: "Configured consistency delay in seconds.", + }, func() float64 { + return c.compactorCfg.ConsistencyDelay.Seconds() + }) + } + // The last successful compaction run metric is exposed as seconds since epoch, so we need to use seconds for this metric. - c.compactionRunInterval.Set(c.compactorCfg.CompactionInterval.Seconds()) + c.CompactionRunInterval.Set(c.compactorCfg.CompactionInterval.Seconds()) return c, nil } @@ -531,7 +529,7 @@ func (c *Compactor) starting(ctx context.Context) error { CleanupConcurrency: c.compactorCfg.CleanupConcurrency, BlockDeletionMarksMigrationEnabled: c.compactorCfg.BlockDeletionMarksMigrationEnabled, TenantCleanupDelay: c.compactorCfg.TenantCleanupDelay, - }, c.bucketClient, c.usersScanner, c.limits, c.parentLogger, c.registerer) + }, c.bucketClient, c.usersScanner, c.limits, c.parentLogger, c.registerer, c.compactorMetrics.syncerBlocksMarkedForDeletion) // Initialize the compactors ring if sharding is enabled. if c.compactorCfg.ShardingEnabled { @@ -648,28 +646,28 @@ func (c *Compactor) compactUsers(ctx context.Context) { failed := false interrupted := false - c.compactionRunsStarted.Inc() + c.CompactionRunsStarted.Inc() defer func() { // interruptions and successful runs are considered // mutually exclusive but we consider a run failed if any // tenant runs failed even if later runs are interrupted if !interrupted && !failed { - c.compactionRunsCompleted.Inc() - c.compactionRunsLastSuccess.SetToCurrentTime() + c.CompactionRunsCompleted.Inc() + c.CompactionRunsLastSuccess.SetToCurrentTime() } if interrupted { - c.compactionRunsInterrupted.Inc() + c.CompactionRunsInterrupted.Inc() } if failed { - c.compactionRunsFailed.Inc() + c.CompactionRunsFailed.Inc() } // Reset progress metrics once done. - c.compactionRunDiscoveredTenants.Set(0) - c.compactionRunSkippedTenants.Set(0) - c.compactionRunSucceededTenants.Set(0) - c.compactionRunFailedTenants.Set(0) + c.CompactionRunDiscoveredTenants.Set(0) + c.CompactionRunSkippedTenants.Set(0) + c.CompactionRunSucceededTenants.Set(0) + c.CompactionRunFailedTenants.Set(0) }() level.Info(c.logger).Log("msg", "discovering users from bucket") @@ -681,7 +679,7 @@ func (c *Compactor) compactUsers(ctx context.Context) { } level.Info(c.logger).Log("msg", "discovered users from bucket", "users", len(users)) - c.compactionRunDiscoveredTenants.Set(float64(len(users))) + c.CompactionRunDiscoveredTenants.Set(float64(len(users))) // When starting multiple compactor replicas nearly at the same time, running in a cluster with // a large number of tenants, we may end up in a situation where the 1st user is compacted by @@ -702,11 +700,11 @@ func (c *Compactor) compactUsers(ctx context.Context) { // Ensure the user ID belongs to our shard. if owned, err := c.ownUserForCompaction(userID); err != nil { - c.compactionRunSkippedTenants.Inc() + c.CompactionRunSkippedTenants.Inc() level.Warn(c.logger).Log("msg", "unable to check if user is owned by this shard", "user", userID, "err", err) continue } else if !owned { - c.compactionRunSkippedTenants.Inc() + c.CompactionRunSkippedTenants.Inc() level.Debug(c.logger).Log("msg", "skipping user because it is not owned by this shard", "user", userID) continue } @@ -714,7 +712,7 @@ func (c *Compactor) compactUsers(ctx context.Context) { // Skipping compaction if the bucket index failed to sync due to CMK errors. if idxs, err := bucketindex.ReadSyncStatus(ctx, c.bucketClient, userID, util_log.WithUserID(userID, c.logger)); err == nil { if idxs.Status == bucketindex.CustomerManagedKeyError { - c.compactionRunSkippedTenants.Inc() + c.CompactionRunSkippedTenants.Inc() level.Info(c.logger).Log("msg", "skipping compactUser due CustomerManagedKeyError", "user", userID) continue } @@ -723,11 +721,11 @@ func (c *Compactor) compactUsers(ctx context.Context) { ownedUsers[userID] = struct{}{} if markedForDeletion, err := cortex_tsdb.TenantDeletionMarkExists(ctx, c.bucketClient, userID); err != nil { - c.compactionRunSkippedTenants.Inc() + c.CompactionRunSkippedTenants.Inc() level.Warn(c.logger).Log("msg", "unable to check if user is marked for deletion", "user", userID, "err", err) continue } else if markedForDeletion { - c.compactionRunSkippedTenants.Inc() + c.CompactionRunSkippedTenants.Inc() level.Debug(c.logger).Log("msg", "skipping user because it is marked for deletion", "user", userID) continue } @@ -742,13 +740,13 @@ func (c *Compactor) compactUsers(ctx context.Context) { return } - c.compactionRunFailedTenants.Inc() + c.CompactionRunFailedTenants.Inc() failed = true level.Error(c.logger).Log("msg", "failed to compact user blocks", "user", userID, "err", err) continue } - c.compactionRunSucceededTenants.Inc() + c.CompactionRunSucceededTenants.Inc() level.Info(c.logger).Log("msg", "successfully compacted user blocks", "user", userID) } @@ -809,7 +807,6 @@ func (c *Compactor) compactUser(ctx context.Context, userID string) error { bucket := bucket.NewUserBucketClient(userID, c.bucketClient, c.limits) reg := prometheus.NewRegistry() - defer c.syncerMetrics.gatherThanosSyncerMetrics(reg) ulogger := util_log.WithUserID(userID, c.logger) ulogger = util_log.WithExecutionID(ulid.MustNew(ulid.Now(), crypto_rand.Reader).String(), ulogger) @@ -845,13 +842,14 @@ func (c *Compactor) compactUser(ctx context.Context, userID string) error { return cortex_tsdb.ErrBlockDiscoveryStrategy } - fetcher, err := block.NewMetaFetcher( + fetcher, err := block.NewMetaFetcherWithMetrics( ulogger, c.compactorCfg.MetaSyncConcurrency, bucket, blockLister, c.metaSyncDirForUser(userID), - reg, + c.compactorMetrics.getBaseFetcherMetrics(), + c.compactorMetrics.getMetaFetcherMetrics(), // List of filters to apply (order matters). []block.MetadataFilter{ // Remove the ingester ID because we don't shard blocks anymore, while still @@ -867,15 +865,14 @@ func (c *Compactor) compactUser(ctx context.Context, userID string) error { return err } - syncer, err := compact.NewMetaSyncer( + syncerMetrics := c.compactorMetrics.getSyncerMetrics(userID) + syncer, err := compact.NewMetaSyncerWithMetrics( ulogger, - reg, + syncerMetrics, bucket, fetcher, deduplicateBlocksFilter, ignoreDeletionMarkFilter, - c.blocksMarkedForDeletion, - c.garbageCollectedBlocks, ) if err != nil { return errors.Wrap(err, "failed to create syncer") @@ -886,8 +883,8 @@ func (c *Compactor) compactUser(ctx context.Context, userID string) error { compactor, err := compact.NewBucketCompactor( ulogger, syncer, - c.blocksGrouperFactory(currentCtx, c.compactorCfg, bucket, ulogger, reg, c.blocksMarkedForDeletion, c.blocksMarkedForNoCompaction, c.garbageCollectedBlocks, c.remainingPlannedCompactions, c.blockVisitMarkerReadFailed, c.blockVisitMarkerWriteFailed, c.ring, c.ringLifecycler, c.limits, userID, noCompactMarkerFilter), - c.blocksPlannerFactory(currentCtx, bucket, ulogger, c.compactorCfg, noCompactMarkerFilter, c.ringLifecycler, c.blockVisitMarkerReadFailed, c.blockVisitMarkerWriteFailed), + c.blocksGrouperFactory(currentCtx, c.compactorCfg, bucket, ulogger, c.BlocksMarkedForNoCompaction, c.blockVisitMarkerReadFailed, c.blockVisitMarkerWriteFailed, syncerMetrics, c.compactorMetrics, c.ring, c.ringLifecycler, c.limits, userID, noCompactMarkerFilter), + c.blocksPlannerFactory(currentCtx, bucket, ulogger, c.compactorCfg, noCompactMarkerFilter, c.ringLifecycler, userID, c.blockVisitMarkerReadFailed, c.blockVisitMarkerWriteFailed, c.compactorMetrics), c.blocksCompactor, c.compactDirForUser(userID), bucket, diff --git a/pkg/compactor/compactor_metrics.go b/pkg/compactor/compactor_metrics.go new file mode 100644 index 0000000000..3567225919 --- /dev/null +++ b/pkg/compactor/compactor_metrics.go @@ -0,0 +1,199 @@ +package compactor + +import ( + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + "github.com/thanos-io/thanos/pkg/block" + "github.com/thanos-io/thanos/pkg/compact" + "github.com/thanos-io/thanos/pkg/extprom" +) + +type compactorMetrics struct { + reg prometheus.Registerer + commonLabels []string + compactionLabels []string + + // block.BaseFetcherMetrics + baseFetcherSyncs *prometheus.CounterVec + + // block.FetcherMetrics + metaFetcherSyncs *prometheus.CounterVec + metaFetcherSyncFailures *prometheus.CounterVec + metaFetcherSyncDuration *prometheus.HistogramVec + metaFetcherSynced *extprom.TxGaugeVec + metaFetcherModified *extprom.TxGaugeVec + + // compact.SyncerMetrics + syncerGarbageCollectedBlocks *prometheus.CounterVec + syncerGarbageCollections *prometheus.CounterVec + syncerGarbageCollectionFailures *prometheus.CounterVec + syncerGarbageCollectionDuration *prometheus.HistogramVec + syncerBlocksMarkedForDeletion *prometheus.CounterVec + + compactions *prometheus.CounterVec + compactionPlanned *prometheus.CounterVec + compactionRunsStarted *prometheus.CounterVec + compactionRunsCompleted *prometheus.CounterVec + compactionFailures *prometheus.CounterVec + verticalCompactions *prometheus.CounterVec + remainingPlannedCompactions *prometheus.GaugeVec +} + +const ( + UserLabelName = "user" + TimeRangeLabelName = "time_range_milliseconds" + ReasonLabelName = "reason" +) + +var ( + commonLabels = []string{UserLabelName} + compactionLabels = []string{TimeRangeLabelName} +) + +func newDefaultCompactorMetrics(reg prometheus.Registerer) *compactorMetrics { + return newCompactorMetricsWithLabels(reg, commonLabels, []string{"resolution"}) +} + +func newCompactorMetrics(reg prometheus.Registerer) *compactorMetrics { + return newCompactorMetricsWithLabels(reg, commonLabels, append(commonLabels, compactionLabels...)) +} + +func newCompactorMetricsWithLabels(reg prometheus.Registerer, commonLabels []string, compactionLabels []string) *compactorMetrics { + var m compactorMetrics + m.reg = reg + m.commonLabels = commonLabels + m.compactionLabels = compactionLabels + + // Copied from Thanos, pkg/block/fetcher.go + m.baseFetcherSyncs = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Subsystem: block.FetcherSubSys, + Name: "cortex_compactor_meta_base_syncs_total", + Help: "Total blocks metadata synchronization attempts by base Fetcher.", + }, nil) + + // Copied from Thanos, pkg/block/fetcher.go + m.metaFetcherSyncs = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Subsystem: block.FetcherSubSys, + Name: "cortex_compactor_meta_syncs_total", + Help: "Total blocks metadata synchronization attempts.", + }, nil) + m.metaFetcherSyncFailures = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Subsystem: block.FetcherSubSys, + Name: "cortex_compactor_meta_sync_failures_total", + Help: "Total blocks metadata synchronization failures.", + }, nil) + m.metaFetcherSyncDuration = promauto.With(reg).NewHistogramVec(prometheus.HistogramOpts{ + Subsystem: block.FetcherSubSys, + Name: "cortex_compactor_meta_sync_duration_seconds", + Help: "Duration of the blocks metadata synchronization in seconds.", + Buckets: []float64{0.01, 1, 10, 100, 300, 600, 1000}, + }, nil) + m.metaFetcherSynced = extprom.NewTxGaugeVec( + reg, + prometheus.GaugeOpts{ + Subsystem: block.FetcherSubSys, + Name: "cortex_compactor_meta_synced", + Help: "Number of block metadata synced", + }, + []string{"state"}, + block.DefaultSyncedStateLabelValues()..., + ) + m.metaFetcherModified = extprom.NewTxGaugeVec( + reg, + prometheus.GaugeOpts{ + Subsystem: block.FetcherSubSys, + Name: "cortex_compactor_meta_modified", + Help: "Number of blocks whose metadata changed", + }, + []string{"modified"}, + block.DefaultModifiedLabelValues()..., + ) + + // Copied from Thanos, pkg/compact/compact.go. + m.syncerGarbageCollectedBlocks = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Name: "cortex_compactor_garbage_collected_blocks_total", + Help: "Total number of blocks marked for deletion by compactor.", + }, nil) + m.syncerGarbageCollections = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Name: "cortex_compactor_garbage_collection_total", + Help: "Total number of garbage collection operations.", + }, nil) + m.syncerGarbageCollectionFailures = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Name: "cortex_compactor_garbage_collection_failures_total", + Help: "Total number of failed garbage collection operations.", + }, nil) + m.syncerGarbageCollectionDuration = promauto.With(reg).NewHistogramVec(prometheus.HistogramOpts{ + Name: "cortex_compactor_garbage_collection_duration_seconds", + Help: "Time it took to perform garbage collection iteration.", + }, nil) + m.syncerBlocksMarkedForDeletion = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Name: blocksMarkedForDeletionName, + Help: blocksMarkedForDeletionHelp, + }, append(commonLabels, ReasonLabelName)) + + m.compactions = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Name: "cortex_compactor_group_compactions_total", + Help: "Total number of group compaction attempts that resulted in a new block.", + }, compactionLabels) + m.compactionPlanned = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Name: "cortex_compact_group_compaction_planned_total", + Help: "Total number of compaction planned.", + }, compactionLabels) + m.compactionRunsStarted = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Name: "cortex_compactor_group_compaction_runs_started_total", + Help: "Total number of group compaction attempts.", + }, compactionLabels) + m.compactionRunsCompleted = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Name: "cortex_compactor_group_compaction_runs_completed_total", + Help: "Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction.", + }, compactionLabels) + m.compactionFailures = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Name: "cortex_compactor_group_compactions_failures_total", + Help: "Total number of failed group compactions.", + }, compactionLabels) + m.verticalCompactions = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Name: "cortex_compactor_group_vertical_compactions_total", + Help: "Total number of group compaction attempts that resulted in a new block based on overlapping blocks.", + }, compactionLabels) + m.remainingPlannedCompactions = promauto.With(reg).NewGaugeVec(prometheus.GaugeOpts{ + Name: "cortex_compactor_remaining_planned_compactions", + Help: "Total number of plans that remain to be compacted. Only available with shuffle-sharding strategy", + }, commonLabels) + + return &m +} + +func (m *compactorMetrics) getBaseFetcherMetrics() *block.BaseFetcherMetrics { + var baseFetcherMetrics block.BaseFetcherMetrics + baseFetcherMetrics.Syncs = m.baseFetcherSyncs.WithLabelValues() + return &baseFetcherMetrics +} + +func (m *compactorMetrics) getMetaFetcherMetrics() *block.FetcherMetrics { + var fetcherMetrics block.FetcherMetrics + fetcherMetrics.Syncs = m.metaFetcherSyncs.WithLabelValues() + fetcherMetrics.SyncFailures = m.metaFetcherSyncFailures.WithLabelValues() + fetcherMetrics.SyncDuration = m.metaFetcherSyncDuration.WithLabelValues() + fetcherMetrics.Synced = m.metaFetcherSynced + fetcherMetrics.Modified = m.metaFetcherModified + return &fetcherMetrics +} + +func (m *compactorMetrics) getSyncerMetrics(userID string) *compact.SyncerMetrics { + var syncerMetrics compact.SyncerMetrics + labelValues := m.getCommonLabelValues(userID) + syncerMetrics.GarbageCollectedBlocks = m.syncerGarbageCollectedBlocks.WithLabelValues() + syncerMetrics.GarbageCollections = m.syncerGarbageCollections.WithLabelValues() + syncerMetrics.GarbageCollectionFailures = m.syncerGarbageCollectionFailures.WithLabelValues() + syncerMetrics.GarbageCollectionDuration = m.syncerGarbageCollectionDuration.WithLabelValues() + syncerMetrics.BlocksMarkedForDeletion = m.syncerBlocksMarkedForDeletion.WithLabelValues(append(labelValues, "compaction")...) + return &syncerMetrics +} + +func (m *compactorMetrics) getCommonLabelValues(userID string) []string { + var labelValues []string + if len(m.commonLabels) > 0 { + labelValues = append(labelValues, userID) + } + return labelValues +} diff --git a/pkg/compactor/compactor_metrics_test.go b/pkg/compactor/compactor_metrics_test.go new file mode 100644 index 0000000000..8667d892cf --- /dev/null +++ b/pkg/compactor/compactor_metrics_test.go @@ -0,0 +1,166 @@ +package compactor + +import ( + "bytes" + "testing" + + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/testutil" + "github.com/stretchr/testify/require" +) + +func TestSyncerMetrics(t *testing.T) { + reg := prometheus.NewPedanticRegistry() + cm := newCompactorMetricsWithLabels(reg, commonLabels, commonLabels) + + generateTestData(cm, 1234) + generateTestData(cm, 7654) + generateTestData(cm, 2222) + + err := testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP blocks_meta_cortex_compactor_meta_base_syncs_total Total blocks metadata synchronization attempts by base Fetcher. + # TYPE blocks_meta_cortex_compactor_meta_base_syncs_total counter + blocks_meta_cortex_compactor_meta_base_syncs_total 11110 + # HELP blocks_meta_cortex_compactor_meta_modified Number of blocks whose metadata changed + # TYPE blocks_meta_cortex_compactor_meta_modified gauge + blocks_meta_cortex_compactor_meta_modified{modified="replica-label-removed"} 0 + # HELP blocks_meta_cortex_compactor_meta_sync_duration_seconds Duration of the blocks metadata synchronization in seconds. + # TYPE blocks_meta_cortex_compactor_meta_sync_duration_seconds histogram + blocks_meta_cortex_compactor_meta_sync_duration_seconds_bucket{le="0.01"} 0 + blocks_meta_cortex_compactor_meta_sync_duration_seconds_bucket{le="1"} 2 + blocks_meta_cortex_compactor_meta_sync_duration_seconds_bucket{le="10"} 3 + blocks_meta_cortex_compactor_meta_sync_duration_seconds_bucket{le="100"} 3 + blocks_meta_cortex_compactor_meta_sync_duration_seconds_bucket{le="300"} 3 + blocks_meta_cortex_compactor_meta_sync_duration_seconds_bucket{le="600"} 3 + blocks_meta_cortex_compactor_meta_sync_duration_seconds_bucket{le="1000"} 3 + blocks_meta_cortex_compactor_meta_sync_duration_seconds_bucket{le="+Inf"} 3 + blocks_meta_cortex_compactor_meta_sync_duration_seconds_sum 4.444 + blocks_meta_cortex_compactor_meta_sync_duration_seconds_count 3 + # HELP blocks_meta_cortex_compactor_meta_sync_failures_total Total blocks metadata synchronization failures. + # TYPE blocks_meta_cortex_compactor_meta_sync_failures_total counter + blocks_meta_cortex_compactor_meta_sync_failures_total 33330 + # HELP blocks_meta_cortex_compactor_meta_synced Number of block metadata synced + # TYPE blocks_meta_cortex_compactor_meta_synced gauge + blocks_meta_cortex_compactor_meta_synced{state="corrupted-meta-json"} 0 + blocks_meta_cortex_compactor_meta_synced{state="duplicate"} 0 + blocks_meta_cortex_compactor_meta_synced{state="failed"} 0 + blocks_meta_cortex_compactor_meta_synced{state="label-excluded"} 0 + blocks_meta_cortex_compactor_meta_synced{state="loaded"} 0 + blocks_meta_cortex_compactor_meta_synced{state="marked-for-deletion"} 0 + blocks_meta_cortex_compactor_meta_synced{state="marked-for-no-compact"} 0 + blocks_meta_cortex_compactor_meta_synced{state="no-meta-json"} 0 + blocks_meta_cortex_compactor_meta_synced{state="time-excluded"} 0 + blocks_meta_cortex_compactor_meta_synced{state="too-fresh"} 0 + # HELP blocks_meta_cortex_compactor_meta_syncs_total Total blocks metadata synchronization attempts. + # TYPE blocks_meta_cortex_compactor_meta_syncs_total counter + blocks_meta_cortex_compactor_meta_syncs_total 22220 + # HELP cortex_compact_group_compaction_planned_total Total number of compaction planned. + # TYPE cortex_compact_group_compaction_planned_total counter + cortex_compact_group_compaction_planned_total{user="aaa"} 211090 + cortex_compact_group_compaction_planned_total{user="bbb"} 222200 + cortex_compact_group_compaction_planned_total{user="ccc"} 233310 + # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE cortex_compactor_blocks_marked_for_deletion_total counter + cortex_compactor_blocks_marked_for_deletion_total{reason="compaction",user="aaa"} 144430 + cortex_compactor_blocks_marked_for_deletion_total{reason="compaction",user="bbb"} 155540 + cortex_compactor_blocks_marked_for_deletion_total{reason="compaction",user="ccc"} 166650 + # HELP cortex_compactor_garbage_collected_blocks_total Total number of blocks marked for deletion by compactor. + # TYPE cortex_compactor_garbage_collected_blocks_total counter + cortex_compactor_garbage_collected_blocks_total 99990 + # HELP cortex_compactor_garbage_collection_duration_seconds Time it took to perform garbage collection iteration. + # TYPE cortex_compactor_garbage_collection_duration_seconds histogram + cortex_compactor_garbage_collection_duration_seconds_bucket{le="0.005"} 0 + cortex_compactor_garbage_collection_duration_seconds_bucket{le="0.01"} 0 + cortex_compactor_garbage_collection_duration_seconds_bucket{le="0.025"} 0 + cortex_compactor_garbage_collection_duration_seconds_bucket{le="0.05"} 0 + cortex_compactor_garbage_collection_duration_seconds_bucket{le="0.1"} 0 + cortex_compactor_garbage_collection_duration_seconds_bucket{le="0.25"} 0 + cortex_compactor_garbage_collection_duration_seconds_bucket{le="0.5"} 0 + cortex_compactor_garbage_collection_duration_seconds_bucket{le="1"} 0 + cortex_compactor_garbage_collection_duration_seconds_bucket{le="2.5"} 1 + cortex_compactor_garbage_collection_duration_seconds_bucket{le="5"} 2 + cortex_compactor_garbage_collection_duration_seconds_bucket{le="10"} 3 + cortex_compactor_garbage_collection_duration_seconds_bucket{le="+Inf"} 3 + cortex_compactor_garbage_collection_duration_seconds_sum 13.331999999999999 + cortex_compactor_garbage_collection_duration_seconds_count 3 + # HELP cortex_compactor_garbage_collection_failures_total Total number of failed garbage collection operations. + # TYPE cortex_compactor_garbage_collection_failures_total counter + cortex_compactor_garbage_collection_failures_total 122210 + # HELP cortex_compactor_garbage_collection_total Total number of garbage collection operations. + # TYPE cortex_compactor_garbage_collection_total counter + cortex_compactor_garbage_collection_total 111100 + # HELP cortex_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. + # TYPE cortex_compactor_group_compaction_runs_completed_total counter + cortex_compactor_group_compaction_runs_completed_total{user="aaa"} 277750 + cortex_compactor_group_compaction_runs_completed_total{user="bbb"} 288860 + cortex_compactor_group_compaction_runs_completed_total{user="ccc"} 299970 + # HELP cortex_compactor_group_compaction_runs_started_total Total number of group compaction attempts. + # TYPE cortex_compactor_group_compaction_runs_started_total counter + cortex_compactor_group_compaction_runs_started_total{user="aaa"} 244420 + cortex_compactor_group_compaction_runs_started_total{user="bbb"} 255530 + cortex_compactor_group_compaction_runs_started_total{user="ccc"} 266640 + # HELP cortex_compactor_group_compactions_failures_total Total number of failed group compactions. + # TYPE cortex_compactor_group_compactions_failures_total counter + cortex_compactor_group_compactions_failures_total{user="aaa"} 311080 + cortex_compactor_group_compactions_failures_total{user="bbb"} 322190 + cortex_compactor_group_compactions_failures_total{user="ccc"} 333300 + # HELP cortex_compactor_group_compactions_total Total number of group compaction attempts that resulted in a new block. + # TYPE cortex_compactor_group_compactions_total counter + cortex_compactor_group_compactions_total{user="aaa"} 177760 + cortex_compactor_group_compactions_total{user="bbb"} 188870 + cortex_compactor_group_compactions_total{user="ccc"} 199980 + # HELP cortex_compactor_group_vertical_compactions_total Total number of group compaction attempts that resulted in a new block based on overlapping blocks. + # TYPE cortex_compactor_group_vertical_compactions_total counter + cortex_compactor_group_vertical_compactions_total{user="aaa"} 344410 + cortex_compactor_group_vertical_compactions_total{user="bbb"} 355520 + cortex_compactor_group_vertical_compactions_total{user="ccc"} 366630 + # HELP cortex_compactor_remaining_planned_compactions Total number of plans that remain to be compacted. Only available with shuffle-sharding strategy + # TYPE cortex_compactor_remaining_planned_compactions gauge + cortex_compactor_remaining_planned_compactions{user="aaa"} 377740 + cortex_compactor_remaining_planned_compactions{user="bbb"} 388850 + cortex_compactor_remaining_planned_compactions{user="ccc"} 399960 + `)) + require.NoError(t, err) + +} + +func generateTestData(cm *compactorMetrics, base float64) { + cm.baseFetcherSyncs.WithLabelValues().Add(1 * base) + cm.metaFetcherSyncs.WithLabelValues().Add(2 * base) + cm.metaFetcherSyncFailures.WithLabelValues().Add(3 * base) + cm.metaFetcherSyncDuration.WithLabelValues().Observe(4 * base / 10000) + cm.metaFetcherSynced.WithLabelValues("loaded").Add(5 * base) + cm.metaFetcherSynced.WithLabelValues("no-meta-json").Add(6 * base) + cm.metaFetcherSynced.WithLabelValues("failed").Add(7 * base) + cm.metaFetcherModified.WithLabelValues("replica-label-removed").Add(8 * base) + + cm.syncerGarbageCollectedBlocks.WithLabelValues().Add(9 * base) + cm.syncerGarbageCollections.WithLabelValues().Add(10 * base) + cm.syncerGarbageCollectionFailures.WithLabelValues().Add(11 * base) + cm.syncerGarbageCollectionDuration.WithLabelValues().Observe(12 * base / 10000) + cm.syncerBlocksMarkedForDeletion.WithLabelValues("aaa", "compaction").Add(13 * base) + cm.syncerBlocksMarkedForDeletion.WithLabelValues("bbb", "compaction").Add(14 * base) + cm.syncerBlocksMarkedForDeletion.WithLabelValues("ccc", "compaction").Add(15 * base) + + cm.compactions.WithLabelValues("aaa").Add(16 * base) + cm.compactions.WithLabelValues("bbb").Add(17 * base) + cm.compactions.WithLabelValues("ccc").Add(18 * base) + cm.compactionPlanned.WithLabelValues("aaa").Add(19 * base) + cm.compactionPlanned.WithLabelValues("bbb").Add(20 * base) + cm.compactionPlanned.WithLabelValues("ccc").Add(21 * base) + cm.compactionRunsStarted.WithLabelValues("aaa").Add(22 * base) + cm.compactionRunsStarted.WithLabelValues("bbb").Add(23 * base) + cm.compactionRunsStarted.WithLabelValues("ccc").Add(24 * base) + cm.compactionRunsCompleted.WithLabelValues("aaa").Add(25 * base) + cm.compactionRunsCompleted.WithLabelValues("bbb").Add(26 * base) + cm.compactionRunsCompleted.WithLabelValues("ccc").Add(27 * base) + cm.compactionFailures.WithLabelValues("aaa").Add(28 * base) + cm.compactionFailures.WithLabelValues("bbb").Add(29 * base) + cm.compactionFailures.WithLabelValues("ccc").Add(30 * base) + cm.verticalCompactions.WithLabelValues("aaa").Add(31 * base) + cm.verticalCompactions.WithLabelValues("bbb").Add(32 * base) + cm.verticalCompactions.WithLabelValues("ccc").Add(33 * base) + cm.remainingPlannedCompactions.WithLabelValues("aaa").Add(34 * base) + cm.remainingPlannedCompactions.WithLabelValues("bbb").Add(35 * base) + cm.remainingPlannedCompactions.WithLabelValues("ccc").Add(36 * base) +} diff --git a/pkg/compactor/compactor_test.go b/pkg/compactor/compactor_test.go index 9cc92ae63e..6015adff86 100644 --- a/pkg/compactor/compactor_test.go +++ b/pkg/compactor/compactor_test.go @@ -186,7 +186,7 @@ func TestCompactor_SkipCompactionWhenCmkError(t *testing.T) { // Wait until a run has completed. cortex_testutil.Poll(t, time.Second, 1.0, func() interface{} { - return prom_testutil.ToFloat64(c.compactionRunsCompleted) + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) }) require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) @@ -206,12 +206,12 @@ func TestCompactor_ShouldDoNothingOnNoUserBlocks(t *testing.T) { // Wait until a run has completed. cortex_testutil.Poll(t, time.Second, 1.0, func() interface{} { - return prom_testutil.ToFloat64(c.compactionRunsCompleted) + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) }) require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) - assert.Equal(t, prom_testutil.ToFloat64(c.compactionRunInterval), cfg.CompactionInterval.Seconds()) + assert.Equal(t, prom_testutil.ToFloat64(c.CompactionRunInterval), cfg.CompactionInterval.Seconds()) assert.ElementsMatch(t, []string{ `level=info component=cleaner msg="started blocks cleanup and maintenance"`, @@ -234,62 +234,6 @@ func TestCompactor_ShouldDoNothingOnNoUserBlocks(t *testing.T) { # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. cortex_compactor_runs_failed_total 0 - # HELP cortex_compactor_garbage_collected_blocks_total Total number of blocks marked for deletion by compactor. - # TYPE cortex_compactor_garbage_collected_blocks_total counter - cortex_compactor_garbage_collected_blocks_total 0 - - # HELP cortex_compactor_garbage_collection_duration_seconds Time it took to perform garbage collection iteration. - # TYPE cortex_compactor_garbage_collection_duration_seconds histogram - cortex_compactor_garbage_collection_duration_seconds_bucket{le="+Inf"} 0 - cortex_compactor_garbage_collection_duration_seconds_sum 0 - cortex_compactor_garbage_collection_duration_seconds_count 0 - - # HELP cortex_compactor_garbage_collection_failures_total Total number of failed garbage collection operations. - # TYPE cortex_compactor_garbage_collection_failures_total counter - cortex_compactor_garbage_collection_failures_total 0 - - # HELP cortex_compactor_garbage_collection_total Total number of garbage collection operations. - # TYPE cortex_compactor_garbage_collection_total counter - cortex_compactor_garbage_collection_total 0 - - # HELP cortex_compactor_meta_sync_consistency_delay_seconds Configured consistency delay in seconds. - # TYPE cortex_compactor_meta_sync_consistency_delay_seconds gauge - cortex_compactor_meta_sync_consistency_delay_seconds 0 - - # HELP cortex_compactor_meta_sync_duration_seconds Duration of the blocks metadata synchronization in seconds. - # TYPE cortex_compactor_meta_sync_duration_seconds histogram - cortex_compactor_meta_sync_duration_seconds_bucket{le="+Inf"} 0 - cortex_compactor_meta_sync_duration_seconds_sum 0 - cortex_compactor_meta_sync_duration_seconds_count 0 - - # HELP cortex_compactor_meta_sync_failures_total Total blocks metadata synchronization failures. - # TYPE cortex_compactor_meta_sync_failures_total counter - cortex_compactor_meta_sync_failures_total 0 - - # HELP cortex_compactor_meta_syncs_total Total blocks metadata synchronization attempts. - # TYPE cortex_compactor_meta_syncs_total counter - cortex_compactor_meta_syncs_total 0 - - # HELP cortex_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. - # TYPE cortex_compactor_group_compaction_runs_completed_total counter - cortex_compactor_group_compaction_runs_completed_total 0 - - # HELP cortex_compactor_group_compaction_runs_started_total Total number of group compaction attempts. - # TYPE cortex_compactor_group_compaction_runs_started_total counter - cortex_compactor_group_compaction_runs_started_total 0 - - # HELP cortex_compactor_group_compactions_failures_total Total number of failed group compactions. - # TYPE cortex_compactor_group_compactions_failures_total counter - cortex_compactor_group_compactions_failures_total 0 - - # HELP cortex_compactor_group_compactions_total Total number of group compaction attempts that resulted in a new block. - # TYPE cortex_compactor_group_compactions_total counter - cortex_compactor_group_compactions_total 0 - - # HELP cortex_compactor_group_vertical_compactions_total Total number of group compaction attempts that resulted in a new block based on overlapping blocks. - # TYPE cortex_compactor_group_vertical_compactions_total counter - cortex_compactor_group_vertical_compactions_total 0 - # TYPE cortex_compactor_block_cleanup_failures_total counter # HELP cortex_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. cortex_compactor_block_cleanup_failures_total 0 @@ -298,15 +242,14 @@ func TestCompactor_ShouldDoNothingOnNoUserBlocks(t *testing.T) { # TYPE cortex_compactor_blocks_cleaned_total counter cortex_compactor_blocks_cleaned_total 0 - # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE cortex_compactor_blocks_marked_for_deletion_total counter - cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 - # HELP cortex_compactor_blocks_marked_for_no_compaction_total Total number of blocks marked for no compact during a compaction run. # TYPE cortex_compactor_blocks_marked_for_no_compaction_total counter cortex_compactor_blocks_marked_for_no_compaction_total 0 + # HELP cortex_compactor_meta_sync_consistency_delay_seconds Configured consistency delay in seconds. + # TYPE cortex_compactor_meta_sync_consistency_delay_seconds gauge + cortex_compactor_meta_sync_consistency_delay_seconds 0 + # TYPE cortex_compactor_block_cleanup_started_total counter # HELP cortex_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. cortex_compactor_block_cleanup_started_total 1 @@ -358,7 +301,7 @@ func TestCompactor_ShouldRetryCompactionOnFailureWhileDiscoveringUsersFromBucket // Wait until all retry attempts have completed. cortex_testutil.Poll(t, time.Second, 1.0, func() interface{} { - return prom_testutil.ToFloat64(c.compactionRunsFailed) + return prom_testutil.ToFloat64(c.CompactionRunsFailed) }) require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) @@ -387,62 +330,6 @@ func TestCompactor_ShouldRetryCompactionOnFailureWhileDiscoveringUsersFromBucket # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. cortex_compactor_runs_failed_total 1 - # HELP cortex_compactor_garbage_collected_blocks_total Total number of blocks marked for deletion by compactor. - # TYPE cortex_compactor_garbage_collected_blocks_total counter - cortex_compactor_garbage_collected_blocks_total 0 - - # HELP cortex_compactor_garbage_collection_duration_seconds Time it took to perform garbage collection iteration. - # TYPE cortex_compactor_garbage_collection_duration_seconds histogram - cortex_compactor_garbage_collection_duration_seconds_bucket{le="+Inf"} 0 - cortex_compactor_garbage_collection_duration_seconds_sum 0 - cortex_compactor_garbage_collection_duration_seconds_count 0 - - # HELP cortex_compactor_garbage_collection_failures_total Total number of failed garbage collection operations. - # TYPE cortex_compactor_garbage_collection_failures_total counter - cortex_compactor_garbage_collection_failures_total 0 - - # HELP cortex_compactor_garbage_collection_total Total number of garbage collection operations. - # TYPE cortex_compactor_garbage_collection_total counter - cortex_compactor_garbage_collection_total 0 - - # HELP cortex_compactor_meta_sync_consistency_delay_seconds Configured consistency delay in seconds. - # TYPE cortex_compactor_meta_sync_consistency_delay_seconds gauge - cortex_compactor_meta_sync_consistency_delay_seconds 0 - - # HELP cortex_compactor_meta_sync_duration_seconds Duration of the blocks metadata synchronization in seconds. - # TYPE cortex_compactor_meta_sync_duration_seconds histogram - cortex_compactor_meta_sync_duration_seconds_bucket{le="+Inf"} 0 - cortex_compactor_meta_sync_duration_seconds_sum 0 - cortex_compactor_meta_sync_duration_seconds_count 0 - - # HELP cortex_compactor_meta_sync_failures_total Total blocks metadata synchronization failures. - # TYPE cortex_compactor_meta_sync_failures_total counter - cortex_compactor_meta_sync_failures_total 0 - - # HELP cortex_compactor_meta_syncs_total Total blocks metadata synchronization attempts. - # TYPE cortex_compactor_meta_syncs_total counter - cortex_compactor_meta_syncs_total 0 - - # HELP cortex_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. - # TYPE cortex_compactor_group_compaction_runs_completed_total counter - cortex_compactor_group_compaction_runs_completed_total 0 - - # HELP cortex_compactor_group_compaction_runs_started_total Total number of group compaction attempts. - # TYPE cortex_compactor_group_compaction_runs_started_total counter - cortex_compactor_group_compaction_runs_started_total 0 - - # HELP cortex_compactor_group_compactions_failures_total Total number of failed group compactions. - # TYPE cortex_compactor_group_compactions_failures_total counter - cortex_compactor_group_compactions_failures_total 0 - - # HELP cortex_compactor_group_compactions_total Total number of group compaction attempts that resulted in a new block. - # TYPE cortex_compactor_group_compactions_total counter - cortex_compactor_group_compactions_total 0 - - # HELP cortex_compactor_group_vertical_compactions_total Total number of group compaction attempts that resulted in a new block based on overlapping blocks. - # TYPE cortex_compactor_group_vertical_compactions_total counter - cortex_compactor_group_vertical_compactions_total 0 - # TYPE cortex_compactor_block_cleanup_failures_total counter # HELP cortex_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. cortex_compactor_block_cleanup_failures_total 0 @@ -451,11 +338,6 @@ func TestCompactor_ShouldRetryCompactionOnFailureWhileDiscoveringUsersFromBucket # TYPE cortex_compactor_blocks_cleaned_total counter cortex_compactor_blocks_cleaned_total 0 - # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE cortex_compactor_blocks_marked_for_deletion_total counter - cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 - # TYPE cortex_compactor_block_cleanup_started_total counter # HELP cortex_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. cortex_compactor_block_cleanup_started_total 1 @@ -464,6 +346,10 @@ func TestCompactor_ShouldRetryCompactionOnFailureWhileDiscoveringUsersFromBucket # TYPE cortex_compactor_blocks_marked_for_no_compaction_total counter cortex_compactor_blocks_marked_for_no_compaction_total 0 + # HELP cortex_compactor_meta_sync_consistency_delay_seconds Configured consistency delay in seconds. + # TYPE cortex_compactor_meta_sync_consistency_delay_seconds gauge + cortex_compactor_meta_sync_consistency_delay_seconds 0 + # TYPE cortex_compactor_block_cleanup_completed_total counter # HELP cortex_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. cortex_compactor_block_cleanup_completed_total 0 @@ -530,7 +416,7 @@ func TestCompactor_ShouldIncrementCompactionErrorIfFailedToCompactASingleTenant( // Wait until all retry attempts have completed. cortex_testutil.Poll(t, time.Second, 1.0, func() interface{} { - return prom_testutil.ToFloat64(c.compactionRunsFailed) + return prom_testutil.ToFloat64(c.CompactionRunsFailed) }) require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) @@ -590,7 +476,7 @@ func TestCompactor_ShouldCompactAndRemoveUserFolder(t *testing.T) { // Wait until a run has completed. cortex_testutil.Poll(t, time.Second, 1.0, func() interface{} { - return prom_testutil.ToFloat64(c.compactionRunsCompleted) + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) }) _, err := os.Stat(c.compactDirForUser("user-1")) @@ -652,7 +538,7 @@ func TestCompactor_ShouldIterateOverUsersAndRunCompaction(t *testing.T) { // Wait until a run has completed. cortex_testutil.Poll(t, time.Second, 1.0, func() interface{} { - return prom_testutil.ToFloat64(c.compactionRunsCompleted) + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) }) require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) @@ -717,8 +603,8 @@ func TestCompactor_ShouldIterateOverUsersAndRunCompaction(t *testing.T) { # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. # TYPE cortex_compactor_blocks_marked_for_deletion_total counter - cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="compaction",user="user-1"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="compaction",user="user-2"} 0 # TYPE cortex_compactor_block_cleanup_started_total counter # HELP cortex_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. @@ -788,7 +674,7 @@ func TestCompactor_ShouldNotCompactBlocksMarkedForDeletion(t *testing.T) { // Wait until a run has completed. cortex_testutil.Poll(t, time.Second, 1.0, func() interface{} { - return prom_testutil.ToFloat64(c.compactionRunsCompleted) + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) }) require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) @@ -846,8 +732,7 @@ func TestCompactor_ShouldNotCompactBlocksMarkedForDeletion(t *testing.T) { # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. # TYPE cortex_compactor_blocks_marked_for_deletion_total counter - cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="compaction",user="user-1"} 0 # TYPE cortex_compactor_block_cleanup_started_total counter # HELP cortex_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. @@ -923,7 +808,7 @@ func TestCompactor_ShouldNotCompactBlocksMarkedForSkipCompact(t *testing.T) { require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) cortex_testutil.Poll(t, time.Second, 1.0, func() interface{} { - return prom_testutil.ToFloat64(c.compactionRunsCompleted) + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) }) require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) @@ -982,7 +867,7 @@ func TestCompactor_ShouldNotCompactBlocksForUsersMarkedForDeletion(t *testing.T) // Wait until a run has completed. cortex_testutil.Poll(t, time.Second, 1.0, func() interface{} { - return prom_testutil.ToFloat64(c.compactionRunsCompleted) + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) }) require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) @@ -1035,11 +920,6 @@ func TestCompactor_ShouldNotCompactBlocksForUsersMarkedForDeletion(t *testing.T) # TYPE cortex_compactor_blocks_cleaned_total counter cortex_compactor_blocks_cleaned_total 1 - # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE cortex_compactor_blocks_marked_for_deletion_total counter - cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 - # TYPE cortex_compactor_block_cleanup_started_total counter # HELP cortex_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. cortex_compactor_block_cleanup_started_total 1 @@ -1187,7 +1067,7 @@ func TestCompactor_ShouldCompactAllUsersOnShardingEnabledButOnlyOneInstanceRunni // Wait until a run has completed. cortex_testutil.Poll(t, 5*time.Second, 1.0, func() interface{} { - return prom_testutil.ToFloat64(c.compactionRunsCompleted) + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) }) require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) @@ -1291,7 +1171,7 @@ func TestCompactor_ShouldCompactOnlyUsersOwnedByTheInstanceOnShardingEnabledAndM // Wait until a run has been completed on each compactor for _, c := range compactors { cortex_testutil.Poll(t, 10*time.Second, 1.0, func() interface{} { - return prom_testutil.ToFloat64(c.compactionRunsCompleted) + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) }) } @@ -1431,7 +1311,7 @@ func TestCompactor_ShouldCompactOnlyShardsOwnedByTheInstanceOnShardingEnabledWit // Wait until a run has been completed on each compactor for _, c := range compactors { cortex_testutil.Poll(t, 60*time.Second, 2.0, func() interface{} { - return prom_testutil.ToFloat64(c.compactionRunsCompleted) + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) }) } @@ -1704,7 +1584,7 @@ func prepare(t *testing.T, compactorCfg Config, bucketClient objstore.Instrument blocksCompactorFactory := func(ctx context.Context, cfg Config, logger log.Logger, reg prometheus.Registerer) (compact.Compactor, PlannerFactory, error) { return tsdbCompactor, - func(ctx context.Context, bkt objstore.InstrumentedBucket, _ log.Logger, _ Config, noCompactMarkFilter *compact.GatherNoCompactionMarkFilter, ringLifecycle *ring.Lifecycler, _ prometheus.Counter, _ prometheus.Counter) compact.Planner { + func(ctx context.Context, bkt objstore.InstrumentedBucket, _ log.Logger, _ Config, noCompactMarkFilter *compact.GatherNoCompactionMarkFilter, ringLifecycle *ring.Lifecycler, _ string, _ prometheus.Counter, _ prometheus.Counter, _ *compactorMetrics) compact.Planner { tsdbPlanner.noCompactMarkFilters = append(tsdbPlanner.noCompactMarkFilters, noCompactMarkFilter) return tsdbPlanner }, @@ -1917,7 +1797,7 @@ func TestCompactor_DeleteLocalSyncFiles(t *testing.T) { // Wait until a run has been completed on first compactor. This happens as soon as compactor starts. cortex_testutil.Poll(t, 10*time.Second, 1.0, func() interface{} { - return prom_testutil.ToFloat64(c1.compactionRunsCompleted) + return prom_testutil.ToFloat64(c1.CompactionRunsCompleted) }) require.NoError(t, os.Mkdir(c1.metaSyncDirForUser("new-user"), 0600)) @@ -1928,7 +1808,7 @@ func TestCompactor_DeleteLocalSyncFiles(t *testing.T) { // Now start second compactor, and wait until it runs compaction. require.NoError(t, services.StartAndAwaitRunning(context.Background(), c2)) cortex_testutil.Poll(t, 10*time.Second, 1.0, func() interface{} { - return prom_testutil.ToFloat64(c2.compactionRunsCompleted) + return prom_testutil.ToFloat64(c2.CompactionRunsCompleted) }) // Let's check how many users second compactor has. @@ -2012,7 +1892,7 @@ func TestCompactor_ShouldNotTreatInterruptionsAsErrors(t *testing.T) { require.NoError(t, services.StartAndAwaitRunning(ctx, c)) cortex_testutil.Poll(t, 1*time.Second, 1.0, func() interface{} { - return prom_testutil.ToFloat64(c.compactionRunsInterrupted) + return prom_testutil.ToFloat64(c.CompactionRunsInterrupted) }) require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) @@ -2084,7 +1964,7 @@ func TestCompactor_ShouldNotFailCompactionIfAccessDeniedErrDuringMetaSync(t *tes // Wait until a run has completed. cortex_testutil.Poll(t, 20*time.Second, 1.0, func() interface{} { - return prom_testutil.ToFloat64(c.compactionRunsCompleted) + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) }) require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) @@ -2134,7 +2014,7 @@ func TestCompactor_ShouldNotFailCompactionIfAccessDeniedErrReturnedFromBucket(t // Wait until a run has completed. cortex_testutil.Poll(t, 20*time.Second, 1.0, func() interface{} { - return prom_testutil.ToFloat64(c.compactionRunsCompleted) + return prom_testutil.ToFloat64(c.CompactionRunsCompleted) }) require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) diff --git a/pkg/compactor/shuffle_sharding_grouper.go b/pkg/compactor/shuffle_sharding_grouper.go index 892ab05398..a041f55b6b 100644 --- a/pkg/compactor/shuffle_sharding_grouper.go +++ b/pkg/compactor/shuffle_sharding_grouper.go @@ -13,7 +13,6 @@ import ( "github.com/oklog/ulid" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/client_golang/prometheus/promauto" "github.com/prometheus/prometheus/model/labels" "github.com/thanos-io/objstore" "github.com/thanos-io/thanos/pkg/block/metadata" @@ -23,28 +22,21 @@ import ( ) type ShuffleShardingGrouper struct { - ctx context.Context - logger log.Logger - bkt objstore.InstrumentedBucket - acceptMalformedIndex bool - enableVerticalCompaction bool - reg prometheus.Registerer - blocksMarkedForDeletion prometheus.Counter - blocksMarkedForNoCompact prometheus.Counter - garbageCollectedBlocks prometheus.Counter - remainingPlannedCompactions prometheus.Gauge - hashFunc metadata.HashFunc - compactions *prometheus.CounterVec - compactionRunsStarted *prometheus.CounterVec - compactionRunsCompleted *prometheus.CounterVec - compactionFailures *prometheus.CounterVec - verticalCompactions *prometheus.CounterVec - compactorCfg Config - limits Limits - userID string - blockFilesConcurrency int - blocksFetchConcurrency int - compactionConcurrency int + ctx context.Context + logger log.Logger + bkt objstore.InstrumentedBucket + acceptMalformedIndex bool + enableVerticalCompaction bool + blocksMarkedForNoCompact prometheus.Counter + syncerMetrics *compact.SyncerMetrics + compactorMetrics *compactorMetrics + hashFunc metadata.HashFunc + compactorCfg Config + limits Limits + userID string + blockFilesConcurrency int + blocksFetchConcurrency int + compactionConcurrency int ring ring.ReadRing ringLifecyclerAddr string @@ -63,12 +55,10 @@ func NewShuffleShardingGrouper( bkt objstore.InstrumentedBucket, acceptMalformedIndex bool, enableVerticalCompaction bool, - reg prometheus.Registerer, - blocksMarkedForDeletion prometheus.Counter, blocksMarkedForNoCompact prometheus.Counter, - garbageCollectedBlocks prometheus.Counter, - remainingPlannedCompactions prometheus.Gauge, hashFunc metadata.HashFunc, + syncerMetrics *compact.SyncerMetrics, + compactorMetrics *compactorMetrics, compactorCfg Config, ring ring.ReadRing, ringLifecyclerAddr string, @@ -93,33 +83,10 @@ func NewShuffleShardingGrouper( bkt: bkt, acceptMalformedIndex: acceptMalformedIndex, enableVerticalCompaction: enableVerticalCompaction, - reg: reg, - blocksMarkedForDeletion: blocksMarkedForDeletion, blocksMarkedForNoCompact: blocksMarkedForNoCompact, - garbageCollectedBlocks: garbageCollectedBlocks, - remainingPlannedCompactions: remainingPlannedCompactions, hashFunc: hashFunc, - // Metrics are copied from Thanos DefaultGrouper constructor - compactions: promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ - Name: "thanos_compact_group_compactions_total", - Help: "Total number of group compaction attempts that resulted in a new block.", - }, []string{"group"}), - compactionRunsStarted: promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ - Name: "thanos_compact_group_compaction_runs_started_total", - Help: "Total number of group compaction attempts.", - }, []string{"group"}), - compactionRunsCompleted: promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ - Name: "thanos_compact_group_compaction_runs_completed_total", - Help: "Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction.", - }, []string{"group"}), - compactionFailures: promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ - Name: "thanos_compact_group_compactions_failures_total", - Help: "Total number of failed group compactions.", - }, []string{"group"}), - verticalCompactions: promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ - Name: "thanos_compact_group_vertical_compactions_total", - Help: "Total number of group compaction attempts that resulted in a new block based on overlapping blocks.", - }, []string{"group"}), + syncerMetrics: syncerMetrics, + compactorMetrics: compactorMetrics, compactorCfg: compactorCfg, ring: ring, ringLifecyclerAddr: ringLifecyclerAddr, @@ -167,7 +134,9 @@ func (g *ShuffleShardingGrouper) Groups(blocks map[ulid.ULID]*metadata.Meta) (re } // Metrics for the remaining planned compactions var remainingCompactions = 0. - defer func() { g.remainingPlannedCompactions.Set(remainingCompactions) }() + defer func() { + g.compactorMetrics.remainingPlannedCompactions.WithLabelValues(g.userID).Set(remainingCompactions) + }() var groups []blocksGroup for _, mainBlocks := range mainGroups { @@ -242,7 +211,11 @@ mainLoop: // resolution and external labels. resolution := group.blocks[0].Thanos.Downsample.Resolution externalLabels := labels.FromMap(group.blocks[0].Thanos.Labels) - + timeRange := group.rangeEnd - group.rangeStart + metricLabelValues := []string{ + g.userID, + fmt.Sprintf("%d", timeRange), + } thanosGroup, err := compact.NewGroup( log.With(g.logger, "groupKey", groupKey, "rangeStart", group.rangeStartTime().String(), "rangeEnd", group.rangeEndTime().String(), "externalLabels", externalLabels, "downsampleResolution", resolution), g.bkt, @@ -251,13 +224,13 @@ mainLoop: resolution, g.acceptMalformedIndex, true, // Enable vertical compaction. - g.compactions.WithLabelValues(groupKey), - g.compactionRunsStarted.WithLabelValues(groupKey), - g.compactionRunsCompleted.WithLabelValues(groupKey), - g.compactionFailures.WithLabelValues(groupKey), - g.verticalCompactions.WithLabelValues(groupKey), - g.garbageCollectedBlocks, - g.blocksMarkedForDeletion, + g.compactorMetrics.compactions.WithLabelValues(metricLabelValues...), + g.compactorMetrics.compactionRunsStarted.WithLabelValues(metricLabelValues...), + g.compactorMetrics.compactionRunsCompleted.WithLabelValues(metricLabelValues...), + g.compactorMetrics.compactionFailures.WithLabelValues(metricLabelValues...), + g.compactorMetrics.verticalCompactions.WithLabelValues(metricLabelValues...), + g.syncerMetrics.GarbageCollectedBlocks, + g.syncerMetrics.BlocksMarkedForDeletion, g.blocksMarkedForNoCompact, g.hashFunc, g.blockFilesConcurrency, diff --git a/pkg/compactor/shuffle_sharding_grouper_test.go b/pkg/compactor/shuffle_sharding_grouper_test.go index fb935c9c9f..c7aaa4a656 100644 --- a/pkg/compactor/shuffle_sharding_grouper_test.go +++ b/pkg/compactor/shuffle_sharding_grouper_test.go @@ -140,9 +140,9 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { {block1hto2hExt1Ulid, block0hto1hExt1Ulid}, {block3hto4hExt1Ulid, block2hto3hExt1Ulid}, }, - metrics: `# HELP cortex_compactor_remaining_planned_compactions Total number of plans that remain to be compacted. + metrics: `# HELP cortex_compactor_remaining_planned_compactions Total number of plans that remain to be compacted. Only available with shuffle-sharding strategy # TYPE cortex_compactor_remaining_planned_compactions gauge - cortex_compactor_remaining_planned_compactions 3 + cortex_compactor_remaining_planned_compactions{user="test-user"} 3 `, }, "test no compaction": { @@ -150,9 +150,9 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { ranges: []time.Duration{2 * time.Hour, 4 * time.Hour}, blocks: map[ulid.ULID]*metadata.Meta{block0hto1hExt1Ulid: blocks[block0hto1hExt1Ulid], block0hto1hExt2Ulid: blocks[block0hto1hExt2Ulid], block0to1hExt3Ulid: blocks[block0to1hExt3Ulid]}, expected: [][]ulid.ULID{}, - metrics: `# HELP cortex_compactor_remaining_planned_compactions Total number of plans that remain to be compacted. + metrics: `# HELP cortex_compactor_remaining_planned_compactions Total number of plans that remain to be compacted. Only available with shuffle-sharding strategy # TYPE cortex_compactor_remaining_planned_compactions gauge - cortex_compactor_remaining_planned_compactions 0 + cortex_compactor_remaining_planned_compactions{user="test-user"} 0 `, }, "test smallest range first": { @@ -164,9 +164,9 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { {block3hto4hExt1Ulid, block2hto3hExt1Ulid}, {block4hto6hExt2Ulid, block6hto8hExt2Ulid}, }, - metrics: `# HELP cortex_compactor_remaining_planned_compactions Total number of plans that remain to be compacted. + metrics: `# HELP cortex_compactor_remaining_planned_compactions Total number of plans that remain to be compacted. Only available with shuffle-sharding strategy # TYPE cortex_compactor_remaining_planned_compactions gauge - cortex_compactor_remaining_planned_compactions 3 + cortex_compactor_remaining_planned_compactions{user="test-user"} 3 `, }, "test oldest min time first": { @@ -177,9 +177,9 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { {block1hto2hExt1Ulid, block0hto1hExt1Ulid, block1hto2hExt1UlidCopy}, {block3hto4hExt1Ulid, block2hto3hExt1Ulid}, }, - metrics: `# HELP cortex_compactor_remaining_planned_compactions Total number of plans that remain to be compacted. + metrics: `# HELP cortex_compactor_remaining_planned_compactions Total number of plans that remain to be compacted. Only available with shuffle-sharding strategy # TYPE cortex_compactor_remaining_planned_compactions gauge - cortex_compactor_remaining_planned_compactions 2 + cortex_compactor_remaining_planned_compactions{user="test-user"} 2 `, }, "test overlapping blocks": { @@ -189,9 +189,9 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { expected: [][]ulid.ULID{ {block21hto40hExt1Ulid, block21hto40hExt1UlidCopy}, }, - metrics: `# HELP cortex_compactor_remaining_planned_compactions Total number of plans that remain to be compacted. + metrics: `# HELP cortex_compactor_remaining_planned_compactions Total number of plans that remain to be compacted. Only available with shuffle-sharding strategy # TYPE cortex_compactor_remaining_planned_compactions gauge - cortex_compactor_remaining_planned_compactions 1 + cortex_compactor_remaining_planned_compactions{user="test-user"} 1 `, }, "test imperfect maxTime blocks": { @@ -201,9 +201,9 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { expected: [][]ulid.ULID{ {block0hto45mExt1Ulid, block0hto1h30mExt1Ulid}, }, - metrics: `# HELP cortex_compactor_remaining_planned_compactions Total number of plans that remain to be compacted. + metrics: `# HELP cortex_compactor_remaining_planned_compactions Total number of plans that remain to be compacted. Only available with shuffle-sharding strategy # TYPE cortex_compactor_remaining_planned_compactions gauge - cortex_compactor_remaining_planned_compactions 1 + cortex_compactor_remaining_planned_compactions{user="test-user"} 1 `, }, "test prematurely created blocks": { @@ -211,9 +211,9 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { ranges: []time.Duration{2 * time.Hour}, blocks: map[ulid.ULID]*metadata.Meta{blocklast1hExt1UlidCopy: blocks[blocklast1hExt1UlidCopy], blocklast1hExt1Ulid: blocks[blocklast1hExt1Ulid]}, expected: [][]ulid.ULID{}, - metrics: `# HELP cortex_compactor_remaining_planned_compactions Total number of plans that remain to be compacted. + metrics: `# HELP cortex_compactor_remaining_planned_compactions Total number of plans that remain to be compacted. Only available with shuffle-sharding strategy # TYPE cortex_compactor_remaining_planned_compactions gauge - cortex_compactor_remaining_planned_compactions 0 + cortex_compactor_remaining_planned_compactions{user="test-user"} 0 `, }, "test group with all blocks visited": { @@ -231,9 +231,9 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { {id: block1hto2hExt2Ulid, compactorID: otherCompactorID, isExpired: false}, {id: block0hto1hExt2Ulid, compactorID: otherCompactorID, isExpired: false}, }, - metrics: `# HELP cortex_compactor_remaining_planned_compactions Total number of plans that remain to be compacted. + metrics: `# HELP cortex_compactor_remaining_planned_compactions Total number of plans that remain to be compacted. Only available with shuffle-sharding strategy # TYPE cortex_compactor_remaining_planned_compactions gauge - cortex_compactor_remaining_planned_compactions 1 + cortex_compactor_remaining_planned_compactions{user="test-user"} 1 `, }, "test group with one block visited": { @@ -250,9 +250,9 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { }{ {id: block1hto2hExt2Ulid, compactorID: otherCompactorID, isExpired: false}, }, - metrics: `# HELP cortex_compactor_remaining_planned_compactions Total number of plans that remain to be compacted. + metrics: `# HELP cortex_compactor_remaining_planned_compactions Total number of plans that remain to be compacted. Only available with shuffle-sharding strategy # TYPE cortex_compactor_remaining_planned_compactions gauge - cortex_compactor_remaining_planned_compactions 1 + cortex_compactor_remaining_planned_compactions{user="test-user"} 1 `, }, "test group block visit marker file expired": { @@ -270,9 +270,9 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { {id: block1hto2hExt2Ulid, compactorID: otherCompactorID, isExpired: true}, {id: block0hto1hExt2Ulid, compactorID: otherCompactorID, isExpired: true}, }, - metrics: `# HELP cortex_compactor_remaining_planned_compactions Total number of plans that remain to be compacted. + metrics: `# HELP cortex_compactor_remaining_planned_compactions Total number of plans that remain to be compacted. Only available with shuffle-sharding strategy # TYPE cortex_compactor_remaining_planned_compactions gauge - cortex_compactor_remaining_planned_compactions 1 + cortex_compactor_remaining_planned_compactions{user="test-user"} 1 `, }, "test group with one block visited by current compactor": { @@ -289,9 +289,9 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { }{ {id: block1hto2hExt2Ulid, compactorID: testCompactorID, isExpired: false}, }, - metrics: `# HELP cortex_compactor_remaining_planned_compactions Total number of plans that remain to be compacted. + metrics: `# HELP cortex_compactor_remaining_planned_compactions Total number of plans that remain to be compacted. Only available with shuffle-sharding strategy # TYPE cortex_compactor_remaining_planned_compactions gauge - cortex_compactor_remaining_planned_compactions 1 + cortex_compactor_remaining_planned_compactions{user="test-user"} 1 `, }, "test basic grouping with concurrency 2": { @@ -302,9 +302,9 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { {block1hto2hExt2Ulid, block0hto1hExt2Ulid}, {block1hto2hExt1Ulid, block0hto1hExt1Ulid}, }, - metrics: `# HELP cortex_compactor_remaining_planned_compactions Total number of plans that remain to be compacted. + metrics: `# HELP cortex_compactor_remaining_planned_compactions Total number of plans that remain to be compacted. Only available with shuffle-sharding strategy # TYPE cortex_compactor_remaining_planned_compactions gauge - cortex_compactor_remaining_planned_compactions 2 + cortex_compactor_remaining_planned_compactions{user="test-user"} 2 `, }, "test should skip block with no compact marker": { @@ -315,9 +315,9 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { {block1hto2hExt2Ulid, block0hto1hExt2Ulid}, {block1hto2hExt1Ulid, block0hto1hExt1Ulid}, }, - metrics: `# HELP cortex_compactor_remaining_planned_compactions Total number of plans that remain to be compacted. + metrics: `# HELP cortex_compactor_remaining_planned_compactions Total number of plans that remain to be compacted. Only available with shuffle-sharding strategy # TYPE cortex_compactor_remaining_planned_compactions gauge - cortex_compactor_remaining_planned_compactions 2 + cortex_compactor_remaining_planned_compactions{user="test-user"} 2 `, noCompactBlocks: map[ulid.ULID]*metadata.NoCompactMark{block2hto3hExt1Ulid: {}}, }, @@ -347,10 +347,6 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { ring.On("ShuffleShard", mock.Anything, mock.Anything).Return(subring, nil) registerer := prometheus.NewPedanticRegistry() - remainingPlannedCompactions := promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ - Name: "cortex_compactor_remaining_planned_compactions", - Help: "Total number of plans that remain to be compacted.", - }) blockVisitMarkerReadFailed := promauto.With(registerer).NewCounter(prometheus.CounterOpts{ Name: "cortex_compactor_block_visit_marker_read_failed", Help: "Number of block visit marker file failed to be read.", @@ -379,6 +375,8 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { bkt.MockUpload(mock.Anything, nil) bkt.MockGet(mock.Anything, "", nil) + metrics := newCompactorMetrics(registerer) + noCompactFilter := func() map[ulid.ULID]*metadata.NoCompactMark { return testData.noCompactBlocks } @@ -391,18 +389,16 @@ func TestShuffleShardingGrouper_Groups(t *testing.T) { objstore.WithNoopInstr(bkt), false, // Do not accept malformed indexes true, // Enable vertical compaction - registerer, - nil, - nil, nil, - remainingPlannedCompactions, metadata.NoneFunc, + metrics.getSyncerMetrics("test-user"), + metrics, *compactorCfg, ring, "test-addr", testCompactorID, overrides, - "", + "test-user", 10, 3, testData.concurrency, diff --git a/pkg/compactor/syncer_metrics.go b/pkg/compactor/syncer_metrics.go deleted file mode 100644 index c171779270..0000000000 --- a/pkg/compactor/syncer_metrics.go +++ /dev/null @@ -1,124 +0,0 @@ -package compactor - -import ( - "github.com/go-kit/log/level" - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/client_golang/prometheus/promauto" - - "github.com/cortexproject/cortex/pkg/util" - util_log "github.com/cortexproject/cortex/pkg/util/log" -) - -// Copied from Thanos, pkg/compact/compact.go. -// Here we aggregate metrics from all finished syncers. -type syncerMetrics struct { - metaSync prometheus.Counter - metaSyncFailures prometheus.Counter - metaSyncDuration *util.HistogramDataCollector // was prometheus.Histogram before - metaSyncConsistencyDelay prometheus.Gauge - garbageCollections prometheus.Counter - garbageCollectionFailures prometheus.Counter - garbageCollectionDuration *util.HistogramDataCollector // was prometheus.Histogram before - compactions prometheus.Counter - compactionRunsStarted prometheus.Counter - compactionRunsCompleted prometheus.Counter - compactionFailures prometheus.Counter - verticalCompactions prometheus.Counter -} - -// Copied (and modified with Cortex prefix) from Thanos, pkg/compact/compact.go -// We also ignore "group" label, since we only use a single group. -func newSyncerMetrics(reg prometheus.Registerer) *syncerMetrics { - var m syncerMetrics - - m.metaSync = promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_meta_syncs_total", - Help: "Total blocks metadata synchronization attempts.", - }) - m.metaSyncFailures = promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_meta_sync_failures_total", - Help: "Total blocks metadata synchronization failures.", - }) - m.metaSyncDuration = util.NewHistogramDataCollector(prometheus.NewDesc( - "cortex_compactor_meta_sync_duration_seconds", - "Duration of the blocks metadata synchronization in seconds.", - nil, nil)) - m.metaSyncConsistencyDelay = promauto.With(reg).NewGauge(prometheus.GaugeOpts{ - Name: "cortex_compactor_meta_sync_consistency_delay_seconds", - Help: "Configured consistency delay in seconds.", - }) - - m.garbageCollections = promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_garbage_collection_total", - Help: "Total number of garbage collection operations.", - }) - m.garbageCollectionFailures = promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_garbage_collection_failures_total", - Help: "Total number of failed garbage collection operations.", - }) - m.garbageCollectionDuration = util.NewHistogramDataCollector(prometheus.NewDesc( - "cortex_compactor_garbage_collection_duration_seconds", - "Time it took to perform garbage collection iteration.", - nil, nil)) - - m.compactions = promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_group_compactions_total", - Help: "Total number of group compaction attempts that resulted in a new block.", - }) - m.compactionRunsStarted = promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_group_compaction_runs_started_total", - Help: "Total number of group compaction attempts.", - }) - m.compactionRunsCompleted = promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_group_compaction_runs_completed_total", - Help: "Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction.", - }) - m.compactionFailures = promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_group_compactions_failures_total", - Help: "Total number of failed group compactions.", - }) - m.verticalCompactions = promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_group_vertical_compactions_total", - Help: "Total number of group compaction attempts that resulted in a new block based on overlapping blocks.", - }) - - if reg != nil { - reg.MustRegister(m.metaSyncDuration, m.garbageCollectionDuration) - } - - return &m -} - -func (m *syncerMetrics) gatherThanosSyncerMetrics(reg *prometheus.Registry) { - if m == nil { - return - } - - mf, err := reg.Gather() - if err != nil { - level.Warn(util_log.Logger).Log("msg", "failed to gather metrics from syncer registry after compaction", "err", err) - return - } - - mfm, err := util.NewMetricFamilyMap(mf) - if err != nil { - level.Warn(util_log.Logger).Log("msg", "failed to gather metrics from syncer registry after compaction", "err", err) - return - } - - m.metaSync.Add(mfm.SumCounters("blocks_meta_syncs_total")) - m.metaSyncFailures.Add(mfm.SumCounters("blocks_meta_sync_failures_total")) - m.metaSyncDuration.Add(mfm.SumHistograms("blocks_meta_sync_duration_seconds")) - m.metaSyncConsistencyDelay.Set(mfm.MaxGauges("consistency_delay_seconds")) - - m.garbageCollections.Add(mfm.SumCounters("thanos_compact_garbage_collection_total")) - m.garbageCollectionFailures.Add(mfm.SumCounters("thanos_compact_garbage_collection_failures_total")) - m.garbageCollectionDuration.Add(mfm.SumHistograms("thanos_compact_garbage_collection_duration_seconds")) - - // These metrics have "group" label, but we sum them all together. - m.compactions.Add(mfm.SumCounters("thanos_compact_group_compactions_total")) - m.compactionRunsStarted.Add(mfm.SumCounters("thanos_compact_group_compaction_runs_started_total")) - m.compactionRunsCompleted.Add(mfm.SumCounters("thanos_compact_group_compaction_runs_completed_total")) - m.compactionFailures.Add(mfm.SumCounters("thanos_compact_group_compactions_failures_total")) - m.verticalCompactions.Add(mfm.SumCounters("thanos_compact_group_vertical_compactions_total")) -} diff --git a/pkg/compactor/syncer_metrics_test.go b/pkg/compactor/syncer_metrics_test.go deleted file mode 100644 index 7a21955ebf..0000000000 --- a/pkg/compactor/syncer_metrics_test.go +++ /dev/null @@ -1,232 +0,0 @@ -package compactor - -import ( - "bytes" - "testing" - - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/client_golang/prometheus/testutil" - "github.com/stretchr/testify/require" -) - -func TestSyncerMetrics(t *testing.T) { - reg := prometheus.NewPedanticRegistry() - - sm := newSyncerMetrics(reg) - sm.gatherThanosSyncerMetrics(generateTestData(12345)) - sm.gatherThanosSyncerMetrics(generateTestData(76543)) - sm.gatherThanosSyncerMetrics(generateTestData(22222)) - // total base = 111110 - - err := testutil.GatherAndCompare(reg, bytes.NewBufferString(` - # HELP cortex_compactor_meta_sync_consistency_delay_seconds Configured consistency delay in seconds. - # TYPE cortex_compactor_meta_sync_consistency_delay_seconds gauge - cortex_compactor_meta_sync_consistency_delay_seconds 300 - - # HELP cortex_compactor_meta_syncs_total Total blocks metadata synchronization attempts. - # TYPE cortex_compactor_meta_syncs_total counter - cortex_compactor_meta_syncs_total 111110 - - # HELP cortex_compactor_meta_sync_failures_total Total blocks metadata synchronization failures. - # TYPE cortex_compactor_meta_sync_failures_total counter - cortex_compactor_meta_sync_failures_total 222220 - - # HELP cortex_compactor_meta_sync_duration_seconds Duration of the blocks metadata synchronization in seconds. - # TYPE cortex_compactor_meta_sync_duration_seconds histogram - # Observed values: 3.7035, 22.9629, 6.6666 (seconds) - cortex_compactor_meta_sync_duration_seconds_bucket{le="0.01"} 0 - cortex_compactor_meta_sync_duration_seconds_bucket{le="0.1"} 0 - cortex_compactor_meta_sync_duration_seconds_bucket{le="0.3"} 0 - cortex_compactor_meta_sync_duration_seconds_bucket{le="0.6"} 0 - cortex_compactor_meta_sync_duration_seconds_bucket{le="1"} 0 - cortex_compactor_meta_sync_duration_seconds_bucket{le="3"} 0 - cortex_compactor_meta_sync_duration_seconds_bucket{le="6"} 1 - cortex_compactor_meta_sync_duration_seconds_bucket{le="9"} 2 - cortex_compactor_meta_sync_duration_seconds_bucket{le="20"} 2 - cortex_compactor_meta_sync_duration_seconds_bucket{le="30"} 3 - cortex_compactor_meta_sync_duration_seconds_bucket{le="60"} 3 - cortex_compactor_meta_sync_duration_seconds_bucket{le="90"} 3 - cortex_compactor_meta_sync_duration_seconds_bucket{le="120"} 3 - cortex_compactor_meta_sync_duration_seconds_bucket{le="240"} 3 - cortex_compactor_meta_sync_duration_seconds_bucket{le="360"} 3 - cortex_compactor_meta_sync_duration_seconds_bucket{le="720"} 3 - cortex_compactor_meta_sync_duration_seconds_bucket{le="+Inf"} 3 - # rounding error - cortex_compactor_meta_sync_duration_seconds_sum 33.333000000000006 - cortex_compactor_meta_sync_duration_seconds_count 3 - - # HELP cortex_compactor_garbage_collection_total Total number of garbage collection operations. - # TYPE cortex_compactor_garbage_collection_total counter - cortex_compactor_garbage_collection_total 555550 - - # HELP cortex_compactor_garbage_collection_failures_total Total number of failed garbage collection operations. - # TYPE cortex_compactor_garbage_collection_failures_total counter - cortex_compactor_garbage_collection_failures_total 666660 - - # HELP cortex_compactor_garbage_collection_duration_seconds Time it took to perform garbage collection iteration. - # TYPE cortex_compactor_garbage_collection_duration_seconds histogram - # Observed values: 8.6415, 53.5801, 15.5554 - cortex_compactor_garbage_collection_duration_seconds_bucket{le="0.01"} 0 - cortex_compactor_garbage_collection_duration_seconds_bucket{le="0.1"} 0 - cortex_compactor_garbage_collection_duration_seconds_bucket{le="0.3"} 0 - cortex_compactor_garbage_collection_duration_seconds_bucket{le="0.6"} 0 - cortex_compactor_garbage_collection_duration_seconds_bucket{le="1"} 0 - cortex_compactor_garbage_collection_duration_seconds_bucket{le="3"} 0 - cortex_compactor_garbage_collection_duration_seconds_bucket{le="6"} 0 - cortex_compactor_garbage_collection_duration_seconds_bucket{le="9"} 1 - cortex_compactor_garbage_collection_duration_seconds_bucket{le="20"} 2 - cortex_compactor_garbage_collection_duration_seconds_bucket{le="30"} 2 - cortex_compactor_garbage_collection_duration_seconds_bucket{le="60"} 3 - cortex_compactor_garbage_collection_duration_seconds_bucket{le="90"} 3 - cortex_compactor_garbage_collection_duration_seconds_bucket{le="120"} 3 - cortex_compactor_garbage_collection_duration_seconds_bucket{le="240"} 3 - cortex_compactor_garbage_collection_duration_seconds_bucket{le="360"} 3 - cortex_compactor_garbage_collection_duration_seconds_bucket{le="720"} 3 - cortex_compactor_garbage_collection_duration_seconds_bucket{le="+Inf"} 3 - cortex_compactor_garbage_collection_duration_seconds_sum 77.777 - cortex_compactor_garbage_collection_duration_seconds_count 3 - - # HELP cortex_compactor_group_compactions_total Total number of group compaction attempts that resulted in a new block. - # TYPE cortex_compactor_group_compactions_total counter - # Sum across all groups - cortex_compactor_group_compactions_total 2999970 - - # HELP cortex_compactor_group_compaction_runs_started_total Total number of group compaction attempts. - # TYPE cortex_compactor_group_compaction_runs_started_total counter - # Sum across all groups - cortex_compactor_group_compaction_runs_started_total 3999960 - - # HELP cortex_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. - # TYPE cortex_compactor_group_compaction_runs_completed_total counter - # Sum across all groups - cortex_compactor_group_compaction_runs_completed_total 4999950 - - # HELP cortex_compactor_group_compactions_failures_total Total number of failed group compactions. - # TYPE cortex_compactor_group_compactions_failures_total counter - cortex_compactor_group_compactions_failures_total 5999940 - - # HELP cortex_compactor_group_vertical_compactions_total Total number of group compaction attempts that resulted in a new block based on overlapping blocks. - # TYPE cortex_compactor_group_vertical_compactions_total counter - cortex_compactor_group_vertical_compactions_total 6999930 - `)) - require.NoError(t, err) -} - -func generateTestData(base float64) *prometheus.Registry { - r := prometheus.NewRegistry() - m := newTestSyncerMetrics(r) - m.metaSync.Add(1 * base) - m.metaSyncFailures.Add(2 * base) - m.metaSyncDuration.Observe(3 * base / 10000) - m.metaSyncConsistencyDelay.Set(300) - m.garbageCollections.Add(5 * base) - m.garbageCollectionFailures.Add(6 * base) - m.garbageCollectionDuration.Observe(7 * base / 10000) - m.compactions.WithLabelValues("aaa").Add(8 * base) - m.compactions.WithLabelValues("bbb").Add(9 * base) - m.compactions.WithLabelValues("ccc").Add(10 * base) - m.compactionRunsStarted.WithLabelValues("aaa").Add(11 * base) - m.compactionRunsStarted.WithLabelValues("bbb").Add(12 * base) - m.compactionRunsStarted.WithLabelValues("ccc").Add(13 * base) - m.compactionRunsCompleted.WithLabelValues("aaa").Add(14 * base) - m.compactionRunsCompleted.WithLabelValues("bbb").Add(15 * base) - m.compactionRunsCompleted.WithLabelValues("ccc").Add(16 * base) - m.compactionFailures.WithLabelValues("aaa").Add(17 * base) - m.compactionFailures.WithLabelValues("bbb").Add(18 * base) - m.compactionFailures.WithLabelValues("ccc").Add(19 * base) - m.verticalCompactions.WithLabelValues("aaa").Add(20 * base) - m.verticalCompactions.WithLabelValues("bbb").Add(21 * base) - m.verticalCompactions.WithLabelValues("ccc").Add(22 * base) - return r -} - -// directly copied from Thanos (and renamed syncerMetrics to testSyncerMetrics to avoid conflict) -type testSyncerMetrics struct { - metaSync prometheus.Counter - metaSyncFailures prometheus.Counter - metaSyncDuration prometheus.Histogram - metaSyncConsistencyDelay prometheus.Gauge - garbageCollections prometheus.Counter - garbageCollectionFailures prometheus.Counter - garbageCollectionDuration prometheus.Histogram - compactions *prometheus.CounterVec - compactionRunsStarted *prometheus.CounterVec - compactionRunsCompleted *prometheus.CounterVec - compactionFailures *prometheus.CounterVec - verticalCompactions *prometheus.CounterVec -} - -func newTestSyncerMetrics(reg prometheus.Registerer) *testSyncerMetrics { - var m testSyncerMetrics - - m.metaSync = prometheus.NewCounter(prometheus.CounterOpts{ - Name: "blocks_meta_syncs_total", - Help: "Total blocks metadata synchronization attempts.", - }) - m.metaSyncFailures = prometheus.NewCounter(prometheus.CounterOpts{ - Name: "blocks_meta_sync_failures_total", - Help: "Total blocks metadata synchronization failures.", - }) - m.metaSyncDuration = prometheus.NewHistogram(prometheus.HistogramOpts{ - Name: "blocks_meta_sync_duration_seconds", - Help: "Duration of the blocks metadata synchronization in seconds.", - Buckets: []float64{0.01, 0.1, 0.3, 0.6, 1, 3, 6, 9, 20, 30, 60, 90, 120, 240, 360, 720}, - }) - m.metaSyncConsistencyDelay = prometheus.NewGauge(prometheus.GaugeOpts{ - Name: "consistency_delay_seconds", - Help: "Configured consistency delay in seconds.", - }) - - m.garbageCollections = prometheus.NewCounter(prometheus.CounterOpts{ - Name: "thanos_compact_garbage_collection_total", - Help: "Total number of garbage collection operations.", - }) - m.garbageCollectionFailures = prometheus.NewCounter(prometheus.CounterOpts{ - Name: "thanos_compact_garbage_collection_failures_total", - Help: "Total number of failed garbage collection operations.", - }) - m.garbageCollectionDuration = prometheus.NewHistogram(prometheus.HistogramOpts{ - Name: "thanos_compact_garbage_collection_duration_seconds", - Help: "Time it took to perform garbage collection iteration.", - Buckets: []float64{0.01, 0.1, 0.3, 0.6, 1, 3, 6, 9, 20, 30, 60, 90, 120, 240, 360, 720}, - }) - - m.compactions = prometheus.NewCounterVec(prometheus.CounterOpts{ - Name: "thanos_compact_group_compactions_total", - Help: "Total number of group compaction attempts that resulted in a new block.", - }, []string{"group"}) - m.compactionRunsStarted = prometheus.NewCounterVec(prometheus.CounterOpts{ - Name: "thanos_compact_group_compaction_runs_started_total", - Help: "Total number of group compaction attempts.", - }, []string{"group"}) - m.compactionRunsCompleted = prometheus.NewCounterVec(prometheus.CounterOpts{ - Name: "thanos_compact_group_compaction_runs_completed_total", - Help: "Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction.", - }, []string{"group"}) - m.compactionFailures = prometheus.NewCounterVec(prometheus.CounterOpts{ - Name: "thanos_compact_group_compactions_failures_total", - Help: "Total number of failed group compactions.", - }, []string{"group"}) - m.verticalCompactions = prometheus.NewCounterVec(prometheus.CounterOpts{ - Name: "thanos_compact_group_vertical_compactions_total", - Help: "Total number of group compaction attempts that resulted in a new block based on overlapping blocks.", - }, []string{"group"}) - - if reg != nil { - reg.MustRegister( - m.metaSync, - m.metaSyncFailures, - m.metaSyncDuration, - m.metaSyncConsistencyDelay, - m.garbageCollections, - m.garbageCollectionFailures, - m.garbageCollectionDuration, - m.compactions, - m.compactionRunsStarted, - m.compactionRunsCompleted, - m.compactionFailures, - m.verticalCompactions, - ) - } - return &m -} From 01f4d9d522e957417fef286de38fb267a81f6c49 Mon Sep 17 00:00:00 2001 From: Alex Le Date: Tue, 23 Jul 2024 16:46:40 -0700 Subject: [PATCH 11/32] Split cleaner cycle for active and deleted tenants (#6112) * Split cleaner cycle for active and deleted tenants Signed-off-by: Alex Le * update CHANGELOG Signed-off-by: Alex Le * refactor code Signed-off-by: Alex Le * rename label Signed-off-by: Alex Le --------- Signed-off-by: Alex Le --- CHANGELOG.md | 1 + pkg/compactor/blocks_cleaner.go | 257 ++++++++++++++++++++------- pkg/compactor/blocks_cleaner_test.go | 78 +++++--- pkg/compactor/compactor_test.go | 97 ++++------ 4 files changed, 281 insertions(+), 152 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 1a9c050e98..f04808f267 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -30,6 +30,7 @@ * [ENHANCEMENT] Compactor: Centralize metrics used by compactor and add user label to compactor metrics. #6096 * [ENHANCEMENT] Compactor: Add unique execution ID for each compaction cycle in log for easy debugging. #6097 * [ENHANCEMENT] Ruler: Add support for filtering by `state` and `health` field on Rules API. #6040 +* [ENHANCEMENT] Compactor: Split cleaner cycle for active and deleted tenants. #6112 * [BUGFIX] Configsdb: Fix endline issue in db password. #5920 * [BUGFIX] Ingester: Fix `user` and `type` labels for the `cortex_ingester_tsdb_head_samples_appended_total` TSDB metric. #5952 * [BUGFIX] Querier: Enforce max query length check for `/api/v1/series` API even though `ignoreMaxQueryLength` is set to true. #6018 diff --git a/pkg/compactor/blocks_cleaner.go b/pkg/compactor/blocks_cleaner.go index a9c4fa7f3d..a447231b64 100644 --- a/pkg/compactor/blocks_cleaner.go +++ b/pkg/compactor/blocks_cleaner.go @@ -15,6 +15,7 @@ import ( "github.com/thanos-io/objstore" "github.com/thanos-io/thanos/pkg/block" "github.com/thanos-io/thanos/pkg/block/metadata" + "go.uber.org/atomic" "github.com/cortexproject/cortex/pkg/storage/bucket" cortex_tsdb "github.com/cortexproject/cortex/pkg/storage/tsdb" @@ -28,6 +29,8 @@ import ( const ( defaultDeleteBlocksConcurrency = 16 reasonValueRetention = "retention" + activeStatus = "active" + deletedStatus = "deleted" ) type BlocksCleanerConfig struct { @@ -51,10 +54,10 @@ type BlocksCleaner struct { lastOwnedUsers []string // Metrics. - runsStarted prometheus.Counter - runsCompleted prometheus.Counter - runsFailed prometheus.Counter - runsLastSuccess prometheus.Gauge + runsStarted *prometheus.CounterVec + runsCompleted *prometheus.CounterVec + runsFailed *prometheus.CounterVec + runsLastSuccess *prometheus.GaugeVec blocksCleanedTotal prometheus.Counter blocksFailedTotal prometheus.Counter blocksMarkedForDeletion *prometheus.CounterVec @@ -63,6 +66,8 @@ type BlocksCleaner struct { tenantBlocksMarkedForNoCompaction *prometheus.GaugeVec tenantPartialBlocks *prometheus.GaugeVec tenantBucketIndexLastUpdate *prometheus.GaugeVec + tenantBlocksCleanedTotal *prometheus.CounterVec + tenantCleanDuration *prometheus.GaugeVec } func NewBlocksCleaner( @@ -80,22 +85,22 @@ func NewBlocksCleaner( usersScanner: usersScanner, cfgProvider: cfgProvider, logger: log.With(logger, "component", "cleaner"), - runsStarted: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + runsStarted: promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ Name: "cortex_compactor_block_cleanup_started_total", Help: "Total number of blocks cleanup runs started.", - }), - runsCompleted: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + }, []string{"user_status"}), + runsCompleted: promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ Name: "cortex_compactor_block_cleanup_completed_total", Help: "Total number of blocks cleanup runs successfully completed.", - }), - runsFailed: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + }, []string{"user_status"}), + runsFailed: promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ Name: "cortex_compactor_block_cleanup_failed_total", Help: "Total number of blocks cleanup runs failed.", - }), - runsLastSuccess: promauto.With(reg).NewGauge(prometheus.GaugeOpts{ + }, []string{"user_status"}), + runsLastSuccess: promauto.With(reg).NewGaugeVec(prometheus.GaugeOpts{ Name: "cortex_compactor_block_cleanup_last_successful_run_timestamp_seconds", Help: "Unix timestamp of the last successful blocks cleanup run.", - }), + }, []string{"user_status"}), blocksCleanedTotal: promauto.With(reg).NewCounter(prometheus.CounterOpts{ Name: "cortex_compactor_blocks_cleaned_total", Help: "Total number of blocks deleted.", @@ -129,54 +134,162 @@ func NewBlocksCleaner( Name: "cortex_bucket_index_last_successful_update_timestamp_seconds", Help: "Timestamp of the last successful update of a tenant's bucket index.", }, []string{"user"}), + tenantBlocksCleanedTotal: promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Name: "cortex_bucket_blocks_cleaned_total", + Help: "Total number of blocks deleted for a tenant.", + }, commonLabels), + tenantCleanDuration: promauto.With(reg).NewGaugeVec(prometheus.GaugeOpts{ + Name: "cortex_bucket_clean_duration_seconds", + Help: "Duration of cleaner runtime for a tenant in seconds", + }, commonLabels), } - c.Service = services.NewTimerService(cfg.CleanupInterval, c.starting, c.ticker, nil) + c.Service = services.NewBasicService(c.starting, c.loop, nil) return c } +type cleanerJob struct { + users []string + timestamp int64 +} + func (c *BlocksCleaner) starting(ctx context.Context) error { // Run a cleanup so that any other service depending on this service // is guaranteed to start once the initial cleanup has been done. - c.runCleanup(ctx, true) + activeUsers, deletedUsers, err := c.scanUsers(ctx) + if err != nil { + level.Error(c.logger).Log("msg", "failed to scan users on startup", "err", err.Error()) + c.runsFailed.WithLabelValues(deletedStatus).Inc() + c.runsFailed.WithLabelValues(activeStatus).Inc() + return nil + } + err = c.cleanUpActiveUsers(ctx, activeUsers, true) + c.checkRunError(activeStatus, err) + err = c.cleanDeletedUsers(ctx, deletedUsers) + c.checkRunError(deletedStatus, err) return nil } -func (c *BlocksCleaner) ticker(ctx context.Context) error { - c.runCleanup(ctx, false) +func (c *BlocksCleaner) loop(ctx context.Context) error { + t := time.NewTicker(c.cfg.CleanupInterval) + defer t.Stop() - return nil -} + usersChan := make(chan *cleanerJob) + deleteChan := make(chan *cleanerJob) + defer close(usersChan) + defer close(deleteChan) + + go func() { + c.runActiveUserCleanup(ctx, usersChan) + }() + go func() { + c.runDeleteUserCleanup(ctx, deleteChan) + }() + + for { + select { + case <-t.C: + activeUsers, deletedUsers, err := c.scanUsers(ctx) + if err != nil { + level.Error(c.logger).Log("msg", "failed to scan users blocks cleanup and maintenance", "err", err.Error()) + c.runsFailed.WithLabelValues(deletedStatus).Inc() + c.runsFailed.WithLabelValues(activeStatus).Inc() + continue + } + cleanJobTimestamp := time.Now().Unix() + usersChan <- &cleanerJob{ + users: activeUsers, + timestamp: cleanJobTimestamp, + } + deleteChan <- &cleanerJob{ + users: deletedUsers, + timestamp: cleanJobTimestamp, + } -func (c *BlocksCleaner) runCleanup(ctx context.Context, firstRun bool) { - level.Info(c.logger).Log("msg", "started blocks cleanup and maintenance") - c.runsStarted.Inc() + case <-ctx.Done(): + return nil + } + } +} - if err := c.cleanUsers(ctx, firstRun); err == nil { - level.Info(c.logger).Log("msg", "successfully completed blocks cleanup and maintenance") - c.runsCompleted.Inc() - c.runsLastSuccess.SetToCurrentTime() +func (c *BlocksCleaner) checkRunError(runType string, err error) { + if err == nil { + level.Info(c.logger).Log("msg", fmt.Sprintf("successfully completed blocks cleanup and maintenance for %s users", runType)) + c.runsCompleted.WithLabelValues(runType).Inc() + c.runsLastSuccess.WithLabelValues(runType).SetToCurrentTime() } else if errors.Is(err, context.Canceled) { - level.Info(c.logger).Log("msg", "canceled blocks cleanup and maintenance", "err", err) - return + level.Info(c.logger).Log("msg", fmt.Sprintf("canceled blocks cleanup and maintenance for %s users", runType), "err", err) } else { - level.Error(c.logger).Log("msg", "failed to run blocks cleanup and maintenance", "err", err.Error()) - c.runsFailed.Inc() + level.Error(c.logger).Log("msg", fmt.Sprintf("failed to run blocks cleanup and maintenance for %s users", runType), "err", err.Error()) + c.runsFailed.WithLabelValues(runType).Inc() } } -func (c *BlocksCleaner) cleanUsers(ctx context.Context, firstRun bool) error { +func (c *BlocksCleaner) runActiveUserCleanup(ctx context.Context, jobChan chan *cleanerJob) { + for job := range jobChan { + if job.timestamp < time.Now().Add(-c.cfg.CleanupInterval).Unix() { + level.Warn(c.logger).Log("Active user cleaner job too old. Ignoring to get recent data") + continue + } + err := c.cleanUpActiveUsers(ctx, job.users, false) + + c.checkRunError(activeStatus, err) + } +} + +func (c *BlocksCleaner) cleanUpActiveUsers(ctx context.Context, users []string, firstRun bool) error { + level.Info(c.logger).Log("msg", "started blocks cleanup and maintenance for active users") + c.runsStarted.WithLabelValues(activeStatus).Inc() + + return concurrency.ForEachUser(ctx, users, c.cfg.CleanupConcurrency, func(ctx context.Context, userID string) error { + userLogger := util_log.WithUserID(userID, c.logger) + userBucket := bucket.NewUserBucketClient(userID, c.bucketClient, c.cfgProvider) + errChan := make(chan error, 1) + defer func() { + errChan <- nil + }() + return errors.Wrapf(c.cleanUser(ctx, userLogger, userBucket, userID, firstRun), "failed to delete blocks for user: %s", userID) + }) +} + +func (c *BlocksCleaner) runDeleteUserCleanup(ctx context.Context, jobChan chan *cleanerJob) { + for job := range jobChan { + if job.timestamp < time.Now().Add(-c.cfg.CleanupInterval).Unix() { + level.Warn(c.logger).Log("Delete users cleaner job too old. Ignoring to get recent data") + continue + } + err := c.cleanDeletedUsers(ctx, job.users) + + c.checkRunError(deletedStatus, err) + } +} + +func (c *BlocksCleaner) cleanDeletedUsers(ctx context.Context, users []string) error { + level.Info(c.logger).Log("msg", "started blocks cleanup and maintenance for deleted users") + c.runsStarted.WithLabelValues(deletedStatus).Inc() + + return concurrency.ForEachUser(ctx, users, c.cfg.CleanupConcurrency, func(ctx context.Context, userID string) error { + userLogger := util_log.WithUserID(userID, c.logger) + userBucket := bucket.NewUserBucketClient(userID, c.bucketClient, c.cfgProvider) + errChan := make(chan error, 1) + defer func() { + errChan <- nil + }() + return errors.Wrapf(c.deleteUserMarkedForDeletion(ctx, userLogger, userBucket, userID), "failed to delete user marked for deletion: %s", userID) + }) +} + +func (c *BlocksCleaner) scanUsers(ctx context.Context) ([]string, []string, error) { users, deleted, err := c.usersScanner.ScanUsers(ctx) if err != nil { - return errors.Wrap(err, "failed to discover users from bucket") + return nil, nil, errors.Wrap(err, "failed to discover users from bucket") } isActive := util.StringsMap(users) isDeleted := util.StringsMap(deleted) allUsers := append(users, deleted...) - // Delete per-tenant metrics for all tenants not belonging anymore to this shard. // Such tenants have been moved to a different shard, so their updated metrics will // be exported by the new shard. @@ -191,18 +304,11 @@ func (c *BlocksCleaner) cleanUsers(ctx context.Context, firstRun bool) error { } c.lastOwnedUsers = allUsers - return concurrency.ForEachUser(ctx, allUsers, c.cfg.CleanupConcurrency, func(ctx context.Context, userID string) error { - if isDeleted[userID] { - return errors.Wrapf(c.deleteUserMarkedForDeletion(ctx, userID), "failed to delete user marked for deletion: %s", userID) - } - return errors.Wrapf(c.cleanUser(ctx, userID, firstRun), "failed to delete blocks for user: %s", userID) - }) + return users, deleted, nil } // Remove blocks and remaining data for tenant marked for deletion. -func (c *BlocksCleaner) deleteUserMarkedForDeletion(ctx context.Context, userID string) error { - userLogger := util_log.WithUserID(userID, c.logger) - userBucket := bucket.NewUserBucketClient(userID, c.bucketClient, c.cfgProvider) +func (c *BlocksCleaner) deleteUserMarkedForDeletion(ctx context.Context, userLogger log.Logger, userBucket objstore.InstrumentedBucket, userID string) error { level.Info(userLogger).Log("msg", "deleting blocks for tenant marked for deletion") @@ -211,51 +317,58 @@ func (c *BlocksCleaner) deleteUserMarkedForDeletion(ctx context.Context, userID if err := bucketindex.DeleteIndex(ctx, c.bucketClient, userID, c.cfgProvider); err != nil { return err } - // Delete the bucket sync status if err := bucketindex.DeleteIndexSyncStatus(ctx, c.bucketClient, userID); err != nil { return err } c.tenantBucketIndexLastUpdate.DeleteLabelValues(userID) - var deletedBlocks, failed int + var blocksToDelete []interface{} err := userBucket.Iter(ctx, "", func(name string) error { if err := ctx.Err(); err != nil { return err } - id, ok := block.IsBlockDir(name) if !ok { return nil } + blocksToDelete = append(blocksToDelete, id) + return nil + }) + if err != nil { + return err + } - err := block.Delete(ctx, userLogger, userBucket, id) + var deletedBlocks, failed atomic.Int64 + err = concurrency.ForEach(ctx, blocksToDelete, defaultDeleteBlocksConcurrency, func(ctx context.Context, job interface{}) error { + blockID := job.(ulid.ULID) + err := block.Delete(ctx, userLogger, userBucket, blockID) if err != nil { - failed++ + failed.Add(1) c.blocksFailedTotal.Inc() - level.Warn(userLogger).Log("msg", "failed to delete block", "block", id, "err", err) + level.Warn(userLogger).Log("msg", "failed to delete block", "block", blockID, "err", err) return nil // Continue with other blocks. } - deletedBlocks++ + deletedBlocks.Add(1) c.blocksCleanedTotal.Inc() - level.Info(userLogger).Log("msg", "deleted block", "block", id) + c.tenantBlocksCleanedTotal.WithLabelValues(userID).Inc() + level.Info(userLogger).Log("msg", "deleted block", "block", blockID) return nil }) - if err != nil { return err } - if failed > 0 { + if failed.Load() > 0 { // The number of blocks left in the storage is equal to the number of blocks we failed // to delete. We also consider them all marked for deletion given the next run will try // to delete them again. - c.tenantBlocks.WithLabelValues(userID).Set(float64(failed)) - c.tenantBlocksMarkedForDelete.WithLabelValues(userID).Set(float64(failed)) + c.tenantBlocks.WithLabelValues(userID).Set(float64(failed.Load())) + c.tenantBlocksMarkedForDelete.WithLabelValues(userID).Set(float64(failed.Load())) c.tenantPartialBlocks.WithLabelValues(userID).Set(0) - return errors.Errorf("failed to delete %d blocks", failed) + return errors.Errorf("failed to delete %d blocks", failed.Load()) } // Given all blocks have been deleted, we can also remove the metrics. @@ -264,8 +377,8 @@ func (c *BlocksCleaner) deleteUserMarkedForDeletion(ctx context.Context, userID c.tenantBlocksMarkedForNoCompaction.DeleteLabelValues(userID) c.tenantPartialBlocks.DeleteLabelValues(userID) - if deletedBlocks > 0 { - level.Info(userLogger).Log("msg", "deleted blocks for tenant marked for deletion", "deletedBlocks", deletedBlocks) + if deletedBlocks.Load() > 0 { + level.Info(userLogger).Log("msg", "deleted blocks for tenant marked for deletion", "deletedBlocks", deletedBlocks.Load()) } mark, err := cortex_tsdb.ReadTenantDeletionMark(ctx, c.bucketClient, userID) @@ -275,22 +388,18 @@ func (c *BlocksCleaner) deleteUserMarkedForDeletion(ctx context.Context, userID if mark == nil { return errors.Wrap(err, "cannot find tenant deletion mark anymore") } - // If we have just deleted some blocks, update "finished" time. Also update "finished" time if it wasn't set yet, but there are no blocks. // Note: this UPDATES the tenant deletion mark. Components that use caching bucket will NOT SEE this update, // but that is fine -- they only check whether tenant deletion marker exists or not. - if deletedBlocks > 0 || mark.FinishedTime == 0 { + if deletedBlocks.Load() > 0 || mark.FinishedTime == 0 { level.Info(userLogger).Log("msg", "updating finished time in tenant deletion mark") mark.FinishedTime = time.Now().Unix() return errors.Wrap(cortex_tsdb.WriteTenantDeletionMark(ctx, c.bucketClient, userID, mark), "failed to update tenant deletion mark") } - if time.Since(time.Unix(mark.FinishedTime, 0)) < c.cfg.TenantCleanupDelay { return nil } - level.Info(userLogger).Log("msg", "cleaning up remaining blocks data for tenant marked for deletion") - // Let's do final cleanup of tenant. if deleted, err := bucket.DeletePrefix(ctx, userBucket, block.DebugMetas, userLogger); err != nil { return errors.Wrap(err, "failed to delete "+block.DebugMetas) @@ -303,17 +412,14 @@ func (c *BlocksCleaner) deleteUserMarkedForDeletion(ctx context.Context, userID } else if deleted > 0 { level.Info(userLogger).Log("msg", "deleted marker files for tenant marked for deletion", "count", deleted) } - if err := cortex_tsdb.DeleteTenantDeletionMark(ctx, c.bucketClient, userID); err != nil { return errors.Wrap(err, "failed to delete tenant deletion mark") } - return nil } -func (c *BlocksCleaner) cleanUser(ctx context.Context, userID string, firstRun bool) (returnErr error) { - userLogger := util_log.WithUserID(userID, c.logger) - userBucket := bucket.NewUserBucketClient(userID, c.bucketClient, c.cfgProvider) +func (c *BlocksCleaner) cleanUser(ctx context.Context, userLogger log.Logger, userBucket objstore.InstrumentedBucket, userID string, firstRun bool) (returnErr error) { + c.blocksMarkedForDeletion.WithLabelValues(userID, reasonValueRetention) startTime := time.Now() level.Info(userLogger).Log("msg", "started blocks cleanup and maintenance") @@ -323,6 +429,7 @@ func (c *BlocksCleaner) cleanUser(ctx context.Context, userID string, firstRun b } else { level.Info(userLogger).Log("msg", "completed blocks cleanup and maintenance", "duration", time.Since(startTime)) } + c.tenantCleanDuration.WithLabelValues(userID).Set(time.Since(startTime).Seconds()) }() // Migrate block deletion marks to the global markers location. This operation is a best-effort. @@ -346,6 +453,7 @@ func (c *BlocksCleaner) cleanUser(ctx context.Context, userID string, firstRun b idxs.SyncTime = time.Now().Unix() // Read the bucket index. + begin := time.Now() idx, err := bucketindex.ReadIndex(ctx, c.bucketClient, userID, c.cfgProvider, c.logger) defer func() { @@ -370,6 +478,7 @@ func (c *BlocksCleaner) cleanUser(ctx context.Context, userID string, firstRun b idxs.Status = bucketindex.GenericError return err } + level.Info(userLogger).Log("msg", "finish reading index", "duration", time.Since(begin), "duration_ms", time.Since(begin).Milliseconds()) // Mark blocks for future deletion based on the retention period for the user. // Note doing this before UpdateIndex, so it reads in the deletion marks. @@ -383,15 +492,18 @@ func (c *BlocksCleaner) cleanUser(ctx context.Context, userID string, firstRun b } // Generate an updated in-memory version of the bucket index. + begin = time.Now() w := bucketindex.NewUpdater(c.bucketClient, userID, c.cfgProvider, c.logger) idx, partials, totalBlocksBlocksMarkedForNoCompaction, err := w.UpdateIndex(ctx, idx) if err != nil { idxs.Status = bucketindex.GenericError return err } + level.Info(userLogger).Log("msg", "finish updating index", "duration", time.Since(begin), "duration_ms", time.Since(begin).Milliseconds()) // Delete blocks marked for deletion. We iterate over a copy of deletion marks because // we'll need to manipulate the index (removing blocks which get deleted). + begin = time.Now() blocksToDelete := make([]interface{}, 0, len(idx.BlockDeletionMarks)) var mux sync.Mutex for _, mark := range idx.BlockDeletionMarks.Clone() { @@ -400,8 +512,10 @@ func (c *BlocksCleaner) cleanUser(ctx context.Context, userID string, firstRun b } blocksToDelete = append(blocksToDelete, mark.ID) } + level.Info(userLogger).Log("msg", "finish getting blocks to be deleted", "duration", time.Since(begin), "duration_ms", time.Since(begin).Milliseconds()) // Concurrently deletes blocks marked for deletion, and removes blocks from index. + begin = time.Now() _ = concurrency.ForEach(ctx, blocksToDelete, defaultDeleteBlocksConcurrency, func(ctx context.Context, job interface{}) error { blockID := job.(ulid.ULID) @@ -417,20 +531,26 @@ func (c *BlocksCleaner) cleanUser(ctx context.Context, userID string, firstRun b mux.Unlock() c.blocksCleanedTotal.Inc() + c.tenantBlocksCleanedTotal.WithLabelValues(userID).Inc() level.Info(userLogger).Log("msg", "deleted block marked for deletion", "block", blockID) return nil }) + level.Info(userLogger).Log("msg", "finish deleting blocks", "duration", time.Since(begin), "duration_ms", time.Since(begin).Milliseconds()) // Partial blocks with a deletion mark can be cleaned up. This is a best effort, so we don't return // error if the cleanup of partial blocks fail. if len(partials) > 0 { - c.cleanUserPartialBlocks(ctx, partials, idx, userBucket, userLogger) + begin = time.Now() + c.cleanUserPartialBlocks(ctx, userID, partials, idx, userBucket, userLogger) + level.Info(userLogger).Log("msg", "finish cleaning partial blocks", "duration", time.Since(begin), "duration_ms", time.Since(begin).Milliseconds()) } // Upload the updated index to the storage. + begin = time.Now() if err := bucketindex.WriteIndex(ctx, c.bucketClient, userID, c.cfgProvider, idx); err != nil { return err } + level.Info(userLogger).Log("msg", "finish writing new index", "duration", time.Since(begin), "duration_ms", time.Since(begin).Milliseconds()) c.tenantBlocks.WithLabelValues(userID).Set(float64(len(idx.Blocks))) c.tenantBlocksMarkedForDelete.WithLabelValues(userID).Set(float64(len(idx.BlockDeletionMarks))) @@ -442,7 +562,7 @@ func (c *BlocksCleaner) cleanUser(ctx context.Context, userID string, firstRun b // cleanUserPartialBlocks delete partial blocks which are safe to be deleted. The provided partials map // and index are updated accordingly. -func (c *BlocksCleaner) cleanUserPartialBlocks(ctx context.Context, partials map[ulid.ULID]error, idx *bucketindex.Index, userBucket objstore.InstrumentedBucket, userLogger log.Logger) { +func (c *BlocksCleaner) cleanUserPartialBlocks(ctx context.Context, userID string, partials map[ulid.ULID]error, idx *bucketindex.Index, userBucket objstore.InstrumentedBucket, userLogger log.Logger) { // Collect all blocks with missing meta.json into buffered channel. blocks := make([]interface{}, 0, len(partials)) @@ -497,6 +617,7 @@ func (c *BlocksCleaner) cleanUserPartialBlocks(ctx context.Context, partials map mux.Unlock() c.blocksCleanedTotal.Inc() + c.tenantBlocksCleanedTotal.WithLabelValues(userID).Inc() level.Info(userLogger).Log("msg", "deleted partial block marked for deletion", "block", blockID) return nil }) diff --git a/pkg/compactor/blocks_cleaner_test.go b/pkg/compactor/blocks_cleaner_test.go index e33994df8b..b582fceb7c 100644 --- a/pkg/compactor/blocks_cleaner_test.go +++ b/pkg/compactor/blocks_cleaner_test.go @@ -20,9 +20,11 @@ import ( "github.com/thanos-io/thanos/pkg/block" "github.com/thanos-io/thanos/pkg/block/metadata" + "github.com/cortexproject/cortex/pkg/storage/bucket" "github.com/cortexproject/cortex/pkg/storage/tsdb" "github.com/cortexproject/cortex/pkg/storage/tsdb/bucketindex" cortex_testutil "github.com/cortexproject/cortex/pkg/storage/tsdb/testutil" + util_log "github.com/cortexproject/cortex/pkg/util/log" "github.com/cortexproject/cortex/pkg/util/services" ) @@ -89,7 +91,9 @@ func TestBlockCleaner_KeyPermissionDenied(t *testing.T) { // Clean User with no error cleaner.bucketClient = bkt - err := cleaner.cleanUser(ctx, userID, false) + userLogger := util_log.WithUserID(userID, cleaner.logger) + userBucket := bucket.NewUserBucketClient(userID, cleaner.bucketClient, cleaner.cfgProvider) + err := cleaner.cleanUser(ctx, userLogger, userBucket, userID, false) require.NoError(t, err) s, err := bucketindex.ReadSyncStatus(ctx, bkt, userID, logger) require.NoError(t, err) @@ -98,7 +102,9 @@ func TestBlockCleaner_KeyPermissionDenied(t *testing.T) { // Clean with cmk error cleaner.bucketClient = mbucket - err = cleaner.cleanUser(ctx, userID, false) + userLogger = util_log.WithUserID(userID, cleaner.logger) + userBucket = bucket.NewUserBucketClient(userID, cleaner.bucketClient, cleaner.cfgProvider) + err = cleaner.cleanUser(ctx, userLogger, userBucket, userID, false) require.NoError(t, err) s, err = bucketindex.ReadSyncStatus(ctx, bkt, userID, logger) require.NoError(t, err) @@ -107,7 +113,9 @@ func TestBlockCleaner_KeyPermissionDenied(t *testing.T) { // Re grant access to the key cleaner.bucketClient = bkt - err = cleaner.cleanUser(ctx, userID, false) + userLogger = util_log.WithUserID(userID, cleaner.logger) + userBucket = bucket.NewUserBucketClient(userID, cleaner.bucketClient, cleaner.cfgProvider) + err = cleaner.cleanUser(ctx, userLogger, userBucket, userID, false) require.NoError(t, err) s, err = bucketindex.ReadSyncStatus(ctx, bkt, userID, logger) require.NoError(t, err) @@ -238,9 +246,9 @@ func testBlocksCleanerWithOptions(t *testing.T, options testBlocksCleanerOptions assert.Equal(t, tc.expectedExists, exists, tc.user) } - assert.Equal(t, float64(1), testutil.ToFloat64(cleaner.runsStarted)) - assert.Equal(t, float64(1), testutil.ToFloat64(cleaner.runsCompleted)) - assert.Equal(t, float64(0), testutil.ToFloat64(cleaner.runsFailed)) + assert.Equal(t, float64(1), testutil.ToFloat64(cleaner.runsStarted.WithLabelValues(activeStatus))) + assert.Equal(t, float64(1), testutil.ToFloat64(cleaner.runsCompleted.WithLabelValues(activeStatus))) + assert.Equal(t, float64(0), testutil.ToFloat64(cleaner.runsFailed.WithLabelValues(activeStatus))) assert.Equal(t, float64(7), testutil.ToFloat64(cleaner.blocksCleanedTotal)) assert.Equal(t, float64(0), testutil.ToFloat64(cleaner.blocksFailedTotal)) @@ -365,9 +373,9 @@ func TestBlocksCleaner_ShouldContinueOnBlockDeletionFailure(t *testing.T) { assert.Equal(t, tc.expectedExists, exists, tc.path) } - assert.Equal(t, float64(1), testutil.ToFloat64(cleaner.runsStarted)) - assert.Equal(t, float64(1), testutil.ToFloat64(cleaner.runsCompleted)) - assert.Equal(t, float64(0), testutil.ToFloat64(cleaner.runsFailed)) + assert.Equal(t, float64(1), testutil.ToFloat64(cleaner.runsStarted.WithLabelValues(activeStatus))) + assert.Equal(t, float64(1), testutil.ToFloat64(cleaner.runsCompleted.WithLabelValues(activeStatus))) + assert.Equal(t, float64(0), testutil.ToFloat64(cleaner.runsFailed.WithLabelValues(activeStatus))) assert.Equal(t, float64(2), testutil.ToFloat64(cleaner.blocksCleanedTotal)) assert.Equal(t, float64(1), testutil.ToFloat64(cleaner.blocksFailedTotal)) @@ -428,9 +436,9 @@ func TestBlocksCleaner_ShouldRebuildBucketIndexOnCorruptedOne(t *testing.T) { assert.Equal(t, tc.expectedExists, exists, tc.path) } - assert.Equal(t, float64(1), testutil.ToFloat64(cleaner.runsStarted)) - assert.Equal(t, float64(1), testutil.ToFloat64(cleaner.runsCompleted)) - assert.Equal(t, float64(0), testutil.ToFloat64(cleaner.runsFailed)) + assert.Equal(t, float64(1), testutil.ToFloat64(cleaner.runsStarted.WithLabelValues(activeStatus))) + assert.Equal(t, float64(1), testutil.ToFloat64(cleaner.runsCompleted.WithLabelValues(activeStatus))) + assert.Equal(t, float64(0), testutil.ToFloat64(cleaner.runsFailed.WithLabelValues(activeStatus))) assert.Equal(t, float64(1), testutil.ToFloat64(cleaner.blocksCleanedTotal)) assert.Equal(t, float64(0), testutil.ToFloat64(cleaner.blocksFailedTotal)) @@ -470,7 +478,10 @@ func TestBlocksCleaner_ShouldRemoveMetricsForTenantsNotBelongingAnymoreToTheShar }, append(commonLabels, ReasonLabelName)) cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, reg, blocksMarkedForDeletion) - require.NoError(t, cleaner.cleanUsers(ctx, true)) + activeUsers, deleteUsers, err := cleaner.scanUsers(ctx) + require.NoError(t, err) + require.NoError(t, cleaner.cleanUpActiveUsers(ctx, activeUsers, true)) + require.NoError(t, cleaner.cleanDeletedUsers(ctx, deleteUsers)) assert.NoError(t, prom_testutil.GatherAndCompare(reg, strings.NewReader(` # HELP cortex_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. @@ -498,7 +509,10 @@ func TestBlocksCleaner_ShouldRemoveMetricsForTenantsNotBelongingAnymoreToTheShar createTSDBBlock(t, bucketClient, "user-1", 40, 50, nil) createTSDBBlock(t, bucketClient, "user-2", 50, 60, nil) - require.NoError(t, cleaner.cleanUsers(ctx, false)) + activeUsers, deleteUsers, err = cleaner.scanUsers(ctx) + require.NoError(t, err) + require.NoError(t, cleaner.cleanUpActiveUsers(ctx, activeUsers, false)) + require.NoError(t, cleaner.cleanDeletedUsers(ctx, deleteUsers)) assert.NoError(t, prom_testutil.GatherAndCompare(reg, strings.NewReader(` # HELP cortex_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. @@ -617,7 +631,10 @@ func TestBlocksCleaner_ShouldRemoveBlocksOutsideRetentionPeriod(t *testing.T) { cfgProvider.userRetentionPeriods["user-1"] = 0 cfgProvider.userRetentionPeriods["user-2"] = 0 - require.NoError(t, cleaner.cleanUsers(ctx, true)) + activeUsers, deleteUsers, err := cleaner.scanUsers(ctx) + require.NoError(t, err) + require.NoError(t, cleaner.cleanUpActiveUsers(ctx, activeUsers, true)) + require.NoError(t, cleaner.cleanDeletedUsers(ctx, deleteUsers)) assertBlockExists("user-1", block1, true) assertBlockExists("user-1", block2, true) assertBlockExists("user-2", block3, true) @@ -632,6 +649,10 @@ func TestBlocksCleaner_ShouldRemoveBlocksOutsideRetentionPeriod(t *testing.T) { # TYPE cortex_bucket_blocks_marked_for_deletion_count gauge cortex_bucket_blocks_marked_for_deletion_count{user="user-1"} 0 cortex_bucket_blocks_marked_for_deletion_count{user="user-2"} 0 + # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE cortex_compactor_blocks_marked_for_deletion_total counter + cortex_compactor_blocks_marked_for_deletion_total{reason="retention",user="user-1"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention",user="user-2"} 0 `), "cortex_bucket_blocks_count", "cortex_bucket_blocks_marked_for_deletion_count", @@ -643,7 +664,10 @@ func TestBlocksCleaner_ShouldRemoveBlocksOutsideRetentionPeriod(t *testing.T) { { cfgProvider.userRetentionPeriods["user-1"] = 9 * time.Hour - require.NoError(t, cleaner.cleanUsers(ctx, false)) + activeUsers, deleteUsers, err := cleaner.scanUsers(ctx) + require.NoError(t, err) + require.NoError(t, cleaner.cleanUpActiveUsers(ctx, activeUsers, false)) + require.NoError(t, cleaner.cleanDeletedUsers(ctx, deleteUsers)) assertBlockExists("user-1", block1, true) assertBlockExists("user-1", block2, true) assertBlockExists("user-2", block3, true) @@ -655,7 +679,10 @@ func TestBlocksCleaner_ShouldRemoveBlocksOutsideRetentionPeriod(t *testing.T) { { cfgProvider.userRetentionPeriods["user-1"] = 7 * time.Hour - require.NoError(t, cleaner.cleanUsers(ctx, false)) + activeUsers, deleteUsers, err := cleaner.scanUsers(ctx) + require.NoError(t, err) + require.NoError(t, cleaner.cleanUpActiveUsers(ctx, activeUsers, false)) + require.NoError(t, cleaner.cleanDeletedUsers(ctx, deleteUsers)) assertBlockExists("user-1", block1, true) assertBlockExists("user-1", block2, true) assertBlockExists("user-2", block3, true) @@ -673,6 +700,7 @@ func TestBlocksCleaner_ShouldRemoveBlocksOutsideRetentionPeriod(t *testing.T) { # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. # TYPE cortex_compactor_blocks_marked_for_deletion_total counter cortex_compactor_blocks_marked_for_deletion_total{reason="retention",user="user-1"} 1 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention",user="user-2"} 0 `), "cortex_bucket_blocks_count", "cortex_bucket_blocks_marked_for_deletion_count", @@ -682,7 +710,10 @@ func TestBlocksCleaner_ShouldRemoveBlocksOutsideRetentionPeriod(t *testing.T) { // Marking the block again, before the deletion occurs, should not cause an error. { - require.NoError(t, cleaner.cleanUsers(ctx, false)) + activeUsers, deleteUsers, err := cleaner.scanUsers(ctx) + require.NoError(t, err) + require.NoError(t, cleaner.cleanUpActiveUsers(ctx, activeUsers, false)) + require.NoError(t, cleaner.cleanDeletedUsers(ctx, deleteUsers)) assertBlockExists("user-1", block1, true) assertBlockExists("user-1", block2, true) assertBlockExists("user-2", block3, true) @@ -693,7 +724,10 @@ func TestBlocksCleaner_ShouldRemoveBlocksOutsideRetentionPeriod(t *testing.T) { { cleaner.cfg.DeletionDelay = 0 - require.NoError(t, cleaner.cleanUsers(ctx, false)) + activeUsers, deleteUsers, err := cleaner.scanUsers(ctx) + require.NoError(t, err) + require.NoError(t, cleaner.cleanUpActiveUsers(ctx, activeUsers, false)) + require.NoError(t, cleaner.cleanDeletedUsers(ctx, deleteUsers)) assertBlockExists("user-1", block1, false) assertBlockExists("user-1", block2, true) assertBlockExists("user-2", block3, true) @@ -711,6 +745,7 @@ func TestBlocksCleaner_ShouldRemoveBlocksOutsideRetentionPeriod(t *testing.T) { # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. # TYPE cortex_compactor_blocks_marked_for_deletion_total counter cortex_compactor_blocks_marked_for_deletion_total{reason="retention",user="user-1"} 1 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention",user="user-2"} 0 `), "cortex_bucket_blocks_count", "cortex_bucket_blocks_marked_for_deletion_count", @@ -722,7 +757,10 @@ func TestBlocksCleaner_ShouldRemoveBlocksOutsideRetentionPeriod(t *testing.T) { { cfgProvider.userRetentionPeriods["user-2"] = 5 * time.Hour - require.NoError(t, cleaner.cleanUsers(ctx, false)) + activeUsers, deleteUsers, err := cleaner.scanUsers(ctx) + require.NoError(t, err) + require.NoError(t, cleaner.cleanUpActiveUsers(ctx, activeUsers, false)) + require.NoError(t, cleaner.cleanDeletedUsers(ctx, deleteUsers)) assertBlockExists("user-1", block1, false) assertBlockExists("user-1", block2, true) assertBlockExists("user-2", block3, false) diff --git a/pkg/compactor/compactor_test.go b/pkg/compactor/compactor_test.go index 6015adff86..8a77b6b551 100644 --- a/pkg/compactor/compactor_test.go +++ b/pkg/compactor/compactor_test.go @@ -214,8 +214,6 @@ func TestCompactor_ShouldDoNothingOnNoUserBlocks(t *testing.T) { assert.Equal(t, prom_testutil.ToFloat64(c.CompactionRunInterval), cfg.CompactionInterval.Seconds()) assert.ElementsMatch(t, []string{ - `level=info component=cleaner msg="started blocks cleanup and maintenance"`, - `level=info component=cleaner msg="successfully completed blocks cleanup and maintenance"`, `level=info component=compactor msg="compactor started"`, `level=info component=compactor msg="discovering users from bucket"`, `level=info component=compactor msg="discovered users from bucket" users=0`, @@ -252,15 +250,13 @@ func TestCompactor_ShouldDoNothingOnNoUserBlocks(t *testing.T) { # TYPE cortex_compactor_block_cleanup_started_total counter # HELP cortex_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. - cortex_compactor_block_cleanup_started_total 1 + cortex_compactor_block_cleanup_started_total{user_status="active"} 1 + cortex_compactor_block_cleanup_started_total{user_status="deleted"} 1 # TYPE cortex_compactor_block_cleanup_completed_total counter # HELP cortex_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. - cortex_compactor_block_cleanup_completed_total 1 - - # TYPE cortex_compactor_block_cleanup_failed_total counter - # HELP cortex_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. - cortex_compactor_block_cleanup_failed_total 0 + cortex_compactor_block_cleanup_completed_total{user_status="active"} 1 + cortex_compactor_block_cleanup_completed_total{user_status="deleted"} 1 `), "cortex_compactor_runs_started_total", "cortex_compactor_runs_completed_total", @@ -310,8 +306,7 @@ func TestCompactor_ShouldRetryCompactionOnFailureWhileDiscoveringUsersFromBucket bucketClient.AssertNumberOfCalls(t, "Iter", 1+3) assert.ElementsMatch(t, []string{ - `level=info component=cleaner msg="started blocks cleanup and maintenance"`, - `level=error component=cleaner msg="failed to run blocks cleanup and maintenance" err="failed to discover users from bucket: failed to iterate the bucket"`, + `level=error component=cleaner msg="failed to scan users on startup" err="failed to discover users from bucket: failed to iterate the bucket"`, `level=info component=compactor msg="compactor started"`, `level=info component=compactor msg="discovering users from bucket"`, `level=error component=compactor msg="failed to discover users from bucket" err="failed to iterate the bucket"`, @@ -338,10 +333,6 @@ func TestCompactor_ShouldRetryCompactionOnFailureWhileDiscoveringUsersFromBucket # TYPE cortex_compactor_blocks_cleaned_total counter cortex_compactor_blocks_cleaned_total 0 - # TYPE cortex_compactor_block_cleanup_started_total counter - # HELP cortex_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. - cortex_compactor_block_cleanup_started_total 1 - # HELP cortex_compactor_blocks_marked_for_no_compaction_total Total number of blocks marked for no compact during a compaction run. # TYPE cortex_compactor_blocks_marked_for_no_compaction_total counter cortex_compactor_blocks_marked_for_no_compaction_total 0 @@ -350,13 +341,10 @@ func TestCompactor_ShouldRetryCompactionOnFailureWhileDiscoveringUsersFromBucket # TYPE cortex_compactor_meta_sync_consistency_delay_seconds gauge cortex_compactor_meta_sync_consistency_delay_seconds 0 - # TYPE cortex_compactor_block_cleanup_completed_total counter - # HELP cortex_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. - cortex_compactor_block_cleanup_completed_total 0 - # TYPE cortex_compactor_block_cleanup_failed_total counter # HELP cortex_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. - cortex_compactor_block_cleanup_failed_total 1 + cortex_compactor_block_cleanup_failed_total{user_status="active"} 1 + cortex_compactor_block_cleanup_failed_total{user_status="deleted"} 1 `), "cortex_compactor_runs_started_total", "cortex_compactor_runs_completed_total", @@ -549,12 +537,6 @@ func TestCompactor_ShouldIterateOverUsersAndRunCompaction(t *testing.T) { assert.Len(t, tsdbPlanner.getNoCompactBlocks(), 0) assert.ElementsMatch(t, []string{ - `level=info component=cleaner msg="started blocks cleanup and maintenance"`, - `level=info component=cleaner org_id=user-1 msg="started blocks cleanup and maintenance"`, - `level=info component=cleaner org_id=user-1 msg="completed blocks cleanup and maintenance"`, - `level=info component=cleaner org_id=user-2 msg="started blocks cleanup and maintenance"`, - `level=info component=cleaner org_id=user-2 msg="completed blocks cleanup and maintenance"`, - `level=info component=cleaner msg="successfully completed blocks cleanup and maintenance"`, `level=info component=compactor msg="compactor started"`, `level=info component=compactor msg="discovering users from bucket"`, `level=info component=compactor msg="discovered users from bucket" users=2`, @@ -605,18 +587,18 @@ func TestCompactor_ShouldIterateOverUsersAndRunCompaction(t *testing.T) { # TYPE cortex_compactor_blocks_marked_for_deletion_total counter cortex_compactor_blocks_marked_for_deletion_total{reason="compaction",user="user-1"} 0 cortex_compactor_blocks_marked_for_deletion_total{reason="compaction",user="user-2"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention",user="user-1"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention",user="user-2"} 0 # TYPE cortex_compactor_block_cleanup_started_total counter # HELP cortex_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. - cortex_compactor_block_cleanup_started_total 1 + cortex_compactor_block_cleanup_started_total{user_status="active"} 1 + cortex_compactor_block_cleanup_started_total{user_status="deleted"} 1 # TYPE cortex_compactor_block_cleanup_completed_total counter # HELP cortex_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. - cortex_compactor_block_cleanup_completed_total 1 - - # TYPE cortex_compactor_block_cleanup_failed_total counter - # HELP cortex_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. - cortex_compactor_block_cleanup_failed_total 0 + cortex_compactor_block_cleanup_completed_total{user_status="active"} 1 + cortex_compactor_block_cleanup_completed_total{user_status="deleted"} 1 # HELP cortex_compactor_blocks_marked_for_no_compaction_total Total number of blocks marked for no compact during a compaction run. # TYPE cortex_compactor_blocks_marked_for_no_compaction_total counter @@ -683,13 +665,6 @@ func TestCompactor_ShouldNotCompactBlocksMarkedForDeletion(t *testing.T) { tsdbPlanner.AssertNumberOfCalls(t, "Plan", 0) assert.ElementsMatch(t, []string{ - `level=info component=cleaner msg="started blocks cleanup and maintenance"`, - `level=info component=cleaner org_id=user-1 msg="started blocks cleanup and maintenance"`, - `level=debug component=cleaner org_id=user-1 msg="deleted file" file=01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json bucket=mock`, - `level=debug component=cleaner org_id=user-1 msg="deleted file" file=01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json bucket=mock`, - `level=info component=cleaner org_id=user-1 msg="deleted block marked for deletion" block=01DTW0ZCPDDNV4BV83Q2SV4QAZ`, - `level=info component=cleaner org_id=user-1 msg="completed blocks cleanup and maintenance"`, - `level=info component=cleaner msg="successfully completed blocks cleanup and maintenance"`, `level=info component=compactor msg="compactor started"`, `level=info component=compactor msg="discovering users from bucket"`, `level=info component=compactor msg="discovered users from bucket" users=1`, @@ -733,18 +708,17 @@ func TestCompactor_ShouldNotCompactBlocksMarkedForDeletion(t *testing.T) { # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. # TYPE cortex_compactor_blocks_marked_for_deletion_total counter cortex_compactor_blocks_marked_for_deletion_total{reason="compaction",user="user-1"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention",user="user-1"} 0 # TYPE cortex_compactor_block_cleanup_started_total counter # HELP cortex_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. - cortex_compactor_block_cleanup_started_total 1 + cortex_compactor_block_cleanup_started_total{user_status="active"} 1 + cortex_compactor_block_cleanup_started_total{user_status="deleted"} 1 # TYPE cortex_compactor_block_cleanup_completed_total counter # HELP cortex_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. - cortex_compactor_block_cleanup_completed_total 1 - - # TYPE cortex_compactor_block_cleanup_failed_total counter - # HELP cortex_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. - cortex_compactor_block_cleanup_failed_total 0 + cortex_compactor_block_cleanup_completed_total{user_status="active"} 1 + cortex_compactor_block_cleanup_completed_total{user_status="deleted"} 1 # HELP cortex_compactor_blocks_marked_for_no_compaction_total Total number of blocks marked for no compact during a compaction run. # TYPE cortex_compactor_blocks_marked_for_no_compaction_total counter @@ -876,14 +850,6 @@ func TestCompactor_ShouldNotCompactBlocksForUsersMarkedForDeletion(t *testing.T) tsdbPlanner.AssertNumberOfCalls(t, "Plan", 0) assert.ElementsMatch(t, []string{ - `level=info component=cleaner msg="started blocks cleanup and maintenance"`, - `level=info component=cleaner org_id=user-1 msg="deleting blocks for tenant marked for deletion"`, - `level=debug component=cleaner org_id=user-1 msg="deleted file" file=01DTVP434PA9VFXSW2JKB3392D/meta.json bucket=mock`, - `level=debug component=cleaner org_id=user-1 msg="deleted file" file=01DTVP434PA9VFXSW2JKB3392D/index bucket=mock`, - `level=info component=cleaner org_id=user-1 msg="deleted block" block=01DTVP434PA9VFXSW2JKB3392D`, - `level=info component=cleaner org_id=user-1 msg="deleted blocks for tenant marked for deletion" deletedBlocks=1`, - `level=info component=cleaner org_id=user-1 msg="updating finished time in tenant deletion mark"`, - `level=info component=cleaner msg="successfully completed blocks cleanup and maintenance"`, `level=info component=compactor msg="compactor started"`, `level=info component=compactor msg="discovering users from bucket"`, `level=info component=compactor msg="discovered users from bucket" users=1`, @@ -922,15 +888,13 @@ func TestCompactor_ShouldNotCompactBlocksForUsersMarkedForDeletion(t *testing.T) # TYPE cortex_compactor_block_cleanup_started_total counter # HELP cortex_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. - cortex_compactor_block_cleanup_started_total 1 + cortex_compactor_block_cleanup_started_total{user_status="active"} 1 + cortex_compactor_block_cleanup_started_total{user_status="deleted"} 1 # TYPE cortex_compactor_block_cleanup_completed_total counter # HELP cortex_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. - cortex_compactor_block_cleanup_completed_total 1 - - # TYPE cortex_compactor_block_cleanup_failed_total counter - # HELP cortex_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. - cortex_compactor_block_cleanup_failed_total 0 + cortex_compactor_block_cleanup_completed_total{user_status="active"} 1 + cortex_compactor_block_cleanup_completed_total{user_status="deleted"} 1 # HELP cortex_compactor_blocks_marked_for_no_compaction_total Total number of blocks marked for no compact during a compaction run. # TYPE cortex_compactor_blocks_marked_for_no_compaction_total counter @@ -1078,12 +1042,6 @@ func TestCompactor_ShouldCompactAllUsersOnShardingEnabledButOnlyOneInstanceRunni assert.ElementsMatch(t, []string{ `level=info component=compactor msg="waiting until compactor is ACTIVE in the ring"`, `level=info component=compactor msg="compactor is ACTIVE in the ring"`, - `level=info component=cleaner msg="started blocks cleanup and maintenance"`, - `level=info component=cleaner org_id=user-1 msg="started blocks cleanup and maintenance"`, - `level=info component=cleaner org_id=user-1 msg="completed blocks cleanup and maintenance"`, - `level=info component=cleaner org_id=user-2 msg="started blocks cleanup and maintenance"`, - `level=info component=cleaner org_id=user-2 msg="completed blocks cleanup and maintenance"`, - `level=info component=cleaner msg="successfully completed blocks cleanup and maintenance"`, `level=info component=compactor msg="compactor started"`, `level=info component=compactor msg="discovering users from bucket"`, `level=info component=compactor msg="discovered users from bucket" users=2`, @@ -1507,10 +1465,15 @@ func removeIgnoredLogs(input []string) []string { `level=info component=compactor msg="compactor stopped"`: {}, } + ignoredLogStringsRegexList := []*regexp.Regexp{ + regexp.MustCompile(`^level=(info|debug|warn) component=cleaner .+$`), + } + out := make([]string, 0, len(input)) durationRe := regexp.MustCompile(`\s?duration(_ms)?=\S+`) executionIDRe := regexp.MustCompile(`\s?execution_id=\S+`) +main: for i := 0; i < len(input); i++ { log := input[i] @@ -1528,6 +1491,12 @@ func removeIgnoredLogs(input []string) []string { continue } + for _, ignoreRegex := range ignoredLogStringsRegexList { + if ignoreRegex.MatchString(log) { + continue main + } + } + out = append(out, log) } From fe788be26d0ef10d96b1c4f700c2293e122cc873 Mon Sep 17 00:00:00 2001 From: Ben Ye Date: Wed, 24 Jul 2024 10:23:06 -0700 Subject: [PATCH 12/32] cleanup native histogram validation const (#6114) Signed-off-by: Ben Ye --- CHANGELOG.md | 1 + pkg/cortexpb/histograms.go | 5 ----- pkg/util/validation/validate.go | 31 ++++++++++++++-------------- pkg/util/validation/validate_test.go | 5 +++-- 4 files changed, 20 insertions(+), 22 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index f04808f267..ddcaca30eb 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -36,6 +36,7 @@ * [BUGFIX] Querier: Enforce max query length check for `/api/v1/series` API even though `ignoreMaxQueryLength` is set to true. #6018 * [BUGFIX] Ingester: Fix issue with the minimize token generator where it was not taking in consideration the current ownerhip of an instance when generating extra tokens. #6062 * [BUGFIX] Scheduler: Fix user queue in scheduler that was not thread-safe. #6077 +* [BUGFIX] Ingester: Include out-of-order head compaction when compacting TSDB head. #6108 ## 1.17.1 2024-05-20 diff --git a/pkg/cortexpb/histograms.go b/pkg/cortexpb/histograms.go index 129b25d300..2e2afef457 100644 --- a/pkg/cortexpb/histograms.go +++ b/pkg/cortexpb/histograms.go @@ -15,11 +15,6 @@ package cortexpb import "github.com/prometheus/prometheus/model/histogram" -const ( - ExponentialSchemaMax int32 = 8 - ExponentialSchemaMin int32 = -4 -) - func (h Histogram) IsFloatHistogram() bool { _, ok := h.GetCount().(*Histogram_CountFloat) return ok diff --git a/pkg/util/validation/validate.go b/pkg/util/validation/validate.go index 46554e2d5b..c13e2cb72f 100644 --- a/pkg/util/validation/validate.go +++ b/pkg/util/validation/validate.go @@ -11,6 +11,7 @@ import ( "github.com/go-kit/log/level" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/histogram" "github.com/weaveworks/common/httpgrpc" "github.com/cortexproject/cortex/pkg/cortexpb" @@ -265,57 +266,57 @@ func ValidateMetadata(validateMetrics *ValidateMetrics, cfg *Limits, userID stri return nil } -func ValidateNativeHistogram(validateMetrics *ValidateMetrics, limits *Limits, userID string, ls []cortexpb.LabelAdapter, histogram cortexpb.Histogram) (cortexpb.Histogram, error) { +func ValidateNativeHistogram(validateMetrics *ValidateMetrics, limits *Limits, userID string, ls []cortexpb.LabelAdapter, histogramSample cortexpb.Histogram) (cortexpb.Histogram, error) { if limits.MaxNativeHistogramBuckets == 0 { - return histogram, nil + return histogramSample, nil } var ( exceedLimit bool ) - if histogram.IsFloatHistogram() { + if histogramSample.IsFloatHistogram() { // Initial check to see if the bucket limit is exceeded or not. If not, we can avoid type casting. - exceedLimit = len(histogram.PositiveCounts)+len(histogram.NegativeCounts) > limits.MaxNativeHistogramBuckets + exceedLimit = len(histogramSample.PositiveCounts)+len(histogramSample.NegativeCounts) > limits.MaxNativeHistogramBuckets if !exceedLimit { - return histogram, nil + return histogramSample, nil } // Exceed limit. - if histogram.Schema <= cortexpb.ExponentialSchemaMin { + if histogramSample.Schema <= histogram.ExponentialSchemaMin { validateMetrics.DiscardedSamples.WithLabelValues(nativeHistogramBucketCountLimitExceeded, userID).Inc() return cortexpb.Histogram{}, newHistogramBucketLimitExceededError(ls, limits.MaxNativeHistogramBuckets) } - fh := cortexpb.FloatHistogramProtoToFloatHistogram(histogram) + fh := cortexpb.FloatHistogramProtoToFloatHistogram(histogramSample) for len(fh.PositiveBuckets)+len(fh.NegativeBuckets) > limits.MaxNativeHistogramBuckets { - if fh.Schema <= cortexpb.ExponentialSchemaMin { + if fh.Schema <= histogram.ExponentialSchemaMin { validateMetrics.DiscardedSamples.WithLabelValues(nativeHistogramBucketCountLimitExceeded, userID).Inc() return cortexpb.Histogram{}, newHistogramBucketLimitExceededError(ls, limits.MaxNativeHistogramBuckets) } fh = fh.ReduceResolution(fh.Schema - 1) } // If resolution reduced, convert new float histogram to protobuf type again. - return cortexpb.FloatHistogramToHistogramProto(histogram.TimestampMs, fh), nil + return cortexpb.FloatHistogramToHistogramProto(histogramSample.TimestampMs, fh), nil } // Initial check to see if bucket limit is exceeded or not. If not, we can avoid type casting. - exceedLimit = len(histogram.PositiveDeltas)+len(histogram.NegativeDeltas) > limits.MaxNativeHistogramBuckets + exceedLimit = len(histogramSample.PositiveDeltas)+len(histogramSample.NegativeDeltas) > limits.MaxNativeHistogramBuckets if !exceedLimit { - return histogram, nil + return histogramSample, nil } // Exceed limit. - if histogram.Schema <= cortexpb.ExponentialSchemaMin { + if histogramSample.Schema <= histogram.ExponentialSchemaMin { validateMetrics.DiscardedSamples.WithLabelValues(nativeHistogramBucketCountLimitExceeded, userID).Inc() return cortexpb.Histogram{}, newHistogramBucketLimitExceededError(ls, limits.MaxNativeHistogramBuckets) } - h := cortexpb.HistogramProtoToHistogram(histogram) + h := cortexpb.HistogramProtoToHistogram(histogramSample) for len(h.PositiveBuckets)+len(h.NegativeBuckets) > limits.MaxNativeHistogramBuckets { - if h.Schema <= cortexpb.ExponentialSchemaMin { + if h.Schema <= histogram.ExponentialSchemaMin { validateMetrics.DiscardedSamples.WithLabelValues(nativeHistogramBucketCountLimitExceeded, userID).Inc() return cortexpb.Histogram{}, newHistogramBucketLimitExceededError(ls, limits.MaxNativeHistogramBuckets) } h = h.ReduceResolution(h.Schema - 1) } // If resolution reduced, convert new histogram to protobuf type again. - return cortexpb.HistogramToHistogramProto(histogram.TimestampMs, h), nil + return cortexpb.HistogramToHistogramProto(histogramSample.TimestampMs, h), nil } func DeletePerUserValidationMetrics(validateMetrics *ValidateMetrics, userID string, log log.Logger) { diff --git a/pkg/util/validation/validate_test.go b/pkg/util/validation/validate_test.go index 93051a63b8..7957c89a11 100644 --- a/pkg/util/validation/validate_test.go +++ b/pkg/util/validation/validate_test.go @@ -8,6 +8,7 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/testutil" "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/histogram" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/tsdb/tsdbutil" "github.com/stretchr/testify/assert" @@ -303,9 +304,9 @@ func TestValidateNativeHistogram(t *testing.T) { fh := tsdbutil.GenerateTestFloatHistogram(0) histogramWithSchemaMin := tsdbutil.GenerateTestHistogram(0) - histogramWithSchemaMin.Schema = cortexpb.ExponentialSchemaMin + histogramWithSchemaMin.Schema = histogram.ExponentialSchemaMin floatHistogramWithSchemaMin := tsdbutil.GenerateTestFloatHistogram(0) - floatHistogramWithSchemaMin.Schema = cortexpb.ExponentialSchemaMin + floatHistogramWithSchemaMin.Schema = histogram.ExponentialSchemaMin for _, tc := range []struct { name string bucketLimit int From 42d73279408565c3c0fd7601f1e3b3747caba74c Mon Sep 17 00:00:00 2001 From: Alex Le Date: Wed, 24 Jul 2024 10:26:49 -0700 Subject: [PATCH 13/32] Differentiate retry and halt error and retry failed compaction only on retriable error (#6111) * Differentiate retry and halt error. Retry failed compaction only on retriable error Signed-off-by: Alex Le * update CHANGELOG Signed-off-by: Alex Le * fix lint and address comment Signed-off-by: Alex Le * merged halt and retriable error into one and differentiate with type label Signed-off-by: Alex Le * reverted log change Signed-off-by: Alex Le * rename Signed-off-by: Alex Le --------- Signed-off-by: Alex Le --- CHANGELOG.md | 1 + pkg/compactor/blocks_cleaner_test.go | 12 +-- pkg/compactor/compactor.go | 10 +++ pkg/compactor/compactor_metrics.go | 22 +++-- pkg/compactor/compactor_metrics_test.go | 20 +++++ pkg/compactor/compactor_test.go | 110 +++++++++++++++++++++++- 6 files changed, 161 insertions(+), 14 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index ddcaca30eb..a01440613c 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -29,6 +29,7 @@ * [ENHANCEMENT] Distributor: Reduce memory usage when error volume is high. #6095 * [ENHANCEMENT] Compactor: Centralize metrics used by compactor and add user label to compactor metrics. #6096 * [ENHANCEMENT] Compactor: Add unique execution ID for each compaction cycle in log for easy debugging. #6097 +* [ENHANCEMENT] Compactor: Differentiate retry and halt error and retry failed compaction only on retriable error. #6111 * [ENHANCEMENT] Ruler: Add support for filtering by `state` and `health` field on Rules API. #6040 * [ENHANCEMENT] Compactor: Split cleaner cycle for active and deleted tenants. #6112 * [BUGFIX] Configsdb: Fix endline issue in db password. #5920 diff --git a/pkg/compactor/blocks_cleaner_test.go b/pkg/compactor/blocks_cleaner_test.go index b582fceb7c..584b3984dc 100644 --- a/pkg/compactor/blocks_cleaner_test.go +++ b/pkg/compactor/blocks_cleaner_test.go @@ -85,7 +85,7 @@ func TestBlockCleaner_KeyPermissionDenied(t *testing.T) { blocksMarkedForDeletion := prometheus.NewCounterVec(prometheus.CounterOpts{ Name: blocksMarkedForDeletionName, Help: blocksMarkedForDeletionHelp, - }, append(commonLabels, ReasonLabelName)) + }, append(commonLabels, reasonLabelName)) cleaner := NewBlocksCleaner(cfg, mbucket, scanner, cfgProvider, logger, nil, blocksMarkedForDeletion) @@ -192,7 +192,7 @@ func testBlocksCleanerWithOptions(t *testing.T, options testBlocksCleanerOptions blocksMarkedForDeletion := promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ Name: blocksMarkedForDeletionName, Help: blocksMarkedForDeletionHelp, - }, append(commonLabels, ReasonLabelName)) + }, append(commonLabels, reasonLabelName)) cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, reg, blocksMarkedForDeletion) require.NoError(t, services.StartAndAwaitRunning(ctx, cleaner)) @@ -353,7 +353,7 @@ func TestBlocksCleaner_ShouldContinueOnBlockDeletionFailure(t *testing.T) { blocksMarkedForDeletion := prometheus.NewCounterVec(prometheus.CounterOpts{ Name: blocksMarkedForDeletionName, Help: blocksMarkedForDeletionHelp, - }, append(commonLabels, ReasonLabelName)) + }, append(commonLabels, reasonLabelName)) cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, nil, blocksMarkedForDeletion) require.NoError(t, services.StartAndAwaitRunning(ctx, cleaner)) @@ -417,7 +417,7 @@ func TestBlocksCleaner_ShouldRebuildBucketIndexOnCorruptedOne(t *testing.T) { blocksMarkedForDeletion := prometheus.NewCounterVec(prometheus.CounterOpts{ Name: blocksMarkedForDeletionName, Help: blocksMarkedForDeletionHelp, - }, append(commonLabels, ReasonLabelName)) + }, append(commonLabels, reasonLabelName)) cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, nil, blocksMarkedForDeletion) require.NoError(t, services.StartAndAwaitRunning(ctx, cleaner)) @@ -475,7 +475,7 @@ func TestBlocksCleaner_ShouldRemoveMetricsForTenantsNotBelongingAnymoreToTheShar blocksMarkedForDeletion := promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ Name: blocksMarkedForDeletionName, Help: blocksMarkedForDeletionHelp, - }, append(commonLabels, ReasonLabelName)) + }, append(commonLabels, reasonLabelName)) cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, reg, blocksMarkedForDeletion) activeUsers, deleteUsers, err := cleaner.scanUsers(ctx) @@ -616,7 +616,7 @@ func TestBlocksCleaner_ShouldRemoveBlocksOutsideRetentionPeriod(t *testing.T) { blocksMarkedForDeletion := promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ Name: blocksMarkedForDeletionName, Help: blocksMarkedForDeletionHelp, - }, append(commonLabels, ReasonLabelName)) + }, append(commonLabels, reasonLabelName)) cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, reg, blocksMarkedForDeletion) diff --git a/pkg/compactor/compactor.go b/pkg/compactor/compactor.go index 7eac246c4d..0d03348d6b 100644 --- a/pkg/compactor/compactor.go +++ b/pkg/compactor/compactor.go @@ -792,10 +792,20 @@ func (c *Compactor) compactUserWithRetries(ctx context.Context, userID string) e if lastErr == nil { return nil } + if ctx.Err() != nil { + return ctx.Err() + } if c.isCausedByPermissionDenied(lastErr) { level.Warn(c.logger).Log("msg", "skipping compactUser due to PermissionDenied", "user", userID, "err", lastErr) + c.compactorMetrics.compactionErrorsCount.WithLabelValues(userID, unauthorizedError).Inc() return nil } + if compact.IsHaltError(lastErr) { + level.Error(c.logger).Log("msg", "compactor returned critical error", "user", userID, "err", lastErr) + c.compactorMetrics.compactionErrorsCount.WithLabelValues(userID, haltError).Inc() + return lastErr + } + c.compactorMetrics.compactionErrorsCount.WithLabelValues(userID, retriableError).Inc() retries.Wait() } diff --git a/pkg/compactor/compactor_metrics.go b/pkg/compactor/compactor_metrics.go index 3567225919..bdd3fefef0 100644 --- a/pkg/compactor/compactor_metrics.go +++ b/pkg/compactor/compactor_metrics.go @@ -37,17 +37,23 @@ type compactorMetrics struct { compactionFailures *prometheus.CounterVec verticalCompactions *prometheus.CounterVec remainingPlannedCompactions *prometheus.GaugeVec + compactionErrorsCount *prometheus.CounterVec } const ( - UserLabelName = "user" - TimeRangeLabelName = "time_range_milliseconds" - ReasonLabelName = "reason" + userLabelName = "user" + timeRangeLabelName = "time_range_milliseconds" + reasonLabelName = "reason" + compactionErrorTypesLabelName = "type" + + retriableError = "retriable" + haltError = "halt" + unauthorizedError = "unauthorized" ) var ( - commonLabels = []string{UserLabelName} - compactionLabels = []string{TimeRangeLabelName} + commonLabels = []string{userLabelName} + compactionLabels = []string{timeRangeLabelName} ) func newDefaultCompactorMetrics(reg prometheus.Registerer) *compactorMetrics { @@ -129,7 +135,7 @@ func newCompactorMetricsWithLabels(reg prometheus.Registerer, commonLabels []str m.syncerBlocksMarkedForDeletion = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ Name: blocksMarkedForDeletionName, Help: blocksMarkedForDeletionHelp, - }, append(commonLabels, ReasonLabelName)) + }, append(commonLabels, reasonLabelName)) m.compactions = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ Name: "cortex_compactor_group_compactions_total", @@ -159,6 +165,10 @@ func newCompactorMetricsWithLabels(reg prometheus.Registerer, commonLabels []str Name: "cortex_compactor_remaining_planned_compactions", Help: "Total number of plans that remain to be compacted. Only available with shuffle-sharding strategy", }, commonLabels) + m.compactionErrorsCount = promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ + Name: "cortex_compactor_compaction_error_total", + Help: "Total number of errors from compactions.", + }, append(commonLabels, compactionErrorTypesLabelName)) return &m } diff --git a/pkg/compactor/compactor_metrics_test.go b/pkg/compactor/compactor_metrics_test.go index 8667d892cf..da4bb82025 100644 --- a/pkg/compactor/compactor_metrics_test.go +++ b/pkg/compactor/compactor_metrics_test.go @@ -119,6 +119,17 @@ func TestSyncerMetrics(t *testing.T) { cortex_compactor_remaining_planned_compactions{user="aaa"} 377740 cortex_compactor_remaining_planned_compactions{user="bbb"} 388850 cortex_compactor_remaining_planned_compactions{user="ccc"} 399960 + # HELP cortex_compactor_compaction_error_total Total number of errors from compactions. + # TYPE cortex_compactor_compaction_error_total counter + cortex_compactor_compaction_error_total{type="halt",user="aaa"} 444400 + cortex_compactor_compaction_error_total{type="halt",user="bbb"} 455510 + cortex_compactor_compaction_error_total{type="halt",user="ccc"} 466620 + cortex_compactor_compaction_error_total{type="retriable",user="aaa"} 411070 + cortex_compactor_compaction_error_total{type="retriable",user="bbb"} 422180 + cortex_compactor_compaction_error_total{type="retriable",user="ccc"} 433290 + cortex_compactor_compaction_error_total{type="unauthorized",user="aaa"} 477730 + cortex_compactor_compaction_error_total{type="unauthorized",user="bbb"} 488840 + cortex_compactor_compaction_error_total{type="unauthorized",user="ccc"} 499950 `)) require.NoError(t, err) @@ -163,4 +174,13 @@ func generateTestData(cm *compactorMetrics, base float64) { cm.remainingPlannedCompactions.WithLabelValues("aaa").Add(34 * base) cm.remainingPlannedCompactions.WithLabelValues("bbb").Add(35 * base) cm.remainingPlannedCompactions.WithLabelValues("ccc").Add(36 * base) + cm.compactionErrorsCount.WithLabelValues("aaa", retriableError).Add(37 * base) + cm.compactionErrorsCount.WithLabelValues("bbb", retriableError).Add(38 * base) + cm.compactionErrorsCount.WithLabelValues("ccc", retriableError).Add(39 * base) + cm.compactionErrorsCount.WithLabelValues("aaa", haltError).Add(40 * base) + cm.compactionErrorsCount.WithLabelValues("bbb", haltError).Add(41 * base) + cm.compactionErrorsCount.WithLabelValues("ccc", haltError).Add(42 * base) + cm.compactionErrorsCount.WithLabelValues("aaa", unauthorizedError).Add(43 * base) + cm.compactionErrorsCount.WithLabelValues("bbb", unauthorizedError).Add(44 * base) + cm.compactionErrorsCount.WithLabelValues("ccc", unauthorizedError).Add(45 * base) } diff --git a/pkg/compactor/compactor_test.go b/pkg/compactor/compactor_test.go index 8a77b6b551..6f283bdcf7 100644 --- a/pkg/compactor/compactor_test.go +++ b/pkg/compactor/compactor_test.go @@ -1615,8 +1615,8 @@ func (m *tsdbPlannerMock) getNoCompactBlocks() []string { return result } -func mockBlockMetaJSON(id string) string { - meta := tsdb.BlockMeta{ +func mockBlockMeta(id string) tsdb.BlockMeta { + return tsdb.BlockMeta{ Version: 1, ULID: ulid.MustParse(id), MinTime: 1574776800000, @@ -1626,6 +1626,10 @@ func mockBlockMetaJSON(id string) string { Sources: []ulid.ULID{ulid.MustParse(id)}, }, } +} + +func mockBlockMetaJSON(id string) string { + meta := mockBlockMeta(id) content, err := json.Marshal(meta) if err != nil { @@ -1988,3 +1992,105 @@ func TestCompactor_ShouldNotFailCompactionIfAccessDeniedErrReturnedFromBucket(t require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) } + +func TestCompactor_FailedWithRetriableError(t *testing.T) { + t.Parallel() + + ss := bucketindex.Status{Status: bucketindex.Ok, Version: bucketindex.SyncStatusFileVersion} + content, err := json.Marshal(ss) + require.NoError(t, err) + + bucketClient := &bucket.ClientMock{} + bucketClient.MockIter("__markers__", []string{}, nil) + bucketClient.MockIter("", []string{"user-1"}, nil) + bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D", "user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ", "user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", "user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json"}, nil) + bucketClient.MockIter("user-1/markers/", nil, nil) + bucketClient.MockExists(cortex_tsdb.GetGlobalDeletionMarkPath("user-1"), false, nil) + bucketClient.MockExists(cortex_tsdb.GetLocalDeletionMarkPath("user-1"), false, nil) + bucketClient.MockIter("user-1/01DTVP434PA9VFXSW2JKB3392D", nil, errors.New("test retriable error")) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", "", nil) + bucketClient.MockIter("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ", nil, errors.New("test retriable error")) + bucketClient.MockGet("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", mockBlockMetaJSON("01DTW0ZCPDDNV4BV83Q2SV4QAZ"), nil) + bucketClient.MockGet("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-1/bucket-index.json.gz", "", nil) + bucketClient.MockGet("user-1/bucket-index-sync-status.json", string(content), nil) + bucketClient.MockUpload("user-1/bucket-index.json.gz", nil) + bucketClient.MockUpload("user-1/bucket-index-sync-status.json", nil) + + cfg := prepareConfig() + cfg.CompactionRetries = 2 + + c, _, tsdbPlanner, _, registry := prepare(t, cfg, bucketClient, nil) + tsdbPlanner.On("Plan", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return([]*metadata.Meta{{BlockMeta: mockBlockMeta("01DTVP434PA9VFXSW2JKB3392D")}, {BlockMeta: mockBlockMeta("01DTW0ZCPDDNV4BV83Q2SV4QAZ")}}, nil) + + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + + cortex_testutil.Poll(t, 1*time.Second, 2.0, func() interface{} { + return prom_testutil.ToFloat64(c.compactorMetrics.compactionErrorsCount.WithLabelValues("user-1", retriableError)) + }) + + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) + + assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` + # HELP cortex_compactor_compaction_error_total Total number of errors from compactions. + # TYPE cortex_compactor_compaction_error_total counter + cortex_compactor_compaction_error_total{type="retriable", user="user-1"} 2 + `), + "cortex_compactor_compaction_retry_error_total", + "cortex_compactor_compaction_error_total", + )) +} + +func TestCompactor_FailedWithHaltError(t *testing.T) { + t.Parallel() + + ss := bucketindex.Status{Status: bucketindex.Ok, Version: bucketindex.SyncStatusFileVersion} + content, err := json.Marshal(ss) + require.NoError(t, err) + + bucketClient := &bucket.ClientMock{} + bucketClient.MockIter("__markers__", []string{}, nil) + bucketClient.MockIter("", []string{"user-1"}, nil) + bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D", "user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ", "user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", "user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json"}, nil) + bucketClient.MockIter("user-1/markers/", nil, nil) + bucketClient.MockExists(cortex_tsdb.GetGlobalDeletionMarkPath("user-1"), false, nil) + bucketClient.MockExists(cortex_tsdb.GetLocalDeletionMarkPath("user-1"), false, nil) + bucketClient.MockIter("user-1/01DTVP434PA9VFXSW2JKB3392D", nil, compact.HaltError{}) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", "", nil) + bucketClient.MockIter("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ", nil, compact.HaltError{}) + bucketClient.MockGet("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", mockBlockMetaJSON("01DTW0ZCPDDNV4BV83Q2SV4QAZ"), nil) + bucketClient.MockGet("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-1/bucket-index.json.gz", "", nil) + bucketClient.MockGet("user-1/bucket-index-sync-status.json", string(content), nil) + bucketClient.MockUpload("user-1/bucket-index.json.gz", nil) + bucketClient.MockUpload("user-1/bucket-index-sync-status.json", nil) + + cfg := prepareConfig() + cfg.CompactionRetries = 2 + + c, _, tsdbPlanner, _, registry := prepare(t, cfg, bucketClient, nil) + tsdbPlanner.On("Plan", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return([]*metadata.Meta{{BlockMeta: mockBlockMeta("01DTVP434PA9VFXSW2JKB3392D")}, {BlockMeta: mockBlockMeta("01DTW0ZCPDDNV4BV83Q2SV4QAZ")}}, nil) + + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + + cortex_testutil.Poll(t, 1*time.Second, 1.0, func() interface{} { + return prom_testutil.ToFloat64(c.compactorMetrics.compactionErrorsCount.WithLabelValues("user-1", haltError)) + }) + + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) + + assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` + # HELP cortex_compactor_compaction_error_total Total number of errors from compactions. + # TYPE cortex_compactor_compaction_error_total counter + cortex_compactor_compaction_error_total{type="halt", user="user-1"} 1 + `), + "cortex_compactor_compaction_retry_error_total", + "cortex_compactor_compaction_error_total", + )) +} From 5356796efea7e54629fc1be70a9cdf0d75f1352f Mon Sep 17 00:00:00 2001 From: Justin Jung Date: Wed, 24 Jul 2024 14:28:59 -0700 Subject: [PATCH 14/32] [Store Gateway] Token bucket limiter (#6016) * Create TokenBucket Signed-off-by: Justin Jung * Update bucket stores to pass token bucket Signed-off-by: Justin Jung * Move limiters to a new file Signed-off-by: Justin Jung * Added tests for limiters and token bucket Signed-off-by: Justin Jung * Add more tests Signed-off-by: Justin Jung * Added enable flag Signed-off-by: Justin Jung * Add dryrun feature Signed-off-by: Justin Jung * Add doc Signed-off-by: Justin Jung * Add changelog Signed-off-by: Justin Jung * Lint Signed-off-by: Justin Jung * Do not create pod token bucket if the feature is not enabled Signed-off-by: Justin Jung * More docs Signed-off-by: Justin Jung * Address comments Signed-off-by: Justin Jung * Rename podTokenBucket to instanceTokenBucket Signed-off-by: Justin Jung * Updated default values Signed-off-by: Justin Jung * Rename TokenBucketLimiter to TokenBucketBytesLimiter Signed-off-by: Justin Jung * Changed error to httpgrpc Signed-off-by: Justin Jung * Nit Signed-off-by: Justin Jung * Increment failure metric when token bucket returns error Signed-off-by: Justin Jung * Simplify token bucket by making Retrieve to always deduct token Signed-off-by: Justin Jung * Throw 429 and 422 for different failure scenarios Signed-off-by: Justin Jung * Hide token factors from doc Signed-off-by: Justin Jung * Simplified config by combining dryrun and enabled Signed-off-by: Justin Jung * Remove test log Signed-off-by: Justin Jung * Fix tests Signed-off-by: Justin Jung * Fix Signed-off-by: Justin Jung --------- Signed-off-by: Justin Jung --- CHANGELOG.md | 1 + docs/blocks-storage/querier.md | 18 +++ docs/blocks-storage/store-gateway.md | 18 +++ docs/configuration/config-file-reference.md | 18 +++ pkg/storage/tsdb/config.go | 45 ++++++ pkg/storegateway/bucket_stores.go | 101 ++++++------- pkg/storegateway/bucket_stores_test.go | 92 ++++++++++++ pkg/storegateway/limiter.go | 148 ++++++++++++++++++++ pkg/storegateway/limiter_test.go | 148 ++++++++++++++++++++ pkg/util/token_bucket.go | 62 ++++++++ pkg/util/token_bucket_test.go | 23 +++ 11 files changed, 625 insertions(+), 49 deletions(-) create mode 100644 pkg/storegateway/limiter.go create mode 100644 pkg/storegateway/limiter_test.go create mode 100644 pkg/util/token_bucket.go create mode 100644 pkg/util/token_bucket_test.go diff --git a/CHANGELOG.md b/CHANGELOG.md index a01440613c..fb01005e0d 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -10,6 +10,7 @@ * [FEATURE] OTLP: Support ingesting OTLP exponential metrics as native histograms. #6071 * [FEATURE] Ingester: Add `ingester.instance-limits.max-inflight-query-requests` to allow limiting ingester concurrent queries. #6081 * [FEATURE] Distributor: Add `validation.max-native-histogram-buckets` to limit max number of bucket count. Distributor will try to automatically reduce histogram resolution until it is within the bucket limit or resolution cannot be reduced anymore. #6104 +* [FEATURE] Store Gateway: Token bucket limiter. #6016 * [ENHANCEMENT] rulers: Add support to persist tokens in rulers. #5987 * [ENHANCEMENT] Query Frontend/Querier: Added store gateway postings touched count and touched size in Querier stats and log in Query Frontend. #5892 * [ENHANCEMENT] Query Frontend/Querier: Returns `warnings` on prometheus query responses. #5916 diff --git a/docs/blocks-storage/querier.md b/docs/blocks-storage/querier.md index 1ca05c1b05..a9206c82ba 100644 --- a/docs/blocks-storage/querier.md +++ b/docs/blocks-storage/querier.md @@ -1341,6 +1341,24 @@ blocks_storage: # CLI flag: -blocks-storage.bucket-store.series-batch-size [series_batch_size: | default = 10000] + token_bucket_bytes_limiter: + # Token bucket bytes limiter mode. Supported values are: disabled, dryrun, + # enabled + # CLI flag: -blocks-storage.bucket-store.token-bucket-bytes-limiter.mode + [mode: | default = "disabled"] + + # Instance token bucket size + # CLI flag: -blocks-storage.bucket-store.token-bucket-bytes-limiter.instance-token-bucket-size + [instance_token_bucket_size: | default = 859832320] + + # User token bucket size + # CLI flag: -blocks-storage.bucket-store.token-bucket-bytes-limiter.user-token-bucket-size + [user_token_bucket_size: | default = 644874240] + + # Request token bucket size + # CLI flag: -blocks-storage.bucket-store.token-bucket-bytes-limiter.request-token-bucket-size + [request_token_bucket_size: | default = 4194304] + tsdb: # Local directory to store TSDBs in the ingesters. # CLI flag: -blocks-storage.tsdb.dir diff --git a/docs/blocks-storage/store-gateway.md b/docs/blocks-storage/store-gateway.md index 4b7e504fa4..6c31046b1f 100644 --- a/docs/blocks-storage/store-gateway.md +++ b/docs/blocks-storage/store-gateway.md @@ -1466,6 +1466,24 @@ blocks_storage: # CLI flag: -blocks-storage.bucket-store.series-batch-size [series_batch_size: | default = 10000] + token_bucket_bytes_limiter: + # Token bucket bytes limiter mode. Supported values are: disabled, dryrun, + # enabled + # CLI flag: -blocks-storage.bucket-store.token-bucket-bytes-limiter.mode + [mode: | default = "disabled"] + + # Instance token bucket size + # CLI flag: -blocks-storage.bucket-store.token-bucket-bytes-limiter.instance-token-bucket-size + [instance_token_bucket_size: | default = 859832320] + + # User token bucket size + # CLI flag: -blocks-storage.bucket-store.token-bucket-bytes-limiter.user-token-bucket-size + [user_token_bucket_size: | default = 644874240] + + # Request token bucket size + # CLI flag: -blocks-storage.bucket-store.token-bucket-bytes-limiter.request-token-bucket-size + [request_token_bucket_size: | default = 4194304] + tsdb: # Local directory to store TSDBs in the ingesters. # CLI flag: -blocks-storage.tsdb.dir diff --git a/docs/configuration/config-file-reference.md b/docs/configuration/config-file-reference.md index 1e730d99e1..df036ed2f6 100644 --- a/docs/configuration/config-file-reference.md +++ b/docs/configuration/config-file-reference.md @@ -1899,6 +1899,24 @@ bucket_store: # CLI flag: -blocks-storage.bucket-store.series-batch-size [series_batch_size: | default = 10000] + token_bucket_bytes_limiter: + # Token bucket bytes limiter mode. Supported values are: disabled, dryrun, + # enabled + # CLI flag: -blocks-storage.bucket-store.token-bucket-bytes-limiter.mode + [mode: | default = "disabled"] + + # Instance token bucket size + # CLI flag: -blocks-storage.bucket-store.token-bucket-bytes-limiter.instance-token-bucket-size + [instance_token_bucket_size: | default = 859832320] + + # User token bucket size + # CLI flag: -blocks-storage.bucket-store.token-bucket-bytes-limiter.user-token-bucket-size + [user_token_bucket_size: | default = 644874240] + + # Request token bucket size + # CLI flag: -blocks-storage.bucket-store.token-bucket-bytes-limiter.request-token-bucket-size + [request_token_bucket_size: | default = 4194304] + tsdb: # Local directory to store TSDBs in the ingesters. # CLI flag: -blocks-storage.tsdb.dir diff --git a/pkg/storage/tsdb/config.go b/pkg/storage/tsdb/config.go index 5c9f0c23cb..bd3099dba9 100644 --- a/pkg/storage/tsdb/config.go +++ b/pkg/storage/tsdb/config.go @@ -2,6 +2,7 @@ package tsdb import ( "flag" + "fmt" "path/filepath" "strings" "time" @@ -52,6 +53,7 @@ var ( ErrInvalidBucketIndexBlockDiscoveryStrategy = errors.New("bucket index block discovery strategy can only be enabled when bucket index is enabled") ErrBlockDiscoveryStrategy = errors.New("invalid block discovery strategy") + ErrInvalidTokenBucketBytesLimiterMode = errors.New("invalid token bucket bytes limiter mode") ) // BlocksStorageConfig holds the config information for the blocks storage. @@ -292,6 +294,22 @@ type BucketStoreConfig struct { // Controls how many series to fetch per batch in Store Gateway. Default value is 10000. SeriesBatchSize int `yaml:"series_batch_size"` + + // Token bucket configs + TokenBucketBytesLimiter TokenBucketBytesLimiterConfig `yaml:"token_bucket_bytes_limiter"` +} + +type TokenBucketBytesLimiterConfig struct { + Mode string `yaml:"mode"` + InstanceTokenBucketSize int64 `yaml:"instance_token_bucket_size"` + UserTokenBucketSize int64 `yaml:"user_token_bucket_size"` + RequestTokenBucketSize int64 `yaml:"request_token_bucket_size"` + FetchedPostingsTokenFactor float64 `yaml:"fetched_postings_token_factor" doc:"hidden"` + TouchedPostingsTokenFactor float64 `yaml:"touched_postings_token_factor" doc:"hidden"` + FetchedSeriesTokenFactor float64 `yaml:"fetched_series_token_factor" doc:"hidden"` + TouchedSeriesTokenFactor float64 `yaml:"touched_series_token_factor" doc:"hidden"` + FetchedChunksTokenFactor float64 `yaml:"fetched_chunks_token_factor" doc:"hidden"` + TouchedChunksTokenFactor float64 `yaml:"touched_chunks_token_factor" doc:"hidden"` } // RegisterFlags registers the BucketStore flags @@ -325,6 +343,16 @@ func (cfg *BucketStoreConfig) RegisterFlags(f *flag.FlagSet) { f.BoolVar(&cfg.LazyExpandedPostingsEnabled, "blocks-storage.bucket-store.lazy-expanded-postings-enabled", false, "If true, Store Gateway will estimate postings size and try to lazily expand postings if it downloads less data than expanding all postings.") f.IntVar(&cfg.SeriesBatchSize, "blocks-storage.bucket-store.series-batch-size", store.SeriesBatchSize, "Controls how many series to fetch per batch in Store Gateway. Default value is 10000.") f.StringVar(&cfg.BlockDiscoveryStrategy, "blocks-storage.bucket-store.block-discovery-strategy", string(ConcurrentDiscovery), "One of "+strings.Join(supportedBlockDiscoveryStrategies, ", ")+". When set to concurrent, stores will concurrently issue one call per directory to discover active blocks in the bucket. The recursive strategy iterates through all objects in the bucket, recursively traversing into each directory. This avoids N+1 calls at the expense of having slower bucket iterations. bucket_index strategy can be used in Compactor only and utilizes the existing bucket index to fetch block IDs to sync. This avoids iterating the bucket but can be impacted by delays of cleaner creating bucket index.") + f.StringVar(&cfg.TokenBucketBytesLimiter.Mode, "blocks-storage.bucket-store.token-bucket-bytes-limiter.mode", string(TokenBucketBytesLimiterDisabled), fmt.Sprintf("Token bucket bytes limiter mode. Supported values are: %s", strings.Join(supportedTokenBucketBytesLimiterModes, ", "))) + f.Int64Var(&cfg.TokenBucketBytesLimiter.InstanceTokenBucketSize, "blocks-storage.bucket-store.token-bucket-bytes-limiter.instance-token-bucket-size", int64(820*units.Mebibyte), "Instance token bucket size") + f.Int64Var(&cfg.TokenBucketBytesLimiter.UserTokenBucketSize, "blocks-storage.bucket-store.token-bucket-bytes-limiter.user-token-bucket-size", int64(615*units.Mebibyte), "User token bucket size") + f.Int64Var(&cfg.TokenBucketBytesLimiter.RequestTokenBucketSize, "blocks-storage.bucket-store.token-bucket-bytes-limiter.request-token-bucket-size", int64(4*units.Mebibyte), "Request token bucket size") + f.Float64Var(&cfg.TokenBucketBytesLimiter.FetchedPostingsTokenFactor, "blocks-storage.bucket-store.token-bucket-bytes-limiter.fetched-postings-token-factor", 0, "Multiplication factor used for fetched postings token") + f.Float64Var(&cfg.TokenBucketBytesLimiter.TouchedPostingsTokenFactor, "blocks-storage.bucket-store.token-bucket-bytes-limiter.touched-postings-token-factor", 5, "Multiplication factor used for touched postings token") + f.Float64Var(&cfg.TokenBucketBytesLimiter.FetchedSeriesTokenFactor, "blocks-storage.bucket-store.token-bucket-bytes-limiter.fetched-series-token-factor", 0, "Multiplication factor used for fetched series token") + f.Float64Var(&cfg.TokenBucketBytesLimiter.TouchedSeriesTokenFactor, "blocks-storage.bucket-store.token-bucket-bytes-limiter.touched-series-token-factor", 25, "Multiplication factor used for touched series token") + f.Float64Var(&cfg.TokenBucketBytesLimiter.FetchedChunksTokenFactor, "blocks-storage.bucket-store.token-bucket-bytes-limiter.fetched-chunks-token-factor", 0, "Multiplication factor used for fetched chunks token") + f.Float64Var(&cfg.TokenBucketBytesLimiter.TouchedChunksTokenFactor, "blocks-storage.bucket-store.token-bucket-bytes-limiter.touched-chunks-token-factor", 1, "Multiplication factor used for touched chunks token") } // Validate the config. @@ -344,6 +372,9 @@ func (cfg *BucketStoreConfig) Validate() error { if !util.StringsContain(supportedBlockDiscoveryStrategies, cfg.BlockDiscoveryStrategy) { return ErrInvalidBucketIndexBlockDiscoveryStrategy } + if !util.StringsContain(supportedTokenBucketBytesLimiterModes, cfg.TokenBucketBytesLimiter.Mode) { + return ErrInvalidTokenBucketBytesLimiterMode + } return nil } @@ -375,3 +406,17 @@ var supportedBlockDiscoveryStrategies = []string{ string(RecursiveDiscovery), string(BucketIndexDiscovery), } + +type TokenBucketBytesLimiterMode string + +const ( + TokenBucketBytesLimiterDisabled TokenBucketBytesLimiterMode = "disabled" + TokenBucketBytesLimiterDryRun TokenBucketBytesLimiterMode = "dryrun" + TokenBucketBytesLimiterEnabled TokenBucketBytesLimiterMode = "enabled" +) + +var supportedTokenBucketBytesLimiterModes = []string{ + string(TokenBucketBytesLimiterDisabled), + string(TokenBucketBytesLimiterDryRun), + string(TokenBucketBytesLimiterEnabled), +} diff --git a/pkg/storegateway/bucket_stores.go b/pkg/storegateway/bucket_stores.go index 5d2c2d0ec1..e95e7dd6bc 100644 --- a/pkg/storegateway/bucket_stores.go +++ b/pkg/storegateway/bucket_stores.go @@ -4,7 +4,6 @@ import ( "context" "fmt" "math" - "net/http" "os" "path/filepath" "strings" @@ -35,6 +34,7 @@ import ( "github.com/cortexproject/cortex/pkg/storage/bucket" "github.com/cortexproject/cortex/pkg/storage/tsdb" + "github.com/cortexproject/cortex/pkg/util" "github.com/cortexproject/cortex/pkg/util/backoff" cortex_errors "github.com/cortexproject/cortex/pkg/util/errors" util_log "github.com/cortexproject/cortex/pkg/util/log" @@ -73,6 +73,11 @@ type BucketStores struct { storesErrorsMu sync.RWMutex storesErrors map[string]error + instanceTokenBucket *util.TokenBucket + + userTokenBucketsMu sync.RWMutex + userTokenBuckets map[string]*util.TokenBucket + // Keeps number of inflight requests inflightRequestCnt int inflightRequestMu sync.RWMutex @@ -115,6 +120,7 @@ func NewBucketStores(cfg tsdb.BlocksStorageConfig, shardingStrategy ShardingStra metaFetcherMetrics: NewMetadataFetcherMetrics(), queryGate: queryGate, partitioner: newGapBasedPartitioner(cfg.BucketStore.PartitionerMaxGapBytes, reg), + userTokenBuckets: make(map[string]*util.TokenBucket), syncTimes: promauto.With(reg).NewHistogram(prometheus.HistogramOpts{ Name: "cortex_bucket_stores_blocks_sync_seconds", Help: "The total time it takes to perform a sync stores", @@ -144,6 +150,13 @@ func NewBucketStores(cfg tsdb.BlocksStorageConfig, shardingStrategy ShardingStra return nil, errors.Wrap(err, "create chunks bytes pool") } + if u.cfg.BucketStore.TokenBucketBytesLimiter.Mode != string(tsdb.TokenBucketBytesLimiterDisabled) { + u.instanceTokenBucket = util.NewTokenBucket(cfg.BucketStore.TokenBucketBytesLimiter.InstanceTokenBucketSize, promauto.With(reg).NewGauge(prometheus.GaugeOpts{ + Name: "cortex_bucket_stores_instance_token_bucket_remaining", + Help: "Number of tokens left in instance token bucket.", + })) + } + if reg != nil { reg.MustRegister(u.bucketStoreMetrics, u.metaFetcherMetrics) } @@ -475,6 +488,12 @@ func (u *BucketStores) closeEmptyBucketStore(userID string) error { unlockInDefer = false u.storesMu.Unlock() + if u.cfg.BucketStore.TokenBucketBytesLimiter.Mode != string(tsdb.TokenBucketBytesLimiterDisabled) { + u.userTokenBucketsMu.Lock() + delete(u.userTokenBuckets, userID) + u.userTokenBucketsMu.Unlock() + } + u.metaFetcherMetrics.RemoveUserRegistry(userID) u.bucketStoreMetrics.RemoveUserRegistry(userID) return bs.Close() @@ -612,13 +631,19 @@ func (u *BucketStores) getOrCreateStore(userID string) (*store.BucketStore, erro bucketStoreOpts = append(bucketStoreOpts, store.WithDebugLogging()) } + if u.cfg.BucketStore.TokenBucketBytesLimiter.Mode != string(tsdb.TokenBucketBytesLimiterDisabled) { + u.userTokenBucketsMu.Lock() + u.userTokenBuckets[userID] = util.NewTokenBucket(u.cfg.BucketStore.TokenBucketBytesLimiter.UserTokenBucketSize, nil) + u.userTokenBucketsMu.Unlock() + } + bs, err := store.NewBucketStore( userBkt, fetcher, u.syncDirForUser(userID), newChunksLimiterFactory(u.limits, userID), newSeriesLimiterFactory(u.limits, userID), - newBytesLimiterFactory(u.limits, userID), + newBytesLimiterFactory(u.limits, userID, u.getUserTokenBucket(userID), u.instanceTokenBucket, u.cfg.BucketStore.TokenBucketBytesLimiter, u.getTokensToRetrieve), u.partitioner, u.cfg.BucketStore.BlockSyncConcurrency, false, // No need to enable backward compatibility with Thanos pre 0.8.0 queriers @@ -680,6 +705,31 @@ func (u *BucketStores) deleteLocalFilesForExcludedTenants(includeUserIDs map[str } } +func (u *BucketStores) getUserTokenBucket(userID string) *util.TokenBucket { + u.userTokenBucketsMu.RLock() + defer u.userTokenBucketsMu.RUnlock() + return u.userTokenBuckets[userID] +} + +func (u *BucketStores) getTokensToRetrieve(tokens uint64, dataType store.StoreDataType) int64 { + tokensToRetrieve := float64(tokens) + switch dataType { + case store.PostingsFetched: + tokensToRetrieve *= u.cfg.BucketStore.TokenBucketBytesLimiter.FetchedPostingsTokenFactor + case store.PostingsTouched: + tokensToRetrieve *= u.cfg.BucketStore.TokenBucketBytesLimiter.TouchedPostingsTokenFactor + case store.SeriesFetched: + tokensToRetrieve *= u.cfg.BucketStore.TokenBucketBytesLimiter.FetchedSeriesTokenFactor + case store.SeriesTouched: + tokensToRetrieve *= u.cfg.BucketStore.TokenBucketBytesLimiter.TouchedSeriesTokenFactor + case store.ChunksFetched: + tokensToRetrieve *= u.cfg.BucketStore.TokenBucketBytesLimiter.FetchedChunksTokenFactor + case store.ChunksTouched: + tokensToRetrieve *= u.cfg.BucketStore.TokenBucketBytesLimiter.TouchedChunksTokenFactor + } + return int64(tokensToRetrieve) +} + func getUserIDFromGRPCContext(ctx context.Context) string { meta, ok := metadata.FromIncomingContext(ctx) if !ok { @@ -730,50 +780,3 @@ type spanSeriesServer struct { func (s spanSeriesServer) Context() context.Context { return s.ctx } - -type limiter struct { - limiter *store.Limiter -} - -func (c *limiter) Reserve(num uint64) error { - return c.ReserveWithType(num, 0) -} - -func (c *limiter) ReserveWithType(num uint64, _ store.StoreDataType) error { - err := c.limiter.Reserve(num) - if err != nil { - return httpgrpc.Errorf(http.StatusUnprocessableEntity, err.Error()) - } - - return nil -} - -func newChunksLimiterFactory(limits *validation.Overrides, userID string) store.ChunksLimiterFactory { - return func(failedCounter prometheus.Counter) store.ChunksLimiter { - // Since limit overrides could be live reloaded, we have to get the current user's limit - // each time a new limiter is instantiated. - return &limiter{ - limiter: store.NewLimiter(uint64(limits.MaxChunksPerQueryFromStore(userID)), failedCounter), - } - } -} - -func newSeriesLimiterFactory(limits *validation.Overrides, userID string) store.SeriesLimiterFactory { - return func(failedCounter prometheus.Counter) store.SeriesLimiter { - // Since limit overrides could be live reloaded, we have to get the current user's limit - // each time a new limiter is instantiated. - return &limiter{ - limiter: store.NewLimiter(uint64(limits.MaxFetchedSeriesPerQuery(userID)), failedCounter), - } - } -} - -func newBytesLimiterFactory(limits *validation.Overrides, userID string) store.BytesLimiterFactory { - return func(failedCounter prometheus.Counter) store.BytesLimiter { - // Since limit overrides could be live reloaded, we have to get the current user's limit - // each time a new limiter is instantiated. - return &limiter{ - limiter: store.NewLimiter(uint64(limits.MaxDownloadedBytesPerRequest(userID)), failedCounter), - } - } -} diff --git a/pkg/storegateway/bucket_stores_test.go b/pkg/storegateway/bucket_stores_test.go index a10a4599fb..d3efcfc112 100644 --- a/pkg/storegateway/bucket_stores_test.go +++ b/pkg/storegateway/bucket_stores_test.go @@ -763,6 +763,98 @@ func TestBucketStores_deleteLocalFilesForExcludedTenants(t *testing.T) { `), metricNames...)) } +func TestBucketStores_tokenBuckets(t *testing.T) { + const ( + user1 = "user-1" + user2 = "user-2" + ) + + ctx := context.Background() + cfg := prepareStorageConfig(t) + cfg.BucketStore.TokenBucketBytesLimiter.Mode = string(cortex_tsdb.TokenBucketBytesLimiterEnabled) + + storageDir := t.TempDir() + userToMetric := map[string]string{ + user1: "series_1", + user2: "series_2", + } + for userID, metricName := range userToMetric { + generateStorageBlock(t, storageDir, userID, metricName, 10, 100, 15) + } + + sharding := userShardingStrategy{} + sharding.users = []string{user1, user2} + + bucket, err := filesystem.NewBucketClient(filesystem.Config{Directory: storageDir}) + assert.NoError(t, err) + + reg := prometheus.NewPedanticRegistry() + stores, err := NewBucketStores(cfg, &sharding, objstore.WithNoopInstr(bucket), defaultLimitsOverrides(t), mockLoggingLevel(), log.NewNopLogger(), reg) + assert.NoError(t, err) + assert.NotNil(t, stores.instanceTokenBucket) + + assert.NoError(t, stores.InitialSync(ctx)) + assert.NotNil(t, stores.getUserTokenBucket("user-1")) + assert.NotNil(t, stores.getUserTokenBucket("user-2")) + + sharding.users = []string{user1} + assert.NoError(t, stores.SyncBlocks(ctx)) + assert.NotNil(t, stores.getUserTokenBucket("user-1")) + assert.Nil(t, stores.getUserTokenBucket("user-2")) + + sharding.users = []string{} + assert.NoError(t, stores.SyncBlocks(ctx)) + assert.Nil(t, stores.getUserTokenBucket("user-1")) + assert.Nil(t, stores.getUserTokenBucket("user-2")) + + cfg.BucketStore.TokenBucketBytesLimiter.Mode = string(cortex_tsdb.TokenBucketBytesLimiterDryRun) + sharding.users = []string{user1, user2} + reg = prometheus.NewPedanticRegistry() + stores, err = NewBucketStores(cfg, &sharding, objstore.WithNoopInstr(bucket), defaultLimitsOverrides(t), mockLoggingLevel(), log.NewNopLogger(), reg) + assert.NoError(t, err) + assert.NotNil(t, stores.instanceTokenBucket) + + assert.NoError(t, stores.InitialSync(ctx)) + assert.NotNil(t, stores.getUserTokenBucket("user-1")) + assert.NotNil(t, stores.getUserTokenBucket("user-2")) + + cfg.BucketStore.TokenBucketBytesLimiter.Mode = string(cortex_tsdb.TokenBucketBytesLimiterDisabled) + sharding.users = []string{user1, user2} + reg = prometheus.NewPedanticRegistry() + stores, err = NewBucketStores(cfg, &sharding, objstore.WithNoopInstr(bucket), defaultLimitsOverrides(t), mockLoggingLevel(), log.NewNopLogger(), reg) + assert.NoError(t, err) + + assert.NoError(t, stores.InitialSync(ctx)) + assert.Nil(t, stores.instanceTokenBucket) + assert.Nil(t, stores.getUserTokenBucket("user-1")) + assert.Nil(t, stores.getUserTokenBucket("user-2")) +} + +func TestBucketStores_getTokensToRetrieve(t *testing.T) { + cfg := prepareStorageConfig(t) + cfg.BucketStore.TokenBucketBytesLimiter.FetchedPostingsTokenFactor = 1 + cfg.BucketStore.TokenBucketBytesLimiter.TouchedPostingsTokenFactor = 2 + cfg.BucketStore.TokenBucketBytesLimiter.FetchedSeriesTokenFactor = 3 + cfg.BucketStore.TokenBucketBytesLimiter.TouchedSeriesTokenFactor = 4 + cfg.BucketStore.TokenBucketBytesLimiter.FetchedChunksTokenFactor = 0 + cfg.BucketStore.TokenBucketBytesLimiter.TouchedChunksTokenFactor = 0.5 + + storageDir := t.TempDir() + bucket, err := filesystem.NewBucketClient(filesystem.Config{Directory: storageDir}) + assert.NoError(t, err) + + reg := prometheus.NewPedanticRegistry() + stores, err := NewBucketStores(cfg, NewNoShardingStrategy(log.NewNopLogger(), nil), objstore.WithNoopInstr(bucket), defaultLimitsOverrides(t), mockLoggingLevel(), log.NewNopLogger(), reg) + assert.NoError(t, err) + + assert.Equal(t, int64(2), stores.getTokensToRetrieve(2, store.PostingsFetched)) + assert.Equal(t, int64(4), stores.getTokensToRetrieve(2, store.PostingsTouched)) + assert.Equal(t, int64(6), stores.getTokensToRetrieve(2, store.SeriesFetched)) + assert.Equal(t, int64(8), stores.getTokensToRetrieve(2, store.SeriesTouched)) + assert.Equal(t, int64(0), stores.getTokensToRetrieve(2, store.ChunksFetched)) + assert.Equal(t, int64(1), stores.getTokensToRetrieve(2, store.ChunksTouched)) +} + func getUsersInDir(t *testing.T, dir string) []string { fs, err := os.ReadDir(dir) require.NoError(t, err) diff --git a/pkg/storegateway/limiter.go b/pkg/storegateway/limiter.go new file mode 100644 index 0000000000..7a633e2422 --- /dev/null +++ b/pkg/storegateway/limiter.go @@ -0,0 +1,148 @@ +package storegateway + +import ( + "net/http" + "sync" + + "github.com/go-kit/log/level" + "github.com/prometheus/client_golang/prometheus" + "github.com/thanos-io/thanos/pkg/store" + "github.com/weaveworks/common/httpgrpc" + + "github.com/cortexproject/cortex/pkg/storage/tsdb" + "github.com/cortexproject/cortex/pkg/util" + util_log "github.com/cortexproject/cortex/pkg/util/log" + "github.com/cortexproject/cortex/pkg/util/validation" +) + +const tokenBucketBytesLimiterErrStr = "store gateway resource exhausted" + +type limiter struct { + limiter *store.Limiter +} + +func (c *limiter) Reserve(num uint64) error { + return c.ReserveWithType(num, 0) +} + +func (c *limiter) ReserveWithType(num uint64, _ store.StoreDataType) error { + err := c.limiter.Reserve(num) + if err != nil { + return httpgrpc.Errorf(http.StatusUnprocessableEntity, err.Error()) + } + + return nil +} + +type compositeBytesLimiter struct { + limiters []store.BytesLimiter +} + +func (c *compositeBytesLimiter) ReserveWithType(num uint64, dataType store.StoreDataType) error { + for _, l := range c.limiters { + if err := l.ReserveWithType(num, dataType); err != nil { + return err // nested limiters are expected to return httpgrpc error + } + } + return nil +} + +type tokenBucketBytesLimiter struct { + instanceTokenBucket *util.TokenBucket + userTokenBucket *util.TokenBucket + requestTokenBucket *util.TokenBucket + getTokensToRetrieve func(tokens uint64, dataType store.StoreDataType) int64 + dryRun bool + + // Counter metric which we will increase if limit is exceeded. + failedCounter prometheus.Counter + failedOnce sync.Once +} + +func (t *tokenBucketBytesLimiter) Reserve(_ uint64) error { + return nil +} + +func (t *tokenBucketBytesLimiter) ReserveWithType(num uint64, dataType store.StoreDataType) error { + tokensToRetrieve := t.getTokensToRetrieve(num, dataType) + + requestTokenRemaining := t.requestTokenBucket.Retrieve(tokensToRetrieve) + userTokenRemaining := t.userTokenBucket.Retrieve(tokensToRetrieve) + instanceTokenRemaining := t.instanceTokenBucket.Retrieve(tokensToRetrieve) + + // if we can retrieve from request bucket, let the request go through + if requestTokenRemaining >= 0 { + return nil + } + + errCode := 0 + + if tokensToRetrieve > t.userTokenBucket.MaxCapacity() || tokensToRetrieve > t.instanceTokenBucket.MaxCapacity() { + errCode = http.StatusUnprocessableEntity + } else if userTokenRemaining < 0 || instanceTokenRemaining < 0 { + errCode = http.StatusTooManyRequests + } + + if errCode > 0 { + if t.dryRun { + level.Warn(util_log.Logger).Log("msg", tokenBucketBytesLimiterErrStr, "dataType", dataType, "dataSize", num, "tokens", tokensToRetrieve, "errorCode", errCode) + return nil + } + + // We need to protect from the counter being incremented twice due to concurrency + t.failedOnce.Do(t.failedCounter.Inc) + return httpgrpc.Errorf(errCode, tokenBucketBytesLimiterErrStr) + } + + return nil +} + +func newTokenBucketBytesLimiter(requestTokenBucket, userTokenBucket, instanceTokenBucket *util.TokenBucket, dryRun bool, failedCounter prometheus.Counter, getTokensToRetrieve func(tokens uint64, dataType store.StoreDataType) int64) *tokenBucketBytesLimiter { + return &tokenBucketBytesLimiter{ + requestTokenBucket: requestTokenBucket, + userTokenBucket: userTokenBucket, + instanceTokenBucket: instanceTokenBucket, + dryRun: dryRun, + failedCounter: failedCounter, + getTokensToRetrieve: getTokensToRetrieve, + } +} + +func newChunksLimiterFactory(limits *validation.Overrides, userID string) store.ChunksLimiterFactory { + return func(failedCounter prometheus.Counter) store.ChunksLimiter { + // Since limit overrides could be live reloaded, we have to get the current user's limit + // each time a new limiter is instantiated. + return &limiter{ + limiter: store.NewLimiter(uint64(limits.MaxChunksPerQueryFromStore(userID)), failedCounter), + } + } +} + +func newSeriesLimiterFactory(limits *validation.Overrides, userID string) store.SeriesLimiterFactory { + return func(failedCounter prometheus.Counter) store.SeriesLimiter { + // Since limit overrides could be live reloaded, we have to get the current user's limit + // each time a new limiter is instantiated. + return &limiter{ + limiter: store.NewLimiter(uint64(limits.MaxFetchedSeriesPerQuery(userID)), failedCounter), + } + } +} + +func newBytesLimiterFactory(limits *validation.Overrides, userID string, userTokenBucket, instanceTokenBucket *util.TokenBucket, tokenBucketBytesLimiterCfg tsdb.TokenBucketBytesLimiterConfig, getTokensToRetrieve func(tokens uint64, dataType store.StoreDataType) int64) store.BytesLimiterFactory { + return func(failedCounter prometheus.Counter) store.BytesLimiter { + limiters := []store.BytesLimiter{} + // Since limit overrides could be live reloaded, we have to get the current user's limit + // each time a new limiter is instantiated. + limiters = append(limiters, &limiter{limiter: store.NewLimiter(uint64(limits.MaxDownloadedBytesPerRequest(userID)), failedCounter)}) + + if tokenBucketBytesLimiterCfg.Mode != string(tsdb.TokenBucketBytesLimiterDisabled) { + requestTokenBucket := util.NewTokenBucket(tokenBucketBytesLimiterCfg.RequestTokenBucketSize, nil) + dryRun := tokenBucketBytesLimiterCfg.Mode == string(tsdb.TokenBucketBytesLimiterDryRun) + limiters = append(limiters, newTokenBucketBytesLimiter(requestTokenBucket, userTokenBucket, instanceTokenBucket, dryRun, failedCounter, getTokensToRetrieve)) + } + + return &compositeBytesLimiter{ + limiters: limiters, + } + } +} diff --git a/pkg/storegateway/limiter_test.go b/pkg/storegateway/limiter_test.go new file mode 100644 index 0000000000..c2a61de249 --- /dev/null +++ b/pkg/storegateway/limiter_test.go @@ -0,0 +1,148 @@ +package storegateway + +import ( + "fmt" + "testing" + + "github.com/prometheus/client_golang/prometheus" + "github.com/stretchr/testify/assert" + "github.com/thanos-io/thanos/pkg/store" + + "github.com/cortexproject/cortex/pkg/util" +) + +func TestLimiter(t *testing.T) { + l := &limiter{ + limiter: store.NewLimiter(2, prometheus.NewCounter(prometheus.CounterOpts{})), + } + + assert.NoError(t, l.Reserve(1)) + assert.NoError(t, l.ReserveWithType(1, store.PostingsFetched)) + assert.Error(t, l.Reserve(1)) + assert.Error(t, l.ReserveWithType(1, store.PostingsFetched)) +} + +func TestCompositeLimiter(t *testing.T) { + l := &compositeBytesLimiter{ + limiters: []store.BytesLimiter{ + &limiter{limiter: store.NewLimiter(2, prometheus.NewCounter(prometheus.CounterOpts{}))}, + &limiter{limiter: store.NewLimiter(1, prometheus.NewCounter(prometheus.CounterOpts{}))}, + }, + } + + assert.NoError(t, l.ReserveWithType(1, store.PostingsFetched)) + assert.ErrorContains(t, l.ReserveWithType(1, store.PostingsFetched), "(422)") +} + +func TestNewTokenBucketBytesLimiter(t *testing.T) { + tests := map[string]struct { + tokensToRetrieve []uint64 + requestTokenBucketSize int64 + userTokenBucketSize int64 + instanceTokenBucketSize int64 + expectedRequestTokenRemaining int64 + expectedUserTokenRemaining int64 + expectedInstanceTokenRemaining int64 + getTokensToRetrieve func(tokens uint64, dataType store.StoreDataType) int64 + errCode int + dryRun bool + }{ + "should retrieve buckets from all buckets": { + tokensToRetrieve: []uint64{1}, + requestTokenBucketSize: 1, + userTokenBucketSize: 1, + instanceTokenBucketSize: 1, + }, + "should succeed if there is enough request token, regardless of user or instance bucket": { + tokensToRetrieve: []uint64{1}, + requestTokenBucketSize: 1, + userTokenBucketSize: 0, + instanceTokenBucketSize: 0, + expectedUserTokenRemaining: -1, + expectedInstanceTokenRemaining: -1, + }, + "should throw 429 if not enough user tokens remaining": { + tokensToRetrieve: []uint64{1, 1}, + requestTokenBucketSize: 1, + userTokenBucketSize: 1, + instanceTokenBucketSize: 2, + errCode: 429, + expectedRequestTokenRemaining: -1, + expectedUserTokenRemaining: -1, + }, + "should throw 422 if request is greater than user token bucket size": { + tokensToRetrieve: []uint64{2}, + requestTokenBucketSize: 1, + userTokenBucketSize: 1, + instanceTokenBucketSize: 2, + errCode: 422, + expectedRequestTokenRemaining: -1, + expectedUserTokenRemaining: -1, + }, + "should throw 429 if not enough instance tokesn remaining": { + tokensToRetrieve: []uint64{1, 1}, + requestTokenBucketSize: 1, + userTokenBucketSize: 2, + instanceTokenBucketSize: 1, + errCode: 429, + expectedRequestTokenRemaining: -1, + expectedInstanceTokenRemaining: -1, + }, + "should throw 422 if request is greater than instance token bucket size": { + tokensToRetrieve: []uint64{2}, + requestTokenBucketSize: 1, + userTokenBucketSize: 2, + instanceTokenBucketSize: 1, + errCode: 422, + expectedRequestTokenRemaining: -1, + expectedInstanceTokenRemaining: -1, + }, + "should use getTokensToRetrieve to calculate tokens": { + tokensToRetrieve: []uint64{1}, + getTokensToRetrieve: func(tokens uint64, dataType store.StoreDataType) int64 { + if dataType == store.PostingsFetched { + return 0 + } + return 1 + }, + }, + "should not fail if dryRun": { + tokensToRetrieve: []uint64{1}, + expectedRequestTokenRemaining: -1, + expectedUserTokenRemaining: -1, + expectedInstanceTokenRemaining: -1, + dryRun: true, + }, + } + + for testName, testData := range tests { + t.Run(testName, func(t *testing.T) { + requestTokenBucket := util.NewTokenBucket(testData.requestTokenBucketSize, nil) + userTokenBucket := util.NewTokenBucket(testData.userTokenBucketSize, nil) + instanceTokenBucket := util.NewTokenBucket(testData.instanceTokenBucketSize, nil) + + getTokensToRetrieve := func(tokens uint64, dataType store.StoreDataType) int64 { + return int64(tokens) + } + if testData.getTokensToRetrieve != nil { + getTokensToRetrieve = testData.getTokensToRetrieve + } + l := newTokenBucketBytesLimiter(requestTokenBucket, userTokenBucket, instanceTokenBucket, testData.dryRun, prometheus.NewCounter(prometheus.CounterOpts{}), getTokensToRetrieve) + + var err error + for _, token := range testData.tokensToRetrieve { + err = l.ReserveWithType(token, store.PostingsFetched) + } + + assert.Equal(t, testData.expectedRequestTokenRemaining, requestTokenBucket.Retrieve(0)) + assert.Equal(t, testData.expectedUserTokenRemaining, userTokenBucket.Retrieve(0)) + assert.Equal(t, testData.expectedInstanceTokenRemaining, instanceTokenBucket.Retrieve(0)) + + if testData.errCode > 0 { + assert.ErrorContains(t, err, fmt.Sprintf("(%d)", testData.errCode)) + } else { + assert.NoError(t, err) + } + }) + } +} diff --git a/pkg/util/token_bucket.go b/pkg/util/token_bucket.go new file mode 100644 index 0000000000..6e33c7aaf8 --- /dev/null +++ b/pkg/util/token_bucket.go @@ -0,0 +1,62 @@ +package util + +import ( + "sync" + "time" + + "github.com/prometheus/client_golang/prometheus" +) + +type TokenBucket struct { + remainingTokens int64 + maxCapacity int64 + refillRate int64 + lastRefill time.Time + mu sync.Mutex + + remainingTokensMetric prometheus.Gauge +} + +func NewTokenBucket(maxCapacity int64, remainingTokensMetric prometheus.Gauge) *TokenBucket { + if remainingTokensMetric != nil { + remainingTokensMetric.Set(float64(maxCapacity)) + } + + return &TokenBucket{ + remainingTokens: maxCapacity, + maxCapacity: maxCapacity, + refillRate: maxCapacity, + lastRefill: time.Now(), + remainingTokensMetric: remainingTokensMetric, + } +} + +// Retrieve always deducts token, even if there is not enough remaining tokens. +func (t *TokenBucket) Retrieve(amount int64) int64 { + t.mu.Lock() + defer t.mu.Unlock() + + t.updateTokens() + t.remainingTokens -= amount + + if t.remainingTokensMetric != nil { + t.remainingTokensMetric.Set(float64(t.remainingTokens)) + } + + return t.remainingTokens +} + +func (t *TokenBucket) MaxCapacity() int64 { + return t.maxCapacity +} + +func (t *TokenBucket) updateTokens() { + now := time.Now() + refilledTokens := int64(now.Sub(t.lastRefill).Seconds() * float64(t.refillRate)) + t.remainingTokens += refilledTokens + t.lastRefill = now + + if t.remainingTokens > t.maxCapacity { + t.remainingTokens = t.maxCapacity + } +} diff --git a/pkg/util/token_bucket_test.go b/pkg/util/token_bucket_test.go new file mode 100644 index 0000000000..155e36ce75 --- /dev/null +++ b/pkg/util/token_bucket_test.go @@ -0,0 +1,23 @@ +package util + +import ( + "testing" + "time" + + "github.com/stretchr/testify/assert" +) + +func TestTokenBucket_Retrieve(t *testing.T) { + bucket := NewTokenBucket(10, nil) + + assert.Equal(t, int64(0), bucket.Retrieve(10)) + assert.Negative(t, bucket.Retrieve(1)) + time.Sleep(time.Second) + assert.Positive(t, bucket.Retrieve(5)) +} + +func TestTokenBucket_MaxCapacity(t *testing.T) { + bucket := NewTokenBucket(10, nil) + + assert.Equal(t, int64(10), bucket.MaxCapacity()) +} From e6f95d9c772dc07e74e9954aeaa6e70831311cac Mon Sep 17 00:00:00 2001 From: Charlie Le Date: Sat, 20 Jul 2024 15:12:30 -0700 Subject: [PATCH 15/32] Update Kubernetes Getting Started This change updates the Kubernetes example to match the Docker getting started example. - Updates Cortex Helm chart to 2.4.0 - Sets up multi-tenancy - Sets up Alertmanager and Ruler - Sets up Grafana dashboards Signed-off-by: Charlie Le --- docs/getting-started/_index.md | 94 ++++++++++++++++----- docs/getting-started/cortex-values.yaml | 37 ++++---- docs/getting-started/grafana-values.yaml | 11 ++- docs/getting-started/prometheus-values.yaml | 2 + docs/getting-started/seaweedfs.yaml | 5 ++ 5 files changed, 110 insertions(+), 39 deletions(-) diff --git a/docs/getting-started/_index.md b/docs/getting-started/_index.md index 19d744317e..fcc2ab0e9f 100644 --- a/docs/getting-started/_index.md +++ b/docs/getting-started/_index.md @@ -162,14 +162,21 @@ $ kubectl create namespace cortex $ kubectl -n cortex apply -f seaweedfs.yaml ``` +```sh +# Wait for SeaweedFS to be ready +$ kubectl -n cortex wait --for=condition=ready pod -l app=seaweedfs +``` + ```sh # Port-forward to SeaweedFS to create a bucket $ kubectl -n cortex port-forward svc/seaweedfs 8333 ``` -```shell -# Create a bucket -$ curl --aws-sigv4 "aws:amz:local:seaweedfs" --user "any:any" -X PUT http://localhost:8333/cortex-bucket +```sh +# Create buckets in SeaweedFS +$ curl --aws-sigv4 "aws:amz:local:seaweedfs" --user "any:any" -X PUT http://localhost:8333/cortex-blocks +$ curl --aws-sigv4 "aws:amz:local:seaweedfs" --user "any:any" -X PUT http://localhost:8333/cortex-ruler +$ curl --aws-sigv4 "aws:amz:local:seaweedfs" --user "any:any" -X PUT http://localhost:8333/cortex-alertmanager ``` #### Setup Cortex @@ -177,7 +184,7 @@ $ curl --aws-sigv4 "aws:amz:local:seaweedfs" --user "any:any" -X PUT http://loca ```sh # Deploy Cortex using the provided values file which configures # - blocks storage to use the seaweedfs service -$ helm upgrade --install --version=2.3.0 --namespace cortex cortex cortex-helm/cortex -f cortex-values.yaml +$ helm upgrade --install --version=2.4.0 --namespace cortex cortex cortex-helm/cortex -f cortex-values.yaml ``` #### Setup Prometheus @@ -187,6 +194,10 @@ $ helm upgrade --install --version=2.3.0 --namespace cortex cortex cortex-helm/ $ helm upgrade --install --version=25.20.1 --namespace cortex prometheus prometheus-community/prometheus -f prometheus-values.yaml ``` +If everything is working correctly, Prometheus should be sending metrics that it is scraping to Cortex. Prometheus is +configured to send metrics to Cortex via `remote_write`. Check out the `prometheus-config.yaml` file to see +how this is configured. + #### Setup Grafana ```sh @@ -194,48 +205,91 @@ $ helm upgrade --install --version=25.20.1 --namespace cortex prometheus prometh $ helm upgrade --install --version=7.3.9 --namespace cortex grafana grafana/grafana -f grafana-values.yaml ``` -#### Explore +```sh +# Create dashboards for Cortex +$ for dashboard in $(ls dashboards); do + basename=$(basename -s .json $dashboard) + cmname=grafana-dashboard-$basename + kubectl create -n cortex cm $cmname --from-file=$dashboard=dashboards/$dashboard --save-config=true -o yaml --dry-run=client | kubectl apply -f - + kubectl patch -n cortex cm $cmname -p '{"metadata":{"labels":{"grafana_dashboard":""}}}' +done + +``` ```sh # Port-forward to Grafana to visualize kubectl --namespace cortex port-forward deploy/grafana 3000 ``` -Grafana is configured to use Cortex as a data source. You can explore the data source in Grafana and query metrics. For example, this [explore](http://localhost:3000/explore?schemaVersion=1&panes=%7B%22au0%22:%7B%22datasource%22:%22P6693426190CB2316%22,%22queries%22:%5B%7B%22refId%22:%22A%22,%22expr%22:%22rate%28prometheus_remote_storage_samples_total%5B$__rate_interval%5D%29%22,%22range%22:true,%22instant%22:true,%22datasource%22:%7B%22type%22:%22prometheus%22,%22uid%22:%22P6693426190CB2316%22%7D,%22editorMode%22:%22builder%22,%22legendFormat%22:%22__auto%22,%22useBackend%22:false,%22disableTextWrap%22:false,%22fullMetaSearch%22:false,%22includeNullMetadata%22:false%7D%5D,%22range%22:%7B%22from%22:%22now-1h%22,%22to%22:%22now%22%7D%7D%7D&orgId=1) page is showing the rate of samples being sent to Cortex. +#### Configure Cortex Recording Rules and Alerting Rules (Optional) + +We can configure Cortex with [cortextool](https://github.com/cortexproject/cortex-tools/) to load [recording rules](https://prometheus.io/docs/prometheus/latest/configuration/recording_rules/) and [alerting rules](https://prometheus.io/docs/prometheus/latest/configuration/alerting_rules/). This is optional, but it is helpful to see how Cortex can be configured to manage rules and alerts. + +```sh +# Port forward to the alertmanager to configure recording rules and alerts +$ kubectl --namespace cortex port-forward svc/cortex-nginx 8080:80 +``` + +```sh +# Configure recording rules for the cortex tenant +$ cortextool rules sync rules.yaml alerts.yaml --id cortex --address http://localhost:8080 +``` +#### Configure Cortex Alertmanager (Optional) + +Cortex also comes with a multi-tenant Alertmanager. Let's load configuration for it to be able to view them in Grafana. + +```sh +# Configure alertmanager for the cortex tenant +$ cortextool alertmanager load alertmanager-config.yaml --id cortex --address http://localhost:8080 +``` + +You can configure Alertmanager in [Grafana as well](http://localhost:3000/alerting/notifications?search=&alertmanager=Cortex%20Alertmanager). + +There's a list of recording rules and alerts that should be visible in Grafana [here](http://localhost:3000/alerting/list?view=list&search=datasource:Cortex). + +#### Explore + +Grafana is configured to use Cortex as a data source. Grafana is also configured with [Cortex Dashboards](http://localhost:3000/dashboards?tag=cortex) to understand the state of the Cortex instance. The dashboards are generated from the cortex-jsonnet repository. There is a Makefile in the repository that can be used to update the dashboards. + +```sh +# Update the dashboards (optional) +$ make +``` If everything is working correctly, then the metrics seen in Grafana were successfully sent from Prometheus to Cortex -via remote_write! +via `remote_write`! Other things to explore: +[Cortex](http://localhost:9009) - Administrative interface for Cortex + ```sh -# Port forward to the ingester to see the administrative interface for Cortex: -$ kubectl --namespace cortex port-forward deploy/cortex-ingester 8080 +# Port forward to the ingester to see the administrative interface for Cortex +$ kubectl --namespace cortex port-forward deploy/cortex-ingester 9009:8080 ``` -- [Cortex Ingester](http://localhost:8080) - - Try shutting down the [ingester](http://localhost:8080/ingester/shutdown) and see how it affects metric ingestion. - - Restart ingester pod to bring the ingester back online, and see if Prometheus affected. - - Does it affect the querying of metrics in Grafana? How many ingesters must be offline before it affects querying? +- Try shutting down the ingester, and see how it affects metric ingestion. +- Restart Cortex to bring the ingester back online, and see how Prometheus catches up. +- Does it affect the querying of metrics in Grafana? +[Prometheus](http://localhost:9090) - Prometheus instance that is sending metrics to Cortex ```sh -# Port forward to Prometheus to see the metrics that are being scraped: +# Port forward to Prometheus to see the metrics that are being scraped $ kubectl --namespace cortex port-forward deploy/prometheus-server 9090 ``` +- Try querying the metrics in Prometheus. +- Are they the same as what you see in Cortex? -- [Prometheus](http://localhost:9090) - Prometheus instance that is sending metrics to Cortex - - Try querying the metrics in Prometheus. - - Are they the same as what you see in Cortex? +[Grafana](http://localhost:3000) - Grafana instance that is visualizing the metrics. ```sh -# Port forward to Prometheus to see the metrics that are being scraped: +# Port forward to Grafana to visualize $ kubectl --namespace cortex port-forward deploy/grafana 3000 ``` -- [Grafana](http://localhost:3000) - Grafana instance that is visualizing the metrics. - - Try creating a new dashboard and adding a new panel with a query to Cortex. +- Try creating a new dashboard and adding a new panel with a query to Cortex. ### Clean up diff --git a/docs/getting-started/cortex-values.yaml b/docs/getting-started/cortex-values.yaml index 1be7862b1e..656e85f213 100644 --- a/docs/getting-started/cortex-values.yaml +++ b/docs/getting-started/cortex-values.yaml @@ -36,7 +36,7 @@ externalConfigSecretName: 'secret-with-config.yaml' externalConfigVersion: '0' config: - auth_enabled: false + auth_enabled: true api: prometheus_http_prefix: '/prometheus' # -- Use GZIP compression for API responses. Some endpoints serve large YAML or JSON blobs @@ -82,7 +82,7 @@ config: s3: &s3 endpoint: seaweedfs.cortex.svc.cluster.local:8333 region: local - bucket_name: cortex-bucket + bucket_name: cortex-blocks access_key_id: any secret_access_key: any insecure: true @@ -93,9 +93,13 @@ config: bucket_index: enabled: true alertmanager_storage: - s3: *s3 + s3: + <<: *s3 + bucket_name: cortex-alertmanager ruler_storage: - s3: *s3 + s3: + <<: *s3 + bucket_name: cortex-ruler # -- https://cortexmetrics.io/docs/configuration/configuration-file/#store_gateway_config store_gateway: sharding_enabled: false @@ -144,7 +148,7 @@ config: cluster: listen_address: '0.0.0.0:9094' # -- Enable the experimental alertmanager config api. - enable_api: false + enable_api: true external_url: '/api/prom/alertmanager' frontend: log_queries_longer_than: 10s @@ -275,10 +279,14 @@ alertmanager: extraContainers: [] # -- Additional volumes to the cortex pod. - extraVolumes: [] + extraVolumes: + - name: tmp + emptyDir: { } # -- Extra volume mounts that will be added to the cortex container - extraVolumeMounts: [] + extraVolumeMounts: + - name: tmp + mountPath: /tmp # -- Additional ports to the cortex services. Useful to expose extra container ports. extraPorts: [] @@ -1423,11 +1431,7 @@ store_gateway: path: /ready port: http-metrics scheme: HTTP - livenessProbe: - httpGet: - path: /ready - port: http-metrics - scheme: HTTP + livenessProbe: {} readinessProbe: httpGet: path: /ready @@ -1543,11 +1547,7 @@ compactor: path: /ready port: http-metrics scheme: HTTP - livenessProbe: - httpGet: - path: /ready - port: http-metrics - scheme: HTTP + livenessProbe: {} readinessProbe: httpGet: path: /ready @@ -1670,4 +1670,5 @@ memcached-blocks-metadata: memberlist: service: annotations: {} - labels: {} \ No newline at end of file + labels: {} + diff --git a/docs/getting-started/grafana-values.yaml b/docs/getting-started/grafana-values.yaml index d612cb27d8..b36403b0a0 100644 --- a/docs/getting-started/grafana-values.yaml +++ b/docs/getting-started/grafana-values.yaml @@ -605,6 +605,15 @@ datasources: access: proxy isDefault: true editable: true + jsonData: + cacheLevel: None + httpHeaderName1: X-Scope-OrgID + httpMethod: POST + prometheusType: Cortex + prometheusVersion: 1.14.0 + timeInterval: 15s + secureJsonData: + httpHeaderValue1: cortex # - name: CloudWatch # type: cloudwatch # access: proxy @@ -912,7 +921,7 @@ sidecar: # Sets the size limit of the alert sidecar emptyDir volume sizeLimit: {} dashboards: - enabled: false + enabled: true # Additional environment variables for the dashboards sidecar env: {} # Do not reprocess already processed unchanged resources on k8s API reconnect. diff --git a/docs/getting-started/prometheus-values.yaml b/docs/getting-started/prometheus-values.yaml index c7ca147ad7..ef3ea08cbf 100644 --- a/docs/getting-started/prometheus-values.yaml +++ b/docs/getting-started/prometheus-values.yaml @@ -249,6 +249,8 @@ server: ## remoteWrite: - url: http://cortex-nginx/api/v1/push + headers: + X-Scope-OrgID: "cortex" ## https://prometheus.io/docs/prometheus/latest/configuration/configuration/#remote_read ## remoteRead: [] diff --git a/docs/getting-started/seaweedfs.yaml b/docs/getting-started/seaweedfs.yaml index 5ca4f24ddb..974e79102f 100644 --- a/docs/getting-started/seaweedfs.yaml +++ b/docs/getting-started/seaweedfs.yaml @@ -47,8 +47,13 @@ spec: - "server" - "-s3" - "-s3.config=/workspace/seaweedfs-config.json" + readinessProbe: + httpGet: + path: /status + port: 8333 ports: - containerPort: 8333 + name: seaweedfs volumeMounts: - name: seaweedfs-config mountPath: /workspace From 4e7dcfd153aeff454e3d5cdee105201839035989 Mon Sep 17 00:00:00 2001 From: Harry John Date: Fri, 26 Jul 2024 10:06:46 -0700 Subject: [PATCH 16/32] Add Alvin under Emeritus Maintainers (#6118) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: 🌲 Harry 🌊 John 🏔 --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index 920e58deda..f66352b508 100644 --- a/README.md +++ b/README.md @@ -177,6 +177,7 @@ There are several commercial services where you can use Cortex on-demand: * Goutham Veeramachaneni @gouthamve * Jacob Lisi @jtlisi * Tom Wilkie @tomwilkie +* Alvin Lin @alvinlin123 ## History of Cortex From 36539d5b2da915da968375bd1f30cd1eb847db86 Mon Sep 17 00:00:00 2001 From: Dimitar Zahariev Date: Mon, 29 Jul 2024 17:36:10 +0300 Subject: [PATCH 17/32] Update architecture.md Changed the introductory sentences for the Store Gateway. Signed-off-by: Dimitar Zahariev --- docs/architecture.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/architecture.md b/docs/architecture.md index 4d48cd1c29..ebd6b865d5 100644 --- a/docs/architecture.md +++ b/docs/architecture.md @@ -179,7 +179,7 @@ For more information, see the [compactor documentation](./blocks-storage/compact The compactor is **stateless**. ### Store gateway -The **store gateway** is the Cortex service responsible to query series from blocks, it needs to have an almost up-to-date view over the storage bucket. In order to discover blocks belonging to their shard. The store-gateway can keep the bucket view updated in to two different ways: +The **store gateway** is the Cortex service responsible for querying series from blocks. It needs to have an almost up-to-date view of the storage bucket. To discover blocks belonging to their shard, the store gateway can keep the bucket view updated in two different ways: 1. Periodically scanning the bucket (default) 2. Periodically downloading the [bucket index](./blocks-storage/bucket-index.md) From ebaf4a4c227f2abba403bf42833d192048e1ffe5 Mon Sep 17 00:00:00 2001 From: Alvin Lin Date: Tue, 30 Jul 2024 00:21:31 -0700 Subject: [PATCH 18/32] Stepping down as Cortex maintainer (#6082) I can't believe this day would come, but I am stepping down as Cortex maintainer because I no longer have capacity to give Cortex the love it needs anymore. Also, Cortex is in good hands with existing maintainers from diverse set of companies. It's been a fun ride. I still remember my [first Cortex PR](https://github.com/cortexproject/cortex/pull/2741) that allows me to build Cortex in my employer's network, and [the day I became maintainer 3 years ago](https://github.com/cortexproject/cortex/commit/b4daa22055ffec14311d8b5d2d9429f1bd575dad). It was full of joys and emotions. But there is always an end to a party, and for me it's today. Thank you all for the great work and support thus far! Signed-off-by: Alvin Lin --- MAINTAINERS | 1 - 1 file changed, 1 deletion(-) diff --git a/MAINTAINERS b/MAINTAINERS index 55b4e04481..1bedc4815d 100644 --- a/MAINTAINERS +++ b/MAINTAINERS @@ -1,5 +1,4 @@ Alan Protasio, Amazon Web Services (@alanprot) -Alvin Lin, Amazon Web Services (@alvinlin123) Ben Ye, Amazon Web Services (@yeya24) Charlie Le, Apple (@CharlieTLe) Daniel Blando, Amazon Web Services (@danielblando) From 7d6351f005d84965be556c7b718adbd959f481e1 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 30 Jul 2024 10:11:10 -0700 Subject: [PATCH 19/32] Bump github.com/alicebob/miniredis/v2 from 2.32.1 to 2.33.0 (#6123) Bumps [github.com/alicebob/miniredis/v2](https://github.com/alicebob/miniredis) from 2.32.1 to 2.33.0. - [Release notes](https://github.com/alicebob/miniredis/releases) - [Changelog](https://github.com/alicebob/miniredis/blob/master/CHANGELOG.md) - [Commits](https://github.com/alicebob/miniredis/compare/v2.32.1...v2.33.0) --- updated-dependencies: - dependency-name: github.com/alicebob/miniredis/v2 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 | 5 +- .../alicebob/miniredis/v2/CHANGELOG.md | 12 + .../github.com/alicebob/miniredis/v2/Makefile | 37 ++- .../alicebob/miniredis/v2/README.md | 2 +- .../alicebob/miniredis/v2/cmd_generic.go | 237 +++++++------- .../alicebob/miniredis/v2/cmd_set.go | 4 + .../alicebob/miniredis/v2/cmd_stream.go | 17 +- vendor/github.com/alicebob/miniredis/v2/db.go | 12 + .../alicebob/miniredis/v2/direct.go | 3 + .../github.com/alicebob/miniredis/v2/lua.go | 9 + .../alicebob/miniredis/v2/miniredis.go | 27 ++ .../github.com/alicebob/miniredis/v2/opts.go | 11 + .../alicebob/miniredis/v2/proto/Makefile | 2 + .../alicebob/miniredis/v2/proto/client.go | 60 ++++ .../alicebob/miniredis/v2/proto/proto.go | 288 ++++++++++++++++++ .../alicebob/miniredis/v2/proto/types.go | 102 +++++++ .../github.com/alicebob/miniredis/v2/redis.go | 1 + vendor/modules.txt | 5 +- 19 files changed, 698 insertions(+), 138 deletions(-) create mode 100644 vendor/github.com/alicebob/miniredis/v2/proto/Makefile create mode 100644 vendor/github.com/alicebob/miniredis/v2/proto/client.go create mode 100644 vendor/github.com/alicebob/miniredis/v2/proto/proto.go create mode 100644 vendor/github.com/alicebob/miniredis/v2/proto/types.go diff --git a/go.mod b/go.mod index e2634ddf43..8ea64be567 100644 --- a/go.mod +++ b/go.mod @@ -7,7 +7,7 @@ toolchain go1.22.5 require ( github.com/Masterminds/squirrel v1.5.4 github.com/alecthomas/units v0.0.0-20231202071711-9a357b53e9c9 - github.com/alicebob/miniredis/v2 v2.32.1 + github.com/alicebob/miniredis/v2 v2.33.0 github.com/armon/go-metrics v0.4.1 github.com/aws/aws-sdk-go v1.54.19 github.com/bradfitz/gomemcache v0.0.0-20230905024940-24af94b03874 diff --git a/go.sum b/go.sum index ae117c8408..bf2b36ab02 100644 --- a/go.sum +++ b/go.sum @@ -579,8 +579,8 @@ github.com/alecthomas/units v0.0.0-20231202071711-9a357b53e9c9 h1:ez/4by2iGztzR4 github.com/alecthomas/units v0.0.0-20231202071711-9a357b53e9c9/go.mod h1:OMCwj8VM1Kc9e19TLln2VL61YJF0x1XFtfdL4JdbSyE= github.com/alicebob/gopher-json v0.0.0-20200520072559-a9ecdc9d1d3a h1:HbKu58rmZpUGpz5+4FfNmIU+FmZg2P3Xaj2v2bfNWmk= github.com/alicebob/gopher-json v0.0.0-20200520072559-a9ecdc9d1d3a/go.mod h1:SGnFV6hVsYE877CKEZ6tDNTjaSXYUk6QqoIK6PrAtcc= -github.com/alicebob/miniredis/v2 v2.32.1 h1:Bz7CciDnYSaa0mX5xODh6GUITRSx+cVhjNoOR4JssBo= -github.com/alicebob/miniredis/v2 v2.32.1/go.mod h1:AqkLNAfUm0K07J28hnAyyQKf/x0YkCY/g5DCtuL01Mw= +github.com/alicebob/miniredis/v2 v2.33.0 h1:uvTF0EDeu9RLnUEG27Db5I68ESoIxTiXbNUiji6lZrA= +github.com/alicebob/miniredis/v2 v2.33.0/go.mod h1:MhP4a3EU7aENRi9aO+tHfTBZicLqQevyi/DJpoj6mi0= github.com/aliyun/aliyun-oss-go-sdk v2.2.2+incompatible h1:9gWa46nstkJ9miBReJcN8Gq34cBFbzSpQZVVT9N09TM= github.com/aliyun/aliyun-oss-go-sdk v2.2.2+incompatible/go.mod h1:T/Aws4fEfogEE9v+HPhhw+CntffsBHJ8nXQCwKr0/g8= github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156 h1:eMwmnE/GDgah4HI848JfFxHt+iPb26b4zyfspmqY0/8= @@ -1729,7 +1729,6 @@ golang.org/x/sys v0.0.0-20180823144017-11551d06cbcc/go.mod h1:STP8DvDyc/dI5b8T5h golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20190204203706-41f3e6584952/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190222072716-a9d3bda3a223/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= diff --git a/vendor/github.com/alicebob/miniredis/v2/CHANGELOG.md b/vendor/github.com/alicebob/miniredis/v2/CHANGELOG.md index 94cbbf2330..28ed159c31 100644 --- a/vendor/github.com/alicebob/miniredis/v2/CHANGELOG.md +++ b/vendor/github.com/alicebob/miniredis/v2/CHANGELOG.md @@ -1,6 +1,18 @@ ## Changelog +### v2.33.0 + +- minimum Go version is now 1.17 +- fix integer overflow (thanks @wszaranski) +- test against the last BSD redis (7.2.4) +- ignore 'redis.set_repl()' call (thanks @TingluoHuang) +- various build fixes (thanks @wszaranski) +- add StartAddrTLS function (thanks @agriffaut) +- support for the NOMKSTREAM option for XADD (thanks @Jahaja) +- return empty array for SRANDMEMBER on nonexistent key (thanks @WKBae) + + ### v2.32.1 - support for SINTERCARD (thanks @s-barr-fetch) diff --git a/vendor/github.com/alicebob/miniredis/v2/Makefile b/vendor/github.com/alicebob/miniredis/v2/Makefile index 125f574cff..2b5ec3eca9 100644 --- a/vendor/github.com/alicebob/miniredis/v2/Makefile +++ b/vendor/github.com/alicebob/miniredis/v2/Makefile @@ -1,12 +1,33 @@ -.PHONY: all test testrace int - -all: test - -test: +.PHONY: test +test: ### Run unit tests go test ./... -testrace: +.PHONY: testrace +testrace: ### Run unit tests with race detector go test -race ./... -int: - INT=1 go test ./... +.PHONY: int +int: ### Run integration tests (doesn't download redis server) + ${MAKE} -C integration int + +.PHONY: ci +ci: ### Run full tests suite (including download and compilation of proper redis server) + ${MAKE} test + ${MAKE} -C integration redis_src/redis-server int + ${MAKE} testrace + +.PHONY: clean +clean: ### Clean integration test files and remove compiled redis from integration/redis_src + ${MAKE} -C integration clean + +.PHONY: help +help: +ifeq ($(UNAME), Linux) + @grep -P '^[a-zA-Z_-]+:.*?## .*$$' $(MAKEFILE_LIST) | sort | \ + awk 'BEGIN {FS = ":.*?## "}; {printf "\033[36m%-20s\033[0m %s\n", $$1, $$2}' +else + @# this is not tested, but prepared in advance for you, Mac drivers + @awk -F ':.*###' '$$0 ~ FS {printf "%15s%s\n", $$1 ":", $$2}' \ + $(MAKEFILE_LIST) | grep -v '@awk' | sort +endif + diff --git a/vendor/github.com/alicebob/miniredis/v2/README.md b/vendor/github.com/alicebob/miniredis/v2/README.md index 0ffb55468b..5d4e4d8ba7 100644 --- a/vendor/github.com/alicebob/miniredis/v2/README.md +++ b/vendor/github.com/alicebob/miniredis/v2/README.md @@ -327,7 +327,7 @@ Commands which will probably not be implemented: ## &c. -Integration tests are run against Redis 7.2.0. The [./integration](./integration/) subdir +Integration tests are run against Redis 7.2.4. The [./integration](./integration/) subdir compares miniredis against a real redis instance. The Redis 6 RESP3 protocol is supported. If there are problems, please open diff --git a/vendor/github.com/alicebob/miniredis/v2/cmd_generic.go b/vendor/github.com/alicebob/miniredis/v2/cmd_generic.go index f3fd604e19..721ad2fabe 100644 --- a/vendor/github.com/alicebob/miniredis/v2/cmd_generic.go +++ b/vendor/github.com/alicebob/miniredis/v2/cmd_generic.go @@ -3,6 +3,7 @@ package miniredis import ( + "errors" "fmt" "sort" "strconv" @@ -24,10 +25,7 @@ func inSeconds(t time.Time) int { } func inMilliSeconds(t time.Time) int { - // Time.UnixMilli() was added in go 1.17 - // return int(t.UnixNano() / 1000000) is limited to dates between year 1678 and 2262 - // by using following calculation we extend this time without too much complexity - return int(t.Unix())*1000 + t.Nanosecond()/1000000 + return int(t.UnixMilli()) } // commandsGeneric handles EXPIRE, TTL, PERSIST, &c. @@ -60,6 +58,47 @@ func commandsGeneric(m *Miniredis) { m.srv.Register("UNLINK", m.cmdDel) } +type expireOpts struct { + key string + value int + nx bool + xx bool + gt bool + lt bool +} + +func expireParse(cmd string, args []string) (*expireOpts, error) { + var opts expireOpts + + opts.key = args[0] + if err := optIntSimple(args[1], &opts.value); err != nil { + return nil, err + } + args = args[2:] + for len(args) > 0 { + switch strings.ToLower(args[0]) { + case "nx": + opts.nx = true + case "xx": + opts.xx = true + case "gt": + opts.gt = true + case "lt": + opts.lt = true + default: + return nil, fmt.Errorf("ERR Unsupported option %s", args[0]) + } + args = args[1:] + } + if opts.gt && opts.lt { + return nil, errors.New("ERR GT and LT options at the same time are not compatible") + } + if opts.nx && (opts.xx || opts.gt || opts.lt) { + return nil, errors.New("ERR NX and XX, GT or LT options at the same time are not compatible") + } + return &opts, nil +} + // generic expire command for EXPIRE, PEXPIRE, EXPIREAT, PEXPIREAT // d is the time unit. If unix is set it'll be seen as a unixtimestamp and // converted to a duration. @@ -77,44 +116,10 @@ func makeCmdExpire(m *Miniredis, unix bool, d time.Duration) func(*server.Peer, return } - var opts struct { - key string - value int - nx bool - xx bool - gt bool - lt bool - } - opts.key = args[0] - if ok := optInt(c, args[1], &opts.value); !ok { - return - } - args = args[2:] - for len(args) > 0 { - switch strings.ToLower(args[0]) { - case "nx": - opts.nx = true - case "xx": - opts.xx = true - case "gt": - opts.gt = true - case "lt": - opts.lt = true - default: - setDirty(c) - c.WriteError(fmt.Sprintf("ERR Unsupported option %s", args[0])) - return - } - args = args[1:] - } - if opts.gt && opts.lt { - setDirty(c) - c.WriteError("ERR GT and LT options at the same time are not compatible") - return - } - if opts.nx && (opts.xx || opts.gt || opts.lt) { + opts, err := expireParse(cmd, args) + if err != nil { setDirty(c) - c.WriteError("ERR NX and XX, GT or LT options at the same time are not compatible") + c.WriteError(err.Error()) return } @@ -597,31 +602,19 @@ func (m *Miniredis) cmdRenamenx(c *server.Peer, cmd string, args []string) { }) } -// SCAN -func (m *Miniredis) cmdScan(c *server.Peer, cmd string, args []string) { - if len(args) < 1 { - setDirty(c) - c.WriteError(errWrongNumber(cmd)) - return - } - if !m.handleAuth(c) { - return - } - if m.checkPubsub(c, cmd) { - return - } - - var opts struct { - cursor int - count int - withMatch bool - match string - withType bool - _type string - } +type scanOpts struct { + cursor int + count int + withMatch bool + match string + withType bool + _type string +} - if ok := optIntErr(c, args[0], &opts.cursor, msgInvalidCursor); !ok { - return +func scanParse(cmd string, args []string) (*scanOpts, error) { + var opts scanOpts + if err := optIntSimple(args[0], &opts.cursor); err != nil { + return nil, errors.New(msgInvalidCursor) } args = args[1:] @@ -629,20 +622,14 @@ func (m *Miniredis) cmdScan(c *server.Peer, cmd string, args []string) { for len(args) > 0 { if strings.ToLower(args[0]) == "count" { if len(args) < 2 { - setDirty(c) - c.WriteError(msgSyntaxError) - return + return nil, errors.New(msgSyntaxError) } count, err := strconv.Atoi(args[1]) if err != nil || count < 0 { - setDirty(c) - c.WriteError(msgInvalidInt) - return + return nil, errors.New(msgInvalidInt) } if count == 0 { - setDirty(c) - c.WriteError(msgSyntaxError) - return + return nil, errors.New(msgSyntaxError) } opts.count = count args = args[2:] @@ -650,9 +637,7 @@ func (m *Miniredis) cmdScan(c *server.Peer, cmd string, args []string) { } if strings.ToLower(args[0]) == "match" { if len(args) < 2 { - setDirty(c) - c.WriteError(msgSyntaxError) - return + return nil, errors.New(msgSyntaxError) } opts.withMatch = true opts.match, args = args[1], args[2:] @@ -660,16 +645,35 @@ func (m *Miniredis) cmdScan(c *server.Peer, cmd string, args []string) { } if strings.ToLower(args[0]) == "type" { if len(args) < 2 { - setDirty(c) - c.WriteError(msgSyntaxError) - return + return nil, errors.New(msgSyntaxError) } opts.withType = true opts._type, args = strings.ToLower(args[1]), args[2:] continue } + return nil, errors.New(msgSyntaxError) + } + return &opts, nil +} + +// SCAN +func (m *Miniredis) cmdScan(c *server.Peer, cmd string, args []string) { + if len(args) < 1 { + setDirty(c) + c.WriteError(errWrongNumber(cmd)) + return + } + if !m.handleAuth(c) { + return + } + if m.checkPubsub(c, cmd) { + return + } + + opts, err := scanParse(cmd, args) + if err != nil { setDirty(c) - c.WriteError(msgSyntaxError) + c.WriteError(err.Error()) return } @@ -724,26 +728,15 @@ func (m *Miniredis) cmdScan(c *server.Peer, cmd string, args []string) { }) } -// COPY -func (m *Miniredis) cmdCopy(c *server.Peer, cmd string, args []string) { - if len(args) < 2 { - setDirty(c) - c.WriteError(errWrongNumber(cmd)) - return - } - if !m.handleAuth(c) { - return - } - if m.checkPubsub(c, cmd) { - return - } +type copyOpts struct { + from string + to string + destinationDB int + replace bool +} - var opts = struct { - from string - to string - destinationDB int - replace bool - }{ +func copyParse(cmd string, args []string) (*copyOpts, error) { + opts := copyOpts{ destinationDB: -1, } @@ -752,33 +745,45 @@ func (m *Miniredis) cmdCopy(c *server.Peer, cmd string, args []string) { switch strings.ToLower(args[0]) { case "db": if len(args) < 2 { - setDirty(c) - c.WriteError(msgSyntaxError) - return + return nil, errors.New(msgSyntaxError) } - db, err := strconv.Atoi(args[1]) - if err != nil { - setDirty(c) - c.WriteError(msgInvalidInt) - return + if err := optIntSimple(args[1], &opts.destinationDB); err != nil { + return nil, err } - if db < 0 { - setDirty(c) - c.WriteError(msgDBIndexOutOfRange) - return + if opts.destinationDB < 0 { + return nil, errors.New(msgDBIndexOutOfRange) } - opts.destinationDB = db args = args[2:] case "replace": opts.replace = true args = args[1:] default: - setDirty(c) - c.WriteError(msgSyntaxError) - return + return nil, errors.New(msgSyntaxError) } } + return &opts, nil +} +// COPY +func (m *Miniredis) cmdCopy(c *server.Peer, cmd string, args []string) { + if len(args) < 2 { + setDirty(c) + c.WriteError(errWrongNumber(cmd)) + return + } + if !m.handleAuth(c) { + return + } + if m.checkPubsub(c, cmd) { + return + } + + opts, err := copyParse(cmd, args) + if err != nil { + setDirty(c) + c.WriteError(err.Error()) + return + } withTx(m, c, func(c *server.Peer, ctx *connCtx) { fromDB, toDB := ctx.selectedDB, opts.destinationDB if toDB == -1 { diff --git a/vendor/github.com/alicebob/miniredis/v2/cmd_set.go b/vendor/github.com/alicebob/miniredis/v2/cmd_set.go index 2aaa7b0580..abcd1baa11 100644 --- a/vendor/github.com/alicebob/miniredis/v2/cmd_set.go +++ b/vendor/github.com/alicebob/miniredis/v2/cmd_set.go @@ -582,6 +582,10 @@ func (m *Miniredis) cmdSrandmember(c *server.Peer, cmd string, args []string) { db := m.db(ctx.selectedDB) if !db.exists(key) { + if withCount { + c.WriteLen(0) + return + } c.WriteNull() return } diff --git a/vendor/github.com/alicebob/miniredis/v2/cmd_stream.go b/vendor/github.com/alicebob/miniredis/v2/cmd_stream.go index a5a2f8b2f5..b6d00343bc 100644 --- a/vendor/github.com/alicebob/miniredis/v2/cmd_stream.go +++ b/vendor/github.com/alicebob/miniredis/v2/cmd_stream.go @@ -50,6 +50,11 @@ func (m *Miniredis) cmdXadd(c *server.Peer, cmd string, args []string) { withTx(m, c, func(c *server.Peer, ctx *connCtx) { maxlen := -1 minID := "" + makeStream := true + if strings.ToLower(args[0]) == "nomkstream" { + args = args[1:] + makeStream = false + } if strings.ToLower(args[0]) == "maxlen" { args = args[1:] // we don't treat "~" special @@ -101,7 +106,10 @@ func (m *Miniredis) cmdXadd(c *server.Peer, cmd string, args []string) { return } if s == nil { - // TODO: NOMKSTREAM + if !makeStream { + c.WriteNull() + return + } s, _ = db.newStream(key) } @@ -1665,7 +1673,7 @@ func (m *Miniredis) cmdXclaim(c *server.Peer, cmd string, args []string) { c.WriteError("ERR Invalid TIME option argument for XCLAIM") return } - opts.newLastDelivery = unixMilli(timeMs) + opts.newLastDelivery = time.UnixMilli(timeMs) args = args[2:] case "RETRYCOUNT": retryCount, err := strconv.Atoi(args[1]) @@ -1806,8 +1814,3 @@ func parseBlock(cmd string, args []string, block *bool, timeout *time.Duration) (*timeout) = time.Millisecond * time.Duration(ms) return nil } - -// taken from Go's time package. Can be dropped if miniredis supports >= 1.17 -func unixMilli(msec int64) time.Time { - return time.Unix(msec/1e3, (msec%1e3)*1e6) -} diff --git a/vendor/github.com/alicebob/miniredis/v2/db.go b/vendor/github.com/alicebob/miniredis/v2/db.go index 9227866cf2..af56839fab 100644 --- a/vendor/github.com/alicebob/miniredis/v2/db.go +++ b/vendor/github.com/alicebob/miniredis/v2/db.go @@ -3,6 +3,7 @@ package miniredis import ( "errors" "fmt" + "math" "math/big" "sort" "strconv" @@ -180,6 +181,17 @@ func (db *RedisDB) stringIncr(k string, delta int) (int, error) { return 0, ErrIntValueError } } + + if delta > 0 { + if math.MaxInt-delta < v { + return 0, ErrIntValueOverflowError + } + } else { + if math.MinInt-delta > v { + return 0, ErrIntValueOverflowError + } + } + v += delta db.stringSet(k, strconv.Itoa(v)) return v, nil diff --git a/vendor/github.com/alicebob/miniredis/v2/direct.go b/vendor/github.com/alicebob/miniredis/v2/direct.go index 1834aa5abe..31505eb462 100644 --- a/vendor/github.com/alicebob/miniredis/v2/direct.go +++ b/vendor/github.com/alicebob/miniredis/v2/direct.go @@ -21,6 +21,9 @@ var ( // ErrIntValueError can returned by INCRBY ErrIntValueError = errors.New(msgInvalidInt) + // ErrIntValueOverflowError can be returned by INCR, DECR, INCRBY, DECRBY + ErrIntValueOverflowError = errors.New(msgIntOverflow) + // ErrFloatValueError can returned by INCRBYFLOAT ErrFloatValueError = errors.New(msgInvalidFloat) ) diff --git a/vendor/github.com/alicebob/miniredis/v2/lua.go b/vendor/github.com/alicebob/miniredis/v2/lua.go index 7c7298cf24..ff777a45a0 100644 --- a/vendor/github.com/alicebob/miniredis/v2/lua.go +++ b/vendor/github.com/alicebob/miniredis/v2/lua.go @@ -161,6 +161,15 @@ func mkLua(srv *server.Server, c *server.Peer, sha string) (map[string]lua.LGFun // ignored return 1 }, + "set_repl": func(l *lua.LState) int { + top := l.GetTop() + if top != 1 { + l.Error(lua.LString("wrong number of arguments"), 1) + return 0 + } + // ignored + return 1 + }, }, luaRedisConstants } diff --git a/vendor/github.com/alicebob/miniredis/v2/miniredis.go b/vendor/github.com/alicebob/miniredis/v2/miniredis.go index 15dd08fefc..a9600ef621 100644 --- a/vendor/github.com/alicebob/miniredis/v2/miniredis.go +++ b/vendor/github.com/alicebob/miniredis/v2/miniredis.go @@ -25,6 +25,7 @@ import ( "sync" "time" + "github.com/alicebob/miniredis/v2/proto" "github.com/alicebob/miniredis/v2/server" ) @@ -135,6 +136,7 @@ func RunTLS(cfg *tls.Config) (*Miniredis, error) { type Tester interface { Fatalf(string, ...interface{}) Cleanup(func()) + Logf(format string, args ...interface{}) } // RunT start a new miniredis, pass it a testing.T. It also registers the cleanup after your test is done. @@ -148,6 +150,22 @@ func RunT(t Tester) *Miniredis { return m } +func runWithClient(t Tester) (*Miniredis, *proto.Client) { + m := RunT(t) + + c, err := proto.Dial(m.Addr()) + if err != nil { + t.Fatalf("could not connect to miniredis: %s", err) + } + t.Cleanup(func() { + if err = c.Close(); err != nil { + t.Logf("error closing connection to miniredis: %s", err) + } + }) + + return m, c +} + // Start starts a server. It listens on a random port on localhost. See also // Addr(). func (m *Miniredis) Start() error { @@ -177,6 +195,15 @@ func (m *Miniredis) StartAddr(addr string) error { return m.start(s) } +// StartAddrTLS runs miniredis with a given addr, TLS version. +func (m *Miniredis) StartAddrTLS(addr string, cfg *tls.Config) error { + s, err := server.NewServerTLS(addr, cfg) + if err != nil { + return err + } + return m.start(s) +} + func (m *Miniredis) start(s *server.Server) error { m.Lock() defer m.Unlock() diff --git a/vendor/github.com/alicebob/miniredis/v2/opts.go b/vendor/github.com/alicebob/miniredis/v2/opts.go index 666ace7f17..5b29c78c23 100644 --- a/vendor/github.com/alicebob/miniredis/v2/opts.go +++ b/vendor/github.com/alicebob/miniredis/v2/opts.go @@ -1,6 +1,7 @@ package miniredis import ( + "errors" "math" "strconv" "time" @@ -26,6 +27,16 @@ func optIntErr(c *server.Peer, src string, dest *int, errMsg string) bool { return true } +// optIntSimple sets dest or returns an error +func optIntSimple(src string, dest *int) error { + n, err := strconv.Atoi(src) + if err != nil { + return errors.New(msgInvalidInt) + } + *dest = n + return nil +} + func optDuration(c *server.Peer, src string, dest *time.Duration) bool { n, err := strconv.ParseFloat(src, 64) if err != nil { diff --git a/vendor/github.com/alicebob/miniredis/v2/proto/Makefile b/vendor/github.com/alicebob/miniredis/v2/proto/Makefile new file mode 100644 index 0000000000..b9ef39496b --- /dev/null +++ b/vendor/github.com/alicebob/miniredis/v2/proto/Makefile @@ -0,0 +1,2 @@ +test: + go test diff --git a/vendor/github.com/alicebob/miniredis/v2/proto/client.go b/vendor/github.com/alicebob/miniredis/v2/proto/client.go new file mode 100644 index 0000000000..92f57baf1e --- /dev/null +++ b/vendor/github.com/alicebob/miniredis/v2/proto/client.go @@ -0,0 +1,60 @@ +package proto + +import ( + "bufio" + "crypto/tls" + "net" +) + +type Client struct { + c net.Conn + r *bufio.Reader +} + +func Dial(addr string) (*Client, error) { + c, err := net.Dial("tcp", addr) + if err != nil { + return nil, err + } + + return &Client{ + c: c, + r: bufio.NewReader(c), + }, nil +} + +func DialTLS(addr string, cfg *tls.Config) (*Client, error) { + c, err := tls.Dial("tcp", addr, cfg) + if err != nil { + return nil, err + } + + return &Client{ + c: c, + r: bufio.NewReader(c), + }, nil +} + +func (c *Client) Close() error { + return c.c.Close() +} + +func (c *Client) Do(cmd ...string) (string, error) { + if err := Write(c.c, cmd); err != nil { + return "", err + } + return Read(c.r) +} + +func (c *Client) Read() (string, error) { + return Read(c.r) +} + +// Do() + ReadStrings() +func (c *Client) DoStrings(cmd ...string) ([]string, error) { + res, err := c.Do(cmd...) + if err != nil { + return nil, err + } + return ReadStrings(res) +} diff --git a/vendor/github.com/alicebob/miniredis/v2/proto/proto.go b/vendor/github.com/alicebob/miniredis/v2/proto/proto.go new file mode 100644 index 0000000000..e378faf187 --- /dev/null +++ b/vendor/github.com/alicebob/miniredis/v2/proto/proto.go @@ -0,0 +1,288 @@ +package proto + +import ( + "bufio" + "errors" + "fmt" + "io" + "strconv" + "strings" +) + +var ( + ErrProtocol = errors.New("unsupported protocol") + ErrUnexpected = errors.New("not what you asked for") +) + +func readLine(r *bufio.Reader) (string, error) { + line, err := r.ReadString('\n') + if err != nil { + return "", err + } + if len(line) < 3 { + return "", ErrProtocol + } + return line, nil +} + +// Read an array, with all elements are the raw redis commands +// Also reads sets and maps. +func ReadArray(b string) ([]string, error) { + r := bufio.NewReader(strings.NewReader(b)) + line, err := readLine(r) + if err != nil { + return nil, err + } + + elems := 0 + switch line[0] { + default: + return nil, ErrUnexpected + case '*', '>', '~': + // *: array + // >: push data + // ~: set + length, err := strconv.Atoi(line[1 : len(line)-2]) + if err != nil { + return nil, err + } + elems = length + case '%': + // we also read maps. + length, err := strconv.Atoi(line[1 : len(line)-2]) + if err != nil { + return nil, err + } + elems = length * 2 + } + + var res []string + for i := 0; i < elems; i++ { + next, err := Read(r) + if err != nil { + return nil, err + } + res = append(res, next) + } + return res, nil +} + +func ReadString(b string) (string, error) { + r := bufio.NewReader(strings.NewReader(b)) + line, err := readLine(r) + if err != nil { + return "", err + } + + switch line[0] { + default: + return "", ErrUnexpected + case '$': + // bulk strings are: `$5\r\nhello\r\n` + length, err := strconv.Atoi(line[1 : len(line)-2]) + if err != nil { + return "", err + } + if length < 0 { + // -1 is a nil response + return line, nil + } + var ( + buf = make([]byte, length+2) + pos = 0 + ) + for pos < length+2 { + n, err := r.Read(buf[pos:]) + if err != nil { + return "", err + } + pos += n + } + return string(buf[:len(buf)-2]), nil + } +} + +func readInline(b string) (string, error) { + if len(b) < 3 { + return "", ErrUnexpected + } + return b[1 : len(b)-2], nil +} + +func ReadError(b string) (string, error) { + if len(b) < 1 { + return "", ErrUnexpected + } + + switch b[0] { + default: + return "", ErrUnexpected + case '-': + return readInline(b) + } +} + +func ReadStrings(b string) ([]string, error) { + elems, err := ReadArray(b) + if err != nil { + return nil, err + } + var res []string + for _, e := range elems { + s, err := ReadString(e) + if err != nil { + return nil, err + } + res = append(res, s) + } + return res, nil +} + +// Read a single command, returning it raw. Used to read replies from redis. +// Understands RESP3 proto. +func Read(r *bufio.Reader) (string, error) { + line, err := readLine(r) + if err != nil { + return "", err + } + + switch line[0] { + default: + return "", ErrProtocol + case '+', '-', ':', ',', '_': + // +: inline string + // -: errors + // :: integer + // ,: float + // _: null + // Simple line based replies. + return line, nil + case '$': + // bulk strings are: `$5\r\nhello\r\n` + length, err := strconv.Atoi(line[1 : len(line)-2]) + if err != nil { + return "", err + } + if length < 0 { + // -1 is a nil response + return line, nil + } + var ( + buf = make([]byte, length+2) + pos = 0 + ) + for pos < length+2 { + n, err := r.Read(buf[pos:]) + if err != nil { + return "", err + } + pos += n + } + return line + string(buf), nil + case '*', '>', '~': + // arrays are: `*6\r\n...` + // pushdata is: `>6\r\n...` + // sets are: `~6\r\n...` + length, err := strconv.Atoi(line[1 : len(line)-2]) + if err != nil { + return "", err + } + for i := 0; i < length; i++ { + next, err := Read(r) + if err != nil { + return "", err + } + line += next + } + return line, nil + case '%': + // maps are: `%3\r\n...` + length, err := strconv.Atoi(line[1 : len(line)-2]) + if err != nil { + return "", err + } + for i := 0; i < length*2; i++ { + next, err := Read(r) + if err != nil { + return "", err + } + line += next + } + return line, nil + } +} + +// Write a command in RESP3 proto. Used to write commands to redis. +// Currently only supports string arrays. +func Write(w io.Writer, cmd []string) error { + if _, err := fmt.Fprintf(w, "*%d\r\n", len(cmd)); err != nil { + return err + } + for _, c := range cmd { + if _, err := fmt.Fprintf(w, "$%d\r\n%s\r\n", len(c), c); err != nil { + return err + } + } + return nil +} + +// Parse into interfaces. `b` must contain exactly a single command (which can be nested). +func Parse(b string) (interface{}, error) { + if len(b) < 1 { + return nil, ErrUnexpected + } + + switch b[0] { + default: + return "", ErrProtocol + case '+': + return readInline(b) + case '-': + e, err := readInline(b) + if err != nil { + return nil, err + } + return errors.New(e), nil + case ':': + e, err := readInline(b) + if err != nil { + return nil, err + } + return strconv.Atoi(e) + case '$': + return ReadString(b) + case '*': + elems, err := ReadArray(b) + if err != nil { + return nil, err + } + var res []interface{} + for _, elem := range elems { + e, err := Parse(elem) + if err != nil { + return nil, err + } + res = append(res, e) + } + return res, nil + case '%': + elems, err := ReadArray(b) + if err != nil { + return nil, err + } + var res = map[interface{}]interface{}{} + for len(elems) > 1 { + key, err := Parse(elems[0]) + if err != nil { + return nil, err + } + value, err := Parse(elems[1]) + if err != nil { + return nil, err + } + res[key] = value + elems = elems[2:] + } + return res, nil + } +} diff --git a/vendor/github.com/alicebob/miniredis/v2/proto/types.go b/vendor/github.com/alicebob/miniredis/v2/proto/types.go new file mode 100644 index 0000000000..0b3b7c9af2 --- /dev/null +++ b/vendor/github.com/alicebob/miniredis/v2/proto/types.go @@ -0,0 +1,102 @@ +package proto + +import ( + "fmt" + "strings" +) + +// Byte-safe string +func String(s string) string { + return fmt.Sprintf("$%d\r\n%s\r\n", len(s), s) +} + +// Inline string +func Inline(s string) string { + return inline('+', s) +} + +// Error +func Error(s string) string { + return inline('-', s) +} + +func inline(r rune, s string) string { + return fmt.Sprintf("%s%s\r\n", string(r), s) +} + +// Int +func Int(n int) string { + return fmt.Sprintf(":%d\r\n", n) +} + +// Float +func Float(n float64) string { + return fmt.Sprintf(",%g\r\n", n) +} + +const ( + Nil = "$-1\r\n" + NilResp3 = "_\r\n" + NilList = "*-1\r\n" +) + +// Array assembles the args in a list. Args should be raw redis commands. +// Example: Array(String("foo"), String("bar")) +func Array(args ...string) string { + return fmt.Sprintf("*%d\r\n", len(args)) + strings.Join(args, "") +} + +// Push assembles the args for push-data. Args should be raw redis commands. +// Example: Push(String("foo"), String("bar")) +func Push(args ...string) string { + return fmt.Sprintf(">%d\r\n", len(args)) + strings.Join(args, "") +} + +// Strings is a helper to build 1 dimensional string arrays. +func Strings(args ...string) string { + var strings []string + for _, a := range args { + strings = append(strings, String(a)) + } + return Array(strings...) +} + +// Ints is a helper to build 1 dimensional int arrays. +func Ints(args ...int) string { + var ints []string + for _, a := range args { + ints = append(ints, Int(a)) + } + return Array(ints...) +} + +// Map assembles the args in a map. Args should be raw redis commands. +// Must be an even number of arguments. +// Example: Map(String("foo"), String("bar")) +func Map(args ...string) string { + return fmt.Sprintf("%%%d\r\n", len(args)/2) + strings.Join(args, "") +} + +// StringMap is is a wrapper to get a map of (bulk)strings. +func StringMap(args ...string) string { + var strings []string + for _, a := range args { + strings = append(strings, String(a)) + } + return Map(strings...) +} + +// Set assembles the args in a map. Args should be raw redis commands. +// Example: Set(String("foo"), String("bar")) +func Set(args ...string) string { + return fmt.Sprintf("~%d\r\n", len(args)) + strings.Join(args, "") +} + +// StringSet is is a wrapper to get a set of (bulk)strings. +func StringSet(args ...string) string { + var strings []string + for _, a := range args { + strings = append(strings, String(a)) + } + return Set(strings...) +} diff --git a/vendor/github.com/alicebob/miniredis/v2/redis.go b/vendor/github.com/alicebob/miniredis/v2/redis.go index 5a21fe7ba8..f70728e7c7 100644 --- a/vendor/github.com/alicebob/miniredis/v2/redis.go +++ b/vendor/github.com/alicebob/miniredis/v2/redis.go @@ -16,6 +16,7 @@ const ( msgWrongType = "WRONGTYPE Operation against a key holding the wrong kind of value" msgNotValidHllValue = "WRONGTYPE Key is not a valid HyperLogLog string value." msgInvalidInt = "ERR value is not an integer or out of range" + msgIntOverflow = "ERR increment or decrement would overflow" msgInvalidFloat = "ERR value is not a valid float" msgInvalidMinMax = "ERR min or max is not a float" msgInvalidRangeItem = "ERR min or max not valid string range item" diff --git a/vendor/modules.txt b/vendor/modules.txt index 16fe5ec6f8..d470bc0ee3 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -125,13 +125,14 @@ github.com/alecthomas/units # github.com/alicebob/gopher-json v0.0.0-20200520072559-a9ecdc9d1d3a ## explicit github.com/alicebob/gopher-json -# github.com/alicebob/miniredis/v2 v2.32.1 -## explicit; go 1.14 +# github.com/alicebob/miniredis/v2 v2.33.0 +## explicit; go 1.17 github.com/alicebob/miniredis/v2 github.com/alicebob/miniredis/v2/fpconv github.com/alicebob/miniredis/v2/geohash github.com/alicebob/miniredis/v2/hyperloglog github.com/alicebob/miniredis/v2/metro +github.com/alicebob/miniredis/v2/proto github.com/alicebob/miniredis/v2/server github.com/alicebob/miniredis/v2/size # github.com/armon/go-metrics v0.4.1 From 4fa5fe0b11934feb47d6f3f882959c45fe1498e1 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 30 Jul 2024 10:11:31 -0700 Subject: [PATCH 20/32] Bump github.com/aws/aws-sdk-go from 1.54.19 to 1.55.3 (#6121) Bumps [github.com/aws/aws-sdk-go](https://github.com/aws/aws-sdk-go) from 1.54.19 to 1.55.3. - [Release notes](https://github.com/aws/aws-sdk-go/releases) - [Commits](https://github.com/aws/aws-sdk-go/compare/v1.54.19...v1.55.3) --- updated-dependencies: - dependency-name: github.com/aws/aws-sdk-go 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 +- .../aws/aws-sdk-go/aws/endpoints/defaults.go | 82 +++++++++++++++---- .../aws/aws-sdk-go/aws/request/waiter.go | 13 ++- .../github.com/aws/aws-sdk-go/aws/version.go | 2 +- .../aws/aws-sdk-go/service/dynamodb/api.go | 15 ++-- .../aws/aws-sdk-go/service/sns/api.go | 2 +- vendor/modules.txt | 2 +- 8 files changed, 93 insertions(+), 29 deletions(-) diff --git a/go.mod b/go.mod index 8ea64be567..63ab592e82 100644 --- a/go.mod +++ b/go.mod @@ -9,7 +9,7 @@ require ( github.com/alecthomas/units v0.0.0-20231202071711-9a357b53e9c9 github.com/alicebob/miniredis/v2 v2.33.0 github.com/armon/go-metrics v0.4.1 - github.com/aws/aws-sdk-go v1.54.19 + github.com/aws/aws-sdk-go v1.55.3 github.com/bradfitz/gomemcache v0.0.0-20230905024940-24af94b03874 github.com/cespare/xxhash v1.1.0 github.com/cortexproject/promqlsmith v0.0.0-20240506042652-6cfdd9739a5e diff --git a/go.sum b/go.sum index bf2b36ab02..16e5a55ec6 100644 --- a/go.sum +++ b/go.sum @@ -601,8 +601,8 @@ github.com/asaskevich/govalidator v0.0.0-20230301143203-a9d515a09cc2 h1:DklsrG3d github.com/asaskevich/govalidator v0.0.0-20230301143203-a9d515a09cc2/go.mod h1:WaHUgvxTVq04UNunO+XhnAqY/wQc+bxr74GqbsZ/Jqw= github.com/aws/aws-sdk-go v1.27.0/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN924inxo= github.com/aws/aws-sdk-go v1.38.35/go.mod h1:hcU610XS61/+aQV88ixoOzUoG7v3b31pl2zKMmprdro= -github.com/aws/aws-sdk-go v1.54.19 h1:tyWV+07jagrNiCcGRzRhdtVjQs7Vy41NwsuOcl0IbVI= -github.com/aws/aws-sdk-go v1.54.19/go.mod h1:eRwEWoyTWFMVYVQzKMNHWP5/RV4xIUGMQfXQHfHkpNU= +github.com/aws/aws-sdk-go v1.55.3 h1:0B5hOX+mIx7I5XPOrjrHlKSDQV/+ypFZpIHOx5LOk3E= +github.com/aws/aws-sdk-go v1.55.3/go.mod h1:eRwEWoyTWFMVYVQzKMNHWP5/RV4xIUGMQfXQHfHkpNU= github.com/aws/aws-sdk-go-v2 v1.16.0/go.mod h1:lJYcuZZEHWNIb6ugJjbQY1fykdoobWbOS7kJYb4APoI= github.com/aws/aws-sdk-go-v2 v1.16.16 h1:M1fj4FE2lB4NzRb9Y0xdWsn2P0+2UHVxwKyOa4YJNjk= github.com/aws/aws-sdk-go-v2 v1.16.16/go.mod h1:SwiyXi/1zTUZ6KIAmLK5V5ll8SiURNUYOqTerZPaF9k= diff --git a/vendor/github.com/aws/aws-sdk-go/aws/endpoints/defaults.go b/vendor/github.com/aws/aws-sdk-go/aws/endpoints/defaults.go index 84dc7dc08e..069debf1f5 100644 --- a/vendor/github.com/aws/aws-sdk-go/aws/endpoints/defaults.go +++ b/vendor/github.com/aws/aws-sdk-go/aws/endpoints/defaults.go @@ -13068,6 +13068,9 @@ var awsPartition = partition{ endpointKey{ Region: "eu-central-1", }: endpoint{}, + endpointKey{ + Region: "eu-central-2", + }: endpoint{}, endpointKey{ Region: "eu-north-1", }: endpoint{}, @@ -22522,6 +22525,9 @@ var awsPartition = partition{ }: endpoint{ Hostname: "network-firewall-fips.ca-central-1.amazonaws.com", }, + endpointKey{ + Region: "ca-west-1", + }: endpoint{}, endpointKey{ Region: "eu-central-1", }: endpoint{}, @@ -24479,6 +24485,14 @@ var awsPartition = partition{ Region: "ca-central-1", }, }, + endpointKey{ + Region: "ca-west-1", + }: endpoint{ + Hostname: "portal.sso.ca-west-1.amazonaws.com", + CredentialScope: credentialScope{ + Region: "ca-west-1", + }, + }, endpointKey{ Region: "eu-central-1", }: endpoint{ @@ -33621,6 +33635,20 @@ var awsPartition = partition{ }: endpoint{}, }, }, + "tax": service{ + PartitionEndpoint: "aws-global", + IsRegionalized: boxedFalse, + Endpoints: serviceEndpoints{ + endpointKey{ + Region: "aws-global", + }: endpoint{ + Hostname: "tax.us-east-1.amazonaws.com", + CredentialScope: credentialScope{ + Region: "us-east-1", + }, + }, + }, + }, "textract": service{ Endpoints: serviceEndpoints{ endpointKey{ @@ -39973,16 +40001,12 @@ var awsusgovPartition = partition{ Endpoints: serviceEndpoints{ endpointKey{ Region: "us-gov-east-1", - }: endpoint{ - Hostname: "autoscaling-plans.us-gov-east-1.amazonaws.com", - Protocols: []string{"http", "https"}, - }, + }: endpoint{}, endpointKey{ Region: "us-gov-east-1", Variant: fipsVariant, }: endpoint{ - Hostname: "autoscaling-plans.us-gov-east-1.amazonaws.com", - Protocols: []string{"http", "https"}, + Hostname: "autoscaling-plans.us-gov-east-1.amazonaws.com", }, endpointKey{ Region: "us-gov-east-1-fips", @@ -39994,16 +40018,12 @@ var awsusgovPartition = partition{ }, endpointKey{ Region: "us-gov-west-1", - }: endpoint{ - Hostname: "autoscaling-plans.us-gov-west-1.amazonaws.com", - Protocols: []string{"http", "https"}, - }, + }: endpoint{}, endpointKey{ Region: "us-gov-west-1", Variant: fipsVariant, }: endpoint{ - Hostname: "autoscaling-plans.us-gov-west-1.amazonaws.com", - Protocols: []string{"http", "https"}, + Hostname: "autoscaling-plans.us-gov-west-1.amazonaws.com", }, endpointKey{ Region: "us-gov-west-1-fips", @@ -40969,20 +40989,40 @@ var awsusgovPartition = partition{ "directconnect": service{ Endpoints: serviceEndpoints{ endpointKey{ - Region: "us-gov-east-1", + Region: "fips-us-gov-east-1", }: endpoint{ - Hostname: "directconnect.us-gov-east-1.amazonaws.com", + Hostname: "directconnect-fips.us-gov-east-1.amazonaws.com", CredentialScope: credentialScope{ Region: "us-gov-east-1", }, + Deprecated: boxedTrue, }, endpointKey{ - Region: "us-gov-west-1", + Region: "fips-us-gov-west-1", }: endpoint{ - Hostname: "directconnect.us-gov-west-1.amazonaws.com", + Hostname: "directconnect-fips.us-gov-west-1.amazonaws.com", CredentialScope: credentialScope{ Region: "us-gov-west-1", }, + Deprecated: boxedTrue, + }, + endpointKey{ + Region: "us-gov-east-1", + }: endpoint{}, + endpointKey{ + Region: "us-gov-east-1", + Variant: fipsVariant, + }: endpoint{ + Hostname: "directconnect-fips.us-gov-east-1.amazonaws.com", + }, + endpointKey{ + Region: "us-gov-west-1", + }: endpoint{}, + endpointKey{ + Region: "us-gov-west-1", + Variant: fipsVariant, + }: endpoint{ + Hostname: "directconnect-fips.us-gov-west-1.amazonaws.com", }, }, }, @@ -46372,6 +46412,9 @@ var awsisoPartition = partition{ endpointKey{ Region: "us-iso-east-1", }: endpoint{}, + endpointKey{ + Region: "us-iso-west-1", + }: endpoint{}, }, }, "appconfig": service{ @@ -47685,6 +47728,13 @@ var awsisobPartition = partition{ }: endpoint{}, }, }, + "apigateway": service{ + Endpoints: serviceEndpoints{ + endpointKey{ + Region: "us-isob-east-1", + }: endpoint{}, + }, + }, "appconfig": service{ Endpoints: serviceEndpoints{ endpointKey{ diff --git a/vendor/github.com/aws/aws-sdk-go/aws/request/waiter.go b/vendor/github.com/aws/aws-sdk-go/aws/request/waiter.go index 4601f883cc..992ed0464b 100644 --- a/vendor/github.com/aws/aws-sdk-go/aws/request/waiter.go +++ b/vendor/github.com/aws/aws-sdk-go/aws/request/waiter.go @@ -256,8 +256,17 @@ func (a *WaiterAcceptor) match(name string, l aws.Logger, req *Request, err erro s := a.Expected.(int) result = s == req.HTTPResponse.StatusCode case ErrorWaiterMatch: - if aerr, ok := err.(awserr.Error); ok { - result = aerr.Code() == a.Expected.(string) + switch ex := a.Expected.(type) { + case string: + if aerr, ok := err.(awserr.Error); ok { + result = aerr.Code() == ex + } + case bool: + if ex { + result = err != nil + } else { + result = err == nil + } } default: waiterLogf(l, "WARNING: Waiter %s encountered unexpected matcher: %s", diff --git a/vendor/github.com/aws/aws-sdk-go/aws/version.go b/vendor/github.com/aws/aws-sdk-go/aws/version.go index b2040b05e5..514bf3ade2 100644 --- a/vendor/github.com/aws/aws-sdk-go/aws/version.go +++ b/vendor/github.com/aws/aws-sdk-go/aws/version.go @@ -5,4 +5,4 @@ package aws const SDKName = "aws-sdk-go" // SDKVersion is the version of this SDK -const SDKVersion = "1.54.19" +const SDKVersion = "1.55.3" diff --git a/vendor/github.com/aws/aws-sdk-go/service/dynamodb/api.go b/vendor/github.com/aws/aws-sdk-go/service/dynamodb/api.go index 7286d56768..5cae6505d7 100644 --- a/vendor/github.com/aws/aws-sdk-go/service/dynamodb/api.go +++ b/vendor/github.com/aws/aws-sdk-go/service/dynamodb/api.go @@ -62,7 +62,8 @@ func (c *DynamoDB) BatchExecuteStatementRequest(input *BatchExecuteStatementInpu // This operation allows you to perform batch reads or writes on data stored // in DynamoDB, using PartiQL. Each read statement in a BatchExecuteStatement // must specify an equality condition on all key attributes. This enforces that -// each SELECT statement in a batch returns at most a single item. +// each SELECT statement in a batch returns at most a single item. For more +// information, see Running batch operations with PartiQL for DynamoDB (https://docs.aws.amazon.com/amazondynamodb/latest/developerguide/ql-reference.multiplestatements.batching.html). // // The entire batch must consist of either read statements or write statements, // you cannot mix both in one batch. @@ -429,8 +430,12 @@ func (c *DynamoDB) BatchWriteItemRequest(input *BatchWriteItemInput) (req *reque // check for unprocessed items and submit a new BatchWriteItem request with // those unprocessed items until all items have been processed. // -// If none of the items can be processed due to insufficient provisioned throughput -// on all of the tables in the request, then BatchWriteItem returns a ProvisionedThroughputExceededException. +// For tables and indexes with provisioned capacity, if none of the items can +// be processed due to insufficient provisioned throughput on all of the tables +// in the request, then BatchWriteItem returns a ProvisionedThroughputExceededException. +// For all tables and indexes, if none of the items can be processed due to +// other throttling scenarios (such as exceeding partition level limits), then +// BatchWriteItem returns a ThrottlingException. // // If DynamoDB returns any unprocessed items, you should retry the batch operation // on those items. However, we strongly recommend that you use an exponential @@ -1564,7 +1569,7 @@ func (c *DynamoDB) DeleteTableRequest(input *DeleteTableInput) (req *request.Req // // DynamoDB might continue to accept data read and write operations, such as // GetItem and PutItem, on a table in the DELETING state until the table deletion -// is complete. +// is complete. For the full list of table states, see TableStatus (https://docs.aws.amazon.com/amazondynamodb/latest/APIReference/API_TableDescription.html#DDB-Type-TableDescription-TableStatus). // // When you delete a table, any indexes on that table are also deleted. // @@ -21427,7 +21432,7 @@ type QueryOutput struct { // The number of items evaluated, before any QueryFilter is applied. A high // ScannedCount value with few, or no, Count results indicates an inefficient - // Query operation. For more information, see Count and ScannedCount (https://docs.aws.amazon.com/amazondynamodb/latest/developerguide/QueryAndScan.html#Count) + // Query operation. For more information, see Count and ScannedCount (https://docs.aws.amazon.com/amazondynamodb/latest/developerguide/Scan.html#Scan.Count) // in the Amazon DynamoDB Developer Guide. // // If you did not use a filter in the request, then ScannedCount is the same diff --git a/vendor/github.com/aws/aws-sdk-go/service/sns/api.go b/vendor/github.com/aws/aws-sdk-go/service/sns/api.go index 8db93d6e9c..7683b15002 100644 --- a/vendor/github.com/aws/aws-sdk-go/service/sns/api.go +++ b/vendor/github.com/aws/aws-sdk-go/service/sns/api.go @@ -6826,7 +6826,7 @@ type ListPhoneNumbersOptedOutOutput struct { // A list of phone numbers that are opted out of receiving SMS messages. The // list is paginated, and each page can contain up to 100 phone numbers. - PhoneNumbers []*string `locationName:"phoneNumbers" type:"list"` + PhoneNumbers []*string `locationName:"phoneNumbers" type:"list" sensitive:"true"` } // String returns the string representation. diff --git a/vendor/modules.txt b/vendor/modules.txt index d470bc0ee3..d773525454 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -142,7 +142,7 @@ github.com/armon/go-metrics/prometheus # github.com/asaskevich/govalidator v0.0.0-20230301143203-a9d515a09cc2 ## explicit; go 1.13 github.com/asaskevich/govalidator -# github.com/aws/aws-sdk-go v1.54.19 +# github.com/aws/aws-sdk-go v1.55.3 ## explicit; go 1.19 github.com/aws/aws-sdk-go/aws github.com/aws/aws-sdk-go/aws/auth/bearer From e4eba4a5889503f40b3dcdf62e87caf582f6fe03 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 30 Jul 2024 10:11:49 -0700 Subject: [PATCH 21/32] Bump github.com/hashicorp/consul/api from 1.29.1 to 1.29.2 (#6120) Bumps [github.com/hashicorp/consul/api](https://github.com/hashicorp/consul) from 1.29.1 to 1.29.2. - [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.29.1...api/v1.29.2) --- updated-dependencies: - dependency-name: github.com/hashicorp/consul/api 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 | 8 ++++---- vendor/github.com/hashicorp/consul/api/api.go | 7 +++++++ vendor/modules.txt | 2 +- 4 files changed, 13 insertions(+), 6 deletions(-) diff --git a/go.mod b/go.mod index 63ab592e82..25facc7ba6 100644 --- a/go.mod +++ b/go.mod @@ -29,7 +29,7 @@ require ( github.com/gorilla/mux v1.8.1 github.com/grafana/regexp v0.0.0-20240518133315-a468a5bfb3bc github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 - github.com/hashicorp/consul/api v1.29.1 + github.com/hashicorp/consul/api v1.29.2 github.com/hashicorp/go-cleanhttp v0.5.2 github.com/hashicorp/go-sockaddr v1.0.6 github.com/hashicorp/memberlist v0.5.1 diff --git a/go.sum b/go.sum index 16e5a55ec6..e104370fd4 100644 --- a/go.sum +++ b/go.sum @@ -1025,10 +1025,10 @@ github.com/grpc-ecosystem/grpc-gateway/v2 v2.20.0 h1:bkypFPDjIYGfCYD5mRBvpqxfYX1 github.com/grpc-ecosystem/grpc-gateway/v2 v2.20.0/go.mod h1:P+Lt/0by1T8bfcF3z737NnSbmxQAppXMRziHUxPOC8k= github.com/grpc-ecosystem/grpc-opentracing v0.0.0-20180507213350-8e809c8a8645/go.mod h1:6iZfnjpejD4L/4DwD7NryNaJyCQdzwWwH2MWhCA90Kw= github.com/hashicorp/consul/api v1.12.0/go.mod h1:6pVBMo0ebnYdt2S3H87XhekM/HHrUoTD2XXb/VrZVy0= -github.com/hashicorp/consul/api v1.29.1 h1:UEwOjYJrd3lG1x5w7HxDRMGiAUPrb3f103EoeKuuEcc= -github.com/hashicorp/consul/api v1.29.1/go.mod h1:lumfRkY/coLuqMICkI7Fh3ylMG31mQSRZyef2c5YvJI= -github.com/hashicorp/consul/proto-public v0.6.1 h1:+uzH3olCrksXYWAYHKqK782CtK9scfqH+Unlw3UHhCg= -github.com/hashicorp/consul/proto-public v0.6.1/go.mod h1:cXXbOg74KBNGajC+o8RlA502Esf0R9prcoJgiOX/2Tg= +github.com/hashicorp/consul/api v1.29.2 h1:aYyRn8EdE2mSfG14S1+L9Qkjtz8RzmaWh6AcNGRNwPw= +github.com/hashicorp/consul/api v1.29.2/go.mod h1:0YObcaLNDSbtlgzIRtmRXI1ZkeuK0trCBxwZQ4MYnIk= +github.com/hashicorp/consul/proto-public v0.6.2 h1:+DA/3g/IiKlJZb88NBn0ZgXrxJp2NlvCZdEyl+qxvL0= +github.com/hashicorp/consul/proto-public v0.6.2/go.mod h1:cXXbOg74KBNGajC+o8RlA502Esf0R9prcoJgiOX/2Tg= github.com/hashicorp/consul/sdk v0.8.0/go.mod h1:GBvyrGALthsZObzUGsfgHZQDXjg4lOjagTIwIR1vPms= github.com/hashicorp/consul/sdk v0.16.1 h1:V8TxTnImoPD5cj0U9Spl0TUxcytjcbbJeADFF07KdHg= github.com/hashicorp/consul/sdk v0.16.1/go.mod h1:fSXvwxB2hmh1FMZCNl6PwX0Q/1wdWtHJcZ7Ea5tns0s= diff --git a/vendor/github.com/hashicorp/consul/api/api.go b/vendor/github.com/hashicorp/consul/api/api.go index b90a45d92b..d4d853d5d4 100644 --- a/vendor/github.com/hashicorp/consul/api/api.go +++ b/vendor/github.com/hashicorp/consul/api/api.go @@ -10,6 +10,7 @@ import ( "encoding/json" "fmt" "io" + "math" "net" "net/http" "net/url" @@ -1181,6 +1182,9 @@ func parseQueryMeta(resp *http.Response, q *QueryMeta) error { if err != nil { return fmt.Errorf("Failed to parse X-Consul-LastContact: %v", err) } + if last > math.MaxInt64 { + return fmt.Errorf("X-Consul-LastContact Header value is out of range: %d", last) + } q.LastContact = time.Duration(last) * time.Millisecond // Parse the X-Consul-KnownLeader @@ -1222,6 +1226,9 @@ func parseQueryMeta(resp *http.Response, q *QueryMeta) error { if err != nil { return fmt.Errorf("Failed to parse Age Header: %v", err) } + if age > math.MaxInt64 { + return fmt.Errorf("Age Header value is out of range: %d", last) + } q.CacheAge = time.Duration(age) * time.Second } diff --git a/vendor/modules.txt b/vendor/modules.txt index d773525454..c8a803765b 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -547,7 +547,7 @@ github.com/grpc-ecosystem/go-grpc-middleware/v2 github.com/grpc-ecosystem/grpc-gateway/v2/internal/httprule github.com/grpc-ecosystem/grpc-gateway/v2/runtime github.com/grpc-ecosystem/grpc-gateway/v2/utilities -# github.com/hashicorp/consul/api v1.29.1 +# github.com/hashicorp/consul/api v1.29.2 ## explicit; go 1.19 github.com/hashicorp/consul/api # github.com/hashicorp/errwrap v1.1.0 From 8ce3642d6f5bc9900bb5d439ee43dd431b6d46b6 Mon Sep 17 00:00:00 2001 From: Anand Rajagopal Date: Tue, 30 Jul 2024 12:57:01 -0500 Subject: [PATCH 22/32] Add support for exclude_alerts flag in ListRules API (#6011) --- CHANGELOG.md | 1 + integration/e2ecortex/client.go | 2 + integration/ruler_test.go | 42 ++++++++- pkg/ruler/api.go | 21 +++++ pkg/ruler/ruler.go | 29 ++++--- pkg/ruler/ruler.pb.go | 149 +++++++++++++++++++++----------- pkg/ruler/ruler.proto | 1 + 7 files changed, 179 insertions(+), 66 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index fb01005e0d..10d2ccbee1 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -6,6 +6,7 @@ * [CHANGE] Server: Instrument `cortex_request_duration_seconds` metric with native histogram. If `native-histograms` feature is enabled in monitoring Prometheus then the metric name needs to be updated in your dashboards. #6056 * [CHANGE] Distributor/Ingester: Change `cortex_distributor_ingester_appends_total`, `cortex_distributor_ingester_append_failures_total`, `cortex_distributor_ingester_queries_total`, and `cortex_distributor_ingester_query_failures_total` metrics to use the ingester ID instead of its IP as the label value. #6078 * [FEATURE] Ingester: Experimental: Enable native histogram ingestion via `-blocks-storage.tsdb.enable-native-histograms` flag. #5986 +* [FEATURE] Ruler: Add support for filtering out alerts in ListRules API. #6011 * [FEATURE] Query Frontend: Added a query rejection mechanism to block resource-intensive queries. #6005 * [FEATURE] OTLP: Support ingesting OTLP exponential metrics as native histograms. #6071 * [FEATURE] Ingester: Add `ingester.instance-limits.max-inflight-query-requests` to allow limiting ingester concurrent queries. #6081 diff --git a/integration/e2ecortex/client.go b/integration/e2ecortex/client.go index dbc626d6a5..5f45d6d59f 100644 --- a/integration/e2ecortex/client.go +++ b/integration/e2ecortex/client.go @@ -526,6 +526,7 @@ type RuleFilter struct { RuleGroupNames []string RuleNames []string RuleType string + ExcludeAlerts string } func addQueryParams(urlValues url.Values, paramName string, params ...string) { @@ -551,6 +552,7 @@ func (c *Client) GetPrometheusRules(filter RuleFilter) ([]*ruler.RuleGroup, erro addQueryParams(urlValues, "rule_name[]", filter.RuleNames...) addQueryParams(urlValues, "rule_group[]", filter.RuleGroupNames...) addQueryParams(urlValues, "type", filter.RuleType) + addQueryParams(urlValues, "exclude_alerts", filter.ExcludeAlerts) req.URL.RawQuery = urlValues.Encode() ctx, cancel := context.WithTimeout(context.Background(), c.timeout) diff --git a/integration/ruler_test.go b/integration/ruler_test.go index 37c086fe23..2da8db2e01 100644 --- a/integration/ruler_test.go +++ b/integration/ruler_test.go @@ -414,6 +414,7 @@ func testRulerAPIWithSharding(t *testing.T, enableRulesBackup bool) { ruleGroups := make([]rulefmt.RuleGroup, numRulesGroups) expectedNames := make([]string, numRulesGroups) alertCount := 0 + evalInterval, _ := model.ParseDuration("1s") for i := 0; i < numRulesGroups; i++ { num := random.Intn(100) var ruleNode yaml.Node @@ -428,7 +429,7 @@ func testRulerAPIWithSharding(t *testing.T, enableRulesBackup bool) { alertCount++ ruleGroups[i] = rulefmt.RuleGroup{ Name: ruleName, - Interval: 60, + Interval: evalInterval, Rules: []rulefmt.RuleNode{{ Alert: ruleNode, Expr: exprNode, @@ -437,7 +438,7 @@ func testRulerAPIWithSharding(t *testing.T, enableRulesBackup bool) { } else { ruleGroups[i] = rulefmt.RuleGroup{ Name: ruleName, - Interval: 60, + Interval: evalInterval, Rules: []rulefmt.RuleNode{{ Record: ruleNode, Expr: exprNode, @@ -458,6 +459,7 @@ func testRulerAPIWithSharding(t *testing.T, enableRulesBackup bool) { "-querier.store-gateway-addresses": "localhost:12345", // Enable the bucket index so we can skip the initial bucket scan. "-blocks-storage.bucket-store.bucket-index.enabled": "true", + "-ruler.poll-interval": "5s", } if enableRulesBackup { overrides["-ruler.ring.replication-factor"] = "3" @@ -553,6 +555,42 @@ func testRulerAPIWithSharding(t *testing.T, enableRulesBackup bool) { assert.Len(t, ruleNames, 3, "Expected %d rules but got %d", 3, len(ruleNames)) }, }, + "Exclude Alerts": { + filter: e2ecortex.RuleFilter{ + ExcludeAlerts: "true", + }, + resultCheckFn: func(t assert.TestingT, ruleGroups []*ruler.RuleGroup) { + alertsCount := 0 + for _, ruleGroup := range ruleGroups { + for _, rule := range ruleGroup.Rules { + r := rule.(map[string]interface{}) + if v, OK := r["alerts"]; OK { + alerts := v.([]interface{}) + alertsCount = alertsCount + len(alerts) + } + } + } + assert.Equal(t, 0, alertsCount, "Expected 0 alerts but got %d", alertsCount) + }, + }, + "Include Alerts": { + filter: e2ecortex.RuleFilter{ + ExcludeAlerts: "false", + }, + resultCheckFn: func(t assert.TestingT, ruleGroups []*ruler.RuleGroup) { + alertsCount := 0 + for _, ruleGroup := range ruleGroups { + for _, rule := range ruleGroup.Rules { + r := rule.(map[string]interface{}) + if v, OK := r["alerts"]; OK { + alerts := v.([]interface{}) + alertsCount = alertsCount + len(alerts) + } + } + } + assert.Greater(t, alertsCount, 0, "Expected greater than 0 alerts but got %d", alertsCount) + }, + }, } // For each test case, fetch the rules with configured filters, and ensure the results match. if enableRulesBackup { diff --git a/pkg/ruler/api.go b/pkg/ruler/api.go index 6c1a868966..0294a78c61 100644 --- a/pkg/ruler/api.go +++ b/pkg/ruler/api.go @@ -154,6 +154,12 @@ func (a *API) PrometheusRules(w http.ResponseWriter, req *http.Request) { return } + excludeAlerts, err := parseExcludeAlerts(req) + if err != nil { + util_api.RespondError(logger, w, v1.ErrBadData, fmt.Sprintf("invalid parameter %q: %s", "exclude_alerts", err.Error()), http.StatusBadRequest) + return + } + rulesRequest := RulesRequest{ RuleNames: req.Form["rule_name[]"], RuleGroupNames: req.Form["rule_group[]"], @@ -162,6 +168,7 @@ func (a *API) PrometheusRules(w http.ResponseWriter, req *http.Request) { State: state, Health: health, Matchers: req.Form["match[]"], + ExcludeAlerts: excludeAlerts, } w.Header().Set("Content-Type", "application/json") @@ -256,6 +263,20 @@ func (a *API) PrometheusRules(w http.ResponseWriter, req *http.Request) { } } +func parseExcludeAlerts(r *http.Request) (bool, error) { + excludeAlertsParam := strings.ToLower(r.URL.Query().Get("exclude_alerts")) + + if excludeAlertsParam == "" { + return false, nil + } + + excludeAlerts, err := strconv.ParseBool(excludeAlertsParam) + if err != nil { + return false, fmt.Errorf("error converting exclude_alerts: %w", err) + } + return excludeAlerts, nil +} + func (a *API) PrometheusAlerts(w http.ResponseWriter, req *http.Request) { logger := util_log.WithContext(req.Context(), a.logger) userID, err := tenant.TenantID(req.Context()) diff --git a/pkg/ruler/ruler.go b/pkg/ruler/ruler.go index f37bc3ce93..dfacb61d80 100644 --- a/pkg/ruler/ruler.go +++ b/pkg/ruler/ruler.go @@ -950,19 +950,21 @@ func (r *Ruler) getLocalRules(userID string, rulesRequest RulesRequest, includeB continue } alerts := []*AlertStateDesc{} - for _, a := range rule.ActiveAlerts() { - alerts = append(alerts, &AlertStateDesc{ - State: a.State.String(), - Labels: cortexpb.FromLabelsToLabelAdapters(a.Labels), - Annotations: cortexpb.FromLabelsToLabelAdapters(a.Annotations), - Value: a.Value, - ActiveAt: a.ActiveAt, - FiredAt: a.FiredAt, - ResolvedAt: a.ResolvedAt, - LastSentAt: a.LastSentAt, - ValidUntil: a.ValidUntil, - KeepFiringSince: a.KeepFiringSince, - }) + if !rulesRequest.ExcludeAlerts { + for _, a := range rule.ActiveAlerts() { + alerts = append(alerts, &AlertStateDesc{ + State: a.State.String(), + Labels: cortexpb.FromLabelsToLabelAdapters(a.Labels), + Annotations: cortexpb.FromLabelsToLabelAdapters(a.Annotations), + Value: a.Value, + ActiveAt: a.ActiveAt, + FiredAt: a.FiredAt, + ResolvedAt: a.ResolvedAt, + LastSentAt: a.LastSentAt, + ValidUntil: a.ValidUntil, + KeepFiringSince: a.KeepFiringSince, + }) + } } ruleDesc = &RuleStateDesc{ Rule: &rulespb.RuleDesc{ @@ -1174,6 +1176,7 @@ func (r *Ruler) getShardedRules(ctx context.Context, userID string, rulesRequest RuleGroupNames: rulesRequest.GetRuleGroupNames(), Files: rulesRequest.GetFiles(), Type: rulesRequest.GetType(), + ExcludeAlerts: rulesRequest.GetExcludeAlerts(), Matchers: rulesRequest.GetMatchers(), }) diff --git a/pkg/ruler/ruler.pb.go b/pkg/ruler/ruler.pb.go index 213c8ee827..5eb36c7a6a 100644 --- a/pkg/ruler/ruler.pb.go +++ b/pkg/ruler/ruler.pb.go @@ -46,6 +46,7 @@ type RulesRequest struct { State string `protobuf:"bytes,5,opt,name=state,proto3" json:"state,omitempty"` Health string `protobuf:"bytes,6,opt,name=health,proto3" json:"health,omitempty"` Matchers []string `protobuf:"bytes,7,rep,name=matchers,proto3" json:"matchers,omitempty"` + ExcludeAlerts bool `protobuf:"varint,8,opt,name=excludeAlerts,proto3" json:"excludeAlerts,omitempty"` } func (m *RulesRequest) Reset() { *m = RulesRequest{} } @@ -129,6 +130,13 @@ func (m *RulesRequest) GetMatchers() []string { return nil } +func (m *RulesRequest) GetExcludeAlerts() bool { + if m != nil { + return m.ExcludeAlerts + } + return false +} + type RulesResponse struct { Groups []*GroupStateDesc `protobuf:"bytes,1,rep,name=groups,proto3" json:"groups,omitempty"` } @@ -444,56 +452,57 @@ func init() { func init() { proto.RegisterFile("ruler.proto", fileDescriptor_9ecbec0a4cfddea6) } var fileDescriptor_9ecbec0a4cfddea6 = []byte{ - // 774 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x55, 0x4d, 0x4f, 0x13, 0x4f, - 0x18, 0xdf, 0x69, 0xe9, 0xdb, 0x94, 0x3f, 0xe4, 0x3f, 0x54, 0xb3, 0x36, 0x64, 0x4b, 0x6a, 0x62, - 0x88, 0x89, 0xdb, 0xa4, 0x92, 0x18, 0x0f, 0x68, 0x4a, 0x00, 0x2f, 0xc6, 0x90, 0xad, 0x7a, 0x6d, - 0xa6, 0xed, 0x74, 0xbb, 0xb2, 0xdd, 0x59, 0x67, 0x66, 0x1b, 0xbc, 0xf9, 0x11, 0x38, 0x7a, 0xf6, - 0xe4, 0x47, 0xe1, 0x48, 0x3c, 0x11, 0x63, 0x50, 0xca, 0xc5, 0x23, 0x1f, 0xc1, 0xcc, 0xcc, 0x2e, - 0xdb, 0x22, 0x26, 0x34, 0x86, 0x0b, 0xcc, 0xf3, 0xf2, 0x7b, 0x5e, 0x7e, 0xcf, 0xb3, 0x4f, 0x61, - 0x99, 0x45, 0x3e, 0x61, 0x76, 0xc8, 0xa8, 0xa0, 0x28, 0xa7, 0x84, 0x6a, 0xc5, 0xa5, 0x2e, 0x55, - 0x9a, 0x86, 0x7c, 0x69, 0x63, 0xd5, 0x72, 0x29, 0x75, 0x7d, 0xd2, 0x50, 0x52, 0x37, 0x1a, 0x34, - 0xfa, 0x11, 0xc3, 0xc2, 0xa3, 0x41, 0x6c, 0xaf, 0x5d, 0xb5, 0x0b, 0x6f, 0x44, 0xb8, 0xc0, 0xa3, - 0x30, 0x76, 0x78, 0xea, 0x7a, 0x62, 0x18, 0x75, 0xed, 0x1e, 0x1d, 0x35, 0x7a, 0x94, 0x09, 0x72, - 0x10, 0x32, 0xfa, 0x8e, 0xf4, 0x44, 0x2c, 0x35, 0xc2, 0x7d, 0x37, 0x31, 0x74, 0xe3, 0x47, 0x0c, - 0xdd, 0xbc, 0x09, 0x54, 0x15, 0xaf, 0xfe, 0xf2, 0xb0, 0xab, 0xff, 0x6b, 0x78, 0xfd, 0x08, 0xc0, - 0x45, 0x47, 0xca, 0x0e, 0x79, 0x1f, 0x11, 0x2e, 0xd0, 0x2a, 0x2c, 0x49, 0xfb, 0x2b, 0x3c, 0x22, - 0xdc, 0x04, 0x6b, 0xd9, 0xf5, 0x92, 0x93, 0x2a, 0xd0, 0x03, 0xb8, 0x24, 0x85, 0x17, 0x8c, 0x46, - 0xa1, 0x76, 0xc9, 0x28, 0x97, 0x2b, 0x5a, 0x54, 0x81, 0xb9, 0x81, 0xe7, 0x13, 0x6e, 0x66, 0x95, - 0x59, 0x0b, 0x08, 0xc1, 0x05, 0xf1, 0x21, 0x24, 0xe6, 0xc2, 0x1a, 0x58, 0x2f, 0x39, 0xea, 0x2d, - 0x3d, 0xb9, 0xc0, 0x82, 0x98, 0x39, 0xa5, 0xd4, 0x02, 0xba, 0x0b, 0xf3, 0x43, 0x82, 0x7d, 0x31, - 0x34, 0xf3, 0x4a, 0x1d, 0x4b, 0xa8, 0x0a, 0x8b, 0x23, 0x2c, 0x7a, 0x43, 0xc2, 0xb8, 0x59, 0x50, - 0xa1, 0x2f, 0xe5, 0xfa, 0x33, 0xf8, 0x5f, 0xdc, 0x09, 0x0f, 0x69, 0xc0, 0x09, 0x7a, 0x04, 0xf3, - 0xae, 0x2c, 0x49, 0xf7, 0x51, 0x6e, 0xde, 0xb1, 0xf5, 0x44, 0x55, 0x9d, 0x6d, 0x99, 0x67, 0x9b, - 0xf0, 0x9e, 0x13, 0x3b, 0xd5, 0x3f, 0x67, 0xe0, 0xd2, 0xac, 0x09, 0x3d, 0x84, 0x39, 0x65, 0x34, - 0xc1, 0x1a, 0x58, 0x2f, 0x37, 0x2b, 0xb6, 0xa6, 0xce, 0x49, 0x9a, 0x55, 0x78, 0xed, 0x82, 0x9e, - 0xc0, 0x45, 0xdc, 0x13, 0xde, 0x98, 0x74, 0x94, 0x93, 0x22, 0x26, 0x81, 0x30, 0x05, 0x49, 0x53, - 0x96, 0xb5, 0xa7, 0x2a, 0x17, 0xbd, 0x85, 0x2b, 0x64, 0x8c, 0xfd, 0x48, 0x6d, 0xcc, 0xeb, 0x64, - 0x33, 0xcc, 0xac, 0x4a, 0x59, 0xb5, 0xf5, 0xee, 0xd8, 0xc9, 0xee, 0xd8, 0x97, 0x1e, 0x5b, 0xc5, - 0xa3, 0xd3, 0x9a, 0x71, 0xf8, 0xa3, 0x06, 0x9c, 0xeb, 0x02, 0xa0, 0x36, 0x44, 0xa9, 0x7a, 0x3b, - 0xde, 0x48, 0xc5, 0x7d, 0xb9, 0x79, 0xef, 0x8f, 0xb0, 0x89, 0x83, 0x8e, 0xfa, 0x49, 0x46, 0xbd, - 0x06, 0x5e, 0xff, 0x9e, 0xd1, 0x2c, 0xa7, 0x1c, 0xdd, 0x87, 0x0b, 0xb2, 0xc5, 0x98, 0xa2, 0xe5, - 0x29, 0x8a, 0x54, 0xab, 0xca, 0x98, 0x4e, 0x39, 0x73, 0xfd, 0x94, 0xb3, 0x33, 0x53, 0x5e, 0x85, - 0x25, 0x1f, 0x73, 0xb1, 0xc3, 0x18, 0x65, 0xf1, 0xb2, 0xa4, 0x0a, 0x39, 0x56, 0xec, 0x13, 0x26, - 0xb8, 0x99, 0x9b, 0x19, 0x6b, 0x4b, 0x2a, 0xa7, 0xc6, 0xaa, 0x9d, 0xfe, 0x46, 0x6f, 0xfe, 0x76, - 0xe8, 0x2d, 0xfc, 0x1b, 0xbd, 0x5f, 0x73, 0x70, 0x69, 0xb6, 0x8f, 0x94, 0x3a, 0x30, 0x4d, 0x5d, - 0x00, 0xf3, 0x3e, 0xee, 0x12, 0x3f, 0xd9, 0xb3, 0x15, 0x3b, 0x39, 0x0f, 0xf6, 0x4b, 0xa9, 0xdf, - 0xc3, 0x1e, 0xdb, 0x6a, 0xc9, 0x5c, 0xdf, 0x4e, 0x6b, 0x73, 0x9d, 0x17, 0x8d, 0x6f, 0xf5, 0x71, - 0x28, 0x08, 0x73, 0xe2, 0x2c, 0xe8, 0x00, 0x96, 0x71, 0x10, 0x50, 0xa1, 0xca, 0xd4, 0x9f, 0xf5, - 0xed, 0x25, 0x9d, 0x4e, 0x25, 0xfb, 0x97, 0x3c, 0xe9, 0xab, 0x01, 0x1c, 0x2d, 0xa0, 0x16, 0x2c, - 0xc5, 0x5f, 0x1b, 0x16, 0xea, 0x74, 0xdc, 0x74, 0x96, 0x45, 0x0d, 0x6b, 0x09, 0xf4, 0x1c, 0x16, - 0x07, 0x1e, 0x23, 0x7d, 0x19, 0x61, 0x9e, 0x6d, 0x28, 0x28, 0x54, 0x4b, 0xa0, 0x1d, 0x58, 0x66, - 0x84, 0x53, 0x7f, 0xac, 0x63, 0x14, 0xe6, 0x88, 0x01, 0x13, 0x60, 0x4b, 0xa0, 0x5d, 0xb8, 0x28, - 0x97, 0xbb, 0xc3, 0x49, 0x20, 0x64, 0x9c, 0xe2, 0x3c, 0x71, 0x24, 0xb2, 0x4d, 0x02, 0xa1, 0xcb, - 0x19, 0x63, 0xdf, 0xeb, 0x77, 0xa2, 0x40, 0x78, 0xbe, 0x59, 0x9a, 0x27, 0x8c, 0x02, 0xbe, 0x91, - 0x38, 0xb4, 0x07, 0xff, 0xdf, 0x27, 0x24, 0xec, 0x0c, 0x3c, 0xe6, 0x05, 0x6e, 0x87, 0x7b, 0x41, - 0x8f, 0x98, 0x70, 0x8e, 0x60, 0xcb, 0x12, 0xbe, 0xab, 0xd0, 0x6d, 0x09, 0x6e, 0x6e, 0xc2, 0x9c, - 0x3c, 0x07, 0x0c, 0x6d, 0xe8, 0x07, 0x47, 0x2b, 0x53, 0x57, 0x31, 0xf9, 0xe5, 0xa9, 0x56, 0x66, - 0x95, 0xfa, 0x88, 0xd7, 0x8d, 0xad, 0x8d, 0xe3, 0x33, 0xcb, 0x38, 0x39, 0xb3, 0x8c, 0x8b, 0x33, - 0x0b, 0x7c, 0x9c, 0x58, 0xe0, 0xcb, 0xc4, 0x02, 0x47, 0x13, 0x0b, 0x1c, 0x4f, 0x2c, 0xf0, 0x73, - 0x62, 0x81, 0x5f, 0x13, 0xcb, 0xb8, 0x98, 0x58, 0xe0, 0xf0, 0xdc, 0x32, 0x8e, 0xcf, 0x2d, 0xe3, - 0xe4, 0xdc, 0x32, 0xba, 0x79, 0x55, 0xe3, 0xe3, 0xdf, 0x01, 0x00, 0x00, 0xff, 0xff, 0xb1, 0xdb, - 0xbe, 0x81, 0xc6, 0x07, 0x00, 0x00, + // 793 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x55, 0x4d, 0x6b, 0x13, 0x41, + 0x18, 0xde, 0x49, 0x9a, 0xaf, 0x49, 0x3f, 0x70, 0x1a, 0x65, 0x0d, 0x65, 0x13, 0xa2, 0x48, 0x10, + 0xdc, 0x40, 0x2c, 0x88, 0x87, 0x2a, 0x29, 0x6d, 0xbd, 0x88, 0x94, 0x8d, 0x7a, 0x0d, 0x93, 0xcd, + 0x64, 0xb3, 0x76, 0xb3, 0xbb, 0xce, 0xcc, 0x86, 0x7a, 0xf3, 0x27, 0xf4, 0xe8, 0xd9, 0x93, 0x3f, + 0xa5, 0xc7, 0xe2, 0xa9, 0x88, 0x54, 0x9b, 0x5e, 0x3c, 0x49, 0x7f, 0x82, 0xcc, 0xcc, 0x6e, 0x93, + 0xd4, 0x0a, 0x0d, 0xd2, 0x4b, 0x3b, 0xef, 0xc7, 0xf3, 0xce, 0xfb, 0x3e, 0xf3, 0xec, 0x1b, 0x58, + 0xa4, 0x91, 0x47, 0xa8, 0x19, 0xd2, 0x80, 0x07, 0x28, 0x23, 0x8d, 0x72, 0xc9, 0x09, 0x9c, 0x40, + 0x7a, 0x1a, 0xe2, 0xa4, 0x82, 0x65, 0xc3, 0x09, 0x02, 0xc7, 0x23, 0x0d, 0x69, 0x75, 0xa3, 0x7e, + 0xa3, 0x17, 0x51, 0xcc, 0xdd, 0xc0, 0x8f, 0xe3, 0x95, 0xcb, 0x71, 0xee, 0x0e, 0x09, 0xe3, 0x78, + 0x18, 0xc6, 0x09, 0x4f, 0x1d, 0x97, 0x0f, 0xa2, 0xae, 0x69, 0x07, 0xc3, 0x86, 0x1d, 0x50, 0x4e, + 0xf6, 0x43, 0x1a, 0xbc, 0x23, 0x36, 0x8f, 0xad, 0x46, 0xb8, 0xe7, 0x24, 0x81, 0x6e, 0x7c, 0x88, + 0xa1, 0x1b, 0xd7, 0x81, 0xca, 0xe6, 0xe5, 0x5f, 0x16, 0x76, 0xd5, 0x7f, 0x05, 0xaf, 0xfd, 0x06, + 0x70, 0xd1, 0x12, 0xb6, 0x45, 0xde, 0x47, 0x84, 0x71, 0xb4, 0x06, 0x0b, 0x22, 0xfe, 0x0a, 0x0f, + 0x09, 0xd3, 0x41, 0x35, 0x5d, 0x2f, 0x58, 0x13, 0x07, 0x7a, 0x00, 0x97, 0x85, 0xf1, 0x82, 0x06, + 0x51, 0xa8, 0x52, 0x52, 0x32, 0xe5, 0x92, 0x17, 0x95, 0x60, 0xa6, 0xef, 0x7a, 0x84, 0xe9, 0x69, + 0x19, 0x56, 0x06, 0x42, 0x70, 0x81, 0x7f, 0x08, 0x89, 0xbe, 0x50, 0x05, 0xf5, 0x82, 0x25, 0xcf, + 0x22, 0x93, 0x71, 0xcc, 0x89, 0x9e, 0x91, 0x4e, 0x65, 0xa0, 0x3b, 0x30, 0x3b, 0x20, 0xd8, 0xe3, + 0x03, 0x3d, 0x2b, 0xdd, 0xb1, 0x85, 0xca, 0x30, 0x3f, 0xc4, 0xdc, 0x1e, 0x10, 0xca, 0xf4, 0x9c, + 0x2c, 0x7d, 0x61, 0xa3, 0xfb, 0x70, 0x89, 0xec, 0xdb, 0x5e, 0xd4, 0x23, 0x2d, 0x8f, 0x50, 0xce, + 0xf4, 0x7c, 0x15, 0xd4, 0xf3, 0xd6, 0xac, 0xb3, 0xf6, 0x0c, 0x2e, 0xc5, 0xf3, 0xb2, 0x30, 0xf0, + 0x19, 0x41, 0x8f, 0x60, 0xd6, 0x11, 0x8d, 0xab, 0x69, 0x8b, 0xcd, 0xdb, 0xa6, 0x7a, 0x77, 0x39, + 0x4d, 0x5b, 0x74, 0xb3, 0x45, 0x98, 0x6d, 0xc5, 0x49, 0xb5, 0xcf, 0x29, 0xb8, 0x3c, 0x1b, 0x42, + 0x0f, 0x61, 0x46, 0x06, 0x75, 0x50, 0x05, 0xf5, 0x62, 0xb3, 0x64, 0x2a, 0x82, 0xad, 0x84, 0x12, + 0x89, 0x57, 0x29, 0xe8, 0x09, 0x5c, 0xc4, 0x36, 0x77, 0x47, 0xa4, 0x23, 0x93, 0x24, 0x7d, 0x09, + 0x84, 0x4a, 0xc8, 0xe4, 0xca, 0xa2, 0xca, 0x94, 0xed, 0xa2, 0xb7, 0x70, 0x95, 0x8c, 0xb0, 0x17, + 0x49, 0x5d, 0xbd, 0x4e, 0xf4, 0xa3, 0xa7, 0xe5, 0x95, 0x65, 0x53, 0x29, 0xcc, 0x4c, 0x14, 0x66, + 0x5e, 0x64, 0x6c, 0xe6, 0x0f, 0x4f, 0x2a, 0xda, 0xc1, 0x8f, 0x0a, 0xb0, 0xae, 0x2a, 0x80, 0xda, + 0x10, 0x4d, 0xdc, 0x5b, 0xb1, 0x6e, 0xe5, 0x0b, 0x15, 0x9b, 0x77, 0xff, 0x2a, 0x9b, 0x24, 0xa8, + 0xaa, 0x9f, 0x44, 0xd5, 0x2b, 0xe0, 0xb5, 0xef, 0x29, 0xc5, 0xf2, 0x84, 0xa3, 0x7b, 0x70, 0x41, + 0x8c, 0x18, 0x53, 0xb4, 0x32, 0x45, 0x91, 0x1c, 0x55, 0x06, 0x27, 0x5a, 0x48, 0x5d, 0xad, 0x85, + 0xf4, 0x8c, 0x16, 0xd6, 0x60, 0xc1, 0xc3, 0x8c, 0x6f, 0x53, 0x1a, 0xd0, 0x58, 0x52, 0x13, 0x87, + 0x78, 0x56, 0xac, 0x64, 0x90, 0x99, 0x79, 0x56, 0x29, 0x83, 0xa9, 0x67, 0x55, 0x49, 0xff, 0xa2, + 0x37, 0x7b, 0x33, 0xf4, 0xe6, 0xfe, 0x8f, 0xde, 0xaf, 0x19, 0xb8, 0x3c, 0x3b, 0xc7, 0x84, 0x3a, + 0x30, 0x4d, 0x9d, 0x0f, 0xb3, 0x1e, 0xee, 0x12, 0x2f, 0xd1, 0xd9, 0xaa, 0x99, 0x2c, 0x11, 0xf3, + 0xa5, 0xf0, 0xef, 0x62, 0x97, 0x6e, 0xb6, 0xc4, 0x5d, 0xdf, 0x4e, 0x2a, 0x73, 0x2d, 0x21, 0x85, + 0x6f, 0xf5, 0x70, 0xc8, 0x09, 0xb5, 0xe2, 0x5b, 0xd0, 0x3e, 0x2c, 0x62, 0xdf, 0x0f, 0xb8, 0x6c, + 0x53, 0x7d, 0xfc, 0x37, 0x77, 0xe9, 0xf4, 0x55, 0x62, 0x7e, 0xc1, 0x93, 0xda, 0x2d, 0xc0, 0x52, + 0x06, 0x6a, 0xc1, 0x42, 0xfc, 0xb5, 0x61, 0x2e, 0x17, 0xcc, 0x75, 0xdf, 0x32, 0xaf, 0x60, 0x2d, + 0x8e, 0x9e, 0xc3, 0x7c, 0xdf, 0xa5, 0xa4, 0x27, 0x2a, 0xcc, 0xa3, 0x86, 0x9c, 0x44, 0xb5, 0x38, + 0xda, 0x86, 0x45, 0x4a, 0x58, 0xe0, 0x8d, 0x54, 0x8d, 0xdc, 0x1c, 0x35, 0x60, 0x02, 0x6c, 0x71, + 0xb4, 0x03, 0x17, 0x85, 0xb8, 0x3b, 0x8c, 0xf8, 0x5c, 0xd4, 0xc9, 0xcf, 0x53, 0x47, 0x20, 0xdb, + 0xc4, 0xe7, 0xaa, 0x9d, 0x11, 0xf6, 0xdc, 0x5e, 0x27, 0xf2, 0xb9, 0xeb, 0xe9, 0x85, 0x79, 0xca, + 0x48, 0xe0, 0x1b, 0x81, 0x43, 0xbb, 0xf0, 0xd6, 0x1e, 0x21, 0x61, 0xa7, 0xef, 0x52, 0xd7, 0x77, + 0x3a, 0xcc, 0xf5, 0x6d, 0xa2, 0xc3, 0x39, 0x8a, 0xad, 0x08, 0xf8, 0x8e, 0x44, 0xb7, 0x05, 0xb8, + 0xb9, 0x01, 0x33, 0x62, 0x1d, 0x50, 0xb4, 0xae, 0x0e, 0x0c, 0xad, 0x4e, 0x6d, 0xc5, 0xe4, 0xf7, + 0xa9, 0x5c, 0x9a, 0x75, 0xaa, 0x25, 0x5e, 0xd3, 0x36, 0xd7, 0x8f, 0x4e, 0x0d, 0xed, 0xf8, 0xd4, + 0xd0, 0xce, 0x4f, 0x0d, 0xf0, 0x71, 0x6c, 0x80, 0x2f, 0x63, 0x03, 0x1c, 0x8e, 0x0d, 0x70, 0x34, + 0x36, 0xc0, 0xcf, 0xb1, 0x01, 0x7e, 0x8d, 0x0d, 0xed, 0x7c, 0x6c, 0x80, 0x83, 0x33, 0x43, 0x3b, + 0x3a, 0x33, 0xb4, 0xe3, 0x33, 0x43, 0xeb, 0x66, 0x65, 0x8f, 0x8f, 0xff, 0x04, 0x00, 0x00, 0xff, + 0xff, 0x3a, 0xeb, 0x6f, 0xc9, 0xec, 0x07, 0x00, 0x00, } func (this *RulesRequest) Equal(that interface{}) bool { @@ -556,6 +565,9 @@ func (this *RulesRequest) Equal(that interface{}) bool { return false } } + if this.ExcludeAlerts != that1.ExcludeAlerts { + return false + } return true } func (this *RulesResponse) Equal(that interface{}) bool { @@ -737,7 +749,7 @@ func (this *RulesRequest) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 11) + s := make([]string, 0, 12) s = append(s, "&ruler.RulesRequest{") s = append(s, "RuleNames: "+fmt.Sprintf("%#v", this.RuleNames)+",\n") s = append(s, "RuleGroupNames: "+fmt.Sprintf("%#v", this.RuleGroupNames)+",\n") @@ -746,6 +758,7 @@ func (this *RulesRequest) GoString() string { s = append(s, "State: "+fmt.Sprintf("%#v", this.State)+",\n") s = append(s, "Health: "+fmt.Sprintf("%#v", this.Health)+",\n") s = append(s, "Matchers: "+fmt.Sprintf("%#v", this.Matchers)+",\n") + s = append(s, "ExcludeAlerts: "+fmt.Sprintf("%#v", this.ExcludeAlerts)+",\n") s = append(s, "}") return strings.Join(s, "") } @@ -926,6 +939,16 @@ func (m *RulesRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.ExcludeAlerts { + i-- + if m.ExcludeAlerts { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x40 + } if len(m.Matchers) > 0 { for iNdEx := len(m.Matchers) - 1; iNdEx >= 0; iNdEx-- { i -= len(m.Matchers[iNdEx]) @@ -1339,6 +1362,9 @@ func (m *RulesRequest) Size() (n int) { n += 1 + l + sovRuler(uint64(l)) } } + if m.ExcludeAlerts { + n += 2 + } return n } @@ -1473,6 +1499,7 @@ func (this *RulesRequest) String() string { `State:` + fmt.Sprintf("%v", this.State) + `,`, `Health:` + fmt.Sprintf("%v", this.Health) + `,`, `Matchers:` + fmt.Sprintf("%v", this.Matchers) + `,`, + `ExcludeAlerts:` + fmt.Sprintf("%v", this.ExcludeAlerts) + `,`, `}`, }, "") return s @@ -1811,6 +1838,26 @@ func (m *RulesRequest) Unmarshal(dAtA []byte) error { } m.Matchers = append(m.Matchers, string(dAtA[iNdEx:postIndex])) iNdEx = postIndex + case 8: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ExcludeAlerts", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuler + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.ExcludeAlerts = bool(v != 0) default: iNdEx = preIndex skippy, err := skipRuler(dAtA[iNdEx:]) diff --git a/pkg/ruler/ruler.proto b/pkg/ruler/ruler.proto index 0d1aa45525..42828f7352 100644 --- a/pkg/ruler/ruler.proto +++ b/pkg/ruler/ruler.proto @@ -27,6 +27,7 @@ message RulesRequest { string state = 5; string health = 6; repeated string matchers = 7; + bool excludeAlerts = 8; } message RulesResponse { From 85bc55547b45c674f3b5f115d03818d1727c4811 Mon Sep 17 00:00:00 2001 From: Mustafain Ali Khan Date: Tue, 30 Jul 2024 11:08:16 -0700 Subject: [PATCH 23/32] Add support for Prometheus rule query offset (#6085) * Add support for prometheus rule query offset Signed-off-by: Mustafain Ali Khan * Fix tests Signed-off-by: Mustafain Ali Khan * Use per-tenant limit for global query offset Signed-off-by: Mustafain Ali Khan --------- Signed-off-by: Mustafain Ali Khan --- CHANGELOG.md | 1 + docs/configuration/config-file-reference.md | 4 + pkg/ruler/api_test.go | 19 ++- pkg/ruler/compat.go | 4 + pkg/ruler/ruler.go | 12 +- pkg/ruler/ruler_test.go | 37 +++++ pkg/ruler/rulespb/compat.go | 31 ++-- pkg/ruler/rulespb/compat_test.go | 10 +- pkg/ruler/rulespb/rules.pb.go | 175 ++++++++++++++------ pkg/ruler/rulespb/rules.proto | 4 +- pkg/ruler/store_mock_test.go | 37 +++++ pkg/util/validation/limits.go | 7 + 12 files changed, 267 insertions(+), 74 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 10d2ccbee1..eb6e4bad0c 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -12,6 +12,7 @@ * [FEATURE] Ingester: Add `ingester.instance-limits.max-inflight-query-requests` to allow limiting ingester concurrent queries. #6081 * [FEATURE] Distributor: Add `validation.max-native-histogram-buckets` to limit max number of bucket count. Distributor will try to automatically reduce histogram resolution until it is within the bucket limit or resolution cannot be reduced anymore. #6104 * [FEATURE] Store Gateway: Token bucket limiter. #6016 +* [FEATURE] Ruler: Add support for `query_offset` field on RuleGroup and new `ruler_query_offset` per-tenant limit. #6085 * [ENHANCEMENT] rulers: Add support to persist tokens in rulers. #5987 * [ENHANCEMENT] Query Frontend/Querier: Added store gateway postings touched count and touched size in Querier stats and log in Query Frontend. #5892 * [ENHANCEMENT] Query Frontend/Querier: Returns `warnings` on prometheus query responses. #5916 diff --git a/docs/configuration/config-file-reference.md b/docs/configuration/config-file-reference.md index df036ed2f6..0e646f7788 100644 --- a/docs/configuration/config-file-reference.md +++ b/docs/configuration/config-file-reference.md @@ -3349,6 +3349,10 @@ query_rejection: # CLI flag: -ruler.max-rule-groups-per-tenant [ruler_max_rule_groups_per_tenant: | default = 0] +# Duration to offset all rule evaluation queries per-tenant. +# CLI flag: -ruler.query-offset +[ruler_query_offset: | default = 0s] + # The default tenant's shard size when the shuffle-sharding strategy is used. # Must be set when the store-gateway sharding is enabled with the # shuffle-sharding strategy. When this setting is specified in the per-tenant diff --git a/pkg/ruler/api_test.go b/pkg/ruler/api_test.go index 132ecfcf64..cdddcf2432 100644 --- a/pkg/ruler/api_test.go +++ b/pkg/ruler/api_test.go @@ -263,7 +263,7 @@ interval: 15s err: errors.New("invalid rules config: rule group 'rg_name' has no rules"), }, { - name: "with a a valid rules file", + name: "with a valid rules file", status: 202, input: ` name: test @@ -279,7 +279,20 @@ rules: labels: test: test `, - output: "name: test\ninterval: 15s\nrules:\n - record: up_rule\n expr: up{}\n - alert: up_alert\n expr: sum(up{}) > 1\n for: 30s\n labels:\n test: test\n annotations:\n test: test\n", + output: "name: test\ninterval: 15s\nquery_offset: 0s\nrules:\n - record: up_rule\n expr: up{}\n - alert: up_alert\n expr: sum(up{}) > 1\n for: 30s\n labels:\n test: test\n annotations:\n test: test\n", + }, + { + name: "with a valid rule query offset", + status: 202, + input: ` +name: test +interval: 15s +query_offset: 2m +rules: +- record: up_rule + expr: up{} +`, + output: "name: test\ninterval: 15s\nquery_offset: 2m\nrules:\n - record: up_rule\n expr: up{}\n", }, } @@ -329,7 +342,7 @@ func TestRuler_DeleteNamespace(t *testing.T) { router.ServeHTTP(w, req) require.Equal(t, http.StatusOK, w.Code) - require.Equal(t, "name: group1\ninterval: 1m\nrules:\n - record: UP_RULE\n expr: up\n - alert: UP_ALERT\n expr: up < 1\n", w.Body.String()) + require.Equal(t, "name: group1\ninterval: 1m\nquery_offset: 0s\nrules:\n - record: UP_RULE\n expr: up\n - alert: UP_ALERT\n expr: up < 1\n", w.Body.String()) // Delete namespace1 req = requestFor(t, http.MethodDelete, "https://localhost:8080/api/v1/rules/namespace1", nil, "user1") diff --git a/pkg/ruler/compat.go b/pkg/ruler/compat.go index b9b4ba2b35..9c3fd2f0f9 100644 --- a/pkg/ruler/compat.go +++ b/pkg/ruler/compat.go @@ -178,6 +178,7 @@ type RulesLimits interface { RulerTenantShardSize(userID string) int RulerMaxRuleGroupsPerTenant(userID string) int RulerMaxRulesPerRuleGroup(userID string) int + RulerQueryOffset(userID string) time.Duration DisabledRuleGroups(userID string) validation.DisabledRuleGroups } @@ -358,6 +359,9 @@ func DefaultTenantManagerFactory(cfg Config, p Pusher, q storage.Queryable, engi ResendDelay: cfg.ResendDelay, ConcurrentEvalsEnabled: cfg.ConcurrentEvalsEnabled, MaxConcurrentEvals: cfg.MaxConcurrentEvals, + DefaultRuleQueryOffset: func() time.Duration { + return overrides.RulerQueryOffset(userID) + }, }) } } diff --git a/pkg/ruler/ruler.go b/pkg/ruler/ruler.go index dfacb61d80..fb6df8caf3 100644 --- a/pkg/ruler/ruler.go +++ b/pkg/ruler/ruler.go @@ -911,13 +911,15 @@ func (r *Ruler) getLocalRules(userID string, rulesRequest RulesRequest, includeB } interval := group.Interval() + queryOffset := group.QueryOffset() groupDesc := &GroupStateDesc{ Group: &rulespb.RuleGroupDesc{ - Name: group.Name(), - Namespace: string(decodedNamespace), - Interval: interval, - User: userID, - Limit: int64(group.Limit()), + Name: group.Name(), + Namespace: string(decodedNamespace), + Interval: interval, + User: userID, + Limit: int64(group.Limit()), + QueryOffset: &queryOffset, }, EvaluationTimestamp: group.GetLastEvaluation(), diff --git a/pkg/ruler/ruler_test.go b/pkg/ruler/ruler_test.go index f66efede18..befe7d01f4 100644 --- a/pkg/ruler/ruler_test.go +++ b/pkg/ruler/ruler_test.go @@ -88,6 +88,7 @@ type ruleLimits struct { maxRuleGroups int disabledRuleGroups validation.DisabledRuleGroups maxQueryLength time.Duration + queryOffset time.Duration } func (r ruleLimits) EvaluationDelay(_ string) time.Duration { @@ -112,6 +113,10 @@ func (r ruleLimits) DisabledRuleGroups(userID string) validation.DisabledRuleGro func (r ruleLimits) MaxQueryLength(_ string) time.Duration { return r.maxQueryLength } +func (r ruleLimits) RulerQueryOffset(_ string) time.Duration { + return r.queryOffset +} + func newEmptyQueryable() storage.Queryable { return storage.QueryableFunc(func(mint, maxt int64) (storage.Querier, error) { return emptyQuerier{}, nil @@ -2533,3 +2538,35 @@ func TestRulerDisablesRuleGroups(t *testing.T) { }) } } + +func TestRuler_QueryOffset(t *testing.T) { + store := newMockRuleStore(mockRulesQueryOffset, nil) + cfg := defaultRulerConfig(t) + + r := newTestRuler(t, cfg, store, nil) + defer services.StopAndAwaitTerminated(context.Background(), r) //nolint:errcheck + + ctx := user.InjectOrgID(context.Background(), "user1") + rls, err := r.Rules(ctx, &RulesRequest{}) + require.NoError(t, err) + require.Len(t, rls.Groups, 1) + rg := rls.Groups[0] + expectedRg := mockRulesQueryOffset["user1"][0] + compareRuleGroupDescToStateDesc(t, expectedRg, rg) + + // test default query offset=0 when not defined at group level + gotOffset := rg.GetGroup().QueryOffset + require.Equal(t, time.Duration(0), *gotOffset) + + ctx = user.InjectOrgID(context.Background(), "user2") + rls, err = r.Rules(ctx, &RulesRequest{}) + require.NoError(t, err) + require.Len(t, rls.Groups, 1) + rg = rls.Groups[0] + expectedRg = mockRules["user2"][0] + compareRuleGroupDescToStateDesc(t, expectedRg, rg) + + // test group query offset is set + gotOffset = rg.GetGroup().QueryOffset + require.Equal(t, time.Minute*2, *gotOffset) +} diff --git a/pkg/ruler/rulespb/compat.go b/pkg/ruler/rulespb/compat.go index f735eb1734..16bf075e27 100644 --- a/pkg/ruler/rulespb/compat.go +++ b/pkg/ruler/rulespb/compat.go @@ -13,13 +13,19 @@ import ( // ToProto transforms a formatted prometheus rulegroup to a rule group protobuf func ToProto(user string, namespace string, rl rulefmt.RuleGroup) *RuleGroupDesc { + var queryOffset *time.Duration + if rl.QueryOffset != nil { + offset := time.Duration(*rl.QueryOffset) + queryOffset = &offset + } rg := RuleGroupDesc{ - Name: rl.Name, - Namespace: namespace, - Interval: time.Duration(rl.Interval), - Rules: formattedRuleToProto(rl.Rules), - User: user, - Limit: int64(rl.Limit), + Name: rl.Name, + Namespace: namespace, + Interval: time.Duration(rl.Interval), + Rules: formattedRuleToProto(rl.Rules), + User: user, + Limit: int64(rl.Limit), + QueryOffset: queryOffset, } return &rg } @@ -43,11 +49,16 @@ func formattedRuleToProto(rls []rulefmt.RuleNode) []*RuleDesc { // FromProto generates a rulefmt RuleGroup func FromProto(rg *RuleGroupDesc) rulefmt.RuleGroup { + var queryOffset model.Duration + if rg.QueryOffset != nil { + queryOffset = model.Duration(*rg.QueryOffset) + } formattedRuleGroup := rulefmt.RuleGroup{ - Name: rg.GetName(), - Interval: model.Duration(rg.Interval), - Rules: make([]rulefmt.RuleNode, len(rg.GetRules())), - Limit: int(rg.Limit), + Name: rg.GetName(), + Interval: model.Duration(rg.Interval), + Rules: make([]rulefmt.RuleNode, len(rg.GetRules())), + Limit: int(rg.Limit), + QueryOffset: &queryOffset, } for i, rl := range rg.GetRules() { diff --git a/pkg/ruler/rulespb/compat_test.go b/pkg/ruler/rulespb/compat_test.go index c438a3fd07..736366714d 100644 --- a/pkg/ruler/rulespb/compat_test.go +++ b/pkg/ruler/rulespb/compat_test.go @@ -29,14 +29,18 @@ func TestProto(t *testing.T) { rules = append(rules, testRule) + queryOffset := model.Duration(30 * time.Second) rg := rulefmt.RuleGroup{ - Name: "group1", - Rules: rules, - Interval: model.Duration(time.Minute), + Name: "group1", + Rules: rules, + Interval: model.Duration(time.Minute), + QueryOffset: &queryOffset, } + desc := ToProto("test", "namespace", rg) assert.Equal(t, len(rules), len(desc.Rules)) + assert.Equal(t, 30*time.Second, *desc.QueryOffset) ruleDesc := desc.Rules[0] diff --git a/pkg/ruler/rulespb/rules.pb.go b/pkg/ruler/rulespb/rules.pb.go index aa8675c9e0..8f09b2cb40 100644 --- a/pkg/ruler/rulespb/rules.pb.go +++ b/pkg/ruler/rulespb/rules.pb.go @@ -43,8 +43,9 @@ type RuleGroupDesc struct { // having to repeatedly redefine the proto description. It can also be leveraged // to create custom `ManagerOpts` based on rule configs which can then be passed // to the Prometheus Manager. - Options []*types.Any `protobuf:"bytes,9,rep,name=options,proto3" json:"options,omitempty"` - Limit int64 `protobuf:"varint,10,opt,name=limit,proto3" json:"limit,omitempty"` + Options []*types.Any `protobuf:"bytes,9,rep,name=options,proto3" json:"options,omitempty"` + Limit int64 `protobuf:"varint,10,opt,name=limit,proto3" json:"limit,omitempty"` + QueryOffset *time.Duration `protobuf:"bytes,11,opt,name=queryOffset,proto3,stdduration" json:"queryOffset,omitempty"` } func (m *RuleGroupDesc) Reset() { *m = RuleGroupDesc{} } @@ -128,6 +129,13 @@ func (m *RuleGroupDesc) GetLimit() int64 { return 0 } +func (m *RuleGroupDesc) GetQueryOffset() *time.Duration { + if m != nil { + return m.QueryOffset + } + return nil +} + // RuleDesc is a proto representation of a Prometheus Rule type RuleDesc struct { Expr string `protobuf:"bytes,1,opt,name=expr,proto3" json:"expr,omitempty"` @@ -214,40 +222,42 @@ func init() { func init() { proto.RegisterFile("rules.proto", fileDescriptor_8e722d3e922f0937) } var fileDescriptor_8e722d3e922f0937 = []byte{ - // 524 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x52, 0x4f, 0x6f, 0xd3, 0x30, - 0x1c, 0x8d, 0xd7, 0x34, 0x4b, 0x5d, 0x55, 0xab, 0x4c, 0x85, 0xbc, 0x81, 0xdc, 0x6a, 0x12, 0x52, - 0x4f, 0x89, 0x34, 0xc4, 0x81, 0x03, 0x42, 0xad, 0xa6, 0x21, 0x55, 0x1c, 0x50, 0x8e, 0x08, 0x69, - 0x72, 0x52, 0x37, 0x84, 0xa5, 0x71, 0xe4, 0x38, 0x68, 0xbb, 0xf1, 0x11, 0xb8, 0x20, 0xf1, 0x11, - 0xf8, 0x28, 0x3b, 0x96, 0xdb, 0xc4, 0xa1, 0xd0, 0xf4, 0x82, 0x38, 0xed, 0x23, 0x20, 0xdb, 0x09, - 0x7f, 0x0f, 0xc0, 0x81, 0x53, 0x7e, 0xcf, 0x2f, 0xcf, 0xef, 0xf9, 0xd9, 0xb0, 0x2b, 0xca, 0x94, - 0x15, 0x5e, 0x2e, 0xb8, 0xe4, 0xa8, 0xad, 0xc1, 0xc1, 0x20, 0xe6, 0x31, 0xd7, 0x2b, 0xbe, 0x9a, - 0x0c, 0x79, 0x40, 0x62, 0xce, 0xe3, 0x94, 0xf9, 0x1a, 0x85, 0xe5, 0xc2, 0x9f, 0x97, 0x82, 0xca, - 0x84, 0x67, 0x35, 0xbf, 0xff, 0x2b, 0x4f, 0xb3, 0x8b, 0x9a, 0xba, 0x1f, 0x27, 0xf2, 0x79, 0x19, - 0x7a, 0x11, 0x5f, 0xfa, 0x11, 0x17, 0x92, 0x9d, 0xe7, 0x82, 0xbf, 0x60, 0x91, 0xac, 0x91, 0x9f, - 0x9f, 0xc5, 0x0d, 0x11, 0xd6, 0x83, 0x91, 0x1e, 0xbe, 0xd9, 0x81, 0xbd, 0xa0, 0x4c, 0xd9, 0x23, - 0xc1, 0xcb, 0xfc, 0x98, 0x15, 0x11, 0x42, 0xd0, 0xce, 0xe8, 0x92, 0x61, 0x30, 0x02, 0xe3, 0x4e, - 0xa0, 0x67, 0x74, 0x1b, 0x76, 0xd4, 0xb7, 0xc8, 0x69, 0xc4, 0xf0, 0x8e, 0x26, 0xbe, 0x2f, 0xa0, - 0x87, 0xd0, 0x4d, 0x32, 0xc9, 0xc4, 0x4b, 0x9a, 0xe2, 0xd6, 0x08, 0x8c, 0xbb, 0x47, 0xfb, 0x9e, - 0x09, 0xeb, 0x35, 0x61, 0xbd, 0xe3, 0xfa, 0x30, 0x53, 0xf7, 0x72, 0x3d, 0xb4, 0xde, 0x7e, 0x1c, - 0x82, 0xe0, 0x9b, 0x08, 0xdd, 0x81, 0xa6, 0x19, 0x6c, 0x8f, 0x5a, 0xe3, 0xee, 0xd1, 0x9e, 0x67, - 0x4a, 0x53, 0xb9, 0x54, 0xa4, 0xc0, 0xb0, 0x2a, 0x59, 0x59, 0x30, 0x81, 0x1d, 0x93, 0x4c, 0xcd, - 0xc8, 0x83, 0xbb, 0x3c, 0x57, 0x1b, 0x17, 0xb8, 0xa3, 0xc5, 0x83, 0xdf, 0xac, 0x27, 0xd9, 0x45, - 0xd0, 0xfc, 0x84, 0x06, 0xb0, 0x9d, 0x26, 0xcb, 0x44, 0x62, 0x38, 0x02, 0xe3, 0x56, 0x60, 0xc0, - 0xcc, 0x76, 0xdb, 0x7d, 0x67, 0x66, 0xbb, 0xbb, 0x7d, 0x77, 0x66, 0xbb, 0x6e, 0xbf, 0x73, 0xf8, - 0xbe, 0x05, 0xdd, 0xc6, 0x5f, 0x19, 0xab, 0x4a, 0x9b, 0x4a, 0xd4, 0x8c, 0x6e, 0x42, 0x47, 0xb0, - 0x88, 0x8b, 0x79, 0xdd, 0x47, 0x8d, 0x94, 0x01, 0x4d, 0x99, 0x90, 0xba, 0x89, 0x4e, 0x60, 0x00, - 0xba, 0x07, 0x5b, 0x0b, 0x2e, 0xb0, 0xfd, 0xf7, 0xed, 0xa8, 0xff, 0x51, 0x06, 0x9d, 0x94, 0x86, - 0x2c, 0x2d, 0x70, 0x5b, 0x1f, 0xee, 0x86, 0xd7, 0xdc, 0xa2, 0xf7, 0x58, 0xad, 0x3f, 0xa1, 0x89, - 0x98, 0x4e, 0x94, 0xe6, 0xc3, 0x7a, 0xf8, 0x4f, 0xaf, 0xc0, 0xe8, 0x27, 0x73, 0x9a, 0x4b, 0x26, - 0x82, 0xda, 0x05, 0x9d, 0xc3, 0x2e, 0xcd, 0x32, 0x2e, 0xa9, 0x69, 0xd4, 0xf9, 0xaf, 0xa6, 0x3f, - 0x5a, 0xa1, 0x67, 0xb0, 0x77, 0xc6, 0x58, 0x7e, 0x92, 0x88, 0x24, 0x8b, 0x4f, 0xb8, 0xc0, 0xbd, - 0x3f, 0x55, 0x75, 0x4b, 0x25, 0xf8, 0xb2, 0x1e, 0xee, 0x29, 0xdd, 0xe9, 0x42, 0x0b, 0x4f, 0x17, - 0x5c, 0xe8, 0xf6, 0x7e, 0xde, 0x4c, 0xdf, 0x6c, 0x6f, 0xfa, 0x60, 0xb5, 0x21, 0xd6, 0xd5, 0x86, - 0x58, 0xd7, 0x1b, 0x02, 0x5e, 0x55, 0x04, 0xbc, 0xab, 0x08, 0xb8, 0xac, 0x08, 0x58, 0x55, 0x04, - 0x7c, 0xaa, 0x08, 0xf8, 0x5c, 0x11, 0xeb, 0xba, 0x22, 0xe0, 0xf5, 0x96, 0x58, 0xab, 0x2d, 0xb1, - 0xae, 0xb6, 0xc4, 0x7a, 0xba, 0xab, 0x1f, 0x5f, 0x1e, 0x86, 0x8e, 0xce, 0x70, 0xf7, 0x6b, 0x00, - 0x00, 0x00, 0xff, 0xff, 0x77, 0x8c, 0xa0, 0x4e, 0xd3, 0x03, 0x00, 0x00, + // 548 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x52, 0x41, 0x8b, 0xd3, 0x4c, + 0x18, 0xce, 0x6c, 0xd3, 0x6c, 0x3a, 0xa1, 0xec, 0x32, 0x5f, 0xf9, 0xc8, 0xae, 0x32, 0x2d, 0x0b, + 0x42, 0x4f, 0x29, 0xac, 0x78, 0xf0, 0x20, 0xd2, 0xb2, 0xae, 0x50, 0x04, 0x25, 0x47, 0x11, 0x96, + 0x49, 0x3a, 0x89, 0x71, 0xd3, 0x4c, 0x9c, 0x4c, 0x64, 0x7b, 0xf3, 0x27, 0x78, 0xf4, 0x27, 0xf8, + 0x53, 0xf6, 0x58, 0x6f, 0x8b, 0x48, 0xb5, 0xe9, 0x45, 0x3c, 0xed, 0x4f, 0x90, 0x99, 0x49, 0x74, + 0xd5, 0x83, 0xeb, 0xc1, 0x53, 0xde, 0x67, 0x9e, 0x3c, 0xf3, 0x3e, 0xef, 0x33, 0x2f, 0x74, 0x78, + 0x99, 0xd2, 0xc2, 0xcb, 0x39, 0x13, 0x0c, 0xb5, 0x15, 0xd8, 0xef, 0xc5, 0x2c, 0x66, 0xea, 0x64, + 0x24, 0x2b, 0x4d, 0xee, 0xe3, 0x98, 0xb1, 0x38, 0xa5, 0x23, 0x85, 0x82, 0x32, 0x1a, 0xcd, 0x4a, + 0x4e, 0x44, 0xc2, 0xb2, 0x9a, 0xdf, 0xfb, 0x95, 0x27, 0xd9, 0xa2, 0xa6, 0xee, 0xc6, 0x89, 0x78, + 0x5e, 0x06, 0x5e, 0xc8, 0xe6, 0xa3, 0x90, 0x71, 0x41, 0xcf, 0x72, 0xce, 0x5e, 0xd0, 0x50, 0xd4, + 0x68, 0x94, 0x9f, 0xc6, 0x0d, 0x11, 0xd4, 0x85, 0x96, 0x1e, 0x7c, 0xdc, 0x82, 0x5d, 0xbf, 0x4c, + 0xe9, 0x43, 0xce, 0xca, 0xfc, 0x88, 0x16, 0x21, 0x42, 0xd0, 0xcc, 0xc8, 0x9c, 0xba, 0x60, 0x00, + 0x86, 0x1d, 0x5f, 0xd5, 0xe8, 0x26, 0xec, 0xc8, 0x6f, 0x91, 0x93, 0x90, 0xba, 0x5b, 0x8a, 0xf8, + 0x71, 0x80, 0xee, 0x43, 0x3b, 0xc9, 0x04, 0xe5, 0xaf, 0x48, 0xea, 0xb6, 0x06, 0x60, 0xe8, 0x1c, + 0xee, 0x79, 0xda, 0xac, 0xd7, 0x98, 0xf5, 0x8e, 0xea, 0x61, 0x26, 0xf6, 0xf9, 0xaa, 0x6f, 0xbc, + 0xfd, 0xd4, 0x07, 0xfe, 0x77, 0x11, 0xba, 0x05, 0x75, 0x32, 0xae, 0x39, 0x68, 0x0d, 0x9d, 0xc3, + 0x1d, 0x4f, 0x87, 0x26, 0x7d, 0x49, 0x4b, 0xbe, 0x66, 0xa5, 0xb3, 0xb2, 0xa0, 0xdc, 0xb5, 0xb4, + 0x33, 0x59, 0x23, 0x0f, 0x6e, 0xb3, 0x5c, 0x5e, 0x5c, 0xb8, 0x1d, 0x25, 0xee, 0xfd, 0xd6, 0x7a, + 0x9c, 0x2d, 0xfc, 0xe6, 0x27, 0xd4, 0x83, 0xed, 0x34, 0x99, 0x27, 0xc2, 0x85, 0x03, 0x30, 0x6c, + 0xf9, 0x1a, 0xa0, 0x07, 0xd0, 0x79, 0x59, 0x52, 0xbe, 0x78, 0x1c, 0x45, 0x05, 0x15, 0xae, 0x73, + 0x9d, 0x21, 0x80, 0x1a, 0xe2, 0xaa, 0x6e, 0x6a, 0xda, 0xed, 0x5d, 0x6b, 0x6a, 0xda, 0xdb, 0xbb, + 0xf6, 0xd4, 0xb4, 0xed, 0xdd, 0xce, 0xc1, 0xfb, 0x16, 0xb4, 0x9b, 0x31, 0xa4, 0x7f, 0xf9, 0x32, + 0x4d, 0xb2, 0xb2, 0x46, 0xff, 0x43, 0x8b, 0xd3, 0x90, 0xf1, 0x59, 0x1d, 0x6b, 0x8d, 0xa4, 0x4f, + 0x92, 0x52, 0x2e, 0x54, 0xa0, 0x1d, 0x5f, 0x03, 0x74, 0x07, 0xb6, 0x22, 0xc6, 0x5d, 0xf3, 0xfa, + 0x21, 0xcb, 0xff, 0x51, 0x06, 0xad, 0x94, 0x04, 0x34, 0x2d, 0xdc, 0xb6, 0xca, 0xe8, 0x3f, 0xaf, + 0x59, 0x06, 0xef, 0x91, 0x3c, 0x7f, 0x42, 0x12, 0x3e, 0x19, 0x4b, 0xcd, 0x87, 0x55, 0xff, 0xaf, + 0x96, 0x49, 0xeb, 0xc7, 0x33, 0x92, 0x0b, 0xca, 0xfd, 0xba, 0x0b, 0x3a, 0x83, 0x0e, 0xc9, 0x32, + 0x26, 0x88, 0x7e, 0x18, 0xeb, 0x9f, 0x36, 0xbd, 0xda, 0x0a, 0x3d, 0x83, 0xdd, 0x53, 0x4a, 0xf3, + 0xe3, 0x84, 0x27, 0x59, 0x7c, 0xcc, 0xb8, 0xdb, 0xfd, 0x53, 0x54, 0x37, 0xa4, 0x83, 0xaf, 0xab, + 0xfe, 0x8e, 0xd4, 0x9d, 0x44, 0x4a, 0x78, 0x12, 0x31, 0xae, 0xd2, 0xfb, 0xf9, 0x32, 0xf5, 0xb2, + 0xdd, 0xc9, 0xbd, 0xe5, 0x1a, 0x1b, 0x17, 0x6b, 0x6c, 0x5c, 0xae, 0x31, 0x78, 0x5d, 0x61, 0xf0, + 0xae, 0xc2, 0xe0, 0xbc, 0xc2, 0x60, 0x59, 0x61, 0xf0, 0xb9, 0xc2, 0xe0, 0x4b, 0x85, 0x8d, 0xcb, + 0x0a, 0x83, 0x37, 0x1b, 0x6c, 0x2c, 0x37, 0xd8, 0xb8, 0xd8, 0x60, 0xe3, 0xe9, 0xb6, 0xda, 0xe1, + 0x3c, 0x08, 0x2c, 0xe5, 0xe1, 0xf6, 0xb7, 0x00, 0x00, 0x00, 0xff, 0xff, 0x9a, 0x1c, 0xe8, 0x17, + 0x1a, 0x04, 0x00, 0x00, } func (this *RuleGroupDesc) Equal(that interface{}) bool { @@ -300,6 +310,15 @@ func (this *RuleGroupDesc) Equal(that interface{}) bool { if this.Limit != that1.Limit { return false } + if this.QueryOffset != nil && that1.QueryOffset != nil { + if *this.QueryOffset != *that1.QueryOffset { + return false + } + } else if this.QueryOffset != nil { + return false + } else if that1.QueryOffset != nil { + return false + } return true } func (this *RuleDesc) Equal(that interface{}) bool { @@ -358,7 +377,7 @@ func (this *RuleGroupDesc) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 11) + s := make([]string, 0, 12) s = append(s, "&rulespb.RuleGroupDesc{") s = append(s, "Name: "+fmt.Sprintf("%#v", this.Name)+",\n") s = append(s, "Namespace: "+fmt.Sprintf("%#v", this.Namespace)+",\n") @@ -371,6 +390,7 @@ func (this *RuleGroupDesc) GoString() string { s = append(s, "Options: "+fmt.Sprintf("%#v", this.Options)+",\n") } s = append(s, "Limit: "+fmt.Sprintf("%#v", this.Limit)+",\n") + s = append(s, "QueryOffset: "+fmt.Sprintf("%#v", this.QueryOffset)+",\n") s = append(s, "}") return strings.Join(s, "") } @@ -418,6 +438,16 @@ func (m *RuleGroupDesc) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.QueryOffset != nil { + n1, err1 := github_com_gogo_protobuf_types.StdDurationMarshalTo(*m.QueryOffset, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(*m.QueryOffset):]) + if err1 != nil { + return 0, err1 + } + i -= n1 + i = encodeVarintRules(dAtA, i, uint64(n1)) + i-- + dAtA[i] = 0x5a + } if m.Limit != 0 { i = encodeVarintRules(dAtA, i, uint64(m.Limit)) i-- @@ -458,12 +488,12 @@ func (m *RuleGroupDesc) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0x22 } } - n1, err1 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.Interval, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.Interval):]) - if err1 != nil { - return 0, err1 + n2, err2 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.Interval, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.Interval):]) + if err2 != nil { + return 0, err2 } - i -= n1 - i = encodeVarintRules(dAtA, i, uint64(n1)) + i -= n2 + i = encodeVarintRules(dAtA, i, uint64(n2)) i-- dAtA[i] = 0x1a if len(m.Namespace) > 0 { @@ -503,12 +533,12 @@ func (m *RuleDesc) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - n2, err2 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.KeepFiringFor, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.KeepFiringFor):]) - if err2 != nil { - return 0, err2 + n3, err3 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.KeepFiringFor, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.KeepFiringFor):]) + if err3 != nil { + return 0, err3 } - i -= n2 - i = encodeVarintRules(dAtA, i, uint64(n2)) + i -= n3 + i = encodeVarintRules(dAtA, i, uint64(n3)) i-- dAtA[i] = 0x6a if len(m.Annotations) > 0 { @@ -539,12 +569,12 @@ func (m *RuleDesc) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0x2a } } - n3, err3 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.For, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.For):]) - if err3 != nil { - return 0, err3 + n4, err4 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.For, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.For):]) + if err4 != nil { + return 0, err4 } - i -= n3 - i = encodeVarintRules(dAtA, i, uint64(n3)) + i -= n4 + i = encodeVarintRules(dAtA, i, uint64(n4)) i-- dAtA[i] = 0x22 if len(m.Alert) > 0 { @@ -617,6 +647,10 @@ func (m *RuleGroupDesc) Size() (n int) { if m.Limit != 0 { n += 1 + sovRules(uint64(m.Limit)) } + if m.QueryOffset != nil { + l = github_com_gogo_protobuf_types.SizeOfStdDuration(*m.QueryOffset) + n += 1 + l + sovRules(uint64(l)) + } return n } @@ -685,6 +719,7 @@ func (this *RuleGroupDesc) String() string { `User:` + fmt.Sprintf("%v", this.User) + `,`, `Options:` + repeatedStringForOptions + `,`, `Limit:` + fmt.Sprintf("%v", this.Limit) + `,`, + `QueryOffset:` + strings.Replace(fmt.Sprintf("%v", this.QueryOffset), "Duration", "duration.Duration", 1) + `,`, `}`, }, "") return s @@ -958,6 +993,42 @@ func (m *RuleGroupDesc) Unmarshal(dAtA []byte) error { break } } + case 11: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field QueryOffset", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRules + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthRules + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthRules + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.QueryOffset == nil { + m.QueryOffset = new(time.Duration) + } + if err := github_com_gogo_protobuf_types.StdDurationUnmarshal(m.QueryOffset, dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipRules(dAtA[iNdEx:]) diff --git a/pkg/ruler/rulespb/rules.proto b/pkg/ruler/rulespb/rules.proto index 1a6f4182ec..ed4e98a76b 100644 --- a/pkg/ruler/rulespb/rules.proto +++ b/pkg/ruler/rulespb/rules.proto @@ -28,6 +28,8 @@ message RuleGroupDesc { // to the Prometheus Manager. repeated google.protobuf.Any options = 9; int64 limit =10; + google.protobuf.Duration queryOffset = 11 + [(gogoproto.nullable) = true, (gogoproto.stdduration) = true]; } // RuleDesc is a proto representation of a Prometheus Rule @@ -46,4 +48,4 @@ message RuleDesc { (gogoproto.customtype) = "github.com/cortexproject/cortex/pkg/cortexpb.LabelAdapter" ]; google.protobuf.Duration keepFiringFor = 13 [(gogoproto.nullable) = false,(gogoproto.stdduration) = true, (gogoproto.jsontag) = "keep_firing_for"]; -} \ No newline at end of file +} diff --git a/pkg/ruler/store_mock_test.go b/pkg/ruler/store_mock_test.go index da3707e840..30a53fdab1 100644 --- a/pkg/ruler/store_mock_test.go +++ b/pkg/ruler/store_mock_test.go @@ -132,6 +132,43 @@ var ( }, }, } + queryOffset = 2 * time.Minute + mockRulesQueryOffset = map[string]rulespb.RuleGroupList{ + "user1": { + &rulespb.RuleGroupDesc{ + Name: "group1", + Namespace: "namespace1", + User: "user1", + Limit: 5, + Rules: []*rulespb.RuleDesc{ + { + Record: "UP_RULE", + Expr: "up", + }, + { + Alert: "UP_ALERT", + Expr: "up < 1", + }, + }, + Interval: interval, + }, + }, + "user2": { + &rulespb.RuleGroupDesc{ + Name: "group1", + Namespace: "namespace1", + User: "user2", + Rules: []*rulespb.RuleDesc{ + { + Record: "UP_RULE", + Expr: "up", + }, + }, + Interval: interval, + QueryOffset: &queryOffset, + }, + }, + } ) func newMockRuleStore(rules map[string]rulespb.RuleGroupList, errorMap map[string]error) *mockRuleStore { diff --git a/pkg/util/validation/limits.go b/pkg/util/validation/limits.go index fc9faab078..2e885a6c40 100644 --- a/pkg/util/validation/limits.go +++ b/pkg/util/validation/limits.go @@ -178,6 +178,7 @@ type Limits struct { RulerTenantShardSize int `yaml:"ruler_tenant_shard_size" json:"ruler_tenant_shard_size"` RulerMaxRulesPerRuleGroup int `yaml:"ruler_max_rules_per_rule_group" json:"ruler_max_rules_per_rule_group"` RulerMaxRuleGroupsPerTenant int `yaml:"ruler_max_rule_groups_per_tenant" json:"ruler_max_rule_groups_per_tenant"` + RulerQueryOffset model.Duration `yaml:"ruler_query_offset" json:"ruler_query_offset"` // Store-gateway. StoreGatewayTenantShardSize float64 `yaml:"store_gateway_tenant_shard_size" json:"store_gateway_tenant_shard_size"` @@ -268,6 +269,7 @@ func (l *Limits) RegisterFlags(f *flag.FlagSet) { f.IntVar(&l.RulerTenantShardSize, "ruler.tenant-shard-size", 0, "The default tenant's shard size when the shuffle-sharding strategy is used by ruler. When this setting is specified in the per-tenant overrides, a value of 0 disables shuffle sharding for the tenant.") f.IntVar(&l.RulerMaxRulesPerRuleGroup, "ruler.max-rules-per-rule-group", 0, "Maximum number of rules per rule group per-tenant. 0 to disable.") f.IntVar(&l.RulerMaxRuleGroupsPerTenant, "ruler.max-rule-groups-per-tenant", 0, "Maximum number of rule groups per-tenant. 0 to disable.") + f.Var(&l.RulerQueryOffset, "ruler.query-offset", "Duration to offset all rule evaluation queries per-tenant.") f.Var(&l.CompactorBlocksRetentionPeriod, "compactor.blocks-retention-period", "Delete blocks containing samples older than the specified retention period. 0 to disable.") f.IntVar(&l.CompactorTenantShardSize, "compactor.tenant-shard-size", 0, "The default tenant's shard size when the shuffle-sharding strategy is used by the compactor. When this setting is specified in the per-tenant overrides, a value of 0 disables shuffle sharding for the tenant.") @@ -791,6 +793,11 @@ func (o *Overrides) RulerMaxRuleGroupsPerTenant(userID string) int { return o.GetOverridesForUser(userID).RulerMaxRuleGroupsPerTenant } +// RulerQueryOffset returns the rule query offset for a given user. +func (o *Overrides) RulerQueryOffset(userID string) time.Duration { + return time.Duration(o.GetOverridesForUser(userID).RulerQueryOffset) +} + // StoreGatewayTenantShardSize returns the store-gateway shard size for a given user. func (o *Overrides) StoreGatewayTenantShardSize(userID string) float64 { return o.GetOverridesForUser(userID).StoreGatewayTenantShardSize From e5f47e1eaf163a7d250c0b45f463dbc5a3d0e1a5 Mon Sep 17 00:00:00 2001 From: Ben Ye Date: Tue, 30 Jul 2024 16:05:10 -0700 Subject: [PATCH 24/32] Add missing changelogs for native histogram (#6130) --- CHANGELOG.md | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index eb6e4bad0c..b437691982 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -5,7 +5,10 @@ * [CHANGE] Ingester: Remove `-querier.query-store-for-labels-enabled` flag. Querying long-term store for labels is always enabled. #5984 * [CHANGE] Server: Instrument `cortex_request_duration_seconds` metric with native histogram. If `native-histograms` feature is enabled in monitoring Prometheus then the metric name needs to be updated in your dashboards. #6056 * [CHANGE] Distributor/Ingester: Change `cortex_distributor_ingester_appends_total`, `cortex_distributor_ingester_append_failures_total`, `cortex_distributor_ingester_queries_total`, and `cortex_distributor_ingester_query_failures_total` metrics to use the ingester ID instead of its IP as the label value. #6078 -* [FEATURE] Ingester: Experimental: Enable native histogram ingestion via `-blocks-storage.tsdb.enable-native-histograms` flag. #5986 +* [FEATURE] Ingester/Distributor: Experimental: Enable native histogram ingestion via `-blocks-storage.tsdb.enable-native-histograms` flag. #5986 #6010 #6020 +* [FEATURE] Querier: Enable querying native histogram chunks. #5944 #6031 +* [FEATURE] Query Frontend: Support native histogram in query frontend response. #5996 #6043 +* [FEATURE] Ruler: Support sending native histogram samples to Ingester. #6029 * [FEATURE] Ruler: Add support for filtering out alerts in ListRules API. #6011 * [FEATURE] Query Frontend: Added a query rejection mechanism to block resource-intensive queries. #6005 * [FEATURE] OTLP: Support ingesting OTLP exponential metrics as native histograms. #6071 From 4539330d0dbb0b72e168e9d3173a09e820296dbe Mon Sep 17 00:00:00 2001 From: Alex Le Date: Wed, 31 Jul 2024 12:06:16 -0700 Subject: [PATCH 25/32] Introduce cleaner visit marker (#6113) --- CHANGELOG.md | 1 + docs/blocks-storage/compactor.md | 11 + docs/configuration/config-file-reference.md | 10 + pkg/compactor/blocks_cleaner.go | 50 +++- pkg/compactor/blocks_cleaner_test.go | 36 ++- pkg/compactor/cleaner_visit_marker.go | 66 ++++++ pkg/compactor/compactor.go | 29 ++- pkg/compactor/compactor_test.go | 49 +++- pkg/compactor/visit_marker.go | 155 +++++++++++++ pkg/compactor/visit_marker_test.go | 238 ++++++++++++++++++++ 10 files changed, 623 insertions(+), 22 deletions(-) create mode 100644 pkg/compactor/cleaner_visit_marker.go create mode 100644 pkg/compactor/visit_marker.go create mode 100644 pkg/compactor/visit_marker_test.go diff --git a/CHANGELOG.md b/CHANGELOG.md index b437691982..db2a0c1cf3 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -38,6 +38,7 @@ * [ENHANCEMENT] Compactor: Differentiate retry and halt error and retry failed compaction only on retriable error. #6111 * [ENHANCEMENT] Ruler: Add support for filtering by `state` and `health` field on Rules API. #6040 * [ENHANCEMENT] Compactor: Split cleaner cycle for active and deleted tenants. #6112 +* [ENHANCEMENT] Compactor: Introduce cleaner visit marker. #6113 * [BUGFIX] Configsdb: Fix endline issue in db password. #5920 * [BUGFIX] Ingester: Fix `user` and `type` labels for the `cortex_ingester_tsdb_head_samples_appended_total` TSDB metric. #5952 * [BUGFIX] Querier: Enforce max query length check for `/api/v1/series` API even though `ignoreMaxQueryLength` is set to true. #6018 diff --git a/docs/blocks-storage/compactor.md b/docs/blocks-storage/compactor.md index 5fb44208ec..04a08cfa64 100644 --- a/docs/blocks-storage/compactor.md +++ b/docs/blocks-storage/compactor.md @@ -295,6 +295,17 @@ compactor: # CLI flag: -compactor.block-visit-marker-file-update-interval [block_visit_marker_file_update_interval: | default = 1m] + # How long cleaner visit marker file should be considered as expired and able + # to be picked up by cleaner again. The value should be smaller than + # -compactor.cleanup-interval + # CLI flag: -compactor.cleaner-visit-marker-timeout + [cleaner_visit_marker_timeout: | default = 10m] + + # How frequently cleaner visit marker file should be updated when cleaning + # user. + # CLI flag: -compactor.cleaner-visit-marker-file-update-interval + [cleaner_visit_marker_file_update_interval: | default = 5m] + # When enabled, index verification will ignore out of order label names. # CLI flag: -compactor.accept-malformed-index [accept_malformed_index: | default = false] diff --git a/docs/configuration/config-file-reference.md b/docs/configuration/config-file-reference.md index 0e646f7788..745b612fe7 100644 --- a/docs/configuration/config-file-reference.md +++ b/docs/configuration/config-file-reference.md @@ -2232,6 +2232,16 @@ sharding_ring: # CLI flag: -compactor.block-visit-marker-file-update-interval [block_visit_marker_file_update_interval: | default = 1m] +# How long cleaner visit marker file should be considered as expired and able to +# be picked up by cleaner again. The value should be smaller than +# -compactor.cleanup-interval +# CLI flag: -compactor.cleaner-visit-marker-timeout +[cleaner_visit_marker_timeout: | default = 10m] + +# How frequently cleaner visit marker file should be updated when cleaning user. +# CLI flag: -compactor.cleaner-visit-marker-file-update-interval +[cleaner_visit_marker_file_update_interval: | default = 5m] + # When enabled, index verification will ignore out of order label names. # CLI flag: -compactor.accept-malformed-index [accept_malformed_index: | default = false] diff --git a/pkg/compactor/blocks_cleaner.go b/pkg/compactor/blocks_cleaner.go index a447231b64..3ea46a5f38 100644 --- a/pkg/compactor/blocks_cleaner.go +++ b/pkg/compactor/blocks_cleaner.go @@ -50,9 +50,14 @@ type BlocksCleaner struct { bucketClient objstore.InstrumentedBucket usersScanner *cortex_tsdb.UsersScanner + ringLifecyclerID string + // Keep track of the last owned users. lastOwnedUsers []string + cleanerVisitMarkerTimeout time.Duration + cleanerVisitMarkerFileUpdateInterval time.Duration + // Metrics. runsStarted *prometheus.CounterVec runsCompleted *prometheus.CounterVec @@ -76,15 +81,21 @@ func NewBlocksCleaner( usersScanner *cortex_tsdb.UsersScanner, cfgProvider ConfigProvider, logger log.Logger, + ringLifecyclerID string, reg prometheus.Registerer, + cleanerVisitMarkerTimeout time.Duration, + cleanerVisitMarkerFileUpdateInterval time.Duration, blocksMarkedForDeletion *prometheus.CounterVec, ) *BlocksCleaner { c := &BlocksCleaner{ - cfg: cfg, - bucketClient: bucketClient, - usersScanner: usersScanner, - cfgProvider: cfgProvider, - logger: log.With(logger, "component", "cleaner"), + cfg: cfg, + bucketClient: bucketClient, + usersScanner: usersScanner, + cfgProvider: cfgProvider, + logger: log.With(logger, "component", "cleaner"), + ringLifecyclerID: ringLifecyclerID, + cleanerVisitMarkerTimeout: cleanerVisitMarkerTimeout, + cleanerVisitMarkerFileUpdateInterval: cleanerVisitMarkerFileUpdateInterval, runsStarted: promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ Name: "cortex_compactor_block_cleanup_started_total", Help: "Total number of blocks cleanup runs started.", @@ -246,7 +257,15 @@ func (c *BlocksCleaner) cleanUpActiveUsers(ctx context.Context, users []string, return concurrency.ForEachUser(ctx, users, c.cfg.CleanupConcurrency, func(ctx context.Context, userID string) error { userLogger := util_log.WithUserID(userID, c.logger) userBucket := bucket.NewUserBucketClient(userID, c.bucketClient, c.cfgProvider) + visitMarkerManager, isVisited, err := c.obtainVisitMarkerManager(ctx, userLogger, userBucket) + if err != nil { + return err + } + if isVisited { + return nil + } errChan := make(chan error, 1) + go visitMarkerManager.HeartBeat(ctx, errChan, c.cleanerVisitMarkerFileUpdateInterval, true) defer func() { errChan <- nil }() @@ -273,7 +292,15 @@ func (c *BlocksCleaner) cleanDeletedUsers(ctx context.Context, users []string) e return concurrency.ForEachUser(ctx, users, c.cfg.CleanupConcurrency, func(ctx context.Context, userID string) error { userLogger := util_log.WithUserID(userID, c.logger) userBucket := bucket.NewUserBucketClient(userID, c.bucketClient, c.cfgProvider) + visitMarkerManager, isVisited, err := c.obtainVisitMarkerManager(ctx, userLogger, userBucket) + if err != nil { + return err + } + if isVisited { + return nil + } errChan := make(chan error, 1) + go visitMarkerManager.HeartBeat(ctx, errChan, c.cleanerVisitMarkerFileUpdateInterval, true) defer func() { errChan <- nil }() @@ -307,6 +334,19 @@ func (c *BlocksCleaner) scanUsers(ctx context.Context) ([]string, []string, erro return users, deleted, nil } +func (c *BlocksCleaner) obtainVisitMarkerManager(ctx context.Context, userLogger log.Logger, userBucket objstore.InstrumentedBucket) (visitMarkerManager *VisitMarkerManager, isVisited bool, err error) { + cleanerVisitMarker := NewCleanerVisitMarker(c.ringLifecyclerID) + visitMarkerManager = NewVisitMarkerManager(userBucket, userLogger, c.ringLifecyclerID, cleanerVisitMarker) + + existingCleanerVisitMarker := &CleanerVisitMarker{} + err = visitMarkerManager.ReadVisitMarker(ctx, existingCleanerVisitMarker) + if err != nil && !errors.Is(err, errorVisitMarkerNotFound) { + return nil, false, errors.Wrapf(err, "failed to read cleaner visit marker") + } + isVisited = !errors.Is(err, errorVisitMarkerNotFound) && existingCleanerVisitMarker.IsVisited(c.cleanerVisitMarkerTimeout) + return visitMarkerManager, isVisited, nil +} + // Remove blocks and remaining data for tenant marked for deletion. func (c *BlocksCleaner) deleteUserMarkedForDeletion(ctx context.Context, userLogger log.Logger, userBucket objstore.InstrumentedBucket, userID string) error { diff --git a/pkg/compactor/blocks_cleaner_test.go b/pkg/compactor/blocks_cleaner_test.go index 584b3984dc..d3c7aa6da9 100644 --- a/pkg/compactor/blocks_cleaner_test.go +++ b/pkg/compactor/blocks_cleaner_test.go @@ -87,7 +87,7 @@ func TestBlockCleaner_KeyPermissionDenied(t *testing.T) { Help: blocksMarkedForDeletionHelp, }, append(commonLabels, reasonLabelName)) - cleaner := NewBlocksCleaner(cfg, mbucket, scanner, cfgProvider, logger, nil, blocksMarkedForDeletion) + cleaner := NewBlocksCleaner(cfg, mbucket, scanner, cfgProvider, logger, "test-cleaner", nil, time.Minute, 30*time.Second, blocksMarkedForDeletion) // Clean User with no error cleaner.bucketClient = bkt @@ -194,7 +194,7 @@ func testBlocksCleanerWithOptions(t *testing.T, options testBlocksCleanerOptions Help: blocksMarkedForDeletionHelp, }, append(commonLabels, reasonLabelName)) - cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, reg, blocksMarkedForDeletion) + cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, "test-cleaner", reg, time.Minute, 30*time.Second, blocksMarkedForDeletion) require.NoError(t, services.StartAndAwaitRunning(ctx, cleaner)) defer services.StopAndAwaitTerminated(ctx, cleaner) //nolint:errcheck @@ -355,7 +355,7 @@ func TestBlocksCleaner_ShouldContinueOnBlockDeletionFailure(t *testing.T) { Help: blocksMarkedForDeletionHelp, }, append(commonLabels, reasonLabelName)) - cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, nil, blocksMarkedForDeletion) + cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, "test-cleaner", nil, time.Minute, 30*time.Second, blocksMarkedForDeletion) require.NoError(t, services.StartAndAwaitRunning(ctx, cleaner)) defer services.StopAndAwaitTerminated(ctx, cleaner) //nolint:errcheck @@ -419,7 +419,7 @@ func TestBlocksCleaner_ShouldRebuildBucketIndexOnCorruptedOne(t *testing.T) { Help: blocksMarkedForDeletionHelp, }, append(commonLabels, reasonLabelName)) - cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, nil, blocksMarkedForDeletion) + cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, "test-cleaner", nil, time.Minute, 30*time.Second, blocksMarkedForDeletion) require.NoError(t, services.StartAndAwaitRunning(ctx, cleaner)) defer services.StopAndAwaitTerminated(ctx, cleaner) //nolint:errcheck @@ -477,7 +477,7 @@ func TestBlocksCleaner_ShouldRemoveMetricsForTenantsNotBelongingAnymoreToTheShar Help: blocksMarkedForDeletionHelp, }, append(commonLabels, reasonLabelName)) - cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, reg, blocksMarkedForDeletion) + cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, "test-cleaner", reg, time.Minute, 30*time.Second, blocksMarkedForDeletion) activeUsers, deleteUsers, err := cleaner.scanUsers(ctx) require.NoError(t, err) require.NoError(t, cleaner.cleanUpActiveUsers(ctx, activeUsers, true)) @@ -618,7 +618,7 @@ func TestBlocksCleaner_ShouldRemoveBlocksOutsideRetentionPeriod(t *testing.T) { Help: blocksMarkedForDeletionHelp, }, append(commonLabels, reasonLabelName)) - cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, reg, blocksMarkedForDeletion) + cleaner := NewBlocksCleaner(cfg, bucketClient, scanner, cfgProvider, logger, "test-cleaner", reg, time.Minute, 30*time.Second, blocksMarkedForDeletion) assertBlockExists := func(user string, block ulid.ULID, expectExists bool) { exists, err := bucketClient.Exists(ctx, path.Join(user, block.String(), metadata.MetaFilename)) @@ -628,6 +628,10 @@ func TestBlocksCleaner_ShouldRemoveBlocksOutsideRetentionPeriod(t *testing.T) { // Existing behaviour - retention period disabled. { + // clean up cleaner visit marker before running test + bucketClient.Delete(ctx, path.Join("user-1", GetCleanerVisitMarkerFilePath())) //nolint:errcheck + bucketClient.Delete(ctx, path.Join("user-2", GetCleanerVisitMarkerFilePath())) //nolint:errcheck + cfgProvider.userRetentionPeriods["user-1"] = 0 cfgProvider.userRetentionPeriods["user-2"] = 0 @@ -662,6 +666,10 @@ func TestBlocksCleaner_ShouldRemoveBlocksOutsideRetentionPeriod(t *testing.T) { // Retention enabled only for a single user, but does nothing. { + // clean up cleaner visit marker before running test + bucketClient.Delete(ctx, path.Join("user-1", GetCleanerVisitMarkerFilePath())) //nolint:errcheck + bucketClient.Delete(ctx, path.Join("user-2", GetCleanerVisitMarkerFilePath())) //nolint:errcheck + cfgProvider.userRetentionPeriods["user-1"] = 9 * time.Hour activeUsers, deleteUsers, err := cleaner.scanUsers(ctx) @@ -677,6 +685,10 @@ func TestBlocksCleaner_ShouldRemoveBlocksOutsideRetentionPeriod(t *testing.T) { // Retention enabled only for a single user, marking a single block. // Note the block won't be deleted yet due to deletion delay. { + // clean up cleaner visit marker before running test + bucketClient.Delete(ctx, path.Join("user-1", GetCleanerVisitMarkerFilePath())) //nolint:errcheck + bucketClient.Delete(ctx, path.Join("user-2", GetCleanerVisitMarkerFilePath())) //nolint:errcheck + cfgProvider.userRetentionPeriods["user-1"] = 7 * time.Hour activeUsers, deleteUsers, err := cleaner.scanUsers(ctx) @@ -710,6 +722,10 @@ func TestBlocksCleaner_ShouldRemoveBlocksOutsideRetentionPeriod(t *testing.T) { // Marking the block again, before the deletion occurs, should not cause an error. { + // clean up cleaner visit marker before running test + bucketClient.Delete(ctx, path.Join("user-1", GetCleanerVisitMarkerFilePath())) //nolint:errcheck + bucketClient.Delete(ctx, path.Join("user-2", GetCleanerVisitMarkerFilePath())) //nolint:errcheck + activeUsers, deleteUsers, err := cleaner.scanUsers(ctx) require.NoError(t, err) require.NoError(t, cleaner.cleanUpActiveUsers(ctx, activeUsers, false)) @@ -722,6 +738,10 @@ func TestBlocksCleaner_ShouldRemoveBlocksOutsideRetentionPeriod(t *testing.T) { // Reduce the deletion delay. Now the block will be deleted. { + // clean up cleaner visit marker before running test + bucketClient.Delete(ctx, path.Join("user-1", GetCleanerVisitMarkerFilePath())) //nolint:errcheck + bucketClient.Delete(ctx, path.Join("user-2", GetCleanerVisitMarkerFilePath())) //nolint:errcheck + cleaner.cfg.DeletionDelay = 0 activeUsers, deleteUsers, err := cleaner.scanUsers(ctx) @@ -755,6 +775,10 @@ func TestBlocksCleaner_ShouldRemoveBlocksOutsideRetentionPeriod(t *testing.T) { // Retention enabled for other user; test deleting multiple blocks. { + // clean up cleaner visit marker before running test + bucketClient.Delete(ctx, path.Join("user-1", GetCleanerVisitMarkerFilePath())) //nolint:errcheck + bucketClient.Delete(ctx, path.Join("user-2", GetCleanerVisitMarkerFilePath())) //nolint:errcheck + cfgProvider.userRetentionPeriods["user-2"] = 5 * time.Hour activeUsers, deleteUsers, err := cleaner.scanUsers(ctx) diff --git a/pkg/compactor/cleaner_visit_marker.go b/pkg/compactor/cleaner_visit_marker.go new file mode 100644 index 0000000000..b31e881066 --- /dev/null +++ b/pkg/compactor/cleaner_visit_marker.go @@ -0,0 +1,66 @@ +package compactor + +import ( + "fmt" + "path" + "time" + + "github.com/cortexproject/cortex/pkg/storage/tsdb/bucketindex" +) + +const ( + // CleanerVisitMarkerName is the name of cleaner visit marker file. + CleanerVisitMarkerName = "cleaner-visit-marker.json" + // CleanerVisitMarkerVersion1 is the current supported version of cleaner visit mark file. + CleanerVisitMarkerVersion1 = 1 +) + +type CleanerVisitMarker struct { + CompactorID string `json:"compactorID"` + Status VisitStatus `json:"status"` + // VisitTime is a unix timestamp of when the partition was visited (mark updated). + VisitTime int64 `json:"visitTime"` + // Version of the file. + Version int `json:"version"` +} + +func NewCleanerVisitMarker(compactorID string) *CleanerVisitMarker { + return &CleanerVisitMarker{ + CompactorID: compactorID, + Version: CleanerVisitMarkerVersion1, + } +} + +func (b *CleanerVisitMarker) IsExpired(cleanerVisitMarkerTimeout time.Duration) bool { + return !time.Now().Before(time.Unix(b.VisitTime, 0).Add(cleanerVisitMarkerTimeout)) +} + +func (b *CleanerVisitMarker) IsVisited(cleanerVisitMarkerTimeout time.Duration) bool { + return !(b.GetStatus() == Completed) && !(b.GetStatus() == Failed) && !b.IsExpired(cleanerVisitMarkerTimeout) +} + +func (b *CleanerVisitMarker) GetStatus() VisitStatus { + return b.Status +} + +func (b *CleanerVisitMarker) GetVisitMarkerFilePath() string { + return GetCleanerVisitMarkerFilePath() +} + +func (b *CleanerVisitMarker) UpdateStatus(ownerIdentifier string, status VisitStatus) { + b.CompactorID = ownerIdentifier + b.Status = status + b.VisitTime = time.Now().Unix() +} + +func (b *CleanerVisitMarker) String() string { + return fmt.Sprintf("compactor_id=%s status=%s visit_time=%s", + b.CompactorID, + b.Status, + time.Unix(b.VisitTime, 0).String(), + ) +} + +func GetCleanerVisitMarkerFilePath() string { + return path.Join(bucketindex.MarkersPathname, CleanerVisitMarkerName) +} diff --git a/pkg/compactor/compactor.go b/pkg/compactor/compactor.go index 0d03348d6b..ff7907fe2c 100644 --- a/pkg/compactor/compactor.go +++ b/pkg/compactor/compactor.go @@ -216,6 +216,10 @@ type Config struct { BlockVisitMarkerTimeout time.Duration `yaml:"block_visit_marker_timeout"` BlockVisitMarkerFileUpdateInterval time.Duration `yaml:"block_visit_marker_file_update_interval"` + // Cleaner visit marker file config + CleanerVisitMarkerTimeout time.Duration `yaml:"cleaner_visit_marker_timeout"` + CleanerVisitMarkerFileUpdateInterval time.Duration `yaml:"cleaner_visit_marker_file_update_interval"` + AcceptMalformedIndex bool `yaml:"accept_malformed_index"` CachingBucketEnabled bool `yaml:"caching_bucket_enabled"` } @@ -255,6 +259,9 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) { f.DurationVar(&cfg.BlockVisitMarkerTimeout, "compactor.block-visit-marker-timeout", 5*time.Minute, "How long block visit marker file should be considered as expired and able to be picked up by compactor again.") f.DurationVar(&cfg.BlockVisitMarkerFileUpdateInterval, "compactor.block-visit-marker-file-update-interval", 1*time.Minute, "How frequently block visit marker file should be updated duration compaction.") + f.DurationVar(&cfg.CleanerVisitMarkerTimeout, "compactor.cleaner-visit-marker-timeout", 10*time.Minute, "How long cleaner visit marker file should be considered as expired and able to be picked up by cleaner again. The value should be smaller than -compactor.cleanup-interval") + f.DurationVar(&cfg.CleanerVisitMarkerFileUpdateInterval, "compactor.cleaner-visit-marker-file-update-interval", 5*time.Minute, "How frequently cleaner visit marker file should be updated when cleaning user.") + f.BoolVar(&cfg.AcceptMalformedIndex, "compactor.accept-malformed-index", false, "When enabled, index verification will ignore out of order label names.") f.BoolVar(&cfg.CachingBucketEnabled, "compactor.caching-bucket-enabled", false, "When enabled, caching bucket will be used for compactor, except cleaner service, which serves as the source of truth for block status") } @@ -522,15 +529,7 @@ func (c *Compactor) starting(ctx context.Context) error { // Create the users scanner. c.usersScanner = cortex_tsdb.NewUsersScanner(c.bucketClient, c.ownUserForCleanUp, c.parentLogger) - // Create the blocks cleaner (service). - c.blocksCleaner = NewBlocksCleaner(BlocksCleanerConfig{ - DeletionDelay: c.compactorCfg.DeletionDelay, - CleanupInterval: util.DurationWithJitter(c.compactorCfg.CleanupInterval, 0.1), - CleanupConcurrency: c.compactorCfg.CleanupConcurrency, - BlockDeletionMarksMigrationEnabled: c.compactorCfg.BlockDeletionMarksMigrationEnabled, - TenantCleanupDelay: c.compactorCfg.TenantCleanupDelay, - }, c.bucketClient, c.usersScanner, c.limits, c.parentLogger, c.registerer, c.compactorMetrics.syncerBlocksMarkedForDeletion) - + var cleanerRingLifecyclerID = "default-cleaner" // Initialize the compactors ring if sharding is enabled. if c.compactorCfg.ShardingEnabled { lifecyclerCfg := c.compactorCfg.ShardingRing.ToLifecyclerConfig() @@ -539,6 +538,8 @@ func (c *Compactor) starting(ctx context.Context) error { return errors.Wrap(err, "unable to initialize compactor ring lifecycler") } + cleanerRingLifecyclerID = c.ringLifecycler.ID + c.ring, err = ring.New(lifecyclerCfg.RingConfig, "compactor", ringKey, c.logger, prometheus.WrapRegistererWithPrefix("cortex_", c.registerer)) if err != nil { return errors.Wrap(err, "unable to initialize compactor ring") @@ -588,6 +589,16 @@ func (c *Compactor) starting(ctx context.Context) error { } } + // Create the blocks cleaner (service). + c.blocksCleaner = NewBlocksCleaner(BlocksCleanerConfig{ + DeletionDelay: c.compactorCfg.DeletionDelay, + CleanupInterval: util.DurationWithJitter(c.compactorCfg.CleanupInterval, 0.1), + CleanupConcurrency: c.compactorCfg.CleanupConcurrency, + BlockDeletionMarksMigrationEnabled: c.compactorCfg.BlockDeletionMarksMigrationEnabled, + TenantCleanupDelay: c.compactorCfg.TenantCleanupDelay, + }, c.bucketClient, c.usersScanner, c.limits, c.parentLogger, cleanerRingLifecyclerID, c.registerer, c.compactorCfg.CleanerVisitMarkerTimeout, c.compactorCfg.CleanerVisitMarkerFileUpdateInterval, + c.compactorMetrics.syncerBlocksMarkedForDeletion) + // Ensure an initial cleanup occurred before starting the compactor. if err := services.StartAndAwaitRunning(ctx, c.blocksCleaner); err != nil { c.ringSubservices.StopAsync() diff --git a/pkg/compactor/compactor_test.go b/pkg/compactor/compactor_test.go index 6f283bdcf7..908f962cf2 100644 --- a/pkg/compactor/compactor_test.go +++ b/pkg/compactor/compactor_test.go @@ -173,6 +173,9 @@ func TestCompactor_SkipCompactionWhenCmkError(t *testing.T) { bucketClient.MockIter("__markers__", []string{}, nil) bucketClient.MockIter(userID+"/", []string{}, nil) bucketClient.MockIter(userID+"/markers/", nil, nil) + bucketClient.MockGet(userID+"/markers/cleaner-visit-marker.json", "", nil) + bucketClient.MockUpload(userID+"/markers/cleaner-visit-marker.json", nil) + bucketClient.MockDelete(userID+"/markers/cleaner-visit-marker.json", nil) bucketClient.MockGet(userID+"/bucket-index-sync-status.json", string(content), nil) bucketClient.MockGet(userID+"/bucket-index.json.gz", "", nil) bucketClient.MockUpload(userID+"/bucket-index-sync-status.json", nil) @@ -239,7 +242,6 @@ func TestCompactor_ShouldDoNothingOnNoUserBlocks(t *testing.T) { # HELP cortex_compactor_blocks_cleaned_total Total number of blocks deleted. # TYPE cortex_compactor_blocks_cleaned_total counter cortex_compactor_blocks_cleaned_total 0 - # HELP cortex_compactor_blocks_marked_for_no_compaction_total Total number of blocks marked for no compact during a compaction run. # TYPE cortex_compactor_blocks_marked_for_no_compaction_total counter cortex_compactor_blocks_marked_for_no_compaction_total 0 @@ -332,7 +334,6 @@ func TestCompactor_ShouldRetryCompactionOnFailureWhileDiscoveringUsersFromBucket # HELP cortex_compactor_blocks_cleaned_total Total number of blocks deleted. # TYPE cortex_compactor_blocks_cleaned_total counter cortex_compactor_blocks_cleaned_total 0 - # HELP cortex_compactor_blocks_marked_for_no_compaction_total Total number of blocks marked for no compact during a compaction run. # TYPE cortex_compactor_blocks_marked_for_no_compaction_total counter cortex_compactor_blocks_marked_for_no_compaction_total 0 @@ -381,6 +382,9 @@ func TestCompactor_ShouldIncrementCompactionErrorIfFailedToCompactASingleTenant( bucketClient.MockIter("__markers__", []string{}, nil) bucketClient.MockIter(userID+"/", []string{userID + "/01DTVP434PA9VFXSW2JKB3392D/meta.json", userID + "/01FN6CDF3PNEWWRY5MPGJPE3EX/meta.json"}, nil) bucketClient.MockIter(userID+"/markers/", nil, nil) + bucketClient.MockGet(userID+"/markers/cleaner-visit-marker.json", "", nil) + bucketClient.MockUpload(userID+"/markers/cleaner-visit-marker.json", nil) + bucketClient.MockDelete(userID+"/markers/cleaner-visit-marker.json", nil) bucketClient.MockExists(cortex_tsdb.GetGlobalDeletionMarkPath(userID), false, nil) bucketClient.MockExists(cortex_tsdb.GetLocalDeletionMarkPath(userID), false, nil) bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) @@ -436,6 +440,9 @@ func TestCompactor_ShouldCompactAndRemoveUserFolder(t *testing.T) { bucketClient.MockExists(cortex_tsdb.GetLocalDeletionMarkPath("user-1"), false, nil) bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", "user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/meta.json"}, nil) bucketClient.MockIter("user-1/markers/", nil, nil) + bucketClient.MockGet("user-1/markers/cleaner-visit-marker.json", "", nil) + bucketClient.MockUpload("user-1/markers/cleaner-visit-marker.json", nil) + bucketClient.MockDelete("user-1/markers/cleaner-visit-marker.json", nil) bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", "", nil) @@ -485,7 +492,13 @@ func TestCompactor_ShouldIterateOverUsersAndRunCompaction(t *testing.T) { bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", "user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/meta.json"}, nil) bucketClient.MockIter("user-2/", []string{"user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", "user-2/01FN3V83ABR9992RF8WRJZ76ZQ/meta.json"}, nil) bucketClient.MockIter("user-1/markers/", nil, nil) + bucketClient.MockGet("user-1/markers/cleaner-visit-marker.json", "", nil) + bucketClient.MockUpload("user-1/markers/cleaner-visit-marker.json", nil) + bucketClient.MockDelete("user-1/markers/cleaner-visit-marker.json", nil) bucketClient.MockIter("user-2/markers/", nil, nil) + bucketClient.MockGet("user-2/markers/cleaner-visit-marker.json", "", nil) + bucketClient.MockUpload("user-2/markers/cleaner-visit-marker.json", nil) + bucketClient.MockDelete("user-2/markers/cleaner-visit-marker.json", nil) bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", "", nil) @@ -641,6 +654,10 @@ func TestCompactor_ShouldNotCompactBlocksMarkedForDeletion(t *testing.T) { "user-1/markers/01DTW0ZCPDDNV4BV83Q2SV4QAZ-deletion-mark.json", }, nil) + bucketClient.MockGet("user-1/markers/cleaner-visit-marker.json", "", nil) + bucketClient.MockUpload("user-1/markers/cleaner-visit-marker.json", nil) + bucketClient.MockDelete("user-1/markers/cleaner-visit-marker.json", nil) + bucketClient.MockDelete("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", nil) bucketClient.MockDelete("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json", nil) bucketClient.MockDelete("user-1/markers/01DTW0ZCPDDNV4BV83Q2SV4QAZ-deletion-mark.json", nil) @@ -740,7 +757,13 @@ func TestCompactor_ShouldNotCompactBlocksMarkedForSkipCompact(t *testing.T) { bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", "user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/meta.json"}, nil) bucketClient.MockIter("user-2/", []string{"user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", "user-2/01FN3V83ABR9992RF8WRJZ76ZQ/meta.json"}, nil) bucketClient.MockIter("user-1/markers/", nil, nil) + bucketClient.MockGet("user-1/markers/cleaner-visit-marker.json", "", nil) + bucketClient.MockUpload("user-1/markers/cleaner-visit-marker.json", nil) + bucketClient.MockDelete("user-1/markers/cleaner-visit-marker.json", nil) bucketClient.MockIter("user-2/markers/", nil, nil) + bucketClient.MockGet("user-2/markers/cleaner-visit-marker.json", "", nil) + bucketClient.MockUpload("user-2/markers/cleaner-visit-marker.json", nil) + bucketClient.MockDelete("user-2/markers/cleaner-visit-marker.json", nil) bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", mockNoCompactBlockJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) @@ -816,6 +839,10 @@ func TestCompactor_ShouldNotCompactBlocksForUsersMarkedForDeletion(t *testing.T) bucketClient.MockGet(cortex_tsdb.GetGlobalDeletionMarkPath("user-1"), `{"deletion_time": 1}`, nil) bucketClient.MockUpload(cortex_tsdb.GetGlobalDeletionMarkPath("user-1"), nil) + bucketClient.MockGet("user-1/markers/cleaner-visit-marker.json", "", nil) + bucketClient.MockUpload("user-1/markers/cleaner-visit-marker.json", nil) + bucketClient.MockDelete("user-1/markers/cleaner-visit-marker.json", nil) + bucketClient.MockIter("user-1/01DTVP434PA9VFXSW2JKB3392D", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", "user-1/01DTVP434PA9VFXSW2JKB3392D/index"}, nil) bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/index", "some index content", nil) @@ -979,7 +1006,13 @@ func TestCompactor_ShouldCompactAllUsersOnShardingEnabledButOnlyOneInstanceRunni bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", "user-1/01FN6CDF3PNEWWRY5MPGJPE3EX/meta.json"}, nil) bucketClient.MockIter("user-2/", []string{"user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", "user-2/01FN3V83ABR9992RF8WRJZ76ZQ/meta.json"}, nil) bucketClient.MockIter("user-1/markers/", nil, nil) + bucketClient.MockGet("user-1/markers/cleaner-visit-marker.json", "", nil) + bucketClient.MockUpload("user-1/markers/cleaner-visit-marker.json", nil) + bucketClient.MockDelete("user-1/markers/cleaner-visit-marker.json", nil) bucketClient.MockIter("user-2/markers/", nil, nil) + bucketClient.MockGet("user-2/markers/cleaner-visit-marker.json", "", nil) + bucketClient.MockUpload("user-2/markers/cleaner-visit-marker.json", nil) + bucketClient.MockDelete("user-2/markers/cleaner-visit-marker.json", nil) bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", "", nil) @@ -1078,6 +1111,9 @@ func TestCompactor_ShouldCompactOnlyUsersOwnedByTheInstanceOnShardingEnabledAndM for _, userID := range userIDs { bucketClient.MockIter(userID+"/", []string{userID + "/01DTVP434PA9VFXSW2JKB3392D"}, nil) bucketClient.MockIter(userID+"/markers/", nil, nil) + bucketClient.MockGet(userID+"/markers/cleaner-visit-marker.json", "", nil) + bucketClient.MockUpload(userID+"/markers/cleaner-visit-marker.json", nil) + bucketClient.MockDelete(userID+"/markers/cleaner-visit-marker.json", nil) bucketClient.MockExists(cortex_tsdb.GetGlobalDeletionMarkPath(userID), false, nil) bucketClient.MockExists(cortex_tsdb.GetLocalDeletionMarkPath(userID), false, nil) bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) @@ -1212,6 +1248,9 @@ func TestCompactor_ShouldCompactOnlyShardsOwnedByTheInstanceOnShardingEnabledWit bucketClient.MockIter(userID+"/", blockFiles, nil) bucketClient.MockIter(userID+"/markers/", nil, nil) + bucketClient.MockGet(userID+"/markers/cleaner-visit-marker.json", "", nil) + bucketClient.MockUpload(userID+"/markers/cleaner-visit-marker.json", nil) + bucketClient.MockDelete(userID+"/markers/cleaner-visit-marker.json", nil) bucketClient.MockExists(cortex_tsdb.GetGlobalDeletionMarkPath(userID), false, nil) bucketClient.MockExists(cortex_tsdb.GetLocalDeletionMarkPath(userID), false, nil) bucketClient.MockGet(userID+"/bucket-index.json.gz", "", nil) @@ -2005,6 +2044,9 @@ func TestCompactor_FailedWithRetriableError(t *testing.T) { bucketClient.MockIter("", []string{"user-1"}, nil) bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D", "user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ", "user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", "user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json"}, nil) bucketClient.MockIter("user-1/markers/", nil, nil) + bucketClient.MockGet("user-1/markers/cleaner-visit-marker.json", "", nil) + bucketClient.MockUpload("user-1/markers/cleaner-visit-marker.json", nil) + bucketClient.MockDelete("user-1/markers/cleaner-visit-marker.json", nil) bucketClient.MockExists(cortex_tsdb.GetGlobalDeletionMarkPath("user-1"), false, nil) bucketClient.MockExists(cortex_tsdb.GetLocalDeletionMarkPath("user-1"), false, nil) bucketClient.MockIter("user-1/01DTVP434PA9VFXSW2JKB3392D", nil, errors.New("test retriable error")) @@ -2056,6 +2098,9 @@ func TestCompactor_FailedWithHaltError(t *testing.T) { bucketClient.MockIter("", []string{"user-1"}, nil) bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D", "user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ", "user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", "user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json"}, nil) bucketClient.MockIter("user-1/markers/", nil, nil) + bucketClient.MockGet("user-1/markers/cleaner-visit-marker.json", "", nil) + bucketClient.MockUpload("user-1/markers/cleaner-visit-marker.json", nil) + bucketClient.MockDelete("user-1/markers/cleaner-visit-marker.json", nil) bucketClient.MockExists(cortex_tsdb.GetGlobalDeletionMarkPath("user-1"), false, nil) bucketClient.MockExists(cortex_tsdb.GetLocalDeletionMarkPath("user-1"), false, nil) bucketClient.MockIter("user-1/01DTVP434PA9VFXSW2JKB3392D", nil, compact.HaltError{}) diff --git a/pkg/compactor/visit_marker.go b/pkg/compactor/visit_marker.go new file mode 100644 index 0000000000..ebe675556d --- /dev/null +++ b/pkg/compactor/visit_marker.go @@ -0,0 +1,155 @@ +package compactor + +import ( + "bytes" + "context" + "encoding/json" + "fmt" + "io" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/pkg/errors" + "github.com/thanos-io/objstore" + "github.com/thanos-io/thanos/pkg/compact" + + "github.com/cortexproject/cortex/pkg/util/runutil" +) + +var ( + errorVisitMarkerNotFound = errors.New("visit marker not found") + errorUnmarshalVisitMarker = errors.New("unmarshal visit marker JSON") +) + +type VisitStatus string + +const ( + Pending VisitStatus = "pending" + InProgress VisitStatus = "inProgress" + Completed VisitStatus = "completed" + Failed VisitStatus = "failed" +) + +type VisitMarker interface { + GetVisitMarkerFilePath() string + UpdateStatus(ownerIdentifier string, status VisitStatus) + GetStatus() VisitStatus + IsExpired(visitMarkerTimeout time.Duration) bool + String() string +} + +type VisitMarkerManager struct { + bkt objstore.InstrumentedBucket + logger log.Logger + ownerIdentifier string + visitMarker VisitMarker +} + +func NewVisitMarkerManager( + bkt objstore.InstrumentedBucket, + logger log.Logger, + ownerIdentifier string, + visitMarker VisitMarker, +) *VisitMarkerManager { + return &VisitMarkerManager{ + bkt: bkt, + logger: log.With(logger, "type", fmt.Sprintf("%T", visitMarker)), + ownerIdentifier: ownerIdentifier, + visitMarker: visitMarker, + } +} + +func (v *VisitMarkerManager) HeartBeat(ctx context.Context, errChan <-chan error, visitMarkerFileUpdateInterval time.Duration, deleteOnExit bool) { + level.Info(v.getLogger()).Log("msg", "start visit marker heart beat") + ticker := time.NewTicker(visitMarkerFileUpdateInterval) + defer ticker.Stop() +heartBeat: + for { + v.MarkWithStatus(ctx, InProgress) + + select { + case <-ctx.Done(): + level.Warn(v.getLogger()).Log("msg", "visit marker heart beat got cancelled") + v.MarkWithStatus(context.Background(), Pending) + break heartBeat + case <-ticker.C: + continue + case err := <-errChan: + if err == nil { + level.Info(v.getLogger()).Log("msg", "update visit marker to completed status") + v.MarkWithStatus(ctx, Completed) + } else { + level.Warn(v.getLogger()).Log("msg", "stop visit marker heart beat due to error", "err", err) + if compact.IsHaltError(err) { + level.Info(v.getLogger()).Log("msg", "update visit marker to failed status", "err", err) + v.MarkWithStatus(ctx, Failed) + } else { + level.Info(v.getLogger()).Log("msg", "update visit marker to pending status", "err", err) + v.MarkWithStatus(ctx, Pending) + } + } + break heartBeat + } + } + level.Info(v.getLogger()).Log("msg", "stop visit marker heart beat") + if deleteOnExit { + level.Info(v.getLogger()).Log("msg", "delete visit marker when exiting heart beat") + v.DeleteVisitMarker(context.Background()) + } +} + +func (v *VisitMarkerManager) MarkWithStatus(ctx context.Context, status VisitStatus) { + v.visitMarker.UpdateStatus(v.ownerIdentifier, status) + if err := v.updateVisitMarker(ctx); err != nil { + level.Error(v.getLogger()).Log("msg", "unable to upsert visit marker file content", "new_status", status, "err", err) + return + } + level.Debug(v.getLogger()).Log("msg", "marked with new status", "new_status", status) +} + +func (v *VisitMarkerManager) DeleteVisitMarker(ctx context.Context) { + if err := v.bkt.Delete(ctx, v.visitMarker.GetVisitMarkerFilePath()); err != nil { + level.Error(v.getLogger()).Log("msg", "failed to delete visit marker", "err", err) + return + } + level.Debug(v.getLogger()).Log("msg", "visit marker deleted") +} + +func (v *VisitMarkerManager) ReadVisitMarker(ctx context.Context, visitMarker any) error { + visitMarkerFile := v.visitMarker.GetVisitMarkerFilePath() + visitMarkerFileReader, err := v.bkt.ReaderWithExpectedErrs(v.bkt.IsObjNotFoundErr).Get(ctx, visitMarkerFile) + if err != nil { + if v.bkt.IsObjNotFoundErr(err) { + return errors.Wrapf(errorVisitMarkerNotFound, "visit marker file: %s", visitMarkerFile) + } + return errors.Wrapf(err, "get visit marker file: %s", visitMarkerFile) + } + defer runutil.CloseWithLogOnErr(v.getLogger(), visitMarkerFileReader, "close visit marker reader") + b, err := io.ReadAll(visitMarkerFileReader) + if err != nil { + return errors.Wrapf(err, "read visit marker file: %s", visitMarkerFile) + } + if err = json.Unmarshal(b, visitMarker); err != nil { + return errors.Wrapf(errorUnmarshalVisitMarker, "visit marker file: %s, content: %s, error: %v", visitMarkerFile, string(b), err.Error()) + } + level.Debug(v.getLogger()).Log("msg", "visit marker read from file", "visit_marker_file", visitMarkerFile) + return nil +} + +func (v *VisitMarkerManager) updateVisitMarker(ctx context.Context) error { + visitMarkerFileContent, err := json.Marshal(v.visitMarker) + if err != nil { + return err + } + + reader := bytes.NewReader(visitMarkerFileContent) + if err := v.bkt.Upload(ctx, v.visitMarker.GetVisitMarkerFilePath(), reader); err != nil { + return err + } + return nil +} + +func (v *VisitMarkerManager) getLogger() log.Logger { + return log.With(v.logger, "visit_marker", v.visitMarker.String()) +} diff --git a/pkg/compactor/visit_marker_test.go b/pkg/compactor/visit_marker_test.go new file mode 100644 index 0000000000..2e3eae5b60 --- /dev/null +++ b/pkg/compactor/visit_marker_test.go @@ -0,0 +1,238 @@ +package compactor + +import ( + "context" + "crypto/rand" + "fmt" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/oklog/ulid" + "github.com/stretchr/testify/require" + "github.com/thanos-io/objstore" + "github.com/thanos-io/thanos/pkg/compact" + + cortex_testutil "github.com/cortexproject/cortex/pkg/storage/tsdb/testutil" +) + +func TestMarkPending(t *testing.T) { + ctx := context.Background() + bkt, _ := cortex_testutil.PrepareFilesystemBucket(t) + logger := log.NewNopLogger() + + ownerIdentifier := "test-owner" + testVisitMarker := NewTestVisitMarker(ownerIdentifier) + + visitMarkerManager := NewVisitMarkerManager(objstore.WithNoopInstr(bkt), logger, ownerIdentifier, testVisitMarker) + visitMarkerManager.MarkWithStatus(ctx, Pending) + + require.Equal(t, Pending, testVisitMarker.Status) + + visitMarkerFromFile := &TestVisitMarker{} + err := visitMarkerManager.ReadVisitMarker(ctx, visitMarkerFromFile) + require.NoError(t, err) + require.Equal(t, Pending, visitMarkerFromFile.Status) +} + +func TestMarkInProgress(t *testing.T) { + ctx := context.Background() + bkt, _ := cortex_testutil.PrepareFilesystemBucket(t) + logger := log.NewNopLogger() + + ownerIdentifier := "test-owner" + testVisitMarker := NewTestVisitMarker(ownerIdentifier) + + visitMarkerManager := NewVisitMarkerManager(objstore.WithNoopInstr(bkt), logger, ownerIdentifier, testVisitMarker) + visitMarkerManager.MarkWithStatus(ctx, InProgress) + + require.Equal(t, InProgress, testVisitMarker.Status) + + visitMarkerFromFile := &TestVisitMarker{} + err := visitMarkerManager.ReadVisitMarker(ctx, visitMarkerFromFile) + require.NoError(t, err) + require.Equal(t, InProgress, visitMarkerFromFile.Status) +} + +func TestMarkCompleted(t *testing.T) { + ctx := context.Background() + bkt, _ := cortex_testutil.PrepareFilesystemBucket(t) + logger := log.NewNopLogger() + + ownerIdentifier := "test-owner" + testVisitMarker := NewTestVisitMarker(ownerIdentifier) + + visitMarkerManager := NewVisitMarkerManager(objstore.WithNoopInstr(bkt), logger, ownerIdentifier, testVisitMarker) + visitMarkerManager.MarkWithStatus(ctx, Completed) + + require.Equal(t, Completed, testVisitMarker.Status) + + visitMarkerFromFile := &TestVisitMarker{} + err := visitMarkerManager.ReadVisitMarker(ctx, visitMarkerFromFile) + require.NoError(t, err) + require.Equal(t, Completed, visitMarkerFromFile.Status) +} + +func TestUpdateExistingVisitMarker(t *testing.T) { + ctx := context.Background() + bkt, _ := cortex_testutil.PrepareFilesystemBucket(t) + logger := log.NewNopLogger() + + ownerIdentifier1 := "test-owner-1" + testVisitMarker1 := NewTestVisitMarker(ownerIdentifier1) + visitMarkerManager1 := NewVisitMarkerManager(objstore.WithNoopInstr(bkt), logger, ownerIdentifier1, testVisitMarker1) + visitMarkerManager1.MarkWithStatus(ctx, InProgress) + + ownerIdentifier2 := "test-owner-2" + testVisitMarker2 := &TestVisitMarker{ + OwnerIdentifier: ownerIdentifier2, + markerID: testVisitMarker1.markerID, + StoredValue: testVisitMarker1.StoredValue, + } + visitMarkerManager2 := NewVisitMarkerManager(objstore.WithNoopInstr(bkt), logger, ownerIdentifier2, testVisitMarker2) + visitMarkerManager2.MarkWithStatus(ctx, Completed) + + visitMarkerFromFile := &TestVisitMarker{} + err := visitMarkerManager2.ReadVisitMarker(ctx, visitMarkerFromFile) + require.NoError(t, err) + require.Equal(t, ownerIdentifier2, visitMarkerFromFile.OwnerIdentifier) + require.Equal(t, Completed, visitMarkerFromFile.Status) +} + +func TestHeartBeat(t *testing.T) { + for _, tcase := range []struct { + name string + isCancelled bool + callerErr error + expectedStatus VisitStatus + deleteOnExit bool + }{ + { + name: "heart beat got cancelled", + isCancelled: true, + callerErr: nil, + expectedStatus: Pending, + deleteOnExit: false, + }, + { + name: "heart beat complete without error", + isCancelled: false, + callerErr: nil, + expectedStatus: Completed, + deleteOnExit: false, + }, + { + name: "heart beat stopped due to halt error", + isCancelled: false, + callerErr: compact.HaltError{}, + expectedStatus: Failed, + deleteOnExit: false, + }, + { + name: "heart beat stopped due to non halt error", + isCancelled: false, + callerErr: fmt.Errorf("some error"), + expectedStatus: Pending, + deleteOnExit: false, + }, + { + name: "heart beat got cancelled and delete visit marker on exit", + isCancelled: true, + callerErr: nil, + expectedStatus: Pending, + deleteOnExit: true, + }, + { + name: "heart beat complete without error and delete visit marker on exit", + isCancelled: false, + callerErr: nil, + expectedStatus: Completed, + deleteOnExit: true, + }, + { + name: "heart beat stopped due to caller error and delete visit marker on exit", + isCancelled: false, + callerErr: fmt.Errorf("some error"), + expectedStatus: Failed, + deleteOnExit: true, + }, + } { + t.Run(tcase.name, func(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + bkt, _ := cortex_testutil.PrepareFilesystemBucket(t) + logger := log.NewNopLogger() + errChan := make(chan error, 1) + + ownerIdentifier := "test-owner" + testVisitMarker := NewTestVisitMarker(ownerIdentifier) + resultTestVisitMarker := CopyTestVisitMarker(testVisitMarker) + visitMarkerManager := NewVisitMarkerManager(objstore.WithNoopInstr(bkt), logger, ownerIdentifier, testVisitMarker) + go visitMarkerManager.HeartBeat(ctx, errChan, time.Second, tcase.deleteOnExit) + + time.Sleep(2 * time.Second) + if tcase.isCancelled { + cancel() + } else { + errChan <- tcase.callerErr + defer cancel() + } + time.Sleep(2 * time.Second) + + if tcase.deleteOnExit { + exists, err := bkt.Exists(context.Background(), testVisitMarker.GetVisitMarkerFilePath()) + require.NoError(t, err) + require.False(t, exists) + } else { + resultVisitMarkerManager := NewVisitMarkerManager(objstore.WithNoopInstr(bkt), logger, ownerIdentifier, resultTestVisitMarker) + err := resultVisitMarkerManager.ReadVisitMarker(context.Background(), resultTestVisitMarker) + require.NoError(t, err) + require.Equal(t, tcase.expectedStatus, resultTestVisitMarker.Status) + } + }) + } +} + +type TestVisitMarker struct { + OwnerIdentifier string `json:"ownerIdentifier"` + Status VisitStatus `json:"status"` + StoredValue string `json:"storedValue"` + + markerID ulid.ULID +} + +func (t *TestVisitMarker) IsExpired(visitMarkerTimeout time.Duration) bool { + return true +} + +func (t *TestVisitMarker) GetStatus() VisitStatus { + return t.Status +} + +func NewTestVisitMarker(ownerIdentifier string) *TestVisitMarker { + return &TestVisitMarker{ + OwnerIdentifier: ownerIdentifier, + markerID: ulid.MustNew(uint64(time.Now().UnixMilli()), rand.Reader), + StoredValue: "initial value", + } +} + +func CopyTestVisitMarker(sourceVisitMarker *TestVisitMarker) *TestVisitMarker { + return &TestVisitMarker{ + OwnerIdentifier: sourceVisitMarker.OwnerIdentifier, + markerID: sourceVisitMarker.markerID, + StoredValue: sourceVisitMarker.StoredValue, + } +} + +func (t *TestVisitMarker) GetVisitMarkerFilePath() string { + return fmt.Sprintf("test-visit-marker-%s.json", t.markerID.String()) +} + +func (t *TestVisitMarker) UpdateStatus(ownerIdentifier string, status VisitStatus) { + t.OwnerIdentifier = ownerIdentifier + t.Status = status +} + +func (t *TestVisitMarker) String() string { + return fmt.Sprintf("id=%s ownerIdentifier=%s status=%s storedValue=%s", t.markerID.String(), t.OwnerIdentifier, t.Status, t.StoredValue) +} From 2bf1b615c85477f33b5f301307bfa8deab0c8c37 Mon Sep 17 00:00:00 2001 From: Alan Protasio Date: Fri, 2 Aug 2024 16:01:40 -0700 Subject: [PATCH 26/32] Fix cortex_ingester_tsdb_mmap_chunks_total metric (#6134) --- CHANGELOG.md | 1 + pkg/ingester/metrics.go | 2 +- pkg/ingester/metrics_test.go | 8 ++++---- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index db2a0c1cf3..e22f7125fc 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -45,6 +45,7 @@ * [BUGFIX] Ingester: Fix issue with the minimize token generator where it was not taking in consideration the current ownerhip of an instance when generating extra tokens. #6062 * [BUGFIX] Scheduler: Fix user queue in scheduler that was not thread-safe. #6077 * [BUGFIX] Ingester: Include out-of-order head compaction when compacting TSDB head. #6108 +* [BUGFIX] Ingester: Fix `cortex_ingester_tsdb_mmap_chunks_total` metric. #6134 ## 1.17.1 2024-05-20 diff --git a/pkg/ingester/metrics.go b/pkg/ingester/metrics.go index 422cd8d3b8..3e2d98fc4b 100644 --- a/pkg/ingester/metrics.go +++ b/pkg/ingester/metrics.go @@ -632,7 +632,7 @@ func (sm *tsdbMetrics) Collect(out chan<- prometheus.Metric) { data.SendSumOfCountersPerUserWithLabels(out, sm.tsdbOutOfOrderSamplesAppended, "prometheus_tsdb_head_out_of_order_samples_appended_total", "type") data.SendSumOfCounters(out, sm.tsdbSnapshotReplayErrorTotal, "prometheus_tsdb_snapshot_replay_error_total") data.SendSumOfHistograms(out, sm.tsdbOOOHistogram, "prometheus_tsdb_sample_ooo_delta") - data.SendSumOfGauges(out, sm.tsdbMmapChunksTotal, "prometheus_tsdb_mmap_chunks_total") + data.SendSumOfCounters(out, sm.tsdbMmapChunksTotal, "prometheus_tsdb_mmap_chunks_total") data.SendSumOfCounters(out, sm.checkpointDeleteFail, "prometheus_tsdb_checkpoint_deletions_failed_total") data.SendSumOfCounters(out, sm.checkpointDeleteTotal, "prometheus_tsdb_checkpoint_deletions_total") data.SendSumOfCounters(out, sm.checkpointCreationFail, "prometheus_tsdb_checkpoint_creations_failed_total") diff --git a/pkg/ingester/metrics_test.go b/pkg/ingester/metrics_test.go index f5c5e2851f..df992686d3 100644 --- a/pkg/ingester/metrics_test.go +++ b/pkg/ingester/metrics_test.go @@ -324,8 +324,8 @@ func TestTSDBMetrics(t *testing.T) { # TYPE cortex_ingester_tsdb_mmap_chunk_corruptions_total counter cortex_ingester_tsdb_mmap_chunk_corruptions_total 2577406 # HELP cortex_ingester_tsdb_mmap_chunks_total Total number of chunks that were memory-mapped. - # TYPE cortex_ingester_tsdb_mmap_chunks_total gauge - cortex_ingester_tsdb_mmap_chunks_total 0 + # TYPE cortex_ingester_tsdb_mmap_chunks_total counter + cortex_ingester_tsdb_mmap_chunks_total 312 # HELP cortex_ingester_tsdb_out_of_order_samples_total Total number of out of order samples ingestion failed attempts due to out of order being disabled. # TYPE cortex_ingester_tsdb_out_of_order_samples_total counter cortex_ingester_tsdb_out_of_order_samples_total{type="float",user="user1"} 102 @@ -576,8 +576,8 @@ func TestTSDBMetricsWithRemoval(t *testing.T) { # TYPE cortex_ingester_tsdb_mmap_chunk_corruptions_total counter cortex_ingester_tsdb_mmap_chunk_corruptions_total 2577406 # HELP cortex_ingester_tsdb_mmap_chunks_total Total number of chunks that were memory-mapped. - # TYPE cortex_ingester_tsdb_mmap_chunks_total gauge - cortex_ingester_tsdb_mmap_chunks_total 0 + # TYPE cortex_ingester_tsdb_mmap_chunks_total counter + cortex_ingester_tsdb_mmap_chunks_total 312 # HELP cortex_ingester_tsdb_out_of_order_samples_total Total number of out of order samples ingestion failed attempts due to out of order being disabled. # TYPE cortex_ingester_tsdb_out_of_order_samples_total counter cortex_ingester_tsdb_out_of_order_samples_total{type="float",user="user1"} 102 From 3e8789ba10b79c8e88fc06b6fc72167b18a10c57 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 5 Aug 2024 08:21:31 -0700 Subject: [PATCH 27/32] Bump github.com/minio/minio-go/v7 from 7.0.73 to 7.0.74 (#6124) Bumps [github.com/minio/minio-go/v7](https://github.com/minio/minio-go) from 7.0.73 to 7.0.74. - [Release notes](https://github.com/minio/minio-go/releases) - [Commits](https://github.com/minio/minio-go/compare/v7.0.73...v7.0.74) --- updated-dependencies: - dependency-name: github.com/minio/minio-go/v7 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 ++-- .../minio/minio-go/v7/api-get-object.go | 24 +++++++++++++++---- .../github.com/minio/minio-go/v7/api-stat.go | 12 ++++++++-- vendor/github.com/minio/minio-go/v7/api.go | 2 +- .../v7/pkg/credentials/file_minio_client.go | 2 +- .../minio/minio-go/v7/pkg/s3utils/utils.go | 2 +- .../minio/minio-go/v7/post-policy.go | 17 +++++++++++++ vendor/modules.txt | 2 +- 9 files changed, 54 insertions(+), 13 deletions(-) diff --git a/go.mod b/go.mod index 25facc7ba6..1b61d3dd32 100644 --- a/go.mod +++ b/go.mod @@ -36,7 +36,7 @@ require ( github.com/json-iterator/go v1.1.12 github.com/klauspost/compress v1.17.9 github.com/lib/pq v1.10.9 - github.com/minio/minio-go/v7 v7.0.73 + github.com/minio/minio-go/v7 v7.0.74 github.com/mitchellh/go-wordwrap v1.0.1 github.com/oklog/ulid v1.3.1 github.com/opentracing-contrib/go-grpc v0.0.0-20210225150812-73cb765af46e diff --git a/go.sum b/go.sum index e104370fd4..720ff32002 100644 --- a/go.sum +++ b/go.sum @@ -1206,8 +1206,8 @@ github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8/go.mod h1:mC1jAcs github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3/go.mod h1:RagcQ7I8IeTMnF8JTXieKnO4Z6JCsikNEzj0DwauVzE= github.com/minio/md5-simd v1.1.2 h1:Gdi1DZK69+ZVMoNHRXJyNcxrMA4dSxoYHZSQbirFg34= github.com/minio/md5-simd v1.1.2/go.mod h1:MzdKDxYpY2BT9XQFocsiZf/NKVtR7nkE4RoEpN+20RM= -github.com/minio/minio-go/v7 v7.0.73 h1:qr2vi96Qm7kZ4v7LLebjte+MQh621fFWnv93p12htEo= -github.com/minio/minio-go/v7 v7.0.73/go.mod h1:qydcVzV8Hqtj1VtEocfxbmVFa2siu6HGa+LDEPogjD8= +github.com/minio/minio-go/v7 v7.0.74 h1:fTo/XlPBTSpo3BAMshlwKL5RspXRv9us5UeHEGYCFe0= +github.com/minio/minio-go/v7 v7.0.74/go.mod h1:qydcVzV8Hqtj1VtEocfxbmVFa2siu6HGa+LDEPogjD8= github.com/minio/sha256-simd v1.0.1 h1:6kaan5IFmwTNynnKKpDHe6FWHohJOHhCPchzK49dzMM= github.com/minio/sha256-simd v1.0.1/go.mod h1:Pz6AKMiUdngCLpeTL/RJY1M9rUuPMYujV5xJjtbRSN8= github.com/mitchellh/cli v1.1.0/go.mod h1:xcISNoH86gajksDmfB23e/pu+B+GeFRMYmoHXxx3xhI= diff --git a/vendor/github.com/minio/minio-go/v7/api-get-object.go b/vendor/github.com/minio/minio-go/v7/api-get-object.go index 9e6b1543c4..d7fd27835b 100644 --- a/vendor/github.com/minio/minio-go/v7/api-get-object.go +++ b/vendor/github.com/minio/minio-go/v7/api-get-object.go @@ -32,10 +32,18 @@ import ( func (c *Client) GetObject(ctx context.Context, bucketName, objectName string, opts GetObjectOptions) (*Object, error) { // Input validation. if err := s3utils.CheckValidBucketName(bucketName); err != nil { - return nil, err + return nil, ErrorResponse{ + StatusCode: http.StatusBadRequest, + Code: "InvalidBucketName", + Message: err.Error(), + } } if err := s3utils.CheckValidObjectName(objectName); err != nil { - return nil, err + return nil, ErrorResponse{ + StatusCode: http.StatusBadRequest, + Code: "XMinioInvalidObjectName", + Message: err.Error(), + } } gctx, cancel := context.WithCancel(ctx) @@ -649,10 +657,18 @@ func newObject(ctx context.Context, cancel context.CancelFunc, reqCh chan<- getR func (c *Client) getObject(ctx context.Context, bucketName, objectName string, opts GetObjectOptions) (io.ReadCloser, ObjectInfo, http.Header, error) { // Validate input arguments. if err := s3utils.CheckValidBucketName(bucketName); err != nil { - return nil, ObjectInfo{}, nil, err + return nil, ObjectInfo{}, nil, ErrorResponse{ + StatusCode: http.StatusBadRequest, + Code: "InvalidBucketName", + Message: err.Error(), + } } if err := s3utils.CheckValidObjectName(objectName); err != nil { - return nil, ObjectInfo{}, nil, err + return nil, ObjectInfo{}, nil, ErrorResponse{ + StatusCode: http.StatusBadRequest, + Code: "XMinioInvalidObjectName", + Message: err.Error(), + } } // Execute GET on objectName. diff --git a/vendor/github.com/minio/minio-go/v7/api-stat.go b/vendor/github.com/minio/minio-go/v7/api-stat.go index b043dc40c9..11455beb3f 100644 --- a/vendor/github.com/minio/minio-go/v7/api-stat.go +++ b/vendor/github.com/minio/minio-go/v7/api-stat.go @@ -61,10 +61,18 @@ func (c *Client) BucketExists(ctx context.Context, bucketName string) (bool, err func (c *Client) StatObject(ctx context.Context, bucketName, objectName string, opts StatObjectOptions) (ObjectInfo, error) { // Input validation. if err := s3utils.CheckValidBucketName(bucketName); err != nil { - return ObjectInfo{}, err + return ObjectInfo{}, ErrorResponse{ + StatusCode: http.StatusBadRequest, + Code: "InvalidBucketName", + Message: err.Error(), + } } if err := s3utils.CheckValidObjectName(objectName); err != nil { - return ObjectInfo{}, err + return ObjectInfo{}, ErrorResponse{ + StatusCode: http.StatusBadRequest, + Code: "XMinioInvalidObjectName", + Message: err.Error(), + } } headers := opts.Header() if opts.Internal.ReplicationDeleteMarker { diff --git a/vendor/github.com/minio/minio-go/v7/api.go b/vendor/github.com/minio/minio-go/v7/api.go index 6027227829..937551403e 100644 --- a/vendor/github.com/minio/minio-go/v7/api.go +++ b/vendor/github.com/minio/minio-go/v7/api.go @@ -129,7 +129,7 @@ type Options struct { // Global constants. const ( libraryName = "minio-go" - libraryVersion = "v7.0.73" + libraryVersion = "v7.0.74" ) // User Agent should always following the below style. diff --git a/vendor/github.com/minio/minio-go/v7/pkg/credentials/file_minio_client.go b/vendor/github.com/minio/minio-go/v7/pkg/credentials/file_minio_client.go index f1c165b79b..750e26ffa8 100644 --- a/vendor/github.com/minio/minio-go/v7/pkg/credentials/file_minio_client.go +++ b/vendor/github.com/minio/minio-go/v7/pkg/credentials/file_minio_client.go @@ -39,7 +39,7 @@ type FileMinioClient struct { Filename string // MinIO Alias to extract credentials from the shared credentials file. If empty - // will default to environment variable "MINIO_ALIAS" or "default" if + // will default to environment variable "MINIO_ALIAS" or "s3" if // environment variable is also not set. Alias string diff --git a/vendor/github.com/minio/minio-go/v7/pkg/s3utils/utils.go b/vendor/github.com/minio/minio-go/v7/pkg/s3utils/utils.go index 056e78a67a..0e63ce2f7d 100644 --- a/vendor/github.com/minio/minio-go/v7/pkg/s3utils/utils.go +++ b/vendor/github.com/minio/minio-go/v7/pkg/s3utils/utils.go @@ -226,7 +226,7 @@ func IsGoogleEndpoint(endpointURL url.URL) bool { if endpointURL == sentinelURL { return false } - return endpointURL.Host == "storage.googleapis.com" + return endpointURL.Hostname() == "storage.googleapis.com" } // Expects ascii encoded strings - from output of urlEncodePath diff --git a/vendor/github.com/minio/minio-go/v7/post-policy.go b/vendor/github.com/minio/minio-go/v7/post-policy.go index f6dbbf7f6e..3f023704a4 100644 --- a/vendor/github.com/minio/minio-go/v7/post-policy.go +++ b/vendor/github.com/minio/minio-go/v7/post-policy.go @@ -209,6 +209,23 @@ func (p *PostPolicy) SetContentTypeStartsWith(contentTypeStartsWith string) erro return nil } +// SetContentDisposition - Sets content-disposition of the object for this policy +func (p *PostPolicy) SetContentDisposition(contentDisposition string) error { + if strings.TrimSpace(contentDisposition) == "" || contentDisposition == "" { + return errInvalidArgument("No content disposition specified.") + } + policyCond := policyCondition{ + matchType: "eq", + condition: "$Content-Disposition", + value: contentDisposition, + } + if err := p.addNewPolicy(policyCond); err != nil { + return err + } + p.formData["Content-Disposition"] = contentDisposition + return nil +} + // SetContentLengthRange - Set new min and max content length // condition for all incoming uploads. func (p *PostPolicy) SetContentLengthRange(min, max int64) error { diff --git a/vendor/modules.txt b/vendor/modules.txt index c8a803765b..378718f522 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -665,7 +665,7 @@ github.com/miekg/dns # github.com/minio/md5-simd v1.1.2 ## explicit; go 1.14 github.com/minio/md5-simd -# github.com/minio/minio-go/v7 v7.0.73 +# github.com/minio/minio-go/v7 v7.0.74 ## explicit; go 1.21 github.com/minio/minio-go/v7 github.com/minio/minio-go/v7/pkg/credentials From d263c962747175b03d14080fd50e114095893245 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 5 Aug 2024 08:31:37 -0700 Subject: [PATCH 28/32] Bump github.com/aws/aws-sdk-go from 1.55.3 to 1.55.5 (#6140) Bumps [github.com/aws/aws-sdk-go](https://github.com/aws/aws-sdk-go) from 1.55.3 to 1.55.5. - [Release notes](https://github.com/aws/aws-sdk-go/releases) - [Commits](https://github.com/aws/aws-sdk-go/compare/v1.55.3...v1.55.5) --- updated-dependencies: - dependency-name: github.com/aws/aws-sdk-go 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 ++-- .../github.com/aws/aws-sdk-go/aws/endpoints/defaults.go | 9 +++++++++ vendor/github.com/aws/aws-sdk-go/aws/version.go | 2 +- vendor/modules.txt | 2 +- 5 files changed, 14 insertions(+), 5 deletions(-) diff --git a/go.mod b/go.mod index 1b61d3dd32..33a458ec09 100644 --- a/go.mod +++ b/go.mod @@ -9,7 +9,7 @@ require ( github.com/alecthomas/units v0.0.0-20231202071711-9a357b53e9c9 github.com/alicebob/miniredis/v2 v2.33.0 github.com/armon/go-metrics v0.4.1 - github.com/aws/aws-sdk-go v1.55.3 + github.com/aws/aws-sdk-go v1.55.5 github.com/bradfitz/gomemcache v0.0.0-20230905024940-24af94b03874 github.com/cespare/xxhash v1.1.0 github.com/cortexproject/promqlsmith v0.0.0-20240506042652-6cfdd9739a5e diff --git a/go.sum b/go.sum index 720ff32002..6728288619 100644 --- a/go.sum +++ b/go.sum @@ -601,8 +601,8 @@ github.com/asaskevich/govalidator v0.0.0-20230301143203-a9d515a09cc2 h1:DklsrG3d github.com/asaskevich/govalidator v0.0.0-20230301143203-a9d515a09cc2/go.mod h1:WaHUgvxTVq04UNunO+XhnAqY/wQc+bxr74GqbsZ/Jqw= github.com/aws/aws-sdk-go v1.27.0/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN924inxo= github.com/aws/aws-sdk-go v1.38.35/go.mod h1:hcU610XS61/+aQV88ixoOzUoG7v3b31pl2zKMmprdro= -github.com/aws/aws-sdk-go v1.55.3 h1:0B5hOX+mIx7I5XPOrjrHlKSDQV/+ypFZpIHOx5LOk3E= -github.com/aws/aws-sdk-go v1.55.3/go.mod h1:eRwEWoyTWFMVYVQzKMNHWP5/RV4xIUGMQfXQHfHkpNU= +github.com/aws/aws-sdk-go v1.55.5 h1:KKUZBfBoyqy5d3swXyiC7Q76ic40rYcbqH7qjh59kzU= +github.com/aws/aws-sdk-go v1.55.5/go.mod h1:eRwEWoyTWFMVYVQzKMNHWP5/RV4xIUGMQfXQHfHkpNU= github.com/aws/aws-sdk-go-v2 v1.16.0/go.mod h1:lJYcuZZEHWNIb6ugJjbQY1fykdoobWbOS7kJYb4APoI= github.com/aws/aws-sdk-go-v2 v1.16.16 h1:M1fj4FE2lB4NzRb9Y0xdWsn2P0+2UHVxwKyOa4YJNjk= github.com/aws/aws-sdk-go-v2 v1.16.16/go.mod h1:SwiyXi/1zTUZ6KIAmLK5V5ll8SiURNUYOqTerZPaF9k= diff --git a/vendor/github.com/aws/aws-sdk-go/aws/endpoints/defaults.go b/vendor/github.com/aws/aws-sdk-go/aws/endpoints/defaults.go index 069debf1f5..c3516e018a 100644 --- a/vendor/github.com/aws/aws-sdk-go/aws/endpoints/defaults.go +++ b/vendor/github.com/aws/aws-sdk-go/aws/endpoints/defaults.go @@ -9503,6 +9503,12 @@ var awsPartition = partition{ endpointKey{ Region: "eu-central-1", }: endpoint{}, + endpointKey{ + Region: "eu-north-1", + }: endpoint{}, + endpointKey{ + Region: "eu-south-2", + }: endpoint{}, endpointKey{ Region: "eu-west-1", }: endpoint{}, @@ -32566,6 +32572,9 @@ var awsPartition = partition{ endpointKey{ Region: "ca-central-1", }: endpoint{}, + endpointKey{ + Region: "ca-west-1", + }: endpoint{}, endpointKey{ Region: "eu-central-1", }: endpoint{}, diff --git a/vendor/github.com/aws/aws-sdk-go/aws/version.go b/vendor/github.com/aws/aws-sdk-go/aws/version.go index 514bf3ade2..d15e3c84c0 100644 --- a/vendor/github.com/aws/aws-sdk-go/aws/version.go +++ b/vendor/github.com/aws/aws-sdk-go/aws/version.go @@ -5,4 +5,4 @@ package aws const SDKName = "aws-sdk-go" // SDKVersion is the version of this SDK -const SDKVersion = "1.55.3" +const SDKVersion = "1.55.5" diff --git a/vendor/modules.txt b/vendor/modules.txt index 378718f522..d70a2c9e54 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -142,7 +142,7 @@ github.com/armon/go-metrics/prometheus # github.com/asaskevich/govalidator v0.0.0-20230301143203-a9d515a09cc2 ## explicit; go 1.13 github.com/asaskevich/govalidator -# github.com/aws/aws-sdk-go v1.55.3 +# github.com/aws/aws-sdk-go v1.55.5 ## explicit; go 1.19 github.com/aws/aws-sdk-go/aws github.com/aws/aws-sdk-go/aws/auth/bearer From f7f6ed2870b112fabf1e7ab93187f7fc9a92bc2e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 5 Aug 2024 08:57:22 -0700 Subject: [PATCH 29/32] Bump golang.org/x/sync from 0.7.0 to 0.8.0 (#6137) Bumps [golang.org/x/sync](https://github.com/golang/sync) from 0.7.0 to 0.8.0. - [Commits](https://github.com/golang/sync/compare/v0.7.0...v0.8.0) --- updated-dependencies: - dependency-name: golang.org/x/sync 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 ++-- vendor/golang.org/x/sync/LICENSE | 4 ++-- vendor/modules.txt | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/go.mod b/go.mod index 33a458ec09..3667f14bc7 100644 --- a/go.mod +++ b/go.mod @@ -70,7 +70,7 @@ require ( go.opentelemetry.io/otel/trace v1.28.0 go.uber.org/atomic v1.11.0 golang.org/x/net v0.27.0 - golang.org/x/sync v0.7.0 + golang.org/x/sync v0.8.0 golang.org/x/time v0.5.0 google.golang.org/grpc v1.65.0 gopkg.in/yaml.v2 v2.4.0 diff --git a/go.sum b/go.sum index 6728288619..080ad777e9 100644 --- a/go.sum +++ b/go.sum @@ -1723,8 +1723,8 @@ golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20220819030929-7fc1605a5dde/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220929204114-8fcdb60fdcc0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.7.0 h1:YsImfSBoP9QPYL0xyKJPq0gcaJdG3rInoqxTWbfQu9M= -golang.org/x/sync v0.7.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= +golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180823144017-11551d06cbcc/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= diff --git a/vendor/golang.org/x/sync/LICENSE b/vendor/golang.org/x/sync/LICENSE index 6a66aea5ea..2a7cf70da6 100644 --- a/vendor/golang.org/x/sync/LICENSE +++ b/vendor/golang.org/x/sync/LICENSE @@ -1,4 +1,4 @@ -Copyright (c) 2009 The Go Authors. All rights reserved. +Copyright 2009 The Go Authors. Redistribution and use in source and binary forms, with or without modification, are permitted provided that the following conditions are @@ -10,7 +10,7 @@ notice, this list of conditions and the following disclaimer. copyright notice, this list of conditions and the following disclaimer in the documentation and/or other materials provided with the distribution. - * Neither the name of Google Inc. nor the names of its + * Neither the name of Google LLC nor the names of its contributors may be used to endorse or promote products derived from this software without specific prior written permission. diff --git a/vendor/modules.txt b/vendor/modules.txt index d70a2c9e54..2f244c0388 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -1309,7 +1309,7 @@ golang.org/x/oauth2/google/internal/stsexchange golang.org/x/oauth2/internal golang.org/x/oauth2/jws golang.org/x/oauth2/jwt -# golang.org/x/sync v0.7.0 +# golang.org/x/sync v0.8.0 ## explicit; go 1.18 golang.org/x/sync/errgroup golang.org/x/sync/semaphore From b2ce96c8385fd0ff33342014e9a8bcac61d97283 Mon Sep 17 00:00:00 2001 From: Ben Ye Date: Mon, 5 Aug 2024 09:50:17 -0700 Subject: [PATCH 30/32] OTLP handler: enable metric name normalization (#6136) --- CHANGELOG.md | 1 + pkg/util/push/otlp.go | 6 +++++- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index e22f7125fc..edb3d0c5c3 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -5,6 +5,7 @@ * [CHANGE] Ingester: Remove `-querier.query-store-for-labels-enabled` flag. Querying long-term store for labels is always enabled. #5984 * [CHANGE] Server: Instrument `cortex_request_duration_seconds` metric with native histogram. If `native-histograms` feature is enabled in monitoring Prometheus then the metric name needs to be updated in your dashboards. #6056 * [CHANGE] Distributor/Ingester: Change `cortex_distributor_ingester_appends_total`, `cortex_distributor_ingester_append_failures_total`, `cortex_distributor_ingester_queries_total`, and `cortex_distributor_ingester_query_failures_total` metrics to use the ingester ID instead of its IP as the label value. #6078 +* [CHANGE] OTLP: Set `AddMetricSuffixes` to true to always enable metric name normalization. #6136 * [FEATURE] Ingester/Distributor: Experimental: Enable native histogram ingestion via `-blocks-storage.tsdb.enable-native-histograms` flag. #5986 #6010 #6020 * [FEATURE] Querier: Enable querying native histogram chunks. #5944 #6031 * [FEATURE] Query Frontend: Support native histogram in query frontend response. #5996 #6043 diff --git a/pkg/util/push/otlp.go b/pkg/util/push/otlp.go index 6a263fe599..b3bae4dd4e 100644 --- a/pkg/util/push/otlp.go +++ b/pkg/util/push/otlp.go @@ -38,7 +38,11 @@ func OTLPHandler(sourceIPs *middleware.SourceIPExtractor, push Func) http.Handle } promConverter := prometheusremotewrite.NewPrometheusConverter() - err = promConverter.FromMetrics(convertToMetricsAttributes(req.Metrics()), prometheusremotewrite.Settings{DisableTargetInfo: true}) + setting := prometheusremotewrite.Settings{ + AddMetricSuffixes: true, + DisableTargetInfo: true, + } + err = promConverter.FromMetrics(convertToMetricsAttributes(req.Metrics()), setting) if err != nil { level.Error(logger).Log("err", err.Error()) http.Error(w, err.Error(), http.StatusBadRequest) From ce9c4ba016c6a44ecd80e2ba2dd921a0a5e58021 Mon Sep 17 00:00:00 2001 From: Ben Ye Date: Mon, 5 Aug 2024 09:50:34 -0700 Subject: [PATCH 31/32] Fix OTLP native histogram push (#6135) --- CHANGELOG.md | 2 +- integration/e2ecortex/client.go | 2 +- integration/otlp_test.go | 2 +- pkg/cortexpb/histograms.go | 39 ++++++++++++++++++++++++++++++++- pkg/util/push/otlp.go | 15 ++++++++++--- pkg/util/push/otlp_test.go | 5 +++++ 6 files changed, 58 insertions(+), 7 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index edb3d0c5c3..1b97cd3b3f 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -12,7 +12,7 @@ * [FEATURE] Ruler: Support sending native histogram samples to Ingester. #6029 * [FEATURE] Ruler: Add support for filtering out alerts in ListRules API. #6011 * [FEATURE] Query Frontend: Added a query rejection mechanism to block resource-intensive queries. #6005 -* [FEATURE] OTLP: Support ingesting OTLP exponential metrics as native histograms. #6071 +* [FEATURE] OTLP: Support ingesting OTLP exponential metrics as native histograms. #6071 #6135 * [FEATURE] Ingester: Add `ingester.instance-limits.max-inflight-query-requests` to allow limiting ingester concurrent queries. #6081 * [FEATURE] Distributor: Add `validation.max-native-histogram-buckets` to limit max number of bucket count. Distributor will try to automatically reduce histogram resolution until it is within the bucket limit or resolution cannot be reduced anymore. #6104 * [FEATURE] Store Gateway: Token bucket limiter. #6016 diff --git a/integration/e2ecortex/client.go b/integration/e2ecortex/client.go index 5f45d6d59f..b6ea5fb7f9 100644 --- a/integration/e2ecortex/client.go +++ b/integration/e2ecortex/client.go @@ -173,7 +173,7 @@ func createDataPointsGauge(newMetric pmetric.Metric, attributes map[string]any, } func createDataPointsExponentialHistogram(newMetric pmetric.Metric, attributes map[string]any, histograms []prompb.Histogram) { - newMetric.SetEmptyExponentialHistogram() + newMetric.SetEmptyExponentialHistogram().SetAggregationTemporality(pmetric.AggregationTemporalityCumulative) for _, h := range histograms { datapoint := newMetric.ExponentialHistogram().DataPoints().AppendEmpty() datapoint.SetTimestamp(pcommon.Timestamp(h.Timestamp * time.Millisecond.Nanoseconds())) diff --git a/integration/otlp_test.go b/integration/otlp_test.go index 0d56c3751b..9a603e8cb0 100644 --- a/integration/otlp_test.go +++ b/integration/otlp_test.go @@ -77,7 +77,7 @@ func TestOTLP(t *testing.T) { i := rand.Uint32() histogramSeries := e2e.GenerateHistogramSeries("histogram_series", now, i, false, prompb.Label{Name: "job", Value: "test"}) - res, err = c.Push(histogramSeries) + res, err = c.OTLP(histogramSeries) require.NoError(t, err) require.Equal(t, 200, res.StatusCode) diff --git a/pkg/cortexpb/histograms.go b/pkg/cortexpb/histograms.go index 2e2afef457..60e7207a19 100644 --- a/pkg/cortexpb/histograms.go +++ b/pkg/cortexpb/histograms.go @@ -13,13 +13,41 @@ package cortexpb -import "github.com/prometheus/prometheus/model/histogram" +import ( + "github.com/prometheus/prometheus/model/histogram" + "github.com/prometheus/prometheus/prompb" +) func (h Histogram) IsFloatHistogram() bool { _, ok := h.GetCount().(*Histogram_CountFloat) return ok } +// HistogramPromProtoToHistogramProto converts a prometheus protobuf Histogram to cortex protobuf Histogram. +func HistogramPromProtoToHistogramProto(h prompb.Histogram) Histogram { + ph := Histogram{ + Sum: h.Sum, + Schema: h.Schema, + ZeroThreshold: h.ZeroThreshold, + NegativeSpans: spansPromProtoToSpansProto(h.NegativeSpans), + NegativeDeltas: h.NegativeDeltas, + NegativeCounts: h.NegativeCounts, + PositiveSpans: spansPromProtoToSpansProto(h.PositiveSpans), + PositiveDeltas: h.PositiveDeltas, + PositiveCounts: h.PositiveCounts, + ResetHint: Histogram_ResetHint(h.ResetHint), + TimestampMs: h.Timestamp, + } + if h.IsFloatHistogram() { + ph.Count = &Histogram_CountFloat{CountFloat: h.GetCountFloat()} + ph.ZeroCount = &Histogram_ZeroCountFloat{ZeroCountFloat: h.GetZeroCountFloat()} + } else { + ph.Count = &Histogram_CountInt{CountInt: h.GetCountInt()} + ph.ZeroCount = &Histogram_ZeroCountInt{ZeroCountInt: h.GetZeroCountInt()} + } + return ph +} + // HistogramProtoToHistogram extracts a (normal integer) Histogram from the // provided proto message. The caller has to make sure that the proto message // represents an interger histogram and not a float histogram. @@ -118,3 +146,12 @@ func spansToSpansProto(s []histogram.Span) []BucketSpan { return spans } + +func spansPromProtoToSpansProto(s []prompb.BucketSpan) []BucketSpan { + spans := make([]BucketSpan, len(s)) + for i := 0; i < len(s); i++ { + spans[i] = BucketSpan{Offset: s[i].Offset, Length: s[i].Length} + } + + return spans +} diff --git a/pkg/util/push/otlp.go b/pkg/util/push/otlp.go index b3bae4dd4e..14c3c98e5c 100644 --- a/pkg/util/push/otlp.go +++ b/pkg/util/push/otlp.go @@ -58,9 +58,10 @@ func OTLPHandler(sourceIPs *middleware.SourceIPExtractor, push Func) http.Handle tsList := []cortexpb.PreallocTimeseries(nil) for _, v := range promConverter.TimeSeries() { tsList = append(tsList, cortexpb.PreallocTimeseries{TimeSeries: &cortexpb.TimeSeries{ - Labels: makeLabels(v.Labels), - Samples: makeSamples(v.Samples), - Exemplars: makeExemplars(v.Exemplars), + Labels: makeLabels(v.Labels), + Samples: makeSamples(v.Samples), + Exemplars: makeExemplars(v.Exemplars), + Histograms: makeHistograms(v.Histograms), }}) } prwReq.Timeseries = tsList @@ -112,6 +113,14 @@ func makeExemplars(in []prompb.Exemplar) []cortexpb.Exemplar { return out } +func makeHistograms(in []prompb.Histogram) []cortexpb.Histogram { + out := make([]cortexpb.Histogram, 0, len(in)) + for _, h := range in { + out = append(out, cortexpb.HistogramPromProtoToHistogramProto(h)) + } + return out +} + func convertToMetricsAttributes(md pmetric.Metrics) pmetric.Metrics { cloneMd := pmetric.NewMetrics() md.CopyTo(cloneMd) diff --git a/pkg/util/push/otlp_test.go b/pkg/util/push/otlp_test.go index 641fc2c566..be7e1f79b9 100644 --- a/pkg/util/push/otlp_test.go +++ b/pkg/util/push/otlp_test.go @@ -124,6 +124,11 @@ func verifyOTLPWriteRequestHandler(t *testing.T, expectSource cortexpb.WriteRequ // TODO: test more things assert.Equal(t, expectSource, request.Source) assert.False(t, request.SkipLabelNameValidation) + for _, ts := range request.Timeseries { + assert.NotEmpty(t, ts.Labels) + // Make sure at least one of sample, exemplar or histogram is set. + assert.True(t, len(ts.Samples) > 0 || len(ts.Exemplars) > 0 || len(ts.Histograms) > 0) + } return &cortexpb.WriteResponse{}, nil } } From df270eef9fa915b2d0da86139010ac26df0ad293 Mon Sep 17 00:00:00 2001 From: Kevin Ingelman Date: Tue, 6 Aug 2024 10:05:38 -0700 Subject: [PATCH 32/32] Fix query offset issue on individual rule groups (#6131) * Fix rule group query offset issue This is a follow-up to #6085, which added support for setting the `query_offset` field on individual recording rule groups, as well as a per-tenant `ruler_query_offset` limit that should be used when no individual recording rule group offset is set. It turns out that compatibility code to convert from a protobuf RuleGroup to a prometheus RuleGroup was coercing null-value query offsets to explicit 0s, which meant that no rule groups would ever fall back to the per-tenant offset. This PR fixes that issue, and it cleans up handling of the query offset in a few other ruler files. Signed-off-by: Kevin Ingelman * Revert change to rules API response Signed-off-by: Kevin Ingelman --------- Signed-off-by: Kevin Ingelman --- pkg/ruler/api_test.go | 4 ++-- pkg/ruler/ruler.go | 11 ++++++----- pkg/ruler/rulespb/compat.go | 7 ++++--- 3 files changed, 12 insertions(+), 10 deletions(-) diff --git a/pkg/ruler/api_test.go b/pkg/ruler/api_test.go index cdddcf2432..3ef611a47e 100644 --- a/pkg/ruler/api_test.go +++ b/pkg/ruler/api_test.go @@ -279,7 +279,7 @@ rules: labels: test: test `, - output: "name: test\ninterval: 15s\nquery_offset: 0s\nrules:\n - record: up_rule\n expr: up{}\n - alert: up_alert\n expr: sum(up{}) > 1\n for: 30s\n labels:\n test: test\n annotations:\n test: test\n", + output: "name: test\ninterval: 15s\nrules:\n - record: up_rule\n expr: up{}\n - alert: up_alert\n expr: sum(up{}) > 1\n for: 30s\n labels:\n test: test\n annotations:\n test: test\n", }, { name: "with a valid rule query offset", @@ -342,7 +342,7 @@ func TestRuler_DeleteNamespace(t *testing.T) { router.ServeHTTP(w, req) require.Equal(t, http.StatusOK, w.Code) - require.Equal(t, "name: group1\ninterval: 1m\nquery_offset: 0s\nrules:\n - record: UP_RULE\n expr: up\n - alert: UP_ALERT\n expr: up < 1\n", w.Body.String()) + require.Equal(t, "name: group1\ninterval: 1m\nrules:\n - record: UP_RULE\n expr: up\n - alert: UP_ALERT\n expr: up < 1\n", w.Body.String()) // Delete namespace1 req = requestFor(t, http.MethodDelete, "https://localhost:8080/api/v1/rules/namespace1", nil, "user1") diff --git a/pkg/ruler/ruler.go b/pkg/ruler/ruler.go index fb6df8caf3..56c4a63e59 100644 --- a/pkg/ruler/ruler.go +++ b/pkg/ruler/ruler.go @@ -1061,11 +1061,12 @@ func (r *Ruler) ruleGroupListToGroupStateDesc(userID string, backupGroups rulesp groupDesc := &GroupStateDesc{ Group: &rulespb.RuleGroupDesc{ - Name: group.GetName(), - Namespace: group.GetNamespace(), - Interval: interval, - User: userID, - Limit: group.Limit, + Name: group.GetName(), + Namespace: group.GetNamespace(), + Interval: interval, + User: userID, + Limit: group.Limit, + QueryOffset: group.QueryOffset, }, // We are keeping default value for EvaluationTimestamp and EvaluationDuration since the backup is not evaluating } diff --git a/pkg/ruler/rulespb/compat.go b/pkg/ruler/rulespb/compat.go index 16bf075e27..7526062ad4 100644 --- a/pkg/ruler/rulespb/compat.go +++ b/pkg/ruler/rulespb/compat.go @@ -49,16 +49,17 @@ func formattedRuleToProto(rls []rulefmt.RuleNode) []*RuleDesc { // FromProto generates a rulefmt RuleGroup func FromProto(rg *RuleGroupDesc) rulefmt.RuleGroup { - var queryOffset model.Duration + var queryOffset *model.Duration if rg.QueryOffset != nil { - queryOffset = model.Duration(*rg.QueryOffset) + offset := model.Duration(*rg.QueryOffset) + queryOffset = &offset } formattedRuleGroup := rulefmt.RuleGroup{ Name: rg.GetName(), Interval: model.Duration(rg.Interval), Rules: make([]rulefmt.RuleNode, len(rg.GetRules())), Limit: int(rg.Limit), - QueryOffset: &queryOffset, + QueryOffset: queryOffset, } for i, rl := range rg.GetRules() {