diff --git a/cmd/prometheus/main.go b/cmd/prometheus/main.go index f6b7611e8e..5fba5dd748 100644 --- a/cmd/prometheus/main.go +++ b/cmd/prometheus/main.go @@ -275,6 +275,9 @@ func (c *flagConfig) setFeatureListOptions(logger *slog.Logger) error { case "promql-delayed-name-removal": c.promqlEnableDelayedNameRemoval = true logger.Info("Experimental PromQL delayed name removal enabled.") + case "promql-extended-range-selectors": + parser.EnableExtendedRangeSelectors = true + logger.Info("Experimental PromQL extended range selectors enabled.") case "": continue case "old-ui": @@ -561,7 +564,7 @@ func main() { a.Flag("scrape.discovery-reload-interval", "Interval used by scrape manager to throttle target groups updates."). Hidden().Default("5s").SetValue(&cfg.scrape.DiscoveryReloadInterval) - a.Flag("enable-feature", "Comma separated feature names to enable. Valid options: exemplar-storage, expand-external-labels, memory-snapshot-on-shutdown, promql-per-step-stats, promql-experimental-functions, extra-scrape-metrics, auto-gomaxprocs, native-histograms, created-timestamp-zero-ingestion, concurrent-rule-eval, delayed-compaction, old-ui, otlp-deltatocumulative, promql-duration-expr, use-uncached-io. See https://prometheus.io/docs/prometheus/latest/feature_flags/ for more details."). + a.Flag("enable-feature", "Comma separated feature names to enable. Valid options: exemplar-storage, expand-external-labels, memory-snapshot-on-shutdown, promql-per-step-stats, promql-experimental-functions, extra-scrape-metrics, auto-gomaxprocs, native-histograms, created-timestamp-zero-ingestion, concurrent-rule-eval, delayed-compaction, old-ui, otlp-deltatocumulative, promql-duration-expr, use-uncached-io, promql-extended-range-selectors. See https://prometheus.io/docs/prometheus/latest/feature_flags/ for more details."). Default("").StringsVar(&cfg.featureList) a.Flag("agent", "Run Prometheus in 'Agent mode'.").BoolVar(&agentMode) diff --git a/docs/feature_flags.md b/docs/feature_flags.md index 4c390ab92e..f9e5763f8a 100644 --- a/docs/feature_flags.md +++ b/docs/feature_flags.md @@ -302,3 +302,21 @@ memory in response to misleading cache growth. This is currently implemented using direct I/O. For more details, see the [proposal](https://github.com/prometheus/proposals/pull/45). + +## Extended Range Selectors + +`--enable-feature=promql-extended-range-selectors` + +Enables experimental `anchored` and `smoothed` modifiers for PromQL range and instant selectors. These modifiers provide more control over how range boundaries are handled in functions like `rate` and `increase`, especially with missing or irregular data. + +- `anchored`: Treats the range as closed on both ends, using the most recent sample before each boundary or duplicating the first/last sample if needed. Useful for precise counter calculations and improved composability. Use with **increase** and **delta**. +- `smoothed`: Linearly interpolates values at the range boundaries, using all available data for more accurate results with irregular scrapes or missing samples. Use with **rate** and **deriv**. + +Example queries: +`increase(http_requests_total[5m] anchored)` +`rate(http_requests_total[5m] smoothed)` + +> **Note for alerting and recording rules:** +> The `smoothed` modifier requires samples after the evaluation interval, so using it directly in alerting or recording rules will typically *under-estimate* the result, as future samples are not available at evaluation time. +> To use `smoothed` safely in rules, you **must** apply a `query_offset` (e.g., `offset 1m`) to ensure the calculation window is fully in the past and all needed samples are available. +> For critical alerting, set the offset to at least one scrape interval; for less critical or more resilient use cases, consider a larger offset (multiple scrape intervals) to tolerate missed scrapes. diff --git a/promql/engine.go b/promql/engine.go index 3cdf299dff..cab9f80e66 100644 --- a/promql/engine.go +++ b/promql/engine.go @@ -926,13 +926,27 @@ func getTimeRangesForSelector(s *parser.EvalStmt, n *parser.VectorSelector, path // because wo want to exclude samples that are precisely the // lookback delta before the eval time. start -= durationMilliseconds(s.LookbackDelta) - 1 + if n.Smoothed { + end += durationMilliseconds(s.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. We subtract one from the range to - // exclude samples positioned directly at the lower boundary of - // the range. - start -= durationMilliseconds(evalRange) - 1 + // For matrix queries, adjust the start and end times to ensure the + // correct range of data is selected. For "anchored" selectors, extend + // the start time backwards by the lookback delta plus the evaluation + // range. For "smoothed" selectors, extend both the start and end times + // by the lookback delta, and also extend the start time by the + // evaluation range to cover the smoothing window. For standard range + // queries, extend the start time backwards by the range (minus one + // millisecond) to exclude samples exactly at the lower boundary. + switch { + case n.Anchored: + start -= durationMilliseconds(s.LookbackDelta + evalRange) + case n.Smoothed: + start -= durationMilliseconds(s.LookbackDelta + evalRange) + end += durationMilliseconds(s.LookbackDelta) + default: + start -= durationMilliseconds(evalRange) - 1 + } } offsetMilliseconds := durationMilliseconds(n.OriginalOffset) @@ -979,7 +993,6 @@ func (ng *Engine) populateSeries(ctx context.Context, querier storage.Querier, s evalRange = 0 hints.By, hints.Grouping = extractGroupsFromPath(path) n.UnexpandedSeriesSet = querier.Select(ctx, false, hints, n.LabelMatchers...) - case *parser.MatrixSelector: evalRange = n.Range } @@ -1516,6 +1529,57 @@ func (ev *evaluator) rangeEvalAgg(ctx context.Context, aggExpr *parser.Aggregate return result, annos } +func (ev *evaluator) smoothSeries(series []storage.Series, offset time.Duration, recordOrigT bool) Matrix { + dur := ev.endTimestamp - ev.startTimestamp + it := storage.NewBuffer(dur + 2*durationMilliseconds(ev.lookbackDelta)) + var chkIter chunkenc.Iterator + mat := make(Matrix, 0, len(series)) + for _, s := range series { + ss := Series{ + Metric: s.Labels(), + } + chkIter := s.Iterator(chkIter) + it.Reset(chkIter) + + var floats []FPoint + + for ts, step := ev.startTimestamp, -1; ts <= ev.endTimestamp; ts += ev.interval { + matrixStart := ts - durationMilliseconds(ev.lookbackDelta) + matrixEnd := ts + durationMilliseconds(ev.lookbackDelta) + floats, _ = ev.matrixIterSlice(it, matrixStart, matrixEnd, floats, nil) + + step++ + var havePrev bool + var found bool + var prevF float64 + var prevT int64 + for _, f := range floats { + if f.T == ts { + ss.Floats = append(ss.Floats, f) + found = true + break + } else if f.T < ts { + prevT = f.T + prevF = f.F + havePrev = true + continue + } + if havePrev { + value := linear(prevF, f.F, prevT, f.T, ts) + ss.Floats = append(ss.Floats, FPoint{F: value, T: ts}) + found = true + break + } + } + if !found && havePrev { + ss.Floats = append(ss.Floats, FPoint{F: prevF, T: ts}) + } + } + mat = append(mat, ss) + } + return mat +} + // evalSeries generates a Matrix between ev.startTimestamp and ev.endTimestamp (inclusive), each point spaced ev.interval apart, from series given offset. // For every storage.Series iterator in series, the method iterates in ev.interval sized steps from ev.startTimestamp until and including ev.endTimestamp, // collecting every corresponding sample (obtained via ev.vectorSelectorSingle) into a Series. @@ -1786,6 +1850,15 @@ func (ev *evaluator) eval(ctx context.Context, expr parser.Expr) (parser.Value, sel := arg.(*parser.MatrixSelector) selVS := sel.VectorSelector.(*parser.VectorSelector) + // Anchored works with: increase, and delta. + if selVS.Anchored && e.Func.Name != "increase" && e.Func.Name != "delta" { + warnings.Add(annotations.NewAnchoredWithUnsupportedFunctionWarning(e.Func.Name, e.Args[matrixArgIndex].PositionRange())) + } + // Smoothed works with: deriv, and rate. + if selVS.Smoothed && e.Func.Name != "deriv" && e.Func.Name != "rate" { + warnings.Add(annotations.NewSmoothedWithUnsupportedFunctionWarning(e.Func.Name, e.Args[matrixArgIndex].PositionRange())) + } + ws, err := checkAndExpandSeriesSet(ctx, sel) warnings.Merge(ws) if err != nil { @@ -1795,7 +1868,13 @@ func (ev *evaluator) eval(ctx context.Context, expr parser.Expr) (parser.Value, offset := durationMilliseconds(selVS.Offset) selRange := durationMilliseconds(sel.Range) stepRange := selRange - if stepRange > ev.interval { + switch { + case selVS.Anchored: + stepRange += durationMilliseconds(ev.lookbackDelta) + case selVS.Smoothed: + stepRange += durationMilliseconds(2 * ev.lookbackDelta) + } + if stepRange > ev.interval || selVS.Smoothed || selVS.Anchored { stepRange = ev.interval } // Reuse objects across steps to save memory allocations. @@ -1805,7 +1884,18 @@ func (ev *evaluator) eval(ctx context.Context, expr parser.Expr) (parser.Value, inMatrix := make(Matrix, 1) enh := &EvalNodeHelper{Out: make(Vector, 0, 1), enableDelayedNameRemoval: ev.enableDelayedNameRemoval} // Process all the calls for one time series at a time. - it := storage.NewBuffer(selRange) + // For anchored and smoothed selectors, we need to iterate over a + // larger range than the query range to account for the lookback delta. + // For standard range queries, we iterate over the query range. + bufferRange := selRange + switch { + case selVS.Anchored: + bufferRange += durationMilliseconds(ev.lookbackDelta) + case selVS.Smoothed: + bufferRange += durationMilliseconds(2 * ev.lookbackDelta) + } + + it := storage.NewBuffer(bufferRange) var chkIter chunkenc.Iterator // The last_over_time function acts like offset; thus, it @@ -1836,6 +1926,7 @@ func (ev *evaluator) eval(ctx context.Context, expr parser.Expr) (parser.Value, DropName: dropName, } inMatrix[0].Metric = selVS.Series[i].Labels() + var mint, maxt int64 for ts, step := ev.startTimestamp, -1; ts <= ev.endTimestamp; ts += ev.interval { step++ // Set the non-matrix arguments. @@ -1852,14 +1943,30 @@ func (ev *evaluator) eval(ctx context.Context, expr parser.Expr) (parser.Value, // for this step, but only if this is the 1st // iteration or no @ modifier has been used. if ts == ev.startTimestamp || selVS.Timestamp == nil { - maxt := ts - offset - mint := maxt - selRange - floats, histograms = ev.matrixIterSlice(it, mint, maxt, floats, histograms) + maxt = ts - offset + mint = maxt - selRange + matrixMaxt := maxt + matrixMint := mint + if selVS.Anchored || selVS.Smoothed { + matrixMint -= durationMilliseconds(ev.lookbackDelta) + } + if selVS.Smoothed { + matrixMaxt += durationMilliseconds(ev.lookbackDelta) + } + floats, histograms = ev.matrixIterSlice(it, matrixMint, matrixMaxt, floats, histograms) } if len(floats)+len(histograms) == 0 { continue } - inMatrix[0].Floats = floats + switch { + case selVS.Anchored: + inMatrix[0].Floats = anchorFloats(floats, mint, maxt) + case selVS.Smoothed: + counterReset := e.Func.Name == "rate" || e.Func.Name == "increase" || e.Func.Name == "irate" + inMatrix[0].Floats = smoothFloats(floats, mint, maxt, counterReset) + default: + inMatrix[0].Floats = floats + } inMatrix[0].Histograms = histograms enh.Ts = ts @@ -2057,6 +2164,10 @@ func (ev *evaluator) eval(ctx context.Context, expr parser.Expr) (parser.Value, if err != nil { ev.error(errWithWarnings{fmt.Errorf("expanding series: %w", err), ws}) } + if e.Smoothed { + mat := ev.smoothSeries(e.Series, e.Offset, false) + return mat, ws + } mat := ev.evalSeries(ctx, e.Series, e.Offset, false) return mat, ws @@ -2355,13 +2466,24 @@ func (ev *evaluator) matrixSelector(ctx context.Context, node *parser.MatrixSele var ( vs = node.VectorSelector.(*parser.VectorSelector) - offset = durationMilliseconds(vs.Offset) - maxt = ev.startTimestamp - offset - mint = maxt - durationMilliseconds(node.Range) - matrix = make(Matrix, 0, len(vs.Series)) - - it = storage.NewBuffer(durationMilliseconds(node.Range)) + offset = durationMilliseconds(vs.Offset) + maxt = ev.startTimestamp - offset + mint = maxt - durationMilliseconds(node.Range) + matrixMint = mint + matrixMaxt = maxt + matrix = make(Matrix, 0, len(vs.Series)) + bufferRange = durationMilliseconds(node.Range) ) + switch { + case vs.Anchored: + bufferRange += durationMilliseconds(ev.lookbackDelta) + mint -= durationMilliseconds(ev.lookbackDelta) + case vs.Smoothed: + bufferRange += 2 * durationMilliseconds(ev.lookbackDelta) + mint -= durationMilliseconds(ev.lookbackDelta) + maxt += durationMilliseconds(ev.lookbackDelta) + } + it := storage.NewBuffer(bufferRange) ws, err := checkAndExpandSeriesSet(ctx, node) if err != nil { ev.error(errWithWarnings{fmt.Errorf("expanding series: %w", err), ws}) @@ -2380,6 +2502,12 @@ func (ev *evaluator) matrixSelector(ctx context.Context, node *parser.MatrixSele } ss.Floats, ss.Histograms = ev.matrixIterSlice(it, mint, maxt, nil, nil) + switch { + case vs.Anchored: + ss.Floats = anchorFloats(ss.Floats, matrixMint, matrixMaxt) + case vs.Smoothed: + ss.Floats = smoothFloats(ss.Floats, matrixMint, matrixMaxt, false) + } totalSize := int64(len(ss.Floats)) + int64(totalHPointSize(ss.Histograms)) ev.samplesStats.IncrementSamplesAtTimestamp(ev.startTimestamp, totalSize) @@ -4010,3 +4138,88 @@ func (ev *evaluator) gatherVector(ts int64, input Matrix, output Vector, bufHelp return output, bufHelpers } + +func anchorFloats(floats []FPoint, mint, maxt int64) []FPoint { + anchor := make([]FPoint, 0) + for i, f := range floats { + // <= mint because the interval is left-closed. + if f.T <= mint { + continue + } + if i > 0 { + anchor = append(anchor, FPoint{T: mint, F: floats[i-1].F}) + } else { + anchor = append(anchor, FPoint{T: mint, F: f.F}) + } + anchor = append(anchor, floats[i:]...) + if lastF := anchor[len(anchor)-1]; lastF.T != maxt { + anchor = append(anchor, FPoint{T: maxt, F: lastF.F}) + } + break + } + return anchor +} + +// linear interpolates between two points at a given time. +func linear(f1, f2 float64, t1, t2, t int64) float64 { + if f1 == f2 { + return f1 + } + ratio := float64(t-t1) / float64(t2-t1) + return (1.0-ratio)*f1 + ratio*f2 +} + +func smoothFloats(floats []FPoint, mint, maxt int64, counterReset bool) []FPoint { + var out []FPoint + n := len(floats) + if n == 0 { + return out + } + + // Find first point after mint. + i := 0 + for i < n && floats[i].T <= mint { + i++ + } + + // If all points are before mint or first after maxt, nothing to do. + if i == n { + return out + } + + // Add interpolated/anchored point at mint. + if i == 0 { + out = append(out, FPoint{T: mint, F: floats[0].F}) + } else { + prev, next := floats[i-1], floats[i] + prevF := prev.F + if counterReset && prevF > next.F { + prevF = 0 + } + f := linear(prevF, next.F, prev.T, next.T, mint) + out = append(out, FPoint{T: mint, F: f}) + } + + // Add all points strictly between mint and maxt. + for ; i < n && floats[i].T < maxt; i++ { + out = append(out, floats[i]) + } + + // Add interpolated/anchored point at maxt. + if i == n { + // If last point is before maxt, anchor at last value. + out = append(out, FPoint{T: maxt, F: floats[n-1].F}) + } else if floats[i].T == maxt { + out = append(out, floats[i]) + } else if i > 0 { + prev, next := floats[i-1], floats[i] + prevF := prev.F + if counterReset && prevF > next.F { + prevF = 0 + } + f := linear(prevF, next.F, prev.T, next.T, maxt) + out = append(out, FPoint{T: maxt, F: f}) + } + + return out +} diff --git a/promql/engine_test.go b/promql/engine_test.go index 2f087bfcdf..c8359b5c68 100644 --- a/promql/engine_test.go +++ b/promql/engine_test.go @@ -1513,6 +1513,139 @@ load 10s } } +func TestExtendedRangeSelectors(t *testing.T) { + parser.EnableExtendedRangeSelectors = true + t.Cleanup(func() { + parser.EnableExtendedRangeSelectors = false + }) + + engine := newTestEngine(t) + storage := promqltest.LoadedStorage(t, ` + load 10s + metric 1+1x10 + withreset 1+1x4 1+1x5 + `) + t.Cleanup(func() { storage.Close() }) + + tc := []struct { + query string + t time.Time + expected promql.Matrix + }{ + { + query: "metric[10s] smoothed", + t: time.Unix(10, 0), + expected: promql.Matrix{ + promql.Series{ + Floats: []promql.FPoint{{F: 1, T: 0}, {F: 2, T: 10000}}, + Metric: labels.FromStrings("__name__", "metric"), + }, + }, + }, + { + query: "metric[10s] smoothed", + t: time.Unix(15, 0), + expected: promql.Matrix{ + promql.Series{ + Floats: []promql.FPoint{{F: 1.5, T: 5000}, {F: 2, T: 10000}, {F: 2.5, T: 15000}}, + Metric: labels.FromStrings("__name__", "metric"), + }, + }, + }, + { + query: "metric[10s] smoothed", + t: time.Unix(5, 0), + expected: promql.Matrix{ + promql.Series{ + Floats: []promql.FPoint{{F: 1, T: -5000}, {F: 1, T: 0}, {F: 1.5, T: 5000}}, + Metric: labels.FromStrings("__name__", "metric"), + }, + }, + }, + { + query: "metric[10s] smoothed", + t: time.Unix(105, 0), + expected: promql.Matrix{ + promql.Series{ + Floats: []promql.FPoint{{F: 10.5, T: 95000}, {F: 11, T: 100000}, {F: 11, T: 105000}}, + Metric: labels.FromStrings("__name__", "metric"), + }, + }, + }, + { + query: "withreset[10s] smoothed", + t: time.Unix(45, 0), + expected: promql.Matrix{ + promql.Series{ + Floats: []promql.FPoint{{F: 4.5, T: 35000}, {F: 5, T: 40000}, {F: 3, T: 45000}}, + Metric: labels.FromStrings("__name__", "withreset"), + }, + }, + }, + { + query: "metric[10s] anchored", + t: time.Unix(10, 0), + expected: promql.Matrix{ + promql.Series{ + Floats: []promql.FPoint{{F: 1, T: 0}, {F: 2, T: 10000}}, + Metric: labels.FromStrings("__name__", "metric"), + }, + }, + }, + { + query: "metric[10s] anchored", + t: time.Unix(15, 0), + expected: promql.Matrix{ + promql.Series{ + Floats: []promql.FPoint{{F: 1, T: 5000}, {F: 2, T: 10000}, {F: 2, T: 15000}}, + Metric: labels.FromStrings("__name__", "metric"), + }, + }, + }, + { + query: "metric[10s] anchored", + t: time.Unix(5, 0), + expected: promql.Matrix{ + promql.Series{ + Floats: []promql.FPoint{{F: 1, T: -5000}, {F: 1, T: 0}, {F: 1, T: 5000}}, + Metric: labels.FromStrings("__name__", "metric"), + }, + }, + }, + { + query: "metric[10s] anchored", + t: time.Unix(105, 0), + expected: promql.Matrix{ + promql.Series{ + Floats: []promql.FPoint{{F: 10, T: 95000}, {F: 11, T: 100000}, {F: 11, T: 105000}}, + Metric: labels.FromStrings("__name__", "metric"), + }, + }, + }, + { + query: "withreset[10s] anchored", + t: time.Unix(45, 0), + expected: promql.Matrix{ + promql.Series{ + Floats: []promql.FPoint{{F: 4, T: 35000}, {F: 5, T: 40000}, {F: 5, T: 45000}}, + Metric: labels.FromStrings("__name__", "withreset"), + }, + }, + }, + } + + for _, tc := range tc { + t.Run(tc.query, func(t *testing.T) { + engine = promqltest.NewTestEngine(t, false, 0, 100) + qry, err := engine.NewInstantQuery(context.Background(), storage, nil, tc.query, tc.t) + require.NoError(t, err) + res := qry.Exec(context.Background()) + require.NoError(t, res.Err) + require.Equal(t, tc.expected, res.Value) + }) + } +} + func TestAtModifier(t *testing.T) { engine := newTestEngine(t) storage := promqltest.LoadedStorage(t, ` diff --git a/promql/parser/ast.go b/promql/parser/ast.go index dc3e36b5b5..9e1cbd724b 100644 --- a/promql/parser/ast.go +++ b/promql/parser/ast.go @@ -226,6 +226,11 @@ type VectorSelector struct { // This is the case when VectorSelector is used to represent the info function's second argument. BypassEmptyMatcherCheck bool + // Anchored is true when the VectorSelector is anchored. + Anchored bool + // Smoothed is true when the VectorSelector is smoothed. + Smoothed bool + PosRange posrange.PositionRange } diff --git a/promql/parser/generated_parser.y b/promql/parser/generated_parser.y index e7e16cd033..a55947d727 100644 --- a/promql/parser/generated_parser.y +++ b/promql/parser/generated_parser.y @@ -141,6 +141,8 @@ GROUP_LEFT GROUP_RIGHT IGNORING OFFSET +SMOOTHED +ANCHORED ON WITHOUT %token keywordsEnd @@ -187,7 +189,7 @@ START_METRIC_SELECTOR %type int %type uint %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_duration_literal offset_expr on_or_ignoring paren_expr string_literal subquery_expr unary_expr vector_selector duration_expr paren_duration_expr positive_duration_expr offset_duration_expr +%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_duration_literal offset_expr anchored_expr smoothed_expr on_or_ignoring paren_expr string_literal subquery_expr unary_expr vector_selector duration_expr paren_duration_expr positive_duration_expr offset_duration_expr %start start @@ -230,6 +232,8 @@ expr : | matrix_selector | number_duration_literal | offset_expr + | anchored_expr + | smoothed_expr | paren_expr | string_literal | subquery_expr @@ -482,6 +486,20 @@ offset_expr: expr OFFSET offset_duration_expr { yylex.(*parser).unexpected("offset", "number, duration, or step()"); $$ = $1 } ; +/* + * Anchored and smoothed modifiers + */ + +anchored_expr: expr ANCHORED + { + yylex.(*parser).setAnchored($1) + } + +smoothed_expr: expr SMOOTHED + { + yylex.(*parser).setSmoothed($1) + } + /* * @ modifiers. */ diff --git a/promql/parser/generated_parser.y.go b/promql/parser/generated_parser.y.go index e93d1b3de6..59d2949bda 100644 --- a/promql/parser/generated_parser.y.go +++ b/promql/parser/generated_parser.y.go @@ -115,26 +115,28 @@ const GROUP_LEFT = 57422 const GROUP_RIGHT = 57423 const IGNORING = 57424 const OFFSET = 57425 -const ON = 57426 -const WITHOUT = 57427 -const keywordsEnd = 57428 -const preprocessorStart = 57429 -const START = 57430 -const END = 57431 -const STEP = 57432 -const preprocessorEnd = 57433 -const counterResetHintsStart = 57434 -const UNKNOWN_COUNTER_RESET = 57435 -const COUNTER_RESET = 57436 -const NOT_COUNTER_RESET = 57437 -const GAUGE_TYPE = 57438 -const counterResetHintsEnd = 57439 -const startSymbolsStart = 57440 -const START_METRIC = 57441 -const START_SERIES_DESCRIPTION = 57442 -const START_EXPRESSION = 57443 -const START_METRIC_SELECTOR = 57444 -const startSymbolsEnd = 57445 +const SMOOTHED = 57426 +const ANCHORED = 57427 +const ON = 57428 +const WITHOUT = 57429 +const keywordsEnd = 57430 +const preprocessorStart = 57431 +const START = 57432 +const END = 57433 +const STEP = 57434 +const preprocessorEnd = 57435 +const counterResetHintsStart = 57436 +const UNKNOWN_COUNTER_RESET = 57437 +const COUNTER_RESET = 57438 +const NOT_COUNTER_RESET = 57439 +const GAUGE_TYPE = 57440 +const counterResetHintsEnd = 57441 +const startSymbolsStart = 57442 +const START_METRIC = 57443 +const START_SERIES_DESCRIPTION = 57444 +const START_EXPRESSION = 57445 +const START_METRIC_SELECTOR = 57446 +const startSymbolsEnd = 57447 var yyToknames = [...]string{ "$end", @@ -220,6 +222,8 @@ var yyToknames = [...]string{ "GROUP_RIGHT", "IGNORING", "OFFSET", + "SMOOTHED", + "ANCHORED", "ON", "WITHOUT", "keywordsEnd", @@ -253,488 +257,490 @@ var yyExca = [...]int16{ 1, -1, -2, 0, -1, 38, - 1, 143, - 10, 143, - 24, 143, + 1, 147, + 10, 147, + 24, 147, -2, 0, - -1, 66, - 2, 186, - 15, 186, - 79, 186, - 85, 186, - -2, 103, - -1, 67, - 2, 187, - 15, 187, - 79, 187, - 85, 187, - -2, 104, -1, 68, - 2, 188, - 15, 188, - 79, 188, - 85, 188, - -2, 106, - -1, 69, - 2, 189, - 15, 189, - 79, 189, - 85, 189, - -2, 107, - -1, 70, 2, 190, 15, 190, 79, 190, - 85, 190, - -2, 108, - -1, 71, + 87, 190, + -2, 107, + -1, 69, 2, 191, 15, 191, 79, 191, - 85, 191, - -2, 113, - -1, 72, + 87, 191, + -2, 108, + -1, 70, 2, 192, 15, 192, 79, 192, - 85, 192, - -2, 115, - -1, 73, + 87, 192, + -2, 110, + -1, 71, 2, 193, 15, 193, 79, 193, - 85, 193, - -2, 117, - -1, 74, + 87, 193, + -2, 111, + -1, 72, 2, 194, 15, 194, 79, 194, - 85, 194, - -2, 118, - -1, 75, + 87, 194, + -2, 112, + -1, 73, 2, 195, 15, 195, 79, 195, - 85, 195, - -2, 119, - -1, 76, + 87, 195, + -2, 117, + -1, 74, 2, 196, 15, 196, 79, 196, - 85, 196, - -2, 120, - -1, 77, + 87, 196, + -2, 119, + -1, 75, 2, 197, 15, 197, 79, 197, - 85, 197, + 87, 197, -2, 121, - -1, 78, + -1, 76, 2, 198, 15, 198, 79, 198, - 85, 198, - -2, 125, - -1, 79, + 87, 198, + -2, 122, + -1, 77, 2, 199, 15, 199, 79, 199, - 85, 199, - -2, 126, - -1, 129, - 41, 262, - 42, 262, - 52, 262, - 53, 262, - 57, 262, - -2, 20, - -1, 239, - 9, 249, - 12, 249, - 13, 249, - 18, 249, - 19, 249, - 25, 249, - 41, 249, - 47, 249, - 48, 249, - 51, 249, - 57, 249, - 62, 249, - 63, 249, - 64, 249, - 65, 249, - 66, 249, - 67, 249, - 68, 249, - 69, 249, - 70, 249, - 71, 249, - 72, 249, - 73, 249, - 74, 249, - 75, 249, - 79, 249, - 83, 249, - 85, 249, - 88, 249, - 89, 249, - 90, 249, + 87, 199, + -2, 123, + -1, 78, + 2, 200, + 15, 200, + 79, 200, + 87, 200, + -2, 124, + -1, 79, + 2, 201, + 15, 201, + 79, 201, + 87, 201, + -2, 125, + -1, 80, + 2, 202, + 15, 202, + 79, 202, + 87, 202, + -2, 129, + -1, 81, + 2, 203, + 15, 203, + 79, 203, + 87, 203, + -2, 130, + -1, 133, + 41, 266, + 42, 266, + 52, 266, + 53, 266, + 57, 266, + -2, 22, + -1, 243, + 9, 253, + 12, 253, + 13, 253, + 18, 253, + 19, 253, + 25, 253, + 41, 253, + 47, 253, + 48, 253, + 51, 253, + 57, 253, + 62, 253, + 63, 253, + 64, 253, + 65, 253, + 66, 253, + 67, 253, + 68, 253, + 69, 253, + 70, 253, + 71, 253, + 72, 253, + 73, 253, + 74, 253, + 75, 253, + 79, 253, + 83, 253, + 87, 253, + 90, 253, + 91, 253, + 92, 253, -2, 0, - -1, 240, - 9, 249, - 12, 249, - 13, 249, - 18, 249, - 19, 249, - 25, 249, - 41, 249, - 47, 249, - 48, 249, - 51, 249, - 57, 249, - 62, 249, - 63, 249, - 64, 249, - 65, 249, - 66, 249, - 67, 249, - 68, 249, - 69, 249, - 70, 249, - 71, 249, - 72, 249, - 73, 249, - 74, 249, - 75, 249, - 79, 249, - 83, 249, - 85, 249, - 88, 249, - 89, 249, - 90, 249, + -1, 244, + 9, 253, + 12, 253, + 13, 253, + 18, 253, + 19, 253, + 25, 253, + 41, 253, + 47, 253, + 48, 253, + 51, 253, + 57, 253, + 62, 253, + 63, 253, + 64, 253, + 65, 253, + 66, 253, + 67, 253, + 68, 253, + 69, 253, + 70, 253, + 71, 253, + 72, 253, + 73, 253, + 74, 253, + 75, 253, + 79, 253, + 83, 253, + 87, 253, + 90, 253, + 91, 253, + 92, 253, -2, 0, } const yyPrivate = 57344 -const yyLast = 1045 +const yyLast = 1052 var yyAct = [...]int16{ - 53, 393, 391, 176, 398, 324, 272, 231, 187, 179, - 45, 338, 89, 135, 215, 87, 118, 122, 339, 64, - 121, 60, 180, 123, 150, 120, 119, 405, 406, 407, - 408, 62, 237, 124, 238, 112, 239, 240, 116, 388, - 387, 358, 315, 208, 184, 356, 145, 377, 117, 115, - 57, 6, 118, 346, 183, 344, 175, 356, 210, 127, - 56, 129, 94, 96, 97, 314, 98, 99, 100, 101, - 102, 103, 104, 105, 106, 107, 185, 108, 109, 111, - 95, 125, 80, 392, 138, 40, 307, 312, 313, 184, - 364, 318, 186, 130, 136, 137, 342, 122, 81, 183, - 365, 306, 309, 123, 110, 363, 319, 91, 174, 173, - 311, 172, 362, 189, 193, 194, 195, 196, 197, 198, - 235, 168, 320, 190, 190, 190, 190, 190, 190, 190, - 171, 192, 169, 211, 191, 191, 191, 191, 191, 191, - 191, 201, 204, 190, 126, 199, 128, 200, 2, 3, - 4, 5, 221, 57, 191, 233, 223, 316, 341, 259, - 227, 112, 216, 56, 217, 131, 113, 116, 234, 264, - 260, 420, 185, 399, 409, 382, 263, 117, 115, 352, - 256, 118, 114, 226, 351, 80, 190, 421, 186, 258, - 419, 191, 260, 418, 381, 265, 266, 191, 146, 350, - 262, 81, 190, 108, 219, 111, 113, 116, 113, 116, - 207, 174, 173, 191, 218, 220, 203, 117, 115, 117, - 115, 118, 114, 118, 114, 217, 137, 310, 261, 202, - 110, 225, 236, 124, 257, 132, 82, 241, 242, 243, - 244, 245, 246, 247, 248, 249, 250, 251, 252, 253, - 254, 255, 340, 317, 224, 36, 336, 337, 7, 376, - 343, 375, 90, 345, 138, 219, 93, 269, 10, 189, - 190, 268, 88, 190, 136, 218, 220, 347, 84, 190, - 222, 191, 134, 374, 191, 91, 267, 91, 373, 372, - 191, 417, 371, 370, 151, 152, 153, 154, 155, 156, - 157, 158, 159, 160, 161, 162, 163, 164, 165, 354, - 113, 116, 369, 368, 367, 366, 144, 355, 357, 190, - 359, 117, 115, 90, 214, 118, 114, 360, 361, 213, - 191, 349, 385, 88, 51, 8, 37, 184, 57, 38, - 83, 86, 212, 378, 175, 1, 91, 183, 56, 422, - 348, 113, 116, 190, 166, 143, 141, 142, 384, 65, - 386, 140, 117, 115, 191, 50, 118, 114, 390, 185, - 80, 394, 395, 396, 139, 49, 401, 400, 403, 402, - 397, 410, 48, 47, 149, 186, 81, 46, 275, 411, - 412, 190, 44, 353, 413, 379, 174, 173, 285, 147, - 205, 43, 191, 415, 291, 148, 42, 41, 383, 61, - 416, 274, 9, 9, 113, 116, 52, 230, 414, 192, - 190, 321, 423, 92, 228, 117, 115, 270, 85, 118, - 114, 191, 404, 287, 288, 177, 273, 289, 54, 133, - 0, 0, 0, 0, 0, 302, 0, 0, 276, 278, - 280, 281, 282, 290, 292, 295, 296, 297, 298, 299, - 303, 304, 275, 0, 277, 279, 283, 284, 286, 293, - 294, 209, 285, 0, 300, 301, 305, 0, 291, 0, - 0, 188, 271, 0, 0, 274, 0, 0, 57, 0, - 113, 116, 0, 0, 175, 0, 0, 0, 56, 0, - 0, 117, 115, 0, 0, 118, 114, 287, 288, 0, - 0, 289, 0, 0, 0, 0, 0, 0, 0, 302, - 80, 0, 276, 278, 280, 281, 282, 290, 292, 295, - 296, 297, 298, 299, 303, 304, 81, 0, 277, 279, - 283, 284, 286, 293, 294, 0, 174, 173, 300, 301, - 305, 57, 0, 112, 55, 82, 0, 58, 0, 0, - 22, 56, 0, 167, 206, 0, 0, 59, 0, 192, - 57, 0, 0, 0, 0, 0, 175, 0, 0, 0, - 56, 96, 0, 80, 0, 0, 0, 0, 0, 18, - 19, 105, 106, 20, 0, 108, 0, 111, 95, 81, - 0, 0, 80, 0, 66, 67, 68, 69, 70, 71, - 72, 73, 74, 75, 76, 77, 78, 79, 81, 0, - 0, 13, 110, 0, 380, 24, 0, 30, 174, 173, - 31, 32, 63, 57, 39, 0, 55, 82, 0, 58, - 0, 0, 22, 56, 0, 0, 178, 0, 0, 59, - 0, 170, 0, 184, 0, 0, 0, 0, 113, 116, - 0, 0, 0, 183, 0, 80, 0, 0, 0, 117, - 115, 18, 19, 118, 114, 20, 0, 0, 0, 0, - 0, 81, 0, 0, 0, 185, 66, 67, 68, 69, - 70, 71, 72, 73, 74, 75, 76, 77, 78, 79, - 0, 186, 0, 13, 0, 0, 0, 24, 0, 30, - 0, 0, 31, 32, 63, 57, 0, 112, 55, 82, - 0, 58, 0, 0, 22, 56, 0, 0, 0, 0, - 0, 59, 181, 182, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 94, 96, 0, 80, 0, 0, - 0, 0, 0, 18, 19, 105, 106, 20, 308, 108, - 109, 111, 95, 81, 0, 0, 0, 0, 66, 67, + 55, 180, 397, 395, 183, 402, 276, 235, 191, 328, + 91, 45, 342, 139, 66, 219, 89, 241, 17, 84, + 154, 242, 125, 62, 22, 184, 124, 123, 343, 409, + 410, 411, 412, 126, 120, 128, 243, 244, 64, 122, + 392, 127, 391, 362, 121, 119, 149, 319, 122, 192, + 381, 350, 360, 18, 19, 188, 59, 20, 220, 318, + 221, 131, 179, 133, 6, 187, 58, 426, 11, 12, + 14, 15, 16, 21, 23, 25, 26, 27, 28, 29, + 33, 34, 317, 316, 129, 13, 40, 189, 82, 24, + 396, 348, 214, 30, 322, 141, 31, 32, 35, 126, + 223, 360, 134, 190, 83, 369, 315, 127, 239, 323, + 222, 224, 346, 176, 178, 177, 424, 193, 197, 198, + 199, 200, 201, 202, 172, 324, 345, 175, 194, 194, + 194, 194, 194, 194, 194, 423, 173, 215, 422, 196, + 195, 195, 195, 195, 195, 195, 195, 130, 194, 132, + 203, 182, 204, 386, 237, 205, 208, 225, 188, 135, + 195, 227, 320, 2, 3, 4, 5, 268, 187, 171, + 266, 207, 385, 188, 264, 238, 59, 189, 231, 403, + 267, 229, 179, 187, 206, 260, 58, 368, 262, 265, + 189, 194, 425, 190, 269, 270, 264, 195, 311, 150, + 84, 230, 367, 195, 228, 189, 190, 194, 82, 366, + 273, 117, 120, 310, 272, 211, 128, 136, 221, 195, + 218, 190, 121, 119, 83, 217, 122, 118, 141, 271, + 36, 314, 7, 380, 178, 177, 379, 240, 216, 185, + 186, 378, 245, 246, 247, 248, 249, 250, 251, 252, + 253, 254, 255, 256, 257, 258, 259, 344, 223, 174, + 321, 145, 188, 377, 347, 148, 144, 349, 222, 224, + 340, 341, 187, 193, 376, 194, 375, 374, 194, 143, + 373, 351, 95, 10, 194, 59, 116, 195, 372, 371, + 195, 179, 413, 86, 370, 58, 195, 155, 156, 157, + 158, 159, 160, 161, 162, 163, 164, 165, 166, 167, + 168, 169, 53, 358, 147, 421, 146, 82, 85, 37, + 1, 170, 359, 361, 194, 363, 117, 120, 110, 67, + 115, 364, 365, 83, 117, 120, 195, 121, 119, 52, + 51, 122, 118, 178, 177, 121, 119, 382, 63, 122, + 118, 9, 9, 50, 142, 112, 114, 113, 194, 8, + 49, 153, 388, 38, 140, 390, 356, 48, 196, 47, + 195, 355, 313, 398, 399, 400, 394, 93, 46, 401, + 44, 405, 404, 407, 406, 414, 354, 279, 151, 209, + 43, 152, 389, 42, 415, 416, 194, 289, 357, 417, + 383, 41, 54, 295, 234, 419, 418, 387, 195, 325, + 278, 117, 120, 94, 420, 232, 274, 87, 408, 117, + 120, 181, 121, 119, 427, 194, 122, 118, 277, 56, + 121, 119, 291, 292, 122, 118, 293, 195, 137, 0, + 0, 0, 0, 0, 306, 0, 0, 280, 282, 284, + 285, 286, 294, 296, 299, 300, 301, 302, 303, 307, + 308, 0, 279, 281, 283, 287, 288, 290, 297, 92, + 142, 298, 289, 0, 0, 304, 305, 309, 295, 90, + 140, 0, 275, 213, 0, 278, 92, 226, 138, 59, + 0, 0, 93, 93, 0, 263, 90, 0, 0, 58, + 0, 0, 117, 120, 88, 0, 0, 291, 292, 93, + 0, 293, 0, 121, 119, 0, 0, 122, 118, 306, + 0, 82, 280, 282, 284, 285, 286, 294, 296, 299, + 300, 301, 302, 303, 307, 308, 0, 83, 281, 283, + 287, 288, 290, 297, 0, 0, 298, 178, 177, 0, + 304, 305, 309, 59, 0, 116, 57, 84, 0, 60, + 0, 0, 22, 58, 117, 120, 210, 0, 0, 61, + 0, 0, 261, 0, 0, 121, 119, 0, 0, 122, + 118, 0, 96, 98, 0, 82, 0, 0, 0, 0, + 0, 18, 19, 107, 108, 20, 0, 110, 111, 115, + 97, 83, 0, 0, 0, 0, 68, 69, 70, 71, + 72, 73, 74, 75, 76, 77, 78, 79, 80, 81, + 0, 0, 0, 13, 112, 114, 113, 24, 0, 0, + 0, 30, 0, 0, 31, 32, 65, 59, 39, 0, + 57, 84, 0, 60, 0, 0, 22, 58, 0, 0, + 0, 0, 233, 61, 0, 0, 188, 0, 0, 236, + 0, 0, 0, 239, 0, 0, 187, 353, 0, 82, + 0, 0, 0, 0, 59, 18, 19, 0, 0, 20, + 179, 0, 0, 0, 58, 83, 352, 0, 189, 0, 68, 69, 70, 71, 72, 73, 74, 75, 76, 77, - 78, 79, 17, 82, 0, 13, 110, 0, 22, 24, - 0, 30, 113, 116, 31, 32, 63, 0, 0, 0, - 0, 0, 0, 117, 115, 0, 0, 118, 114, 0, - 0, 0, 229, 0, 0, 0, 184, 18, 19, 232, - 0, 20, 0, 235, 0, 0, 183, 0, 0, 0, - 0, 0, 11, 12, 14, 15, 16, 21, 23, 25, - 26, 27, 28, 29, 33, 34, 17, 36, 185, 13, - 0, 0, 22, 24, 323, 30, 0, 0, 31, 32, - 35, 322, 0, 0, 186, 326, 327, 325, 332, 334, - 331, 333, 328, 329, 330, 335, 0, 0, 0, 0, - 0, 18, 19, 0, 0, 20, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 11, 12, 14, 15, - 16, 21, 23, 25, 26, 27, 28, 29, 33, 34, - 112, 0, 0, 13, 0, 0, 0, 24, 0, 30, - 0, 0, 31, 32, 35, 0, 0, 0, 0, 112, - 0, 0, 0, 0, 0, 0, 0, 94, 96, 97, - 0, 98, 99, 100, 101, 102, 103, 104, 105, 106, - 107, 0, 108, 109, 111, 95, 94, 96, 97, 0, - 98, 99, 100, 0, 102, 103, 104, 105, 106, 107, - 389, 108, 109, 111, 95, 112, 0, 0, 0, 110, - 0, 326, 327, 325, 332, 334, 331, 333, 328, 329, - 330, 335, 0, 0, 0, 0, 0, 0, 110, 0, - 0, 0, 94, 96, 97, 0, 98, 99, 0, 0, - 102, 103, 0, 105, 106, 107, 0, 108, 109, 111, - 95, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 110, + 78, 79, 80, 81, 190, 384, 82, 13, 0, 0, + 0, 24, 0, 0, 0, 30, 0, 0, 31, 32, + 65, 59, 83, 0, 57, 84, 0, 60, 0, 0, + 22, 58, 178, 177, 0, 0, 0, 61, 0, 117, + 120, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 121, 119, 0, 82, 122, 118, 0, 196, 0, 18, + 19, 17, 36, 20, 0, 0, 0, 22, 0, 83, + 0, 0, 0, 0, 68, 69, 70, 71, 72, 73, + 74, 75, 76, 77, 78, 79, 80, 81, 0, 0, + 0, 13, 0, 0, 0, 24, 18, 19, 0, 30, + 20, 0, 31, 32, 65, 0, 0, 0, 0, 0, + 0, 11, 12, 14, 15, 16, 21, 23, 25, 26, + 27, 28, 29, 33, 34, 116, 0, 0, 13, 0, + 0, 0, 24, 212, 0, 0, 30, 0, 0, 31, + 32, 35, 0, 0, 116, 0, 0, 0, 0, 0, + 0, 0, 96, 98, 99, 0, 100, 101, 102, 103, + 104, 105, 106, 107, 108, 109, 0, 110, 111, 115, + 97, 96, 98, 99, 0, 100, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 0, 110, 111, 115, 97, + 116, 0, 0, 0, 112, 114, 113, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 116, + 0, 0, 0, 112, 114, 113, 0, 96, 98, 99, + 0, 100, 101, 102, 0, 104, 105, 106, 107, 108, + 109, 0, 110, 111, 115, 97, 96, 98, 99, 0, + 100, 101, 0, 116, 104, 105, 0, 107, 108, 109, + 0, 110, 111, 115, 97, 312, 0, 0, 0, 112, + 114, 113, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 98, 0, 0, 0, 0, 0, 0, 112, 114, + 113, 107, 108, 0, 0, 110, 0, 115, 97, 117, + 120, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 121, 119, 0, 0, 122, 118, 0, 0, 327, 0, + 0, 0, 112, 114, 113, 326, 0, 0, 0, 330, + 331, 329, 336, 338, 335, 337, 332, 333, 334, 339, + 393, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 330, 331, 329, 336, 338, 335, 337, 332, 333, + 334, 339, } var yyPact = [...]int16{ - 49, 248, 834, 834, 624, 770, -1000, -1000, -1000, 242, + 62, 222, 749, 749, 628, 6, -1000, -1000, -1000, 217, -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, 321, -1000, 264, -1000, - 896, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, -1000, 125, 18, 218, -1000, -1000, 706, -1000, - 706, 223, -1000, 150, 220, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, 484, -1000, 280, -1000, + 830, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, 523, 20, 201, -1000, -1000, + 712, -1000, 712, 187, -1000, 144, 202, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, - -1000, -1000, 262, -1000, -1000, 354, -1000, -1000, 353, 312, - -1000, -1000, 22, -1000, -54, -54, -54, -54, -54, -54, - -54, -54, -54, -54, -54, -54, -54, -54, -54, -54, - 561, 644, 479, 41, 41, 41, 41, 41, 41, 218, - -62, -1000, 214, 214, 542, -1000, 21, 449, 147, -40, - -1000, 36, 41, 322, -1000, -1000, 160, 221, -1000, -1000, - 260, -1000, 229, -1000, 158, 807, 706, -1000, -50, -44, - -1000, 706, 706, 706, 706, 706, 706, 706, 706, 706, - 706, 706, 706, 706, 706, 706, -1000, -1000, -1000, 144, - 213, 185, 125, -1000, -1000, 41, -1000, 154, -1000, -1000, - -1000, -1000, -1000, -1000, -1000, 80, 80, 265, -1000, 125, - -1000, 41, 150, -4, -4, -40, -40, -40, -40, -1000, - -1000, -1000, 460, -1000, -1000, 79, -1000, 896, -1000, -1000, - -1000, 751, -1000, 82, -1000, 85, -1000, -1000, -1000, -1000, - -1000, 63, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 16, - 131, 65, -1000, -1000, -1000, 837, 539, 214, 214, 214, - 214, 147, 147, 703, 703, 703, 961, 915, 703, 703, - 961, 147, 147, 703, 147, 539, -1000, 143, 81, 41, - -40, 33, 41, 449, 31, -1000, -1000, -1000, 329, -1000, - 177, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, 468, -1000, -1000, 259, -1000, -1000, + 312, 261, -1000, -1000, 22, -1000, -58, -58, -58, -58, + -58, -58, -58, -58, -58, -58, -58, -58, -58, -58, + -58, -58, 167, -1000, -1000, 149, 47, 276, 276, 276, + 276, 276, 276, 201, -46, -1000, 169, 169, 544, -1000, + 811, 461, 272, -17, -1000, 70, 276, 218, -1000, -1000, + 56, 214, -1000, -1000, 467, -1000, 179, -1000, 176, 647, + 712, -1000, -65, -44, -1000, 712, 712, 712, 712, 712, + 712, 712, 712, 712, 712, 712, 712, 712, 712, 712, + -1000, -1000, -1000, 480, 174, 155, 523, -1000, -1000, 276, + -1000, 152, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 253, + 253, 208, -1000, 523, -1000, 276, 144, -8, -8, -17, + -17, -17, -17, -1000, -1000, -1000, 460, -1000, -1000, 191, + -1000, 830, -1000, -1000, -1000, 948, -1000, 352, -1000, 81, + -1000, -1000, -1000, -1000, -1000, 57, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, 21, 136, 68, -1000, -1000, -1000, 991, + 929, 169, 169, 169, 169, 272, 272, 541, 541, 541, + 895, 876, 541, 541, 895, 272, 272, 541, 272, 929, + -1000, 111, 97, 276, -17, 69, 276, 461, 29, -1000, + -1000, -1000, 665, -1000, 364, -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, -1000, 706, 41, -1000, - -1000, -1000, -1000, -1000, -1000, 38, 38, 15, 38, 104, - 104, 88, 83, -1000, -1000, 309, 308, 307, 306, 287, - 286, 283, 282, 277, 255, 253, -1000, -1000, -1000, -1000, - -1000, 25, 41, 373, -1000, 617, -1000, 173, -1000, -1000, - -1000, 386, -1000, 896, 310, -1000, -1000, -1000, 38, -1000, - 14, 13, 953, -1000, -1000, -1000, 26, 35, 35, 35, - 80, 159, 159, 26, 159, 26, -66, -1000, 167, -1000, - 41, -1000, -1000, -1000, -1000, -1000, -1000, 38, 38, -1000, - -1000, -1000, 38, -1000, -1000, -1000, -1000, -1000, -1000, 35, - -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, 41, - 269, -1000, -1000, -1000, 169, -1000, 165, -1000, 328, -1000, - -1000, -1000, -1000, -1000, + -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, 712, 276, -1000, -1000, -1000, -1000, -1000, -1000, 82, + 82, 17, 82, 92, 92, 185, 88, -1000, -1000, 288, + 283, 282, 274, 271, 270, 268, 257, 235, 230, 227, + -1000, -1000, -1000, -1000, -1000, 28, 276, 378, -1000, 698, + -1000, 151, -1000, -1000, -1000, 385, -1000, 830, 370, -1000, + -1000, -1000, 82, -1000, 16, 14, 1013, -1000, -1000, -1000, + 33, 164, 164, 164, 253, 165, 165, 33, 165, 33, + -66, -1000, 285, -1000, 276, -1000, -1000, -1000, -1000, -1000, + -1000, 82, 82, -1000, -1000, -1000, 82, -1000, -1000, -1000, + -1000, -1000, -1000, 164, -1000, -1000, -1000, -1000, -1000, -1000, + -1000, -1000, -1000, 276, 293, -1000, -1000, -1000, 114, -1000, + 170, -1000, 46, -1000, -1000, -1000, -1000, -1000, } var yyPgo = [...]int16{ - 0, 439, 13, 438, 6, 14, 436, 409, 21, 435, - 12, 432, 19, 268, 335, 428, 15, 427, 18, 11, - 424, 423, 7, 421, 5, 4, 418, 2, 1, 9, - 417, 22, 3, 416, 407, 26, 198, 406, 405, 85, - 401, 400, 25, 399, 31, 392, 10, 387, 384, 383, - 382, 375, 365, 334, 0, 359, 8, 354, 345, 336, + 0, 438, 13, 429, 6, 15, 428, 348, 23, 421, + 10, 418, 14, 283, 359, 417, 16, 416, 28, 12, + 415, 413, 7, 409, 9, 5, 406, 3, 2, 4, + 404, 25, 1, 402, 401, 27, 199, 393, 391, 86, + 390, 389, 26, 388, 38, 380, 11, 378, 369, 367, + 361, 360, 353, 340, 339, 312, 0, 329, 8, 321, + 320, 319, } var yyR1 = [...]int8{ - 0, 58, 58, 58, 58, 58, 58, 58, 39, 39, + 0, 60, 60, 60, 60, 60, 60, 60, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, - 39, 34, 34, 34, 34, 35, 35, 37, 37, 37, + 39, 39, 39, 34, 34, 34, 34, 35, 35, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, - 37, 37, 37, 36, 38, 38, 48, 48, 43, 43, - 43, 43, 18, 18, 18, 18, 17, 17, 17, 4, - 4, 4, 40, 42, 42, 41, 41, 41, 49, 56, - 47, 47, 33, 33, 33, 9, 9, 45, 51, 51, - 51, 51, 51, 51, 52, 53, 53, 53, 44, 44, - 44, 1, 1, 1, 2, 2, 2, 2, 2, 2, - 2, 14, 14, 7, 7, 7, 7, 7, 7, 7, + 37, 37, 37, 37, 37, 36, 38, 38, 50, 50, + 43, 43, 43, 43, 18, 18, 18, 18, 17, 17, + 17, 4, 4, 4, 40, 42, 42, 41, 41, 41, + 51, 58, 47, 47, 48, 49, 33, 33, 33, 9, + 9, 45, 53, 53, 53, 53, 53, 53, 54, 55, + 55, 55, 44, 44, 44, 1, 1, 1, 2, 2, + 2, 2, 2, 2, 2, 14, 14, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, - 7, 7, 7, 7, 7, 7, 7, 7, 13, 13, - 13, 13, 15, 15, 15, 16, 16, 16, 16, 16, - 16, 16, 59, 21, 21, 21, 21, 20, 20, 20, - 20, 20, 20, 20, 20, 20, 30, 30, 30, 22, - 22, 22, 22, 23, 23, 23, 24, 24, 24, 24, - 24, 24, 24, 24, 24, 24, 24, 25, 25, 26, - 26, 26, 11, 11, 11, 11, 3, 3, 3, 3, + 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, + 7, 7, 13, 13, 13, 13, 15, 15, 15, 16, + 16, 16, 16, 16, 16, 16, 61, 21, 21, 21, + 21, 20, 20, 20, 20, 20, 20, 20, 20, 20, + 30, 30, 30, 22, 22, 22, 22, 23, 23, 23, + 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, + 24, 25, 25, 26, 26, 26, 11, 11, 11, 11, 3, 3, 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, 6, 6, 6, 6, 6, - 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, - 8, 8, 5, 5, 5, 5, 46, 46, 29, 29, - 31, 31, 32, 32, 28, 27, 27, 50, 10, 19, - 19, 57, 57, 57, 57, 57, 57, 57, 57, 12, - 12, 54, 54, 54, 54, 54, 54, 54, 54, 54, - 54, 54, 55, + 6, 6, 6, 6, 8, 8, 5, 5, 5, 5, + 46, 46, 29, 29, 31, 31, 32, 32, 28, 27, + 27, 52, 10, 19, 19, 59, 59, 59, 59, 59, + 59, 59, 59, 12, 12, 56, 56, 56, 56, 56, + 56, 56, 56, 56, 56, 56, 57, } var yyR2 = [...]int8{ 0, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, - 1, 3, 3, 2, 2, 2, 2, 4, 4, 4, + 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, 1, 2, 3, 2, 3, 1, 2, 3, 1, - 3, 3, 3, 5, 3, 1, 1, 4, 6, 5, - 6, 5, 4, 3, 2, 2, 1, 1, 3, 4, - 2, 3, 1, 2, 3, 3, 1, 3, 3, 2, - 1, 2, 1, 1, 1, 1, 1, 1, 1, 1, + 4, 4, 4, 4, 4, 1, 0, 1, 3, 3, + 1, 1, 3, 3, 3, 4, 2, 1, 3, 1, + 2, 1, 1, 1, 2, 3, 2, 3, 1, 2, + 3, 1, 3, 3, 2, 2, 3, 5, 3, 1, + 1, 4, 6, 5, 6, 5, 4, 3, 2, 2, + 1, 1, 3, 4, 2, 3, 1, 2, 3, 3, + 1, 3, 3, 2, 1, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, - 1, 1, 1, 1, 1, 1, 1, 1, 3, 4, - 2, 0, 3, 1, 2, 3, 3, 1, 3, 3, - 2, 1, 2, 0, 3, 2, 1, 1, 3, 1, - 3, 4, 1, 3, 5, 5, 1, 1, 1, 4, - 3, 3, 2, 3, 1, 2, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 4, 3, 3, - 1, 2, 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, 1, 3, 3, 2, 1, 2, 0, 3, 2, + 1, 1, 3, 1, 3, 4, 1, 3, 5, 5, + 1, 1, 1, 4, 3, 3, 2, 3, 1, 2, + 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, + 3, 4, 3, 3, 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, 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, 2, 1, 1, 1, 0, - 1, 1, 2, 3, 4, 6, 7, 4, 1, 1, - 1, 1, 2, 3, 3, 3, 3, 3, 3, 3, - 6, 1, 3, + 1, 1, 1, 1, 2, 2, 1, 1, 1, 2, + 1, 1, 1, 0, 1, 1, 2, 3, 4, 6, + 7, 4, 1, 1, 1, 1, 2, 3, 3, 3, + 3, 3, 3, 3, 6, 1, 3, } var yyChk = [...]int16{ - -1000, -58, 99, 100, 101, 102, 2, 10, -14, -7, + -1000, -60, 101, 102, 103, 104, 2, 10, -14, -7, -13, 62, 63, 79, 64, 65, 66, 12, 47, 48, 51, 67, 18, 68, 83, 69, 70, 71, 72, 73, - 85, 88, 89, 74, 75, 90, 13, -59, -14, 10, - -39, -34, -37, -40, -45, -46, -47, -49, -50, -51, - -52, -53, -33, -54, -3, 12, 19, 9, 15, 25, - -8, -7, -44, 90, -12, -55, 62, 63, 64, 65, - 66, 67, 68, 69, 70, 71, 72, 73, 74, 75, - 41, 57, 13, -53, -13, -15, 20, -16, 12, -10, - 2, 25, -21, 2, 41, 59, 42, 43, 45, 46, - 47, 48, 49, 50, 51, 52, 53, 54, 56, 57, - 83, 58, 14, 41, 57, 53, 42, 52, 56, -35, - -42, 2, 79, 85, 15, -42, -39, -54, -39, -54, - -44, 15, 15, -1, 20, -2, 12, -10, 2, 20, - 7, 2, 4, 2, 4, 24, -36, -43, -38, -48, - 78, -36, -36, -36, -36, -36, -36, -36, -36, -36, - -36, -36, -36, -36, -36, -36, -57, 2, -46, -8, - 90, -12, -54, 68, 67, 15, -32, -9, 2, -29, - -31, 88, 89, 19, 9, 41, 57, -56, 2, -54, - -46, -8, 90, -54, -54, -54, -54, -54, -54, -42, - -35, -18, 15, 2, -18, -41, 22, -39, 22, 22, - 22, -54, 20, 7, 2, -5, 2, 4, 54, 44, - 55, -5, 20, -16, 25, 2, 25, 2, -20, 5, - -30, -22, 12, -29, -31, 16, -39, 82, 84, 80, - 81, -39, -39, -39, -39, -39, -39, -39, -39, -39, - -39, -39, -39, -39, -39, -39, -46, 90, -12, 15, - -54, 15, 15, -54, 15, -29, -29, 21, 6, 2, - -17, 22, -4, -6, 25, 2, 62, 78, 63, 79, - 64, 65, 66, 80, 81, 12, 82, 47, 48, 51, - 67, 18, 68, 83, 84, 69, 70, 71, 72, 73, - 88, 89, 59, 74, 75, 90, 22, 7, 7, 20, - -2, 25, 2, 25, 2, 26, 26, -31, 26, 41, - 57, -23, 24, 17, -24, 30, 28, 29, 35, 36, - 37, 33, 31, 34, 32, 38, -18, -18, -19, -18, - -19, 15, 15, -54, 22, -54, 22, -56, 21, 2, - 22, 7, 2, -39, -54, -28, 19, -28, 26, -28, - -22, -22, 24, 17, 2, 17, 6, 6, 6, 6, - 6, 6, 6, 6, 6, 6, 6, 22, -54, 22, - 7, 21, 2, 22, -4, 22, -28, 26, 26, 17, - -24, -27, 57, -28, -32, -32, -32, -29, -25, 14, - -25, -27, -25, -27, -11, 93, 94, 95, 96, 7, - -54, -28, -28, -28, -26, -32, -54, 22, 24, 21, - 2, 22, 21, -32, + 87, 90, 91, 74, 75, 92, 13, -61, -14, 10, + -39, -34, -37, -40, -45, -46, -47, -48, -49, -51, + -52, -53, -54, -55, -33, -56, -3, 12, 19, 9, + 15, 25, -8, -7, -44, 92, -12, -57, 62, 63, + 64, 65, 66, 67, 68, 69, 70, 71, 72, 73, + 74, 75, 41, 57, 13, -55, -13, -15, 20, -16, + 12, -10, 2, 25, -21, 2, 41, 59, 42, 43, + 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, + 56, 57, 83, 85, 84, 58, 14, 41, 57, 53, + 42, 52, 56, -35, -42, 2, 79, 87, 15, -42, + -39, -56, -39, -56, -44, 15, 15, -1, 20, -2, + 12, -10, 2, 20, 7, 2, 4, 2, 4, 24, + -36, -43, -38, -50, 78, -36, -36, -36, -36, -36, + -36, -36, -36, -36, -36, -36, -36, -36, -36, -36, + -59, 2, -46, -8, 92, -12, -56, 68, 67, 15, + -32, -9, 2, -29, -31, 90, 91, 19, 9, 41, + 57, -58, 2, -56, -46, -8, 92, -56, -56, -56, + -56, -56, -56, -42, -35, -18, 15, 2, -18, -41, + 22, -39, 22, 22, 22, -56, 20, 7, 2, -5, + 2, 4, 54, 44, 55, -5, 20, -16, 25, 2, + 25, 2, -20, 5, -30, -22, 12, -29, -31, 16, + -39, 82, 86, 80, 81, -39, -39, -39, -39, -39, + -39, -39, -39, -39, -39, -39, -39, -39, -39, -39, + -46, 92, -12, 15, -56, 15, 15, -56, 15, -29, + -29, 21, 6, 2, -17, 22, -4, -6, 25, 2, + 62, 78, 63, 79, 64, 65, 66, 80, 81, 12, + 82, 47, 48, 51, 67, 18, 68, 83, 86, 69, + 70, 71, 72, 73, 90, 91, 59, 74, 75, 92, + 22, 7, 7, 20, -2, 25, 2, 25, 2, 26, + 26, -31, 26, 41, 57, -23, 24, 17, -24, 30, + 28, 29, 35, 36, 37, 33, 31, 34, 32, 38, + -18, -18, -19, -18, -19, 15, 15, -56, 22, -56, + 22, -58, 21, 2, 22, 7, 2, -39, -56, -28, + 19, -28, 26, -28, -22, -22, 24, 17, 2, 17, + 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, + 6, 22, -56, 22, 7, 21, 2, 22, -4, 22, + -28, 26, 26, 17, -24, -27, 57, -28, -32, -32, + -32, -29, -25, 14, -25, -27, -25, -27, -11, 95, + 96, 97, 98, 7, -56, -28, -28, -28, -26, -32, + -56, 22, 24, 21, 2, 22, 21, -32, } var yyDef = [...]int16{ - 0, -2, 131, 131, 0, 0, 7, 6, 1, 131, - 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, - 112, 113, 114, 115, 116, 117, 118, 119, 120, 121, - 122, 123, 124, 125, 126, 127, 0, 2, -2, 3, + 0, -2, 135, 135, 0, 0, 7, 6, 1, 135, + 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, + 116, 117, 118, 119, 120, 121, 122, 123, 124, 125, + 126, 127, 128, 129, 130, 131, 0, 2, -2, 3, 4, 8, 9, 10, 11, 12, 13, 14, 15, 16, - 17, 18, 19, 20, 0, 109, 236, 237, 0, 247, - 0, 86, 87, 127, 0, 271, -2, -2, -2, -2, + 17, 18, 19, 20, 21, 22, 0, 113, 240, 241, + 0, 251, 0, 90, 91, 131, 0, 275, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, -2, - 230, 231, 0, 5, 101, 0, 130, 133, 0, 137, - 141, 248, 142, 146, 44, 44, 44, 44, 44, 44, - 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, + -2, -2, 234, 235, 0, 5, 105, 0, 134, 137, + 0, 141, 145, 252, 146, 150, 46, 46, 46, 46, + 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, + 46, 46, 0, 74, 75, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 25, 26, 0, 0, 0, 64, + 0, 22, 88, -2, 89, 0, 0, 0, 94, 96, + 0, 100, 104, 132, 0, 138, 0, 144, 0, 149, + 0, 45, 50, 51, 47, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 23, 24, 0, 0, 0, 62, 0, 20, 84, -2, - 85, 0, 0, 0, 90, 92, 0, 96, 100, 128, - 0, 134, 0, 140, 0, 145, 0, 43, 48, 49, - 45, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 70, 71, 251, 0, - 0, 0, 258, 259, 260, 0, 72, 0, 74, 242, - 243, 75, 76, 238, 239, 0, 0, 0, 83, 69, - 261, 0, 0, 263, 264, 265, 266, 267, 268, 21, - 22, 25, 0, 55, 26, 0, 64, 66, 68, 272, - 269, 0, 88, 0, 93, 0, 99, 232, 233, 234, - 235, 0, 129, 132, 135, 138, 136, 139, 144, 147, - 149, 152, 156, 157, 158, 0, 27, 0, 0, -2, - -2, 28, 29, 30, 31, 32, 33, 34, 35, 36, - 37, 38, 39, 40, 41, 42, 252, 0, 0, 0, - 262, 0, 0, 0, 0, 240, 241, 77, 0, 82, - 0, 54, 57, 59, 60, 61, 200, 201, 202, 203, + 72, 73, 255, 0, 0, 0, 262, 263, 264, 0, + 76, 0, 78, 246, 247, 79, 80, 242, 243, 0, + 0, 0, 87, 71, 265, 0, 0, 267, 268, 269, + 270, 271, 272, 23, 24, 27, 0, 57, 28, 0, + 66, 68, 70, 276, 273, 0, 92, 0, 97, 0, + 103, 236, 237, 238, 239, 0, 133, 136, 139, 142, + 140, 143, 148, 151, 153, 156, 160, 161, 162, 0, + 29, 0, 0, -2, -2, 30, 31, 32, 33, 34, + 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, + 256, 0, 0, 0, 266, 0, 0, 0, 0, 244, + 245, 81, 0, 86, 0, 56, 59, 61, 62, 63, 204, 205, 206, 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, 217, 218, 219, 220, 221, 222, 223, - 224, 225, 226, 227, 228, 229, 63, 67, 0, 89, - 91, 94, 98, 95, 97, 0, 0, 0, 0, 0, - 0, 0, 0, 162, 164, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 46, 47, 50, 250, - 51, 0, 0, 0, 253, 0, 73, 0, 79, 81, - 52, 0, 58, 65, 0, 148, 244, 150, 0, 153, - 0, 0, 0, 160, 165, 161, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 254, 0, 257, - 0, 78, 80, 53, 56, 270, 151, 0, 0, 159, - 163, 166, 0, 246, 167, 168, 169, 170, 171, 0, - 172, 173, 174, 175, 176, 182, 183, 184, 185, 0, - 0, 154, 155, 245, 0, 180, 0, 255, 0, 178, - 181, 256, 177, 179, + 224, 225, 226, 227, 228, 229, 230, 231, 232, 233, + 65, 69, 0, 93, 95, 98, 102, 99, 101, 0, + 0, 0, 0, 0, 0, 0, 0, 166, 168, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 48, 49, 52, 254, 53, 0, 0, 0, 257, 0, + 77, 0, 83, 85, 54, 0, 60, 67, 0, 152, + 248, 154, 0, 157, 0, 0, 0, 164, 169, 165, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 258, 0, 261, 0, 82, 84, 55, 58, 274, + 155, 0, 0, 163, 167, 170, 0, 250, 171, 172, + 173, 174, 175, 0, 176, 177, 178, 179, 180, 186, + 187, 188, 189, 0, 0, 158, 159, 249, 0, 184, + 0, 259, 0, 182, 185, 260, 181, 183, } var yyTok1 = [...]int8{ @@ -752,7 +758,7 @@ var yyTok2 = [...]int8{ 72, 73, 74, 75, 76, 77, 78, 79, 80, 81, 82, 83, 84, 85, 86, 87, 88, 89, 90, 91, 92, 93, 94, 95, 96, 97, 98, 99, 100, 101, - 102, 103, + 102, 103, 104, 105, } var yyTok3 = [...]int8{ @@ -1119,7 +1125,7 @@ yydefault: { yylex.(*parser).unexpected("", "") } - case 21: + case 23: yyDollar = yyS[yypt-3 : yypt+1] { // Need to consume the position of the first RIGHT_PAREN. It might not exist on garbage input @@ -1129,7 +1135,7 @@ yydefault: } yyVAL.node = yylex.(*parser).newAggregateExpr(yyDollar[1].item, yyDollar[2].node, yyDollar[3].node) } - case 22: + case 24: yyDollar = yyS[yypt-3 : yypt+1] { // Need to consume the position of the first RIGHT_PAREN. It might not exist on garbage input @@ -1139,25 +1145,25 @@ yydefault: } yyVAL.node = yylex.(*parser).newAggregateExpr(yyDollar[1].item, yyDollar[3].node, yyDollar[2].node) } - case 23: + case 25: yyDollar = yyS[yypt-2 : yypt+1] { yyVAL.node = yylex.(*parser).newAggregateExpr(yyDollar[1].item, &AggregateExpr{}, yyDollar[2].node) } - case 24: + case 26: yyDollar = yyS[yypt-2 : yypt+1] { yylex.(*parser).unexpected("aggregation", "") yyVAL.node = yylex.(*parser).newAggregateExpr(yyDollar[1].item, &AggregateExpr{}, Expressions{}) } - case 25: + case 27: yyDollar = yyS[yypt-2 : yypt+1] { yyVAL.node = &AggregateExpr{ Grouping: yyDollar[2].strings, } } - case 26: + case 28: yyDollar = yyS[yypt-2 : yypt+1] { yyVAL.node = &AggregateExpr{ @@ -1165,16 +1171,6 @@ yydefault: Without: true, } } - case 27: - yyDollar = yyS[yypt-4 : yypt+1] - { - yyVAL.node = yylex.(*parser).newBinaryExpression(yyDollar[1].node, yyDollar[2].item, yyDollar[3].node, yyDollar[4].node) - } - case 28: - yyDollar = yyS[yypt-4 : yypt+1] - { - yyVAL.node = yylex.(*parser).newBinaryExpression(yyDollar[1].node, yyDollar[2].item, yyDollar[3].node, yyDollar[4].node) - } case 29: yyDollar = yyS[yypt-4 : yypt+1] { @@ -1245,14 +1241,24 @@ yydefault: { yyVAL.node = yylex.(*parser).newBinaryExpression(yyDollar[1].node, yyDollar[2].item, yyDollar[3].node, yyDollar[4].node) } + case 43: + yyDollar = yyS[yypt-4 : yypt+1] + { + yyVAL.node = yylex.(*parser).newBinaryExpression(yyDollar[1].node, yyDollar[2].item, yyDollar[3].node, yyDollar[4].node) + } case 44: + yyDollar = yyS[yypt-4 : yypt+1] + { + yyVAL.node = yylex.(*parser).newBinaryExpression(yyDollar[1].node, yyDollar[2].item, yyDollar[3].node, yyDollar[4].node) + } + case 46: yyDollar = yyS[yypt-0 : yypt+1] { yyVAL.node = &BinaryExpr{ VectorMatching: &VectorMatching{Card: CardOneToOne}, } } - case 45: + case 47: yyDollar = yyS[yypt-1 : yypt+1] { yyVAL.node = &BinaryExpr{ @@ -1260,71 +1266,71 @@ yydefault: ReturnBool: true, } } - case 46: + case 48: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.node = yyDollar[1].node yyVAL.node.(*BinaryExpr).VectorMatching.MatchingLabels = yyDollar[3].strings } - case 47: + case 49: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.node = yyDollar[1].node yyVAL.node.(*BinaryExpr).VectorMatching.MatchingLabels = yyDollar[3].strings yyVAL.node.(*BinaryExpr).VectorMatching.On = true } - case 50: + case 52: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.node = yyDollar[1].node yyVAL.node.(*BinaryExpr).VectorMatching.Card = CardManyToOne yyVAL.node.(*BinaryExpr).VectorMatching.Include = yyDollar[3].strings } - case 51: + case 53: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.node = yyDollar[1].node yyVAL.node.(*BinaryExpr).VectorMatching.Card = CardOneToMany yyVAL.node.(*BinaryExpr).VectorMatching.Include = yyDollar[3].strings } - case 52: + case 54: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.strings = yyDollar[2].strings } - case 53: + case 55: yyDollar = yyS[yypt-4 : yypt+1] { yyVAL.strings = yyDollar[2].strings } - case 54: + case 56: yyDollar = yyS[yypt-2 : yypt+1] { yyVAL.strings = []string{} } - case 55: + case 57: yyDollar = yyS[yypt-1 : yypt+1] { yylex.(*parser).unexpected("grouping opts", "\"(\"") yyVAL.strings = nil } - case 56: + case 58: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.strings = append(yyDollar[1].strings, yyDollar[3].item.Val) } - case 57: + case 59: yyDollar = yyS[yypt-1 : yypt+1] { yyVAL.strings = []string{yyDollar[1].item.Val} } - case 58: + case 60: yyDollar = yyS[yypt-2 : yypt+1] { yylex.(*parser).unexpected("grouping opts", "\",\" or \")\"") yyVAL.strings = yyDollar[1].strings } - case 59: + case 61: yyDollar = yyS[yypt-1 : yypt+1] { if !model.LabelName(yyDollar[1].item.Val).IsValid() { @@ -1332,7 +1338,7 @@ yydefault: } yyVAL.item = yyDollar[1].item } - case 60: + case 62: yyDollar = yyS[yypt-1 : yypt+1] { unquoted := yylex.(*parser).unquoteString(yyDollar[1].item.Val) @@ -1343,13 +1349,13 @@ yydefault: yyVAL.item.Pos++ yyVAL.item.Val = unquoted } - case 61: + case 63: yyDollar = yyS[yypt-1 : yypt+1] { yylex.(*parser).unexpected("grouping opts", "label") yyVAL.item = Item{} } - case 62: + case 64: yyDollar = yyS[yypt-2 : yypt+1] { fn, exist := getFunction(yyDollar[1].item.Val, yylex.(*parser).functions) @@ -1369,39 +1375,39 @@ yydefault: } yylex.(*parser).closingParens = yylex.(*parser).closingParens[1:] } - case 63: + case 65: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.node = yyDollar[2].node } - case 64: + case 66: yyDollar = yyS[yypt-2 : yypt+1] { yyVAL.node = Expressions{} } - case 65: + case 67: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.node = append(yyDollar[1].node.(Expressions), yyDollar[3].node.(Expr)) } - case 66: + case 68: yyDollar = yyS[yypt-1 : yypt+1] { yyVAL.node = Expressions{yyDollar[1].node.(Expr)} } - case 67: + case 69: yyDollar = yyS[yypt-2 : yypt+1] { yylex.(*parser).addParseErrf(yyDollar[2].item.PositionRange(), "trailing commas not allowed in function call args") yyVAL.node = yyDollar[1].node } - case 68: + case 70: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.node = &ParenExpr{Expr: yyDollar[2].node.(Expr), PosRange: mergeRanges(&yyDollar[1].item, &yyDollar[3].item)} yylex.(*parser).closingParens = yylex.(*parser).closingParens[1:] } - case 69: + case 71: yyDollar = yyS[yypt-1 : yypt+1] { if numLit, ok := yyDollar[1].node.(*NumberLiteral); ok { @@ -1415,7 +1421,7 @@ yydefault: } yyVAL.node = yyDollar[1].node } - case 70: + case 72: yyDollar = yyS[yypt-3 : yypt+1] { if numLit, ok := yyDollar[3].node.(*NumberLiteral); ok { @@ -1426,31 +1432,41 @@ yydefault: yylex.(*parser).addOffsetExpr(yyDollar[1].node, yyDollar[3].node.(*DurationExpr)) yyVAL.node = yyDollar[1].node } - case 71: + case 73: yyDollar = yyS[yypt-3 : yypt+1] { yylex.(*parser).unexpected("offset", "number, duration, or step()") yyVAL.node = yyDollar[1].node } - case 72: + case 74: + yyDollar = yyS[yypt-2 : yypt+1] + { + yylex.(*parser).setAnchored(yyDollar[1].node) + } + case 75: + yyDollar = yyS[yypt-2 : yypt+1] + { + yylex.(*parser).setSmoothed(yyDollar[1].node) + } + case 76: yyDollar = yyS[yypt-3 : yypt+1] { yylex.(*parser).setTimestamp(yyDollar[1].node, yyDollar[3].float) yyVAL.node = yyDollar[1].node } - case 73: + case 77: yyDollar = yyS[yypt-5 : yypt+1] { yylex.(*parser).setAtModifierPreprocessor(yyDollar[1].node, yyDollar[3].item) yyVAL.node = yyDollar[1].node } - case 74: + case 78: yyDollar = yyS[yypt-3 : yypt+1] { yylex.(*parser).unexpected("@", "timestamp") yyVAL.node = yyDollar[1].node } - case 77: + case 81: yyDollar = yyS[yypt-4 : yypt+1] { var errMsg string @@ -1480,7 +1496,7 @@ yydefault: EndPos: yylex.(*parser).lastClosing, } } - case 78: + case 82: yyDollar = yyS[yypt-6 : yypt+1] { var rangeNl time.Duration @@ -1502,7 +1518,7 @@ yydefault: EndPos: yyDollar[6].item.Pos + 1, } } - case 79: + case 83: yyDollar = yyS[yypt-5 : yypt+1] { var rangeNl time.Duration @@ -1517,31 +1533,31 @@ yydefault: EndPos: yyDollar[5].item.Pos + 1, } } - case 80: + case 84: yyDollar = yyS[yypt-6 : yypt+1] { yylex.(*parser).unexpected("subquery selector", "\"]\"") yyVAL.node = yyDollar[1].node } - case 81: + case 85: yyDollar = yyS[yypt-5 : yypt+1] { yylex.(*parser).unexpected("subquery selector", "number, duration, or step() or \"]\"") yyVAL.node = yyDollar[1].node } - case 82: + case 86: yyDollar = yyS[yypt-4 : yypt+1] { yylex.(*parser).unexpected("subquery or range", "\":\" or \"]\"") yyVAL.node = yyDollar[1].node } - case 83: + case 87: yyDollar = yyS[yypt-3 : yypt+1] { yylex.(*parser).unexpected("subquery or range selector", "number, duration, or step()") yyVAL.node = yyDollar[1].node } - case 84: + case 88: yyDollar = yyS[yypt-2 : yypt+1] { if nl, ok := yyDollar[2].node.(*NumberLiteral); ok { @@ -1554,7 +1570,7 @@ yydefault: yyVAL.node = &UnaryExpr{Op: yyDollar[1].item.Typ, Expr: yyDollar[2].node.(Expr), StartPos: yyDollar[1].item.Pos} } } - case 85: + case 89: yyDollar = yyS[yypt-2 : yypt+1] { vs := yyDollar[2].node.(*VectorSelector) @@ -1563,7 +1579,7 @@ yydefault: yylex.(*parser).assembleVectorSelector(vs) yyVAL.node = vs } - case 86: + case 90: yyDollar = yyS[yypt-1 : yypt+1] { vs := &VectorSelector{ @@ -1574,14 +1590,14 @@ yydefault: yylex.(*parser).assembleVectorSelector(vs) yyVAL.node = vs } - case 87: + case 91: yyDollar = yyS[yypt-1 : yypt+1] { vs := yyDollar[1].node.(*VectorSelector) yylex.(*parser).assembleVectorSelector(vs) yyVAL.node = vs } - case 88: + case 92: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.node = &VectorSelector{ @@ -1589,7 +1605,7 @@ yydefault: PosRange: mergeRanges(&yyDollar[1].item, &yyDollar[3].item), } } - case 89: + case 93: yyDollar = yyS[yypt-4 : yypt+1] { yyVAL.node = &VectorSelector{ @@ -1597,7 +1613,7 @@ yydefault: PosRange: mergeRanges(&yyDollar[1].item, &yyDollar[4].item), } } - case 90: + case 94: yyDollar = yyS[yypt-2 : yypt+1] { yyVAL.node = &VectorSelector{ @@ -1605,7 +1621,7 @@ yydefault: PosRange: mergeRanges(&yyDollar[1].item, &yyDollar[2].item), } } - case 91: + case 95: yyDollar = yyS[yypt-3 : yypt+1] { if yyDollar[1].matchers != nil { @@ -1614,144 +1630,144 @@ yydefault: yyVAL.matchers = yyDollar[1].matchers } } - case 92: + case 96: yyDollar = yyS[yypt-1 : yypt+1] { yyVAL.matchers = []*labels.Matcher{yyDollar[1].matcher} } - case 93: + case 97: yyDollar = yyS[yypt-2 : yypt+1] { yylex.(*parser).unexpected("label matching", "\",\" or \"}\"") yyVAL.matchers = yyDollar[1].matchers } - case 94: + case 98: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.matcher = yylex.(*parser).newLabelMatcher(yyDollar[1].item, yyDollar[2].item, yyDollar[3].item) } - case 95: + case 99: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.matcher = yylex.(*parser).newLabelMatcher(yyDollar[1].item, yyDollar[2].item, yyDollar[3].item) } - case 96: + case 100: yyDollar = yyS[yypt-1 : yypt+1] { yyVAL.matcher = yylex.(*parser).newMetricNameMatcher(yyDollar[1].item) } - case 97: + case 101: yyDollar = yyS[yypt-3 : yypt+1] { yylex.(*parser).unexpected("label matching", "string") yyVAL.matcher = nil } - case 98: + case 102: yyDollar = yyS[yypt-3 : yypt+1] { yylex.(*parser).unexpected("label matching", "string") yyVAL.matcher = nil } - case 99: + case 103: yyDollar = yyS[yypt-2 : yypt+1] { yylex.(*parser).unexpected("label matching", "label matching operator") yyVAL.matcher = nil } - case 100: + case 104: yyDollar = yyS[yypt-1 : yypt+1] { yylex.(*parser).unexpected("label matching", "identifier or \"}\"") yyVAL.matcher = nil } - case 101: + case 105: yyDollar = yyS[yypt-2 : yypt+1] { b := labels.NewBuilder(yyDollar[2].labels) b.Set(labels.MetricName, yyDollar[1].item.Val) yyVAL.labels = b.Labels() } - case 102: + case 106: yyDollar = yyS[yypt-1 : yypt+1] { yyVAL.labels = yyDollar[1].labels } - case 128: + case 132: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.labels = labels.New(yyDollar[2].lblList...) } - case 129: + case 133: yyDollar = yyS[yypt-4 : yypt+1] { yyVAL.labels = labels.New(yyDollar[2].lblList...) } - case 130: + case 134: yyDollar = yyS[yypt-2 : yypt+1] { yyVAL.labels = labels.New() } - case 131: + case 135: yyDollar = yyS[yypt-0 : yypt+1] { yyVAL.labels = labels.New() } - case 132: + case 136: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.lblList = append(yyDollar[1].lblList, yyDollar[3].label) } - case 133: + case 137: yyDollar = yyS[yypt-1 : yypt+1] { yyVAL.lblList = []labels.Label{yyDollar[1].label} } - case 134: + case 138: yyDollar = yyS[yypt-2 : yypt+1] { yylex.(*parser).unexpected("label set", "\",\" or \"}\"") yyVAL.lblList = yyDollar[1].lblList } - case 135: + case 139: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.label = labels.Label{Name: yyDollar[1].item.Val, Value: yylex.(*parser).unquoteString(yyDollar[3].item.Val)} } - case 136: + case 140: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.label = labels.Label{Name: yyDollar[1].item.Val, Value: yylex.(*parser).unquoteString(yyDollar[3].item.Val)} } - case 137: + case 141: yyDollar = yyS[yypt-1 : yypt+1] { yyVAL.label = labels.Label{Name: labels.MetricName, Value: yyDollar[1].item.Val} } - case 138: + case 142: yyDollar = yyS[yypt-3 : yypt+1] { yylex.(*parser).unexpected("label set", "string") yyVAL.label = labels.Label{} } - case 139: + case 143: yyDollar = yyS[yypt-3 : yypt+1] { yylex.(*parser).unexpected("label set", "string") yyVAL.label = labels.Label{} } - case 140: + case 144: yyDollar = yyS[yypt-2 : yypt+1] { yylex.(*parser).unexpected("label set", "\"=\"") yyVAL.label = labels.Label{} } - case 141: + case 145: yyDollar = yyS[yypt-1 : yypt+1] { yylex.(*parser).unexpected("label set", "identifier or \"}\"") yyVAL.label = labels.Label{} } - case 142: + case 146: yyDollar = yyS[yypt-2 : yypt+1] { yylex.(*parser).generatedParserResult = &seriesDescription{ @@ -1759,33 +1775,33 @@ yydefault: values: yyDollar[2].series, } } - case 143: + case 147: yyDollar = yyS[yypt-0 : yypt+1] { yyVAL.series = []SequenceValue{} } - case 144: + case 148: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.series = append(yyDollar[1].series, yyDollar[3].series...) } - case 145: + case 149: yyDollar = yyS[yypt-2 : yypt+1] { yyVAL.series = yyDollar[1].series } - case 146: + case 150: yyDollar = yyS[yypt-1 : yypt+1] { yylex.(*parser).unexpected("series values", "") yyVAL.series = nil } - case 147: + case 151: yyDollar = yyS[yypt-1 : yypt+1] { yyVAL.series = []SequenceValue{{Omitted: true}} } - case 148: + case 152: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.series = []SequenceValue{} @@ -1793,12 +1809,12 @@ yydefault: yyVAL.series = append(yyVAL.series, SequenceValue{Omitted: true}) } } - case 149: + case 153: yyDollar = yyS[yypt-1 : yypt+1] { yyVAL.series = []SequenceValue{{Value: yyDollar[1].float}} } - case 150: + case 154: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.series = []SequenceValue{} @@ -1807,7 +1823,7 @@ yydefault: yyVAL.series = append(yyVAL.series, SequenceValue{Value: yyDollar[1].float}) } } - case 151: + case 155: yyDollar = yyS[yypt-4 : yypt+1] { yyVAL.series = []SequenceValue{} @@ -1817,12 +1833,12 @@ yydefault: yyDollar[1].float += yyDollar[2].float } } - case 152: + case 156: yyDollar = yyS[yypt-1 : yypt+1] { yyVAL.series = []SequenceValue{{Histogram: yyDollar[1].histogram}} } - case 153: + case 157: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.series = []SequenceValue{} @@ -1832,7 +1848,7 @@ yydefault: //$1 += $2 } } - case 154: + case 158: yyDollar = yyS[yypt-5 : yypt+1] { val, err := yylex.(*parser).histogramsIncreaseSeries(yyDollar[1].histogram, yyDollar[3].histogram, yyDollar[5].uint) @@ -1841,7 +1857,7 @@ yydefault: } yyVAL.series = val } - case 155: + case 159: yyDollar = yyS[yypt-5 : yypt+1] { val, err := yylex.(*parser).histogramsDecreaseSeries(yyDollar[1].histogram, yyDollar[3].histogram, yyDollar[5].uint) @@ -1850,7 +1866,7 @@ yydefault: } yyVAL.series = val } - case 156: + case 160: yyDollar = yyS[yypt-1 : yypt+1] { if yyDollar[1].item.Val != "stale" { @@ -1858,130 +1874,130 @@ yydefault: } yyVAL.float = math.Float64frombits(value.StaleNaN) } - case 159: + case 163: yyDollar = yyS[yypt-4 : yypt+1] { yyVAL.histogram = yylex.(*parser).buildHistogramFromMap(&yyDollar[2].descriptors) } - case 160: + case 164: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.histogram = yylex.(*parser).buildHistogramFromMap(&yyDollar[2].descriptors) } - case 161: - yyDollar = yyS[yypt-3 : yypt+1] - { - m := yylex.(*parser).newMap() - yyVAL.histogram = yylex.(*parser).buildHistogramFromMap(&m) - } - case 162: - yyDollar = yyS[yypt-2 : yypt+1] - { - m := yylex.(*parser).newMap() - yyVAL.histogram = yylex.(*parser).buildHistogramFromMap(&m) - } - case 163: - yyDollar = yyS[yypt-3 : yypt+1] - { - yyVAL.descriptors = *(yylex.(*parser).mergeMaps(&yyDollar[1].descriptors, &yyDollar[3].descriptors)) - } - case 164: - yyDollar = yyS[yypt-1 : yypt+1] - { - yyVAL.descriptors = yyDollar[1].descriptors - } case 165: - yyDollar = yyS[yypt-2 : yypt+1] + yyDollar = yyS[yypt-3 : yypt+1] { - yylex.(*parser).unexpected("histogram description", "histogram description key, e.g. buckets:[5 10 7]") + m := yylex.(*parser).newMap() + yyVAL.histogram = yylex.(*parser).buildHistogramFromMap(&m) } case 166: - yyDollar = yyS[yypt-3 : yypt+1] + yyDollar = yyS[yypt-2 : yypt+1] { - yyVAL.descriptors = yylex.(*parser).newMap() - yyVAL.descriptors["schema"] = yyDollar[3].int + m := yylex.(*parser).newMap() + yyVAL.histogram = yylex.(*parser).buildHistogramFromMap(&m) } case 167: yyDollar = yyS[yypt-3 : yypt+1] { - yyVAL.descriptors = yylex.(*parser).newMap() - yyVAL.descriptors["sum"] = yyDollar[3].float + yyVAL.descriptors = *(yylex.(*parser).mergeMaps(&yyDollar[1].descriptors, &yyDollar[3].descriptors)) } case 168: - yyDollar = yyS[yypt-3 : yypt+1] + yyDollar = yyS[yypt-1 : yypt+1] { - yyVAL.descriptors = yylex.(*parser).newMap() - yyVAL.descriptors["count"] = yyDollar[3].float + yyVAL.descriptors = yyDollar[1].descriptors } case 169: - yyDollar = yyS[yypt-3 : yypt+1] + yyDollar = yyS[yypt-2 : yypt+1] { - yyVAL.descriptors = yylex.(*parser).newMap() - yyVAL.descriptors["z_bucket"] = yyDollar[3].float + yylex.(*parser).unexpected("histogram description", "histogram description key, e.g. buckets:[5 10 7]") } case 170: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.descriptors = yylex.(*parser).newMap() - yyVAL.descriptors["z_bucket_w"] = yyDollar[3].float + yyVAL.descriptors["schema"] = yyDollar[3].int } case 171: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.descriptors = yylex.(*parser).newMap() - yyVAL.descriptors["custom_values"] = yyDollar[3].bucket_set + yyVAL.descriptors["sum"] = yyDollar[3].float } case 172: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.descriptors = yylex.(*parser).newMap() - yyVAL.descriptors["buckets"] = yyDollar[3].bucket_set + yyVAL.descriptors["count"] = yyDollar[3].float } case 173: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.descriptors = yylex.(*parser).newMap() - yyVAL.descriptors["offset"] = yyDollar[3].int + yyVAL.descriptors["z_bucket"] = yyDollar[3].float } case 174: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.descriptors = yylex.(*parser).newMap() - yyVAL.descriptors["n_buckets"] = yyDollar[3].bucket_set + yyVAL.descriptors["z_bucket_w"] = yyDollar[3].float } case 175: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.descriptors = yylex.(*parser).newMap() - yyVAL.descriptors["n_offset"] = yyDollar[3].int + yyVAL.descriptors["custom_values"] = yyDollar[3].bucket_set } case 176: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.descriptors = yylex.(*parser).newMap() - yyVAL.descriptors["counter_reset_hint"] = yyDollar[3].item + yyVAL.descriptors["buckets"] = yyDollar[3].bucket_set } case 177: - yyDollar = yyS[yypt-4 : yypt+1] + yyDollar = yyS[yypt-3 : yypt+1] { - yyVAL.bucket_set = yyDollar[2].bucket_set + yyVAL.descriptors = yylex.(*parser).newMap() + yyVAL.descriptors["offset"] = yyDollar[3].int } case 178: yyDollar = yyS[yypt-3 : yypt+1] { - yyVAL.bucket_set = yyDollar[2].bucket_set + yyVAL.descriptors = yylex.(*parser).newMap() + yyVAL.descriptors["n_buckets"] = yyDollar[3].bucket_set } case 179: yyDollar = yyS[yypt-3 : yypt+1] { - yyVAL.bucket_set = append(yyDollar[1].bucket_set, yyDollar[3].float) + yyVAL.descriptors = yylex.(*parser).newMap() + yyVAL.descriptors["n_offset"] = yyDollar[3].int } case 180: + yyDollar = yyS[yypt-3 : yypt+1] + { + yyVAL.descriptors = yylex.(*parser).newMap() + yyVAL.descriptors["counter_reset_hint"] = yyDollar[3].item + } + case 181: + yyDollar = yyS[yypt-4 : yypt+1] + { + yyVAL.bucket_set = yyDollar[2].bucket_set + } + case 182: + yyDollar = yyS[yypt-3 : yypt+1] + { + yyVAL.bucket_set = yyDollar[2].bucket_set + } + case 183: + yyDollar = yyS[yypt-3 : yypt+1] + { + yyVAL.bucket_set = append(yyDollar[1].bucket_set, yyDollar[3].float) + } + case 184: yyDollar = yyS[yypt-1 : yypt+1] { yyVAL.bucket_set = []float64{yyDollar[1].float} } - case 236: + case 240: yyDollar = yyS[yypt-1 : yypt+1] { yyVAL.node = &NumberLiteral{ @@ -1989,7 +2005,7 @@ yydefault: PosRange: yyDollar[1].item.PositionRange(), } } - case 237: + case 241: yyDollar = yyS[yypt-1 : yypt+1] { var err error @@ -2004,12 +2020,12 @@ yydefault: Duration: true, } } - case 238: + case 242: yyDollar = yyS[yypt-1 : yypt+1] { yyVAL.float = yylex.(*parser).number(yyDollar[1].item.Val) } - case 239: + case 243: yyDollar = yyS[yypt-1 : yypt+1] { var err error @@ -2020,17 +2036,17 @@ yydefault: } yyVAL.float = dur.Seconds() } - case 240: + case 244: yyDollar = yyS[yypt-2 : yypt+1] { yyVAL.float = yyDollar[2].float } - case 241: + case 245: yyDollar = yyS[yypt-2 : yypt+1] { yyVAL.float = -yyDollar[2].float } - case 244: + case 248: yyDollar = yyS[yypt-1 : yypt+1] { var err error @@ -2039,17 +2055,17 @@ yydefault: yylex.(*parser).addParseErrf(yyDollar[1].item.PositionRange(), "invalid repetition in series values: %s", err) } } - case 245: + case 249: yyDollar = yyS[yypt-2 : yypt+1] { yyVAL.int = -int64(yyDollar[2].uint) } - case 246: + case 250: yyDollar = yyS[yypt-1 : yypt+1] { yyVAL.int = int64(yyDollar[1].uint) } - case 247: + case 251: yyDollar = yyS[yypt-1 : yypt+1] { yyVAL.node = &StringLiteral{ @@ -2057,7 +2073,7 @@ yydefault: PosRange: yyDollar[1].item.PositionRange(), } } - case 248: + case 252: yyDollar = yyS[yypt-1 : yypt+1] { yyVAL.item = Item{ @@ -2066,12 +2082,12 @@ yydefault: Val: yylex.(*parser).unquoteString(yyDollar[1].item.Val), } } - case 249: + case 253: yyDollar = yyS[yypt-0 : yypt+1] { yyVAL.strings = nil } - case 251: + case 255: yyDollar = yyS[yypt-1 : yypt+1] { nl := yyDollar[1].node.(*NumberLiteral) @@ -2082,7 +2098,7 @@ yydefault: } yyVAL.node = nl } - case 252: + case 256: yyDollar = yyS[yypt-2 : yypt+1] { nl := yyDollar[2].node.(*NumberLiteral) @@ -2097,7 +2113,7 @@ yydefault: nl.PosRange.Start = yyDollar[1].item.Pos yyVAL.node = nl } - case 253: + case 257: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.node = &DurationExpr{ @@ -2106,7 +2122,7 @@ yydefault: EndPos: yyDollar[3].item.PositionRange().End, } } - case 254: + case 258: yyDollar = yyS[yypt-4 : yypt+1] { yyVAL.node = &DurationExpr{ @@ -2119,7 +2135,7 @@ yydefault: StartPos: yyDollar[1].item.Pos, } } - case 255: + case 259: yyDollar = yyS[yypt-6 : yypt+1] { yyVAL.node = &DurationExpr{ @@ -2130,7 +2146,7 @@ yydefault: RHS: yyDollar[5].node.(Expr), } } - case 256: + case 260: yyDollar = yyS[yypt-7 : yypt+1] { yyVAL.node = &DurationExpr{ @@ -2146,7 +2162,7 @@ yydefault: }, } } - case 257: + case 261: yyDollar = yyS[yypt-4 : yypt+1] { de := yyDollar[3].node.(*DurationExpr) @@ -2161,7 +2177,7 @@ yydefault: } yyVAL.node = yyDollar[3].node } - case 261: + case 265: yyDollar = yyS[yypt-1 : yypt+1] { nl := yyDollar[1].node.(*NumberLiteral) @@ -2172,7 +2188,7 @@ yydefault: } yyVAL.node = nl } - case 262: + case 266: yyDollar = yyS[yypt-2 : yypt+1] { switch expr := yyDollar[2].node.(type) { @@ -2205,25 +2221,25 @@ yydefault: break } } - case 263: + case 267: yyDollar = yyS[yypt-3 : yypt+1] { yylex.(*parser).experimentalDurationExpr(yyDollar[1].node.(Expr)) yyVAL.node = &DurationExpr{Op: ADD, LHS: yyDollar[1].node.(Expr), RHS: yyDollar[3].node.(Expr)} } - case 264: + case 268: yyDollar = yyS[yypt-3 : yypt+1] { yylex.(*parser).experimentalDurationExpr(yyDollar[1].node.(Expr)) yyVAL.node = &DurationExpr{Op: SUB, LHS: yyDollar[1].node.(Expr), RHS: yyDollar[3].node.(Expr)} } - case 265: + case 269: yyDollar = yyS[yypt-3 : yypt+1] { yylex.(*parser).experimentalDurationExpr(yyDollar[1].node.(Expr)) yyVAL.node = &DurationExpr{Op: MUL, LHS: yyDollar[1].node.(Expr), RHS: yyDollar[3].node.(Expr)} } - case 266: + case 270: yyDollar = yyS[yypt-3 : yypt+1] { yylex.(*parser).experimentalDurationExpr(yyDollar[1].node.(Expr)) @@ -2234,7 +2250,7 @@ yydefault: } yyVAL.node = &DurationExpr{Op: DIV, LHS: yyDollar[1].node.(Expr), RHS: yyDollar[3].node.(Expr)} } - case 267: + case 271: yyDollar = yyS[yypt-3 : yypt+1] { yylex.(*parser).experimentalDurationExpr(yyDollar[1].node.(Expr)) @@ -2245,13 +2261,13 @@ yydefault: } yyVAL.node = &DurationExpr{Op: MOD, LHS: yyDollar[1].node.(Expr), RHS: yyDollar[3].node.(Expr)} } - case 268: + case 272: yyDollar = yyS[yypt-3 : yypt+1] { yylex.(*parser).experimentalDurationExpr(yyDollar[1].node.(Expr)) yyVAL.node = &DurationExpr{Op: POW, LHS: yyDollar[1].node.(Expr), RHS: yyDollar[3].node.(Expr)} } - case 269: + case 273: yyDollar = yyS[yypt-3 : yypt+1] { yyVAL.node = &DurationExpr{ @@ -2260,7 +2276,7 @@ yydefault: EndPos: yyDollar[3].item.PositionRange().End, } } - case 270: + case 274: yyDollar = yyS[yypt-6 : yypt+1] { yyVAL.node = &DurationExpr{ @@ -2271,7 +2287,7 @@ yydefault: RHS: yyDollar[5].node.(Expr), } } - case 272: + case 276: yyDollar = yyS[yypt-3 : yypt+1] { yylex.(*parser).experimentalDurationExpr(yyDollar[2].node.(Expr)) diff --git a/promql/parser/lex.go b/promql/parser/lex.go index 2b3eecbadd..03c7645f79 100644 --- a/promql/parser/lex.go +++ b/promql/parser/lex.go @@ -129,6 +129,8 @@ var key = map[string]ItemType{ // Keywords. "offset": OFFSET, + "smoothed": SMOOTHED, + "anchored": ANCHORED, "by": BY, "without": WITHOUT, "on": ON, diff --git a/promql/parser/parse.go b/promql/parser/parse.go index e99f5f4570..d0e18efa2e 100644 --- a/promql/parser/parse.go +++ b/promql/parser/parse.go @@ -42,6 +42,9 @@ var parserPool = sync.Pool{ // ExperimentalDurationExpr is a flag to enable experimental duration expression parsing. var ExperimentalDurationExpr bool +// EnableExtendedRangeSelectors is a flag to enable experimental extended range selectors. +var EnableExtendedRangeSelectors bool + type Parser interface { ParseExpr() (Expr, error) Close() @@ -1037,6 +1040,48 @@ func (p *parser) addOffsetExpr(e Node, expr *DurationExpr) { *endPosp = p.lastClosing } +func (p *parser) setAnchored(e Node) { + if !EnableExtendedRangeSelectors { + p.addParseErrf(e.PositionRange(), "anchored modifier is experimental and not enabled") + return + } + switch s := e.(type) { + case *VectorSelector: + s.Anchored = true + if s.Smoothed { + p.addParseErrf(e.PositionRange(), "anchored and smoothed modifiers cannot be used together") + } + case *MatrixSelector: + s.VectorSelector.(*VectorSelector).Anchored = true + if s.VectorSelector.(*VectorSelector).Smoothed { + p.addParseErrf(e.PositionRange(), "anchored and smoothed modifiers cannot be used together") + } + default: + p.addParseErrf(e.PositionRange(), "anchored modifier not implemented") + } +} + +func (p *parser) setSmoothed(e Node) { + if !EnableExtendedRangeSelectors { + p.addParseErrf(e.PositionRange(), "smoothed modifier is experimental and not enabled") + return + } + switch s := e.(type) { + case *VectorSelector: + s.Smoothed = true + if s.Anchored { + p.addParseErrf(e.PositionRange(), "anchored and smoothed modifiers cannot be used together") + } + case *MatrixSelector: + s.VectorSelector.(*VectorSelector).Smoothed = true + if s.VectorSelector.(*VectorSelector).Anchored { + p.addParseErrf(e.PositionRange(), "anchored and smoothed modifiers cannot be used together") + } + default: + p.addParseErrf(e.PositionRange(), "smoothed modifier not implemented") + } +} + // 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) || diff --git a/promql/parser/printer.go b/promql/parser/printer.go index 9dae10a70e..2cf933f481 100644 --- a/promql/parser/printer.go +++ b/promql/parser/printer.go @@ -222,11 +222,17 @@ func (node *MatrixSelector) String() string { vecSelector.Timestamp = nil vecSelector.StartOrEnd = 0 + extendedAttribute := "" + if vecSelector.Anchored { + extendedAttribute = " anchored" + } else if vecSelector.Smoothed { + extendedAttribute = " smoothed" + } rangeStr := model.Duration(node.Range).String() if node.RangeExpr != nil { rangeStr = node.RangeExpr.String() } - str := fmt.Sprintf("%s[%s]%s%s", vecSelector.String(), rangeStr, at, offset) + str := fmt.Sprintf("%s[%s]%s%s%s", vecSelector.String(), rangeStr, extendedAttribute, at, offset) vecSelector.OriginalOffset, vecSelector.OriginalOffsetExpr, vecSelector.Timestamp, vecSelector.StartOrEnd = offsetVal, offsetExprVal, atVal, preproc @@ -321,6 +327,12 @@ func (node *VectorSelector) String() string { } labelStrings = append(labelStrings, matcher.String()) } + extendedAttribute := "" + if node.Anchored { + extendedAttribute = " anchored" + } else if node.Smoothed { + extendedAttribute = " smoothed" + } offset := "" switch { case node.OriginalOffsetExpr != nil: @@ -344,5 +356,5 @@ func (node *VectorSelector) String() string { return fmt.Sprintf("%s%s%s", node.Name, at, offset) } sort.Strings(labelStrings) - return fmt.Sprintf("%s{%s}%s%s", node.Name, strings.Join(labelStrings, ","), at, offset) + return fmt.Sprintf("%s{%s}%s%s%s", node.Name, strings.Join(labelStrings, ","), extendedAttribute, at, offset) } diff --git a/promql/promqltest/test.go b/promql/promqltest/test.go index 1754f6635d..37a963d504 100644 --- a/promql/promqltest/test.go +++ b/promql/promqltest/test.go @@ -123,6 +123,7 @@ func RunBuiltinTestsWithStorage(t TBRun, engine promql.QueryEngine, newStorage f }) parser.EnableExperimentalFunctions = true parser.ExperimentalDurationExpr = true + parser.EnableExtendedRangeSelectors = true files, err := fs.Glob(testsFs, "*/*.test") require.NoError(t, err) diff --git a/promql/promqltest/testdata/extended_vectors.test b/promql/promqltest/testdata/extended_vectors.test new file mode 100644 index 0000000000..19e4118bbe --- /dev/null +++ b/promql/promqltest/testdata/extended_vectors.test @@ -0,0 +1,216 @@ +# Reference from PROM-52: Complete dataset + +load 15s + metric 1+1x4 9+1x4 + +eval instant at 5s increase(metric[1m]) + +eval instant at 20s increase(metric[1m]) + {} 1.833333333 + +eval instant at 35s increase(metric[1m]) + {} 2.833333333 + +eval instant at 50s increase(metric[1m]) + {} 4 + +eval instant at 65s increase(metric[1m]) + {} 4 + +eval instant at 80s increase(metric[1m]) + {} 8 + +eval instant at 95s increase(metric[1m]) + {} 8 + +eval instant at 110s increase(metric[1m]) + {} 8 + +eval instant at 125s increase(metric[1m]) + {} 4 + +eval instant at 5s increase(metric[1m] anchored) + {} 0 + +eval instant at 20s increase(metric[1m] anchored) + {} 1 + +eval instant at 35s increase(metric[1m] anchored) + {} 2 + +eval instant at 50s increase(metric[1m] anchored) + {} 3 + +eval instant at 65s increase(metric[1m] anchored) + {} 4 + +eval instant at 80s increase(metric[1m] anchored) + {} 7 + +eval instant at 95s increase(metric[1m] anchored) + {} 7 + +eval instant at 110s increase(metric[1m] anchored) + {} 7 + +eval instant at 125s increase(metric[1m] anchored) + {} 7 + +eval instant at 5s increase(metric[1m] smoothed) + {} 0.333333333 + +eval instant at 20s increase(metric[1m] smoothed) + {} 1.333333333 + +eval instant at 35s increase(metric[1m] smoothed) + {} 2.333333333 + +eval instant at 50s increase(metric[1m] smoothed) + {} 3.333333333 + +eval instant at 65s increase(metric[1m] smoothed) + {} 5 + +eval instant at 80s increase(metric[1m] smoothed) + {} 7 + +eval instant at 95s increase(metric[1m] smoothed) + {} 7 + +eval instant at 110s increase(metric[1m] smoothed) + {} 7 + +eval instant at 125s increase(metric[1m] smoothed) + {} 6 + +# Reference from PROM-52: Partial dataset + +clear +load 15s + metric 1+1x2 _ _ 9+1x4 + +eval instant at 5s increase(metric[1m]) + +eval instant at 20s increase(metric[1m]) + {} 1.833333333 + +eval instant at 35s increase(metric[1m]) + {} 2.833333333 + +eval instant at 50s increase(metric[1m]) + {} 3.166666666 + +eval instant at 65s increase(metric[1m]) + {} 2.166666666 + +eval instant at 80s increase(metric[1m]) + {} 8 + +eval instant at 95s increase(metric[1m]) + {} 1.833333333 + +eval instant at 110s increase(metric[1m]) + {} 2.833333333 + +eval instant at 125s increase(metric[1m]) + {} 4 + +eval instant at 5s increase(metric[1m] anchored) + {} 0 + +eval instant at 20s increase(metric[1m] anchored) + {} 1 + +eval instant at 35s increase(metric[1m] anchored) + {} 2 + +eval instant at 50s increase(metric[1m] anchored) + {} 2 + +eval instant at 65s increase(metric[1m] anchored) + {} 2 + +eval instant at 80s increase(metric[1m] anchored) + {} 7 + +eval instant at 95s increase(metric[1m] anchored) + {} 7 + +eval instant at 110s increase(metric[1m] anchored) + {} 8 + +eval instant at 125s increase(metric[1m] anchored) + {} 9 + +eval instant at 5s increase(metric[1m] smoothed) + {} 0.333333333 + +eval instant at 20s increase(metric[1m] smoothed) + {} 1.333333333 + +eval instant at 35s increase(metric[1m] smoothed) + {} 2.666666666 + +eval instant at 50s increase(metric[1m] smoothed) + {} 4.666666666 + +eval instant at 65s increase(metric[1m] smoothed) + {} 6.333333333 + +eval instant at 80s increase(metric[1m] smoothed) + {} 7 + +eval instant at 95s increase(metric[1m] smoothed) + {} 6.666666666 + +eval instant at 110s increase(metric[1m] smoothed) + {} 5.666666666 + +eval instant at 125s increase(metric[1m] smoothed) + {} 4.666666666 + +# Test that inverval is left-open. + +clear +load 1m + metric 1 2 _ 4 5 + +eval instant at 2m increase(metric[1m] smoothed) + {} 1 + +eval instant at 2m increase(metric[1m] anchored) + +# Basic test with counter resets + +clear +load 1m + metric{id="1"} 1+1x4 1+1x4 + metric{id="2"} 3 2+2x9 + metric{id="3"} 5+3x2 3+3x6 + +eval instant at 1m30s increase(metric[1m]) + +eval instant at 1m30s increase(metric[1m] smoothed) + {id="1"} 1 + {id="2"} 2 + {id="3"} 3 + +eval instant at 1m30s increase(metric[1m] anchored) + {id="1"} 1 + {id="2"} 2 + {id="3"} 3 + +eval instant at 6m increase(metric[5m]) + {id="1"} 5 + {id="2"} 10 + {id="3"} 15 + +eval instant at 6m increase(metric[5m] smoothed) + {id="1"} 5 + {id="2"} 10 + {id="3"} 15 + +eval instant at 5m increase(metric[5m] anchored) + {id="1"} 5 + {id="2"} 10 + {id="3"} 15 \ No newline at end of file diff --git a/util/annotations/annotations.go b/util/annotations/annotations.go index f8070ff343..c9e75e9551 100644 --- a/util/annotations/annotations.go +++ b/util/annotations/annotations.go @@ -145,6 +145,8 @@ var ( MixedExponentialCustomHistogramsWarning = fmt.Errorf("%w: vector contains a mix of histograms with exponential and custom buckets schemas for metric name", PromQLWarning) IncompatibleCustomBucketsHistogramsWarning = fmt.Errorf("%w: vector contains histograms with incompatible custom buckets for metric name", PromQLWarning) IncompatibleBucketLayoutInBinOpWarning = fmt.Errorf("%w: incompatible bucket layout encountered for binary operator", PromQLWarning) + AnchoredWithUnsupportedFunctionWarning = fmt.Errorf("%w: anchored vector with unsupported function:", PromQLWarning) + SmoothedWithUnsupportedFunctionWarning = fmt.Errorf("%w: smoothed vector with unsupported function:", PromQLWarning) PossibleNonCounterInfo = fmt.Errorf("%w: metric might not be a counter, name does not end in _total/_sum/_count/_bucket:", PromQLInfo) PossibleNonCounterLabelInfo = fmt.Errorf("%w: metric might not be a counter, __type__ label is not set to %q or %q", PromQLInfo, model.MetricTypeCounter, model.MetricTypeHistogram) @@ -348,3 +350,17 @@ func NewNativeHistogramFractionNaNsInfo(metricName string, pos posrange.Position Err: fmt.Errorf("%w %q", NativeHistogramFractionNaNsInfo, metricName), } } + +func NewAnchoredWithUnsupportedFunctionWarning(function string, pos posrange.PositionRange) error { + return annoErr{ + PositionRange: pos, + Err: fmt.Errorf("%w %s", AnchoredWithUnsupportedFunctionWarning, function), + } +} + +func NewSmoothedWithUnsupportedFunctionWarning(function string, pos posrange.PositionRange) error { + return annoErr{ + PositionRange: pos, + Err: fmt.Errorf("%w %s", SmoothedWithUnsupportedFunctionWarning, function), + } +}