diff --git a/promql/durations.go b/promql/durations.go new file mode 100644 index 0000000000..c047ac0bfc --- /dev/null +++ b/promql/durations.go @@ -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) + } +} diff --git a/promql/durations_test.go b/promql/durations_test.go new file mode 100644 index 0000000000..d1ac4b77d7 --- /dev/null +++ b/promql/durations_test.go @@ -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) + }) + } +} diff --git a/promql/engine.go b/promql/engine.go index 8c37f12e42..60d361a62c 100644 --- a/promql/engine.go +++ b/promql/engine.go @@ -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 diff --git a/promql/engine_test.go b/promql/engine_test.go index 281e4524f9..0181d01534 100644 --- a/promql/engine_test.go +++ b/promql/engine_test.go @@ -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) } diff --git a/promql/parser/ast.go b/promql/parser/ast.go index a9f41978e0..9eebaed9ab 100644 --- a/promql/parser/ast.go +++ b/promql/parser/ast.go @@ -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 } diff --git a/promql/parser/generated_parser.y b/promql/parser/generated_parser.y index d9f9eb5949..de9234589c 100644 --- a/promql/parser/generated_parser.y +++ b/promql/parser/generated_parser.y @@ -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 + } ; %% diff --git a/promql/parser/generated_parser.y.go b/promql/parser/generated_parser.y.go index 26a7914d6d..8c84b42f14 100644 --- a/promql/parser/generated_parser.y.go +++ b/promql/parser/generated_parser.y.go @@ -255,91 +255,98 @@ var yyExca = [...]int16{ 10, 142, 24, 142, -2, 0, - -1, 61, + -1, 63, 2, 185, 15, 185, 79, 185, 85, 185, -2, 103, - -1, 62, + -1, 64, 2, 186, 15, 186, 79, 186, 85, 186, -2, 104, - -1, 63, + -1, 65, 2, 187, 15, 187, 79, 187, 85, 187, -2, 106, - -1, 64, + -1, 66, 2, 188, 15, 188, 79, 188, 85, 188, -2, 107, - -1, 65, + -1, 67, 2, 189, 15, 189, 79, 189, 85, 189, -2, 108, - -1, 66, + -1, 68, 2, 190, 15, 190, 79, 190, 85, 190, -2, 113, - -1, 67, + -1, 69, 2, 191, 15, 191, 79, 191, 85, 191, -2, 115, - -1, 68, + -1, 70, 2, 192, 15, 192, 79, 192, 85, 192, -2, 117, - -1, 69, + -1, 71, 2, 193, 15, 193, 79, 193, 85, 193, -2, 118, - -1, 70, + -1, 72, 2, 194, 15, 194, 79, 194, 85, 194, -2, 119, - -1, 71, + -1, 73, 2, 195, 15, 195, 79, 195, 85, 195, -2, 120, - -1, 72, + -1, 74, 2, 196, 15, 196, 79, 196, 85, 196, -2, 121, - -1, 73, + -1, 75, 2, 197, 15, 197, 79, 197, 85, 197, -2, 125, - -1, 74, + -1, 76, 2, 198, 15, 198, 79, 198, 85, 198, -2, 126, - -1, 209, + -1, 126, + 41, 250, + 42, 250, + 52, 250, + 53, 250, + 57, 250, + -2, 20, + -1, 224, 9, 247, 12, 247, 13, 247, @@ -371,7 +378,7 @@ var yyExca = [...]int16{ 88, 247, 89, 247, -2, 0, - -1, 210, + -1, 225, 9, 247, 12, 247, 13, 247, @@ -407,159 +414,160 @@ var yyExca = [...]int16{ const yyPrivate = 57344 -const yyLast = 882 +const yyLast = 892 var yyAct = [...]int16{ - 158, 358, 356, 161, 363, 243, 39, 201, 293, 44, - 308, 307, 84, 120, 82, 246, 185, 109, 108, 160, - 135, 110, 169, 58, 111, 256, 166, 162, 326, 171, - 112, 262, 209, 210, 113, 349, 165, 287, 245, 370, - 371, 372, 373, 207, 231, 208, 353, 352, 60, 328, - 284, 381, 288, 130, 317, 326, 335, 205, 167, 166, - 258, 259, 364, 115, 260, 116, 357, 107, 289, 165, - 380, 114, 273, 379, 168, 247, 249, 251, 252, 253, - 261, 263, 266, 267, 268, 269, 270, 274, 275, 229, - 122, 248, 250, 254, 255, 257, 264, 265, 111, 230, - 228, 271, 272, 231, 112, 163, 164, 131, 117, 103, - 285, 106, 235, 113, 77, 154, 35, 154, 6, 7, - 180, 346, 174, 177, 129, 167, 172, 55, 173, 155, - 345, 155, 166, 157, 203, 151, 105, 54, 206, 191, - 193, 168, 165, 211, 212, 213, 214, 215, 216, 217, - 218, 219, 220, 221, 222, 223, 224, 225, 204, 75, - 55, 176, 154, 283, 167, 154, 157, 154, 334, 344, - 54, 236, 237, 343, 175, 76, 155, 226, 229, 155, - 168, 155, 232, 333, 187, 233, 282, 234, 230, 228, - 332, 342, 231, 227, 341, 340, 122, 279, 136, 137, - 138, 139, 140, 141, 142, 143, 144, 145, 146, 147, - 148, 149, 150, 316, 2, 3, 4, 5, 305, 306, - 10, 339, 309, 186, 189, 187, 88, 277, 286, 338, - 79, 337, 226, 229, 188, 190, 154, 154, 154, 154, - 154, 154, 276, 230, 228, 281, 197, 231, 227, 154, - 155, 155, 155, 155, 155, 155, 310, 311, 312, 313, - 314, 315, 318, 155, 336, 189, 36, 123, 280, 196, - 85, 195, 348, 107, 184, 188, 190, 121, 323, 183, - 83, 181, 1, 322, 324, 278, 325, 327, 192, 329, - 86, 347, 182, 86, 194, 8, 330, 331, 321, 37, - 89, 91, 92, 156, 93, 94, 95, 96, 97, 98, - 99, 100, 101, 102, 170, 103, 104, 106, 90, 126, - 50, 55, 49, 128, 125, 127, 78, 157, 350, 48, - 351, 54, 240, 107, 47, 46, 239, 124, 359, 360, - 361, 355, 105, 134, 362, 320, 366, 365, 368, 367, - 246, 238, 55, 75, 374, 375, 45, 43, 157, 376, - 256, 91, 54, 132, 319, 378, 262, 178, 42, 76, - 242, 100, 101, 245, 133, 103, 166, 106, 90, 41, - 383, 40, 51, 123, 75, 200, 165, 59, 382, 377, - 9, 9, 290, 121, 87, 258, 259, 198, 241, 260, - 76, 119, 105, 80, 369, 159, 86, 273, 167, 244, - 247, 249, 251, 252, 253, 261, 263, 266, 267, 268, - 269, 270, 274, 275, 168, 52, 248, 250, 254, 255, - 257, 264, 265, 118, 0, 55, 271, 272, 53, 77, - 0, 56, 85, 0, 22, 54, 0, 0, 179, 0, - 199, 57, 83, 0, 166, 0, 0, 202, 0, 0, - 81, 205, 0, 0, 165, 86, 0, 75, 153, 0, - 0, 0, 0, 18, 19, 55, 0, 20, 0, 0, - 0, 157, 0, 76, 0, 54, 167, 0, 61, 62, - 63, 64, 65, 66, 67, 68, 69, 70, 71, 72, - 73, 74, 168, 0, 0, 13, 0, 75, 0, 24, - 0, 30, 0, 0, 31, 32, 55, 38, 107, 53, - 77, 0, 56, 152, 0, 22, 54, 0, 0, 0, - 0, 0, 57, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 89, 91, 0, 75, 0, - 0, 0, 0, 0, 18, 19, 100, 101, 20, 0, - 103, 104, 106, 90, 76, 0, 0, 0, 0, 61, - 62, 63, 64, 65, 66, 67, 68, 69, 70, 71, - 72, 73, 74, 0, 0, 0, 13, 105, 0, 0, - 24, 0, 30, 0, 55, 31, 32, 53, 77, 0, - 56, 292, 0, 22, 54, 0, 0, 0, 291, 0, - 57, 0, 295, 296, 294, 301, 303, 300, 302, 297, - 298, 299, 304, 0, 0, 0, 75, 0, 0, 0, - 0, 0, 18, 19, 0, 0, 20, 0, 0, 0, - 17, 77, 76, 0, 0, 0, 22, 61, 62, 63, - 64, 65, 66, 67, 68, 69, 70, 71, 72, 73, - 74, 0, 0, 0, 13, 0, 0, 0, 24, 0, - 30, 0, 0, 31, 32, 18, 19, 0, 0, 20, - 0, 0, 0, 17, 35, 0, 0, 0, 0, 22, - 11, 12, 14, 15, 16, 21, 23, 25, 26, 27, - 28, 29, 33, 34, 0, 0, 0, 13, 0, 0, - 0, 24, 0, 30, 0, 0, 31, 32, 18, 19, - 0, 0, 20, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 11, 12, 14, 15, 16, 21, 23, - 25, 26, 27, 28, 29, 33, 34, 107, 0, 0, + 166, 359, 357, 169, 364, 251, 39, 216, 301, 52, + 177, 315, 86, 130, 84, 6, 316, 109, 200, 44, + 145, 117, 116, 61, 118, 195, 109, 295, 119, 170, + 371, 372, 373, 374, 120, 109, 222, 121, 223, 224, + 225, 115, 296, 327, 91, 93, 94, 140, 95, 96, + 97, 98, 99, 100, 101, 102, 103, 104, 297, 105, + 106, 108, 92, 93, 123, 202, 125, 124, 105, 126, + 108, 354, 293, 102, 103, 353, 122, 105, 59, 108, + 92, 358, 110, 113, 127, 329, 107, 175, 292, 349, + 318, 285, 132, 114, 112, 107, 327, 115, 111, 174, + 336, 119, 141, 176, 107, 204, 284, 120, 348, 173, + 220, 2, 3, 4, 5, 203, 205, 161, 291, 179, + 180, 181, 182, 183, 184, 185, 190, 163, 194, 163, + 163, 163, 163, 163, 163, 163, 188, 191, 186, 189, + 187, 290, 243, 121, 218, 365, 289, 79, 221, 382, + 208, 206, 35, 226, 227, 228, 229, 230, 231, 232, + 233, 234, 235, 236, 237, 238, 239, 240, 381, 288, + 219, 380, 7, 10, 241, 242, 347, 346, 345, 244, + 245, 344, 343, 81, 163, 163, 164, 109, 164, 164, + 164, 164, 164, 164, 164, 146, 147, 148, 149, 150, + 151, 152, 153, 154, 155, 156, 157, 158, 159, 160, + 342, 132, 287, 212, 91, 93, 94, 139, 95, 96, + 97, 98, 99, 100, 101, 102, 103, 104, 210, 105, + 106, 108, 92, 109, 341, 168, 211, 317, 340, 313, + 314, 339, 174, 164, 164, 294, 178, 338, 337, 50, + 8, 209, 173, 56, 37, 80, 107, 179, 319, 165, + 91, 93, 94, 55, 95, 96, 97, 163, 99, 100, + 101, 102, 103, 104, 175, 105, 106, 108, 92, 248, + 138, 335, 137, 247, 60, 77, 90, 9, 9, 196, + 176, 201, 325, 202, 326, 328, 334, 330, 246, 321, + 113, 78, 107, 333, 331, 332, 56, 36, 110, 113, + 114, 112, 165, 1, 115, 62, 55, 49, 320, 114, + 112, 171, 172, 115, 111, 174, 164, 48, 47, 351, + 46, 352, 144, 204, 45, 173, 43, 383, 77, 360, + 361, 362, 356, 203, 205, 363, 162, 367, 366, 369, + 368, 254, 142, 56, 78, 375, 376, 175, 214, 165, + 377, 264, 174, 55, 192, 217, 379, 270, 133, 220, + 87, 350, 173, 176, 253, 42, 143, 199, 131, 324, + 85, 384, 198, 41, 323, 77, 286, 136, 207, 40, + 51, 88, 135, 88, 175, 197, 266, 267, 215, 322, + 268, 78, 378, 298, 89, 134, 213, 249, 281, 82, + 176, 255, 257, 259, 260, 261, 269, 271, 274, 275, + 276, 277, 278, 282, 283, 254, 370, 256, 258, 262, + 263, 265, 272, 273, 133, 264, 56, 279, 280, 167, + 87, 270, 165, 252, 131, 250, 55, 53, 253, 128, + 85, 174, 129, 0, 0, 0, 0, 88, 83, 0, + 0, 173, 0, 88, 0, 0, 0, 0, 77, 0, + 266, 267, 0, 0, 268, 0, 0, 0, 0, 0, + 0, 0, 281, 175, 78, 255, 257, 259, 260, 261, + 269, 271, 274, 275, 276, 277, 278, 282, 283, 176, + 0, 256, 258, 262, 263, 265, 272, 273, 0, 0, + 56, 279, 280, 54, 79, 0, 57, 300, 0, 22, + 55, 0, 0, 193, 299, 0, 58, 0, 303, 304, + 302, 309, 311, 308, 310, 305, 306, 307, 312, 0, + 0, 0, 77, 0, 0, 0, 0, 0, 18, 19, + 0, 0, 20, 0, 0, 0, 0, 0, 78, 0, + 0, 0, 0, 63, 64, 65, 66, 67, 68, 69, + 70, 71, 72, 73, 74, 75, 76, 0, 0, 0, 13, 0, 0, 0, 24, 0, 30, 0, 0, 31, - 32, 0, 0, 0, 0, 0, 107, 0, 0, 0, - 0, 0, 0, 0, 89, 91, 92, 0, 93, 94, - 95, 96, 97, 98, 99, 100, 101, 102, 0, 103, - 104, 106, 90, 89, 91, 92, 0, 93, 94, 95, - 0, 97, 98, 99, 100, 101, 102, 354, 103, 104, - 106, 90, 107, 0, 0, 0, 105, 0, 295, 296, - 294, 301, 303, 300, 302, 297, 298, 299, 304, 0, - 0, 0, 0, 0, 0, 105, 0, 0, 0, 89, - 91, 92, 0, 93, 94, 0, 0, 97, 98, 0, - 100, 101, 102, 0, 103, 104, 106, 90, 0, 0, + 32, 56, 38, 109, 54, 79, 0, 57, 355, 0, + 22, 55, 0, 0, 0, 0, 0, 58, 0, 303, + 304, 302, 309, 311, 308, 310, 305, 306, 307, 312, + 91, 93, 0, 77, 0, 0, 0, 0, 0, 18, + 19, 102, 103, 20, 0, 105, 106, 108, 92, 78, + 0, 0, 0, 0, 63, 64, 65, 66, 67, 68, + 69, 70, 71, 72, 73, 74, 75, 76, 0, 0, + 0, 13, 107, 0, 0, 24, 0, 30, 0, 56, + 31, 32, 54, 79, 0, 57, 0, 0, 22, 55, + 0, 0, 0, 0, 0, 58, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 77, 0, 0, 0, 0, 0, 18, 19, 0, + 0, 20, 0, 0, 0, 17, 79, 78, 0, 0, + 0, 22, 63, 64, 65, 66, 67, 68, 69, 70, + 71, 72, 73, 74, 75, 76, 0, 0, 0, 13, + 0, 0, 0, 24, 0, 30, 0, 0, 31, 32, + 18, 19, 0, 0, 20, 0, 0, 0, 17, 35, + 0, 0, 0, 0, 22, 11, 12, 14, 15, 16, + 21, 23, 25, 26, 27, 28, 29, 33, 34, 0, + 0, 0, 13, 0, 0, 0, 24, 0, 30, 0, + 0, 31, 32, 18, 19, 0, 0, 20, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 11, 12, + 14, 15, 16, 21, 23, 25, 26, 27, 28, 29, + 33, 34, 109, 0, 0, 13, 0, 0, 0, 24, + 0, 30, 0, 0, 31, 32, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 91, + 93, 94, 0, 95, 96, 0, 0, 99, 100, 0, + 102, 103, 104, 0, 105, 106, 108, 92, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 105, + 0, 107, } var yyPact = [...]int16{ - 116, 109, 671, 671, 507, 628, -1000, -1000, -1000, 103, + 13, 162, 746, 746, 582, 703, -1000, -1000, -1000, 139, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, 440, -1000, 224, -1000, 733, + -1000, -1000, -1000, -1000, -1000, 438, -1000, 284, -1000, 173, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, 19, 98, -1000, -1000, 585, -1000, 585, 101, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, 381, -1000, -1000, - 317, -1000, -1000, 321, 120, -1000, -1000, 29, -1000, -58, - -58, -58, -58, -58, -58, -58, -58, -58, -58, -58, - -58, -58, -58, -58, -58, 466, 17, 312, 98, -55, - -1000, 159, 159, 426, -1000, 259, 53, -1000, 272, -1000, - -1000, 221, 180, -1000, -1000, 268, -1000, 269, -1000, 244, - 445, 585, -1000, -39, -48, -1000, 585, 585, 585, 585, - 585, 585, 585, 585, 585, 585, 585, 585, 585, 585, - 585, -1000, 151, -1000, -1000, 118, -1000, 118, -1000, 97, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, 50, 50, 330, - -1000, 136, -1000, -1000, -1000, 348, -1000, -1000, 220, -1000, - 733, -1000, -1000, 265, -1000, 243, -1000, -1000, -1000, -1000, - -1000, 161, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 24, - 84, 11, -1000, -1000, -1000, 584, 319, 159, 159, 159, - 159, 53, 53, 504, 504, 504, 798, 752, 504, 504, - 798, 53, 53, 504, 53, 319, 118, 118, 118, 118, - 118, 118, 47, -12, 191, 32, -1000, -1000, -1000, 343, - -1000, 276, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, 41, 22, 128, -1000, -1000, 660, -1000, 660, + 134, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 432, + -1000, -1000, 385, -1000, -1000, 278, 213, -1000, -1000, 23, + -1000, -58, -58, -58, -58, -58, -58, -58, -58, -58, + -58, -58, -58, -58, -58, -58, -58, 344, 233, 244, + 427, 427, 427, 427, 427, 427, 128, -51, -1000, 124, + 124, 501, -1000, 3, 267, 12, -15, -1000, 375, -1000, + -1000, 289, 61, -1000, -1000, 368, -1000, 226, -1000, 211, + 353, 660, -1000, -46, -41, -1000, 660, 660, 660, 660, + 660, 660, 660, 660, 660, 660, 660, 660, 660, 660, + 660, -1000, -1000, -1000, 427, 427, -1000, 127, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, 90, 90, 277, -1000, 41, + 258, 258, -15, -15, -15, -15, -1000, -1000, -1000, 423, + -1000, -1000, 84, -1000, 173, -1000, -1000, -1000, 366, -1000, + 144, -1000, -1000, -1000, -1000, -1000, 116, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, 62, 46, 1, -1000, -1000, -1000, + 500, 21, 124, 124, 124, 124, 12, 12, 579, 579, + 579, 808, 219, 579, 579, 808, 12, 12, 579, 12, + 21, -15, 267, 68, -1000, -1000, -1000, 297, -1000, 377, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, 585, -1000, -1000, - -1000, -1000, -1000, -1000, 36, 36, 23, 36, 41, 41, - 166, 39, -1000, -1000, 258, 225, 223, 215, 189, 188, - 185, 167, 163, 124, 115, -1000, -1000, -1000, -1000, -1000, - 47, 47, -12, -12, -12, -12, -1000, -1000, 270, -1000, - -1000, -1000, 13, -1000, 733, -1000, -1000, -1000, 36, -1000, - 21, 20, 790, -1000, -1000, -1000, 9, 123, 123, 123, - 50, 48, 48, 9, 48, 9, -53, -1000, -1000, -1000, - -1000, -1000, 36, 36, -1000, -1000, -1000, 36, -1000, -1000, - -1000, -1000, -1000, -1000, 123, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, 49, -1000, 367, - -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, 660, -1000, -1000, -1000, -1000, + -1000, -1000, 77, 77, 59, 77, 94, 94, 279, 83, + -1000, -1000, 242, 241, 235, 232, 228, 204, 176, 175, + 172, 171, 170, -1000, -1000, -1000, -1000, -1000, -1000, 87, + -1000, -1000, -1000, 349, -1000, 173, -1000, -1000, -1000, 77, + -1000, 49, 45, 581, -1000, -1000, -1000, 24, 442, 442, + 442, 90, 131, 131, 24, 131, 24, -62, -1000, -1000, + -1000, -1000, -1000, 77, 77, -1000, -1000, -1000, 77, -1000, + -1000, -1000, -1000, -1000, -1000, 442, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 147, -1000, + 316, -1000, -1000, -1000, -1000, } var yyPgo = [...]int16{ - 0, 433, 13, 425, 5, 16, 409, 387, 23, 405, - 12, 404, 220, 295, 403, 14, 398, 10, 11, 397, - 394, 7, 392, 8, 4, 389, 2, 1, 3, 385, - 27, 0, 382, 381, 18, 107, 379, 374, 6, 368, - 367, 17, 363, 48, 357, 9, 356, 343, 335, 334, - 329, 322, 320, 29, 303, 22, 282, 266, + 0, 449, 13, 447, 5, 18, 443, 284, 78, 439, + 12, 426, 173, 250, 409, 14, 407, 16, 11, 406, + 404, 7, 403, 8, 4, 402, 2, 1, 3, 398, + 29, 0, 390, 389, 22, 102, 383, 376, 6, 375, + 364, 21, 352, 23, 336, 19, 334, 332, 330, 328, + 327, 317, 249, 9, 315, 10, 313, 307, } var yyR1 = [...]int8{ 0, 56, 56, 56, 56, 56, 56, 56, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, - 33, 33, 33, 33, 34, 34, 36, 36, 36, 36, + 38, 33, 33, 33, 33, 34, 34, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, - 36, 36, 35, 37, 37, 47, 47, 42, 42, 42, - 42, 17, 17, 17, 17, 16, 16, 16, 4, 4, - 4, 39, 41, 41, 40, 40, 40, 48, 55, 46, + 36, 36, 36, 35, 37, 37, 47, 47, 42, 42, + 42, 42, 17, 17, 17, 17, 16, 16, 16, 4, + 4, 4, 39, 41, 41, 40, 40, 40, 48, 55, 46, 46, 32, 32, 32, 9, 9, 44, 50, 50, 50, 50, 50, 50, 51, 52, 52, 52, 43, 43, 43, 1, 1, 1, 2, 2, 2, 2, 2, 2, @@ -584,12 +592,12 @@ var yyR1 = [...]int8{ var yyR2 = [...]int8{ 0, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, - 3, 3, 2, 2, 2, 2, 4, 4, 4, 4, + 1, 3, 3, 2, 2, 2, 2, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, - 4, 4, 1, 0, 1, 3, 3, 1, 1, 3, - 3, 3, 4, 2, 1, 3, 1, 2, 1, 1, - 1, 2, 3, 2, 3, 1, 2, 3, 1, 3, - 4, 3, 3, 5, 3, 1, 1, 4, 6, 5, + 4, 4, 4, 1, 0, 1, 3, 3, 1, 1, + 3, 3, 3, 4, 2, 1, 3, 1, 2, 1, + 1, 1, 2, 3, 2, 3, 1, 2, 3, 1, + 3, 3, 3, 5, 3, 1, 1, 4, 6, 5, 6, 5, 4, 3, 2, 2, 1, 1, 3, 4, 2, 3, 1, 2, 3, 3, 1, 3, 3, 2, 1, 2, 1, 1, 1, 1, 1, 1, 1, 1, @@ -616,40 +624,40 @@ var yyChk = [...]int16{ 51, 67, 18, 68, 83, 69, 70, 71, 72, 73, 85, 88, 89, 74, 75, 13, -57, -13, 10, -38, -33, -36, -39, -44, -45, -46, -48, -49, -50, -51, - -52, -32, -3, 12, 19, 9, 15, 25, -8, -7, - -43, 62, 63, 64, 65, 66, 67, 68, 69, 70, - 71, 72, 73, 74, 75, 41, 57, 13, -52, -12, - -14, 20, -15, 12, -10, 2, 25, -20, 2, 41, - 59, 42, 43, 45, 46, 47, 48, 49, 50, 51, - 52, 53, 54, 56, 57, 83, 58, 14, -34, -41, - 2, 79, 85, 15, -41, -38, -38, -43, -1, 20, + -52, -32, -53, -3, 12, 19, 9, 15, 25, -8, + -7, -43, -54, 62, 63, 64, 65, 66, 67, 68, + 69, 70, 71, 72, 73, 74, 75, 41, 57, 13, + -52, -12, -14, 20, -15, 12, -10, 2, 25, -20, + 2, 41, 59, 42, 43, 45, 46, 47, 48, 49, + 50, 51, 52, 53, 54, 56, 57, 83, 58, 14, + 41, 57, 53, 42, 52, 56, -34, -41, 2, 79, + 85, 15, -41, -38, -53, -38, -53, -43, -1, 20, -2, 12, -10, 2, 20, 7, 2, 4, 2, 4, 24, -35, -42, -37, -47, 78, -35, -35, -35, -35, -35, -35, -35, -35, -35, -35, -35, -35, -35, -35, - -35, -53, 57, 2, -45, -8, -54, 15, -31, -9, - 2, -28, -30, 88, 89, 19, 9, 41, 57, -55, - 2, -53, -41, -34, -17, 15, 2, -17, -40, 22, - -38, 22, 20, 7, 2, -5, 2, 4, 54, 44, - 55, -5, 20, -15, 25, 2, 25, 2, -19, 5, - -29, -21, 12, -28, -30, 16, -38, 82, 84, 80, - 81, -38, -38, -38, -38, -38, -38, -38, -38, -38, - -38, -38, -38, -38, -38, -38, 41, 57, 53, 42, - 52, 56, -53, -53, -53, 15, -28, -28, 21, 6, - 2, -16, 22, -4, -6, 25, 2, 62, 78, 63, - 79, 64, 65, 66, 80, 81, 12, 82, 47, 48, - 51, 67, 18, 68, 83, 84, 69, 70, 71, 72, - 73, 88, 89, 59, 74, 75, 22, 7, 20, -2, - 25, 2, 25, 2, 26, 26, -30, 26, 41, 57, - -22, 24, 17, -23, 30, 28, 29, 35, 36, 37, - 33, 31, 34, 32, 38, -17, -17, -18, -17, -18, - -53, -53, -53, -53, -53, -53, 22, 22, -55, 21, - 2, 22, 7, 2, -38, -27, 19, -27, 26, -27, - -21, -21, 24, 17, 2, 17, 6, 6, 6, 6, - 6, 6, 6, 6, 6, 6, 6, 21, 2, 22, - -4, -27, 26, 26, 17, -23, -26, 57, -27, -31, - -31, -31, -28, -24, 14, -24, -26, -24, -26, -11, - 92, 93, 94, 95, -27, -27, -27, -25, -31, 24, - 21, 2, 21, -31, + -35, -53, 2, -45, -8, 15, -31, -9, 2, -28, + -30, 88, 89, 19, 9, 41, 57, -55, 2, -53, + -53, -53, -53, -53, -53, -53, -41, -34, -17, 15, + 2, -17, -40, 22, -38, 22, 22, 20, 7, 2, + -5, 2, 4, 54, 44, 55, -5, 20, -15, 25, + 2, 25, 2, -19, 5, -29, -21, 12, -28, -30, + 16, -38, 82, 84, 80, 81, -38, -38, -38, -38, + -38, -38, -38, -38, -38, -38, -38, -38, -38, -38, + -38, -53, -53, 15, -28, -28, 21, 6, 2, -16, + 22, -4, -6, 25, 2, 62, 78, 63, 79, 64, + 65, 66, 80, 81, 12, 82, 47, 48, 51, 67, + 18, 68, 83, 84, 69, 70, 71, 72, 73, 88, + 89, 59, 74, 75, 22, 7, 20, -2, 25, 2, + 25, 2, 26, 26, -30, 26, 41, 57, -22, 24, + 17, -23, 30, 28, 29, 35, 36, 37, 33, 31, + 34, 32, 38, -17, -17, -18, -17, -18, 22, -55, + 21, 2, 22, 7, 2, -38, -27, 19, -27, 26, + -27, -21, -21, 24, 17, 2, 17, 6, 6, 6, + 6, 6, 6, 6, 6, 6, 6, 6, 21, 2, + 22, -4, -27, 26, 26, 17, -23, -26, 57, -27, + -31, -31, -31, -28, -24, 14, -24, -26, -24, -26, + -11, 92, 93, 94, 95, -27, -27, -27, -25, -31, + 24, 21, 2, 21, -31, } var yyDef = [...]int16{ @@ -658,40 +666,40 @@ var yyDef = [...]int16{ 112, 113, 114, 115, 116, 117, 118, 119, 120, 121, 122, 123, 124, 125, 126, 0, 2, -2, 3, 4, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, - 18, 19, 0, 109, 234, 235, 0, 245, 0, 86, - 87, -2, -2, -2, -2, -2, -2, -2, -2, -2, - -2, -2, -2, -2, -2, 228, 229, 0, 5, 101, - 0, 129, 132, 0, 136, 140, 246, 141, 145, 43, - 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, - 43, 43, 43, 43, 43, 0, 0, 0, 0, 22, - 23, 0, 0, 0, 61, 0, 84, 85, 0, 90, + 18, 19, 20, 0, 109, 234, 235, 0, 245, 0, + 86, 87, 257, -2, -2, -2, -2, -2, -2, -2, + -2, -2, -2, -2, -2, -2, -2, 228, 229, 0, + 5, 101, 0, 129, 132, 0, 136, 140, 246, 141, + 145, 44, 44, 44, 44, 44, 44, 44, 44, 44, + 44, 44, 44, 44, 44, 44, 44, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 23, 24, 0, + 0, 0, 62, 0, 20, 84, -2, 85, 0, 90, 92, 0, 96, 100, 127, 0, 133, 0, 139, 0, - 144, 0, 42, 47, 48, 44, 0, 0, 0, 0, + 144, 0, 43, 48, 49, 45, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 69, 229, 71, 249, 0, 257, 0, 72, 0, - 74, 240, 241, 75, 76, 236, 237, 0, 0, 0, - 83, 68, 20, 21, 24, 0, 54, 25, 0, 63, - 65, 67, 88, 0, 93, 0, 99, 230, 231, 232, - 233, 0, 128, 131, 134, 137, 135, 138, 143, 146, - 148, 151, 155, 156, 157, 0, 26, 0, 0, -2, - -2, 27, 28, 29, 30, 31, 32, 33, 34, 35, - 36, 37, 38, 39, 40, 41, 0, 0, 0, 0, - 0, 0, 70, 250, 0, 0, 238, 239, 77, 0, - 82, 0, 53, 56, 58, 59, 60, 199, 200, 201, - 202, 203, 204, 205, 206, 207, 208, 209, 210, 211, - 212, 213, 214, 215, 216, 217, 218, 219, 220, 221, - 222, 223, 224, 225, 226, 227, 62, 66, 89, 91, - 94, 98, 95, 97, 0, 0, 0, 0, 0, 0, - 0, 0, 161, 163, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 45, 46, 49, 248, 50, - 251, 252, 253, 254, 255, 256, 258, 73, 0, 79, - 81, 51, 0, 57, 64, 147, 242, 149, 0, 152, - 0, 0, 0, 159, 164, 160, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 78, 80, 52, - 55, 150, 0, 0, 158, 162, 165, 0, 244, 166, - 167, 168, 169, 170, 0, 171, 172, 173, 174, 175, - 181, 182, 183, 184, 153, 154, 243, 0, 179, 0, - 177, 180, 176, 178, + 0, 70, 71, 249, 0, 0, 72, 0, 74, 240, + 241, 75, 76, 236, 237, 0, 0, 0, 83, 69, + 251, 252, 253, 254, 255, 256, 21, 22, 25, 0, + 55, 26, 0, 64, 66, 68, 258, 88, 0, 93, + 0, 99, 230, 231, 232, 233, 0, 128, 131, 134, + 137, 135, 138, 143, 146, 148, 151, 155, 156, 157, + 0, 27, 0, 0, -2, -2, 28, 29, 30, 31, + 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, + 42, 250, 0, 0, 238, 239, 77, 0, 82, 0, + 54, 57, 59, 60, 61, 199, 200, 201, 202, 203, + 204, 205, 206, 207, 208, 209, 210, 211, 212, 213, + 214, 215, 216, 217, 218, 219, 220, 221, 222, 223, + 224, 225, 226, 227, 63, 67, 89, 91, 94, 98, + 95, 97, 0, 0, 0, 0, 0, 0, 0, 0, + 161, 163, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 46, 47, 50, 248, 51, 73, 0, + 79, 81, 52, 0, 58, 65, 147, 242, 149, 0, + 152, 0, 0, 0, 159, 164, 160, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 78, 80, + 53, 56, 150, 0, 0, 158, 162, 165, 0, 244, + 166, 167, 168, 169, 170, 0, 171, 172, 173, 174, + 175, 181, 182, 183, 184, 153, 154, 243, 0, 179, + 0, 177, 180, 176, 178, } var yyTok1 = [...]int8{ @@ -1076,46 +1084,41 @@ yydefault: { yylex.(*parser).unexpected("", "") } - case 20: + case 21: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.node = yylex.(*parser).newAggregateExpr(yyDollar[1].item, yyDollar[2].node, yyDollar[3].node) } - case 21: + case 22: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.node = yylex.(*parser).newAggregateExpr(yyDollar[1].item, yyDollar[3].node, yyDollar[2].node) } - case 22: + case 23: yyDollar = yyS[yypt-2 : yypt+1] { yyVAL.node = yylex.(*parser).newAggregateExpr(yyDollar[1].item, &AggregateExpr{}, yyDollar[2].node) } - case 23: + case 24: yyDollar = yyS[yypt-2 : yypt+1] { yylex.(*parser).unexpected("aggregation", "") yyVAL.node = yylex.(*parser).newAggregateExpr(yyDollar[1].item, &AggregateExpr{}, Expressions{}) } - case 24: - yyDollar = yyS[yypt-2 : yypt+1] - { - yyVAL.node = &AggregateExpr{ - Grouping: yyDollar[2].strings, - } - } case 25: yyDollar = yyS[yypt-2 : yypt+1] { yyVAL.node = &AggregateExpr{ Grouping: yyDollar[2].strings, - Without: true, } } case 26: - yyDollar = yyS[yypt-4 : yypt+1] + yyDollar = yyS[yypt-2 : yypt+1] { - yyVAL.node = yylex.(*parser).newBinaryExpression(yyDollar[1].node, yyDollar[2].item, yyDollar[3].node, yyDollar[4].node) + yyVAL.node = &AggregateExpr{ + Grouping: yyDollar[2].strings, + Without: true, + } } case 27: yyDollar = yyS[yypt-4 : yypt+1] @@ -1192,14 +1195,19 @@ yydefault: { yyVAL.node = yylex.(*parser).newBinaryExpression(yyDollar[1].node, yyDollar[2].item, yyDollar[3].node, yyDollar[4].node) } - case 43: + case 42: + yyDollar = yyS[yypt-4 : yypt+1] + { + yyVAL.node = yylex.(*parser).newBinaryExpression(yyDollar[1].node, yyDollar[2].item, yyDollar[3].node, yyDollar[4].node) + } + case 44: yyDollar = yyS[yypt-0 : yypt+1] { yyVAL.node = &BinaryExpr{ VectorMatching: &VectorMatching{Card: CardOneToOne}, } } - case 44: + case 45: yyDollar = yyS[yypt-1 : yypt+1] { yyVAL.node = &BinaryExpr{ @@ -1207,71 +1215,71 @@ yydefault: ReturnBool: true, } } - case 45: + case 46: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.node = yyDollar[1].node yyVAL.node.(*BinaryExpr).VectorMatching.MatchingLabels = yyDollar[3].strings } - case 46: + case 47: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.node = yyDollar[1].node yyVAL.node.(*BinaryExpr).VectorMatching.MatchingLabels = yyDollar[3].strings yyVAL.node.(*BinaryExpr).VectorMatching.On = true } - case 49: + case 50: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.node = yyDollar[1].node yyVAL.node.(*BinaryExpr).VectorMatching.Card = CardManyToOne yyVAL.node.(*BinaryExpr).VectorMatching.Include = yyDollar[3].strings } - case 50: + case 51: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.node = yyDollar[1].node yyVAL.node.(*BinaryExpr).VectorMatching.Card = CardOneToMany yyVAL.node.(*BinaryExpr).VectorMatching.Include = yyDollar[3].strings } - case 51: + case 52: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.strings = yyDollar[2].strings } - case 52: + case 53: yyDollar = yyS[yypt-4 : yypt+1] { yyVAL.strings = yyDollar[2].strings } - case 53: + case 54: yyDollar = yyS[yypt-2 : yypt+1] { yyVAL.strings = []string{} } - case 54: + case 55: yyDollar = yyS[yypt-1 : yypt+1] { yylex.(*parser).unexpected("grouping opts", "\"(\"") yyVAL.strings = nil } - case 55: + case 56: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.strings = append(yyDollar[1].strings, yyDollar[3].item.Val) } - case 56: + case 57: yyDollar = yyS[yypt-1 : yypt+1] { yyVAL.strings = []string{yyDollar[1].item.Val} } - case 57: + case 58: yyDollar = yyS[yypt-2 : yypt+1] { yylex.(*parser).unexpected("grouping opts", "\",\" or \")\"") yyVAL.strings = yyDollar[1].strings } - case 58: + case 59: yyDollar = yyS[yypt-1 : yypt+1] { if !model.LabelName(yyDollar[1].item.Val).IsValid() { @@ -1279,7 +1287,7 @@ yydefault: } yyVAL.item = yyDollar[1].item } - case 59: + case 60: yyDollar = yyS[yypt-1 : yypt+1] { unquoted := yylex.(*parser).unquoteString(yyDollar[1].item.Val) @@ -1290,13 +1298,13 @@ yydefault: yyVAL.item.Pos++ yyVAL.item.Val = unquoted } - case 60: + case 61: yyDollar = yyS[yypt-1 : yypt+1] { yylex.(*parser).unexpected("grouping opts", "label") yyVAL.item = Item{} } - case 61: + case 62: yyDollar = yyS[yypt-2 : yypt+1] { fn, exist := getFunction(yyDollar[1].item.Val, yylex.(*parser).functions) @@ -1315,66 +1323,60 @@ yydefault: }, } } - case 62: + case 63: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.node = yyDollar[2].node } - case 63: + case 64: yyDollar = yyS[yypt-2 : yypt+1] { yyVAL.node = Expressions{} } - case 64: + case 65: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.node = append(yyDollar[1].node.(Expressions), yyDollar[3].node.(Expr)) } - case 65: + case 66: yyDollar = yyS[yypt-1 : yypt+1] { yyVAL.node = Expressions{yyDollar[1].node.(Expr)} } - case 66: + case 67: yyDollar = yyS[yypt-2 : yypt+1] { yylex.(*parser).addParseErrf(yyDollar[2].item.PositionRange(), "trailing commas not allowed in function call args") yyVAL.node = yyDollar[1].node } - case 67: + case 68: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.node = &ParenExpr{Expr: yyDollar[2].node.(Expr), PosRange: mergeRanges(&yyDollar[1].item, &yyDollar[3].item)} } - case 68: + case 69: yyDollar = yyS[yypt-1 : yypt+1] { - numLit, ok := yyDollar[1].node.(*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") - yyVAL.node = &NumberLiteral{Val: 1} // Use 1 as fallback to prevent cascading errors. - } else if numLit.Val > 0 { - yyVAL.node = numLit - } else { - yylex.(*parser).addParseErrf(numLit.PosRange, "duration must be greater than 0") - yyVAL.node = &NumberLiteral{Val: 1, PosRange: numLit.PosRange} // Use 1 as fallback. + if numLit, ok := yyDollar[1].node.(*NumberLiteral); ok { + if numLit.Val <= 0 { + yylex.(*parser).addParseErrf(numLit.PositionRange(), "duration must be greater than 0") + yyVAL.node = &NumberLiteral{Val: 0} // Return 0 on error. + break + } + yyVAL.node = yyDollar[1].node + break } - } - case 69: - yyDollar = yyS[yypt-3 : yypt+1] - { - numLit, _ := yyDollar[3].node.(*NumberLiteral) - dur := time.Duration(numLit.Val*1000) * time.Millisecond - yylex.(*parser).addOffset(yyDollar[1].node, dur) yyVAL.node = yyDollar[1].node } case 70: - yyDollar = yyS[yypt-4 : yypt+1] + yyDollar = yyS[yypt-3 : yypt+1] { - numLit, _ := yyDollar[4].node.(*NumberLiteral) - dur := time.Duration(numLit.Val*1000) * time.Millisecond - yylex.(*parser).addOffset(yyDollar[1].node, -dur) + if numLit, ok := yyDollar[3].node.(*NumberLiteral); ok { + yylex.(*parser).addOffset(yyDollar[1].node, time.Duration(numLit.Val*1000)*time.Millisecond) + yyVAL.node = yyDollar[1].node + break + } + yylex.(*parser).addOffsetExpr(yyDollar[1].node, yyDollar[3].node.(*DurationExpr)) yyVAL.node = yyDollar[1].node } case 71: @@ -1419,34 +1421,53 @@ yydefault: yylex.(*parser).addParseErrf(errRange, "%s", errMsg) } - numLit, _ := yyDollar[3].node.(*NumberLiteral) + var rangeNl time.Duration + if numLit, ok := yyDollar[3].node.(*NumberLiteral); ok { + rangeNl = time.Duration(numLit.Val*1000) * time.Millisecond + } + rangeExpr, _ := yyDollar[3].node.(*DurationExpr) yyVAL.node = &MatrixSelector{ VectorSelector: yyDollar[1].node.(Expr), - Range: time.Duration(numLit.Val*1000) * time.Millisecond, + Range: rangeNl, + RangeExpr: rangeExpr, EndPos: yylex.(*parser).lastClosing, } } case 78: yyDollar = yyS[yypt-6 : yypt+1] { - numLitRange, _ := yyDollar[3].node.(*NumberLiteral) - numLitStep, _ := yyDollar[5].node.(*NumberLiteral) + var rangeNl time.Duration + var stepNl time.Duration + if numLit, ok := yyDollar[3].node.(*NumberLiteral); ok { + rangeNl = time.Duration(numLit.Val*1000) * time.Millisecond + } + rangeExpr, _ := yyDollar[3].node.(*DurationExpr) + if numLit, ok := yyDollar[5].node.(*NumberLiteral); ok { + stepNl = time.Duration(numLit.Val*1000) * time.Millisecond + } + stepExpr, _ := yyDollar[5].node.(*DurationExpr) yyVAL.node = &SubqueryExpr{ - Expr: yyDollar[1].node.(Expr), - Range: time.Duration(numLitRange.Val*1000) * time.Millisecond, - Step: time.Duration(numLitStep.Val*1000) * time.Millisecond, - EndPos: yyDollar[6].item.Pos + 1, + Expr: yyDollar[1].node.(Expr), + Range: rangeNl, + RangeExpr: rangeExpr, + Step: stepNl, + StepExpr: stepExpr, + EndPos: yyDollar[6].item.Pos + 1, } } case 79: yyDollar = yyS[yypt-5 : yypt+1] { - numLitRange, _ := yyDollar[3].node.(*NumberLiteral) + var rangeNl time.Duration + if numLit, ok := yyDollar[3].node.(*NumberLiteral); ok { + rangeNl = time.Duration(numLit.Val*1000) * time.Millisecond + } + rangeExpr, _ := yyDollar[3].node.(*DurationExpr) yyVAL.node = &SubqueryExpr{ - Expr: yyDollar[1].node.(Expr), - Range: time.Duration(numLitRange.Val*1000) * time.Millisecond, - Step: 0, - EndPos: yyDollar[5].item.Pos + 1, + Expr: yyDollar[1].node.(Expr), + Range: rangeNl, + RangeExpr: rangeExpr, + EndPos: yyDollar[5].item.Pos + 1, } } case 80: @@ -1933,6 +1954,7 @@ yydefault: yyVAL.node = &NumberLiteral{ Val: dur.Seconds(), PosRange: yyDollar[1].item.PositionRange(), + Duration: true, } } case 236: @@ -2002,54 +2024,105 @@ yydefault: { yyVAL.strings = nil } - case 250: - yyDollar = yyS[yypt-2 : yypt+1] + case 249: + yyDollar = yyS[yypt-1 : yypt+1] { - nl, ok := yyDollar[2].node.(*NumberLiteral) - if !ok { - yylex.(*parser).addParseErrf(yyDollar[1].item.PositionRange(), "expected number literal in duration expression") + nl := yyDollar[1].node.(*NumberLiteral) + if nl.Val > 1<<63/1e9 || nl.Val < -(1<<63)/1e9 { + yylex.(*parser).addParseErrf(nl.PosRange, "duration out of range") yyVAL.node = &NumberLiteral{Val: 0} break } - if yyDollar[1].item.Typ == SUB { - nl.Val *= -1 - } - nl.PosRange.Start = yyDollar[1].item.Pos yyVAL.node = nl } + case 250: + yyDollar = yyS[yypt-2 : yypt+1] + { + switch expr := yyDollar[2].node.(type) { + case *NumberLiteral: + if yyDollar[1].item.Typ == SUB { + expr.Val *= -1 + } + if expr.Val > 1<<63/1e9 || expr.Val < -(1<<63)/1e9 { + yylex.(*parser).addParseErrf(yyDollar[1].item.PositionRange(), "duration out of range") + yyVAL.node = &NumberLiteral{Val: 0} + break + } + expr.PosRange.Start = yyDollar[1].item.Pos + yyVAL.node = expr + break + case *DurationExpr: + if yyDollar[1].item.Typ == SUB { + yyVAL.node = &DurationExpr{ + Op: SUB, + RHS: expr, + StartPos: yyDollar[1].item.Pos, + } + break + } + yyVAL.node = expr + break + default: + yylex.(*parser).addParseErrf(yyDollar[1].item.PositionRange(), "expected number literal or duration expression") + yyVAL.node = &NumberLiteral{Val: 0} + break + } + } case 251: yyDollar = yyS[yypt-3 : yypt+1] { - yyVAL.node = yylex.(*parser).evalDurationExprBinOp(yyDollar[1].node, yyDollar[3].node, yyDollar[2].item) + yylex.(*parser).experimentalDurationExpr(yyDollar[1].node.(Expr)) + yyVAL.node = &DurationExpr{Op: ADD, LHS: yyDollar[1].node.(Expr), RHS: yyDollar[3].node.(Expr)} } case 252: yyDollar = yyS[yypt-3 : yypt+1] { - yyVAL.node = yylex.(*parser).evalDurationExprBinOp(yyDollar[1].node, yyDollar[3].node, yyDollar[2].item) + yylex.(*parser).experimentalDurationExpr(yyDollar[1].node.(Expr)) + yyVAL.node = &DurationExpr{Op: SUB, LHS: yyDollar[1].node.(Expr), RHS: yyDollar[3].node.(Expr)} } case 253: yyDollar = yyS[yypt-3 : yypt+1] { - yyVAL.node = yylex.(*parser).evalDurationExprBinOp(yyDollar[1].node, yyDollar[3].node, yyDollar[2].item) + yylex.(*parser).experimentalDurationExpr(yyDollar[1].node.(Expr)) + yyVAL.node = &DurationExpr{Op: MUL, LHS: yyDollar[1].node.(Expr), RHS: yyDollar[3].node.(Expr)} } case 254: yyDollar = yyS[yypt-3 : yypt+1] { - yyVAL.node = yylex.(*parser).evalDurationExprBinOp(yyDollar[1].node, yyDollar[3].node, yyDollar[2].item) + yylex.(*parser).experimentalDurationExpr(yyDollar[1].node.(Expr)) + if nl, ok := yyDollar[3].node.(*NumberLiteral); ok && nl.Val == 0 { + yylex.(*parser).addParseErrf(yyDollar[2].item.PositionRange(), "division by zero") + yyVAL.node = &NumberLiteral{Val: 0} + break + } + yyVAL.node = &DurationExpr{Op: DIV, LHS: yyDollar[1].node.(Expr), RHS: yyDollar[3].node.(Expr)} } case 255: yyDollar = yyS[yypt-3 : yypt+1] { - yyVAL.node = yylex.(*parser).evalDurationExprBinOp(yyDollar[1].node, yyDollar[3].node, yyDollar[2].item) + yylex.(*parser).experimentalDurationExpr(yyDollar[1].node.(Expr)) + if nl, ok := yyDollar[3].node.(*NumberLiteral); ok && nl.Val == 0 { + yylex.(*parser).addParseErrf(yyDollar[2].item.PositionRange(), "modulo by zero") + yyVAL.node = &NumberLiteral{Val: 0} + break + } + yyVAL.node = &DurationExpr{Op: MOD, LHS: yyDollar[1].node.(Expr), RHS: yyDollar[3].node.(Expr)} } case 256: yyDollar = yyS[yypt-3 : yypt+1] { - yyVAL.node = yylex.(*parser).evalDurationExprBinOp(yyDollar[1].node, yyDollar[3].node, yyDollar[2].item) + yylex.(*parser).experimentalDurationExpr(yyDollar[1].node.(Expr)) + yyVAL.node = &DurationExpr{Op: POW, LHS: yyDollar[1].node.(Expr), RHS: yyDollar[3].node.(Expr)} } case 258: yyDollar = yyS[yypt-3 : yypt+1] { + yylex.(*parser).experimentalDurationExpr(yyDollar[2].node.(Expr)) + if durationExpr, ok := yyDollar[2].node.(*DurationExpr); ok { + durationExpr.Wrapped = true + yyVAL.node = durationExpr + break + } yyVAL.node = yyDollar[2].node } } diff --git a/promql/parser/parse.go b/promql/parser/parse.go index b7962defc8..5cf85ea350 100644 --- a/promql/parser/parse.go +++ b/promql/parser/parse.go @@ -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, - }, - } -} diff --git a/promql/parser/parse_test.go b/promql/parser/parse_test.go index a58dd1bf0d..4c6b1d1b23 100644 --- a/promql/parser/parse_test.go +++ b/promql/parser/parse_test.go @@ -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`, }, diff --git a/promql/parser/prettier.go b/promql/parser/prettier.go index 9870d6da74..568e65eab5 100644 --- a/promql/parser/prettier.go +++ b/promql/parser/prettier.go @@ -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) { diff --git a/promql/parser/prettier_test.go b/promql/parser/prettier_test.go index 16f4906f62..ea9a7a1a26 100644 --- a/promql/parser/prettier_test.go +++ b/promql/parser/prettier_test.go @@ -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)) + }) + } +} diff --git a/promql/parser/printer.go b/promql/parser/printer.go index 6f234a0290..dc22f8fb52 100644 --- a/promql/parser/printer.go +++ b/promql/parser/printer.go @@ -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):