mirror of
https://github.com/prometheus/prometheus.git
synced 2025-03-05 20:59:13 -08:00
[FEATURE] PromQL: Add experimental info function MVP (#14495)
The `info` function is an experiment to improve UX around including labels from info metrics. `info` has to be enabled via the feature flag `--enable-feature=promql-experimental-functions`. This MVP of info simplifies the implementation by assuming: * Only support for the target_info metric * That target_info's identifying labels are job and instance Also: * Encode info samples' original timestamp as sample value * Deduce info series select hints from top-most VectorSelector --------- Signed-off-by: Arve Knudsen <arve.knudsen@gmail.com> Co-authored-by: Ying WANG <ying.wang@grafana.com> Co-authored-by: Augustin Husson <augustin.husson@amadeus.com> Co-authored-by: Bartlomiej Plotka <bwplotka@gmail.com> Co-authored-by: Björn Rabenstein <github@rabenste.in> Co-authored-by: Bryan Boreham <bjboreham@gmail.com>
This commit is contained in:
parent
249969d7ff
commit
de16f5e387
|
@ -2,6 +2,8 @@
|
|||
|
||||
## unreleased
|
||||
|
||||
* [FEATURE] PromQL: Add experimental `info` function. #14495
|
||||
|
||||
## 2.55.0-rc.0 / 2024-09-20
|
||||
|
||||
* [FEATURE] Support UTF-8 characters in label names - feature flag `utf8-names`. #14482, #14880, #14736, #14727
|
||||
|
|
|
@ -432,6 +432,97 @@ by the number of seconds under the specified time range window, and should be
|
|||
used primarily for human readability. Use `rate` in recording rules so that
|
||||
increases are tracked consistently on a per-second basis.
|
||||
|
||||
## `info()` (experimental)
|
||||
|
||||
_The `info` function is an experiment to improve UX
|
||||
around including labels from [info metrics](https://grafana.com/blog/2021/08/04/how-to-use-promql-joins-for-more-effective-queries-of-prometheus-metrics-at-scale/#info-metrics).
|
||||
The behavior of this function may change in future versions of Prometheus,
|
||||
including its removal from PromQL. `info` has to be enabled via the
|
||||
[feature flag](../feature_flags.md#experimental-promql-functions) `--enable-feature=promql-experimental-functions`._
|
||||
|
||||
`info(v instant-vector, [data-label-selector instant-vector])` finds, for each time
|
||||
series in `v`, all info series with matching _identifying_ labels (more on
|
||||
this later), and adds the union of their _data_ (i.e., non-identifying) labels
|
||||
to the time series. The second argument `data-label-selector` is optional.
|
||||
It is not a real instant vector, but uses a subset of its syntax.
|
||||
It must start and end with curly braces (`{ ... }`) and may only contain label matchers.
|
||||
The label matchers are used to constrain which info series to consider
|
||||
and which data labels to add to `v`.
|
||||
|
||||
Identifying labels of an info series are the subset of labels that uniquely
|
||||
identify the info series. The remaining labels are considered
|
||||
_data labels_ (also called non-identifying). (Note that Prometheus's concept
|
||||
of time series identity always includes _all_ the labels. For the sake of the `info`
|
||||
function, we “logically” define info series identity in a different way than
|
||||
in the conventional Prometheus view.) The identifying labels of an info series
|
||||
are used to join it to regular (non-info) series, i.e. those series that have
|
||||
the same labels as the identifying labels of the info series. The data labels, which are
|
||||
the ones added to the regular series by the `info` function, effectively encode
|
||||
metadata key value pairs. (This implies that a change in the data labels
|
||||
in the conventional Prometheus view constitutes the end of one info series and
|
||||
the beginning of a new info series, while the “logical” view of the `info` function is
|
||||
that the same info series continues to exist, just with different “data”.)
|
||||
|
||||
The conventional approach of adding data labels is sometimes called a “join query”,
|
||||
as illustrated by the following example:
|
||||
|
||||
```
|
||||
rate(http_server_request_duration_seconds_count[2m])
|
||||
* on (job, instance) group_left (k8s_cluster_name)
|
||||
target_info
|
||||
```
|
||||
|
||||
The core of the query is the expression `rate(http_server_request_duration_seconds_count[2m])`.
|
||||
But to add data labels from an info metric, the user has to use elaborate
|
||||
(and not very obvious) syntax to specify which info metric to use (`target_info`), what the
|
||||
identifying labels are (`on (job, instance)`), and which data labels to add
|
||||
(`group_left (k8s_cluster_name)`).
|
||||
|
||||
This query is not only verbose and hard to write, it might also run into an “identity crisis”:
|
||||
If any of the data labels of `target_info` changes, Prometheus sees that as a change of series
|
||||
(as alluded to above, Prometheus just has no native concept of non-identifying labels).
|
||||
If the old `target_info` series is not properly marked as stale (which can happen with certain ingestion paths),
|
||||
the query above will fail for up to 5m (the lookback delta) because it will find a conflicting
|
||||
match with both the old and the new version of `target_info`.
|
||||
|
||||
The `info` function not only resolves this conflict in favor of the newer series, it also simplifies the syntax
|
||||
because it knows about the available info series and what their identifying labels are. The example query
|
||||
looks like this with the `info` function:
|
||||
|
||||
```
|
||||
info(
|
||||
rate(http_server_request_duration_seconds_count[2m]),
|
||||
{k8s_cluster_name=~".+"}
|
||||
)
|
||||
```
|
||||
|
||||
The common case of adding _all_ data labels can be achieved by
|
||||
omitting the 2nd argument of the `info` function entirely, simplifying
|
||||
the example even more:
|
||||
|
||||
```
|
||||
info(rate(http_server_request_duration_seconds_count[2m]))
|
||||
```
|
||||
|
||||
While `info` normally automatically finds all matching info series, it's possible to
|
||||
restrict them by providing a `__name__` label matcher, e.g.
|
||||
`{__name__="target_info"}`.
|
||||
|
||||
### Limitations
|
||||
|
||||
In its current iteration, `info` defaults to considering only info series with
|
||||
the name `target_info`. It also assumes that the identifying info series labels are
|
||||
`instance` and `job`. `info` does support other info series names however, through
|
||||
`__name__` label matchers. E.g., one can explicitly say to consider both
|
||||
`target_info` and `build_info` as follows:
|
||||
`{__name__=~"(target|build)_info"}`. However, the identifying labels always
|
||||
have to be `instance` and `job`.
|
||||
|
||||
These limitations are partially defeating the purpose of the `info` function.
|
||||
At the current stage, this is an experiment to find out how useful the approach
|
||||
turns out to be in practice. A final version of the `info` function will indeed
|
||||
consider all matching info series and with their appropriate identifying labels.
|
||||
|
||||
## `irate()`
|
||||
|
||||
`irate(v range-vector)` calculates the per-second instant rate of increase of
|
||||
|
|
|
@ -21,6 +21,8 @@ import (
|
|||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
"github.com/prometheus/prometheus/model/histogram"
|
||||
"github.com/prometheus/prometheus/model/labels"
|
||||
"github.com/prometheus/prometheus/promql"
|
||||
|
@ -380,6 +382,126 @@ func BenchmarkNativeHistograms(b *testing.B) {
|
|||
}
|
||||
}
|
||||
|
||||
func BenchmarkInfoFunction(b *testing.B) {
|
||||
// Initialize test storage and generate test series data.
|
||||
testStorage := teststorage.New(b)
|
||||
defer testStorage.Close()
|
||||
|
||||
start := time.Unix(0, 0)
|
||||
end := start.Add(2 * time.Hour)
|
||||
step := 30 * time.Second
|
||||
|
||||
// Generate time series data for the benchmark.
|
||||
generateInfoFunctionTestSeries(b, testStorage, 100, 2000, 3600)
|
||||
|
||||
// Define test cases with queries to benchmark.
|
||||
cases := []struct {
|
||||
name string
|
||||
query string
|
||||
}{
|
||||
{
|
||||
name: "Joining info metrics with other metrics with group_left example 1",
|
||||
query: "rate(http_server_request_duration_seconds_count[2m]) * on (job, instance) group_left (k8s_cluster_name) target_info{k8s_cluster_name=\"us-east\"}",
|
||||
},
|
||||
{
|
||||
name: "Joining info metrics with other metrics with info() example 1",
|
||||
query: `info(rate(http_server_request_duration_seconds_count[2m]), {k8s_cluster_name="us-east"})`,
|
||||
},
|
||||
{
|
||||
name: "Joining info metrics with other metrics with group_left example 2",
|
||||
query: "sum by (k8s_cluster_name, http_status_code) (rate(http_server_request_duration_seconds_count[2m]) * on (job, instance) group_left (k8s_cluster_name) target_info)",
|
||||
},
|
||||
{
|
||||
name: "Joining info metrics with other metrics with info() example 2",
|
||||
query: `sum by (k8s_cluster_name, http_status_code) (info(rate(http_server_request_duration_seconds_count[2m]), {k8s_cluster_name=~".+"}))`,
|
||||
},
|
||||
}
|
||||
|
||||
// Benchmark each query type.
|
||||
for _, tc := range cases {
|
||||
// Initialize the PromQL engine once for all benchmarks.
|
||||
opts := promql.EngineOpts{
|
||||
Logger: nil,
|
||||
Reg: nil,
|
||||
MaxSamples: 50000000,
|
||||
Timeout: 100 * time.Second,
|
||||
EnableAtModifier: true,
|
||||
EnableNegativeOffset: true,
|
||||
}
|
||||
engine := promql.NewEngine(opts)
|
||||
b.Run(tc.name, func(b *testing.B) {
|
||||
b.ResetTimer()
|
||||
for i := 0; i < b.N; i++ {
|
||||
b.StopTimer() // Stop the timer to exclude setup time.
|
||||
qry, err := engine.NewRangeQuery(context.Background(), testStorage, nil, tc.query, start, end, step)
|
||||
require.NoError(b, err)
|
||||
b.StartTimer()
|
||||
result := qry.Exec(context.Background())
|
||||
require.NoError(b, result.Err)
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
// Report allocations.
|
||||
b.ReportAllocs()
|
||||
}
|
||||
|
||||
// Helper function to generate target_info and http_server_request_duration_seconds_count series for info function benchmarking.
|
||||
func generateInfoFunctionTestSeries(tb testing.TB, stor *teststorage.TestStorage, infoSeriesNum, interval, numIntervals int) {
|
||||
tb.Helper()
|
||||
|
||||
ctx := context.Background()
|
||||
statusCodes := []string{"200", "400", "500"}
|
||||
|
||||
// Generate target_info metrics with instance and job labels, and k8s_cluster_name label.
|
||||
// Generate http_server_request_duration_seconds_count metrics with instance and job labels, and http_status_code label.
|
||||
// the classic target_info metrics is gauge type.
|
||||
metrics := make([]labels.Labels, 0, infoSeriesNum+len(statusCodes))
|
||||
for i := 0; i < infoSeriesNum; i++ {
|
||||
clusterName := "us-east"
|
||||
if i >= infoSeriesNum/2 {
|
||||
clusterName = "eu-south"
|
||||
}
|
||||
metrics = append(metrics, labels.FromStrings(
|
||||
"__name__", "target_info",
|
||||
"instance", "instance"+strconv.Itoa(i),
|
||||
"job", "job"+strconv.Itoa(i),
|
||||
"k8s_cluster_name", clusterName,
|
||||
))
|
||||
}
|
||||
|
||||
for _, statusCode := range statusCodes {
|
||||
metrics = append(metrics, labels.FromStrings(
|
||||
"__name__", "http_server_request_duration_seconds_count",
|
||||
"instance", "instance0",
|
||||
"job", "job0",
|
||||
"http_status_code", statusCode,
|
||||
))
|
||||
}
|
||||
|
||||
// Append the generated metrics and samples to the storage.
|
||||
refs := make([]storage.SeriesRef, len(metrics))
|
||||
|
||||
for i := 0; i < numIntervals; i++ {
|
||||
a := stor.Appender(context.Background())
|
||||
ts := int64(i * interval)
|
||||
for j, metric := range metrics[:infoSeriesNum] {
|
||||
ref, _ := a.Append(refs[j], metric, ts, 1)
|
||||
refs[j] = ref
|
||||
}
|
||||
|
||||
for j, metric := range metrics[infoSeriesNum:] {
|
||||
ref, _ := a.Append(refs[j+infoSeriesNum], metric, ts, float64(i))
|
||||
refs[j+infoSeriesNum] = ref
|
||||
}
|
||||
|
||||
require.NoError(tb, a.Commit())
|
||||
}
|
||||
|
||||
stor.DB.ForceHeadMMap() // Ensure we have at most one head chunk for every series.
|
||||
stor.DB.Compact(ctx)
|
||||
}
|
||||
|
||||
func generateNativeHistogramSeries(app storage.Appender, numSeries int) error {
|
||||
commonLabels := []string{labels.MetricName, "native_histogram_series", "foo", "bar"}
|
||||
series := make([][]*histogram.Histogram, numSeries)
|
||||
|
|
|
@ -734,6 +734,7 @@ func (ng *Engine) execEvalStmt(ctx context.Context, query *query, s *parser.Eval
|
|||
samplesStats: query.sampleStats,
|
||||
noStepSubqueryIntervalFn: ng.noStepSubqueryIntervalFn,
|
||||
enableDelayedNameRemoval: ng.enableDelayedNameRemoval,
|
||||
querier: querier,
|
||||
}
|
||||
query.sampleStats.InitStepTracking(start, start, 1)
|
||||
|
||||
|
@ -792,6 +793,7 @@ func (ng *Engine) execEvalStmt(ctx context.Context, query *query, s *parser.Eval
|
|||
samplesStats: query.sampleStats,
|
||||
noStepSubqueryIntervalFn: ng.noStepSubqueryIntervalFn,
|
||||
enableDelayedNameRemoval: ng.enableDelayedNameRemoval,
|
||||
querier: querier,
|
||||
}
|
||||
query.sampleStats.InitStepTracking(evaluator.startTimestamp, evaluator.endTimestamp, evaluator.interval)
|
||||
val, warnings, err := evaluator.Eval(ctxInnerEval, s.Expr)
|
||||
|
@ -1058,6 +1060,7 @@ type evaluator struct {
|
|||
samplesStats *stats.QuerySamples
|
||||
noStepSubqueryIntervalFn func(rangeMillis int64) int64
|
||||
enableDelayedNameRemoval bool
|
||||
querier storage.Querier
|
||||
}
|
||||
|
||||
// errorf causes a panic with the input formatted into an error.
|
||||
|
@ -1430,19 +1433,18 @@ func (ev *evaluator) rangeEvalAgg(ctx context.Context, aggExpr *parser.Aggregate
|
|||
return result, warnings
|
||||
}
|
||||
|
||||
// evalVectorSelector generates a Matrix between ev.startTimestamp and ev.endTimestamp (inclusive), each point spaced ev.interval apart, from vs.
|
||||
// vs.Series has to be expanded before calling this method.
|
||||
// For every series iterator in vs.Series, the method iterates in ev.interval sized steps from ev.startTimestamp until and including ev.endTimestamp,
|
||||
// evalSeries generates a Matrix between ev.startTimestamp and ev.endTimestamp (inclusive), each point spaced ev.interval apart, from series given offset.
|
||||
// For every storage.Series iterator in series, the method iterates in ev.interval sized steps from ev.startTimestamp until and including ev.endTimestamp,
|
||||
// collecting every corresponding sample (obtained via ev.vectorSelectorSingle) into a Series.
|
||||
// All of the generated Series are collected into a Matrix, that gets returned.
|
||||
func (ev *evaluator) evalVectorSelector(ctx context.Context, vs *parser.VectorSelector) Matrix {
|
||||
func (ev *evaluator) evalSeries(ctx context.Context, series []storage.Series, offset time.Duration, recordOrigT bool) Matrix {
|
||||
numSteps := int((ev.endTimestamp-ev.startTimestamp)/ev.interval) + 1
|
||||
|
||||
mat := make(Matrix, 0, len(vs.Series))
|
||||
mat := make(Matrix, 0, len(series))
|
||||
var prevSS *Series
|
||||
it := storage.NewMemoizedEmptyIterator(durationMilliseconds(ev.lookbackDelta))
|
||||
var chkIter chunkenc.Iterator
|
||||
for _, s := range vs.Series {
|
||||
for _, s := range series {
|
||||
if err := contextDone(ctx, "expression evaluation"); err != nil {
|
||||
ev.error(err)
|
||||
}
|
||||
|
@ -1455,7 +1457,7 @@ func (ev *evaluator) evalVectorSelector(ctx context.Context, vs *parser.VectorSe
|
|||
|
||||
for ts, step := ev.startTimestamp, -1; ts <= ev.endTimestamp; ts += ev.interval {
|
||||
step++
|
||||
_, f, h, ok := ev.vectorSelectorSingle(it, vs, ts)
|
||||
origT, f, h, ok := ev.vectorSelectorSingle(it, offset, ts)
|
||||
if !ok {
|
||||
continue
|
||||
}
|
||||
|
@ -1469,8 +1471,18 @@ func (ev *evaluator) evalVectorSelector(ctx context.Context, vs *parser.VectorSe
|
|||
if ss.Floats == nil {
|
||||
ss.Floats = reuseOrGetFPointSlices(prevSS, numSteps)
|
||||
}
|
||||
if recordOrigT {
|
||||
// This is an info metric, where we want to track the original sample timestamp.
|
||||
// Info metric values should be 1 by convention, therefore we can re-use this
|
||||
// space in the sample.
|
||||
f = float64(origT)
|
||||
}
|
||||
ss.Floats = append(ss.Floats, FPoint{F: f, T: ts})
|
||||
} else {
|
||||
if recordOrigT {
|
||||
ev.error(fmt.Errorf("this should be an info metric, with float samples: %s", ss.Metric))
|
||||
}
|
||||
|
||||
point := HPoint{H: h, T: ts}
|
||||
histSize := point.size()
|
||||
ev.currentSamples += histSize
|
||||
|
@ -1640,6 +1652,8 @@ func (ev *evaluator) eval(ctx context.Context, expr parser.Expr) (parser.Value,
|
|||
return ev.evalLabelReplace(ctx, e.Args)
|
||||
case "label_join":
|
||||
return ev.evalLabelJoin(ctx, e.Args)
|
||||
case "info":
|
||||
return ev.evalInfo(ctx, e.Args)
|
||||
}
|
||||
|
||||
if !matrixArg {
|
||||
|
@ -1932,7 +1946,7 @@ func (ev *evaluator) eval(ctx context.Context, expr parser.Expr) (parser.Value,
|
|||
if err != nil {
|
||||
ev.error(errWithWarnings{fmt.Errorf("expanding series: %w", err), ws})
|
||||
}
|
||||
mat := ev.evalVectorSelector(ctx, e)
|
||||
mat := ev.evalSeries(ctx, e.Series, e.Offset, false)
|
||||
return mat, ws
|
||||
|
||||
case *parser.MatrixSelector:
|
||||
|
@ -1953,6 +1967,7 @@ func (ev *evaluator) eval(ctx context.Context, expr parser.Expr) (parser.Value,
|
|||
samplesStats: ev.samplesStats.NewChild(),
|
||||
noStepSubqueryIntervalFn: ev.noStepSubqueryIntervalFn,
|
||||
enableDelayedNameRemoval: ev.enableDelayedNameRemoval,
|
||||
querier: ev.querier,
|
||||
}
|
||||
|
||||
if e.Step != 0 {
|
||||
|
@ -1997,6 +2012,7 @@ func (ev *evaluator) eval(ctx context.Context, expr parser.Expr) (parser.Value,
|
|||
samplesStats: ev.samplesStats.NewChild(),
|
||||
noStepSubqueryIntervalFn: ev.noStepSubqueryIntervalFn,
|
||||
enableDelayedNameRemoval: ev.enableDelayedNameRemoval,
|
||||
querier: ev.querier,
|
||||
}
|
||||
res, ws := newEv.eval(ctx, e.Expr)
|
||||
ev.currentSamples = newEv.currentSamples
|
||||
|
@ -2097,7 +2113,7 @@ func (ev *evaluator) rangeEvalTimestampFunctionOverVectorSelector(ctx context.Co
|
|||
vec := make(Vector, 0, len(vs.Series))
|
||||
for i, s := range vs.Series {
|
||||
it := seriesIterators[i]
|
||||
t, _, _, ok := ev.vectorSelectorSingle(it, vs, enh.Ts)
|
||||
t, _, _, ok := ev.vectorSelectorSingle(it, vs.Offset, enh.Ts)
|
||||
if !ok {
|
||||
continue
|
||||
}
|
||||
|
@ -2121,10 +2137,10 @@ func (ev *evaluator) rangeEvalTimestampFunctionOverVectorSelector(ctx context.Co
|
|||
}
|
||||
|
||||
// vectorSelectorSingle evaluates an instant vector for the iterator of one time series.
|
||||
func (ev *evaluator) vectorSelectorSingle(it *storage.MemoizedSeriesIterator, node *parser.VectorSelector, ts int64) (
|
||||
func (ev *evaluator) vectorSelectorSingle(it *storage.MemoizedSeriesIterator, offset time.Duration, ts int64) (
|
||||
int64, float64, *histogram.FloatHistogram, bool,
|
||||
) {
|
||||
refTime := ts - durationMilliseconds(node.Offset)
|
||||
refTime := ts - durationMilliseconds(offset)
|
||||
var t int64
|
||||
var v float64
|
||||
var h *histogram.FloatHistogram
|
||||
|
|
|
@ -1677,6 +1677,7 @@ var FunctionCalls = map[string]FunctionCall{
|
|||
"hour": funcHour,
|
||||
"idelta": funcIdelta,
|
||||
"increase": funcIncrease,
|
||||
"info": nil,
|
||||
"irate": funcIrate,
|
||||
"label_replace": funcLabelReplace,
|
||||
"label_join": funcLabelJoin,
|
||||
|
|
454
promql/info.go
Normal file
454
promql/info.go
Normal file
|
@ -0,0 +1,454 @@
|
|||
// 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 promql
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"slices"
|
||||
"strings"
|
||||
|
||||
"github.com/grafana/regexp"
|
||||
|
||||
"github.com/prometheus/prometheus/model/labels"
|
||||
"github.com/prometheus/prometheus/promql/parser"
|
||||
"github.com/prometheus/prometheus/storage"
|
||||
"github.com/prometheus/prometheus/util/annotations"
|
||||
)
|
||||
|
||||
const targetInfo = "target_info"
|
||||
|
||||
// identifyingLabels are the labels we consider as identifying for info metrics.
|
||||
// Currently hard coded, so we don't need knowledge of individual info metrics.
|
||||
var identifyingLabels = []string{"instance", "job"}
|
||||
|
||||
// evalInfo implements the info PromQL function.
|
||||
func (ev *evaluator) evalInfo(ctx context.Context, args parser.Expressions) (parser.Value, annotations.Annotations) {
|
||||
val, annots := ev.eval(ctx, args[0])
|
||||
mat := val.(Matrix)
|
||||
// Map from data label name to matchers.
|
||||
dataLabelMatchers := map[string][]*labels.Matcher{}
|
||||
var infoNameMatchers []*labels.Matcher
|
||||
if len(args) > 1 {
|
||||
// TODO: Introduce a dedicated LabelSelector type.
|
||||
labelSelector := args[1].(*parser.VectorSelector)
|
||||
for _, m := range labelSelector.LabelMatchers {
|
||||
dataLabelMatchers[m.Name] = append(dataLabelMatchers[m.Name], m)
|
||||
if m.Name == labels.MetricName {
|
||||
infoNameMatchers = append(infoNameMatchers, m)
|
||||
}
|
||||
}
|
||||
} else {
|
||||
infoNameMatchers = []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, labels.MetricName, targetInfo)}
|
||||
}
|
||||
|
||||
// Don't try to enrich info series.
|
||||
ignoreSeries := map[int]struct{}{}
|
||||
loop:
|
||||
for i, s := range mat {
|
||||
name := s.Metric.Get(labels.MetricName)
|
||||
for _, m := range infoNameMatchers {
|
||||
if m.Matches(name) {
|
||||
ignoreSeries[i] = struct{}{}
|
||||
continue loop
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
selectHints := ev.infoSelectHints(args[0])
|
||||
infoSeries, ws, err := ev.fetchInfoSeries(ctx, mat, ignoreSeries, dataLabelMatchers, selectHints)
|
||||
if err != nil {
|
||||
ev.error(err)
|
||||
}
|
||||
annots.Merge(ws)
|
||||
|
||||
res, ws := ev.combineWithInfoSeries(ctx, mat, infoSeries, ignoreSeries, dataLabelMatchers)
|
||||
annots.Merge(ws)
|
||||
return res, annots
|
||||
}
|
||||
|
||||
// infoSelectHints calculates the storage.SelectHints for selecting info series, given expr (first argument to info call).
|
||||
func (ev *evaluator) infoSelectHints(expr parser.Expr) storage.SelectHints {
|
||||
var nodeTimestamp *int64
|
||||
var offset int64
|
||||
parser.Inspect(expr, func(node parser.Node, path []parser.Node) error {
|
||||
switch n := node.(type) {
|
||||
case *parser.VectorSelector:
|
||||
if n.Timestamp != nil {
|
||||
nodeTimestamp = n.Timestamp
|
||||
}
|
||||
offset = durationMilliseconds(n.OriginalOffset)
|
||||
return fmt.Errorf("end traversal")
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
})
|
||||
|
||||
start := ev.startTimestamp
|
||||
end := ev.endTimestamp
|
||||
if nodeTimestamp != nil {
|
||||
// The timestamp on the selector overrides everything.
|
||||
start = *nodeTimestamp
|
||||
end = *nodeTimestamp
|
||||
}
|
||||
// Reduce the start by one fewer ms than the lookback delta
|
||||
// because wo want to exclude samples that are precisely the
|
||||
// lookback delta before the eval time.
|
||||
start -= durationMilliseconds(ev.lookbackDelta) - 1
|
||||
start -= offset
|
||||
end -= offset
|
||||
|
||||
return storage.SelectHints{
|
||||
Start: start,
|
||||
End: end,
|
||||
Step: ev.interval,
|
||||
Func: "info",
|
||||
}
|
||||
}
|
||||
|
||||
// fetchInfoSeries fetches info series given matching identifying labels in mat.
|
||||
// Series in ignoreSeries are not fetched.
|
||||
// dataLabelMatchers may be mutated.
|
||||
func (ev *evaluator) fetchInfoSeries(ctx context.Context, mat Matrix, ignoreSeries map[int]struct{}, dataLabelMatchers map[string][]*labels.Matcher, selectHints storage.SelectHints) (Matrix, annotations.Annotations, error) {
|
||||
// A map of values for all identifying labels we are interested in.
|
||||
idLblValues := map[string]map[string]struct{}{}
|
||||
for i, s := range mat {
|
||||
if _, exists := ignoreSeries[i]; exists {
|
||||
continue
|
||||
}
|
||||
|
||||
// Register relevant values per identifying label for this series.
|
||||
for _, l := range identifyingLabels {
|
||||
val := s.Metric.Get(l)
|
||||
if val == "" {
|
||||
continue
|
||||
}
|
||||
|
||||
if idLblValues[l] == nil {
|
||||
idLblValues[l] = map[string]struct{}{}
|
||||
}
|
||||
idLblValues[l][val] = struct{}{}
|
||||
}
|
||||
}
|
||||
if len(idLblValues) == 0 {
|
||||
return nil, nil, nil
|
||||
}
|
||||
|
||||
// Generate regexps for every interesting value per identifying label.
|
||||
var sb strings.Builder
|
||||
idLblRegexps := make(map[string]string, len(idLblValues))
|
||||
for name, vals := range idLblValues {
|
||||
sb.Reset()
|
||||
i := 0
|
||||
for v := range vals {
|
||||
if i > 0 {
|
||||
sb.WriteRune('|')
|
||||
}
|
||||
sb.WriteString(regexp.QuoteMeta(v))
|
||||
i++
|
||||
}
|
||||
idLblRegexps[name] = sb.String()
|
||||
}
|
||||
|
||||
var infoLabelMatchers []*labels.Matcher
|
||||
for name, re := range idLblRegexps {
|
||||
infoLabelMatchers = append(infoLabelMatchers, labels.MustNewMatcher(labels.MatchRegexp, name, re))
|
||||
}
|
||||
var nameMatcher *labels.Matcher
|
||||
for name, ms := range dataLabelMatchers {
|
||||
for i, m := range ms {
|
||||
if m.Name == labels.MetricName {
|
||||
nameMatcher = m
|
||||
ms = slices.Delete(ms, i, i+1)
|
||||
}
|
||||
infoLabelMatchers = append(infoLabelMatchers, m)
|
||||
}
|
||||
if len(ms) > 0 {
|
||||
dataLabelMatchers[name] = ms
|
||||
} else {
|
||||
delete(dataLabelMatchers, name)
|
||||
}
|
||||
}
|
||||
if nameMatcher == nil {
|
||||
// Default to using the target_info metric.
|
||||
infoLabelMatchers = append([]*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, labels.MetricName, targetInfo)}, infoLabelMatchers...)
|
||||
}
|
||||
|
||||
infoIt := ev.querier.Select(ctx, false, &selectHints, infoLabelMatchers...)
|
||||
infoSeries, ws, err := expandSeriesSet(ctx, infoIt)
|
||||
if err != nil {
|
||||
return nil, ws, err
|
||||
}
|
||||
|
||||
infoMat := ev.evalSeries(ctx, infoSeries, 0, true)
|
||||
return infoMat, ws, nil
|
||||
}
|
||||
|
||||
// combineWithInfoSeries combines mat with select data labels from infoMat.
|
||||
func (ev *evaluator) combineWithInfoSeries(ctx context.Context, mat, infoMat Matrix, ignoreSeries map[int]struct{}, dataLabelMatchers map[string][]*labels.Matcher) (Matrix, annotations.Annotations) {
|
||||
buf := make([]byte, 0, 1024)
|
||||
lb := labels.NewScratchBuilder(0)
|
||||
sigFunction := func(name string) func(labels.Labels) string {
|
||||
return func(lset labels.Labels) string {
|
||||
lb.Reset()
|
||||
lb.Add(labels.MetricName, name)
|
||||
lset.MatchLabels(true, identifyingLabels...).Range(func(l labels.Label) {
|
||||
lb.Add(l.Name, l.Value)
|
||||
})
|
||||
lb.Sort()
|
||||
return string(lb.Labels().Bytes(buf))
|
||||
}
|
||||
}
|
||||
|
||||
infoMetrics := map[string]struct{}{}
|
||||
for _, is := range infoMat {
|
||||
lblMap := is.Metric.Map()
|
||||
infoMetrics[lblMap[labels.MetricName]] = struct{}{}
|
||||
}
|
||||
sigfs := make(map[string]func(labels.Labels) string, len(infoMetrics))
|
||||
for name := range infoMetrics {
|
||||
sigfs[name] = sigFunction(name)
|
||||
}
|
||||
|
||||
// Keep a copy of the original point slices so they can be returned to the pool.
|
||||
origMatrices := []Matrix{
|
||||
make(Matrix, len(mat)),
|
||||
make(Matrix, len(infoMat)),
|
||||
}
|
||||
copy(origMatrices[0], mat)
|
||||
copy(origMatrices[1], infoMat)
|
||||
|
||||
numSteps := int((ev.endTimestamp-ev.startTimestamp)/ev.interval) + 1
|
||||
originalNumSamples := ev.currentSamples
|
||||
|
||||
// Create an output vector that is as big as the input matrix with
|
||||
// the most time series.
|
||||
biggestLen := max(len(mat), len(infoMat))
|
||||
baseVector := make(Vector, 0, len(mat))
|
||||
infoVector := make(Vector, 0, len(infoMat))
|
||||
enh := &EvalNodeHelper{
|
||||
Out: make(Vector, 0, biggestLen),
|
||||
}
|
||||
type seriesAndTimestamp struct {
|
||||
Series
|
||||
ts int64
|
||||
}
|
||||
seriess := make(map[uint64]seriesAndTimestamp, biggestLen) // Output series by series hash.
|
||||
tempNumSamples := ev.currentSamples
|
||||
|
||||
// For every base series, compute signature per info metric.
|
||||
baseSigs := make([]map[string]string, 0, len(mat))
|
||||
for _, s := range mat {
|
||||
sigs := make(map[string]string, len(infoMetrics))
|
||||
for infoName := range infoMetrics {
|
||||
sigs[infoName] = sigfs[infoName](s.Metric)
|
||||
}
|
||||
baseSigs = append(baseSigs, sigs)
|
||||
}
|
||||
|
||||
infoSigs := make([]string, 0, len(infoMat))
|
||||
for _, s := range infoMat {
|
||||
name := s.Metric.Map()[labels.MetricName]
|
||||
infoSigs = append(infoSigs, sigfs[name](s.Metric))
|
||||
}
|
||||
|
||||
var warnings annotations.Annotations
|
||||
for ts := ev.startTimestamp; ts <= ev.endTimestamp; ts += ev.interval {
|
||||
if err := contextDone(ctx, "expression evaluation"); err != nil {
|
||||
ev.error(err)
|
||||
}
|
||||
|
||||
// Reset number of samples in memory after each timestamp.
|
||||
ev.currentSamples = tempNumSamples
|
||||
// Gather input vectors for this timestamp.
|
||||
baseVector, _ = ev.gatherVector(ts, mat, baseVector, nil, nil)
|
||||
infoVector, _ = ev.gatherVector(ts, infoMat, infoVector, nil, nil)
|
||||
|
||||
enh.Ts = ts
|
||||
result, err := ev.combineWithInfoVector(baseVector, infoVector, ignoreSeries, baseSigs, infoSigs, enh, dataLabelMatchers)
|
||||
if err != nil {
|
||||
ev.error(err)
|
||||
}
|
||||
enh.Out = result[:0] // Reuse result vector.
|
||||
|
||||
vecNumSamples := result.TotalSamples()
|
||||
ev.currentSamples += vecNumSamples
|
||||
// When we reset currentSamples to tempNumSamples during the next iteration of the loop it also
|
||||
// needs to include the samples from the result here, as they're still in memory.
|
||||
tempNumSamples += vecNumSamples
|
||||
ev.samplesStats.UpdatePeak(ev.currentSamples)
|
||||
if ev.currentSamples > ev.maxSamples {
|
||||
ev.error(ErrTooManySamples(env))
|
||||
}
|
||||
|
||||
// Add samples in result vector to output series.
|
||||
for _, sample := range result {
|
||||
h := sample.Metric.Hash()
|
||||
ss, exists := seriess[h]
|
||||
if exists {
|
||||
if ss.ts == ts { // If we've seen this output series before at this timestamp, it's a duplicate.
|
||||
ev.errorf("vector cannot contain metrics with the same labelset")
|
||||
}
|
||||
ss.ts = ts
|
||||
} else {
|
||||
ss = seriesAndTimestamp{Series{Metric: sample.Metric}, ts}
|
||||
}
|
||||
addToSeries(&ss.Series, enh.Ts, sample.F, sample.H, numSteps)
|
||||
seriess[h] = ss
|
||||
}
|
||||
}
|
||||
|
||||
// Reuse the original point slices.
|
||||
for _, m := range origMatrices {
|
||||
for _, s := range m {
|
||||
putFPointSlice(s.Floats)
|
||||
putHPointSlice(s.Histograms)
|
||||
}
|
||||
}
|
||||
// Assemble the output matrix. By the time we get here we know we don't have too many samples.
|
||||
numSamples := 0
|
||||
output := make(Matrix, 0, len(seriess))
|
||||
for _, ss := range seriess {
|
||||
numSamples += len(ss.Floats) + totalHPointSize(ss.Histograms)
|
||||
output = append(output, ss.Series)
|
||||
}
|
||||
ev.currentSamples = originalNumSamples + numSamples
|
||||
ev.samplesStats.UpdatePeak(ev.currentSamples)
|
||||
return output, warnings
|
||||
}
|
||||
|
||||
// combineWithInfoVector combines base and info Vectors.
|
||||
// Base series in ignoreSeries are not combined.
|
||||
func (ev *evaluator) combineWithInfoVector(base, info Vector, ignoreSeries map[int]struct{}, baseSigs []map[string]string, infoSigs []string, enh *EvalNodeHelper, dataLabelMatchers map[string][]*labels.Matcher) (Vector, error) {
|
||||
if len(base) == 0 {
|
||||
return nil, nil // Short-circuit: nothing is going to match.
|
||||
}
|
||||
|
||||
// All samples from the info Vector hashed by the matching label/values.
|
||||
if enh.rightSigs == nil {
|
||||
enh.rightSigs = make(map[string]Sample, len(enh.Out))
|
||||
} else {
|
||||
clear(enh.rightSigs)
|
||||
}
|
||||
|
||||
for i, s := range info {
|
||||
if s.H != nil {
|
||||
ev.error(errors.New("info sample should be float"))
|
||||
}
|
||||
// We encode original info sample timestamps via the float value.
|
||||
origT := int64(s.F)
|
||||
|
||||
sig := infoSigs[i]
|
||||
if existing, exists := enh.rightSigs[sig]; exists {
|
||||
// We encode original info sample timestamps via the float value.
|
||||
existingOrigT := int64(existing.F)
|
||||
switch {
|
||||
case existingOrigT > origT:
|
||||
// Keep the other info sample, since it's newer.
|
||||
case existingOrigT < origT:
|
||||
// Keep this info sample, since it's newer.
|
||||
enh.rightSigs[sig] = s
|
||||
default:
|
||||
// The two info samples have the same timestamp - conflict.
|
||||
name := s.Metric.Map()[labels.MetricName]
|
||||
ev.errorf("found duplicate series for info metric %s", name)
|
||||
}
|
||||
} else {
|
||||
enh.rightSigs[sig] = s
|
||||
}
|
||||
}
|
||||
|
||||
for i, bs := range base {
|
||||
if _, exists := ignoreSeries[i]; exists {
|
||||
// This series should not be enriched with info metric data labels.
|
||||
enh.Out = append(enh.Out, Sample{
|
||||
Metric: bs.Metric,
|
||||
F: bs.F,
|
||||
H: bs.H,
|
||||
})
|
||||
continue
|
||||
}
|
||||
|
||||
baseLabels := bs.Metric.Map()
|
||||
enh.resetBuilder(labels.Labels{})
|
||||
|
||||
// For every info metric name, try to find an info series with the same signature.
|
||||
seenInfoMetrics := map[string]struct{}{}
|
||||
for infoName, sig := range baseSigs[i] {
|
||||
is, exists := enh.rightSigs[sig]
|
||||
if !exists {
|
||||
continue
|
||||
}
|
||||
if _, exists := seenInfoMetrics[infoName]; exists {
|
||||
continue
|
||||
}
|
||||
|
||||
err := is.Metric.Validate(func(l labels.Label) error {
|
||||
if l.Name == labels.MetricName {
|
||||
return nil
|
||||
}
|
||||
if _, exists := dataLabelMatchers[l.Name]; len(dataLabelMatchers) > 0 && !exists {
|
||||
// Not among the specified data label matchers.
|
||||
return nil
|
||||
}
|
||||
|
||||
if v := enh.lb.Get(l.Name); v != "" && v != l.Value {
|
||||
return fmt.Errorf("conflicting label: %s", l.Name)
|
||||
}
|
||||
if _, exists := baseLabels[l.Name]; exists {
|
||||
// Skip labels already on the base metric.
|
||||
return nil
|
||||
}
|
||||
|
||||
enh.lb.Set(l.Name, l.Value)
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
seenInfoMetrics[infoName] = struct{}{}
|
||||
}
|
||||
|
||||
infoLbls := enh.lb.Labels()
|
||||
if infoLbls.Len() == 0 {
|
||||
// If there's at least one data label matcher not matching the empty string,
|
||||
// we have to ignore this series as there are no matching info series.
|
||||
allMatchersMatchEmpty := true
|
||||
for _, ms := range dataLabelMatchers {
|
||||
for _, m := range ms {
|
||||
if !m.Matches("") {
|
||||
allMatchersMatchEmpty = false
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
if !allMatchersMatchEmpty {
|
||||
continue
|
||||
}
|
||||
}
|
||||
|
||||
enh.resetBuilder(bs.Metric)
|
||||
infoLbls.Range(func(l labels.Label) {
|
||||
enh.lb.Set(l.Name, l.Value)
|
||||
})
|
||||
|
||||
enh.Out = append(enh.Out, Sample{
|
||||
Metric: enh.lb.Labels(),
|
||||
F: bs.F,
|
||||
H: bs.H,
|
||||
})
|
||||
}
|
||||
return enh.Out, nil
|
||||
}
|
140
promql/info_test.go
Normal file
140
promql/info_test.go
Normal file
|
@ -0,0 +1,140 @@
|
|||
// 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 promql_test
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/prometheus/prometheus/promql/promqltest"
|
||||
)
|
||||
|
||||
// The "info" function is experimental. This is why we write those tests here for now instead of promqltest/testdata/info.test.
|
||||
func TestInfo(t *testing.T) {
|
||||
engine := promqltest.NewTestEngine(t, false, 0, promqltest.DefaultMaxSamplesPerQuery)
|
||||
promqltest.RunTest(t, `
|
||||
load 5m
|
||||
metric{instance="a", job="1", label="value"} 0 1 2
|
||||
metric_not_matching_target_info{instance="a", job="2", label="value"} 0 1 2
|
||||
metric_with_overlapping_label{instance="a", job="1", label="value", data="base"} 0 1 2
|
||||
target_info{instance="a", job="1", data="info", another_data="another info"} 1 1 1
|
||||
build_info{instance="a", job="1", build_data="build"} 1 1 1
|
||||
|
||||
# Include one info metric data label.
|
||||
eval range from 0m to 10m step 5m info(metric, {data=~".+"})
|
||||
metric{data="info", instance="a", job="1", label="value"} 0 1 2
|
||||
|
||||
# Include all info metric data labels.
|
||||
eval range from 0m to 10m step 5m info(metric)
|
||||
metric{data="info", instance="a", job="1", label="value", another_data="another info"} 0 1 2
|
||||
|
||||
# Try including all info metric data labels, but non-matching identifying labels.
|
||||
eval range from 0m to 10m step 5m info(metric_not_matching_target_info)
|
||||
metric_not_matching_target_info{instance="a", job="2", label="value"} 0 1 2
|
||||
|
||||
# Try including a certain info metric data label with a non-matching matcher not accepting empty labels.
|
||||
# Metric is ignored, due there being a data label matcher not matching empty labels,
|
||||
# and there being no info series matches.
|
||||
eval range from 0m to 10m step 5m info(metric, {non_existent=~".+"})
|
||||
|
||||
# Include a certain info metric data label together with a non-matching matcher accepting empty labels.
|
||||
# Since the non_existent matcher matches empty labels, it's simply ignored when there's no match.
|
||||
# XXX: This case has to include a matcher not matching empty labels, due the PromQL limitation
|
||||
# that vector selectors have to contain at least one matcher not accepting empty labels.
|
||||
# We might need another construct than vector selector to get around this limitation.
|
||||
eval range from 0m to 10m step 5m info(metric, {data=~".+", non_existent=~".*"})
|
||||
metric{data="info", instance="a", job="1", label="value"} 0 1 2
|
||||
|
||||
# Info series data labels overlapping with those of base series are ignored.
|
||||
eval range from 0m to 10m step 5m info(metric_with_overlapping_label)
|
||||
metric_with_overlapping_label{data="base", instance="a", job="1", label="value", another_data="another info"} 0 1 2
|
||||
|
||||
# Include data labels from target_info specifically.
|
||||
eval range from 0m to 10m step 5m info(metric, {__name__="target_info"})
|
||||
metric{data="info", instance="a", job="1", label="value", another_data="another info"} 0 1 2
|
||||
|
||||
# Try to include all data labels from a non-existent info metric.
|
||||
eval range from 0m to 10m step 5m info(metric, {__name__="non_existent"})
|
||||
metric{instance="a", job="1", label="value"} 0 1 2
|
||||
|
||||
# Try to include a certain data label from a non-existent info metric.
|
||||
eval range from 0m to 10m step 5m info(metric, {__name__="non_existent", data=~".+"})
|
||||
|
||||
# Include data labels from build_info.
|
||||
eval range from 0m to 10m step 5m info(metric, {__name__="build_info"})
|
||||
metric{instance="a", job="1", label="value", build_data="build"} 0 1 2
|
||||
|
||||
# Include data labels from build_info and target_info.
|
||||
eval range from 0m to 10m step 5m info(metric, {__name__=~".+_info"})
|
||||
metric{instance="a", job="1", label="value", build_data="build", data="info", another_data="another info"} 0 1 2
|
||||
|
||||
# Info metrics themselves are ignored when it comes to enriching with info metric data labels.
|
||||
eval range from 0m to 10m step 5m info(build_info, {__name__=~".+_info", build_data=~".+"})
|
||||
build_info{instance="a", job="1", build_data="build"} 1 1 1
|
||||
|
||||
clear
|
||||
|
||||
# Overlapping target_info series.
|
||||
load 5m
|
||||
metric{instance="a", job="1", label="value"} 0 1 2
|
||||
target_info{instance="a", job="1", data="info", another_data="another info"} 1 1 _
|
||||
target_info{instance="a", job="1", data="updated info", another_data="another info"} _ _ 1
|
||||
|
||||
# Conflicting info series are resolved through picking the latest sample.
|
||||
eval range from 0m to 10m step 5m info(metric)
|
||||
metric{data="info", instance="a", job="1", label="value", another_data="another info"} 0 1 _
|
||||
metric{data="updated info", instance="a", job="1", label="value", another_data="another info"} _ _ 2
|
||||
|
||||
clear
|
||||
|
||||
# Non-overlapping target_info series.
|
||||
load 5m
|
||||
metric{instance="a", job="1", label="value"} 0 1 2
|
||||
target_info{instance="a", job="1", data="info"} 1 1 stale
|
||||
target_info{instance="a", job="1", data="updated info"} _ _ 1
|
||||
|
||||
# Include info metric data labels from a metric which data labels change over time.
|
||||
eval range from 0m to 10m step 5m info(metric)
|
||||
metric{data="info", instance="a", job="1", label="value"} 0 1 _
|
||||
metric{data="updated info", instance="a", job="1", label="value"} _ _ 2
|
||||
|
||||
clear
|
||||
|
||||
# Info series selector matches histogram series, info metrics should be float type.
|
||||
load 5m
|
||||
metric{instance="a", job="1", label="value"} 0 1 2
|
||||
histogram{instance="a", job="1"} {{schema:1 sum:3 count:22 buckets:[5 10 7]}}
|
||||
|
||||
eval_fail range from 0m to 10m step 5m info(metric, {__name__="histogram"})
|
||||
|
||||
clear
|
||||
|
||||
# Series with skipped scrape.
|
||||
load 1m
|
||||
metric{instance="a", job="1", label="value"} 0 _ 2 3 4
|
||||
target_info{instance="a", job="1", data="info"} 1 _ 1 1 1
|
||||
|
||||
# Lookback works also for the info series.
|
||||
eval range from 1m to 4m step 1m info(metric)
|
||||
metric{data="info", instance="a", job="1", label="value"} 0 2 3 4
|
||||
|
||||
# @ operator works also with info.
|
||||
# Note that we pick the timestamp missing a sample, lookback should pick previous sample.
|
||||
eval range from 1m to 4m step 1m info(metric @ 60)
|
||||
metric{data="info", instance="a", job="1", label="value"} 0 0 0 0
|
||||
|
||||
# offset operator works also with info.
|
||||
eval range from 1m to 4m step 1m info(metric offset 1m)
|
||||
metric{data="info", instance="a", job="1", label="value"} 0 0 2 3
|
||||
`, engine)
|
||||
}
|
|
@ -208,6 +208,10 @@ type VectorSelector struct {
|
|||
UnexpandedSeriesSet storage.SeriesSet
|
||||
Series []storage.Series
|
||||
|
||||
// BypassEmptyMatcherCheck is true when the VectorSelector isn't required to have at least one matcher matching the empty string.
|
||||
// This is the case when VectorSelector is used to represent the info function's second argument.
|
||||
BypassEmptyMatcherCheck bool
|
||||
|
||||
PosRange posrange.PositionRange
|
||||
}
|
||||
|
||||
|
|
|
@ -223,6 +223,13 @@ var Functions = map[string]*Function{
|
|||
ArgTypes: []ValueType{ValueTypeMatrix},
|
||||
ReturnType: ValueTypeVector,
|
||||
},
|
||||
"info": {
|
||||
Name: "info",
|
||||
ArgTypes: []ValueType{ValueTypeVector, ValueTypeVector},
|
||||
ReturnType: ValueTypeVector,
|
||||
Experimental: true,
|
||||
Variadic: 1,
|
||||
},
|
||||
"irate": {
|
||||
Name: "irate",
|
||||
ArgTypes: []ValueType{ValueTypeMatrix},
|
||||
|
|
|
@ -784,6 +784,19 @@ func (p *parser) checkAST(node Node) (typ ValueType) {
|
|||
}
|
||||
}
|
||||
|
||||
if n.Func.Name == "info" && len(n.Args) > 1 {
|
||||
// Check the type is correct first
|
||||
if n.Args[1].Type() != ValueTypeVector {
|
||||
p.addParseErrf(node.PositionRange(), "expected type %s in %s, got %s", DocumentedType(ValueTypeVector), fmt.Sprintf("call to function %q", n.Func.Name), DocumentedType(n.Args[1].Type()))
|
||||
}
|
||||
// Check the vector selector in the input doesn't contain a metric name
|
||||
if n.Args[1].(*VectorSelector).Name != "" {
|
||||
p.addParseErrf(n.Args[1].PositionRange(), "expected label selectors only, got vector selector instead")
|
||||
}
|
||||
// Set Vector Selector flag to bypass empty matcher check
|
||||
n.Args[1].(*VectorSelector).BypassEmptyMatcherCheck = true
|
||||
}
|
||||
|
||||
for i, arg := range n.Args {
|
||||
if i >= len(n.Func.ArgTypes) {
|
||||
if n.Func.Variadic == 0 {
|
||||
|
@ -830,17 +843,19 @@ func (p *parser) checkAST(node Node) (typ ValueType) {
|
|||
// metric name is a non-empty matcher.
|
||||
break
|
||||
}
|
||||
// A Vector selector must contain at least one non-empty matcher to prevent
|
||||
// implicit selection of all metrics (e.g. by a typo).
|
||||
notEmpty := false
|
||||
for _, lm := range n.LabelMatchers {
|
||||
if lm != nil && !lm.Matches("") {
|
||||
notEmpty = true
|
||||
break
|
||||
if !n.BypassEmptyMatcherCheck {
|
||||
// A Vector selector must contain at least one non-empty matcher to prevent
|
||||
// implicit selection of all metrics (e.g. by a typo).
|
||||
notEmpty := false
|
||||
for _, lm := range n.LabelMatchers {
|
||||
if lm != nil && !lm.Matches("") {
|
||||
notEmpty = true
|
||||
break
|
||||
}
|
||||
}
|
||||
if !notEmpty {
|
||||
p.addParseErrf(n.PositionRange(), "vector selector must contain at least one non-empty matcher")
|
||||
}
|
||||
}
|
||||
if !notEmpty {
|
||||
p.addParseErrf(n.PositionRange(), "vector selector must contain at least one non-empty matcher")
|
||||
}
|
||||
|
||||
case *NumberLiteral, *StringLiteral:
|
||||
|
|
|
@ -3872,6 +3872,81 @@ var testExpr = []struct {
|
|||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
input: `info(rate(http_request_counter_total{}[5m]))`,
|
||||
expected: &Call{
|
||||
Func: MustGetFunction("info"),
|
||||
Args: Expressions{
|
||||
&Call{
|
||||
Func: MustGetFunction("rate"),
|
||||
PosRange: posrange.PositionRange{
|
||||
Start: 5,
|
||||
End: 43,
|
||||
},
|
||||
Args: Expressions{
|
||||
&MatrixSelector{
|
||||
VectorSelector: &VectorSelector{
|
||||
Name: "http_request_counter_total",
|
||||
OriginalOffset: 0,
|
||||
LabelMatchers: []*labels.Matcher{
|
||||
MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "http_request_counter_total"),
|
||||
},
|
||||
PosRange: posrange.PositionRange{
|
||||
Start: 10,
|
||||
End: 38,
|
||||
},
|
||||
},
|
||||
EndPos: 42,
|
||||
Range: 5 * time.Minute,
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
PosRange: posrange.PositionRange{
|
||||
Start: 0,
|
||||
End: 44,
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
input: `info(rate(http_request_counter_total{}[5m]), target_info{foo="bar"})`,
|
||||
fail: true,
|
||||
errMsg: `1:46: parse error: expected label selectors only, got vector selector instead`,
|
||||
},
|
||||
{
|
||||
input: `info(http_request_counter_total{namespace="zzz"}, {foo="bar", bar="baz"})`,
|
||||
expected: &Call{
|
||||
Func: MustGetFunction("info"),
|
||||
Args: Expressions{
|
||||
&VectorSelector{
|
||||
Name: "http_request_counter_total",
|
||||
LabelMatchers: []*labels.Matcher{
|
||||
MustLabelMatcher(labels.MatchEqual, "namespace", "zzz"),
|
||||
MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "http_request_counter_total"),
|
||||
},
|
||||
PosRange: posrange.PositionRange{
|
||||
Start: 5,
|
||||
End: 48,
|
||||
},
|
||||
},
|
||||
&VectorSelector{
|
||||
LabelMatchers: []*labels.Matcher{
|
||||
MustLabelMatcher(labels.MatchEqual, "foo", "bar"),
|
||||
MustLabelMatcher(labels.MatchEqual, "bar", "baz"),
|
||||
},
|
||||
PosRange: posrange.PositionRange{
|
||||
Start: 50,
|
||||
End: 72,
|
||||
},
|
||||
BypassEmptyMatcherCheck: true,
|
||||
},
|
||||
},
|
||||
PosRange: posrange.PositionRange{
|
||||
Start: 0,
|
||||
End: 73,
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
func makeInt64Pointer(val int64) *int64 {
|
||||
|
@ -3889,6 +3964,12 @@ func readable(s string) string {
|
|||
}
|
||||
|
||||
func TestParseExpressions(t *testing.T) {
|
||||
// Enable experimental functions testing.
|
||||
EnableExperimentalFunctions = true
|
||||
t.Cleanup(func() {
|
||||
EnableExperimentalFunctions = false
|
||||
})
|
||||
|
||||
model.NameValidationScheme = model.UTF8Validation
|
||||
for _, test := range testExpr {
|
||||
t.Run(readable(test.input), func(t *testing.T) {
|
||||
|
|
|
@ -281,6 +281,12 @@ export const functionIdentifierTerms = [
|
|||
info: 'Calculate the increase in value over a range of time (for counters)',
|
||||
type: 'function',
|
||||
},
|
||||
{
|
||||
label: 'info',
|
||||
detail: 'function',
|
||||
info: 'Add data labels from corresponding info metrics',
|
||||
type: 'function',
|
||||
},
|
||||
{
|
||||
label: 'irate',
|
||||
detail: 'function',
|
||||
|
|
|
@ -968,6 +968,23 @@ describe('promql operations', () => {
|
|||
expectedValueType: ValueType.vector,
|
||||
expectedDiag: [],
|
||||
},
|
||||
{
|
||||
expr: 'info(rate(http_request_counter_total{}[5m]))',
|
||||
expectedValueType: ValueType.vector,
|
||||
expectedDiag: [],
|
||||
},
|
||||
{
|
||||
expr: 'info(rate(http_request_counter_total[5m]), target_info{service_version=~".+"})',
|
||||
expectedValueType: ValueType.vector,
|
||||
expectedDiag: [
|
||||
{
|
||||
from: 0,
|
||||
to: 78,
|
||||
message: `expected label selectors as the second argument to "info" function, got [object Object]`,
|
||||
severity: 'error',
|
||||
},
|
||||
],
|
||||
},
|
||||
];
|
||||
testCases.forEach((value) => {
|
||||
const state = createEditorState(value.expr);
|
||||
|
|
|
@ -275,6 +275,13 @@ export class Parser {
|
|||
}
|
||||
}
|
||||
|
||||
if (funcSignature.name === 'info') {
|
||||
// Verify that the data label selector expression is not prefixed with metric name.
|
||||
if (args.length > 1 && args[1].getChild(Identifier)) {
|
||||
this.addDiagnostic(node, `expected label selectors as the second argument to "info" function, got ${args[1].type}`);
|
||||
}
|
||||
}
|
||||
|
||||
let j = 0;
|
||||
for (let i = 0; i < args.length; i++) {
|
||||
j = i;
|
||||
|
|
|
@ -50,6 +50,7 @@ import {
|
|||
Hour,
|
||||
Idelta,
|
||||
Increase,
|
||||
Info,
|
||||
Irate,
|
||||
LabelJoin,
|
||||
LabelReplace,
|
||||
|
@ -336,6 +337,12 @@ const promqlFunctions: { [key: number]: PromQLFunction } = {
|
|||
variadic: 0,
|
||||
returnType: ValueType.vector,
|
||||
},
|
||||
[Info]: {
|
||||
name: 'info',
|
||||
argTypes: [ValueType.vector, ValueType.vector],
|
||||
variadic: 1,
|
||||
returnType: ValueType.vector,
|
||||
},
|
||||
[Irate]: {
|
||||
name: 'irate',
|
||||
argTypes: [ValueType.matrix],
|
||||
|
|
|
@ -145,6 +145,7 @@ FunctionIdentifier {
|
|||
Hour |
|
||||
Idelta |
|
||||
Increase |
|
||||
Info |
|
||||
Irate |
|
||||
LabelReplace |
|
||||
LabelJoin |
|
||||
|
@ -392,6 +393,7 @@ NumberDurationLiteralInDurationContext {
|
|||
Hour { condFn<"hour"> }
|
||||
Idelta { condFn<"idelta"> }
|
||||
Increase { condFn<"increase"> }
|
||||
Info { condFn<"info"> }
|
||||
Irate { condFn<"irate"> }
|
||||
LabelReplace { condFn<"label_replace"> }
|
||||
LabelJoin { condFn<"label_join"> }
|
||||
|
|
Loading…
Reference in a new issue