Merge pull request #16249 from roidelapluie/arythmetics
PromQL: allow arithmetic operations in durations in PromQL parser
This commit is contained in:
commit
f4ca1368ff
|
@ -249,6 +249,9 @@ func (c *flagConfig) setFeatureListOptions(logger *slog.Logger) error {
|
|||
case "promql-experimental-functions":
|
||||
parser.EnableExperimentalFunctions = true
|
||||
logger.Info("Experimental PromQL functions enabled.")
|
||||
case "promql-duration-expr":
|
||||
parser.ExperimentalDurationExpr = true
|
||||
logger.Info("Experimental duration expression parsing enabled.")
|
||||
case "native-histograms":
|
||||
c.tsdb.EnableNativeHistograms = true
|
||||
c.scrape.EnableNativeHistogramsIngestion = true
|
||||
|
@ -539,7 +542,7 @@ func main() {
|
|||
a.Flag("scrape.discovery-reload-interval", "Interval used by scrape manager to throttle target groups updates.").
|
||||
Hidden().Default("5s").SetValue(&cfg.scrape.DiscoveryReloadInterval)
|
||||
|
||||
a.Flag("enable-feature", "Comma separated feature names to enable. Valid options: exemplar-storage, expand-external-labels, memory-snapshot-on-shutdown, promql-per-step-stats, promql-experimental-functions, extra-scrape-metrics, auto-gomaxprocs, native-histograms, created-timestamp-zero-ingestion, concurrent-rule-eval, delayed-compaction, old-ui, otlp-deltatocumulative. See https://prometheus.io/docs/prometheus/latest/feature_flags/ for more details.").
|
||||
a.Flag("enable-feature", "Comma separated feature names to enable. Valid options: exemplar-storage, expand-external-labels, memory-snapshot-on-shutdown, promql-per-step-stats, promql-experimental-functions, extra-scrape-metrics, auto-gomaxprocs, native-histograms, created-timestamp-zero-ingestion, concurrent-rule-eval, delayed-compaction, old-ui, otlp-deltatocumulative, promql-duration-expr. See https://prometheus.io/docs/prometheus/latest/feature_flags/ for more details.").
|
||||
Default("").StringsVar(&cfg.featureList)
|
||||
|
||||
a.Flag("agent", "Run Prometheus in 'Agent mode'.").BoolVar(&agentMode)
|
||||
|
|
|
@ -61,7 +61,7 @@ The Prometheus monitoring server
|
|||
| <code class="text-nowrap">--query.timeout</code> | Maximum time a query may take before being aborted. Use with server mode only. | `2m` |
|
||||
| <code class="text-nowrap">--query.max-concurrency</code> | Maximum number of queries executed concurrently. Use with server mode only. | `20` |
|
||||
| <code class="text-nowrap">--query.max-samples</code> | Maximum number of samples a single query can load into memory. Note that queries will fail if they try to load more samples than this into memory, so this also limits the number of samples a query can return. Use with server mode only. | `50000000` |
|
||||
| <code class="text-nowrap">--enable-feature</code> <code class="text-nowrap">...<code class="text-nowrap"> | Comma separated feature names to enable. Valid options: exemplar-storage, expand-external-labels, memory-snapshot-on-shutdown, promql-per-step-stats, promql-experimental-functions, extra-scrape-metrics, auto-gomaxprocs, native-histograms, created-timestamp-zero-ingestion, concurrent-rule-eval, delayed-compaction, old-ui, otlp-deltatocumulative. See https://prometheus.io/docs/prometheus/latest/feature_flags/ for more details. | |
|
||||
| <code class="text-nowrap">--enable-feature</code> <code class="text-nowrap">...<code class="text-nowrap"> | Comma separated feature names to enable. Valid options: exemplar-storage, expand-external-labels, memory-snapshot-on-shutdown, promql-per-step-stats, promql-experimental-functions, extra-scrape-metrics, auto-gomaxprocs, native-histograms, created-timestamp-zero-ingestion, concurrent-rule-eval, delayed-compaction, old-ui, otlp-deltatocumulative, promql-duration-expr. See https://prometheus.io/docs/prometheus/latest/feature_flags/ for more details. | |
|
||||
| <code class="text-nowrap">--agent</code> | Run Prometheus in 'Agent mode'. | |
|
||||
| <code class="text-nowrap">--log.level</code> | Only log messages with the given severity or above. One of: [debug, info, warn, error] | `info` |
|
||||
| <code class="text-nowrap">--log.format</code> | Output format of log messages. One of: [logfmt, json] | `logfmt` |
|
||||
|
|
|
@ -183,4 +183,38 @@ This state is periodically ([`max_stale`][d2c]) cleared of inactive series.
|
|||
Enabling this _can_ have negative impact on performance, because the in-memory
|
||||
state is mutex guarded. Cumulative-only OTLP requests are not affected.
|
||||
|
||||
### PromQL arithmetic expressions in time durations
|
||||
|
||||
`--enable-feature=promql-duration-expr`
|
||||
|
||||
With this flag, arithmetic expressions can be used in time durations in range queries and offset durations. For example:
|
||||
|
||||
In range queries:
|
||||
rate(http_requests_total[5m * 2]) # 10 minute range
|
||||
rate(http_requests_total[(5+2) * 1m]) # 7 minute range
|
||||
|
||||
In offset durations:
|
||||
http_requests_total offset (1h / 2) # 30 minute offset
|
||||
http_requests_total offset ((2 ^ 3) * 1m) # 8 minute offset
|
||||
|
||||
Note: Duration expressions are not supported in the @ timestamp operator.
|
||||
|
||||
The following operators are supported:
|
||||
|
||||
* `+` - addition
|
||||
* `-` - subtraction
|
||||
* `*` - multiplication
|
||||
* `/` - division
|
||||
* `%` - modulo
|
||||
* `^` - exponentiation
|
||||
|
||||
Examples of equivalent durations:
|
||||
|
||||
* `5m * 2` is the equivalent to `10m` or `600s`
|
||||
* `10m - 1m` is the equivalent to `9m` or `540s`
|
||||
* `(5+2) * 1m` is the equivalent to `7m` or `420s`
|
||||
* `1h / 2` is the equivalent to `30m` or `1800s`
|
||||
* `4h % 3h` is the equivalent to `1h` or `3600s`
|
||||
* `(2 ^ 3) * 1m` is the equivalent to `8m` or `480s`
|
||||
|
||||
[d2c]: https://github.com/open-telemetry/opentelemetry-collector-contrib/tree/main/processor/deltatocumulativeprocessor
|
||||
|
|
|
@ -0,0 +1,136 @@
|
|||
// Copyright 2025 The Prometheus Authors
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package promql
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"math"
|
||||
"time"
|
||||
|
||||
"github.com/prometheus/prometheus/promql/parser"
|
||||
)
|
||||
|
||||
// durationVisitor is a visitor that visits a duration expression and calculates the duration.
|
||||
type durationVisitor struct{}
|
||||
|
||||
func (v *durationVisitor) Visit(node parser.Node, _ []parser.Node) (parser.Visitor, error) {
|
||||
switch n := node.(type) {
|
||||
case *parser.VectorSelector:
|
||||
if n.OriginalOffsetExpr != nil {
|
||||
duration, err := calculateDuration(n.OriginalOffsetExpr, true)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
n.OriginalOffset = duration
|
||||
}
|
||||
case *parser.MatrixSelector:
|
||||
if n.RangeExpr != nil {
|
||||
duration, err := calculateDuration(n.RangeExpr, false)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
n.Range = duration
|
||||
}
|
||||
case *parser.SubqueryExpr:
|
||||
if n.OriginalOffsetExpr != nil {
|
||||
duration, err := calculateDuration(n.OriginalOffsetExpr, true)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
n.OriginalOffset = duration
|
||||
}
|
||||
if n.StepExpr != nil {
|
||||
duration, err := calculateDuration(n.StepExpr, false)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
n.Step = duration
|
||||
}
|
||||
if n.RangeExpr != nil {
|
||||
duration, err := calculateDuration(n.RangeExpr, false)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
n.Range = duration
|
||||
}
|
||||
}
|
||||
return v, nil
|
||||
}
|
||||
|
||||
// calculateDuration computes the duration from a duration expression.
|
||||
func calculateDuration(expr parser.Expr, allowedNegative bool) (time.Duration, error) {
|
||||
duration, err := evaluateDurationExpr(expr)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
if duration <= 0 && !allowedNegative {
|
||||
return 0, fmt.Errorf("%d:%d: duration must be greater than 0", expr.PositionRange().Start, expr.PositionRange().End)
|
||||
}
|
||||
if duration > 1<<63-1 || duration < -1<<63 {
|
||||
return 0, fmt.Errorf("%d:%d: duration is out of range", expr.PositionRange().Start, expr.PositionRange().End)
|
||||
}
|
||||
return time.Duration(duration*1000) * time.Millisecond, nil
|
||||
}
|
||||
|
||||
// evaluateDurationExpr recursively evaluates a duration expression to a float64 value.
|
||||
func evaluateDurationExpr(expr parser.Expr) (float64, error) {
|
||||
switch n := expr.(type) {
|
||||
case *parser.NumberLiteral:
|
||||
return n.Val, nil
|
||||
case *parser.DurationExpr:
|
||||
var lhs, rhs float64
|
||||
var err error
|
||||
|
||||
if n.LHS != nil {
|
||||
lhs, err = evaluateDurationExpr(n.LHS)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
}
|
||||
|
||||
rhs, err = evaluateDurationExpr(n.RHS)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
|
||||
switch n.Op {
|
||||
case parser.ADD:
|
||||
return lhs + rhs, nil
|
||||
case parser.SUB:
|
||||
if n.LHS == nil {
|
||||
// Unary negative duration expression.
|
||||
return -rhs, nil
|
||||
}
|
||||
return lhs - rhs, nil
|
||||
case parser.MUL:
|
||||
return lhs * rhs, nil
|
||||
case parser.DIV:
|
||||
if rhs == 0 {
|
||||
return 0, fmt.Errorf("%d:%d: division by zero", expr.PositionRange().Start, expr.PositionRange().End)
|
||||
}
|
||||
return lhs / rhs, nil
|
||||
case parser.MOD:
|
||||
if rhs == 0 {
|
||||
return 0, fmt.Errorf("%d:%d: modulo by zero", expr.PositionRange().Start, expr.PositionRange().End)
|
||||
}
|
||||
return math.Mod(lhs, rhs), nil
|
||||
case parser.POW:
|
||||
return math.Pow(lhs, rhs), nil
|
||||
default:
|
||||
return 0, fmt.Errorf("unexpected duration expression operator %q", n.Op)
|
||||
}
|
||||
default:
|
||||
return 0, fmt.Errorf("unexpected duration expression type %T", n)
|
||||
}
|
||||
}
|
|
@ -0,0 +1,238 @@
|
|||
// Copyright 2025 The Prometheus Authors
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package promql
|
||||
|
||||
import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
"github.com/prometheus/prometheus/promql/parser"
|
||||
)
|
||||
|
||||
func TestDurationVisitor(t *testing.T) {
|
||||
// Enable experimental duration expression parsing.
|
||||
parser.ExperimentalDurationExpr = true
|
||||
t.Cleanup(func() {
|
||||
parser.ExperimentalDurationExpr = false
|
||||
})
|
||||
complexExpr := `sum_over_time(
|
||||
rate(metric[5m] offset 1h)[10m:30s] offset 2h
|
||||
) +
|
||||
avg_over_time(
|
||||
metric[1h + 30m] offset -1h
|
||||
) *
|
||||
count_over_time(
|
||||
metric[2h * 0.5]
|
||||
)`
|
||||
|
||||
expr, err := parser.ParseExpr(complexExpr)
|
||||
require.NoError(t, err)
|
||||
|
||||
err = parser.Walk(&durationVisitor{}, expr, nil)
|
||||
require.NoError(t, err)
|
||||
|
||||
// Verify different parts of the expression have correct durations.
|
||||
// This is a binary expression at the top level.
|
||||
binExpr, ok := expr.(*parser.BinaryExpr)
|
||||
require.True(t, ok, "Expected binary expression at top level")
|
||||
|
||||
// Left side should be sum_over_time with subquery.
|
||||
leftCall, ok := binExpr.LHS.(*parser.Call)
|
||||
require.True(t, ok, "Expected call expression on left side")
|
||||
require.Equal(t, "sum_over_time", leftCall.Func.Name)
|
||||
|
||||
// Extract the subquery from sum_over_time.
|
||||
sumSubquery, ok := leftCall.Args[0].(*parser.SubqueryExpr)
|
||||
require.True(t, ok, "Expected subquery in sum_over_time")
|
||||
require.Equal(t, 10*time.Minute, sumSubquery.Range)
|
||||
require.Equal(t, 30*time.Second, sumSubquery.Step)
|
||||
require.Equal(t, 2*time.Hour, sumSubquery.OriginalOffset)
|
||||
|
||||
// Extract the rate call inside the subquery.
|
||||
rateCall, ok := sumSubquery.Expr.(*parser.Call)
|
||||
require.True(t, ok, "Expected rate call in subquery")
|
||||
require.Equal(t, "rate", rateCall.Func.Name)
|
||||
|
||||
// Extract the matrix selector from rate.
|
||||
rateMatrix, ok := rateCall.Args[0].(*parser.MatrixSelector)
|
||||
require.True(t, ok, "Expected matrix selector in rate")
|
||||
require.Equal(t, 5*time.Minute, rateMatrix.Range)
|
||||
require.Equal(t, 1*time.Hour, rateMatrix.VectorSelector.(*parser.VectorSelector).OriginalOffset)
|
||||
|
||||
// Right side should be another binary expression (multiplication).
|
||||
rightBinExpr, ok := binExpr.RHS.(*parser.BinaryExpr)
|
||||
require.True(t, ok, "Expected binary expression on right side")
|
||||
|
||||
// Left side of multiplication should be avg_over_time.
|
||||
avgCall, ok := rightBinExpr.LHS.(*parser.Call)
|
||||
require.True(t, ok, "Expected call expression on left side of multiplication")
|
||||
require.Equal(t, "avg_over_time", avgCall.Func.Name)
|
||||
|
||||
// Extract the matrix selector from avg_over_time.
|
||||
avgMatrix, ok := avgCall.Args[0].(*parser.MatrixSelector)
|
||||
require.True(t, ok, "Expected matrix selector in avg_over_time")
|
||||
require.Equal(t, 90*time.Minute, avgMatrix.Range) // 1h + 30m
|
||||
require.Equal(t, -1*time.Hour, avgMatrix.VectorSelector.(*parser.VectorSelector).OriginalOffset)
|
||||
|
||||
// Right side of multiplication should be count_over_time.
|
||||
countCall, ok := rightBinExpr.RHS.(*parser.Call)
|
||||
require.True(t, ok, "Expected call expression on right side of multiplication")
|
||||
require.Equal(t, "count_over_time", countCall.Func.Name)
|
||||
|
||||
// Extract the matrix selector from count_over_time.
|
||||
countMatrix, ok := countCall.Args[0].(*parser.MatrixSelector)
|
||||
require.True(t, ok, "Expected matrix selector in count_over_time")
|
||||
require.Equal(t, 1*time.Hour, countMatrix.Range) // 2h * 0.5
|
||||
}
|
||||
|
||||
func TestCalculateDuration(t *testing.T) {
|
||||
tests := []struct {
|
||||
name string
|
||||
expr parser.Expr
|
||||
expected time.Duration
|
||||
errorMessage string
|
||||
allowedNegative bool
|
||||
}{
|
||||
{
|
||||
name: "addition",
|
||||
expr: &parser.DurationExpr{
|
||||
LHS: &parser.NumberLiteral{Val: 5},
|
||||
RHS: &parser.NumberLiteral{Val: 10},
|
||||
Op: parser.ADD,
|
||||
},
|
||||
expected: 15 * time.Second,
|
||||
},
|
||||
{
|
||||
name: "subtraction",
|
||||
expr: &parser.DurationExpr{
|
||||
LHS: &parser.NumberLiteral{Val: 15},
|
||||
RHS: &parser.NumberLiteral{Val: 5},
|
||||
Op: parser.SUB,
|
||||
},
|
||||
expected: 10 * time.Second,
|
||||
},
|
||||
{
|
||||
name: "subtraction with negative",
|
||||
expr: &parser.DurationExpr{
|
||||
LHS: &parser.NumberLiteral{Val: 5},
|
||||
RHS: &parser.NumberLiteral{Val: 10},
|
||||
Op: parser.SUB,
|
||||
},
|
||||
errorMessage: "duration must be greater than 0",
|
||||
},
|
||||
{
|
||||
name: "multiplication",
|
||||
expr: &parser.DurationExpr{
|
||||
LHS: &parser.NumberLiteral{Val: 5},
|
||||
RHS: &parser.NumberLiteral{Val: 3},
|
||||
Op: parser.MUL,
|
||||
},
|
||||
expected: 15 * time.Second,
|
||||
},
|
||||
{
|
||||
name: "division",
|
||||
expr: &parser.DurationExpr{
|
||||
LHS: &parser.NumberLiteral{Val: 15},
|
||||
RHS: &parser.NumberLiteral{Val: 3},
|
||||
Op: parser.DIV,
|
||||
},
|
||||
expected: 5 * time.Second,
|
||||
},
|
||||
{
|
||||
name: "modulo with numbers",
|
||||
expr: &parser.DurationExpr{
|
||||
LHS: &parser.NumberLiteral{Val: 17},
|
||||
RHS: &parser.NumberLiteral{Val: 5},
|
||||
Op: parser.MOD,
|
||||
},
|
||||
expected: 2 * time.Second,
|
||||
},
|
||||
{
|
||||
name: "power",
|
||||
expr: &parser.DurationExpr{
|
||||
LHS: &parser.NumberLiteral{Val: 2},
|
||||
RHS: &parser.NumberLiteral{Val: 3},
|
||||
Op: parser.POW,
|
||||
},
|
||||
expected: 8 * time.Second,
|
||||
},
|
||||
{
|
||||
name: "complex expression",
|
||||
expr: &parser.DurationExpr{
|
||||
LHS: &parser.DurationExpr{
|
||||
LHS: &parser.NumberLiteral{Val: 2},
|
||||
RHS: &parser.DurationExpr{
|
||||
LHS: &parser.NumberLiteral{Val: 3},
|
||||
RHS: &parser.NumberLiteral{Val: 4},
|
||||
Op: parser.ADD,
|
||||
},
|
||||
Op: parser.MUL,
|
||||
},
|
||||
RHS: &parser.NumberLiteral{Val: 1},
|
||||
Op: parser.SUB,
|
||||
},
|
||||
expected: 13 * time.Second,
|
||||
},
|
||||
{
|
||||
name: "unary negative",
|
||||
expr: &parser.DurationExpr{
|
||||
RHS: &parser.NumberLiteral{Val: 5},
|
||||
Op: parser.SUB,
|
||||
},
|
||||
expected: -5 * time.Second,
|
||||
allowedNegative: true,
|
||||
},
|
||||
{
|
||||
name: "division by zero",
|
||||
expr: &parser.DurationExpr{
|
||||
LHS: &parser.NumberLiteral{Val: 5},
|
||||
RHS: &parser.DurationExpr{
|
||||
LHS: &parser.NumberLiteral{Val: 5},
|
||||
RHS: &parser.NumberLiteral{Val: 5},
|
||||
Op: parser.SUB,
|
||||
},
|
||||
Op: parser.DIV,
|
||||
},
|
||||
errorMessage: "division by zero",
|
||||
},
|
||||
{
|
||||
name: "modulo by zero",
|
||||
expr: &parser.DurationExpr{
|
||||
LHS: &parser.NumberLiteral{Val: 5},
|
||||
RHS: &parser.DurationExpr{
|
||||
LHS: &parser.NumberLiteral{Val: 5},
|
||||
RHS: &parser.NumberLiteral{Val: 5},
|
||||
Op: parser.SUB,
|
||||
},
|
||||
Op: parser.MOD,
|
||||
},
|
||||
errorMessage: "modulo by zero",
|
||||
},
|
||||
}
|
||||
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
result, err := calculateDuration(tt.expr, tt.allowedNegative)
|
||||
if tt.errorMessage != "" {
|
||||
require.Error(t, err)
|
||||
require.Contains(t, err.Error(), tt.errorMessage)
|
||||
return
|
||||
}
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, tt.expected, result)
|
||||
})
|
||||
}
|
||||
}
|
|
@ -489,9 +489,9 @@ func (ng *Engine) NewInstantQuery(ctx context.Context, q storage.Queryable, opts
|
|||
if err := ng.validateOpts(expr); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
*pExpr = PreprocessExpr(expr, ts, ts)
|
||||
*pExpr, err = PreprocessExpr(expr, ts, ts)
|
||||
|
||||
return qry, nil
|
||||
return qry, err
|
||||
}
|
||||
|
||||
// NewRangeQuery returns an evaluation query for the given time range and with
|
||||
|
@ -513,9 +513,9 @@ func (ng *Engine) NewRangeQuery(ctx context.Context, q storage.Queryable, opts Q
|
|||
if expr.Type() != parser.ValueTypeVector && expr.Type() != parser.ValueTypeScalar {
|
||||
return nil, fmt.Errorf("invalid expression type %q for range query, must be Scalar or instant Vector", parser.DocumentedType(expr.Type()))
|
||||
}
|
||||
*pExpr = PreprocessExpr(expr, start, end)
|
||||
*pExpr, err = PreprocessExpr(expr, start, end)
|
||||
|
||||
return qry, nil
|
||||
return qry, err
|
||||
}
|
||||
|
||||
func (ng *Engine) newQuery(q storage.Queryable, qs string, opts QueryOpts, start, end time.Time, interval time.Duration) (*parser.Expr, *query) {
|
||||
|
@ -3596,15 +3596,20 @@ func unwrapStepInvariantExpr(e parser.Expr) parser.Expr {
|
|||
}
|
||||
|
||||
// PreprocessExpr wraps all possible step invariant parts of the given expression with
|
||||
// StepInvariantExpr. It also resolves the preprocessors.
|
||||
func PreprocessExpr(expr parser.Expr, start, end time.Time) parser.Expr {
|
||||
// StepInvariantExpr. It also resolves the preprocessors and evaluates duration expressions
|
||||
// into their numeric values.
|
||||
func PreprocessExpr(expr parser.Expr, start, end time.Time) (parser.Expr, error) {
|
||||
detectHistogramStatsDecoding(expr)
|
||||
|
||||
if err := parser.Walk(&durationVisitor{}, expr, nil); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
isStepInvariant := preprocessExprHelper(expr, start, end)
|
||||
if isStepInvariant {
|
||||
return newStepInvariantExpr(expr)
|
||||
return newStepInvariantExpr(expr), nil
|
||||
}
|
||||
return expr
|
||||
return expr, nil
|
||||
}
|
||||
|
||||
// preprocessExprHelper wraps the child nodes of the expression
|
||||
|
|
|
@ -1900,15 +1900,6 @@ func TestSubquerySelector(t *testing.T) {
|
|||
},
|
||||
Start: time.Unix(35, 0),
|
||||
},
|
||||
{
|
||||
Query: "metric[0:10s]",
|
||||
Result: promql.Result{
|
||||
nil,
|
||||
promql.Matrix{},
|
||||
nil,
|
||||
},
|
||||
Start: time.Unix(10, 0),
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
|
@ -3096,7 +3087,8 @@ func TestPreprocessAndWrapWithStepInvariantExpr(t *testing.T) {
|
|||
t.Run(test.input, func(t *testing.T) {
|
||||
expr, err := parser.ParseExpr(test.input)
|
||||
require.NoError(t, err)
|
||||
expr = promql.PreprocessExpr(expr, startTime, endTime)
|
||||
expr, err = promql.PreprocessExpr(expr, startTime, endTime)
|
||||
require.NoError(t, err)
|
||||
if test.outputTest {
|
||||
require.Equal(t, test.input, expr.String(), "error on input '%s'", test.input)
|
||||
}
|
||||
|
@ -3268,11 +3260,6 @@ func TestInstantQueryWithRangeVectorSelector(t *testing.T) {
|
|||
},
|
||||
},
|
||||
},
|
||||
"matches series but range is 0": {
|
||||
expr: "some_metric[0]",
|
||||
ts: baseT.Add(2 * time.Minute),
|
||||
expected: promql.Matrix{},
|
||||
},
|
||||
}
|
||||
|
||||
for name, testCase := range testCases {
|
||||
|
|
|
@ -110,6 +110,15 @@ type BinaryExpr struct {
|
|||
ReturnBool bool
|
||||
}
|
||||
|
||||
// DurationExpr represents a binary expression between two duration expressions.
|
||||
type DurationExpr struct {
|
||||
Op ItemType // The operation of the expression.
|
||||
LHS, RHS Expr // The operands on the respective sides of the operator.
|
||||
Wrapped bool // Set when the duration is wrapped in parentheses.
|
||||
|
||||
StartPos posrange.Pos // For unary operations, the position of the operator.
|
||||
}
|
||||
|
||||
// Call represents a function call.
|
||||
type Call struct {
|
||||
Func *Function // The function that was called.
|
||||
|
@ -124,7 +133,7 @@ type MatrixSelector struct {
|
|||
// if the parser hasn't returned an error.
|
||||
VectorSelector Expr
|
||||
Range time.Duration
|
||||
|
||||
RangeExpr *DurationExpr
|
||||
EndPos posrange.Pos
|
||||
}
|
||||
|
||||
|
@ -132,16 +141,19 @@ type MatrixSelector struct {
|
|||
type SubqueryExpr struct {
|
||||
Expr Expr
|
||||
Range time.Duration
|
||||
RangeExpr *DurationExpr
|
||||
// OriginalOffset is the actual offset that was set in the query.
|
||||
// This never changes.
|
||||
OriginalOffset time.Duration
|
||||
// OriginalOffsetExpr is the actual offset expression that was set in the query.
|
||||
OriginalOffsetExpr *DurationExpr
|
||||
// Offset is the offset used during the query execution
|
||||
// which is calculated using the original offset, at modifier time,
|
||||
// which is calculated using the original offset, offset expression, at modifier time,
|
||||
// eval time, and subquery offsets in the AST tree.
|
||||
Offset time.Duration
|
||||
Timestamp *int64
|
||||
StartOrEnd ItemType // Set when @ is used with start() or end()
|
||||
Step time.Duration
|
||||
StepExpr *DurationExpr
|
||||
|
||||
EndPos posrange.Pos
|
||||
}
|
||||
|
@ -150,6 +162,7 @@ type SubqueryExpr struct {
|
|||
type NumberLiteral struct {
|
||||
Val float64
|
||||
|
||||
Duration bool // Used to format the number as a duration.
|
||||
PosRange posrange.PositionRange
|
||||
}
|
||||
|
||||
|
@ -191,9 +204,10 @@ func (e *StepInvariantExpr) PositionRange() posrange.PositionRange {
|
|||
// VectorSelector represents a Vector selection.
|
||||
type VectorSelector struct {
|
||||
Name string
|
||||
// OriginalOffset is the actual offset that was set in the query.
|
||||
// This never changes.
|
||||
// OriginalOffset is the actual offset calculated from OriginalOffsetExpr.
|
||||
OriginalOffset time.Duration
|
||||
// OriginalOffsetExpr is the actual offset that was set in the query.
|
||||
OriginalOffsetExpr *DurationExpr
|
||||
// Offset is the offset used during the query execution
|
||||
// which is calculated using the original offset, at modifier time,
|
||||
// eval time, and subquery offsets in the AST tree.
|
||||
|
@ -244,6 +258,7 @@ func (e *BinaryExpr) Type() ValueType {
|
|||
return ValueTypeVector
|
||||
}
|
||||
func (e *StepInvariantExpr) Type() ValueType { return e.Expr.Type() }
|
||||
func (e *DurationExpr) Type() ValueType { return ValueTypeScalar }
|
||||
|
||||
func (*AggregateExpr) PromQLExpr() {}
|
||||
func (*BinaryExpr) PromQLExpr() {}
|
||||
|
@ -256,6 +271,7 @@ func (*StringLiteral) PromQLExpr() {}
|
|||
func (*UnaryExpr) PromQLExpr() {}
|
||||
func (*VectorSelector) PromQLExpr() {}
|
||||
func (*StepInvariantExpr) PromQLExpr() {}
|
||||
func (*DurationExpr) PromQLExpr() {}
|
||||
|
||||
// VectorMatchCardinality describes the cardinality relationship
|
||||
// of two Vectors in a binary operation.
|
||||
|
@ -438,6 +454,16 @@ func (e *BinaryExpr) PositionRange() posrange.PositionRange {
|
|||
return mergeRanges(e.LHS, e.RHS)
|
||||
}
|
||||
|
||||
func (e *DurationExpr) PositionRange() posrange.PositionRange {
|
||||
if e.LHS == nil {
|
||||
return posrange.PositionRange{
|
||||
Start: e.StartPos,
|
||||
End: e.RHS.PositionRange().End,
|
||||
}
|
||||
}
|
||||
return mergeRanges(e.LHS, e.RHS)
|
||||
}
|
||||
|
||||
func (e *Call) PositionRange() posrange.PositionRange {
|
||||
return e.PosRange
|
||||
}
|
||||
|
|
|
@ -186,7 +186,7 @@ START_METRIC_SELECTOR
|
|||
%type <int> int
|
||||
%type <uint> uint
|
||||
%type <float> number series_value signed_number signed_or_unsigned_number
|
||||
%type <node> step_invariant_expr aggregate_expr aggregate_modifier bin_modifier binary_expr bool_modifier expr function_call function_call_args function_call_body group_modifiers label_matchers matrix_selector number_duration_literal offset_expr on_or_ignoring paren_expr string_literal subquery_expr unary_expr vector_selector
|
||||
%type <node> step_invariant_expr aggregate_expr aggregate_modifier bin_modifier binary_expr bool_modifier expr function_call function_call_args function_call_body group_modifiers label_matchers matrix_selector number_duration_literal offset_expr on_or_ignoring paren_expr string_literal subquery_expr unary_expr vector_selector duration_expr paren_duration_expr positive_duration_expr
|
||||
|
||||
%start start
|
||||
|
||||
|
@ -235,6 +235,7 @@ expr :
|
|||
| unary_expr
|
||||
| vector_selector
|
||||
| step_invariant_expr
|
||||
| duration_expr
|
||||
;
|
||||
|
||||
/*
|
||||
|
@ -433,23 +434,35 @@ paren_expr : LEFT_PAREN expr RIGHT_PAREN
|
|||
* Offset modifiers.
|
||||
*/
|
||||
|
||||
offset_expr: expr OFFSET number_duration_literal
|
||||
positive_duration_expr : duration_expr
|
||||
{
|
||||
numLit, _ := $3.(*NumberLiteral)
|
||||
dur := time.Duration(numLit.Val * 1000) * time.Millisecond
|
||||
yylex.(*parser).addOffset($1, dur)
|
||||
if numLit, ok := $1.(*NumberLiteral); ok {
|
||||
if numLit.Val <= 0 {
|
||||
yylex.(*parser).addParseErrf(numLit.PositionRange(), "duration must be greater than 0")
|
||||
$$ = &NumberLiteral{Val: 0} // Return 0 on error.
|
||||
break
|
||||
}
|
||||
$$ = $1
|
||||
break
|
||||
}
|
||||
$$ = $1
|
||||
}
|
||||
| expr OFFSET SUB number_duration_literal
|
||||
;
|
||||
|
||||
offset_expr: expr OFFSET duration_expr
|
||||
{
|
||||
numLit, _ := $4.(*NumberLiteral)
|
||||
dur := time.Duration(numLit.Val * 1000) * time.Millisecond
|
||||
yylex.(*parser).addOffset($1, -dur)
|
||||
if numLit, ok := $3.(*NumberLiteral); ok {
|
||||
yylex.(*parser).addOffset($1, time.Duration(numLit.Val*1000)*time.Millisecond)
|
||||
$$ = $1
|
||||
break
|
||||
}
|
||||
yylex.(*parser).addOffsetExpr($1, $3.(*DurationExpr))
|
||||
$$ = $1
|
||||
}
|
||||
| expr OFFSET error
|
||||
{ yylex.(*parser).unexpected("offset", "number or duration"); $$ = $1 }
|
||||
;
|
||||
|
||||
/*
|
||||
* @ modifiers.
|
||||
*/
|
||||
|
@ -474,7 +487,7 @@ at_modifier_preprocessors: START | END;
|
|||
* Subquery and range selectors.
|
||||
*/
|
||||
|
||||
matrix_selector : expr LEFT_BRACKET number_duration_literal RIGHT_BRACKET
|
||||
matrix_selector : expr LEFT_BRACKET positive_duration_expr RIGHT_BRACKET
|
||||
{
|
||||
var errMsg string
|
||||
vs, ok := $1.(*VectorSelector)
|
||||
|
@ -491,41 +504,60 @@ matrix_selector : expr LEFT_BRACKET number_duration_literal RIGHT_BRACKET
|
|||
yylex.(*parser).addParseErrf(errRange, "%s", errMsg)
|
||||
}
|
||||
|
||||
numLit, _ := $3.(*NumberLiteral)
|
||||
var rangeNl time.Duration
|
||||
if numLit, ok := $3.(*NumberLiteral); ok {
|
||||
rangeNl = time.Duration(numLit.Val*1000)*time.Millisecond
|
||||
}
|
||||
rangeExpr, _ := $3.(*DurationExpr)
|
||||
$$ = &MatrixSelector{
|
||||
VectorSelector: $1.(Expr),
|
||||
Range: time.Duration(numLit.Val * 1000) * time.Millisecond,
|
||||
Range: rangeNl,
|
||||
RangeExpr: rangeExpr,
|
||||
EndPos: yylex.(*parser).lastClosing,
|
||||
}
|
||||
}
|
||||
;
|
||||
|
||||
subquery_expr : expr LEFT_BRACKET number_duration_literal COLON number_duration_literal RIGHT_BRACKET
|
||||
subquery_expr : expr LEFT_BRACKET positive_duration_expr COLON positive_duration_expr RIGHT_BRACKET
|
||||
{
|
||||
numLitRange, _ := $3.(*NumberLiteral)
|
||||
numLitStep, _ := $5.(*NumberLiteral)
|
||||
var rangeNl time.Duration
|
||||
var stepNl time.Duration
|
||||
if numLit, ok := $3.(*NumberLiteral); ok {
|
||||
rangeNl = time.Duration(numLit.Val*1000)*time.Millisecond
|
||||
}
|
||||
rangeExpr, _ := $3.(*DurationExpr)
|
||||
if numLit, ok := $5.(*NumberLiteral); ok {
|
||||
stepNl = time.Duration(numLit.Val*1000)*time.Millisecond
|
||||
}
|
||||
stepExpr, _ := $5.(*DurationExpr)
|
||||
$$ = &SubqueryExpr{
|
||||
Expr: $1.(Expr),
|
||||
Range: time.Duration(numLitRange.Val * 1000) * time.Millisecond,
|
||||
Step: time.Duration(numLitStep.Val * 1000) * time.Millisecond,
|
||||
Range: rangeNl,
|
||||
RangeExpr: rangeExpr,
|
||||
Step: stepNl,
|
||||
StepExpr: stepExpr,
|
||||
EndPos: $6.Pos + 1,
|
||||
}
|
||||
}
|
||||
| expr LEFT_BRACKET number_duration_literal COLON RIGHT_BRACKET
|
||||
| expr LEFT_BRACKET positive_duration_expr COLON RIGHT_BRACKET
|
||||
{
|
||||
numLitRange, _ := $3.(*NumberLiteral)
|
||||
var rangeNl time.Duration
|
||||
if numLit, ok := $3.(*NumberLiteral); ok {
|
||||
rangeNl = time.Duration(numLit.Val*1000)*time.Millisecond
|
||||
}
|
||||
rangeExpr, _ := $3.(*DurationExpr)
|
||||
$$ = &SubqueryExpr{
|
||||
Expr: $1.(Expr),
|
||||
Range: time.Duration(numLitRange.Val * 1000) * time.Millisecond,
|
||||
Step: 0,
|
||||
Range: rangeNl,
|
||||
RangeExpr: rangeExpr,
|
||||
EndPos: $5.Pos + 1,
|
||||
}
|
||||
}
|
||||
| expr LEFT_BRACKET number_duration_literal COLON number_duration_literal error
|
||||
| expr LEFT_BRACKET positive_duration_expr COLON positive_duration_expr error
|
||||
{ yylex.(*parser).unexpected("subquery selector", "\"]\""); $$ = $1 }
|
||||
| expr LEFT_BRACKET number_duration_literal COLON error
|
||||
| expr LEFT_BRACKET positive_duration_expr COLON error
|
||||
{ yylex.(*parser).unexpected("subquery selector", "number or duration or \"]\""); $$ = $1 }
|
||||
| expr LEFT_BRACKET number_duration_literal error
|
||||
| expr LEFT_BRACKET positive_duration_expr error
|
||||
{ yylex.(*parser).unexpected("subquery or range", "\":\" or \"]\""); $$ = $1 }
|
||||
| expr LEFT_BRACKET error
|
||||
{ yylex.(*parser).unexpected("subquery selector", "number or duration"); $$ = $1 }
|
||||
|
@ -930,6 +962,7 @@ number_duration_literal : NUMBER
|
|||
$$ = &NumberLiteral{
|
||||
Val: dur.Seconds(),
|
||||
PosRange: $1.PositionRange(),
|
||||
Duration: true,
|
||||
}
|
||||
}
|
||||
;
|
||||
|
@ -997,4 +1030,105 @@ maybe_grouping_labels: /* empty */ { $$ = nil }
|
|||
| grouping_labels
|
||||
;
|
||||
|
||||
/*
|
||||
* Duration expressions.
|
||||
*/
|
||||
|
||||
duration_expr : number_duration_literal
|
||||
{
|
||||
nl := $1.(*NumberLiteral)
|
||||
if nl.Val > 1<<63/1e9 || nl.Val < -(1<<63)/1e9 {
|
||||
yylex.(*parser).addParseErrf(nl.PosRange, "duration out of range")
|
||||
$$ = &NumberLiteral{Val: 0}
|
||||
break
|
||||
}
|
||||
$$ = nl
|
||||
}
|
||||
| unary_op duration_expr %prec MUL
|
||||
{
|
||||
switch expr := $2.(type) {
|
||||
case *NumberLiteral:
|
||||
if $1.Typ == SUB {
|
||||
expr.Val *= -1
|
||||
}
|
||||
if expr.Val > 1<<63/1e9 || expr.Val < -(1<<63)/1e9 {
|
||||
yylex.(*parser).addParseErrf($1.PositionRange(), "duration out of range")
|
||||
$$ = &NumberLiteral{Val: 0}
|
||||
break
|
||||
}
|
||||
expr.PosRange.Start = $1.Pos
|
||||
$$ = expr
|
||||
break
|
||||
case *DurationExpr:
|
||||
if $1.Typ == SUB {
|
||||
$$ = &DurationExpr{
|
||||
Op: SUB,
|
||||
RHS: expr,
|
||||
StartPos: $1.Pos,
|
||||
}
|
||||
break
|
||||
}
|
||||
$$ = expr
|
||||
break
|
||||
default:
|
||||
yylex.(*parser).addParseErrf($1.PositionRange(), "expected number literal or duration expression")
|
||||
$$ = &NumberLiteral{Val: 0}
|
||||
break
|
||||
}
|
||||
}
|
||||
| duration_expr ADD duration_expr
|
||||
{
|
||||
yylex.(*parser).experimentalDurationExpr($1.(Expr))
|
||||
$$ = &DurationExpr{Op: ADD, LHS: $1.(Expr), RHS: $3.(Expr)}
|
||||
}
|
||||
| duration_expr SUB duration_expr
|
||||
{
|
||||
yylex.(*parser).experimentalDurationExpr($1.(Expr))
|
||||
$$ = &DurationExpr{Op: SUB, LHS: $1.(Expr), RHS: $3.(Expr)}
|
||||
}
|
||||
| duration_expr MUL duration_expr
|
||||
{
|
||||
yylex.(*parser).experimentalDurationExpr($1.(Expr))
|
||||
$$ = &DurationExpr{Op: MUL, LHS: $1.(Expr), RHS: $3.(Expr)}
|
||||
}
|
||||
| duration_expr DIV duration_expr
|
||||
{
|
||||
yylex.(*parser).experimentalDurationExpr($1.(Expr))
|
||||
if nl, ok := $3.(*NumberLiteral); ok && nl.Val == 0 {
|
||||
yylex.(*parser).addParseErrf($2.PositionRange(), "division by zero")
|
||||
$$ = &NumberLiteral{Val: 0}
|
||||
break
|
||||
}
|
||||
$$ = &DurationExpr{Op: DIV, LHS: $1.(Expr), RHS: $3.(Expr)}
|
||||
}
|
||||
| duration_expr MOD duration_expr
|
||||
{
|
||||
yylex.(*parser).experimentalDurationExpr($1.(Expr))
|
||||
if nl, ok := $3.(*NumberLiteral); ok && nl.Val == 0 {
|
||||
yylex.(*parser).addParseErrf($2.PositionRange(), "modulo by zero")
|
||||
$$ = &NumberLiteral{Val: 0}
|
||||
break
|
||||
}
|
||||
$$ = &DurationExpr{Op: MOD, LHS: $1.(Expr), RHS: $3.(Expr)}
|
||||
}
|
||||
| duration_expr POW duration_expr
|
||||
{
|
||||
yylex.(*parser).experimentalDurationExpr($1.(Expr))
|
||||
$$ = &DurationExpr{Op: POW, LHS: $1.(Expr), RHS: $3.(Expr)}
|
||||
}
|
||||
| paren_duration_expr
|
||||
;
|
||||
|
||||
paren_duration_expr : LEFT_PAREN duration_expr RIGHT_PAREN
|
||||
{
|
||||
yylex.(*parser).experimentalDurationExpr($2.(Expr))
|
||||
if durationExpr, ok := $2.(*DurationExpr); ok {
|
||||
durationExpr.Wrapped = true
|
||||
$$ = durationExpr
|
||||
break
|
||||
}
|
||||
$$ = $2
|
||||
}
|
||||
;
|
||||
|
||||
%%
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -277,6 +277,7 @@ type Lexer struct {
|
|||
braceOpen bool // Whether a { is opened.
|
||||
bracketOpen bool // Whether a [ is opened.
|
||||
gotColon bool // Whether we got a ':' after [ was opened.
|
||||
gotDuration bool // Whether we got a duration after [ was opened.
|
||||
stringOpen rune // Quote rune of the string currently being read.
|
||||
|
||||
// series description variables for internal PromQL testing framework as well as in promtool rules unit tests.
|
||||
|
@ -491,7 +492,7 @@ func lexStatements(l *Lexer) stateFn {
|
|||
skipSpaces(l)
|
||||
}
|
||||
l.bracketOpen = true
|
||||
return lexNumberOrDuration
|
||||
return lexDurationExpr
|
||||
case r == ']':
|
||||
if !l.bracketOpen {
|
||||
return l.errorf("unexpected right bracket %q", r)
|
||||
|
@ -549,6 +550,8 @@ func lexHistogram(l *Lexer) stateFn {
|
|||
return lexNumber
|
||||
case r == '[':
|
||||
l.bracketOpen = true
|
||||
l.gotColon = false
|
||||
l.gotDuration = false
|
||||
l.emit(LEFT_BRACKET)
|
||||
return lexBuckets
|
||||
case r == '}' && l.peek() == '}':
|
||||
|
@ -1077,3 +1080,64 @@ func isDigit(r rune) bool {
|
|||
func isAlpha(r rune) bool {
|
||||
return r == '_' || ('a' <= r && r <= 'z') || ('A' <= r && r <= 'Z')
|
||||
}
|
||||
|
||||
// lexDurationExpr scans arithmetic expressions within brackets for duration expressions.
|
||||
func lexDurationExpr(l *Lexer) stateFn {
|
||||
switch r := l.next(); {
|
||||
case r == eof:
|
||||
return l.errorf("unexpected end of input in duration expression")
|
||||
case r == ']':
|
||||
l.emit(RIGHT_BRACKET)
|
||||
l.bracketOpen = false
|
||||
l.gotColon = false
|
||||
return lexStatements
|
||||
case r == ':':
|
||||
l.emit(COLON)
|
||||
if !l.gotDuration {
|
||||
return l.errorf("unexpected colon before duration in duration expression")
|
||||
}
|
||||
if l.gotColon {
|
||||
return l.errorf("unexpected repeated colon in duration expression")
|
||||
}
|
||||
l.gotColon = true
|
||||
return lexDurationExpr
|
||||
case r == '(':
|
||||
l.emit(LEFT_PAREN)
|
||||
l.parenDepth++
|
||||
return lexDurationExpr
|
||||
case r == ')':
|
||||
l.emit(RIGHT_PAREN)
|
||||
l.parenDepth--
|
||||
if l.parenDepth < 0 {
|
||||
return l.errorf("unexpected right parenthesis %q", r)
|
||||
}
|
||||
return lexDurationExpr
|
||||
case isSpace(r):
|
||||
skipSpaces(l)
|
||||
return lexDurationExpr
|
||||
case r == '+':
|
||||
l.emit(ADD)
|
||||
return lexDurationExpr
|
||||
case r == '-':
|
||||
l.emit(SUB)
|
||||
return lexDurationExpr
|
||||
case r == '*':
|
||||
l.emit(MUL)
|
||||
return lexDurationExpr
|
||||
case r == '/':
|
||||
l.emit(DIV)
|
||||
return lexDurationExpr
|
||||
case r == '%':
|
||||
l.emit(MOD)
|
||||
return lexDurationExpr
|
||||
case r == '^':
|
||||
l.emit(POW)
|
||||
return lexDurationExpr
|
||||
case isDigit(r) || (r == '.' && isDigit(l.peek())):
|
||||
l.backup()
|
||||
l.gotDuration = true
|
||||
return lexNumberOrDuration
|
||||
default:
|
||||
return l.errorf("unexpected character in duration expression: %q", r)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -951,6 +951,10 @@ var tests = []struct {
|
|||
input: `test:name{on!~"bar"}[:4s]`,
|
||||
fail: true,
|
||||
},
|
||||
{
|
||||
input: `test:name{on!~"bar"}[1s:1s:1s]`,
|
||||
fail: true,
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
|
|
@ -39,6 +39,9 @@ var parserPool = sync.Pool{
|
|||
},
|
||||
}
|
||||
|
||||
// ExperimentalDurationExpr is a flag to enable experimental duration expression parsing.
|
||||
var ExperimentalDurationExpr bool
|
||||
|
||||
type Parser interface {
|
||||
ParseExpr() (Expr, error)
|
||||
Close()
|
||||
|
@ -881,9 +884,6 @@ func parseDuration(ds string) (time.Duration, error) {
|
|||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
if dur == 0 {
|
||||
return 0, errors.New("duration must be greater than 0")
|
||||
}
|
||||
return time.Duration(dur), nil
|
||||
}
|
||||
|
||||
|
@ -939,11 +939,13 @@ func (p *parser) newMetricNameMatcher(value Item) *labels.Matcher {
|
|||
// addOffset is used to set the offset in the generated parser.
|
||||
func (p *parser) addOffset(e Node, offset time.Duration) {
|
||||
var orgoffsetp *time.Duration
|
||||
var orgoffsetexprp *DurationExpr
|
||||
var endPosp *posrange.Pos
|
||||
|
||||
switch s := e.(type) {
|
||||
case *VectorSelector:
|
||||
orgoffsetp = &s.OriginalOffset
|
||||
orgoffsetexprp = s.OriginalOffsetExpr
|
||||
endPosp = &s.PosRange.End
|
||||
case *MatrixSelector:
|
||||
vs, ok := s.VectorSelector.(*VectorSelector)
|
||||
|
@ -952,9 +954,11 @@ func (p *parser) addOffset(e Node, offset time.Duration) {
|
|||
return
|
||||
}
|
||||
orgoffsetp = &vs.OriginalOffset
|
||||
orgoffsetexprp = vs.OriginalOffsetExpr
|
||||
endPosp = &s.EndPos
|
||||
case *SubqueryExpr:
|
||||
orgoffsetp = &s.OriginalOffset
|
||||
orgoffsetexprp = s.OriginalOffsetExpr
|
||||
endPosp = &s.EndPos
|
||||
default:
|
||||
p.addParseErrf(e.PositionRange(), "offset modifier must be preceded by an instant vector selector or range vector selector or a subquery")
|
||||
|
@ -963,7 +967,7 @@ func (p *parser) addOffset(e Node, offset time.Duration) {
|
|||
|
||||
// it is already ensured by parseDuration func that there never will be a zero offset modifier
|
||||
switch {
|
||||
case *orgoffsetp != 0:
|
||||
case *orgoffsetp != 0 || orgoffsetexprp != nil:
|
||||
p.addParseErrf(e.PositionRange(), "offset may not be set multiple times")
|
||||
case orgoffsetp != nil:
|
||||
*orgoffsetp = offset
|
||||
|
@ -972,6 +976,45 @@ func (p *parser) addOffset(e Node, offset time.Duration) {
|
|||
*endPosp = p.lastClosing
|
||||
}
|
||||
|
||||
// addOffsetExpr is used to set the offset expression in the generated parser.
|
||||
func (p *parser) addOffsetExpr(e Node, expr *DurationExpr) {
|
||||
var orgoffsetp *time.Duration
|
||||
var orgoffsetexprp **DurationExpr
|
||||
var endPosp *posrange.Pos
|
||||
|
||||
switch s := e.(type) {
|
||||
case *VectorSelector:
|
||||
orgoffsetp = &s.OriginalOffset
|
||||
orgoffsetexprp = &s.OriginalOffsetExpr
|
||||
endPosp = &s.PosRange.End
|
||||
case *MatrixSelector:
|
||||
vs, ok := s.VectorSelector.(*VectorSelector)
|
||||
if !ok {
|
||||
p.addParseErrf(e.PositionRange(), "ranges only allowed for vector selectors")
|
||||
return
|
||||
}
|
||||
orgoffsetp = &vs.OriginalOffset
|
||||
orgoffsetexprp = &vs.OriginalOffsetExpr
|
||||
endPosp = &s.EndPos
|
||||
case *SubqueryExpr:
|
||||
orgoffsetp = &s.OriginalOffset
|
||||
orgoffsetexprp = &s.OriginalOffsetExpr
|
||||
endPosp = &s.EndPos
|
||||
default:
|
||||
p.addParseErrf(e.PositionRange(), "offset modifier must be preceded by an instant vector selector or range vector selector or a subquery")
|
||||
return
|
||||
}
|
||||
|
||||
switch {
|
||||
case *orgoffsetp != 0 || *orgoffsetexprp != nil:
|
||||
p.addParseErrf(e.PositionRange(), "offset may not be set multiple times")
|
||||
case orgoffsetexprp != nil:
|
||||
*orgoffsetexprp = expr
|
||||
}
|
||||
|
||||
*endPosp = p.lastClosing
|
||||
}
|
||||
|
||||
// setTimestamp is used to set the timestamp from the @ modifier in the generated parser.
|
||||
func (p *parser) setTimestamp(e Node, ts float64) {
|
||||
if math.IsInf(ts, -1) || math.IsInf(ts, 1) || math.IsNaN(ts) ||
|
||||
|
@ -1045,6 +1088,12 @@ func (p *parser) getAtModifierVars(e Node) (**int64, *ItemType, *posrange.Pos, b
|
|||
return timestampp, preprocp, endPosp, true
|
||||
}
|
||||
|
||||
func (p *parser) experimentalDurationExpr(e Expr) {
|
||||
if !ExperimentalDurationExpr {
|
||||
p.addParseErrf(e.PositionRange(), "experimental duration expression is not enabled")
|
||||
}
|
||||
}
|
||||
|
||||
func MustLabelMatcher(mt labels.MatchType, name, val string) *labels.Matcher {
|
||||
m, err := labels.NewMatcher(mt, name, val)
|
||||
if err != nil {
|
||||
|
|
|
@ -2337,12 +2337,12 @@ var testExpr = []struct {
|
|||
{
|
||||
input: `foo[]`,
|
||||
fail: true,
|
||||
errMsg: "bad number or duration syntax: \"\"",
|
||||
errMsg: "unexpected \"]\" in subquery selector, expected number or duration",
|
||||
},
|
||||
{
|
||||
input: `foo[-1]`,
|
||||
fail: true,
|
||||
errMsg: "bad number or duration syntax: \"\"",
|
||||
errMsg: "duration must be greater than 0",
|
||||
},
|
||||
{
|
||||
input: `some_metric[5m] OFFSET 1mm`,
|
||||
|
@ -3091,7 +3091,7 @@ var testExpr = []struct {
|
|||
{
|
||||
input: `foo{bar="baz"}[`,
|
||||
fail: true,
|
||||
errMsg: `1:16: parse error: bad number or duration syntax: ""`,
|
||||
errMsg: `unexpected end of input in duration expression`,
|
||||
},
|
||||
{
|
||||
input: `foo{bar="baz"}[10m:6s]`,
|
||||
|
@ -3946,6 +3946,304 @@ var testExpr = []struct {
|
|||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
input: `foo[11s+10s-5*2^2]`,
|
||||
expected: &MatrixSelector{
|
||||
VectorSelector: &VectorSelector{
|
||||
Name: "foo",
|
||||
LabelMatchers: []*labels.Matcher{
|
||||
MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"),
|
||||
},
|
||||
PosRange: posrange.PositionRange{
|
||||
Start: 0,
|
||||
End: 3,
|
||||
},
|
||||
},
|
||||
RangeExpr: &DurationExpr{
|
||||
Op: SUB,
|
||||
LHS: &DurationExpr{
|
||||
Op: ADD,
|
||||
LHS: &NumberLiteral{
|
||||
Val: 11,
|
||||
PosRange: posrange.PositionRange{
|
||||
Start: 4,
|
||||
End: 7,
|
||||
},
|
||||
Duration: true,
|
||||
},
|
||||
RHS: &NumberLiteral{
|
||||
Val: 10,
|
||||
PosRange: posrange.PositionRange{
|
||||
Start: 8,
|
||||
End: 11,
|
||||
},
|
||||
Duration: true,
|
||||
},
|
||||
},
|
||||
RHS: &DurationExpr{
|
||||
Op: MUL,
|
||||
LHS: &NumberLiteral{Val: 5, PosRange: posrange.PositionRange{Start: 12, End: 13}},
|
||||
RHS: &DurationExpr{
|
||||
Op: POW,
|
||||
LHS: &NumberLiteral{Val: 2, PosRange: posrange.PositionRange{Start: 14, End: 15}},
|
||||
RHS: &NumberLiteral{Val: 2, PosRange: posrange.PositionRange{Start: 16, End: 17}},
|
||||
},
|
||||
},
|
||||
},
|
||||
EndPos: 18,
|
||||
},
|
||||
},
|
||||
{
|
||||
input: `foo[-(10s-5s)+20s]`,
|
||||
expected: &MatrixSelector{
|
||||
VectorSelector: &VectorSelector{
|
||||
Name: "foo",
|
||||
LabelMatchers: []*labels.Matcher{
|
||||
MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"),
|
||||
},
|
||||
PosRange: posrange.PositionRange{
|
||||
Start: 0,
|
||||
End: 3,
|
||||
},
|
||||
},
|
||||
RangeExpr: &DurationExpr{
|
||||
Op: ADD,
|
||||
LHS: &DurationExpr{
|
||||
Op: SUB,
|
||||
StartPos: 4,
|
||||
RHS: &DurationExpr{
|
||||
Op: SUB,
|
||||
LHS: &NumberLiteral{
|
||||
Val: 10,
|
||||
PosRange: posrange.PositionRange{
|
||||
Start: 6,
|
||||
End: 9,
|
||||
},
|
||||
Duration: true,
|
||||
},
|
||||
RHS: &NumberLiteral{
|
||||
Val: 5,
|
||||
PosRange: posrange.PositionRange{
|
||||
Start: 10,
|
||||
End: 12,
|
||||
},
|
||||
Duration: true,
|
||||
},
|
||||
Wrapped: true,
|
||||
},
|
||||
},
|
||||
RHS: &NumberLiteral{
|
||||
Val: 20,
|
||||
PosRange: posrange.PositionRange{
|
||||
Start: 14,
|
||||
End: 17,
|
||||
},
|
||||
Duration: true,
|
||||
},
|
||||
},
|
||||
EndPos: 18,
|
||||
},
|
||||
},
|
||||
{
|
||||
input: `foo[-10s+15s]`,
|
||||
expected: &MatrixSelector{
|
||||
VectorSelector: &VectorSelector{
|
||||
Name: "foo",
|
||||
LabelMatchers: []*labels.Matcher{
|
||||
MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"),
|
||||
},
|
||||
PosRange: posrange.PositionRange{
|
||||
Start: 0,
|
||||
End: 3,
|
||||
},
|
||||
},
|
||||
RangeExpr: &DurationExpr{
|
||||
Op: ADD,
|
||||
LHS: &NumberLiteral{
|
||||
Val: -10,
|
||||
PosRange: posrange.PositionRange{
|
||||
Start: 4,
|
||||
End: 8,
|
||||
},
|
||||
Duration: true,
|
||||
},
|
||||
RHS: &NumberLiteral{
|
||||
Val: 15,
|
||||
PosRange: posrange.PositionRange{
|
||||
Start: 9,
|
||||
End: 12,
|
||||
},
|
||||
Duration: true,
|
||||
},
|
||||
},
|
||||
EndPos: 13,
|
||||
},
|
||||
},
|
||||
{
|
||||
input: `foo[4s+4s:1s*2] offset (5s-8)`,
|
||||
expected: &SubqueryExpr{
|
||||
Expr: &VectorSelector{
|
||||
Name: "foo",
|
||||
LabelMatchers: []*labels.Matcher{
|
||||
MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"),
|
||||
},
|
||||
PosRange: posrange.PositionRange{
|
||||
Start: 0,
|
||||
End: 3,
|
||||
},
|
||||
},
|
||||
RangeExpr: &DurationExpr{
|
||||
Op: ADD,
|
||||
LHS: &NumberLiteral{
|
||||
Val: 4,
|
||||
PosRange: posrange.PositionRange{
|
||||
Start: 4,
|
||||
End: 6,
|
||||
},
|
||||
Duration: true,
|
||||
},
|
||||
RHS: &NumberLiteral{
|
||||
Val: 4,
|
||||
PosRange: posrange.PositionRange{
|
||||
Start: 7,
|
||||
End: 9,
|
||||
},
|
||||
Duration: true,
|
||||
},
|
||||
},
|
||||
StepExpr: &DurationExpr{
|
||||
Op: MUL,
|
||||
LHS: &NumberLiteral{
|
||||
Val: 1,
|
||||
PosRange: posrange.PositionRange{
|
||||
Start: 10,
|
||||
End: 12,
|
||||
},
|
||||
Duration: true,
|
||||
},
|
||||
RHS: &NumberLiteral{
|
||||
Val: 2,
|
||||
PosRange: posrange.PositionRange{
|
||||
Start: 13,
|
||||
End: 14,
|
||||
},
|
||||
},
|
||||
},
|
||||
OriginalOffsetExpr: &DurationExpr{
|
||||
Op: SUB,
|
||||
LHS: &NumberLiteral{
|
||||
Val: 5,
|
||||
PosRange: posrange.PositionRange{
|
||||
Start: 24,
|
||||
End: 26,
|
||||
},
|
||||
Duration: true,
|
||||
},
|
||||
RHS: &NumberLiteral{
|
||||
Val: 8,
|
||||
PosRange: posrange.PositionRange{
|
||||
Start: 27,
|
||||
End: 28,
|
||||
},
|
||||
},
|
||||
Wrapped: true,
|
||||
},
|
||||
EndPos: 29,
|
||||
},
|
||||
},
|
||||
{
|
||||
input: `foo offset 5s-8`,
|
||||
expected: &BinaryExpr{
|
||||
Op: SUB,
|
||||
LHS: &VectorSelector{
|
||||
Name: "foo",
|
||||
OriginalOffset: 5 * time.Second,
|
||||
LabelMatchers: []*labels.Matcher{
|
||||
MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"),
|
||||
},
|
||||
PosRange: posrange.PositionRange{
|
||||
Start: 0,
|
||||
End: 13,
|
||||
},
|
||||
},
|
||||
RHS: &NumberLiteral{
|
||||
Val: 8,
|
||||
PosRange: posrange.PositionRange{
|
||||
Start: 14,
|
||||
End: 15,
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
input: `rate(foo[2m+2m])`,
|
||||
expected: &Call{
|
||||
Func: MustGetFunction("rate"),
|
||||
Args: Expressions{
|
||||
&MatrixSelector{
|
||||
VectorSelector: &VectorSelector{
|
||||
Name: "foo",
|
||||
LabelMatchers: []*labels.Matcher{
|
||||
MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"),
|
||||
},
|
||||
PosRange: posrange.PositionRange{
|
||||
Start: 5,
|
||||
End: 8,
|
||||
},
|
||||
},
|
||||
RangeExpr: &DurationExpr{
|
||||
Op: ADD,
|
||||
LHS: &NumberLiteral{
|
||||
Val: 120,
|
||||
PosRange: posrange.PositionRange{
|
||||
Start: 9,
|
||||
End: 11,
|
||||
},
|
||||
Duration: true,
|
||||
},
|
||||
RHS: &NumberLiteral{
|
||||
Val: 120,
|
||||
PosRange: posrange.PositionRange{
|
||||
Start: 12,
|
||||
End: 14,
|
||||
},
|
||||
Duration: true,
|
||||
},
|
||||
},
|
||||
EndPos: 15,
|
||||
},
|
||||
},
|
||||
PosRange: posrange.PositionRange{
|
||||
Start: 0,
|
||||
End: 16,
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
input: `foo[5s/0d]`,
|
||||
fail: true,
|
||||
errMsg: `division by zero`,
|
||||
},
|
||||
{
|
||||
input: `foo offset (4d/0)`,
|
||||
fail: true,
|
||||
errMsg: `division by zero`,
|
||||
},
|
||||
{
|
||||
input: `foo[5s%0d]`,
|
||||
fail: true,
|
||||
errMsg: `modulo by zero`,
|
||||
},
|
||||
{
|
||||
input: `foo offset 9.5e10`,
|
||||
fail: true,
|
||||
errMsg: `duration out of range`,
|
||||
},
|
||||
{
|
||||
input: `foo[9.5e10]`,
|
||||
fail: true,
|
||||
errMsg: `duration out of range`,
|
||||
},
|
||||
}
|
||||
|
||||
func makeInt64Pointer(val int64) *int64 {
|
||||
|
@ -3965,8 +4263,11 @@ func readable(s string) string {
|
|||
func TestParseExpressions(t *testing.T) {
|
||||
// Enable experimental functions testing.
|
||||
EnableExperimentalFunctions = true
|
||||
// Enable experimental duration expression parsing.
|
||||
ExperimentalDurationExpr = true
|
||||
t.Cleanup(func() {
|
||||
EnableExperimentalFunctions = false
|
||||
ExperimentalDurationExpr = false
|
||||
})
|
||||
|
||||
for _, test := range testExpr {
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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))
|
||||
})
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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):
|
||||
|
|
|
@ -117,8 +117,12 @@ func RunBuiltinTests(t TBRun, engine promql.QueryEngine) {
|
|||
|
||||
// RunBuiltinTestsWithStorage runs an acceptance test suite against the provided engine and storage.
|
||||
func RunBuiltinTestsWithStorage(t TBRun, engine promql.QueryEngine, newStorage func(testutil.T) storage.Storage) {
|
||||
t.Cleanup(func() { parser.EnableExperimentalFunctions = false })
|
||||
t.Cleanup(func() {
|
||||
parser.EnableExperimentalFunctions = false
|
||||
parser.ExperimentalDurationExpr = false
|
||||
})
|
||||
parser.EnableExperimentalFunctions = true
|
||||
parser.ExperimentalDurationExpr = true
|
||||
|
||||
files, err := fs.Glob(testsFs, "*/*.test")
|
||||
require.NoError(t, err)
|
||||
|
|
|
@ -0,0 +1,121 @@
|
|||
# Test for different duration expression formats in range selectors.
|
||||
# This tests the parser's ability to handle various duration expression.
|
||||
|
||||
# Set up a basic counter that increases steadily.
|
||||
load 5m
|
||||
http_requests{path="/foo"} 1 2 3 0 1 0 0 1 2 0
|
||||
http_requests{path="/bar"} 1 2 3 4 5 1 2 3 4 5
|
||||
http_requests{path="/biz"} 0 0 0 0 0 1 1 1 1 1
|
||||
|
||||
# Test basic duration with unit: [30m]
|
||||
eval instant at 50m changes(http_requests[30m])
|
||||
{path="/foo"} 3
|
||||
{path="/bar"} 4
|
||||
{path="/biz"} 0
|
||||
|
||||
# Test addition in duration: [26m+4m]
|
||||
eval instant at 50m changes(http_requests[26m+4m])
|
||||
{path="/foo"} 3
|
||||
{path="/bar"} 4
|
||||
{path="/biz"} 0
|
||||
|
||||
# Test addition with 0 in duration: [30m+0s]
|
||||
eval instant at 50m changes(http_requests[30m+0s])
|
||||
{path="/foo"} 3
|
||||
{path="/bar"} 4
|
||||
{path="/biz"} 0
|
||||
|
||||
# Test raw seconds: [1800]
|
||||
eval instant at 50m changes(http_requests[1800])
|
||||
{path="/foo"} 3
|
||||
{path="/bar"} 4
|
||||
{path="/biz"} 0
|
||||
|
||||
# Test seconds with multiplication: [60*30]
|
||||
eval instant at 50m changes(http_requests[60*30])
|
||||
{path="/foo"} 3
|
||||
{path="/bar"} 4
|
||||
{path="/biz"} 0
|
||||
|
||||
# Test minutes with multiplication: [2m*15]
|
||||
eval instant at 50m changes(http_requests[2m*15])
|
||||
{path="/foo"} 3
|
||||
{path="/bar"} 4
|
||||
{path="/biz"} 0
|
||||
|
||||
# Test complex expression with parentheses: [2m*(10+5)]
|
||||
eval instant at 50m changes(http_requests[2m*(10+5)])
|
||||
{path="/foo"} 3
|
||||
{path="/bar"} 4
|
||||
{path="/biz"} 0
|
||||
|
||||
# Test mixed units: [29m+60s]
|
||||
eval instant at 50m changes(http_requests[29m+60s])
|
||||
{path="/foo"} 3
|
||||
{path="/bar"} 4
|
||||
{path="/biz"} 0
|
||||
|
||||
# Test nested parentheses: [24m+((1.5*2m)+2m)]
|
||||
eval instant at 50m changes(http_requests[24m+((1.5*2m)+2m)])
|
||||
{path="/foo"} 3
|
||||
{path="/bar"} 4
|
||||
{path="/biz"} 0
|
||||
|
||||
# Test start with -: [-5m+35m]
|
||||
eval instant at 50m changes(http_requests[-5m+35m])
|
||||
{path="/foo"} 3
|
||||
{path="/bar"} 4
|
||||
{path="/biz"} 0
|
||||
|
||||
# Test division: [1h/2]
|
||||
eval instant at 50m changes(http_requests[1h/2])
|
||||
{path="/foo"} 3
|
||||
{path="/bar"} 4
|
||||
{path="/biz"} 0
|
||||
|
||||
# Test modulo: [1h30m % 1h]
|
||||
eval instant at 50m changes(http_requests[1h30m % 1h])
|
||||
{path="/foo"} 3
|
||||
{path="/bar"} 4
|
||||
{path="/biz"} 0
|
||||
|
||||
# Test modulo and calculation: [30m1s-30m1s % 1m]
|
||||
eval instant at 50m changes(http_requests[30m1s-30m1s % 1m])
|
||||
{path="/foo"} 3
|
||||
{path="/bar"} 4
|
||||
{path="/biz"} 0
|
||||
|
||||
# Test combination of operations: [(9m30s+30s)*3]
|
||||
eval instant at 50m changes(http_requests[(9m30s+30s)*3])
|
||||
{path="/foo"} 3
|
||||
{path="/bar"} 4
|
||||
{path="/biz"} 0
|
||||
|
||||
clear
|
||||
|
||||
load 10s
|
||||
metric1_total 0+1x1000
|
||||
|
||||
# In subquery expression.
|
||||
eval instant at 1000s sum_over_time(metric1_total[29s+1s:5s+5s])
|
||||
{} 297
|
||||
|
||||
# Test complex expressions in subquery ranges.
|
||||
eval instant at 1000s sum_over_time(metric1_total[29s+1s:((((8 - 2) / 3) * 7s) % 4) + 8000ms])
|
||||
{} 297
|
||||
|
||||
# Test complex expressions in offset ranges.
|
||||
eval instant at 1200s sum_over_time(metric1_total[29s+1s:20*500ms] offset (20*(((((8 - 2) / 3) * 7s) % 4) + 8000ms)))
|
||||
{} 297
|
||||
|
||||
# Test complex expressions in offset ranges with negative offset.
|
||||
eval instant at 800s sum_over_time(metric1_total[29s+1s:20*500ms] offset -(20*(((((8 - 2) / 3) * 7s) % 4) + 8000ms)))
|
||||
{} 297
|
||||
|
||||
# Test offset precedence with parentheses: offset (100 + 2)
|
||||
eval instant at 1000s metric1_total offset (100 + 2)
|
||||
{__name__="metric1_total"} 89
|
||||
|
||||
# Test offset precedence without parentheses: offset 100 + 2
|
||||
eval instant at 1000s metric1_total offset 100 + 2
|
||||
{} 92
|
Loading…
Reference in New Issue