created docker-prometheus compose file and edited the Prometheus yml file
This commit is contained in:
168
prom/promql/analyzer.go
Normal file
168
prom/promql/analyzer.go
Normal file
@ -0,0 +1,168 @@
|
||||
// Copyright 2013 The Prometheus Authors
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package promql
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"time"
|
||||
|
||||
"golang.org/x/net/context"
|
||||
|
||||
clientmodel "github.com/prometheus/client_golang/model"
|
||||
|
||||
"github.com/prometheus/prometheus/storage/local"
|
||||
)
|
||||
|
||||
// An Analyzer traverses an expression and determines which data has to be requested
|
||||
// from the storage. It is bound to a context that allows cancellation and timing out.
|
||||
type Analyzer struct {
|
||||
// The storage from which to query data.
|
||||
Storage local.Storage
|
||||
// The expression being analyzed.
|
||||
Expr Expr
|
||||
// The time range for evaluation of Expr.
|
||||
Start, End clientmodel.Timestamp
|
||||
|
||||
// The preload times for different query time offsets.
|
||||
offsetPreloadTimes map[time.Duration]preloadTimes
|
||||
}
|
||||
|
||||
// preloadTimes tracks which instants or ranges to preload for a set of
|
||||
// fingerprints. One of these structs is collected for each offset by the query
|
||||
// analyzer.
|
||||
type preloadTimes struct {
|
||||
// Instants require single samples to be loaded along the entire query
|
||||
// range, with intervals between the samples corresponding to the query
|
||||
// resolution.
|
||||
instants map[clientmodel.Fingerprint]struct{}
|
||||
// Ranges require loading a range of samples at each resolution step,
|
||||
// stretching backwards from the current evaluation timestamp. The length of
|
||||
// the range into the past is given by the duration, as in "foo[5m]".
|
||||
ranges map[clientmodel.Fingerprint]time.Duration
|
||||
}
|
||||
|
||||
// Analyze the provided expression and attach metrics and fingerprints to data-selecting
|
||||
// AST nodes that are later used to preload the data from the storage.
|
||||
func (a *Analyzer) Analyze(ctx context.Context) error {
|
||||
a.offsetPreloadTimes = map[time.Duration]preloadTimes{}
|
||||
|
||||
getPreloadTimes := func(offset time.Duration) preloadTimes {
|
||||
if _, ok := a.offsetPreloadTimes[offset]; !ok {
|
||||
a.offsetPreloadTimes[offset] = preloadTimes{
|
||||
instants: map[clientmodel.Fingerprint]struct{}{},
|
||||
ranges: map[clientmodel.Fingerprint]time.Duration{},
|
||||
}
|
||||
}
|
||||
return a.offsetPreloadTimes[offset]
|
||||
}
|
||||
|
||||
// Retrieve fingerprints and metrics for the required time range for
|
||||
// each metric or matrix selector node.
|
||||
Inspect(a.Expr, func(node Node) bool {
|
||||
switch n := node.(type) {
|
||||
case *VectorSelector:
|
||||
n.metrics = a.Storage.MetricsForLabelMatchers(n.LabelMatchers...)
|
||||
n.iterators = make(map[clientmodel.Fingerprint]local.SeriesIterator, len(n.metrics))
|
||||
|
||||
pt := getPreloadTimes(n.Offset)
|
||||
for fp := range n.metrics {
|
||||
// Only add the fingerprint to the instants if not yet present in the
|
||||
// ranges. Ranges always contain more points and span more time than
|
||||
// instants for the same offset.
|
||||
if _, alreadyInRanges := pt.ranges[fp]; !alreadyInRanges {
|
||||
pt.instants[fp] = struct{}{}
|
||||
}
|
||||
}
|
||||
case *MatrixSelector:
|
||||
n.metrics = a.Storage.MetricsForLabelMatchers(n.LabelMatchers...)
|
||||
n.iterators = make(map[clientmodel.Fingerprint]local.SeriesIterator, len(n.metrics))
|
||||
|
||||
pt := getPreloadTimes(n.Offset)
|
||||
for fp := range n.metrics {
|
||||
if pt.ranges[fp] < n.Range {
|
||||
pt.ranges[fp] = n.Range
|
||||
// Delete the fingerprint from the instants. Ranges always contain more
|
||||
// points and span more time than instants, so we don't need to track
|
||||
// an instant for the same fingerprint, should we have one.
|
||||
delete(pt.instants, fp)
|
||||
}
|
||||
}
|
||||
}
|
||||
return true
|
||||
})
|
||||
|
||||
// Currently we do not return an error but we might place a context check in here
|
||||
// or extend the stage in some other way.
|
||||
return nil
|
||||
}
|
||||
|
||||
// Prepare the expression evaluation by preloading all required chunks from the storage
|
||||
// and setting the respective storage iterators in the AST nodes.
|
||||
func (a *Analyzer) Prepare(ctx context.Context) (local.Preloader, error) {
|
||||
const env = "query preparation"
|
||||
|
||||
if a.offsetPreloadTimes == nil {
|
||||
return nil, errors.New("analysis must be performed before preparing query")
|
||||
}
|
||||
var err error
|
||||
// The preloader must not be closed unless an error ocurred as closing
|
||||
// unpins the preloaded chunks.
|
||||
p := a.Storage.NewPreloader()
|
||||
defer func() {
|
||||
if err != nil {
|
||||
p.Close()
|
||||
}
|
||||
}()
|
||||
|
||||
// Preload all analyzed ranges.
|
||||
for offset, pt := range a.offsetPreloadTimes {
|
||||
start := a.Start.Add(-offset)
|
||||
end := a.End.Add(-offset)
|
||||
for fp, rangeDuration := range pt.ranges {
|
||||
if err = contextDone(ctx, env); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
err = p.PreloadRange(fp, start.Add(-rangeDuration), end, StalenessDelta)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
for fp := range pt.instants {
|
||||
if err = contextDone(ctx, env); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
err = p.PreloadRange(fp, start, end, StalenessDelta)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Attach storage iterators to AST nodes.
|
||||
Inspect(a.Expr, func(node Node) bool {
|
||||
switch n := node.(type) {
|
||||
case *VectorSelector:
|
||||
for fp := range n.metrics {
|
||||
n.iterators[fp] = a.Storage.NewIterator(fp)
|
||||
}
|
||||
case *MatrixSelector:
|
||||
for fp := range n.metrics {
|
||||
n.iterators[fp] = a.Storage.NewIterator(fp)
|
||||
}
|
||||
}
|
||||
return true
|
||||
})
|
||||
|
||||
return p, nil
|
||||
}
|
346
prom/promql/ast.go
Normal file
346
prom/promql/ast.go
Normal file
@ -0,0 +1,346 @@
|
||||
// Copyright 2015 The Prometheus Authors
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package promql
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
clientmodel "github.com/prometheus/client_golang/model"
|
||||
|
||||
"github.com/prometheus/prometheus/storage/local"
|
||||
"github.com/prometheus/prometheus/storage/metric"
|
||||
)
|
||||
|
||||
// Node is a generic interface for all nodes in an AST.
|
||||
//
|
||||
// Whenever numerous nodes are listed such as in a switch-case statement
|
||||
// or a chain of function definitions (e.g. String(), expr(), etc.) convention is
|
||||
// to list them as follows:
|
||||
//
|
||||
// - Statements
|
||||
// - statement types (alphabetical)
|
||||
// - ...
|
||||
// - Expressions
|
||||
// - expression types (alphabetical)
|
||||
// - ...
|
||||
//
|
||||
type Node interface {
|
||||
// String representation of the node that returns the given node when parsed
|
||||
// as part of a valid query.
|
||||
String() string
|
||||
}
|
||||
|
||||
// Statement is a generic interface for all statements.
|
||||
type Statement interface {
|
||||
Node
|
||||
|
||||
// stmt ensures that no other type accidentally implements the interface
|
||||
stmt()
|
||||
}
|
||||
|
||||
// Statements is a list of statement nodes that implements Node.
|
||||
type Statements []Statement
|
||||
|
||||
// AlertStmt represents an added alert rule.
|
||||
type AlertStmt struct {
|
||||
Name string
|
||||
Expr Expr
|
||||
Duration time.Duration
|
||||
Labels clientmodel.LabelSet
|
||||
Summary string
|
||||
Description string
|
||||
Runbook string
|
||||
}
|
||||
|
||||
// EvalStmt holds an expression and information on the range it should
|
||||
// be evaluated on.
|
||||
type EvalStmt struct {
|
||||
Expr Expr // Expression to be evaluated.
|
||||
|
||||
// The time boundaries for the evaluation. If Start equals End an instant
|
||||
// is evaluated.
|
||||
Start, End clientmodel.Timestamp
|
||||
// Time between two evaluated instants for the range [Start:End].
|
||||
Interval time.Duration
|
||||
}
|
||||
|
||||
// RecordStmt represents an added recording rule.
|
||||
type RecordStmt struct {
|
||||
Name string
|
||||
Expr Expr
|
||||
Labels clientmodel.LabelSet
|
||||
}
|
||||
|
||||
func (*AlertStmt) stmt() {}
|
||||
func (*EvalStmt) stmt() {}
|
||||
func (*RecordStmt) stmt() {}
|
||||
|
||||
// ExprType is the type an evaluated expression returns.
|
||||
type ExprType int
|
||||
|
||||
const (
|
||||
ExprNone ExprType = iota
|
||||
ExprScalar
|
||||
ExprVector
|
||||
ExprMatrix
|
||||
ExprString
|
||||
)
|
||||
|
||||
// MarshalJSON implements json.Marshaler.
|
||||
func (et ExprType) MarshalJSON() ([]byte, error) {
|
||||
return json.Marshal(et.String())
|
||||
}
|
||||
|
||||
func (e ExprType) String() string {
|
||||
switch e {
|
||||
case ExprNone:
|
||||
return "<ExprNone>"
|
||||
case ExprScalar:
|
||||
return "scalar"
|
||||
case ExprVector:
|
||||
return "vector"
|
||||
case ExprMatrix:
|
||||
return "matrix"
|
||||
case ExprString:
|
||||
return "string"
|
||||
}
|
||||
panic("promql.ExprType.String: unhandled expression type")
|
||||
}
|
||||
|
||||
// Expr is a generic interface for all expression types.
|
||||
type Expr interface {
|
||||
Node
|
||||
|
||||
// Type returns the type the expression evaluates to. It does not perform
|
||||
// in-depth checks as this is done at parsing-time.
|
||||
Type() ExprType
|
||||
// expr ensures that no other types accidentally implement the interface.
|
||||
expr()
|
||||
}
|
||||
|
||||
// Expressions is a list of expression nodes that implements Node.
|
||||
type Expressions []Expr
|
||||
|
||||
// AggregateExpr represents an aggregation operation on a vector.
|
||||
type AggregateExpr struct {
|
||||
Op itemType // The used aggregation operation.
|
||||
Expr Expr // The vector expression over which is aggregated.
|
||||
Grouping clientmodel.LabelNames // The labels by which to group the vector.
|
||||
KeepExtraLabels bool // Whether to keep extra labels common among result elements.
|
||||
}
|
||||
|
||||
// BinaryExpr represents a binary expression between two child expressions.
|
||||
type BinaryExpr struct {
|
||||
Op itemType // The operation of the expression.
|
||||
LHS, RHS Expr // The operands on the respective sides of the operator.
|
||||
|
||||
// The matching behavior for the operation if both operands are vectors.
|
||||
// If they are not this field is nil.
|
||||
VectorMatching *VectorMatching
|
||||
}
|
||||
|
||||
// Call represents a function call.
|
||||
type Call struct {
|
||||
Func *Function // The function that was called.
|
||||
Args Expressions // Arguments used in the call.
|
||||
}
|
||||
|
||||
// MatrixSelector represents a matrix selection.
|
||||
type MatrixSelector struct {
|
||||
Name string
|
||||
Range time.Duration
|
||||
Offset time.Duration
|
||||
LabelMatchers metric.LabelMatchers
|
||||
|
||||
// The series iterators are populated at query analysis time.
|
||||
iterators map[clientmodel.Fingerprint]local.SeriesIterator
|
||||
metrics map[clientmodel.Fingerprint]clientmodel.COWMetric
|
||||
}
|
||||
|
||||
// NumberLiteral represents a number.
|
||||
type NumberLiteral struct {
|
||||
Val clientmodel.SampleValue
|
||||
}
|
||||
|
||||
// ParenExpr wraps an expression so it cannot be disassembled as a consequence
|
||||
// of operator precendence.
|
||||
type ParenExpr struct {
|
||||
Expr Expr
|
||||
}
|
||||
|
||||
// StringLiteral represents a string.
|
||||
type StringLiteral struct {
|
||||
Val string
|
||||
}
|
||||
|
||||
// UnaryExpr represents a unary operation on another expression.
|
||||
// Currently unary operations are only supported for scalars.
|
||||
type UnaryExpr struct {
|
||||
Op itemType
|
||||
Expr Expr
|
||||
}
|
||||
|
||||
// VectorSelector represents a vector selection.
|
||||
type VectorSelector struct {
|
||||
Name string
|
||||
Offset time.Duration
|
||||
LabelMatchers metric.LabelMatchers
|
||||
|
||||
// The series iterators are populated at query analysis time.
|
||||
iterators map[clientmodel.Fingerprint]local.SeriesIterator
|
||||
metrics map[clientmodel.Fingerprint]clientmodel.COWMetric
|
||||
}
|
||||
|
||||
func (e *AggregateExpr) Type() ExprType { return ExprVector }
|
||||
func (e *Call) Type() ExprType { return e.Func.ReturnType }
|
||||
func (e *MatrixSelector) Type() ExprType { return ExprMatrix }
|
||||
func (e *NumberLiteral) Type() ExprType { return ExprScalar }
|
||||
func (e *ParenExpr) Type() ExprType { return e.Expr.Type() }
|
||||
func (e *StringLiteral) Type() ExprType { return ExprString }
|
||||
func (e *UnaryExpr) Type() ExprType { return e.Expr.Type() }
|
||||
func (e *VectorSelector) Type() ExprType { return ExprVector }
|
||||
|
||||
func (e *BinaryExpr) Type() ExprType {
|
||||
if e.LHS.Type() == ExprScalar && e.RHS.Type() == ExprScalar {
|
||||
return ExprScalar
|
||||
}
|
||||
return ExprVector
|
||||
}
|
||||
|
||||
func (*AggregateExpr) expr() {}
|
||||
func (*BinaryExpr) expr() {}
|
||||
func (*Call) expr() {}
|
||||
func (*MatrixSelector) expr() {}
|
||||
func (*NumberLiteral) expr() {}
|
||||
func (*ParenExpr) expr() {}
|
||||
func (*StringLiteral) expr() {}
|
||||
func (*UnaryExpr) expr() {}
|
||||
func (*VectorSelector) expr() {}
|
||||
|
||||
// VectorMatchCardinaly describes the cardinality relationship
|
||||
// of two vectors in a binary operation.
|
||||
type VectorMatchCardinality int
|
||||
|
||||
const (
|
||||
CardOneToOne VectorMatchCardinality = iota
|
||||
CardManyToOne
|
||||
CardOneToMany
|
||||
CardManyToMany
|
||||
)
|
||||
|
||||
func (vmc VectorMatchCardinality) String() string {
|
||||
switch vmc {
|
||||
case CardOneToOne:
|
||||
return "one-to-one"
|
||||
case CardManyToOne:
|
||||
return "many-to-one"
|
||||
case CardOneToMany:
|
||||
return "one-to-many"
|
||||
case CardManyToMany:
|
||||
return "many-to-many"
|
||||
}
|
||||
panic("promql.VectorMatchCardinality.String: unknown match cardinality")
|
||||
}
|
||||
|
||||
// VectorMatching describes how elements from two vectors in a binary
|
||||
// operation are supposed to be matched.
|
||||
type VectorMatching struct {
|
||||
// The cardinality of the two vectors.
|
||||
Card VectorMatchCardinality
|
||||
// On contains the labels which define equality of a pair
|
||||
// of elements from the vectors.
|
||||
On clientmodel.LabelNames
|
||||
// Include contains additional labels that should be included in
|
||||
// the result from the side with the higher cardinality.
|
||||
Include clientmodel.LabelNames
|
||||
}
|
||||
|
||||
// A Visitor's Visit method is invoked for each node encountered by Walk.
|
||||
// If the result visitor w is not nil, Walk visits each of the children
|
||||
// of node with the visitor w, followed by a call of w.Visit(nil).
|
||||
type Visitor interface {
|
||||
Visit(node Node) (w Visitor)
|
||||
}
|
||||
|
||||
// Walk traverses an AST in depth-first order: It starts by calling
|
||||
// v.Visit(node); node must not be nil. If the visitor w returned by
|
||||
// v.Visit(node) is not nil, Walk is invoked recursively with visitor
|
||||
// w for each of the non-nil children of node, followed by a call of
|
||||
// w.Visit(nil).
|
||||
func Walk(v Visitor, node Node) {
|
||||
if v = v.Visit(node); v == nil {
|
||||
return
|
||||
}
|
||||
|
||||
switch n := node.(type) {
|
||||
case Statements:
|
||||
for _, s := range n {
|
||||
Walk(v, s)
|
||||
}
|
||||
case *AlertStmt:
|
||||
Walk(v, n.Expr)
|
||||
|
||||
case *EvalStmt:
|
||||
Walk(v, n.Expr)
|
||||
|
||||
case *RecordStmt:
|
||||
Walk(v, n.Expr)
|
||||
|
||||
case Expressions:
|
||||
for _, e := range n {
|
||||
Walk(v, e)
|
||||
}
|
||||
case *AggregateExpr:
|
||||
Walk(v, n.Expr)
|
||||
|
||||
case *BinaryExpr:
|
||||
Walk(v, n.LHS)
|
||||
Walk(v, n.RHS)
|
||||
|
||||
case *Call:
|
||||
Walk(v, n.Args)
|
||||
|
||||
case *ParenExpr:
|
||||
Walk(v, n.Expr)
|
||||
|
||||
case *UnaryExpr:
|
||||
Walk(v, n.Expr)
|
||||
|
||||
case *MatrixSelector, *NumberLiteral, *StringLiteral, *VectorSelector:
|
||||
// nothing to do
|
||||
|
||||
default:
|
||||
panic(fmt.Errorf("promql.Walk: unhandled node type %T", node))
|
||||
}
|
||||
|
||||
v.Visit(nil)
|
||||
}
|
||||
|
||||
type inspector func(Node) bool
|
||||
|
||||
func (f inspector) Visit(node Node) Visitor {
|
||||
if f(node) {
|
||||
return f
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// Inspect traverses an AST in depth-first order: It starts by calling
|
||||
// f(node); node must not be nil. If f returns true, Inspect invokes f
|
||||
// for all the non-nil children of node, recursively.
|
||||
func Inspect(node Node, f func(Node) bool) {
|
||||
Walk(inspector(f), node)
|
||||
}
|
1254
prom/promql/engine.go
Normal file
1254
prom/promql/engine.go
Normal file
File diff suppressed because it is too large
Load Diff
181
prom/promql/engine_test.go
Normal file
181
prom/promql/engine_test.go
Normal file
@ -0,0 +1,181 @@
|
||||
package promql
|
||||
|
||||
import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"golang.org/x/net/context"
|
||||
)
|
||||
|
||||
var noop = testStmt(func(context.Context) error {
|
||||
return nil
|
||||
})
|
||||
|
||||
func TestQueryConcurreny(t *testing.T) {
|
||||
engine := NewEngine(nil, nil)
|
||||
defer engine.Stop()
|
||||
|
||||
block := make(chan struct{})
|
||||
processing := make(chan struct{})
|
||||
|
||||
f := func(context.Context) error {
|
||||
processing <- struct{}{}
|
||||
<-block
|
||||
return nil
|
||||
}
|
||||
|
||||
for i := 0; i < DefaultEngineOptions.MaxConcurrentQueries; i++ {
|
||||
q := engine.newTestQuery(f)
|
||||
go q.Exec()
|
||||
select {
|
||||
case <-processing:
|
||||
// Expected.
|
||||
case <-time.After(5 * time.Millisecond):
|
||||
t.Fatalf("Query within concurrency threshold not being executed")
|
||||
}
|
||||
}
|
||||
|
||||
q := engine.newTestQuery(f)
|
||||
go q.Exec()
|
||||
|
||||
select {
|
||||
case <-processing:
|
||||
t.Fatalf("Query above concurrency threhosld being executed")
|
||||
case <-time.After(5 * time.Millisecond):
|
||||
// Expected.
|
||||
}
|
||||
|
||||
// Terminate a running query.
|
||||
block <- struct{}{}
|
||||
|
||||
select {
|
||||
case <-processing:
|
||||
// Expected.
|
||||
case <-time.After(5 * time.Millisecond):
|
||||
t.Fatalf("Query within concurrency threshold not being executed")
|
||||
}
|
||||
|
||||
// Terminate remaining queries.
|
||||
for i := 0; i < DefaultEngineOptions.MaxConcurrentQueries; i++ {
|
||||
block <- struct{}{}
|
||||
}
|
||||
}
|
||||
|
||||
func TestQueryTimeout(t *testing.T) {
|
||||
engine := NewEngine(nil, &EngineOptions{
|
||||
Timeout: 5 * time.Millisecond,
|
||||
MaxConcurrentQueries: 20,
|
||||
})
|
||||
defer engine.Stop()
|
||||
|
||||
query := engine.newTestQuery(func(ctx context.Context) error {
|
||||
time.Sleep(10 * time.Millisecond)
|
||||
return contextDone(ctx, "test statement execution")
|
||||
})
|
||||
|
||||
res := query.Exec()
|
||||
if res.Err == nil {
|
||||
t.Fatalf("expected timeout error but got none")
|
||||
}
|
||||
if _, ok := res.Err.(ErrQueryTimeout); res.Err != nil && !ok {
|
||||
t.Fatalf("expected timeout error but got: %s", res.Err)
|
||||
}
|
||||
}
|
||||
|
||||
func TestQueryCancel(t *testing.T) {
|
||||
engine := NewEngine(nil, nil)
|
||||
defer engine.Stop()
|
||||
|
||||
// Cancel a running query before it completes.
|
||||
block := make(chan struct{})
|
||||
processing := make(chan struct{})
|
||||
|
||||
query1 := engine.newTestQuery(func(ctx context.Context) error {
|
||||
processing <- struct{}{}
|
||||
<-block
|
||||
return contextDone(ctx, "test statement execution")
|
||||
})
|
||||
|
||||
var res *Result
|
||||
|
||||
go func() {
|
||||
res = query1.Exec()
|
||||
processing <- struct{}{}
|
||||
}()
|
||||
|
||||
<-processing
|
||||
query1.Cancel()
|
||||
block <- struct{}{}
|
||||
<-processing
|
||||
|
||||
if res.Err == nil {
|
||||
t.Fatalf("expected cancellation error for query1 but got none")
|
||||
}
|
||||
if ee := ErrQueryCanceled("test statement execution"); res.Err != ee {
|
||||
t.Fatalf("expected error %q, got %q")
|
||||
}
|
||||
|
||||
// Canceling a query before starting it must have no effect.
|
||||
query2 := engine.newTestQuery(func(ctx context.Context) error {
|
||||
return contextDone(ctx, "test statement execution")
|
||||
})
|
||||
|
||||
query2.Cancel()
|
||||
res = query2.Exec()
|
||||
if res.Err != nil {
|
||||
t.Fatalf("unexpeceted error on executing query2: %s", res.Err)
|
||||
}
|
||||
}
|
||||
|
||||
func TestEngineShutdown(t *testing.T) {
|
||||
engine := NewEngine(nil, nil)
|
||||
|
||||
block := make(chan struct{})
|
||||
processing := make(chan struct{})
|
||||
|
||||
// Shutdown engine on first handler execution. Should handler execution ever become
|
||||
// concurrent this test has to be adjusted accordingly.
|
||||
f := func(ctx context.Context) error {
|
||||
processing <- struct{}{}
|
||||
<-block
|
||||
return contextDone(ctx, "test statement execution")
|
||||
}
|
||||
query1 := engine.newTestQuery(f)
|
||||
|
||||
// Stopping the engine must cancel the base context. While executing queries is
|
||||
// still possible, their context is canceled from the beginning and execution should
|
||||
// terminate immediately.
|
||||
|
||||
var res *Result
|
||||
go func() {
|
||||
res = query1.Exec()
|
||||
processing <- struct{}{}
|
||||
}()
|
||||
|
||||
<-processing
|
||||
engine.Stop()
|
||||
block <- struct{}{}
|
||||
<-processing
|
||||
|
||||
if res.Err == nil {
|
||||
t.Fatalf("expected error on shutdown during query but got none")
|
||||
}
|
||||
if ee := ErrQueryCanceled("test statement execution"); res.Err != ee {
|
||||
t.Fatalf("expected error %q, got %q", ee, res.Err)
|
||||
}
|
||||
|
||||
query2 := engine.newTestQuery(func(context.Context) error {
|
||||
t.Fatalf("reached query execution unexpectedly")
|
||||
return nil
|
||||
})
|
||||
|
||||
// The second query is started after the engine shut down. It must
|
||||
// be canceled immediately.
|
||||
res2 := query2.Exec()
|
||||
if res2.Err == nil {
|
||||
t.Fatalf("expected error on querying shutdown engine but got none")
|
||||
}
|
||||
if _, ok := res2.Err.(ErrQueryCanceled); !ok {
|
||||
t.Fatalf("expected cancelation error, got %q", res2.Err)
|
||||
}
|
||||
}
|
838
prom/promql/functions.go
Normal file
838
prom/promql/functions.go
Normal file
@ -0,0 +1,838 @@
|
||||
// Copyright 2015 The Prometheus Authors
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package promql
|
||||
|
||||
import (
|
||||
"container/heap"
|
||||
"math"
|
||||
"sort"
|
||||
"strconv"
|
||||
"time"
|
||||
|
||||
clientmodel "github.com/prometheus/client_golang/model"
|
||||
|
||||
"github.com/prometheus/prometheus/storage/metric"
|
||||
)
|
||||
|
||||
// Function represents a function of the expression language and is
|
||||
// used by function nodes.
|
||||
type Function struct {
|
||||
Name string
|
||||
ArgTypes []ExprType
|
||||
OptionalArgs int
|
||||
ReturnType ExprType
|
||||
Call func(ev *evaluator, args Expressions) Value
|
||||
}
|
||||
|
||||
// === time() clientmodel.SampleValue ===
|
||||
func funcTime(ev *evaluator, args Expressions) Value {
|
||||
return &Scalar{
|
||||
Value: clientmodel.SampleValue(ev.Timestamp.Unix()),
|
||||
Timestamp: ev.Timestamp,
|
||||
}
|
||||
}
|
||||
|
||||
// === delta(matrix ExprMatrix, isCounter=0 ExprScalar) Vector ===
|
||||
func funcDelta(ev *evaluator, args Expressions) Value {
|
||||
isCounter := len(args) >= 2 && ev.evalInt(args[1]) > 0
|
||||
resultVector := Vector{}
|
||||
|
||||
// If we treat these metrics as counters, we need to fetch all values
|
||||
// in the interval to find breaks in the timeseries' monotonicity.
|
||||
// I.e. if a counter resets, we want to ignore that reset.
|
||||
var matrixValue Matrix
|
||||
if isCounter {
|
||||
matrixValue = ev.evalMatrix(args[0])
|
||||
} else {
|
||||
matrixValue = ev.evalMatrixBounds(args[0])
|
||||
}
|
||||
for _, samples := range matrixValue {
|
||||
// No sense in trying to compute a delta without at least two points. Drop
|
||||
// this vector element.
|
||||
if len(samples.Values) < 2 {
|
||||
continue
|
||||
}
|
||||
|
||||
counterCorrection := clientmodel.SampleValue(0)
|
||||
lastValue := clientmodel.SampleValue(0)
|
||||
for _, sample := range samples.Values {
|
||||
currentValue := sample.Value
|
||||
if isCounter && currentValue < lastValue {
|
||||
counterCorrection += lastValue - currentValue
|
||||
}
|
||||
lastValue = currentValue
|
||||
}
|
||||
resultValue := lastValue - samples.Values[0].Value + counterCorrection
|
||||
|
||||
targetInterval := args[0].(*MatrixSelector).Range
|
||||
sampledInterval := samples.Values[len(samples.Values)-1].Timestamp.Sub(samples.Values[0].Timestamp)
|
||||
if sampledInterval == 0 {
|
||||
// Only found one sample. Cannot compute a rate from this.
|
||||
continue
|
||||
}
|
||||
// Correct for differences in target vs. actual delta interval.
|
||||
//
|
||||
// Above, we didn't actually calculate the delta for the specified target
|
||||
// interval, but for an interval between the first and last found samples
|
||||
// under the target interval, which will usually have less time between
|
||||
// them. Depending on how many samples are found under a target interval,
|
||||
// the delta results are distorted and temporal aliasing occurs (ugly
|
||||
// bumps). This effect is corrected for below.
|
||||
intervalCorrection := clientmodel.SampleValue(targetInterval) / clientmodel.SampleValue(sampledInterval)
|
||||
resultValue *= intervalCorrection
|
||||
|
||||
resultSample := &Sample{
|
||||
Metric: samples.Metric,
|
||||
Value: resultValue,
|
||||
Timestamp: ev.Timestamp,
|
||||
}
|
||||
resultSample.Metric.Delete(clientmodel.MetricNameLabel)
|
||||
resultVector = append(resultVector, resultSample)
|
||||
}
|
||||
return resultVector
|
||||
}
|
||||
|
||||
// === rate(node ExprMatrix) Vector ===
|
||||
func funcRate(ev *evaluator, args Expressions) Value {
|
||||
args = append(args, &NumberLiteral{1})
|
||||
vector := funcDelta(ev, args).(Vector)
|
||||
|
||||
// TODO: could be other type of ExprMatrix in the future (right now, only
|
||||
// MatrixSelector exists). Find a better way of getting the duration of a
|
||||
// matrix, such as looking at the samples themselves.
|
||||
interval := args[0].(*MatrixSelector).Range
|
||||
for i := range vector {
|
||||
vector[i].Value /= clientmodel.SampleValue(interval / time.Second)
|
||||
}
|
||||
return vector
|
||||
}
|
||||
|
||||
// === increase(node ExprMatrix) Vector ===
|
||||
func funcIncrease(ev *evaluator, args Expressions) Value {
|
||||
args = append(args, &NumberLiteral{1})
|
||||
vector := funcDelta(ev, args).(Vector)
|
||||
return vector
|
||||
}
|
||||
|
||||
// === sort(node ExprVector) Vector ===
|
||||
func funcSort(ev *evaluator, args Expressions) Value {
|
||||
byValueSorter := vectorByValueHeap(ev.evalVector(args[0]))
|
||||
sort.Sort(byValueSorter)
|
||||
return Vector(byValueSorter)
|
||||
}
|
||||
|
||||
// === sortDesc(node ExprVector) Vector ===
|
||||
func funcSortDesc(ev *evaluator, args Expressions) Value {
|
||||
byValueSorter := vectorByValueHeap(ev.evalVector(args[0]))
|
||||
sort.Sort(sort.Reverse(byValueSorter))
|
||||
return Vector(byValueSorter)
|
||||
}
|
||||
|
||||
// === topk(k ExprScalar, node ExprVector) Vector ===
|
||||
func funcTopk(ev *evaluator, args Expressions) Value {
|
||||
k := ev.evalInt(args[0])
|
||||
if k < 1 {
|
||||
return Vector{}
|
||||
}
|
||||
vector := ev.evalVector(args[1])
|
||||
|
||||
topk := make(vectorByValueHeap, 0, k)
|
||||
|
||||
for _, el := range vector {
|
||||
if len(topk) < k || topk[0].Value < el.Value {
|
||||
if len(topk) == k {
|
||||
heap.Pop(&topk)
|
||||
}
|
||||
heap.Push(&topk, el)
|
||||
}
|
||||
}
|
||||
sort.Sort(sort.Reverse(topk))
|
||||
return Vector(topk)
|
||||
}
|
||||
|
||||
// === bottomk(k ExprScalar, node ExprVector) Vector ===
|
||||
func funcBottomk(ev *evaluator, args Expressions) Value {
|
||||
k := ev.evalInt(args[0])
|
||||
if k < 1 {
|
||||
return Vector{}
|
||||
}
|
||||
vector := ev.evalVector(args[1])
|
||||
|
||||
bottomk := make(vectorByValueHeap, 0, k)
|
||||
bkHeap := reverseHeap{Interface: &bottomk}
|
||||
|
||||
for _, el := range vector {
|
||||
if len(bottomk) < k || bottomk[0].Value > el.Value {
|
||||
if len(bottomk) == k {
|
||||
heap.Pop(&bkHeap)
|
||||
}
|
||||
heap.Push(&bkHeap, el)
|
||||
}
|
||||
}
|
||||
sort.Sort(bottomk)
|
||||
return Vector(bottomk)
|
||||
}
|
||||
|
||||
// === drop_common_labels(node ExprVector) Vector ===
|
||||
func funcDropCommonLabels(ev *evaluator, args Expressions) Value {
|
||||
vector := ev.evalVector(args[0])
|
||||
if len(vector) < 1 {
|
||||
return Vector{}
|
||||
}
|
||||
common := clientmodel.LabelSet{}
|
||||
for k, v := range vector[0].Metric.Metric {
|
||||
// TODO(julius): Should we also drop common metric names?
|
||||
if k == clientmodel.MetricNameLabel {
|
||||
continue
|
||||
}
|
||||
common[k] = v
|
||||
}
|
||||
|
||||
for _, el := range vector[1:] {
|
||||
for k, v := range common {
|
||||
if el.Metric.Metric[k] != v {
|
||||
// Deletion of map entries while iterating over them is safe.
|
||||
// From http://golang.org/ref/spec#For_statements:
|
||||
// "If map entries that have not yet been reached are deleted during
|
||||
// iteration, the corresponding iteration values will not be produced."
|
||||
delete(common, k)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
for _, el := range vector {
|
||||
for k := range el.Metric.Metric {
|
||||
if _, ok := common[k]; ok {
|
||||
el.Metric.Delete(k)
|
||||
}
|
||||
}
|
||||
}
|
||||
return vector
|
||||
}
|
||||
|
||||
// === round(vector ExprVector, toNearest=1 Scalar) Vector ===
|
||||
func funcRound(ev *evaluator, args Expressions) Value {
|
||||
// round returns a number rounded to toNearest.
|
||||
// Ties are solved by rounding up.
|
||||
toNearest := float64(1)
|
||||
if len(args) >= 2 {
|
||||
toNearest = ev.evalFloat(args[1])
|
||||
}
|
||||
// Invert as it seems to cause fewer floating point accuracy issues.
|
||||
toNearestInverse := 1.0 / toNearest
|
||||
|
||||
vector := ev.evalVector(args[0])
|
||||
for _, el := range vector {
|
||||
el.Metric.Delete(clientmodel.MetricNameLabel)
|
||||
el.Value = clientmodel.SampleValue(math.Floor(float64(el.Value)*toNearestInverse+0.5) / toNearestInverse)
|
||||
}
|
||||
return vector
|
||||
}
|
||||
|
||||
// === scalar(node ExprVector) Scalar ===
|
||||
func funcScalar(ev *evaluator, args Expressions) Value {
|
||||
v := ev.evalVector(args[0])
|
||||
if len(v) != 1 {
|
||||
return &Scalar{clientmodel.SampleValue(math.NaN()), ev.Timestamp}
|
||||
}
|
||||
return &Scalar{clientmodel.SampleValue(v[0].Value), ev.Timestamp}
|
||||
}
|
||||
|
||||
// === count_scalar(vector ExprVector) model.SampleValue ===
|
||||
func funcCountScalar(ev *evaluator, args Expressions) Value {
|
||||
return &Scalar{
|
||||
Value: clientmodel.SampleValue(len(ev.evalVector(args[0]))),
|
||||
Timestamp: ev.Timestamp,
|
||||
}
|
||||
}
|
||||
|
||||
func aggrOverTime(ev *evaluator, args Expressions, aggrFn func(metric.Values) clientmodel.SampleValue) Value {
|
||||
matrix := ev.evalMatrix(args[0])
|
||||
resultVector := Vector{}
|
||||
|
||||
for _, el := range matrix {
|
||||
if len(el.Values) == 0 {
|
||||
continue
|
||||
}
|
||||
|
||||
el.Metric.Delete(clientmodel.MetricNameLabel)
|
||||
resultVector = append(resultVector, &Sample{
|
||||
Metric: el.Metric,
|
||||
Value: aggrFn(el.Values),
|
||||
Timestamp: ev.Timestamp,
|
||||
})
|
||||
}
|
||||
return resultVector
|
||||
}
|
||||
|
||||
// === avg_over_time(matrix ExprMatrix) Vector ===
|
||||
func funcAvgOverTime(ev *evaluator, args Expressions) Value {
|
||||
return aggrOverTime(ev, args, func(values metric.Values) clientmodel.SampleValue {
|
||||
var sum clientmodel.SampleValue
|
||||
for _, v := range values {
|
||||
sum += v.Value
|
||||
}
|
||||
return sum / clientmodel.SampleValue(len(values))
|
||||
})
|
||||
}
|
||||
|
||||
// === count_over_time(matrix ExprMatrix) Vector ===
|
||||
func funcCountOverTime(ev *evaluator, args Expressions) Value {
|
||||
return aggrOverTime(ev, args, func(values metric.Values) clientmodel.SampleValue {
|
||||
return clientmodel.SampleValue(len(values))
|
||||
})
|
||||
}
|
||||
|
||||
// === floor(vector ExprVector) Vector ===
|
||||
func funcFloor(ev *evaluator, args Expressions) Value {
|
||||
vector := ev.evalVector(args[0])
|
||||
for _, el := range vector {
|
||||
el.Metric.Delete(clientmodel.MetricNameLabel)
|
||||
el.Value = clientmodel.SampleValue(math.Floor(float64(el.Value)))
|
||||
}
|
||||
return vector
|
||||
}
|
||||
|
||||
// === max_over_time(matrix ExprMatrix) Vector ===
|
||||
func funcMaxOverTime(ev *evaluator, args Expressions) Value {
|
||||
return aggrOverTime(ev, args, func(values metric.Values) clientmodel.SampleValue {
|
||||
max := math.Inf(-1)
|
||||
for _, v := range values {
|
||||
max = math.Max(max, float64(v.Value))
|
||||
}
|
||||
return clientmodel.SampleValue(max)
|
||||
})
|
||||
}
|
||||
|
||||
// === min_over_time(matrix ExprMatrix) Vector ===
|
||||
func funcMinOverTime(ev *evaluator, args Expressions) Value {
|
||||
return aggrOverTime(ev, args, func(values metric.Values) clientmodel.SampleValue {
|
||||
min := math.Inf(1)
|
||||
for _, v := range values {
|
||||
min = math.Min(min, float64(v.Value))
|
||||
}
|
||||
return clientmodel.SampleValue(min)
|
||||
})
|
||||
}
|
||||
|
||||
// === sum_over_time(matrix ExprMatrix) Vector ===
|
||||
func funcSumOverTime(ev *evaluator, args Expressions) Value {
|
||||
return aggrOverTime(ev, args, func(values metric.Values) clientmodel.SampleValue {
|
||||
var sum clientmodel.SampleValue
|
||||
for _, v := range values {
|
||||
sum += v.Value
|
||||
}
|
||||
return sum
|
||||
})
|
||||
}
|
||||
|
||||
// === abs(vector ExprVector) Vector ===
|
||||
func funcAbs(ev *evaluator, args Expressions) Value {
|
||||
vector := ev.evalVector(args[0])
|
||||
for _, el := range vector {
|
||||
el.Metric.Delete(clientmodel.MetricNameLabel)
|
||||
el.Value = clientmodel.SampleValue(math.Abs(float64(el.Value)))
|
||||
}
|
||||
return vector
|
||||
}
|
||||
|
||||
// === absent(vector ExprVector) Vector ===
|
||||
func funcAbsent(ev *evaluator, args Expressions) Value {
|
||||
if len(ev.evalVector(args[0])) > 0 {
|
||||
return Vector{}
|
||||
}
|
||||
m := clientmodel.Metric{}
|
||||
if vs, ok := args[0].(*VectorSelector); ok {
|
||||
for _, matcher := range vs.LabelMatchers {
|
||||
if matcher.Type == metric.Equal && matcher.Name != clientmodel.MetricNameLabel {
|
||||
m[matcher.Name] = matcher.Value
|
||||
}
|
||||
}
|
||||
}
|
||||
return Vector{
|
||||
&Sample{
|
||||
Metric: clientmodel.COWMetric{
|
||||
Metric: m,
|
||||
Copied: true,
|
||||
},
|
||||
Value: 1,
|
||||
Timestamp: ev.Timestamp,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
// === ceil(vector ExprVector) Vector ===
|
||||
func funcCeil(ev *evaluator, args Expressions) Value {
|
||||
vector := ev.evalVector(args[0])
|
||||
for _, el := range vector {
|
||||
el.Metric.Delete(clientmodel.MetricNameLabel)
|
||||
el.Value = clientmodel.SampleValue(math.Ceil(float64(el.Value)))
|
||||
}
|
||||
return vector
|
||||
}
|
||||
|
||||
// === exp(vector ExprVector) Vector ===
|
||||
func funcExp(ev *evaluator, args Expressions) Value {
|
||||
vector := ev.evalVector(args[0])
|
||||
for _, el := range vector {
|
||||
el.Metric.Delete(clientmodel.MetricNameLabel)
|
||||
el.Value = clientmodel.SampleValue(math.Exp(float64(el.Value)))
|
||||
}
|
||||
return vector
|
||||
}
|
||||
|
||||
// === sqrt(vector VectorNode) Vector ===
|
||||
func funcSqrt(ev *evaluator, args Expressions) Value {
|
||||
vector := ev.evalVector(args[0])
|
||||
for _, el := range vector {
|
||||
el.Metric.Delete(clientmodel.MetricNameLabel)
|
||||
el.Value = clientmodel.SampleValue(math.Sqrt(float64(el.Value)))
|
||||
}
|
||||
return vector
|
||||
}
|
||||
|
||||
// === ln(vector ExprVector) Vector ===
|
||||
func funcLn(ev *evaluator, args Expressions) Value {
|
||||
vector := ev.evalVector(args[0])
|
||||
for _, el := range vector {
|
||||
el.Metric.Delete(clientmodel.MetricNameLabel)
|
||||
el.Value = clientmodel.SampleValue(math.Log(float64(el.Value)))
|
||||
}
|
||||
return vector
|
||||
}
|
||||
|
||||
// === log2(vector ExprVector) Vector ===
|
||||
func funcLog2(ev *evaluator, args Expressions) Value {
|
||||
vector := ev.evalVector(args[0])
|
||||
for _, el := range vector {
|
||||
el.Metric.Delete(clientmodel.MetricNameLabel)
|
||||
el.Value = clientmodel.SampleValue(math.Log2(float64(el.Value)))
|
||||
}
|
||||
return vector
|
||||
}
|
||||
|
||||
// === log10(vector ExprVector) Vector ===
|
||||
func funcLog10(ev *evaluator, args Expressions) Value {
|
||||
vector := ev.evalVector(args[0])
|
||||
for _, el := range vector {
|
||||
el.Metric.Delete(clientmodel.MetricNameLabel)
|
||||
el.Value = clientmodel.SampleValue(math.Log10(float64(el.Value)))
|
||||
}
|
||||
return vector
|
||||
}
|
||||
|
||||
// === deriv(node ExprMatrix) Vector ===
|
||||
func funcDeriv(ev *evaluator, args Expressions) Value {
|
||||
resultVector := Vector{}
|
||||
matrix := ev.evalMatrix(args[0])
|
||||
|
||||
for _, samples := range matrix {
|
||||
// No sense in trying to compute a derivative without at least two points.
|
||||
// Drop this vector element.
|
||||
if len(samples.Values) < 2 {
|
||||
continue
|
||||
}
|
||||
|
||||
// Least squares.
|
||||
n := clientmodel.SampleValue(0)
|
||||
sumY := clientmodel.SampleValue(0)
|
||||
sumX := clientmodel.SampleValue(0)
|
||||
sumXY := clientmodel.SampleValue(0)
|
||||
sumX2 := clientmodel.SampleValue(0)
|
||||
for _, sample := range samples.Values {
|
||||
x := clientmodel.SampleValue(sample.Timestamp.UnixNano() / 1e9)
|
||||
n += 1.0
|
||||
sumY += sample.Value
|
||||
sumX += x
|
||||
sumXY += x * sample.Value
|
||||
sumX2 += x * x
|
||||
}
|
||||
numerator := sumXY - sumX*sumY/n
|
||||
denominator := sumX2 - (sumX*sumX)/n
|
||||
|
||||
resultValue := numerator / denominator
|
||||
|
||||
resultSample := &Sample{
|
||||
Metric: samples.Metric,
|
||||
Value: resultValue,
|
||||
Timestamp: ev.Timestamp,
|
||||
}
|
||||
resultSample.Metric.Delete(clientmodel.MetricNameLabel)
|
||||
resultVector = append(resultVector, resultSample)
|
||||
}
|
||||
return resultVector
|
||||
}
|
||||
|
||||
// === predict_linear(node ExprMatrix, k ExprScalar) Vector ===
|
||||
func funcPredictLinear(ev *evaluator, args Expressions) Value {
|
||||
vector := funcDeriv(ev, args[0:1]).(Vector)
|
||||
duration := clientmodel.SampleValue(clientmodel.SampleValue(ev.evalFloat(args[1])))
|
||||
|
||||
excludedLabels := map[clientmodel.LabelName]struct{}{
|
||||
clientmodel.MetricNameLabel: {},
|
||||
}
|
||||
|
||||
// Calculate predicted delta over the duration.
|
||||
signatureToDelta := map[uint64]clientmodel.SampleValue{}
|
||||
for _, el := range vector {
|
||||
signature := clientmodel.SignatureWithoutLabels(el.Metric.Metric, excludedLabels)
|
||||
signatureToDelta[signature] = el.Value * duration
|
||||
}
|
||||
|
||||
// add predicted delta to last value.
|
||||
matrixBounds := ev.evalMatrixBounds(args[0])
|
||||
outVec := make(Vector, 0, len(signatureToDelta))
|
||||
for _, samples := range matrixBounds {
|
||||
if len(samples.Values) < 2 {
|
||||
continue
|
||||
}
|
||||
signature := clientmodel.SignatureWithoutLabels(samples.Metric.Metric, excludedLabels)
|
||||
delta, ok := signatureToDelta[signature]
|
||||
if ok {
|
||||
samples.Metric.Delete(clientmodel.MetricNameLabel)
|
||||
outVec = append(outVec, &Sample{
|
||||
Metric: samples.Metric,
|
||||
Value: delta + samples.Values[1].Value,
|
||||
Timestamp: ev.Timestamp,
|
||||
})
|
||||
}
|
||||
}
|
||||
return outVec
|
||||
}
|
||||
|
||||
// === histogram_quantile(k ExprScalar, vector ExprVector) Vector ===
|
||||
func funcHistogramQuantile(ev *evaluator, args Expressions) Value {
|
||||
q := clientmodel.SampleValue(ev.evalFloat(args[0]))
|
||||
inVec := ev.evalVector(args[1])
|
||||
|
||||
outVec := Vector{}
|
||||
signatureToMetricWithBuckets := map[uint64]*metricWithBuckets{}
|
||||
for _, el := range inVec {
|
||||
upperBound, err := strconv.ParseFloat(
|
||||
string(el.Metric.Metric[clientmodel.BucketLabel]), 64,
|
||||
)
|
||||
if err != nil {
|
||||
// Oops, no bucket label or malformed label value. Skip.
|
||||
// TODO(beorn7): Issue a warning somehow.
|
||||
continue
|
||||
}
|
||||
signature := clientmodel.SignatureWithoutLabels(el.Metric.Metric, excludedLabels)
|
||||
mb, ok := signatureToMetricWithBuckets[signature]
|
||||
if !ok {
|
||||
el.Metric.Delete(clientmodel.BucketLabel)
|
||||
el.Metric.Delete(clientmodel.MetricNameLabel)
|
||||
mb = &metricWithBuckets{el.Metric, nil}
|
||||
signatureToMetricWithBuckets[signature] = mb
|
||||
}
|
||||
mb.buckets = append(mb.buckets, bucket{upperBound, el.Value})
|
||||
}
|
||||
|
||||
for _, mb := range signatureToMetricWithBuckets {
|
||||
outVec = append(outVec, &Sample{
|
||||
Metric: mb.metric,
|
||||
Value: clientmodel.SampleValue(quantile(q, mb.buckets)),
|
||||
Timestamp: ev.Timestamp,
|
||||
})
|
||||
}
|
||||
|
||||
return outVec
|
||||
}
|
||||
|
||||
// === resets(matrix ExprMatrix) Vector ===
|
||||
func funcResets(ev *evaluator, args Expressions) Value {
|
||||
in := ev.evalMatrix(args[0])
|
||||
out := make(Vector, 0, len(in))
|
||||
|
||||
for _, samples := range in {
|
||||
resets := 0
|
||||
prev := clientmodel.SampleValue(samples.Values[0].Value)
|
||||
for _, sample := range samples.Values[1:] {
|
||||
current := sample.Value
|
||||
if current < prev {
|
||||
resets++
|
||||
}
|
||||
prev = current
|
||||
}
|
||||
|
||||
rs := &Sample{
|
||||
Metric: samples.Metric,
|
||||
Value: clientmodel.SampleValue(resets),
|
||||
Timestamp: ev.Timestamp,
|
||||
}
|
||||
rs.Metric.Delete(clientmodel.MetricNameLabel)
|
||||
out = append(out, rs)
|
||||
}
|
||||
return out
|
||||
}
|
||||
|
||||
// === changes(matrix ExprMatrix) Vector ===
|
||||
func funcChanges(ev *evaluator, args Expressions) Value {
|
||||
in := ev.evalMatrix(args[0])
|
||||
out := make(Vector, 0, len(in))
|
||||
|
||||
for _, samples := range in {
|
||||
changes := 0
|
||||
prev := clientmodel.SampleValue(samples.Values[0].Value)
|
||||
for _, sample := range samples.Values[1:] {
|
||||
current := sample.Value
|
||||
if current != prev {
|
||||
changes++
|
||||
}
|
||||
prev = current
|
||||
}
|
||||
|
||||
rs := &Sample{
|
||||
Metric: samples.Metric,
|
||||
Value: clientmodel.SampleValue(changes),
|
||||
Timestamp: ev.Timestamp,
|
||||
}
|
||||
rs.Metric.Delete(clientmodel.MetricNameLabel)
|
||||
out = append(out, rs)
|
||||
}
|
||||
return out
|
||||
}
|
||||
|
||||
var functions = map[string]*Function{
|
||||
"abs": {
|
||||
Name: "abs",
|
||||
ArgTypes: []ExprType{ExprVector},
|
||||
ReturnType: ExprVector,
|
||||
Call: funcAbs,
|
||||
},
|
||||
"absent": {
|
||||
Name: "absent",
|
||||
ArgTypes: []ExprType{ExprVector},
|
||||
ReturnType: ExprVector,
|
||||
Call: funcAbsent,
|
||||
},
|
||||
"increase": {
|
||||
Name: "increase",
|
||||
ArgTypes: []ExprType{ExprMatrix},
|
||||
ReturnType: ExprVector,
|
||||
Call: funcIncrease,
|
||||
},
|
||||
"avg_over_time": {
|
||||
Name: "avg_over_time",
|
||||
ArgTypes: []ExprType{ExprMatrix},
|
||||
ReturnType: ExprVector,
|
||||
Call: funcAvgOverTime,
|
||||
},
|
||||
"bottomk": {
|
||||
Name: "bottomk",
|
||||
ArgTypes: []ExprType{ExprScalar, ExprVector},
|
||||
ReturnType: ExprVector,
|
||||
Call: funcBottomk,
|
||||
},
|
||||
"ceil": {
|
||||
Name: "ceil",
|
||||
ArgTypes: []ExprType{ExprVector},
|
||||
ReturnType: ExprVector,
|
||||
Call: funcCeil,
|
||||
},
|
||||
"changes": {
|
||||
Name: "changes",
|
||||
ArgTypes: []ExprType{ExprMatrix},
|
||||
ReturnType: ExprVector,
|
||||
Call: funcChanges,
|
||||
},
|
||||
"count_over_time": {
|
||||
Name: "count_over_time",
|
||||
ArgTypes: []ExprType{ExprMatrix},
|
||||
ReturnType: ExprVector,
|
||||
Call: funcCountOverTime,
|
||||
},
|
||||
"count_scalar": {
|
||||
Name: "count_scalar",
|
||||
ArgTypes: []ExprType{ExprVector},
|
||||
ReturnType: ExprScalar,
|
||||
Call: funcCountScalar,
|
||||
},
|
||||
"delta": {
|
||||
Name: "delta",
|
||||
ArgTypes: []ExprType{ExprMatrix, ExprScalar},
|
||||
OptionalArgs: 1, // The 2nd argument is deprecated.
|
||||
ReturnType: ExprVector,
|
||||
Call: funcDelta,
|
||||
},
|
||||
"deriv": {
|
||||
Name: "deriv",
|
||||
ArgTypes: []ExprType{ExprMatrix},
|
||||
ReturnType: ExprVector,
|
||||
Call: funcDeriv,
|
||||
},
|
||||
"drop_common_labels": {
|
||||
Name: "drop_common_labels",
|
||||
ArgTypes: []ExprType{ExprVector},
|
||||
ReturnType: ExprVector,
|
||||
Call: funcDropCommonLabels,
|
||||
},
|
||||
"exp": {
|
||||
Name: "exp",
|
||||
ArgTypes: []ExprType{ExprVector},
|
||||
ReturnType: ExprVector,
|
||||
Call: funcExp,
|
||||
},
|
||||
"floor": {
|
||||
Name: "floor",
|
||||
ArgTypes: []ExprType{ExprVector},
|
||||
ReturnType: ExprVector,
|
||||
Call: funcFloor,
|
||||
},
|
||||
"histogram_quantile": {
|
||||
Name: "histogram_quantile",
|
||||
ArgTypes: []ExprType{ExprScalar, ExprVector},
|
||||
ReturnType: ExprVector,
|
||||
Call: funcHistogramQuantile,
|
||||
},
|
||||
"ln": {
|
||||
Name: "ln",
|
||||
ArgTypes: []ExprType{ExprVector},
|
||||
ReturnType: ExprVector,
|
||||
Call: funcLn,
|
||||
},
|
||||
"log10": {
|
||||
Name: "log10",
|
||||
ArgTypes: []ExprType{ExprVector},
|
||||
ReturnType: ExprVector,
|
||||
Call: funcLog10,
|
||||
},
|
||||
"log2": {
|
||||
Name: "log2",
|
||||
ArgTypes: []ExprType{ExprVector},
|
||||
ReturnType: ExprVector,
|
||||
Call: funcLog2,
|
||||
},
|
||||
"max_over_time": {
|
||||
Name: "max_over_time",
|
||||
ArgTypes: []ExprType{ExprMatrix},
|
||||
ReturnType: ExprVector,
|
||||
Call: funcMaxOverTime,
|
||||
},
|
||||
"min_over_time": {
|
||||
Name: "min_over_time",
|
||||
ArgTypes: []ExprType{ExprMatrix},
|
||||
ReturnType: ExprVector,
|
||||
Call: funcMinOverTime,
|
||||
},
|
||||
"predict_linear": {
|
||||
Name: "predict_linear",
|
||||
ArgTypes: []ExprType{ExprMatrix, ExprScalar},
|
||||
ReturnType: ExprVector,
|
||||
Call: funcPredictLinear,
|
||||
},
|
||||
"rate": {
|
||||
Name: "rate",
|
||||
ArgTypes: []ExprType{ExprMatrix},
|
||||
ReturnType: ExprVector,
|
||||
Call: funcRate,
|
||||
},
|
||||
"resets": {
|
||||
Name: "resets",
|
||||
ArgTypes: []ExprType{ExprMatrix},
|
||||
ReturnType: ExprVector,
|
||||
Call: funcResets,
|
||||
},
|
||||
"round": {
|
||||
Name: "round",
|
||||
ArgTypes: []ExprType{ExprVector, ExprScalar},
|
||||
OptionalArgs: 1,
|
||||
ReturnType: ExprVector,
|
||||
Call: funcRound,
|
||||
},
|
||||
"scalar": {
|
||||
Name: "scalar",
|
||||
ArgTypes: []ExprType{ExprVector},
|
||||
ReturnType: ExprScalar,
|
||||
Call: funcScalar,
|
||||
},
|
||||
"sort": {
|
||||
Name: "sort",
|
||||
ArgTypes: []ExprType{ExprVector},
|
||||
ReturnType: ExprVector,
|
||||
Call: funcSort,
|
||||
},
|
||||
"sort_desc": {
|
||||
Name: "sort_desc",
|
||||
ArgTypes: []ExprType{ExprVector},
|
||||
ReturnType: ExprVector,
|
||||
Call: funcSortDesc,
|
||||
},
|
||||
"sqrt": {
|
||||
Name: "sqrt",
|
||||
ArgTypes: []ExprType{ExprVector},
|
||||
ReturnType: ExprVector,
|
||||
Call: funcSqrt,
|
||||
},
|
||||
"sum_over_time": {
|
||||
Name: "sum_over_time",
|
||||
ArgTypes: []ExprType{ExprMatrix},
|
||||
ReturnType: ExprVector,
|
||||
Call: funcSumOverTime,
|
||||
},
|
||||
"time": {
|
||||
Name: "time",
|
||||
ArgTypes: []ExprType{},
|
||||
ReturnType: ExprScalar,
|
||||
Call: funcTime,
|
||||
},
|
||||
"topk": {
|
||||
Name: "topk",
|
||||
ArgTypes: []ExprType{ExprScalar, ExprVector},
|
||||
ReturnType: ExprVector,
|
||||
Call: funcTopk,
|
||||
},
|
||||
}
|
||||
|
||||
// getFunction returns a predefined Function object for the given name.
|
||||
func getFunction(name string) (*Function, bool) {
|
||||
function, ok := functions[name]
|
||||
return function, ok
|
||||
}
|
||||
|
||||
type vectorByValueHeap Vector
|
||||
|
||||
func (s vectorByValueHeap) Len() int {
|
||||
return len(s)
|
||||
}
|
||||
|
||||
func (s vectorByValueHeap) Less(i, j int) bool {
|
||||
if math.IsNaN(float64(s[i].Value)) {
|
||||
return true
|
||||
}
|
||||
return s[i].Value < s[j].Value
|
||||
}
|
||||
|
||||
func (s vectorByValueHeap) Swap(i, j int) {
|
||||
s[i], s[j] = s[j], s[i]
|
||||
}
|
||||
|
||||
func (s *vectorByValueHeap) Push(x interface{}) {
|
||||
*s = append(*s, x.(*Sample))
|
||||
}
|
||||
|
||||
func (s *vectorByValueHeap) Pop() interface{} {
|
||||
old := *s
|
||||
n := len(old)
|
||||
el := old[n-1]
|
||||
*s = old[0 : n-1]
|
||||
return el
|
||||
}
|
||||
|
||||
type reverseHeap struct {
|
||||
heap.Interface
|
||||
}
|
||||
|
||||
func (s reverseHeap) Less(i, j int) bool {
|
||||
return s.Interface.Less(j, i)
|
||||
}
|
754
prom/promql/lex.go
Normal file
754
prom/promql/lex.go
Normal file
@ -0,0 +1,754 @@
|
||||
// Copyright 2015 The Prometheus Authors
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package promql
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"strings"
|
||||
"unicode/utf8"
|
||||
)
|
||||
|
||||
// item represents a token or text string returned from the scanner.
|
||||
type item struct {
|
||||
typ itemType // The type of this item.
|
||||
pos Pos // The starting position, in bytes, of this item in the input string.
|
||||
val string // The value of this item.
|
||||
}
|
||||
|
||||
// String returns a descriptive string for the item.
|
||||
func (i item) String() string {
|
||||
switch {
|
||||
case i.typ == itemEOF:
|
||||
return "EOF"
|
||||
case i.typ == itemError:
|
||||
return i.val
|
||||
case i.typ == itemIdentifier || i.typ == itemMetricIdentifier:
|
||||
return fmt.Sprintf("%q", i.val)
|
||||
case i.typ.isKeyword():
|
||||
return fmt.Sprintf("<%s>", i.val)
|
||||
case i.typ.isOperator():
|
||||
return fmt.Sprintf("<op:%s>", i.val)
|
||||
case i.typ.isAggregator():
|
||||
return fmt.Sprintf("<aggr:%s>", i.val)
|
||||
case len(i.val) > 10:
|
||||
return fmt.Sprintf("%.10q...", i.val)
|
||||
}
|
||||
return fmt.Sprintf("%q", i.val)
|
||||
}
|
||||
|
||||
// isOperator returns true if the item corresponds to a logical or arithmetic operator.
|
||||
// Returns false otherwise.
|
||||
func (i itemType) isOperator() bool { return i > operatorsStart && i < operatorsEnd }
|
||||
|
||||
// isAggregator returns true if the item belongs to the aggregator functions.
|
||||
// Returns false otherwise
|
||||
func (i itemType) isAggregator() bool { return i > aggregatorsStart && i < aggregatorsEnd }
|
||||
|
||||
// isKeyword returns true if the item corresponds to a keyword.
|
||||
// Returns false otherwise.
|
||||
func (i itemType) isKeyword() bool { return i > keywordsStart && i < keywordsEnd }
|
||||
|
||||
// Constants for operator precedence in expressions.
|
||||
//
|
||||
const LowestPrec = 0 // Non-operators.
|
||||
|
||||
// Precedence returns the operator precedence of the binary
|
||||
// operator op. If op is not a binary operator, the result
|
||||
// is LowestPrec.
|
||||
func (i itemType) precedence() int {
|
||||
switch i {
|
||||
case itemLOR:
|
||||
return 1
|
||||
case itemLAND:
|
||||
return 2
|
||||
case itemEQL, itemNEQ, itemLTE, itemLSS, itemGTE, itemGTR:
|
||||
return 3
|
||||
case itemADD, itemSUB:
|
||||
return 4
|
||||
case itemMUL, itemDIV, itemMOD:
|
||||
return 5
|
||||
default:
|
||||
return LowestPrec
|
||||
}
|
||||
}
|
||||
|
||||
type itemType int
|
||||
|
||||
const (
|
||||
itemError itemType = iota // Error occurred, value is error message
|
||||
itemEOF
|
||||
itemComment
|
||||
itemIdentifier
|
||||
itemMetricIdentifier
|
||||
itemLeftParen
|
||||
itemRightParen
|
||||
itemLeftBrace
|
||||
itemRightBrace
|
||||
itemLeftBracket
|
||||
itemRightBracket
|
||||
itemComma
|
||||
itemAssign
|
||||
itemSemicolon
|
||||
itemString
|
||||
itemNumber
|
||||
itemDuration
|
||||
itemBlank
|
||||
itemTimes
|
||||
|
||||
operatorsStart
|
||||
// Operators.
|
||||
itemSUB
|
||||
itemADD
|
||||
itemMUL
|
||||
itemMOD
|
||||
itemDIV
|
||||
itemLAND
|
||||
itemLOR
|
||||
itemEQL
|
||||
itemNEQ
|
||||
itemLTE
|
||||
itemLSS
|
||||
itemGTE
|
||||
itemGTR
|
||||
itemEQLRegex
|
||||
itemNEQRegex
|
||||
operatorsEnd
|
||||
|
||||
aggregatorsStart
|
||||
// Aggregators.
|
||||
itemAvg
|
||||
itemCount
|
||||
itemSum
|
||||
itemMin
|
||||
itemMax
|
||||
itemStddev
|
||||
itemStdvar
|
||||
aggregatorsEnd
|
||||
|
||||
keywordsStart
|
||||
// Keywords.
|
||||
itemAlert
|
||||
itemIf
|
||||
itemFor
|
||||
itemWith
|
||||
itemSummary
|
||||
itemRunbook
|
||||
itemDescription
|
||||
itemKeepCommon
|
||||
itemOffset
|
||||
itemBy
|
||||
itemOn
|
||||
itemGroupLeft
|
||||
itemGroupRight
|
||||
keywordsEnd
|
||||
)
|
||||
|
||||
var key = map[string]itemType{
|
||||
// Operators.
|
||||
"and": itemLAND,
|
||||
"or": itemLOR,
|
||||
|
||||
// Aggregators.
|
||||
"sum": itemSum,
|
||||
"avg": itemAvg,
|
||||
"count": itemCount,
|
||||
"min": itemMin,
|
||||
"max": itemMax,
|
||||
"stddev": itemStddev,
|
||||
"stdvar": itemStdvar,
|
||||
|
||||
// Keywords.
|
||||
"alert": itemAlert,
|
||||
"if": itemIf,
|
||||
"for": itemFor,
|
||||
"with": itemWith,
|
||||
"summary": itemSummary,
|
||||
"runbook": itemRunbook,
|
||||
"description": itemDescription,
|
||||
"offset": itemOffset,
|
||||
"by": itemBy,
|
||||
"keeping_extra": itemKeepCommon,
|
||||
"keep_common": itemKeepCommon,
|
||||
"on": itemOn,
|
||||
"group_left": itemGroupLeft,
|
||||
"group_right": itemGroupRight,
|
||||
}
|
||||
|
||||
// These are the default string representations for common items. It does not
|
||||
// imply that those are the only character sequences that can be lexed to such an item.
|
||||
var itemTypeStr = map[itemType]string{
|
||||
itemLeftParen: "(",
|
||||
itemRightParen: ")",
|
||||
itemLeftBrace: "{",
|
||||
itemRightBrace: "}",
|
||||
itemLeftBracket: "[",
|
||||
itemRightBracket: "]",
|
||||
itemComma: ",",
|
||||
itemAssign: "=",
|
||||
itemSemicolon: ";",
|
||||
itemBlank: "_",
|
||||
itemTimes: "x",
|
||||
|
||||
itemSUB: "-",
|
||||
itemADD: "+",
|
||||
itemMUL: "*",
|
||||
itemMOD: "%",
|
||||
itemDIV: "/",
|
||||
itemEQL: "==",
|
||||
itemNEQ: "!=",
|
||||
itemLTE: "<=",
|
||||
itemLSS: "<",
|
||||
itemGTE: ">=",
|
||||
itemGTR: ">",
|
||||
itemEQLRegex: "=~",
|
||||
itemNEQRegex: "!~",
|
||||
}
|
||||
|
||||
func init() {
|
||||
// Add keywords to item type strings.
|
||||
for s, ty := range key {
|
||||
itemTypeStr[ty] = s
|
||||
}
|
||||
// Special numbers.
|
||||
key["inf"] = itemNumber
|
||||
key["nan"] = itemNumber
|
||||
}
|
||||
|
||||
func (t itemType) String() string {
|
||||
if s, ok := itemTypeStr[t]; ok {
|
||||
return s
|
||||
}
|
||||
return fmt.Sprintf("<item %d>", t)
|
||||
}
|
||||
|
||||
func (i item) desc() string {
|
||||
if _, ok := itemTypeStr[i.typ]; ok {
|
||||
return i.String()
|
||||
}
|
||||
if i.typ == itemEOF {
|
||||
return i.typ.desc()
|
||||
}
|
||||
return fmt.Sprintf("%s %s", i.typ.desc(), i)
|
||||
}
|
||||
|
||||
func (t itemType) desc() string {
|
||||
switch t {
|
||||
case itemError:
|
||||
return "error"
|
||||
case itemEOF:
|
||||
return "end of input"
|
||||
case itemComment:
|
||||
return "comment"
|
||||
case itemIdentifier:
|
||||
return "identifier"
|
||||
case itemMetricIdentifier:
|
||||
return "metric identifier"
|
||||
case itemString:
|
||||
return "string"
|
||||
case itemNumber:
|
||||
return "number"
|
||||
case itemDuration:
|
||||
return "duration"
|
||||
}
|
||||
return fmt.Sprintf("%q", t)
|
||||
}
|
||||
|
||||
const eof = -1
|
||||
|
||||
// stateFn represents the state of the scanner as a function that returns the next state.
|
||||
type stateFn func(*lexer) stateFn
|
||||
|
||||
// Pos is the position in a string.
|
||||
type Pos int
|
||||
|
||||
// lexer holds the state of the scanner.
|
||||
type lexer struct {
|
||||
input string // The string being scanned.
|
||||
state stateFn // The next lexing function to enter.
|
||||
pos Pos // Current position in the input.
|
||||
start Pos // Start position of this item.
|
||||
width Pos // Width of last rune read from input.
|
||||
lastPos Pos // Position of most recent item returned by nextItem.
|
||||
items chan item // Channel of scanned items.
|
||||
|
||||
parenDepth int // Nesting depth of ( ) exprs.
|
||||
braceOpen bool // Whether a { is opened.
|
||||
bracketOpen bool // Whether a [ is opened.
|
||||
stringOpen rune // Quote rune of the string currently being read.
|
||||
|
||||
// seriesDesc is set when a series description for the testing
|
||||
// language is lexed.
|
||||
seriesDesc bool
|
||||
}
|
||||
|
||||
// next returns the next rune in the input.
|
||||
func (l *lexer) next() rune {
|
||||
if int(l.pos) >= len(l.input) {
|
||||
l.width = 0
|
||||
return eof
|
||||
}
|
||||
r, w := utf8.DecodeRuneInString(l.input[l.pos:])
|
||||
l.width = Pos(w)
|
||||
l.pos += l.width
|
||||
return r
|
||||
}
|
||||
|
||||
// peek returns but does not consume the next rune in the input.
|
||||
func (l *lexer) peek() rune {
|
||||
r := l.next()
|
||||
l.backup()
|
||||
return r
|
||||
}
|
||||
|
||||
// backup steps back one rune. Can only be called once per call of next.
|
||||
func (l *lexer) backup() {
|
||||
l.pos -= l.width
|
||||
}
|
||||
|
||||
// emit passes an item back to the client.
|
||||
func (l *lexer) emit(t itemType) {
|
||||
l.items <- item{t, l.start, l.input[l.start:l.pos]}
|
||||
l.start = l.pos
|
||||
}
|
||||
|
||||
// ignore skips over the pending input before this point.
|
||||
func (l *lexer) ignore() {
|
||||
l.start = l.pos
|
||||
}
|
||||
|
||||
// accept consumes the next rune if it's from the valid set.
|
||||
func (l *lexer) accept(valid string) bool {
|
||||
if strings.IndexRune(valid, l.next()) >= 0 {
|
||||
return true
|
||||
}
|
||||
l.backup()
|
||||
return false
|
||||
}
|
||||
|
||||
// acceptRun consumes a run of runes from the valid set.
|
||||
func (l *lexer) acceptRun(valid string) {
|
||||
for strings.IndexRune(valid, l.next()) >= 0 {
|
||||
// consume
|
||||
}
|
||||
l.backup()
|
||||
}
|
||||
|
||||
// lineNumber reports which line we're on, based on the position of
|
||||
// the previous item returned by nextItem. Doing it this way
|
||||
// means we don't have to worry about peek double counting.
|
||||
func (l *lexer) lineNumber() int {
|
||||
return 1 + strings.Count(l.input[:l.lastPos], "\n")
|
||||
}
|
||||
|
||||
// linePosition reports at which character in the current line
|
||||
// we are on.
|
||||
func (l *lexer) linePosition() int {
|
||||
lb := strings.LastIndex(l.input[:l.lastPos], "\n")
|
||||
if lb == -1 {
|
||||
return 1 + int(l.lastPos)
|
||||
}
|
||||
return 1 + int(l.lastPos) - lb
|
||||
}
|
||||
|
||||
// errorf returns an error token and terminates the scan by passing
|
||||
// back a nil pointer that will be the next state, terminating l.nextItem.
|
||||
func (l *lexer) errorf(format string, args ...interface{}) stateFn {
|
||||
l.items <- item{itemError, l.start, fmt.Sprintf(format, args...)}
|
||||
return nil
|
||||
}
|
||||
|
||||
// nextItem returns the next item from the input.
|
||||
func (l *lexer) nextItem() item {
|
||||
item := <-l.items
|
||||
l.lastPos = item.pos
|
||||
return item
|
||||
}
|
||||
|
||||
// lex creates a new scanner for the input string.
|
||||
func lex(input string) *lexer {
|
||||
l := &lexer{
|
||||
input: input,
|
||||
items: make(chan item),
|
||||
}
|
||||
go l.run()
|
||||
return l
|
||||
}
|
||||
|
||||
// run runs the state machine for the lexer.
|
||||
func (l *lexer) run() {
|
||||
for l.state = lexStatements; l.state != nil; {
|
||||
l.state = l.state(l)
|
||||
}
|
||||
close(l.items)
|
||||
}
|
||||
|
||||
// lineComment is the character that starts a line comment.
|
||||
const lineComment = "#"
|
||||
|
||||
// lexStatements is the top-level state for lexing.
|
||||
func lexStatements(l *lexer) stateFn {
|
||||
if l.braceOpen {
|
||||
return lexInsideBraces
|
||||
}
|
||||
if strings.HasPrefix(l.input[l.pos:], lineComment) {
|
||||
return lexLineComment
|
||||
}
|
||||
|
||||
switch r := l.next(); {
|
||||
case r == eof:
|
||||
if l.parenDepth != 0 {
|
||||
return l.errorf("unclosed left parenthesis")
|
||||
} else if l.bracketOpen {
|
||||
return l.errorf("unclosed left bracket")
|
||||
}
|
||||
l.emit(itemEOF)
|
||||
return nil
|
||||
case r == ',':
|
||||
l.emit(itemComma)
|
||||
case isSpace(r):
|
||||
return lexSpace
|
||||
case r == '*':
|
||||
l.emit(itemMUL)
|
||||
case r == '/':
|
||||
l.emit(itemDIV)
|
||||
case r == '%':
|
||||
l.emit(itemMOD)
|
||||
case r == '+':
|
||||
l.emit(itemADD)
|
||||
case r == '-':
|
||||
l.emit(itemSUB)
|
||||
case r == '=':
|
||||
if t := l.peek(); t == '=' {
|
||||
l.next()
|
||||
l.emit(itemEQL)
|
||||
} else if t == '~' {
|
||||
return l.errorf("unexpected character after '=': %q", t)
|
||||
} else {
|
||||
l.emit(itemAssign)
|
||||
}
|
||||
case r == '!':
|
||||
if t := l.next(); t == '=' {
|
||||
l.emit(itemNEQ)
|
||||
} else {
|
||||
return l.errorf("unexpected character after '!': %q", t)
|
||||
}
|
||||
case r == '<':
|
||||
if t := l.peek(); t == '=' {
|
||||
l.next()
|
||||
l.emit(itemLTE)
|
||||
} else {
|
||||
l.emit(itemLSS)
|
||||
}
|
||||
case r == '>':
|
||||
if t := l.peek(); t == '=' {
|
||||
l.next()
|
||||
l.emit(itemGTE)
|
||||
} else {
|
||||
l.emit(itemGTR)
|
||||
}
|
||||
case isDigit(r) || (r == '.' && isDigit(l.peek())):
|
||||
l.backup()
|
||||
return lexNumberOrDuration
|
||||
case r == '"' || r == '\'':
|
||||
l.stringOpen = r
|
||||
return lexString
|
||||
case isAlpha(r) || r == ':':
|
||||
l.backup()
|
||||
return lexKeywordOrIdentifier
|
||||
case r == '(':
|
||||
l.emit(itemLeftParen)
|
||||
l.parenDepth++
|
||||
return lexStatements
|
||||
case r == ')':
|
||||
l.emit(itemRightParen)
|
||||
l.parenDepth--
|
||||
if l.parenDepth < 0 {
|
||||
return l.errorf("unexpected right parenthesis %q", r)
|
||||
}
|
||||
return lexStatements
|
||||
case r == '{':
|
||||
l.emit(itemLeftBrace)
|
||||
l.braceOpen = true
|
||||
return lexInsideBraces(l)
|
||||
case r == '[':
|
||||
if l.bracketOpen {
|
||||
return l.errorf("unexpected left bracket %q", r)
|
||||
}
|
||||
l.emit(itemLeftBracket)
|
||||
l.bracketOpen = true
|
||||
return lexDuration
|
||||
case r == ']':
|
||||
if !l.bracketOpen {
|
||||
return l.errorf("unexpected right bracket %q", r)
|
||||
}
|
||||
l.emit(itemRightBracket)
|
||||
l.bracketOpen = false
|
||||
|
||||
default:
|
||||
return l.errorf("unexpected character: %q", r)
|
||||
}
|
||||
return lexStatements
|
||||
}
|
||||
|
||||
// lexInsideBraces scans the inside of a vector selector. Keywords are ignored and
|
||||
// scanned as identifiers.
|
||||
func lexInsideBraces(l *lexer) stateFn {
|
||||
if strings.HasPrefix(l.input[l.pos:], lineComment) {
|
||||
return lexLineComment
|
||||
}
|
||||
|
||||
switch r := l.next(); {
|
||||
case r == eof:
|
||||
return l.errorf("unexpected end of input inside braces")
|
||||
case isSpace(r):
|
||||
return lexSpace
|
||||
case isAlpha(r):
|
||||
l.backup()
|
||||
return lexIdentifier
|
||||
case r == ',':
|
||||
l.emit(itemComma)
|
||||
case r == '"' || r == '\'':
|
||||
l.stringOpen = r
|
||||
return lexString
|
||||
case r == '=':
|
||||
if l.next() == '~' {
|
||||
l.emit(itemEQLRegex)
|
||||
break
|
||||
}
|
||||
l.backup()
|
||||
l.emit(itemEQL)
|
||||
case r == '!':
|
||||
switch nr := l.next(); {
|
||||
case nr == '~':
|
||||
l.emit(itemNEQRegex)
|
||||
case nr == '=':
|
||||
l.emit(itemNEQ)
|
||||
default:
|
||||
return l.errorf("unexpected character after '!' inside braces: %q", nr)
|
||||
}
|
||||
case r == '{':
|
||||
return l.errorf("unexpected left brace %q", r)
|
||||
case r == '}':
|
||||
l.emit(itemRightBrace)
|
||||
l.braceOpen = false
|
||||
|
||||
if l.seriesDesc {
|
||||
return lexValueSequence
|
||||
}
|
||||
return lexStatements
|
||||
default:
|
||||
return l.errorf("unexpected character inside braces: %q", r)
|
||||
}
|
||||
return lexInsideBraces
|
||||
}
|
||||
|
||||
// lexValueSequence scans a value sequence of a series description.
|
||||
func lexValueSequence(l *lexer) stateFn {
|
||||
switch r := l.next(); {
|
||||
case r == eof:
|
||||
return lexStatements
|
||||
case isSpace(r):
|
||||
lexSpace(l)
|
||||
case r == '+':
|
||||
l.emit(itemADD)
|
||||
case r == '-':
|
||||
l.emit(itemSUB)
|
||||
case r == 'x':
|
||||
l.emit(itemTimes)
|
||||
case r == '_':
|
||||
l.emit(itemBlank)
|
||||
case isDigit(r) || (r == '.' && isDigit(l.peek())):
|
||||
l.backup()
|
||||
lexNumber(l)
|
||||
case isAlpha(r):
|
||||
l.backup()
|
||||
// We might lex invalid items here but this will be caught by the parser.
|
||||
return lexKeywordOrIdentifier
|
||||
default:
|
||||
return l.errorf("unexpected character in series sequence: %q", r)
|
||||
}
|
||||
return lexValueSequence
|
||||
}
|
||||
|
||||
// lexString scans a quoted string. The initial quote has already been seen.
|
||||
func lexString(l *lexer) stateFn {
|
||||
Loop:
|
||||
for {
|
||||
switch l.next() {
|
||||
case '\\':
|
||||
if r := l.next(); r != eof && r != '\n' {
|
||||
break
|
||||
}
|
||||
fallthrough
|
||||
case eof, '\n':
|
||||
return l.errorf("unterminated quoted string")
|
||||
case l.stringOpen:
|
||||
break Loop
|
||||
}
|
||||
}
|
||||
l.emit(itemString)
|
||||
return lexStatements
|
||||
}
|
||||
|
||||
// lexSpace scans a run of space characters. One space has already been seen.
|
||||
func lexSpace(l *lexer) stateFn {
|
||||
for isSpace(l.peek()) {
|
||||
l.next()
|
||||
}
|
||||
l.ignore()
|
||||
return lexStatements
|
||||
}
|
||||
|
||||
// lexLineComment scans a line comment. Left comment marker is known to be present.
|
||||
func lexLineComment(l *lexer) stateFn {
|
||||
l.pos += Pos(len(lineComment))
|
||||
for r := l.next(); !isEndOfLine(r) && r != eof; {
|
||||
r = l.next()
|
||||
}
|
||||
l.backup()
|
||||
l.emit(itemComment)
|
||||
return lexStatements
|
||||
}
|
||||
|
||||
func lexDuration(l *lexer) stateFn {
|
||||
if l.scanNumber() {
|
||||
return l.errorf("missing unit character in duration")
|
||||
}
|
||||
// Next two chars must be a valid unit and a non-alphanumeric.
|
||||
if l.accept("smhdwy") {
|
||||
if isAlphaNumeric(l.next()) {
|
||||
return l.errorf("bad duration syntax: %q", l.input[l.start:l.pos])
|
||||
}
|
||||
l.backup()
|
||||
l.emit(itemDuration)
|
||||
return lexStatements
|
||||
}
|
||||
return l.errorf("bad duration syntax: %q", l.input[l.start:l.pos])
|
||||
}
|
||||
|
||||
// lexNumber scans a number: decimal, hex, oct or float.
|
||||
func lexNumber(l *lexer) stateFn {
|
||||
if !l.scanNumber() {
|
||||
return l.errorf("bad number syntax: %q", l.input[l.start:l.pos])
|
||||
}
|
||||
l.emit(itemNumber)
|
||||
return lexStatements
|
||||
}
|
||||
|
||||
// lexNumberOrDuration scans a number or a duration item.
|
||||
func lexNumberOrDuration(l *lexer) stateFn {
|
||||
if l.scanNumber() {
|
||||
l.emit(itemNumber)
|
||||
return lexStatements
|
||||
}
|
||||
// Next two chars must be a valid unit and a non-alphanumeric.
|
||||
if l.accept("smhdwy") {
|
||||
if isAlphaNumeric(l.next()) {
|
||||
return l.errorf("bad number or duration syntax: %q", l.input[l.start:l.pos])
|
||||
}
|
||||
l.backup()
|
||||
l.emit(itemDuration)
|
||||
return lexStatements
|
||||
}
|
||||
return l.errorf("bad number or duration syntax: %q", l.input[l.start:l.pos])
|
||||
}
|
||||
|
||||
// scanNumber scans numbers of different formats. The scanned item is
|
||||
// not necessarily a valid number. This case is caught by the parser.
|
||||
func (l *lexer) scanNumber() bool {
|
||||
digits := "0123456789"
|
||||
// Disallow hexadecimal in series descriptions as the syntax is ambiguous.
|
||||
if !l.seriesDesc && l.accept("0") && l.accept("xX") {
|
||||
digits = "0123456789abcdefABCDEF"
|
||||
}
|
||||
l.acceptRun(digits)
|
||||
if l.accept(".") {
|
||||
l.acceptRun(digits)
|
||||
}
|
||||
if l.accept("eE") {
|
||||
l.accept("+-")
|
||||
l.acceptRun("0123456789")
|
||||
}
|
||||
// Next thing must not be alphanumeric unless it's the times token
|
||||
// for series repetitions.
|
||||
if r := l.peek(); (l.seriesDesc && r == 'x') || !isAlphaNumeric(r) {
|
||||
return true
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
// lexIdentifier scans an alphanumeric identifier. The next character
|
||||
// is known to be a letter.
|
||||
func lexIdentifier(l *lexer) stateFn {
|
||||
for isAlphaNumeric(l.next()) {
|
||||
// absorb
|
||||
}
|
||||
l.backup()
|
||||
l.emit(itemIdentifier)
|
||||
return lexStatements
|
||||
}
|
||||
|
||||
// lexKeywordOrIdentifier scans an alphanumeric identifier which may contain
|
||||
// a colon rune. If the identifier is a keyword the respective keyword item
|
||||
// is scanned.
|
||||
func lexKeywordOrIdentifier(l *lexer) stateFn {
|
||||
Loop:
|
||||
for {
|
||||
switch r := l.next(); {
|
||||
case isAlphaNumeric(r) || r == ':':
|
||||
// absorb.
|
||||
default:
|
||||
l.backup()
|
||||
word := l.input[l.start:l.pos]
|
||||
if kw, ok := key[strings.ToLower(word)]; ok {
|
||||
l.emit(kw)
|
||||
} else if !strings.Contains(word, ":") {
|
||||
l.emit(itemIdentifier)
|
||||
} else {
|
||||
l.emit(itemMetricIdentifier)
|
||||
}
|
||||
break Loop
|
||||
}
|
||||
}
|
||||
if l.seriesDesc && l.peek() != '{' {
|
||||
return lexValueSequence
|
||||
}
|
||||
return lexStatements
|
||||
}
|
||||
|
||||
func isSpace(r rune) bool {
|
||||
return r == ' ' || r == '\t' || r == '\n' || r == '\r'
|
||||
}
|
||||
|
||||
// isEndOfLine reports whether r is an end-of-line character.
|
||||
func isEndOfLine(r rune) bool {
|
||||
return r == '\r' || r == '\n'
|
||||
}
|
||||
|
||||
// isAlphaNumeric reports whether r is an alphabetic, digit, or underscore.
|
||||
func isAlphaNumeric(r rune) bool {
|
||||
return isAlpha(r) || isDigit(r)
|
||||
}
|
||||
|
||||
// isDigit reports whether r is a digit. Note: we cannot use unicode.IsDigit()
|
||||
// instead because that also classifies non-Latin digits as digits. See
|
||||
// https://github.com/prometheus/prometheus/issues/939.
|
||||
func isDigit(r rune) bool {
|
||||
return '0' <= r && r <= '9'
|
||||
}
|
||||
|
||||
// isAlpha reports whether r is an alphabetic or underscore.
|
||||
func isAlpha(r rune) bool {
|
||||
return r == '_' || ('a' <= r && r <= 'z') || ('A' <= r && r <= 'Z')
|
||||
}
|
454
prom/promql/lex_test.go
Normal file
454
prom/promql/lex_test.go
Normal file
@ -0,0 +1,454 @@
|
||||
// Copyright 2015 The Prometheus Authors
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package promql
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"reflect"
|
||||
"testing"
|
||||
)
|
||||
|
||||
var tests = []struct {
|
||||
input string
|
||||
expected []item
|
||||
fail bool
|
||||
seriesDesc bool // Whether to lex a series description.
|
||||
}{
|
||||
// Test common stuff.
|
||||
{
|
||||
input: ",",
|
||||
expected: []item{{itemComma, 0, ","}},
|
||||
}, {
|
||||
input: "()",
|
||||
expected: []item{{itemLeftParen, 0, `(`}, {itemRightParen, 1, `)`}},
|
||||
}, {
|
||||
input: "{}",
|
||||
expected: []item{{itemLeftBrace, 0, `{`}, {itemRightBrace, 1, `}`}},
|
||||
}, {
|
||||
input: "[5m]",
|
||||
expected: []item{
|
||||
{itemLeftBracket, 0, `[`},
|
||||
{itemDuration, 1, `5m`},
|
||||
{itemRightBracket, 3, `]`},
|
||||
},
|
||||
}, {
|
||||
input: "\r\n\r",
|
||||
expected: []item{},
|
||||
},
|
||||
// Test numbers.
|
||||
{
|
||||
input: "1",
|
||||
expected: []item{{itemNumber, 0, "1"}},
|
||||
}, {
|
||||
input: "4.23",
|
||||
expected: []item{{itemNumber, 0, "4.23"}},
|
||||
}, {
|
||||
input: ".3",
|
||||
expected: []item{{itemNumber, 0, ".3"}},
|
||||
}, {
|
||||
input: "5.",
|
||||
expected: []item{{itemNumber, 0, "5."}},
|
||||
}, {
|
||||
input: "NaN",
|
||||
expected: []item{{itemNumber, 0, "NaN"}},
|
||||
}, {
|
||||
input: "nAN",
|
||||
expected: []item{{itemNumber, 0, "nAN"}},
|
||||
}, {
|
||||
input: "NaN 123",
|
||||
expected: []item{{itemNumber, 0, "NaN"}, {itemNumber, 4, "123"}},
|
||||
}, {
|
||||
input: "NaN123",
|
||||
expected: []item{{itemIdentifier, 0, "NaN123"}},
|
||||
}, {
|
||||
input: "iNf",
|
||||
expected: []item{{itemNumber, 0, "iNf"}},
|
||||
}, {
|
||||
input: "Inf",
|
||||
expected: []item{{itemNumber, 0, "Inf"}},
|
||||
}, {
|
||||
input: "+Inf",
|
||||
expected: []item{{itemADD, 0, "+"}, {itemNumber, 1, "Inf"}},
|
||||
}, {
|
||||
input: "+Inf 123",
|
||||
expected: []item{{itemADD, 0, "+"}, {itemNumber, 1, "Inf"}, {itemNumber, 5, "123"}},
|
||||
}, {
|
||||
input: "-Inf",
|
||||
expected: []item{{itemSUB, 0, "-"}, {itemNumber, 1, "Inf"}},
|
||||
}, {
|
||||
input: "Infoo",
|
||||
expected: []item{{itemIdentifier, 0, "Infoo"}},
|
||||
}, {
|
||||
input: "-Infoo",
|
||||
expected: []item{{itemSUB, 0, "-"}, {itemIdentifier, 1, "Infoo"}},
|
||||
}, {
|
||||
input: "-Inf 123",
|
||||
expected: []item{{itemSUB, 0, "-"}, {itemNumber, 1, "Inf"}, {itemNumber, 5, "123"}},
|
||||
}, {
|
||||
input: "0x123",
|
||||
expected: []item{{itemNumber, 0, "0x123"}},
|
||||
},
|
||||
{
|
||||
// See https://github.com/prometheus/prometheus/issues/939.
|
||||
input: ".٩",
|
||||
fail: true,
|
||||
},
|
||||
// Test duration.
|
||||
{
|
||||
input: "5s",
|
||||
expected: []item{{itemDuration, 0, "5s"}},
|
||||
}, {
|
||||
input: "123m",
|
||||
expected: []item{{itemDuration, 0, "123m"}},
|
||||
}, {
|
||||
input: "1h",
|
||||
expected: []item{{itemDuration, 0, "1h"}},
|
||||
}, {
|
||||
input: "3w",
|
||||
expected: []item{{itemDuration, 0, "3w"}},
|
||||
}, {
|
||||
input: "1y",
|
||||
expected: []item{{itemDuration, 0, "1y"}},
|
||||
},
|
||||
// Test identifiers.
|
||||
{
|
||||
input: "abc",
|
||||
expected: []item{{itemIdentifier, 0, "abc"}},
|
||||
}, {
|
||||
input: "a:bc",
|
||||
expected: []item{{itemMetricIdentifier, 0, "a:bc"}},
|
||||
}, {
|
||||
input: "abc d",
|
||||
expected: []item{{itemIdentifier, 0, "abc"}, {itemIdentifier, 4, "d"}},
|
||||
}, {
|
||||
input: ":bc",
|
||||
expected: []item{{itemMetricIdentifier, 0, ":bc"}},
|
||||
}, {
|
||||
input: "0a:bc",
|
||||
fail: true,
|
||||
},
|
||||
// Test comments.
|
||||
{
|
||||
input: "# some comment",
|
||||
expected: []item{{itemComment, 0, "# some comment"}},
|
||||
}, {
|
||||
input: "5 # 1+1\n5",
|
||||
expected: []item{
|
||||
{itemNumber, 0, "5"},
|
||||
{itemComment, 2, "# 1+1"},
|
||||
{itemNumber, 8, "5"},
|
||||
},
|
||||
},
|
||||
// Test operators.
|
||||
{
|
||||
input: `=`,
|
||||
expected: []item{{itemAssign, 0, `=`}},
|
||||
}, {
|
||||
// Inside braces equality is a single '=' character.
|
||||
input: `{=}`,
|
||||
expected: []item{{itemLeftBrace, 0, `{`}, {itemEQL, 1, `=`}, {itemRightBrace, 2, `}`}},
|
||||
}, {
|
||||
input: `==`,
|
||||
expected: []item{{itemEQL, 0, `==`}},
|
||||
}, {
|
||||
input: `!=`,
|
||||
expected: []item{{itemNEQ, 0, `!=`}},
|
||||
}, {
|
||||
input: `<`,
|
||||
expected: []item{{itemLSS, 0, `<`}},
|
||||
}, {
|
||||
input: `>`,
|
||||
expected: []item{{itemGTR, 0, `>`}},
|
||||
}, {
|
||||
input: `>=`,
|
||||
expected: []item{{itemGTE, 0, `>=`}},
|
||||
}, {
|
||||
input: `<=`,
|
||||
expected: []item{{itemLTE, 0, `<=`}},
|
||||
}, {
|
||||
input: `+`,
|
||||
expected: []item{{itemADD, 0, `+`}},
|
||||
}, {
|
||||
input: `-`,
|
||||
expected: []item{{itemSUB, 0, `-`}},
|
||||
}, {
|
||||
input: `*`,
|
||||
expected: []item{{itemMUL, 0, `*`}},
|
||||
}, {
|
||||
input: `/`,
|
||||
expected: []item{{itemDIV, 0, `/`}},
|
||||
}, {
|
||||
input: `%`,
|
||||
expected: []item{{itemMOD, 0, `%`}},
|
||||
}, {
|
||||
input: `AND`,
|
||||
expected: []item{{itemLAND, 0, `AND`}},
|
||||
}, {
|
||||
input: `or`,
|
||||
expected: []item{{itemLOR, 0, `or`}},
|
||||
},
|
||||
// Test aggregators.
|
||||
{
|
||||
input: `sum`,
|
||||
expected: []item{{itemSum, 0, `sum`}},
|
||||
}, {
|
||||
input: `AVG`,
|
||||
expected: []item{{itemAvg, 0, `AVG`}},
|
||||
}, {
|
||||
input: `MAX`,
|
||||
expected: []item{{itemMax, 0, `MAX`}},
|
||||
}, {
|
||||
input: `min`,
|
||||
expected: []item{{itemMin, 0, `min`}},
|
||||
}, {
|
||||
input: `count`,
|
||||
expected: []item{{itemCount, 0, `count`}},
|
||||
}, {
|
||||
input: `stdvar`,
|
||||
expected: []item{{itemStdvar, 0, `stdvar`}},
|
||||
}, {
|
||||
input: `stddev`,
|
||||
expected: []item{{itemStddev, 0, `stddev`}},
|
||||
},
|
||||
// Test keywords.
|
||||
{
|
||||
input: "alert",
|
||||
expected: []item{{itemAlert, 0, "alert"}},
|
||||
}, {
|
||||
input: "keeping_extra",
|
||||
expected: []item{{itemKeepCommon, 0, "keeping_extra"}},
|
||||
}, {
|
||||
input: "keep_common",
|
||||
expected: []item{{itemKeepCommon, 0, "keep_common"}},
|
||||
}, {
|
||||
input: "if",
|
||||
expected: []item{{itemIf, 0, "if"}},
|
||||
}, {
|
||||
input: "for",
|
||||
expected: []item{{itemFor, 0, "for"}},
|
||||
}, {
|
||||
input: "with",
|
||||
expected: []item{{itemWith, 0, "with"}},
|
||||
}, {
|
||||
input: "description",
|
||||
expected: []item{{itemDescription, 0, "description"}},
|
||||
}, {
|
||||
input: "summary",
|
||||
expected: []item{{itemSummary, 0, "summary"}},
|
||||
}, {
|
||||
input: "runbook",
|
||||
expected: []item{{itemRunbook, 0, "runbook"}},
|
||||
}, {
|
||||
input: "offset",
|
||||
expected: []item{{itemOffset, 0, "offset"}},
|
||||
}, {
|
||||
input: "by",
|
||||
expected: []item{{itemBy, 0, "by"}},
|
||||
}, {
|
||||
input: "on",
|
||||
expected: []item{{itemOn, 0, "on"}},
|
||||
}, {
|
||||
input: "group_left",
|
||||
expected: []item{{itemGroupLeft, 0, "group_left"}},
|
||||
}, {
|
||||
input: "group_right",
|
||||
expected: []item{{itemGroupRight, 0, "group_right"}},
|
||||
},
|
||||
// Test Selector.
|
||||
{
|
||||
input: `台北`,
|
||||
fail: true,
|
||||
}, {
|
||||
input: `{台北='a'}`,
|
||||
fail: true,
|
||||
}, {
|
||||
input: `{0a='a'}`,
|
||||
fail: true,
|
||||
}, {
|
||||
input: `{foo='bar'}`,
|
||||
expected: []item{
|
||||
{itemLeftBrace, 0, `{`},
|
||||
{itemIdentifier, 1, `foo`},
|
||||
{itemEQL, 4, `=`},
|
||||
{itemString, 5, `'bar'`},
|
||||
{itemRightBrace, 10, `}`},
|
||||
},
|
||||
}, {
|
||||
input: `{foo="bar"}`,
|
||||
expected: []item{
|
||||
{itemLeftBrace, 0, `{`},
|
||||
{itemIdentifier, 1, `foo`},
|
||||
{itemEQL, 4, `=`},
|
||||
{itemString, 5, `"bar"`},
|
||||
{itemRightBrace, 10, `}`},
|
||||
},
|
||||
}, {
|
||||
input: `{foo="bar\"bar"}`,
|
||||
expected: []item{
|
||||
{itemLeftBrace, 0, `{`},
|
||||
{itemIdentifier, 1, `foo`},
|
||||
{itemEQL, 4, `=`},
|
||||
{itemString, 5, `"bar\"bar"`},
|
||||
{itemRightBrace, 15, `}`},
|
||||
},
|
||||
}, {
|
||||
input: `{NaN != "bar" }`,
|
||||
expected: []item{
|
||||
{itemLeftBrace, 0, `{`},
|
||||
{itemIdentifier, 1, `NaN`},
|
||||
{itemNEQ, 5, `!=`},
|
||||
{itemString, 8, `"bar"`},
|
||||
{itemRightBrace, 14, `}`},
|
||||
},
|
||||
}, {
|
||||
input: `{alert=~"bar" }`,
|
||||
expected: []item{
|
||||
{itemLeftBrace, 0, `{`},
|
||||
{itemIdentifier, 1, `alert`},
|
||||
{itemEQLRegex, 6, `=~`},
|
||||
{itemString, 8, `"bar"`},
|
||||
{itemRightBrace, 14, `}`},
|
||||
},
|
||||
}, {
|
||||
input: `{on!~"bar"}`,
|
||||
expected: []item{
|
||||
{itemLeftBrace, 0, `{`},
|
||||
{itemIdentifier, 1, `on`},
|
||||
{itemNEQRegex, 3, `!~`},
|
||||
{itemString, 5, `"bar"`},
|
||||
{itemRightBrace, 10, `}`},
|
||||
},
|
||||
}, {
|
||||
input: `{alert!#"bar"}`, fail: true,
|
||||
}, {
|
||||
input: `{foo:a="bar"}`, fail: true,
|
||||
},
|
||||
// Test common errors.
|
||||
{
|
||||
input: `=~`, fail: true,
|
||||
}, {
|
||||
input: `!~`, fail: true,
|
||||
}, {
|
||||
input: `!(`, fail: true,
|
||||
}, {
|
||||
input: "1a", fail: true,
|
||||
},
|
||||
// Test mismatched parens.
|
||||
{
|
||||
input: `(`, fail: true,
|
||||
}, {
|
||||
input: `())`, fail: true,
|
||||
}, {
|
||||
input: `(()`, fail: true,
|
||||
}, {
|
||||
input: `{`, fail: true,
|
||||
}, {
|
||||
input: `}`, fail: true,
|
||||
}, {
|
||||
input: "{{", fail: true,
|
||||
}, {
|
||||
input: "{{}}", fail: true,
|
||||
}, {
|
||||
input: `[`, fail: true,
|
||||
}, {
|
||||
input: `[[`, fail: true,
|
||||
}, {
|
||||
input: `[]]`, fail: true,
|
||||
}, {
|
||||
input: `[[]]`, fail: true,
|
||||
}, {
|
||||
input: `]`, fail: true,
|
||||
},
|
||||
// Test series description.
|
||||
{
|
||||
input: `{} _ 1 x .3`,
|
||||
expected: []item{
|
||||
{itemLeftBrace, 0, `{`},
|
||||
{itemRightBrace, 1, `}`},
|
||||
{itemBlank, 3, `_`},
|
||||
{itemNumber, 5, `1`},
|
||||
{itemTimes, 7, `x`},
|
||||
{itemNumber, 9, `.3`},
|
||||
},
|
||||
seriesDesc: true,
|
||||
},
|
||||
{
|
||||
input: `metric +Inf Inf NaN`,
|
||||
expected: []item{
|
||||
{itemIdentifier, 0, `metric`},
|
||||
{itemADD, 7, `+`},
|
||||
{itemNumber, 8, `Inf`},
|
||||
{itemNumber, 12, `Inf`},
|
||||
{itemNumber, 16, `NaN`},
|
||||
},
|
||||
seriesDesc: true,
|
||||
},
|
||||
{
|
||||
input: `metric 1+1x4`,
|
||||
expected: []item{
|
||||
{itemIdentifier, 0, `metric`},
|
||||
{itemNumber, 7, `1`},
|
||||
{itemADD, 8, `+`},
|
||||
{itemNumber, 9, `1`},
|
||||
{itemTimes, 10, `x`},
|
||||
{itemNumber, 11, `4`},
|
||||
},
|
||||
seriesDesc: true,
|
||||
},
|
||||
}
|
||||
|
||||
// TestLexer tests basic functionality of the lexer. More elaborate tests are implemented
|
||||
// for the parser to avoid duplicated effort.
|
||||
func TestLexer(t *testing.T) {
|
||||
for i, test := range tests {
|
||||
l := lex(test.input)
|
||||
l.seriesDesc = test.seriesDesc
|
||||
|
||||
out := []item{}
|
||||
for it := range l.items {
|
||||
out = append(out, it)
|
||||
}
|
||||
|
||||
lastItem := out[len(out)-1]
|
||||
if test.fail {
|
||||
if lastItem.typ != itemError {
|
||||
t.Logf("%d: input %q", i, test.input)
|
||||
t.Fatalf("expected lexing error but did not fail")
|
||||
}
|
||||
continue
|
||||
}
|
||||
if lastItem.typ == itemError {
|
||||
t.Logf("%d: input %q", i, test.input)
|
||||
t.Fatalf("unexpected lexing error at position %d: %s", lastItem.pos, lastItem)
|
||||
}
|
||||
|
||||
if !reflect.DeepEqual(lastItem, item{itemEOF, Pos(len(test.input)), ""}) {
|
||||
t.Logf("%d: input %q", i, test.input)
|
||||
t.Fatalf("lexing error: expected output to end with EOF item.\ngot:\n%s", expectedList(out))
|
||||
}
|
||||
out = out[:len(out)-1]
|
||||
if !reflect.DeepEqual(out, test.expected) {
|
||||
t.Logf("%d: input %q", i, test.input)
|
||||
t.Fatalf("lexing mismatch:\nexpected:\n%s\ngot:\n%s", expectedList(test.expected), expectedList(out))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func expectedList(exp []item) string {
|
||||
s := ""
|
||||
for _, it := range exp {
|
||||
s += fmt.Sprintf("\t%#v\n", it)
|
||||
}
|
||||
return s
|
||||
}
|
1111
prom/promql/parse.go
Normal file
1111
prom/promql/parse.go
Normal file
File diff suppressed because it is too large
Load Diff
1456
prom/promql/parse_test.go
Normal file
1456
prom/promql/parse_test.go
Normal file
File diff suppressed because it is too large
Load Diff
247
prom/promql/printer.go
Normal file
247
prom/promql/printer.go
Normal file
@ -0,0 +1,247 @@
|
||||
// Copyright 2015 The Prometheus Authors
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package promql
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"sort"
|
||||
"strings"
|
||||
|
||||
clientmodel "github.com/prometheus/client_golang/model"
|
||||
|
||||
"github.com/prometheus/prometheus/storage/metric"
|
||||
"github.com/prometheus/prometheus/util/strutil"
|
||||
)
|
||||
|
||||
func (matrix Matrix) String() string {
|
||||
metricStrings := make([]string, 0, len(matrix))
|
||||
for _, sampleStream := range matrix {
|
||||
metricName, hasName := sampleStream.Metric.Metric[clientmodel.MetricNameLabel]
|
||||
numLabels := len(sampleStream.Metric.Metric)
|
||||
if hasName {
|
||||
numLabels--
|
||||
}
|
||||
labelStrings := make([]string, 0, numLabels)
|
||||
for label, value := range sampleStream.Metric.Metric {
|
||||
if label != clientmodel.MetricNameLabel {
|
||||
labelStrings = append(labelStrings, fmt.Sprintf("%s=%q", label, value))
|
||||
}
|
||||
}
|
||||
sort.Strings(labelStrings)
|
||||
valueStrings := make([]string, 0, len(sampleStream.Values))
|
||||
for _, value := range sampleStream.Values {
|
||||
valueStrings = append(valueStrings,
|
||||
fmt.Sprintf("\n%v @[%v]", value.Value, value.Timestamp))
|
||||
}
|
||||
metricStrings = append(metricStrings,
|
||||
fmt.Sprintf("%s{%s} => %s",
|
||||
metricName,
|
||||
strings.Join(labelStrings, ", "),
|
||||
strings.Join(valueStrings, ", ")))
|
||||
}
|
||||
sort.Strings(metricStrings)
|
||||
return strings.Join(metricStrings, "\n")
|
||||
}
|
||||
|
||||
func (vector Vector) String() string {
|
||||
metricStrings := make([]string, 0, len(vector))
|
||||
for _, sample := range vector {
|
||||
metricStrings = append(metricStrings,
|
||||
fmt.Sprintf("%s => %v @[%v]",
|
||||
sample.Metric,
|
||||
sample.Value, sample.Timestamp))
|
||||
}
|
||||
return strings.Join(metricStrings, "\n")
|
||||
}
|
||||
|
||||
// Tree returns a string of the tree structure of the given node.
|
||||
func Tree(node Node) string {
|
||||
return tree(node, "")
|
||||
}
|
||||
|
||||
func tree(node Node, level string) string {
|
||||
if node == nil {
|
||||
return fmt.Sprintf("%s |---- %T\n", level, node)
|
||||
}
|
||||
typs := strings.Split(fmt.Sprintf("%T", node), ".")[1]
|
||||
|
||||
var t string
|
||||
// Only print the number of statements for readability.
|
||||
if stmts, ok := node.(Statements); ok {
|
||||
t = fmt.Sprintf("%s |---- %s :: %d\n", level, typs, len(stmts))
|
||||
} else {
|
||||
t = fmt.Sprintf("%s |---- %s :: %s\n", level, typs, node)
|
||||
}
|
||||
|
||||
level += " · · ·"
|
||||
|
||||
switch n := node.(type) {
|
||||
case Statements:
|
||||
for _, s := range n {
|
||||
t += tree(s, level)
|
||||
}
|
||||
case *AlertStmt:
|
||||
t += tree(n.Expr, level)
|
||||
|
||||
case *EvalStmt:
|
||||
t += tree(n.Expr, level)
|
||||
|
||||
case *RecordStmt:
|
||||
t += tree(n.Expr, level)
|
||||
|
||||
case Expressions:
|
||||
for _, e := range n {
|
||||
t += tree(e, level)
|
||||
}
|
||||
case *AggregateExpr:
|
||||
t += tree(n.Expr, level)
|
||||
|
||||
case *BinaryExpr:
|
||||
t += tree(n.LHS, level)
|
||||
t += tree(n.RHS, level)
|
||||
|
||||
case *Call:
|
||||
t += tree(n.Args, level)
|
||||
|
||||
case *ParenExpr:
|
||||
t += tree(n.Expr, level)
|
||||
|
||||
case *UnaryExpr:
|
||||
t += tree(n.Expr, level)
|
||||
|
||||
case *MatrixSelector, *NumberLiteral, *StringLiteral, *VectorSelector:
|
||||
// nothing to do
|
||||
|
||||
default:
|
||||
panic("promql.Tree: not all node types covered")
|
||||
}
|
||||
return t
|
||||
}
|
||||
|
||||
func (stmts Statements) String() (s string) {
|
||||
if len(stmts) == 0 {
|
||||
return ""
|
||||
}
|
||||
for _, stmt := range stmts {
|
||||
s += stmt.String()
|
||||
s += "\n\n"
|
||||
}
|
||||
return s[:len(s)-2]
|
||||
}
|
||||
|
||||
func (node *AlertStmt) String() string {
|
||||
s := fmt.Sprintf("ALERT %s", node.Name)
|
||||
s += fmt.Sprintf("\n\tIF %s", node.Expr)
|
||||
if node.Duration > 0 {
|
||||
s += fmt.Sprintf("\n\tFOR %s", strutil.DurationToString(node.Duration))
|
||||
}
|
||||
if len(node.Labels) > 0 {
|
||||
s += fmt.Sprintf("\n\tWITH %s", node.Labels)
|
||||
}
|
||||
s += fmt.Sprintf("\n\tSUMMARY %q", node.Summary)
|
||||
s += fmt.Sprintf("\n\tDESCRIPTION %q", node.Description)
|
||||
return s
|
||||
}
|
||||
|
||||
func (node *EvalStmt) String() string {
|
||||
return "EVAL " + node.Expr.String()
|
||||
}
|
||||
|
||||
func (node *RecordStmt) String() string {
|
||||
s := fmt.Sprintf("%s%s = %s", node.Name, node.Labels, node.Expr)
|
||||
return s
|
||||
}
|
||||
|
||||
func (es Expressions) String() (s string) {
|
||||
if len(es) == 0 {
|
||||
return ""
|
||||
}
|
||||
for _, e := range es {
|
||||
s += e.String()
|
||||
s += ", "
|
||||
}
|
||||
return s[:len(s)-2]
|
||||
}
|
||||
|
||||
func (node *AggregateExpr) String() string {
|
||||
aggrString := fmt.Sprintf("%s(%s)", node.Op, node.Expr)
|
||||
if len(node.Grouping) > 0 {
|
||||
format := "%s BY (%s)"
|
||||
if node.KeepExtraLabels {
|
||||
format += " KEEP_COMMON"
|
||||
}
|
||||
return fmt.Sprintf(format, aggrString, node.Grouping)
|
||||
}
|
||||
return aggrString
|
||||
}
|
||||
|
||||
func (node *BinaryExpr) String() string {
|
||||
matching := ""
|
||||
vm := node.VectorMatching
|
||||
if vm != nil && len(vm.On) > 0 {
|
||||
matching = fmt.Sprintf(" ON(%s)", vm.On)
|
||||
if vm.Card == CardManyToOne {
|
||||
matching += fmt.Sprintf(" GROUP_LEFT(%s)", vm.Include)
|
||||
}
|
||||
if vm.Card == CardOneToMany {
|
||||
matching += fmt.Sprintf(" GROUP_RIGHT(%s)", vm.Include)
|
||||
}
|
||||
}
|
||||
return fmt.Sprintf("%s %s%s %s", node.LHS, node.Op, matching, node.RHS)
|
||||
}
|
||||
|
||||
func (node *Call) String() string {
|
||||
return fmt.Sprintf("%s(%s)", node.Func.Name, node.Args)
|
||||
}
|
||||
|
||||
func (node *MatrixSelector) String() string {
|
||||
vecSelector := &VectorSelector{
|
||||
Name: node.Name,
|
||||
LabelMatchers: node.LabelMatchers,
|
||||
}
|
||||
return fmt.Sprintf("%s[%s]", vecSelector.String(), strutil.DurationToString(node.Range))
|
||||
}
|
||||
|
||||
func (node *NumberLiteral) String() string {
|
||||
return fmt.Sprint(node.Val)
|
||||
}
|
||||
|
||||
func (node *ParenExpr) String() string {
|
||||
return fmt.Sprintf("(%s)", node.Expr)
|
||||
}
|
||||
|
||||
func (node *StringLiteral) String() string {
|
||||
return fmt.Sprintf("%q", node.Val)
|
||||
}
|
||||
|
||||
func (node *UnaryExpr) String() string {
|
||||
return fmt.Sprintf("%s%s", node.Op, node.Expr)
|
||||
}
|
||||
|
||||
func (node *VectorSelector) String() string {
|
||||
labelStrings := make([]string, 0, len(node.LabelMatchers)-1)
|
||||
for _, matcher := range node.LabelMatchers {
|
||||
// Only include the __name__ label if its no equality matching.
|
||||
if matcher.Name == clientmodel.MetricNameLabel && matcher.Type == metric.Equal {
|
||||
continue
|
||||
}
|
||||
labelStrings = append(labelStrings, matcher.String())
|
||||
}
|
||||
|
||||
if len(labelStrings) == 0 {
|
||||
return node.Name
|
||||
}
|
||||
sort.Strings(labelStrings)
|
||||
return fmt.Sprintf("%s{%s}", node.Name, strings.Join(labelStrings, ","))
|
||||
}
|
48
prom/promql/printer_test.go
Normal file
48
prom/promql/printer_test.go
Normal file
@ -0,0 +1,48 @@
|
||||
// Copyright 2015 The Prometheus Authors
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package promql
|
||||
|
||||
import (
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestExprString(t *testing.T) {
|
||||
// A list of valid expressions that are expected to be
|
||||
// returned as out when calling String(). If out is empty the output
|
||||
// is expected to equal the input.
|
||||
inputs := []struct {
|
||||
in, out string
|
||||
}{
|
||||
{
|
||||
in: `sum(task:errors:rate10s{job="s"}) BY (code)`,
|
||||
},
|
||||
{
|
||||
in: `sum(task:errors:rate10s{job="s"}) BY (code) KEEP_COMMON`,
|
||||
},
|
||||
}
|
||||
|
||||
for _, test := range inputs {
|
||||
expr, err := ParseExpr(test.in)
|
||||
if err != nil {
|
||||
t.Fatalf("parsing error for %q: %s", test.in, err)
|
||||
}
|
||||
exp := test.in
|
||||
if test.out != "" {
|
||||
exp = test.out
|
||||
}
|
||||
if expr.String() != exp {
|
||||
t.Fatalf("expected %q to be returned as:\n%s\ngot:\n%s\n", test.in, exp, expr.String())
|
||||
}
|
||||
}
|
||||
}
|
37
prom/promql/promql_test.go
Normal file
37
prom/promql/promql_test.go
Normal file
@ -0,0 +1,37 @@
|
||||
// Copyright 2015 The Prometheus Authors
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package promql
|
||||
|
||||
import (
|
||||
"path/filepath"
|
||||
"testing"
|
||||
)
|
||||
|
||||
func TestEvaluations(t *testing.T) {
|
||||
files, err := filepath.Glob("testdata/*.test")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
for _, fn := range files {
|
||||
test, err := NewTestFromFile(t, fn)
|
||||
if err != nil {
|
||||
t.Errorf("error creating test for %s: %s", fn, err)
|
||||
}
|
||||
err = test.Run()
|
||||
if err != nil {
|
||||
t.Errorf("error running test %s: %s", fn, err)
|
||||
}
|
||||
test.Close()
|
||||
}
|
||||
}
|
106
prom/promql/quantile.go
Normal file
106
prom/promql/quantile.go
Normal file
@ -0,0 +1,106 @@
|
||||
// Copyright 2015 The Prometheus Authors
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package promql
|
||||
|
||||
import (
|
||||
"math"
|
||||
"sort"
|
||||
|
||||
clientmodel "github.com/prometheus/client_golang/model"
|
||||
)
|
||||
|
||||
// Helpers to calculate quantiles.
|
||||
|
||||
// excludedLabels are the labels to exclude from signature calculation for
|
||||
// quantiles.
|
||||
var excludedLabels = map[clientmodel.LabelName]struct{}{
|
||||
clientmodel.MetricNameLabel: {},
|
||||
clientmodel.BucketLabel: {},
|
||||
}
|
||||
|
||||
type bucket struct {
|
||||
upperBound float64
|
||||
count clientmodel.SampleValue
|
||||
}
|
||||
|
||||
// buckets implements sort.Interface.
|
||||
type buckets []bucket
|
||||
|
||||
func (b buckets) Len() int { return len(b) }
|
||||
func (b buckets) Swap(i, j int) { b[i], b[j] = b[j], b[i] }
|
||||
func (b buckets) Less(i, j int) bool { return b[i].upperBound < b[j].upperBound }
|
||||
|
||||
type metricWithBuckets struct {
|
||||
metric clientmodel.COWMetric
|
||||
buckets buckets
|
||||
}
|
||||
|
||||
// quantile calculates the quantile 'q' based on the given buckets. The buckets
|
||||
// will be sorted by upperBound by this function (i.e. no sorting needed before
|
||||
// calling this function). The quantile value is interpolated assuming a linear
|
||||
// distribution within a bucket. However, if the quantile falls into the highest
|
||||
// bucket, the upper bound of the 2nd highest bucket is returned. A natural
|
||||
// lower bound of 0 is assumed if the upper bound of the lowest bucket is
|
||||
// greater 0. In that case, interpolation in the lowest bucket happens linearly
|
||||
// between 0 and the upper bound of the lowest bucket. However, if the lowest
|
||||
// bucket has an upper bound less or equal 0, this upper bound is returned if
|
||||
// the quantile falls into the lowest bucket.
|
||||
//
|
||||
// There are a number of special cases (once we have a way to report errors
|
||||
// happening during evaluations of AST functions, we should report those
|
||||
// explicitly):
|
||||
//
|
||||
// If 'buckets' has fewer than 2 elements, NaN is returned.
|
||||
//
|
||||
// If the highest bucket is not +Inf, NaN is returned.
|
||||
//
|
||||
// If q<0, -Inf is returned.
|
||||
//
|
||||
// If q>1, +Inf is returned.
|
||||
func quantile(q clientmodel.SampleValue, buckets buckets) float64 {
|
||||
if q < 0 {
|
||||
return math.Inf(-1)
|
||||
}
|
||||
if q > 1 {
|
||||
return math.Inf(+1)
|
||||
}
|
||||
if len(buckets) < 2 {
|
||||
return math.NaN()
|
||||
}
|
||||
sort.Sort(buckets)
|
||||
if !math.IsInf(buckets[len(buckets)-1].upperBound, +1) {
|
||||
return math.NaN()
|
||||
}
|
||||
|
||||
rank := q * buckets[len(buckets)-1].count
|
||||
b := sort.Search(len(buckets)-1, func(i int) bool { return buckets[i].count >= rank })
|
||||
|
||||
if b == len(buckets)-1 {
|
||||
return buckets[len(buckets)-2].upperBound
|
||||
}
|
||||
if b == 0 && buckets[0].upperBound <= 0 {
|
||||
return buckets[0].upperBound
|
||||
}
|
||||
var (
|
||||
bucketStart float64
|
||||
bucketEnd = buckets[b].upperBound
|
||||
count = buckets[b].count
|
||||
)
|
||||
if b > 0 {
|
||||
bucketStart = buckets[b-1].upperBound
|
||||
count -= buckets[b-1].count
|
||||
rank -= buckets[b-1].count
|
||||
}
|
||||
return bucketStart + (bucketEnd-bucketStart)*float64(rank/count)
|
||||
}
|
518
prom/promql/test.go
Normal file
518
prom/promql/test.go
Normal file
@ -0,0 +1,518 @@
|
||||
// Copyright 2015 The Prometheus Authors
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package promql
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"math"
|
||||
"regexp"
|
||||
"strconv"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
clientmodel "github.com/prometheus/client_golang/model"
|
||||
|
||||
"github.com/prometheus/prometheus/storage"
|
||||
"github.com/prometheus/prometheus/storage/local"
|
||||
"github.com/prometheus/prometheus/storage/metric"
|
||||
"github.com/prometheus/prometheus/util/strutil"
|
||||
"github.com/prometheus/prometheus/util/testutil"
|
||||
)
|
||||
|
||||
var (
|
||||
minNormal = math.Float64frombits(0x0010000000000000) // The smallest positive normal value of type float64.
|
||||
|
||||
patSpace = regexp.MustCompile("[\t ]+")
|
||||
patLoad = regexp.MustCompile(`^load\s+(.+?)$`)
|
||||
patEvalInstant = regexp.MustCompile(`^eval(?:_(fail|ordered))?\s+instant\s+(?:at\s+(.+?))?\s+(.+)$`)
|
||||
)
|
||||
|
||||
const (
|
||||
testStartTime = clientmodel.Timestamp(0)
|
||||
epsilon = 0.000001 // Relative error allowed for sample values.
|
||||
maxErrorCount = 10
|
||||
)
|
||||
|
||||
// Test is a sequence of read and write commands that are run
|
||||
// against a test storage.
|
||||
type Test struct {
|
||||
testutil.T
|
||||
|
||||
cmds []testCommand
|
||||
|
||||
storage local.Storage
|
||||
closeStorage func()
|
||||
queryEngine *Engine
|
||||
}
|
||||
|
||||
// NewTest returns an initialized empty Test.
|
||||
func NewTest(t testutil.T, input string) (*Test, error) {
|
||||
test := &Test{
|
||||
T: t,
|
||||
cmds: []testCommand{},
|
||||
}
|
||||
err := test.parse(input)
|
||||
test.clear()
|
||||
|
||||
return test, err
|
||||
}
|
||||
|
||||
func NewTestFromFile(t testutil.T, filename string) (*Test, error) {
|
||||
content, err := ioutil.ReadFile(filename)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return NewTest(t, string(content))
|
||||
}
|
||||
|
||||
// QueryEngine returns the test's query engine.
|
||||
func (t *Test) QueryEngine() *Engine {
|
||||
return t.queryEngine
|
||||
}
|
||||
|
||||
// Storage returns the test's storage.
|
||||
func (t *Test) Storage() local.Storage {
|
||||
return t.storage
|
||||
}
|
||||
|
||||
func raise(line int, format string, v ...interface{}) error {
|
||||
return &ParseErr{
|
||||
Line: line + 1,
|
||||
Err: fmt.Errorf(format, v...),
|
||||
}
|
||||
}
|
||||
|
||||
func (t *Test) parseLoad(lines []string, i int) (int, *loadCmd, error) {
|
||||
if !patLoad.MatchString(lines[i]) {
|
||||
return i, nil, raise(i, "invalid load command. (load <step:duration>)")
|
||||
}
|
||||
parts := patLoad.FindStringSubmatch(lines[i])
|
||||
|
||||
gap, err := strutil.StringToDuration(parts[1])
|
||||
if err != nil {
|
||||
return i, nil, raise(i, "invalid step definition %q: %s", parts[1], err)
|
||||
}
|
||||
cmd := newLoadCmd(gap)
|
||||
for i+1 < len(lines) {
|
||||
i++
|
||||
defLine := lines[i]
|
||||
if len(defLine) == 0 {
|
||||
i--
|
||||
break
|
||||
}
|
||||
metric, vals, err := parseSeriesDesc(defLine)
|
||||
if err != nil {
|
||||
if perr, ok := err.(*ParseErr); ok {
|
||||
perr.Line = i + 1
|
||||
}
|
||||
return i, nil, err
|
||||
}
|
||||
cmd.set(metric, vals...)
|
||||
}
|
||||
return i, cmd, nil
|
||||
}
|
||||
|
||||
func (t *Test) parseEval(lines []string, i int) (int, *evalCmd, error) {
|
||||
if !patEvalInstant.MatchString(lines[i]) {
|
||||
return i, nil, raise(i, "invalid evaluation command. (eval[_fail|_ordered] instant [at <offset:duration>] <query>")
|
||||
}
|
||||
parts := patEvalInstant.FindStringSubmatch(lines[i])
|
||||
var (
|
||||
mod = parts[1]
|
||||
at = parts[2]
|
||||
qry = parts[3]
|
||||
)
|
||||
expr, err := ParseExpr(qry)
|
||||
if err != nil {
|
||||
if perr, ok := err.(*ParseErr); ok {
|
||||
perr.Line = i + 1
|
||||
perr.Pos += strings.Index(lines[i], qry)
|
||||
}
|
||||
return i, nil, err
|
||||
}
|
||||
|
||||
offset, err := strutil.StringToDuration(at)
|
||||
if err != nil {
|
||||
return i, nil, raise(i, "invalid step definition %q: %s", parts[1], err)
|
||||
}
|
||||
ts := testStartTime.Add(offset)
|
||||
|
||||
cmd := newEvalCmd(expr, ts, ts, 0)
|
||||
switch mod {
|
||||
case "ordered":
|
||||
cmd.ordered = true
|
||||
case "fail":
|
||||
cmd.fail = true
|
||||
}
|
||||
|
||||
for j := 1; i+1 < len(lines); j++ {
|
||||
i++
|
||||
defLine := lines[i]
|
||||
if len(defLine) == 0 {
|
||||
i--
|
||||
break
|
||||
}
|
||||
if f, err := parseNumber(defLine); err == nil {
|
||||
cmd.expect(0, nil, sequenceValue{value: clientmodel.SampleValue(f)})
|
||||
break
|
||||
}
|
||||
metric, vals, err := parseSeriesDesc(defLine)
|
||||
if err != nil {
|
||||
if perr, ok := err.(*ParseErr); ok {
|
||||
perr.Line = i + 1
|
||||
}
|
||||
return i, nil, err
|
||||
}
|
||||
|
||||
// Currently, we are not expecting any matrices.
|
||||
if len(vals) > 1 {
|
||||
return i, nil, raise(i, "expecting multiple values in instant evaluation not allowed")
|
||||
}
|
||||
cmd.expect(j, metric, vals...)
|
||||
}
|
||||
return i, cmd, nil
|
||||
}
|
||||
|
||||
// parse the given command sequence and appends it to the test.
|
||||
func (t *Test) parse(input string) error {
|
||||
// Trim lines and remove comments.
|
||||
lines := strings.Split(input, "\n")
|
||||
for i, l := range lines {
|
||||
l = strings.TrimSpace(l)
|
||||
if strings.HasPrefix(l, "#") {
|
||||
l = ""
|
||||
}
|
||||
lines[i] = l
|
||||
}
|
||||
var err error
|
||||
|
||||
// Scan for steps line by line.
|
||||
for i := 0; i < len(lines); i++ {
|
||||
l := lines[i]
|
||||
if len(l) == 0 {
|
||||
continue
|
||||
}
|
||||
var cmd testCommand
|
||||
|
||||
switch c := strings.ToLower(patSpace.Split(l, 2)[0]); {
|
||||
case c == "clear":
|
||||
cmd = &clearCmd{}
|
||||
case c == "load":
|
||||
i, cmd, err = t.parseLoad(lines, i)
|
||||
case strings.HasPrefix(c, "eval"):
|
||||
i, cmd, err = t.parseEval(lines, i)
|
||||
default:
|
||||
return raise(i, "invalid command %q", l)
|
||||
}
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
t.cmds = append(t.cmds, cmd)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// testCommand is an interface that ensures that only the package internal
|
||||
// types can be a valid command for a test.
|
||||
type testCommand interface {
|
||||
testCmd()
|
||||
}
|
||||
|
||||
func (*clearCmd) testCmd() {}
|
||||
func (*loadCmd) testCmd() {}
|
||||
func (*evalCmd) testCmd() {}
|
||||
|
||||
// loadCmd is a command that loads sequences of sample values for specific
|
||||
// metrics into the storage.
|
||||
type loadCmd struct {
|
||||
gap time.Duration
|
||||
metrics map[clientmodel.Fingerprint]clientmodel.Metric
|
||||
defs map[clientmodel.Fingerprint]metric.Values
|
||||
}
|
||||
|
||||
func newLoadCmd(gap time.Duration) *loadCmd {
|
||||
return &loadCmd{
|
||||
gap: gap,
|
||||
metrics: map[clientmodel.Fingerprint]clientmodel.Metric{},
|
||||
defs: map[clientmodel.Fingerprint]metric.Values{},
|
||||
}
|
||||
}
|
||||
|
||||
func (cmd loadCmd) String() string {
|
||||
return "load"
|
||||
}
|
||||
|
||||
// set a sequence of sample values for the given metric.
|
||||
func (cmd *loadCmd) set(m clientmodel.Metric, vals ...sequenceValue) {
|
||||
fp := m.Fingerprint()
|
||||
|
||||
samples := make(metric.Values, 0, len(vals))
|
||||
ts := testStartTime
|
||||
for _, v := range vals {
|
||||
if !v.omitted {
|
||||
samples = append(samples, metric.SamplePair{
|
||||
Timestamp: ts,
|
||||
Value: v.value,
|
||||
})
|
||||
}
|
||||
ts = ts.Add(cmd.gap)
|
||||
}
|
||||
cmd.defs[fp] = samples
|
||||
cmd.metrics[fp] = m
|
||||
}
|
||||
|
||||
// append the defined time series to the storage.
|
||||
func (cmd *loadCmd) append(a storage.SampleAppender) {
|
||||
for fp, samples := range cmd.defs {
|
||||
met := cmd.metrics[fp]
|
||||
for _, smpl := range samples {
|
||||
s := &clientmodel.Sample{
|
||||
Metric: met,
|
||||
Value: smpl.Value,
|
||||
Timestamp: smpl.Timestamp,
|
||||
}
|
||||
a.Append(s)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// evalCmd is a command that evaluates an expression for the given time (range)
|
||||
// and expects a specific result.
|
||||
type evalCmd struct {
|
||||
expr Expr
|
||||
start, end clientmodel.Timestamp
|
||||
interval time.Duration
|
||||
|
||||
instant bool
|
||||
fail, ordered bool
|
||||
|
||||
metrics map[clientmodel.Fingerprint]clientmodel.Metric
|
||||
expected map[clientmodel.Fingerprint]entry
|
||||
}
|
||||
|
||||
type entry struct {
|
||||
pos int
|
||||
vals []sequenceValue
|
||||
}
|
||||
|
||||
func (e entry) String() string {
|
||||
return fmt.Sprintf("%d: %s", e.pos, e.vals)
|
||||
}
|
||||
|
||||
func newEvalCmd(expr Expr, start, end clientmodel.Timestamp, interval time.Duration) *evalCmd {
|
||||
return &evalCmd{
|
||||
expr: expr,
|
||||
start: start,
|
||||
end: end,
|
||||
interval: interval,
|
||||
instant: start == end && interval == 0,
|
||||
|
||||
metrics: map[clientmodel.Fingerprint]clientmodel.Metric{},
|
||||
expected: map[clientmodel.Fingerprint]entry{},
|
||||
}
|
||||
}
|
||||
|
||||
func (ev *evalCmd) String() string {
|
||||
return "eval"
|
||||
}
|
||||
|
||||
// expect adds a new metric with a sequence of values to the set of expected
|
||||
// results for the query.
|
||||
func (ev *evalCmd) expect(pos int, m clientmodel.Metric, vals ...sequenceValue) {
|
||||
if m == nil {
|
||||
ev.expected[0] = entry{pos: pos, vals: vals}
|
||||
return
|
||||
}
|
||||
fp := m.Fingerprint()
|
||||
ev.metrics[fp] = m
|
||||
ev.expected[fp] = entry{pos: pos, vals: vals}
|
||||
}
|
||||
|
||||
// compareResult compares the result value with the defined expectation.
|
||||
func (ev *evalCmd) compareResult(result Value) error {
|
||||
switch val := result.(type) {
|
||||
case Matrix:
|
||||
if ev.instant {
|
||||
return fmt.Errorf("received range result on instant evaluation")
|
||||
}
|
||||
seen := map[clientmodel.Fingerprint]bool{}
|
||||
for pos, v := range val {
|
||||
fp := v.Metric.Metric.Fingerprint()
|
||||
if _, ok := ev.metrics[fp]; !ok {
|
||||
return fmt.Errorf("unexpected metric %s in result", v.Metric.Metric)
|
||||
}
|
||||
exp := ev.expected[fp]
|
||||
if ev.ordered && exp.pos != pos+1 {
|
||||
return fmt.Errorf("expected metric %s with %v at position %d but was at %d", v.Metric.Metric, exp.vals, exp.pos, pos+1)
|
||||
}
|
||||
for i, expVal := range exp.vals {
|
||||
if !almostEqual(float64(expVal.value), float64(v.Values[i].Value)) {
|
||||
return fmt.Errorf("expected %v for %s but got %v", expVal, v.Metric.Metric, v.Values)
|
||||
}
|
||||
}
|
||||
seen[fp] = true
|
||||
}
|
||||
for fp, expVals := range ev.expected {
|
||||
if !seen[fp] {
|
||||
return fmt.Errorf("expected metric %s with %v not found", ev.metrics[fp], expVals)
|
||||
}
|
||||
}
|
||||
|
||||
case Vector:
|
||||
if !ev.instant {
|
||||
fmt.Errorf("received instant result on range evaluation")
|
||||
}
|
||||
seen := map[clientmodel.Fingerprint]bool{}
|
||||
for pos, v := range val {
|
||||
fp := v.Metric.Metric.Fingerprint()
|
||||
if _, ok := ev.metrics[fp]; !ok {
|
||||
return fmt.Errorf("unexpected metric %s in result", v.Metric.Metric)
|
||||
}
|
||||
exp := ev.expected[fp]
|
||||
if ev.ordered && exp.pos != pos+1 {
|
||||
return fmt.Errorf("expected metric %s with %v at position %d but was at %d", v.Metric.Metric, exp.vals, exp.pos, pos+1)
|
||||
}
|
||||
if !almostEqual(float64(exp.vals[0].value), float64(v.Value)) {
|
||||
return fmt.Errorf("expected %v for %s but got %v", exp.vals[0].value, v.Metric.Metric, v.Value)
|
||||
}
|
||||
|
||||
seen[fp] = true
|
||||
}
|
||||
for fp, expVals := range ev.expected {
|
||||
if !seen[fp] {
|
||||
return fmt.Errorf("expected metric %s with %v not found", ev.metrics[fp], expVals)
|
||||
}
|
||||
}
|
||||
|
||||
case *Scalar:
|
||||
if !almostEqual(float64(ev.expected[0].vals[0].value), float64(val.Value)) {
|
||||
return fmt.Errorf("expected scalar %v but got %v", val.Value, ev.expected[0].vals[0].value)
|
||||
}
|
||||
|
||||
default:
|
||||
panic(fmt.Errorf("promql.Test.compareResult: unexpected result type %T", result))
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// clearCmd is a command that wipes the test's storage state.
|
||||
type clearCmd struct{}
|
||||
|
||||
func (cmd clearCmd) String() string {
|
||||
return "clear"
|
||||
}
|
||||
|
||||
// Run executes the command sequence of the test. Until the maximum error number
|
||||
// is reached, evaluation errors do not terminate execution.
|
||||
func (t *Test) Run() error {
|
||||
for _, cmd := range t.cmds {
|
||||
err := t.exec(cmd)
|
||||
// TODO(fabxc): aggregate command errors, yield diffs for result
|
||||
// comparison errors.
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// exec processes a single step of the test
|
||||
func (t *Test) exec(tc testCommand) error {
|
||||
switch cmd := tc.(type) {
|
||||
case *clearCmd:
|
||||
t.clear()
|
||||
|
||||
case *loadCmd:
|
||||
cmd.append(t.storage)
|
||||
t.storage.WaitForIndexing()
|
||||
|
||||
case *evalCmd:
|
||||
q := t.queryEngine.newQuery(cmd.expr, cmd.start, cmd.end, cmd.interval)
|
||||
res := q.Exec()
|
||||
if res.Err != nil {
|
||||
if cmd.fail {
|
||||
return nil
|
||||
}
|
||||
return fmt.Errorf("error evaluating query: %s", res.Err)
|
||||
}
|
||||
if res.Err == nil && cmd.fail {
|
||||
return fmt.Errorf("expected error evaluating query but got none")
|
||||
}
|
||||
|
||||
err := cmd.compareResult(res.Value)
|
||||
if err != nil {
|
||||
return fmt.Errorf("error in %s %s: %s", cmd, cmd.expr, err)
|
||||
}
|
||||
|
||||
default:
|
||||
panic("promql.Test.exec: unknown test command type")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// clear the current test storage of all inserted samples.
|
||||
func (t *Test) clear() {
|
||||
if t.closeStorage != nil {
|
||||
t.closeStorage()
|
||||
}
|
||||
if t.queryEngine != nil {
|
||||
t.queryEngine.Stop()
|
||||
}
|
||||
|
||||
var closer testutil.Closer
|
||||
t.storage, closer = local.NewTestStorage(t, 1)
|
||||
|
||||
t.closeStorage = closer.Close
|
||||
t.queryEngine = NewEngine(t.storage, nil)
|
||||
}
|
||||
|
||||
func (t *Test) Close() {
|
||||
t.queryEngine.Stop()
|
||||
t.closeStorage()
|
||||
}
|
||||
|
||||
// samplesAlmostEqual returns true if the two sample lines only differ by a
|
||||
// small relative error in their sample value.
|
||||
func almostEqual(a, b float64) bool {
|
||||
// NaN has no equality but for testing we still want to know whether both values
|
||||
// are NaN.
|
||||
if math.IsNaN(a) && math.IsNaN(b) {
|
||||
return true
|
||||
}
|
||||
|
||||
// Cf. http://floating-point-gui.de/errors/comparison/
|
||||
if a == b {
|
||||
return true
|
||||
}
|
||||
|
||||
diff := math.Abs(a - b)
|
||||
|
||||
if a == 0 || b == 0 || diff < minNormal {
|
||||
return diff < epsilon*minNormal
|
||||
}
|
||||
return diff/(math.Abs(a)+math.Abs(b)) < epsilon
|
||||
}
|
||||
|
||||
func parseNumber(s string) (float64, error) {
|
||||
n, err := strconv.ParseInt(s, 0, 64)
|
||||
f := float64(n)
|
||||
if err != nil {
|
||||
f, err = strconv.ParseFloat(s, 64)
|
||||
}
|
||||
if err != nil {
|
||||
return 0, fmt.Errorf("error parsing number: %s", err)
|
||||
}
|
||||
return f, nil
|
||||
}
|
94
prom/promql/testdata/functions.test
vendored
Normal file
94
prom/promql/testdata/functions.test
vendored
Normal file
@ -0,0 +1,94 @@
|
||||
# Testdata for resets() and changes().
|
||||
load 5m
|
||||
http_requests{path="/foo"} 1 2 3 0 1 0 0 1 2 0
|
||||
http_requests{path="/bar"} 1 2 3 4 5 1 2 3 4 5
|
||||
http_requests{path="/biz"} 0 0 0 0 0 1 1 1 1 1
|
||||
|
||||
# Tests for resets().
|
||||
eval instant at 50m resets(http_requests[5m])
|
||||
{path="/foo"} 0
|
||||
{path="/bar"} 0
|
||||
{path="/biz"} 0
|
||||
|
||||
eval instant at 50m resets(http_requests[20m])
|
||||
{path="/foo"} 1
|
||||
{path="/bar"} 0
|
||||
{path="/biz"} 0
|
||||
|
||||
eval instant at 50m resets(http_requests[30m])
|
||||
{path="/foo"} 2
|
||||
{path="/bar"} 1
|
||||
{path="/biz"} 0
|
||||
|
||||
eval instant at 50m resets(http_requests[50m])
|
||||
{path="/foo"} 3
|
||||
{path="/bar"} 1
|
||||
{path="/biz"} 0
|
||||
|
||||
eval instant at 50m resets(nonexistent_metric[50m])
|
||||
|
||||
# Tests for changes().
|
||||
eval instant at 50m changes(http_requests[5m])
|
||||
{path="/foo"} 0
|
||||
{path="/bar"} 0
|
||||
{path="/biz"} 0
|
||||
|
||||
eval instant at 50m changes(http_requests[20m])
|
||||
{path="/foo"} 3
|
||||
{path="/bar"} 3
|
||||
{path="/biz"} 0
|
||||
|
||||
eval instant at 50m changes(http_requests[30m])
|
||||
{path="/foo"} 4
|
||||
{path="/bar"} 5
|
||||
{path="/biz"} 1
|
||||
|
||||
eval instant at 50m changes(http_requests[50m])
|
||||
{path="/foo"} 8
|
||||
{path="/bar"} 9
|
||||
{path="/biz"} 1
|
||||
|
||||
eval instant at 50m changes(nonexistent_metric[50m])
|
||||
|
||||
|
||||
clear
|
||||
|
||||
# Tests for increase().
|
||||
load 5m
|
||||
http_requests{path="/foo"} 0+10x10
|
||||
http_requests{path="/bar"} 0+10x5 0+10x5
|
||||
|
||||
# Tests for increase().
|
||||
eval instant at 50m increase(http_requests[50m])
|
||||
{path="/foo"} 100
|
||||
{path="/bar"} 90
|
||||
|
||||
|
||||
clear
|
||||
|
||||
# Tests for deriv() and predict_linear().
|
||||
load 5m
|
||||
testcounter_reset_middle 0+10x4 0+10x5
|
||||
http_requests{job="app-server", instance="1", group="canary"} 0+80x10
|
||||
|
||||
# Deriv should return the same as rate in simple cases.
|
||||
eval instant at 50m rate(http_requests{group="canary", instance="1", job="app-server"}[60m])
|
||||
{group="canary", instance="1", job="app-server"} 0.26666666666666666
|
||||
|
||||
eval instant at 50m deriv(http_requests{group="canary", instance="1", job="app-server"}[60m])
|
||||
{group="canary", instance="1", job="app-server"} 0.26666666666666666
|
||||
|
||||
# Deriv should return correct result.
|
||||
eval instant at 50m deriv(testcounter_reset_middle[100m])
|
||||
{} 0.010606060606060607
|
||||
|
||||
# Predict_linear should return correct result.
|
||||
eval instant at 50m predict_linear(testcounter_reset_middle[100m], 3600)
|
||||
{} 88.181818181818185200
|
||||
|
||||
# Predict_linear is syntactic sugar around deriv.
|
||||
eval instant at 50m predict_linear(http_requests[50m], 3600) - (http_requests + deriv(http_requests[50m]) * 3600)
|
||||
{group="canary", instance="1", job="app-server"} 0
|
||||
|
||||
eval instant at 50m predict_linear(testcounter_reset_middle[100m], 3600) - (testcounter_reset_middle + deriv(testcounter_reset_middle[100m]) * 3600)
|
||||
{} 0
|
141
prom/promql/testdata/histograms.test
vendored
Normal file
141
prom/promql/testdata/histograms.test
vendored
Normal file
@ -0,0 +1,141 @@
|
||||
# Two histograms with 4 buckets each (x_sum and x_count not included,
|
||||
# only buckets). Lowest bucket for one histogram < 0, for the other >
|
||||
# 0. They have the same name, just separated by label. Not useful in
|
||||
# practice, but can happen (if clients change bucketing), and the
|
||||
# server has to cope with it.
|
||||
|
||||
# Test histogram.
|
||||
load 5m
|
||||
testhistogram_bucket{le="0.1", start="positive"} 0+5x10
|
||||
testhistogram_bucket{le=".2", start="positive"} 0+7x10
|
||||
testhistogram_bucket{le="1e0", start="positive"} 0+11x10
|
||||
testhistogram_bucket{le="+Inf", start="positive"} 0+12x10
|
||||
testhistogram_bucket{le="-.2", start="negative"} 0+1x10
|
||||
testhistogram_bucket{le="-0.1", start="negative"} 0+2x10
|
||||
testhistogram_bucket{le="0.3", start="negative"} 0+2x10
|
||||
testhistogram_bucket{le="+Inf", start="negative"} 0+3x10
|
||||
|
||||
|
||||
# Now a more realistic histogram per job and instance to test aggregation.
|
||||
load 5m
|
||||
request_duration_seconds_bucket{job="job1", instance="ins1", le="0.1"} 0+1x10
|
||||
request_duration_seconds_bucket{job="job1", instance="ins1", le="0.2"} 0+3x10
|
||||
request_duration_seconds_bucket{job="job1", instance="ins1", le="+Inf"} 0+4x10
|
||||
request_duration_seconds_bucket{job="job1", instance="ins2", le="0.1"} 0+2x10
|
||||
request_duration_seconds_bucket{job="job1", instance="ins2", le="0.2"} 0+5x10
|
||||
request_duration_seconds_bucket{job="job1", instance="ins2", le="+Inf"} 0+6x10
|
||||
request_duration_seconds_bucket{job="job2", instance="ins1", le="0.1"} 0+3x10
|
||||
request_duration_seconds_bucket{job="job2", instance="ins1", le="0.2"} 0+4x10
|
||||
request_duration_seconds_bucket{job="job2", instance="ins1", le="+Inf"} 0+6x10
|
||||
request_duration_seconds_bucket{job="job2", instance="ins2", le="0.1"} 0+4x10
|
||||
request_duration_seconds_bucket{job="job2", instance="ins2", le="0.2"} 0+7x10
|
||||
request_duration_seconds_bucket{job="job2", instance="ins2", le="+Inf"} 0+9x10
|
||||
|
||||
|
||||
# Quantile too low.
|
||||
eval instant at 50m histogram_quantile(-0.1, testhistogram_bucket)
|
||||
{start="positive"} -Inf
|
||||
{start="negative"} -Inf
|
||||
|
||||
# Quantile too high.
|
||||
eval instant at 50m histogram_quantile(1.01, testhistogram_bucket)
|
||||
{start="positive"} +Inf
|
||||
{start="negative"} +Inf
|
||||
|
||||
# Quantile value in lowest bucket, which is positive.
|
||||
eval instant at 50m histogram_quantile(0, testhistogram_bucket{start="positive"})
|
||||
{start="positive"} 0
|
||||
|
||||
# Quantile value in lowest bucket, which is negative.
|
||||
eval instant at 50m histogram_quantile(0, testhistogram_bucket{start="negative"})
|
||||
{start="negative"} -0.2
|
||||
|
||||
# Quantile value in highest bucket.
|
||||
eval instant at 50m histogram_quantile(1, testhistogram_bucket)
|
||||
{start="positive"} 1
|
||||
{start="negative"} 0.3
|
||||
|
||||
# Finally some useful quantiles.
|
||||
eval instant at 50m histogram_quantile(0.2, testhistogram_bucket)
|
||||
{start="positive"} 0.048
|
||||
{start="negative"} -0.2
|
||||
|
||||
|
||||
eval instant at 50m histogram_quantile(0.5, testhistogram_bucket)
|
||||
{start="positive"} 0.15
|
||||
{start="negative"} -0.15
|
||||
|
||||
eval instant at 50m histogram_quantile(0.8, testhistogram_bucket)
|
||||
{start="positive"} 0.72
|
||||
{start="negative"} 0.3
|
||||
|
||||
# More realistic with rates.
|
||||
eval instant at 50m histogram_quantile(0.2, rate(testhistogram_bucket[5m]))
|
||||
{start="positive"} 0.048
|
||||
{start="negative"} -0.2
|
||||
|
||||
eval instant at 50m histogram_quantile(0.5, rate(testhistogram_bucket[5m]))
|
||||
{start="positive"} 0.15
|
||||
{start="negative"} -0.15
|
||||
|
||||
eval instant at 50m histogram_quantile(0.8, rate(testhistogram_bucket[5m]))
|
||||
{start="positive"} 0.72
|
||||
{start="negative"} 0.3
|
||||
|
||||
# Aggregated histogram: Everything in one.
|
||||
eval instant at 50m histogram_quantile(0.3, sum(rate(request_duration_seconds_bucket[5m])) by (le))
|
||||
{} 0.075
|
||||
|
||||
eval instant at 50m histogram_quantile(0.5, sum(rate(request_duration_seconds_bucket[5m])) by (le))
|
||||
{} 0.1277777777777778
|
||||
|
||||
# Aggregated histogram: Everything in one. Now with avg, which does not change anything.
|
||||
eval instant at 50m histogram_quantile(0.3, avg(rate(request_duration_seconds_bucket[5m])) by (le))
|
||||
{} 0.075
|
||||
|
||||
eval instant at 50m histogram_quantile(0.5, avg(rate(request_duration_seconds_bucket[5m])) by (le))
|
||||
{} 0.12777777777777778
|
||||
|
||||
# Aggregated histogram: By job.
|
||||
eval instant at 50m histogram_quantile(0.3, sum(rate(request_duration_seconds_bucket[5m])) by (le, instance))
|
||||
{instance="ins1"} 0.075
|
||||
{instance="ins2"} 0.075
|
||||
|
||||
eval instant at 50m histogram_quantile(0.5, sum(rate(request_duration_seconds_bucket[5m])) by (le, instance))
|
||||
{instance="ins1"} 0.1333333333
|
||||
{instance="ins2"} 0.125
|
||||
|
||||
# Aggregated histogram: By instance.
|
||||
eval instant at 50m histogram_quantile(0.3, sum(rate(request_duration_seconds_bucket[5m])) by (le, job))
|
||||
{job="job1"} 0.1
|
||||
{job="job2"} 0.0642857142857143
|
||||
|
||||
eval instant at 50m histogram_quantile(0.5, sum(rate(request_duration_seconds_bucket[5m])) by (le, job))
|
||||
{job="job1"} 0.14
|
||||
{job="job2"} 0.1125
|
||||
|
||||
# Aggregated histogram: By job and instance.
|
||||
eval instant at 50m histogram_quantile(0.3, sum(rate(request_duration_seconds_bucket[5m])) by (le, job, instance))
|
||||
{instance="ins1", job="job1"} 0.11
|
||||
{instance="ins2", job="job1"} 0.09
|
||||
{instance="ins1", job="job2"} 0.06
|
||||
{instance="ins2", job="job2"} 0.0675
|
||||
|
||||
eval instant at 50m histogram_quantile(0.5, sum(rate(request_duration_seconds_bucket[5m])) by (le, job, instance))
|
||||
{instance="ins1", job="job1"} 0.15
|
||||
{instance="ins2", job="job1"} 0.1333333333333333
|
||||
{instance="ins1", job="job2"} 0.1
|
||||
{instance="ins2", job="job2"} 0.1166666666666667
|
||||
|
||||
# The unaggregated histogram for comparison. Same result as the previous one.
|
||||
eval instant at 50m histogram_quantile(0.3, rate(request_duration_seconds_bucket[5m]))
|
||||
{instance="ins1", job="job1"} 0.11
|
||||
{instance="ins2", job="job1"} 0.09
|
||||
{instance="ins1", job="job2"} 0.06
|
||||
{instance="ins2", job="job2"} 0.0675
|
||||
|
||||
eval instant at 50m histogram_quantile(0.5, rate(request_duration_seconds_bucket[5m]))
|
||||
{instance="ins1", job="job1"} 0.15
|
||||
{instance="ins2", job="job1"} 0.13333333333333333
|
||||
{instance="ins1", job="job2"} 0.1
|
||||
{instance="ins2", job="job2"} 0.11666666666666667
|
665
prom/promql/testdata/legacy.test
vendored
Normal file
665
prom/promql/testdata/legacy.test
vendored
Normal file
@ -0,0 +1,665 @@
|
||||
load 5m
|
||||
http_requests{job="api-server", instance="0", group="production"} 0+10x10
|
||||
http_requests{job="api-server", instance="1", group="production"} 0+20x10
|
||||
http_requests{job="api-server", instance="0", group="canary"} 0+30x10
|
||||
http_requests{job="api-server", instance="1", group="canary"} 0+40x10
|
||||
http_requests{job="app-server", instance="0", group="production"} 0+50x10
|
||||
http_requests{job="app-server", instance="1", group="production"} 0+60x10
|
||||
http_requests{job="app-server", instance="0", group="canary"} 0+70x10
|
||||
http_requests{job="app-server", instance="1", group="canary"} 0+80x10
|
||||
|
||||
load 5m
|
||||
x{y="testvalue"} 0+10x10
|
||||
|
||||
load 5m
|
||||
testcounter_reset_middle 0+10x4 0+10x5
|
||||
testcounter_reset_end 0+10x9 0 10
|
||||
|
||||
load 5m
|
||||
label_grouping_test{a="aa", b="bb"} 0+10x10
|
||||
label_grouping_test{a="a", b="abb"} 0+20x10
|
||||
|
||||
load 5m
|
||||
vector_matching_a{l="x"} 0+1x100
|
||||
vector_matching_a{l="y"} 0+2x50
|
||||
vector_matching_b{l="x"} 0+4x25
|
||||
|
||||
load 5m
|
||||
cpu_count{instance="0", type="numa"} 0+30x10
|
||||
cpu_count{instance="0", type="smp"} 0+10x20
|
||||
cpu_count{instance="1", type="smp"} 0+20x10
|
||||
|
||||
|
||||
eval instant at 50m SUM(http_requests)
|
||||
{} 3600
|
||||
|
||||
eval instant at 50m SUM(http_requests{instance="0"}) BY(job)
|
||||
{job="api-server"} 400
|
||||
{job="app-server"} 1200
|
||||
|
||||
eval instant at 50m SUM(http_requests{instance="0"}) BY(job) KEEP_COMMON
|
||||
{instance="0", job="api-server"} 400
|
||||
{instance="0", job="app-server"} 1200
|
||||
|
||||
eval instant at 50m SUM(http_requests) BY (job)
|
||||
{job="api-server"} 1000
|
||||
{job="app-server"} 2600
|
||||
|
||||
# Non-existent labels mentioned in BY-clauses shouldn't propagate to output.
|
||||
eval instant at 50m SUM(http_requests) BY (job, nonexistent)
|
||||
{job="api-server"} 1000
|
||||
{job="app-server"} 2600
|
||||
|
||||
|
||||
eval instant at 50m COUNT(http_requests) BY (job)
|
||||
{job="api-server"} 4
|
||||
{job="app-server"} 4
|
||||
|
||||
|
||||
eval instant at 50m SUM(http_requests) BY (job, group)
|
||||
{group="canary", job="api-server"} 700
|
||||
{group="canary", job="app-server"} 1500
|
||||
{group="production", job="api-server"} 300
|
||||
{group="production", job="app-server"} 1100
|
||||
|
||||
|
||||
eval instant at 50m AVG(http_requests) BY (job)
|
||||
{job="api-server"} 250
|
||||
{job="app-server"} 650
|
||||
|
||||
|
||||
eval instant at 50m MIN(http_requests) BY (job)
|
||||
{job="api-server"} 100
|
||||
{job="app-server"} 500
|
||||
|
||||
|
||||
eval instant at 50m MAX(http_requests) BY (job)
|
||||
{job="api-server"} 400
|
||||
{job="app-server"} 800
|
||||
|
||||
|
||||
eval instant at 50m SUM(http_requests) BY (job) - COUNT(http_requests) BY (job)
|
||||
{job="api-server"} 996
|
||||
{job="app-server"} 2596
|
||||
|
||||
|
||||
eval instant at 50m 2 - SUM(http_requests) BY (job)
|
||||
{job="api-server"} -998
|
||||
{job="app-server"} -2598
|
||||
|
||||
|
||||
eval instant at 50m 1000 / SUM(http_requests) BY (job)
|
||||
{job="api-server"} 1
|
||||
{job="app-server"} 0.38461538461538464
|
||||
|
||||
|
||||
eval instant at 50m SUM(http_requests) BY (job) - 2
|
||||
{job="api-server"} 998
|
||||
{job="app-server"} 2598
|
||||
|
||||
|
||||
eval instant at 50m SUM(http_requests) BY (job) % 3
|
||||
{job="api-server"} 1
|
||||
{job="app-server"} 2
|
||||
|
||||
|
||||
eval instant at 50m SUM(http_requests) BY (job) / 0
|
||||
{job="api-server"} +Inf
|
||||
{job="app-server"} +Inf
|
||||
|
||||
|
||||
eval instant at 50m SUM(http_requests) BY (job) > 1000
|
||||
{job="app-server"} 2600
|
||||
|
||||
|
||||
eval instant at 50m 1000 < SUM(http_requests) BY (job)
|
||||
{job="app-server"} 1000
|
||||
|
||||
|
||||
eval instant at 50m SUM(http_requests) BY (job) <= 1000
|
||||
{job="api-server"} 1000
|
||||
|
||||
|
||||
eval instant at 50m SUM(http_requests) BY (job) != 1000
|
||||
{job="app-server"} 2600
|
||||
|
||||
|
||||
eval instant at 50m SUM(http_requests) BY (job) == 1000
|
||||
{job="api-server"} 1000
|
||||
|
||||
|
||||
eval instant at 50m SUM(http_requests) BY (job) + SUM(http_requests) BY (job)
|
||||
{job="api-server"} 2000
|
||||
{job="app-server"} 5200
|
||||
|
||||
|
||||
eval instant at 50m http_requests{job="api-server", group="canary"}
|
||||
http_requests{group="canary", instance="0", job="api-server"} 300
|
||||
http_requests{group="canary", instance="1", job="api-server"} 400
|
||||
|
||||
|
||||
eval instant at 50m http_requests{job="api-server", group="canary"} + rate(http_requests{job="api-server"}[5m]) * 5 * 60
|
||||
{group="canary", instance="0", job="api-server"} 330
|
||||
{group="canary", instance="1", job="api-server"} 440
|
||||
|
||||
|
||||
eval instant at 50m rate(http_requests[25m]) * 25 * 60
|
||||
{group="canary", instance="0", job="api-server"} 150
|
||||
{group="canary", instance="0", job="app-server"} 350
|
||||
{group="canary", instance="1", job="api-server"} 200
|
||||
{group="canary", instance="1", job="app-server"} 400
|
||||
{group="production", instance="0", job="api-server"} 50
|
||||
{group="production", instance="0", job="app-server"} 249.99999999999997
|
||||
{group="production", instance="1", job="api-server"} 100
|
||||
{group="production", instance="1", job="app-server"} 300
|
||||
|
||||
eval instant at 50m delta(http_requests[25m], 1)
|
||||
{group="canary", instance="0", job="api-server"} 150
|
||||
{group="canary", instance="0", job="app-server"} 350
|
||||
{group="canary", instance="1", job="api-server"} 200
|
||||
{group="canary", instance="1", job="app-server"} 400
|
||||
{group="production", instance="0", job="api-server"} 50
|
||||
{group="production", instance="0", job="app-server"} 250
|
||||
{group="production", instance="1", job="api-server"} 100
|
||||
{group="production", instance="1", job="app-server"} 300
|
||||
|
||||
eval_ordered instant at 50m sort(http_requests)
|
||||
http_requests{group="production", instance="0", job="api-server"} 100
|
||||
http_requests{group="production", instance="1", job="api-server"} 200
|
||||
http_requests{group="canary", instance="0", job="api-server"} 300
|
||||
http_requests{group="canary", instance="1", job="api-server"} 400
|
||||
http_requests{group="production", instance="0", job="app-server"} 500
|
||||
http_requests{group="production", instance="1", job="app-server"} 600
|
||||
http_requests{group="canary", instance="0", job="app-server"} 700
|
||||
http_requests{group="canary", instance="1", job="app-server"} 800
|
||||
|
||||
eval_ordered instant at 50m sort(0 / round(http_requests, 400) + http_requests)
|
||||
{group="production", instance="0", job="api-server"} NaN
|
||||
{group="production", instance="1", job="api-server"} 200
|
||||
{group="canary", instance="0", job="api-server"} 300
|
||||
{group="canary", instance="1", job="api-server"} 400
|
||||
{group="production", instance="0", job="app-server"} 500
|
||||
{group="production", instance="1", job="app-server"} 600
|
||||
{group="canary", instance="0", job="app-server"} 700
|
||||
{group="canary", instance="1", job="app-server"} 800
|
||||
|
||||
|
||||
eval_ordered instant at 50m sort_desc(http_requests)
|
||||
http_requests{group="canary", instance="1", job="app-server"} 800
|
||||
http_requests{group="canary", instance="0", job="app-server"} 700
|
||||
http_requests{group="production", instance="1", job="app-server"} 600
|
||||
http_requests{group="production", instance="0", job="app-server"} 500
|
||||
http_requests{group="canary", instance="1", job="api-server"} 400
|
||||
http_requests{group="canary", instance="0", job="api-server"} 300
|
||||
http_requests{group="production", instance="1", job="api-server"} 200
|
||||
http_requests{group="production", instance="0", job="api-server"} 100
|
||||
|
||||
eval_ordered instant at 50m topk(3, http_requests)
|
||||
http_requests{group="canary", instance="1", job="app-server"} 800
|
||||
http_requests{group="canary", instance="0", job="app-server"} 700
|
||||
http_requests{group="production", instance="1", job="app-server"} 600
|
||||
|
||||
eval_ordered instant at 50m topk(5, http_requests{group="canary",job="app-server"})
|
||||
http_requests{group="canary", instance="1", job="app-server"} 800
|
||||
http_requests{group="canary", instance="0", job="app-server"} 700
|
||||
|
||||
eval_ordered instant at 50m bottomk(3, http_requests)
|
||||
http_requests{group="production", instance="0", job="api-server"} 100
|
||||
http_requests{group="production", instance="1", job="api-server"} 200
|
||||
http_requests{group="canary", instance="0", job="api-server"} 300
|
||||
|
||||
eval_ordered instant at 50m bottomk(5, http_requests{group="canary",job="app-server"})
|
||||
http_requests{group="canary", instance="0", job="app-server"} 700
|
||||
http_requests{group="canary", instance="1", job="app-server"} 800
|
||||
|
||||
|
||||
# Single-letter label names and values.
|
||||
eval instant at 50m x{y="testvalue"}
|
||||
x{y="testvalue"} 100
|
||||
|
||||
|
||||
# Lower-cased aggregation operators should work too.
|
||||
eval instant at 50m sum(http_requests) by (job) + min(http_requests) by (job) + max(http_requests) by (job) + avg(http_requests) by (job)
|
||||
{job="app-server"} 4550
|
||||
{job="api-server"} 1750
|
||||
|
||||
|
||||
# Deltas should be adjusted for target interval vs. samples under target interval.
|
||||
eval instant at 50m delta(http_requests{group="canary", instance="1", job="app-server"}[18m])
|
||||
{group="canary", instance="1", job="app-server"} 288
|
||||
|
||||
|
||||
# Deltas should perform the same operation when 2nd argument is 0.
|
||||
eval instant at 50m delta(http_requests{group="canary", instance="1", job="app-server"}[18m], 0)
|
||||
{group="canary", instance="1", job="app-server"} 288
|
||||
|
||||
|
||||
# Rates should calculate per-second rates.
|
||||
eval instant at 50m rate(http_requests{group="canary", instance="1", job="app-server"}[60m])
|
||||
{group="canary", instance="1", job="app-server"} 0.26666666666666666
|
||||
|
||||
# Counter resets at in the middle of range are handled correctly by rate().
|
||||
eval instant at 50m rate(testcounter_reset_middle[60m])
|
||||
{} 0.03
|
||||
|
||||
|
||||
# Counter resets at end of range are ignored by rate().
|
||||
eval instant at 50m rate(testcounter_reset_end[5m])
|
||||
{} 0
|
||||
|
||||
# count_scalar for a non-empty vector should return scalar element count.
|
||||
eval instant at 50m count_scalar(http_requests)
|
||||
8
|
||||
|
||||
# count_scalar for an empty vector should return scalar 0.
|
||||
eval instant at 50m count_scalar(nonexistent)
|
||||
0
|
||||
|
||||
eval instant at 50m http_requests{group!="canary"}
|
||||
http_requests{group="production", instance="1", job="app-server"} 600
|
||||
http_requests{group="production", instance="0", job="app-server"} 500
|
||||
http_requests{group="production", instance="1", job="api-server"} 200
|
||||
http_requests{group="production", instance="0", job="api-server"} 100
|
||||
|
||||
eval instant at 50m http_requests{job=~"server",group!="canary"}
|
||||
http_requests{group="production", instance="1", job="app-server"} 600
|
||||
http_requests{group="production", instance="0", job="app-server"} 500
|
||||
http_requests{group="production", instance="1", job="api-server"} 200
|
||||
http_requests{group="production", instance="0", job="api-server"} 100
|
||||
|
||||
eval instant at 50m http_requests{job!~"api",group!="canary"}
|
||||
http_requests{group="production", instance="1", job="app-server"} 600
|
||||
http_requests{group="production", instance="0", job="app-server"} 500
|
||||
|
||||
eval instant at 50m count_scalar(http_requests{job=~"^server$"})
|
||||
0
|
||||
|
||||
eval instant at 50m http_requests{group="production",job=~"^api"}
|
||||
http_requests{group="production", instance="0", job="api-server"} 100
|
||||
http_requests{group="production", instance="1", job="api-server"} 200
|
||||
|
||||
eval instant at 50m abs(-1 * http_requests{group="production",job="api-server"})
|
||||
{group="production", instance="0", job="api-server"} 100
|
||||
{group="production", instance="1", job="api-server"} 200
|
||||
|
||||
eval instant at 50m floor(0.004 * http_requests{group="production",job="api-server"})
|
||||
{group="production", instance="0", job="api-server"} 0
|
||||
{group="production", instance="1", job="api-server"} 0
|
||||
|
||||
eval instant at 50m ceil(0.004 * http_requests{group="production",job="api-server"})
|
||||
{group="production", instance="0", job="api-server"} 1
|
||||
{group="production", instance="1", job="api-server"} 1
|
||||
|
||||
eval instant at 50m round(0.004 * http_requests{group="production",job="api-server"})
|
||||
{group="production", instance="0", job="api-server"} 0
|
||||
{group="production", instance="1", job="api-server"} 1
|
||||
|
||||
# Round should correctly handle negative numbers.
|
||||
eval instant at 50m round(-1 * (0.004 * http_requests{group="production",job="api-server"}))
|
||||
{group="production", instance="0", job="api-server"} 0
|
||||
{group="production", instance="1", job="api-server"} -1
|
||||
|
||||
# Round should round half up.
|
||||
eval instant at 50m round(0.005 * http_requests{group="production",job="api-server"})
|
||||
{group="production", instance="0", job="api-server"} 1
|
||||
{group="production", instance="1", job="api-server"} 1
|
||||
|
||||
eval instant at 50m round(-1 * (0.005 * http_requests{group="production",job="api-server"}))
|
||||
{group="production", instance="0", job="api-server"} 0
|
||||
{group="production", instance="1", job="api-server"} -1
|
||||
|
||||
eval instant at 50m round(1 + 0.005 * http_requests{group="production",job="api-server"})
|
||||
{group="production", instance="0", job="api-server"} 2
|
||||
{group="production", instance="1", job="api-server"} 2
|
||||
|
||||
eval instant at 50m round(-1 * (1 + 0.005 * http_requests{group="production",job="api-server"}))
|
||||
{group="production", instance="0", job="api-server"} -1
|
||||
{group="production", instance="1", job="api-server"} -2
|
||||
|
||||
# Round should accept the number to round nearest to.
|
||||
eval instant at 50m round(0.0005 * http_requests{group="production",job="api-server"}, 0.1)
|
||||
{group="production", instance="0", job="api-server"} 0.1
|
||||
{group="production", instance="1", job="api-server"} 0.1
|
||||
|
||||
eval instant at 50m round(2.1 + 0.0005 * http_requests{group="production",job="api-server"}, 0.1)
|
||||
{group="production", instance="0", job="api-server"} 2.2
|
||||
{group="production", instance="1", job="api-server"} 2.2
|
||||
|
||||
eval instant at 50m round(5.2 + 0.0005 * http_requests{group="production",job="api-server"}, 0.1)
|
||||
{group="production", instance="0", job="api-server"} 5.3
|
||||
{group="production", instance="1", job="api-server"} 5.3
|
||||
|
||||
# Round should work correctly with negative numbers and multiple decimal places.
|
||||
eval instant at 50m round(-1 * (5.2 + 0.0005 * http_requests{group="production",job="api-server"}), 0.1)
|
||||
{group="production", instance="0", job="api-server"} -5.2
|
||||
{group="production", instance="1", job="api-server"} -5.3
|
||||
|
||||
# Round should work correctly with big toNearests.
|
||||
eval instant at 50m round(0.025 * http_requests{group="production",job="api-server"}, 5)
|
||||
{group="production", instance="0", job="api-server"} 5
|
||||
{group="production", instance="1", job="api-server"} 5
|
||||
|
||||
eval instant at 50m round(0.045 * http_requests{group="production",job="api-server"}, 5)
|
||||
{group="production", instance="0", job="api-server"} 5
|
||||
{group="production", instance="1", job="api-server"} 10
|
||||
|
||||
eval instant at 50m avg_over_time(http_requests{group="production",job="api-server"}[1h])
|
||||
{group="production", instance="0", job="api-server"} 50
|
||||
{group="production", instance="1", job="api-server"} 100
|
||||
|
||||
eval instant at 50m count_over_time(http_requests{group="production",job="api-server"}[1h])
|
||||
{group="production", instance="0", job="api-server"} 11
|
||||
{group="production", instance="1", job="api-server"} 11
|
||||
|
||||
eval instant at 50m max_over_time(http_requests{group="production",job="api-server"}[1h])
|
||||
{group="production", instance="0", job="api-server"} 100
|
||||
{group="production", instance="1", job="api-server"} 200
|
||||
|
||||
eval instant at 50m min_over_time(http_requests{group="production",job="api-server"}[1h])
|
||||
{group="production", instance="0", job="api-server"} 0
|
||||
{group="production", instance="1", job="api-server"} 0
|
||||
|
||||
eval instant at 50m sum_over_time(http_requests{group="production",job="api-server"}[1h])
|
||||
{group="production", instance="0", job="api-server"} 550
|
||||
{group="production", instance="1", job="api-server"} 1100
|
||||
|
||||
eval instant at 50m time()
|
||||
3000
|
||||
|
||||
eval instant at 50m drop_common_labels(http_requests{group="production",job="api-server"})
|
||||
http_requests{instance="0"} 100
|
||||
http_requests{instance="1"} 200
|
||||
|
||||
eval instant at 50m {__name__=~".+"}
|
||||
http_requests{group="canary", instance="0", job="api-server"} 300
|
||||
http_requests{group="canary", instance="0", job="app-server"} 700
|
||||
http_requests{group="canary", instance="1", job="api-server"} 400
|
||||
http_requests{group="canary", instance="1", job="app-server"} 800
|
||||
http_requests{group="production", instance="0", job="api-server"} 100
|
||||
http_requests{group="production", instance="0", job="app-server"} 500
|
||||
http_requests{group="production", instance="1", job="api-server"} 200
|
||||
http_requests{group="production", instance="1", job="app-server"} 600
|
||||
testcounter_reset_end 0
|
||||
testcounter_reset_middle 50
|
||||
x{y="testvalue"} 100
|
||||
label_grouping_test{a="a", b="abb"} 200
|
||||
label_grouping_test{a="aa", b="bb"} 100
|
||||
vector_matching_a{l="x"} 10
|
||||
vector_matching_a{l="y"} 20
|
||||
vector_matching_b{l="x"} 40
|
||||
cpu_count{instance="1", type="smp"} 200
|
||||
cpu_count{instance="0", type="smp"} 100
|
||||
cpu_count{instance="0", type="numa"} 300
|
||||
|
||||
|
||||
eval instant at 50m {job=~"server", job!~"api"}
|
||||
http_requests{group="canary", instance="0", job="app-server"} 700
|
||||
http_requests{group="canary", instance="1", job="app-server"} 800
|
||||
http_requests{group="production", instance="0", job="app-server"} 500
|
||||
http_requests{group="production", instance="1", job="app-server"} 600
|
||||
|
||||
# Test alternative "by"-clause order.
|
||||
eval instant at 50m sum by (group) (http_requests{job="api-server"})
|
||||
{group="canary"} 700
|
||||
{group="production"} 300
|
||||
|
||||
# Test alternative "by"-clause order with "keep_common".
|
||||
eval instant at 50m sum by (group) keep_common (http_requests{job="api-server"})
|
||||
{group="canary", job="api-server"} 700
|
||||
{group="production", job="api-server"} 300
|
||||
|
||||
# Test both alternative "by"-clause orders in one expression.
|
||||
# Public health warning: stick to one form within an expression (or even
|
||||
# in an organization), or risk serious user confusion.
|
||||
eval instant at 50m sum(sum by (group) keep_common (http_requests{job="api-server"})) by (job)
|
||||
{job="api-server"} 1000
|
||||
|
||||
eval instant at 50m http_requests{group="canary"} and http_requests{instance="0"}
|
||||
http_requests{group="canary", instance="0", job="api-server"} 300
|
||||
http_requests{group="canary", instance="0", job="app-server"} 700
|
||||
|
||||
eval instant at 50m (http_requests{group="canary"} + 1) and http_requests{instance="0"}
|
||||
{group="canary", instance="0", job="api-server"} 301
|
||||
{group="canary", instance="0", job="app-server"} 701
|
||||
|
||||
eval instant at 50m (http_requests{group="canary"} + 1) and on(instance, job) http_requests{instance="0", group="production"}
|
||||
{group="canary", instance="0", job="api-server"} 301
|
||||
{group="canary", instance="0", job="app-server"} 701
|
||||
|
||||
eval instant at 50m (http_requests{group="canary"} + 1) and on(instance) http_requests{instance="0", group="production"}
|
||||
{group="canary", instance="0", job="api-server"} 301
|
||||
{group="canary", instance="0", job="app-server"} 701
|
||||
|
||||
eval instant at 50m http_requests{group="canary"} or http_requests{group="production"}
|
||||
http_requests{group="canary", instance="0", job="api-server"} 300
|
||||
http_requests{group="canary", instance="0", job="app-server"} 700
|
||||
http_requests{group="canary", instance="1", job="api-server"} 400
|
||||
http_requests{group="canary", instance="1", job="app-server"} 800
|
||||
http_requests{group="production", instance="0", job="api-server"} 100
|
||||
http_requests{group="production", instance="0", job="app-server"} 500
|
||||
http_requests{group="production", instance="1", job="api-server"} 200
|
||||
http_requests{group="production", instance="1", job="app-server"} 600
|
||||
|
||||
# On overlap the rhs samples must be dropped.
|
||||
eval instant at 50m (http_requests{group="canary"} + 1) or http_requests{instance="1"}
|
||||
{group="canary", instance="0", job="api-server"} 301
|
||||
{group="canary", instance="0", job="app-server"} 701
|
||||
{group="canary", instance="1", job="api-server"} 401
|
||||
{group="canary", instance="1", job="app-server"} 801
|
||||
http_requests{group="production", instance="1", job="api-server"} 200
|
||||
http_requests{group="production", instance="1", job="app-server"} 600
|
||||
|
||||
# Matching only on instance excludes everything that has instance=0/1 but includes
|
||||
# entries without the instance label.
|
||||
eval instant at 50m (http_requests{group="canary"} + 1) or on(instance) (http_requests or cpu_count or vector_matching_a)
|
||||
{group="canary", instance="0", job="api-server"} 301
|
||||
{group="canary", instance="0", job="app-server"} 701
|
||||
{group="canary", instance="1", job="api-server"} 401
|
||||
{group="canary", instance="1", job="app-server"} 801
|
||||
vector_matching_a{l="x"} 10
|
||||
vector_matching_a{l="y"} 20
|
||||
|
||||
eval instant at 50m http_requests{group="canary"} / on(instance,job) http_requests{group="production"}
|
||||
{instance="0", job="api-server"} 3
|
||||
{instance="0", job="app-server"} 1.4
|
||||
{instance="1", job="api-server"} 2
|
||||
{instance="1", job="app-server"} 1.3333333333333333
|
||||
|
||||
# Include labels must guarantee uniquely identifiable time series.
|
||||
eval_fail instant at 50m http_requests{group="production"} / on(instance) group_left(group) cpu_count{type="smp"}
|
||||
|
||||
# Many-to-many matching is not allowed.
|
||||
eval_fail instant at 50m http_requests{group="production"} / on(instance) group_left(job,type) cpu_count
|
||||
|
||||
# Many-to-one matching must be explicit.
|
||||
eval_fail instant at 50m http_requests{group="production"} / on(instance) cpu_count{type="smp"}
|
||||
|
||||
eval instant at 50m http_requests{group="production"} / on(instance) group_left(job) cpu_count{type="smp"}
|
||||
{instance="1", job="api-server"} 1
|
||||
{instance="0", job="app-server"} 5
|
||||
{instance="1", job="app-server"} 3
|
||||
{instance="0", job="api-server"} 1
|
||||
|
||||
# Ensure sidedness of grouping preserves operand sides.
|
||||
eval instant at 50m cpu_count{type="smp"} / on(instance) group_right(job) http_requests{group="production"}
|
||||
{instance="1", job="app-server"} 0.3333333333333333
|
||||
{instance="0", job="app-server"} 0.2
|
||||
{instance="1", job="api-server"} 1
|
||||
{instance="0", job="api-server"} 1
|
||||
|
||||
# Include labels from both sides.
|
||||
eval instant at 50m http_requests{group="production"} / on(instance) group_left(job) cpu_count{type="smp"}
|
||||
{instance="1", job="api-server"} 1
|
||||
{instance="0", job="app-server"} 5
|
||||
{instance="1", job="app-server"} 3
|
||||
{instance="0", job="api-server"} 1
|
||||
|
||||
eval instant at 50m http_requests{group="production"} < on(instance,job) http_requests{group="canary"}
|
||||
{instance="1", job="app-server"} 600
|
||||
{instance="0", job="app-server"} 500
|
||||
{instance="1", job="api-server"} 200
|
||||
{instance="0", job="api-server"} 100
|
||||
|
||||
|
||||
eval instant at 50m http_requests{group="production"} > on(instance,job) http_requests{group="canary"}
|
||||
# no output
|
||||
|
||||
eval instant at 50m http_requests{group="production"} == on(instance,job) http_requests{group="canary"}
|
||||
# no output
|
||||
|
||||
eval instant at 50m http_requests > on(instance) group_left(group,job) cpu_count{type="smp"}
|
||||
{group="canary", instance="0", job="app-server"} 700
|
||||
{group="canary", instance="1", job="app-server"} 800
|
||||
{group="canary", instance="0", job="api-server"} 300
|
||||
{group="canary", instance="1", job="api-server"} 400
|
||||
{group="production", instance="0", job="app-server"} 500
|
||||
{group="production", instance="1", job="app-server"} 600
|
||||
|
||||
eval instant at 50m {l="x"} + on(__name__) {l="y"}
|
||||
vector_matching_a 30
|
||||
|
||||
eval instant at 50m absent(nonexistent)
|
||||
{} 1
|
||||
|
||||
|
||||
eval instant at 50m absent(nonexistent{job="testjob", instance="testinstance", method=~".x"})
|
||||
{instance="testinstance", job="testjob"} 1
|
||||
|
||||
eval instant at 50m count_scalar(absent(http_requests))
|
||||
0
|
||||
|
||||
eval instant at 50m count_scalar(absent(sum(http_requests)))
|
||||
0
|
||||
|
||||
eval instant at 50m absent(sum(nonexistent{job="testjob", instance="testinstance"}))
|
||||
{} 1
|
||||
|
||||
eval instant at 50m http_requests{group="production",job="api-server"} offset 5m
|
||||
http_requests{group="production", instance="0", job="api-server"} 90
|
||||
http_requests{group="production", instance="1", job="api-server"} 180
|
||||
|
||||
eval instant at 50m rate(http_requests{group="production",job="api-server"}[10m] offset 5m)
|
||||
{group="production", instance="0", job="api-server"} 0.03333333333333333
|
||||
{group="production", instance="1", job="api-server"} 0.06666666666666667
|
||||
|
||||
# Regression test for missing separator byte in labelsToGroupingKey.
|
||||
eval instant at 50m sum(label_grouping_test) by (a, b)
|
||||
{a="a", b="abb"} 200
|
||||
{a="aa", b="bb"} 100
|
||||
|
||||
eval instant at 50m http_requests{group="canary", instance="0", job="api-server"} / 0
|
||||
{group="canary", instance="0", job="api-server"} +Inf
|
||||
|
||||
eval instant at 50m -1 * http_requests{group="canary", instance="0", job="api-server"} / 0
|
||||
{group="canary", instance="0", job="api-server"} -Inf
|
||||
|
||||
eval instant at 50m 0 * http_requests{group="canary", instance="0", job="api-server"} / 0
|
||||
{group="canary", instance="0", job="api-server"} NaN
|
||||
|
||||
eval instant at 50m 0 * http_requests{group="canary", instance="0", job="api-server"} % 0
|
||||
{group="canary", instance="0", job="api-server"} NaN
|
||||
|
||||
eval instant at 50m exp(vector_matching_a)
|
||||
{l="x"} 22026.465794806718
|
||||
{l="y"} 485165195.4097903
|
||||
|
||||
eval instant at 50m exp(vector_matching_a - 10)
|
||||
{l="y"} 22026.465794806718
|
||||
{l="x"} 1
|
||||
|
||||
eval instant at 50m exp(vector_matching_a - 20)
|
||||
{l="x"} 4.5399929762484854e-05
|
||||
{l="y"} 1
|
||||
|
||||
eval instant at 50m ln(vector_matching_a)
|
||||
{l="x"} 2.302585092994046
|
||||
{l="y"} 2.995732273553991
|
||||
|
||||
eval instant at 50m ln(vector_matching_a - 10)
|
||||
{l="y"} 2.302585092994046
|
||||
{l="x"} -Inf
|
||||
|
||||
eval instant at 50m ln(vector_matching_a - 20)
|
||||
{l="y"} -Inf
|
||||
{l="x"} NaN
|
||||
|
||||
eval instant at 50m exp(ln(vector_matching_a))
|
||||
{l="y"} 20
|
||||
{l="x"} 10
|
||||
|
||||
eval instant at 50m sqrt(vector_matching_a)
|
||||
{l="x"} 3.1622776601683795
|
||||
{l="y"} 4.47213595499958
|
||||
|
||||
eval instant at 50m log2(vector_matching_a)
|
||||
{l="x"} 3.3219280948873626
|
||||
{l="y"} 4.321928094887363
|
||||
|
||||
eval instant at 50m log2(vector_matching_a - 10)
|
||||
{l="y"} 3.3219280948873626
|
||||
{l="x"} -Inf
|
||||
|
||||
eval instant at 50m log2(vector_matching_a - 20)
|
||||
{l="x"} NaN
|
||||
{l="y"} -Inf
|
||||
|
||||
eval instant at 50m log10(vector_matching_a)
|
||||
{l="x"} 1
|
||||
{l="y"} 1.301029995663981
|
||||
|
||||
eval instant at 50m log10(vector_matching_a - 10)
|
||||
{l="y"} 1
|
||||
{l="x"} -Inf
|
||||
|
||||
eval instant at 50m log10(vector_matching_a - 20)
|
||||
{l="x"} NaN
|
||||
{l="y"} -Inf
|
||||
|
||||
eval instant at 50m stddev(http_requests)
|
||||
{} 229.12878474779
|
||||
|
||||
eval instant at 50m stddev by (instance)(http_requests)
|
||||
{instance="0"} 223.60679774998
|
||||
{instance="1"} 223.60679774998
|
||||
|
||||
eval instant at 50m stdvar(http_requests)
|
||||
{} 52500
|
||||
|
||||
eval instant at 50m stdvar by (instance)(http_requests)
|
||||
{instance="0"} 50000
|
||||
{instance="1"} 50000
|
||||
|
||||
|
||||
# Matrix tests.
|
||||
|
||||
clear
|
||||
load 1h
|
||||
testmetric{testlabel="1"} 1 1
|
||||
testmetric{testlabel="2"} _ 2
|
||||
|
||||
eval instant at 0h drop_common_labels(testmetric)
|
||||
testmetric 1
|
||||
|
||||
eval instant at 1h drop_common_labels(testmetric)
|
||||
testmetric{testlabel="1"} 1
|
||||
testmetric{testlabel="2"} 2
|
||||
|
||||
clear
|
||||
load 1h
|
||||
testmetric{testlabel="1"} 1 1
|
||||
testmetric{testlabel="2"} 2 _
|
||||
|
||||
eval instant at 0h sum(testmetric) keep_common
|
||||
{} 3
|
||||
|
||||
eval instant at 1h sum(testmetric) keep_common
|
||||
{testlabel="1"} 1
|
||||
|
||||
clear
|
||||
load 1h
|
||||
testmetric{aa="bb"} 1
|
||||
testmetric{a="abb"} 2
|
||||
|
||||
eval instant at 0h testmetric
|
||||
testmetric{aa="bb"} 1
|
||||
testmetric{a="abb"} 2
|
56
prom/promql/testdata/literals.test
vendored
Normal file
56
prom/promql/testdata/literals.test
vendored
Normal file
@ -0,0 +1,56 @@
|
||||
eval instant at 50m 12.34e6
|
||||
12340000
|
||||
|
||||
eval instant at 50m 12.34e+6
|
||||
12340000
|
||||
|
||||
eval instant at 50m 12.34e-6
|
||||
0.00001234
|
||||
|
||||
eval instant at 50m 1+1
|
||||
2
|
||||
|
||||
eval instant at 50m 1-1
|
||||
0
|
||||
|
||||
eval instant at 50m 1 - -1
|
||||
2
|
||||
|
||||
eval instant at 50m .2
|
||||
0.2
|
||||
|
||||
eval instant at 50m +0.2
|
||||
0.2
|
||||
|
||||
eval instant at 50m -0.2e-6
|
||||
-0.0000002
|
||||
|
||||
eval instant at 50m +Inf
|
||||
+Inf
|
||||
|
||||
eval instant at 50m inF
|
||||
+Inf
|
||||
|
||||
eval instant at 50m -inf
|
||||
-Inf
|
||||
|
||||
eval instant at 50m NaN
|
||||
NaN
|
||||
|
||||
eval instant at 50m nan
|
||||
NaN
|
||||
|
||||
eval instant at 50m 2.
|
||||
2
|
||||
|
||||
eval instant at 50m 1 / 0
|
||||
+Inf
|
||||
|
||||
eval instant at 50m -1 / 0
|
||||
-Inf
|
||||
|
||||
eval instant at 50m 0 / 0
|
||||
NaN
|
||||
|
||||
eval instant at 50m 1 % 0
|
||||
NaN
|
Reference in New Issue
Block a user