Merge pull request #16249 from roidelapluie/arythmetics

PromQL: allow arithmetic operations in durations in PromQL parser
This commit is contained in:
Julien 2025-04-22 10:17:41 +02:00 committed by GitHub
commit f4ca1368ff
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
19 changed files with 1918 additions and 559 deletions

View File

@ -249,6 +249,9 @@ func (c *flagConfig) setFeatureListOptions(logger *slog.Logger) error {
case "promql-experimental-functions":
parser.EnableExperimentalFunctions = true
logger.Info("Experimental PromQL functions enabled.")
case "promql-duration-expr":
parser.ExperimentalDurationExpr = true
logger.Info("Experimental duration expression parsing enabled.")
case "native-histograms":
c.tsdb.EnableNativeHistograms = true
c.scrape.EnableNativeHistogramsIngestion = true
@ -539,7 +542,7 @@ func main() {
a.Flag("scrape.discovery-reload-interval", "Interval used by scrape manager to throttle target groups updates.").
Hidden().Default("5s").SetValue(&cfg.scrape.DiscoveryReloadInterval)
a.Flag("enable-feature", "Comma separated feature names to enable. Valid options: exemplar-storage, expand-external-labels, memory-snapshot-on-shutdown, promql-per-step-stats, promql-experimental-functions, extra-scrape-metrics, auto-gomaxprocs, native-histograms, created-timestamp-zero-ingestion, concurrent-rule-eval, delayed-compaction, old-ui, otlp-deltatocumulative. See https://prometheus.io/docs/prometheus/latest/feature_flags/ for more details.").
a.Flag("enable-feature", "Comma separated feature names to enable. Valid options: exemplar-storage, expand-external-labels, memory-snapshot-on-shutdown, promql-per-step-stats, promql-experimental-functions, extra-scrape-metrics, auto-gomaxprocs, native-histograms, created-timestamp-zero-ingestion, concurrent-rule-eval, delayed-compaction, old-ui, otlp-deltatocumulative, promql-duration-expr. See https://prometheus.io/docs/prometheus/latest/feature_flags/ for more details.").
Default("").StringsVar(&cfg.featureList)
a.Flag("agent", "Run Prometheus in 'Agent mode'.").BoolVar(&agentMode)

View File

@ -61,7 +61,7 @@ The Prometheus monitoring server
| <code class="text-nowrap">--query.timeout</code> | Maximum time a query may take before being aborted. Use with server mode only. | `2m` |
| <code class="text-nowrap">--query.max-concurrency</code> | Maximum number of queries executed concurrently. Use with server mode only. | `20` |
| <code class="text-nowrap">--query.max-samples</code> | Maximum number of samples a single query can load into memory. Note that queries will fail if they try to load more samples than this into memory, so this also limits the number of samples a query can return. Use with server mode only. | `50000000` |
| <code class="text-nowrap">--enable-feature</code> <code class="text-nowrap">...<code class="text-nowrap"> | Comma separated feature names to enable. Valid options: exemplar-storage, expand-external-labels, memory-snapshot-on-shutdown, promql-per-step-stats, promql-experimental-functions, extra-scrape-metrics, auto-gomaxprocs, native-histograms, created-timestamp-zero-ingestion, concurrent-rule-eval, delayed-compaction, old-ui, otlp-deltatocumulative. See https://prometheus.io/docs/prometheus/latest/feature_flags/ for more details. | |
| <code class="text-nowrap">--enable-feature</code> <code class="text-nowrap">...<code class="text-nowrap"> | Comma separated feature names to enable. Valid options: exemplar-storage, expand-external-labels, memory-snapshot-on-shutdown, promql-per-step-stats, promql-experimental-functions, extra-scrape-metrics, auto-gomaxprocs, native-histograms, created-timestamp-zero-ingestion, concurrent-rule-eval, delayed-compaction, old-ui, otlp-deltatocumulative, promql-duration-expr. See https://prometheus.io/docs/prometheus/latest/feature_flags/ for more details. | |
| <code class="text-nowrap">--agent</code> | Run Prometheus in 'Agent mode'. | |
| <code class="text-nowrap">--log.level</code> | Only log messages with the given severity or above. One of: [debug, info, warn, error] | `info` |
| <code class="text-nowrap">--log.format</code> | Output format of log messages. One of: [logfmt, json] | `logfmt` |

View File

@ -183,4 +183,38 @@ This state is periodically ([`max_stale`][d2c]) cleared of inactive series.
Enabling this _can_ have negative impact on performance, because the in-memory
state is mutex guarded. Cumulative-only OTLP requests are not affected.
### PromQL arithmetic expressions in time durations
`--enable-feature=promql-duration-expr`
With this flag, arithmetic expressions can be used in time durations in range queries and offset durations. For example:
In range queries:
rate(http_requests_total[5m * 2]) # 10 minute range
rate(http_requests_total[(5+2) * 1m]) # 7 minute range
In offset durations:
http_requests_total offset (1h / 2) # 30 minute offset
http_requests_total offset ((2 ^ 3) * 1m) # 8 minute offset
Note: Duration expressions are not supported in the @ timestamp operator.
The following operators are supported:
* `+` - addition
* `-` - subtraction
* `*` - multiplication
* `/` - division
* `%` - modulo
* `^` - exponentiation
Examples of equivalent durations:
* `5m * 2` is the equivalent to `10m` or `600s`
* `10m - 1m` is the equivalent to `9m` or `540s`
* `(5+2) * 1m` is the equivalent to `7m` or `420s`
* `1h / 2` is the equivalent to `30m` or `1800s`
* `4h % 3h` is the equivalent to `1h` or `3600s`
* `(2 ^ 3) * 1m` is the equivalent to `8m` or `480s`
[d2c]: https://github.com/open-telemetry/opentelemetry-collector-contrib/tree/main/processor/deltatocumulativeprocessor

136
promql/durations.go Normal file
View File

@ -0,0 +1,136 @@
// Copyright 2025 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 (
"fmt"
"math"
"time"
"github.com/prometheus/prometheus/promql/parser"
)
// durationVisitor is a visitor that visits a duration expression and calculates the duration.
type durationVisitor struct{}
func (v *durationVisitor) Visit(node parser.Node, _ []parser.Node) (parser.Visitor, error) {
switch n := node.(type) {
case *parser.VectorSelector:
if n.OriginalOffsetExpr != nil {
duration, err := calculateDuration(n.OriginalOffsetExpr, true)
if err != nil {
return nil, err
}
n.OriginalOffset = duration
}
case *parser.MatrixSelector:
if n.RangeExpr != nil {
duration, err := calculateDuration(n.RangeExpr, false)
if err != nil {
return nil, err
}
n.Range = duration
}
case *parser.SubqueryExpr:
if n.OriginalOffsetExpr != nil {
duration, err := calculateDuration(n.OriginalOffsetExpr, true)
if err != nil {
return nil, err
}
n.OriginalOffset = duration
}
if n.StepExpr != nil {
duration, err := calculateDuration(n.StepExpr, false)
if err != nil {
return nil, err
}
n.Step = duration
}
if n.RangeExpr != nil {
duration, err := calculateDuration(n.RangeExpr, false)
if err != nil {
return nil, err
}
n.Range = duration
}
}
return v, nil
}
// calculateDuration computes the duration from a duration expression.
func calculateDuration(expr parser.Expr, allowedNegative bool) (time.Duration, error) {
duration, err := evaluateDurationExpr(expr)
if err != nil {
return 0, err
}
if duration <= 0 && !allowedNegative {
return 0, fmt.Errorf("%d:%d: duration must be greater than 0", expr.PositionRange().Start, expr.PositionRange().End)
}
if duration > 1<<63-1 || duration < -1<<63 {
return 0, fmt.Errorf("%d:%d: duration is out of range", expr.PositionRange().Start, expr.PositionRange().End)
}
return time.Duration(duration*1000) * time.Millisecond, nil
}
// evaluateDurationExpr recursively evaluates a duration expression to a float64 value.
func evaluateDurationExpr(expr parser.Expr) (float64, error) {
switch n := expr.(type) {
case *parser.NumberLiteral:
return n.Val, nil
case *parser.DurationExpr:
var lhs, rhs float64
var err error
if n.LHS != nil {
lhs, err = evaluateDurationExpr(n.LHS)
if err != nil {
return 0, err
}
}
rhs, err = evaluateDurationExpr(n.RHS)
if err != nil {
return 0, err
}
switch n.Op {
case parser.ADD:
return lhs + rhs, nil
case parser.SUB:
if n.LHS == nil {
// Unary negative duration expression.
return -rhs, nil
}
return lhs - rhs, nil
case parser.MUL:
return lhs * rhs, nil
case parser.DIV:
if rhs == 0 {
return 0, fmt.Errorf("%d:%d: division by zero", expr.PositionRange().Start, expr.PositionRange().End)
}
return lhs / rhs, nil
case parser.MOD:
if rhs == 0 {
return 0, fmt.Errorf("%d:%d: modulo by zero", expr.PositionRange().Start, expr.PositionRange().End)
}
return math.Mod(lhs, rhs), nil
case parser.POW:
return math.Pow(lhs, rhs), nil
default:
return 0, fmt.Errorf("unexpected duration expression operator %q", n.Op)
}
default:
return 0, fmt.Errorf("unexpected duration expression type %T", n)
}
}

238
promql/durations_test.go Normal file
View File

@ -0,0 +1,238 @@
// Copyright 2025 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 (
"testing"
"time"
"github.com/stretchr/testify/require"
"github.com/prometheus/prometheus/promql/parser"
)
func TestDurationVisitor(t *testing.T) {
// Enable experimental duration expression parsing.
parser.ExperimentalDurationExpr = true
t.Cleanup(func() {
parser.ExperimentalDurationExpr = false
})
complexExpr := `sum_over_time(
rate(metric[5m] offset 1h)[10m:30s] offset 2h
) +
avg_over_time(
metric[1h + 30m] offset -1h
) *
count_over_time(
metric[2h * 0.5]
)`
expr, err := parser.ParseExpr(complexExpr)
require.NoError(t, err)
err = parser.Walk(&durationVisitor{}, expr, nil)
require.NoError(t, err)
// Verify different parts of the expression have correct durations.
// This is a binary expression at the top level.
binExpr, ok := expr.(*parser.BinaryExpr)
require.True(t, ok, "Expected binary expression at top level")
// Left side should be sum_over_time with subquery.
leftCall, ok := binExpr.LHS.(*parser.Call)
require.True(t, ok, "Expected call expression on left side")
require.Equal(t, "sum_over_time", leftCall.Func.Name)
// Extract the subquery from sum_over_time.
sumSubquery, ok := leftCall.Args[0].(*parser.SubqueryExpr)
require.True(t, ok, "Expected subquery in sum_over_time")
require.Equal(t, 10*time.Minute, sumSubquery.Range)
require.Equal(t, 30*time.Second, sumSubquery.Step)
require.Equal(t, 2*time.Hour, sumSubquery.OriginalOffset)
// Extract the rate call inside the subquery.
rateCall, ok := sumSubquery.Expr.(*parser.Call)
require.True(t, ok, "Expected rate call in subquery")
require.Equal(t, "rate", rateCall.Func.Name)
// Extract the matrix selector from rate.
rateMatrix, ok := rateCall.Args[0].(*parser.MatrixSelector)
require.True(t, ok, "Expected matrix selector in rate")
require.Equal(t, 5*time.Minute, rateMatrix.Range)
require.Equal(t, 1*time.Hour, rateMatrix.VectorSelector.(*parser.VectorSelector).OriginalOffset)
// Right side should be another binary expression (multiplication).
rightBinExpr, ok := binExpr.RHS.(*parser.BinaryExpr)
require.True(t, ok, "Expected binary expression on right side")
// Left side of multiplication should be avg_over_time.
avgCall, ok := rightBinExpr.LHS.(*parser.Call)
require.True(t, ok, "Expected call expression on left side of multiplication")
require.Equal(t, "avg_over_time", avgCall.Func.Name)
// Extract the matrix selector from avg_over_time.
avgMatrix, ok := avgCall.Args[0].(*parser.MatrixSelector)
require.True(t, ok, "Expected matrix selector in avg_over_time")
require.Equal(t, 90*time.Minute, avgMatrix.Range) // 1h + 30m
require.Equal(t, -1*time.Hour, avgMatrix.VectorSelector.(*parser.VectorSelector).OriginalOffset)
// Right side of multiplication should be count_over_time.
countCall, ok := rightBinExpr.RHS.(*parser.Call)
require.True(t, ok, "Expected call expression on right side of multiplication")
require.Equal(t, "count_over_time", countCall.Func.Name)
// Extract the matrix selector from count_over_time.
countMatrix, ok := countCall.Args[0].(*parser.MatrixSelector)
require.True(t, ok, "Expected matrix selector in count_over_time")
require.Equal(t, 1*time.Hour, countMatrix.Range) // 2h * 0.5
}
func TestCalculateDuration(t *testing.T) {
tests := []struct {
name string
expr parser.Expr
expected time.Duration
errorMessage string
allowedNegative bool
}{
{
name: "addition",
expr: &parser.DurationExpr{
LHS: &parser.NumberLiteral{Val: 5},
RHS: &parser.NumberLiteral{Val: 10},
Op: parser.ADD,
},
expected: 15 * time.Second,
},
{
name: "subtraction",
expr: &parser.DurationExpr{
LHS: &parser.NumberLiteral{Val: 15},
RHS: &parser.NumberLiteral{Val: 5},
Op: parser.SUB,
},
expected: 10 * time.Second,
},
{
name: "subtraction with negative",
expr: &parser.DurationExpr{
LHS: &parser.NumberLiteral{Val: 5},
RHS: &parser.NumberLiteral{Val: 10},
Op: parser.SUB,
},
errorMessage: "duration must be greater than 0",
},
{
name: "multiplication",
expr: &parser.DurationExpr{
LHS: &parser.NumberLiteral{Val: 5},
RHS: &parser.NumberLiteral{Val: 3},
Op: parser.MUL,
},
expected: 15 * time.Second,
},
{
name: "division",
expr: &parser.DurationExpr{
LHS: &parser.NumberLiteral{Val: 15},
RHS: &parser.NumberLiteral{Val: 3},
Op: parser.DIV,
},
expected: 5 * time.Second,
},
{
name: "modulo with numbers",
expr: &parser.DurationExpr{
LHS: &parser.NumberLiteral{Val: 17},
RHS: &parser.NumberLiteral{Val: 5},
Op: parser.MOD,
},
expected: 2 * time.Second,
},
{
name: "power",
expr: &parser.DurationExpr{
LHS: &parser.NumberLiteral{Val: 2},
RHS: &parser.NumberLiteral{Val: 3},
Op: parser.POW,
},
expected: 8 * time.Second,
},
{
name: "complex expression",
expr: &parser.DurationExpr{
LHS: &parser.DurationExpr{
LHS: &parser.NumberLiteral{Val: 2},
RHS: &parser.DurationExpr{
LHS: &parser.NumberLiteral{Val: 3},
RHS: &parser.NumberLiteral{Val: 4},
Op: parser.ADD,
},
Op: parser.MUL,
},
RHS: &parser.NumberLiteral{Val: 1},
Op: parser.SUB,
},
expected: 13 * time.Second,
},
{
name: "unary negative",
expr: &parser.DurationExpr{
RHS: &parser.NumberLiteral{Val: 5},
Op: parser.SUB,
},
expected: -5 * time.Second,
allowedNegative: true,
},
{
name: "division by zero",
expr: &parser.DurationExpr{
LHS: &parser.NumberLiteral{Val: 5},
RHS: &parser.DurationExpr{
LHS: &parser.NumberLiteral{Val: 5},
RHS: &parser.NumberLiteral{Val: 5},
Op: parser.SUB,
},
Op: parser.DIV,
},
errorMessage: "division by zero",
},
{
name: "modulo by zero",
expr: &parser.DurationExpr{
LHS: &parser.NumberLiteral{Val: 5},
RHS: &parser.DurationExpr{
LHS: &parser.NumberLiteral{Val: 5},
RHS: &parser.NumberLiteral{Val: 5},
Op: parser.SUB,
},
Op: parser.MOD,
},
errorMessage: "modulo by zero",
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
result, err := calculateDuration(tt.expr, tt.allowedNegative)
if tt.errorMessage != "" {
require.Error(t, err)
require.Contains(t, err.Error(), tt.errorMessage)
return
}
require.NoError(t, err)
require.Equal(t, tt.expected, result)
})
}
}

View File

@ -489,9 +489,9 @@ func (ng *Engine) NewInstantQuery(ctx context.Context, q storage.Queryable, opts
if err := ng.validateOpts(expr); err != nil {
return nil, err
}
*pExpr = PreprocessExpr(expr, ts, ts)
*pExpr, err = PreprocessExpr(expr, ts, ts)
return qry, nil
return qry, err
}
// NewRangeQuery returns an evaluation query for the given time range and with
@ -513,9 +513,9 @@ func (ng *Engine) NewRangeQuery(ctx context.Context, q storage.Queryable, opts Q
if expr.Type() != parser.ValueTypeVector && expr.Type() != parser.ValueTypeScalar {
return nil, fmt.Errorf("invalid expression type %q for range query, must be Scalar or instant Vector", parser.DocumentedType(expr.Type()))
}
*pExpr = PreprocessExpr(expr, start, end)
*pExpr, err = PreprocessExpr(expr, start, end)
return qry, nil
return qry, err
}
func (ng *Engine) newQuery(q storage.Queryable, qs string, opts QueryOpts, start, end time.Time, interval time.Duration) (*parser.Expr, *query) {
@ -3596,15 +3596,20 @@ func unwrapStepInvariantExpr(e parser.Expr) parser.Expr {
}
// PreprocessExpr wraps all possible step invariant parts of the given expression with
// StepInvariantExpr. It also resolves the preprocessors.
func PreprocessExpr(expr parser.Expr, start, end time.Time) parser.Expr {
// StepInvariantExpr. It also resolves the preprocessors and evaluates duration expressions
// into their numeric values.
func PreprocessExpr(expr parser.Expr, start, end time.Time) (parser.Expr, error) {
detectHistogramStatsDecoding(expr)
if err := parser.Walk(&durationVisitor{}, expr, nil); err != nil {
return nil, err
}
isStepInvariant := preprocessExprHelper(expr, start, end)
if isStepInvariant {
return newStepInvariantExpr(expr)
return newStepInvariantExpr(expr), nil
}
return expr
return expr, nil
}
// preprocessExprHelper wraps the child nodes of the expression

View File

@ -1900,15 +1900,6 @@ func TestSubquerySelector(t *testing.T) {
},
Start: time.Unix(35, 0),
},
{
Query: "metric[0:10s]",
Result: promql.Result{
nil,
promql.Matrix{},
nil,
},
Start: time.Unix(10, 0),
},
},
},
{
@ -3096,7 +3087,8 @@ func TestPreprocessAndWrapWithStepInvariantExpr(t *testing.T) {
t.Run(test.input, func(t *testing.T) {
expr, err := parser.ParseExpr(test.input)
require.NoError(t, err)
expr = promql.PreprocessExpr(expr, startTime, endTime)
expr, err = promql.PreprocessExpr(expr, startTime, endTime)
require.NoError(t, err)
if test.outputTest {
require.Equal(t, test.input, expr.String(), "error on input '%s'", test.input)
}
@ -3268,11 +3260,6 @@ func TestInstantQueryWithRangeVectorSelector(t *testing.T) {
},
},
},
"matches series but range is 0": {
expr: "some_metric[0]",
ts: baseT.Add(2 * time.Minute),
expected: promql.Matrix{},
},
}
for name, testCase := range testCases {

View File

@ -110,6 +110,15 @@ type BinaryExpr struct {
ReturnBool bool
}
// DurationExpr represents a binary expression between two duration expressions.
type DurationExpr struct {
Op ItemType // The operation of the expression.
LHS, RHS Expr // The operands on the respective sides of the operator.
Wrapped bool // Set when the duration is wrapped in parentheses.
StartPos posrange.Pos // For unary operations, the position of the operator.
}
// Call represents a function call.
type Call struct {
Func *Function // The function that was called.
@ -124,7 +133,7 @@ type MatrixSelector struct {
// if the parser hasn't returned an error.
VectorSelector Expr
Range time.Duration
RangeExpr *DurationExpr
EndPos posrange.Pos
}
@ -132,16 +141,19 @@ type MatrixSelector struct {
type SubqueryExpr struct {
Expr Expr
Range time.Duration
RangeExpr *DurationExpr
// OriginalOffset is the actual offset that was set in the query.
// This never changes.
OriginalOffset time.Duration
// OriginalOffsetExpr is the actual offset expression that was set in the query.
OriginalOffsetExpr *DurationExpr
// Offset is the offset used during the query execution
// which is calculated using the original offset, at modifier time,
// which is calculated using the original offset, offset expression, at modifier time,
// eval time, and subquery offsets in the AST tree.
Offset time.Duration
Timestamp *int64
StartOrEnd ItemType // Set when @ is used with start() or end()
Step time.Duration
StepExpr *DurationExpr
EndPos posrange.Pos
}
@ -150,6 +162,7 @@ type SubqueryExpr struct {
type NumberLiteral struct {
Val float64
Duration bool // Used to format the number as a duration.
PosRange posrange.PositionRange
}
@ -191,9 +204,10 @@ func (e *StepInvariantExpr) PositionRange() posrange.PositionRange {
// VectorSelector represents a Vector selection.
type VectorSelector struct {
Name string
// OriginalOffset is the actual offset that was set in the query.
// This never changes.
// OriginalOffset is the actual offset calculated from OriginalOffsetExpr.
OriginalOffset time.Duration
// OriginalOffsetExpr is the actual offset that was set in the query.
OriginalOffsetExpr *DurationExpr
// Offset is the offset used during the query execution
// which is calculated using the original offset, at modifier time,
// eval time, and subquery offsets in the AST tree.
@ -244,6 +258,7 @@ func (e *BinaryExpr) Type() ValueType {
return ValueTypeVector
}
func (e *StepInvariantExpr) Type() ValueType { return e.Expr.Type() }
func (e *DurationExpr) Type() ValueType { return ValueTypeScalar }
func (*AggregateExpr) PromQLExpr() {}
func (*BinaryExpr) PromQLExpr() {}
@ -256,6 +271,7 @@ func (*StringLiteral) PromQLExpr() {}
func (*UnaryExpr) PromQLExpr() {}
func (*VectorSelector) PromQLExpr() {}
func (*StepInvariantExpr) PromQLExpr() {}
func (*DurationExpr) PromQLExpr() {}
// VectorMatchCardinality describes the cardinality relationship
// of two Vectors in a binary operation.
@ -438,6 +454,16 @@ func (e *BinaryExpr) PositionRange() posrange.PositionRange {
return mergeRanges(e.LHS, e.RHS)
}
func (e *DurationExpr) PositionRange() posrange.PositionRange {
if e.LHS == nil {
return posrange.PositionRange{
Start: e.StartPos,
End: e.RHS.PositionRange().End,
}
}
return mergeRanges(e.LHS, e.RHS)
}
func (e *Call) PositionRange() posrange.PositionRange {
return e.PosRange
}

View File

@ -186,7 +186,7 @@ START_METRIC_SELECTOR
%type <int> int
%type <uint> uint
%type <float> number series_value signed_number signed_or_unsigned_number
%type <node> 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
%type <node> 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 duration_expr paren_duration_expr positive_duration_expr
%start start
@ -235,6 +235,7 @@ expr :
| unary_expr
| vector_selector
| step_invariant_expr
| duration_expr
;
/*
@ -433,23 +434,35 @@ paren_expr : LEFT_PAREN expr RIGHT_PAREN
* Offset modifiers.
*/
offset_expr: expr OFFSET number_duration_literal
positive_duration_expr : duration_expr
{
numLit, _ := $3.(*NumberLiteral)
dur := time.Duration(numLit.Val * 1000) * time.Millisecond
yylex.(*parser).addOffset($1, dur)
if numLit, ok := $1.(*NumberLiteral); ok {
if numLit.Val <= 0 {
yylex.(*parser).addParseErrf(numLit.PositionRange(), "duration must be greater than 0")
$$ = &NumberLiteral{Val: 0} // Return 0 on error.
break
}
$$ = $1
break
}
$$ = $1
}
| expr OFFSET SUB number_duration_literal
;
offset_expr: expr OFFSET duration_expr
{
numLit, _ := $4.(*NumberLiteral)
dur := time.Duration(numLit.Val * 1000) * time.Millisecond
yylex.(*parser).addOffset($1, -dur)
if numLit, ok := $3.(*NumberLiteral); ok {
yylex.(*parser).addOffset($1, time.Duration(numLit.Val*1000)*time.Millisecond)
$$ = $1
break
}
yylex.(*parser).addOffsetExpr($1, $3.(*DurationExpr))
$$ = $1
}
| expr OFFSET error
{ yylex.(*parser).unexpected("offset", "number or duration"); $$ = $1 }
;
/*
* @ modifiers.
*/
@ -474,7 +487,7 @@ at_modifier_preprocessors: START | END;
* Subquery and range selectors.
*/
matrix_selector : expr LEFT_BRACKET number_duration_literal RIGHT_BRACKET
matrix_selector : expr LEFT_BRACKET positive_duration_expr RIGHT_BRACKET
{
var errMsg string
vs, ok := $1.(*VectorSelector)
@ -491,41 +504,60 @@ matrix_selector : expr LEFT_BRACKET number_duration_literal RIGHT_BRACKET
yylex.(*parser).addParseErrf(errRange, "%s", errMsg)
}
numLit, _ := $3.(*NumberLiteral)
var rangeNl time.Duration
if numLit, ok := $3.(*NumberLiteral); ok {
rangeNl = time.Duration(numLit.Val*1000)*time.Millisecond
}
rangeExpr, _ := $3.(*DurationExpr)
$$ = &MatrixSelector{
VectorSelector: $1.(Expr),
Range: time.Duration(numLit.Val * 1000) * time.Millisecond,
Range: rangeNl,
RangeExpr: rangeExpr,
EndPos: yylex.(*parser).lastClosing,
}
}
;
subquery_expr : expr LEFT_BRACKET number_duration_literal COLON number_duration_literal RIGHT_BRACKET
subquery_expr : expr LEFT_BRACKET positive_duration_expr COLON positive_duration_expr RIGHT_BRACKET
{
numLitRange, _ := $3.(*NumberLiteral)
numLitStep, _ := $5.(*NumberLiteral)
var rangeNl time.Duration
var stepNl time.Duration
if numLit, ok := $3.(*NumberLiteral); ok {
rangeNl = time.Duration(numLit.Val*1000)*time.Millisecond
}
rangeExpr, _ := $3.(*DurationExpr)
if numLit, ok := $5.(*NumberLiteral); ok {
stepNl = time.Duration(numLit.Val*1000)*time.Millisecond
}
stepExpr, _ := $5.(*DurationExpr)
$$ = &SubqueryExpr{
Expr: $1.(Expr),
Range: time.Duration(numLitRange.Val * 1000) * time.Millisecond,
Step: time.Duration(numLitStep.Val * 1000) * time.Millisecond,
Range: rangeNl,
RangeExpr: rangeExpr,
Step: stepNl,
StepExpr: stepExpr,
EndPos: $6.Pos + 1,
}
}
| expr LEFT_BRACKET number_duration_literal COLON RIGHT_BRACKET
| expr LEFT_BRACKET positive_duration_expr COLON RIGHT_BRACKET
{
numLitRange, _ := $3.(*NumberLiteral)
var rangeNl time.Duration
if numLit, ok := $3.(*NumberLiteral); ok {
rangeNl = time.Duration(numLit.Val*1000)*time.Millisecond
}
rangeExpr, _ := $3.(*DurationExpr)
$$ = &SubqueryExpr{
Expr: $1.(Expr),
Range: time.Duration(numLitRange.Val * 1000) * time.Millisecond,
Step: 0,
Range: rangeNl,
RangeExpr: rangeExpr,
EndPos: $5.Pos + 1,
}
}
| expr LEFT_BRACKET number_duration_literal COLON number_duration_literal error
| expr LEFT_BRACKET positive_duration_expr COLON positive_duration_expr error
{ yylex.(*parser).unexpected("subquery selector", "\"]\""); $$ = $1 }
| expr LEFT_BRACKET number_duration_literal COLON error
| expr LEFT_BRACKET positive_duration_expr COLON error
{ yylex.(*parser).unexpected("subquery selector", "number or duration or \"]\""); $$ = $1 }
| expr LEFT_BRACKET number_duration_literal error
| expr LEFT_BRACKET positive_duration_expr error
{ yylex.(*parser).unexpected("subquery or range", "\":\" or \"]\""); $$ = $1 }
| expr LEFT_BRACKET error
{ yylex.(*parser).unexpected("subquery selector", "number or duration"); $$ = $1 }
@ -930,6 +962,7 @@ number_duration_literal : NUMBER
$$ = &NumberLiteral{
Val: dur.Seconds(),
PosRange: $1.PositionRange(),
Duration: true,
}
}
;
@ -997,4 +1030,105 @@ maybe_grouping_labels: /* empty */ { $$ = nil }
| grouping_labels
;
/*
* Duration expressions.
*/
duration_expr : number_duration_literal
{
nl := $1.(*NumberLiteral)
if nl.Val > 1<<63/1e9 || nl.Val < -(1<<63)/1e9 {
yylex.(*parser).addParseErrf(nl.PosRange, "duration out of range")
$$ = &NumberLiteral{Val: 0}
break
}
$$ = nl
}
| unary_op duration_expr %prec MUL
{
switch expr := $2.(type) {
case *NumberLiteral:
if $1.Typ == SUB {
expr.Val *= -1
}
if expr.Val > 1<<63/1e9 || expr.Val < -(1<<63)/1e9 {
yylex.(*parser).addParseErrf($1.PositionRange(), "duration out of range")
$$ = &NumberLiteral{Val: 0}
break
}
expr.PosRange.Start = $1.Pos
$$ = expr
break
case *DurationExpr:
if $1.Typ == SUB {
$$ = &DurationExpr{
Op: SUB,
RHS: expr,
StartPos: $1.Pos,
}
break
}
$$ = expr
break
default:
yylex.(*parser).addParseErrf($1.PositionRange(), "expected number literal or duration expression")
$$ = &NumberLiteral{Val: 0}
break
}
}
| duration_expr ADD duration_expr
{
yylex.(*parser).experimentalDurationExpr($1.(Expr))
$$ = &DurationExpr{Op: ADD, LHS: $1.(Expr), RHS: $3.(Expr)}
}
| duration_expr SUB duration_expr
{
yylex.(*parser).experimentalDurationExpr($1.(Expr))
$$ = &DurationExpr{Op: SUB, LHS: $1.(Expr), RHS: $3.(Expr)}
}
| duration_expr MUL duration_expr
{
yylex.(*parser).experimentalDurationExpr($1.(Expr))
$$ = &DurationExpr{Op: MUL, LHS: $1.(Expr), RHS: $3.(Expr)}
}
| duration_expr DIV duration_expr
{
yylex.(*parser).experimentalDurationExpr($1.(Expr))
if nl, ok := $3.(*NumberLiteral); ok && nl.Val == 0 {
yylex.(*parser).addParseErrf($2.PositionRange(), "division by zero")
$$ = &NumberLiteral{Val: 0}
break
}
$$ = &DurationExpr{Op: DIV, LHS: $1.(Expr), RHS: $3.(Expr)}
}
| duration_expr MOD duration_expr
{
yylex.(*parser).experimentalDurationExpr($1.(Expr))
if nl, ok := $3.(*NumberLiteral); ok && nl.Val == 0 {
yylex.(*parser).addParseErrf($2.PositionRange(), "modulo by zero")
$$ = &NumberLiteral{Val: 0}
break
}
$$ = &DurationExpr{Op: MOD, LHS: $1.(Expr), RHS: $3.(Expr)}
}
| duration_expr POW duration_expr
{
yylex.(*parser).experimentalDurationExpr($1.(Expr))
$$ = &DurationExpr{Op: POW, LHS: $1.(Expr), RHS: $3.(Expr)}
}
| paren_duration_expr
;
paren_duration_expr : LEFT_PAREN duration_expr RIGHT_PAREN
{
yylex.(*parser).experimentalDurationExpr($2.(Expr))
if durationExpr, ok := $2.(*DurationExpr); ok {
durationExpr.Wrapped = true
$$ = durationExpr
break
}
$$ = $2
}
;
%%

File diff suppressed because it is too large Load Diff

View File

@ -277,6 +277,7 @@ type Lexer struct {
braceOpen bool // Whether a { is opened.
bracketOpen bool // Whether a [ is opened.
gotColon bool // Whether we got a ':' after [ was opened.
gotDuration bool // Whether we got a duration after [ was opened.
stringOpen rune // Quote rune of the string currently being read.
// series description variables for internal PromQL testing framework as well as in promtool rules unit tests.
@ -491,7 +492,7 @@ func lexStatements(l *Lexer) stateFn {
skipSpaces(l)
}
l.bracketOpen = true
return lexNumberOrDuration
return lexDurationExpr
case r == ']':
if !l.bracketOpen {
return l.errorf("unexpected right bracket %q", r)
@ -549,6 +550,8 @@ func lexHistogram(l *Lexer) stateFn {
return lexNumber
case r == '[':
l.bracketOpen = true
l.gotColon = false
l.gotDuration = false
l.emit(LEFT_BRACKET)
return lexBuckets
case r == '}' && l.peek() == '}':
@ -1077,3 +1080,64 @@ func isDigit(r rune) bool {
func isAlpha(r rune) bool {
return r == '_' || ('a' <= r && r <= 'z') || ('A' <= r && r <= 'Z')
}
// lexDurationExpr scans arithmetic expressions within brackets for duration expressions.
func lexDurationExpr(l *Lexer) stateFn {
switch r := l.next(); {
case r == eof:
return l.errorf("unexpected end of input in duration expression")
case r == ']':
l.emit(RIGHT_BRACKET)
l.bracketOpen = false
l.gotColon = false
return lexStatements
case r == ':':
l.emit(COLON)
if !l.gotDuration {
return l.errorf("unexpected colon before duration in duration expression")
}
if l.gotColon {
return l.errorf("unexpected repeated colon in duration expression")
}
l.gotColon = true
return lexDurationExpr
case r == '(':
l.emit(LEFT_PAREN)
l.parenDepth++
return lexDurationExpr
case r == ')':
l.emit(RIGHT_PAREN)
l.parenDepth--
if l.parenDepth < 0 {
return l.errorf("unexpected right parenthesis %q", r)
}
return lexDurationExpr
case isSpace(r):
skipSpaces(l)
return lexDurationExpr
case r == '+':
l.emit(ADD)
return lexDurationExpr
case r == '-':
l.emit(SUB)
return lexDurationExpr
case r == '*':
l.emit(MUL)
return lexDurationExpr
case r == '/':
l.emit(DIV)
return lexDurationExpr
case r == '%':
l.emit(MOD)
return lexDurationExpr
case r == '^':
l.emit(POW)
return lexDurationExpr
case isDigit(r) || (r == '.' && isDigit(l.peek())):
l.backup()
l.gotDuration = true
return lexNumberOrDuration
default:
return l.errorf("unexpected character in duration expression: %q", r)
}
}

View File

@ -951,6 +951,10 @@ var tests = []struct {
input: `test:name{on!~"bar"}[:4s]`,
fail: true,
},
{
input: `test:name{on!~"bar"}[1s:1s:1s]`,
fail: true,
},
},
},
}

View File

@ -39,6 +39,9 @@ var parserPool = sync.Pool{
},
}
// ExperimentalDurationExpr is a flag to enable experimental duration expression parsing.
var ExperimentalDurationExpr bool
type Parser interface {
ParseExpr() (Expr, error)
Close()
@ -881,9 +884,6 @@ func parseDuration(ds string) (time.Duration, error) {
if err != nil {
return 0, err
}
if dur == 0 {
return 0, errors.New("duration must be greater than 0")
}
return time.Duration(dur), nil
}
@ -939,11 +939,13 @@ func (p *parser) newMetricNameMatcher(value Item) *labels.Matcher {
// addOffset is used to set the offset in the generated parser.
func (p *parser) addOffset(e Node, offset time.Duration) {
var orgoffsetp *time.Duration
var orgoffsetexprp *DurationExpr
var endPosp *posrange.Pos
switch s := e.(type) {
case *VectorSelector:
orgoffsetp = &s.OriginalOffset
orgoffsetexprp = s.OriginalOffsetExpr
endPosp = &s.PosRange.End
case *MatrixSelector:
vs, ok := s.VectorSelector.(*VectorSelector)
@ -952,9 +954,11 @@ func (p *parser) addOffset(e Node, offset time.Duration) {
return
}
orgoffsetp = &vs.OriginalOffset
orgoffsetexprp = vs.OriginalOffsetExpr
endPosp = &s.EndPos
case *SubqueryExpr:
orgoffsetp = &s.OriginalOffset
orgoffsetexprp = s.OriginalOffsetExpr
endPosp = &s.EndPos
default:
p.addParseErrf(e.PositionRange(), "offset modifier must be preceded by an instant vector selector or range vector selector or a subquery")
@ -963,7 +967,7 @@ func (p *parser) addOffset(e Node, offset time.Duration) {
// it is already ensured by parseDuration func that there never will be a zero offset modifier
switch {
case *orgoffsetp != 0:
case *orgoffsetp != 0 || orgoffsetexprp != nil:
p.addParseErrf(e.PositionRange(), "offset may not be set multiple times")
case orgoffsetp != nil:
*orgoffsetp = offset
@ -972,6 +976,45 @@ func (p *parser) addOffset(e Node, offset time.Duration) {
*endPosp = p.lastClosing
}
// addOffsetExpr is used to set the offset expression in the generated parser.
func (p *parser) addOffsetExpr(e Node, expr *DurationExpr) {
var orgoffsetp *time.Duration
var orgoffsetexprp **DurationExpr
var endPosp *posrange.Pos
switch s := e.(type) {
case *VectorSelector:
orgoffsetp = &s.OriginalOffset
orgoffsetexprp = &s.OriginalOffsetExpr
endPosp = &s.PosRange.End
case *MatrixSelector:
vs, ok := s.VectorSelector.(*VectorSelector)
if !ok {
p.addParseErrf(e.PositionRange(), "ranges only allowed for vector selectors")
return
}
orgoffsetp = &vs.OriginalOffset
orgoffsetexprp = &vs.OriginalOffsetExpr
endPosp = &s.EndPos
case *SubqueryExpr:
orgoffsetp = &s.OriginalOffset
orgoffsetexprp = &s.OriginalOffsetExpr
endPosp = &s.EndPos
default:
p.addParseErrf(e.PositionRange(), "offset modifier must be preceded by an instant vector selector or range vector selector or a subquery")
return
}
switch {
case *orgoffsetp != 0 || *orgoffsetexprp != nil:
p.addParseErrf(e.PositionRange(), "offset may not be set multiple times")
case orgoffsetexprp != nil:
*orgoffsetexprp = expr
}
*endPosp = p.lastClosing
}
// setTimestamp is used to set the timestamp from the @ modifier in the generated parser.
func (p *parser) setTimestamp(e Node, ts float64) {
if math.IsInf(ts, -1) || math.IsInf(ts, 1) || math.IsNaN(ts) ||
@ -1045,6 +1088,12 @@ func (p *parser) getAtModifierVars(e Node) (**int64, *ItemType, *posrange.Pos, b
return timestampp, preprocp, endPosp, true
}
func (p *parser) experimentalDurationExpr(e Expr) {
if !ExperimentalDurationExpr {
p.addParseErrf(e.PositionRange(), "experimental duration expression is not enabled")
}
}
func MustLabelMatcher(mt labels.MatchType, name, val string) *labels.Matcher {
m, err := labels.NewMatcher(mt, name, val)
if err != nil {

View File

@ -2337,12 +2337,12 @@ var testExpr = []struct {
{
input: `foo[]`,
fail: true,
errMsg: "bad number or duration syntax: \"\"",
errMsg: "unexpected \"]\" in subquery selector, expected number or duration",
},
{
input: `foo[-1]`,
fail: true,
errMsg: "bad number or duration syntax: \"\"",
errMsg: "duration must be greater than 0",
},
{
input: `some_metric[5m] OFFSET 1mm`,
@ -3091,7 +3091,7 @@ var testExpr = []struct {
{
input: `foo{bar="baz"}[`,
fail: true,
errMsg: `1:16: parse error: bad number or duration syntax: ""`,
errMsg: `unexpected end of input in duration expression`,
},
{
input: `foo{bar="baz"}[10m:6s]`,
@ -3946,6 +3946,304 @@ var testExpr = []struct {
},
},
},
{
input: `foo[11s+10s-5*2^2]`,
expected: &MatrixSelector{
VectorSelector: &VectorSelector{
Name: "foo",
LabelMatchers: []*labels.Matcher{
MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"),
},
PosRange: posrange.PositionRange{
Start: 0,
End: 3,
},
},
RangeExpr: &DurationExpr{
Op: SUB,
LHS: &DurationExpr{
Op: ADD,
LHS: &NumberLiteral{
Val: 11,
PosRange: posrange.PositionRange{
Start: 4,
End: 7,
},
Duration: true,
},
RHS: &NumberLiteral{
Val: 10,
PosRange: posrange.PositionRange{
Start: 8,
End: 11,
},
Duration: true,
},
},
RHS: &DurationExpr{
Op: MUL,
LHS: &NumberLiteral{Val: 5, PosRange: posrange.PositionRange{Start: 12, End: 13}},
RHS: &DurationExpr{
Op: POW,
LHS: &NumberLiteral{Val: 2, PosRange: posrange.PositionRange{Start: 14, End: 15}},
RHS: &NumberLiteral{Val: 2, PosRange: posrange.PositionRange{Start: 16, End: 17}},
},
},
},
EndPos: 18,
},
},
{
input: `foo[-(10s-5s)+20s]`,
expected: &MatrixSelector{
VectorSelector: &VectorSelector{
Name: "foo",
LabelMatchers: []*labels.Matcher{
MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"),
},
PosRange: posrange.PositionRange{
Start: 0,
End: 3,
},
},
RangeExpr: &DurationExpr{
Op: ADD,
LHS: &DurationExpr{
Op: SUB,
StartPos: 4,
RHS: &DurationExpr{
Op: SUB,
LHS: &NumberLiteral{
Val: 10,
PosRange: posrange.PositionRange{
Start: 6,
End: 9,
},
Duration: true,
},
RHS: &NumberLiteral{
Val: 5,
PosRange: posrange.PositionRange{
Start: 10,
End: 12,
},
Duration: true,
},
Wrapped: true,
},
},
RHS: &NumberLiteral{
Val: 20,
PosRange: posrange.PositionRange{
Start: 14,
End: 17,
},
Duration: true,
},
},
EndPos: 18,
},
},
{
input: `foo[-10s+15s]`,
expected: &MatrixSelector{
VectorSelector: &VectorSelector{
Name: "foo",
LabelMatchers: []*labels.Matcher{
MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"),
},
PosRange: posrange.PositionRange{
Start: 0,
End: 3,
},
},
RangeExpr: &DurationExpr{
Op: ADD,
LHS: &NumberLiteral{
Val: -10,
PosRange: posrange.PositionRange{
Start: 4,
End: 8,
},
Duration: true,
},
RHS: &NumberLiteral{
Val: 15,
PosRange: posrange.PositionRange{
Start: 9,
End: 12,
},
Duration: true,
},
},
EndPos: 13,
},
},
{
input: `foo[4s+4s:1s*2] offset (5s-8)`,
expected: &SubqueryExpr{
Expr: &VectorSelector{
Name: "foo",
LabelMatchers: []*labels.Matcher{
MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"),
},
PosRange: posrange.PositionRange{
Start: 0,
End: 3,
},
},
RangeExpr: &DurationExpr{
Op: ADD,
LHS: &NumberLiteral{
Val: 4,
PosRange: posrange.PositionRange{
Start: 4,
End: 6,
},
Duration: true,
},
RHS: &NumberLiteral{
Val: 4,
PosRange: posrange.PositionRange{
Start: 7,
End: 9,
},
Duration: true,
},
},
StepExpr: &DurationExpr{
Op: MUL,
LHS: &NumberLiteral{
Val: 1,
PosRange: posrange.PositionRange{
Start: 10,
End: 12,
},
Duration: true,
},
RHS: &NumberLiteral{
Val: 2,
PosRange: posrange.PositionRange{
Start: 13,
End: 14,
},
},
},
OriginalOffsetExpr: &DurationExpr{
Op: SUB,
LHS: &NumberLiteral{
Val: 5,
PosRange: posrange.PositionRange{
Start: 24,
End: 26,
},
Duration: true,
},
RHS: &NumberLiteral{
Val: 8,
PosRange: posrange.PositionRange{
Start: 27,
End: 28,
},
},
Wrapped: true,
},
EndPos: 29,
},
},
{
input: `foo offset 5s-8`,
expected: &BinaryExpr{
Op: SUB,
LHS: &VectorSelector{
Name: "foo",
OriginalOffset: 5 * time.Second,
LabelMatchers: []*labels.Matcher{
MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"),
},
PosRange: posrange.PositionRange{
Start: 0,
End: 13,
},
},
RHS: &NumberLiteral{
Val: 8,
PosRange: posrange.PositionRange{
Start: 14,
End: 15,
},
},
},
},
{
input: `rate(foo[2m+2m])`,
expected: &Call{
Func: MustGetFunction("rate"),
Args: Expressions{
&MatrixSelector{
VectorSelector: &VectorSelector{
Name: "foo",
LabelMatchers: []*labels.Matcher{
MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"),
},
PosRange: posrange.PositionRange{
Start: 5,
End: 8,
},
},
RangeExpr: &DurationExpr{
Op: ADD,
LHS: &NumberLiteral{
Val: 120,
PosRange: posrange.PositionRange{
Start: 9,
End: 11,
},
Duration: true,
},
RHS: &NumberLiteral{
Val: 120,
PosRange: posrange.PositionRange{
Start: 12,
End: 14,
},
Duration: true,
},
},
EndPos: 15,
},
},
PosRange: posrange.PositionRange{
Start: 0,
End: 16,
},
},
},
{
input: `foo[5s/0d]`,
fail: true,
errMsg: `division by zero`,
},
{
input: `foo offset (4d/0)`,
fail: true,
errMsg: `division by zero`,
},
{
input: `foo[5s%0d]`,
fail: true,
errMsg: `modulo by zero`,
},
{
input: `foo offset 9.5e10`,
fail: true,
errMsg: `duration out of range`,
},
{
input: `foo[9.5e10]`,
fail: true,
errMsg: `duration out of range`,
},
}
func makeInt64Pointer(val int64) *int64 {
@ -3965,8 +4263,11 @@ func readable(s string) string {
func TestParseExpressions(t *testing.T) {
// Enable experimental functions testing.
EnableExperimentalFunctions = true
// Enable experimental duration expression parsing.
ExperimentalDurationExpr = true
t.Cleanup(func() {
EnableExperimentalFunctions = false
ExperimentalDurationExpr = false
})
for _, test := range testExpr {

View File

@ -79,6 +79,22 @@ func (e *BinaryExpr) Pretty(level int) string {
return fmt.Sprintf("%s\n%s%s%s%s\n%s", e.LHS.Pretty(level+1), indent(level), e.Op, returnBool, matching, e.RHS.Pretty(level+1))
}
func (e *DurationExpr) Pretty(int) string {
var s string
fmt.Println("e.LHS", e.LHS)
fmt.Println("e.RHS", e.RHS)
if e.LHS == nil {
// This is a unary negative duration expression.
s = fmt.Sprintf("%s %s", e.Op, e.RHS.Pretty(0))
} else {
s = fmt.Sprintf("%s %s %s", e.LHS.Pretty(0), e.Op, e.RHS.Pretty(0))
}
if e.Wrapped {
s = fmt.Sprintf("(%s)", s)
}
return s
}
func (e *Call) Pretty(level int) string {
s := indent(level)
if !needsSplit(e) {

View File

@ -668,3 +668,41 @@ func TestUnaryPretty(t *testing.T) {
})
}
}
func TestDurationExprPretty(t *testing.T) {
// Enable experimental duration expression parsing.
ExperimentalDurationExpr = true
t.Cleanup(func() {
ExperimentalDurationExpr = false
})
maxCharactersPerLine = 10
inputs := []struct {
in, out string
}{
{
in: `rate(foo[2*1h])`,
out: `rate(
foo[2 * 1h]
)`,
},
{
in: `rate(foo[2*1h])`,
out: `rate(
foo[2 * 1h]
)`,
},
{
in: `rate(foo[-5m+35m])`,
out: `rate(
foo[-5m + 35m]
)`,
},
}
for _, test := range inputs {
t.Run(test.in, func(t *testing.T) {
expr, err := ParseExpr(test.in)
require.NoError(t, err)
require.Equal(t, test.out, Prettify(expr))
})
}
}

View File

@ -146,6 +146,24 @@ func (node *BinaryExpr) getMatchingStr() string {
return matching
}
func (node *DurationExpr) String() string {
var expr string
if node.LHS == nil {
// This is a unary negative duration expression.
expr = fmt.Sprintf("%s%s", node.Op, node.RHS)
} else {
expr = fmt.Sprintf("%s %s %s", node.LHS, node.Op, node.RHS)
}
if node.Wrapped {
return fmt.Sprintf("(%s)", expr)
}
return expr
}
func (node *DurationExpr) ShortString() string {
return node.Op.String()
}
func (node *Call) String() string {
return fmt.Sprintf("%s(%s)", node.Func.Name, node.Args)
}
@ -159,6 +177,8 @@ func (node *MatrixSelector) atOffset() (string, string) {
vecSelector := node.VectorSelector.(*VectorSelector)
offset := ""
switch {
case vecSelector.OriginalOffsetExpr != nil:
offset = fmt.Sprintf(" offset %s", vecSelector.OriginalOffsetExpr)
case vecSelector.OriginalOffset > time.Duration(0):
offset = fmt.Sprintf(" offset %s", model.Duration(vecSelector.OriginalOffset))
case vecSelector.OriginalOffset < time.Duration(0):
@ -181,21 +201,30 @@ func (node *MatrixSelector) String() string {
// Copy the Vector selector before changing the offset
vecSelector := *node.VectorSelector.(*VectorSelector)
// Do not print the @ and offset twice.
offsetVal, atVal, preproc := vecSelector.OriginalOffset, vecSelector.Timestamp, vecSelector.StartOrEnd
offsetVal, offsetExprVal, atVal, preproc := vecSelector.OriginalOffset, vecSelector.OriginalOffsetExpr, vecSelector.Timestamp, vecSelector.StartOrEnd
vecSelector.OriginalOffset = 0
vecSelector.OriginalOffsetExpr = nil
vecSelector.Timestamp = nil
vecSelector.StartOrEnd = 0
str := fmt.Sprintf("%s[%s]%s%s", vecSelector.String(), model.Duration(node.Range), at, offset)
rangeStr := model.Duration(node.Range).String()
if node.RangeExpr != nil {
rangeStr = node.RangeExpr.String()
}
str := fmt.Sprintf("%s[%s]%s%s", vecSelector.String(), rangeStr, at, offset)
vecSelector.OriginalOffset, vecSelector.Timestamp, vecSelector.StartOrEnd = offsetVal, atVal, preproc
vecSelector.OriginalOffset, vecSelector.OriginalOffsetExpr, vecSelector.Timestamp, vecSelector.StartOrEnd = offsetVal, offsetExprVal, atVal, preproc
return str
}
func (node *MatrixSelector) ShortString() string {
at, offset := node.atOffset()
return fmt.Sprintf("[%s]%s%s", model.Duration(node.Range), at, offset)
rangeStr := model.Duration(node.Range).String()
if node.RangeExpr != nil {
rangeStr = node.RangeExpr.String()
}
return fmt.Sprintf("[%s]%s%s", rangeStr, at, offset)
}
func (node *SubqueryExpr) String() string {
@ -211,9 +240,13 @@ func (node *SubqueryExpr) getSubqueryTimeSuffix() string {
step := ""
if node.Step != 0 {
step = model.Duration(node.Step).String()
} else if node.StepExpr != nil {
step = node.StepExpr.String()
}
offset := ""
switch {
case node.OriginalOffsetExpr != nil:
offset = fmt.Sprintf(" offset %s", node.OriginalOffsetExpr)
case node.OriginalOffset > time.Duration(0):
offset = fmt.Sprintf(" offset %s", model.Duration(node.OriginalOffset))
case node.OriginalOffset < time.Duration(0):
@ -228,10 +261,20 @@ func (node *SubqueryExpr) getSubqueryTimeSuffix() string {
case node.StartOrEnd == END:
at = " @ end()"
}
return fmt.Sprintf("[%s:%s]%s%s", model.Duration(node.Range), step, at, offset)
rangeStr := model.Duration(node.Range).String()
if node.RangeExpr != nil {
rangeStr = node.RangeExpr.String()
}
return fmt.Sprintf("[%s:%s]%s%s", rangeStr, step, at, offset)
}
func (node *NumberLiteral) String() string {
if node.Duration {
if node.Val < 0 {
return fmt.Sprintf("-%s", model.Duration(-node.Val*1e9).String())
}
return model.Duration(node.Val * 1e9).String()
}
return strconv.FormatFloat(node.Val, 'f', -1, 64)
}
@ -265,6 +308,8 @@ func (node *VectorSelector) String() string {
}
offset := ""
switch {
case node.OriginalOffsetExpr != nil:
offset = fmt.Sprintf(" offset %s", node.OriginalOffsetExpr)
case node.OriginalOffset > time.Duration(0):
offset = fmt.Sprintf(" offset %s", model.Duration(node.OriginalOffset))
case node.OriginalOffset < time.Duration(0):

View File

@ -117,8 +117,12 @@ func RunBuiltinTests(t TBRun, engine promql.QueryEngine) {
// RunBuiltinTestsWithStorage runs an acceptance test suite against the provided engine and storage.
func RunBuiltinTestsWithStorage(t TBRun, engine promql.QueryEngine, newStorage func(testutil.T) storage.Storage) {
t.Cleanup(func() { parser.EnableExperimentalFunctions = false })
t.Cleanup(func() {
parser.EnableExperimentalFunctions = false
parser.ExperimentalDurationExpr = false
})
parser.EnableExperimentalFunctions = true
parser.ExperimentalDurationExpr = true
files, err := fs.Glob(testsFs, "*/*.test")
require.NoError(t, err)

View File

@ -0,0 +1,121 @@
# Test for different duration expression formats in range selectors.
# This tests the parser's ability to handle various duration expression.
# Set up a basic counter that increases steadily.
load 5m
http_requests{path="/foo"} 1 2 3 0 1 0 0 1 2 0
http_requests{path="/bar"} 1 2 3 4 5 1 2 3 4 5
http_requests{path="/biz"} 0 0 0 0 0 1 1 1 1 1
# Test basic duration with unit: [30m]
eval instant at 50m changes(http_requests[30m])
{path="/foo"} 3
{path="/bar"} 4
{path="/biz"} 0
# Test addition in duration: [26m+4m]
eval instant at 50m changes(http_requests[26m+4m])
{path="/foo"} 3
{path="/bar"} 4
{path="/biz"} 0
# Test addition with 0 in duration: [30m+0s]
eval instant at 50m changes(http_requests[30m+0s])
{path="/foo"} 3
{path="/bar"} 4
{path="/biz"} 0
# Test raw seconds: [1800]
eval instant at 50m changes(http_requests[1800])
{path="/foo"} 3
{path="/bar"} 4
{path="/biz"} 0
# Test seconds with multiplication: [60*30]
eval instant at 50m changes(http_requests[60*30])
{path="/foo"} 3
{path="/bar"} 4
{path="/biz"} 0
# Test minutes with multiplication: [2m*15]
eval instant at 50m changes(http_requests[2m*15])
{path="/foo"} 3
{path="/bar"} 4
{path="/biz"} 0
# Test complex expression with parentheses: [2m*(10+5)]
eval instant at 50m changes(http_requests[2m*(10+5)])
{path="/foo"} 3
{path="/bar"} 4
{path="/biz"} 0
# Test mixed units: [29m+60s]
eval instant at 50m changes(http_requests[29m+60s])
{path="/foo"} 3
{path="/bar"} 4
{path="/biz"} 0
# Test nested parentheses: [24m+((1.5*2m)+2m)]
eval instant at 50m changes(http_requests[24m+((1.5*2m)+2m)])
{path="/foo"} 3
{path="/bar"} 4
{path="/biz"} 0
# Test start with -: [-5m+35m]
eval instant at 50m changes(http_requests[-5m+35m])
{path="/foo"} 3
{path="/bar"} 4
{path="/biz"} 0
# Test division: [1h/2]
eval instant at 50m changes(http_requests[1h/2])
{path="/foo"} 3
{path="/bar"} 4
{path="/biz"} 0
# Test modulo: [1h30m % 1h]
eval instant at 50m changes(http_requests[1h30m % 1h])
{path="/foo"} 3
{path="/bar"} 4
{path="/biz"} 0
# Test modulo and calculation: [30m1s-30m1s % 1m]
eval instant at 50m changes(http_requests[30m1s-30m1s % 1m])
{path="/foo"} 3
{path="/bar"} 4
{path="/biz"} 0
# Test combination of operations: [(9m30s+30s)*3]
eval instant at 50m changes(http_requests[(9m30s+30s)*3])
{path="/foo"} 3
{path="/bar"} 4
{path="/biz"} 0
clear
load 10s
metric1_total 0+1x1000
# In subquery expression.
eval instant at 1000s sum_over_time(metric1_total[29s+1s:5s+5s])
{} 297
# Test complex expressions in subquery ranges.
eval instant at 1000s sum_over_time(metric1_total[29s+1s:((((8 - 2) / 3) * 7s) % 4) + 8000ms])
{} 297
# Test complex expressions in offset ranges.
eval instant at 1200s sum_over_time(metric1_total[29s+1s:20*500ms] offset (20*(((((8 - 2) / 3) * 7s) % 4) + 8000ms)))
{} 297
# Test complex expressions in offset ranges with negative offset.
eval instant at 800s sum_over_time(metric1_total[29s+1s:20*500ms] offset -(20*(((((8 - 2) / 3) * 7s) % 4) + 8000ms)))
{} 297
# Test offset precedence with parentheses: offset (100 + 2)
eval instant at 1000s metric1_total offset (100 + 2)
{__name__="metric1_total"} 89
# Test offset precedence without parentheses: offset 100 + 2
eval instant at 1000s metric1_total offset 100 + 2
{} 92