From 9199fcb8d100afba384bf566779712a2b8094b6c Mon Sep 17 00:00:00 2001 From: Ganesh Vernekar <15064823+codesome@users.noreply.github.com> Date: Wed, 20 Jan 2021 16:27:39 +0530 Subject: [PATCH] '@ ' modifier (#8121) This commit adds `@ ` modifier as per this design doc: https://docs.google.com/document/d/1uSbD3T2beM-iX4-Hp7V074bzBRiRNlqUdcWP6JTDQSs/edit. An example query: ``` rate(process_cpu_seconds_total[1m]) and topk(7, rate(process_cpu_seconds_total[1h] @ 1234)) ``` which ranks based on last 1h rate and w.r.t. unix timestamp 1234 but actually plots the 1m rate. Signed-off-by: Ganesh Vernekar --- cmd/prometheus/main.go | 80 +- docs/disabled_features.md | 19 + docs/querying/basics.md | 42 +- pkg/timestamp/timestamp.go | 10 +- promql/engine.go | 445 +++++++++-- promql/engine_test.go | 1088 ++++++++++++++++++++++++++- promql/functions.go | 35 +- promql/parser/ast.go | 59 +- promql/parser/generated_parser.y | 29 +- promql/parser/generated_parser.y.go | 1075 +++++++++++++------------- promql/parser/lex.go | 3 +- promql/parser/parse.go | 82 +- promql/parser/parse_test.go | 771 ++++++++++++++----- promql/parser/printer.go | 42 +- promql/test.go | 184 +++-- promql/testdata/at_modifier.test | 166 ++++ promql/testdata/functions.test | 14 +- promql/testdata/histograms.test | 6 +- 18 files changed, 3208 insertions(+), 942 deletions(-) create mode 100644 docs/disabled_features.md create mode 100644 promql/testdata/at_modifier.test diff --git a/cmd/prometheus/main.go b/cmd/prometheus/main.go index c6d30ebb91..0dd07e6374 100644 --- a/cmd/prometheus/main.go +++ b/cmd/prometheus/main.go @@ -97,6 +97,53 @@ func init() { } } +type flagConfig struct { + configFile string + + localStoragePath string + notifier notifier.Options + notifierTimeout model.Duration + forGracePeriod model.Duration + outageTolerance model.Duration + resendDelay model.Duration + web web.Options + tsdb tsdbOptions + lookbackDelta model.Duration + webTimeout model.Duration + queryTimeout model.Duration + queryConcurrency int + queryMaxSamples int + RemoteFlushDeadline model.Duration + + featureList []string + // These options are extracted from featureList + // for ease of use. + enablePromQLAtModifier bool + + prometheusURL string + corsRegexString string + + promlogConfig promlog.Config +} + +// setFeatureListOptions sets the corresponding options from the featureList. +func (c *flagConfig) setFeatureListOptions(logger log.Logger) error { + for _, f := range c.featureList { + opts := strings.Split(f, ",") + for _, o := range opts { + switch o { + case "promql-at-modifier": + c.enablePromQLAtModifier = true + case "": + continue + default: + level.Warn(logger).Log("msg", "Unknown option for --enable-feature", "option", o) + } + } + } + return nil +} + func main() { if os.Getenv("DEBUG") != "" { runtime.SetBlockProfileRate(20) @@ -108,29 +155,7 @@ func main() { newFlagRetentionDuration model.Duration ) - cfg := struct { - configFile string - - localStoragePath string - notifier notifier.Options - notifierTimeout model.Duration - forGracePeriod model.Duration - outageTolerance model.Duration - resendDelay model.Duration - web web.Options - tsdb tsdbOptions - lookbackDelta model.Duration - webTimeout model.Duration - queryTimeout model.Duration - queryConcurrency int - queryMaxSamples int - RemoteFlushDeadline model.Duration - - prometheusURL string - corsRegexString string - - promlogConfig promlog.Config - }{ + cfg := flagConfig{ notifier: notifier.Options{ Registerer: prometheus.DefaultRegisterer, }, @@ -265,6 +290,9 @@ func main() { a.Flag("query.max-samples", "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."). Default("50000000").IntVar(&cfg.queryMaxSamples) + a.Flag("enable-feature", "Comma separated feature names to enable. Valid options: 'promql-at-modifier' to enable the @ modifier. See https://prometheus.io/docs/prometheus/latest/disabled_features/ for more details."). + Default("").StringsVar(&cfg.featureList) + promlogflag.AddFlags(a, &cfg.promlogConfig) _, err := a.Parse(os.Args[1:]) @@ -276,6 +304,11 @@ func main() { logger := promlog.New(&cfg.promlogConfig) + if err := cfg.setFeatureListOptions(logger); err != nil { + fmt.Fprintln(os.Stderr, errors.Wrapf(err, "Error parsing feature list")) + os.Exit(1) + } + cfg.web.ExternalURL, err = computeExternalURL(cfg.prometheusURL, cfg.web.ListenAddress) if err != nil { fmt.Fprintln(os.Stderr, errors.Wrapf(err, "parse external URL %q", cfg.prometheusURL)) @@ -400,6 +433,7 @@ func main() { ActiveQueryTracker: promql.NewActiveQueryTracker(cfg.localStoragePath, cfg.queryConcurrency, log.With(logger, "component", "activeQueryTracker")), LookbackDelta: time.Duration(cfg.lookbackDelta), NoStepSubqueryIntervalFn: noStepSubqueryInterval.Get, + EnableAtModifier: cfg.enablePromQLAtModifier, } queryEngine = promql.NewEngine(opts) diff --git a/docs/disabled_features.md b/docs/disabled_features.md new file mode 100644 index 0000000000..2af789311d --- /dev/null +++ b/docs/disabled_features.md @@ -0,0 +1,19 @@ +--- +title: Disabled Features +sort_rank: 10 +--- + +# Disabled Features + +Here is a list of features that are disabled by default since they are breaking changes or are considered experimental. +Their behaviour can change in future releases which will be communicated via the [release changelog](https://github.com/prometheus/prometheus/blob/master/CHANGELOG.md). + +You can enable them using the `--enable-feature` flag with a comma separated list of features. +They may be enabled by default in future versions. + +## `@` Modifier in PromQL + +`--enable-feature=promql-at-modifier` + +This feature lets you specify the evaluation time for instant vector selectors, +range vector selectors, and subqueries. More details can be found [here](querying/basics.md#@-modifier). diff --git a/docs/querying/basics.md b/docs/querying/basics.md index 7a26910d71..6c76a83b0a 100644 --- a/docs/querying/basics.md +++ b/docs/querying/basics.md @@ -204,11 +204,51 @@ The same works for range vectors. This returns the 5-minute rate that rate(http_requests_total[5m] offset 1w) +### @ modifier + +The `@` modifier allows changing the evaluation time for individual instant +and range vectors in a query. The time supplied to `@` modifier +is a unix timestamp and described with a float literal. + +For example, the following expression returns the value of +`http_requests_total` at `2021-01-04T07:40:00+00:00`: + + http_requests_total @ 1609746000 + +Note that the `@` modifier always needs to follow the selector +immediately, i.e. the following would be correct: + + sum(http_requests_total{method="GET"} @ 1609746000) // GOOD. + +While the following would be *incorrect*: + + sum(http_requests_total{method="GET"}) @ 1609746000 // INVALID. + +The same works for range vectors. This returns the 5-minute rate that +`http_requests_total` had at `2021-01-04T07:40:00+00:00`: + + rate(http_requests_total[5m] @ 1609746000) + +`@` modifier supports all representation of float literals described +above within the limits of `int64`. It can also be used along +with `offset` modifier where the offset is applied relative to the `@` +modifier time irrespective of which modifier is written first. +These 2 queries will produce the same result. + + # offset after @ + http_requests_total @ 1609746000 offset 5m + # offset before @ + http_requests_total offset 5m @ 1609746000 + +This modifier is disabled by default since it breaks the invariant that PromQL +does not look ahead of the evaluation time for samples. It can be enabled by setting +`--enable-feature=promql-at-modifier` flag. It will be enabled by default in the future. + ## Subquery Subquery allows you to run an instant query for a given range and resolution. The result of a subquery is a range vector. -Syntax: ` '[' ':' [] ']' [ offset ]` +Syntax: ` '[' ':' [] ']' [ @ ] [ offset ]` * `` is optional. Default is the global evaluation interval. diff --git a/pkg/timestamp/timestamp.go b/pkg/timestamp/timestamp.go index a7f03b0caf..93458f644d 100644 --- a/pkg/timestamp/timestamp.go +++ b/pkg/timestamp/timestamp.go @@ -13,7 +13,10 @@ package timestamp -import "time" +import ( + "math" + "time" +) // FromTime returns a new millisecond timestamp from a time. func FromTime(t time.Time) int64 { @@ -24,3 +27,8 @@ func FromTime(t time.Time) int64 { func Time(ts int64) time.Time { return time.Unix(ts/1000, (ts%1000)*int64(time.Millisecond)).UTC() } + +// FromFloatSeconds returns a millisecond timestamp from float seconds. +func FromFloatSeconds(ts float64) int64 { + return int64(math.Round(ts * 1000)) +} diff --git a/promql/engine.go b/promql/engine.go index 81bd70d5fb..4486313e41 100644 --- a/promql/engine.go +++ b/promql/engine.go @@ -208,6 +208,9 @@ type EngineOpts struct { // NoStepSubqueryIntervalFn is the default evaluation interval of // a subquery in milliseconds if no step in range vector was specified `[30m:]`. NoStepSubqueryIntervalFn func(rangeMillis int64) int64 + + // EnableAtModifier if true enables @ modifier. Disabled otherwise. + EnableAtModifier bool } // Engine handles the lifetime of queries from beginning to end. @@ -222,6 +225,7 @@ type Engine struct { queryLoggerLock sync.RWMutex lookbackDelta time.Duration noStepSubqueryIntervalFn func(rangeMillis int64) int64 + enableAtModifier bool } // NewEngine returns a new engine. @@ -302,6 +306,7 @@ func NewEngine(opts EngineOpts) *Engine { activeQueryTracker: opts.ActiveQueryTracker, lookbackDelta: opts.LookbackDelta, noStepSubqueryIntervalFn: opts.NoStepSubqueryIntervalFn, + enableAtModifier: opts.EnableAtModifier, } } @@ -334,7 +339,10 @@ func (ng *Engine) NewInstantQuery(q storage.Queryable, qs string, ts time.Time) if err != nil { return nil, err } - qry := ng.newQuery(q, expr, ts, ts, 0) + qry, err := ng.newQuery(q, expr, ts, ts, 0) + if err != nil { + return nil, err + } qry.q = qs return qry, nil @@ -350,15 +358,22 @@ func (ng *Engine) NewRangeQuery(q storage.Queryable, qs string, start, end time. if expr.Type() != parser.ValueTypeVector && expr.Type() != parser.ValueTypeScalar { return nil, errors.Errorf("invalid expression type %q for range query, must be Scalar or instant Vector", parser.DocumentedType(expr.Type())) } - qry := ng.newQuery(q, expr, start, end, interval) + qry, err := ng.newQuery(q, expr, start, end, interval) + if err != nil { + return nil, err + } qry.q = qs return qry, nil } -func (ng *Engine) newQuery(q storage.Queryable, expr parser.Expr, start, end time.Time, interval time.Duration) *query { +func (ng *Engine) newQuery(q storage.Queryable, expr parser.Expr, start, end time.Time, interval time.Duration) (*query, error) { + if err := ng.validateOpts(expr); err != nil { + return nil, err + } + es := &parser.EvalStmt{ - Expr: expr, + Expr: WrapWithStepInvariantExpr(expr), Start: start, End: end, Interval: interval, @@ -369,7 +384,39 @@ func (ng *Engine) newQuery(q storage.Queryable, expr parser.Expr, start, end tim stats: stats.NewQueryTimers(), queryable: q, } - return qry + return qry, nil +} + +func (ng *Engine) validateOpts(expr parser.Expr) error { + if ng.enableAtModifier { + return nil + } + + var validationErr error + parser.Inspect(expr, func(node parser.Node, path []parser.Node) error { + switch n := node.(type) { + case *parser.VectorSelector: + if n.Timestamp != nil { + validationErr = errors.New("@ modifier is disabled") + return validationErr + } + + case *parser.MatrixSelector: + if n.VectorSelector.(*parser.VectorSelector).Timestamp != nil { + validationErr = errors.New("@ modifier is disabled") + return validationErr + } + + case *parser.SubqueryExpr: + if n.Timestamp != nil { + validationErr = errors.New("@ modifier is disabled") + return validationErr + } + } + return nil + }) + + return validationErr } func (ng *Engine) newTestQuery(f func(context.Context) error) Query { @@ -477,8 +524,8 @@ func durationMilliseconds(d time.Duration) int64 { // execEvalStmt evaluates the expression of an evaluation statement for the given time range. func (ng *Engine) execEvalStmt(ctx context.Context, query *query, s *parser.EvalStmt) (parser.Value, storage.Warnings, error) { prepareSpanTimer, ctxPrepare := query.stats.GetSpanTimer(ctx, stats.QueryPreparationTime, ng.metrics.queryPrepareTime) - mint := ng.findMinTime(s) - querier, err := query.queryable.Querier(ctxPrepare, timestamp.FromTime(mint), timestamp.FromTime(s.End)) + mint, maxt := ng.findMinMaxTime(s) + querier, err := query.queryable.Querier(ctxPrepare, mint, maxt) if err != nil { prepareSpanTimer.Finish() return nil, nil, err @@ -488,6 +535,9 @@ func (ng *Engine) execEvalStmt(ctx context.Context, query *query, s *parser.Eval ng.populateSeries(querier, s) prepareSpanTimer.Finish() + // Modify the offset of vector and matrix selectors for the @ modifier + // w.r.t. the start time since only 1 evaluation will be done on them. + setOffsetForAtModifier(timeMilliseconds(s.Start), s.Expr) evalSpanTimer, ctxInnerEval := query.stats.GetSpanTimer(ctx, stats.InnerEvalTime, ng.metrics.queryInnerEval) // Instant evaluation. This is executed as a range evaluation with one step. if s.Start == s.End && s.Interval == 0 { @@ -576,45 +626,102 @@ func (ng *Engine) execEvalStmt(ctx context.Context, query *query, s *parser.Eval return mat, warnings, nil } -// subqueryOffsetRange returns the sum of offsets and ranges of all subqueries in the path. -func (ng *Engine) subqueryOffsetRange(path []parser.Node) (time.Duration, time.Duration) { +// subqueryTimes returns the sum of offsets and ranges of all subqueries in the path. +// If the @ modifier is used, then the offset and range is w.r.t. that timestamp +// (i.e. the sum is reset when we have @ modifier). +// The returned *int64 is the closest timestamp that was seen. nil for no @ modifier. +func subqueryTimes(path []parser.Node) (time.Duration, time.Duration, *int64) { var ( - subqOffset time.Duration - subqRange time.Duration + subqOffset, subqRange time.Duration + ts int64 = math.MaxInt64 ) for _, node := range path { switch n := node.(type) { case *parser.SubqueryExpr: - subqOffset += n.Offset + subqOffset += n.OriginalOffset subqRange += n.Range + if n.Timestamp != nil { + // The @ modifier on subquery invalidates all the offset and + // range till now. Hence resetting it here. + subqOffset = n.OriginalOffset + subqRange = n.Range + ts = *n.Timestamp + } } } - return subqOffset, subqRange + var tsp *int64 + if ts != math.MaxInt64 { + tsp = &ts + } + return subqOffset, subqRange, tsp } -func (ng *Engine) findMinTime(s *parser.EvalStmt) time.Time { - var maxOffset time.Duration +func (ng *Engine) findMinMaxTime(s *parser.EvalStmt) (int64, int64) { + var minTimestamp, maxTimestamp int64 = math.MaxInt64, math.MinInt64 + // Whenever a MatrixSelector is evaluated, evalRange is set to the corresponding range. + // The evaluation of the VectorSelector inside then evaluates the given range and unsets + // the variable. + var evalRange time.Duration parser.Inspect(s.Expr, func(node parser.Node, path []parser.Node) error { - subqOffset, subqRange := ng.subqueryOffsetRange(path) switch n := node.(type) { case *parser.VectorSelector: - if maxOffset < ng.lookbackDelta+subqOffset+subqRange { - maxOffset = ng.lookbackDelta + subqOffset + subqRange + start, end := ng.getTimeRangesForSelector(s, n, path, evalRange) + if start < minTimestamp { + minTimestamp = start } - if n.Offset+ng.lookbackDelta+subqOffset+subqRange > maxOffset { - maxOffset = n.Offset + ng.lookbackDelta + subqOffset + subqRange + if end > maxTimestamp { + maxTimestamp = end } + evalRange = 0 + case *parser.MatrixSelector: - if maxOffset < n.Range+subqOffset+subqRange { - maxOffset = n.Range + subqOffset + subqRange - } - if m := n.VectorSelector.(*parser.VectorSelector).Offset + n.Range + subqOffset + subqRange; m > maxOffset { - maxOffset = m - } + evalRange = n.Range } return nil }) - return s.Start.Add(-maxOffset) + + if maxTimestamp == math.MinInt64 { + // This happens when there was no selector. Hence no time range to select. + minTimestamp = 0 + maxTimestamp = 0 + } + + return minTimestamp, maxTimestamp +} + +func (ng *Engine) getTimeRangesForSelector(s *parser.EvalStmt, n *parser.VectorSelector, path []parser.Node, evalRange time.Duration) (int64, int64) { + start, end := timestamp.FromTime(s.Start), timestamp.FromTime(s.End) + subqOffset, subqRange, subqTs := subqueryTimes(path) + + if subqTs != nil { + // The timestamp on the subquery overrides the eval statement time ranges. + start = *subqTs + end = *subqTs + } + + if n.Timestamp != nil { + // The timestamp on the selector overrides everything. + start = *n.Timestamp + end = *n.Timestamp + } else { + offsetMilliseconds := durationMilliseconds(subqOffset) + start = start - offsetMilliseconds - durationMilliseconds(subqRange) + end = end - offsetMilliseconds + } + + if evalRange == 0 { + start = start - durationMilliseconds(ng.lookbackDelta) + } else { + // For all matrix queries we want to ensure that we have (end-start) + range selected + // this way we have `range` data before the start time + start = start - durationMilliseconds(evalRange) + } + + offsetMilliseconds := durationMilliseconds(n.OriginalOffset) + start = start - offsetMilliseconds + end = end - offsetMilliseconds + + return start, end } func (ng *Engine) populateSeries(querier storage.Querier, s *parser.EvalStmt) { @@ -626,40 +733,18 @@ func (ng *Engine) populateSeries(querier storage.Querier, s *parser.EvalStmt) { parser.Inspect(s.Expr, func(node parser.Node, path []parser.Node) error { switch n := node.(type) { case *parser.VectorSelector: + start, end := ng.getTimeRangesForSelector(s, n, path, evalRange) hints := &storage.SelectHints{ - Start: timestamp.FromTime(s.Start), - End: timestamp.FromTime(s.End), + Start: start, + End: end, Step: durationMilliseconds(s.Interval), + Range: durationMilliseconds(evalRange), + Func: extractFuncFromPath(path), } - - // We need to make sure we select the timerange selected by the subquery. - // The subqueryOffsetRange function gives the sum of range and the - // sum of offset. - // TODO(bwplotka): Add support for better hints when subquerying. See: https://github.com/prometheus/prometheus/issues/7630. - subqOffset, subqRange := ng.subqueryOffsetRange(path) - offsetMilliseconds := durationMilliseconds(subqOffset) - hints.Start = hints.Start - offsetMilliseconds - durationMilliseconds(subqRange) - hints.End = hints.End - offsetMilliseconds - - if evalRange == 0 { - hints.Start = hints.Start - durationMilliseconds(ng.lookbackDelta) - } else { - hints.Range = durationMilliseconds(evalRange) - // For all matrix queries we want to ensure that we have (end-start) + range selected - // this way we have `range` data before the start time - hints.Start = hints.Start - durationMilliseconds(evalRange) - evalRange = 0 - } - - hints.Func = extractFuncFromPath(path) + evalRange = 0 hints.By, hints.Grouping = extractGroupsFromPath(path) - if n.Offset > 0 { - offsetMilliseconds := durationMilliseconds(n.Offset) - hints.Start = hints.Start - offsetMilliseconds - hints.End = hints.End - offsetMilliseconds - } - n.UnexpandedSeriesSet = querier.Select(false, hints, n.LabelMatchers...) + case *parser.MatrixSelector: evalRange = n.Range } @@ -852,7 +937,7 @@ func (enh *EvalNodeHelper) signatureFunc(on bool, names ...string) func(labels.L // the given function with the values computed for each expression at that // step. The return value is the combination into time series of all the // function call results. -func (ev *evaluator) rangeEval(f func([]parser.Value, *EvalNodeHelper) (Vector, storage.Warnings), exprs ...parser.Expr) (Matrix, storage.Warnings) { +func (ev *evaluator) rangeEval(funcCall func([]parser.Value, *EvalNodeHelper) (Vector, storage.Warnings), exprs ...parser.Expr) (Matrix, storage.Warnings) { numSteps := int((ev.endTimestamp-ev.startTimestamp)/ev.interval) + 1 matrixes := make([]Matrix, len(exprs)) origMatrixes := make([]Matrix, len(exprs)) @@ -917,7 +1002,7 @@ func (ev *evaluator) rangeEval(f func([]parser.Value, *EvalNodeHelper) (Vector, } // Make the function call. enh.Ts = ts - result, ws := f(args, enh) + result, ws := funcCall(args, enh) if result.ContainsSameLabelset() { ev.errorf("vector cannot contain metrics with the same labelset") } @@ -978,21 +1063,30 @@ func (ev *evaluator) rangeEval(f func([]parser.Value, *EvalNodeHelper) (Vector, // evalSubquery evaluates given SubqueryExpr and returns an equivalent // evaluated MatrixSelector in its place. Note that the Name and LabelMatchers are not set. -func (ev *evaluator) evalSubquery(subq *parser.SubqueryExpr) (*parser.MatrixSelector, storage.Warnings) { +func (ev *evaluator) evalSubquery(subq *parser.SubqueryExpr) (*parser.MatrixSelector, int, storage.Warnings) { val, ws := ev.eval(subq) mat := val.(Matrix) vs := &parser.VectorSelector{ - Offset: subq.Offset, - Series: make([]storage.Series, 0, len(mat)), + OriginalOffset: subq.OriginalOffset, + Offset: subq.Offset, + Series: make([]storage.Series, 0, len(mat)), + Timestamp: subq.Timestamp, + } + if subq.Timestamp != nil { + // The offset of subquery is not modified in case of @ modifier. + // Hence we take care of that here for the result. + vs.Offset = subq.OriginalOffset + time.Duration(ev.startTimestamp-*subq.Timestamp)*time.Millisecond } ms := &parser.MatrixSelector{ Range: subq.Range, VectorSelector: vs, } + totalSamples := 0 for _, s := range mat { + totalSamples += len(s.Points) vs.Series = append(vs.Series, NewStorageSeries(s)) } - return ms, ws + return ms, totalSamples, ws } // eval evaluates the given expression as the given AST expression node requires. @@ -1007,7 +1101,7 @@ func (ev *evaluator) eval(expr parser.Expr) (parser.Value, storage.Warnings) { switch e := expr.(type) { case *parser.AggregateExpr: unwrapParenExpr(&e.Param) - if s, ok := e.Param.(*parser.StringLiteral); ok { + if s, ok := unwrapStepInvariantExpr(e.Param).(*parser.StringLiteral); ok { return ev.rangeEval(func(v []parser.Value, enh *EvalNodeHelper) (Vector, storage.Warnings) { return ev.aggregation(e.Op, e.Grouping, e.Without, s.Val, v[0].(Vector), enh), nil }, e.Expr) @@ -1022,13 +1116,13 @@ func (ev *evaluator) eval(expr parser.Expr) (parser.Value, storage.Warnings) { case *parser.Call: call := FunctionCalls[e.Func.Name] - if e.Func.Name == "timestamp" { // Matrix evaluation always returns the evaluation time, // so this function needs special handling when given // a vector selector. unwrapParenExpr(&e.Args[0]) - vs, ok := e.Args[0].(*parser.VectorSelector) + arg := unwrapStepInvariantExpr(e.Args[0]) + vs, ok := arg.(*parser.VectorSelector) if ok { return ev.rangeEval(func(v []parser.Value, enh *EvalNodeHelper) (Vector, storage.Warnings) { val, ws := ev.vectorSelector(vs, enh.Ts) @@ -1045,7 +1139,7 @@ func (ev *evaluator) eval(expr parser.Expr) (parser.Value, storage.Warnings) { ) for i := range e.Args { unwrapParenExpr(&e.Args[i]) - a := e.Args[i] + a := unwrapStepInvariantExpr(e.Args[i]) if _, ok := a.(*parser.MatrixSelector); ok { matrixArgIndex = i matrixArg = true @@ -1056,9 +1150,14 @@ func (ev *evaluator) eval(expr parser.Expr) (parser.Value, storage.Warnings) { matrixArgIndex = i matrixArg = true // Replacing parser.SubqueryExpr with parser.MatrixSelector. - val, ws := ev.evalSubquery(subq) + val, totalSamples, ws := ev.evalSubquery(subq) e.Args[i] = val warnings = append(warnings, ws...) + defer func() { + // subquery result takes space in the memory. Get rid of that at the end. + val.VectorSelector.(*parser.VectorSelector).Series = nil + ev.currentSamples -= totalSamples + }() break } } @@ -1083,7 +1182,7 @@ func (ev *evaluator) eval(expr parser.Expr) (parser.Value, storage.Warnings) { } } - sel := e.Args[matrixArgIndex].(*parser.MatrixSelector) + sel := unwrapStepInvariantExpr(e.Args[matrixArgIndex]).(*parser.MatrixSelector) selVS := sel.VectorSelector.(*parser.VectorSelector) ws, err := checkAndExpandSeriesSet(ev.ctx, sel) @@ -1146,7 +1245,7 @@ func (ev *evaluator) eval(expr parser.Expr) (parser.Value, storage.Warnings) { it.ReduceDelta(stepRange) } if len(ss.Points) > 0 { - if ev.currentSamples < ev.maxSamples { + if ev.currentSamples+len(ss.Points) <= ev.maxSamples { mat = append(mat, ss) ev.currentSamples += len(ss.Points) } else { @@ -1266,6 +1365,9 @@ func (ev *evaluator) eval(expr parser.Expr) (parser.Value, storage.Warnings) { return append(enh.Out, Sample{Point: Point{V: e.Val}}), nil }) + case *parser.StringLiteral: + return String{V: e.Val, T: ev.startTimestamp}, nil + case *parser.VectorSelector: ws, err := checkAndExpandSeriesSet(ev.ctx, e) if err != nil { @@ -1332,11 +1434,65 @@ func (ev *evaluator) eval(expr parser.Expr) (parser.Value, storage.Warnings) { newEv.startTimestamp += newEv.interval } + if newEv.startTimestamp != ev.startTimestamp { + // Adjust the offset of selectors based on the new + // start time of the evaluator since the calculation + // of the offset with @ happens w.r.t. the start time. + setOffsetForAtModifier(newEv.startTimestamp, e.Expr) + } + res, ws := newEv.eval(e.Expr) ev.currentSamples = newEv.currentSamples return res, ws - case *parser.StringLiteral: - return String{V: e.Val, T: ev.startTimestamp}, nil + case *parser.StepInvariantExpr: + switch ce := e.Expr.(type) { + case *parser.StringLiteral, *parser.NumberLiteral: + return ev.eval(ce) + } + + newEv := &evaluator{ + startTimestamp: ev.startTimestamp, + endTimestamp: ev.startTimestamp, // Always a single evaluation. + interval: ev.interval, + ctx: ev.ctx, + currentSamples: ev.currentSamples, + maxSamples: ev.maxSamples, + logger: ev.logger, + lookbackDelta: ev.lookbackDelta, + noStepSubqueryIntervalFn: ev.noStepSubqueryIntervalFn, + } + res, ws := newEv.eval(e.Expr) + ev.currentSamples = newEv.currentSamples + switch e.Expr.(type) { + case *parser.MatrixSelector, *parser.SubqueryExpr: + // We do not duplicate results for range selectors since result is a matrix + // with their unique timestamps which does not depend on the step. + return res, ws + } + + // For every evaluation while the value remains same, the timestamp for that + // value would change for different eval times. Hence we duplicate the result + // with changed timestamps. + mat, ok := res.(Matrix) + if !ok { + panic(errors.Errorf("unexpected result in StepInvariantExpr evaluation: %T", expr)) + } + for i := range mat { + if len(mat[i].Points) != 1 { + panic(errors.Errorf("unexpected number of samples")) + } + for ts := ev.startTimestamp + ev.interval; ts <= ev.endTimestamp; ts = ts + ev.interval { + mat[i].Points = append(mat[i].Points, Point{ + T: ts, + V: mat[i].Points[0].V, + }) + ev.currentSamples++ + if ev.currentSamples > ev.maxSamples { + ev.error(ErrTooManySamples(env)) + } + } + } + return res, ws } panic(errors.Errorf("unhandled expression of type: %T", expr)) @@ -1359,12 +1515,13 @@ func (ev *evaluator) vectorSelector(node *parser.VectorSelector, ts int64) (Vect Metric: node.Series[i].Labels(), Point: Point{V: v, T: t}, }) + ev.currentSamples++ + if ev.currentSamples > ev.maxSamples { + ev.error(ErrTooManySamples(env)) + } } - if ev.currentSamples >= ev.maxSamples { - ev.error(ErrTooManySamples(env)) - } } return vec, ws } @@ -1497,8 +1654,8 @@ func (ev *evaluator) matrixIterSlice(it *storage.BufferedSeriesIterator, mint, m if ev.currentSamples >= ev.maxSamples { ev.error(ErrTooManySamples(env)) } - out = append(out, Point{T: t, V: v}) ev.currentSamples++ + out = append(out, Point{T: t, V: v}) } } // The seeked sample might also be in the range. @@ -2141,3 +2298,141 @@ func unwrapParenExpr(e *parser.Expr) { } } } + +func unwrapStepInvariantExpr(e parser.Expr) parser.Expr { + if p, ok := e.(*parser.StepInvariantExpr); ok { + return p.Expr + } + return e +} + +// WrapWithStepInvariantExpr wraps all possible parts of the given +// expression with StepInvariantExpr wherever valid. +func WrapWithStepInvariantExpr(expr parser.Expr) parser.Expr { + isStepInvariant := wrapWithStepInvariantExprHelper(expr) + if isStepInvariant { + return newStepInvariantExpr(expr) + } + return expr +} + +// wrapWithStepInvariantExprHelper wraps the child nodes of the expression +// with a StepInvariantExpr wherever valid. The returned boolean is true if the +// passed expression qualifies to be wrapped by StepInvariantExpr. +func wrapWithStepInvariantExprHelper(expr parser.Expr) bool { + switch n := expr.(type) { + case *parser.VectorSelector: + return n.Timestamp != nil + + case *parser.AggregateExpr: + return wrapWithStepInvariantExprHelper(n.Expr) + + case *parser.BinaryExpr: + isInvariant1, isInvariant2 := wrapWithStepInvariantExprHelper(n.LHS), wrapWithStepInvariantExprHelper(n.RHS) + if isInvariant1 && isInvariant2 { + return true + } + + if isInvariant1 { + n.LHS = newStepInvariantExpr(n.LHS) + } + if isInvariant2 { + n.RHS = newStepInvariantExpr(n.RHS) + } + + return false + + case *parser.Call: + _, ok := AtModifierUnsafeFunctions[n.Func.Name] + isStepInvariant := !ok + isStepInvariantSlice := make([]bool, len(n.Args)) + for i := range n.Args { + isStepInvariantSlice[i] = wrapWithStepInvariantExprHelper(n.Args[i]) + isStepInvariant = isStepInvariant && isStepInvariantSlice[i] + } + + if isStepInvariant { + + // The function and all arguments are step invariant. + return true + } + + for i, isi := range isStepInvariantSlice { + if isi { + n.Args[i] = newStepInvariantExpr(n.Args[i]) + } + } + return false + + case *parser.MatrixSelector: + return n.VectorSelector.(*parser.VectorSelector).Timestamp != nil + + case *parser.SubqueryExpr: + // Since we adjust offset for the @ modifier evaluation, + // it gets tricky to adjust it for every subquery step. + // Hence we wrap the inside of subquery irrespective of + // @ on subquery (given it is also step invariant) so that + // it is evaluated only once w.r.t. the start time of subquery. + isInvariant := wrapWithStepInvariantExprHelper(n.Expr) + if isInvariant { + n.Expr = newStepInvariantExpr(n.Expr) + } + return n.Timestamp != nil + + case *parser.ParenExpr: + return wrapWithStepInvariantExprHelper(n.Expr) + + case *parser.UnaryExpr: + return wrapWithStepInvariantExprHelper(n.Expr) + + case *parser.StringLiteral, *parser.NumberLiteral: + return true + } + + panic(fmt.Sprintf("found unexpected node %#v", expr)) +} + +func newStepInvariantExpr(expr parser.Expr) parser.Expr { + if e, ok := expr.(*parser.ParenExpr); ok { + // Wrapping the inside of () makes it easy to unwrap the paren later. + // But this effectively unwraps the paren. + return newStepInvariantExpr(e.Expr) + + } + return &parser.StepInvariantExpr{Expr: expr} +} + +// setOffsetForAtModifier modifies the offset of vector and matrix selector +// and subquery in the tree to accommodate the timestamp of @ modifier. +// The offset is adjusted w.r.t. the given evaluation time. +func setOffsetForAtModifier(evalTime int64, expr parser.Expr) { + getOffset := func(ts *int64, originalOffset time.Duration, path []parser.Node) time.Duration { + if ts == nil { + return originalOffset + } + + subqOffset, _, subqTs := subqueryTimes(path) + if subqTs != nil { + subqOffset += time.Duration(evalTime-*subqTs) * time.Millisecond + } + + offsetForTs := time.Duration(evalTime-*ts) * time.Millisecond + offsetDiff := offsetForTs - subqOffset + return originalOffset + offsetDiff + } + + parser.Inspect(expr, func(node parser.Node, path []parser.Node) error { + switch n := node.(type) { + case *parser.VectorSelector: + n.Offset = getOffset(n.Timestamp, n.OriginalOffset, path) + + case *parser.MatrixSelector: + vs := n.VectorSelector.(*parser.VectorSelector) + vs.Offset = getOffset(vs.Timestamp, vs.OriginalOffset, path) + + case *parser.SubqueryExpr: + n.Offset = getOffset(n.Timestamp, n.OriginalOffset, path) + } + return nil + }) +} diff --git a/promql/engine_test.go b/promql/engine_test.go index c4c4aff906..273ae272cd 100644 --- a/promql/engine_test.go +++ b/promql/engine_test.go @@ -246,11 +246,12 @@ func (h *hintRecordingQuerier) Select(sortSeries bool, hints *storage.SelectHint func TestSelectHintsSetCorrectly(t *testing.T) { opts := EngineOpts{ - Logger: nil, - Reg: nil, - MaxSamples: 10, - Timeout: 10 * time.Second, - LookbackDelta: 5 * time.Second, + Logger: nil, + Reg: nil, + MaxSamples: 10, + Timeout: 10 * time.Second, + LookbackDelta: 5 * time.Second, + EnableAtModifier: true, } for _, tc := range []struct { @@ -267,16 +268,46 @@ func TestSelectHintsSetCorrectly(t *testing.T) { expected: []*storage.SelectHints{ {Start: 5000, End: 10000}, }, + }, { + query: "foo @ 15", start: 10000, + expected: []*storage.SelectHints{ + {Start: 10000, End: 15000}, + }, + }, { + query: "foo @ 1", start: 10000, + expected: []*storage.SelectHints{ + {Start: -4000, End: 1000}, + }, }, { query: "foo[2m]", start: 200000, expected: []*storage.SelectHints{ {Start: 80000, End: 200000, Range: 120000}, }, + }, { + query: "foo[2m] @ 180", start: 200000, + expected: []*storage.SelectHints{ + {Start: 60000, End: 180000, Range: 120000}, + }, + }, { + query: "foo[2m] @ 300", start: 200000, + expected: []*storage.SelectHints{ + {Start: 180000, End: 300000, Range: 120000}, + }, + }, { + query: "foo[2m] @ 60", start: 200000, + expected: []*storage.SelectHints{ + {Start: -60000, End: 60000, Range: 120000}, + }, }, { query: "foo[2m] offset 2m", start: 300000, expected: []*storage.SelectHints{ {Start: 60000, End: 180000, Range: 120000}, }, + }, { + query: "foo[2m] @ 200 offset 2m", start: 300000, + expected: []*storage.SelectHints{ + {Start: -40000, End: 80000, Range: 120000}, + }, }, { query: "foo[2m:1s]", start: 300000, expected: []*storage.SelectHints{ @@ -287,6 +318,21 @@ func TestSelectHintsSetCorrectly(t *testing.T) { expected: []*storage.SelectHints{ {Start: 175000, End: 300000, Func: "count_over_time"}, }, + }, { + query: "count_over_time(foo[2m:1s] @ 300)", start: 200000, + expected: []*storage.SelectHints{ + {Start: 175000, End: 300000, Func: "count_over_time"}, + }, + }, { + query: "count_over_time(foo[2m:1s] @ 200)", start: 200000, + expected: []*storage.SelectHints{ + {Start: 75000, End: 200000, Func: "count_over_time"}, + }, + }, { + query: "count_over_time(foo[2m:1s] @ 100)", start: 200000, + expected: []*storage.SelectHints{ + {Start: -25000, End: 100000, Func: "count_over_time"}, + }, }, { query: "count_over_time(foo[2m:1s] offset 10s)", start: 300000, expected: []*storage.SelectHints{ @@ -298,11 +344,55 @@ func TestSelectHintsSetCorrectly(t *testing.T) { {Start: 155000, End: 280000, Func: "count_over_time"}, }, }, { + // When the @ is on the vector selector, the enclosing subquery parameters + // don't affect the hint ranges. + query: "count_over_time((foo @ 200 offset 10s)[2m:1s] offset 10s)", start: 300000, + expected: []*storage.SelectHints{ + {Start: 185000, End: 190000, Func: "count_over_time"}, + }, + }, { + // When the @ is on the vector selector, the enclosing subquery parameters + // don't affect the hint ranges. + query: "count_over_time((foo @ 200 offset 10s)[2m:1s] @ 100 offset 10s)", start: 300000, + expected: []*storage.SelectHints{ + {Start: 185000, End: 190000, Func: "count_over_time"}, + }, + }, { + query: "count_over_time((foo offset 10s)[2m:1s] @ 100 offset 10s)", start: 300000, + expected: []*storage.SelectHints{ + {Start: -45000, End: 80000, Func: "count_over_time"}, + }, + }, { query: "foo", start: 10000, end: 20000, expected: []*storage.SelectHints{ {Start: 5000, End: 20000, Step: 1000}, }, + }, { + query: "foo @ 15", start: 10000, end: 20000, + expected: []*storage.SelectHints{ + {Start: 10000, End: 15000, Step: 1000}, + }, + }, { + query: "foo @ 1", start: 10000, end: 20000, + expected: []*storage.SelectHints{ + {Start: -4000, End: 1000, Step: 1000}, + }, + }, { + query: "rate(foo[2m] @ 180)", start: 200000, end: 500000, + expected: []*storage.SelectHints{ + {Start: 60000, End: 180000, Range: 120000, Func: "rate", Step: 1000}, + }, + }, { + query: "rate(foo[2m] @ 300)", start: 200000, end: 500000, + expected: []*storage.SelectHints{ + {Start: 180000, End: 300000, Range: 120000, Func: "rate", Step: 1000}, + }, + }, { + query: "rate(foo[2m] @ 60)", start: 200000, end: 500000, + expected: []*storage.SelectHints{ + {Start: -60000, End: 60000, Range: 120000, Func: "rate", Step: 1000}, + }, }, { query: "rate(foo[2m])", start: 200000, end: 500000, expected: []*storage.SelectHints{ @@ -328,11 +418,45 @@ func TestSelectHintsSetCorrectly(t *testing.T) { expected: []*storage.SelectHints{ {Start: 165000, End: 490000, Func: "count_over_time", Step: 1000}, }, + }, { + query: "count_over_time(foo[2m:1s] @ 300)", start: 200000, end: 500000, + expected: []*storage.SelectHints{ + {Start: 175000, End: 300000, Func: "count_over_time", Step: 1000}, + }, + }, { + query: "count_over_time(foo[2m:1s] @ 200)", start: 200000, end: 500000, + expected: []*storage.SelectHints{ + {Start: 75000, End: 200000, Func: "count_over_time", Step: 1000}, + }, + }, { + query: "count_over_time(foo[2m:1s] @ 100)", start: 200000, end: 500000, + expected: []*storage.SelectHints{ + {Start: -25000, End: 100000, Func: "count_over_time", Step: 1000}, + }, }, { query: "count_over_time((foo offset 10s)[2m:1s] offset 10s)", start: 300000, end: 500000, expected: []*storage.SelectHints{ {Start: 155000, End: 480000, Func: "count_over_time", Step: 1000}, }, + }, { + // When the @ is on the vector selector, the enclosing subquery parameters + // don't affect the hint ranges. + query: "count_over_time((foo @ 200 offset 10s)[2m:1s] offset 10s)", start: 300000, end: 500000, + expected: []*storage.SelectHints{ + {Start: 185000, End: 190000, Func: "count_over_time", Step: 1000}, + }, + }, { + // When the @ is on the vector selector, the enclosing subquery parameters + // don't affect the hint ranges. + query: "count_over_time((foo @ 200 offset 10s)[2m:1s] @ 100 offset 10s)", start: 300000, end: 500000, + expected: []*storage.SelectHints{ + {Start: 185000, End: 190000, Func: "count_over_time", Step: 1000}, + }, + }, { + query: "count_over_time((foo offset 10s)[2m:1s] @ 100 offset 10s)", start: 300000, end: 500000, + expected: []*storage.SelectHints{ + {Start: -45000, End: 80000, Func: "count_over_time", Step: 1000}, + }, }, { query: "sum by (dim1) (foo)", start: 10000, expected: []*storage.SelectHints{ @@ -364,7 +488,53 @@ func TestSelectHintsSetCorrectly(t *testing.T) { {Start: 95000, End: 120000, Func: "sum", By: true}, {Start: 95000, End: 120000, Func: "max", By: true}, }, - }} { + }, { + query: "foo @ 50 + bar @ 250 + baz @ 900", start: 100000, end: 500000, + expected: []*storage.SelectHints{ + {Start: 45000, End: 50000, Step: 1000}, + {Start: 245000, End: 250000, Step: 1000}, + {Start: 895000, End: 900000, Step: 1000}, + }, + }, { + query: "foo @ 50 + bar + baz @ 900", start: 100000, end: 500000, + expected: []*storage.SelectHints{ + {Start: 45000, End: 50000, Step: 1000}, + {Start: 95000, End: 500000, Step: 1000}, + {Start: 895000, End: 900000, Step: 1000}, + }, + }, { + query: "rate(foo[2s] @ 50) + bar @ 250 + baz @ 900", start: 100000, end: 500000, + expected: []*storage.SelectHints{ + {Start: 48000, End: 50000, Step: 1000, Func: "rate", Range: 2000}, + {Start: 245000, End: 250000, Step: 1000}, + {Start: 895000, End: 900000, Step: 1000}, + }, + }, { + query: "rate(foo[2s:1s] @ 50) + bar + baz", start: 100000, end: 500000, + expected: []*storage.SelectHints{ + {Start: 43000, End: 50000, Step: 1000, Func: "rate"}, + {Start: 95000, End: 500000, Step: 1000}, + {Start: 95000, End: 500000, Step: 1000}, + }, + }, { + query: "rate(foo[2s:1s] @ 50) + bar + rate(baz[2m:1s] @ 900 offset 2m) ", start: 100000, end: 500000, + expected: []*storage.SelectHints{ + {Start: 43000, End: 50000, Step: 1000, Func: "rate"}, + {Start: 95000, End: 500000, Step: 1000}, + {Start: 655000, End: 780000, Step: 1000, Func: "rate"}, + }, + }, { // Hints are based on the inner most subquery timestamp. + query: `sum_over_time(sum_over_time(metric{job="1"}[100s])[100s:25s] @ 50)[3s:1s] @ 3000`, start: 100000, + expected: []*storage.SelectHints{ + {Start: -150000, End: 50000, Range: 100000, Func: "sum_over_time"}, + }, + }, { // Hints are based on the inner most subquery timestamp. + query: `sum_over_time(sum_over_time(metric{job="1"}[100s])[100s:25s] @ 3000)[3s:1s] @ 50`, + expected: []*storage.SelectHints{ + {Start: 2800000, End: 3000000, Range: 100000, Func: "sum_over_time"}, + }, + }, + } { t.Run(tc.query, func(t *testing.T) { engine := NewEngine(opts) hintsRecorder := &noopHintRecordingQueryable{} @@ -545,9 +715,9 @@ load 10s func TestMaxQuerySamples(t *testing.T) { test, err := NewTest(t, ` load 10s - metric 1 2 - bigmetric{a="1"} 1 2 - bigmetric{a="2"} 1 2 + metric 1+1x100 + bigmetric{a="1"} 1+1x100 + bigmetric{a="2"} 1+1x100 `) require.NoError(t, err) defer test.Close() @@ -585,6 +755,10 @@ load 10s Query: "metric[20s:5s]", MaxSamples: 3, Start: time.Unix(10, 0), + }, { + Query: "metric[20s] @ 10", + MaxSamples: 2, + Start: time.Unix(0, 0), }, // Range queries. { @@ -617,36 +791,272 @@ load 10s Start: time.Unix(0, 0), End: time.Unix(10, 0), Interval: 5 * time.Second, + }, { + // Result is duplicated, so @ also produces 3 samples. + Query: "metric @ 10", + MaxSamples: 3, + Start: time.Unix(0, 0), + End: time.Unix(10, 0), + Interval: 5 * time.Second, + }, { + // The peak samples in memory is during the first evaluation: + // - Subquery takes 22 samples, 11 for each bigmetric, + // - Result is calculated per series where the series samples is buffered, hence 11 more here. + // - The result of two series is added before the last series buffer is discarded, so 2 more here. + // Hence at peak it is 22 (subquery) + 11 (buffer of a series) + 2 (result from 2 series). + // The subquery samples and the buffer is discarded before duplicating. + Query: `rate(bigmetric[10s:1s] @ 10)`, + MaxSamples: 35, + Start: time.Unix(0, 0), + End: time.Unix(10, 0), + Interval: 5 * time.Second, + }, { + // Here the reasoning is same as above. But LHS and RHS are done one after another. + // So while one of them takes 35 samples at peak, we need to hold the 2 sample + // result of the other till then. + Query: `rate(bigmetric[10s:1s] @ 10) + rate(bigmetric[10s:1s] @ 30)`, + MaxSamples: 37, + Start: time.Unix(0, 0), + End: time.Unix(10, 0), + Interval: 5 * time.Second, + }, { + // Sample as above but with only 1 part as step invariant. + // Here the peak is caused by the non-step invariant part as it touches more time range. + // Hence at peak it is 2*21 (subquery from 0s to 20s) + // + 11 (buffer of a series per evaluation) + // + 6 (result from 2 series at 3 eval times). + Query: `rate(bigmetric[10s:1s]) + rate(bigmetric[10s:1s] @ 30)`, + MaxSamples: 59, + Start: time.Unix(10, 0), + End: time.Unix(20, 0), + Interval: 5 * time.Second, + }, { + // Nested subquery. + // We saw that innermost rate takes 35 samples which is still the peak + // since the other two subqueries just duplicate the result. + Query: `rate(rate(bigmetric[10s:1s] @ 10)[100s:25s] @ 1000)[100s:20s] @ 2000`, + MaxSamples: 35, + Start: time.Unix(10, 0), + }, { + // Nested subquery. + // Now the outmost subquery produces more samples than inner most rate. + Query: `rate(rate(bigmetric[10s:1s] @ 10)[100s:25s] @ 1000)[17s:1s] @ 2000`, + MaxSamples: 36, + Start: time.Unix(10, 0), }, } engine := test.QueryEngine() for _, c := range cases { t.Run(c.Query, func(t *testing.T) { - var err error - var qry Query + testFunc := func(expError error) { + var err error + var qry Query + if c.Interval == 0 { + qry, err = engine.NewInstantQuery(test.Queryable(), c.Query, c.Start) + } else { + qry, err = engine.NewRangeQuery(test.Queryable(), c.Query, c.Start, c.End, c.Interval) + } + require.NoError(t, err) - engine.maxSamplesPerQuery = c.MaxSamples - - if c.Interval == 0 { - qry, err = engine.NewInstantQuery(test.Queryable(), c.Query, c.Start) - } else { - qry, err = engine.NewRangeQuery(test.Queryable(), c.Query, c.Start, c.End, c.Interval) + res := qry.Exec(test.Context()) + require.Equal(t, expError, res.Err) } - require.NoError(t, err) // Within limit. - res := qry.Exec(test.Context()) - require.NoError(t, res.Err) + engine.maxSamplesPerQuery = c.MaxSamples + testFunc(nil) // Exceeding limit. engine.maxSamplesPerQuery = c.MaxSamples - 1 - res = qry.Exec(test.Context()) - require.Equal(t, ErrTooManySamples(env), res.Err) + testFunc(ErrTooManySamples(env)) }) } } +func TestAtModifier(t *testing.T) { + test, err := NewTest(t, ` +load 10s + metric{job="1"} 0+1x1000 + metric{job="2"} 0+2x1000 + metric_topk{instance="1"} 0+1x1000 + metric_topk{instance="2"} 0+2x1000 + metric_topk{instance="3"} 1000-1x1000 + +load 1ms + metric_ms 0+1x10000 +`) + require.NoError(t, err) + defer test.Close() + + err = test.Run() + require.NoError(t, err) + + lbls1 := labels.FromStrings("__name__", "metric", "job", "1") + lbls2 := labels.FromStrings("__name__", "metric", "job", "2") + lblstopk2 := labels.FromStrings("__name__", "metric_topk", "instance", "2") + lblstopk3 := labels.FromStrings("__name__", "metric_topk", "instance", "3") + lblsms := labels.FromStrings("__name__", "metric_ms") + lblsneg := labels.FromStrings("__name__", "metric_neg") + + // Add some samples with negative timestamp. + db := test.TSDB() + app := db.Appender(context.Background()) + ref, err := app.Add(lblsneg, -1000000, 1000) + require.NoError(t, err) + for ts := int64(-1000000 + 1000); ts <= 0; ts += 1000 { + require.NoError(t, app.AddFast(ref, ts, -float64(ts/1000)+1)) + } + require.NoError(t, app.Commit()) + + cases := []struct { + query string + start, end, interval int64 // Time in seconds. + result parser.Value + }{ + { // Time of the result is the evaluation time. + query: `metric_neg @ 0`, + start: 100, + result: Vector{ + Sample{Point: Point{V: 1, T: 100000}, Metric: lblsneg}, + }, + }, { + query: `metric_neg @ -200`, + start: 100, + result: Vector{ + Sample{Point: Point{V: 201, T: 100000}, Metric: lblsneg}, + }, + }, { + query: `metric{job="2"} @ 50`, + start: -2, end: 2, interval: 1, + result: Matrix{ + Series{ + Points: []Point{{V: 10, T: -2000}, {V: 10, T: -1000}, {V: 10, T: 0}, {V: 10, T: 1000}, {V: 10, T: 2000}}, + Metric: lbls2, + }, + }, + }, { // Timestamps for matrix selector does not depend on the evaluation time. + query: "metric[20s] @ 300", + start: 10, + result: Matrix{ + Series{ + Points: []Point{{V: 28, T: 280000}, {V: 29, T: 290000}, {V: 30, T: 300000}}, + Metric: lbls1, + }, + Series{ + Points: []Point{{V: 56, T: 280000}, {V: 58, T: 290000}, {V: 60, T: 300000}}, + Metric: lbls2, + }, + }, + }, { + query: `metric_neg[2s] @ 0`, + start: 100, + result: Matrix{ + Series{ + Points: []Point{{V: 3, T: -2000}, {V: 2, T: -1000}, {V: 1, T: 0}}, + Metric: lblsneg, + }, + }, + }, { + query: `metric_neg[3s] @ -500`, + start: 100, + result: Matrix{ + Series{ + Points: []Point{{V: 504, T: -503000}, {V: 503, T: -502000}, {V: 502, T: -501000}, {V: 501, T: -500000}}, + Metric: lblsneg, + }, + }, + }, { + query: `metric_ms[3ms] @ 2.345`, + start: 100, + result: Matrix{ + Series{ + Points: []Point{{V: 2342, T: 2342}, {V: 2343, T: 2343}, {V: 2344, T: 2344}, {V: 2345, T: 2345}}, + Metric: lblsms, + }, + }, + }, { + query: "metric[100s:25s] @ 300", + start: 100, + result: Matrix{ + Series{ + Points: []Point{{V: 20, T: 200000}, {V: 22, T: 225000}, {V: 25, T: 250000}, {V: 27, T: 275000}, {V: 30, T: 300000}}, + Metric: lbls1, + }, + Series{ + Points: []Point{{V: 40, T: 200000}, {V: 44, T: 225000}, {V: 50, T: 250000}, {V: 54, T: 275000}, {V: 60, T: 300000}}, + Metric: lbls2, + }, + }, + }, { + query: "metric_neg[50s:25s] @ 0", + start: 100, + result: Matrix{ + Series{ + Points: []Point{{V: 51, T: -50000}, {V: 26, T: -25000}, {V: 1, T: 0}}, + Metric: lblsneg, + }, + }, + }, { + query: "metric_neg[50s:25s] @ -100", + start: 100, + result: Matrix{ + Series{ + Points: []Point{{V: 151, T: -150000}, {V: 126, T: -125000}, {V: 101, T: -100000}}, + Metric: lblsneg, + }, + }, + }, { + query: `metric_ms[100ms:25ms] @ 2.345`, + start: 100, + result: Matrix{ + Series{ + Points: []Point{{V: 2250, T: 2250}, {V: 2275, T: 2275}, {V: 2300, T: 2300}, {V: 2325, T: 2325}}, + Metric: lblsms, + }, + }, + }, { + query: `metric_topk and topk(1, sum_over_time(metric_topk[50s] @ 100))`, + start: 50, end: 80, interval: 10, + result: Matrix{ + Series{ + Points: []Point{{V: 995, T: 50000}, {V: 994, T: 60000}, {V: 993, T: 70000}, {V: 992, T: 80000}}, + Metric: lblstopk3, + }, + }, + }, { + query: `metric_topk and topk(1, sum_over_time(metric_topk[50s] @ 5000))`, + start: 50, end: 80, interval: 10, + result: Matrix{ + Series{ + Points: []Point{{V: 10, T: 50000}, {V: 12, T: 60000}, {V: 14, T: 70000}, {V: 16, T: 80000}}, + Metric: lblstopk2, + }, + }, + }, + } + + for _, c := range cases { + t.Run(c.query, func(t *testing.T) { + if c.interval == 0 { + c.interval = 1 + } + start, end, interval := time.Unix(c.start, 0), time.Unix(c.end, 0), time.Duration(c.interval)*time.Second + var err error + var qry Query + if c.end == 0 { + qry, err = test.QueryEngine().NewInstantQuery(test.Queryable(), c.query, start) + } else { + qry, err = test.QueryEngine().NewRangeQuery(test.Queryable(), c.query, start, end, interval) + } + require.NoError(t, err) + + res := qry.Exec(test.Context()) + require.NoError(t, res.Err) + require.Equal(t, c.result, res.Value, "query %q failed", c.query) + }) + } +} func TestRecoverEvaluatorRuntime(t *testing.T) { ev := &evaluator{logger: log.NewNopLogger()} @@ -1076,3 +1486,637 @@ func TestQueryLogger_error(t *testing.T) { require.Equal(t, f1.logs[i], field) } } + +func TestWrapWithStepInvariantExpr(t *testing.T) { + var testCases = []struct { + input string // The input to be parsed. + expected parser.Expr // The expected expression AST. + }{ + { + input: "123.4567", + expected: &parser.StepInvariantExpr{ + Expr: &parser.NumberLiteral{ + Val: 123.4567, + PosRange: parser.PositionRange{Start: 0, End: 8}, + }, + }, + }, { + input: `"foo"`, + expected: &parser.StepInvariantExpr{ + Expr: &parser.StringLiteral{ + Val: "foo", + PosRange: parser.PositionRange{Start: 0, End: 5}, + }, + }, + }, { + input: "foo * bar", + expected: &parser.BinaryExpr{ + Op: parser.MUL, + LHS: &parser.VectorSelector{ + Name: "foo", + LabelMatchers: []*labels.Matcher{ + parser.MustLabelMatcher(labels.MatchEqual, "__name__", "foo"), + }, + PosRange: parser.PositionRange{ + Start: 0, + End: 3, + }, + }, + RHS: &parser.VectorSelector{ + Name: "bar", + LabelMatchers: []*labels.Matcher{ + parser.MustLabelMatcher(labels.MatchEqual, "__name__", "bar"), + }, + PosRange: parser.PositionRange{ + Start: 6, + End: 9, + }, + }, + VectorMatching: &parser.VectorMatching{Card: parser.CardOneToOne}, + }, + }, { + input: "foo * bar @ 10", + expected: &parser.BinaryExpr{ + Op: parser.MUL, + LHS: &parser.VectorSelector{ + Name: "foo", + LabelMatchers: []*labels.Matcher{ + parser.MustLabelMatcher(labels.MatchEqual, "__name__", "foo"), + }, + PosRange: parser.PositionRange{ + Start: 0, + End: 3, + }, + }, + RHS: &parser.StepInvariantExpr{ + Expr: &parser.VectorSelector{ + Name: "bar", + LabelMatchers: []*labels.Matcher{ + parser.MustLabelMatcher(labels.MatchEqual, "__name__", "bar"), + }, + PosRange: parser.PositionRange{ + Start: 6, + End: 14, + }, + Timestamp: makeInt64Pointer(10000), + }, + }, + VectorMatching: &parser.VectorMatching{Card: parser.CardOneToOne}, + }, + }, { + input: "foo @ 20 * bar @ 10", + expected: &parser.StepInvariantExpr{ + Expr: &parser.BinaryExpr{ + Op: parser.MUL, + LHS: &parser.VectorSelector{ + Name: "foo", + LabelMatchers: []*labels.Matcher{ + parser.MustLabelMatcher(labels.MatchEqual, "__name__", "foo"), + }, + PosRange: parser.PositionRange{ + Start: 0, + End: 8, + }, + Timestamp: makeInt64Pointer(20000), + }, + RHS: &parser.VectorSelector{ + Name: "bar", + LabelMatchers: []*labels.Matcher{ + parser.MustLabelMatcher(labels.MatchEqual, "__name__", "bar"), + }, + PosRange: parser.PositionRange{ + Start: 11, + End: 19, + }, + Timestamp: makeInt64Pointer(10000), + }, + VectorMatching: &parser.VectorMatching{Card: parser.CardOneToOne}, + }, + }, + }, { + input: "test[5s]", + expected: &parser.MatrixSelector{ + VectorSelector: &parser.VectorSelector{ + Name: "test", + LabelMatchers: []*labels.Matcher{ + parser.MustLabelMatcher(labels.MatchEqual, "__name__", "test"), + }, + PosRange: parser.PositionRange{ + Start: 0, + End: 4, + }, + }, + Range: 5 * time.Second, + EndPos: 8, + }, + }, { + input: `test{a="b"}[5y] @ 1603774699`, + expected: &parser.StepInvariantExpr{ + Expr: &parser.MatrixSelector{ + VectorSelector: &parser.VectorSelector{ + Name: "test", + Timestamp: makeInt64Pointer(1603774699000), + LabelMatchers: []*labels.Matcher{ + parser.MustLabelMatcher(labels.MatchEqual, "a", "b"), + parser.MustLabelMatcher(labels.MatchEqual, "__name__", "test"), + }, + PosRange: parser.PositionRange{ + Start: 0, + End: 11, + }, + }, + Range: 5 * 365 * 24 * time.Hour, + EndPos: 28, + }, + }, + }, { + input: "sum by (foo)(some_metric)", + expected: &parser.AggregateExpr{ + Op: parser.SUM, + Expr: &parser.VectorSelector{ + Name: "some_metric", + LabelMatchers: []*labels.Matcher{ + parser.MustLabelMatcher(labels.MatchEqual, "__name__", "some_metric"), + }, + PosRange: parser.PositionRange{ + Start: 13, + End: 24, + }, + }, + Grouping: []string{"foo"}, + PosRange: parser.PositionRange{ + Start: 0, + End: 25, + }, + }, + }, { + input: "sum by (foo)(some_metric @ 10)", + expected: &parser.StepInvariantExpr{ + Expr: &parser.AggregateExpr{ + Op: parser.SUM, + Expr: &parser.VectorSelector{ + Name: "some_metric", + LabelMatchers: []*labels.Matcher{ + parser.MustLabelMatcher(labels.MatchEqual, "__name__", "some_metric"), + }, + PosRange: parser.PositionRange{ + Start: 13, + End: 29, + }, + Timestamp: makeInt64Pointer(10000), + }, + Grouping: []string{"foo"}, + PosRange: parser.PositionRange{ + Start: 0, + End: 30, + }, + }, + }, + }, { + input: "sum(some_metric1 @ 10) + sum(some_metric2 @ 20)", + expected: &parser.StepInvariantExpr{ + Expr: &parser.BinaryExpr{ + Op: parser.ADD, + VectorMatching: &parser.VectorMatching{}, + LHS: &parser.AggregateExpr{ + Op: parser.SUM, + Expr: &parser.VectorSelector{ + Name: "some_metric1", + LabelMatchers: []*labels.Matcher{ + parser.MustLabelMatcher(labels.MatchEqual, "__name__", "some_metric1"), + }, + PosRange: parser.PositionRange{ + Start: 4, + End: 21, + }, + Timestamp: makeInt64Pointer(10000), + }, + PosRange: parser.PositionRange{ + Start: 0, + End: 22, + }, + }, + RHS: &parser.AggregateExpr{ + Op: parser.SUM, + Expr: &parser.VectorSelector{ + Name: "some_metric2", + LabelMatchers: []*labels.Matcher{ + parser.MustLabelMatcher(labels.MatchEqual, "__name__", "some_metric2"), + }, + PosRange: parser.PositionRange{ + Start: 29, + End: 46, + }, + Timestamp: makeInt64Pointer(20000), + }, + PosRange: parser.PositionRange{ + Start: 25, + End: 47, + }, + }, + }, + }, + }, { + input: "some_metric and topk(5, rate(some_metric[1m] @ 20))", + expected: &parser.BinaryExpr{ + Op: parser.LAND, + VectorMatching: &parser.VectorMatching{ + Card: parser.CardManyToMany, + }, + LHS: &parser.VectorSelector{ + Name: "some_metric", + LabelMatchers: []*labels.Matcher{ + parser.MustLabelMatcher(labels.MatchEqual, "__name__", "some_metric"), + }, + PosRange: parser.PositionRange{ + Start: 0, + End: 11, + }, + }, + RHS: &parser.StepInvariantExpr{ + Expr: &parser.AggregateExpr{ + Op: parser.TOPK, + Expr: &parser.Call{ + Func: parser.MustGetFunction("rate"), + Args: parser.Expressions{ + &parser.MatrixSelector{ + VectorSelector: &parser.VectorSelector{ + Name: "some_metric", + LabelMatchers: []*labels.Matcher{ + parser.MustLabelMatcher(labels.MatchEqual, "__name__", "some_metric"), + }, + PosRange: parser.PositionRange{ + Start: 29, + End: 40, + }, + Timestamp: makeInt64Pointer(20000), + }, + Range: 1 * time.Minute, + EndPos: 49, + }, + }, + PosRange: parser.PositionRange{ + Start: 24, + End: 50, + }, + }, + Param: &parser.NumberLiteral{ + Val: 5, + PosRange: parser.PositionRange{ + Start: 21, + End: 22, + }, + }, + PosRange: parser.PositionRange{ + Start: 16, + End: 51, + }, + }, + }, + }, + }, { + input: "time()", + expected: &parser.Call{ + Func: parser.MustGetFunction("time"), + Args: parser.Expressions{}, + PosRange: parser.PositionRange{ + Start: 0, + End: 6, + }, + }, + }, { + input: `foo{bar="baz"}[10m:6s]`, + expected: &parser.SubqueryExpr{ + Expr: &parser.VectorSelector{ + Name: "foo", + LabelMatchers: []*labels.Matcher{ + parser.MustLabelMatcher(labels.MatchEqual, "bar", "baz"), + parser.MustLabelMatcher(labels.MatchEqual, "__name__", "foo"), + }, + PosRange: parser.PositionRange{ + Start: 0, + End: 14, + }, + }, + Range: 10 * time.Minute, + Step: 6 * time.Second, + EndPos: 22, + }, + }, { + input: `foo{bar="baz"}[10m:6s] @ 10`, + expected: &parser.StepInvariantExpr{ + Expr: &parser.SubqueryExpr{ + Expr: &parser.VectorSelector{ + Name: "foo", + LabelMatchers: []*labels.Matcher{ + parser.MustLabelMatcher(labels.MatchEqual, "bar", "baz"), + parser.MustLabelMatcher(labels.MatchEqual, "__name__", "foo"), + }, + PosRange: parser.PositionRange{ + Start: 0, + End: 14, + }, + }, + Range: 10 * time.Minute, + Step: 6 * time.Second, + Timestamp: makeInt64Pointer(10000), + EndPos: 27, + }, + }, + }, { // Even though the subquery is step invariant, the inside is also wrapped separately. + input: `sum(foo{bar="baz"} @ 20)[10m:6s] @ 10`, + expected: &parser.StepInvariantExpr{ + Expr: &parser.SubqueryExpr{ + Expr: &parser.StepInvariantExpr{ + Expr: &parser.AggregateExpr{ + Op: parser.SUM, + Expr: &parser.VectorSelector{ + Name: "foo", + LabelMatchers: []*labels.Matcher{ + parser.MustLabelMatcher(labels.MatchEqual, "bar", "baz"), + parser.MustLabelMatcher(labels.MatchEqual, "__name__", "foo"), + }, + PosRange: parser.PositionRange{ + Start: 4, + End: 23, + }, + Timestamp: makeInt64Pointer(20000), + }, + PosRange: parser.PositionRange{ + Start: 0, + End: 24, + }, + }, + }, + Range: 10 * time.Minute, + Step: 6 * time.Second, + Timestamp: makeInt64Pointer(10000), + EndPos: 37, + }, + }, + }, { + input: `min_over_time(rate(foo{bar="baz"}[2s])[5m:] @ 1603775091)[4m:3s]`, + expected: &parser.SubqueryExpr{ + Expr: &parser.StepInvariantExpr{ + Expr: &parser.Call{ + Func: parser.MustGetFunction("min_over_time"), + Args: parser.Expressions{ + &parser.SubqueryExpr{ + Expr: &parser.Call{ + Func: parser.MustGetFunction("rate"), + Args: parser.Expressions{ + &parser.MatrixSelector{ + VectorSelector: &parser.VectorSelector{ + Name: "foo", + LabelMatchers: []*labels.Matcher{ + parser.MustLabelMatcher(labels.MatchEqual, "bar", "baz"), + parser.MustLabelMatcher(labels.MatchEqual, "__name__", "foo"), + }, + PosRange: parser.PositionRange{ + Start: 19, + End: 33, + }, + }, + Range: 2 * time.Second, + EndPos: 37, + }, + }, + PosRange: parser.PositionRange{ + Start: 14, + End: 38, + }, + }, + Range: 5 * time.Minute, + Timestamp: makeInt64Pointer(1603775091000), + EndPos: 56, + }, + }, + PosRange: parser.PositionRange{ + Start: 0, + End: 57, + }, + }, + }, + Range: 4 * time.Minute, + Step: 3 * time.Second, + EndPos: 64, + }, + }, { + input: `some_metric @ 123 offset 1m [10m:5s]`, + expected: &parser.SubqueryExpr{ + Expr: &parser.StepInvariantExpr{ + Expr: &parser.VectorSelector{ + Name: "some_metric", + LabelMatchers: []*labels.Matcher{ + parser.MustLabelMatcher(labels.MatchEqual, "__name__", "some_metric"), + }, + PosRange: parser.PositionRange{ + Start: 0, + End: 27, + }, + Timestamp: makeInt64Pointer(123000), + OriginalOffset: 1 * time.Minute, + }, + }, + Range: 10 * time.Minute, + Step: 5 * time.Second, + EndPos: 36, + }, + }, { + input: `some_metric[10m:5s] offset 1m @ 123`, + expected: &parser.StepInvariantExpr{ + Expr: &parser.SubqueryExpr{ + Expr: &parser.VectorSelector{ + Name: "some_metric", + LabelMatchers: []*labels.Matcher{ + parser.MustLabelMatcher(labels.MatchEqual, "__name__", "some_metric"), + }, + PosRange: parser.PositionRange{ + Start: 0, + End: 11, + }, + }, + Timestamp: makeInt64Pointer(123000), + OriginalOffset: 1 * time.Minute, + Range: 10 * time.Minute, + Step: 5 * time.Second, + EndPos: 35, + }, + }, + }, { + input: `(foo + bar{nm="val"} @ 1234)[5m:] @ 1603775019`, + expected: &parser.StepInvariantExpr{ + Expr: &parser.SubqueryExpr{ + Expr: &parser.ParenExpr{ + Expr: &parser.BinaryExpr{ + Op: parser.ADD, + VectorMatching: &parser.VectorMatching{ + Card: parser.CardOneToOne, + }, + LHS: &parser.VectorSelector{ + Name: "foo", + LabelMatchers: []*labels.Matcher{ + parser.MustLabelMatcher(labels.MatchEqual, "__name__", "foo"), + }, + PosRange: parser.PositionRange{ + Start: 1, + End: 4, + }, + }, + RHS: &parser.StepInvariantExpr{ + Expr: &parser.VectorSelector{ + Name: "bar", + LabelMatchers: []*labels.Matcher{ + parser.MustLabelMatcher(labels.MatchEqual, "nm", "val"), + parser.MustLabelMatcher(labels.MatchEqual, "__name__", "bar"), + }, + Timestamp: makeInt64Pointer(1234000), + PosRange: parser.PositionRange{ + Start: 7, + End: 27, + }, + }, + }, + }, + PosRange: parser.PositionRange{ + Start: 0, + End: 28, + }, + }, + Range: 5 * time.Minute, + Timestamp: makeInt64Pointer(1603775019000), + EndPos: 46, + }, + }, + }, { + input: "abs(abs(metric @ 10))", + expected: &parser.StepInvariantExpr{ + Expr: &parser.Call{ + Func: &parser.Function{ + Name: "abs", + ArgTypes: []parser.ValueType{parser.ValueTypeVector}, + ReturnType: parser.ValueTypeVector, + }, + Args: parser.Expressions{&parser.Call{ + Func: &parser.Function{ + Name: "abs", + ArgTypes: []parser.ValueType{parser.ValueTypeVector}, + ReturnType: parser.ValueTypeVector, + }, + Args: parser.Expressions{&parser.VectorSelector{ + Name: "metric", + LabelMatchers: []*labels.Matcher{ + parser.MustLabelMatcher(labels.MatchEqual, "__name__", "metric"), + }, + PosRange: parser.PositionRange{ + Start: 8, + End: 19, + }, + Timestamp: makeInt64Pointer(10000), + }}, + PosRange: parser.PositionRange{ + Start: 4, + End: 20, + }, + }}, + PosRange: parser.PositionRange{ + Start: 0, + End: 21, + }, + }, + }, + }, { + input: "sum(sum(some_metric1 @ 10) + sum(some_metric2 @ 20))", + expected: &parser.StepInvariantExpr{ + Expr: &parser.AggregateExpr{ + Op: parser.SUM, + Expr: &parser.BinaryExpr{ + Op: parser.ADD, + VectorMatching: &parser.VectorMatching{}, + LHS: &parser.AggregateExpr{ + Op: parser.SUM, + Expr: &parser.VectorSelector{ + Name: "some_metric1", + LabelMatchers: []*labels.Matcher{ + parser.MustLabelMatcher(labels.MatchEqual, "__name__", "some_metric1"), + }, + PosRange: parser.PositionRange{ + Start: 8, + End: 25, + }, + Timestamp: makeInt64Pointer(10000), + }, + PosRange: parser.PositionRange{ + Start: 4, + End: 26, + }, + }, + RHS: &parser.AggregateExpr{ + Op: parser.SUM, + Expr: &parser.VectorSelector{ + Name: "some_metric2", + LabelMatchers: []*labels.Matcher{ + parser.MustLabelMatcher(labels.MatchEqual, "__name__", "some_metric2"), + }, + PosRange: parser.PositionRange{ + Start: 33, + End: 50, + }, + Timestamp: makeInt64Pointer(20000), + }, + PosRange: parser.PositionRange{ + Start: 29, + End: 52, + }, + }, + }, + PosRange: parser.PositionRange{ + Start: 0, + End: 52, + }, + }, + }, + }, + } + + for _, test := range testCases { + t.Run(test.input, func(t *testing.T) { + expr, err := parser.ParseExpr(test.input) + require.NoError(t, err) + expr = WrapWithStepInvariantExpr(expr) + require.Equal(t, test.expected, expr, "error on input '%s'", test.input) + }) + } +} + +func TestEngineOptsValidation(t *testing.T) { + cases := []struct { + opts EngineOpts + query string + fail bool + expError string + }{ + { + opts: EngineOpts{EnableAtModifier: false}, + query: "metric @ 100", + fail: true, + expError: "@ modifier is disabled", + }, { + opts: EngineOpts{EnableAtModifier: true}, + query: "metric @ 100", + }, + } + + for _, c := range cases { + eng := NewEngine(c.opts) + _, err1 := eng.NewInstantQuery(nil, c.query, time.Unix(10, 0)) + _, err2 := eng.NewRangeQuery(nil, c.query, time.Unix(0, 0), time.Unix(10, 0), time.Second) + if c.fail { + require.Equal(t, c.expError, err1.Error()) + require.Equal(t, c.expError, err2.Error()) + } else { + require.Nil(t, err1) + require.Nil(t, err2) + } + } +} diff --git a/promql/functions.go b/promql/functions.go index 0499e8f15c..3a96a9ecec 100644 --- a/promql/functions.go +++ b/promql/functions.go @@ -59,7 +59,6 @@ func funcTime(vals []parser.Value, args parser.Expressions, enh *EvalNodeHelper) func extrapolatedRate(vals []parser.Value, args parser.Expressions, enh *EvalNodeHelper, isCounter bool, isRate bool) Vector { ms := args[0].(*parser.MatrixSelector) vs := ms.VectorSelector.(*parser.VectorSelector) - var ( samples = vals[0].(Matrix)[0] rangeStart = enh.Ts - durationMilliseconds(ms.Range+vs.Offset) @@ -598,7 +597,6 @@ func funcDeriv(vals []parser.Value, args parser.Expressions, enh *EvalNodeHelper func funcPredictLinear(vals []parser.Value, args parser.Expressions, enh *EvalNodeHelper) Vector { samples := vals[0].(Matrix)[0] duration := vals[1].(Vector)[0].V - // No sense in trying to predict anything without at least two points. // Drop this Vector element. if len(samples.Points) < 2 { @@ -701,10 +699,10 @@ func funcChanges(vals []parser.Value, args parser.Expressions, enh *EvalNodeHelp func funcLabelReplace(vals []parser.Value, args parser.Expressions, enh *EvalNodeHelper) Vector { var ( vector = vals[0].(Vector) - dst = args[1].(*parser.StringLiteral).Val - repl = args[2].(*parser.StringLiteral).Val - src = args[3].(*parser.StringLiteral).Val - regexStr = args[4].(*parser.StringLiteral).Val + dst = stringFromArg(args[1]) + repl = stringFromArg(args[2]) + src = stringFromArg(args[3]) + regexStr = stringFromArg(args[4]) ) if enh.regex == nil { @@ -764,8 +762,8 @@ func funcVector(vals []parser.Value, args parser.Expressions, enh *EvalNodeHelpe func funcLabelJoin(vals []parser.Value, args parser.Expressions, enh *EvalNodeHelper) Vector { var ( vector = vals[0].(Vector) - dst = args[1].(*parser.StringLiteral).Val - sep = args[2].(*parser.StringLiteral).Val + dst = stringFromArg(args[1]) + sep = stringFromArg(args[2]) srcLabels = make([]string, len(args)-3) ) @@ -774,7 +772,7 @@ func funcLabelJoin(vals []parser.Value, args parser.Expressions, enh *EvalNodeHe } for i := 3; i < len(args); i++ { - src := args[i].(*parser.StringLiteral).Val + src := stringFromArg(args[i]) if !model.LabelName(src).IsValid() { panic(errors.Errorf("invalid source label name in label_join(): %s", src)) } @@ -938,6 +936,21 @@ var FunctionCalls = map[string]FunctionCall{ "year": funcYear, } +// AtModifierUnsafeFunctions are the functions whose result +// can vary if evaluation time is changed when the arguments are +// step invariant. It also includes functions that use the timestamps +// of the passed instant vector argument to calculate a result since +// that can also change with change in eval time. +var AtModifierUnsafeFunctions = map[string]struct{}{ + // Step invariant functions. + "days_in_month": {}, "day_of_month": {}, "day_of_week": {}, + "hour": {}, "minute": {}, "month": {}, "year": {}, + "predict_linear": {}, "time": {}, + // Uses timestamp of the argument for the result, + // hence unsafe to use with @ modifier. + "timestamp": {}, +} + type vectorByValueHeap Vector func (s vectorByValueHeap) Len() int { @@ -1028,3 +1041,7 @@ func createLabelsForAbsentFunction(expr parser.Expr) labels.Labels { } return m } + +func stringFromArg(e parser.Expr) string { + return unwrapStepInvariantExpr(e).(*parser.StringLiteral).Val +} diff --git a/promql/parser/ast.go b/promql/parser/ast.go index de82d67254..459f1f47f5 100644 --- a/promql/parser/ast.go +++ b/promql/parser/ast.go @@ -125,10 +125,17 @@ type MatrixSelector struct { // SubqueryExpr represents a subquery. type SubqueryExpr struct { - Expr Expr - Range time.Duration - Offset time.Duration - Step time.Duration + Expr Expr + Range time.Duration + // OriginalOffset is the actual offset that was set in the query. + // This never changes. + OriginalOffset time.Duration + // 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. + Offset time.Duration + Timestamp *int64 + Step time.Duration EndPos Pos } @@ -162,10 +169,28 @@ type UnaryExpr struct { StartPos Pos } +// StepInvariantExpr represents a query which evaluates to the same result +// irrespective of the evaluation time given the raw samples from TSDB remain unchanged. +// Currently this is only used for engine optimisations and the parser does not produce this. +type StepInvariantExpr struct { + Expr Expr +} + +func (e *StepInvariantExpr) String() string { return e.Expr.String() } + +func (e *StepInvariantExpr) PositionRange() PositionRange { return e.Expr.PositionRange() } + // VectorSelector represents a Vector selection. type VectorSelector struct { - Name string + Name string + // OriginalOffset is the actual offset that was set in the query. + // This never changes. + OriginalOffset time.Duration + // 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. Offset time.Duration + Timestamp *int64 LabelMatchers []*labels.Matcher // The unexpanded seriesSet populated at query preparation time. @@ -203,17 +228,19 @@ func (e *BinaryExpr) Type() ValueType { } return ValueTypeVector } +func (e *StepInvariantExpr) Type() ValueType { return e.Expr.Type() } -func (*AggregateExpr) PromQLExpr() {} -func (*BinaryExpr) PromQLExpr() {} -func (*Call) PromQLExpr() {} -func (*MatrixSelector) PromQLExpr() {} -func (*SubqueryExpr) PromQLExpr() {} -func (*NumberLiteral) PromQLExpr() {} -func (*ParenExpr) PromQLExpr() {} -func (*StringLiteral) PromQLExpr() {} -func (*UnaryExpr) PromQLExpr() {} -func (*VectorSelector) PromQLExpr() {} +func (*AggregateExpr) PromQLExpr() {} +func (*BinaryExpr) PromQLExpr() {} +func (*Call) PromQLExpr() {} +func (*MatrixSelector) PromQLExpr() {} +func (*SubqueryExpr) PromQLExpr() {} +func (*NumberLiteral) PromQLExpr() {} +func (*ParenExpr) PromQLExpr() {} +func (*StringLiteral) PromQLExpr() {} +func (*UnaryExpr) PromQLExpr() {} +func (*VectorSelector) PromQLExpr() {} +func (*StepInvariantExpr) PromQLExpr() {} // VectorMatchCardinality describes the cardinality relationship // of two Vectors in a binary operation. @@ -347,6 +374,8 @@ func Children(node Node) []Node { return []Node{n.Expr} case *MatrixSelector: return []Node{n.VectorSelector} + case *StepInvariantExpr: + return []Node{n.Expr} case *NumberLiteral, *StringLiteral, *VectorSelector: // nothing to do return []Node{} diff --git a/promql/parser/generated_parser.y b/promql/parser/generated_parser.y index f0bdc320fc..7964bf35f8 100644 --- a/promql/parser/generated_parser.y +++ b/promql/parser/generated_parser.y @@ -83,6 +83,7 @@ NEQ NEQ_REGEX POW SUB +AT %token operatorsEnd // Aggregators. @@ -137,8 +138,8 @@ START_METRIC_SELECTOR %type grouping_label_list grouping_labels maybe_grouping_labels %type series_item series_values %type uint -%type number series_value signed_number -%type 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_literal offset_expr on_or_ignoring paren_expr string_literal subquery_expr unary_expr vector_selector +%type number series_value signed_number signed_or_unsigned_number +%type 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_literal offset_expr on_or_ignoring paren_expr string_literal subquery_expr unary_expr vector_selector %type duration maybe_duration %start start @@ -187,6 +188,7 @@ expr : | subquery_expr | unary_expr | vector_selector + | step_invariant_expr ; /* @@ -200,8 +202,8 @@ aggregate_expr : aggregate_op aggregate_modifier function_call_body | aggregate_op function_call_body { $$ = yylex.(*parser).newAggregateExpr($1, &AggregateExpr{}, $2) } | aggregate_op error - { - yylex.(*parser).unexpected("aggregation",""); + { + yylex.(*parser).unexpected("aggregation",""); $$ = yylex.(*parser).newAggregateExpr($1, &AggregateExpr{}, Expressions{}) } ; @@ -380,6 +382,19 @@ offset_expr: expr OFFSET duration | expr OFFSET error { yylex.(*parser).unexpected("offset", "duration"); $$ = $1 } ; +/* + * @ modifiers. + */ + +step_invariant_expr: expr AT signed_or_unsigned_number + { + yylex.(*parser).setTimestamp($1, $3) + $$ = $1 + } + + | expr AT error + { yylex.(*parser).unexpected("@", "timestamp"); $$ = $1 } + ; /* * Subquery and range selectors. @@ -391,8 +406,10 @@ matrix_selector : expr LEFT_BRACKET duration RIGHT_BRACKET vs, ok := $1.(*VectorSelector) if !ok{ errMsg = "ranges only allowed for vector selectors" - } else if vs.Offset != 0{ + } else if vs.OriginalOffset != 0{ errMsg = "no offset modifiers allowed before range" + } else if vs.Timestamp != nil { + errMsg = "no @ modifiers allowed before range" } if errMsg != ""{ @@ -664,6 +681,8 @@ signed_number : ADD number { $$ = $2 } | SUB number { $$ = -$2 } ; +signed_or_unsigned_number: number | signed_number ; + uint : NUMBER { var err error diff --git a/promql/parser/generated_parser.y.go b/promql/parser/generated_parser.y.go index 4117f0e715..8d6f8b64de 100644 --- a/promql/parser/generated_parser.y.go +++ b/promql/parser/generated_parser.y.go @@ -3,8 +3,11 @@ //line generated_parser.y:15 package parser +import __yyfmt__ "fmt" + +//line generated_parser.y:15 + import ( - __yyfmt__ "fmt" "math" "sort" "strconv" @@ -12,7 +15,7 @@ import ( "github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/pkg/value" -) //line generated_parser.y:15 +) //line generated_parser.y:28 type yySymType struct { @@ -69,37 +72,38 @@ const NEQ = 57381 const NEQ_REGEX = 57382 const POW = 57383 const SUB = 57384 -const operatorsEnd = 57385 -const aggregatorsStart = 57386 -const AVG = 57387 -const BOTTOMK = 57388 -const COUNT = 57389 -const COUNT_VALUES = 57390 -const GROUP = 57391 -const MAX = 57392 -const MIN = 57393 -const QUANTILE = 57394 -const STDDEV = 57395 -const STDVAR = 57396 -const SUM = 57397 -const TOPK = 57398 -const aggregatorsEnd = 57399 -const keywordsStart = 57400 -const BOOL = 57401 -const BY = 57402 -const GROUP_LEFT = 57403 -const GROUP_RIGHT = 57404 -const IGNORING = 57405 -const OFFSET = 57406 -const ON = 57407 -const WITHOUT = 57408 -const keywordsEnd = 57409 -const startSymbolsStart = 57410 -const START_METRIC = 57411 -const START_SERIES_DESCRIPTION = 57412 -const START_EXPRESSION = 57413 -const START_METRIC_SELECTOR = 57414 -const startSymbolsEnd = 57415 +const AT = 57385 +const operatorsEnd = 57386 +const aggregatorsStart = 57387 +const AVG = 57388 +const BOTTOMK = 57389 +const COUNT = 57390 +const COUNT_VALUES = 57391 +const GROUP = 57392 +const MAX = 57393 +const MIN = 57394 +const QUANTILE = 57395 +const STDDEV = 57396 +const STDVAR = 57397 +const SUM = 57398 +const TOPK = 57399 +const aggregatorsEnd = 57400 +const keywordsStart = 57401 +const BOOL = 57402 +const BY = 57403 +const GROUP_LEFT = 57404 +const GROUP_RIGHT = 57405 +const IGNORING = 57406 +const OFFSET = 57407 +const ON = 57408 +const WITHOUT = 57409 +const keywordsEnd = 57410 +const startSymbolsStart = 57411 +const START_METRIC = 57412 +const START_SERIES_DESCRIPTION = 57413 +const START_EXPRESSION = 57414 +const START_METRIC_SELECTOR = 57415 +const startSymbolsEnd = 57416 var yyToknames = [...]string{ "$end", @@ -144,6 +148,7 @@ var yyToknames = [...]string{ "NEQ_REGEX", "POW", "SUB", + "AT", "operatorsEnd", "aggregatorsStart", "AVG", @@ -182,7 +187,7 @@ const yyEofCode = 1 const yyErrCode = 2 const yyInitialStackSize = 16 -//line generated_parser.y:710 +//line generated_parser.y:729 //line yacctab:1 var yyExca = [...]int{ @@ -190,349 +195,357 @@ var yyExca = [...]int{ 1, -1, -2, 0, -1, 33, - 1, 121, - 10, 121, - 22, 121, + 1, 124, + 10, 124, + 22, 124, -2, 0, - -1, 55, - 2, 133, - 15, 133, - 60, 133, - 66, 133, - -2, 89, -1, 56, - 2, 134, - 15, 134, - 60, 134, - 66, 134, - -2, 90, - -1, 57, - 2, 135, - 15, 135, - 60, 135, - 66, 135, - -2, 92, - -1, 58, 2, 136, 15, 136, - 60, 136, - 66, 136, - -2, 93, - -1, 59, + 61, 136, + 67, 136, + -2, 92, + -1, 57, 2, 137, 15, 137, - 60, 137, - 66, 137, - -2, 94, - -1, 60, + 61, 137, + 67, 137, + -2, 93, + -1, 58, 2, 138, 15, 138, - 60, 138, - 66, 138, - -2, 99, - -1, 61, + 61, 138, + 67, 138, + -2, 95, + -1, 59, 2, 139, 15, 139, - 60, 139, - 66, 139, - -2, 101, - -1, 62, + 61, 139, + 67, 139, + -2, 96, + -1, 60, 2, 140, 15, 140, - 60, 140, - 66, 140, - -2, 103, - -1, 63, + 61, 140, + 67, 140, + -2, 97, + -1, 61, 2, 141, 15, 141, - 60, 141, - 66, 141, - -2, 104, - -1, 64, + 61, 141, + 67, 141, + -2, 102, + -1, 62, 2, 142, 15, 142, - 60, 142, - 66, 142, - -2, 105, - -1, 65, + 61, 142, + 67, 142, + -2, 104, + -1, 63, 2, 143, 15, 143, - 60, 143, - 66, 143, + 61, 143, + 67, 143, -2, 106, - -1, 66, + -1, 64, 2, 144, 15, 144, - 60, 144, - 66, 144, + 61, 144, + 67, 144, -2, 107, - -1, 176, - 12, 184, - 13, 184, - 16, 184, - 17, 184, - 23, 184, - 26, 184, - 32, 184, - 33, 184, - 36, 184, - 42, 184, - 45, 184, - 46, 184, - 47, 184, - 48, 184, - 49, 184, - 50, 184, - 51, 184, - 52, 184, - 53, 184, - 54, 184, - 55, 184, - 56, 184, - 60, 184, - 64, 184, - 66, 184, + -1, 65, + 2, 145, + 15, 145, + 61, 145, + 67, 145, + -2, 108, + -1, 66, + 2, 146, + 15, 146, + 61, 146, + 67, 146, + -2, 109, + -1, 67, + 2, 147, + 15, 147, + 61, 147, + 67, 147, + -2, 110, + -1, 182, + 12, 189, + 13, 189, + 16, 189, + 17, 189, + 23, 189, + 26, 189, + 32, 189, + 33, 189, + 36, 189, + 42, 189, + 46, 189, + 47, 189, + 48, 189, + 49, 189, + 50, 189, + 51, 189, + 52, 189, + 53, 189, + 54, 189, + 55, 189, + 56, 189, + 57, 189, + 61, 189, + 65, 189, + 67, 189, -2, 0, - -1, 177, - 12, 184, - 13, 184, - 16, 184, - 17, 184, - 23, 184, - 26, 184, - 32, 184, - 33, 184, - 36, 184, - 42, 184, - 45, 184, - 46, 184, - 47, 184, - 48, 184, - 49, 184, - 50, 184, - 51, 184, - 52, 184, - 53, 184, - 54, 184, - 55, 184, - 56, 184, - 60, 184, - 64, 184, - 66, 184, + -1, 183, + 12, 189, + 13, 189, + 16, 189, + 17, 189, + 23, 189, + 26, 189, + 32, 189, + 33, 189, + 36, 189, + 42, 189, + 46, 189, + 47, 189, + 48, 189, + 49, 189, + 50, 189, + 51, 189, + 52, 189, + 53, 189, + 54, 189, + 55, 189, + 56, 189, + 57, 189, + 61, 189, + 65, 189, + 67, 189, -2, 0, - -1, 193, - 19, 182, + -1, 201, + 19, 187, -2, 0, - -1, 241, - 19, 183, + -1, 247, + 19, 188, -2, 0, } const yyPrivate = 57344 -const yyLast = 598 +const yyLast = 629 var yyAct = [...]int{ - 247, 197, 35, 136, 237, 238, 168, 169, 108, 74, - 97, 96, 99, 174, 121, 175, 98, 250, 100, 176, - 177, 230, 95, 54, 231, 229, 171, 48, 69, 101, - 50, 22, 49, 163, 245, 148, 251, 225, 51, 244, - 116, 67, 172, 6, 248, 170, 228, 18, 19, 92, - 224, 20, 243, 103, 162, 104, 69, 68, 117, 102, - 55, 56, 57, 58, 59, 60, 61, 62, 63, 64, - 65, 66, 94, 95, 99, 13, 101, 105, 31, 24, - 100, 30, 7, 252, 8, 79, 80, 81, 33, 82, - 83, 84, 85, 86, 87, 88, 89, 90, 91, 139, - 92, 93, 145, 78, 149, 143, 146, 141, 110, 142, - 2, 3, 4, 5, 242, 144, 32, 115, 109, 114, - 173, 138, 161, 94, 226, 178, 179, 180, 181, 182, - 183, 184, 185, 186, 187, 188, 189, 190, 191, 122, - 123, 124, 125, 126, 127, 128, 129, 130, 131, 132, - 133, 134, 135, 153, 46, 140, 10, 137, 152, 1, - 70, 227, 138, 155, 138, 156, 71, 240, 45, 151, - 34, 95, 48, 69, 232, 50, 22, 49, 233, 234, - 235, 236, 239, 51, 80, 53, 67, 194, 9, 9, - 158, 193, 18, 19, 89, 90, 20, 241, 92, 44, - 157, 159, 68, 43, 192, 55, 56, 57, 58, 59, - 60, 61, 62, 63, 64, 65, 66, 42, 165, 76, - 13, 94, 120, 41, 24, 167, 30, 40, 246, 75, - 170, 39, 249, 48, 69, 160, 50, 22, 49, 171, - 113, 118, 110, 147, 51, 112, 254, 67, 38, 76, - 119, 255, 109, 18, 19, 172, 111, 20, 107, 75, - 37, 36, 166, 68, 77, 73, 55, 56, 57, 58, - 59, 60, 61, 62, 63, 64, 65, 66, 199, 164, - 195, 13, 72, 52, 198, 24, 154, 30, 209, 47, - 106, 0, 215, 0, 0, 0, 253, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 211, 212, - 0, 0, 213, 0, 0, 0, 0, 0, 0, 0, - 0, 200, 202, 204, 205, 206, 214, 216, 219, 220, - 221, 222, 223, 199, 0, 201, 203, 207, 208, 210, - 217, 218, 0, 209, 0, 0, 0, 215, 0, 0, - 0, 196, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 211, 212, 0, 0, 213, 0, 0, - 0, 0, 0, 0, 0, 0, 200, 202, 204, 205, - 206, 214, 216, 219, 220, 221, 222, 223, 0, 0, - 201, 203, 207, 208, 210, 217, 218, 17, 69, 0, - 0, 22, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 18, 19, 0, - 0, 20, 17, 31, 0, 0, 22, 0, 0, 0, - 11, 12, 14, 15, 16, 21, 23, 25, 26, 27, - 28, 29, 18, 19, 0, 13, 20, 0, 0, 24, - 0, 30, 0, 0, 0, 11, 12, 14, 15, 16, - 21, 23, 25, 26, 27, 28, 29, 95, 0, 0, - 13, 0, 0, 150, 24, 0, 30, 0, 0, 79, - 80, 81, 0, 82, 83, 84, 85, 86, 87, 88, - 89, 90, 91, 0, 92, 93, 0, 0, 95, 0, + 253, 35, 205, 138, 244, 243, 144, 110, 75, 99, + 98, 180, 101, 181, 182, 183, 6, 100, 102, 123, + 55, 256, 238, 143, 237, 172, 118, 97, 49, 70, + 103, 51, 22, 50, 233, 103, 157, 257, 254, 52, + 81, 145, 68, 70, 154, 236, 171, 232, 18, 19, + 90, 91, 20, 105, 93, 106, 96, 153, 69, 104, + 31, 119, 56, 57, 58, 59, 60, 61, 62, 63, + 64, 65, 66, 67, 251, 107, 101, 13, 95, 250, + 142, 24, 102, 30, 2, 3, 4, 5, 258, 248, + 97, 239, 249, 146, 202, 145, 140, 79, 201, 7, + 32, 148, 80, 81, 146, 158, 152, 155, 150, 147, + 151, 200, 8, 90, 91, 149, 33, 93, 94, 96, + 147, 179, 140, 170, 1, 178, 184, 185, 186, 187, + 188, 189, 190, 191, 192, 193, 194, 195, 196, 197, + 246, 95, 177, 124, 125, 126, 127, 128, 129, 130, + 131, 132, 133, 134, 135, 136, 137, 164, 139, 165, + 46, 45, 162, 10, 44, 140, 71, 161, 43, 235, + 198, 199, 97, 72, 34, 42, 49, 70, 160, 51, + 22, 50, 240, 122, 167, 241, 242, 52, 112, 245, + 68, 41, 77, 112, 166, 168, 18, 19, 111, 93, + 20, 96, 76, 111, 234, 247, 69, 40, 169, 109, + 56, 57, 58, 59, 60, 61, 62, 63, 64, 65, + 66, 67, 117, 95, 116, 13, 39, 120, 156, 24, + 38, 30, 121, 54, 77, 252, 9, 9, 49, 70, + 255, 51, 22, 50, 76, 37, 36, 47, 174, 52, + 74, 115, 68, 260, 141, 176, 114, 261, 18, 19, + 145, 175, 20, 78, 173, 203, 73, 113, 69, 146, + 53, 206, 56, 57, 58, 59, 60, 61, 62, 63, + 64, 65, 66, 67, 207, 147, 163, 13, 48, 108, + 0, 24, 0, 30, 217, 0, 0, 0, 223, 0, + 0, 0, 259, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 219, 220, 0, 0, 221, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 208, 210, + 212, 213, 214, 222, 224, 227, 228, 229, 230, 231, + 207, 0, 209, 211, 215, 216, 218, 225, 226, 0, + 217, 0, 0, 0, 223, 0, 0, 0, 204, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 79, 80, 81, 0, 82, 83, 84, 94, 86, 87, - 88, 89, 90, 91, 0, 92, 93, 0, 0, 95, + 219, 220, 0, 0, 221, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 208, 210, 212, 213, 214, 222, + 224, 227, 228, 229, 230, 231, 0, 0, 209, 211, + 215, 216, 218, 225, 226, 17, 70, 0, 0, 22, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 79, 80, 81, 0, 82, 83, 95, 94, 86, - 87, 0, 89, 90, 91, 0, 92, 93, 0, 79, - 80, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 89, 90, 0, 0, 92, 93, 0, 0, 0, 94, + 0, 0, 0, 0, 0, 18, 19, 0, 0, 20, + 0, 17, 31, 0, 0, 22, 0, 0, 0, 11, + 12, 14, 15, 16, 21, 23, 25, 26, 27, 28, + 29, 18, 19, 0, 13, 20, 0, 0, 24, 0, + 30, 0, 0, 0, 0, 11, 12, 14, 15, 16, + 21, 23, 25, 26, 27, 28, 29, 97, 0, 0, + 13, 0, 0, 159, 24, 0, 30, 0, 0, 80, + 81, 82, 0, 83, 84, 85, 86, 87, 88, 89, + 90, 91, 92, 0, 93, 94, 96, 0, 0, 0, + 0, 0, 0, 97, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 80, 81, 82, 95, 83, + 84, 85, 86, 87, 88, 89, 90, 91, 92, 0, + 93, 94, 96, 0, 0, 97, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 80, 81, 82, + 0, 83, 84, 85, 95, 87, 88, 89, 90, 91, + 92, 0, 93, 94, 96, 0, 0, 97, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 80, + 81, 82, 0, 83, 84, 0, 95, 87, 88, 0, + 90, 91, 92, 0, 93, 94, 96, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 94, + 0, 0, 0, 0, 0, 0, 0, 0, 95, } var yyPact = [...]int{ - 41, 72, 410, 410, 160, 385, -1000, -1000, -1000, 65, + 14, 89, 419, 419, 164, 393, -1000, -1000, -1000, 47, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, 247, -1000, 101, -1000, 59, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, 14, 61, -1000, - 221, -1000, 221, 43, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 240, - -1000, -1000, 238, -1000, -1000, 115, -1000, 18, -1000, -45, - -45, -45, -45, -45, -45, -45, -45, -45, -45, -45, - -45, -45, -45, -45, 155, 153, 61, -48, -1000, 100, - 100, 15, -1000, 453, 8, -1000, 151, -1000, -1000, 161, - -1000, -1000, 217, -1000, 31, -1000, 213, 221, -1000, -50, - -42, -1000, 221, 221, 221, 221, 221, 221, 221, 221, - 221, 221, 221, 221, 221, 221, -1000, -1000, -1000, 185, - -1000, -1000, -1000, -1000, 331, -1000, -1000, 30, -1000, 59, - -1000, -1000, 106, -1000, 23, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, -3, 0, -1000, -1000, -1000, - -1000, 28, 28, 157, 100, 100, 100, 100, 8, 533, - 533, 533, 515, 484, 533, 533, 515, 8, 8, 533, - 8, 157, -1000, 112, -1000, 32, -1000, -1000, -1000, -1000, + -1000, 232, -1000, 95, -1000, 499, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 15, 20, + -1000, 226, -1000, 226, 30, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + 191, -1000, -1000, 249, -1000, -1000, 220, -1000, 4, -1000, + -41, -41, -41, -41, -41, -41, -41, -41, -41, -41, + -41, -41, -41, -41, -41, 156, 78, 113, 20, -49, + -1000, 42, 42, 16, -1000, 463, 158, -1000, 160, -1000, + -1000, 155, -1000, -1000, 190, -1000, 23, -1000, 243, 226, + -1000, -53, -48, -1000, 226, 226, 226, 226, 226, 226, + 226, 226, 226, 226, 226, 226, 226, 226, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, 24, 24, 92, -1000, + -1000, -1000, -1000, 338, -1000, -1000, 27, -1000, 499, -1000, + -1000, 186, -1000, 22, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -2, 67, -1000, -1000, -1000, 13, + 42, 42, 42, 42, 158, 76, 76, 76, 563, 531, + 76, 76, 563, 158, 158, 76, 158, 13, -1000, -1000, + -1000, 87, -1000, 72, -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, -1000, -1000, -1000, -1000, 221, -1000, -1000, -1000, -1000, - 27, 27, -7, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - 17, 81, -1000, -1000, 276, -1000, 59, -1000, -1000, -1000, - 27, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, 226, -1000, -1000, -1000, -1000, 21, 21, + -3, -1000, -1000, -1000, -1000, -1000, 18, 86, -1000, -1000, + 282, -1000, 499, -1000, -1000, -1000, 21, -1000, -1000, -1000, + -1000, -1000, } var yyPgo = [...]int{ - 0, 290, 8, 289, 1, 286, 284, 185, 283, 156, - 282, 84, 9, 280, 5, 4, 279, 264, 0, 6, - 262, 7, 261, 11, 58, 260, 250, 2, 248, 243, - 10, 241, 23, 231, 227, 223, 222, 217, 203, 199, - 168, 154, 3, 167, 159, 116, + 0, 289, 7, 288, 2, 286, 271, 233, 270, 163, + 266, 112, 8, 265, 4, 5, 264, 263, 0, 23, + 261, 6, 254, 247, 246, 10, 61, 245, 232, 1, + 230, 228, 9, 227, 20, 226, 207, 191, 183, 175, + 168, 164, 161, 160, 3, 140, 124, 100, } var yyR1 = [...]int{ - 0, 44, 44, 44, 44, 44, 44, 44, 27, 27, - 27, 27, 27, 27, 27, 27, 27, 27, 27, 22, - 22, 22, 22, 23, 23, 25, 25, 25, 25, 25, - 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, - 24, 26, 26, 36, 36, 31, 31, 31, 31, 14, - 14, 14, 14, 13, 13, 13, 4, 4, 28, 30, - 30, 29, 29, 29, 37, 35, 35, 33, 39, 39, - 39, 39, 39, 40, 41, 41, 41, 32, 32, 32, - 1, 1, 1, 2, 2, 2, 2, 11, 11, 7, + 0, 46, 46, 46, 46, 46, 46, 46, 29, 29, + 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, + 24, 24, 24, 24, 25, 25, 27, 27, 27, 27, + 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, + 27, 26, 28, 28, 38, 38, 33, 33, 33, 33, + 14, 14, 14, 14, 13, 13, 13, 4, 4, 30, + 32, 32, 31, 31, 31, 39, 37, 37, 23, 23, + 35, 41, 41, 41, 41, 41, 42, 43, 43, 43, + 34, 34, 34, 1, 1, 1, 2, 2, 2, 2, + 11, 11, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, - 7, 7, 7, 7, 7, 7, 7, 7, 7, 9, - 9, 9, 9, 10, 10, 10, 12, 12, 12, 12, - 45, 17, 17, 17, 17, 16, 16, 16, 16, 16, - 20, 20, 20, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 6, 6, 6, 6, 6, + 7, 7, 9, 9, 9, 9, 10, 10, 10, 12, + 12, 12, 12, 47, 17, 17, 17, 17, 16, 16, + 16, 16, 16, 20, 20, 20, 3, 3, 3, 3, + 3, 3, 3, 3, 3, 3, 3, 3, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, - 6, 6, 6, 6, 6, 6, 6, 6, 6, 8, - 8, 5, 5, 5, 5, 34, 19, 21, 21, 18, - 42, 38, 43, 43, 15, 15, + 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, + 6, 6, 8, 8, 5, 5, 5, 5, 36, 19, + 21, 21, 22, 22, 18, 44, 40, 45, 45, 15, + 15, } var yyR2 = [...]int{ 0, 2, 2, 2, 2, 2, 2, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, 1, 3, - 3, 2, 2, 2, 2, 4, 4, 4, 4, 4, + 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, + 3, 3, 2, 2, 2, 2, 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, 2, 3, - 2, 3, 1, 2, 3, 3, 3, 4, 6, 6, - 5, 4, 3, 2, 2, 1, 1, 3, 4, 2, - 3, 1, 2, 3, 3, 2, 1, 2, 1, 1, + 4, 1, 0, 1, 3, 3, 1, 1, 3, 3, + 3, 4, 2, 1, 3, 1, 2, 1, 1, 2, + 3, 2, 3, 1, 2, 3, 3, 3, 3, 3, + 4, 6, 6, 5, 4, 3, 2, 2, 1, 1, + 3, 4, 2, 3, 1, 2, 3, 3, 2, 1, + 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, 1, 3, - 4, 2, 0, 3, 1, 2, 3, 3, 2, 1, - 2, 0, 3, 2, 1, 1, 3, 1, 3, 4, + 1, 1, 3, 4, 2, 0, 3, 1, 2, 3, + 3, 2, 1, 2, 0, 3, 2, 1, 1, 3, + 1, 3, 4, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 2, 2, 1, - 1, 1, 0, 1, 0, 1, + 2, 2, 1, 1, 1, 1, 1, 0, 1, 0, + 1, } var yyChk = [...]int{ - -1000, -44, 69, 70, 71, 72, 2, 10, -11, -7, - -9, 45, 46, 60, 47, 48, 49, 12, 32, 33, - 36, 50, 16, 51, 64, 52, 53, 54, 55, 56, - 66, 13, -45, -11, 10, -27, -22, -25, -28, -33, - -34, -35, -37, -38, -39, -40, -41, -3, 12, 17, - 15, 23, -8, -7, -32, 45, 46, 47, 48, 49, - 50, 51, 52, 53, 54, 55, 56, 26, 42, 13, - -41, -9, -10, 18, -12, 12, 2, -17, 2, 26, - 27, 28, 30, 31, 32, 33, 34, 35, 36, 37, - 38, 39, 41, 42, 64, 14, -23, -30, 2, 60, - 66, 15, -30, -27, -27, -32, -1, 18, -2, 12, - 2, 18, 7, 2, 4, 2, 22, -24, -31, -26, - -36, 59, -24, -24, -24, -24, -24, -24, -24, -24, - -24, -24, -24, -24, -24, -24, -42, 2, 9, -42, - 2, -30, -23, -14, 15, 2, -14, -29, 20, -27, - 20, 18, 7, 2, -5, 2, 4, 39, 29, 40, - 18, -12, 23, 2, -16, 5, -20, 12, -19, -21, - 17, 26, 42, -27, 63, 65, 61, 62, -27, -27, - -27, -27, -27, -27, -27, -27, -27, -27, -27, -27, - -27, -27, 19, 6, 2, -13, 20, -4, -6, 2, - 45, 59, 46, 60, 47, 48, 49, 61, 62, 12, - 63, 32, 33, 36, 50, 16, 51, 64, 65, 52, - 53, 54, 55, 56, 20, 7, 18, -2, 23, 2, - 24, 24, -21, -19, -19, -14, -14, -15, -14, -15, - -43, -42, 2, 20, 7, 2, -27, -18, 17, -18, - 24, 19, 2, 20, -4, -18, + -1000, -46, 70, 71, 72, 73, 2, 10, -11, -7, + -9, 46, 47, 61, 48, 49, 50, 12, 32, 33, + 36, 51, 16, 52, 65, 53, 54, 55, 56, 57, + 67, 13, -47, -11, 10, -29, -24, -27, -30, -35, + -36, -37, -39, -40, -41, -42, -43, -23, -3, 12, + 17, 15, 23, -8, -7, -34, 46, 47, 48, 49, + 50, 51, 52, 53, 54, 55, 56, 57, 26, 42, + 13, -43, -9, -10, 18, -12, 12, 2, -17, 2, + 26, 27, 28, 30, 31, 32, 33, 34, 35, 36, + 37, 38, 39, 41, 42, 65, 43, 14, -25, -32, + 2, 61, 67, 15, -32, -29, -29, -34, -1, 18, + -2, 12, 2, 18, 7, 2, 4, 2, 22, -26, + -33, -28, -38, 60, -26, -26, -26, -26, -26, -26, + -26, -26, -26, -26, -26, -26, -26, -26, -44, 2, + 9, -22, 2, -19, -21, 17, 26, 42, -44, 2, + -32, -25, -14, 15, 2, -14, -31, 20, -29, 20, + 18, 7, 2, -5, 2, 4, 39, 29, 40, 18, + -12, 23, 2, -16, 5, -20, 12, -19, -21, -29, + 64, 66, 62, 63, -29, -29, -29, -29, -29, -29, + -29, -29, -29, -29, -29, -29, -29, -29, -19, -19, + 19, 6, 2, -13, 20, -4, -6, 2, 46, 60, + 47, 61, 48, 49, 50, 62, 63, 12, 64, 32, + 33, 36, 51, 16, 52, 65, 66, 53, 54, 55, + 56, 57, 20, 7, 18, -2, 23, 2, 24, 24, + -21, -14, -14, -15, -14, -15, -45, -44, 2, 20, + 7, 2, -29, -18, 17, -18, 24, 19, 2, 20, + -4, -18, } var yyDef = [...]int{ - 0, -2, 112, 112, 0, 0, 7, 6, 1, 112, - 88, 89, 90, 91, 92, 93, 94, 95, 96, 97, - 98, 99, 100, 101, 102, 103, 104, 105, 106, 107, - 108, 0, 2, -2, 3, 4, 8, 9, 10, 11, - 12, 13, 14, 15, 16, 17, 18, 0, 95, 175, - 0, 181, 0, 75, 76, -2, -2, -2, -2, -2, - -2, -2, -2, -2, -2, -2, -2, 169, 170, 0, - 5, 87, 0, 111, 114, 0, 119, 120, 124, 41, - 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, - 41, 41, 41, 41, 0, 0, 0, 21, 22, 0, - 0, 0, 58, 0, 73, 74, 0, 79, 81, 0, - 86, 109, 0, 115, 0, 118, 123, 0, 40, 45, - 46, 42, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 65, 66, 180, 0, - 72, 19, 20, 23, 0, 52, 24, 0, 60, 62, - 64, 77, 0, 82, 0, 85, 171, 172, 173, 174, - 110, 113, 116, 117, 122, 125, 127, 130, 131, 132, - 176, 0, 0, 25, 0, 0, -2, -2, 26, 27, - 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, - 38, 39, 67, -2, 71, 0, 51, 54, 56, 57, - 145, 146, 147, 148, 149, 150, 151, 152, 153, 154, - 155, 156, 157, 158, 159, 160, 161, 162, 163, 164, - 165, 166, 167, 168, 59, 63, 78, 80, 83, 84, - 0, 0, 0, 177, 178, 43, 44, 47, 185, 48, - 0, -2, 70, 49, 0, 55, 61, 126, 179, 128, - 0, 68, 69, 50, 53, 129, + 0, -2, 115, 115, 0, 0, 7, 6, 1, 115, + 91, 92, 93, 94, 95, 96, 97, 98, 99, 100, + 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, + 111, 0, 2, -2, 3, 4, 8, 9, 10, 11, + 12, 13, 14, 15, 16, 17, 18, 19, 0, 98, + 178, 0, 186, 0, 78, 79, -2, -2, -2, -2, + -2, -2, -2, -2, -2, -2, -2, -2, 172, 173, + 0, 5, 90, 0, 114, 117, 0, 122, 123, 127, + 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, + 42, 42, 42, 42, 42, 0, 0, 0, 0, 22, + 23, 0, 0, 0, 59, 0, 76, 77, 0, 82, + 84, 0, 89, 112, 0, 118, 0, 121, 126, 0, + 41, 46, 47, 43, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 66, 67, + 185, 68, 69, 182, 183, 179, 0, 0, 0, 75, + 20, 21, 24, 0, 53, 25, 0, 61, 63, 65, + 80, 0, 85, 0, 88, 174, 175, 176, 177, 113, + 116, 119, 120, 125, 128, 130, 133, 134, 135, 26, + 0, 0, -2, -2, 27, 28, 29, 30, 31, 32, + 33, 34, 35, 36, 37, 38, 39, 40, 180, 181, + 70, -2, 74, 0, 52, 55, 57, 58, 148, 149, + 150, 151, 152, 153, 154, 155, 156, 157, 158, 159, + 160, 161, 162, 163, 164, 165, 166, 167, 168, 169, + 170, 171, 60, 64, 81, 83, 86, 87, 0, 0, + 0, 44, 45, 48, 190, 49, 0, -2, 73, 50, + 0, 56, 62, 129, 184, 131, 0, 71, 72, 51, + 54, 132, } var yyTok1 = [...]int{ @@ -547,7 +560,7 @@ var yyTok2 = [...]int{ 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63, 64, 65, 66, 67, 68, 69, 70, 71, - 72, 73, + 72, 73, 74, } var yyTok3 = [...]int{ 0, @@ -892,277 +905,277 @@ yydefault: case 1: yyDollar = yyS[yypt-2 : yypt+1] -//line generated_parser.y:165 +//line generated_parser.y:166 { yylex.(*parser).generatedParserResult = yyDollar[2].labels } case 3: yyDollar = yyS[yypt-2 : yypt+1] -//line generated_parser.y:168 +//line generated_parser.y:169 { yylex.(*parser).addParseErrf(PositionRange{}, "no expression found in input") } case 4: yyDollar = yyS[yypt-2 : yypt+1] -//line generated_parser.y:170 +//line generated_parser.y:171 { yylex.(*parser).generatedParserResult = yyDollar[2].node } case 5: yyDollar = yyS[yypt-2 : yypt+1] -//line generated_parser.y:172 +//line generated_parser.y:173 { yylex.(*parser).generatedParserResult = yyDollar[2].node } case 7: yyDollar = yyS[yypt-1 : yypt+1] -//line generated_parser.y:175 +//line generated_parser.y:176 { yylex.(*parser).unexpected("", "") } - case 19: - yyDollar = yyS[yypt-3 : yypt+1] -//line generated_parser.y:197 - { - yyVAL.node = yylex.(*parser).newAggregateExpr(yyDollar[1].item, yyDollar[2].node, yyDollar[3].node) - } case 20: yyDollar = yyS[yypt-3 : yypt+1] //line generated_parser.y:199 { - yyVAL.node = yylex.(*parser).newAggregateExpr(yyDollar[1].item, yyDollar[3].node, yyDollar[2].node) + yyVAL.node = yylex.(*parser).newAggregateExpr(yyDollar[1].item, yyDollar[2].node, yyDollar[3].node) } case 21: - yyDollar = yyS[yypt-2 : yypt+1] + yyDollar = yyS[yypt-3 : yypt+1] //line generated_parser.y:201 { - yyVAL.node = yylex.(*parser).newAggregateExpr(yyDollar[1].item, &AggregateExpr{}, yyDollar[2].node) + yyVAL.node = yylex.(*parser).newAggregateExpr(yyDollar[1].item, yyDollar[3].node, yyDollar[2].node) } case 22: yyDollar = yyS[yypt-2 : yypt+1] //line generated_parser.y:203 { - yylex.(*parser).unexpected("aggregation", "") - yyVAL.node = yylex.(*parser).newAggregateExpr(yyDollar[1].item, &AggregateExpr{}, Expressions{}) + yyVAL.node = yylex.(*parser).newAggregateExpr(yyDollar[1].item, &AggregateExpr{}, yyDollar[2].node) } case 23: yyDollar = yyS[yypt-2 : yypt+1] -//line generated_parser.y:211 +//line generated_parser.y:205 + { + yylex.(*parser).unexpected("aggregation", "") + yyVAL.node = yylex.(*parser).newAggregateExpr(yyDollar[1].item, &AggregateExpr{}, Expressions{}) + } + case 24: + yyDollar = yyS[yypt-2 : yypt+1] +//line generated_parser.y:213 { yyVAL.node = &AggregateExpr{ Grouping: yyDollar[2].strings, } } - case 24: + case 25: yyDollar = yyS[yypt-2 : yypt+1] -//line generated_parser.y:217 +//line generated_parser.y:219 { yyVAL.node = &AggregateExpr{ Grouping: yyDollar[2].strings, Without: true, } } - case 25: - yyDollar = yyS[yypt-4 : yypt+1] -//line generated_parser.y:230 - { - yyVAL.node = yylex.(*parser).newBinaryExpression(yyDollar[1].node, yyDollar[2].item, yyDollar[3].node, yyDollar[4].node) - } case 26: yyDollar = yyS[yypt-4 : yypt+1] -//line generated_parser.y:231 - { - yyVAL.node = yylex.(*parser).newBinaryExpression(yyDollar[1].node, yyDollar[2].item, yyDollar[3].node, yyDollar[4].node) - } - case 27: - yyDollar = yyS[yypt-4 : yypt+1] //line generated_parser.y:232 { yyVAL.node = yylex.(*parser).newBinaryExpression(yyDollar[1].node, yyDollar[2].item, yyDollar[3].node, yyDollar[4].node) } - case 28: + case 27: yyDollar = yyS[yypt-4 : yypt+1] //line generated_parser.y:233 { yyVAL.node = yylex.(*parser).newBinaryExpression(yyDollar[1].node, yyDollar[2].item, yyDollar[3].node, yyDollar[4].node) } - case 29: + case 28: yyDollar = yyS[yypt-4 : yypt+1] //line generated_parser.y:234 { yyVAL.node = yylex.(*parser).newBinaryExpression(yyDollar[1].node, yyDollar[2].item, yyDollar[3].node, yyDollar[4].node) } - case 30: + case 29: yyDollar = yyS[yypt-4 : yypt+1] //line generated_parser.y:235 { yyVAL.node = yylex.(*parser).newBinaryExpression(yyDollar[1].node, yyDollar[2].item, yyDollar[3].node, yyDollar[4].node) } - case 31: + case 30: yyDollar = yyS[yypt-4 : yypt+1] //line generated_parser.y:236 { yyVAL.node = yylex.(*parser).newBinaryExpression(yyDollar[1].node, yyDollar[2].item, yyDollar[3].node, yyDollar[4].node) } - case 32: + case 31: yyDollar = yyS[yypt-4 : yypt+1] //line generated_parser.y:237 { yyVAL.node = yylex.(*parser).newBinaryExpression(yyDollar[1].node, yyDollar[2].item, yyDollar[3].node, yyDollar[4].node) } - case 33: + case 32: yyDollar = yyS[yypt-4 : yypt+1] //line generated_parser.y:238 { yyVAL.node = yylex.(*parser).newBinaryExpression(yyDollar[1].node, yyDollar[2].item, yyDollar[3].node, yyDollar[4].node) } - case 34: + case 33: yyDollar = yyS[yypt-4 : yypt+1] //line generated_parser.y:239 { yyVAL.node = yylex.(*parser).newBinaryExpression(yyDollar[1].node, yyDollar[2].item, yyDollar[3].node, yyDollar[4].node) } - case 35: + case 34: yyDollar = yyS[yypt-4 : yypt+1] //line generated_parser.y:240 { yyVAL.node = yylex.(*parser).newBinaryExpression(yyDollar[1].node, yyDollar[2].item, yyDollar[3].node, yyDollar[4].node) } - case 36: + case 35: yyDollar = yyS[yypt-4 : yypt+1] //line generated_parser.y:241 { yyVAL.node = yylex.(*parser).newBinaryExpression(yyDollar[1].node, yyDollar[2].item, yyDollar[3].node, yyDollar[4].node) } - case 37: + case 36: yyDollar = yyS[yypt-4 : yypt+1] //line generated_parser.y:242 { yyVAL.node = yylex.(*parser).newBinaryExpression(yyDollar[1].node, yyDollar[2].item, yyDollar[3].node, yyDollar[4].node) } - case 38: + case 37: yyDollar = yyS[yypt-4 : yypt+1] //line generated_parser.y:243 { yyVAL.node = yylex.(*parser).newBinaryExpression(yyDollar[1].node, yyDollar[2].item, yyDollar[3].node, yyDollar[4].node) } - case 39: + case 38: yyDollar = yyS[yypt-4 : yypt+1] //line generated_parser.y:244 { yyVAL.node = yylex.(*parser).newBinaryExpression(yyDollar[1].node, yyDollar[2].item, yyDollar[3].node, yyDollar[4].node) } - case 41: + case 39: + yyDollar = yyS[yypt-4 : yypt+1] +//line generated_parser.y:245 + { + yyVAL.node = yylex.(*parser).newBinaryExpression(yyDollar[1].node, yyDollar[2].item, yyDollar[3].node, yyDollar[4].node) + } + case 40: + yyDollar = yyS[yypt-4 : yypt+1] +//line generated_parser.y:246 + { + yyVAL.node = yylex.(*parser).newBinaryExpression(yyDollar[1].node, yyDollar[2].item, yyDollar[3].node, yyDollar[4].node) + } + case 42: yyDollar = yyS[yypt-0 : yypt+1] -//line generated_parser.y:252 +//line generated_parser.y:254 { yyVAL.node = &BinaryExpr{ VectorMatching: &VectorMatching{Card: CardOneToOne}, } } - case 42: + case 43: yyDollar = yyS[yypt-1 : yypt+1] -//line generated_parser.y:257 +//line generated_parser.y:259 { yyVAL.node = &BinaryExpr{ VectorMatching: &VectorMatching{Card: CardOneToOne}, ReturnBool: true, } } - case 43: + case 44: yyDollar = yyS[yypt-3 : yypt+1] -//line generated_parser.y:265 +//line generated_parser.y:267 { yyVAL.node = yyDollar[1].node yyVAL.node.(*BinaryExpr).VectorMatching.MatchingLabels = yyDollar[3].strings } - case 44: + case 45: yyDollar = yyS[yypt-3 : yypt+1] -//line generated_parser.y:270 +//line generated_parser.y:272 { yyVAL.node = yyDollar[1].node yyVAL.node.(*BinaryExpr).VectorMatching.MatchingLabels = yyDollar[3].strings yyVAL.node.(*BinaryExpr).VectorMatching.On = true } - case 47: + case 48: yyDollar = yyS[yypt-3 : yypt+1] -//line generated_parser.y:280 +//line generated_parser.y:282 { yyVAL.node = yyDollar[1].node yyVAL.node.(*BinaryExpr).VectorMatching.Card = CardManyToOne yyVAL.node.(*BinaryExpr).VectorMatching.Include = yyDollar[3].strings } - case 48: + case 49: yyDollar = yyS[yypt-3 : yypt+1] -//line generated_parser.y:286 +//line generated_parser.y:288 { yyVAL.node = yyDollar[1].node yyVAL.node.(*BinaryExpr).VectorMatching.Card = CardOneToMany yyVAL.node.(*BinaryExpr).VectorMatching.Include = yyDollar[3].strings } - case 49: - yyDollar = yyS[yypt-3 : yypt+1] -//line generated_parser.y:295 - { - yyVAL.strings = yyDollar[2].strings - } case 50: - yyDollar = yyS[yypt-4 : yypt+1] + yyDollar = yyS[yypt-3 : yypt+1] //line generated_parser.y:297 { yyVAL.strings = yyDollar[2].strings } case 51: - yyDollar = yyS[yypt-2 : yypt+1] + yyDollar = yyS[yypt-4 : yypt+1] //line generated_parser.y:299 + { + yyVAL.strings = yyDollar[2].strings + } + case 52: + yyDollar = yyS[yypt-2 : yypt+1] +//line generated_parser.y:301 { yyVAL.strings = []string{} } - case 52: + case 53: yyDollar = yyS[yypt-1 : yypt+1] -//line generated_parser.y:301 +//line generated_parser.y:303 { yylex.(*parser).unexpected("grouping opts", "\"(\"") yyVAL.strings = nil } - case 53: + case 54: yyDollar = yyS[yypt-3 : yypt+1] -//line generated_parser.y:307 +//line generated_parser.y:309 { yyVAL.strings = append(yyDollar[1].strings, yyDollar[3].item.Val) } - case 54: + case 55: yyDollar = yyS[yypt-1 : yypt+1] -//line generated_parser.y:309 +//line generated_parser.y:311 { yyVAL.strings = []string{yyDollar[1].item.Val} } - case 55: + case 56: yyDollar = yyS[yypt-2 : yypt+1] -//line generated_parser.y:311 +//line generated_parser.y:313 { yylex.(*parser).unexpected("grouping opts", "\",\" or \")\"") yyVAL.strings = yyDollar[1].strings } - case 56: + case 57: yyDollar = yyS[yypt-1 : yypt+1] -//line generated_parser.y:315 +//line generated_parser.y:317 { if !isLabel(yyDollar[1].item.Val) { yylex.(*parser).unexpected("grouping opts", "label") } yyVAL.item = yyDollar[1].item } - case 57: + case 58: yyDollar = yyS[yypt-1 : yypt+1] -//line generated_parser.y:322 +//line generated_parser.y:324 { yylex.(*parser).unexpected("grouping opts", "label") yyVAL.item = Item{} } - case 58: + case 59: yyDollar = yyS[yypt-2 : yypt+1] -//line generated_parser.y:330 +//line generated_parser.y:332 { fn, exist := getFunction(yyDollar[1].item.Val) if !exist { @@ -1177,67 +1190,83 @@ yydefault: }, } } - case 59: + case 60: yyDollar = yyS[yypt-3 : yypt+1] -//line generated_parser.y:347 +//line generated_parser.y:349 { yyVAL.node = yyDollar[2].node } - case 60: + case 61: yyDollar = yyS[yypt-2 : yypt+1] -//line generated_parser.y:349 +//line generated_parser.y:351 { yyVAL.node = Expressions{} } - case 61: + case 62: yyDollar = yyS[yypt-3 : yypt+1] -//line generated_parser.y:353 +//line generated_parser.y:355 { yyVAL.node = append(yyDollar[1].node.(Expressions), yyDollar[3].node.(Expr)) } - case 62: + case 63: yyDollar = yyS[yypt-1 : yypt+1] -//line generated_parser.y:355 +//line generated_parser.y:357 { yyVAL.node = Expressions{yyDollar[1].node.(Expr)} } - case 63: + case 64: yyDollar = yyS[yypt-2 : yypt+1] -//line generated_parser.y:357 +//line generated_parser.y:359 { yylex.(*parser).addParseErrf(yyDollar[2].item.PositionRange(), "trailing commas not allowed in function call args") yyVAL.node = yyDollar[1].node } - case 64: + case 65: yyDollar = yyS[yypt-3 : yypt+1] -//line generated_parser.y:368 +//line generated_parser.y:370 { yyVAL.node = &ParenExpr{Expr: yyDollar[2].node.(Expr), PosRange: mergeRanges(&yyDollar[1].item, &yyDollar[3].item)} } - case 65: + case 66: yyDollar = yyS[yypt-3 : yypt+1] -//line generated_parser.y:376 +//line generated_parser.y:378 { yylex.(*parser).addOffset(yyDollar[1].node, yyDollar[3].duration) yyVAL.node = yyDollar[1].node } - case 66: + case 67: yyDollar = yyS[yypt-3 : yypt+1] -//line generated_parser.y:381 +//line generated_parser.y:383 { yylex.(*parser).unexpected("offset", "duration") yyVAL.node = yyDollar[1].node } - case 67: + case 68: + yyDollar = yyS[yypt-3 : yypt+1] +//line generated_parser.y:390 + { + yylex.(*parser).setTimestamp(yyDollar[1].node, yyDollar[3].float) + yyVAL.node = yyDollar[1].node + } + case 69: + yyDollar = yyS[yypt-3 : yypt+1] +//line generated_parser.y:396 + { + yylex.(*parser).unexpected("@", "timestamp") + yyVAL.node = yyDollar[1].node + } + case 70: yyDollar = yyS[yypt-4 : yypt+1] -//line generated_parser.y:389 +//line generated_parser.y:404 { var errMsg string vs, ok := yyDollar[1].node.(*VectorSelector) if !ok { errMsg = "ranges only allowed for vector selectors" - } else if vs.Offset != 0 { + } else if vs.OriginalOffset != 0 { errMsg = "no offset modifiers allowed before range" + } else if vs.Timestamp != nil { + errMsg = "no @ modifiers allowed before range" } if errMsg != "" { @@ -1251,9 +1280,9 @@ yydefault: EndPos: yylex.(*parser).lastClosing, } } - case 68: + case 71: yyDollar = yyS[yypt-6 : yypt+1] -//line generated_parser.y:412 +//line generated_parser.y:429 { yyVAL.node = &SubqueryExpr{ Expr: yyDollar[1].node.(Expr), @@ -1263,37 +1292,37 @@ yydefault: EndPos: yyDollar[6].item.Pos + 1, } } - case 69: + case 72: yyDollar = yyS[yypt-6 : yypt+1] -//line generated_parser.y:422 +//line generated_parser.y:439 { yylex.(*parser).unexpected("subquery selector", "\"]\"") yyVAL.node = yyDollar[1].node } - case 70: + case 73: yyDollar = yyS[yypt-5 : yypt+1] -//line generated_parser.y:424 +//line generated_parser.y:441 { yylex.(*parser).unexpected("subquery selector", "duration or \"]\"") yyVAL.node = yyDollar[1].node } - case 71: + case 74: yyDollar = yyS[yypt-4 : yypt+1] -//line generated_parser.y:426 +//line generated_parser.y:443 { yylex.(*parser).unexpected("subquery or range", "\":\" or \"]\"") yyVAL.node = yyDollar[1].node } - case 72: + case 75: yyDollar = yyS[yypt-3 : yypt+1] -//line generated_parser.y:428 +//line generated_parser.y:445 { yylex.(*parser).unexpected("subquery selector", "duration") yyVAL.node = yyDollar[1].node } - case 73: + case 76: yyDollar = yyS[yypt-2 : yypt+1] -//line generated_parser.y:438 +//line generated_parser.y:455 { if nl, ok := yyDollar[2].node.(*NumberLiteral); ok { if yyDollar[1].item.Typ == SUB { @@ -1305,9 +1334,9 @@ yydefault: yyVAL.node = &UnaryExpr{Op: yyDollar[1].item.Typ, Expr: yyDollar[2].node.(Expr), StartPos: yyDollar[1].item.Pos} } } - case 74: + case 77: yyDollar = yyS[yypt-2 : yypt+1] -//line generated_parser.y:456 +//line generated_parser.y:473 { vs := yyDollar[2].node.(*VectorSelector) vs.PosRange = mergeRanges(&yyDollar[1].item, vs) @@ -1315,9 +1344,9 @@ yydefault: yylex.(*parser).assembleVectorSelector(vs) yyVAL.node = vs } - case 75: + case 78: yyDollar = yyS[yypt-1 : yypt+1] -//line generated_parser.y:464 +//line generated_parser.y:481 { vs := &VectorSelector{ Name: yyDollar[1].item.Val, @@ -1327,44 +1356,44 @@ yydefault: yylex.(*parser).assembleVectorSelector(vs) yyVAL.node = vs } - case 76: + case 79: yyDollar = yyS[yypt-1 : yypt+1] -//line generated_parser.y:474 +//line generated_parser.y:491 { vs := yyDollar[1].node.(*VectorSelector) yylex.(*parser).assembleVectorSelector(vs) yyVAL.node = vs } - case 77: + case 80: yyDollar = yyS[yypt-3 : yypt+1] -//line generated_parser.y:482 +//line generated_parser.y:499 { yyVAL.node = &VectorSelector{ LabelMatchers: yyDollar[2].matchers, PosRange: mergeRanges(&yyDollar[1].item, &yyDollar[3].item), } } - case 78: + case 81: yyDollar = yyS[yypt-4 : yypt+1] -//line generated_parser.y:489 +//line generated_parser.y:506 { yyVAL.node = &VectorSelector{ LabelMatchers: yyDollar[2].matchers, PosRange: mergeRanges(&yyDollar[1].item, &yyDollar[4].item), } } - case 79: + case 82: yyDollar = yyS[yypt-2 : yypt+1] -//line generated_parser.y:496 +//line generated_parser.y:513 { yyVAL.node = &VectorSelector{ LabelMatchers: []*labels.Matcher{}, PosRange: mergeRanges(&yyDollar[1].item, &yyDollar[2].item), } } - case 80: + case 83: yyDollar = yyS[yypt-3 : yypt+1] -//line generated_parser.y:505 +//line generated_parser.y:522 { if yyDollar[1].matchers != nil { yyVAL.matchers = append(yyDollar[1].matchers, yyDollar[3].matcher) @@ -1372,196 +1401,196 @@ yydefault: yyVAL.matchers = yyDollar[1].matchers } } - case 81: + case 84: yyDollar = yyS[yypt-1 : yypt+1] -//line generated_parser.y:513 +//line generated_parser.y:530 { yyVAL.matchers = []*labels.Matcher{yyDollar[1].matcher} } - case 82: + case 85: yyDollar = yyS[yypt-2 : yypt+1] -//line generated_parser.y:515 +//line generated_parser.y:532 { yylex.(*parser).unexpected("label matching", "\",\" or \"}\"") yyVAL.matchers = yyDollar[1].matchers } - case 83: + case 86: yyDollar = yyS[yypt-3 : yypt+1] -//line generated_parser.y:519 +//line generated_parser.y:536 { yyVAL.matcher = yylex.(*parser).newLabelMatcher(yyDollar[1].item, yyDollar[2].item, yyDollar[3].item) } - case 84: + case 87: yyDollar = yyS[yypt-3 : yypt+1] -//line generated_parser.y:521 +//line generated_parser.y:538 { yylex.(*parser).unexpected("label matching", "string") yyVAL.matcher = nil } - case 85: + case 88: yyDollar = yyS[yypt-2 : yypt+1] -//line generated_parser.y:523 +//line generated_parser.y:540 { yylex.(*parser).unexpected("label matching", "label matching operator") yyVAL.matcher = nil } - case 86: + case 89: yyDollar = yyS[yypt-1 : yypt+1] -//line generated_parser.y:525 +//line generated_parser.y:542 { yylex.(*parser).unexpected("label matching", "identifier or \"}\"") yyVAL.matcher = nil } - case 87: + case 90: yyDollar = yyS[yypt-2 : yypt+1] -//line generated_parser.y:533 +//line generated_parser.y:550 { yyVAL.labels = append(yyDollar[2].labels, labels.Label{Name: labels.MetricName, Value: yyDollar[1].item.Val}) sort.Sort(yyVAL.labels) } - case 88: + case 91: yyDollar = yyS[yypt-1 : yypt+1] -//line generated_parser.y:535 +//line generated_parser.y:552 { yyVAL.labels = yyDollar[1].labels } - case 109: - yyDollar = yyS[yypt-3 : yypt+1] -//line generated_parser.y:542 - { - yyVAL.labels = labels.New(yyDollar[2].labels...) - } - case 110: - yyDollar = yyS[yypt-4 : yypt+1] -//line generated_parser.y:544 - { - yyVAL.labels = labels.New(yyDollar[2].labels...) - } - case 111: - yyDollar = yyS[yypt-2 : yypt+1] -//line generated_parser.y:546 - { - yyVAL.labels = labels.New() - } case 112: - yyDollar = yyS[yypt-0 : yypt+1] -//line generated_parser.y:548 + yyDollar = yyS[yypt-3 : yypt+1] +//line generated_parser.y:559 { - yyVAL.labels = labels.New() + yyVAL.labels = labels.New(yyDollar[2].labels...) } case 113: + yyDollar = yyS[yypt-4 : yypt+1] +//line generated_parser.y:561 + { + yyVAL.labels = labels.New(yyDollar[2].labels...) + } + case 114: + yyDollar = yyS[yypt-2 : yypt+1] +//line generated_parser.y:563 + { + yyVAL.labels = labels.New() + } + case 115: + yyDollar = yyS[yypt-0 : yypt+1] +//line generated_parser.y:565 + { + yyVAL.labels = labels.New() + } + case 116: yyDollar = yyS[yypt-3 : yypt+1] -//line generated_parser.y:552 +//line generated_parser.y:569 { yyVAL.labels = append(yyDollar[1].labels, yyDollar[3].label) } - case 114: + case 117: yyDollar = yyS[yypt-1 : yypt+1] -//line generated_parser.y:554 +//line generated_parser.y:571 { yyVAL.labels = []labels.Label{yyDollar[1].label} } - case 115: + case 118: yyDollar = yyS[yypt-2 : yypt+1] -//line generated_parser.y:556 +//line generated_parser.y:573 { yylex.(*parser).unexpected("label set", "\",\" or \"}\"") yyVAL.labels = yyDollar[1].labels } - case 116: + case 119: yyDollar = yyS[yypt-3 : yypt+1] -//line generated_parser.y:561 +//line generated_parser.y:578 { yyVAL.label = labels.Label{Name: yyDollar[1].item.Val, Value: yylex.(*parser).unquoteString(yyDollar[3].item.Val)} } - case 117: + case 120: yyDollar = yyS[yypt-3 : yypt+1] -//line generated_parser.y:563 +//line generated_parser.y:580 { yylex.(*parser).unexpected("label set", "string") yyVAL.label = labels.Label{} } - case 118: + case 121: yyDollar = yyS[yypt-2 : yypt+1] -//line generated_parser.y:565 +//line generated_parser.y:582 { yylex.(*parser).unexpected("label set", "\"=\"") yyVAL.label = labels.Label{} } - case 119: + case 122: yyDollar = yyS[yypt-1 : yypt+1] -//line generated_parser.y:567 +//line generated_parser.y:584 { yylex.(*parser).unexpected("label set", "identifier or \"}\"") yyVAL.label = labels.Label{} } - case 120: + case 123: yyDollar = yyS[yypt-2 : yypt+1] -//line generated_parser.y:575 +//line generated_parser.y:592 { yylex.(*parser).generatedParserResult = &seriesDescription{ labels: yyDollar[1].labels, values: yyDollar[2].series, } } - case 121: + case 124: yyDollar = yyS[yypt-0 : yypt+1] -//line generated_parser.y:584 +//line generated_parser.y:601 { yyVAL.series = []SequenceValue{} } - case 122: + case 125: yyDollar = yyS[yypt-3 : yypt+1] -//line generated_parser.y:586 +//line generated_parser.y:603 { yyVAL.series = append(yyDollar[1].series, yyDollar[3].series...) } - case 123: + case 126: yyDollar = yyS[yypt-2 : yypt+1] -//line generated_parser.y:588 +//line generated_parser.y:605 { yyVAL.series = yyDollar[1].series } - case 124: + case 127: yyDollar = yyS[yypt-1 : yypt+1] -//line generated_parser.y:590 +//line generated_parser.y:607 { yylex.(*parser).unexpected("series values", "") yyVAL.series = nil } - case 125: + case 128: yyDollar = yyS[yypt-1 : yypt+1] -//line generated_parser.y:594 +//line generated_parser.y:611 { yyVAL.series = []SequenceValue{{Omitted: true}} } - case 126: + case 129: yyDollar = yyS[yypt-3 : yypt+1] -//line generated_parser.y:596 +//line generated_parser.y:613 { yyVAL.series = []SequenceValue{} for i := uint64(0); i < yyDollar[3].uint; i++ { yyVAL.series = append(yyVAL.series, SequenceValue{Omitted: true}) } } - case 127: + case 130: yyDollar = yyS[yypt-1 : yypt+1] -//line generated_parser.y:603 +//line generated_parser.y:620 { yyVAL.series = []SequenceValue{{Value: yyDollar[1].float}} } - case 128: + case 131: yyDollar = yyS[yypt-3 : yypt+1] -//line generated_parser.y:605 +//line generated_parser.y:622 { yyVAL.series = []SequenceValue{} for i := uint64(0); i <= yyDollar[3].uint; i++ { yyVAL.series = append(yyVAL.series, SequenceValue{Value: yyDollar[1].float}) } } - case 129: + case 132: yyDollar = yyS[yypt-4 : yypt+1] -//line generated_parser.y:612 +//line generated_parser.y:629 { yyVAL.series = []SequenceValue{} for i := uint64(0); i <= yyDollar[4].uint; i++ { @@ -1569,45 +1598,45 @@ yydefault: yyDollar[1].float += yyDollar[2].float } } - case 130: + case 133: yyDollar = yyS[yypt-1 : yypt+1] -//line generated_parser.y:622 +//line generated_parser.y:639 { if yyDollar[1].item.Val != "stale" { yylex.(*parser).unexpected("series values", "number or \"stale\"") } yyVAL.float = math.Float64frombits(value.StaleNaN) } - case 175: + case 178: yyDollar = yyS[yypt-1 : yypt+1] -//line generated_parser.y:653 +//line generated_parser.y:670 { yyVAL.node = &NumberLiteral{ Val: yylex.(*parser).number(yyDollar[1].item.Val), PosRange: yyDollar[1].item.PositionRange(), } } - case 176: + case 179: yyDollar = yyS[yypt-1 : yypt+1] -//line generated_parser.y:661 +//line generated_parser.y:678 { yyVAL.float = yylex.(*parser).number(yyDollar[1].item.Val) } - case 177: + case 180: yyDollar = yyS[yypt-2 : yypt+1] -//line generated_parser.y:663 +//line generated_parser.y:680 { yyVAL.float = yyDollar[2].float } - case 178: + case 181: yyDollar = yyS[yypt-2 : yypt+1] -//line generated_parser.y:664 +//line generated_parser.y:681 { yyVAL.float = -yyDollar[2].float } - case 179: + case 184: yyDollar = yyS[yypt-1 : yypt+1] -//line generated_parser.y:668 +//line generated_parser.y:687 { var err error yyVAL.uint, err = strconv.ParseUint(yyDollar[1].item.Val, 10, 64) @@ -1615,9 +1644,9 @@ yydefault: yylex.(*parser).addParseErrf(yyDollar[1].item.PositionRange(), "invalid repetition in series values: %s", err) } } - case 180: + case 185: yyDollar = yyS[yypt-1 : yypt+1] -//line generated_parser.y:678 +//line generated_parser.y:697 { var err error yyVAL.duration, err = parseDuration(yyDollar[1].item.Val) @@ -1625,24 +1654,24 @@ yydefault: yylex.(*parser).addParseErr(yyDollar[1].item.PositionRange(), err) } } - case 181: + case 186: yyDollar = yyS[yypt-1 : yypt+1] -//line generated_parser.y:689 +//line generated_parser.y:708 { yyVAL.node = &StringLiteral{ Val: yylex.(*parser).unquoteString(yyDollar[1].item.Val), PosRange: yyDollar[1].item.PositionRange(), } } - case 182: + case 187: yyDollar = yyS[yypt-0 : yypt+1] -//line generated_parser.y:702 +//line generated_parser.y:721 { yyVAL.duration = 0 } - case 184: + case 189: yyDollar = yyS[yypt-0 : yypt+1] -//line generated_parser.y:706 +//line generated_parser.y:725 { yyVAL.strings = nil } diff --git a/promql/parser/lex.go b/promql/parser/lex.go index ada5d70d16..11ce35c54c 100644 --- a/promql/parser/lex.go +++ b/promql/parser/lex.go @@ -440,7 +440,8 @@ func lexStatements(l *Lexer) stateFn { } l.emit(RIGHT_BRACKET) l.bracketOpen = false - + case r == '@': + l.emit(AT) default: return l.errorf("unexpected character: %q", r) } diff --git a/promql/parser/parse.go b/promql/parser/parse.go index a89c9424c2..2960b14b28 100644 --- a/promql/parser/parse.go +++ b/promql/parser/parse.go @@ -15,6 +15,7 @@ package parser import ( "fmt" + "math" "os" "runtime" "strconv" @@ -26,6 +27,7 @@ import ( "github.com/prometheus/common/model" "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/pkg/timestamp" "github.com/prometheus/prometheus/util/strutil" ) @@ -318,7 +320,7 @@ func (p *parser) Lex(lval *yySymType) int { case EOF: lval.item.Typ = EOF p.InjectItem(0) - case RIGHT_BRACE, RIGHT_PAREN, RIGHT_BRACKET, DURATION: + case RIGHT_BRACE, RIGHT_PAREN, RIGHT_BRACKET, DURATION, NUMBER: p.lastClosing = lval.item.Pos + Pos(len(lval.item.Val)) } @@ -680,34 +682,92 @@ func (p *parser) newLabelMatcher(label Item, operator Item, value Item) *labels. return m } +// addOffset is used to set the offset in the generated parser. func (p *parser) addOffset(e Node, offset time.Duration) { - var offsetp *time.Duration + var orgoffsetp *time.Duration var endPosp *Pos switch s := e.(type) { case *VectorSelector: - offsetp = &s.Offset + orgoffsetp = &s.OriginalOffset endPosp = &s.PosRange.End case *MatrixSelector: - if vs, ok := s.VectorSelector.(*VectorSelector); ok { - offsetp = &vs.Offset + vs, ok := s.VectorSelector.(*VectorSelector) + if !ok { + p.addParseErrf(e.PositionRange(), "ranges only allowed for vector selectors") + return } + orgoffsetp = &vs.OriginalOffset endPosp = &s.EndPos case *SubqueryExpr: - offsetp = &s.Offset + orgoffsetp = &s.OriginalOffset endPosp = &s.EndPos default: - p.addParseErrf(e.PositionRange(), "offset modifier must be preceded by an instant or range selector, but follows a %T instead", e) + p.addParseErrf(e.PositionRange(), "offset modifier must be preceded by an instant selector vector or range vector selector or a subquery") return } // it is already ensured by parseDuration func that there never will be a zero offset modifier - if *offsetp != 0 { + if *orgoffsetp != 0 { p.addParseErrf(e.PositionRange(), "offset may not be set multiple times") - } else if offsetp != nil { - *offsetp = offset + } else if orgoffsetp != nil { + *orgoffsetp = offset } *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) || + ts >= float64(math.MaxInt64) || ts <= float64(math.MinInt64) { + p.addParseErrf(e.PositionRange(), "timestamp out of bounds for @ modifier: %f", ts) + } + var timestampp **int64 + var endPosp *Pos + + switch s := e.(type) { + case *VectorSelector: + timestampp = &s.Timestamp + endPosp = &s.PosRange.End + case *MatrixSelector: + vs, ok := s.VectorSelector.(*VectorSelector) + if !ok { + p.addParseErrf(e.PositionRange(), "ranges only allowed for vector selectors") + return + } + timestampp = &vs.Timestamp + endPosp = &s.EndPos + case *SubqueryExpr: + timestampp = &s.Timestamp + endPosp = &s.EndPos + default: + p.addParseErrf(e.PositionRange(), "@ modifier must be preceded by an instant selector vector or range vector selector or a subquery") + return + } + + if *timestampp != nil { + p.addParseErrf(e.PositionRange(), "@ may not be set multiple times") + } else if timestampp != nil { + *timestampp = new(int64) + **timestampp = timestamp.FromFloatSeconds(ts) + } + + *endPosp = p.lastClosing +} + +func MustLabelMatcher(mt labels.MatchType, name, val string) *labels.Matcher { + m, err := labels.NewMatcher(mt, name, val) + if err != nil { + panic(err) + } + return m +} + +func MustGetFunction(name string) *Function { + f, ok := getFunction(name) + if !ok { + panic(errors.Errorf("function %q does not exist", name)) + } + return f } diff --git a/promql/parser/parse_test.go b/promql/parser/parse_test.go index e94689f70f..f1f9b9a6ca 100644 --- a/promql/parser/parse_test.go +++ b/promql/parser/parse_test.go @@ -14,6 +14,7 @@ package parser import ( + "fmt" "math" "strings" "testing" @@ -397,7 +398,7 @@ var testExpr = []struct { Expr: &VectorSelector{ Name: "some_metric", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), }, PosRange: PositionRange{ Start: 1, @@ -412,7 +413,7 @@ var testExpr = []struct { Expr: &VectorSelector{ Name: "some_metric", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), }, PosRange: PositionRange{ Start: 1, @@ -427,7 +428,7 @@ var testExpr = []struct { Expr: &VectorSelector{ Name: "some_metric", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), }, PosRange: PositionRange{ Start: 2, @@ -527,7 +528,7 @@ var testExpr = []struct { }, { input: "1 offset 1d", fail: true, - errMsg: "offset modifier must be preceded by an instant or range selector", + errMsg: "1:1: parse error: offset modifier must be preceded by an instant selector vector or range vector selector or a subquery", }, { input: "foo offset 1s offset 2s", fail: true, @@ -545,7 +546,7 @@ var testExpr = []struct { LHS: &VectorSelector{ Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 0, @@ -555,7 +556,7 @@ var testExpr = []struct { RHS: &VectorSelector{ Name: "bar", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), }, PosRange: PositionRange{ Start: 6, @@ -571,7 +572,7 @@ var testExpr = []struct { LHS: &VectorSelector{ Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 0, @@ -581,7 +582,7 @@ var testExpr = []struct { RHS: &VectorSelector{ Name: "sum", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "sum"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "sum"), }, PosRange: PositionRange{ Start: 6, @@ -597,7 +598,7 @@ var testExpr = []struct { LHS: &VectorSelector{ Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 0, @@ -616,7 +617,7 @@ var testExpr = []struct { LHS: &VectorSelector{ Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 0, @@ -640,7 +641,7 @@ var testExpr = []struct { RHS: &VectorSelector{ Name: "bar", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), }, PosRange: PositionRange{ Start: 6, @@ -655,7 +656,7 @@ var testExpr = []struct { LHS: &VectorSelector{ Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 0, @@ -665,7 +666,7 @@ var testExpr = []struct { RHS: &VectorSelector{ Name: "bar", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), }, PosRange: PositionRange{ Start: 8, @@ -681,7 +682,7 @@ var testExpr = []struct { LHS: &VectorSelector{ Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 0, @@ -691,7 +692,7 @@ var testExpr = []struct { RHS: &VectorSelector{ Name: "bar", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), }, PosRange: PositionRange{ Start: 7, @@ -707,7 +708,7 @@ var testExpr = []struct { LHS: &VectorSelector{ Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 0, @@ -717,7 +718,7 @@ var testExpr = []struct { RHS: &VectorSelector{ Name: "bar", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), }, PosRange: PositionRange{ Start: 11, @@ -736,7 +737,7 @@ var testExpr = []struct { LHS: &VectorSelector{ Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 0, @@ -746,7 +747,7 @@ var testExpr = []struct { RHS: &VectorSelector{ Name: "bar", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), }, PosRange: PositionRange{ Start: 6, @@ -760,7 +761,7 @@ var testExpr = []struct { LHS: &VectorSelector{ Name: "bla", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bla"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bla"), }, PosRange: PositionRange{ Start: 13, @@ -770,7 +771,7 @@ var testExpr = []struct { RHS: &VectorSelector{ Name: "blub", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "blub"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "blub"), }, PosRange: PositionRange{ Start: 21, @@ -793,7 +794,7 @@ var testExpr = []struct { LHS: &VectorSelector{ Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 0, @@ -803,7 +804,7 @@ var testExpr = []struct { RHS: &VectorSelector{ Name: "bar", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), }, PosRange: PositionRange{ Start: 8, @@ -815,7 +816,7 @@ var testExpr = []struct { RHS: &VectorSelector{ Name: "baz", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "baz"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "baz"), }, PosRange: PositionRange{ Start: 19, @@ -827,7 +828,7 @@ var testExpr = []struct { RHS: &VectorSelector{ Name: "qux", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "qux"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "qux"), }, PosRange: PositionRange{ Start: 26, @@ -844,7 +845,7 @@ var testExpr = []struct { LHS: &VectorSelector{ Name: "bar", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), }, PosRange: PositionRange{ Start: 0, @@ -856,7 +857,7 @@ var testExpr = []struct { LHS: &VectorSelector{ Name: "bla", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bla"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bla"), }, PosRange: PositionRange{ Start: 14, @@ -866,7 +867,7 @@ var testExpr = []struct { RHS: &VectorSelector{ Name: "blub", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "blub"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "blub"), }, PosRange: PositionRange{ Start: 51, @@ -893,7 +894,7 @@ var testExpr = []struct { LHS: &VectorSelector{ Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 0, @@ -903,7 +904,7 @@ var testExpr = []struct { RHS: &VectorSelector{ Name: "bar", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), }, PosRange: PositionRange{ Start: 20, @@ -923,7 +924,7 @@ var testExpr = []struct { LHS: &VectorSelector{ Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 0, @@ -933,7 +934,7 @@ var testExpr = []struct { RHS: &VectorSelector{ Name: "bar", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), }, PosRange: PositionRange{ Start: 31, @@ -953,7 +954,7 @@ var testExpr = []struct { LHS: &VectorSelector{ Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 0, @@ -963,7 +964,7 @@ var testExpr = []struct { RHS: &VectorSelector{ Name: "bar", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), }, PosRange: PositionRange{ Start: 22, @@ -983,7 +984,7 @@ var testExpr = []struct { LHS: &VectorSelector{ Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 0, @@ -993,7 +994,7 @@ var testExpr = []struct { RHS: &VectorSelector{ Name: "bar", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), }, PosRange: PositionRange{ Start: 13, @@ -1013,7 +1014,7 @@ var testExpr = []struct { LHS: &VectorSelector{ Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 0, @@ -1023,7 +1024,7 @@ var testExpr = []struct { RHS: &VectorSelector{ Name: "bar", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), }, PosRange: PositionRange{ Start: 28, @@ -1042,7 +1043,7 @@ var testExpr = []struct { LHS: &VectorSelector{ Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 0, @@ -1052,7 +1053,7 @@ var testExpr = []struct { RHS: &VectorSelector{ Name: "bar", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), }, PosRange: PositionRange{ Start: 19, @@ -1071,7 +1072,7 @@ var testExpr = []struct { LHS: &VectorSelector{ Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 0, @@ -1081,7 +1082,7 @@ var testExpr = []struct { RHS: &VectorSelector{ Name: "baz", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "baz"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "baz"), }, PosRange: PositionRange{ Start: 19, @@ -1101,7 +1102,7 @@ var testExpr = []struct { LHS: &VectorSelector{ Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 0, @@ -1111,7 +1112,7 @@ var testExpr = []struct { RHS: &VectorSelector{ Name: "bar", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), }, PosRange: PositionRange{ Start: 36, @@ -1132,7 +1133,7 @@ var testExpr = []struct { LHS: &VectorSelector{ Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 0, @@ -1142,7 +1143,7 @@ var testExpr = []struct { RHS: &VectorSelector{ Name: "bar", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), }, PosRange: PositionRange{ Start: 43, @@ -1162,7 +1163,7 @@ var testExpr = []struct { LHS: &VectorSelector{ Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 0, @@ -1172,7 +1173,7 @@ var testExpr = []struct { RHS: &VectorSelector{ Name: "bar", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), }, PosRange: PositionRange{ Start: 42, @@ -1192,7 +1193,7 @@ var testExpr = []struct { LHS: &VectorSelector{ Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 0, @@ -1202,7 +1203,7 @@ var testExpr = []struct { RHS: &VectorSelector{ Name: "bar", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), }, PosRange: PositionRange{ Start: 41, @@ -1223,7 +1224,7 @@ var testExpr = []struct { LHS: &VectorSelector{ Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 0, @@ -1233,7 +1234,7 @@ var testExpr = []struct { RHS: &VectorSelector{ Name: "bar", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), }, PosRange: PositionRange{ Start: 47, @@ -1327,10 +1328,9 @@ var testExpr = []struct { { input: "foo", expected: &VectorSelector{ - Name: "foo", - Offset: 0, + Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 0, @@ -1340,10 +1340,9 @@ var testExpr = []struct { }, { input: "min", expected: &VectorSelector{ - Name: "min", - Offset: 0, + Name: "min", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "min"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "min"), }, PosRange: PositionRange{ Start: 0, @@ -1353,10 +1352,10 @@ var testExpr = []struct { }, { input: "foo offset 5m", expected: &VectorSelector{ - Name: "foo", - Offset: 5 * time.Minute, + Name: "foo", + OriginalOffset: 5 * time.Minute, LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 0, @@ -1366,10 +1365,10 @@ var testExpr = []struct { }, { input: `foo OFFSET 1h30m`, expected: &VectorSelector{ - Name: "foo", - Offset: 90 * time.Minute, + Name: "foo", + OriginalOffset: 90 * time.Minute, LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 0, @@ -1379,24 +1378,186 @@ var testExpr = []struct { }, { input: `foo OFFSET 1m30ms`, expected: &VectorSelector{ - Name: "foo", - Offset: time.Minute + 30*time.Millisecond, + Name: "foo", + OriginalOffset: time.Minute + 30*time.Millisecond, LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 0, End: 17, }, }, + }, { + input: `foo @ 1603774568`, + expected: &VectorSelector{ + Name: "foo", + Timestamp: makeInt64Pointer(1603774568000), + LabelMatchers: []*labels.Matcher{ + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + }, + PosRange: PositionRange{ + Start: 0, + End: 16, + }, + }, + }, { + input: `foo @ -100`, + expected: &VectorSelector{ + Name: "foo", + Timestamp: makeInt64Pointer(-100000), + LabelMatchers: []*labels.Matcher{ + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + }, + PosRange: PositionRange{ + Start: 0, + End: 10, + }, + }, + }, { + input: `foo @ .3`, + expected: &VectorSelector{ + Name: "foo", + Timestamp: makeInt64Pointer(300), + LabelMatchers: []*labels.Matcher{ + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + }, + PosRange: PositionRange{ + Start: 0, + End: 8, + }, + }, + }, { + input: `foo @ 3.`, + expected: &VectorSelector{ + Name: "foo", + Timestamp: makeInt64Pointer(3000), + LabelMatchers: []*labels.Matcher{ + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + }, + PosRange: PositionRange{ + Start: 0, + End: 8, + }, + }, + }, { + input: `foo @ 3.33`, + expected: &VectorSelector{ + Name: "foo", + Timestamp: makeInt64Pointer(3330), + LabelMatchers: []*labels.Matcher{ + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + }, + PosRange: PositionRange{ + Start: 0, + End: 10, + }, + }, + }, { // Rounding off. + input: `foo @ 3.3333`, + expected: &VectorSelector{ + Name: "foo", + Timestamp: makeInt64Pointer(3333), + LabelMatchers: []*labels.Matcher{ + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + }, + PosRange: PositionRange{ + Start: 0, + End: 12, + }, + }, + }, { // Rounding off. + input: `foo @ 3.3335`, + expected: &VectorSelector{ + Name: "foo", + Timestamp: makeInt64Pointer(3334), + LabelMatchers: []*labels.Matcher{ + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + }, + PosRange: PositionRange{ + Start: 0, + End: 12, + }, + }, + }, { + input: `foo @ 3e2`, + expected: &VectorSelector{ + Name: "foo", + Timestamp: makeInt64Pointer(300000), + LabelMatchers: []*labels.Matcher{ + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + }, + PosRange: PositionRange{ + Start: 0, + End: 9, + }, + }, + }, { + input: `foo @ 3e-1`, + expected: &VectorSelector{ + Name: "foo", + Timestamp: makeInt64Pointer(300), + LabelMatchers: []*labels.Matcher{ + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + }, + PosRange: PositionRange{ + Start: 0, + End: 10, + }, + }, + }, { + input: `foo @ 0xA`, + expected: &VectorSelector{ + Name: "foo", + Timestamp: makeInt64Pointer(10000), + LabelMatchers: []*labels.Matcher{ + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + }, + PosRange: PositionRange{ + Start: 0, + End: 9, + }, + }, + }, { + input: `foo @ -3.3e1`, + expected: &VectorSelector{ + Name: "foo", + Timestamp: makeInt64Pointer(-33000), + LabelMatchers: []*labels.Matcher{ + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + }, + PosRange: PositionRange{ + Start: 0, + End: 12, + }, + }, + }, { + input: `foo @ +Inf`, + fail: true, + errMsg: "1:1: parse error: timestamp out of bounds for @ modifier: +Inf", + }, { + input: `foo @ -Inf`, + fail: true, + errMsg: "1:1: parse error: timestamp out of bounds for @ modifier: -Inf", + }, { + input: `foo @ NaN`, + fail: true, + errMsg: "1:1: parse error: timestamp out of bounds for @ modifier: NaN", + }, { + input: fmt.Sprintf(`foo @ %f`, float64(math.MaxInt64)+1), + fail: true, + errMsg: fmt.Sprintf("1:1: parse error: timestamp out of bounds for @ modifier: %f", float64(math.MaxInt64)+1), + }, { + input: fmt.Sprintf(`foo @ %f`, float64(math.MinInt64)-1), + fail: true, + errMsg: fmt.Sprintf("1:1: parse error: timestamp out of bounds for @ modifier: %f", float64(math.MinInt64)-1), }, { input: `foo:bar{a="bc"}`, expected: &VectorSelector{ - Name: "foo:bar", - Offset: 0, + Name: "foo:bar", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, "a", "bc"), - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo:bar"), + MustLabelMatcher(labels.MatchEqual, "a", "bc"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo:bar"), }, PosRange: PositionRange{ Start: 0, @@ -1406,11 +1567,10 @@ var testExpr = []struct { }, { input: `foo{NaN='bc'}`, expected: &VectorSelector{ - Name: "foo", - Offset: 0, + Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, "NaN", "bc"), - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, "NaN", "bc"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 0, @@ -1420,11 +1580,10 @@ var testExpr = []struct { }, { input: `foo{bar='}'}`, expected: &VectorSelector{ - Name: "foo", - Offset: 0, + Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, "bar", "}"), - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, "bar", "}"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 0, @@ -1434,14 +1593,13 @@ var testExpr = []struct { }, { input: `foo{a="b", foo!="bar", test=~"test", bar!~"baz"}`, expected: &VectorSelector{ - Name: "foo", - Offset: 0, + Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, "a", "b"), - mustLabelMatcher(labels.MatchNotEqual, "foo", "bar"), - mustLabelMatcher(labels.MatchRegexp, "test", "test"), - mustLabelMatcher(labels.MatchNotRegexp, "bar", "baz"), - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, "a", "b"), + MustLabelMatcher(labels.MatchNotEqual, "foo", "bar"), + MustLabelMatcher(labels.MatchRegexp, "test", "test"), + MustLabelMatcher(labels.MatchNotRegexp, "bar", "baz"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 0, @@ -1451,14 +1609,13 @@ var testExpr = []struct { }, { input: `foo{a="b", foo!="bar", test=~"test", bar!~"baz",}`, expected: &VectorSelector{ - Name: "foo", - Offset: 0, + Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, "a", "b"), - mustLabelMatcher(labels.MatchNotEqual, "foo", "bar"), - mustLabelMatcher(labels.MatchRegexp, "test", "test"), - mustLabelMatcher(labels.MatchNotRegexp, "bar", "baz"), - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, "a", "b"), + MustLabelMatcher(labels.MatchNotEqual, "foo", "bar"), + MustLabelMatcher(labels.MatchRegexp, "test", "test"), + MustLabelMatcher(labels.MatchNotRegexp, "bar", "baz"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 0, @@ -1557,10 +1714,9 @@ var testExpr = []struct { input: "test[5s]", expected: &MatrixSelector{ VectorSelector: &VectorSelector{ - Name: "test", - Offset: 0, + Name: "test", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "test"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "test"), }, PosRange: PositionRange{ Start: 0, @@ -1574,10 +1730,9 @@ var testExpr = []struct { input: "test[5m]", expected: &MatrixSelector{ VectorSelector: &VectorSelector{ - Name: "test", - Offset: 0, + Name: "test", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "test"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "test"), }, PosRange: PositionRange{ Start: 0, @@ -1591,10 +1746,9 @@ var testExpr = []struct { input: `foo[5m30s]`, expected: &MatrixSelector{ VectorSelector: &VectorSelector{ - Name: "foo", - Offset: 0, + Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 0, @@ -1608,10 +1762,10 @@ var testExpr = []struct { input: "test[5h] OFFSET 5m", expected: &MatrixSelector{ VectorSelector: &VectorSelector{ - Name: "test", - Offset: 5 * time.Minute, + Name: "test", + OriginalOffset: 5 * time.Minute, LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "test"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "test"), }, PosRange: PositionRange{ Start: 0, @@ -1625,10 +1779,10 @@ var testExpr = []struct { input: "test[5d] OFFSET 10s", expected: &MatrixSelector{ VectorSelector: &VectorSelector{ - Name: "test", - Offset: 10 * time.Second, + Name: "test", + OriginalOffset: 10 * time.Second, LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "test"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "test"), }, PosRange: PositionRange{ Start: 0, @@ -1642,10 +1796,10 @@ var testExpr = []struct { input: "test[5w] offset 2w", expected: &MatrixSelector{ VectorSelector: &VectorSelector{ - Name: "test", - Offset: 14 * 24 * time.Hour, + Name: "test", + OriginalOffset: 14 * 24 * time.Hour, LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "test"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "test"), }, PosRange: PositionRange{ Start: 0, @@ -1659,11 +1813,11 @@ var testExpr = []struct { input: `test{a="b"}[5y] OFFSET 3d`, expected: &MatrixSelector{ VectorSelector: &VectorSelector{ - Name: "test", - Offset: 3 * 24 * time.Hour, + Name: "test", + OriginalOffset: 3 * 24 * time.Hour, LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, "a", "b"), - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "test"), + MustLabelMatcher(labels.MatchEqual, "a", "b"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "test"), }, PosRange: PositionRange{ Start: 0, @@ -1673,6 +1827,24 @@ var testExpr = []struct { Range: 5 * 365 * 24 * time.Hour, EndPos: 25, }, + }, { + input: `test{a="b"}[5y] @ 1603774699`, + expected: &MatrixSelector{ + VectorSelector: &VectorSelector{ + Name: "test", + Timestamp: makeInt64Pointer(1603774699000), + LabelMatchers: []*labels.Matcher{ + MustLabelMatcher(labels.MatchEqual, "a", "b"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "test"), + }, + PosRange: PositionRange{ + Start: 0, + End: 11, + }, + }, + Range: 5 * 365 * 24 * time.Hour, + EndPos: 28, + }, }, { input: `foo[5mm]`, fail: true, @@ -1729,6 +1901,19 @@ var testExpr = []struct { fail: true, errMsg: "1:22: parse error: no offset modifiers allowed before range", }, { + input: `some_metric[5m] @ 1m`, + fail: true, + errMsg: "1:19: parse error: unexpected duration \"1m\" in @, expected timestamp", + }, { + input: `some_metric[5m] @`, + fail: true, + errMsg: "1:18: parse error: unexpected end of input in @, expected timestamp", + }, { + input: `some_metric @ 1234 [5m]`, + fail: true, + errMsg: "1:20: parse error: no @ modifiers allowed before range", + }, + { input: `(foo + bar)[5m]`, fail: true, errMsg: "1:12: parse error: ranges only allowed for vector selectors", @@ -1741,7 +1926,7 @@ var testExpr = []struct { Expr: &VectorSelector{ Name: "some_metric", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), }, PosRange: PositionRange{ Start: 13, @@ -1761,7 +1946,7 @@ var testExpr = []struct { Expr: &VectorSelector{ Name: "some_metric", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), }, PosRange: PositionRange{ Start: 13, @@ -1781,7 +1966,7 @@ var testExpr = []struct { Expr: &VectorSelector{ Name: "some_metric", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), }, PosRange: PositionRange{ Start: 13, @@ -1802,7 +1987,7 @@ var testExpr = []struct { Expr: &VectorSelector{ Name: "some_metric", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), }, PosRange: PositionRange{ Start: 19, @@ -1823,7 +2008,7 @@ var testExpr = []struct { Expr: &VectorSelector{ Name: "some_metric", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), }, PosRange: PositionRange{ Start: 5, @@ -1843,7 +2028,7 @@ var testExpr = []struct { Expr: &VectorSelector{ Name: "some_metric", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), }, PosRange: PositionRange{ Start: 7, @@ -1862,7 +2047,7 @@ var testExpr = []struct { Expr: &VectorSelector{ Name: "some_metric", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), }, PosRange: PositionRange{ Start: 16, @@ -1882,7 +2067,7 @@ var testExpr = []struct { Expr: &VectorSelector{ Name: "some_metric", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), }, PosRange: PositionRange{ Start: 10, @@ -1902,7 +2087,7 @@ var testExpr = []struct { Expr: &VectorSelector{ Name: "some_metric", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), }, PosRange: PositionRange{ Start: 18, @@ -1922,7 +2107,7 @@ var testExpr = []struct { Expr: &VectorSelector{ Name: "some_metric", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), }, PosRange: PositionRange{ Start: 14, @@ -1942,7 +2127,7 @@ var testExpr = []struct { Expr: &VectorSelector{ Name: "some_metric", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), }, PosRange: PositionRange{ Start: 8, @@ -1968,7 +2153,7 @@ var testExpr = []struct { Expr: &VectorSelector{ Name: "some_metric", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), }, PosRange: PositionRange{ Start: 22, @@ -1996,7 +2181,7 @@ var testExpr = []struct { Expr: &VectorSelector{ Name: "some_metric", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), }, PosRange: PositionRange{ Start: 53, @@ -2069,12 +2254,16 @@ var testExpr = []struct { input: `count_values(5, other_metric)`, fail: true, errMsg: "1:14: parse error: expected type string in aggregation parameter, got scalar", + }, { + input: `rate(some_metric[5m]) @ 1234`, + fail: true, + errMsg: "1:1: parse error: @ modifier must be preceded by an instant selector vector or range vector selector or a subquery", }, // Test function calls. { input: "time()", expected: &Call{ - Func: mustGetFunction("time"), + Func: MustGetFunction("time"), Args: Expressions{}, PosRange: PositionRange{ Start: 0, @@ -2084,13 +2273,13 @@ var testExpr = []struct { }, { input: `floor(some_metric{foo!="bar"})`, expected: &Call{ - Func: mustGetFunction("floor"), + Func: MustGetFunction("floor"), Args: Expressions{ &VectorSelector{ Name: "some_metric", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchNotEqual, "foo", "bar"), - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), + MustLabelMatcher(labels.MatchNotEqual, "foo", "bar"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), }, PosRange: PositionRange{ Start: 6, @@ -2106,13 +2295,13 @@ var testExpr = []struct { }, { input: "rate(some_metric[5m])", expected: &Call{ - Func: mustGetFunction("rate"), + Func: MustGetFunction("rate"), Args: Expressions{ &MatrixSelector{ VectorSelector: &VectorSelector{ Name: "some_metric", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), }, PosRange: PositionRange{ Start: 5, @@ -2131,12 +2320,12 @@ var testExpr = []struct { }, { input: "round(some_metric)", expected: &Call{ - Func: mustGetFunction("round"), + Func: MustGetFunction("round"), Args: Expressions{ &VectorSelector{ Name: "some_metric", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), }, PosRange: PositionRange{ Start: 6, @@ -2152,12 +2341,12 @@ var testExpr = []struct { }, { input: "round(some_metric, 5)", expected: &Call{ - Func: mustGetFunction("round"), + Func: MustGetFunction("round"), Args: Expressions{ &VectorSelector{ Name: "some_metric", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), }, PosRange: PositionRange{ Start: 6, @@ -2247,7 +2436,7 @@ var testExpr = []struct { Expr: &VectorSelector{ Name: "sum", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "sum"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "sum"), }, PosRange: PositionRange{ Start: 4, @@ -2266,7 +2455,7 @@ var testExpr = []struct { LHS: &VectorSelector{ Name: "a", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "a"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "a"), }, PosRange: PositionRange{ Start: 0, @@ -2276,7 +2465,7 @@ var testExpr = []struct { RHS: &VectorSelector{ Name: "sum", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "sum"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "sum"), }, PosRange: PositionRange{ Start: 4, @@ -2347,8 +2536,8 @@ var testExpr = []struct { Expr: &VectorSelector{ Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, "bar", "baz"), - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, "bar", "baz"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 0, @@ -2366,8 +2555,8 @@ var testExpr = []struct { Expr: &VectorSelector{ Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, "bar", "baz"), - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, "bar", "baz"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 0, @@ -2384,7 +2573,7 @@ var testExpr = []struct { Expr: &VectorSelector{ Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 0, @@ -2397,18 +2586,18 @@ var testExpr = []struct { }, { input: `min_over_time(rate(foo{bar="baz"}[2s])[5m:5s])`, expected: &Call{ - Func: mustGetFunction("min_over_time"), + Func: MustGetFunction("min_over_time"), Args: Expressions{ &SubqueryExpr{ Expr: &Call{ - Func: mustGetFunction("rate"), + Func: MustGetFunction("rate"), Args: Expressions{ &MatrixSelector{ VectorSelector: &VectorSelector{ Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, "bar", "baz"), - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, "bar", "baz"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 19, @@ -2439,18 +2628,18 @@ var testExpr = []struct { input: `min_over_time(rate(foo{bar="baz"}[2s])[5m:])[4m:3s]`, expected: &SubqueryExpr{ Expr: &Call{ - Func: mustGetFunction("min_over_time"), + Func: MustGetFunction("min_over_time"), Args: Expressions{ &SubqueryExpr{ Expr: &Call{ - Func: mustGetFunction("rate"), + Func: MustGetFunction("rate"), Args: Expressions{ &MatrixSelector{ VectorSelector: &VectorSelector{ Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, "bar", "baz"), - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, "bar", "baz"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 19, @@ -2483,18 +2672,18 @@ var testExpr = []struct { input: `min_over_time(rate(foo{bar="baz"}[2s])[5m:] offset 4m)[4m:3s]`, expected: &SubqueryExpr{ Expr: &Call{ - Func: mustGetFunction("min_over_time"), + Func: MustGetFunction("min_over_time"), Args: Expressions{ &SubqueryExpr{ Expr: &Call{ - Func: mustGetFunction("rate"), + Func: MustGetFunction("rate"), Args: Expressions{ &MatrixSelector{ VectorSelector: &VectorSelector{ Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, "bar", "baz"), - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, "bar", "baz"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 19, @@ -2510,9 +2699,9 @@ var testExpr = []struct { End: 38, }, }, - Range: 5 * time.Minute, - Offset: 4 * time.Minute, - EndPos: 53, + Range: 5 * time.Minute, + OriginalOffset: 4 * time.Minute, + EndPos: 53, }, }, PosRange: PositionRange{ @@ -2524,6 +2713,96 @@ var testExpr = []struct { Step: 3 * time.Second, EndPos: 61, }, + }, { + input: `min_over_time(rate(foo{bar="baz"}[2s])[5m:] @ 1603775091)[4m:3s]`, + expected: &SubqueryExpr{ + Expr: &Call{ + Func: MustGetFunction("min_over_time"), + Args: Expressions{ + &SubqueryExpr{ + Expr: &Call{ + Func: MustGetFunction("rate"), + Args: Expressions{ + &MatrixSelector{ + VectorSelector: &VectorSelector{ + Name: "foo", + LabelMatchers: []*labels.Matcher{ + MustLabelMatcher(labels.MatchEqual, "bar", "baz"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + }, + PosRange: PositionRange{ + Start: 19, + End: 33, + }, + }, + Range: 2 * time.Second, + EndPos: 37, + }, + }, + PosRange: PositionRange{ + Start: 14, + End: 38, + }, + }, + Range: 5 * time.Minute, + Timestamp: makeInt64Pointer(1603775091000), + EndPos: 56, + }, + }, + PosRange: PositionRange{ + Start: 0, + End: 57, + }, + }, + Range: 4 * time.Minute, + Step: 3 * time.Second, + EndPos: 64, + }, + }, { + input: `min_over_time(rate(foo{bar="baz"}[2s])[5m:] @ -160377509)[4m:3s]`, + expected: &SubqueryExpr{ + Expr: &Call{ + Func: MustGetFunction("min_over_time"), + Args: Expressions{ + &SubqueryExpr{ + Expr: &Call{ + Func: MustGetFunction("rate"), + Args: Expressions{ + &MatrixSelector{ + VectorSelector: &VectorSelector{ + Name: "foo", + LabelMatchers: []*labels.Matcher{ + MustLabelMatcher(labels.MatchEqual, "bar", "baz"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + }, + PosRange: PositionRange{ + Start: 19, + End: 33, + }, + }, + Range: 2 * time.Second, + EndPos: 37, + }, + }, + PosRange: PositionRange{ + Start: 14, + End: 38, + }, + }, + Range: 5 * time.Minute, + Timestamp: makeInt64Pointer(-160377509000), + EndPos: 56, + }, + }, + PosRange: PositionRange{ + Start: 0, + End: 57, + }, + }, + Range: 4 * time.Minute, + Step: 3 * time.Second, + EndPos: 64, + }, }, { input: "sum without(and, by, avg, count, alert, annotations)(some_metric) [30m:10s]", expected: &SubqueryExpr{ @@ -2533,7 +2812,7 @@ var testExpr = []struct { Expr: &VectorSelector{ Name: "some_metric", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), }, PosRange: PositionRange{ Start: 53, @@ -2556,18 +2835,93 @@ var testExpr = []struct { Expr: &VectorSelector{ Name: "some_metric", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), }, PosRange: PositionRange{ Start: 0, End: 21, }, - Offset: 1 * time.Minute, + OriginalOffset: 1 * time.Minute, }, Range: 10 * time.Minute, Step: 5 * time.Second, EndPos: 30, }, + }, { + input: `some_metric @ 123 [10m:5s]`, + expected: &SubqueryExpr{ + Expr: &VectorSelector{ + Name: "some_metric", + LabelMatchers: []*labels.Matcher{ + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), + }, + PosRange: PositionRange{ + Start: 0, + End: 17, + }, + Timestamp: makeInt64Pointer(123000), + }, + Range: 10 * time.Minute, + Step: 5 * time.Second, + EndPos: 26, + }, + }, { + input: `some_metric @ 123 offset 1m [10m:5s]`, + expected: &SubqueryExpr{ + Expr: &VectorSelector{ + Name: "some_metric", + LabelMatchers: []*labels.Matcher{ + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), + }, + PosRange: PositionRange{ + Start: 0, + End: 27, + }, + Timestamp: makeInt64Pointer(123000), + OriginalOffset: 1 * time.Minute, + }, + Range: 10 * time.Minute, + Step: 5 * time.Second, + EndPos: 36, + }, + }, { + input: `some_metric offset 1m @ 123 [10m:5s]`, + expected: &SubqueryExpr{ + Expr: &VectorSelector{ + Name: "some_metric", + LabelMatchers: []*labels.Matcher{ + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), + }, + PosRange: PositionRange{ + Start: 0, + End: 27, + }, + Timestamp: makeInt64Pointer(123000), + OriginalOffset: 1 * time.Minute, + }, + Range: 10 * time.Minute, + Step: 5 * time.Second, + EndPos: 36, + }, + }, { + input: `some_metric[10m:5s] offset 1m @ 123`, + expected: &SubqueryExpr{ + Expr: &VectorSelector{ + Name: "some_metric", + LabelMatchers: []*labels.Matcher{ + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "some_metric"), + }, + PosRange: PositionRange{ + Start: 0, + End: 11, + }, + }, + Timestamp: makeInt64Pointer(123000), + OriginalOffset: 1 * time.Minute, + Range: 10 * time.Minute, + Step: 5 * time.Second, + EndPos: 35, + }, }, { input: `(foo + bar{nm="val"})[5m:]`, expected: &SubqueryExpr{ @@ -2580,7 +2934,7 @@ var testExpr = []struct { LHS: &VectorSelector{ Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 1, @@ -2590,8 +2944,8 @@ var testExpr = []struct { RHS: &VectorSelector{ Name: "bar", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, "nm", "val"), - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), + MustLabelMatcher(labels.MatchEqual, "nm", "val"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), }, PosRange: PositionRange{ Start: 7, @@ -2619,7 +2973,7 @@ var testExpr = []struct { LHS: &VectorSelector{ Name: "foo", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), }, PosRange: PositionRange{ Start: 1, @@ -2629,8 +2983,8 @@ var testExpr = []struct { RHS: &VectorSelector{ Name: "bar", LabelMatchers: []*labels.Matcher{ - mustLabelMatcher(labels.MatchEqual, "nm", "val"), - mustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), + MustLabelMatcher(labels.MatchEqual, "nm", "val"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), }, PosRange: PositionRange{ Start: 7, @@ -2643,9 +2997,50 @@ var testExpr = []struct { End: 21, }, }, - Range: 5 * time.Minute, - Offset: 10 * time.Minute, - EndPos: 37, + Range: 5 * time.Minute, + OriginalOffset: 10 * time.Minute, + EndPos: 37, + }, + }, { + input: `(foo + bar{nm="val"} @ 1234)[5m:] @ 1603775019`, + expected: &SubqueryExpr{ + Expr: &ParenExpr{ + Expr: &BinaryExpr{ + Op: ADD, + VectorMatching: &VectorMatching{ + Card: CardOneToOne, + }, + LHS: &VectorSelector{ + Name: "foo", + LabelMatchers: []*labels.Matcher{ + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "foo"), + }, + PosRange: PositionRange{ + Start: 1, + End: 4, + }, + }, + RHS: &VectorSelector{ + Name: "bar", + LabelMatchers: []*labels.Matcher{ + MustLabelMatcher(labels.MatchEqual, "nm", "val"), + MustLabelMatcher(labels.MatchEqual, model.MetricNameLabel, "bar"), + }, + Timestamp: makeInt64Pointer(1234000), + PosRange: PositionRange{ + Start: 7, + End: 27, + }, + }, + }, + PosRange: PositionRange{ + Start: 0, + End: 28, + }, + }, + Range: 5 * time.Minute, + Timestamp: makeInt64Pointer(1603775019000), + EndPos: 46, }, }, { input: "test[5d] OFFSET 10s [10m:5s]", @@ -2655,9 +3050,23 @@ var testExpr = []struct { input: `(foo + bar{nm="val"})[5m:][10m:5s]`, fail: true, errMsg: `1:1: parse error: subquery is only allowed on instant vector, got matrix`, + }, { + input: "rate(food[1m])[1h] offset 1h", + fail: true, + errMsg: `1:15: parse error: ranges only allowed for vector selectors`, + }, { + input: "rate(food[1m])[1h] @ 100", + fail: true, + errMsg: `1:15: parse error: ranges only allowed for vector selectors`, }, } +func makeInt64Pointer(val int64) *int64 { + valp := new(int64) + *valp = val + return valp +} + func TestParseExpressions(t *testing.T) { for _, test := range testExpr { t.Run(test.input, func(t *testing.T) { @@ -2697,22 +3106,6 @@ func TestNaNExpression(t *testing.T) { require.True(t, math.IsNaN(float64(nl.Val)), "expected 'NaN' in number literal but got %v", nl.Val) } -func mustLabelMatcher(mt labels.MatchType, name, val string) *labels.Matcher { - m, err := labels.NewMatcher(mt, name, val) - if err != nil { - panic(err) - } - return m -} - -func mustGetFunction(name string) *Function { - f, ok := getFunction(name) - if !ok { - panic(errors.Errorf("function %q does not exist", name)) - } - return f -} - var testSeries = []struct { input string expectedMetric labels.Labels diff --git a/promql/parser/printer.go b/promql/parser/printer.go index eef4aa8e41..ba18a3db10 100644 --- a/promql/parser/printer.go +++ b/promql/parser/printer.go @@ -116,14 +116,24 @@ func (node *MatrixSelector) String() string { // Copy the Vector selector before changing the offset vecSelector := *node.VectorSelector.(*VectorSelector) offset := "" - if vecSelector.Offset != time.Duration(0) { - offset = fmt.Sprintf(" offset %s", model.Duration(vecSelector.Offset)) + if vecSelector.OriginalOffset != time.Duration(0) { + offset = fmt.Sprintf(" offset %s", model.Duration(vecSelector.OriginalOffset)) + } + at := "" + if vecSelector.Timestamp != nil { + at = fmt.Sprintf(" @ %.3f", float64(*vecSelector.Timestamp)/1000.0) } - // Do not print the offset twice. - vecSelector.Offset = 0 + // Do not print the @ and offset twice. + offsetVal, atVal := vecSelector.OriginalOffset, vecSelector.Timestamp + vecSelector.OriginalOffset = 0 + vecSelector.Timestamp = nil - return fmt.Sprintf("%s[%s]%s", vecSelector.String(), model.Duration(node.Range), offset) + str := fmt.Sprintf("%s[%s]%s%s", vecSelector.String(), model.Duration(node.Range), at, offset) + + vecSelector.OriginalOffset, vecSelector.Timestamp = offsetVal, atVal + + return str } func (node *SubqueryExpr) String() string { @@ -132,10 +142,14 @@ func (node *SubqueryExpr) String() string { step = model.Duration(node.Step).String() } offset := "" - if node.Offset != time.Duration(0) { - offset = fmt.Sprintf(" offset %s", model.Duration(node.Offset)) + if node.OriginalOffset != time.Duration(0) { + offset = fmt.Sprintf(" offset %s", model.Duration(node.OriginalOffset)) } - return fmt.Sprintf("%s[%s:%s]%s", node.Expr.String(), model.Duration(node.Range), step, offset) + at := "" + if node.Timestamp != nil { + at = fmt.Sprintf(" @ %.3f", float64(*node.Timestamp)/1000.0) + } + return fmt.Sprintf("%s[%s:%s]%s%s", node.Expr.String(), model.Duration(node.Range), step, at, offset) } func (node *NumberLiteral) String() string { @@ -164,13 +178,17 @@ func (node *VectorSelector) String() string { labelStrings = append(labelStrings, matcher.String()) } offset := "" - if node.Offset != time.Duration(0) { - offset = fmt.Sprintf(" offset %s", model.Duration(node.Offset)) + if node.OriginalOffset != time.Duration(0) { + offset = fmt.Sprintf(" offset %s", model.Duration(node.OriginalOffset)) + } + at := "" + if node.Timestamp != nil { + at = fmt.Sprintf(" @ %.3f", float64(*node.Timestamp)/1000.0) } if len(labelStrings) == 0 { - return fmt.Sprintf("%s%s", node.Name, offset) + return fmt.Sprintf("%s%s%s", node.Name, at, offset) } sort.Strings(labelStrings) - return fmt.Sprintf("%s{%s}%s", node.Name, strings.Join(labelStrings, ","), offset) + return fmt.Sprintf("%s{%s}%s%s", node.Name, strings.Join(labelStrings, ","), at, offset) } diff --git a/promql/test.go b/promql/test.go index 7992c5b23a..d02f263175 100644 --- a/promql/test.go +++ b/promql/test.go @@ -27,6 +27,7 @@ import ( "github.com/prometheus/common/model" "github.com/prometheus/prometheus/pkg/labels" + "github.com/prometheus/prometheus/pkg/timestamp" "github.com/prometheus/prometheus/promql/parser" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb" @@ -427,6 +428,74 @@ func (t *Test) Run() error { return nil } +type atModifierTestCase struct { + expr string + evalTime time.Time +} + +func atModifierTestCases(exprStr string, evalTime time.Time) ([]atModifierTestCase, error) { + expr, err := parser.ParseExpr(exprStr) + if err != nil { + return nil, err + } + ts := timestamp.FromTime(evalTime) + + containsNonStepInvariant := false + // Setting the @ timestamp for all selectors to be evalTime. + // If there is a subquery, then the selectors inside it don't get the @ timestamp. + // If any selector already has the @ timestamp set, then it is untouched. + parser.Inspect(expr, func(node parser.Node, path []parser.Node) error { + _, _, subqTs := subqueryTimes(path) + if subqTs != nil { + // There is a subquery with timestamp in the path, + // hence don't change any timestamps further. + return nil + } + switch n := node.(type) { + case *parser.VectorSelector: + if n.Timestamp == nil { + n.Timestamp = makeInt64Pointer(ts) + } + + case *parser.MatrixSelector: + if vs := n.VectorSelector.(*parser.VectorSelector); vs.Timestamp == nil { + vs.Timestamp = makeInt64Pointer(ts) + } + + case *parser.SubqueryExpr: + if n.Timestamp == nil { + n.Timestamp = makeInt64Pointer(ts) + } + + case *parser.Call: + _, ok := AtModifierUnsafeFunctions[n.Func.Name] + containsNonStepInvariant = containsNonStepInvariant || ok + } + return nil + }) + + if containsNonStepInvariant { + // Since there is a step invariant function, we cannot automatically + // generate step invariant test cases for it sanely. + return nil, nil + } + + newExpr := expr.String() // With all the @ evalTime set. + additionalEvalTimes := []int64{-10 * ts, 0, ts / 5, ts, 10 * ts} + if ts == 0 { + additionalEvalTimes = []int64{-1000, -ts, 1000} + } + testCases := make([]atModifierTestCase, 0, len(additionalEvalTimes)) + for _, et := range additionalEvalTimes { + testCases = append(testCases, atModifierTestCase{ + expr: newExpr, + evalTime: timestamp.Time(et), + }) + } + + return testCases, nil +} + // exec processes a single step of the test. func (t *Test) exec(tc testCommand) error { switch cmd := tc.(type) { @@ -445,59 +514,66 @@ func (t *Test) exec(tc testCommand) error { } case *evalCmd: - q, err := t.QueryEngine().NewInstantQuery(t.storage, cmd.expr, cmd.start) + queries, err := atModifierTestCases(cmd.expr, cmd.start) if err != nil { return err } - defer q.Close() - res := q.Exec(t.context) - if res.Err != nil { - if cmd.fail { - return nil + queries = append([]atModifierTestCase{{expr: cmd.expr, evalTime: cmd.start}}, queries...) + for _, iq := range queries { + q, err := t.QueryEngine().NewInstantQuery(t.storage, iq.expr, iq.evalTime) + if err != nil { + return err + } + defer q.Close() + res := q.Exec(t.context) + if res.Err != nil { + if cmd.fail { + continue + } + return errors.Wrapf(res.Err, "error evaluating query %q (line %d)", iq.expr, cmd.line) + } + if res.Err == nil && cmd.fail { + return errors.Errorf("expected error evaluating query %q (line %d) but got none", iq.expr, cmd.line) + } + err = cmd.compareResult(res.Value) + if err != nil { + return errors.Wrapf(err, "error in %s %s", cmd, iq.expr) } - return errors.Wrapf(res.Err, "error evaluating query %q (line %d)", cmd.expr, cmd.line) - } - if res.Err == nil && cmd.fail { - return errors.Errorf("expected error evaluating query %q (line %d) but got none", cmd.expr, cmd.line) - } - err = cmd.compareResult(res.Value) - if err != nil { - return errors.Wrapf(err, "error in %s %s", cmd, cmd.expr) - } - - // Check query returns same result in range mode, - // by checking against the middle step. - q, err = t.queryEngine.NewRangeQuery(t.storage, cmd.expr, cmd.start.Add(-time.Minute), cmd.start.Add(time.Minute), time.Minute) - if err != nil { - return err - } - rangeRes := q.Exec(t.context) - if rangeRes.Err != nil { - return errors.Wrapf(rangeRes.Err, "error evaluating query %q (line %d) in range mode", cmd.expr, cmd.line) - } - defer q.Close() - if cmd.ordered { - // Ordering isn't defined for range queries. - return nil - } - mat := rangeRes.Value.(Matrix) - vec := make(Vector, 0, len(mat)) - for _, series := range mat { - for _, point := range series.Points { - if point.T == timeMilliseconds(cmd.start) { - vec = append(vec, Sample{Metric: series.Metric, Point: point}) - break + // Check query returns same result in range mode, + // by checking against the middle step. + q, err = t.queryEngine.NewRangeQuery(t.storage, iq.expr, iq.evalTime.Add(-time.Minute), iq.evalTime.Add(time.Minute), time.Minute) + if err != nil { + return err + } + rangeRes := q.Exec(t.context) + if rangeRes.Err != nil { + return errors.Wrapf(rangeRes.Err, "error evaluating query %q (line %d) in range mode", iq.expr, cmd.line) + } + defer q.Close() + if cmd.ordered { + // Ordering isn't defined for range queries. + continue + } + mat := rangeRes.Value.(Matrix) + vec := make(Vector, 0, len(mat)) + for _, series := range mat { + for _, point := range series.Points { + if point.T == timeMilliseconds(iq.evalTime) { + vec = append(vec, Sample{Metric: series.Metric, Point: point}) + break + } } } - } - if _, ok := res.Value.(Scalar); ok { - err = cmd.compareResult(Scalar{V: vec[0].Point.V}) - } else { - err = cmd.compareResult(vec) - } - if err != nil { - return errors.Wrapf(err, "error in %s %s (line %d) rande mode", cmd, cmd.expr, cmd.line) + if _, ok := res.Value.(Scalar); ok { + err = cmd.compareResult(Scalar{V: vec[0].Point.V}) + } else { + err = cmd.compareResult(vec) + } + if err != nil { + return errors.Wrapf(err, "error in %s %s (line %d) rande mode", cmd, iq.expr, cmd.line) + } + } default: @@ -524,6 +600,7 @@ func (t *Test) clear() { MaxSamples: 10000, Timeout: 100 * time.Second, NoStepSubqueryIntervalFn: func(int64) int64 { return durationMilliseconds(1 * time.Minute) }, + EnableAtModifier: true, } t.queryEngine = NewEngine(opts) @@ -633,10 +710,11 @@ func (ll *LazyLoader) clear() { ll.storage = teststorage.New(ll) opts := EngineOpts{ - Logger: nil, - Reg: nil, - MaxSamples: 10000, - Timeout: 100 * time.Second, + Logger: nil, + Reg: nil, + MaxSamples: 10000, + Timeout: 100 * time.Second, + EnableAtModifier: true, } ll.queryEngine = NewEngine(opts) @@ -701,3 +779,9 @@ func (ll *LazyLoader) Close() { ll.T.Fatalf("closing test storage: %s", err) } } + +func makeInt64Pointer(val int64) *int64 { + valp := new(int64) + *valp = val + return valp +} diff --git a/promql/testdata/at_modifier.test b/promql/testdata/at_modifier.test new file mode 100644 index 0000000000..18cd93a4ba --- /dev/null +++ b/promql/testdata/at_modifier.test @@ -0,0 +1,166 @@ +load 10s + metric{job="1"} 0+1x1000 + metric{job="2"} 0+2x1000 + +load 1ms + metric_ms 0+1x10000 + +# Instant vector selectors. +eval instant at 10s metric @ 100 + metric{job="1"} 10 + metric{job="2"} 20 + +eval instant at 10s metric @ 100 offset 50s + metric{job="1"} 5 + metric{job="2"} 10 + +eval instant at 10s metric offset 50s @ 100 + metric{job="1"} 5 + metric{job="2"} 10 + +eval instant at 10s -metric @ 100 + {job="1"} -10 + {job="2"} -20 + +eval instant at 10s ---metric @ 100 + {job="1"} -10 + {job="2"} -20 + +# Millisecond precision. +eval instant at 100s metric_ms @ 1.234 + metric_ms 1234 + +# Range vector selectors. +eval instant at 25s sum_over_time(metric{job="1"}[100s] @ 100) + {job="1"} 55 + +eval instant at 25s sum_over_time(metric{job="1"}[100s] @ 100 offset 50s) + {job="1"} 15 + +eval instant at 25s sum_over_time(metric{job="1"}[100s] offset 50s @ 100) + {job="1"} 15 + +# Different timestamps. +eval instant at 25s metric{job="1"} @ 50 + metric{job="1"} @ 100 + {job="1"} 15 + +eval instant at 25s rate(metric{job="1"}[100s] @ 100) + label_replace(rate(metric{job="2"}[123s] @ 200), "job", "1", "", "") + {job="1"} 0.3 + +eval instant at 25s sum_over_time(metric{job="1"}[100s] @ 100) + label_replace(sum_over_time(metric{job="2"}[100s] @ 100), "job", "1", "", "") + {job="1"} 165 + +# Subqueries. + +# 10*(1+2+...+9) + 10. +eval instant at 25s sum_over_time(metric{job="1"}[100s:1s] @ 100) + {job="1"} 460 + +# 10*(1+2+...+7) + 8. +eval instant at 25s sum_over_time(metric{job="1"}[100s:1s] @ 100 offset 20s) + {job="1"} 288 + +# 10*(1+2+...+7) + 8. +eval instant at 25s sum_over_time(metric{job="1"}[100s:1s] offset 20s @ 100) + {job="1"} 288 + +# Subquery with different timestamps. + +# Since vector selector has timestamp, the result value does not depend on the timestamp of subqueries. +# Inner most sum=1+2+...+10=55. +# With [100s:25s] subquery, it's 55*5. +eval instant at 100s sum_over_time(sum_over_time(metric{job="1"}[100s] @ 100)[100s:25s] @ 50) + {job="1"} 275 + +# Nested subqueries with different timestamps on both. + +# Since vector selector has timestamp, the result value does not depend on the timestamp of subqueries. +# Sum of innermost subquery is 275 as above. The outer subquery repeats it 4 times. +eval instant at 0s sum_over_time(sum_over_time(sum_over_time(metric{job="1"}[100s] @ 100)[100s:25s] @ 50)[3s:1s] @ 3000) + {job="1"} 1100 + +# Testing the inner subquery timestamp since vector selector does not have @. + +# Inner sum for subquery [100s:25s] @ 50 are +# at -50 nothing, at -25 nothing, at 0=0, at 25=2, at 50=4+5=9. +# This sum of 11 is repeated 4 times by outer subquery. +eval instant at 0s sum_over_time(sum_over_time(sum_over_time(metric{job="1"}[10s])[100s:25s] @ 50)[3s:1s] @ 200) + {job="1"} 44 + +# Inner sum for subquery [100s:25s] @ 200 are +# at 100=9+10, at 125=12, at 150=14+15, at 175=17, at 200=19+20. +# This sum of 116 is repeated 4 times by outer subquery. +eval instant at 0s sum_over_time(sum_over_time(sum_over_time(metric{job="1"}[10s])[100s:25s] @ 200)[3s:1s] @ 50) + {job="1"} 464 + +# Nested subqueries with timestamp only on outer subquery. +# Outer most subquery: +# at 900=783 +# inner subquery: at 870=87+86+85, at 880=88+87+86, at 890=89+88+87 +# at 925=537 +# inner subquery: at 895=89+88, at 905=90+89, at 915=90+91 +# at 950=828 +# inner subquery: at 920=92+91+90, at 930=93+92+91, at 940=94+93+92 +# at 975=567 +# inner subquery: at 945=94+93, at 955=95+94, at 965=96+95 +# at 1000=873 +# inner subquery: at 970=97+96+95, at 980=98+97+96, at 990=99+98+97 +eval instant at 0s sum_over_time(sum_over_time(sum_over_time(metric{job="1"}[20s])[20s:10s] offset 10s)[100s:25s] @ 1000) + {job="1"} 3588 + +# minute is counted on the value of the sample. +eval instant at 10s minute(metric @ 1500) + {job="1"} 2 + {job="2"} 5 + +# timestamp() takes the time of the sample and not the evaluation time. +eval instant at 10m timestamp(metric{job="1"} @ 10) + {job="1"} 10 + +# The result of inner timestamp() will have the timestamp as the +# eval time, hence entire expression is not step invariant and depends on eval time. +eval instant at 10m timestamp(timestamp(metric{job="1"} @ 10)) + {job="1"} 600 + +eval instant at 15m timestamp(timestamp(metric{job="1"} @ 10)) + {job="1"} 900 + +# Time functions inside a subquery. + +# minute is counted on the value of the sample. +eval instant at 0s sum_over_time(minute(metric @ 1500)[100s:10s]) + {job="1"} 22 + {job="2"} 55 + +# If nothing passed, minute() takes eval time. +# Here the eval time is determined by the subquery. +# [50m:1m] at 6000, i.e. 100m, is 50m to 100m. +# sum=50+51+52+...+59+0+1+2+...+40. +eval instant at 0s sum_over_time(minute()[50m:1m] @ 6000) + {} 1365 + +# sum=45+46+47+...+59+0+1+2+...+35. +eval instant at 0s sum_over_time(minute()[50m:1m] @ 6000 offset 5m) + {} 1410 + +# time() is the eval time which is determined by subquery here. +# 2900+2901+...+3000 = (3000*3001 - 2899*2900)/2. +eval instant at 0s sum_over_time(vector(time())[100s:1s] @ 3000) + {} 297950 + +# 2300+2301+...+2400 = (2400*2401 - 2299*2300)/2. +eval instant at 0s sum_over_time(vector(time())[100s:1s] @ 3000 offset 600s) + {} 237350 + +# timestamp() takes the time of the sample and not the evaluation time. +eval instant at 0s sum_over_time(timestamp(metric{job="1"} @ 10)[100s:10s] @ 3000) + {job="1"} 110 + +# The result of inner timestamp() will have the timestamp as the +# eval time, hence entire expression is not step invariant and depends on eval time. +# Here eval time is determined by the subquery. +eval instant at 0s sum_over_time(timestamp(timestamp(metric{job="1"} @ 999))[10s:1s] @ 10) + {job="1"} 55 + + +clear diff --git a/promql/testdata/functions.test b/promql/testdata/functions.test index 0279f880d1..b9dc27cecb 100644 --- a/promql/testdata/functions.test +++ b/promql/testdata/functions.test @@ -224,6 +224,18 @@ eval instant at 50m deriv(testcounter_reset_middle[100m]) eval instant at 50m predict_linear(testcounter_reset_middle[100m], 3600) {} 76.81818181818181 +# intercept at t = 3000+3600 = 6600 +eval instant at 50m predict_linear(testcounter_reset_middle[100m] @ 3000, 3600) + {} 76.81818181818181 + +# intercept at t = 600+3600 = 4200 +eval instant at 10m predict_linear(testcounter_reset_middle[100m] @ 3000, 3600) + {} 51.36363636363637 + +# intercept at t = 4200+3600 = 7800 +eval instant at 70m predict_linear(testcounter_reset_middle[100m] @ 3000, 3600) + {} 89.54545454545455 + # With http_requests, there is a sample value exactly at the end of # the range, and it has exactly the predicted value, so predict_linear # can be emulated with deriv. @@ -829,8 +841,6 @@ eval instant at 16m absent_over_time(httpd_handshake_failures_total[1m]) eval instant at 16m absent_over_time({instance="127.0.0.1"}[5m]) -eval instant at 16m absent_over_time({instance="127.0.0.1"}[5m]) - eval instant at 21m absent_over_time({instance="127.0.0.1"}[5m]) {instance="127.0.0.1"} 1 diff --git a/promql/testdata/histograms.test b/promql/testdata/histograms.test index ec236576b4..a9bb9b5e45 100644 --- a/promql/testdata/histograms.test +++ b/promql/testdata/histograms.test @@ -105,7 +105,7 @@ eval instant at 50m histogram_quantile(0.3, avg(rate(request_duration_seconds_bu eval instant at 50m histogram_quantile(0.5, avg(rate(request_duration_seconds_bucket[5m])) by (le)) {} 0.12777777777777778 -# Aggregated histogram: By job. +# Aggregated histogram: By instance. eval instant at 50m histogram_quantile(0.3, sum(rate(request_duration_seconds_bucket[5m])) by (le, instance)) {instance="ins1"} 0.075 {instance="ins2"} 0.075 @@ -114,7 +114,7 @@ eval instant at 50m histogram_quantile(0.5, sum(rate(request_duration_seconds_bu {instance="ins1"} 0.1333333333 {instance="ins2"} 0.125 -# Aggregated histogram: By instance. +# Aggregated histogram: By job. eval instant at 50m histogram_quantile(0.3, sum(rate(request_duration_seconds_bucket[5m])) by (le, job)) {job="job1"} 0.1 {job="job2"} 0.0642857142857143 @@ -190,4 +190,4 @@ load 5m empty_bucket{le="+Inf", job="job1", instance="ins1"} 0x10 eval instant at 50m histogram_quantile(0.2, rate(empty_bucket[5m])) - {instance="ins1", job="job1"} NaN \ No newline at end of file + {instance="ins1", job="job1"} NaN