Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in>
This commit is contained in:
Ganesh Vernekar 2018-12-22 19:17:13 +05:30 committed by Brian Brazil
parent 2e725a195a
commit dbe55c1352
14 changed files with 1070 additions and 57 deletions

View file

@ -233,6 +233,7 @@ func main() {
} }
promql.LookbackDelta = time.Duration(cfg.lookbackDelta) promql.LookbackDelta = time.Duration(cfg.lookbackDelta)
promql.SetDefaultEvaluationInterval(time.Duration(config.DefaultGlobalConfig.EvaluationInterval))
logger := promlog.New(&cfg.promlogConfig) logger := promlog.New(&cfg.promlogConfig)
@ -654,6 +655,7 @@ func reloadConfig(filename string, logger log.Logger, rls ...func(*config.Config
if failed { if failed {
return fmt.Errorf("one or more errors occurred while applying the new configuration (--config.file=%q)", filename) return fmt.Errorf("one or more errors occurred while applying the new configuration (--config.file=%q)", filename)
} }
promql.SetDefaultEvaluationInterval(time.Duration(conf.GlobalConfig.EvaluationInterval))
level.Info(logger).Log("msg", "Completed loading of configuration file", "filename", filename) level.Info(logger).Log("msg", "Completed loading of configuration file", "filename", filename)
return nil return nil
} }

View file

@ -170,6 +170,14 @@ The same works for range vectors. This returns the 5-minutes rate that
rate(http_requests_total[5m] offset 1w) rate(http_requests_total[5m] offset 1w)
## 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: `<instant_query> '[' <range> ':' [<resolution>] ']' [ offset <duration> ]`
* `<resolution>` is optional. Default is the global evaluation interval.
## Operators ## Operators
Prometheus supports many binary and aggregation operators. These are described Prometheus supports many binary and aggregation operators. These are described

View file

@ -38,6 +38,16 @@ To select all HTTP status codes except 4xx ones, you could run:
http_requests_total{status!~"4.."} http_requests_total{status!~"4.."}
## Subquery
This query returns 5-minute rate of `http_requests_total` metric for the past 30 minutes, at a resolution of 1 minute.
rate(http_requests_total[5m])[30m:1m]
This is an example of nested subquery. The subquery for the `deriv` function uses default resolution. Note that using subqueries unnecessarily is unwise.
max_over_time(deriv(rate(distance_covered_total[5s])[30s:5s])[10m:])
## Using functions, operators, etc. ## Using functions, operators, etc.
Return the per-second rate for all time series with the `http_requests_total` Return the per-second rate for all time series with the `http_requests_total`

View file

@ -116,6 +116,14 @@ type MatrixSelector struct {
series []storage.Series series []storage.Series
} }
// SubqueryExpr represents a subquery.
type SubqueryExpr struct {
Expr Expr
Range time.Duration
Offset time.Duration
Step time.Duration
}
// NumberLiteral represents a number. // NumberLiteral represents a number.
type NumberLiteral struct { type NumberLiteral struct {
Val float64 Val float64
@ -153,6 +161,7 @@ type VectorSelector struct {
func (e *AggregateExpr) Type() ValueType { return ValueTypeVector } func (e *AggregateExpr) Type() ValueType { return ValueTypeVector }
func (e *Call) Type() ValueType { return e.Func.ReturnType } func (e *Call) Type() ValueType { return e.Func.ReturnType }
func (e *MatrixSelector) Type() ValueType { return ValueTypeMatrix } func (e *MatrixSelector) Type() ValueType { return ValueTypeMatrix }
func (e *SubqueryExpr) Type() ValueType { return ValueTypeMatrix }
func (e *NumberLiteral) Type() ValueType { return ValueTypeScalar } func (e *NumberLiteral) Type() ValueType { return ValueTypeScalar }
func (e *ParenExpr) Type() ValueType { return e.Expr.Type() } func (e *ParenExpr) Type() ValueType { return e.Expr.Type() }
func (e *StringLiteral) Type() ValueType { return ValueTypeString } func (e *StringLiteral) Type() ValueType { return ValueTypeString }
@ -169,6 +178,7 @@ func (*AggregateExpr) expr() {}
func (*BinaryExpr) expr() {} func (*BinaryExpr) expr() {}
func (*Call) expr() {} func (*Call) expr() {}
func (*MatrixSelector) expr() {} func (*MatrixSelector) expr() {}
func (*SubqueryExpr) expr() {}
func (*NumberLiteral) expr() {} func (*NumberLiteral) expr() {}
func (*ParenExpr) expr() {} func (*ParenExpr) expr() {}
func (*StringLiteral) expr() {} func (*StringLiteral) expr() {}
@ -267,6 +277,11 @@ func Walk(v Visitor, node Node, path []Node) error {
return err return err
} }
case *SubqueryExpr:
if err := Walk(v, n.Expr, path); err != nil {
return err
}
case *ParenExpr: case *ParenExpr:
if err := Walk(v, n.Expr, path); err != nil { if err := Walk(v, n.Expr, path); err != nil {
return err return err

View file

@ -23,6 +23,7 @@ import (
"sort" "sort"
"strconv" "strconv"
"sync" "sync"
"sync/atomic"
"time" "time"
"github.com/go-kit/kit/log" "github.com/go-kit/kit/log"
@ -51,6 +52,26 @@ const (
minInt64 = -9223372036854775808 minInt64 = -9223372036854775808
) )
var (
// LookbackDelta determines the time since the last sample after which a time
// series is considered stale.
LookbackDelta = 5 * time.Minute
// DefaultEvaluationInterval is the default evaluation interval of
// a subquery in milliseconds.
DefaultEvaluationInterval int64
)
// SetDefaultEvaluationInterval sets DefaultEvaluationInterval.
func SetDefaultEvaluationInterval(ev time.Duration) {
atomic.StoreInt64(&DefaultEvaluationInterval, durationToInt64Millis(ev))
}
// GetDefaultEvaluationInterval returns the DefaultEvaluationInterval as time.Duration.
func GetDefaultEvaluationInterval() int64 {
return atomic.LoadInt64(&DefaultEvaluationInterval)
}
type engineMetrics struct { type engineMetrics struct {
currentQueries prometheus.Gauge currentQueries prometheus.Gauge
maxConcurrentQueries prometheus.Gauge maxConcurrentQueries prometheus.Gauge
@ -409,6 +430,7 @@ func (ng *Engine) execEvalStmt(ctx context.Context, query *query, s *EvalStmt) (
interval: 1, interval: 1,
ctx: ctx, ctx: ctx,
maxSamples: ng.maxSamplesPerQuery, maxSamples: ng.maxSamplesPerQuery,
defaultEvalInterval: GetDefaultEvaluationInterval(),
logger: ng.logger, logger: ng.logger,
} }
val, err := evaluator.Eval(s.Expr) val, err := evaluator.Eval(s.Expr)
@ -450,6 +472,7 @@ func (ng *Engine) execEvalStmt(ctx context.Context, query *query, s *EvalStmt) (
interval: durationMilliseconds(s.Interval), interval: durationMilliseconds(s.Interval),
ctx: ctx, ctx: ctx,
maxSamples: ng.maxSamplesPerQuery, maxSamples: ng.maxSamplesPerQuery,
defaultEvalInterval: GetDefaultEvaluationInterval(),
logger: ng.logger, logger: ng.logger,
} }
val, err := evaluator.Eval(s.Expr) val, err := evaluator.Eval(s.Expr)
@ -477,23 +500,36 @@ func (ng *Engine) execEvalStmt(ctx context.Context, query *query, s *EvalStmt) (
return mat, nil, warnings return mat, nil, warnings
} }
// cumulativeSubqueryOffset returns the sum of range and offset of all subqueries in the path.
func (ng *Engine) cumulativeSubqueryOffset(path []Node) time.Duration {
var subqOffset time.Duration
for _, node := range path {
switch n := node.(type) {
case *SubqueryExpr:
subqOffset += n.Range + n.Offset
}
}
return subqOffset
}
func (ng *Engine) populateSeries(ctx context.Context, q storage.Queryable, s *EvalStmt) (storage.Querier, error, storage.Warnings) { func (ng *Engine) populateSeries(ctx context.Context, q storage.Queryable, s *EvalStmt) (storage.Querier, error, storage.Warnings) {
var maxOffset time.Duration var maxOffset time.Duration
Inspect(s.Expr, func(node Node, _ []Node) error { Inspect(s.Expr, func(node Node, path []Node) error {
subqOffset := ng.cumulativeSubqueryOffset(path)
switch n := node.(type) { switch n := node.(type) {
case *VectorSelector: case *VectorSelector:
if maxOffset < LookbackDelta { if maxOffset < LookbackDelta+subqOffset {
maxOffset = LookbackDelta maxOffset = LookbackDelta + subqOffset
} }
if n.Offset+LookbackDelta > maxOffset { if n.Offset+LookbackDelta+subqOffset > maxOffset {
maxOffset = n.Offset + LookbackDelta maxOffset = n.Offset + LookbackDelta + subqOffset
} }
case *MatrixSelector: case *MatrixSelector:
if maxOffset < n.Range { if maxOffset < n.Range+subqOffset {
maxOffset = n.Range maxOffset = n.Range + subqOffset
} }
if n.Offset+n.Range > maxOffset { if n.Offset+n.Range+subqOffset > maxOffset {
maxOffset = n.Offset + n.Range maxOffset = n.Offset + n.Range + subqOffset
} }
} }
return nil return nil
@ -514,7 +550,7 @@ func (ng *Engine) populateSeries(ctx context.Context, q storage.Queryable, s *Ev
params := &storage.SelectParams{ params := &storage.SelectParams{
Start: timestamp.FromTime(s.Start), Start: timestamp.FromTime(s.Start),
End: timestamp.FromTime(s.End), End: timestamp.FromTime(s.End),
Step: int64(s.Interval / time.Millisecond), Step: durationToInt64Millis(s.Interval),
} }
switch n := node.(type) { switch n := node.(type) {
@ -626,6 +662,7 @@ type evaluator struct {
maxSamples int maxSamples int
currentSamples int currentSamples int
defaultEvalInterval int64
logger log.Logger logger log.Logger
} }
@ -839,6 +876,21 @@ func (ev *evaluator) rangeEval(f func([]Value, *EvalNodeHelper) Vector, exprs ..
return mat return mat
} }
// 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 *SubqueryExpr) *MatrixSelector {
val := ev.eval(subq).(Matrix)
ms := &MatrixSelector{
Range: subq.Range,
Offset: subq.Offset,
series: make([]storage.Series, 0, len(val)),
}
for _, s := range val {
ms.series = append(ms.series, NewStorageSeries(s))
}
return ms
}
// eval evaluates the given expression as the given AST expression node requires. // eval evaluates the given expression as the given AST expression node requires.
func (ev *evaluator) eval(expr Expr) Value { func (ev *evaluator) eval(expr Expr) Value {
// This is the top-level evaluation method. // This is the top-level evaluation method.
@ -880,12 +932,19 @@ func (ev *evaluator) eval(expr Expr) Value {
var matrixArgIndex int var matrixArgIndex int
var matrixArg bool var matrixArg bool
for i, a := range e.Args { for i, a := range e.Args {
_, ok := a.(*MatrixSelector) if _, ok := a.(*MatrixSelector); ok {
if ok {
matrixArgIndex = i matrixArgIndex = i
matrixArg = true matrixArg = true
break break
} }
// SubqueryExpr can be used in place of MatrixSelector.
if subq, ok := a.(*SubqueryExpr); ok {
matrixArgIndex = i
matrixArg = true
// Replacing SubqueryExpr with MatrixSelector.
e.Args[i] = ev.evalSubquery(subq)
break
}
} }
if !matrixArg { if !matrixArg {
// Does not have a matrix argument. // Does not have a matrix argument.
@ -1077,11 +1136,43 @@ func (ev *evaluator) eval(expr Expr) Value {
panic(fmt.Errorf("cannot do range evaluation of matrix selector")) panic(fmt.Errorf("cannot do range evaluation of matrix selector"))
} }
return ev.matrixSelector(e) return ev.matrixSelector(e)
case *SubqueryExpr:
offsetMillis := durationToInt64Millis(e.Offset)
rangeMillis := durationToInt64Millis(e.Range)
newEv := &evaluator{
endTimestamp: ev.endTimestamp - offsetMillis,
interval: ev.defaultEvalInterval,
ctx: ev.ctx,
currentSamples: ev.currentSamples,
maxSamples: ev.maxSamples,
defaultEvalInterval: ev.defaultEvalInterval,
logger: ev.logger,
}
if e.Step != 0 {
newEv.interval = durationToInt64Millis(e.Step)
}
// Start with the first timestamp after (ev.startTimestamp - offset - range)
// that is aligned with the step (multiple of 'newEv.interval').
newEv.startTimestamp = newEv.interval * ((ev.startTimestamp - offsetMillis - rangeMillis) / newEv.interval)
if newEv.startTimestamp < (ev.startTimestamp - offsetMillis - rangeMillis) {
newEv.startTimestamp += newEv.interval
}
res := newEv.eval(e.Expr)
ev.currentSamples = newEv.currentSamples
return res
} }
panic(fmt.Errorf("unhandled expression of type: %T", expr)) panic(fmt.Errorf("unhandled expression of type: %T", expr))
} }
func durationToInt64Millis(d time.Duration) int64 {
return int64(d / time.Millisecond)
}
// vectorSelector evaluates a *VectorSelector expression. // vectorSelector evaluates a *VectorSelector expression.
func (ev *evaluator) vectorSelector(node *VectorSelector, ts int64) Vector { func (ev *evaluator) vectorSelector(node *VectorSelector, ts int64) Vector {
if err := checkForSeriesSetExpansion(node, ev.ctx); err != nil { if err := checkForSeriesSetExpansion(node, ev.ctx); err != nil {
@ -1825,10 +1916,6 @@ func shouldDropMetricName(op ItemType) bool {
} }
} }
// LookbackDelta determines the time since the last sample after which a time
// series is considered stale.
var LookbackDelta = 5 * time.Minute
// documentedType returns the internal type to the equivalent // documentedType returns the internal type to the equivalent
// user facing terminology as defined in the documentation. // user facing terminology as defined in the documentation.
func documentedType(t ValueType) string { func documentedType(t ValueType) string {

View file

@ -475,6 +475,34 @@ load 10s
}, },
Start: time.Unix(10, 0), Start: time.Unix(10, 0),
}, },
{
Query: "rate(metric[20s])",
MaxSamples: 3,
Result: Result{
nil,
Vector{
Sample{
Point: Point{V: 0.1, T: 10000},
Metric: labels.Labels{},
},
},
nil,
},
Start: time.Unix(10, 0),
},
{
Query: "metric[20s:5s]",
MaxSamples: 3,
Result: Result{
nil,
Matrix{Series{
Points: []Point{{V: 1, T: 0}, {V: 1, T: 5000}, {V: 2, T: 10000}},
Metric: labels.FromStrings("__name__", "metric")},
},
nil,
},
Start: time.Unix(10, 0),
},
{ {
Query: "metric[20s]", Query: "metric[20s]",
MaxSamples: 0, MaxSamples: 0,
@ -624,3 +652,265 @@ func TestRecoverEvaluatorError(t *testing.T) {
panic(e) panic(e)
} }
func TestSubquerySelector(t *testing.T) {
tests := []struct {
loadString string
cases []struct {
Query string
Result Result
Start time.Time
}
}{
{
loadString: `load 10s
metric 1 2`,
cases: []struct {
Query string
Result Result
Start time.Time
}{
{
Query: "metric[20s:10s]",
Result: Result{
nil,
Matrix{Series{
Points: []Point{{V: 1, T: 0}, {V: 2, T: 10000}},
Metric: labels.FromStrings("__name__", "metric")},
},
nil,
},
Start: time.Unix(10, 0),
},
{
Query: "metric[20s:5s]",
Result: Result{
nil,
Matrix{Series{
Points: []Point{{V: 1, T: 0}, {V: 1, T: 5000}, {V: 2, T: 10000}},
Metric: labels.FromStrings("__name__", "metric")},
},
nil,
},
Start: time.Unix(10, 0),
},
{
Query: "metric[20s:5s] offset 2s",
Result: Result{
nil,
Matrix{Series{
Points: []Point{{V: 1, T: 0}, {V: 1, T: 5000}, {V: 2, T: 10000}},
Metric: labels.FromStrings("__name__", "metric")},
},
nil,
},
Start: time.Unix(12, 0),
},
{
Query: "metric[20s:5s] offset 6s",
Result: Result{
nil,
Matrix{Series{
Points: []Point{{V: 1, T: 0}, {V: 1, T: 5000}, {V: 2, T: 10000}},
Metric: labels.FromStrings("__name__", "metric")},
},
nil,
},
Start: time.Unix(20, 0),
},
{
Query: "metric[20s:5s] offset 4s",
Result: Result{
nil,
Matrix{Series{
Points: []Point{{V: 2, T: 15000}, {V: 2, T: 20000}, {V: 2, T: 25000}, {V: 2, T: 30000}},
Metric: labels.FromStrings("__name__", "metric")},
},
nil,
},
Start: time.Unix(35, 0),
},
{
Query: "metric[20s:5s] offset 5s",
Result: Result{
nil,
Matrix{Series{
Points: []Point{{V: 2, T: 10000}, {V: 2, T: 15000}, {V: 2, T: 20000}, {V: 2, T: 25000}, {V: 2, T: 30000}},
Metric: labels.FromStrings("__name__", "metric")},
},
nil,
},
Start: time.Unix(35, 0),
},
{
Query: "metric[20s:5s] offset 6s",
Result: Result{
nil,
Matrix{Series{
Points: []Point{{V: 2, T: 10000}, {V: 2, T: 15000}, {V: 2, T: 20000}, {V: 2, T: 25000}},
Metric: labels.FromStrings("__name__", "metric")},
},
nil,
},
Start: time.Unix(35, 0),
},
{
Query: "metric[20s:5s] offset 7s",
Result: Result{
nil,
Matrix{Series{
Points: []Point{{V: 2, T: 10000}, {V: 2, T: 15000}, {V: 2, T: 20000}, {V: 2, T: 25000}},
Metric: labels.FromStrings("__name__", "metric")},
},
nil,
},
Start: time.Unix(35, 0),
},
},
},
{
loadString: `load 10s
http_requests{job="api-server", instance="0", group="production"} 0+10x1000 100+30x1000
http_requests{job="api-server", instance="1", group="production"} 0+20x1000 200+30x1000
http_requests{job="api-server", instance="0", group="canary"} 0+30x1000 300+80x1000
http_requests{job="api-server", instance="1", group="canary"} 0+40x2000`,
cases: []struct {
Query string
Result Result
Start time.Time
}{
{ // Normal selector.
Query: `http_requests{group=~"pro.*",instance="0"}[30s:10s]`,
Result: Result{
nil,
Matrix{Series{
Points: []Point{{V: 9990, T: 9990000}, {V: 10000, T: 10000000}, {V: 100, T: 10010000}, {V: 130, T: 10020000}},
Metric: labels.FromStrings("__name__", "http_requests", "job", "api-server", "instance", "0", "group", "production")},
},
nil,
},
Start: time.Unix(10020, 0),
},
{ // Default step.
Query: `http_requests{group=~"pro.*",instance="0"}[5m:]`,
Result: Result{
nil,
Matrix{Series{
Points: []Point{{V: 9840, T: 9840000}, {V: 9900, T: 9900000}, {V: 9960, T: 9960000}, {V: 130, T: 10020000}, {V: 310, T: 10080000}},
Metric: labels.FromStrings("__name__", "http_requests", "job", "api-server", "instance", "0", "group", "production")},
},
nil,
},
Start: time.Unix(10100, 0),
},
{ // Checking if high offset (>LookbackDelta) is being taken care of.
Query: `http_requests{group=~"pro.*",instance="0"}[5m:] offset 20m`,
Result: Result{
nil,
Matrix{Series{
Points: []Point{{V: 8640, T: 8640000}, {V: 8700, T: 8700000}, {V: 8760, T: 8760000}, {V: 8820, T: 8820000}, {V: 8880, T: 8880000}},
Metric: labels.FromStrings("__name__", "http_requests", "job", "api-server", "instance", "0", "group", "production")},
},
nil,
},
Start: time.Unix(10100, 0),
},
{
Query: `rate(http_requests[1m])[15s:5s]`,
Result: Result{
nil,
Matrix{
Series{
Points: []Point{{V: 3, T: 7985000}, {V: 3, T: 7990000}, {V: 3, T: 7995000}, {V: 3, T: 8000000}},
Metric: labels.FromStrings("job", "api-server", "instance", "0", "group", "canary"),
},
Series{
Points: []Point{{V: 4, T: 7985000}, {V: 4, T: 7990000}, {V: 4, T: 7995000}, {V: 4, T: 8000000}},
Metric: labels.FromStrings("job", "api-server", "instance", "1", "group", "canary"),
},
Series{
Points: []Point{{V: 1, T: 7985000}, {V: 1, T: 7990000}, {V: 1, T: 7995000}, {V: 1, T: 8000000}},
Metric: labels.FromStrings("job", "api-server", "instance", "0", "group", "production"),
},
Series{
Points: []Point{{V: 2, T: 7985000}, {V: 2, T: 7990000}, {V: 2, T: 7995000}, {V: 2, T: 8000000}},
Metric: labels.FromStrings("job", "api-server", "instance", "1", "group", "production"),
},
},
nil,
},
Start: time.Unix(8000, 0),
},
{
Query: `sum(http_requests{group=~"pro.*"})[30s:10s]`,
Result: Result{
nil,
Matrix{Series{
Points: []Point{{V: 270, T: 90000}, {V: 300, T: 100000}, {V: 330, T: 110000}, {V: 360, T: 120000}},
Metric: labels.Labels{}},
},
nil,
},
Start: time.Unix(120, 0),
},
{
Query: `sum(http_requests)[40s:10s]`,
Result: Result{
nil,
Matrix{Series{
Points: []Point{{V: 800, T: 80000}, {V: 900, T: 90000}, {V: 1000, T: 100000}, {V: 1100, T: 110000}, {V: 1200, T: 120000}},
Metric: labels.Labels{}},
},
nil,
},
Start: time.Unix(120, 0),
},
{
Query: `(sum(http_requests{group=~"p.*"})+sum(http_requests{group=~"c.*"}))[20s:5s]`,
Result: Result{
nil,
Matrix{Series{
Points: []Point{{V: 1000, T: 100000}, {V: 1000, T: 105000}, {V: 1100, T: 110000}, {V: 1100, T: 115000}, {V: 1200, T: 120000}},
Metric: labels.Labels{}},
},
nil,
},
Start: time.Unix(120, 0),
},
},
},
}
SetDefaultEvaluationInterval(1 * time.Minute)
for _, tst := range tests {
test, err := NewTest(t, tst.loadString)
if err != nil {
t.Fatalf("unexpected error creating test: %q", err)
}
defer test.Close()
err = test.Run()
if err != nil {
t.Fatalf("unexpected error initializing test: %q", err)
}
engine := test.QueryEngine()
for _, c := range tst.cases {
var err error
var qry Query
qry, err = engine.NewInstantQuery(test.Queryable(), c.Query, c.Start)
if err != nil {
t.Fatalf("unexpected error creating query: %q", err)
}
res := qry.Exec(test.Context())
if res.Err != nil && res.Err != c.Result.Err {
t.Fatalf("unexpected error running query: %q, expected to get result: %q", res.Err, c.Result.Value)
}
if !reflect.DeepEqual(res.Value, c.Result.Value) {
t.Fatalf("unexpected result for query %q: got %q wanted %q", c.Query, res.Value.String(), c.Result.String())
}
}
}
}

View file

@ -137,6 +137,7 @@ const (
itemRightBracket itemRightBracket
itemComma itemComma
itemAssign itemAssign
itemColon
itemSemicolon itemSemicolon
itemString itemString
itemNumber itemNumber
@ -235,6 +236,7 @@ var itemTypeStr = map[ItemType]string{
itemRightBracket: "]", itemRightBracket: "]",
itemComma: ",", itemComma: ",",
itemAssign: "=", itemAssign: "=",
itemColon: ":",
itemSemicolon: ";", itemSemicolon: ";",
itemBlank: "_", itemBlank: "_",
itemTimes: "x", itemTimes: "x",
@ -326,6 +328,7 @@ type lexer struct {
parenDepth int // Nesting depth of ( ) exprs. parenDepth int // Nesting depth of ( ) exprs.
braceOpen bool // Whether a { is opened. braceOpen bool // Whether a { is opened.
bracketOpen bool // Whether a [ is opened. bracketOpen bool // Whether a [ is opened.
gotColon bool // Whether we got a ':' after [ was opened.
stringOpen rune // Quote rune of the string currently being read. stringOpen rune // Quote rune of the string currently being read.
// seriesDesc is set when a series description for the testing // seriesDesc is set when a series description for the testing
@ -517,8 +520,15 @@ func lexStatements(l *lexer) stateFn {
l.stringOpen = r l.stringOpen = r
return lexRawString return lexRawString
case isAlpha(r) || r == ':': case isAlpha(r) || r == ':':
if !l.bracketOpen {
l.backup() l.backup()
return lexKeywordOrIdentifier return lexKeywordOrIdentifier
}
if l.gotColon {
return l.errorf("unexpected colon %q", r)
}
l.emit(itemColon)
l.gotColon = true
case r == '(': case r == '(':
l.emit(itemLeftParen) l.emit(itemLeftParen)
l.parenDepth++ l.parenDepth++
@ -538,6 +548,7 @@ func lexStatements(l *lexer) stateFn {
if l.bracketOpen { if l.bracketOpen {
return l.errorf("unexpected left bracket %q", r) return l.errorf("unexpected left bracket %q", r)
} }
l.gotColon = false
l.emit(itemLeftBracket) l.emit(itemLeftBracket)
l.bracketOpen = true l.bracketOpen = true
return lexDuration return lexDuration

View file

@ -429,6 +429,167 @@ var tests = []struct {
}, },
seriesDesc: true, seriesDesc: true,
}, },
// Test subquery.
{
input: `test_name{on!~"bar"}[4m:4s]`,
expected: []item{
{itemIdentifier, 0, `test_name`},
{itemLeftBrace, 9, `{`},
{itemIdentifier, 10, `on`},
{itemNEQRegex, 12, `!~`},
{itemString, 14, `"bar"`},
{itemRightBrace, 19, `}`},
{itemLeftBracket, 20, `[`},
{itemDuration, 21, `4m`},
{itemColon, 23, `:`},
{itemDuration, 24, `4s`},
{itemRightBracket, 26, `]`},
},
},
{
input: `test:name{on!~"bar"}[4m:4s]`,
expected: []item{
{itemMetricIdentifier, 0, `test:name`},
{itemLeftBrace, 9, `{`},
{itemIdentifier, 10, `on`},
{itemNEQRegex, 12, `!~`},
{itemString, 14, `"bar"`},
{itemRightBrace, 19, `}`},
{itemLeftBracket, 20, `[`},
{itemDuration, 21, `4m`},
{itemColon, 23, `:`},
{itemDuration, 24, `4s`},
{itemRightBracket, 26, `]`},
},
}, {
input: `test:name{on!~"b:ar"}[4m:4s]`,
expected: []item{
{itemMetricIdentifier, 0, `test:name`},
{itemLeftBrace, 9, `{`},
{itemIdentifier, 10, `on`},
{itemNEQRegex, 12, `!~`},
{itemString, 14, `"b:ar"`},
{itemRightBrace, 20, `}`},
{itemLeftBracket, 21, `[`},
{itemDuration, 22, `4m`},
{itemColon, 24, `:`},
{itemDuration, 25, `4s`},
{itemRightBracket, 27, `]`},
},
}, {
input: `test:name{on!~"b:ar"}[4m:]`,
expected: []item{
{itemMetricIdentifier, 0, `test:name`},
{itemLeftBrace, 9, `{`},
{itemIdentifier, 10, `on`},
{itemNEQRegex, 12, `!~`},
{itemString, 14, `"b:ar"`},
{itemRightBrace, 20, `}`},
{itemLeftBracket, 21, `[`},
{itemDuration, 22, `4m`},
{itemColon, 24, `:`},
{itemRightBracket, 25, `]`},
},
}, { // Nested Subquery.
input: `min_over_time(rate(foo{bar="baz"}[2s])[5m:])[4m:3s]`,
expected: []item{
{itemIdentifier, 0, `min_over_time`},
{itemLeftParen, 13, `(`},
{itemIdentifier, 14, `rate`},
{itemLeftParen, 18, `(`},
{itemIdentifier, 19, `foo`},
{itemLeftBrace, 22, `{`},
{itemIdentifier, 23, `bar`},
{itemEQL, 26, `=`},
{itemString, 27, `"baz"`},
{itemRightBrace, 32, `}`},
{itemLeftBracket, 33, `[`},
{itemDuration, 34, `2s`},
{itemRightBracket, 36, `]`},
{itemRightParen, 37, `)`},
{itemLeftBracket, 38, `[`},
{itemDuration, 39, `5m`},
{itemColon, 41, `:`},
{itemRightBracket, 42, `]`},
{itemRightParen, 43, `)`},
{itemLeftBracket, 44, `[`},
{itemDuration, 45, `4m`},
{itemColon, 47, `:`},
{itemDuration, 48, `3s`},
{itemRightBracket, 50, `]`},
},
},
// Subquery with offset.
{
input: `test:name{on!~"b:ar"}[4m:4s] offset 10m`,
expected: []item{
{itemMetricIdentifier, 0, `test:name`},
{itemLeftBrace, 9, `{`},
{itemIdentifier, 10, `on`},
{itemNEQRegex, 12, `!~`},
{itemString, 14, `"b:ar"`},
{itemRightBrace, 20, `}`},
{itemLeftBracket, 21, `[`},
{itemDuration, 22, `4m`},
{itemColon, 24, `:`},
{itemDuration, 25, `4s`},
{itemRightBracket, 27, `]`},
{itemOffset, 29, "offset"},
{itemDuration, 36, "10m"},
},
}, {
input: `min_over_time(rate(foo{bar="baz"}[2s])[5m:] offset 6m)[4m:3s]`,
expected: []item{
{itemIdentifier, 0, `min_over_time`},
{itemLeftParen, 13, `(`},
{itemIdentifier, 14, `rate`},
{itemLeftParen, 18, `(`},
{itemIdentifier, 19, `foo`},
{itemLeftBrace, 22, `{`},
{itemIdentifier, 23, `bar`},
{itemEQL, 26, `=`},
{itemString, 27, `"baz"`},
{itemRightBrace, 32, `}`},
{itemLeftBracket, 33, `[`},
{itemDuration, 34, `2s`},
{itemRightBracket, 36, `]`},
{itemRightParen, 37, `)`},
{itemLeftBracket, 38, `[`},
{itemDuration, 39, `5m`},
{itemColon, 41, `:`},
{itemRightBracket, 42, `]`},
{itemOffset, 44, `offset`},
{itemDuration, 51, `6m`},
{itemRightParen, 53, `)`},
{itemLeftBracket, 54, `[`},
{itemDuration, 55, `4m`},
{itemColon, 57, `:`},
{itemDuration, 58, `3s`},
{itemRightBracket, 60, `]`},
},
},
{
input: `test:name{o:n!~"bar"}[4m:4s]`,
fail: true,
},
{
input: `test:name{on!~"bar"}[4m:4s:4h]`,
fail: true,
},
{
input: `test:name{on!~"bar"}[4m:4s:]`,
fail: true,
},
{
input: `test:name{on!~"bar"}[4m::]`,
fail: true,
},
{
input: `test:name{on!~"bar"}[:4s]`,
fail: true,
},
} }
// TestLexer tests basic functionality of the lexer. More elaborate tests are implemented // TestLexer tests basic functionality of the lexer. More elaborate tests are implemented

View file

@ -351,6 +351,17 @@ func (p *parser) expr() Expr {
// If the next token is not an operator the expression is done. // If the next token is not an operator the expression is done.
op := p.peek().typ op := p.peek().typ
if !op.isOperator() { if !op.isOperator() {
// Check for subquery.
if op == itemLeftBracket {
expr = p.subqueryOrRangeSelector(expr, false)
if s, ok := expr.(*SubqueryExpr); ok {
// Parse optional offset.
if p.peek().typ == itemOffset {
offset := p.offset()
s.Offset = offset
}
}
}
return expr return expr
} }
p.next() // Consume operator. p.next() // Consume operator.
@ -471,11 +482,7 @@ func (p *parser) unaryExpr() Expr {
// Expression might be followed by a range selector. // Expression might be followed by a range selector.
if p.peek().typ == itemLeftBracket { if p.peek().typ == itemLeftBracket {
vs, ok := e.(*VectorSelector) e = p.subqueryOrRangeSelector(e, true)
if !ok {
p.errorf("range specification must be preceded by a metric selector, but follows a %T instead", e)
}
e = p.rangeSelector(vs)
} }
// Parse optional offset. // Parse optional offset.
@ -487,6 +494,8 @@ func (p *parser) unaryExpr() Expr {
s.Offset = offset s.Offset = offset
case *MatrixSelector: case *MatrixSelector:
s.Offset = offset s.Offset = offset
case *SubqueryExpr:
s.Offset = offset
default: default:
p.errorf("offset modifier must be preceded by an instant or range selector, but follows a %T instead", e) p.errorf("offset modifier must be preceded by an instant or range selector, but follows a %T instead", e)
} }
@ -495,13 +504,17 @@ func (p *parser) unaryExpr() Expr {
return e return e
} }
// rangeSelector parses a Matrix (a.k.a. range) selector based on a given // subqueryOrRangeSelector parses a Subquery based on given Expr (or)
// Vector selector. // a Matrix (a.k.a. range) selector based on a given Vector selector.
// //
// <Vector_selector> '[' <duration> ']' // <Vector_selector> '[' <duration> ']' | <Vector_selector> '[' <duration> ':' [<duration>] ']'
// //
func (p *parser) rangeSelector(vs *VectorSelector) *MatrixSelector { func (p *parser) subqueryOrRangeSelector(expr Expr, checkRange bool) Expr {
const ctx = "range selector" ctx := "subquery selector"
if checkRange {
ctx = "range/subquery selector"
}
p.next() p.next()
var erange time.Duration var erange time.Duration
@ -513,14 +526,43 @@ func (p *parser) rangeSelector(vs *VectorSelector) *MatrixSelector {
p.error(err) p.error(err)
} }
p.expect(itemRightBracket, ctx) var itm item
if checkRange {
e := &MatrixSelector{ itm = p.expectOneOf(itemRightBracket, itemColon, ctx)
if itm.typ == itemRightBracket {
// Range selector.
vs, ok := expr.(*VectorSelector)
if !ok {
p.errorf("range specification must be preceded by a metric selector, but follows a %T instead", expr)
}
return &MatrixSelector{
Name: vs.Name, Name: vs.Name,
LabelMatchers: vs.LabelMatchers, LabelMatchers: vs.LabelMatchers,
Range: erange, Range: erange,
} }
return e }
} else {
itm = p.expect(itemColon, ctx)
}
// Subquery.
var estep time.Duration
itm = p.expectOneOf(itemRightBracket, itemDuration, ctx)
if itm.typ == itemDuration {
estepStr := itm.val
estep, err = parseDuration(estepStr)
if err != nil {
p.error(err)
}
p.expect(itemRightBracket, ctx)
}
return &SubqueryExpr{
Expr: expr,
Range: erange,
Step: estep,
}
} }
// number parses a number. // number parses a number.
@ -1000,6 +1042,12 @@ func (p *parser) checkType(node Node) (typ ValueType) {
p.errorf("unary expression only allowed on expressions of type scalar or instant vector, got %q", documentedType(t)) p.errorf("unary expression only allowed on expressions of type scalar or instant vector, got %q", documentedType(t))
} }
case *SubqueryExpr:
ty := p.checkType(n.Expr)
if ty != ValueTypeVector {
p.errorf("subquery is only allowed on instant vector, got %s in %q instead", ty, n.String())
}
case *NumberLiteral, *MatrixSelector, *StringLiteral, *VectorSelector: case *NumberLiteral, *MatrixSelector, *StringLiteral, *VectorSelector:
// Nothing to do for terminals. // Nothing to do for terminals.

View file

@ -942,10 +942,6 @@ var testExpr = []struct {
input: `foo{__name__="bar"}`, input: `foo{__name__="bar"}`,
fail: true, fail: true,
errMsg: "metric name must not be set twice: \"foo\" or \"bar\"", errMsg: "metric name must not be set twice: \"foo\" or \"bar\"",
// }, {
// input: `:foo`,
// fail: true,
// errMsg: "bla",
}, },
// Test matrix selector. // Test matrix selector.
{ {
@ -1051,11 +1047,11 @@ var testExpr = []struct {
}, { }, {
input: `some_metric OFFSET 1m[5m]`, input: `some_metric OFFSET 1m[5m]`,
fail: true, fail: true,
errMsg: "could not parse remaining input \"[5m]\"...", errMsg: "parse error at char 25: unexpected \"]\" in subquery selector, expected \":\"",
}, { }, {
input: `(foo + bar)[5m]`, input: `(foo + bar)[5m]`,
fail: true, fail: true,
errMsg: "could not parse remaining input \"[5m]\"...", errMsg: "parse error at char 15: unexpected \"]\" in subquery selector, expected \":\"",
}, },
// Test aggregation. // Test aggregation.
{ {
@ -1390,6 +1386,202 @@ var testExpr = []struct {
fail: true, fail: true,
errMsg: "illegal character U+002E '.' in escape sequence", errMsg: "illegal character U+002E '.' in escape sequence",
}, },
// Subquery.
{
input: `foo{bar="baz"}[10m:6s]`,
expected: &SubqueryExpr{
Expr: &VectorSelector{
Name: "foo",
LabelMatchers: []*labels.Matcher{
mustLabelMatcher(labels.MatchEqual, "bar", "baz"),
mustLabelMatcher(labels.MatchEqual, string(model.MetricNameLabel), "foo"),
},
},
Range: 10 * time.Minute,
Step: 6 * time.Second,
},
}, {
input: `foo[10m:]`,
expected: &SubqueryExpr{
Expr: &VectorSelector{
Name: "foo",
LabelMatchers: []*labels.Matcher{
mustLabelMatcher(labels.MatchEqual, string(model.MetricNameLabel), "foo"),
},
},
Range: 10 * time.Minute,
},
}, {
input: `min_over_time(rate(foo{bar="baz"}[2s])[5m:5s])`,
expected: &Call{
Func: mustGetFunction("min_over_time"),
Args: Expressions{
&SubqueryExpr{
Expr: &Call{
Func: mustGetFunction("rate"),
Args: Expressions{
&MatrixSelector{
Name: "foo",
Range: 2 * time.Second,
LabelMatchers: []*labels.Matcher{
mustLabelMatcher(labels.MatchEqual, "bar", "baz"),
mustLabelMatcher(labels.MatchEqual, string(model.MetricNameLabel), "foo"),
},
},
},
},
Range: 5 * time.Minute,
Step: 5 * time.Second,
},
},
},
}, {
input: `min_over_time(rate(foo{bar="baz"}[2s])[5m:])[4m:3s]`,
expected: &SubqueryExpr{
Expr: &Call{
Func: mustGetFunction("min_over_time"),
Args: Expressions{
&SubqueryExpr{
Expr: &Call{
Func: mustGetFunction("rate"),
Args: Expressions{
&MatrixSelector{
Name: "foo",
Range: 2 * time.Second,
LabelMatchers: []*labels.Matcher{
mustLabelMatcher(labels.MatchEqual, "bar", "baz"),
mustLabelMatcher(labels.MatchEqual, string(model.MetricNameLabel), "foo"),
},
},
},
},
Range: 5 * time.Minute,
},
},
},
Range: 4 * time.Minute,
Step: 3 * time.Second,
},
}, {
input: `min_over_time(rate(foo{bar="baz"}[2s])[5m:] offset 4m)[4m:3s]`,
expected: &SubqueryExpr{
Expr: &Call{
Func: mustGetFunction("min_over_time"),
Args: Expressions{
&SubqueryExpr{
Expr: &Call{
Func: mustGetFunction("rate"),
Args: Expressions{
&MatrixSelector{
Name: "foo",
Range: 2 * time.Second,
LabelMatchers: []*labels.Matcher{
mustLabelMatcher(labels.MatchEqual, "bar", "baz"),
mustLabelMatcher(labels.MatchEqual, string(model.MetricNameLabel), "foo"),
},
},
},
},
Range: 5 * time.Minute,
Offset: 4 * time.Minute,
},
},
},
Range: 4 * time.Minute,
Step: 3 * time.Second,
},
}, {
input: "sum without(and, by, avg, count, alert, annotations)(some_metric) [30m:10s]",
expected: &SubqueryExpr{
Expr: &AggregateExpr{
Op: itemSum,
Without: true,
Expr: &VectorSelector{
Name: "some_metric",
LabelMatchers: []*labels.Matcher{
mustLabelMatcher(labels.MatchEqual, string(model.MetricNameLabel), "some_metric"),
},
},
Grouping: []string{"and", "by", "avg", "count", "alert", "annotations"},
},
Range: 30 * time.Minute,
Step: 10 * time.Second,
},
}, {
input: `some_metric OFFSET 1m [10m:5s]`,
expected: &SubqueryExpr{
Expr: &VectorSelector{
Name: "some_metric",
LabelMatchers: []*labels.Matcher{
mustLabelMatcher(labels.MatchEqual, string(model.MetricNameLabel), "some_metric"),
},
Offset: 1 * time.Minute,
},
Range: 10 * time.Minute,
Step: 5 * time.Second,
},
}, {
input: `(foo + bar{nm="val"})[5m:]`,
expected: &SubqueryExpr{
Expr: &ParenExpr{
Expr: &BinaryExpr{
Op: itemADD,
VectorMatching: &VectorMatching{
Card: CardOneToOne,
},
LHS: &VectorSelector{
Name: "foo",
LabelMatchers: []*labels.Matcher{
mustLabelMatcher(labels.MatchEqual, string(model.MetricNameLabel), "foo"),
},
},
RHS: &VectorSelector{
Name: "bar",
LabelMatchers: []*labels.Matcher{
mustLabelMatcher(labels.MatchEqual, "nm", "val"),
mustLabelMatcher(labels.MatchEqual, string(model.MetricNameLabel), "bar"),
},
},
},
},
Range: 5 * time.Minute,
},
}, {
input: `(foo + bar{nm="val"})[5m:] offset 10m`,
expected: &SubqueryExpr{
Expr: &ParenExpr{
Expr: &BinaryExpr{
Op: itemADD,
VectorMatching: &VectorMatching{
Card: CardOneToOne,
},
LHS: &VectorSelector{
Name: "foo",
LabelMatchers: []*labels.Matcher{
mustLabelMatcher(labels.MatchEqual, string(model.MetricNameLabel), "foo"),
},
},
RHS: &VectorSelector{
Name: "bar",
LabelMatchers: []*labels.Matcher{
mustLabelMatcher(labels.MatchEqual, "nm", "val"),
mustLabelMatcher(labels.MatchEqual, string(model.MetricNameLabel), "bar"),
},
},
},
},
Range: 5 * time.Minute,
Offset: 10 * time.Minute,
},
}, {
input: "test[5d] OFFSET 10s [10m:5s]",
fail: true,
errMsg: "parse error at char 29: subquery is only allowed on instant vector, got matrix in \"test[5d] offset 10s[10m:5s]\"",
}, {
input: `(foo + bar{nm="val"})[5m:][10m:5s]`,
fail: true,
errMsg: "parse error at char 27: could not parse remaining input \"[10m:5s]\"...",
},
} }
func TestParseExpressions(t *testing.T) { func TestParseExpressions(t *testing.T) {

View file

@ -62,6 +62,9 @@ func tree(node Node, level string) string {
case *UnaryExpr: case *UnaryExpr:
t += tree(n.Expr, level) t += tree(n.Expr, level)
case *SubqueryExpr:
t += tree(n.Expr, level)
case *MatrixSelector, *NumberLiteral, *StringLiteral, *VectorSelector: case *MatrixSelector, *NumberLiteral, *StringLiteral, *VectorSelector:
// nothing to do // nothing to do
@ -149,6 +152,14 @@ func (node *MatrixSelector) String() string {
return fmt.Sprintf("%s[%s]%s", vecSelector.String(), model.Duration(node.Range), offset) return fmt.Sprintf("%s[%s]%s", vecSelector.String(), model.Duration(node.Range), offset)
} }
func (node *SubqueryExpr) String() string {
step := ""
if node.Step != 0 {
step = fmt.Sprintf("%s", model.Duration(node.Step))
}
return fmt.Sprintf("%s[%s:%s]", node.Expr.String(), model.Duration(node.Range), step)
}
func (node *NumberLiteral) String() string { func (node *NumberLiteral) String() string {
return fmt.Sprint(node.Val) return fmt.Sprint(node.Val)
} }

View file

@ -515,7 +515,7 @@ func (t *Test) clear() {
Logger: nil, Logger: nil,
Reg: nil, Reg: nil,
MaxConcurrent: 20, MaxConcurrent: 20,
MaxSamples: 1000, MaxSamples: 10000,
Timeout: 100 * time.Second, Timeout: 100 * time.Second,
} }

113
promql/testdata/subquery.test vendored Normal file
View file

@ -0,0 +1,113 @@
load 10s
metric 1 2
# Evaluation before 0s gets no sample.
eval instant at 10s sum_over_time(metric[50s:10s])
{} 3
eval instant at 10s sum_over_time(metric[50s:5s])
{} 4
# Every evaluation yields the last value, i.e. 2
eval instant at 5m sum_over_time(metric[50s:10s])
{} 12
# Series becomes stale at 5m10s (5m after last sample)
# Hence subquery gets a single sample at 6m-50s=5m10s.
eval instant at 6m sum_over_time(metric[50s:10s])
{} 2
eval instant at 10s rate(metric[20s:10s])
{} 0.1
eval instant at 20s rate(metric[20s:5s])
{} 0.05
clear
load 10s
http_requests{job="api-server", instance="1", group="production"} 0+20x1000 200+30x1000
http_requests{job="api-server", instance="0", group="production"} 0+10x1000 100+30x1000
http_requests{job="api-server", instance="0", group="canary"} 0+30x1000 300+80x1000
http_requests{job="api-server", instance="1", group="canary"} 0+40x2000
eval instant at 8000s rate(http_requests{group=~"pro.*"}[1m:10s])
{job="api-server", instance="0", group="production"} 1
{job="api-server", instance="1", group="production"} 2
eval instant at 20000s avg_over_time(rate(http_requests[1m])[1m:1s])
{job="api-server", instance="0", group="canary"} 8
{job="api-server", instance="1", group="canary"} 4
{job="api-server", instance="1", group="production"} 3
{job="api-server", instance="0", group="production"} 3
clear
load 10s
metric1 0+1x1000
metric2 0+2x1000
metric3 0+3x1000
eval instant at 1000s sum_over_time(metric1[30s:10s])
{} 394
# This is (394*2 - 100), because other than the last 100 at 1000s,
# everything else is repeated with the 5s step.
eval instant at 1000s sum_over_time(metric1[30s:5s])
{} 688
# Offset is aligned with the step.
eval instant at 1010s sum_over_time(metric1[30s:10s] offset 10s)
{} 394
# Same result for different offsets due to step alignment.
eval instant at 1010s sum_over_time(metric1[30s:10s] offset 9s)
{} 297
eval instant at 1010s sum_over_time(metric1[30s:10s] offset 7s)
{} 297
eval instant at 1010s sum_over_time(metric1[30s:10s] offset 5s)
{} 297
eval instant at 1010s sum_over_time(metric1[30s:10s] offset 3s)
{} 297
# Nested subqueries
eval instant at 1000s rate(sum_over_time(metric1[30s:10s])[50s:10s])
{} 0.4
eval instant at 1000s rate(sum_over_time(metric2[30s:10s])[50s:10s])
{} 0.8
eval instant at 1000s rate(sum_over_time(metric3[30s:10s])[50s:10s])
{} 1.2
eval instant at 1000s rate(sum_over_time((metric1+metric2+metric3)[30s:10s])[30s:10s])
{} 2.4
clear
# Fibonacci sequence, to ensure the rate is not constant.
# Additional note: using subqueries unnecessarily is unwise.
load 7s
metric 1 1 2 3 5 8 13 21 34 55 89 144 233 377 610 987 1597 2584 4181 6765 10946 17711 28657 46368 75025 121393 196418 317811 514229 832040 1346269 2178309 3524578 5702887 9227465 14930352 24157817 39088169 63245986 102334155 165580141 267914296 433494437 701408733 1134903170 1836311903 2971215073 4807526976 7778742049 12586269025 20365011074 32951280099 53316291173 86267571272 139583862445 225851433717 365435296162 591286729879 956722026041 1548008755920 2504730781961 4052739537881 6557470319842 10610209857723 17167680177565 27777890035288 44945570212853 72723460248141 117669030460994 190392490709135 308061521170129 498454011879264 806515533049393 1304969544928657 2111485077978050 3416454622906707 5527939700884757 8944394323791464 14472334024676221 23416728348467685 37889062373143906 61305790721611591 99194853094755497 160500643816367088 259695496911122585 420196140727489673 679891637638612258 1100087778366101931 1779979416004714189 2880067194370816120 4660046610375530309 7540113804746346429 12200160415121876738 19740274219868223167 31940434634990099905 51680708854858323072 83621143489848422977 135301852344706746049 218922995834555169026 354224848179261915075 573147844013817084101 927372692193078999176 1500520536206896083277 2427893228399975082453 3928413764606871165730 6356306993006846248183 10284720757613717413913 16641027750620563662096 26925748508234281076009 43566776258854844738105 70492524767089125814114 114059301025943970552219 184551825793033096366333 298611126818977066918552 483162952612010163284885 781774079430987230203437 1264937032042997393488322 2046711111473984623691759 3311648143516982017180081 5358359254990966640871840 8670007398507948658051921 14028366653498915298923761 22698374052006863956975682 36726740705505779255899443 59425114757512643212875125 96151855463018422468774568 155576970220531065681649693 251728825683549488150424261 407305795904080553832073954 659034621587630041982498215 1066340417491710595814572169 1725375039079340637797070384 2791715456571051233611642553 4517090495650391871408712937 7308805952221443105020355490 11825896447871834976429068427 19134702400093278081449423917 30960598847965113057878492344 50095301248058391139327916261 81055900096023504197206408605 131151201344081895336534324866 212207101440105399533740733471 343358302784187294870275058337 555565404224292694404015791808 898923707008479989274290850145 1454489111232772683678306641953 2353412818241252672952597492098 3807901929474025356630904134051 6161314747715278029583501626149 9969216677189303386214405760200 16130531424904581415797907386349 26099748102093884802012313146549 42230279526998466217810220532898 68330027629092351019822533679447 110560307156090817237632754212345 178890334785183168257455287891792 289450641941273985495088042104137 468340976726457153752543329995929 757791618667731139247631372100066 1226132595394188293000174702095995 1983924214061919432247806074196061 3210056809456107725247980776292056 5193981023518027157495786850488117 8404037832974134882743767626780173 13598018856492162040239554477268290 22002056689466296922983322104048463 35600075545958458963222876581316753 57602132235424755886206198685365216 93202207781383214849429075266681969 150804340016807970735635273952047185 244006547798191185585064349218729154 394810887814999156320699623170776339 638817435613190341905763972389505493 1033628323428189498226463595560281832 1672445759041379840132227567949787325 2706074082469569338358691163510069157 4378519841510949178490918731459856482 7084593923980518516849609894969925639 11463113765491467695340528626429782121 18547707689471986212190138521399707760
# Extrapolated from [3@21, 144@77]: (144 - 3) / (77 - 21)
eval instant at 80s rate(metric[1m])
{} 2.517857143
# No extrapolation, [2@20, 144@80]: (144 - 2) / 60
eval instant at 80s rate(metric[1m:10s])
{} 2.366666667
# Only one value between 10s and 20s, 2@14
eval instant at 20s min_over_time(metric[10s])
{} 2
# min(1@10, 2@20)
eval instant at 20s min_over_time(metric[10s:10s])
{} 1
eval instant at 20m min_over_time(rate(metric[5m])[20m:1m])
{} 0.12119047619047618

View file

@ -255,3 +255,68 @@ func (r *Result) String() string {
} }
return r.Value.String() return r.Value.String()
} }
// StorageSeries simulates promql.Series as storage.Series.
type StorageSeries struct {
series Series
}
// NewStorageSeries returns a StorageSeries fromfor series.
func NewStorageSeries(series Series) *StorageSeries {
return &StorageSeries{
series: series,
}
}
func (ss *StorageSeries) Labels() labels.Labels {
return ss.series.Metric
}
// Iterator returns a new iterator of the data of the series.
func (ss *StorageSeries) Iterator() storage.SeriesIterator {
return newStorageSeriesIterator(ss.series)
}
type storageSeriesIterator struct {
points []Point
curr int
}
func newStorageSeriesIterator(series Series) *storageSeriesIterator {
return &storageSeriesIterator{
points: series.Points,
curr: -1,
}
}
func (ssi *storageSeriesIterator) Seek(t int64) bool {
i := ssi.curr
if i < 0 {
i = 0
}
for ; i < len(ssi.points); i++ {
if ssi.points[i].T >= t {
ssi.curr = i
return true
}
}
ssi.curr = len(ssi.points) - 1
return false
}
func (ssi *storageSeriesIterator) At() (t int64, v float64) {
p := ssi.points[ssi.curr]
return p.T, p.V
}
func (ssi *storageSeriesIterator) Next() bool {
ssi.curr++
if ssi.curr >= len(ssi.points) {
return false
}
return true
}
func (ssi *storageSeriesIterator) Err() error {
return nil
}