Move calculation to the engine

Signed-off-by: Julien Pivotto <291750+roidelapluie@users.noreply.github.com>
This commit is contained in:
Julien Pivotto 2025-03-27 14:39:23 +01:00
parent 2e0a4ef096
commit 2f6ad79edf
12 changed files with 1336 additions and 462 deletions

137
promql/durations.go Normal file
View File

@ -0,0 +1,137 @@
// 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 (
"errors"
"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, errors.New("duration must be greater than 0")
}
if duration > 1<<63-1 || duration < -1<<63 {
return 0, errors.New("duration is out of range")
}
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, errors.New("division by zero")
}
return lhs / rhs, nil
case parser.MOD:
if rhs == 0 {
return 0, errors.New("modulo by zero")
}
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)
}
}

305
promql/durations_test.go Normal file
View File

@ -0,0 +1,305 @@
// 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) {
// Enable experimental duration expression parsing.
parser.ExperimentalDurationExpr = true
t.Cleanup(func() {
parser.ExperimentalDurationExpr = false
})
tests := []struct {
name string
expr string
expected time.Duration
expectError bool
}{
{
name: "number literal",
expr: "5",
expected: 5 * time.Second,
},
{
name: "time unit literal",
expr: "1h",
expected: time.Hour,
},
{
name: "addition with numbers",
expr: "5 + 10",
expected: 15 * time.Second,
},
{
name: "addition with time units",
expr: "1h + 30m",
expected: 90 * time.Minute,
},
{
name: "subtraction with numbers",
expr: "15 - 5",
expected: 10 * time.Second,
},
{
name: "subtraction with time units",
expr: "2h - 30m",
expected: 90 * time.Minute,
},
{
name: "multiplication with numbers",
expr: "5 * 3",
expected: 15 * time.Second,
},
{
name: "multiplication with time unit and number",
expr: "2h * 1.5",
expected: 3 * time.Hour,
},
{
name: "division with numbers",
expr: "15 / 3",
expected: 5 * time.Second,
},
{
name: "division with time unit and number",
expr: "1h / 2",
expected: 30 * time.Minute,
},
{
name: "modulo with numbers",
expr: "17 % 5",
expected: 2 * time.Second,
},
{
name: "modulo with time unit and number",
expr: "70m % 60m",
expected: 10 * time.Minute,
},
{
name: "power with numbers",
expr: "2 ^ 3",
expected: 8 * time.Second,
},
{
name: "complex expression with numbers",
expr: "2 * (3 + 4) - 1",
expected: 13 * time.Second,
},
{
name: "complex expression with time units",
expr: "2 * (1h + 30m) - 15m",
expected: 165 * time.Minute,
},
{
name: "unary negative with number",
expr: "-5",
expected: -5 * time.Second,
},
{
name: "unary negative with time unit",
expr: "-1h",
expected: -time.Hour,
},
{
name: "division by zero with numbers",
expr: "5 / 0",
expectError: true,
},
{
name: "division by zero with time unit",
expr: "1h / 0",
expectError: true,
},
{
name: "modulo by zero with numbers",
expr: "5 % 0",
expectError: true,
},
{
name: "modulo by zero with time unit",
expr: "1h % 0",
expectError: true,
},
}
for _, tt := range tests {
t.Run(tt.name+" offset", func(t *testing.T) {
expr, err := parser.ParseExpr("foo offset (" + tt.expr + ")")
if tt.expectError {
require.Error(t, err)
return
}
require.NoError(t, err)
// Extract the duration expression from the vector selector
vectorSelector, ok := expr.(*parser.VectorSelector)
require.True(t, ok, "Expected vector selector, got %T", expr)
result := vectorSelector.OriginalOffset
if vectorSelector.OriginalOffsetExpr != nil {
result, err = calculateDuration(vectorSelector.OriginalOffsetExpr, false)
require.NoError(t, err)
}
require.Equal(t, tt.expected, result)
})
t.Run(tt.name+" subquery with fixed step", func(t *testing.T) {
expr, err := parser.ParseExpr("foo[5m:(" + tt.expr + ")]")
if tt.expectError || tt.expected < 0 {
require.Error(t, err)
return
}
require.NoError(t, err)
// Extract the duration expression from the subquery
subquery, ok := expr.(*parser.SubqueryExpr)
require.True(t, ok, "Expected subquery, got %T", expr)
require.Equal(t, 5*time.Minute, subquery.Range)
result := subquery.Step
if subquery.StepExpr != nil {
result, err = calculateDuration(subquery.StepExpr, false)
require.NoError(t, err)
}
require.Equal(t, tt.expected, result)
})
t.Run(tt.name+" subquery with fixed range", func(t *testing.T) {
expr, err := parser.ParseExpr("foo[(" + tt.expr + "):5m]")
if tt.expectError || tt.expected < 0 {
require.Error(t, err)
return
}
require.NoError(t, err)
// Extract the duration expression from the subquery
subquery, ok := expr.(*parser.SubqueryExpr)
require.True(t, ok, "Expected subquery, got %T", expr)
require.Equal(t, 5*time.Minute, subquery.Step)
result := subquery.Range
if subquery.RangeExpr != nil {
result, err = calculateDuration(subquery.RangeExpr, false)
require.NoError(t, err)
}
require.Equal(t, tt.expected, result)
})
t.Run(tt.name+" matrix selector", func(t *testing.T) {
expr, err := parser.ParseExpr("foo[(" + tt.expr + ")]")
if tt.expectError || tt.expected < 0 {
require.Error(t, err)
return
}
require.NoError(t, err)
// Extract the duration expression from the matrix selector
matrixSelector, ok := expr.(*parser.MatrixSelector)
require.True(t, ok, "Expected matrix selector, got %T", expr)
result := matrixSelector.Range
if matrixSelector.RangeExpr != nil {
result, err = calculateDuration(matrixSelector.RangeExpr, false)
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) {
@ -3591,15 +3591,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

@ -3087,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)
}

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,24 +133,27 @@ type MatrixSelector struct {
// if the parser hasn't returned an error.
VectorSelector Expr
Range time.Duration
EndPos posrange.Pos
RangeExpr *DurationExpr
EndPos posrange.Pos
}
// SubqueryExpr represents a subquery.
type SubqueryExpr struct {
Expr Expr
Range time.Duration
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

@ -235,6 +235,7 @@ expr :
| unary_expr
| vector_selector
| step_invariant_expr
| duration_expr
;
/*
@ -435,33 +436,28 @@ paren_expr : LEFT_PAREN expr RIGHT_PAREN
positive_duration_expr : duration_expr
{
numLit, ok := $1.(*NumberLiteral)
if !ok {
// This should never happen but handle it gracefully.
yylex.(*parser).addParseErrf(posrange.PositionRange{}, "internal error: duration expression did not evaluate to a number")
$$ = &NumberLiteral{Val: 1} // Use 1 as fallback to prevent cascading errors.
} else if numLit.Val > 0 {
$$ = numLit
} else {
yylex.(*parser).addParseErrf(numLit.PosRange, "duration must be greater than 0")
$$ = &NumberLiteral{Val: 1, PosRange: numLit.PosRange} // Use 1 as fallback.
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
}
;
offset_expr: expr OFFSET duration_expr
{
numLit, _ := $3.(*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
}
| expr OFFSET SUB duration_expr
{
numLit, _ := $4.(*NumberLiteral)
dur := time.Duration(numLit.Val * 1000) * time.Millisecond
yylex.(*parser).addOffset($1, -dur)
$$ = $1
yylex.(*parser).addOffsetExpr($1, $3.(*DurationExpr))
$$ = $1
}
| expr OFFSET error
{ yylex.(*parser).unexpected("offset", "number or duration"); $$ = $1 }
@ -508,10 +504,15 @@ matrix_selector : expr LEFT_BRACKET positive_duration_expr 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,
}
}
@ -519,25 +520,39 @@ matrix_selector : expr LEFT_BRACKET positive_duration_expr 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 positive_duration_expr COLON RIGHT_BRACKET
{
numLitRange, _ := $3.(*NumberLiteral)
$$ = &SubqueryExpr{
Expr: $1.(Expr),
Range: time.Duration(numLitRange.Val * 1000) * time.Millisecond,
Step: 0,
EndPos: $5.Pos + 1,
}
}
{
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: rangeNl,
RangeExpr: rangeExpr,
EndPos: $5.Pos + 1,
}
}
| expr LEFT_BRACKET positive_duration_expr COLON positive_duration_expr error
{ yylex.(*parser).unexpected("subquery selector", "\"]\""); $$ = $1 }
| expr LEFT_BRACKET positive_duration_expr COLON error
@ -947,6 +962,7 @@ number_duration_literal : NUMBER
$$ = &NumberLiteral{
Val: dur.Seconds(),
PosRange: $1.PositionRange(),
Duration: true,
}
}
;
@ -1019,38 +1035,100 @@ maybe_grouping_labels: /* empty */ { $$ = nil }
*/
duration_expr : number_duration_literal
/* Gives the rule the same precedence as MUL. This aligns with mathematical conventions. */
| unary_op duration_expr %prec MUL
{
nl, ok := $2.(*NumberLiteral)
if !ok {
yylex.(*parser).addParseErrf($1.PositionRange(), "expected number literal in duration expression")
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
}
if $1.Typ == SUB {
nl.Val *= -1
}
nl.PosRange.Start = $1.Pos
$$ = 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).evalDurationExprBinOp($1, $3, $2) }
{
yylex.(*parser).experimentalDurationExpr($1.(Expr))
$$ = &DurationExpr{Op: ADD, LHS: $1.(Expr), RHS: $3.(Expr)}
}
| duration_expr SUB duration_expr
{ $$ = yylex.(*parser).evalDurationExprBinOp($1, $3, $2) }
{
yylex.(*parser).experimentalDurationExpr($1.(Expr))
$$ = &DurationExpr{Op: SUB, LHS: $1.(Expr), RHS: $3.(Expr)}
}
| duration_expr MUL duration_expr
{ $$ = yylex.(*parser).evalDurationExprBinOp($1, $3, $2) }
{
yylex.(*parser).experimentalDurationExpr($1.(Expr))
$$ = &DurationExpr{Op: MUL, LHS: $1.(Expr), RHS: $3.(Expr)}
}
| duration_expr DIV duration_expr
{ $$ = yylex.(*parser).evalDurationExprBinOp($1, $3, $2) }
{
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).evalDurationExprBinOp($1, $3, $2) }
{
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).evalDurationExprBinOp($1, $3, $2) }
{
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
{ $$ = $2 }
{
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

@ -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 {
@ -1060,71 +1109,3 @@ func MustGetFunction(name string) *Function {
}
return f
}
// evalDurationExprBinOp evaluates binary operations for duration expressions.
// It handles type checking, performs the operation using the specified operator,
// and constructs a new NumberLiteral with the result.
func (p *parser) evalDurationExprBinOp(lhs, rhs Node, op Item) *NumberLiteral {
if !ExperimentalDurationExpr {
p.addParseErrf(op.PositionRange(), "experimental duration expression parsing is experimental and must be enabled with --enable-feature=promql-duration-expr")
return &NumberLiteral{Val: 0}
}
numLit1, ok1 := lhs.(*NumberLiteral)
numLit2, ok2 := rhs.(*NumberLiteral)
if !ok1 || !ok2 {
p.addParseErrf(posrange.PositionRange{
Start: lhs.PositionRange().Start,
End: rhs.PositionRange().End,
}, "invalid operands for %s", op.Val)
return &NumberLiteral{Val: 0}
}
var val float64
var err error
switch op.Typ {
case ADD:
val = numLit1.Val + numLit2.Val
case SUB:
val = numLit1.Val - numLit2.Val
case MUL:
val = numLit1.Val * numLit2.Val
case DIV:
if numLit2.Val == 0 {
err = errors.New("division by zero")
} else {
val = numLit1.Val / numLit2.Val
}
case MOD:
if numLit2.Val == 0 {
err = errors.New("modulo by zero")
} else {
val = math.Mod(numLit1.Val, numLit2.Val)
}
case POW:
val = math.Pow(numLit1.Val, numLit2.Val)
default:
p.addParseErrf(op.PositionRange(), "unknown operator for duration expression: %s", op.Val)
return &NumberLiteral{Val: 0}
}
if err != nil {
p.addParseErrf(numLit2.PosRange, err.Error())
return &NumberLiteral{Val: 0}
}
if val > 1<<63/1e9 || val < -(1<<63)/1e9 {
p.addParseErrf(op.PositionRange(), "duration out of range")
return &NumberLiteral{Val: 0}
}
return &NumberLiteral{
Val: val,
PosRange: posrange.PositionRange{
Start: numLit1.PosRange.Start,
End: numLit2.PosRange.End,
},
}
}

View File

@ -3959,7 +3959,37 @@ var testExpr = []struct {
End: 3,
},
},
Range: 1 * time.Second, // 11s+10s-5*2^2 = 21s-20s = 1s
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,
},
},
@ -3976,7 +4006,41 @@ var testExpr = []struct {
End: 3,
},
},
Range: 15 * time.Second, // -(10s-5s)+20s = -5s+20s = 15s
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,
},
},
@ -3993,7 +4057,25 @@ var testExpr = []struct {
End: 3,
},
},
Range: 5 * time.Second, // -10s+15s = 5s
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,
},
},
@ -4010,10 +4092,63 @@ var testExpr = []struct {
End: 3,
},
},
Range: 8 * time.Second, // 4s+4s = 8s
Step: 2 * time.Second, // 1s*2 = 2s
OriginalOffset: -3 * time.Second, // 5s-8 = -3s
EndPos: 29,
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,
},
},
{
@ -4040,6 +4175,50 @@ var testExpr = []struct {
},
},
},
{
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,
@ -4056,22 +4235,12 @@ var testExpr = []struct {
errMsg: `modulo by zero`,
},
{
input: `foo offset (5s%(2d-2d))`,
fail: true,
errMsg: `modulo by zero`,
},
{
input: `foo[150y+150y]`,
input: `foo offset 9.5e10`,
fail: true,
errMsg: `duration out of range`,
},
{
input: `foo offset (150y+150y)`,
fail: true,
errMsg: `duration out of range`,
},
{
input: `foo offset (-2*150y)`,
input: `foo[9.5e10]`,
fail: true,
errMsg: `duration out of range`,
},

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):