From 32b7595c4771b193b5869f5a97f935e404f16e30 Mon Sep 17 00:00:00 2001 From: Fabian Reinartz Date: Mon, 30 Mar 2015 18:12:51 +0200 Subject: [PATCH 01/10] Create promql package with lexer/parser. This commit creates a (so far unused) package. It contains the a custom lexer/parser for the query language. ast.go: New AST that interacts well with the parser. lex.go: Custom lexer (new). lex_test.go: Lexer tests (new). parse.go: Custom parser (new). parse_test.go: Parser tests (new). functions.go: Changed function type, dummies for parser testing (barely changed/dummies). printer.go: Adapted from rules/ and adjusted to new AST (mostly unchanged, few additions). --- promql/ast.go | 345 ++++++++++++ promql/functions.go | 191 +++++++ promql/lex.go | 657 ++++++++++++++++++++++ promql/lex_test.go | 358 ++++++++++++ promql/parse.go | 867 +++++++++++++++++++++++++++++ promql/parse_test.go | 1077 +++++++++++++++++++++++++++++++++++++ promql/printer.go | 355 ++++++++++++ storage/metric/matcher.go | 5 + 8 files changed, 3855 insertions(+) create mode 100644 promql/ast.go create mode 100644 promql/functions.go create mode 100644 promql/lex.go create mode 100644 promql/lex_test.go create mode 100644 promql/parse.go create mode 100644 promql/parse_test.go create mode 100644 promql/printer.go diff --git a/promql/ast.go b/promql/ast.go new file mode 100644 index 000000000..861f10eb3 --- /dev/null +++ b/promql/ast.go @@ -0,0 +1,345 @@ +// 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" + "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 + // DotGraph returns a dot graph representation of the node. + DotGraph() 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 +} + +// 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 +) + +func (e ExprType) String() string { + switch e { + case ExprNone: + return "" + 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 + // Fingerprints are populated from label matchers at query analysis time. + fingerprints clientmodel.Fingerprints +} + +// 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 { + Str 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 + // Fingerprints are populated from label matchers at query analysis time. + fingerprints clientmodel.Fingerprints +} + +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) +} diff --git a/promql/functions.go b/promql/functions.go new file mode 100644 index 000000000..4d747ed75 --- /dev/null +++ b/promql/functions.go @@ -0,0 +1,191 @@ +// 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 + +// 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() +} + +var functions = map[string]*Function{ + "abs": { + Name: "abs", + ArgTypes: []ExprType{ExprVector}, + ReturnType: ExprVector, + Call: func() {}, + }, + "absent": { + Name: "absent", + ArgTypes: []ExprType{ExprVector}, + ReturnType: ExprVector, + Call: func() {}, + }, + "avg_over_time": { + Name: "avg_over_time", + ArgTypes: []ExprType{ExprMatrix}, + ReturnType: ExprVector, + Call: func() {}, + }, + "bottomk": { + Name: "bottomk", + ArgTypes: []ExprType{ExprScalar, ExprVector}, + ReturnType: ExprVector, + Call: func() {}, + }, + "ceil": { + Name: "ceil", + ArgTypes: []ExprType{ExprVector}, + ReturnType: ExprVector, + Call: func() {}, + }, + "count_over_time": { + Name: "count_over_time", + ArgTypes: []ExprType{ExprMatrix}, + ReturnType: ExprVector, + Call: func() {}, + }, + "count_scalar": { + Name: "count_scalar", + ArgTypes: []ExprType{ExprVector}, + ReturnType: ExprScalar, + Call: func() {}, + }, + "delta": { + Name: "delta", + ArgTypes: []ExprType{ExprMatrix, ExprScalar}, + OptionalArgs: 1, // The 2nd argument is deprecated. + ReturnType: ExprVector, + Call: func() {}, + }, + "deriv": { + Name: "deriv", + ArgTypes: []ExprType{ExprMatrix}, + ReturnType: ExprVector, + Call: func() {}, + }, + "drop_common_labels": { + Name: "drop_common_labels", + ArgTypes: []ExprType{ExprVector}, + ReturnType: ExprVector, + Call: func() {}, + }, + "exp": { + Name: "exp", + ArgTypes: []ExprType{ExprVector}, + ReturnType: ExprVector, + Call: func() {}, + }, + "floor": { + Name: "floor", + ArgTypes: []ExprType{ExprVector}, + ReturnType: ExprVector, + Call: func() {}, + }, + "histogram_quantile": { + Name: "histogram_quantile", + ArgTypes: []ExprType{ExprScalar, ExprVector}, + ReturnType: ExprVector, + Call: func() {}, + }, + "ln": { + Name: "ln", + ArgTypes: []ExprType{ExprVector}, + ReturnType: ExprVector, + Call: func() {}, + }, + "log10": { + Name: "log10", + ArgTypes: []ExprType{ExprVector}, + ReturnType: ExprVector, + Call: func() {}, + }, + "log2": { + Name: "log2", + ArgTypes: []ExprType{ExprVector}, + ReturnType: ExprVector, + Call: func() {}, + }, + "max_over_time": { + Name: "max_over_time", + ArgTypes: []ExprType{ExprMatrix}, + ReturnType: ExprVector, + Call: func() {}, + }, + "min_over_time": { + Name: "min_over_time", + ArgTypes: []ExprType{ExprMatrix}, + ReturnType: ExprVector, + Call: func() {}, + }, + "rate": { + Name: "rate", + ArgTypes: []ExprType{ExprMatrix}, + ReturnType: ExprVector, + Call: func() {}, + }, + "round": { + Name: "round", + ArgTypes: []ExprType{ExprVector, ExprScalar}, + OptionalArgs: 1, + ReturnType: ExprVector, + Call: func() {}, + }, + "scalar": { + Name: "scalar", + ArgTypes: []ExprType{ExprVector}, + ReturnType: ExprScalar, + Call: func() {}, + }, + "sort": { + Name: "sort", + ArgTypes: []ExprType{ExprVector}, + ReturnType: ExprVector, + Call: func() {}, + }, + "sort_desc": { + Name: "sort_desc", + ArgTypes: []ExprType{ExprVector}, + ReturnType: ExprVector, + Call: func() {}, + }, + "sum_over_time": { + Name: "sum_over_time", + ArgTypes: []ExprType{ExprMatrix}, + ReturnType: ExprVector, + Call: func() {}, + }, + "time": { + Name: "time", + ArgTypes: []ExprType{}, + ReturnType: ExprScalar, + Call: func() {}, + }, + "topk": { + Name: "topk", + ArgTypes: []ExprType{ExprScalar, ExprVector}, + ReturnType: ExprVector, + Call: func() {}, + }, +} + +// GetFunction returns a predefined Function object for the given name. +func GetFunction(name string) (*Function, bool) { + function, ok := functions[name] + return function, ok +} diff --git a/promql/lex.go b/promql/lex.go new file mode 100644 index 000000000..4a27bbcc5 --- /dev/null +++ b/promql/lex.go @@ -0,0 +1,657 @@ +// 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" + "strings" + "unicode" + "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.isKeyword(): + return fmt.Sprintf("<%s>", i.val) + case i.typ.isOperator(): + return fmt.Sprintf("", i.val) + case i.typ.isAggregator(): + return fmt.Sprintf("", 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 + + 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 + itemDescription + itemKeepingExtra + 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, + "description": itemDescription, + "offset": itemOffset, + "by": itemBy, + "keeping_extra": itemKeepingExtra, + "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{ + 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 + } +} + +func (t itemType) String() string { + if s, ok := itemTypeStr[t]; ok { + return s + } + return reflect.TypeOf(t).Name() +} + +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 { + name string // The name of the input; used only for error reports. + 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. +} + +// 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() Pos { + lb := Pos(strings.LastIndex(l.input[:l.lastPos], "\n")) + if lb == -1 { + return 1 + l.lastPos + } + return 1 + 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(name, input string) *lexer { + l := &lexer{ + name: name, + 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("unrecognized character after '=': %#U", t) + } else { + l.emit(itemAssign) + } + case r == '!': + if t := l.next(); t == '=' { + l.emit(itemNEQ) + } else { + return l.errorf("unrecognized character after '!': %#U", 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 '0' <= r && r <= '9' || r == '.': + l.backup() + return lexNumberOrDuration + case r == '"' || r == '\'': + l.stringOpen = r + return lexString + case r == 'N' || r == 'n' || r == 'I' || r == 'i': + n2 := strings.ToLower(l.input[l.pos:]) + if len(n2) < 3 || !isAlphaNumeric(rune(n2[2])) { + if (r == 'N' || r == 'n') && strings.HasPrefix(n2, "an") { + l.pos += 2 + l.emit(itemNumber) + break + } + if (r == 'I' || r == 'i') && strings.HasPrefix(n2, "nf") { + l.pos += 2 + l.emit(itemNumber) + break + } + } + fallthrough + case isAlphaNumeric(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 %#U", 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 %#U", r) + } + l.emit(itemLeftBracket) + l.bracketOpen = true + return lexDuration + case r == ']': + if !l.bracketOpen { + return l.errorf("unexpected right bracket %#U", r) + } + l.emit(itemRightBracket) + l.bracketOpen = false + + default: + return l.errorf("unrecognized character in statement: %#U", 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 EOF inside braces") + case isSpace(r): + return lexSpace + case isAlphaNumeric(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("unrecognized character after '!' inside braces: %#U", nr) + } + case r == '{': + return l.errorf("unexpected left brace %#U", r) + case r == '}': + l.emit(itemRightBrace) + l.braceOpen = false + return lexStatements + default: + return l.errorf("unrecognized character inside braces: %#U", r) + } + return lexInsideBraces +} + +// 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") && !isAlphaNumeric(l.peek()) { + 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") && !isAlphaNumeric(l.peek()) { + 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" + if 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. + if isAlphaNumeric(l.peek()) { + return false + } + return true +} + +// lexIdentifier scans an alphanumeric identifier. +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 + } + } + return lexStatements +} + +func isSpace(r rune) bool { + return r == ' ' || r == '\t' || r == '\n' +} + +// 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 r == '_' || unicode.IsLetter(r) || unicode.IsDigit(r) +} diff --git a/promql/lex_test.go b/promql/lex_test.go new file mode 100644 index 000000000..c9c45b58c --- /dev/null +++ b/promql/lex_test.go @@ -0,0 +1,358 @@ +// 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 +}{ + // 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, `]`}, + }, + }, + // 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"}}, + }, + // 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"}}, + }, + // 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{{itemKeepingExtra, 0, "keeping_extra"}}, + }, { + 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: "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: `{foo="bar"}`, + expected: []item{ + {itemLeftBrace, 0, `{`}, + {itemIdentifier, 1, `foo`}, + {itemEQL, 4, `=`}, + {itemString, 5, `"bar"`}, + {itemRightBrace, 10, `}`}, + }, + }, { + 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, + }, +} + +// 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 { + tn := fmt.Sprintf("test.%d \"%s\"", i, test.input) + l := lex(tn, test.input) + + out := []item{} + for it := range l.items { + out = append(out, it) + } + + lastItem := out[len(out)-1] + if test.fail { + if lastItem.typ != itemError { + t.Fatalf("%s: expected lexing error but did not fail", tn) + } + continue + } + if lastItem.typ == itemError { + t.Fatalf("%s: unexpected lexing error: %s", tn, lastItem) + } + + if !reflect.DeepEqual(lastItem, item{itemEOF, Pos(len(test.input)), ""}) { + t.Fatalf("%s: lexing error: expected output to end with EOF item", tn) + } + out = out[:len(out)-1] + if !reflect.DeepEqual(out, test.expected) { + t.Errorf("%s: lexing mismatch:\nexpected: %#v\n-----\ngot: %#v", tn, test.expected, out) + } + } +} diff --git a/promql/parse.go b/promql/parse.go new file mode 100644 index 000000000..a683782ad --- /dev/null +++ b/promql/parse.go @@ -0,0 +1,867 @@ +// 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" + "runtime" + "strconv" + "time" + + clientmodel "github.com/prometheus/client_golang/model" + "github.com/prometheus/prometheus/storage/metric" + "github.com/prometheus/prometheus/utility" +) + +type parser struct { + name string + lex *lexer + token [3]item + peekCount int +} + +// ParseStmts parses the input and returns the resulting statements or any ocurring error. +func ParseStmts(name, input string) (Statements, error) { + p := newParser(name, input) + + stmts, err := p.parseStmts() + if err != nil { + return nil, err + } + err = p.typecheck(stmts) + return stmts, err +} + +// ParseExpr returns the expression parsed from the input. +func ParseExpr(name, input string) (Expr, error) { + p := newParser(name, input) + + expr, err := p.parseExpr() + if err != nil { + return nil, err + } + err = p.typecheck(expr) + return expr, err +} + +// newParser returns a new parser. +func newParser(name, input string) *parser { + p := &parser{ + name: name, + lex: lex(name, input), + } + return p +} + +// parseStmts parses a sequence of statements from the input. +func (p *parser) parseStmts() (stmts Statements, err error) { + defer p.recover(&err) + stmts = Statements{} + + for p.peek().typ != itemEOF { + if p.peek().typ == itemComment { + continue + } + stmts = append(stmts, p.stmt()) + } + return +} + +// parseExpr parses a single expression from the input. +func (p *parser) parseExpr() (expr Expr, err error) { + defer p.recover(&err) + + for p.peek().typ != itemEOF { + if p.peek().typ == itemComment { + continue + } + if expr != nil { + p.errorf("expression read but input remaining") + } + expr = p.expr() + } + + if expr == nil { + p.errorf("no expression found in input") + } + return +} + +// typecheck checks correct typing of the parsed statements or expression. +func (p *parser) typecheck(node Node) (err error) { + defer p.recover(&err) + + p.checkType(node) + return nil +} + +// next returns the next token. +func (p *parser) next() item { + if p.peekCount > 0 { + p.peekCount-- + } else { + t := p.lex.nextItem() + // Skip comments. + for t.typ == itemComment { + t = p.lex.nextItem() + } + p.token[0] = t + } + return p.token[p.peekCount] +} + +// peek returns but does not consume the next token. +func (p *parser) peek() item { + if p.peekCount > 0 { + return p.token[p.peekCount-1] + } + p.peekCount = 1 + + t := p.lex.nextItem() + // Skip comments. + for t.typ == itemComment { + t = p.lex.nextItem() + } + p.token[0] = t + return p.token[0] +} + +// backup backs the input stream up one token. +func (p *parser) backup() { + p.peekCount++ +} + +// errorf formats the error and terminates processing. +func (p *parser) errorf(format string, args ...interface{}) { + format = fmt.Sprintf("%s:%d,%d %s", p.name, p.lex.lineNumber(), p.lex.linePosition(), format) + panic(fmt.Errorf(format, args...)) +} + +// error terminates processing. +func (p *parser) error(err error) { + p.errorf("%s", err) +} + +// expect consumes the next token and guarantees it has the required type. +func (p *parser) expect(expected itemType, context string) item { + token := p.next() + if token.typ != expected { + p.unexpected(token, context) + } + return token +} + +// expectOneOf consumes the next token and guarantees it has one of the required types. +func (p *parser) expectOneOf(expected1, expected2 itemType, context string) item { + token := p.next() + if token.typ != expected1 && token.typ != expected2 { + p.unexpected(token, context) + } + return token +} + +// unexpected complains about the token and terminates processing. +func (p *parser) unexpected(token item, context string) { + p.errorf("unexpected %s in %s", token, context) +} + +// recover is the handler that turns panics into returns from the top level of Parse. +func (p *parser) recover(errp *error) { + e := recover() + if e != nil { + if _, ok := e.(runtime.Error); ok { + panic(e) + } + *errp = e.(error) + } + return +} + +// stmt parses any statement. +// +// alertStatement | recordStatement +// +func (p *parser) stmt() Statement { + switch tok := p.peek(); tok.typ { + case itemAlert: + return p.alertStmt() + case itemIdentifier, itemMetricIdentifier: + return p.recordStmt() + } + p.errorf("no valid statement detected") + return nil +} + +// alertStmt parses an alert rule. +// +// ALERT name IF expr [FOR duration] [WITH label_set] +// SUMMARY "summary" +// DESCRIPTION "description" +// +func (p *parser) alertStmt() *AlertStmt { + const ctx = "alert statement" + + p.expect(itemAlert, ctx) + name := p.expect(itemIdentifier, ctx) + // Alerts require a vector typed expression. + p.expect(itemIf, ctx) + expr := p.expr() + + // Optional for clause. + var duration time.Duration + var err error + + if p.peek().typ == itemFor { + p.next() + dur := p.expect(itemDuration, ctx) + duration, err = parseDuration(dur.val) + if err != nil { + p.error(err) + } + } + + lset := clientmodel.LabelSet{} + if p.peek().typ == itemWith { + p.expect(itemWith, ctx) + lset = p.labelSet() + } + + p.expect(itemSummary, ctx) + sum, err := strconv.Unquote(p.expect(itemString, ctx).val) + if err != nil { + p.error(err) + } + + p.expect(itemDescription, ctx) + desc, err := strconv.Unquote(p.expect(itemString, ctx).val) + if err != nil { + p.error(err) + } + + return &AlertStmt{ + Name: name.val, + Expr: expr, + Duration: duration, + Labels: lset, + Summary: sum, + Description: desc, + } +} + +// recordStmt parses a recording rule. +func (p *parser) recordStmt() *RecordStmt { + const ctx = "record statement" + + name := p.expectOneOf(itemIdentifier, itemMetricIdentifier, ctx).val + + var lset clientmodel.LabelSet + if p.peek().typ == itemLeftBrace { + lset = p.labelSet() + } + + p.expect(itemAssign, ctx) + expr := p.expr() + + return &RecordStmt{ + Name: name, + Labels: lset, + Expr: expr, + } +} + +// expr parses any expression. +func (p *parser) expr() Expr { + const ctx = "binary expression" + + // Parse the starting expression. + expr := p.unaryExpr() + + // Loop through the operations and construct a binary operation tree based + // on the operators' precedence. + for { + // If the next token is not an operator the expression is done. + op := p.peek().typ + if !op.isOperator() { + return expr + } + p.next() // Consume operator. + + // Parse optional operator matching options. Its validity + // is checked in the type-checking stage. + vecMatching := &VectorMatching{ + Card: CardOneToOne, + } + if op == itemLAND || op == itemLOR { + vecMatching.Card = CardManyToMany + } + + // Parse ON clause. + if p.peek().typ == itemOn { + p.next() + vecMatching.On = p.labels() + + // Parse grouping. + if t := p.peek().typ; t == itemGroupLeft { + p.next() + vecMatching.Card = CardManyToOne + vecMatching.Include = p.labels() + } else if t == itemGroupRight { + p.next() + vecMatching.Card = CardOneToMany + vecMatching.Include = p.labels() + } + } + + for _, ln := range vecMatching.On { + for _, ln2 := range vecMatching.Include { + if ln == ln2 { + p.errorf("label %q must not occur in ON and INCLUDE clause at once", ln) + } + } + } + + // Parse the next operand. + rhs := p.unaryExpr() + + // Assign the new root based on the precendence of the LHS and RHS operators. + if lhs, ok := expr.(*BinaryExpr); ok && lhs.Op.precedence() < op.precedence() { + expr = &BinaryExpr{ + Op: lhs.Op, + LHS: lhs.LHS, + RHS: &BinaryExpr{ + Op: op, + LHS: lhs.RHS, + RHS: rhs, + VectorMatching: vecMatching, + }, + VectorMatching: lhs.VectorMatching, + } + } else { + expr = &BinaryExpr{ + Op: op, + LHS: expr, + RHS: rhs, + VectorMatching: vecMatching, + } + } + } + return nil +} + +// unaryExpr parses a unary expression. +// +// | | (+|-) | '(' ')' +// +func (p *parser) unaryExpr() Expr { + switch t := p.peek(); t.typ { + case itemADD, itemSUB: + p.next() + e := p.unaryExpr() + // Simplify unary expressions for number literals. + if nl, ok := e.(*NumberLiteral); ok { + if t.typ == itemSUB { + nl.Val *= -1 + } + return nl + } + return &UnaryExpr{Op: t.typ, Expr: e} + + case itemLeftParen: + p.next() + e := p.expr() + p.expect(itemRightParen, "paren expression") + + return &ParenExpr{Expr: e} + } + e := p.primaryExpr() + + // Expression might be followed by a range selector. + if p.peek().typ == itemLeftBracket { + vs, ok := e.(*VectorSelector) + if !ok { + p.errorf("range specification must be preceded by a metric selector, but follows a %T instead", e) + } + e = p.rangeSelector(vs) + } + return e +} + +// rangeSelector parses a matrix selector based on a given vector selector. +// +// '[' ']' +// +func (p *parser) rangeSelector(vs *VectorSelector) *MatrixSelector { + const ctx = "matrix selector" + p.next() + + var erange, offset time.Duration + var err error + + erangeStr := p.expect(itemDuration, ctx).val + erange, err = parseDuration(erangeStr) + if err != nil { + p.error(err) + } + + p.expect(itemRightBracket, ctx) + + // Parse optional offset. + if p.peek().typ == itemOffset { + p.next() + offi := p.expect(itemDuration, ctx) + + offset, err = parseDuration(offi.val) + if err != nil { + p.error(err) + } + } + + e := &MatrixSelector{ + Name: vs.Name, + LabelMatchers: vs.LabelMatchers, + Range: erange, + Offset: offset, + } + return e +} + +// primaryExpr parses a primary expression. +// +// | | | +// +func (p *parser) primaryExpr() Expr { + switch t := p.next(); { + case t.typ == itemNumber: + n, err := strconv.ParseInt(t.val, 0, 64) + f := float64(n) + if err != nil { + f, err = strconv.ParseFloat(t.val, 64) + } + if err != nil { + p.errorf("error parsing number: %s", err) + } + return &NumberLiteral{clientmodel.SampleValue(f)} + + case t.typ == itemString: + s := t.val[1 : len(t.val)-1] + return &StringLiteral{s} + + case t.typ == itemLeftBrace: + // Metric selector without metric name. + p.backup() + return p.vectorSelector("") + + case t.typ == itemIdentifier: + // Check for function call. + if p.peek().typ == itemLeftParen { + return p.call(t.val) + } + fallthrough // Else metric selector. + + case t.typ == itemMetricIdentifier: + return p.vectorSelector(t.val) + + case t.typ.isAggregator(): + p.backup() + return p.aggrExpr() + } + p.errorf("invalid primary expression") + return nil +} + +// labels parses a list of labelnames. +// +// '(' , ... ')' +// +func (p *parser) labels() clientmodel.LabelNames { + const ctx = "grouping opts" + + p.expect(itemLeftParen, ctx) + + labels := clientmodel.LabelNames{} + for { + id := p.expect(itemIdentifier, ctx) + labels = append(labels, clientmodel.LabelName(id.val)) + + if p.peek().typ != itemComma { + break + } + p.next() + } + p.expect(itemRightParen, ctx) + + return labels +} + +// aggrExpr parses an aggregation expression. +// +// () [by ] [keeping_extra] +// [by ] [keeping_extra] () +// +func (p *parser) aggrExpr() *AggregateExpr { + const ctx = "aggregation" + + agop := p.next() + if !agop.typ.isAggregator() { + p.errorf("%s is not an aggregation operator", agop) + } + var grouping clientmodel.LabelNames + var keepExtra bool + + modifiersFirst := false + + if p.peek().typ == itemBy { + p.next() + grouping = p.labels() + modifiersFirst = true + } + if p.peek().typ == itemKeepingExtra { + p.next() + keepExtra = true + modifiersFirst = true + } + + p.expect(itemLeftParen, ctx) + e := p.expr() + p.expect(itemRightParen, ctx) + + if !modifiersFirst { + if p.peek().typ == itemBy { + if len(grouping) > 0 { + p.errorf("aggregation must only contain one grouping clause") + } + p.next() + grouping = p.labels() + } + if p.peek().typ == itemKeepingExtra { + p.next() + keepExtra = true + } + } + + return &AggregateExpr{ + Op: agop.typ, + Expr: e, + Grouping: grouping, + KeepExtraLabels: keepExtra, + } +} + +// call parses a function call. +// +// '(' [ , ...] ')' +// +func (p *parser) call(name string) *Call { + const ctx = "function call" + + fn, exist := GetFunction(name) + if !exist { + p.errorf("unknown function with name %q", name) + } + + p.expect(itemLeftParen, ctx) + // Might be call without args. + if p.peek().typ == itemRightParen { + p.next() // Consume. + return &Call{fn, nil} + } + + var args []Expr + for { + e := p.expr() + args = append(args, e) + + // Terminate if no more arguments. + if p.peek().typ != itemComma { + break + } + p.next() + } + + // Call must be closed. + p.expect(itemRightParen, ctx) + + return &Call{Func: fn, Args: args} +} + +// labelSet parses a set of label matchers +// +// '{' [ '=' , ... ] '}' +// +func (p *parser) labelSet() clientmodel.LabelSet { + set := clientmodel.LabelSet{} + for _, lm := range p.labelMatchers(itemEQL) { + set[lm.Name] = lm.Value + } + return set +} + +// labelMatchers parses a set of label matchers. +// +// '{' [ , ... ] '}' +// +func (p *parser) labelMatchers(operators ...itemType) metric.LabelMatchers { + const ctx = "label matching" + + matchers := metric.LabelMatchers{} + + p.expect(itemLeftBrace, ctx) + + // Check if no matchers are provided. + if p.peek().typ == itemRightBrace { + p.next() + return matchers + } + + for { + label := p.expect(itemIdentifier, ctx) + + op := p.next().typ + if !op.isOperator() { + p.errorf("item %s is not a valid operator for label matching", op) + } + var validOp = false + for _, allowedOp := range operators { + if op == allowedOp { + validOp = true + } + } + if !validOp { + p.errorf("operator must be one of %q, is %q", operators, op) + } + + val, err := strconv.Unquote(p.expect(itemString, ctx).val) + if err != nil { + p.error(err) + } + + // Map the item to the respective match type. + var matchType metric.MatchType + switch op { + case itemEQL: + matchType = metric.Equal + case itemNEQ: + matchType = metric.NotEqual + case itemEQLRegex: + matchType = metric.RegexMatch + case itemNEQRegex: + matchType = metric.RegexNoMatch + default: + p.errorf("item %q is not a metric match type", op) + } + + m, err := metric.NewLabelMatcher( + matchType, + clientmodel.LabelName(label.val), + clientmodel.LabelValue(val), + ) + if err != nil { + p.error(err) + } + + matchers = append(matchers, m) + + // Terminate list if last matcher. + if p.peek().typ != itemComma { + break + } + p.next() + } + + p.expect(itemRightBrace, ctx) + + return matchers +} + +// metricSelector parses a new metric selector. +// +// [] [ offset ] +// [] [ offset ] +// +func (p *parser) vectorSelector(name string) *VectorSelector { + const ctx = "metric selector" + + var matchers metric.LabelMatchers + // Parse label matching if any. + if t := p.peek(); t.typ == itemLeftBrace { + matchers = p.labelMatchers(itemEQL, itemNEQ, itemEQLRegex, itemNEQRegex) + } + // Metric name must not be set in the label matchers and before at the same time. + if name != "" { + for _, m := range matchers { + if m.Name == clientmodel.MetricNameLabel { + p.errorf("metric name must not be set twice: %q or %q", name, m.Value) + } + } + // Set name label matching. + matchers = append(matchers, &metric.LabelMatcher{ + Type: metric.Equal, + Name: clientmodel.MetricNameLabel, + Value: clientmodel.LabelValue(name), + }) + } + + if len(matchers) == 0 { + p.errorf("vector selector must contain label matchers or metric name") + } + + var err error + var offset time.Duration + // Parse optional offset. + if p.peek().typ == itemOffset { + p.next() + offi := p.expect(itemDuration, ctx) + + offset, err = parseDuration(offi.val) + if err != nil { + p.error(err) + } + } + return &VectorSelector{ + Name: name, + LabelMatchers: matchers, + Offset: offset, + } +} + +// expectType checks the type of the node and raises an error if it +// is not of the expected type. +func (p *parser) expectType(node Node, want ExprType, context string) { + t := p.checkType(node) + if t != want { + p.errorf("expected type %s in %s, got %s", want, context, t) + } +} + +// check the types of the children of each node and raise an error +// if they do not form a valid node. +// +// Some of these checks are redundant as the the parsing stage does not allow +// them, but the costs are small and might reveal errors when making changes. +func (p *parser) checkType(node Node) (typ ExprType) { + // For expressions the type is determined by their Type function. + // Statements and lists do not have a type but are not invalid either. + switch n := node.(type) { + case Statements, Expressions, Statement: + typ = ExprNone + case Expr: + typ = n.Type() + default: + p.errorf("unknown node type: %T", node) + } + + // Recursively check correct typing for child nodes and raise + // errors in case of bad typing. + switch n := node.(type) { + case Statements: + for _, s := range n { + p.expectType(s, ExprNone, "statement list") + } + case *AlertStmt: + p.expectType(n.Expr, ExprVector, "alert statement") + + case *EvalStmt: + ty := p.checkType(n.Expr) + if ty == ExprNone { + p.errorf("evaluation statement must have a valid expression type but got %s", ty) + } + + case *RecordStmt: + p.expectType(n.Expr, ExprVector, "record statement") + + case Expressions: + for _, e := range n { + ty := p.checkType(e) + if ty == ExprNone { + p.errorf("expression must have a valid expression type but got %s", ty) + } + } + case *AggregateExpr: + if !n.Op.isAggregator() { + p.errorf("aggregation operator expected in aggregation expression but got %q", n.Op) + } + p.expectType(n.Expr, ExprVector, "aggregation expression") + + case *BinaryExpr: + lt := p.checkType(n.LHS) + rt := p.checkType(n.RHS) + + if !n.Op.isOperator() { + p.errorf("only logical and arithmetic operators allowed in binary expression, got %q", n.Op) + } + if (lt != ExprScalar && lt != ExprVector) || (rt != ExprScalar && rt != ExprVector) { + p.errorf("binary expression must contain only scalar and vector types") + } + + if (lt != ExprVector || rt != ExprVector) && n.VectorMatching != nil { + if len(n.VectorMatching.On) > 0 { + p.errorf("vector matching only allowed between vectors") + } + n.VectorMatching = nil + } else { + // Both operands are vectors. + if n.Op == itemLAND || n.Op == itemLOR { + if n.VectorMatching.Card == CardOneToMany || n.VectorMatching.Card == CardManyToOne { + p.errorf("no grouping allowed for AND and OR operations") + } + if n.VectorMatching.Card != CardManyToMany { + p.errorf("AND and OR operations must always be many-to-many") + } + } + } + + if (lt == ExprScalar || rt == ExprScalar) && (n.Op == itemLAND || n.Op == itemLOR) { + p.errorf("AND and OR not allowed in binary scalar expression") + } + + case *Call: + nargs := len(n.Func.ArgTypes) + if na := nargs - n.Func.OptionalArgs; na > len(n.Args) { + p.errorf("expected at least %d arguments in call to %q, got %d", na, n.Func.Name, len(n.Args)) + } + if nargs < len(n.Args) { + p.errorf("expected at most %d arguments in call to %q, got %d", nargs, n.Func.Name, len(n.Args)) + } + for i, arg := range n.Args { + p.expectType(arg, n.Func.ArgTypes[i], fmt.Sprintf("call to function %q", n.Func.Name)) + } + + case *ParenExpr: + p.checkType(n.Expr) + + case *UnaryExpr: + if n.Op != itemADD && n.Op != itemSUB { + p.errorf("only + and - operators allowed for unary expressions") + } + p.expectType(n.Expr, ExprScalar, "unary expression") + + case *NumberLiteral, *MatrixSelector, *StringLiteral, *VectorSelector: + // Nothing to do for terminals. + + default: + p.errorf("unknown node type: %T", node) + } + return +} + +func parseDuration(ds string) (time.Duration, error) { + dur, err := utility.StringToDuration(ds) + if err != nil { + return 0, err + } + if dur == 0 { + return 0, fmt.Errorf("duration must be greater than 0") + } + return dur, nil +} diff --git a/promql/parse_test.go b/promql/parse_test.go new file mode 100644 index 000000000..fc990c0a1 --- /dev/null +++ b/promql/parse_test.go @@ -0,0 +1,1077 @@ +// 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" + "math" + "reflect" + "testing" + "time" + + clientmodel "github.com/prometheus/client_golang/model" + "github.com/prometheus/prometheus/storage/metric" +) + +var testExpr = []struct { + input string + expected Expr + fail bool +}{ + // Scalars and scalar-to-scalar operations. + { + input: "1", + expected: &NumberLiteral{1}, + }, { + input: "+Inf", + expected: &NumberLiteral{clientmodel.SampleValue(math.Inf(1))}, + }, { + input: "-Inf", + expected: &NumberLiteral{clientmodel.SampleValue(math.Inf(-1))}, + }, { + input: ".5", + expected: &NumberLiteral{0.5}, + }, { + input: "5.", + expected: &NumberLiteral{5}, + }, { + input: "123.4567", + expected: &NumberLiteral{123.4567}, + }, { + input: "5e-3", + expected: &NumberLiteral{0.005}, + }, { + input: "5e3", + expected: &NumberLiteral{5000}, + }, { + input: "0xc", + expected: &NumberLiteral{12}, + }, { + input: "0755", + expected: &NumberLiteral{493}, + }, { + input: "+5.5e-3", + expected: &NumberLiteral{0.0055}, + }, { + input: "-0755", + expected: &NumberLiteral{-493}, + }, { + input: "1 + 1", + expected: &BinaryExpr{itemADD, &NumberLiteral{1}, &NumberLiteral{1}, nil}, + }, { + input: "1 - 1", + expected: &BinaryExpr{itemSUB, &NumberLiteral{1}, &NumberLiteral{1}, nil}, + }, { + input: "1 * 1", + expected: &BinaryExpr{itemMUL, &NumberLiteral{1}, &NumberLiteral{1}, nil}, + }, { + input: "1 % 1", + expected: &BinaryExpr{itemMOD, &NumberLiteral{1}, &NumberLiteral{1}, nil}, + }, { + input: "1 / 1", + expected: &BinaryExpr{itemDIV, &NumberLiteral{1}, &NumberLiteral{1}, nil}, + }, { + input: "1 == 1", + expected: &BinaryExpr{itemEQL, &NumberLiteral{1}, &NumberLiteral{1}, nil}, + }, { + input: "1 != 1", + expected: &BinaryExpr{itemNEQ, &NumberLiteral{1}, &NumberLiteral{1}, nil}, + }, { + input: "1 > 1", + expected: &BinaryExpr{itemGTR, &NumberLiteral{1}, &NumberLiteral{1}, nil}, + }, { + input: "1 >= 1", + expected: &BinaryExpr{itemGTE, &NumberLiteral{1}, &NumberLiteral{1}, nil}, + }, { + input: "1 < 1", + expected: &BinaryExpr{itemLSS, &NumberLiteral{1}, &NumberLiteral{1}, nil}, + }, { + input: "1 <= 1", + expected: &BinaryExpr{itemLTE, &NumberLiteral{1}, &NumberLiteral{1}, nil}, + }, { + input: "+1 + -2 * 1", + expected: &BinaryExpr{ + Op: itemADD, + LHS: &NumberLiteral{1}, + RHS: &BinaryExpr{ + Op: itemMUL, LHS: &NumberLiteral{-2}, RHS: &NumberLiteral{1}, + }, + }, + }, { + input: "1 + 2/(3*1)", + expected: &BinaryExpr{ + Op: itemADD, + LHS: &NumberLiteral{1}, + RHS: &BinaryExpr{ + Op: itemDIV, + LHS: &NumberLiteral{2}, + RHS: &ParenExpr{&BinaryExpr{ + Op: itemMUL, LHS: &NumberLiteral{3}, RHS: &NumberLiteral{1}, + }}, + }, + }, + }, { + input: "", fail: true, + }, { + input: "# just a comment\n\n", fail: true, + }, { + input: "1+", fail: true, + }, { + input: "2.5.", fail: true, + }, { + input: "100..4", fail: true, + }, { + input: "0deadbeef", fail: true, + }, { + input: "1 /", fail: true, + }, { + input: "*1", fail: true, + }, { + input: "(1))", fail: true, + }, { + input: "((1)", fail: true, + }, { + input: "(", fail: true, + }, { + input: "1 and 1", fail: true, + }, { + input: "1 or 1", fail: true, + }, { + input: "1 !~ 1", fail: true, + }, { + input: "1 =~ 1", fail: true, + }, { + input: "-some_metric", fail: true, + }, { + input: `-"string"`, fail: true, + }, + // Vector binary operations. + { + input: "foo * bar", + expected: &BinaryExpr{ + Op: itemMUL, + LHS: &VectorSelector{ + Name: "foo", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "foo"}, + }, + }, + RHS: &VectorSelector{ + Name: "bar", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "bar"}, + }, + }, + VectorMatching: &VectorMatching{Card: CardOneToOne}, + }, + }, { + input: "foo == 1", + expected: &BinaryExpr{ + Op: itemEQL, + LHS: &VectorSelector{ + Name: "foo", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "foo"}, + }, + }, + RHS: &NumberLiteral{1}, + }, + }, { + input: "2.5 / bar", + expected: &BinaryExpr{ + Op: itemDIV, + LHS: &NumberLiteral{2.5}, + RHS: &VectorSelector{ + Name: "bar", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "bar"}, + }, + }, + }, + }, { + input: "foo and bar", + expected: &BinaryExpr{ + Op: itemLAND, + LHS: &VectorSelector{ + Name: "foo", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "foo"}, + }, + }, + RHS: &VectorSelector{ + Name: "bar", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "bar"}, + }, + }, + VectorMatching: &VectorMatching{Card: CardManyToMany}, + }, + }, { + input: "foo or bar", + expected: &BinaryExpr{ + Op: itemLOR, + LHS: &VectorSelector{ + Name: "foo", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "foo"}, + }, + }, + RHS: &VectorSelector{ + Name: "bar", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "bar"}, + }, + }, + VectorMatching: &VectorMatching{Card: CardManyToMany}, + }, + }, { + // Test and/or precedence and reassigning of operands. + input: "foo + bar or bla and blub", + expected: &BinaryExpr{ + Op: itemLOR, + LHS: &BinaryExpr{ + Op: itemADD, + LHS: &VectorSelector{ + Name: "foo", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "foo"}, + }, + }, + RHS: &VectorSelector{ + Name: "bar", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "bar"}, + }, + }, + VectorMatching: &VectorMatching{Card: CardOneToOne}, + }, + RHS: &BinaryExpr{ + Op: itemLAND, + LHS: &VectorSelector{ + Name: "bla", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "bla"}, + }, + }, + RHS: &VectorSelector{ + Name: "blub", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "blub"}, + }, + }, + VectorMatching: &VectorMatching{Card: CardManyToMany}, + }, + VectorMatching: &VectorMatching{Card: CardManyToMany}, + }, + }, { + // Test precedence and reassigning of operands. + input: "bar + on(foo) bla / on(baz, buz) group_right(test) blub", + expected: &BinaryExpr{ + Op: itemADD, + LHS: &VectorSelector{ + Name: "bar", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "bar"}, + }, + }, + RHS: &BinaryExpr{ + Op: itemDIV, + LHS: &VectorSelector{ + Name: "bla", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "bla"}, + }, + }, + RHS: &VectorSelector{ + Name: "blub", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "blub"}, + }, + }, + VectorMatching: &VectorMatching{ + Card: CardOneToMany, + On: clientmodel.LabelNames{"baz", "buz"}, + Include: clientmodel.LabelNames{"test"}, + }, + }, + VectorMatching: &VectorMatching{ + Card: CardOneToOne, + On: clientmodel.LabelNames{"foo"}, + }, + }, + }, { + input: "foo * on(test,blub) bar", + expected: &BinaryExpr{ + Op: itemMUL, + LHS: &VectorSelector{ + Name: "foo", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "foo"}, + }, + }, + RHS: &VectorSelector{ + Name: "bar", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "bar"}, + }, + }, + VectorMatching: &VectorMatching{ + Card: CardOneToOne, + On: clientmodel.LabelNames{"test", "blub"}, + }, + }, + }, { + input: "foo and on(test,blub) bar", + expected: &BinaryExpr{ + Op: itemLAND, + LHS: &VectorSelector{ + Name: "foo", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "foo"}, + }, + }, + RHS: &VectorSelector{ + Name: "bar", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "bar"}, + }, + }, + VectorMatching: &VectorMatching{ + Card: CardManyToMany, + On: clientmodel.LabelNames{"test", "blub"}, + }, + }, + }, { + input: "foo / on(test,blub) group_left(bar) bar", + expected: &BinaryExpr{ + Op: itemDIV, + LHS: &VectorSelector{ + Name: "foo", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "foo"}, + }, + }, + RHS: &VectorSelector{ + Name: "bar", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "bar"}, + }, + }, + VectorMatching: &VectorMatching{ + Card: CardManyToOne, + On: clientmodel.LabelNames{"test", "blub"}, + Include: clientmodel.LabelNames{"bar"}, + }, + }, + }, { + input: "foo - on(test,blub) group_right(bar,foo) bar", + expected: &BinaryExpr{ + Op: itemSUB, + LHS: &VectorSelector{ + Name: "foo", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "foo"}, + }, + }, + RHS: &VectorSelector{ + Name: "bar", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "bar"}, + }, + }, + VectorMatching: &VectorMatching{ + Card: CardOneToMany, + On: clientmodel.LabelNames{"test", "blub"}, + Include: clientmodel.LabelNames{"bar", "foo"}, + }, + }, + }, { + input: "foo and 1", fail: true, + }, { + input: "1 and foo", fail: true, + }, { + input: "foo or 1", fail: true, + }, { + input: "1 or foo", fail: true, + }, { + input: "1 or on(bar) foo", fail: true, + }, { + input: "foo == on(bar) 10", fail: true, + }, { + input: "foo and on(bar) group_left(baz) bar", fail: true, + }, { + input: "foo and on(bar) group_right(baz) bar", fail: true, + }, { + input: "foo or on(bar) group_left(baz) bar", fail: true, + }, { + input: "foo or on(bar) group_right(baz) bar", fail: true, + }, + // Test vector selector. + { + input: "foo", + expected: &VectorSelector{ + Name: "foo", + Offset: 0, + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "foo"}, + }, + }, + }, { + input: "foo offset 5m", + expected: &VectorSelector{ + Name: "foo", + Offset: 5 * time.Minute, + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "foo"}, + }, + }, + }, { + input: `foo:bar{a="b"}`, + expected: &VectorSelector{ + Name: "foo:bar", + Offset: 0, + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: "a", Value: "b"}, + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "foo:bar"}, + }, + }, + }, { + input: `foo{NaN='b'}`, + expected: &VectorSelector{ + Name: "foo", + Offset: 0, + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: "NaN", Value: "b"}, + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "foo"}, + }, + }, + }, { + input: `foo{a="b", foo!="bar", test=~"test", bar!~"baz"}`, + expected: &VectorSelector{ + Name: "foo", + Offset: 0, + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: "a", Value: "b"}, + {Type: metric.NotEqual, Name: "foo", Value: "bar"}, + mustLabelMatcher(metric.RegexMatch, "test", "test"), + mustLabelMatcher(metric.RegexNoMatch, "bar", "baz"), + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "foo"}, + }, + }, + }, { + input: `{`, fail: true, + }, { + input: `}`, fail: true, + }, { + input: `some{`, fail: true, + }, { + input: `some}`, fail: true, + }, { + input: `some_metric{a=b}`, fail: true, + }, { + input: `some_metric{a:b="b"}`, fail: true, + }, { + input: `foo{a*"b"}`, fail: true, + }, { + input: `foo{a>="b"}`, fail: true, + }, { + input: `foo{gibberish}`, fail: true, + }, { + input: `foo{1}`, fail: true, + }, { + input: `{}`, fail: true, + }, { + input: `foo{__name__="bar"}`, fail: true, + }, { + input: `:foo`, fail: true, + }, + // Test matrix selector. + { + input: "test[5s]", + expected: &MatrixSelector{ + Name: "test", + Offset: 0, + Range: 5 * time.Second, + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "test"}, + }, + }, + }, { + input: "test[5m]", + expected: &MatrixSelector{ + Name: "test", + Offset: 0, + Range: 5 * time.Minute, + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "test"}, + }, + }, + }, { + input: "test[5h] OFFSET 5m", + expected: &MatrixSelector{ + Name: "test", + Offset: 5 * time.Minute, + Range: 5 * time.Hour, + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "test"}, + }, + }, + }, { + input: "test[5d] OFFSET 10s", + expected: &MatrixSelector{ + Name: "test", + Offset: 10 * time.Second, + Range: 5 * 24 * time.Hour, + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "test"}, + }, + }, + }, { + input: "test[5w] offset 2w", + expected: &MatrixSelector{ + Name: "test", + Offset: 14 * 24 * time.Hour, + Range: 5 * 7 * 24 * time.Hour, + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "test"}, + }, + }, + }, { + input: `test{a="b"}[5y] OFFSET 3d`, + expected: &MatrixSelector{ + Name: "test", + Offset: 3 * 24 * time.Hour, + Range: 5 * 365 * 24 * time.Hour, + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: "a", Value: "b"}, + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "test"}, + }, + }, + }, { + input: `foo[5mm]`, fail: true, + }, { + input: `foo[0m]`, fail: true, + }, { + input: `foo[5m30s]`, fail: true, + }, { + input: `foo[5m] OFFSET 1h30m`, fail: true, + }, { + input: `foo[]`, fail: true, + }, { + input: `foo[1]`, fail: true, + }, { + input: `some_metric[5m] OFFSET 1`, fail: true, + }, { + input: `some_metric[5m] OFFSET 1mm`, fail: true, + }, { + input: `some_metric[5m] OFFSET`, fail: true, + }, { + input: `(foo + bar)[5m]`, fail: true, + }, + // Test aggregation. + { + input: "sum by (foo)(some_metric)", + expected: &AggregateExpr{ + Op: itemSum, + Expr: &VectorSelector{ + Name: "some_metric", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "some_metric"}, + }, + }, + Grouping: clientmodel.LabelNames{"foo"}, + }, + }, { + input: "sum by (foo) keeping_extra (some_metric)", + expected: &AggregateExpr{ + Op: itemSum, + KeepExtraLabels: true, + Expr: &VectorSelector{ + Name: "some_metric", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "some_metric"}, + }, + }, + Grouping: clientmodel.LabelNames{"foo"}, + }, + }, { + input: "sum (some_metric) by (foo,bar) keeping_extra", + expected: &AggregateExpr{ + Op: itemSum, + KeepExtraLabels: true, + Expr: &VectorSelector{ + Name: "some_metric", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "some_metric"}, + }, + }, + Grouping: clientmodel.LabelNames{"foo", "bar"}, + }, + }, { + input: "avg by (foo)(some_metric)", + expected: &AggregateExpr{ + Op: itemAvg, + Expr: &VectorSelector{ + Name: "some_metric", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "some_metric"}, + }, + }, + Grouping: clientmodel.LabelNames{"foo"}, + }, + }, { + input: "COUNT by (foo) keeping_extra (some_metric)", + expected: &AggregateExpr{ + Op: itemCount, + Expr: &VectorSelector{ + Name: "some_metric", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "some_metric"}, + }, + }, + Grouping: clientmodel.LabelNames{"foo"}, + KeepExtraLabels: true, + }, + }, { + input: "MIN (some_metric) by (foo) keeping_extra", + expected: &AggregateExpr{ + Op: itemMin, + Expr: &VectorSelector{ + Name: "some_metric", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "some_metric"}, + }, + }, + Grouping: clientmodel.LabelNames{"foo"}, + KeepExtraLabels: true, + }, + }, { + input: "max by (foo)(some_metric)", + expected: &AggregateExpr{ + Op: itemMax, + Expr: &VectorSelector{ + Name: "some_metric", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "some_metric"}, + }, + }, + Grouping: clientmodel.LabelNames{"foo"}, + }, + }, { + input: "stddev(some_metric)", + expected: &AggregateExpr{ + Op: itemStddev, + Expr: &VectorSelector{ + Name: "some_metric", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "some_metric"}, + }, + }, + }, + }, { + input: "stdvar by (foo)(some_metric)", + expected: &AggregateExpr{ + Op: itemStdvar, + Expr: &VectorSelector{ + Name: "some_metric", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "some_metric"}, + }, + }, + Grouping: clientmodel.LabelNames{"foo"}, + }, + }, { + input: `sum some_metric by (test)`, fail: true, + }, { + input: `sum (some_metric) by test`, fail: true, + }, { + input: `sum (some_metric) by ()`, fail: true, + }, { + input: `sum (some_metric) by test`, fail: true, + }, { + input: `some_metric[5m] OFFSET`, fail: true, + }, { + input: `sum () by (test)`, fail: true, + }, { + input: "MIN keeping_extra (some_metric) by (foo)", fail: true, + }, { + input: "MIN by(test) (some_metric) keeping_extra", fail: true, + }, + // Test function calls. + { + input: "time()", + expected: &Call{ + Func: mustGetFunction("time"), + }, + }, { + input: `floor(some_metric{foo!="bar"})`, + expected: &Call{ + Func: mustGetFunction("floor"), + Args: Expressions{ + &VectorSelector{ + Name: "some_metric", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.NotEqual, Name: "foo", Value: "bar"}, + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "some_metric"}, + }, + }, + }, + }, + }, { + input: "rate(some_metric[5m])", + expected: &Call{ + Func: mustGetFunction("rate"), + Args: Expressions{ + &MatrixSelector{ + Name: "some_metric", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "some_metric"}, + }, + Range: 5 * time.Minute, + }, + }, + }, + }, { + input: "round(some_metric)", + expected: &Call{ + Func: mustGetFunction("round"), + Args: Expressions{ + &VectorSelector{ + Name: "some_metric", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "some_metric"}, + }, + }, + }, + }, + }, { + input: "round(some_metric, 5)", + expected: &Call{ + Func: mustGetFunction("round"), + Args: Expressions{ + &VectorSelector{ + Name: "some_metric", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "some_metric"}, + }, + }, + &NumberLiteral{5}, + }, + }, + }, { + input: "floor()", fail: true, + }, { + input: "floor(some_metric, other_metric)", fail: true, + }, { + input: "floor(1)", fail: true, + }, { + input: "non_existant_function_far_bar()", fail: true, + }, { + input: "rate(some_metric)", fail: true, + }, +} + +func TestParseExpressions(t *testing.T) { + for _, test := range testExpr { + + parser := newParser("test", test.input) + + expr, err := parser.parseExpr() + if !test.fail && err != nil { + t.Errorf("error in input '%s'", test.input) + t.Fatalf("could not parse: %s", err) + } + if test.fail && err != nil { + continue + } + + err = parser.typecheck(expr) + if !test.fail && err != nil { + t.Errorf("error on input '%s'", test.input) + t.Fatalf("typecheck failed: %s", err) + } + + if test.fail { + if err != nil { + continue + } + t.Errorf("error on input '%s'", test.input) + t.Fatalf("failure expected, but passed with result: %q", expr) + } + + if !reflect.DeepEqual(expr, test.expected) { + t.Errorf("error on input '%s'", test.input) + t.Fatalf("no match\n\nexpected:\n%s\ngot: \n%s\n", Tree(test.expected), Tree(expr)) + } + } +} + +// NaN has no equality. Thus, we need a separate test for it. +func TestNaNExpression(t *testing.T) { + parser := newParser("test", "NaN") + + expr, err := parser.parseExpr() + if err != nil { + t.Errorf("error on input 'NaN'") + t.Fatalf("coud not parse: %s", err) + } + + nl, ok := expr.(*NumberLiteral) + if !ok { + t.Errorf("error on input 'NaN'") + t.Fatalf("expected number literal but got %T", expr) + } + + if !math.IsNaN(float64(nl.Val)) { + t.Errorf("error on input 'NaN'") + t.Fatalf("expected 'NaN' in number literal but got %d", nl.Val) + } +} + +var testStatement = []struct { + input string + expected Statements + fail bool +}{ + { + // Test a file-like input. + input: ` + # A simple test recording rule. + dc:http_request:rate5m = sum(rate(http_request_count[5m])) by (dc) + + # A simple test alerting rule. + ALERT GlobalRequestRateLow IF(dc:http_request:rate5m < 10000) FOR 5m WITH { + service = "testservice" + # ... more fields here ... + } + SUMMARY "Global request rate low" + DESCRIPTION "The global request rate is low" + + foo = bar{label1="value1"} + + ALERT BazAlert IF foo > 10 WITH {} + SUMMARY "Baz" + DESCRIPTION "BazAlert" + `, + expected: Statements{ + &RecordStmt{ + Name: "dc:http_request:rate5m", + Expr: &AggregateExpr{ + Op: itemSum, + Grouping: clientmodel.LabelNames{"dc"}, + Expr: &Call{ + Func: mustGetFunction("rate"), + Args: Expressions{ + &MatrixSelector{ + Name: "http_request_count", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "http_request_count"}, + }, + Range: 5 * time.Minute, + }, + }, + }, + }, + Labels: nil, + }, + &AlertStmt{ + Name: "GlobalRequestRateLow", + Expr: &ParenExpr{&BinaryExpr{ + Op: itemLSS, + LHS: &VectorSelector{ + Name: "dc:http_request:rate5m", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "dc:http_request:rate5m"}, + }, + }, + RHS: &NumberLiteral{10000}, + }}, + Labels: clientmodel.LabelSet{"service": "testservice"}, + Duration: 5 * time.Minute, + Summary: "Global request rate low", + Description: "The global request rate is low", + }, + &RecordStmt{ + Name: "foo", + Expr: &VectorSelector{ + Name: "bar", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: "label1", Value: "value1"}, + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "bar"}, + }, + }, + Labels: nil, + }, + &AlertStmt{ + Name: "BazAlert", + Expr: &BinaryExpr{ + Op: itemGTR, + LHS: &VectorSelector{ + Name: "foo", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "foo"}, + }, + }, + RHS: &NumberLiteral{10}, + }, + Labels: clientmodel.LabelSet{}, + Summary: "Baz", + Description: "BazAlert", + }, + }, + }, { + input: `foo{x="", a="z"} = bar{a="b", x=~"y"}`, + expected: Statements{ + &RecordStmt{ + Name: "foo", + Expr: &VectorSelector{ + Name: "bar", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: "a", Value: "b"}, + mustLabelMatcher(metric.RegexMatch, "x", "y"), + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "bar"}, + }, + }, + Labels: clientmodel.LabelSet{"x": "", "a": "z"}, + }, + }, + }, { + input: `ALERT SomeName IF some_metric > 1 + SUMMARY "Global request rate low" + DESCRIPTION "The global request rate is low" + `, + expected: Statements{ + &AlertStmt{ + Name: "SomeName", + Expr: &BinaryExpr{ + Op: itemGTR, + LHS: &VectorSelector{ + Name: "some_metric", + LabelMatchers: metric.LabelMatchers{ + {Type: metric.Equal, Name: clientmodel.MetricNameLabel, Value: "some_metric"}, + }, + }, + RHS: &NumberLiteral{1}, + }, + Labels: clientmodel.LabelSet{}, + Summary: "Global request rate low", + Description: "The global request rate is low", + }, + }, + }, { + input: ` + # A simple test alerting rule. + ALERT GlobalRequestRateLow IF(dc:http_request:rate5m < 10000) FOR 5 WITH { + service = "testservice" + # ... more fields here ... + } + SUMMARY "Global request rate low" + DESCRIPTION "The global request rate is low" + `, + fail: true, + }, { + input: "", + expected: Statements{}, + }, { + input: "foo = time()", + fail: true, + }, { + input: "foo = 1", + fail: true, + }, { + input: "foo = bar[5m]", + fail: true, + }, { + input: `foo = "test"`, + fail: true, + }, { + input: `foo = `, + fail: true, + }, { + input: `foo{a!="b"} = bar`, + fail: true, + }, { + input: `foo{a=~"b"} = bar`, + fail: true, + }, { + input: `foo{a!~"b"} = bar`, + fail: true, + }, { + input: `ALERT SomeName IF time() WITH {} + SUMMARY "Global request rate low" + DESCRIPTION "The global request rate is low" + `, + fail: true, + }, { + input: `ALERT SomeName IF some_metric > 1 WITH {} + SUMMARY "Global request rate low" + `, + fail: true, + }, { + input: `ALERT SomeName IF some_metric > 1 + DESCRIPTION "The global request rate is low" + `, + fail: true, + }, +} + +func TestParseStatements(t *testing.T) { + for _, test := range testStatement { + parser := newParser("test", test.input) + + stmts, err := parser.parseStmts() + if !test.fail && err != nil { + t.Errorf("error in input: \n\n%s\n", test.input) + t.Fatalf("could not parse: %s", err) + } + if test.fail && err != nil { + continue + } + + err = parser.typecheck(stmts) + if !test.fail && err != nil { + t.Errorf("error in input: \n\n%s\n", test.input) + t.Fatalf("typecheck failed: %s", err) + } + + if test.fail { + if err != nil { + continue + } + t.Errorf("error in input: \n\n%s\n", test.input) + t.Fatalf("failure expected, but passed") + } + + if !reflect.DeepEqual(stmts, test.expected) { + t.Errorf("error in input: \n\n%s\n", test.input) + t.Fatalf("no match\n\nexpected:\n%s\ngot: \n%s\n", Tree(test.expected), Tree(stmts)) + } + } +} + +func mustLabelMatcher(mt metric.MatchType, name clientmodel.LabelName, val clientmodel.LabelValue) *metric.LabelMatcher { + m, err := metric.NewLabelMatcher(mt, name, val) + if err != nil { + panic(err) + } + return m +} + +func mustGetFunction(name string) *Function { + f, ok := GetFunction(name) + if !ok { + panic(fmt.Errorf("function %q does not exist", name)) + } + return f +} diff --git a/promql/printer.go b/promql/printer.go new file mode 100644 index 000000000..03833accb --- /dev/null +++ b/promql/printer.go @@ -0,0 +1,355 @@ +// 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" + "sort" + "strings" + + clientmodel "github.com/prometheus/client_golang/model" + + "github.com/prometheus/prometheus/storage/metric" + "github.com/prometheus/prometheus/utility" +) + +// 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 { + 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", utility.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 { + return fmt.Sprintf("%s BY (%s)", 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(), utility.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.Str) +} + +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, ",")) +} + +// DotGraph returns a DOT representation of a statement list. +func (ss Statements) DotGraph() string { + graph := "" + for _, stmt := range ss { + graph += stmt.DotGraph() + } + return graph +} + +// DotGraph returns a DOT representation of the alert statement. +func (node *AlertStmt) DotGraph() string { + graph := fmt.Sprintf( + `digraph "Alert Statement" { + %#p[shape="box",label="ALERT %s IF FOR %s"]; + %#p -> %x; + %s + }`, + node, node.Name, utility.DurationToString(node.Duration), + node, reflect.ValueOf(node.Expr).Pointer(), + node.Expr.DotGraph(), + ) + return graph +} + +// DotGraph returns a DOT representation of the eval statement. +func (node *EvalStmt) DotGraph() string { + graph := fmt.Sprintf( + `%#p[shape="box",label="[%d:%s:%d]"; + %#p -> %x; + %s + }`, + node, node.Start, node.End, node.Interval, + node, reflect.ValueOf(node.Expr).Pointer(), + node.Expr.DotGraph(), + ) + return graph +} + +// DotGraph returns a DOT representation of the record statement. +func (node *RecordStmt) DotGraph() string { + graph := fmt.Sprintf( + `%#p[shape="box",label="%s = "]; + %#p -> %x; + %s + }`, + node, node.Name, + node, reflect.ValueOf(node.Expr).Pointer(), + node.Expr.DotGraph(), + ) + return graph +} + +// DotGraph returns a DOT representation of // DotGraph returns a DOT representation of the record statement. +// DotGraph returns a DOT representation of a statement list. +func (es Expressions) DotGraph() string { + graph := "" + for _, expr := range es { + graph += expr.DotGraph() + } + return graph +} + +// DotGraph returns a DOT representation of the vector aggregation. +func (node *AggregateExpr) DotGraph() string { + groupByStrings := make([]string, 0, len(node.Grouping)) + for _, label := range node.Grouping { + groupByStrings = append(groupByStrings, string(label)) + } + + graph := fmt.Sprintf("%#p[label=\"%s BY (%s)\"]\n", + node, + node.Op, + strings.Join(groupByStrings, ", ")) + graph += fmt.Sprintf("%#p -> %x;\n", node, reflect.ValueOf(node.Expr).Pointer()) + graph += node.Expr.DotGraph() + return graph +} + +// DotGraph returns a DOT representation of the expression. +func (node *BinaryExpr) DotGraph() string { + nodeAddr := reflect.ValueOf(node).Pointer() + graph := fmt.Sprintf( + ` + %x[label="%s"]; + %x -> %x; + %x -> %x; + %s + %s + }`, + nodeAddr, node.Op, + nodeAddr, reflect.ValueOf(node.LHS).Pointer(), + nodeAddr, reflect.ValueOf(node.RHS).Pointer(), + node.LHS.DotGraph(), + node.RHS.DotGraph(), + ) + return graph +} + +// DotGraph returns a DOT representation of the function call. +func (node *Call) DotGraph() string { + graph := fmt.Sprintf("%#p[label=\"%s\"];\n", node, node.Func.Name) + graph += functionArgsToDotGraph(node, node.Args) + return graph +} + +// DotGraph returns a DOT representation of the number literal. +func (node *NumberLiteral) DotGraph() string { + return fmt.Sprintf("%#p[label=\"%v\"];\n", node, node.Val) +} + +// DotGraph returns a DOT representation of the encapsulated expression. +func (node *ParenExpr) DotGraph() string { + return node.Expr.DotGraph() +} + +// DotGraph returns a DOT representation of the matrix selector. +func (node *MatrixSelector) DotGraph() string { + return fmt.Sprintf("%#p[label=\"%s\"];\n", node, node) +} + +// DotGraph returns a DOT representation of the string literal. +func (node *StringLiteral) DotGraph() string { + return fmt.Sprintf("%#p[label=\"'%q'\"];\n", node, node.Str) +} + +// DotGraph returns a DOT representation of the unary expression. +func (node *UnaryExpr) DotGraph() string { + nodeAddr := reflect.ValueOf(node).Pointer() + graph := fmt.Sprintf( + ` + %x[label="%s"]; + %x -> %x; + %s + %s + }`, + nodeAddr, node.Op, + nodeAddr, reflect.ValueOf(node.Expr).Pointer(), + node.Expr.DotGraph(), + ) + return graph +} + +// DotGraph returns a DOT representation of the vector selector. +func (node *VectorSelector) DotGraph() string { + return fmt.Sprintf("%#p[label=\"%s\"];\n", node, node) +} + +func functionArgsToDotGraph(node Node, args Expressions) string { + graph := args.DotGraph() + for _, arg := range args { + graph += fmt.Sprintf("%x -> %x;\n", reflect.ValueOf(node).Pointer(), reflect.ValueOf(arg).Pointer()) + } + return graph +} diff --git a/storage/metric/matcher.go b/storage/metric/matcher.go index 26a8df706..c01d8d5b7 100644 --- a/storage/metric/matcher.go +++ b/storage/metric/matcher.go @@ -14,6 +14,7 @@ package metric import ( + "fmt" "regexp" clientmodel "github.com/prometheus/client_golang/model" @@ -71,6 +72,10 @@ func NewLabelMatcher(matchType MatchType, name clientmodel.LabelName, value clie return m, nil } +func (m *LabelMatcher) String() string { + return fmt.Sprintf("%s%s%q", m.Name, m.Type, m.Value) +} + // Match returns true if the label matcher matches the supplied label value. func (m *LabelMatcher) Match(v clientmodel.LabelValue) bool { switch m.Type { From 5602328c7ce724d9d0fb395147771f4fedabcb2f Mon Sep 17 00:00:00 2001 From: Fabian Reinartz Date: Mon, 30 Mar 2015 19:13:36 +0200 Subject: [PATCH 02/10] Refactor query evaluation. This copies the evaluation logic from the current rules/ package. The new engine handles the execution process from query string to final result. It provides query timeout and cancellation and general flexibility for future changes. functions.go: Add evaluation implementation. Slight changes to in/out data but not to the processing logic. quantile.go: No changes. analyzer.go: No changes. engine.go: Actually new part. Mainly consists of evaluation methods which were not changed. setup_test.go: Copy of rules/helpers_test.go to setup test storage. promql_test.go: Copy of rules/rules_test.go. --- Godeps/Godeps.json | 4 + .../src/golang.org/x/net/context/context.go | 447 +++++ .../golang.org/x/net/context/context_test.go | 575 ++++++ .../x/net/context/withtimeout_test.go | 26 + promql/analyzer.go | 170 ++ promql/ast.go | 2 +- promql/engine.go | 1219 ++++++++++++ promql/engine_test.go | 272 +++ promql/functions.go | 583 +++++- promql/parse.go | 2 +- promql/parse_test.go | 2 +- promql/printer.go | 45 +- promql/promql_test.go | 1656 +++++++++++++++++ promql/quantile.go | 106 ++ promql/setup_test.go | 486 +++++ 15 files changed, 5561 insertions(+), 34 deletions(-) create mode 100644 Godeps/_workspace/src/golang.org/x/net/context/context.go create mode 100644 Godeps/_workspace/src/golang.org/x/net/context/context_test.go create mode 100644 Godeps/_workspace/src/golang.org/x/net/context/withtimeout_test.go create mode 100644 promql/analyzer.go create mode 100644 promql/engine.go create mode 100644 promql/engine_test.go create mode 100644 promql/promql_test.go create mode 100644 promql/quantile.go create mode 100644 promql/setup_test.go diff --git a/Godeps/Godeps.json b/Godeps/Godeps.json index 1f3b26468..426f1e078 100644 --- a/Godeps/Godeps.json +++ b/Godeps/Godeps.json @@ -7,6 +7,10 @@ "Comment": "null-5", "Rev": "75cd24fc2f2c2a2088577d12123ddee5f54e0675" }, + { + "ImportPath": "github.com/prometheus/procfs", + "Rev": "92faa308558161acab0ada1db048e9996ecec160" + }, { "ImportPath": "github.com/beorn7/perks/quantile", "Rev": "b965b613227fddccbfffe13eae360ed3fa822f8d" diff --git a/Godeps/_workspace/src/golang.org/x/net/context/context.go b/Godeps/_workspace/src/golang.org/x/net/context/context.go new file mode 100644 index 000000000..ef2f3e86f --- /dev/null +++ b/Godeps/_workspace/src/golang.org/x/net/context/context.go @@ -0,0 +1,447 @@ +// Copyright 2014 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// Package context defines the Context type, which carries deadlines, +// cancelation signals, and other request-scoped values across API boundaries +// and between processes. +// +// Incoming requests to a server should create a Context, and outgoing calls to +// servers should accept a Context. The chain of function calls between must +// propagate the Context, optionally replacing it with a modified copy created +// using WithDeadline, WithTimeout, WithCancel, or WithValue. +// +// Programs that use Contexts should follow these rules to keep interfaces +// consistent across packages and enable static analysis tools to check context +// propagation: +// +// Do not store Contexts inside a struct type; instead, pass a Context +// explicitly to each function that needs it. The Context should be the first +// parameter, typically named ctx: +// +// func DoSomething(ctx context.Context, arg Arg) error { +// // ... use ctx ... +// } +// +// Do not pass a nil Context, even if a function permits it. Pass context.TODO +// if you are unsure about which Context to use. +// +// Use context Values only for request-scoped data that transits processes and +// APIs, not for passing optional parameters to functions. +// +// The same Context may be passed to functions running in different goroutines; +// Contexts are safe for simultaneous use by multiple goroutines. +// +// See http://blog.golang.org/context for example code for a server that uses +// Contexts. +package context + +import ( + "errors" + "fmt" + "sync" + "time" +) + +// A Context carries a deadline, a cancelation signal, and other values across +// API boundaries. +// +// Context's methods may be called by multiple goroutines simultaneously. +type Context interface { + // Deadline returns the time when work done on behalf of this context + // should be canceled. Deadline returns ok==false when no deadline is + // set. Successive calls to Deadline return the same results. + Deadline() (deadline time.Time, ok bool) + + // Done returns a channel that's closed when work done on behalf of this + // context should be canceled. Done may return nil if this context can + // never be canceled. Successive calls to Done return the same value. + // + // WithCancel arranges for Done to be closed when cancel is called; + // WithDeadline arranges for Done to be closed when the deadline + // expires; WithTimeout arranges for Done to be closed when the timeout + // elapses. + // + // Done is provided for use in select statements: + // + // // Stream generates values with DoSomething and sends them to out + // // until DoSomething returns an error or ctx.Done is closed. + // func Stream(ctx context.Context, out <-chan Value) error { + // for { + // v, err := DoSomething(ctx) + // if err != nil { + // return err + // } + // select { + // case <-ctx.Done(): + // return ctx.Err() + // case out <- v: + // } + // } + // } + // + // See http://blog.golang.org/pipelines for more examples of how to use + // a Done channel for cancelation. + Done() <-chan struct{} + + // Err returns a non-nil error value after Done is closed. Err returns + // Canceled if the context was canceled or DeadlineExceeded if the + // context's deadline passed. No other values for Err are defined. + // After Done is closed, successive calls to Err return the same value. + Err() error + + // Value returns the value associated with this context for key, or nil + // if no value is associated with key. Successive calls to Value with + // the same key returns the same result. + // + // Use context values only for request-scoped data that transits + // processes and API boundaries, not for passing optional parameters to + // functions. + // + // A key identifies a specific value in a Context. Functions that wish + // to store values in Context typically allocate a key in a global + // variable then use that key as the argument to context.WithValue and + // Context.Value. A key can be any type that supports equality; + // packages should define keys as an unexported type to avoid + // collisions. + // + // Packages that define a Context key should provide type-safe accessors + // for the values stores using that key: + // + // // Package user defines a User type that's stored in Contexts. + // package user + // + // import "golang.org/x/net/context" + // + // // User is the type of value stored in the Contexts. + // type User struct {...} + // + // // key is an unexported type for keys defined in this package. + // // This prevents collisions with keys defined in other packages. + // type key int + // + // // userKey is the key for user.User values in Contexts. It is + // // unexported; clients use user.NewContext and user.FromContext + // // instead of using this key directly. + // var userKey key = 0 + // + // // NewContext returns a new Context that carries value u. + // func NewContext(ctx context.Context, u *User) context.Context { + // return context.WithValue(ctx, userKey, u) + // } + // + // // FromContext returns the User value stored in ctx, if any. + // func FromContext(ctx context.Context) (*User, bool) { + // u, ok := ctx.Value(userKey).(*User) + // return u, ok + // } + Value(key interface{}) interface{} +} + +// Canceled is the error returned by Context.Err when the context is canceled. +var Canceled = errors.New("context canceled") + +// DeadlineExceeded is the error returned by Context.Err when the context's +// deadline passes. +var DeadlineExceeded = errors.New("context deadline exceeded") + +// An emptyCtx is never canceled, has no values, and has no deadline. It is not +// struct{}, since vars of this type must have distinct addresses. +type emptyCtx int + +func (*emptyCtx) Deadline() (deadline time.Time, ok bool) { + return +} + +func (*emptyCtx) Done() <-chan struct{} { + return nil +} + +func (*emptyCtx) Err() error { + return nil +} + +func (*emptyCtx) Value(key interface{}) interface{} { + return nil +} + +func (e *emptyCtx) String() string { + switch e { + case background: + return "context.Background" + case todo: + return "context.TODO" + } + return "unknown empty Context" +} + +var ( + background = new(emptyCtx) + todo = new(emptyCtx) +) + +// Background returns a non-nil, empty Context. It is never canceled, has no +// values, and has no deadline. It is typically used by the main function, +// initialization, and tests, and as the top-level Context for incoming +// requests. +func Background() Context { + return background +} + +// TODO returns a non-nil, empty Context. Code should use context.TODO when +// it's unclear which Context to use or it's is not yet available (because the +// surrounding function has not yet been extended to accept a Context +// parameter). TODO is recognized by static analysis tools that determine +// whether Contexts are propagated correctly in a program. +func TODO() Context { + return todo +} + +// A CancelFunc tells an operation to abandon its work. +// A CancelFunc does not wait for the work to stop. +// After the first call, subsequent calls to a CancelFunc do nothing. +type CancelFunc func() + +// WithCancel returns a copy of parent with a new Done channel. The returned +// context's Done channel is closed when the returned cancel function is called +// or when the parent context's Done channel is closed, whichever happens first. +// +// Canceling this context releases resources associated with it, so code should +// call cancel as soon as the operations running in this Context complete. +func WithCancel(parent Context) (ctx Context, cancel CancelFunc) { + c := newCancelCtx(parent) + propagateCancel(parent, &c) + return &c, func() { c.cancel(true, Canceled) } +} + +// newCancelCtx returns an initialized cancelCtx. +func newCancelCtx(parent Context) cancelCtx { + return cancelCtx{ + Context: parent, + done: make(chan struct{}), + } +} + +// propagateCancel arranges for child to be canceled when parent is. +func propagateCancel(parent Context, child canceler) { + if parent.Done() == nil { + return // parent is never canceled + } + if p, ok := parentCancelCtx(parent); ok { + p.mu.Lock() + if p.err != nil { + // parent has already been canceled + child.cancel(false, p.err) + } else { + if p.children == nil { + p.children = make(map[canceler]bool) + } + p.children[child] = true + } + p.mu.Unlock() + } else { + go func() { + select { + case <-parent.Done(): + child.cancel(false, parent.Err()) + case <-child.Done(): + } + }() + } +} + +// parentCancelCtx follows a chain of parent references until it finds a +// *cancelCtx. This function understands how each of the concrete types in this +// package represents its parent. +func parentCancelCtx(parent Context) (*cancelCtx, bool) { + for { + switch c := parent.(type) { + case *cancelCtx: + return c, true + case *timerCtx: + return &c.cancelCtx, true + case *valueCtx: + parent = c.Context + default: + return nil, false + } + } +} + +// removeChild removes a context from its parent. +func removeChild(parent Context, child canceler) { + p, ok := parentCancelCtx(parent) + if !ok { + return + } + p.mu.Lock() + if p.children != nil { + delete(p.children, child) + } + p.mu.Unlock() +} + +// A canceler is a context type that can be canceled directly. The +// implementations are *cancelCtx and *timerCtx. +type canceler interface { + cancel(removeFromParent bool, err error) + Done() <-chan struct{} +} + +// A cancelCtx can be canceled. When canceled, it also cancels any children +// that implement canceler. +type cancelCtx struct { + Context + + done chan struct{} // closed by the first cancel call. + + mu sync.Mutex + children map[canceler]bool // set to nil by the first cancel call + err error // set to non-nil by the first cancel call +} + +func (c *cancelCtx) Done() <-chan struct{} { + return c.done +} + +func (c *cancelCtx) Err() error { + c.mu.Lock() + defer c.mu.Unlock() + return c.err +} + +func (c *cancelCtx) String() string { + return fmt.Sprintf("%v.WithCancel", c.Context) +} + +// cancel closes c.done, cancels each of c's children, and, if +// removeFromParent is true, removes c from its parent's children. +func (c *cancelCtx) cancel(removeFromParent bool, err error) { + if err == nil { + panic("context: internal error: missing cancel error") + } + c.mu.Lock() + if c.err != nil { + c.mu.Unlock() + return // already canceled + } + c.err = err + close(c.done) + for child := range c.children { + // NOTE: acquiring the child's lock while holding parent's lock. + child.cancel(false, err) + } + c.children = nil + c.mu.Unlock() + + if removeFromParent { + removeChild(c.Context, c) + } +} + +// WithDeadline returns a copy of the parent context with the deadline adjusted +// to be no later than d. If the parent's deadline is already earlier than d, +// WithDeadline(parent, d) is semantically equivalent to parent. The returned +// context's Done channel is closed when the deadline expires, when the returned +// cancel function is called, or when the parent context's Done channel is +// closed, whichever happens first. +// +// Canceling this context releases resources associated with it, so code should +// call cancel as soon as the operations running in this Context complete. +func WithDeadline(parent Context, deadline time.Time) (Context, CancelFunc) { + if cur, ok := parent.Deadline(); ok && cur.Before(deadline) { + // The current deadline is already sooner than the new one. + return WithCancel(parent) + } + c := &timerCtx{ + cancelCtx: newCancelCtx(parent), + deadline: deadline, + } + propagateCancel(parent, c) + d := deadline.Sub(time.Now()) + if d <= 0 { + c.cancel(true, DeadlineExceeded) // deadline has already passed + return c, func() { c.cancel(true, Canceled) } + } + c.mu.Lock() + defer c.mu.Unlock() + if c.err == nil { + c.timer = time.AfterFunc(d, func() { + c.cancel(true, DeadlineExceeded) + }) + } + return c, func() { c.cancel(true, Canceled) } +} + +// A timerCtx carries a timer and a deadline. It embeds a cancelCtx to +// implement Done and Err. It implements cancel by stopping its timer then +// delegating to cancelCtx.cancel. +type timerCtx struct { + cancelCtx + timer *time.Timer // Under cancelCtx.mu. + + deadline time.Time +} + +func (c *timerCtx) Deadline() (deadline time.Time, ok bool) { + return c.deadline, true +} + +func (c *timerCtx) String() string { + return fmt.Sprintf("%v.WithDeadline(%s [%s])", c.cancelCtx.Context, c.deadline, c.deadline.Sub(time.Now())) +} + +func (c *timerCtx) cancel(removeFromParent bool, err error) { + c.cancelCtx.cancel(false, err) + if removeFromParent { + // Remove this timerCtx from its parent cancelCtx's children. + removeChild(c.cancelCtx.Context, c) + } + c.mu.Lock() + if c.timer != nil { + c.timer.Stop() + c.timer = nil + } + c.mu.Unlock() +} + +// WithTimeout returns WithDeadline(parent, time.Now().Add(timeout)). +// +// Canceling this context releases resources associated with it, so code should +// call cancel as soon as the operations running in this Context complete: +// +// func slowOperationWithTimeout(ctx context.Context) (Result, error) { +// ctx, cancel := context.WithTimeout(ctx, 100*time.Millisecond) +// defer cancel() // releases resources if slowOperation completes before timeout elapses +// return slowOperation(ctx) +// } +func WithTimeout(parent Context, timeout time.Duration) (Context, CancelFunc) { + return WithDeadline(parent, time.Now().Add(timeout)) +} + +// WithValue returns a copy of parent in which the value associated with key is +// val. +// +// Use context Values only for request-scoped data that transits processes and +// APIs, not for passing optional parameters to functions. +func WithValue(parent Context, key interface{}, val interface{}) Context { + return &valueCtx{parent, key, val} +} + +// A valueCtx carries a key-value pair. It implements Value for that key and +// delegates all other calls to the embedded Context. +type valueCtx struct { + Context + key, val interface{} +} + +func (c *valueCtx) String() string { + return fmt.Sprintf("%v.WithValue(%#v, %#v)", c.Context, c.key, c.val) +} + +func (c *valueCtx) Value(key interface{}) interface{} { + if c.key == key { + return c.val + } + return c.Context.Value(key) +} diff --git a/Godeps/_workspace/src/golang.org/x/net/context/context_test.go b/Godeps/_workspace/src/golang.org/x/net/context/context_test.go new file mode 100644 index 000000000..faf67722a --- /dev/null +++ b/Godeps/_workspace/src/golang.org/x/net/context/context_test.go @@ -0,0 +1,575 @@ +// Copyright 2014 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package context + +import ( + "fmt" + "math/rand" + "runtime" + "strings" + "sync" + "testing" + "time" +) + +// otherContext is a Context that's not one of the types defined in context.go. +// This lets us test code paths that differ based on the underlying type of the +// Context. +type otherContext struct { + Context +} + +func TestBackground(t *testing.T) { + c := Background() + if c == nil { + t.Fatalf("Background returned nil") + } + select { + case x := <-c.Done(): + t.Errorf("<-c.Done() == %v want nothing (it should block)", x) + default: + } + if got, want := fmt.Sprint(c), "context.Background"; got != want { + t.Errorf("Background().String() = %q want %q", got, want) + } +} + +func TestTODO(t *testing.T) { + c := TODO() + if c == nil { + t.Fatalf("TODO returned nil") + } + select { + case x := <-c.Done(): + t.Errorf("<-c.Done() == %v want nothing (it should block)", x) + default: + } + if got, want := fmt.Sprint(c), "context.TODO"; got != want { + t.Errorf("TODO().String() = %q want %q", got, want) + } +} + +func TestWithCancel(t *testing.T) { + c1, cancel := WithCancel(Background()) + + if got, want := fmt.Sprint(c1), "context.Background.WithCancel"; got != want { + t.Errorf("c1.String() = %q want %q", got, want) + } + + o := otherContext{c1} + c2, _ := WithCancel(o) + contexts := []Context{c1, o, c2} + + for i, c := range contexts { + if d := c.Done(); d == nil { + t.Errorf("c[%d].Done() == %v want non-nil", i, d) + } + if e := c.Err(); e != nil { + t.Errorf("c[%d].Err() == %v want nil", i, e) + } + + select { + case x := <-c.Done(): + t.Errorf("<-c.Done() == %v want nothing (it should block)", x) + default: + } + } + + cancel() + time.Sleep(100 * time.Millisecond) // let cancelation propagate + + for i, c := range contexts { + select { + case <-c.Done(): + default: + t.Errorf("<-c[%d].Done() blocked, but shouldn't have", i) + } + if e := c.Err(); e != Canceled { + t.Errorf("c[%d].Err() == %v want %v", i, e, Canceled) + } + } +} + +func TestParentFinishesChild(t *testing.T) { + // Context tree: + // parent -> cancelChild + // parent -> valueChild -> timerChild + parent, cancel := WithCancel(Background()) + cancelChild, stop := WithCancel(parent) + defer stop() + valueChild := WithValue(parent, "key", "value") + timerChild, stop := WithTimeout(valueChild, 10000*time.Hour) + defer stop() + + select { + case x := <-parent.Done(): + t.Errorf("<-parent.Done() == %v want nothing (it should block)", x) + case x := <-cancelChild.Done(): + t.Errorf("<-cancelChild.Done() == %v want nothing (it should block)", x) + case x := <-timerChild.Done(): + t.Errorf("<-timerChild.Done() == %v want nothing (it should block)", x) + case x := <-valueChild.Done(): + t.Errorf("<-valueChild.Done() == %v want nothing (it should block)", x) + default: + } + + // The parent's children should contain the two cancelable children. + pc := parent.(*cancelCtx) + cc := cancelChild.(*cancelCtx) + tc := timerChild.(*timerCtx) + pc.mu.Lock() + if len(pc.children) != 2 || !pc.children[cc] || !pc.children[tc] { + t.Errorf("bad linkage: pc.children = %v, want %v and %v", + pc.children, cc, tc) + } + pc.mu.Unlock() + + if p, ok := parentCancelCtx(cc.Context); !ok || p != pc { + t.Errorf("bad linkage: parentCancelCtx(cancelChild.Context) = %v, %v want %v, true", p, ok, pc) + } + if p, ok := parentCancelCtx(tc.Context); !ok || p != pc { + t.Errorf("bad linkage: parentCancelCtx(timerChild.Context) = %v, %v want %v, true", p, ok, pc) + } + + cancel() + + pc.mu.Lock() + if len(pc.children) != 0 { + t.Errorf("pc.cancel didn't clear pc.children = %v", pc.children) + } + pc.mu.Unlock() + + // parent and children should all be finished. + check := func(ctx Context, name string) { + select { + case <-ctx.Done(): + default: + t.Errorf("<-%s.Done() blocked, but shouldn't have", name) + } + if e := ctx.Err(); e != Canceled { + t.Errorf("%s.Err() == %v want %v", name, e, Canceled) + } + } + check(parent, "parent") + check(cancelChild, "cancelChild") + check(valueChild, "valueChild") + check(timerChild, "timerChild") + + // WithCancel should return a canceled context on a canceled parent. + precanceledChild := WithValue(parent, "key", "value") + select { + case <-precanceledChild.Done(): + default: + t.Errorf("<-precanceledChild.Done() blocked, but shouldn't have") + } + if e := precanceledChild.Err(); e != Canceled { + t.Errorf("precanceledChild.Err() == %v want %v", e, Canceled) + } +} + +func TestChildFinishesFirst(t *testing.T) { + cancelable, stop := WithCancel(Background()) + defer stop() + for _, parent := range []Context{Background(), cancelable} { + child, cancel := WithCancel(parent) + + select { + case x := <-parent.Done(): + t.Errorf("<-parent.Done() == %v want nothing (it should block)", x) + case x := <-child.Done(): + t.Errorf("<-child.Done() == %v want nothing (it should block)", x) + default: + } + + cc := child.(*cancelCtx) + pc, pcok := parent.(*cancelCtx) // pcok == false when parent == Background() + if p, ok := parentCancelCtx(cc.Context); ok != pcok || (ok && pc != p) { + t.Errorf("bad linkage: parentCancelCtx(cc.Context) = %v, %v want %v, %v", p, ok, pc, pcok) + } + + if pcok { + pc.mu.Lock() + if len(pc.children) != 1 || !pc.children[cc] { + t.Errorf("bad linkage: pc.children = %v, cc = %v", pc.children, cc) + } + pc.mu.Unlock() + } + + cancel() + + if pcok { + pc.mu.Lock() + if len(pc.children) != 0 { + t.Errorf("child's cancel didn't remove self from pc.children = %v", pc.children) + } + pc.mu.Unlock() + } + + // child should be finished. + select { + case <-child.Done(): + default: + t.Errorf("<-child.Done() blocked, but shouldn't have") + } + if e := child.Err(); e != Canceled { + t.Errorf("child.Err() == %v want %v", e, Canceled) + } + + // parent should not be finished. + select { + case x := <-parent.Done(): + t.Errorf("<-parent.Done() == %v want nothing (it should block)", x) + default: + } + if e := parent.Err(); e != nil { + t.Errorf("parent.Err() == %v want nil", e) + } + } +} + +func testDeadline(c Context, wait time.Duration, t *testing.T) { + select { + case <-time.After(wait): + t.Fatalf("context should have timed out") + case <-c.Done(): + } + if e := c.Err(); e != DeadlineExceeded { + t.Errorf("c.Err() == %v want %v", e, DeadlineExceeded) + } +} + +func TestDeadline(t *testing.T) { + c, _ := WithDeadline(Background(), time.Now().Add(100*time.Millisecond)) + if got, prefix := fmt.Sprint(c), "context.Background.WithDeadline("; !strings.HasPrefix(got, prefix) { + t.Errorf("c.String() = %q want prefix %q", got, prefix) + } + testDeadline(c, 200*time.Millisecond, t) + + c, _ = WithDeadline(Background(), time.Now().Add(100*time.Millisecond)) + o := otherContext{c} + testDeadline(o, 200*time.Millisecond, t) + + c, _ = WithDeadline(Background(), time.Now().Add(100*time.Millisecond)) + o = otherContext{c} + c, _ = WithDeadline(o, time.Now().Add(300*time.Millisecond)) + testDeadline(c, 200*time.Millisecond, t) +} + +func TestTimeout(t *testing.T) { + c, _ := WithTimeout(Background(), 100*time.Millisecond) + if got, prefix := fmt.Sprint(c), "context.Background.WithDeadline("; !strings.HasPrefix(got, prefix) { + t.Errorf("c.String() = %q want prefix %q", got, prefix) + } + testDeadline(c, 200*time.Millisecond, t) + + c, _ = WithTimeout(Background(), 100*time.Millisecond) + o := otherContext{c} + testDeadline(o, 200*time.Millisecond, t) + + c, _ = WithTimeout(Background(), 100*time.Millisecond) + o = otherContext{c} + c, _ = WithTimeout(o, 300*time.Millisecond) + testDeadline(c, 200*time.Millisecond, t) +} + +func TestCanceledTimeout(t *testing.T) { + c, _ := WithTimeout(Background(), 200*time.Millisecond) + o := otherContext{c} + c, cancel := WithTimeout(o, 400*time.Millisecond) + cancel() + time.Sleep(100 * time.Millisecond) // let cancelation propagate + select { + case <-c.Done(): + default: + t.Errorf("<-c.Done() blocked, but shouldn't have") + } + if e := c.Err(); e != Canceled { + t.Errorf("c.Err() == %v want %v", e, Canceled) + } +} + +type key1 int +type key2 int + +var k1 = key1(1) +var k2 = key2(1) // same int as k1, different type +var k3 = key2(3) // same type as k2, different int + +func TestValues(t *testing.T) { + check := func(c Context, nm, v1, v2, v3 string) { + if v, ok := c.Value(k1).(string); ok == (len(v1) == 0) || v != v1 { + t.Errorf(`%s.Value(k1).(string) = %q, %t want %q, %t`, nm, v, ok, v1, len(v1) != 0) + } + if v, ok := c.Value(k2).(string); ok == (len(v2) == 0) || v != v2 { + t.Errorf(`%s.Value(k2).(string) = %q, %t want %q, %t`, nm, v, ok, v2, len(v2) != 0) + } + if v, ok := c.Value(k3).(string); ok == (len(v3) == 0) || v != v3 { + t.Errorf(`%s.Value(k3).(string) = %q, %t want %q, %t`, nm, v, ok, v3, len(v3) != 0) + } + } + + c0 := Background() + check(c0, "c0", "", "", "") + + c1 := WithValue(Background(), k1, "c1k1") + check(c1, "c1", "c1k1", "", "") + + if got, want := fmt.Sprint(c1), `context.Background.WithValue(1, "c1k1")`; got != want { + t.Errorf("c.String() = %q want %q", got, want) + } + + c2 := WithValue(c1, k2, "c2k2") + check(c2, "c2", "c1k1", "c2k2", "") + + c3 := WithValue(c2, k3, "c3k3") + check(c3, "c2", "c1k1", "c2k2", "c3k3") + + c4 := WithValue(c3, k1, nil) + check(c4, "c4", "", "c2k2", "c3k3") + + o0 := otherContext{Background()} + check(o0, "o0", "", "", "") + + o1 := otherContext{WithValue(Background(), k1, "c1k1")} + check(o1, "o1", "c1k1", "", "") + + o2 := WithValue(o1, k2, "o2k2") + check(o2, "o2", "c1k1", "o2k2", "") + + o3 := otherContext{c4} + check(o3, "o3", "", "c2k2", "c3k3") + + o4 := WithValue(o3, k3, nil) + check(o4, "o4", "", "c2k2", "") +} + +func TestAllocs(t *testing.T) { + bg := Background() + for _, test := range []struct { + desc string + f func() + limit float64 + gccgoLimit float64 + }{ + { + desc: "Background()", + f: func() { Background() }, + limit: 0, + gccgoLimit: 0, + }, + { + desc: fmt.Sprintf("WithValue(bg, %v, nil)", k1), + f: func() { + c := WithValue(bg, k1, nil) + c.Value(k1) + }, + limit: 3, + gccgoLimit: 3, + }, + { + desc: "WithTimeout(bg, 15*time.Millisecond)", + f: func() { + c, _ := WithTimeout(bg, 15*time.Millisecond) + <-c.Done() + }, + limit: 8, + gccgoLimit: 13, + }, + { + desc: "WithCancel(bg)", + f: func() { + c, cancel := WithCancel(bg) + cancel() + <-c.Done() + }, + limit: 5, + gccgoLimit: 8, + }, + { + desc: "WithTimeout(bg, 100*time.Millisecond)", + f: func() { + c, cancel := WithTimeout(bg, 100*time.Millisecond) + cancel() + <-c.Done() + }, + limit: 8, + gccgoLimit: 25, + }, + } { + limit := test.limit + if runtime.Compiler == "gccgo" { + // gccgo does not yet do escape analysis. + // TOOD(iant): Remove this when gccgo does do escape analysis. + limit = test.gccgoLimit + } + if n := testing.AllocsPerRun(100, test.f); n > limit { + t.Errorf("%s allocs = %f want %d", test.desc, n, int(limit)) + } + } +} + +func TestSimultaneousCancels(t *testing.T) { + root, cancel := WithCancel(Background()) + m := map[Context]CancelFunc{root: cancel} + q := []Context{root} + // Create a tree of contexts. + for len(q) != 0 && len(m) < 100 { + parent := q[0] + q = q[1:] + for i := 0; i < 4; i++ { + ctx, cancel := WithCancel(parent) + m[ctx] = cancel + q = append(q, ctx) + } + } + // Start all the cancels in a random order. + var wg sync.WaitGroup + wg.Add(len(m)) + for _, cancel := range m { + go func(cancel CancelFunc) { + cancel() + wg.Done() + }(cancel) + } + // Wait on all the contexts in a random order. + for ctx := range m { + select { + case <-ctx.Done(): + case <-time.After(1 * time.Second): + buf := make([]byte, 10<<10) + n := runtime.Stack(buf, true) + t.Fatalf("timed out waiting for <-ctx.Done(); stacks:\n%s", buf[:n]) + } + } + // Wait for all the cancel functions to return. + done := make(chan struct{}) + go func() { + wg.Wait() + close(done) + }() + select { + case <-done: + case <-time.After(1 * time.Second): + buf := make([]byte, 10<<10) + n := runtime.Stack(buf, true) + t.Fatalf("timed out waiting for cancel functions; stacks:\n%s", buf[:n]) + } +} + +func TestInterlockedCancels(t *testing.T) { + parent, cancelParent := WithCancel(Background()) + child, cancelChild := WithCancel(parent) + go func() { + parent.Done() + cancelChild() + }() + cancelParent() + select { + case <-child.Done(): + case <-time.After(1 * time.Second): + buf := make([]byte, 10<<10) + n := runtime.Stack(buf, true) + t.Fatalf("timed out waiting for child.Done(); stacks:\n%s", buf[:n]) + } +} + +func TestLayersCancel(t *testing.T) { + testLayers(t, time.Now().UnixNano(), false) +} + +func TestLayersTimeout(t *testing.T) { + testLayers(t, time.Now().UnixNano(), true) +} + +func testLayers(t *testing.T, seed int64, testTimeout bool) { + rand.Seed(seed) + errorf := func(format string, a ...interface{}) { + t.Errorf(fmt.Sprintf("seed=%d: %s", seed, format), a...) + } + const ( + timeout = 200 * time.Millisecond + minLayers = 30 + ) + type value int + var ( + vals []*value + cancels []CancelFunc + numTimers int + ctx = Background() + ) + for i := 0; i < minLayers || numTimers == 0 || len(cancels) == 0 || len(vals) == 0; i++ { + switch rand.Intn(3) { + case 0: + v := new(value) + ctx = WithValue(ctx, v, v) + vals = append(vals, v) + case 1: + var cancel CancelFunc + ctx, cancel = WithCancel(ctx) + cancels = append(cancels, cancel) + case 2: + var cancel CancelFunc + ctx, cancel = WithTimeout(ctx, timeout) + cancels = append(cancels, cancel) + numTimers++ + } + } + checkValues := func(when string) { + for _, key := range vals { + if val := ctx.Value(key).(*value); key != val { + errorf("%s: ctx.Value(%p) = %p want %p", when, key, val, key) + } + } + } + select { + case <-ctx.Done(): + errorf("ctx should not be canceled yet") + default: + } + if s, prefix := fmt.Sprint(ctx), "context.Background."; !strings.HasPrefix(s, prefix) { + t.Errorf("ctx.String() = %q want prefix %q", s, prefix) + } + t.Log(ctx) + checkValues("before cancel") + if testTimeout { + select { + case <-ctx.Done(): + case <-time.After(timeout + timeout/10): + errorf("ctx should have timed out") + } + checkValues("after timeout") + } else { + cancel := cancels[rand.Intn(len(cancels))] + cancel() + select { + case <-ctx.Done(): + default: + errorf("ctx should be canceled") + } + checkValues("after cancel") + } +} + +func TestCancelRemoves(t *testing.T) { + checkChildren := func(when string, ctx Context, want int) { + if got := len(ctx.(*cancelCtx).children); got != want { + t.Errorf("%s: context has %d children, want %d", when, got, want) + } + } + + ctx, _ := WithCancel(Background()) + checkChildren("after creation", ctx, 0) + _, cancel := WithCancel(ctx) + checkChildren("with WithCancel child ", ctx, 1) + cancel() + checkChildren("after cancelling WithCancel child", ctx, 0) + + ctx, _ = WithCancel(Background()) + checkChildren("after creation", ctx, 0) + _, cancel = WithTimeout(ctx, 60*time.Minute) + checkChildren("with WithTimeout child ", ctx, 1) + cancel() + checkChildren("after cancelling WithTimeout child", ctx, 0) +} diff --git a/Godeps/_workspace/src/golang.org/x/net/context/withtimeout_test.go b/Godeps/_workspace/src/golang.org/x/net/context/withtimeout_test.go new file mode 100644 index 000000000..a6754dc36 --- /dev/null +++ b/Godeps/_workspace/src/golang.org/x/net/context/withtimeout_test.go @@ -0,0 +1,26 @@ +// Copyright 2014 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package context_test + +import ( + "fmt" + "time" + + "golang.org/x/net/context" +) + +func ExampleWithTimeout() { + // Pass a context with a timeout to tell a blocking function that it + // should abandon its work after the timeout elapses. + ctx, _ := context.WithTimeout(context.Background(), 100*time.Millisecond) + select { + case <-time.After(200 * time.Millisecond): + fmt.Println("overslept") + case <-ctx.Done(): + fmt.Println(ctx.Err()) // prints "context deadline exceeded" + } + // Output: + // context deadline exceeded +} diff --git a/promql/analyzer.go b/promql/analyzer.go new file mode 100644 index 000000000..a4d2521f9 --- /dev/null +++ b/promql/analyzer.go @@ -0,0 +1,170 @@ +// 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: + pt := getPreloadTimes(n.Offset) + fpts := a.Storage.GetFingerprintsForLabelMatchers(n.LabelMatchers) + n.fingerprints = fpts + n.metrics = map[clientmodel.Fingerprint]clientmodel.COWMetric{} + n.iterators = map[clientmodel.Fingerprint]local.SeriesIterator{} + for _, fp := range fpts { + // 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{}{} + } + n.metrics[fp] = a.Storage.GetMetricForFingerprint(fp) + } + case *MatrixSelector: + pt := getPreloadTimes(n.Offset) + fpts := a.Storage.GetFingerprintsForLabelMatchers(n.LabelMatchers) + n.fingerprints = fpts + n.metrics = map[clientmodel.Fingerprint]clientmodel.COWMetric{} + n.iterators = map[clientmodel.Fingerprint]local.SeriesIterator{} + for _, fp := range fpts { + 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) + } + n.metrics[fp] = a.Storage.GetMetricForFingerprint(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 { + if err = contextDone(ctx, env); err != nil { + return nil, err + } + + start := a.Start.Add(-offset) + end := a.End.Add(-offset) + for fp, rangeDuration := range pt.ranges { + err = p.PreloadRange(fp, start.Add(-rangeDuration), end, *stalenessDelta) + if err != nil { + return nil, err + } + } + for fp := range pt.instants { + 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.fingerprints { + n.iterators[fp] = a.Storage.NewIterator(fp) + } + case *MatrixSelector: + for _, fp := range n.fingerprints { + n.iterators[fp] = a.Storage.NewIterator(fp) + } + } + return true + }) + + return p, nil +} diff --git a/promql/ast.go b/promql/ast.go index 861f10eb3..fc7c26a75 100644 --- a/promql/ast.go +++ b/promql/ast.go @@ -180,7 +180,7 @@ type ParenExpr struct { // StringLiteral represents a string. type StringLiteral struct { - Str string + Val string } // UnaryExpr represents a unary operation on another expression. diff --git a/promql/engine.go b/promql/engine.go new file mode 100644 index 000000000..dc3714ab0 --- /dev/null +++ b/promql/engine.go @@ -0,0 +1,1219 @@ +// 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 ( + "flag" + "fmt" + "io/ioutil" + "math" + "runtime" + "sort" + "sync" + "time" + + "golang.org/x/net/context" + + clientmodel "github.com/prometheus/client_golang/model" + + "github.com/prometheus/prometheus/stats" + "github.com/prometheus/prometheus/storage/local" + "github.com/prometheus/prometheus/storage/metric" +) + +var ( + stalenessDelta = flag.Duration("query.staleness-delta", 300*time.Second, "Staleness delta allowance during expression evaluations.") + defaultQueryTimeout = flag.Duration("query.timeout", 2*time.Minute, "Maximum time a query may take before being aborted.") +) + +// SampleStream is a stream of Values belonging to an attached COWMetric. +type SampleStream struct { + Metric clientmodel.COWMetric `json:"metric"` + Values metric.Values `json:"values"` +} + +// Sample is a single sample belonging to a COWMetric. +type Sample struct { + Metric clientmodel.COWMetric `json:"metric"` + Value clientmodel.SampleValue `json:"value"` + Timestamp clientmodel.Timestamp `json:"timestamp"` +} + +// Scalar is a scalar value evaluated at the set timestamp. +type Scalar struct { + Value clientmodel.SampleValue + Timestamp clientmodel.Timestamp +} + +func (s *Scalar) String() string { + return fmt.Sprintf("scalar: %v @[%v]", s.Value, s.Timestamp) +} + +// String is a string value evaluated at the set timestamp. +type String struct { + Value string + Timestamp clientmodel.Timestamp +} + +func (s *String) String() string { + return s.Value +} + +// Vector is basically only an alias for clientmodel.Samples, but the +// contract is that in a Vector, all Samples have the same timestamp. +type Vector []*Sample + +// Matrix is a slice of SampleStreams that implements sort.Interface and +// has a String method. +type Matrix []*SampleStream + +// Len implements sort.Interface. +func (matrix Matrix) Len() int { + return len(matrix) +} + +// Less implements sort.Interface. +func (matrix Matrix) Less(i, j int) bool { + return matrix[i].Metric.String() < matrix[j].Metric.String() +} + +// Swap implements sort.Interface. +func (matrix Matrix) Swap(i, j int) { + matrix[i], matrix[j] = matrix[j], matrix[i] +} + +// Value is a generic interface for values resulting from a query evaluation. +type Value interface { + Type() ExprType + String() string +} + +func (Matrix) Type() ExprType { return ExprMatrix } +func (Vector) Type() ExprType { return ExprVector } +func (*Scalar) Type() ExprType { return ExprScalar } +func (*String) Type() ExprType { return ExprString } + +// Result holds the resulting value of an execution or an error +// if any occurred. +type Result struct { + Err error + Value Value +} + +// Vector returns a vector if the result value is one. An error is returned if +// the result was an error or the result value is not a vector. +func (r *Result) Vector() (Vector, error) { + if r.Err != nil { + return nil, r.Err + } + v, ok := r.Value.(Vector) + if !ok { + return nil, fmt.Errorf("query result is not a vector") + } + return v, nil +} + +// Matrix returns a matrix. An error is returned if +// the result was an error or the result value is not a matrix. +func (r *Result) Matrix() (Matrix, error) { + if r.Err != nil { + return nil, r.Err + } + v, ok := r.Value.(Matrix) + if !ok { + return nil, fmt.Errorf("query result is not a matrix") + } + return v, nil +} + +// Scalar returns a scalar value. An error is returned if +// the result was an error or the result value is not a scalar. +func (r *Result) Scalar() (*Scalar, error) { + if r.Err != nil { + return nil, r.Err + } + v, ok := r.Value.(*Scalar) + if !ok { + return nil, fmt.Errorf("query result is not a scalar") + } + return v, nil +} + +func (r *Result) String() string { + if r.Err != nil { + return r.Err.Error() + } + if r.Value == nil { + return "" + } + return r.Value.String() +} + +type ( + // ErrQueryTimeout is returned if a query timed out during processing. + ErrQueryTimeout string + // ErrQueryCanceled is returned if a query was canceled during processing. + ErrQueryCanceled string + // ErrNoHandlers is returned if no handlers were registered for the + // execution of a statement. + ErrNoHandlers string +) + +func (e ErrQueryTimeout) Error() string { return fmt.Sprintf("query timed out in %s", e) } +func (e ErrQueryCanceled) Error() string { return fmt.Sprintf("query was canceled in %s", e) } +func (e ErrNoHandlers) Error() string { return fmt.Sprintf("no handlers registered to process %s", e) } + +// A Query is derived from an a raw query string and can be run against an engine +// it is associated with. +type Query interface { + // Exec processes the query and + Exec() *Result + // Statements returns the parsed statements of the query. + Statements() Statements + // Stats returns statistics about the lifetime of the query. + Stats() *stats.TimerGroup + // Cancel signals that a running query execution should be aborted. + Cancel() +} + +// query implements the Query interface. +type query struct { + // The original query string. + q string + // Statements of the parsed query. + stmts Statements + // On finished execution two bools indicating success of the execution + // are sent on the channel. + done chan bool + // Timer stats for the query execution. + stats *stats.TimerGroup + // Cancelation function for the query. + cancel func() + + // The engine against which the query is executed. + ng *Engine +} + +// Statements implements the Query interface. +func (q *query) Statements() Statements { + return q.stmts +} + +// Stats implements the Query interface. +func (q *query) Stats() *stats.TimerGroup { + return q.stats +} + +// Cancel implements the Query interface. +func (q *query) Cancel() { + if q.cancel != nil { + q.cancel() + } +} + +// Exec implements the Query interface. +func (q *query) Exec() *Result { + ctx, cancel := context.WithTimeout(q.ng.baseCtx, *defaultQueryTimeout) + q.cancel = cancel + + res, err := q.ng.exec(ctx, q) + return &Result{Err: err, Value: res} +} + +type ( + // AlertHandlers can be registered with an engine and are called on + // each executed alert statement. + AlertHandler func(context.Context, *AlertStmt) error + // RecordHandlers can be registered with an engine and are called on + // each executed record statement. + RecordHandler func(context.Context, *RecordStmt) error +) + +// contextDone returns an error if the context was canceled or timed out. +func contextDone(ctx context.Context, env string) error { + select { + case <-ctx.Done(): + err := ctx.Err() + switch err { + case context.Canceled: + return ErrQueryCanceled(env) + case context.DeadlineExceeded: + return ErrQueryTimeout(env) + default: + return err + } + default: + return nil + } +} + +// Engine handles the liftetime of queries from beginning to end. It is connected +// to a storage. +type Engine struct { + sync.RWMutex + + // The storage on which the engine operates. + storage local.Storage + + // The base context for all queries and its cancellation function. + baseCtx context.Context + cancelQueries func() + + // Handlers for the statements. + alertHandlers map[string]AlertHandler + recordHandlers map[string]RecordHandler +} + +// NewEngine returns a new engine. +func NewEngine(storage local.Storage) *Engine { + ctx, cancel := context.WithCancel(context.Background()) + return &Engine{ + storage: storage, + baseCtx: ctx, + cancelQueries: cancel, + alertHandlers: map[string]AlertHandler{}, + recordHandlers: map[string]RecordHandler{}, + } +} + +// Stop the engine and cancel all running queries. +func (ng *Engine) Stop() { + ng.cancelQueries() +} + +// NewQuery returns a new query of the given query string. +func (ng *Engine) NewQuery(qs string) (Query, error) { + stmts, err := ParseStmts("query", qs) + if err != nil { + return nil, err + } + query := &query{ + q: qs, + stmts: stmts, + ng: ng, + done: make(chan bool, 2), + stats: stats.NewTimerGroup(), + } + return query, nil +} + +// NewQueryFromFile reads a file and returns a query of statements it contains. +func (ng *Engine) NewQueryFromFile(filename string) (Query, error) { + content, err := ioutil.ReadFile(filename) + if err != nil { + return nil, err + } + return ng.NewQuery(string(content)) +} + +// NewInstantQuery returns an evaluation query for the given expression at the given time. +func (ng *Engine) NewInstantQuery(es string, ts clientmodel.Timestamp) (Query, error) { + return ng.NewRangeQuery(es, ts, ts, 0) +} + +// NewRangeQuery returns an evaluation query for the given time range and with +// the resolution set by the interval. +func (ng *Engine) NewRangeQuery(qs string, start, end clientmodel.Timestamp, interval time.Duration) (Query, error) { + expr, err := ParseExpr("query", qs) + if err != nil { + return nil, err + } + es := &EvalStmt{ + Expr: expr, + Start: start, + End: end, + Interval: interval, + } + + query := &query{ + q: qs, + stmts: Statements{es}, + ng: ng, + done: make(chan bool, 2), + stats: stats.NewTimerGroup(), + } + return query, nil +} + +// exec executes all statements in the query. For evaluation statements only +// one statement per query is allowed, after which the execution returns. +func (ng *Engine) exec(ctx context.Context, q *query) (Value, error) { + const env = "query execution" + + // Cancel when execution is done or an error was raised. + defer q.cancel() + + // The base context might already be canceled (e.g. during shutdown). + if err := contextDone(ctx, env); err != nil { + return nil, err + } + + evalTimer := q.stats.GetTimer(stats.TotalEvalTime).Start() + defer evalTimer.Stop() + + ng.RLock() + alertHandlers := []AlertHandler{} + for _, h := range ng.alertHandlers { + alertHandlers = append(alertHandlers, h) + } + recordHandlers := []RecordHandler{} + for _, h := range ng.recordHandlers { + recordHandlers = append(recordHandlers, h) + } + ng.RUnlock() + + for _, stmt := range q.stmts { + switch s := stmt.(type) { + case *AlertStmt: + if len(alertHandlers) == 0 { + return nil, ErrNoHandlers("alert statement") + } + for _, h := range alertHandlers { + if err := contextDone(ctx, env); err != nil { + return nil, err + } + err := h(ctx, s) + if err != nil { + return nil, err + } + } + case *RecordStmt: + if len(recordHandlers) == 0 { + return nil, ErrNoHandlers("record statement") + } + for _, h := range recordHandlers { + if err := contextDone(ctx, env); err != nil { + return nil, err + } + err := h(ctx, s) + if err != nil { + return nil, err + } + } + case *EvalStmt: + // Currently, only one execution statement per query is allowed. + return ng.execEvalStmt(ctx, q, s) + + default: + panic(fmt.Errorf("statement of unknown type %T", stmt)) + } + } + return nil, nil +} + +// execEvalStmt evaluates the expression of an evaluation statement for the given time range. +func (ng *Engine) execEvalStmt(ctx context.Context, query *query, s *EvalStmt) (Value, error) { + prepareTimer := query.stats.GetTimer(stats.TotalQueryPreparationTime).Start() + analyzeTimer := query.stats.GetTimer(stats.QueryAnalysisTime).Start() + + // Only one execution statement per query is allowed. + analyzer := &Analyzer{ + Storage: ng.storage, + Expr: s.Expr, + Start: s.Start, + End: s.End, + } + err := analyzer.Analyze(ctx) + if err != nil { + analyzeTimer.Stop() + prepareTimer.Stop() + return nil, err + } + analyzeTimer.Stop() + + preloadTimer := query.stats.GetTimer(stats.PreloadTime).Start() + closer, err := analyzer.Prepare(ctx) + if err != nil { + preloadTimer.Stop() + prepareTimer.Stop() + return nil, err + } + defer closer.Close() + + preloadTimer.Stop() + prepareTimer.Stop() + + evalTimer := query.stats.GetTimer(stats.InnerEvalTime).Start() + // Instant evaluation. + if s.Start == s.End && s.Interval == 0 { + evaluator := &evaluator{ + Timestamp: s.Start, + ctx: ctx, + } + val, err := evaluator.Eval(s.Expr) + if err != nil { + return nil, err + } + + evalTimer.Stop() + return val, nil + } + + // Range evaluation. + sampleStreams := map[clientmodel.Fingerprint]*SampleStream{} + for ts := s.Start; !ts.After(s.End); ts = ts.Add(s.Interval) { + + if err := contextDone(ctx, "range evaluation"); err != nil { + return nil, err + } + + evaluator := &evaluator{ + Timestamp: ts, + ctx: ctx, + } + val, err := evaluator.Eval(s.Expr) + if err != nil { + return nil, err + } + vector, ok := val.(Vector) + if !ok { + return nil, fmt.Errorf("value for expression %q must be of type vector but is %s", s.Expr, val.Type()) + } + + for _, sample := range vector { + samplePair := metric.SamplePair{ + Value: sample.Value, + Timestamp: sample.Timestamp, + } + fp := sample.Metric.Metric.Fingerprint() + if sampleStreams[fp] == nil { + sampleStreams[fp] = &SampleStream{ + Metric: sample.Metric, + Values: metric.Values{samplePair}, + } + } else { + sampleStreams[fp].Values = append(sampleStreams[fp].Values, samplePair) + } + + } + } + evalTimer.Stop() + + if err := contextDone(ctx, "expression evaluation"); err != nil { + return nil, err + } + + appendTimer := query.stats.GetTimer(stats.ResultAppendTime).Start() + matrix := Matrix{} + for _, sampleStream := range sampleStreams { + matrix = append(matrix, sampleStream) + } + appendTimer.Stop() + + if err := contextDone(ctx, "expression evaluation"); err != nil { + return nil, err + } + + sortTimer := query.stats.GetTimer(stats.ResultSortTime).Start() + sort.Sort(matrix) + sortTimer.Stop() + + return matrix, nil +} + +// An evaluator evaluates given expressions at a fixed timestamp. It is attached to an +// engine through which it connects to a storage and reports errors. On timeout or +// cancellation of its context it terminates. +type evaluator struct { + ctx context.Context + + Timestamp clientmodel.Timestamp +} + +// fatalf causes a panic with the input formatted into an error. +func (ev *evaluator) errorf(format string, args ...interface{}) { + ev.error(fmt.Errorf(format, args...)) +} + +// fatal causes a panic with the given error. +func (ev *evaluator) error(err error) { + panic(err) +} + +// recover is the handler that turns panics into returns from the top level of evaluation. +func (ev *evaluator) recover(errp *error) { + e := recover() + if e != nil { + // Do not recover from runtime errors. + if _, ok := e.(runtime.Error); ok { + panic(e) + } + *errp = e.(error) + } +} + +// evalScalar attempts to evaluate e to a scalar value and errors otherwise. +func (ev *evaluator) evalScalar(e Expr) *Scalar { + val := ev.eval(e) + sv, ok := val.(*Scalar) + if !ok { + ev.errorf("expected scalar but got %s", val.Type()) + } + return sv +} + +// evalVector attempts to evaluate e to a vector value and errors otherwise. +func (ev *evaluator) evalVector(e Expr) Vector { + val := ev.eval(e) + vec, ok := val.(Vector) + if !ok { + ev.errorf("expected vector but got %s", val.Type()) + } + return vec +} + +// evalInt attempts to evaluate e into an integer and errors otherwise. +func (ev *evaluator) evalInt(e Expr) int { + sc := ev.evalScalar(e) + return int(sc.Value) +} + +// evalFloat attempts to evaluate e into a float and errors otherwise. +func (ev *evaluator) evalFloat(e Expr) float64 { + sc := ev.evalScalar(e) + return float64(sc.Value) +} + +// evalMatrix attempts to evaluate e into a matrix and errors otherwise. +func (ev *evaluator) evalMatrix(e Expr) Matrix { + val := ev.eval(e) + mat, ok := val.(Matrix) + if !ok { + ev.errorf("expected matrix but got %s", val.Type()) + } + return mat +} + +// evalMatrixBounds attempts to evaluate e to matrix boundaries and errors otherwise. +func (ev *evaluator) evalMatrixBounds(e Expr) Matrix { + ms, ok := e.(*MatrixSelector) + if !ok { + ev.errorf("matrix bounds can only be evaluated for matrix selectors, got %T", e) + } + return ev.matrixSelectorBounds(ms) +} + +// evalOneOf evaluates e and errors unless the result is of one of the given types. +func (ev *evaluator) evalOneOf(e Expr, t1, t2 ExprType) Value { + val := ev.eval(e) + if val.Type() != t1 && val.Type() != t2 { + ev.errorf("expected %s or %s but got %s", t1, t2, val.Type()) + } + return val +} + +func (ev *evaluator) Eval(expr Expr) (v Value, err error) { + defer ev.recover(&err) + return ev.eval(expr), nil +} + +// eval evaluates the given expression as the given AST expression node requires. +func (ev *evaluator) eval(expr Expr) Value { + // This is the top-level evaluation method. + // Thus, we check for timeout/cancellation here. + if err := contextDone(ev.ctx, "expression evaluation"); err != nil { + ev.error(err) + } + + switch e := expr.(type) { + case *AggregateExpr: + vector := ev.evalVector(e.Expr) + return ev.aggregation(e.Op, e.Grouping, e.KeepExtraLabels, vector) + + case *BinaryExpr: + lhs := ev.evalOneOf(e.LHS, ExprScalar, ExprVector) + rhs := ev.evalOneOf(e.RHS, ExprScalar, ExprVector) + + switch lt, rt := lhs.Type(), rhs.Type(); { + case lt == ExprScalar && rt == ExprScalar: + return &Scalar{ + Value: scalarBinop(e.Op, lhs.(*Scalar).Value, rhs.(*Scalar).Value), + Timestamp: ev.Timestamp, + } + + case lt == ExprVector && rt == ExprVector: + return ev.vectorBinop(e.Op, lhs.(Vector), rhs.(Vector), e.VectorMatching) + + case lt == ExprVector && rt == ExprScalar: + return ev.vectorScalarBinop(e.Op, lhs.(Vector), rhs.(*Scalar), false) + + case lt == ExprScalar && rt == ExprVector: + return ev.vectorScalarBinop(e.Op, rhs.(Vector), lhs.(*Scalar), true) + } + + case *Call: + return e.Func.Call(ev, e.Args) + + case *MatrixSelector: + return ev.matrixSelector(e) + + case *NumberLiteral: + return &Scalar{Value: e.Val, Timestamp: ev.Timestamp} + + case *ParenExpr: + return ev.eval(e.Expr) + + case *StringLiteral: + return &String{Value: e.Val, Timestamp: ev.Timestamp} + + case *UnaryExpr: + smpl := ev.evalScalar(e.Expr) + if e.Op == itemSUB { + smpl.Value = -smpl.Value + } + return smpl + + case *VectorSelector: + return ev.vectorSelector(e) + } + panic(fmt.Errorf("unhandled expression of type: %T", expr)) +} + +// vectorSelector evaluates a *VectorSelector expression. +func (ev *evaluator) vectorSelector(node *VectorSelector) Vector { + vec := Vector{} + for fp, it := range node.iterators { + sampleCandidates := it.GetValueAtTime(ev.Timestamp.Add(-node.Offset)) + samplePair := chooseClosestSample(sampleCandidates, ev.Timestamp.Add(-node.Offset)) + if samplePair != nil { + vec = append(vec, &Sample{ + Metric: node.metrics[fp], + Value: samplePair.Value, + Timestamp: ev.Timestamp, + }) + } + } + return vec +} + +// matrixSelector evaluates a *MatrixSelector expression. +func (ev *evaluator) matrixSelector(node *MatrixSelector) Matrix { + interval := metric.Interval{ + OldestInclusive: ev.Timestamp.Add(-node.Range - node.Offset), + NewestInclusive: ev.Timestamp.Add(-node.Offset), + } + + sampleStreams := make([]*SampleStream, 0, len(node.iterators)) + for fp, it := range node.iterators { + samplePairs := it.GetRangeValues(interval) + if len(samplePairs) == 0 { + continue + } + + if node.Offset != 0 { + for _, sp := range samplePairs { + sp.Timestamp = sp.Timestamp.Add(node.Offset) + } + } + + sampleStream := &SampleStream{ + Metric: node.metrics[fp], + Values: samplePairs, + } + sampleStreams = append(sampleStreams, sampleStream) + } + return Matrix(sampleStreams) +} + +// matrixSelectorBounds evaluates the boundaries of a *MatrixSelector. +func (ev *evaluator) matrixSelectorBounds(node *MatrixSelector) Matrix { + interval := metric.Interval{ + OldestInclusive: ev.Timestamp.Add(-node.Range - node.Offset), + NewestInclusive: ev.Timestamp.Add(-node.Offset), + } + + sampleStreams := make([]*SampleStream, 0, len(node.iterators)) + for fp, it := range node.iterators { + samplePairs := it.GetBoundaryValues(interval) + if len(samplePairs) == 0 { + continue + } + + sampleStream := &SampleStream{ + Metric: node.metrics[fp], + Values: samplePairs, + } + sampleStreams = append(sampleStreams, sampleStream) + } + return Matrix(sampleStreams) +} + +// vectorBinop evaluates a binary operation between two vector values. +func (ev *evaluator) vectorBinop(op itemType, lhs, rhs Vector, matching *VectorMatching) Vector { + result := make(Vector, 0, len(rhs)) + // The control flow below handles one-to-one or many-to-one matching. + // For one-to-many, swap sidedness and account for the swap when calculating + // values. + if matching.Card == CardOneToMany { + lhs, rhs = rhs, lhs + } + // All samples from the rhs hashed by the matching label/values. + rm := map[uint64]*Sample{} + // Maps the hash of the label values used for matching to the hashes of the label + // values of the include labels (if any). It is used to keep track of already + // inserted samples. + added := map[uint64][]uint64{} + + // Add all rhs samples to a map so we can easily find matches later. + for _, rs := range rhs { + hash := hashForMetric(rs.Metric.Metric, matching.On) + // The rhs is guaranteed to be the 'one' side. Having multiple samples + // with the same hash means that the matching is many-to-many, + // which is not supported. + if _, found := rm[hash]; matching.Card != CardManyToMany && found { + // Many-to-many matching not allowed. + ev.errorf("many-to-many matching not allowed") + } + // In many-to-many matching the entry is simply overwritten. It can thus only + // be used to check whether any matching rhs entry exists but not retrieve them all. + rm[hash] = rs + } + + // For all lhs samples find a respective rhs sample and perform + // the binary operation. + for _, ls := range lhs { + hash := hashForMetric(ls.Metric.Metric, matching.On) + // Any lhs sample we encounter in an OR operation belongs to the result. + if op == itemLOR { + ls.Metric = resultMetric(op, ls, nil, matching) + result = append(result, ls) + added[hash] = nil // Ensure matching rhs sample is not added later. + continue + } + + rs, found := rm[hash] // Look for a match in the rhs vector. + if !found { + continue + } + var value clientmodel.SampleValue + var keep bool + + if op == itemLAND { + value = ls.Value + keep = true + } else { + if _, exists := added[hash]; matching.Card == CardOneToOne && exists { + // Many-to-one matching must be explicit. + ev.errorf("many-to-one matching must be explicit") + } + // Account for potentially swapped sidedness. + vl, vr := ls.Value, rs.Value + if matching.Card == CardOneToMany { + vl, vr = vr, vl + } + value, keep = vectorElemBinop(op, vl, vr) + } + + if keep { + metric := resultMetric(op, ls, rs, matching) + // Check if the same label set has been added for a many-to-one matching before. + if matching.Card == CardManyToOne || matching.Card == CardOneToMany { + insHash := clientmodel.SignatureForLabels(metric.Metric, matching.Include) + if ihs, exists := added[hash]; exists { + for _, ih := range ihs { + if ih == insHash { + ev.errorf("metric with label set has already been matched") + } + } + added[hash] = append(ihs, insHash) + } else { + added[hash] = []uint64{insHash} + } + } + ns := &Sample{ + Metric: metric, + Value: value, + Timestamp: ev.Timestamp, + } + result = append(result, ns) + added[hash] = added[hash] // Set existance to true. + } + } + + // Add all remaining samples in the rhs in an OR operation if they + // have not been matched up with a lhs sample. + if op == itemLOR { + for hash, rs := range rm { + if _, exists := added[hash]; !exists { + rs.Metric = resultMetric(op, rs, nil, matching) + result = append(result, rs) + } + } + } + return result +} + +// vectorScalarBinop evaluates a binary operation between a vector and a scalar. +func (ev *evaluator) vectorScalarBinop(op itemType, lhs Vector, rhs *Scalar, swap bool) Vector { + vector := make(Vector, 0, len(lhs)) + + for _, lhsSample := range lhs { + lv, rv := lhsSample.Value, rhs.Value + // lhs always contains the vector. If the original position was different + // swap for calculating the value. + if swap { + lv, rv = rv, lv + } + value, keep := vectorElemBinop(op, lv, rv) + if keep { + lhsSample.Value = value + if shouldDropMetricName(op) { + lhsSample.Metric.Delete(clientmodel.MetricNameLabel) + } + vector = append(vector, lhsSample) + } + } + return vector +} + +// scalarBinop evaluates a binary operation between two scalars. +func scalarBinop(op itemType, lhs, rhs clientmodel.SampleValue) clientmodel.SampleValue { + switch op { + case itemADD: + return lhs + rhs + case itemSUB: + return lhs - rhs + case itemMUL: + return lhs * rhs + case itemDIV: + return lhs / rhs + case itemMOD: + if rhs != 0 { + return clientmodel.SampleValue(int(lhs) % int(rhs)) + } + return clientmodel.SampleValue(math.NaN()) + case itemEQL: + return btos(lhs == rhs) + case itemNEQ: + return btos(lhs != rhs) + case itemGTR: + return btos(lhs > rhs) + case itemLSS: + return btos(lhs < rhs) + case itemGTE: + return btos(lhs >= rhs) + case itemLTE: + return btos(lhs <= rhs) + } + panic(fmt.Errorf("operator %q not allowed for scalar operations", op)) +} + +// vectorElemBinop evaluates a binary operation between two vector elements. +func vectorElemBinop(op itemType, lhs, rhs clientmodel.SampleValue) (clientmodel.SampleValue, bool) { + switch op { + case itemADD: + return lhs + rhs, true + case itemSUB: + return lhs - rhs, true + case itemMUL: + return lhs * rhs, true + case itemDIV: + return lhs / rhs, true + case itemMOD: + if rhs != 0 { + return clientmodel.SampleValue(int(lhs) % int(rhs)), true + } + return clientmodel.SampleValue(math.NaN()), true + case itemEQL: + return lhs, lhs == rhs + case itemNEQ: + return lhs, lhs != rhs + case itemGTR: + return lhs, lhs > rhs + case itemLSS: + return lhs, lhs < rhs + case itemGTE: + return lhs, lhs >= rhs + case itemLTE: + return lhs, lhs <= rhs + } + panic(fmt.Errorf("operator %q not allowed for operations between vectors", op)) +} + +// labelIntersection returns the metric of common label/value pairs of two input metrics. +func labelIntersection(metric1, metric2 clientmodel.COWMetric) clientmodel.COWMetric { + for label, value := range metric1.Metric { + if metric2.Metric[label] != value { + metric1.Delete(label) + } + } + return metric1 +} + +type groupedAggregation struct { + labels clientmodel.COWMetric + value clientmodel.SampleValue + valuesSquaredSum clientmodel.SampleValue + groupCount int +} + +// aggregation evaluates an aggregation operation on a vector. +func (ev *evaluator) aggregation(op itemType, grouping clientmodel.LabelNames, keepExtra bool, vector Vector) Vector { + + result := map[uint64]*groupedAggregation{} + + for _, sample := range vector { + groupingKey := clientmodel.SignatureForLabels(sample.Metric.Metric, grouping) + + groupedResult, ok := result[groupingKey] + // Add a new group if it doesn't exist. + if !ok { + var m clientmodel.COWMetric + if keepExtra { + m = sample.Metric + m.Delete(clientmodel.MetricNameLabel) + } else { + m = clientmodel.COWMetric{ + Metric: clientmodel.Metric{}, + Copied: true, + } + for _, l := range grouping { + if v, ok := sample.Metric.Metric[l]; ok { + m.Set(l, v) + } + } + } + result[groupingKey] = &groupedAggregation{ + labels: m, + value: sample.Value, + valuesSquaredSum: sample.Value * sample.Value, + groupCount: 1, + } + continue + } + // Add the sample to the existing group. + if keepExtra { + groupedResult.labels = labelIntersection(groupedResult.labels, sample.Metric) + } + + switch op { + case itemSum: + groupedResult.value += sample.Value + case itemAvg: + groupedResult.value += sample.Value + groupedResult.groupCount++ + case itemMax: + if groupedResult.value < sample.Value { + groupedResult.value = sample.Value + } + case itemMin: + if groupedResult.value > sample.Value { + groupedResult.value = sample.Value + } + case itemCount: + groupedResult.groupCount++ + case itemStdvar, itemStddev: + groupedResult.value += sample.Value + groupedResult.valuesSquaredSum += sample.Value * sample.Value + groupedResult.groupCount++ + default: + panic(fmt.Errorf("expected aggregation operator but got %q", op)) + } + } + + // Construct the result vector from the aggregated groups. + resultVector := make(Vector, 0, len(result)) + + for _, aggr := range result { + switch op { + case itemAvg: + aggr.value = aggr.value / clientmodel.SampleValue(aggr.groupCount) + case itemCount: + aggr.value = clientmodel.SampleValue(aggr.groupCount) + case itemStdvar: + avg := float64(aggr.value) / float64(aggr.groupCount) + aggr.value = clientmodel.SampleValue(float64(aggr.valuesSquaredSum)/float64(aggr.groupCount) - avg*avg) + case itemStddev: + avg := float64(aggr.value) / float64(aggr.groupCount) + aggr.value = clientmodel.SampleValue(math.Sqrt(float64(aggr.valuesSquaredSum)/float64(aggr.groupCount) - avg*avg)) + default: + // For other aggregations, we already have the right value. + } + sample := &Sample{ + Metric: aggr.labels, + Value: aggr.value, + Timestamp: ev.Timestamp, + } + resultVector = append(resultVector, sample) + } + return resultVector +} + +// RegisterAlertHandler registers a new alert handler of the given name. +func (ng *Engine) RegisterAlertHandler(name string, h AlertHandler) { + ng.Lock() + ng.alertHandlers[name] = h + ng.Unlock() +} + +// RegisterRecordHandler registers a new record handler of the given name. +func (ng *Engine) RegisterRecordHandler(name string, h RecordHandler) { + ng.Lock() + ng.recordHandlers[name] = h + ng.Unlock() +} + +// UnregisterAlertHandler removes the alert handler with the given name. +func (ng *Engine) UnregisterAlertHandler(name string) { + ng.Lock() + delete(ng.alertHandlers, name) + ng.Unlock() +} + +// UnregisterRecordHandler removes the record handler with the given name. +func (ng *Engine) UnregisterRecordHandler(name string) { + ng.Lock() + delete(ng.recordHandlers, name) + ng.Unlock() +} + +// btos returns 1 if b is true, 0 otherwise. +func btos(b bool) clientmodel.SampleValue { + if b { + return 1 + } + return 0 +} + +// shouldDropMetricName returns whether the metric name should be dropped in the +// result of the op operation. +func shouldDropMetricName(op itemType) bool { + switch op { + case itemADD, itemSUB, itemDIV, itemMUL, itemMOD: + return true + default: + return false + } +} + +// resultMetric returns the metric for the given sample(s) based on the vector +// binary operation and the matching options. +func resultMetric(op itemType, ls, rs *Sample, matching *VectorMatching) clientmodel.COWMetric { + if len(matching.On) == 0 || op == itemLOR || op == itemLAND { + if shouldDropMetricName(op) { + ls.Metric.Delete(clientmodel.MetricNameLabel) + } + return ls.Metric + } + + m := clientmodel.Metric{} + for _, ln := range matching.On { + m[ln] = ls.Metric.Metric[ln] + } + + for _, ln := range matching.Include { + // Included labels from the `group_x` modifier are taken from the "many"-side. + v, ok := ls.Metric.Metric[ln] + if ok { + m[ln] = v + } + } + return clientmodel.COWMetric{false, m} +} + +// hashForMetric calculates a hash value for the given metric based on the matching +// options for the binary operation. +func hashForMetric(metric clientmodel.Metric, withLabels clientmodel.LabelNames) uint64 { + var labels clientmodel.LabelNames + + if len(withLabels) > 0 { + var match bool + for _, ln := range withLabels { + if _, match = metric[ln]; !match { + break + } + } + // If the metric does not contain the labels to match on, build the hash + // over the whole metric to give it a unique hash. + if !match { + labels = make(clientmodel.LabelNames, 0, len(metric)) + for ln := range metric { + labels = append(labels, ln) + } + } else { + labels = withLabels + } + } else { + labels = make(clientmodel.LabelNames, 0, len(metric)) + for ln := range metric { + if ln != clientmodel.MetricNameLabel { + labels = append(labels, ln) + } + } + } + return clientmodel.SignatureForLabels(metric, labels) +} + +// chooseClosestSample chooses the closest sample of a list of samples +// surrounding a given target time. If samples are found both before and after +// the target time, the sample value is interpolated between these. Otherwise, +// the single closest sample is returned verbatim. +func chooseClosestSample(samples metric.Values, timestamp clientmodel.Timestamp) *metric.SamplePair { + var closestBefore *metric.SamplePair + var closestAfter *metric.SamplePair + for _, candidate := range samples { + delta := candidate.Timestamp.Sub(timestamp) + // Samples before target time. + if delta < 0 { + // Ignore samples outside of staleness policy window. + if -delta > *stalenessDelta { + continue + } + // Ignore samples that are farther away than what we've seen before. + if closestBefore != nil && candidate.Timestamp.Before(closestBefore.Timestamp) { + continue + } + sample := candidate + closestBefore = &sample + } + + // Samples after target time. + if delta >= 0 { + // Ignore samples outside of staleness policy window. + if delta > *stalenessDelta { + continue + } + // Ignore samples that are farther away than samples we've seen before. + if closestAfter != nil && candidate.Timestamp.After(closestAfter.Timestamp) { + continue + } + sample := candidate + closestAfter = &sample + } + } + + switch { + case closestBefore != nil && closestAfter != nil: + return interpolateSamples(closestBefore, closestAfter, timestamp) + case closestBefore != nil: + return closestBefore + default: + return closestAfter + } +} + +// interpolateSamples interpolates a value at a target time between two +// provided sample pairs. +func interpolateSamples(first, second *metric.SamplePair, timestamp clientmodel.Timestamp) *metric.SamplePair { + dv := second.Value - first.Value + dt := second.Timestamp.Sub(first.Timestamp) + + dDt := dv / clientmodel.SampleValue(dt) + offset := clientmodel.SampleValue(timestamp.Sub(first.Timestamp)) + + return &metric.SamplePair{ + Value: first.Value + (offset * dDt), + Timestamp: timestamp, + } +} diff --git a/promql/engine_test.go b/promql/engine_test.go new file mode 100644 index 000000000..35e9188af --- /dev/null +++ b/promql/engine_test.go @@ -0,0 +1,272 @@ +package promql + +import ( + "reflect" + "sync" + "testing" + "time" + + "golang.org/x/net/context" + + "github.com/prometheus/prometheus/storage/local" +) + +func TestQueryTimeout(t *testing.T) { + *defaultQueryTimeout = 5 * time.Millisecond + defer func() { + // Restore default query timeout + *defaultQueryTimeout = 2 * time.Minute + }() + + storage, closer := local.NewTestStorage(t, 1) + defer closer.Close() + + engine := NewEngine(storage) + defer engine.Stop() + + query, err := engine.NewQuery("foo = bar") + if err != nil { + t.Fatalf("error parsing query: %s", err) + } + + // Timeouts are not exact but checked in designated places. For example between + // invoking handlers. Thus, we reigster two handlers that take some time to ensure we check + // after exceeding the timeout. + // Should the implementation of this area change, the test might have to be adjusted. + engine.RegisterRecordHandler("test", func(context.Context, *RecordStmt) error { + time.Sleep(10 * time.Millisecond) + return nil + }) + engine.RegisterRecordHandler("test2", func(context.Context, *RecordStmt) error { + time.Sleep(10 * time.Millisecond) + return nil + }) + + 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) { + storage, closer := local.NewTestStorage(t, 1) + defer closer.Close() + + engine := NewEngine(storage) + defer engine.Stop() + + query1, err := engine.NewQuery("foo = bar") + if err != nil { + t.Fatalf("error parsing query: %s", err) + } + query2, err := engine.NewQuery("foo = baz") + if err != nil { + t.Fatalf("error parsing query: %s", err) + } + + // As for timeouts, cancellation is only checked at designated points. We ensure + // that we reach one of those points using the same method. + engine.RegisterRecordHandler("test1", func(context.Context, *RecordStmt) error { + <-time.After(2 * time.Millisecond) + return nil + }) + engine.RegisterRecordHandler("test2", func(context.Context, *RecordStmt) error { + <-time.After(2 * time.Millisecond) + return nil + }) + + // Cancel query after starting it. + var wg sync.WaitGroup + var res *Result + + wg.Add(1) + go func() { + res = query1.Exec() + wg.Done() + }() + <-time.After(1 * time.Millisecond) + query1.Cancel() + wg.Wait() + + if res.Err == nil { + t.Fatalf("expected cancellation error for query1 but got none") + } + if _, ok := res.Err.(ErrQueryCanceled); res.Err != nil && !ok { + t.Fatalf("expected cancellation error for query1 but got: %s", res.Err) + } + + // Canceling query before starting it must have no effect. + query2.Cancel() + res = query2.Exec() + if res.Err != nil { + t.Fatalf("unexpeceted error on executing query2: %s", res.Err) + } +} + +func TestEngineShutdown(t *testing.T) { + storage, closer := local.NewTestStorage(t, 1) + defer closer.Close() + + engine := NewEngine(storage) + + query1, err := engine.NewQuery("foo = bar") + if err != nil { + t.Fatalf("error parsing query: %s", err) + } + query2, err := engine.NewQuery("foo = baz") + if err != nil { + t.Fatalf("error parsing query: %s", err) + } + + handlerExecutions := 0 + + // Shutdown engine on first handler execution. Should handler execution ever become + // concurrent this test has to be adjusted accordingly. + engine.RegisterRecordHandler("test", func(context.Context, *RecordStmt) error { + handlerExecutions++ + engine.Stop() + time.Sleep(10 * time.Millisecond) + return nil + }) + engine.RegisterRecordHandler("test2", func(context.Context, *RecordStmt) error { + handlerExecutions++ + engine.Stop() + time.Sleep(10 * time.Millisecond) + return nil + }) + + // Stopping the engine should cancel the base context. While setting up queries is + // still possible their context is canceled from the beginning and execution should + // terminate immediately. + + res := query1.Exec() + if res.Err == nil { + t.Fatalf("expected error on shutdown during query but got none") + } + if handlerExecutions != 1 { + t.Fatalf("expected only one handler to be executed before query cancellation but got %d executons", handlerExecutions) + } + + res2 := query2.Exec() + if res2.Err == nil { + t.Fatalf("expected error on querying shutdown engine but got none") + } + if handlerExecutions != 1 { + t.Fatalf("expected no handler execution for query after engine shutdown") + } + +} + +func TestAlertHandler(t *testing.T) { + storage, closer := local.NewTestStorage(t, 1) + defer closer.Close() + + engine := NewEngine(storage) + defer engine.Stop() + + qs := `ALERT Foo IF bar FOR 5m WITH {a="b"} SUMMARY "sum" DESCRIPTION "desc"` + + doQuery := func(expectFailure bool) *AlertStmt { + query, err := engine.NewQuery(qs) + if err != nil { + t.Fatalf("error parsing query: %s", err) + } + res := query.Exec() + if expectFailure && res.Err == nil { + t.Fatalf("expected error but got none.") + } + if res.Err != nil && !expectFailure { + t.Fatalf("error on executing alert query: %s", res.Err) + } + // That this alert statement is correct is tested elsewhere. + return query.Statements()[0].(*AlertStmt) + } + + // We expect an error if nothing is registered to handle the query. + alertStmt := doQuery(true) + + receivedCalls := 0 + + // Ensure that we receive the correct statement. + engine.RegisterAlertHandler("test", func(ctx context.Context, as *AlertStmt) error { + if !reflect.DeepEqual(alertStmt, as) { + t.Errorf("received alert statement did not match input: %q", qs) + t.Fatalf("no match\n\nexpected:\n%s\ngot: \n%s\n", Tree(alertStmt), Tree(as)) + } + receivedCalls++ + return nil + }) + + for i := 0; i < 10; i++ { + doQuery(false) + if receivedCalls != i+1 { + t.Fatalf("alert handler was not called on query execution") + } + } + + engine.UnregisterAlertHandler("test") + + // We must receive no further calls after unregistering. + doQuery(true) + if receivedCalls != 10 { + t.Fatalf("received calls after unregistering alert handler") + } +} + +func TestRecordHandler(t *testing.T) { + storage, closer := local.NewTestStorage(t, 1) + defer closer.Close() + + engine := NewEngine(storage) + defer engine.Stop() + + qs := `foo = bar` + + doQuery := func(expectFailure bool) *RecordStmt { + query, err := engine.NewQuery(qs) + if err != nil { + t.Fatalf("error parsing query: %s", err) + } + res := query.Exec() + if expectFailure && res.Err == nil { + t.Fatalf("expected error but got none.") + } + if res.Err != nil && !expectFailure { + t.Fatalf("error on executing record query: %s", res.Err) + } + return query.Statements()[0].(*RecordStmt) + } + + // We expect an error if nothing is registered to handle the query. + recordStmt := doQuery(true) + + receivedCalls := 0 + + // Ensure that we receive the correct statement. + engine.RegisterRecordHandler("test", func(ctx context.Context, rs *RecordStmt) error { + if !reflect.DeepEqual(recordStmt, rs) { + t.Errorf("received record statement did not match input: %q", qs) + t.Fatalf("no match\n\nexpected:\n%s\ngot: \n%s\n", Tree(recordStmt), Tree(rs)) + } + receivedCalls++ + return nil + }) + + for i := 0; i < 10; i++ { + doQuery(false) + if receivedCalls != i+1 { + t.Fatalf("record handler was not called on query execution") + } + } + + engine.UnregisterRecordHandler("test") + + // We must receive no further calls after unregistering. + doQuery(true) + if receivedCalls != 10 { + t.Fatalf("received calls after unregistering record handler") + } +} diff --git a/promql/functions.go b/promql/functions.go index 4d747ed75..f1b9c0fd7 100644 --- a/promql/functions.go +++ b/promql/functions.go @@ -13,6 +13,18 @@ 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 { @@ -20,7 +32,477 @@ type Function struct { ArgTypes []ExprType OptionalArgs int ReturnType ExprType - Call func() + 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 +} + +// === 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 +} + +// === 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 } var functions = map[string]*Function{ @@ -28,164 +510,207 @@ var functions = map[string]*Function{ Name: "abs", ArgTypes: []ExprType{ExprVector}, ReturnType: ExprVector, - Call: func() {}, + Call: funcAbs, }, "absent": { Name: "absent", ArgTypes: []ExprType{ExprVector}, ReturnType: ExprVector, - Call: func() {}, + Call: funcAbsent, }, "avg_over_time": { Name: "avg_over_time", ArgTypes: []ExprType{ExprMatrix}, ReturnType: ExprVector, - Call: func() {}, + Call: funcAvgOverTime, }, "bottomk": { Name: "bottomk", ArgTypes: []ExprType{ExprScalar, ExprVector}, ReturnType: ExprVector, - Call: func() {}, + Call: funcBottomk, }, "ceil": { Name: "ceil", ArgTypes: []ExprType{ExprVector}, ReturnType: ExprVector, - Call: func() {}, + Call: funcCeil, }, "count_over_time": { Name: "count_over_time", ArgTypes: []ExprType{ExprMatrix}, ReturnType: ExprVector, - Call: func() {}, + Call: funcCountOverTime, }, "count_scalar": { Name: "count_scalar", ArgTypes: []ExprType{ExprVector}, ReturnType: ExprScalar, - Call: func() {}, + Call: funcCountScalar, }, "delta": { Name: "delta", ArgTypes: []ExprType{ExprMatrix, ExprScalar}, OptionalArgs: 1, // The 2nd argument is deprecated. ReturnType: ExprVector, - Call: func() {}, + Call: funcDelta, }, "deriv": { Name: "deriv", ArgTypes: []ExprType{ExprMatrix}, ReturnType: ExprVector, - Call: func() {}, + Call: funcDeriv, }, "drop_common_labels": { Name: "drop_common_labels", ArgTypes: []ExprType{ExprVector}, ReturnType: ExprVector, - Call: func() {}, + Call: funcDropCommonLabels, }, "exp": { Name: "exp", ArgTypes: []ExprType{ExprVector}, ReturnType: ExprVector, - Call: func() {}, + Call: funcExp, }, "floor": { Name: "floor", ArgTypes: []ExprType{ExprVector}, ReturnType: ExprVector, - Call: func() {}, + Call: funcFloor, }, "histogram_quantile": { Name: "histogram_quantile", ArgTypes: []ExprType{ExprScalar, ExprVector}, ReturnType: ExprVector, - Call: func() {}, + Call: funcHistogramQuantile, }, "ln": { Name: "ln", ArgTypes: []ExprType{ExprVector}, ReturnType: ExprVector, - Call: func() {}, + Call: funcLn, }, "log10": { Name: "log10", ArgTypes: []ExprType{ExprVector}, ReturnType: ExprVector, - Call: func() {}, + Call: funcLog10, }, "log2": { Name: "log2", ArgTypes: []ExprType{ExprVector}, ReturnType: ExprVector, - Call: func() {}, + Call: funcLog2, }, "max_over_time": { Name: "max_over_time", ArgTypes: []ExprType{ExprMatrix}, ReturnType: ExprVector, - Call: func() {}, + Call: funcMaxOverTime, }, "min_over_time": { Name: "min_over_time", ArgTypes: []ExprType{ExprMatrix}, ReturnType: ExprVector, - Call: func() {}, + Call: funcMinOverTime, }, "rate": { Name: "rate", ArgTypes: []ExprType{ExprMatrix}, ReturnType: ExprVector, - Call: func() {}, + Call: funcRate, }, "round": { Name: "round", ArgTypes: []ExprType{ExprVector, ExprScalar}, OptionalArgs: 1, ReturnType: ExprVector, - Call: func() {}, + Call: funcRound, }, "scalar": { Name: "scalar", ArgTypes: []ExprType{ExprVector}, ReturnType: ExprScalar, - Call: func() {}, + Call: funcScalar, }, "sort": { Name: "sort", ArgTypes: []ExprType{ExprVector}, ReturnType: ExprVector, - Call: func() {}, + Call: funcSort, }, "sort_desc": { Name: "sort_desc", ArgTypes: []ExprType{ExprVector}, ReturnType: ExprVector, - Call: func() {}, + 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: func() {}, + Call: funcSumOverTime, }, "time": { Name: "time", ArgTypes: []ExprType{}, ReturnType: ExprScalar, - Call: func() {}, + Call: funcTime, }, "topk": { Name: "topk", ArgTypes: []ExprType{ExprScalar, ExprVector}, ReturnType: ExprVector, - Call: func() {}, + Call: funcTopk, }, } -// GetFunction returns a predefined Function object for the given name. -func GetFunction(name string) (*Function, bool) { +// 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) +} diff --git a/promql/parse.go b/promql/parse.go index a683782ad..ec388c450 100644 --- a/promql/parse.go +++ b/promql/parse.go @@ -565,7 +565,7 @@ func (p *parser) aggrExpr() *AggregateExpr { func (p *parser) call(name string) *Call { const ctx = "function call" - fn, exist := GetFunction(name) + fn, exist := getFunction(name) if !exist { p.errorf("unknown function with name %q", name) } diff --git a/promql/parse_test.go b/promql/parse_test.go index fc990c0a1..49750859a 100644 --- a/promql/parse_test.go +++ b/promql/parse_test.go @@ -1069,7 +1069,7 @@ func mustLabelMatcher(mt metric.MatchType, name clientmodel.LabelName, val clien } func mustGetFunction(name string) *Function { - f, ok := GetFunction(name) + f, ok := getFunction(name) if !ok { panic(fmt.Errorf("function %q does not exist", name)) } diff --git a/promql/printer.go b/promql/printer.go index 03833accb..6576d2332 100644 --- a/promql/printer.go +++ b/promql/printer.go @@ -25,6 +25,47 @@ import ( "github.com/prometheus/prometheus/utility" ) +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, "") @@ -175,7 +216,7 @@ func (node *ParenExpr) String() string { } func (node *StringLiteral) String() string { - return fmt.Sprintf("%q", node.Str) + return fmt.Sprintf("%q", node.Val) } func (node *UnaryExpr) String() string { @@ -321,7 +362,7 @@ func (node *MatrixSelector) DotGraph() string { // DotGraph returns a DOT representation of the string literal. func (node *StringLiteral) DotGraph() string { - return fmt.Sprintf("%#p[label=\"'%q'\"];\n", node, node.Str) + return fmt.Sprintf("%#p[label=\"'%q'\"];\n", node, node.Val) } // DotGraph returns a DOT representation of the unary expression. diff --git a/promql/promql_test.go b/promql/promql_test.go new file mode 100644 index 000000000..93af620d3 --- /dev/null +++ b/promql/promql_test.go @@ -0,0 +1,1656 @@ +// 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 ( + "fmt" + "math" + "regexp" + "strconv" + "strings" + "testing" + "time" + + clientmodel "github.com/prometheus/client_golang/model" + + "github.com/prometheus/prometheus/storage/local" + "github.com/prometheus/prometheus/storage/metric" + "github.com/prometheus/prometheus/utility/test" +) + +var ( + testEvalTime = testStartTime.Add(testSampleInterval * 10) + fixturesPath = "fixtures" + + reSample = regexp.MustCompile(`^(.*)(?: \=\>|:) (\-?\d+\.?\d*(?:e-?\d+)?|[+-]Inf|NaN) \@\[(\d+)\]$`) + minNormal = math.Float64frombits(0x0010000000000000) // The smallest positive normal value of type float64. +) + +const ( + epsilon = 0.000001 // Relative error allowed for sample values. +) + +func annotateWithTime(lines []string, timestamp clientmodel.Timestamp) []string { + annotatedLines := []string{} + for _, line := range lines { + annotatedLines = append(annotatedLines, fmt.Sprintf(line, timestamp)) + } + return annotatedLines +} + +func vectorComparisonString(expected []string, actual []string) string { + separator := "\n--------------\n" + return fmt.Sprintf("Expected:%v%v%v\nActual:%v%v%v ", + separator, + strings.Join(expected, "\n"), + separator, + separator, + strings.Join(actual, "\n"), + separator) +} + +// samplesAlmostEqual returns true if the two sample lines only differ by a +// small relative error in their sample value. +func samplesAlmostEqual(a, b string) bool { + if a == b { + // Fast path if strings are equal. + return true + } + aMatches := reSample.FindStringSubmatch(a) + if aMatches == nil { + panic(fmt.Errorf("sample %q did not match regular expression", a)) + } + bMatches := reSample.FindStringSubmatch(b) + if bMatches == nil { + panic(fmt.Errorf("sample %q did not match regular expression", b)) + } + if aMatches[1] != bMatches[1] { + return false // Labels don't match. + } + if aMatches[3] != bMatches[3] { + return false // Timestamps don't match. + } + // If we are here, we have the diff in the floats. + // We have to check if they are almost equal. + aVal, err := strconv.ParseFloat(aMatches[2], 64) + if err != nil { + panic(err) + } + bVal, err := strconv.ParseFloat(bMatches[2], 64) + if err != nil { + panic(err) + } + + // Cf. http://floating-point-gui.de/errors/comparison/ + if aVal == bVal { + return true + } + + diff := math.Abs(aVal - bVal) + + if aVal == 0 || bVal == 0 || diff < minNormal { + return diff < epsilon*minNormal + } + return diff/(math.Abs(aVal)+math.Abs(bVal)) < epsilon +} + +func newTestStorage(t testing.TB) (storage local.Storage, closer test.Closer) { + storage, closer = local.NewTestStorage(t, 1) + storeMatrix(storage, testMatrix) + return storage, closer +} + +func TestExpressions(t *testing.T) { + // Labels in expected output need to be alphabetically sorted. + expressionTests := []struct { + expr string + output []string + shouldFail bool + checkOrder bool + }{ + { + expr: `SUM(http_requests)`, + output: []string{`{} => 3600 @[%v]`}, + }, { + expr: `SUM(http_requests{instance="0"}) BY(job)`, + output: []string{ + `{job="api-server"} => 400 @[%v]`, + `{job="app-server"} => 1200 @[%v]`, + }, + }, { + expr: `SUM(http_requests{instance="0"}) BY(job) KEEPING_EXTRA`, + output: []string{ + `{instance="0", job="api-server"} => 400 @[%v]`, + `{instance="0", job="app-server"} => 1200 @[%v]`, + }, + }, { + expr: `SUM(http_requests) BY (job)`, + output: []string{ + `{job="api-server"} => 1000 @[%v]`, + `{job="app-server"} => 2600 @[%v]`, + }, + }, { + // Non-existent labels mentioned in BY-clauses shouldn't propagate to output. + expr: `SUM(http_requests) BY (job, nonexistent)`, + output: []string{ + `{job="api-server"} => 1000 @[%v]`, + `{job="app-server"} => 2600 @[%v]`, + }, + }, { + expr: ` + # Test comment. + SUM(http_requests) BY # comments shouldn't have any effect + (job) # another comment`, + output: []string{ + `{job="api-server"} => 1000 @[%v]`, + `{job="app-server"} => 2600 @[%v]`, + }, + }, { + expr: `COUNT(http_requests) BY (job)`, + output: []string{ + `{job="api-server"} => 4 @[%v]`, + `{job="app-server"} => 4 @[%v]`, + }, + }, { + expr: `SUM(http_requests) BY (job, group)`, + output: []string{ + `{group="canary", job="api-server"} => 700 @[%v]`, + `{group="canary", job="app-server"} => 1500 @[%v]`, + `{group="production", job="api-server"} => 300 @[%v]`, + `{group="production", job="app-server"} => 1100 @[%v]`, + }, + }, { + expr: `AVG(http_requests) BY (job)`, + output: []string{ + `{job="api-server"} => 250 @[%v]`, + `{job="app-server"} => 650 @[%v]`, + }, + }, { + expr: `MIN(http_requests) BY (job)`, + output: []string{ + `{job="api-server"} => 100 @[%v]`, + `{job="app-server"} => 500 @[%v]`, + }, + }, { + expr: `MAX(http_requests) BY (job)`, + output: []string{ + `{job="api-server"} => 400 @[%v]`, + `{job="app-server"} => 800 @[%v]`, + }, + }, { + expr: `SUM(http_requests) BY (job) - COUNT(http_requests) BY (job)`, + output: []string{ + `{job="api-server"} => 996 @[%v]`, + `{job="app-server"} => 2596 @[%v]`, + }, + }, { + expr: `2 - SUM(http_requests) BY (job)`, + output: []string{ + `{job="api-server"} => -998 @[%v]`, + `{job="app-server"} => -2598 @[%v]`, + }, + }, { + expr: `1000 / SUM(http_requests) BY (job)`, + output: []string{ + `{job="api-server"} => 1 @[%v]`, + `{job="app-server"} => 0.38461538461538464 @[%v]`, + }, + }, { + expr: `SUM(http_requests) BY (job) - 2`, + output: []string{ + `{job="api-server"} => 998 @[%v]`, + `{job="app-server"} => 2598 @[%v]`, + }, + }, { + expr: `SUM(http_requests) BY (job) % 3`, + output: []string{ + `{job="api-server"} => 1 @[%v]`, + `{job="app-server"} => 2 @[%v]`, + }, + }, { + expr: `SUM(http_requests) BY (job) / 0`, + output: []string{ + `{job="api-server"} => +Inf @[%v]`, + `{job="app-server"} => +Inf @[%v]`, + }, + }, { + expr: `SUM(http_requests) BY (job) > 1000`, + output: []string{ + `{job="app-server"} => 2600 @[%v]`, + }, + }, { + expr: `1000 < SUM(http_requests) BY (job)`, + output: []string{ + `{job="app-server"} => 1000 @[%v]`, + }, + }, { + expr: `SUM(http_requests) BY (job) <= 1000`, + output: []string{ + `{job="api-server"} => 1000 @[%v]`, + }, + }, { + expr: `SUM(http_requests) BY (job) != 1000`, + output: []string{ + `{job="app-server"} => 2600 @[%v]`, + }, + }, { + expr: `SUM(http_requests) BY (job) == 1000`, + output: []string{ + `{job="api-server"} => 1000 @[%v]`, + }, + }, { + expr: `SUM(http_requests) BY (job) + SUM(http_requests) BY (job)`, + output: []string{ + `{job="api-server"} => 2000 @[%v]`, + `{job="app-server"} => 5200 @[%v]`, + }, + }, { + expr: `http_requests{job="api-server", group="canary"}`, + output: []string{ + `http_requests{group="canary", instance="0", job="api-server"} => 300 @[%v]`, + `http_requests{group="canary", instance="1", job="api-server"} => 400 @[%v]`, + }, + }, { + expr: `http_requests{job="api-server", group="canary"} + rate(http_requests{job="api-server"}[5m]) * 5 * 60`, + output: []string{ + `{group="canary", instance="0", job="api-server"} => 330 @[%v]`, + `{group="canary", instance="1", job="api-server"} => 440 @[%v]`, + }, + }, { + expr: `rate(http_requests[25m]) * 25 * 60`, + output: []string{ + `{group="canary", instance="0", job="api-server"} => 150 @[%v]`, + `{group="canary", instance="0", job="app-server"} => 350 @[%v]`, + `{group="canary", instance="1", job="api-server"} => 200 @[%v]`, + `{group="canary", instance="1", job="app-server"} => 400 @[%v]`, + `{group="production", instance="0", job="api-server"} => 50 @[%v]`, + `{group="production", instance="0", job="app-server"} => 249.99999999999997 @[%v]`, + `{group="production", instance="1", job="api-server"} => 100 @[%v]`, + `{group="production", instance="1", job="app-server"} => 300 @[%v]`, + }, + }, { + expr: `delta(http_requests[25m], 1)`, + output: []string{ + `{group="canary", instance="0", job="api-server"} => 150 @[%v]`, + `{group="canary", instance="0", job="app-server"} => 350 @[%v]`, + `{group="canary", instance="1", job="api-server"} => 200 @[%v]`, + `{group="canary", instance="1", job="app-server"} => 400 @[%v]`, + `{group="production", instance="0", job="api-server"} => 50 @[%v]`, + `{group="production", instance="0", job="app-server"} => 250 @[%v]`, + `{group="production", instance="1", job="api-server"} => 100 @[%v]`, + `{group="production", instance="1", job="app-server"} => 300 @[%v]`, + }, + }, + { + expr: `sort(http_requests)`, + output: []string{ + `http_requests{group="production", instance="0", job="api-server"} => 100 @[%v]`, + `http_requests{group="production", instance="1", job="api-server"} => 200 @[%v]`, + `http_requests{group="canary", instance="0", job="api-server"} => 300 @[%v]`, + `http_requests{group="canary", instance="1", job="api-server"} => 400 @[%v]`, + `http_requests{group="production", instance="0", job="app-server"} => 500 @[%v]`, + `http_requests{group="production", instance="1", job="app-server"} => 600 @[%v]`, + `http_requests{group="canary", instance="0", job="app-server"} => 700 @[%v]`, + `http_requests{group="canary", instance="1", job="app-server"} => 800 @[%v]`, + }, + checkOrder: true, + }, { + expr: `sort(0 / round(http_requests, 400) + http_requests)`, + output: []string{ + `{group="production", instance="0", job="api-server"} => NaN @[%v]`, + `{group="production", instance="1", job="api-server"} => 200 @[%v]`, + `{group="canary", instance="0", job="api-server"} => 300 @[%v]`, + `{group="canary", instance="1", job="api-server"} => 400 @[%v]`, + `{group="production", instance="0", job="app-server"} => 500 @[%v]`, + `{group="production", instance="1", job="app-server"} => 600 @[%v]`, + `{group="canary", instance="0", job="app-server"} => 700 @[%v]`, + `{group="canary", instance="1", job="app-server"} => 800 @[%v]`, + }, + checkOrder: true, + }, { + expr: `sort_desc(http_requests)`, + output: []string{ + `http_requests{group="canary", instance="1", job="app-server"} => 800 @[%v]`, + `http_requests{group="canary", instance="0", job="app-server"} => 700 @[%v]`, + `http_requests{group="production", instance="1", job="app-server"} => 600 @[%v]`, + `http_requests{group="production", instance="0", job="app-server"} => 500 @[%v]`, + `http_requests{group="canary", instance="1", job="api-server"} => 400 @[%v]`, + `http_requests{group="canary", instance="0", job="api-server"} => 300 @[%v]`, + `http_requests{group="production", instance="1", job="api-server"} => 200 @[%v]`, + `http_requests{group="production", instance="0", job="api-server"} => 100 @[%v]`, + }, + checkOrder: true, + }, + { + expr: `topk(3, http_requests)`, + output: []string{ + `http_requests{group="canary", instance="1", job="app-server"} => 800 @[%v]`, + `http_requests{group="canary", instance="0", job="app-server"} => 700 @[%v]`, + `http_requests{group="production", instance="1", job="app-server"} => 600 @[%v]`, + }, + checkOrder: true, + }, { + expr: `topk(5, http_requests{group="canary",job="app-server"})`, + output: []string{ + `http_requests{group="canary", instance="1", job="app-server"} => 800 @[%v]`, + `http_requests{group="canary", instance="0", job="app-server"} => 700 @[%v]`, + }, + checkOrder: true, + }, { + expr: `bottomk(3, http_requests)`, + output: []string{ + `http_requests{group="production", instance="0", job="api-server"} => 100 @[%v]`, + `http_requests{group="production", instance="1", job="api-server"} => 200 @[%v]`, + `http_requests{group="canary", instance="0", job="api-server"} => 300 @[%v]`, + }, + checkOrder: true, + }, { + expr: `bottomk(5, http_requests{group="canary",job="app-server"})`, + output: []string{ + `http_requests{group="canary", instance="0", job="app-server"} => 700 @[%v]`, + `http_requests{group="canary", instance="1", job="app-server"} => 800 @[%v]`, + }, + checkOrder: true, + }, + { + // Single-letter label names and values. + expr: `x{y="testvalue"}`, + output: []string{ + `x{y="testvalue"} => 100 @[%v]`, + }, + }, { + // Lower-cased aggregation operators should work too. + expr: `sum(http_requests) by (job) + min(http_requests) by (job) + max(http_requests) by (job) + avg(http_requests) by (job)`, + output: []string{ + `{job="app-server"} => 4550 @[%v]`, + `{job="api-server"} => 1750 @[%v]`, + }, + }, { + // Deltas should be adjusted for target interval vs. samples under target interval. + expr: `delta(http_requests{group="canary", instance="1", job="app-server"}[18m])`, + output: []string{`{group="canary", instance="1", job="app-server"} => 288 @[%v]`}, + }, { + // Deltas should perform the same operation when 2nd argument is 0. + expr: `delta(http_requests{group="canary", instance="1", job="app-server"}[18m], 0)`, + output: []string{`{group="canary", instance="1", job="app-server"} => 288 @[%v]`}, + }, { + // Rates should calculate per-second rates. + expr: `rate(http_requests{group="canary", instance="1", job="app-server"}[60m])`, + output: []string{`{group="canary", instance="1", job="app-server"} => 0.26666666666666666 @[%v]`}, + }, + { + // Deriv should return the same as rate in simple cases. + expr: `deriv(http_requests{group="canary", instance="1", job="app-server"}[60m])`, + output: []string{`{group="canary", instance="1", job="app-server"} => 0.26666666666666666 @[%v]`}, + }, + { + // Counter resets at in the middle of range are handled correctly by rate(). + expr: `rate(testcounter_reset_middle[60m])`, + output: []string{`{} => 0.03 @[%v]`}, + }, { + // Counter resets at end of range are ignored by rate(). + expr: `rate(testcounter_reset_end[5m])`, + output: []string{`{} => 0 @[%v]`}, + }, + { + // Deriv should return correct result. + expr: `deriv(testcounter_reset_middle[100m])`, + output: []string{`{} => 0.010606060606060607 @[%v]`}, + }, + { + // count_scalar for a non-empty vector should return scalar element count. + expr: `count_scalar(http_requests)`, + output: []string{`scalar: 8 @[%v]`}, + }, { + // count_scalar for an empty vector should return scalar 0. + expr: `count_scalar(nonexistent)`, + output: []string{`scalar: 0 @[%v]`}, + }, { + // Empty expressions shouldn't parse. + expr: ``, + shouldFail: true, + }, { + // Interval durations can't be in quotes. + expr: `http_requests["1m"]`, + shouldFail: true, + }, { + // Binop arguments need to be scalar or vector. + expr: `http_requests - http_requests[1m]`, + shouldFail: true, + }, { + expr: `http_requests{group!="canary"}`, + output: []string{ + `http_requests{group="production", instance="1", job="app-server"} => 600 @[%v]`, + `http_requests{group="production", instance="0", job="app-server"} => 500 @[%v]`, + `http_requests{group="production", instance="1", job="api-server"} => 200 @[%v]`, + `http_requests{group="production", instance="0", job="api-server"} => 100 @[%v]`, + }, + }, { + expr: `http_requests{job=~"server",group!="canary"}`, + output: []string{ + `http_requests{group="production", instance="1", job="app-server"} => 600 @[%v]`, + `http_requests{group="production", instance="0", job="app-server"} => 500 @[%v]`, + `http_requests{group="production", instance="1", job="api-server"} => 200 @[%v]`, + `http_requests{group="production", instance="0", job="api-server"} => 100 @[%v]`, + }, + }, { + expr: `http_requests{job!~"api",group!="canary"}`, + output: []string{ + `http_requests{group="production", instance="1", job="app-server"} => 600 @[%v]`, + `http_requests{group="production", instance="0", job="app-server"} => 500 @[%v]`, + }, + }, { + expr: `count_scalar(http_requests{job=~"^server$"})`, + output: []string{`scalar: 0 @[%v]`}, + }, { + expr: `http_requests{group="production",job=~"^api"}`, + output: []string{ + `http_requests{group="production", instance="0", job="api-server"} => 100 @[%v]`, + `http_requests{group="production", instance="1", job="api-server"} => 200 @[%v]`, + }, + }, + { + expr: `abs(-1 * http_requests{group="production",job="api-server"})`, + output: []string{ + `{group="production", instance="0", job="api-server"} => 100 @[%v]`, + `{group="production", instance="1", job="api-server"} => 200 @[%v]`, + }, + }, + { + expr: `floor(0.004 * http_requests{group="production",job="api-server"})`, + output: []string{ + `{group="production", instance="0", job="api-server"} => 0 @[%v]`, + `{group="production", instance="1", job="api-server"} => 0 @[%v]`, + }, + }, + { + expr: `ceil(0.004 * http_requests{group="production",job="api-server"})`, + output: []string{ + `{group="production", instance="0", job="api-server"} => 1 @[%v]`, + `{group="production", instance="1", job="api-server"} => 1 @[%v]`, + }, + }, + { + expr: `round(0.004 * http_requests{group="production",job="api-server"})`, + output: []string{ + `{group="production", instance="0", job="api-server"} => 0 @[%v]`, + `{group="production", instance="1", job="api-server"} => 1 @[%v]`, + }, + }, + { // Round should correctly handle negative numbers. + expr: `round(-1 * (0.004 * http_requests{group="production",job="api-server"}))`, + output: []string{ + `{group="production", instance="0", job="api-server"} => 0 @[%v]`, + `{group="production", instance="1", job="api-server"} => -1 @[%v]`, + }, + }, + { // Round should round half up. + expr: `round(0.005 * http_requests{group="production",job="api-server"})`, + output: []string{ + `{group="production", instance="0", job="api-server"} => 1 @[%v]`, + `{group="production", instance="1", job="api-server"} => 1 @[%v]`, + }, + }, + { + expr: `round(-1 * (0.005 * http_requests{group="production",job="api-server"}))`, + output: []string{ + `{group="production", instance="0", job="api-server"} => 0 @[%v]`, + `{group="production", instance="1", job="api-server"} => -1 @[%v]`, + }, + }, + { + expr: `round(1 + 0.005 * http_requests{group="production",job="api-server"})`, + output: []string{ + `{group="production", instance="0", job="api-server"} => 2 @[%v]`, + `{group="production", instance="1", job="api-server"} => 2 @[%v]`, + }, + }, + { + expr: `round(-1 * (1 + 0.005 * http_requests{group="production",job="api-server"}))`, + output: []string{ + `{group="production", instance="0", job="api-server"} => -1 @[%v]`, + `{group="production", instance="1", job="api-server"} => -2 @[%v]`, + }, + }, + { // Round should accept the number to round nearest to. + expr: `round(0.0005 * http_requests{group="production",job="api-server"}, 0.1)`, + output: []string{ + `{group="production", instance="0", job="api-server"} => 0.1 @[%v]`, + `{group="production", instance="1", job="api-server"} => 0.1 @[%v]`, + }, + }, + { + expr: `round(2.1 + 0.0005 * http_requests{group="production",job="api-server"}, 0.1)`, + output: []string{ + `{group="production", instance="0", job="api-server"} => 2.2 @[%v]`, + `{group="production", instance="1", job="api-server"} => 2.2 @[%v]`, + }, + }, + { + expr: `round(5.2 + 0.0005 * http_requests{group="production",job="api-server"}, 0.1)`, + output: []string{ + `{group="production", instance="0", job="api-server"} => 5.3 @[%v]`, + `{group="production", instance="1", job="api-server"} => 5.3 @[%v]`, + }, + }, + { // Round should work correctly with negative numbers and multiple decimal places. + expr: `round(-1 * (5.2 + 0.0005 * http_requests{group="production",job="api-server"}), 0.1)`, + output: []string{ + `{group="production", instance="0", job="api-server"} => -5.2 @[%v]`, + `{group="production", instance="1", job="api-server"} => -5.3 @[%v]`, + }, + }, + { // Round should work correctly with big toNearests. + expr: `round(0.025 * http_requests{group="production",job="api-server"}, 5)`, + output: []string{ + `{group="production", instance="0", job="api-server"} => 5 @[%v]`, + `{group="production", instance="1", job="api-server"} => 5 @[%v]`, + }, + }, + { + expr: `round(0.045 * http_requests{group="production",job="api-server"}, 5)`, + output: []string{ + `{group="production", instance="0", job="api-server"} => 5 @[%v]`, + `{group="production", instance="1", job="api-server"} => 10 @[%v]`, + }, + }, + { + expr: `avg_over_time(http_requests{group="production",job="api-server"}[1h])`, + output: []string{ + `{group="production", instance="0", job="api-server"} => 50 @[%v]`, + `{group="production", instance="1", job="api-server"} => 100 @[%v]`, + }, + }, + { + expr: `count_over_time(http_requests{group="production",job="api-server"}[1h])`, + output: []string{ + `{group="production", instance="0", job="api-server"} => 11 @[%v]`, + `{group="production", instance="1", job="api-server"} => 11 @[%v]`, + }, + }, + { + expr: `max_over_time(http_requests{group="production",job="api-server"}[1h])`, + output: []string{ + `{group="production", instance="0", job="api-server"} => 100 @[%v]`, + `{group="production", instance="1", job="api-server"} => 200 @[%v]`, + }, + }, + { + expr: `min_over_time(http_requests{group="production",job="api-server"}[1h])`, + output: []string{ + `{group="production", instance="0", job="api-server"} => 0 @[%v]`, + `{group="production", instance="1", job="api-server"} => 0 @[%v]`, + }, + }, + { + expr: `sum_over_time(http_requests{group="production",job="api-server"}[1h])`, + output: []string{ + `{group="production", instance="0", job="api-server"} => 550 @[%v]`, + `{group="production", instance="1", job="api-server"} => 1100 @[%v]`, + }, + }, + { + expr: `time()`, + output: []string{`scalar: 3000 @[%v]`}, + }, + { + expr: `drop_common_labels(http_requests{group="production",job="api-server"})`, + output: []string{ + `http_requests{instance="0"} => 100 @[%v]`, + `http_requests{instance="1"} => 200 @[%v]`, + }, + }, + { + expr: `{` + string(clientmodel.MetricNameLabel) + `=~".*"}`, + output: []string{ + `http_requests{group="canary", instance="0", job="api-server"} => 300 @[%v]`, + `http_requests{group="canary", instance="0", job="app-server"} => 700 @[%v]`, + `http_requests{group="canary", instance="1", job="api-server"} => 400 @[%v]`, + `http_requests{group="canary", instance="1", job="app-server"} => 800 @[%v]`, + `http_requests{group="production", instance="0", job="api-server"} => 100 @[%v]`, + `http_requests{group="production", instance="0", job="app-server"} => 500 @[%v]`, + `http_requests{group="production", instance="1", job="api-server"} => 200 @[%v]`, + `http_requests{group="production", instance="1", job="app-server"} => 600 @[%v]`, + `testcounter_reset_end => 0 @[%v]`, + `testcounter_reset_middle => 50 @[%v]`, + `x{y="testvalue"} => 100 @[%v]`, + `label_grouping_test{a="a", b="abb"} => 200 @[%v]`, + `label_grouping_test{a="aa", b="bb"} => 100 @[%v]`, + `testhistogram_bucket{le="0.1", start="positive"} => 50 @[%v]`, + `testhistogram_bucket{le=".2", start="positive"} => 70 @[%v]`, + `testhistogram_bucket{le="1e0", start="positive"} => 110 @[%v]`, + `testhistogram_bucket{le="+Inf", start="positive"} => 120 @[%v]`, + `testhistogram_bucket{le="-.2", start="negative"} => 10 @[%v]`, + `testhistogram_bucket{le="-0.1", start="negative"} => 20 @[%v]`, + `testhistogram_bucket{le="0.3", start="negative"} => 20 @[%v]`, + `testhistogram_bucket{le="+Inf", start="negative"} => 30 @[%v]`, + `request_duration_seconds_bucket{instance="ins1", job="job1", le="0.1"} => 10 @[%v]`, + `request_duration_seconds_bucket{instance="ins1", job="job1", le="0.2"} => 30 @[%v]`, + `request_duration_seconds_bucket{instance="ins1", job="job1", le="+Inf"} => 40 @[%v]`, + `request_duration_seconds_bucket{instance="ins2", job="job1", le="0.1"} => 20 @[%v]`, + `request_duration_seconds_bucket{instance="ins2", job="job1", le="0.2"} => 50 @[%v]`, + `request_duration_seconds_bucket{instance="ins2", job="job1", le="+Inf"} => 60 @[%v]`, + `request_duration_seconds_bucket{instance="ins1", job="job2", le="0.1"} => 30 @[%v]`, + `request_duration_seconds_bucket{instance="ins1", job="job2", le="0.2"} => 40 @[%v]`, + `request_duration_seconds_bucket{instance="ins1", job="job2", le="+Inf"} => 60 @[%v]`, + `request_duration_seconds_bucket{instance="ins2", job="job2", le="0.1"} => 40 @[%v]`, + `request_duration_seconds_bucket{instance="ins2", job="job2", le="0.2"} => 70 @[%v]`, + `request_duration_seconds_bucket{instance="ins2", job="job2", le="+Inf"} => 90 @[%v]`, + `vector_matching_a{l="x"} => 10 @[%v]`, + `vector_matching_a{l="y"} => 20 @[%v]`, + `vector_matching_b{l="x"} => 40 @[%v]`, + `cpu_count{instance="1", type="smp"} => 200 @[%v]`, + `cpu_count{instance="0", type="smp"} => 100 @[%v]`, + `cpu_count{instance="0", type="numa"} => 300 @[%v]`, + }, + }, + { + expr: `{job=~"server", job!~"api"}`, + output: []string{ + `http_requests{group="canary", instance="0", job="app-server"} => 700 @[%v]`, + `http_requests{group="canary", instance="1", job="app-server"} => 800 @[%v]`, + `http_requests{group="production", instance="0", job="app-server"} => 500 @[%v]`, + `http_requests{group="production", instance="1", job="app-server"} => 600 @[%v]`, + }, + }, + { + // Test alternative "by"-clause order. + expr: `sum by (group) (http_requests{job="api-server"})`, + output: []string{ + `{group="canary"} => 700 @[%v]`, + `{group="production"} => 300 @[%v]`, + }, + }, + { + // Test alternative "by"-clause order with "keeping_extra". + expr: `sum by (group) keeping_extra (http_requests{job="api-server"})`, + output: []string{ + `{group="canary", job="api-server"} => 700 @[%v]`, + `{group="production", job="api-server"} => 300 @[%v]`, + }, + }, + { + // 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. + expr: `sum(sum by (group) keeping_extra (http_requests{job="api-server"})) by (job)`, + output: []string{ + `{job="api-server"} => 1000 @[%v]`, + }, + }, + { + expr: `http_requests{group="canary"} and http_requests{instance="0"}`, + output: []string{ + `http_requests{group="canary", instance="0", job="api-server"} => 300 @[%v]`, + `http_requests{group="canary", instance="0", job="app-server"} => 700 @[%v]`, + }, + }, + { + expr: `(http_requests{group="canary"} + 1) and http_requests{instance="0"}`, + output: []string{ + `{group="canary", instance="0", job="api-server"} => 301 @[%v]`, + `{group="canary", instance="0", job="app-server"} => 701 @[%v]`, + }, + }, + { + expr: `(http_requests{group="canary"} + 1) and on(instance, job) http_requests{instance="0", group="production"}`, + output: []string{ + `{group="canary", instance="0", job="api-server"} => 301 @[%v]`, + `{group="canary", instance="0", job="app-server"} => 701 @[%v]`, + }, + }, + { + expr: `(http_requests{group="canary"} + 1) and on(instance) http_requests{instance="0", group="production"}`, + output: []string{ + `{group="canary", instance="0", job="api-server"} => 301 @[%v]`, + `{group="canary", instance="0", job="app-server"} => 701 @[%v]`, + }, + }, + { + expr: `http_requests{group="canary"} or http_requests{group="production"}`, + output: []string{ + `http_requests{group="canary", instance="0", job="api-server"} => 300 @[%v]`, + `http_requests{group="canary", instance="0", job="app-server"} => 700 @[%v]`, + `http_requests{group="canary", instance="1", job="api-server"} => 400 @[%v]`, + `http_requests{group="canary", instance="1", job="app-server"} => 800 @[%v]`, + `http_requests{group="production", instance="0", job="api-server"} => 100 @[%v]`, + `http_requests{group="production", instance="0", job="app-server"} => 500 @[%v]`, + `http_requests{group="production", instance="1", job="api-server"} => 200 @[%v]`, + `http_requests{group="production", instance="1", job="app-server"} => 600 @[%v]`, + }, + }, + { + // On overlap the rhs samples must be dropped. + expr: `(http_requests{group="canary"} + 1) or http_requests{instance="1"}`, + output: []string{ + `{group="canary", instance="0", job="api-server"} => 301 @[%v]`, + `{group="canary", instance="0", job="app-server"} => 701 @[%v]`, + `{group="canary", instance="1", job="api-server"} => 401 @[%v]`, + `{group="canary", instance="1", job="app-server"} => 801 @[%v]`, + `http_requests{group="production", instance="1", job="api-server"} => 200 @[%v]`, + `http_requests{group="production", instance="1", job="app-server"} => 600 @[%v]`, + }, + }, + { + // Matching only on instance excludes everything that has instance=0/1 but includes + // entries without the instance label. + expr: `(http_requests{group="canary"} + 1) or on(instance) (http_requests or cpu_count or vector_matching_a)`, + output: []string{ + `{group="canary", instance="0", job="api-server"} => 301 @[%v]`, + `{group="canary", instance="0", job="app-server"} => 701 @[%v]`, + `{group="canary", instance="1", job="api-server"} => 401 @[%v]`, + `{group="canary", instance="1", job="app-server"} => 801 @[%v]`, + `vector_matching_a{l="x"} => 10 @[%v]`, + `vector_matching_a{l="y"} => 20 @[%v]`, + }, + }, + { + expr: `http_requests{group="canary"} / on(instance,job) http_requests{group="production"}`, + output: []string{ + `{instance="0", job="api-server"} => 3 @[%v]`, + `{instance="0", job="app-server"} => 1.4 @[%v]`, + `{instance="1", job="api-server"} => 2 @[%v]`, + `{instance="1", job="app-server"} => 1.3333333333333333 @[%v]`, + }, + }, + { + // Include labels must guarantee uniquely identifiable time series. + expr: `http_requests{group="production"} / on(instance) group_left(group) cpu_count{type="smp"}`, + shouldFail: true, + }, + { + // Many-to-many matching is not allowed. + expr: `http_requests{group="production"} / on(instance) group_left(job,type) cpu_count`, + shouldFail: true, + }, + { + // Many-to-one matching must be explicit. + expr: `http_requests{group="production"} / on(instance) cpu_count{type="smp"}`, + shouldFail: true, + }, + { + expr: `http_requests{group="production"} / on(instance) group_left(job) cpu_count{type="smp"}`, + output: []string{ + `{instance="1", job="api-server"} => 1 @[%v]`, + `{instance="0", job="app-server"} => 5 @[%v]`, + `{instance="1", job="app-server"} => 3 @[%v]`, + `{instance="0", job="api-server"} => 1 @[%v]`, + }, + }, + { + // Ensure sidedness of grouping preserves operand sides. + expr: `cpu_count{type="smp"} / on(instance) group_right(job) http_requests{group="production"}`, + output: []string{ + `{instance="1", job="app-server"} => 0.3333333333333333 @[%v]`, + `{instance="0", job="app-server"} => 0.2 @[%v]`, + `{instance="1", job="api-server"} => 1 @[%v]`, + `{instance="0", job="api-server"} => 1 @[%v]`, + }, + }, + { + // Include labels from both sides. + expr: `http_requests{group="production"} / on(instance) group_left(job) cpu_count{type="smp"}`, + output: []string{ + `{instance="1", job="api-server"} => 1 @[%v]`, + `{instance="0", job="app-server"} => 5 @[%v]`, + `{instance="1", job="app-server"} => 3 @[%v]`, + `{instance="0", job="api-server"} => 1 @[%v]`, + }, + }, + { + expr: `http_requests{group="production"} < on(instance,job) http_requests{group="canary"}`, + output: []string{ + `{instance="1", job="app-server"} => 600 @[%v]`, + `{instance="0", job="app-server"} => 500 @[%v]`, + `{instance="1", job="api-server"} => 200 @[%v]`, + `{instance="0", job="api-server"} => 100 @[%v]`, + }, + }, + { + expr: `http_requests{group="production"} > on(instance,job) http_requests{group="canary"}`, + output: []string{}, + }, + { + expr: `http_requests{group="production"} == on(instance,job) http_requests{group="canary"}`, + output: []string{}, + }, + { + expr: `http_requests > on(instance) group_left(group,job) cpu_count{type="smp"}`, + output: []string{ + `{group="canary", instance="0", job="app-server"} => 700 @[%v]`, + `{group="canary", instance="1", job="app-server"} => 800 @[%v]`, + `{group="canary", instance="0", job="api-server"} => 300 @[%v]`, + `{group="canary", instance="1", job="api-server"} => 400 @[%v]`, + `{group="production", instance="0", job="app-server"} => 500 @[%v]`, + `{group="production", instance="1", job="app-server"} => 600 @[%v]`, + }, + }, + { + expr: `http_requests / on(instance) 3`, + shouldFail: true, + }, + { + expr: `3 / on(instance) http_requests_total`, + shouldFail: true, + }, + { + expr: `3 / on(instance) 3`, + shouldFail: true, + }, + { + // Missing label list for grouping mod. + expr: `http_requests{group="production"} / on(instance) group_left cpu_count{type="smp"}`, + shouldFail: true, + }, + { + // No group mod allowed for logical operations. + expr: `http_requests{group="production"} or on(instance) group_left(type) cpu_count{type="smp"}`, + shouldFail: true, + }, + { + // No group mod allowed for logical operations. + expr: `http_requests{group="production"} and on(instance) group_left(type) cpu_count{type="smp"}`, + shouldFail: true, + }, + { + // No duplicate use of label. + expr: `http_requests{group="production"} + on(instance) group_left(job,instance) cpu_count{type="smp"}`, + shouldFail: true, + }, + { + expr: `{l="x"} + on(__name__) {l="y"}`, + output: []string{ + `vector_matching_a => 30 @[%v]`, + }, + }, + { + expr: `absent(nonexistent)`, + output: []string{ + `{} => 1 @[%v]`, + }, + }, + { + expr: `absent(nonexistent{job="testjob", instance="testinstance", method=~".*"})`, + output: []string{ + `{instance="testinstance", job="testjob"} => 1 @[%v]`, + }, + }, + { + expr: `count_scalar(absent(http_requests))`, + output: []string{ + `scalar: 0 @[%v]`, + }, + }, + { + expr: `count_scalar(absent(sum(http_requests)))`, + output: []string{ + `scalar: 0 @[%v]`, + }, + }, + { + expr: `absent(sum(nonexistent{job="testjob", instance="testinstance"}))`, + output: []string{ + `{} => 1 @[%v]`, + }, + }, + { + expr: `http_requests{group="production",job="api-server"} offset 5m`, + output: []string{ + `http_requests{group="production", instance="0", job="api-server"} => 90 @[%v]`, + `http_requests{group="production", instance="1", job="api-server"} => 180 @[%v]`, + }, + }, + { + expr: `rate(http_requests{group="production",job="api-server"}[10m] offset 5m)`, + output: []string{ + `{group="production", instance="0", job="api-server"} => 0.03333333333333333 @[%v]`, + `{group="production", instance="1", job="api-server"} => 0.06666666666666667 @[%v]`, + }, + }, + { + expr: `rate(http_requests[10m]) offset 5m`, + shouldFail: true, + }, + { + expr: `sum(http_requests) offset 5m`, + shouldFail: true, + }, + // Regression test for missing separator byte in labelsToGroupingKey. + { + expr: `sum(label_grouping_test) by (a, b)`, + output: []string{ + `{a="a", b="abb"} => 200 @[%v]`, + `{a="aa", b="bb"} => 100 @[%v]`, + }, + }, + // Quantile too low. + { + expr: `histogram_quantile(-0.1, testhistogram_bucket)`, + output: []string{ + `{start="positive"} => -Inf @[%v]`, + `{start="negative"} => -Inf @[%v]`, + }, + }, + // Quantile too high. + { + expr: `histogram_quantile(1.01, testhistogram_bucket)`, + output: []string{ + `{start="positive"} => +Inf @[%v]`, + `{start="negative"} => +Inf @[%v]`, + }, + }, + // Quantile value in lowest bucket, which is positive. + { + expr: `histogram_quantile(0, testhistogram_bucket{start="positive"})`, + output: []string{ + `{start="positive"} => 0 @[%v]`, + }, + }, + // Quantile value in lowest bucket, which is negative. + { + expr: `histogram_quantile(0, testhistogram_bucket{start="negative"})`, + output: []string{ + `{start="negative"} => -0.2 @[%v]`, + }, + }, + // Quantile value in highest bucket. + { + expr: `histogram_quantile(1, testhistogram_bucket)`, + output: []string{ + `{start="positive"} => 1 @[%v]`, + `{start="negative"} => 0.3 @[%v]`, + }, + }, + // Finally some useful quantiles. + { + expr: `histogram_quantile(0.2, testhistogram_bucket)`, + output: []string{ + `{start="positive"} => 0.048 @[%v]`, + `{start="negative"} => -0.2 @[%v]`, + }, + }, + { + expr: `histogram_quantile(0.5, testhistogram_bucket)`, + output: []string{ + `{start="positive"} => 0.15 @[%v]`, + `{start="negative"} => -0.15 @[%v]`, + }, + }, + { + expr: `histogram_quantile(0.8, testhistogram_bucket)`, + output: []string{ + `{start="positive"} => 0.72 @[%v]`, + `{start="negative"} => 0.3 @[%v]`, + }, + }, + // More realistic with rates. + { + expr: `histogram_quantile(0.2, rate(testhistogram_bucket[5m]))`, + output: []string{ + `{start="positive"} => 0.048 @[%v]`, + `{start="negative"} => -0.2 @[%v]`, + }, + }, + { + expr: `histogram_quantile(0.5, rate(testhistogram_bucket[5m]))`, + output: []string{ + `{start="positive"} => 0.15 @[%v]`, + `{start="negative"} => -0.15 @[%v]`, + }, + }, + { + expr: `histogram_quantile(0.8, rate(testhistogram_bucket[5m]))`, + output: []string{ + `{start="positive"} => 0.72 @[%v]`, + `{start="negative"} => 0.3 @[%v]`, + }, + }, + // Aggregated histogram: Everything in one. + { + expr: `histogram_quantile(0.3, sum(rate(request_duration_seconds_bucket[5m])) by (le))`, + output: []string{ + `{} => 0.075 @[%v]`, + }, + }, + { + expr: `histogram_quantile(0.5, sum(rate(request_duration_seconds_bucket[5m])) by (le))`, + output: []string{ + `{} => 0.1277777777777778 @[%v]`, + }, + }, + // Aggregated histogram: Everything in one. Now with avg, which does not change anything. + { + expr: `histogram_quantile(0.3, avg(rate(request_duration_seconds_bucket[5m])) by (le))`, + output: []string{ + `{} => 0.075 @[%v]`, + }, + }, + { + expr: `histogram_quantile(0.5, avg(rate(request_duration_seconds_bucket[5m])) by (le))`, + output: []string{ + `{} => 0.12777777777777778 @[%v]`, + }, + }, + // Aggregated histogram: By job. + { + expr: `histogram_quantile(0.3, sum(rate(request_duration_seconds_bucket[5m])) by (le, instance))`, + output: []string{ + `{instance="ins1"} => 0.075 @[%v]`, + `{instance="ins2"} => 0.075 @[%v]`, + }, + }, + { + expr: `histogram_quantile(0.5, sum(rate(request_duration_seconds_bucket[5m])) by (le, instance))`, + output: []string{ + `{instance="ins1"} => 0.1333333333 @[%v]`, + `{instance="ins2"} => 0.125 @[%v]`, + }, + }, + // Aggregated histogram: By instance. + { + expr: `histogram_quantile(0.3, sum(rate(request_duration_seconds_bucket[5m])) by (le, job))`, + output: []string{ + `{job="job1"} => 0.1 @[%v]`, + `{job="job2"} => 0.0642857142857143 @[%v]`, + }, + }, + { + expr: `histogram_quantile(0.5, sum(rate(request_duration_seconds_bucket[5m])) by (le, job))`, + output: []string{ + `{job="job1"} => 0.14 @[%v]`, + `{job="job2"} => 0.1125 @[%v]`, + }, + }, + // Aggregated histogram: By job and instance. + { + expr: `histogram_quantile(0.3, sum(rate(request_duration_seconds_bucket[5m])) by (le, job, instance))`, + output: []string{ + `{instance="ins1", job="job1"} => 0.11 @[%v]`, + `{instance="ins2", job="job1"} => 0.09 @[%v]`, + `{instance="ins1", job="job2"} => 0.06 @[%v]`, + `{instance="ins2", job="job2"} => 0.0675 @[%v]`, + }, + }, + { + expr: `histogram_quantile(0.5, sum(rate(request_duration_seconds_bucket[5m])) by (le, job, instance))`, + output: []string{ + `{instance="ins1", job="job1"} => 0.15 @[%v]`, + `{instance="ins2", job="job1"} => 0.1333333333333333 @[%v]`, + `{instance="ins1", job="job2"} => 0.1 @[%v]`, + `{instance="ins2", job="job2"} => 0.1166666666666667 @[%v]`, + }, + }, + // The unaggregated histogram for comparison. Same result as the previous one. + { + expr: `histogram_quantile(0.3, rate(request_duration_seconds_bucket[5m]))`, + output: []string{ + `{instance="ins1", job="job1"} => 0.11 @[%v]`, + `{instance="ins2", job="job1"} => 0.09 @[%v]`, + `{instance="ins1", job="job2"} => 0.06 @[%v]`, + `{instance="ins2", job="job2"} => 0.0675 @[%v]`, + }, + }, + { + expr: `histogram_quantile(0.5, rate(request_duration_seconds_bucket[5m]))`, + output: []string{ + `{instance="ins1", job="job1"} => 0.15 @[%v]`, + `{instance="ins2", job="job1"} => 0.13333333333333333 @[%v]`, + `{instance="ins1", job="job2"} => 0.1 @[%v]`, + `{instance="ins2", job="job2"} => 0.11666666666666667 @[%v]`, + }, + }, + { + expr: `12.34e6`, + output: []string{`scalar: 12340000 @[%v]`}, + }, + { + expr: `12.34e+6`, + output: []string{`scalar: 12340000 @[%v]`}, + }, + { + expr: `12.34e-6`, + output: []string{`scalar: 0.00001234 @[%v]`}, + }, + { + expr: `1+1`, + output: []string{`scalar: 2 @[%v]`}, + }, + { + expr: `1-1`, + output: []string{`scalar: 0 @[%v]`}, + }, + { + expr: `1 - -1`, + output: []string{`scalar: 2 @[%v]`}, + }, + { + expr: `.2`, + output: []string{`scalar: 0.2 @[%v]`}, + }, + { + expr: `+0.2`, + output: []string{`scalar: 0.2 @[%v]`}, + }, + { + expr: `-0.2e-6`, + output: []string{`scalar: -0.0000002 @[%v]`}, + }, + { + expr: `+Inf`, + output: []string{`scalar: +Inf @[%v]`}, + }, + { + expr: `inF`, + output: []string{`scalar: +Inf @[%v]`}, + }, + { + expr: `-inf`, + output: []string{`scalar: -Inf @[%v]`}, + }, + { + expr: `NaN`, + output: []string{`scalar: NaN @[%v]`}, + }, + { + expr: `nan`, + output: []string{`scalar: NaN @[%v]`}, + }, + { + expr: `2.`, + output: []string{`scalar: 2 @[%v]`}, + }, + { + expr: `999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999`, + shouldFail: true, + }, + { + expr: `1 / 0`, + output: []string{`scalar: +Inf @[%v]`}, + }, + { + expr: `-1 / 0`, + output: []string{`scalar: -Inf @[%v]`}, + }, + { + expr: `0 / 0`, + output: []string{`scalar: NaN @[%v]`}, + }, + { + expr: `1 % 0`, + output: []string{`scalar: NaN @[%v]`}, + }, + { + expr: `http_requests{group="canary", instance="0", job="api-server"} / 0`, + output: []string{ + `{group="canary", instance="0", job="api-server"} => +Inf @[%v]`, + }, + }, + { + expr: `-1 * http_requests{group="canary", instance="0", job="api-server"} / 0`, + output: []string{ + `{group="canary", instance="0", job="api-server"} => -Inf @[%v]`, + }, + }, + { + expr: `0 * http_requests{group="canary", instance="0", job="api-server"} / 0`, + output: []string{ + `{group="canary", instance="0", job="api-server"} => NaN @[%v]`, + }, + }, + { + expr: `0 * http_requests{group="canary", instance="0", job="api-server"} % 0`, + output: []string{ + `{group="canary", instance="0", job="api-server"} => NaN @[%v]`, + }, + }, + { + expr: `exp(vector_matching_a)`, + output: []string{ + `{l="x"} => 22026.465794806718 @[%v]`, + `{l="y"} => 485165195.4097903 @[%v]`, + }, + }, + { + expr: `exp(vector_matching_a - 10)`, + output: []string{ + `{l="y"} => 22026.465794806718 @[%v]`, + `{l="x"} => 1 @[%v]`, + }, + }, + { + expr: `exp(vector_matching_a - 20)`, + output: []string{ + `{l="x"} => 4.5399929762484854e-05 @[%v]`, + `{l="y"} => 1 @[%v]`, + }, + }, + { + expr: `ln(vector_matching_a)`, + output: []string{ + `{l="x"} => 2.302585092994046 @[%v]`, + `{l="y"} => 2.995732273553991 @[%v]`, + }, + }, + { + expr: `ln(vector_matching_a - 10)`, + output: []string{ + `{l="y"} => 2.302585092994046 @[%v]`, + `{l="x"} => -Inf @[%v]`, + }, + }, + { + expr: `ln(vector_matching_a - 20)`, + output: []string{ + `{l="y"} => -Inf @[%v]`, + `{l="x"} => NaN @[%v]`, + }, + }, + { + expr: `exp(ln(vector_matching_a))`, + output: []string{ + `{l="y"} => 20 @[%v]`, + `{l="x"} => 10 @[%v]`, + }, + }, + { + expr: `sqrt(vector_matching_a)`, + output: []string{ + `{l="x"} => 3.1622776601683795 @[%v]`, + `{l="y"} => 4.47213595499958 @[%v]`, + }, + }, + { + expr: `log2(vector_matching_a)`, + output: []string{ + `{l="x"} => 3.3219280948873626 @[%v]`, + `{l="y"} => 4.321928094887363 @[%v]`, + }, + }, + { + expr: `log2(vector_matching_a - 10)`, + output: []string{ + `{l="y"} => 3.3219280948873626 @[%v]`, + `{l="x"} => -Inf @[%v]`, + }, + }, + { + expr: `log2(vector_matching_a - 20)`, + output: []string{ + `{l="x"} => NaN @[%v]`, + `{l="y"} => -Inf @[%v]`, + }, + }, + { + expr: `log10(vector_matching_a)`, + output: []string{ + `{l="x"} => 1 @[%v]`, + `{l="y"} => 1.301029995663981 @[%v]`, + }, + }, + { + expr: `log10(vector_matching_a - 10)`, + output: []string{ + `{l="y"} => 1 @[%v]`, + `{l="x"} => -Inf @[%v]`, + }, + }, + { + expr: `log10(vector_matching_a - 20)`, + output: []string{ + `{l="x"} => NaN @[%v]`, + `{l="y"} => -Inf @[%v]`, + }, + }, + { + expr: `stddev(http_requests)`, + output: []string{ + `{} => 229.12878474779 @[%v]`, + }, + }, + { + expr: `stddev by (instance)(http_requests)`, + output: []string{ + `{instance="0"} => 223.60679774998 @[%v]`, + `{instance="1"} => 223.60679774998 @[%v]`, + }, + }, + { + expr: `stdvar(http_requests)`, + output: []string{ + `{} => 52500 @[%v]`, + }, + }, + { + expr: `stdvar by (instance)(http_requests)`, + output: []string{ + `{instance="0"} => 50000 @[%v]`, + `{instance="1"} => 50000 @[%v]`, + }, + }, + } + + storage, closer := newTestStorage(t) + defer closer.Close() + + engine := NewEngine(storage) + + for i, exprTest := range expressionTests { + expectedLines := annotateWithTime(exprTest.output, testEvalTime) + + query, err := engine.NewInstantQuery(exprTest.expr, testEvalTime) + + if err != nil { + if !exprTest.shouldFail { + t.Errorf("%d. Error during parsing: %v", i, err) + t.Errorf("%d. Expression: %v", i, exprTest.expr) + } + continue + } + + failed := false + + res := query.Exec() + if res.Err != nil { + if !exprTest.shouldFail { + t.Errorf("%d. Error evaluating query: %s", res.Err) + t.Errorf("%d. Expression: %v", i, exprTest.expr) + } + continue + } + if exprTest.shouldFail { + t.Errorf("%d. Expression should fail but did not", i) + continue + } + resultLines := strings.Split(res.String(), "\n") + // resultStr := ast.EvalToString(testExpr, testEvalTime, ast.Text, storage, stats.NewTimerGroup()) + // resultLines := strings.Split(resultStr, "\n") + + if len(exprTest.output) == 0 && strings.Trim(res.String(), "\n") == "" { + // expected and received empty vector, everything is fine + continue + } else if len(exprTest.output) != len(resultLines) { + t.Errorf("%d. Number of samples in expected and actual output don't match", i) + failed = true + } + + if exprTest.checkOrder { + for j, expectedSample := range expectedLines { + if resultLines[j] != expectedSample { + t.Errorf("%d.%d. Expected sample '%v', got '%v'", i, j, resultLines[j], expectedSample) + failed = true + } + } + } else { + for j, expectedSample := range expectedLines { + found := false + for _, actualSample := range resultLines { + if samplesAlmostEqual(actualSample, expectedSample) { + found = true + } + } + if !found { + t.Errorf("%d.%d. Couldn't find expected sample in output: '%v'", i, j, expectedSample) + failed = true + } + } + } + + if failed { + t.Errorf("%d. Expression: %v\n%v", i, exprTest.expr, vectorComparisonString(expectedLines, resultLines)) + } + + } +} + +func TestRangedEvaluationRegressions(t *testing.T) { + scenarios := []struct { + in Matrix + out Matrix + expr string + }{ + { + // Testing COWMetric behavior in drop_common_labels. + in: Matrix{ + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "testmetric", + "testlabel": "1", + }, + }, + Values: metric.Values{ + { + Timestamp: testStartTime, + Value: 1, + }, + { + Timestamp: testStartTime.Add(time.Hour), + Value: 1, + }, + }, + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "testmetric", + "testlabel": "2", + }, + }, + Values: metric.Values{ + { + Timestamp: testStartTime.Add(time.Hour), + Value: 2, + }, + }, + }, + }, + out: Matrix{ + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "testmetric", + }, + }, + Values: metric.Values{ + { + Timestamp: testStartTime, + Value: 1, + }, + }, + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "testmetric", + "testlabel": "1", + }, + }, + Values: metric.Values{ + { + Timestamp: testStartTime.Add(time.Hour), + Value: 1, + }, + }, + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "testmetric", + "testlabel": "2", + }, + }, + Values: metric.Values{ + { + Timestamp: testStartTime.Add(time.Hour), + Value: 2, + }, + }, + }, + }, + expr: "drop_common_labels(testmetric)", + }, + { + // Testing COWMetric behavior in vector aggregation. + in: Matrix{ + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "testmetric", + "testlabel": "1", + }, + }, + Values: metric.Values{ + { + Timestamp: testStartTime, + Value: 1, + }, + { + Timestamp: testStartTime.Add(time.Hour), + Value: 1, + }, + }, + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "testmetric", + "testlabel": "2", + }, + }, + Values: metric.Values{ + { + Timestamp: testStartTime, + Value: 2, + }, + }, + }, + }, + out: Matrix{ + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{}, + }, + Values: metric.Values{ + { + Timestamp: testStartTime, + Value: 3, + }, + }, + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + "testlabel": "1", + }, + }, + Values: metric.Values{ + { + Timestamp: testStartTime.Add(time.Hour), + Value: 1, + }, + }, + }, + }, + expr: "sum(testmetric) keeping_extra", + }, + { + // Testing metric fingerprint grouping behavior. + in: Matrix{ + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "testmetric", + "aa": "bb", + }, + }, + Values: metric.Values{ + { + Timestamp: testStartTime, + Value: 1, + }, + }, + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "testmetric", + "a": "abb", + }, + }, + Values: metric.Values{ + { + Timestamp: testStartTime, + Value: 2, + }, + }, + }, + }, + out: Matrix{ + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "testmetric", + "aa": "bb", + }, + }, + Values: metric.Values{ + { + Timestamp: testStartTime, + Value: 1, + }, + }, + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "testmetric", + "a": "abb", + }, + }, + Values: metric.Values{ + { + Timestamp: testStartTime, + Value: 2, + }, + }, + }, + }, + expr: "testmetric", + }, + } + + for i, s := range scenarios { + storage, closer := local.NewTestStorage(t, 1) + storeMatrix(storage, s.in) + + engine := NewEngine(storage) + query, err := engine.NewRangeQuery(s.expr, testStartTime, testStartTime.Add(time.Hour), time.Hour) + if err != nil { + t.Errorf("%d. Error in expression %q", i, s.expr) + t.Fatalf("%d. Error parsing expression: %v", i, err) + } + res := query.Exec() + if res.Err != nil { + t.Errorf("%d. Error in expression %q", i, s.expr) + t.Fatalf("%d. Error evaluating expression: %v", i, err) + } + + if res.String() != s.out.String() { + t.Errorf("%d. Error in expression %q", i, s.expr) + t.Fatalf("%d. Expression: %s\n\ngot:\n=====\n%v\n====\n\nwant:\n=====\n%v\n=====\n", i, s.expr, res.String(), s.out.String()) + } + + closer.Close() + } +} diff --git a/promql/quantile.go b/promql/quantile.go new file mode 100644 index 000000000..11edad85f --- /dev/null +++ b/promql/quantile.go @@ -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: struct{}{}, + clientmodel.BucketLabel: struct{}{}, +} + +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) +} diff --git a/promql/setup_test.go b/promql/setup_test.go new file mode 100644 index 000000000..5796d7f87 --- /dev/null +++ b/promql/setup_test.go @@ -0,0 +1,486 @@ +// 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 ( + "time" + + clientmodel "github.com/prometheus/client_golang/model" + + "github.com/prometheus/prometheus/storage/local" + "github.com/prometheus/prometheus/storage/metric" +) + +var testSampleInterval = time.Duration(5) * time.Minute +var testStartTime = clientmodel.Timestamp(0) + +func getTestValueStream(startVal, endVal, stepVal clientmodel.SampleValue, startTime clientmodel.Timestamp) (resultValues metric.Values) { + currentTime := startTime + for currentVal := startVal; currentVal <= endVal; currentVal += stepVal { + sample := metric.SamplePair{ + Value: currentVal, + Timestamp: currentTime, + } + resultValues = append(resultValues, sample) + currentTime = currentTime.Add(testSampleInterval) + } + return resultValues +} + +func getTestVectorFromTestMatrix(matrix Matrix) Vector { + vector := Vector{} + for _, sampleStream := range matrix { + lastSample := sampleStream.Values[len(sampleStream.Values)-1] + vector = append(vector, &Sample{ + Metric: sampleStream.Metric, + Value: lastSample.Value, + Timestamp: lastSample.Timestamp, + }) + } + return vector +} + +func storeMatrix(storage local.Storage, matrix Matrix) { + pendingSamples := clientmodel.Samples{} + for _, sampleStream := range matrix { + for _, sample := range sampleStream.Values { + pendingSamples = append(pendingSamples, &clientmodel.Sample{ + Metric: sampleStream.Metric.Metric, + Value: sample.Value, + Timestamp: sample.Timestamp, + }) + } + } + for _, s := range pendingSamples { + storage.Append(s) + } + storage.WaitForIndexing() +} + +var testVector = getTestVectorFromTestMatrix(testMatrix) + +var testMatrix = Matrix{ + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "http_requests", + clientmodel.JobLabel: "api-server", + "instance": "0", + "group": "production", + }, + }, + Values: getTestValueStream(0, 100, 10, testStartTime), + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "http_requests", + clientmodel.JobLabel: "api-server", + "instance": "1", + "group": "production", + }, + }, + Values: getTestValueStream(0, 200, 20, testStartTime), + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "http_requests", + clientmodel.JobLabel: "api-server", + "instance": "0", + "group": "canary", + }, + }, + Values: getTestValueStream(0, 300, 30, testStartTime), + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "http_requests", + clientmodel.JobLabel: "api-server", + "instance": "1", + "group": "canary", + }, + }, + Values: getTestValueStream(0, 400, 40, testStartTime), + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "http_requests", + clientmodel.JobLabel: "app-server", + "instance": "0", + "group": "production", + }, + }, + Values: getTestValueStream(0, 500, 50, testStartTime), + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "http_requests", + clientmodel.JobLabel: "app-server", + "instance": "1", + "group": "production", + }, + }, + Values: getTestValueStream(0, 600, 60, testStartTime), + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "http_requests", + clientmodel.JobLabel: "app-server", + "instance": "0", + "group": "canary", + }, + }, + Values: getTestValueStream(0, 700, 70, testStartTime), + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "http_requests", + clientmodel.JobLabel: "app-server", + "instance": "1", + "group": "canary", + }, + }, + Values: getTestValueStream(0, 800, 80, testStartTime), + }, + // Single-letter metric and label names. + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "x", + "y": "testvalue", + }, + }, + Values: getTestValueStream(0, 100, 10, testStartTime), + }, + // Counter reset in the middle of range. + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "testcounter_reset_middle", + }, + }, + Values: append(getTestValueStream(0, 40, 10, testStartTime), getTestValueStream(0, 50, 10, testStartTime.Add(testSampleInterval*5))...), + }, + // Counter reset at the end of range. + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "testcounter_reset_end", + }, + }, + Values: append(getTestValueStream(0, 90, 10, testStartTime), getTestValueStream(0, 0, 10, testStartTime.Add(testSampleInterval*10))...), + }, + // For label-key grouping regression test. + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "label_grouping_test", + "a": "aa", + "b": "bb", + }, + }, + Values: getTestValueStream(0, 100, 10, testStartTime), + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "label_grouping_test", + "a": "a", + "b": "abb", + }, + }, + Values: getTestValueStream(0, 200, 20, testStartTime), + }, + // Two histograms with 4 buckets each (*_sum and *_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. + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "testhistogram_bucket", + "le": "0.1", + "start": "positive", + }, + }, + Values: getTestValueStream(0, 50, 5, testStartTime), + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "testhistogram_bucket", + "le": ".2", + "start": "positive", + }, + }, + Values: getTestValueStream(0, 70, 7, testStartTime), + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "testhistogram_bucket", + "le": "1e0", + "start": "positive", + }, + }, + Values: getTestValueStream(0, 110, 11, testStartTime), + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "testhistogram_bucket", + "le": "+Inf", + "start": "positive", + }, + }, + Values: getTestValueStream(0, 120, 12, testStartTime), + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "testhistogram_bucket", + "le": "-.2", + "start": "negative", + }, + }, + Values: getTestValueStream(0, 10, 1, testStartTime), + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "testhistogram_bucket", + "le": "-0.1", + "start": "negative", + }, + }, + Values: getTestValueStream(0, 20, 2, testStartTime), + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "testhistogram_bucket", + "le": "0.3", + "start": "negative", + }, + }, + Values: getTestValueStream(0, 20, 2, testStartTime), + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "testhistogram_bucket", + "le": "+Inf", + "start": "negative", + }, + }, + Values: getTestValueStream(0, 30, 3, testStartTime), + }, + // Now a more realistic histogram per job and instance to test aggregation. + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "request_duration_seconds_bucket", + clientmodel.JobLabel: "job1", + "instance": "ins1", + "le": "0.1", + }, + }, + Values: getTestValueStream(0, 10, 1, testStartTime), + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "request_duration_seconds_bucket", + clientmodel.JobLabel: "job1", + "instance": "ins1", + "le": "0.2", + }, + }, + Values: getTestValueStream(0, 30, 3, testStartTime), + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "request_duration_seconds_bucket", + clientmodel.JobLabel: "job1", + "instance": "ins1", + "le": "+Inf", + }, + }, + Values: getTestValueStream(0, 40, 4, testStartTime), + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "request_duration_seconds_bucket", + clientmodel.JobLabel: "job1", + "instance": "ins2", + "le": "0.1", + }, + }, + Values: getTestValueStream(0, 20, 2, testStartTime), + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "request_duration_seconds_bucket", + clientmodel.JobLabel: "job1", + "instance": "ins2", + "le": "0.2", + }, + }, + Values: getTestValueStream(0, 50, 5, testStartTime), + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "request_duration_seconds_bucket", + clientmodel.JobLabel: "job1", + "instance": "ins2", + "le": "+Inf", + }, + }, + Values: getTestValueStream(0, 60, 6, testStartTime), + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "request_duration_seconds_bucket", + clientmodel.JobLabel: "job2", + "instance": "ins1", + "le": "0.1", + }, + }, + Values: getTestValueStream(0, 30, 3, testStartTime), + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "request_duration_seconds_bucket", + clientmodel.JobLabel: "job2", + "instance": "ins1", + "le": "0.2", + }, + }, + Values: getTestValueStream(0, 40, 4, testStartTime), + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "request_duration_seconds_bucket", + clientmodel.JobLabel: "job2", + "instance": "ins1", + "le": "+Inf", + }, + }, + Values: getTestValueStream(0, 60, 6, testStartTime), + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "request_duration_seconds_bucket", + clientmodel.JobLabel: "job2", + "instance": "ins2", + "le": "0.1", + }, + }, + Values: getTestValueStream(0, 40, 4, testStartTime), + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "request_duration_seconds_bucket", + clientmodel.JobLabel: "job2", + "instance": "ins2", + "le": "0.2", + }, + }, + Values: getTestValueStream(0, 70, 7, testStartTime), + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "request_duration_seconds_bucket", + clientmodel.JobLabel: "job2", + "instance": "ins2", + "le": "+Inf", + }, + }, + Values: getTestValueStream(0, 90, 9, testStartTime), + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "vector_matching_a", + "l": "x", + }, + }, + Values: getTestValueStream(0, 100, 1, testStartTime), + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "vector_matching_a", + "l": "y", + }, + }, + Values: getTestValueStream(0, 100, 2, testStartTime), + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "vector_matching_b", + "l": "x", + }, + }, + Values: getTestValueStream(0, 100, 4, testStartTime), + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "cpu_count", + "instance": "0", + "type": "numa", + }, + }, + Values: getTestValueStream(0, 500, 30, testStartTime), + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "cpu_count", + "instance": "0", + "type": "smp", + }, + }, + Values: getTestValueStream(0, 200, 10, testStartTime), + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "cpu_count", + "instance": "1", + "type": "smp", + }, + }, + Values: getTestValueStream(0, 200, 20, testStartTime), + }, +} From 3ca11bcaf58685d3994d6f643fff57075cbed595 Mon Sep 17 00:00:00 2001 From: Fabian Reinartz Date: Mon, 30 Mar 2015 19:43:19 +0200 Subject: [PATCH 03/10] Switch Prometheus to promql package. This commit removes all functionality from rules/ that is now handled in promql/. All parts of Prometheus are changed to use the promql/ package. --- main.go | 36 +- rules/Makefile | 27 - rules/alerting.go | 37 +- rules/ast/ast.go | 1175 ------------- rules/ast/functions.go | 752 -------- rules/ast/functions_test.go | 63 - rules/ast/printer.go | 441 ----- rules/ast/quantile.go | 106 -- rules/ast/query_analyzer.go | 229 --- rules/ast/walk.go | 28 - rules/fixtures/empty.rules | 0 rules/fixtures/mixed.rules | 16 - rules/fixtures/non_vector.rules | 1 - rules/fixtures/syntax_error.rules | 15 - rules/helpers.go | 221 --- rules/helpers_test.go | 487 ----- rules/lexer.l | 118 -- rules/lexer.l.go | 2738 ----------------------------- rules/load.go | 164 -- rules/{manager => }/manager.go | 90 +- rules/parser.y | 281 --- rules/parser.y.go | 784 --------- rules/recording.go | 34 +- rules/rules.go | 11 +- rules/rules_test.go | 1720 ++---------------- templates/templates.go | 25 +- templates/templates_test.go | 5 +- tools/rule_checker/main.go | 10 +- utility/strconv.go | 23 + web/alerts.go | 6 +- web/api/api.go | 18 +- web/api/api_test.go | 10 +- web/api/query.go | 64 +- web/consoles.go | 8 +- web/httputils/httputils.go | 40 + web/status.go | 4 +- 36 files changed, 351 insertions(+), 9436 deletions(-) delete mode 100644 rules/Makefile delete mode 100644 rules/ast/ast.go delete mode 100644 rules/ast/functions.go delete mode 100644 rules/ast/functions_test.go delete mode 100644 rules/ast/printer.go delete mode 100644 rules/ast/quantile.go delete mode 100644 rules/ast/query_analyzer.go delete mode 100644 rules/ast/walk.go delete mode 100644 rules/fixtures/empty.rules delete mode 100644 rules/fixtures/mixed.rules delete mode 100644 rules/fixtures/non_vector.rules delete mode 100644 rules/fixtures/syntax_error.rules delete mode 100644 rules/helpers.go delete mode 100644 rules/helpers_test.go delete mode 100644 rules/lexer.l delete mode 100644 rules/lexer.l.go delete mode 100644 rules/load.go rename rules/{manager => }/manager.go (78%) delete mode 100644 rules/parser.y delete mode 100644 rules/parser.y.go diff --git a/main.go b/main.go index 1deabd339..3e6f77550 100644 --- a/main.go +++ b/main.go @@ -32,8 +32,9 @@ import ( "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/notification" + "github.com/prometheus/prometheus/promql" "github.com/prometheus/prometheus/retrieval" - "github.com/prometheus/prometheus/rules/manager" + "github.com/prometheus/prometheus/rules" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/storage/local" "github.com/prometheus/prometheus/storage/remote" @@ -76,7 +77,8 @@ var ( ) type prometheus struct { - ruleManager manager.RuleManager + queryEngine *promql.Engine + ruleManager rules.RuleManager targetManager retrieval.TargetManager notificationHandler *notification.NotificationHandler storage local.Storage @@ -155,17 +157,26 @@ func NewPrometheus() *prometheus { targetManager := retrieval.NewTargetManager(sampleAppender, conf.GlobalLabels()) targetManager.AddTargetsFromConfig(conf) - ruleManager := manager.NewRuleManager(&manager.RuleManagerOptions{ + queryEngine := promql.NewEngine(memStorage) + + ruleManager := rules.NewRuleManager(&rules.RuleManagerOptions{ SampleAppender: sampleAppender, NotificationHandler: notificationHandler, EvaluationInterval: conf.EvaluationInterval(), - Storage: memStorage, + QueryEngine: queryEngine, PrometheusURL: web.MustBuildServerURL(*pathPrefix), PathPrefix: *pathPrefix, }) - if err := ruleManager.AddRulesFromConfig(conf); err != nil { - glog.Error("Error loading rule files: ", err) - os.Exit(1) + for _, rf := range conf.Global.GetRuleFile() { + query, err := queryEngine.NewQueryFromFile(rf) + if err != nil { + glog.Errorf("Error loading rule file %q: %s", rf, err) + os.Exit(1) + } + if res := query.Exec(); res.Err != nil { + glog.Errorf("Error initializing rules: %s", res.Err) + os.Exit(1) + } } flags := map[string]string{} @@ -188,8 +199,8 @@ func NewPrometheus() *prometheus { } consolesHandler := &web.ConsolesHandler{ - Storage: memStorage, - PathPrefix: *pathPrefix, + QueryEngine: queryEngine, + PathPrefix: *pathPrefix, } graphsHandler := &web.GraphsHandler{ @@ -197,8 +208,9 @@ func NewPrometheus() *prometheus { } metricsService := &api.MetricsService{ - Now: clientmodel.Now, - Storage: memStorage, + Now: clientmodel.Now, + Storage: memStorage, + QueryEngine: queryEngine, } webService := &web.WebService{ @@ -210,6 +222,7 @@ func NewPrometheus() *prometheus { } p := &prometheus{ + queryEngine: queryEngine, ruleManager: ruleManager, targetManager: targetManager, notificationHandler: notificationHandler, @@ -252,6 +265,7 @@ func (p *prometheus) Serve() { p.targetManager.Stop() p.ruleManager.Stop() + p.queryEngine.Stop() if err := p.storage.Stop(); err != nil { glog.Error("Error stopping local storage: ", err) diff --git a/rules/Makefile b/rules/Makefile deleted file mode 100644 index 12df33ff0..000000000 --- a/rules/Makefile +++ /dev/null @@ -1,27 +0,0 @@ -# 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. - -all: parser.y.go lexer.l.go - -include ../Makefile.INCLUDE - -parser.y.go: parser.y - $(GOCC) tool yacc -o parser.y.go -v "" parser.y - -lexer.l.go: parser.y.go lexer.l - # This is golex from https://github.com/cznic/golex. - $(GO_GET) github.com/cznic/golex - golex -o="lexer.l.go" lexer.l - -clean: - rm lexer.l.go parser.y.go diff --git a/rules/alerting.go b/rules/alerting.go index 7703db1c2..8a558aeca 100644 --- a/rules/alerting.go +++ b/rules/alerting.go @@ -22,9 +22,7 @@ import ( clientmodel "github.com/prometheus/client_golang/model" - "github.com/prometheus/prometheus/rules/ast" - "github.com/prometheus/prometheus/stats" - "github.com/prometheus/prometheus/storage/local" + "github.com/prometheus/prometheus/promql" "github.com/prometheus/prometheus/utility" ) @@ -80,7 +78,7 @@ type Alert struct { } // sample returns a Sample suitable for recording the alert. -func (a Alert) sample(timestamp clientmodel.Timestamp, value clientmodel.SampleValue) *ast.Sample { +func (a Alert) sample(timestamp clientmodel.Timestamp, value clientmodel.SampleValue) *promql.Sample { recordedMetric := clientmodel.Metric{} for label, value := range a.Labels { recordedMetric[label] = value @@ -90,7 +88,7 @@ func (a Alert) sample(timestamp clientmodel.Timestamp, value clientmodel.SampleV recordedMetric[AlertNameLabel] = clientmodel.LabelValue(a.Name) recordedMetric[AlertStateLabel] = clientmodel.LabelValue(a.State.String()) - return &ast.Sample{ + return &promql.Sample{ Metric: clientmodel.COWMetric{ Metric: recordedMetric, Copied: true, @@ -105,7 +103,7 @@ type AlertingRule struct { // The name of the alert. name string // The vector expression from which to generate alerts. - Vector ast.VectorNode + Vector promql.Expr // The duration for which a labelset needs to persist in the expression // output vector before an alert transitions from Pending to Firing state. holdDuration time.Duration @@ -129,14 +127,18 @@ func (rule *AlertingRule) Name() string { } // EvalRaw returns the raw value of the rule expression, without creating alerts. -func (rule *AlertingRule) EvalRaw(timestamp clientmodel.Timestamp, storage local.Storage) (ast.Vector, error) { - return ast.EvalVectorInstant(rule.Vector, timestamp, storage, stats.NewTimerGroup()) +func (rule *AlertingRule) EvalRaw(timestamp clientmodel.Timestamp, engine *promql.Engine) (promql.Vector, error) { + query, err := engine.NewInstantQuery(rule.Vector.String(), timestamp) + if err != nil { + return nil, err + } + return query.Exec().Vector() } // Eval evaluates the rule expression and then creates pending alerts and fires // or removes previously pending alerts accordingly. -func (rule *AlertingRule) Eval(timestamp clientmodel.Timestamp, storage local.Storage) (ast.Vector, error) { - exprResult, err := rule.EvalRaw(timestamp, storage) +func (rule *AlertingRule) Eval(timestamp clientmodel.Timestamp, engine *promql.Engine) (promql.Vector, error) { + exprResult, err := rule.EvalRaw(timestamp, engine) if err != nil { return nil, err } @@ -170,7 +172,7 @@ func (rule *AlertingRule) Eval(timestamp clientmodel.Timestamp, storage local.St } } - vector := ast.Vector{} + vector := promql.Vector{} // Check if any pending alerts should be removed or fire now. Write out alert timeseries. for fp, activeAlert := range rule.activeAlerts { @@ -191,8 +193,8 @@ func (rule *AlertingRule) Eval(timestamp clientmodel.Timestamp, storage local.St return vector, nil } -// ToDotGraph returns the text representation of a dot graph. -func (rule *AlertingRule) ToDotGraph() string { +// DotGraph returns the text representation of a dot graph. +func (rule *AlertingRule) DotGraph() string { graph := fmt.Sprintf( `digraph "Rules" { %#p[shape="box",label="ALERT %s IF FOR %s"]; @@ -201,7 +203,8 @@ func (rule *AlertingRule) ToDotGraph() string { }`, &rule, rule.name, utility.DurationToString(rule.holdDuration), &rule, reflect.ValueOf(rule.Vector).Pointer(), - rule.Vector.NodeTreeToDotGraph()) + rule.Vector.DotGraph(), + ) return graph } @@ -217,9 +220,9 @@ func (rule *AlertingRule) HTMLSnippet() template.HTML { } return template.HTML(fmt.Sprintf( `ALERT %s IF %s FOR %s WITH %s`, - GraphLinkForExpression(alertMetric.String()), + utility.GraphLinkForExpression(alertMetric.String()), rule.name, - GraphLinkForExpression(rule.Vector.String()), + utility.GraphLinkForExpression(rule.Vector.String()), rule.Vector, utility.DurationToString(rule.holdDuration), rule.Labels)) @@ -252,7 +255,7 @@ func (rule *AlertingRule) ActiveAlerts() []Alert { } // NewAlertingRule constructs a new AlertingRule. -func NewAlertingRule(name string, vector ast.VectorNode, holdDuration time.Duration, labels clientmodel.LabelSet, summary string, description string) *AlertingRule { +func NewAlertingRule(name string, vector promql.Expr, holdDuration time.Duration, labels clientmodel.LabelSet, summary string, description string) *AlertingRule { return &AlertingRule{ name: name, Vector: vector, diff --git a/rules/ast/ast.go b/rules/ast/ast.go deleted file mode 100644 index a279a4f9d..000000000 --- a/rules/ast/ast.go +++ /dev/null @@ -1,1175 +0,0 @@ -// 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 ast - -import ( - "errors" - "flag" - "fmt" - "math" - "time" - - clientmodel "github.com/prometheus/client_golang/model" - - "github.com/prometheus/prometheus/stats" - "github.com/prometheus/prometheus/storage/local" - "github.com/prometheus/prometheus/storage/metric" -) - -var ( - stalenessDelta = flag.Duration("query.staleness-delta", 300*time.Second, "Staleness delta allowance during expression evaluations.") - queryTimeout = flag.Duration("query.timeout", 2*time.Minute, "Maximum time a query may take before being aborted.") -) - -type queryTimeoutError struct { - timeoutAfter time.Duration -} - -func (e queryTimeoutError) Error() string { - return fmt.Sprintf("query timeout after %v", e.timeoutAfter) -} - -// ---------------------------------------------------------------------------- -// Raw data value types. - -// SampleStream is a stream of Values belonging to an attached COWMetric. -type SampleStream struct { - Metric clientmodel.COWMetric `json:"metric"` - Values metric.Values `json:"values"` -} - -// Sample is a single sample belonging to a COWMetric. -type Sample struct { - Metric clientmodel.COWMetric `json:"metric"` - Value clientmodel.SampleValue `json:"value"` - Timestamp clientmodel.Timestamp `json:"timestamp"` -} - -// Vector is basically only an alias for clientmodel.Samples, but the -// contract is that in a Vector, all Samples have the same timestamp. -type Vector []*Sample - -// Matrix is a slice of SampleStreams that implements sort.Interface and -// has a String method. -// BUG(julius): Pointerize this. -type Matrix []SampleStream - -type groupedAggregation struct { - labels clientmodel.COWMetric - value clientmodel.SampleValue - groupCount int -} - -// ---------------------------------------------------------------------------- -// Enums. - -// ExprType is an enum for the rule language expression types. -type ExprType int - -// Possible language expression types. We define these as integer constants -// because sometimes we need to pass around just the type without an object of -// that type. -const ( - ScalarType ExprType = iota - VectorType - MatrixType - StringType -) - -// BinOpType is an enum for binary operator types. -type BinOpType int - -// Possible binary operator types. -const ( - Add BinOpType = iota - Sub - Mul - Div - Mod - NE - EQ - GT - LT - GE - LE - And - Or -) - -// shouldDropMetric indicates whether the metric name should be dropped after -// applying this operator to a vector. -func (opType BinOpType) shouldDropMetric() bool { - switch opType { - case Add, Sub, Mul, Div, Mod: - return true - default: - return false - } -} - -// AggrType is an enum for aggregation types. -type AggrType int - -// Possible aggregation types. -const ( - Sum AggrType = iota - Avg - Min - Max - Count -) - -// ---------------------------------------------------------------------------- -// Interfaces. - -// Nodes is a slice of any mix of node types as all node types -// implement the Node interface. -type Nodes []Node - -// Node is the top-level interface for any kind of nodes. Each node -// type implements one of the ...Node interfaces, each of which embeds -// this Node interface. -type Node interface { - Type() ExprType - Children() Nodes - NodeTreeToDotGraph() string - String() string -} - -// ScalarNode is a Node for scalar values. -type ScalarNode interface { - Node - // Eval evaluates and returns the value of the scalar represented by this node. - Eval(timestamp clientmodel.Timestamp) clientmodel.SampleValue -} - -// VectorNode is a Node for vector values. -type VectorNode interface { - Node - // Eval evaluates the node recursively and returns the result - // as a Vector (i.e. a slice of Samples all at the given - // Timestamp). - Eval(timestamp clientmodel.Timestamp) Vector -} - -// MatrixNode is a Node for matrix values. -type MatrixNode interface { - Node - // Eval evaluates the node recursively and returns the result as a Matrix. - Eval(timestamp clientmodel.Timestamp) Matrix - // Eval evaluates the node recursively and returns the result - // as a Matrix that only contains the boundary values. - EvalBoundaries(timestamp clientmodel.Timestamp) Matrix -} - -// StringNode is a Node for string values. -type StringNode interface { - Node - // Eval evaluates and returns the value of the string - // represented by this node. - Eval(timestamp clientmodel.Timestamp) string -} - -// ---------------------------------------------------------------------------- -// ScalarNode types. - -type ( - // ScalarLiteral represents a numeric selector. - ScalarLiteral struct { - value clientmodel.SampleValue - } - - // ScalarFunctionCall represents a function with a numeric - // return type. - ScalarFunctionCall struct { - function *Function - args Nodes - } - - // ScalarArithExpr represents an arithmetic expression of - // numeric type. - ScalarArithExpr struct { - opType BinOpType - lhs ScalarNode - rhs ScalarNode - } -) - -// ---------------------------------------------------------------------------- -// VectorNode types. - -type ( - // A VectorSelector represents a metric name plus labelset. - VectorSelector struct { - labelMatchers metric.LabelMatchers - offset time.Duration - // The series iterators are populated at query analysis time. - iterators map[clientmodel.Fingerprint]local.SeriesIterator - metrics map[clientmodel.Fingerprint]clientmodel.COWMetric - // Fingerprints are populated from label matchers at query analysis time. - fingerprints clientmodel.Fingerprints - } - - // VectorFunctionCall represents a function with vector return - // type. - VectorFunctionCall struct { - function *Function - args Nodes - } - - // A VectorAggregation with vector return type. - VectorAggregation struct { - aggrType AggrType - groupBy clientmodel.LabelNames - keepExtraLabels bool - vector VectorNode - } - - // VectorArithExpr represents an arithmetic expression of vector type. At - // least one of the two operand Nodes must be a VectorNode. The other may be - // a VectorNode or ScalarNode. Both criteria are checked at runtime. - VectorArithExpr struct { - opType BinOpType - lhs Node - rhs Node - matchCardinality VectorMatchCardinality - matchOn clientmodel.LabelNames - includeLabels clientmodel.LabelNames - } -) - -// VectorMatchCardinality is an enum describing vector matches (1:1, n:1, 1:n, n:m). -type VectorMatchCardinality int - -// Constants for VectorMatchCardinality enum. -const ( - MatchOneToOne VectorMatchCardinality = iota - MatchManyToOne - MatchOneToMany - MatchManyToMany -) - -// ---------------------------------------------------------------------------- -// MatrixNode types. - -type ( - // A MatrixSelector represents a metric name plus labelset and - // timerange. - MatrixSelector struct { - labelMatchers metric.LabelMatchers - // The series iterators are populated at query analysis time. - iterators map[clientmodel.Fingerprint]local.SeriesIterator - metrics map[clientmodel.Fingerprint]clientmodel.COWMetric - // Fingerprints are populated from label matchers at query analysis time. - fingerprints clientmodel.Fingerprints - interval time.Duration - offset time.Duration - } -) - -// ---------------------------------------------------------------------------- -// StringNode types. - -type ( - // A StringLiteral is what you think it is. - StringLiteral struct { - str string - } - - // StringFunctionCall represents a function with string return - // type. - StringFunctionCall struct { - function *Function - args Nodes - } -) - -// ---------------------------------------------------------------------------- -// Implementations. - -// Type implements the Node interface. -func (node ScalarLiteral) Type() ExprType { return ScalarType } - -// Type implements the Node interface. -func (node ScalarFunctionCall) Type() ExprType { return ScalarType } - -// Type implements the Node interface. -func (node ScalarArithExpr) Type() ExprType { return ScalarType } - -// Type implements the Node interface. -func (node VectorSelector) Type() ExprType { return VectorType } - -// Type implements the Node interface. -func (node VectorFunctionCall) Type() ExprType { return VectorType } - -// Type implements the Node interface. -func (node VectorAggregation) Type() ExprType { return VectorType } - -// Type implements the Node interface. -func (node VectorArithExpr) Type() ExprType { return VectorType } - -// Type implements the Node interface. -func (node MatrixSelector) Type() ExprType { return MatrixType } - -// Type implements the Node interface. -func (node StringLiteral) Type() ExprType { return StringType } - -// Type implements the Node interface. -func (node StringFunctionCall) Type() ExprType { return StringType } - -// Children implements the Node interface and returns an empty slice. -func (node ScalarLiteral) Children() Nodes { return Nodes{} } - -// Children implements the Node interface and returns the args of the -// function call. -func (node ScalarFunctionCall) Children() Nodes { return node.args } - -// Children implements the Node interface and returns the LHS and the RHS -// of the expression. -func (node ScalarArithExpr) Children() Nodes { return Nodes{node.lhs, node.rhs} } - -// Children implements the Node interface and returns an empty slice. -func (node VectorSelector) Children() Nodes { return Nodes{} } - -// Children implements the Node interface and returns the args of the -// function call. -func (node VectorFunctionCall) Children() Nodes { return node.args } - -// Children implements the Node interface and returns the vector to be -// aggregated. -func (node VectorAggregation) Children() Nodes { return Nodes{node.vector} } - -// Children implements the Node interface and returns the LHS and the RHS -// of the expression. -func (node VectorArithExpr) Children() Nodes { return Nodes{node.lhs, node.rhs} } - -// Children implements the Node interface and returns an empty slice. -func (node MatrixSelector) Children() Nodes { return Nodes{} } - -// Children implements the Node interface and returns an empty slice. -func (node StringLiteral) Children() Nodes { return Nodes{} } - -// Children implements the Node interface and returns the args of the -// function call. -func (node StringFunctionCall) Children() Nodes { return node.args } - -// Eval implements the ScalarNode interface and returns the selector -// value. -func (node *ScalarLiteral) Eval(timestamp clientmodel.Timestamp) clientmodel.SampleValue { - return node.value -} - -// Eval implements the ScalarNode interface and returns the result of -// the expression. -func (node *ScalarArithExpr) Eval(timestamp clientmodel.Timestamp) clientmodel.SampleValue { - lhs := node.lhs.Eval(timestamp) - rhs := node.rhs.Eval(timestamp) - return evalScalarBinop(node.opType, lhs, rhs) -} - -// Eval implements the ScalarNode interface and returns the result of -// the function call. -func (node *ScalarFunctionCall) Eval(timestamp clientmodel.Timestamp) clientmodel.SampleValue { - return node.function.callFn(timestamp, node.args).(clientmodel.SampleValue) -} - -// EvalVectorInstant evaluates a VectorNode with an instant query. -func EvalVectorInstant(node VectorNode, timestamp clientmodel.Timestamp, storage local.Storage, queryStats *stats.TimerGroup) (Vector, error) { - totalEvalTimer := queryStats.GetTimer(stats.TotalEvalTime).Start() - defer totalEvalTimer.Stop() - - closer, err := prepareInstantQuery(node, timestamp, storage, queryStats) - if err != nil { - return nil, err - } - defer closer.Close() - if et := totalEvalTimer.ElapsedTime(); et > *queryTimeout { - return nil, queryTimeoutError{et} - } - return node.Eval(timestamp), nil -} - -// EvalVectorRange evaluates a VectorNode with a range query. -func EvalVectorRange(node VectorNode, start clientmodel.Timestamp, end clientmodel.Timestamp, interval time.Duration, storage local.Storage, queryStats *stats.TimerGroup) (Matrix, error) { - totalEvalTimer := queryStats.GetTimer(stats.TotalEvalTime).Start() - defer totalEvalTimer.Stop() - // Explicitly initialize to an empty matrix since a nil Matrix encodes to - // null in JSON. - matrix := Matrix{} - - prepareTimer := queryStats.GetTimer(stats.TotalQueryPreparationTime).Start() - closer, err := prepareRangeQuery(node, start, end, interval, storage, queryStats) - prepareTimer.Stop() - if err != nil { - return nil, err - } - defer closer.Close() - - evalTimer := queryStats.GetTimer(stats.InnerEvalTime).Start() - sampleStreams := map[clientmodel.Fingerprint]*SampleStream{} - for t := start; !t.After(end); t = t.Add(interval) { - if et := totalEvalTimer.ElapsedTime(); et > *queryTimeout { - evalTimer.Stop() - return nil, queryTimeoutError{et} - } - vector := node.Eval(t) - for _, sample := range vector { - samplePair := metric.SamplePair{ - Value: sample.Value, - Timestamp: sample.Timestamp, - } - fp := sample.Metric.Metric.Fingerprint() - if sampleStreams[fp] == nil { - sampleStreams[fp] = &SampleStream{ - Metric: sample.Metric, - Values: metric.Values{samplePair}, - } - } else { - sampleStreams[fp].Values = append(sampleStreams[fp].Values, samplePair) - } - } - } - evalTimer.Stop() - - appendTimer := queryStats.GetTimer(stats.ResultAppendTime).Start() - for _, sampleStream := range sampleStreams { - matrix = append(matrix, *sampleStream) - } - appendTimer.Stop() - - return matrix, nil -} - -func labelIntersection(metric1, metric2 clientmodel.COWMetric) clientmodel.COWMetric { - for label, value := range metric1.Metric { - if metric2.Metric[label] != value { - metric1.Delete(label) - } - } - return metric1 -} - -func (node *VectorAggregation) groupedAggregationsToVector(aggregations map[uint64]*groupedAggregation, timestamp clientmodel.Timestamp) Vector { - vector := Vector{} - for _, aggregation := range aggregations { - switch node.aggrType { - case Avg: - aggregation.value = aggregation.value / clientmodel.SampleValue(aggregation.groupCount) - case Count: - aggregation.value = clientmodel.SampleValue(aggregation.groupCount) - default: - // For other aggregations, we already have the right value. - } - sample := &Sample{ - Metric: aggregation.labels, - Value: aggregation.value, - Timestamp: timestamp, - } - vector = append(vector, sample) - } - return vector -} - -// Eval implements the VectorNode interface and returns the aggregated -// Vector. -func (node *VectorAggregation) Eval(timestamp clientmodel.Timestamp) Vector { - vector := node.vector.Eval(timestamp) - result := map[uint64]*groupedAggregation{} - for _, sample := range vector { - groupingKey := clientmodel.SignatureForLabels(sample.Metric.Metric, node.groupBy) - if groupedResult, ok := result[groupingKey]; ok { - if node.keepExtraLabels { - groupedResult.labels = labelIntersection(groupedResult.labels, sample.Metric) - } - - switch node.aggrType { - case Sum: - groupedResult.value += sample.Value - case Avg: - groupedResult.value += sample.Value - groupedResult.groupCount++ - case Max: - if groupedResult.value < sample.Value { - groupedResult.value = sample.Value - } - case Min: - if groupedResult.value > sample.Value { - groupedResult.value = sample.Value - } - case Count: - groupedResult.groupCount++ - default: - panic("Unknown aggregation type") - } - } else { - var m clientmodel.COWMetric - if node.keepExtraLabels { - m = sample.Metric - m.Delete(clientmodel.MetricNameLabel) - } else { - m = clientmodel.COWMetric{ - Metric: clientmodel.Metric{}, - Copied: true, - } - for _, l := range node.groupBy { - if v, ok := sample.Metric.Metric[l]; ok { - m.Set(l, v) - } - } - } - result[groupingKey] = &groupedAggregation{ - labels: m, - value: sample.Value, - groupCount: 1, - } - } - } - - return node.groupedAggregationsToVector(result, timestamp) -} - -// Eval implements the VectorNode interface and returns the value of -// the selector. -func (node *VectorSelector) Eval(timestamp clientmodel.Timestamp) Vector { - //// timer := v.stats.GetTimer(stats.GetValueAtTimeTime).Start() - samples := Vector{} - for fp, it := range node.iterators { - sampleCandidates := it.GetValueAtTime(timestamp.Add(-node.offset)) - samplePair := chooseClosestSample(sampleCandidates, timestamp.Add(-node.offset)) - if samplePair != nil { - samples = append(samples, &Sample{ - Metric: node.metrics[fp], - Value: samplePair.Value, - Timestamp: timestamp, - }) - } - } - //// timer.Stop() - return samples -} - -// chooseClosestSample chooses the closest sample of a list of samples -// surrounding a given target time. If samples are found both before and after -// the target time, the sample value is interpolated between these. Otherwise, -// the single closest sample is returned verbatim. -func chooseClosestSample(samples metric.Values, timestamp clientmodel.Timestamp) *metric.SamplePair { - var closestBefore *metric.SamplePair - var closestAfter *metric.SamplePair - for _, candidate := range samples { - delta := candidate.Timestamp.Sub(timestamp) - // Samples before target time. - if delta < 0 { - // Ignore samples outside of staleness policy window. - if -delta > *stalenessDelta { - continue - } - // Ignore samples that are farther away than what we've seen before. - if closestBefore != nil && candidate.Timestamp.Before(closestBefore.Timestamp) { - continue - } - sample := candidate - closestBefore = &sample - } - - // Samples after target time. - if delta >= 0 { - // Ignore samples outside of staleness policy window. - if delta > *stalenessDelta { - continue - } - // Ignore samples that are farther away than samples we've seen before. - if closestAfter != nil && candidate.Timestamp.After(closestAfter.Timestamp) { - continue - } - sample := candidate - closestAfter = &sample - } - } - - switch { - case closestBefore != nil && closestAfter != nil: - return interpolateSamples(closestBefore, closestAfter, timestamp) - case closestBefore != nil: - return closestBefore - default: - return closestAfter - } -} - -// interpolateSamples interpolates a value at a target time between two -// provided sample pairs. -func interpolateSamples(first, second *metric.SamplePair, timestamp clientmodel.Timestamp) *metric.SamplePair { - dv := second.Value - first.Value - dt := second.Timestamp.Sub(first.Timestamp) - - dDt := dv / clientmodel.SampleValue(dt) - offset := clientmodel.SampleValue(timestamp.Sub(first.Timestamp)) - - return &metric.SamplePair{ - Value: first.Value + (offset * dDt), - Timestamp: timestamp, - } -} - -// Eval implements the VectorNode interface and returns the result of -// the function call. -func (node *VectorFunctionCall) Eval(timestamp clientmodel.Timestamp) Vector { - return node.function.callFn(timestamp, node.args).(Vector) -} - -func evalScalarBinop(opType BinOpType, - lhs clientmodel.SampleValue, - rhs clientmodel.SampleValue) clientmodel.SampleValue { - switch opType { - case Add: - return lhs + rhs - case Sub: - return lhs - rhs - case Mul: - return lhs * rhs - case Div: - return lhs / rhs - case Mod: - if rhs != 0 { - return clientmodel.SampleValue(int(lhs) % int(rhs)) - } - return clientmodel.SampleValue(math.NaN()) - case EQ: - if lhs == rhs { - return 1 - } - return 0 - case NE: - if lhs != rhs { - return 1 - } - return 0 - case GT: - if lhs > rhs { - return 1 - } - return 0 - case LT: - if lhs < rhs { - return 1 - } - return 0 - case GE: - if lhs >= rhs { - return 1 - } - return 0 - case LE: - if lhs <= rhs { - return 1 - } - return 0 - } - panic("Not all enum values enumerated in switch") -} - -func evalVectorBinop(opType BinOpType, - lhs clientmodel.SampleValue, - rhs clientmodel.SampleValue) (clientmodel.SampleValue, bool) { - switch opType { - case Add: - return lhs + rhs, true - case Sub: - return lhs - rhs, true - case Mul: - return lhs * rhs, true - case Div: - return lhs / rhs, true - case Mod: - if rhs != 0 { - return clientmodel.SampleValue(int(lhs) % int(rhs)), true - } - return clientmodel.SampleValue(math.NaN()), true - case EQ: - if lhs == rhs { - return lhs, true - } - return 0, false - case NE: - if lhs != rhs { - return lhs, true - } - return 0, false - case GT: - if lhs > rhs { - return lhs, true - } - return 0, false - case LT: - if lhs < rhs { - return lhs, true - } - return 0, false - case GE: - if lhs >= rhs { - return lhs, true - } - return 0, false - case LE: - if lhs <= rhs { - return lhs, true - } - return 0, false - } - panic("Not all enum values enumerated in switch") -} - -func labelsEqual(labels1, labels2 clientmodel.Metric) bool { - for label, value := range labels1 { - if labels2[label] != value && label != clientmodel.MetricNameLabel { - return false - } - } - return true -} - -// Eval implements the VectorNode interface and returns the result of -// the expression. -func (node *VectorArithExpr) Eval(timestamp clientmodel.Timestamp) Vector { - // Calculate vector-to-vector operation. - if node.lhs.Type() == VectorType && node.rhs.Type() == VectorType { - lhs := node.lhs.(VectorNode).Eval(timestamp) - rhs := node.rhs.(VectorNode).Eval(timestamp) - - return node.evalVectors(timestamp, lhs, rhs) - } - - // Calculate vector-to-scalar operation. - var lhs Vector - var rhs clientmodel.SampleValue - swap := false - - if node.lhs.Type() == ScalarType && node.rhs.Type() == VectorType { - lhs = node.rhs.(VectorNode).Eval(timestamp) - rhs = node.lhs.(ScalarNode).Eval(timestamp) - swap = true - } else { - lhs = node.lhs.(VectorNode).Eval(timestamp) - rhs = node.rhs.(ScalarNode).Eval(timestamp) - } - - result := make(Vector, 0, len(lhs)) - - for _, lhsSample := range lhs { - lv, rv := lhsSample.Value, rhs - // lhs always contains the vector. If the original position was different - // swap for calculating the value. - if swap { - lv, rv = rv, lv - } - value, keep := evalVectorBinop(node.opType, lv, rv) - if keep { - lhsSample.Value = value - if node.opType.shouldDropMetric() { - lhsSample.Metric.Delete(clientmodel.MetricNameLabel) - } - result = append(result, lhsSample) - } - } - return result -} - -// evalVectors evaluates the binary operation for the given vectors. -func (node *VectorArithExpr) evalVectors(timestamp clientmodel.Timestamp, lhs, rhs Vector) Vector { - result := make(Vector, 0, len(rhs)) - // The control flow below handles one-to-one or many-to-one matching. - // For one-to-many, swap sidedness and account for the swap when calculating - // values. - if node.matchCardinality == MatchOneToMany { - lhs, rhs = rhs, lhs - } - // All samples from the rhs hashed by the matching label/values. - rm := make(map[uint64]*Sample) - // Maps the hash of the label values used for matching to the hashes of the label - // values of the include labels (if any). It is used to keep track of already - // inserted samples. - added := make(map[uint64][]uint64) - - // Add all rhs samples to a map so we can easily find matches later. - for _, rs := range rhs { - hash := node.hashForMetric(rs.Metric.Metric) - // The rhs is guaranteed to be the 'one' side. Having multiple samples - // with the same hash means that the matching is many-to-many, - // which is not supported. - if _, found := rm[hash]; node.matchCardinality != MatchManyToMany && found { - // Many-to-many matching not allowed. - // TODO(fabxc): Return a query error here once AST nodes support that. - return Vector{} - } - // In many-to-many matching the entry is simply overwritten. It can thus only - // be used to check whether any matching rhs entry exists but not retrieve them all. - rm[hash] = rs - } - - // For all lhs samples find a respective rhs sample and perform - // the binary operation. - for _, ls := range lhs { - hash := node.hashForMetric(ls.Metric.Metric) - // Any lhs sample we encounter in an OR operation belongs to the result. - if node.opType == Or { - ls.Metric = node.resultMetric(ls, nil) - result = append(result, ls) - added[hash] = nil // Ensure matching rhs sample is not added later. - continue - } - - rs, found := rm[hash] // Look for a match in the rhs vector. - if !found { - continue - } - var value clientmodel.SampleValue - var keep bool - - if node.opType == And { - value = ls.Value - keep = true - } else { - if _, exists := added[hash]; node.matchCardinality == MatchOneToOne && exists { - // Many-to-one matching must be explicit. - // TODO(fabxc): Return a query error here once AST nodes support that. - return Vector{} - } - // Account for potentially swapped sidedness. - vl, vr := ls.Value, rs.Value - if node.matchCardinality == MatchOneToMany { - vl, vr = vr, vl - } - value, keep = evalVectorBinop(node.opType, vl, vr) - } - - if keep { - metric := node.resultMetric(ls, rs) - // Check if the same label set has been added for a many-to-one matching before. - if node.matchCardinality == MatchManyToOne || node.matchCardinality == MatchOneToMany { - insHash := clientmodel.SignatureForLabels(metric.Metric, node.includeLabels) - if ihs, exists := added[hash]; exists { - for _, ih := range ihs { - if ih == insHash { - // TODO(fabxc): Return a query error here once AST nodes support that. - return Vector{} - } - } - added[hash] = append(ihs, insHash) - } else { - added[hash] = []uint64{insHash} - } - } - ns := &Sample{ - Metric: metric, - Value: value, - Timestamp: timestamp, - } - result = append(result, ns) - if _, ok := added[hash]; !ok { - added[hash] = nil // Set existence to true. - } - } - } - - // Add all remaining samples in the rhs in an OR operation if they - // have not been matched up with a lhs sample. - if node.opType == Or { - for hash, rs := range rm { - if _, exists := added[hash]; !exists { - rs.Metric = node.resultMetric(rs, nil) - result = append(result, rs) - } - } - } - return result -} - -// resultMetric returns the metric for the given sample(s) based on the vector -// binary operation and the matching options. If a label that has to be included is set on -// both sides an error is returned. -func (node *VectorArithExpr) resultMetric(ls, rs *Sample) clientmodel.COWMetric { - if len(node.matchOn) == 0 || node.opType == Or || node.opType == And { - if node.opType.shouldDropMetric() { - ls.Metric.Delete(clientmodel.MetricNameLabel) - } - return ls.Metric - } - - m := clientmodel.Metric{} - for _, ln := range node.matchOn { - m[ln] = ls.Metric.Metric[ln] - } - - for _, ln := range node.includeLabels { - // Included labels from the `group_x` modifier are taken from the "many"-side. - v, ok := ls.Metric.Metric[ln] - if ok { - m[ln] = v - } - } - return clientmodel.COWMetric{false, m} -} - -// hashForMetric calculates a hash value for the given metric based on the matching -// options for the binary operation. -func (node *VectorArithExpr) hashForMetric(metric clientmodel.Metric) uint64 { - var labels clientmodel.LabelNames - - if len(node.matchOn) > 0 { - var match bool - for _, ln := range node.matchOn { - if _, match = metric[ln]; !match { - break - } - } - // If the metric does not contain the labels to match on, build the hash - // over the whole metric to give it a unique hash. - if !match { - labels = make(clientmodel.LabelNames, 0, len(metric)) - for ln := range metric { - labels = append(labels, ln) - } - } else { - labels = node.matchOn - } - } else { - labels = make(clientmodel.LabelNames, 0, len(metric)) - for ln := range metric { - if ln != clientmodel.MetricNameLabel { - labels = append(labels, ln) - } - } - } - return clientmodel.SignatureForLabels(metric, labels) -} - -// Eval implements the MatrixNode interface and returns the value of -// the selector. -func (node *MatrixSelector) Eval(timestamp clientmodel.Timestamp) Matrix { - interval := &metric.Interval{ - OldestInclusive: timestamp.Add(-node.interval - node.offset), - NewestInclusive: timestamp.Add(-node.offset), - } - - //// timer := v.stats.GetTimer(stats.GetRangeValuesTime).Start() - sampleStreams := []SampleStream{} - for fp, it := range node.iterators { - samplePairs := it.GetRangeValues(*interval) - if len(samplePairs) == 0 { - continue - } - - if node.offset != 0 { - for _, sp := range samplePairs { - sp.Timestamp = sp.Timestamp.Add(node.offset) - } - } - - sampleStream := SampleStream{ - Metric: node.metrics[fp], - Values: samplePairs, - } - sampleStreams = append(sampleStreams, sampleStream) - } - //// timer.Stop() - return sampleStreams -} - -// EvalBoundaries implements the MatrixNode interface and returns the -// boundary values of the selector. -func (node *MatrixSelector) EvalBoundaries(timestamp clientmodel.Timestamp) Matrix { - interval := &metric.Interval{ - OldestInclusive: timestamp.Add(-node.interval), - NewestInclusive: timestamp, - } - - //// timer := v.stats.GetTimer(stats.GetBoundaryValuesTime).Start() - sampleStreams := []SampleStream{} - for fp, it := range node.iterators { - samplePairs := it.GetBoundaryValues(*interval) - if len(samplePairs) == 0 { - continue - } - - sampleStream := SampleStream{ - Metric: node.metrics[fp], - Values: samplePairs, - } - sampleStreams = append(sampleStreams, sampleStream) - } - //// timer.Stop() - return sampleStreams -} - -// Len implements sort.Interface. -func (matrix Matrix) Len() int { - return len(matrix) -} - -// Less implements sort.Interface. -func (matrix Matrix) Less(i, j int) bool { - return matrix[i].Metric.String() < matrix[j].Metric.String() -} - -// Swap implements sort.Interface. -func (matrix Matrix) Swap(i, j int) { - matrix[i], matrix[j] = matrix[j], matrix[i] -} - -// Eval implements the StringNode interface and returns the value of -// the selector. -func (node *StringLiteral) Eval(timestamp clientmodel.Timestamp) string { - return node.str -} - -// Eval implements the StringNode interface and returns the result of -// the function call. -func (node *StringFunctionCall) Eval(timestamp clientmodel.Timestamp) string { - return node.function.callFn(timestamp, node.args).(string) -} - -// ---------------------------------------------------------------------------- -// Constructors. - -// NewScalarLiteral returns a ScalarLiteral with the given value. -func NewScalarLiteral(value clientmodel.SampleValue) *ScalarLiteral { - return &ScalarLiteral{ - value: value, - } -} - -// NewVectorSelector returns a (not yet evaluated) VectorSelector with -// the given LabelSet. -func NewVectorSelector(m metric.LabelMatchers, offset time.Duration) *VectorSelector { - return &VectorSelector{ - labelMatchers: m, - offset: offset, - iterators: map[clientmodel.Fingerprint]local.SeriesIterator{}, - metrics: map[clientmodel.Fingerprint]clientmodel.COWMetric{}, - } -} - -// NewVectorAggregation returns a (not yet evaluated) -// VectorAggregation, aggregating the given VectorNode using the given -// AggrType, grouping by the given LabelNames. -func NewVectorAggregation(aggrType AggrType, vector VectorNode, groupBy clientmodel.LabelNames, keepExtraLabels bool) *VectorAggregation { - return &VectorAggregation{ - aggrType: aggrType, - groupBy: groupBy, - keepExtraLabels: keepExtraLabels, - vector: vector, - } -} - -// NewFunctionCall returns a (not yet evaluated) function call node -// (of type ScalarFunctionCall, VectorFunctionCall, or -// StringFunctionCall). -func NewFunctionCall(function *Function, args Nodes) (Node, error) { - if err := function.CheckArgTypes(args); err != nil { - return nil, err - } - switch function.returnType { - case ScalarType: - return &ScalarFunctionCall{ - function: function, - args: args, - }, nil - case VectorType: - return &VectorFunctionCall{ - function: function, - args: args, - }, nil - case StringType: - return &StringFunctionCall{ - function: function, - args: args, - }, nil - } - panic("Function with invalid return type") -} - -func nodesHaveTypes(nodes Nodes, exprTypes []ExprType) bool { - for _, node := range nodes { - correctType := false - for _, exprType := range exprTypes { - if node.Type() == exprType { - correctType = true - } - } - if !correctType { - return false - } - } - return true -} - -// NewArithExpr returns a (not yet evaluated) expression node (of type -// VectorArithExpr or ScalarArithExpr). -func NewArithExpr(opType BinOpType, lhs Node, rhs Node, matchCard VectorMatchCardinality, matchOn, include clientmodel.LabelNames) (Node, error) { - if !nodesHaveTypes(Nodes{lhs, rhs}, []ExprType{ScalarType, VectorType}) { - return nil, errors.New("binary operands must be of vector or scalar type") - } - - if opType == And || opType == Or { - if lhs.Type() == ScalarType || rhs.Type() == ScalarType { - return nil, errors.New("AND and OR operators may only be used between vectors") - } - // Logical operations must never be used with group modifiers. - if len(include) > 0 { - return nil, errors.New("AND and OR operators must not have a group modifier") - } - } - if lhs.Type() != VectorType || rhs.Type() != VectorType { - if matchCard != MatchOneToOne || matchOn != nil || include != nil { - return nil, errors.New("binary scalar expressions cannot have vector matching options") - } - } - - if lhs.Type() == VectorType || rhs.Type() == VectorType { - return &VectorArithExpr{ - opType: opType, - lhs: lhs, - rhs: rhs, - matchCardinality: matchCard, - matchOn: matchOn, - includeLabels: include, - }, nil - } - - return &ScalarArithExpr{ - opType: opType, - lhs: lhs.(ScalarNode), - rhs: rhs.(ScalarNode), - }, nil -} - -// NewMatrixSelector returns a (not yet evaluated) MatrixSelector with -// the given VectorSelector and Duration. -func NewMatrixSelector(vector *VectorSelector, interval time.Duration, offset time.Duration) *MatrixSelector { - return &MatrixSelector{ - labelMatchers: vector.labelMatchers, - interval: interval, - offset: offset, - iterators: map[clientmodel.Fingerprint]local.SeriesIterator{}, - metrics: map[clientmodel.Fingerprint]clientmodel.COWMetric{}, - } -} - -// NewStringLiteral returns a StringLiteral with the given string as -// value. -func NewStringLiteral(str string) *StringLiteral { - return &StringLiteral{ - str: str, - } -} diff --git a/rules/ast/functions.go b/rules/ast/functions.go deleted file mode 100644 index ea95a1966..000000000 --- a/rules/ast/functions.go +++ /dev/null @@ -1,752 +0,0 @@ -// 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 ast - -import ( - "container/heap" - "fmt" - "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 - callFn func(timestamp clientmodel.Timestamp, args []Node) interface{} -} - -// CheckArgTypes returns a non-nil error if the number or types of -// passed in arg nodes do not match the function's expectations. -func (function *Function) CheckArgTypes(args []Node) error { - if len(function.argTypes) < len(args) { - return fmt.Errorf( - "too many arguments to function %v(): %v expected at most, %v given", - function.name, len(function.argTypes), len(args), - ) - } - if len(function.argTypes)-function.optionalArgs > len(args) { - return fmt.Errorf( - "too few arguments to function %v(): %v expected at least, %v given", - function.name, len(function.argTypes)-function.optionalArgs, len(args), - ) - } - for idx, arg := range args { - invalidType := false - var expectedType string - if _, ok := arg.(ScalarNode); function.argTypes[idx] == ScalarType && !ok { - invalidType = true - expectedType = "scalar" - } - if _, ok := arg.(VectorNode); function.argTypes[idx] == VectorType && !ok { - invalidType = true - expectedType = "vector" - } - if _, ok := arg.(MatrixNode); function.argTypes[idx] == MatrixType && !ok { - invalidType = true - expectedType = "matrix" - } - if _, ok := arg.(StringNode); function.argTypes[idx] == StringType && !ok { - invalidType = true - expectedType = "string" - } - - if invalidType { - return fmt.Errorf( - "wrong type for argument %v in function %v(), expected %v", - idx, function.name, expectedType, - ) - } - } - return nil -} - -// === time() clientmodel.SampleValue === -func timeImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { - return clientmodel.SampleValue(timestamp.Unix()) -} - -// === delta(matrix MatrixNode, isCounter=0 ScalarNode) Vector === -func deltaImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { - matrixNode := args[0].(MatrixNode) - isCounter := len(args) >= 2 && args[1].(ScalarNode).Eval(timestamp) > 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 = matrixNode.Eval(timestamp) - } else { - matrixValue = matrixNode.EvalBoundaries(timestamp) - } - 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).interval - 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: timestamp, - } - resultSample.Metric.Delete(clientmodel.MetricNameLabel) - resultVector = append(resultVector, resultSample) - } - return resultVector -} - -// === rate(node MatrixNode) Vector === -func rateImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { - args = append(args, &ScalarLiteral{value: 1}) - vector := deltaImpl(timestamp, args).(Vector) - - // TODO: could be other type of MatrixNode 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).interval - for i := range vector { - vector[i].Value /= clientmodel.SampleValue(interval / time.Second) - } - return vector -} - -type vectorByValueHeap Vector - -func (s vectorByValueHeap) Len() int { - return len(s) -} - -func (s vectorByValueHeap) Less(i, j int) bool { - 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) -} - -// === sort(node VectorNode) Vector === -func sortImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { - byValueSorter := vectorByValueHeap(args[0].(VectorNode).Eval(timestamp)) - sort.Sort(byValueSorter) - return Vector(byValueSorter) -} - -// === sortDesc(node VectorNode) Vector === -func sortDescImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { - byValueSorter := vectorByValueHeap(args[0].(VectorNode).Eval(timestamp)) - sort.Sort(sort.Reverse(byValueSorter)) - return Vector(byValueSorter) -} - -// === topk(k ScalarNode, node VectorNode) Vector === -func topkImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { - k := int(args[0].(ScalarNode).Eval(timestamp)) - if k < 1 { - return Vector{} - } - - topk := make(vectorByValueHeap, 0, k) - vector := args[1].(VectorNode).Eval(timestamp) - - 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 ScalarNode, node VectorNode) Vector === -func bottomkImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { - k := int(args[0].(ScalarNode).Eval(timestamp)) - if k < 1 { - return Vector{} - } - - bottomk := make(vectorByValueHeap, 0, k) - bkHeap := reverseHeap{Interface: &bottomk} - vector := args[1].(VectorNode).Eval(timestamp) - - 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 VectorNode) Vector === -func dropCommonLabelsImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { - vector := args[0].(VectorNode).Eval(timestamp) - 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 VectorNode, toNearest=1 Scalar) Vector === -func roundImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { - // round returns a number rounded to toNearest. - // Ties are solved by rounding up. - toNearest := float64(1) - if len(args) >= 2 { - toNearest = float64(args[1].(ScalarNode).Eval(timestamp)) - } - // Invert as it seems to cause fewer floating point accuracy issues. - toNearestInverse := 1.0 / toNearest - - n := args[0].(VectorNode) - vector := n.Eval(timestamp) - 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 VectorNode) Scalar === -func scalarImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { - v := args[0].(VectorNode).Eval(timestamp) - if len(v) != 1 { - return clientmodel.SampleValue(math.NaN()) - } - return clientmodel.SampleValue(v[0].Value) -} - -// === count_scalar(vector VectorNode) model.SampleValue === -func countScalarImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { - return clientmodel.SampleValue(len(args[0].(VectorNode).Eval(timestamp))) -} - -func aggrOverTime(timestamp clientmodel.Timestamp, args []Node, aggrFn func(metric.Values) clientmodel.SampleValue) interface{} { - n := args[0].(MatrixNode) - matrixVal := n.Eval(timestamp) - resultVector := Vector{} - - for _, el := range matrixVal { - if len(el.Values) == 0 { - continue - } - - el.Metric.Delete(clientmodel.MetricNameLabel) - resultVector = append(resultVector, &Sample{ - Metric: el.Metric, - Value: aggrFn(el.Values), - Timestamp: timestamp, - }) - } - return resultVector -} - -// === avg_over_time(matrix MatrixNode) Vector === -func avgOverTimeImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { - return aggrOverTime(timestamp, 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 MatrixNode) Vector === -func countOverTimeImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { - return aggrOverTime(timestamp, args, func(values metric.Values) clientmodel.SampleValue { - return clientmodel.SampleValue(len(values)) - }) -} - -// === floor(vector VectorNode) Vector === -func floorImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { - n := args[0].(VectorNode) - vector := n.Eval(timestamp) - for _, el := range vector { - el.Metric.Delete(clientmodel.MetricNameLabel) - el.Value = clientmodel.SampleValue(math.Floor(float64(el.Value))) - } - return vector -} - -// === max_over_time(matrix MatrixNode) Vector === -func maxOverTimeImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { - return aggrOverTime(timestamp, 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 MatrixNode) Vector === -func minOverTimeImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { - return aggrOverTime(timestamp, 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 MatrixNode) Vector === -func sumOverTimeImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { - return aggrOverTime(timestamp, args, func(values metric.Values) clientmodel.SampleValue { - var sum clientmodel.SampleValue - for _, v := range values { - sum += v.Value - } - return sum - }) -} - -// === abs(vector VectorNode) Vector === -func absImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { - n := args[0].(VectorNode) - vector := n.Eval(timestamp) - for _, el := range vector { - el.Metric.Delete(clientmodel.MetricNameLabel) - el.Value = clientmodel.SampleValue(math.Abs(float64(el.Value))) - } - return vector -} - -// === absent(vector VectorNode) Vector === -func absentImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { - n := args[0].(VectorNode) - if len(n.Eval(timestamp)) > 0 { - return Vector{} - } - m := clientmodel.Metric{} - if vs, ok := n.(*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: timestamp, - }, - } -} - -// === ceil(vector VectorNode) Vector === -func ceilImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { - n := args[0].(VectorNode) - vector := n.Eval(timestamp) - for _, el := range vector { - el.Metric.Delete(clientmodel.MetricNameLabel) - el.Value = clientmodel.SampleValue(math.Ceil(float64(el.Value))) - } - return vector -} - -// === exp(vector VectorNode) Vector === -func expImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { - n := args[0].(VectorNode) - vector := n.Eval(timestamp) - for _, el := range vector { - el.Metric.Delete(clientmodel.MetricNameLabel) - el.Value = clientmodel.SampleValue(math.Exp(float64(el.Value))) - } - return vector -} - -// === ln(vector VectorNode) Vector === -func lnImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { - n := args[0].(VectorNode) - vector := n.Eval(timestamp) - for _, el := range vector { - el.Metric.Delete(clientmodel.MetricNameLabel) - el.Value = clientmodel.SampleValue(math.Log(float64(el.Value))) - } - return vector -} - -// === log2(vector VectorNode) Vector === -func log2Impl(timestamp clientmodel.Timestamp, args []Node) interface{} { - n := args[0].(VectorNode) - vector := n.Eval(timestamp) - for _, el := range vector { - el.Metric.Delete(clientmodel.MetricNameLabel) - el.Value = clientmodel.SampleValue(math.Log2(float64(el.Value))) - } - return vector -} - -// === log10(vector VectorNode) Vector === -func log10Impl(timestamp clientmodel.Timestamp, args []Node) interface{} { - n := args[0].(VectorNode) - vector := n.Eval(timestamp) - for _, el := range vector { - el.Metric.Delete(clientmodel.MetricNameLabel) - el.Value = clientmodel.SampleValue(math.Log10(float64(el.Value))) - } - return vector -} - -// === deriv(node MatrixNode) Vector === -func derivImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { - matrixNode := args[0].(MatrixNode) - resultVector := Vector{} - - matrixValue := matrixNode.Eval(timestamp) - for _, samples := range matrixValue { - // 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: timestamp, - } - resultSample.Metric.Delete(clientmodel.MetricNameLabel) - resultVector = append(resultVector, resultSample) - } - return resultVector -} - -// === histogram_quantile(k ScalarNode, vector VectorNode) Vector === -func histogramQuantileImpl(timestamp clientmodel.Timestamp, args []Node) interface{} { - q := args[0].(ScalarNode).Eval(timestamp) - inVec := args[1].(VectorNode).Eval(timestamp) - 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: timestamp, - }) - } - - return outVec -} - -var functions = map[string]*Function{ - "abs": { - name: "abs", - argTypes: []ExprType{VectorType}, - returnType: VectorType, - callFn: absImpl, - }, - "absent": { - name: "absent", - argTypes: []ExprType{VectorType}, - returnType: VectorType, - callFn: absentImpl, - }, - "avg_over_time": { - name: "avg_over_time", - argTypes: []ExprType{MatrixType}, - returnType: VectorType, - callFn: avgOverTimeImpl, - }, - "bottomk": { - name: "bottomk", - argTypes: []ExprType{ScalarType, VectorType}, - returnType: VectorType, - callFn: bottomkImpl, - }, - "ceil": { - name: "ceil", - argTypes: []ExprType{VectorType}, - returnType: VectorType, - callFn: ceilImpl, - }, - "count_over_time": { - name: "count_over_time", - argTypes: []ExprType{MatrixType}, - returnType: VectorType, - callFn: countOverTimeImpl, - }, - "count_scalar": { - name: "count_scalar", - argTypes: []ExprType{VectorType}, - returnType: ScalarType, - callFn: countScalarImpl, - }, - "delta": { - name: "delta", - argTypes: []ExprType{MatrixType, ScalarType}, - optionalArgs: 1, // The 2nd argument is deprecated. - returnType: VectorType, - callFn: deltaImpl, - }, - "deriv": { - name: "deriv", - argTypes: []ExprType{MatrixType}, - returnType: VectorType, - callFn: derivImpl, - }, - "drop_common_labels": { - name: "drop_common_labels", - argTypes: []ExprType{VectorType}, - returnType: VectorType, - callFn: dropCommonLabelsImpl, - }, - "exp": { - name: "exp", - argTypes: []ExprType{VectorType}, - returnType: VectorType, - callFn: expImpl, - }, - "floor": { - name: "floor", - argTypes: []ExprType{VectorType}, - returnType: VectorType, - callFn: floorImpl, - }, - "histogram_quantile": { - name: "histogram_quantile", - argTypes: []ExprType{ScalarType, VectorType}, - returnType: VectorType, - callFn: histogramQuantileImpl, - }, - "ln": { - name: "ln", - argTypes: []ExprType{VectorType}, - returnType: VectorType, - callFn: lnImpl, - }, - "log10": { - name: "log10", - argTypes: []ExprType{VectorType}, - returnType: VectorType, - callFn: log10Impl, - }, - "log2": { - name: "log2", - argTypes: []ExprType{VectorType}, - returnType: VectorType, - callFn: log2Impl, - }, - "max_over_time": { - name: "max_over_time", - argTypes: []ExprType{MatrixType}, - returnType: VectorType, - callFn: maxOverTimeImpl, - }, - "min_over_time": { - name: "min_over_time", - argTypes: []ExprType{MatrixType}, - returnType: VectorType, - callFn: minOverTimeImpl, - }, - "rate": { - name: "rate", - argTypes: []ExprType{MatrixType}, - returnType: VectorType, - callFn: rateImpl, - }, - "round": { - name: "round", - argTypes: []ExprType{VectorType, ScalarType}, - optionalArgs: 1, - returnType: VectorType, - callFn: roundImpl, - }, - "scalar": { - name: "scalar", - argTypes: []ExprType{VectorType}, - returnType: ScalarType, - callFn: scalarImpl, - }, - "sort": { - name: "sort", - argTypes: []ExprType{VectorType}, - returnType: VectorType, - callFn: sortImpl, - }, - "sort_desc": { - name: "sort_desc", - argTypes: []ExprType{VectorType}, - returnType: VectorType, - callFn: sortDescImpl, - }, - "sum_over_time": { - name: "sum_over_time", - argTypes: []ExprType{MatrixType}, - returnType: VectorType, - callFn: sumOverTimeImpl, - }, - "time": { - name: "time", - argTypes: []ExprType{}, - returnType: ScalarType, - callFn: timeImpl, - }, - "topk": { - name: "topk", - argTypes: []ExprType{ScalarType, VectorType}, - returnType: VectorType, - callFn: topkImpl, - }, -} - -// GetFunction returns a predefined Function object for the given -// name. -func GetFunction(name string) (*Function, error) { - function, ok := functions[name] - if !ok { - return nil, fmt.Errorf("couldn't find function %v()", name) - } - return function, nil -} diff --git a/rules/ast/functions_test.go b/rules/ast/functions_test.go deleted file mode 100644 index ca3f44a57..000000000 --- a/rules/ast/functions_test.go +++ /dev/null @@ -1,63 +0,0 @@ -// 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 ast - -import ( - "testing" - - clientmodel "github.com/prometheus/client_golang/model" - - "github.com/prometheus/prometheus/storage/metric" -) - -type emptyRangeNode struct{} - -func (node emptyRangeNode) Type() ExprType { return MatrixType } -func (node emptyRangeNode) NodeTreeToDotGraph() string { return "" } -func (node emptyRangeNode) String() string { return "" } -func (node emptyRangeNode) Children() Nodes { return Nodes{} } - -func (node emptyRangeNode) Eval(timestamp clientmodel.Timestamp) Matrix { - return Matrix{ - SampleStream{ - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{clientmodel.MetricNameLabel: "empty_metric"}, - }, - Values: metric.Values{}, - }, - } -} - -func (node emptyRangeNode) EvalBoundaries(timestamp clientmodel.Timestamp) Matrix { - return Matrix{ - SampleStream{ - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{clientmodel.MetricNameLabel: "empty_metric"}, - }, - Values: metric.Values{}, - }, - } -} - -func TestDeltaWithEmptyElementDoesNotCrash(t *testing.T) { - now := clientmodel.Now() - vector := deltaImpl(now, []Node{emptyRangeNode{}, &ScalarLiteral{value: 0}}).(Vector) - if len(vector) != 0 { - t.Fatalf("Expected empty result vector, got: %v", vector) - } - vector = deltaImpl(now, []Node{emptyRangeNode{}, &ScalarLiteral{value: 1}}).(Vector) - if len(vector) != 0 { - t.Fatalf("Expected empty result vector, got: %v", vector) - } -} diff --git a/rules/ast/printer.go b/rules/ast/printer.go deleted file mode 100644 index 7c9ebcade..000000000 --- a/rules/ast/printer.go +++ /dev/null @@ -1,441 +0,0 @@ -// 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 ast - -import ( - "encoding/json" - "errors" - "fmt" - "reflect" - "sort" - "strings" - - clientmodel "github.com/prometheus/client_golang/model" - - "github.com/prometheus/prometheus/stats" - "github.com/prometheus/prometheus/storage/local" - "github.com/prometheus/prometheus/utility" -) - -// OutputFormat is an enum for the possible output formats. -type OutputFormat int - -// Possible output formats. -const ( - Text OutputFormat = iota - JSON -) - -const jsonFormatVersion = 1 - -func (opType BinOpType) String() string { - opTypeMap := map[BinOpType]string{ - Add: "+", - Sub: "-", - Mul: "*", - Div: "/", - Mod: "%", - GT: ">", - LT: "<", - EQ: "==", - NE: "!=", - GE: ">=", - LE: "<=", - And: "AND", - Or: "OR", - } - return opTypeMap[opType] -} - -func (aggrType AggrType) String() string { - aggrTypeMap := map[AggrType]string{ - Sum: "SUM", - Avg: "AVG", - Min: "MIN", - Max: "MAX", - Count: "COUNT", - } - return aggrTypeMap[aggrType] -} - -func (exprType ExprType) String() string { - exprTypeMap := map[ExprType]string{ - ScalarType: "scalar", - VectorType: "vector", - MatrixType: "matrix", - StringType: "string", - } - return exprTypeMap[exprType] -} - -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") -} - -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") -} - -// ErrorToJSON converts the given error into JSON. -func ErrorToJSON(err error) string { - errorStruct := struct { - Type string `json:"type"` - Value string `json:"value"` - Version int `json:"version"` - }{ - Type: "error", - Value: err.Error(), - Version: jsonFormatVersion, - } - - errorJSON, err := json.Marshal(errorStruct) - if err != nil { - return "" - } - return string(errorJSON) -} - -// TypedValueToJSON converts the given data of type 'scalar', -// 'vector', or 'matrix' into its JSON representation. -func TypedValueToJSON(data interface{}, typeStr string) string { - dataStruct := struct { - Type string `json:"type"` - Value interface{} `json:"value"` - Version int `json:"version"` - }{ - Type: typeStr, - Value: data, - Version: jsonFormatVersion, - } - dataJSON, err := json.Marshal(dataStruct) - if err != nil { - return ErrorToJSON(err) - } - return string(dataJSON) -} - -// EvalToString evaluates the given node into a string of the given format. -func EvalToString(node Node, timestamp clientmodel.Timestamp, format OutputFormat, storage local.Storage, queryStats *stats.TimerGroup) string { - totalEvalTimer := queryStats.GetTimer(stats.TotalEvalTime).Start() - defer totalEvalTimer.Stop() - - prepareTimer := queryStats.GetTimer(stats.TotalQueryPreparationTime).Start() - closer, err := prepareInstantQuery(node, timestamp, storage, queryStats) - prepareTimer.Stop() - if err != nil { - panic(err) - } - defer closer.Close() - - evalTimer := queryStats.GetTimer(stats.InnerEvalTime).Start() - switch node.Type() { - case ScalarType: - scalar := node.(ScalarNode).Eval(timestamp) - evalTimer.Stop() - switch format { - case Text: - return fmt.Sprintf("scalar: %v @[%v]", scalar, timestamp) - case JSON: - return TypedValueToJSON(scalar, "scalar") - } - case VectorType: - vector := node.(VectorNode).Eval(timestamp) - evalTimer.Stop() - switch format { - case Text: - return vector.String() - case JSON: - return TypedValueToJSON(vector, "vector") - } - case MatrixType: - matrix := node.(MatrixNode).Eval(timestamp) - evalTimer.Stop() - switch format { - case Text: - return matrix.String() - case JSON: - return TypedValueToJSON(matrix, "matrix") - } - case StringType: - str := node.(StringNode).Eval(timestamp) - evalTimer.Stop() - switch format { - case Text: - return str - case JSON: - return TypedValueToJSON(str, "string") - } - } - panic("Switch didn't cover all node types") -} - -// EvalToVector evaluates the given node into a Vector. Matrices aren't supported. -func EvalToVector(node Node, timestamp clientmodel.Timestamp, storage local.Storage, queryStats *stats.TimerGroup) (Vector, error) { - totalEvalTimer := queryStats.GetTimer(stats.TotalEvalTime).Start() - defer totalEvalTimer.Stop() - - prepareTimer := queryStats.GetTimer(stats.TotalQueryPreparationTime).Start() - closer, err := prepareInstantQuery(node, timestamp, storage, queryStats) - prepareTimer.Stop() - if err != nil { - panic(err) - } - defer closer.Close() - - evalTimer := queryStats.GetTimer(stats.InnerEvalTime).Start() - switch node.Type() { - case ScalarType: - scalar := node.(ScalarNode).Eval(timestamp) - evalTimer.Stop() - return Vector{&Sample{Value: scalar}}, nil - case VectorType: - vector := node.(VectorNode).Eval(timestamp) - evalTimer.Stop() - return vector, nil - case MatrixType: - return nil, errors.New("matrices not supported by EvalToVector") - case StringType: - str := node.(StringNode).Eval(timestamp) - evalTimer.Stop() - return Vector{ - &Sample{ - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - "__value__": clientmodel.LabelValue(str), - }, - Copied: true, - }, - }, - }, nil - } - panic("Switch didn't cover all node types") -} - -// NodeTreeToDotGraph returns a DOT representation of the scalar -// literal. -func (node *ScalarLiteral) NodeTreeToDotGraph() string { - return fmt.Sprintf("%#p[label=\"%v\"];\n", node, node.value) -} - -func functionArgsToDotGraph(node Node, args []Node) string { - graph := "" - for _, arg := range args { - graph += fmt.Sprintf("%x -> %x;\n", reflect.ValueOf(node).Pointer(), reflect.ValueOf(arg).Pointer()) - } - for _, arg := range args { - graph += arg.NodeTreeToDotGraph() - } - return graph -} - -// NodeTreeToDotGraph returns a DOT representation of the function -// call. -func (node *ScalarFunctionCall) NodeTreeToDotGraph() string { - graph := fmt.Sprintf("%#p[label=\"%s\"];\n", node, node.function.name) - graph += functionArgsToDotGraph(node, node.args) - return graph -} - -// NodeTreeToDotGraph returns a DOT representation of the expression. -func (node *ScalarArithExpr) NodeTreeToDotGraph() string { - nodeAddr := reflect.ValueOf(node).Pointer() - graph := fmt.Sprintf( - ` - %x[label="%s"]; - %x -> %x; - %x -> %x; - %s - %s - }`, - nodeAddr, node.opType, - nodeAddr, reflect.ValueOf(node.lhs).Pointer(), - nodeAddr, reflect.ValueOf(node.rhs).Pointer(), - node.lhs.NodeTreeToDotGraph(), - node.rhs.NodeTreeToDotGraph(), - ) - return graph -} - -// NodeTreeToDotGraph returns a DOT representation of the vector selector. -func (node *VectorSelector) NodeTreeToDotGraph() string { - return fmt.Sprintf("%#p[label=\"%s\"];\n", node, node) -} - -// NodeTreeToDotGraph returns a DOT representation of the function -// call. -func (node *VectorFunctionCall) NodeTreeToDotGraph() string { - graph := fmt.Sprintf("%#p[label=\"%s\"];\n", node, node.function.name) - graph += functionArgsToDotGraph(node, node.args) - return graph -} - -// NodeTreeToDotGraph returns a DOT representation of the vector -// aggregation. -func (node *VectorAggregation) NodeTreeToDotGraph() string { - groupByStrings := make([]string, 0, len(node.groupBy)) - for _, label := range node.groupBy { - groupByStrings = append(groupByStrings, string(label)) - } - - graph := fmt.Sprintf("%#p[label=\"%s BY (%s)\"]\n", - node, - node.aggrType, - strings.Join(groupByStrings, ", ")) - graph += fmt.Sprintf("%#p -> %x;\n", node, reflect.ValueOf(node.vector).Pointer()) - graph += node.vector.NodeTreeToDotGraph() - return graph -} - -// NodeTreeToDotGraph returns a DOT representation of the expression. -func (node *VectorArithExpr) NodeTreeToDotGraph() string { - nodeAddr := reflect.ValueOf(node).Pointer() - graph := fmt.Sprintf( - ` - %x[label="%s"]; - %x -> %x; - %x -> %x; - %s - %s - }`, - nodeAddr, node.opType, - nodeAddr, reflect.ValueOf(node.lhs).Pointer(), - nodeAddr, reflect.ValueOf(node.rhs).Pointer(), - node.lhs.NodeTreeToDotGraph(), - node.rhs.NodeTreeToDotGraph(), - ) - return graph -} - -// NodeTreeToDotGraph returns a DOT representation of the matrix -// selector. -func (node *MatrixSelector) NodeTreeToDotGraph() string { - return fmt.Sprintf("%#p[label=\"%s\"];\n", node, node) -} - -// NodeTreeToDotGraph returns a DOT representation of the string -// literal. -func (node *StringLiteral) NodeTreeToDotGraph() string { - return fmt.Sprintf("%#p[label=\"'%q'\"];\n", node, node.str) -} - -// NodeTreeToDotGraph returns a DOT representation of the function -// call. -func (node *StringFunctionCall) NodeTreeToDotGraph() string { - graph := fmt.Sprintf("%#p[label=\"%s\"];\n", node, node.function.name) - graph += functionArgsToDotGraph(node, node.args) - return graph -} - -func (nodes Nodes) String() string { - nodeStrings := make([]string, 0, len(nodes)) - for _, node := range nodes { - nodeStrings = append(nodeStrings, node.String()) - } - return strings.Join(nodeStrings, ", ") -} - -func (node *ScalarLiteral) String() string { - return fmt.Sprint(node.value) -} - -func (node *ScalarFunctionCall) String() string { - return fmt.Sprintf("%s(%s)", node.function.name, node.args) -} - -func (node *ScalarArithExpr) String() string { - return fmt.Sprintf("(%s %s %s)", node.lhs, node.opType, node.rhs) -} - -func (node *VectorSelector) String() string { - labelStrings := make([]string, 0, len(node.labelMatchers)-1) - var metricName clientmodel.LabelValue - for _, matcher := range node.labelMatchers { - if matcher.Name != clientmodel.MetricNameLabel { - labelStrings = append(labelStrings, fmt.Sprintf("%s%s%q", matcher.Name, matcher.Type, matcher.Value)) - } else { - metricName = matcher.Value - } - } - - switch len(labelStrings) { - case 0: - return string(metricName) - default: - sort.Strings(labelStrings) - return fmt.Sprintf("%s{%s}", metricName, strings.Join(labelStrings, ",")) - } -} - -func (node *VectorFunctionCall) String() string { - return fmt.Sprintf("%s(%s)", node.function.name, node.args) -} - -func (node *VectorAggregation) String() string { - aggrString := fmt.Sprintf("%s(%s)", node.aggrType, node.vector) - if len(node.groupBy) > 0 { - return fmt.Sprintf("%s BY (%s)", aggrString, node.groupBy) - } - return aggrString -} - -func (node *VectorArithExpr) String() string { - return fmt.Sprintf("(%s %s %s)", node.lhs, node.opType, node.rhs) -} - -func (node *MatrixSelector) String() string { - vectorString := (&VectorSelector{labelMatchers: node.labelMatchers}).String() - intervalString := fmt.Sprintf("[%s]", utility.DurationToString(node.interval)) - return vectorString + intervalString -} - -func (node *StringLiteral) String() string { - return fmt.Sprintf("%q", node.str) -} - -func (node *StringFunctionCall) String() string { - return fmt.Sprintf("%s(%s)", node.function.name, node.args) -} diff --git a/rules/ast/quantile.go b/rules/ast/quantile.go deleted file mode 100644 index 38ddc7976..000000000 --- a/rules/ast/quantile.go +++ /dev/null @@ -1,106 +0,0 @@ -// 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 ast - -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) -} diff --git a/rules/ast/query_analyzer.go b/rules/ast/query_analyzer.go deleted file mode 100644 index 9807f7527..000000000 --- a/rules/ast/query_analyzer.go +++ /dev/null @@ -1,229 +0,0 @@ -// 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 ast - -import ( - "time" - - clientmodel "github.com/prometheus/client_golang/model" - - "github.com/prometheus/prometheus/stats" - "github.com/prometheus/prometheus/storage/local" -) - -// 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 -} - -// A queryAnalyzer recursively traverses the AST to look for any nodes -// which will need data from the datastore. Instantiate with -// newQueryAnalyzer. -type queryAnalyzer struct { - // Tracks one set of times to preload per offset that occurs in the query - // expression. - offsetPreloadTimes map[time.Duration]preloadTimes - // The underlying storage to which the query will be applied. Needed for - // extracting timeseries fingerprint information during query analysis. - storage local.Storage -} - -// newQueryAnalyzer returns a pointer to a newly instantiated -// queryAnalyzer. The storage is needed to extract timeseries -// fingerprint information during query analysis. -func newQueryAnalyzer(storage local.Storage) *queryAnalyzer { - return &queryAnalyzer{ - offsetPreloadTimes: map[time.Duration]preloadTimes{}, - storage: storage, - } -} - -func (analyzer *queryAnalyzer) getPreloadTimes(offset time.Duration) preloadTimes { - if _, ok := analyzer.offsetPreloadTimes[offset]; !ok { - analyzer.offsetPreloadTimes[offset] = preloadTimes{ - instants: map[clientmodel.Fingerprint]struct{}{}, - ranges: map[clientmodel.Fingerprint]time.Duration{}, - } - } - return analyzer.offsetPreloadTimes[offset] -} - -// visit implements the visitor interface. -func (analyzer *queryAnalyzer) visit(node Node) { - switch n := node.(type) { - case *VectorSelector: - pt := analyzer.getPreloadTimes(n.offset) - fingerprints := analyzer.storage.GetFingerprintsForLabelMatchers(n.labelMatchers) - n.fingerprints = fingerprints - for _, fp := range fingerprints { - // 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{}{} - } - - n.metrics[fp] = analyzer.storage.GetMetricForFingerprint(fp) - } - case *MatrixSelector: - pt := analyzer.getPreloadTimes(n.offset) - fingerprints := analyzer.storage.GetFingerprintsForLabelMatchers(n.labelMatchers) - n.fingerprints = fingerprints - for _, fp := range fingerprints { - if pt.ranges[fp] < n.interval { - pt.ranges[fp] = n.interval - // 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) - } - - n.metrics[fp] = analyzer.storage.GetMetricForFingerprint(fp) - } - } -} - -type iteratorInitializer struct { - storage local.Storage -} - -func (i *iteratorInitializer) visit(node Node) { - switch n := node.(type) { - case *VectorSelector: - for _, fp := range n.fingerprints { - n.iterators[fp] = i.storage.NewIterator(fp) - } - case *MatrixSelector: - for _, fp := range n.fingerprints { - n.iterators[fp] = i.storage.NewIterator(fp) - } - } -} - -func prepareInstantQuery(node Node, timestamp clientmodel.Timestamp, storage local.Storage, queryStats *stats.TimerGroup) (local.Preloader, error) { - totalTimer := queryStats.GetTimer(stats.TotalEvalTime) - - analyzeTimer := queryStats.GetTimer(stats.QueryAnalysisTime).Start() - analyzer := newQueryAnalyzer(storage) - Walk(analyzer, node) - analyzeTimer.Stop() - - preloadTimer := queryStats.GetTimer(stats.PreloadTime).Start() - p := storage.NewPreloader() - for offset, pt := range analyzer.offsetPreloadTimes { - ts := timestamp.Add(-offset) - for fp, rangeDuration := range pt.ranges { - if et := totalTimer.ElapsedTime(); et > *queryTimeout { - preloadTimer.Stop() - p.Close() - return nil, queryTimeoutError{et} - } - if err := p.PreloadRange(fp, ts.Add(-rangeDuration), ts, *stalenessDelta); err != nil { - preloadTimer.Stop() - p.Close() - return nil, err - } - } - for fp := range pt.instants { - if et := totalTimer.ElapsedTime(); et > *queryTimeout { - preloadTimer.Stop() - p.Close() - return nil, queryTimeoutError{et} - } - if err := p.PreloadRange(fp, ts, ts, *stalenessDelta); err != nil { - preloadTimer.Stop() - p.Close() - return nil, err - } - } - } - preloadTimer.Stop() - - ii := &iteratorInitializer{ - storage: storage, - } - Walk(ii, node) - - return p, nil -} - -func prepareRangeQuery(node Node, start clientmodel.Timestamp, end clientmodel.Timestamp, interval time.Duration, storage local.Storage, queryStats *stats.TimerGroup) (local.Preloader, error) { - totalTimer := queryStats.GetTimer(stats.TotalEvalTime) - - analyzeTimer := queryStats.GetTimer(stats.QueryAnalysisTime).Start() - analyzer := newQueryAnalyzer(storage) - Walk(analyzer, node) - analyzeTimer.Stop() - - preloadTimer := queryStats.GetTimer(stats.PreloadTime).Start() - p := storage.NewPreloader() - for offset, pt := range analyzer.offsetPreloadTimes { - offsetStart := start.Add(-offset) - offsetEnd := end.Add(-offset) - for fp, rangeDuration := range pt.ranges { - if et := totalTimer.ElapsedTime(); et > *queryTimeout { - preloadTimer.Stop() - p.Close() - return nil, queryTimeoutError{et} - } - if err := p.PreloadRange(fp, offsetStart.Add(-rangeDuration), offsetEnd, *stalenessDelta); err != nil { - preloadTimer.Stop() - p.Close() - return nil, err - } - /* - if interval < rangeDuration { - if err := p.GetMetricRange(fp, offsetEnd, offsetEnd.Sub(offsetStart)+rangeDuration); err != nil { - p.Close() - return nil, err - } - } else { - if err := p.GetMetricRangeAtInterval(fp, offsetStart, offsetEnd, interval, rangeDuration); err != nil { - p.Close() - return nil, err - } - } - */ - } - for fp := range pt.instants { - if et := totalTimer.ElapsedTime(); et > *queryTimeout { - preloadTimer.Stop() - p.Close() - return nil, queryTimeoutError{et} - } - if err := p.PreloadRange(fp, offsetStart, offsetEnd, *stalenessDelta); err != nil { - preloadTimer.Stop() - p.Close() - return nil, err - } - } - } - preloadTimer.Stop() - - ii := &iteratorInitializer{ - storage: storage, - } - Walk(ii, node) - - return p, nil -} diff --git a/rules/ast/walk.go b/rules/ast/walk.go deleted file mode 100644 index 143120e8f..000000000 --- a/rules/ast/walk.go +++ /dev/null @@ -1,28 +0,0 @@ -// 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 ast - -// visitor is the interface for a Node visitor. -type visitor interface { - visit(node Node) -} - -// Walk does a depth-first traversal of the AST, starting at node, -// calling visitor.visit for each encountered Node in the tree. -func Walk(v visitor, node Node) { - v.visit(node) - for _, childNode := range node.Children() { - Walk(v, childNode) - } -} diff --git a/rules/fixtures/empty.rules b/rules/fixtures/empty.rules deleted file mode 100644 index e69de29bb..000000000 diff --git a/rules/fixtures/mixed.rules b/rules/fixtures/mixed.rules deleted file mode 100644 index 7b9520607..000000000 --- a/rules/fixtures/mixed.rules +++ /dev/null @@ -1,16 +0,0 @@ -// A simple test recording rule. -dc_http_request_rate5m = sum(rate(http_request_count[5m])) by (dc) - -// A simple test alerting rule. -ALERT GlobalRequestRateLow IF(dc_http_request_rate5m < 10000) FOR 5m WITH { - service = "testservice" - /* ... more fields here ... */ - } - SUMMARY "Global request rate low" - DESCRIPTION "The global request rate is low" - -foo = bar{label1="value1"} - -ALERT BazAlert IF(foo > 10) WITH {} - SUMMARY "Baz" - DESCRIPTION "BazAlert" diff --git a/rules/fixtures/non_vector.rules b/rules/fixtures/non_vector.rules deleted file mode 100644 index f3b1046f2..000000000 --- a/rules/fixtures/non_vector.rules +++ /dev/null @@ -1 +0,0 @@ -now = time() diff --git a/rules/fixtures/syntax_error.rules b/rules/fixtures/syntax_error.rules deleted file mode 100644 index d44c8c9a8..000000000 --- a/rules/fixtures/syntax_error.rules +++ /dev/null @@ -1,15 +0,0 @@ -// A simple test recording rule. -dc_http_request_rate5m = sum(rate(http_request_count[5m])) by (dc) - -// A simple test alerting rule with a syntax error (invalid duration string "5"). -ALERT GlobalRequestRateLow IF(dc_http_request_rate5m < 10000) FOR 5 WITH { - description = "Global HTTP request rate low!", - summary = "Request rate low" - /* ... more fields here ... */ - } - SUMMARY "summary" - DESCRIPTION "description" - -foo = bar{label1="value1"} - -ALERT BazAlert IF(foo > 10) WITH {} SUMMARY "summary" DESCRIPTION "description" diff --git a/rules/helpers.go b/rules/helpers.go deleted file mode 100644 index 9f4162215..000000000 --- a/rules/helpers.go +++ /dev/null @@ -1,221 +0,0 @@ -// 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 rules - -import ( - "fmt" - "net/url" - "strings" - - clientmodel "github.com/prometheus/client_golang/model" - - "github.com/prometheus/prometheus/rules/ast" - "github.com/prometheus/prometheus/storage/metric" - "github.com/prometheus/prometheus/utility" -) - -// CreateRecordingRule is a convenience function to create a recording rule. -func CreateRecordingRule(name string, labels clientmodel.LabelSet, expr ast.Node, permanent bool) (*RecordingRule, error) { - if _, ok := expr.(ast.VectorNode); !ok { - return nil, fmt.Errorf("recording rule expression %v does not evaluate to vector type", expr) - } - return &RecordingRule{ - name: name, - labels: labels, - vector: expr.(ast.VectorNode), - permanent: permanent, - }, nil -} - -// CreateAlertingRule is a convenience function to create a new alerting rule. -func CreateAlertingRule(name string, expr ast.Node, holdDurationStr string, labels clientmodel.LabelSet, summary string, description string) (*AlertingRule, error) { - if _, ok := expr.(ast.VectorNode); !ok { - return nil, fmt.Errorf("alert rule expression %v does not evaluate to vector type", expr) - } - holdDuration, err := utility.StringToDuration(holdDurationStr) - if err != nil { - return nil, err - } - return NewAlertingRule(name, expr.(ast.VectorNode), holdDuration, labels, summary, description), nil -} - -// NewScalarLiteral returns a ScalarLiteral with the given value. If sign is "-" -// the value is negated. -func NewScalarLiteral(value clientmodel.SampleValue, sign string) *ast.ScalarLiteral { - if sign == "-" { - value = -value - } - return ast.NewScalarLiteral(value) -} - -// NewFunctionCall is a convenience function to create a new AST function-call node. -func NewFunctionCall(name string, args []ast.Node) (ast.Node, error) { - function, err := ast.GetFunction(name) - if err != nil { - return nil, fmt.Errorf("unknown function %q", name) - } - functionCall, err := ast.NewFunctionCall(function, args) - if err != nil { - return nil, fmt.Errorf(err.Error()) - } - return functionCall, nil -} - -// NewVectorAggregation is a convenience function to create a new AST vector aggregation. -func NewVectorAggregation(aggrTypeStr string, vector ast.Node, groupBy clientmodel.LabelNames, keepExtraLabels bool) (*ast.VectorAggregation, error) { - if _, ok := vector.(ast.VectorNode); !ok { - return nil, fmt.Errorf("operand of %v aggregation must be of vector type", aggrTypeStr) - } - var aggrTypes = map[string]ast.AggrType{ - "SUM": ast.Sum, - "MAX": ast.Max, - "MIN": ast.Min, - "AVG": ast.Avg, - "COUNT": ast.Count, - } - aggrType, ok := aggrTypes[aggrTypeStr] - if !ok { - return nil, fmt.Errorf("unknown aggregation type %q", aggrTypeStr) - } - return ast.NewVectorAggregation(aggrType, vector.(ast.VectorNode), groupBy, keepExtraLabels), nil -} - -// vectorMatching combines data used to match samples between vectors. -type vectorMatching struct { - matchCardinality ast.VectorMatchCardinality - matchOn clientmodel.LabelNames - includeLabels clientmodel.LabelNames -} - -// newVectorMatching is a convenience function to create a new vectorMatching. -func newVectorMatching(card string, matchOn, include clientmodel.LabelNames) (*vectorMatching, error) { - var matchCardinalities = map[string]ast.VectorMatchCardinality{ - "": ast.MatchOneToOne, - "GROUP_LEFT": ast.MatchManyToOne, - "GROUP_RIGHT": ast.MatchOneToMany, - } - matchCard, ok := matchCardinalities[card] - if !ok { - return nil, fmt.Errorf("invalid vector match cardinality %q", card) - } - if matchCard != ast.MatchOneToOne && len(include) == 0 { - return nil, fmt.Errorf("grouped vector matching must provide labels") - } - // There must be no overlap between both labelname lists. - for _, matchLabel := range matchOn { - for _, incLabel := range include { - if matchLabel == incLabel { - return nil, fmt.Errorf("use of label %s in ON and %s clauses not allowed", incLabel, card) - } - } - } - return &vectorMatching{matchCard, matchOn, include}, nil -} - -// NewArithExpr is a convenience function to create a new AST arithmetic expression. -func NewArithExpr(opTypeStr string, lhs ast.Node, rhs ast.Node, vecMatching *vectorMatching) (ast.Node, error) { - var opTypes = map[string]ast.BinOpType{ - "+": ast.Add, - "-": ast.Sub, - "*": ast.Mul, - "/": ast.Div, - "%": ast.Mod, - ">": ast.GT, - "<": ast.LT, - "==": ast.EQ, - "!=": ast.NE, - ">=": ast.GE, - "<=": ast.LE, - "AND": ast.And, - "OR": ast.Or, - } - opType, ok := opTypes[opTypeStr] - if !ok { - return nil, fmt.Errorf("invalid binary operator %q", opTypeStr) - } - var vm vectorMatching - if vecMatching != nil { - vm = *vecMatching - // And/or always do many-to-many matching. - if opType == ast.And || opType == ast.Or { - vm.matchCardinality = ast.MatchManyToMany - } - } - expr, err := ast.NewArithExpr(opType, lhs, rhs, vm.matchCardinality, vm.matchOn, vm.includeLabels) - if err != nil { - return nil, fmt.Errorf(err.Error()) - } - return expr, nil -} - -// NewVectorSelector is a convenience function to create a new AST vector selector. -func NewVectorSelector(m metric.LabelMatchers, offsetStr string) (ast.VectorNode, error) { - offset, err := utility.StringToDuration(offsetStr) - if err != nil { - return nil, err - } - return ast.NewVectorSelector(m, offset), nil -} - -// NewMatrixSelector is a convenience function to create a new AST matrix selector. -func NewMatrixSelector(vector ast.Node, intervalStr string, offsetStr string) (ast.MatrixNode, error) { - interval, err := utility.StringToDuration(intervalStr) - if err != nil { - return nil, err - } - offset, err := utility.StringToDuration(offsetStr) - if err != nil { - return nil, err - } - vectorSelector, ok := vector.(*ast.VectorSelector) - if !ok { - return nil, fmt.Errorf("intervals are currently only supported for vector selectors") - } - return ast.NewMatrixSelector(vectorSelector, interval, offset), nil -} - -func newLabelMatcher(matchTypeStr string, name clientmodel.LabelName, value clientmodel.LabelValue) (*metric.LabelMatcher, error) { - matchTypes := map[string]metric.MatchType{ - "=": metric.Equal, - "!=": metric.NotEqual, - "=~": metric.RegexMatch, - "!~": metric.RegexNoMatch, - } - matchType, ok := matchTypes[matchTypeStr] - if !ok { - return nil, fmt.Errorf("invalid label matching operator %q", matchTypeStr) - } - return metric.NewLabelMatcher(matchType, name, value) -} - -// TableLinkForExpression creates an escaped relative link to the table view of -// the provided expression. -func TableLinkForExpression(expr string) string { - // url.QueryEscape percent-escapes everything except spaces, for which it - // uses "+". However, in the non-query part of a URI, only percent-escaped - // spaces are legal, so we need to manually replace "+" with "%20" after - // query-escaping the string. - // - // See also: - // http://stackoverflow.com/questions/1634271/url-encoding-the-space-character-or-20. - urlData := url.QueryEscape(fmt.Sprintf(`[{"expr":%q,"tab":1}]`, expr)) - return fmt.Sprintf("/graph#%s", strings.Replace(urlData, "+", "%20", -1)) -} - -// GraphLinkForExpression creates an escaped relative link to the graph view of -// the provided expression. -func GraphLinkForExpression(expr string) string { - urlData := url.QueryEscape(fmt.Sprintf(`[{"expr":%q,"tab":0}]`, expr)) - return fmt.Sprintf("/graph#%s", strings.Replace(urlData, "+", "%20", -1)) -} diff --git a/rules/helpers_test.go b/rules/helpers_test.go deleted file mode 100644 index 037fec86e..000000000 --- a/rules/helpers_test.go +++ /dev/null @@ -1,487 +0,0 @@ -// 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 rules - -import ( - "time" - - clientmodel "github.com/prometheus/client_golang/model" - - "github.com/prometheus/prometheus/rules/ast" - "github.com/prometheus/prometheus/storage/local" - "github.com/prometheus/prometheus/storage/metric" -) - -var testSampleInterval = time.Duration(5) * time.Minute -var testStartTime = clientmodel.Timestamp(0) - -func getTestValueStream(startVal clientmodel.SampleValue, endVal clientmodel.SampleValue, stepVal clientmodel.SampleValue, startTime clientmodel.Timestamp) (resultValues metric.Values) { - currentTime := startTime - for currentVal := startVal; currentVal <= endVal; currentVal += stepVal { - sample := metric.SamplePair{ - Value: currentVal, - Timestamp: currentTime, - } - resultValues = append(resultValues, sample) - currentTime = currentTime.Add(testSampleInterval) - } - return resultValues -} - -func getTestVectorFromTestMatrix(matrix ast.Matrix) ast.Vector { - vector := ast.Vector{} - for _, sampleStream := range matrix { - lastSample := sampleStream.Values[len(sampleStream.Values)-1] - vector = append(vector, &ast.Sample{ - Metric: sampleStream.Metric, - Value: lastSample.Value, - Timestamp: lastSample.Timestamp, - }) - } - return vector -} - -func storeMatrix(storage local.Storage, matrix ast.Matrix) { - pendingSamples := clientmodel.Samples{} - for _, sampleStream := range matrix { - for _, sample := range sampleStream.Values { - pendingSamples = append(pendingSamples, &clientmodel.Sample{ - Metric: sampleStream.Metric.Metric, - Value: sample.Value, - Timestamp: sample.Timestamp, - }) - } - } - for _, s := range pendingSamples { - storage.Append(s) - } - storage.WaitForIndexing() -} - -var testMatrix = ast.Matrix{ - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "http_requests", - clientmodel.JobLabel: "api-server", - "instance": "0", - "group": "production", - }, - }, - Values: getTestValueStream(0, 100, 10, testStartTime), - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "http_requests", - clientmodel.JobLabel: "api-server", - "instance": "1", - "group": "production", - }, - }, - Values: getTestValueStream(0, 200, 20, testStartTime), - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "http_requests", - clientmodel.JobLabel: "api-server", - "instance": "0", - "group": "canary", - }, - }, - Values: getTestValueStream(0, 300, 30, testStartTime), - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "http_requests", - clientmodel.JobLabel: "api-server", - "instance": "1", - "group": "canary", - }, - }, - Values: getTestValueStream(0, 400, 40, testStartTime), - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "http_requests", - clientmodel.JobLabel: "app-server", - "instance": "0", - "group": "production", - }, - }, - Values: getTestValueStream(0, 500, 50, testStartTime), - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "http_requests", - clientmodel.JobLabel: "app-server", - "instance": "1", - "group": "production", - }, - }, - Values: getTestValueStream(0, 600, 60, testStartTime), - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "http_requests", - clientmodel.JobLabel: "app-server", - "instance": "0", - "group": "canary", - }, - }, - Values: getTestValueStream(0, 700, 70, testStartTime), - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "http_requests", - clientmodel.JobLabel: "app-server", - "instance": "1", - "group": "canary", - }, - }, - Values: getTestValueStream(0, 800, 80, testStartTime), - }, - // Single-letter metric and label names. - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "x", - "y": "testvalue", - }, - }, - Values: getTestValueStream(0, 100, 10, testStartTime), - }, - // Counter reset in the middle of range. - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "testcounter_reset_middle", - }, - }, - Values: append(getTestValueStream(0, 40, 10, testStartTime), getTestValueStream(0, 50, 10, testStartTime.Add(testSampleInterval*5))...), - }, - // Counter reset at the end of range. - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "testcounter_reset_end", - }, - }, - Values: append(getTestValueStream(0, 90, 10, testStartTime), getTestValueStream(0, 0, 10, testStartTime.Add(testSampleInterval*10))...), - }, - // For label-key grouping regression test. - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "label_grouping_test", - "a": "aa", - "b": "bb", - }, - }, - Values: getTestValueStream(0, 100, 10, testStartTime), - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "label_grouping_test", - "a": "a", - "b": "abb", - }, - }, - Values: getTestValueStream(0, 200, 20, testStartTime), - }, - // Two histograms with 4 buckets each (*_sum and *_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. - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "testhistogram_bucket", - "le": "0.1", - "start": "positive", - }, - }, - Values: getTestValueStream(0, 50, 5, testStartTime), - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "testhistogram_bucket", - "le": ".2", - "start": "positive", - }, - }, - Values: getTestValueStream(0, 70, 7, testStartTime), - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "testhistogram_bucket", - "le": "1e0", - "start": "positive", - }, - }, - Values: getTestValueStream(0, 110, 11, testStartTime), - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "testhistogram_bucket", - "le": "+Inf", - "start": "positive", - }, - }, - Values: getTestValueStream(0, 120, 12, testStartTime), - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "testhistogram_bucket", - "le": "-.2", - "start": "negative", - }, - }, - Values: getTestValueStream(0, 10, 1, testStartTime), - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "testhistogram_bucket", - "le": "-0.1", - "start": "negative", - }, - }, - Values: getTestValueStream(0, 20, 2, testStartTime), - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "testhistogram_bucket", - "le": "0.3", - "start": "negative", - }, - }, - Values: getTestValueStream(0, 20, 2, testStartTime), - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "testhistogram_bucket", - "le": "+Inf", - "start": "negative", - }, - }, - Values: getTestValueStream(0, 30, 3, testStartTime), - }, - // Now a more realistic histogram per job and instance to test aggregation. - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "request_duration_seconds_bucket", - clientmodel.JobLabel: "job1", - "instance": "ins1", - "le": "0.1", - }, - }, - Values: getTestValueStream(0, 10, 1, testStartTime), - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "request_duration_seconds_bucket", - clientmodel.JobLabel: "job1", - "instance": "ins1", - "le": "0.2", - }, - }, - Values: getTestValueStream(0, 30, 3, testStartTime), - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "request_duration_seconds_bucket", - clientmodel.JobLabel: "job1", - "instance": "ins1", - "le": "+Inf", - }, - }, - Values: getTestValueStream(0, 40, 4, testStartTime), - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "request_duration_seconds_bucket", - clientmodel.JobLabel: "job1", - "instance": "ins2", - "le": "0.1", - }, - }, - Values: getTestValueStream(0, 20, 2, testStartTime), - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "request_duration_seconds_bucket", - clientmodel.JobLabel: "job1", - "instance": "ins2", - "le": "0.2", - }, - }, - Values: getTestValueStream(0, 50, 5, testStartTime), - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "request_duration_seconds_bucket", - clientmodel.JobLabel: "job1", - "instance": "ins2", - "le": "+Inf", - }, - }, - Values: getTestValueStream(0, 60, 6, testStartTime), - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "request_duration_seconds_bucket", - clientmodel.JobLabel: "job2", - "instance": "ins1", - "le": "0.1", - }, - }, - Values: getTestValueStream(0, 30, 3, testStartTime), - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "request_duration_seconds_bucket", - clientmodel.JobLabel: "job2", - "instance": "ins1", - "le": "0.2", - }, - }, - Values: getTestValueStream(0, 40, 4, testStartTime), - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "request_duration_seconds_bucket", - clientmodel.JobLabel: "job2", - "instance": "ins1", - "le": "+Inf", - }, - }, - Values: getTestValueStream(0, 60, 6, testStartTime), - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "request_duration_seconds_bucket", - clientmodel.JobLabel: "job2", - "instance": "ins2", - "le": "0.1", - }, - }, - Values: getTestValueStream(0, 40, 4, testStartTime), - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "request_duration_seconds_bucket", - clientmodel.JobLabel: "job2", - "instance": "ins2", - "le": "0.2", - }, - }, - Values: getTestValueStream(0, 70, 7, testStartTime), - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "request_duration_seconds_bucket", - clientmodel.JobLabel: "job2", - "instance": "ins2", - "le": "+Inf", - }, - }, - Values: getTestValueStream(0, 90, 9, testStartTime), - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "vector_matching_a", - "l": "x", - }, - }, - Values: getTestValueStream(0, 100, 1, testStartTime), - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "vector_matching_a", - "l": "y", - }, - }, - Values: getTestValueStream(0, 100, 2, testStartTime), - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "vector_matching_b", - "l": "x", - }, - }, - Values: getTestValueStream(0, 100, 4, testStartTime), - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "cpu_count", - "instance": "0", - "type": "numa", - }, - }, - Values: getTestValueStream(0, 500, 30, testStartTime), - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "cpu_count", - "instance": "0", - "type": "smp", - }, - }, - Values: getTestValueStream(0, 200, 10, testStartTime), - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "cpu_count", - "instance": "1", - "type": "smp", - }, - }, - Values: getTestValueStream(0, 200, 20, testStartTime), - }, -} - -var testVector = getTestVectorFromTestMatrix(testMatrix) diff --git a/rules/lexer.l b/rules/lexer.l deleted file mode 100644 index d4c9d19af..000000000 --- a/rules/lexer.l +++ /dev/null @@ -1,118 +0,0 @@ -/* 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 rules - -import ( - "fmt" - "strconv" - "strings" - - clientmodel "github.com/prometheus/client_golang/model" -) - -// Lex is called by the parser generated by "go tool yacc" to obtain each -// token. The method is opened before the matching rules block and closed at -// the end of the file. -func (lexer *RulesLexer) Lex(lval *yySymType) int { - // Internal lexer states. - const ( - S_INITIAL = iota - S_COMMENTS - ) - - // We simulate multiple start symbols for closely-related grammars via dummy tokens. See - // http://www.gnu.org/software/bison/manual/html_node/Multiple-start_002dsymbols.html - // Reason: we want to be able to parse lists of named rules as well as single expressions. - if lexer.startToken != 0 { - startToken := lexer.startToken - lexer.startToken = 0 - return startToken - } - - c := lexer.current - currentState := 0 - - if lexer.empty { - c, lexer.empty = lexer.getChar(), false - } - -%} - -D [0-9] -L [a-zA-Z_] -M [a-zA-Z_:] -U [smhdwy] - -FLOAT ({D}*\.?{D}+|{D}+\.?{D}*){EXPONENT}?|[+-]?[iI][nN][fF]|[nN][aA][nN] -EXPONENT [eE][-+]?[0-9]+ - -STR \"(\\.|[^\\"])*\"|\'(\\.|[^\\'])*\' - -%x S_COMMENTS - -%yyc c -%yyn c = lexer.getChar() -%yyt currentState - -%% - lexer.buf = lexer.buf[:0] // The code before the first rule executed before every scan cycle (rule #0 / state 0 action) - -"/*" currentState = S_COMMENTS -"*/" currentState = S_INITIAL -.|\n /* ignore chars within multi-line comments */ - -\/\/[^\r\n]*\n /* gobble up one-line comments */ - -ALERT|alert return ALERT -IF|if return IF -FOR|for return FOR -WITH|with return WITH -SUMMARY|summary return SUMMARY -DESCRIPTION|description return DESCRIPTION - -PERMANENT|permanent return PERMANENT -BY|by return GROUP_OP -ON|on return MATCH_OP -GROUP_LEFT|GROUP_RIGHT lval.str = lexer.token(); return MATCH_MOD -group_left|group_right lval.str = strings.ToUpper(lexer.token()); return MATCH_MOD -KEEPING_EXTRA|keeping_extra return KEEPING_EXTRA -OFFSET|offset return OFFSET -AVG|SUM|MAX|MIN|COUNT lval.str = lexer.token(); return AGGR_OP -avg|sum|max|min|count lval.str = strings.ToUpper(lexer.token()); return AGGR_OP -\<|>|AND|OR|and|or lval.str = strings.ToUpper(lexer.token()); return CMP_OP -==|!=|>=|<=|=~|!~ lval.str = lexer.token(); return CMP_OP -[+\-] lval.str = lexer.token(); return ADDITIVE_OP -[*/%] lval.str = lexer.token(); return MULT_OP - -{FLOAT} num, err := strconv.ParseFloat(lexer.token(), 64); - if (err != nil && err.(*strconv.NumError).Err == strconv.ErrSyntax) { - panic("Invalid float") - } - lval.num = clientmodel.SampleValue(num) - return NUMBER - -{D}+{U} lval.str = lexer.token(); return DURATION -{L}({L}|{D})* lval.str = lexer.token(); return IDENTIFIER -{M}({M}|{D})* lval.str = lexer.token(); return METRICNAME - -{STR} lval.str = lexer.token()[1:len(lexer.token()) - 1]; return STRING - -[{}\[\]()=,] return int(lexer.buf[0]) -[\t\n\r ] /* gobble up any whitespace */ -%% - - lexer.empty = true - return int(c) -} diff --git a/rules/lexer.l.go b/rules/lexer.l.go deleted file mode 100644 index 892d1befd..000000000 --- a/rules/lexer.l.go +++ /dev/null @@ -1,2738 +0,0 @@ -/* 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 rules - -import ( - "fmt" - "strconv" - "strings" - - clientmodel "github.com/prometheus/client_golang/model" -) - -// Lex is called by the parser generated by "go tool yacc" to obtain each -// token. The method is opened before the matching rules block and closed at -// the end of the file. -func (lexer *RulesLexer) Lex(lval *yySymType) int { - // Internal lexer states. - const ( - S_INITIAL = iota - S_COMMENTS - ) - - // We simulate multiple start symbols for closely-related grammars via dummy tokens. See - // http://www.gnu.org/software/bison/manual/html_node/Multiple-start_002dsymbols.html - // Reason: we want to be able to parse lists of named rules as well as single expressions. - if lexer.startToken != 0 { - startToken := lexer.startToken - lexer.startToken = 0 - return startToken - } - - c := lexer.current - currentState := 0 - - if lexer.empty { - c, lexer.empty = lexer.getChar(), false - } - -yystate0: - - lexer.buf = lexer.buf[:0] // The code before the first rule executed before every scan cycle (rule #0 / state 0 action) - - switch yyt := currentState; yyt { - default: - panic(fmt.Errorf(`invalid start condition %d`, yyt)) - case 0: // start condition: INITIAL - goto yystart1 - case 1: // start condition: S_COMMENTS - goto yystart197 - } - - goto yystate0 // silence unused label error - goto yystate1 // silence unused label error -yystate1: - c = lexer.getChar() -yystart1: - switch { - default: - goto yyabort - case c == '!': - goto yystate3 - case c == '"': - goto yystate5 - case c == '%' || c == '*': - goto yystate8 - case c == '(' || c == ')' || c == ',' || c == '[' || c == ']' || c == '{' || c == '}': - goto yystate11 - case c == '+' || c == '-': - goto yystate12 - case c == '.': - goto yystate16 - case c == '/': - goto yystate21 - case c == ':': - goto yystate27 - case c == '<' || c == '>': - goto yystate28 - case c == '=': - goto yystate29 - case c == 'A': - goto yystate30 - case c == 'B': - goto yystate40 - case c == 'C': - goto yystate42 - case c == 'D': - goto yystate46 - case c == 'E' || c == 'H' || c == 'J' || c == 'L' || c == 'Q' || c == 'R' || c >= 'T' && c <= 'V' || c >= 'X' && c <= 'Z' || c == '_' || c == 'e' || c == 'h' || c == 'j' || c == 'l' || c == 'q' || c == 'r' || c >= 't' && c <= 'v' || c >= 'x' && c <= 'z': - goto yystate31 - case c == 'F': - goto yystate57 - case c == 'G': - goto yystate60 - case c == 'I': - goto yystate73 - case c == 'K': - goto yystate77 - case c == 'M': - goto yystate90 - case c == 'N' || c == 'n': - goto yystate93 - case c == 'O': - goto yystate95 - case c == 'P': - goto yystate102 - case c == 'S': - goto yystate111 - case c == 'W': - goto yystate118 - case c == '\'': - goto yystate9 - case c == '\t' || c == '\n' || c == '\r' || c == ' ': - goto yystate2 - case c == 'a': - goto yystate122 - case c == 'b': - goto yystate129 - case c == 'c': - goto yystate130 - case c == 'd': - goto yystate134 - case c == 'f': - goto yystate144 - case c == 'g': - goto yystate146 - case c == 'i': - goto yystate159 - case c == 'k': - goto yystate160 - case c == 'm': - goto yystate172 - case c == 'o': - goto yystate175 - case c == 'p': - goto yystate180 - case c == 's': - goto yystate188 - case c == 'w': - goto yystate194 - case c >= '0' && c <= '9': - goto yystate25 - } - -yystate2: - c = lexer.getChar() - goto yyrule30 - -yystate3: - c = lexer.getChar() - switch { - default: - goto yyabort - case c == '=' || c == '~': - goto yystate4 - } - -yystate4: - c = lexer.getChar() - goto yyrule21 - -yystate5: - c = lexer.getChar() - switch { - default: - goto yyabort - case c == '"': - goto yystate6 - case c == '\\': - goto yystate7 - case c >= '\x01' && c <= '!' || c >= '#' && c <= '[' || c >= ']' && c <= 'ÿ': - goto yystate5 - } - -yystate6: - c = lexer.getChar() - goto yyrule28 - -yystate7: - c = lexer.getChar() - switch { - default: - goto yyabort - case c >= '\x01' && c <= '\t' || c >= '\v' && c <= 'ÿ': - goto yystate5 - } - -yystate8: - c = lexer.getChar() - goto yyrule23 - -yystate9: - c = lexer.getChar() - switch { - default: - goto yyabort - case c == '\'': - goto yystate6 - case c == '\\': - goto yystate10 - case c >= '\x01' && c <= '&' || c >= '(' && c <= '[' || c >= ']' && c <= 'ÿ': - goto yystate9 - } - -yystate10: - c = lexer.getChar() - switch { - default: - goto yyabort - case c >= '\x01' && c <= '\t' || c >= '\v' && c <= 'ÿ': - goto yystate9 - } - -yystate11: - c = lexer.getChar() - goto yyrule29 - -yystate12: - c = lexer.getChar() - switch { - default: - goto yyrule22 - case c == 'I' || c == 'i': - goto yystate13 - } - -yystate13: - c = lexer.getChar() - switch { - default: - goto yyabort - case c == 'N' || c == 'n': - goto yystate14 - } - -yystate14: - c = lexer.getChar() - switch { - default: - goto yyabort - case c == 'F' || c == 'f': - goto yystate15 - } - -yystate15: - c = lexer.getChar() - goto yyrule24 - -yystate16: - c = lexer.getChar() - switch { - default: - goto yyabort - case c >= '0' && c <= '9': - goto yystate17 - } - -yystate17: - c = lexer.getChar() - switch { - default: - goto yyrule24 - case c == 'E' || c == 'e': - goto yystate18 - case c >= '0' && c <= '9': - goto yystate17 - } - -yystate18: - c = lexer.getChar() - switch { - default: - goto yyabort - case c == '+' || c == '-': - goto yystate19 - case c >= '0' && c <= '9': - goto yystate20 - } - -yystate19: - c = lexer.getChar() - switch { - default: - goto yyabort - case c >= '0' && c <= '9': - goto yystate20 - } - -yystate20: - c = lexer.getChar() - switch { - default: - goto yyrule24 - case c >= '0' && c <= '9': - goto yystate20 - } - -yystate21: - c = lexer.getChar() - switch { - default: - goto yyrule23 - case c == '*': - goto yystate22 - case c == '/': - goto yystate23 - } - -yystate22: - c = lexer.getChar() - goto yyrule1 - -yystate23: - c = lexer.getChar() - switch { - default: - goto yyabort - case c == '\n': - goto yystate24 - case c >= '\x01' && c <= '\t' || c == '\v' || c == '\f' || c >= '\x0e' && c <= 'ÿ': - goto yystate23 - } - -yystate24: - c = lexer.getChar() - goto yyrule4 - -yystate25: - c = lexer.getChar() - switch { - default: - goto yyrule24 - case c == '.': - goto yystate17 - case c == 'E' || c == 'e': - goto yystate18 - case c == 'd' || c == 'h' || c == 'm' || c == 's' || c == 'w' || c == 'y': - goto yystate26 - case c >= '0' && c <= '9': - goto yystate25 - } - -yystate26: - c = lexer.getChar() - goto yyrule25 - -yystate27: - c = lexer.getChar() - switch { - default: - goto yyrule27 - case c >= '0' && c <= ':' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate27 - } - -yystate28: - c = lexer.getChar() - switch { - default: - goto yyrule20 - case c == '=': - goto yystate4 - } - -yystate29: - c = lexer.getChar() - switch { - default: - goto yyrule29 - case c == '=' || c == '~': - goto yystate4 - } - -yystate30: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'L': - goto yystate32 - case c == 'N': - goto yystate36 - case c == 'V': - goto yystate38 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'K' || c == 'M' || c >= 'O' && c <= 'U' || c >= 'W' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate31: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate32: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'E': - goto yystate33 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'D' || c >= 'F' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate33: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'R': - goto yystate34 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Q' || c >= 'S' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate34: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'T': - goto yystate35 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'S' || c >= 'U' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate35: - c = lexer.getChar() - switch { - default: - goto yyrule5 - case c == ':': - goto yystate27 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate36: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'D': - goto yystate37 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'C' || c >= 'E' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate37: - c = lexer.getChar() - switch { - default: - goto yyrule20 - case c == ':': - goto yystate27 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate38: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'G': - goto yystate39 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'F' || c >= 'H' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate39: - c = lexer.getChar() - switch { - default: - goto yyrule18 - case c == ':': - goto yystate27 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate40: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'Y': - goto yystate41 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'X' || c == 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate41: - c = lexer.getChar() - switch { - default: - goto yyrule12 - case c == ':': - goto yystate27 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate42: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'O': - goto yystate43 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'N' || c >= 'P' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate43: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'U': - goto yystate44 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'T' || c >= 'V' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate44: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'N': - goto yystate45 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'M' || c >= 'O' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate45: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'T': - goto yystate39 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'S' || c >= 'U' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate46: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'E': - goto yystate47 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'D' || c >= 'F' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate47: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'S': - goto yystate48 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'R' || c >= 'T' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate48: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'C': - goto yystate49 - case c >= '0' && c <= '9' || c == 'A' || c == 'B' || c >= 'D' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate49: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'R': - goto yystate50 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Q' || c >= 'S' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate50: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'I': - goto yystate51 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'H' || c >= 'J' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate51: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'P': - goto yystate52 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'O' || c >= 'Q' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate52: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'T': - goto yystate53 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'S' || c >= 'U' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate53: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'I': - goto yystate54 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'H' || c >= 'J' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate54: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'O': - goto yystate55 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'N' || c >= 'P' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate55: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'N': - goto yystate56 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'M' || c >= 'O' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate56: - c = lexer.getChar() - switch { - default: - goto yyrule10 - case c == ':': - goto yystate27 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate57: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'O': - goto yystate58 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'N' || c >= 'P' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate58: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'R': - goto yystate59 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Q' || c >= 'S' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate59: - c = lexer.getChar() - switch { - default: - goto yyrule7 - case c == ':': - goto yystate27 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate60: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'R': - goto yystate61 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Q' || c >= 'S' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate61: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'O': - goto yystate62 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'N' || c >= 'P' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate62: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'U': - goto yystate63 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'T' || c >= 'V' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate63: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'P': - goto yystate64 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'O' || c >= 'Q' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate64: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == '_': - goto yystate65 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate65: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'L': - goto yystate66 - case c == 'R': - goto yystate70 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'K' || c >= 'M' && c <= 'Q' || c >= 'S' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate66: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'E': - goto yystate67 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'D' || c >= 'F' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate67: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'F': - goto yystate68 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'E' || c >= 'G' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate68: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'T': - goto yystate69 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'S' || c >= 'U' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate69: - c = lexer.getChar() - switch { - default: - goto yyrule14 - case c == ':': - goto yystate27 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate70: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'I': - goto yystate71 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'H' || c >= 'J' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate71: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'G': - goto yystate72 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'F' || c >= 'H' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate72: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'H': - goto yystate68 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'G' || c >= 'I' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate73: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'F': - goto yystate74 - case c == 'N' || c == 'n': - goto yystate75 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'E' || c >= 'G' && c <= 'M' || c >= 'O' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'm' || c >= 'o' && c <= 'z': - goto yystate31 - } - -yystate74: - c = lexer.getChar() - switch { - default: - goto yyrule6 - case c == ':': - goto yystate27 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate75: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'F' || c == 'f': - goto yystate76 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'E' || c >= 'G' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'e' || c >= 'g' && c <= 'z': - goto yystate31 - } - -yystate76: - c = lexer.getChar() - switch { - default: - goto yyrule24 - case c == ':': - goto yystate27 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate77: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'E': - goto yystate78 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'D' || c >= 'F' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate78: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'E': - goto yystate79 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'D' || c >= 'F' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate79: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'P': - goto yystate80 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'O' || c >= 'Q' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate80: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'I': - goto yystate81 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'H' || c >= 'J' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate81: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'N': - goto yystate82 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'M' || c >= 'O' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate82: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'G': - goto yystate83 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'F' || c >= 'H' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate83: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == '_': - goto yystate84 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate84: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'E': - goto yystate85 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'D' || c >= 'F' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate85: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'X': - goto yystate86 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'W' || c == 'Y' || c == 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate86: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'T': - goto yystate87 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'S' || c >= 'U' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate87: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'R': - goto yystate88 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Q' || c >= 'S' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate88: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'A': - goto yystate89 - case c >= '0' && c <= '9' || c >= 'B' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate89: - c = lexer.getChar() - switch { - default: - goto yyrule16 - case c == ':': - goto yystate27 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate90: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'A': - goto yystate91 - case c == 'I': - goto yystate92 - case c >= '0' && c <= '9' || c >= 'B' && c <= 'H' || c >= 'J' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate91: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'X': - goto yystate39 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'W' || c == 'Y' || c == 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate92: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'N': - goto yystate39 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'M' || c >= 'O' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate93: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'A' || c == 'a': - goto yystate94 - case c >= '0' && c <= '9' || c >= 'B' && c <= 'Z' || c == '_' || c >= 'b' && c <= 'z': - goto yystate31 - } - -yystate94: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'N' || c == 'n': - goto yystate76 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'M' || c >= 'O' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'm' || c >= 'o' && c <= 'z': - goto yystate31 - } - -yystate95: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'F': - goto yystate96 - case c == 'N': - goto yystate101 - case c == 'R': - goto yystate37 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'E' || c >= 'G' && c <= 'M' || c >= 'O' && c <= 'Q' || c >= 'S' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate96: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'F': - goto yystate97 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'E' || c >= 'G' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate97: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'S': - goto yystate98 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'R' || c >= 'T' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate98: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'E': - goto yystate99 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'D' || c >= 'F' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate99: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'T': - goto yystate100 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'S' || c >= 'U' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate100: - c = lexer.getChar() - switch { - default: - goto yyrule17 - case c == ':': - goto yystate27 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate101: - c = lexer.getChar() - switch { - default: - goto yyrule13 - case c == ':': - goto yystate27 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate102: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'E': - goto yystate103 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'D' || c >= 'F' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate103: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'R': - goto yystate104 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Q' || c >= 'S' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate104: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'M': - goto yystate105 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'L' || c >= 'N' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate105: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'A': - goto yystate106 - case c >= '0' && c <= '9' || c >= 'B' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate106: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'N': - goto yystate107 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'M' || c >= 'O' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate107: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'E': - goto yystate108 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'D' || c >= 'F' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate108: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'N': - goto yystate109 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'M' || c >= 'O' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate109: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'T': - goto yystate110 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'S' || c >= 'U' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate110: - c = lexer.getChar() - switch { - default: - goto yyrule11 - case c == ':': - goto yystate27 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate111: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'U': - goto yystate112 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'T' || c >= 'V' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate112: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'M': - goto yystate113 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'L' || c >= 'N' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate113: - c = lexer.getChar() - switch { - default: - goto yyrule18 - case c == ':': - goto yystate27 - case c == 'M': - goto yystate114 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'L' || c >= 'N' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate114: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'A': - goto yystate115 - case c >= '0' && c <= '9' || c >= 'B' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate115: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'R': - goto yystate116 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Q' || c >= 'S' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate116: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'Y': - goto yystate117 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'X' || c == 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate117: - c = lexer.getChar() - switch { - default: - goto yyrule9 - case c == ':': - goto yystate27 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate118: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'I': - goto yystate119 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'H' || c >= 'J' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate119: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'T': - goto yystate120 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'S' || c >= 'U' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate120: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'H': - goto yystate121 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'G' || c >= 'I' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate121: - c = lexer.getChar() - switch { - default: - goto yyrule8 - case c == ':': - goto yystate27 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate122: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'l': - goto yystate123 - case c == 'n': - goto yystate126 - case c == 'v': - goto yystate127 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'k' || c == 'm' || c >= 'o' && c <= 'u' || c >= 'w' && c <= 'z': - goto yystate31 - } - -yystate123: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'e': - goto yystate124 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'd' || c >= 'f' && c <= 'z': - goto yystate31 - } - -yystate124: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'r': - goto yystate125 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'q' || c >= 's' && c <= 'z': - goto yystate31 - } - -yystate125: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 't': - goto yystate35 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 's' || c >= 'u' && c <= 'z': - goto yystate31 - } - -yystate126: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'd': - goto yystate37 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'c' || c >= 'e' && c <= 'z': - goto yystate31 - } - -yystate127: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'g': - goto yystate128 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'f' || c >= 'h' && c <= 'z': - goto yystate31 - } - -yystate128: - c = lexer.getChar() - switch { - default: - goto yyrule19 - case c == ':': - goto yystate27 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate129: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'y': - goto yystate41 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'x' || c == 'z': - goto yystate31 - } - -yystate130: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'o': - goto yystate131 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'n' || c >= 'p' && c <= 'z': - goto yystate31 - } - -yystate131: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'u': - goto yystate132 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 't' || c >= 'v' && c <= 'z': - goto yystate31 - } - -yystate132: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'n': - goto yystate133 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'm' || c >= 'o' && c <= 'z': - goto yystate31 - } - -yystate133: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 't': - goto yystate128 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 's' || c >= 'u' && c <= 'z': - goto yystate31 - } - -yystate134: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'e': - goto yystate135 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'd' || c >= 'f' && c <= 'z': - goto yystate31 - } - -yystate135: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 's': - goto yystate136 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'r' || c >= 't' && c <= 'z': - goto yystate31 - } - -yystate136: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'c': - goto yystate137 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c == 'a' || c == 'b' || c >= 'd' && c <= 'z': - goto yystate31 - } - -yystate137: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'r': - goto yystate138 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'q' || c >= 's' && c <= 'z': - goto yystate31 - } - -yystate138: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'i': - goto yystate139 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'h' || c >= 'j' && c <= 'z': - goto yystate31 - } - -yystate139: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'p': - goto yystate140 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'o' || c >= 'q' && c <= 'z': - goto yystate31 - } - -yystate140: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 't': - goto yystate141 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 's' || c >= 'u' && c <= 'z': - goto yystate31 - } - -yystate141: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'i': - goto yystate142 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'h' || c >= 'j' && c <= 'z': - goto yystate31 - } - -yystate142: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'o': - goto yystate143 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'n' || c >= 'p' && c <= 'z': - goto yystate31 - } - -yystate143: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'n': - goto yystate56 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'm' || c >= 'o' && c <= 'z': - goto yystate31 - } - -yystate144: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'o': - goto yystate145 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'n' || c >= 'p' && c <= 'z': - goto yystate31 - } - -yystate145: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'r': - goto yystate59 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'q' || c >= 's' && c <= 'z': - goto yystate31 - } - -yystate146: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'r': - goto yystate147 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'q' || c >= 's' && c <= 'z': - goto yystate31 - } - -yystate147: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'o': - goto yystate148 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'n' || c >= 'p' && c <= 'z': - goto yystate31 - } - -yystate148: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'u': - goto yystate149 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 't' || c >= 'v' && c <= 'z': - goto yystate31 - } - -yystate149: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'p': - goto yystate150 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'o' || c >= 'q' && c <= 'z': - goto yystate31 - } - -yystate150: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == '_': - goto yystate151 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate151: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'l': - goto yystate152 - case c == 'r': - goto yystate156 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'k' || c >= 'm' && c <= 'q' || c >= 's' && c <= 'z': - goto yystate31 - } - -yystate152: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'e': - goto yystate153 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'd' || c >= 'f' && c <= 'z': - goto yystate31 - } - -yystate153: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'f': - goto yystate154 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'e' || c >= 'g' && c <= 'z': - goto yystate31 - } - -yystate154: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 't': - goto yystate155 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 's' || c >= 'u' && c <= 'z': - goto yystate31 - } - -yystate155: - c = lexer.getChar() - switch { - default: - goto yyrule15 - case c == ':': - goto yystate27 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate156: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'i': - goto yystate157 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'h' || c >= 'j' && c <= 'z': - goto yystate31 - } - -yystate157: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'g': - goto yystate158 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'f' || c >= 'h' && c <= 'z': - goto yystate31 - } - -yystate158: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'h': - goto yystate154 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'g' || c >= 'i' && c <= 'z': - goto yystate31 - } - -yystate159: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'N' || c == 'n': - goto yystate75 - case c == 'f': - goto yystate74 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'M' || c >= 'O' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'e' || c >= 'g' && c <= 'm' || c >= 'o' && c <= 'z': - goto yystate31 - } - -yystate160: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'e': - goto yystate161 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'd' || c >= 'f' && c <= 'z': - goto yystate31 - } - -yystate161: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'e': - goto yystate162 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'd' || c >= 'f' && c <= 'z': - goto yystate31 - } - -yystate162: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'p': - goto yystate163 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'o' || c >= 'q' && c <= 'z': - goto yystate31 - } - -yystate163: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'i': - goto yystate164 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'h' || c >= 'j' && c <= 'z': - goto yystate31 - } - -yystate164: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'n': - goto yystate165 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'm' || c >= 'o' && c <= 'z': - goto yystate31 - } - -yystate165: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'g': - goto yystate166 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'f' || c >= 'h' && c <= 'z': - goto yystate31 - } - -yystate166: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == '_': - goto yystate167 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c >= 'a' && c <= 'z': - goto yystate31 - } - -yystate167: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'e': - goto yystate168 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'd' || c >= 'f' && c <= 'z': - goto yystate31 - } - -yystate168: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'x': - goto yystate169 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'w' || c == 'y' || c == 'z': - goto yystate31 - } - -yystate169: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 't': - goto yystate170 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 's' || c >= 'u' && c <= 'z': - goto yystate31 - } - -yystate170: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'r': - goto yystate171 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'q' || c >= 's' && c <= 'z': - goto yystate31 - } - -yystate171: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'a': - goto yystate89 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'b' && c <= 'z': - goto yystate31 - } - -yystate172: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'a': - goto yystate173 - case c == 'i': - goto yystate174 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'b' && c <= 'h' || c >= 'j' && c <= 'z': - goto yystate31 - } - -yystate173: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'x': - goto yystate128 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'w' || c == 'y' || c == 'z': - goto yystate31 - } - -yystate174: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'n': - goto yystate128 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'm' || c >= 'o' && c <= 'z': - goto yystate31 - } - -yystate175: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'f': - goto yystate176 - case c == 'n': - goto yystate101 - case c == 'r': - goto yystate37 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'e' || c >= 'g' && c <= 'm' || c >= 'o' && c <= 'q' || c >= 's' && c <= 'z': - goto yystate31 - } - -yystate176: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'f': - goto yystate177 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'e' || c >= 'g' && c <= 'z': - goto yystate31 - } - -yystate177: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 's': - goto yystate178 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'r' || c >= 't' && c <= 'z': - goto yystate31 - } - -yystate178: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'e': - goto yystate179 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'd' || c >= 'f' && c <= 'z': - goto yystate31 - } - -yystate179: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 't': - goto yystate100 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 's' || c >= 'u' && c <= 'z': - goto yystate31 - } - -yystate180: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'e': - goto yystate181 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'd' || c >= 'f' && c <= 'z': - goto yystate31 - } - -yystate181: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'r': - goto yystate182 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'q' || c >= 's' && c <= 'z': - goto yystate31 - } - -yystate182: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'm': - goto yystate183 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'l' || c >= 'n' && c <= 'z': - goto yystate31 - } - -yystate183: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'a': - goto yystate184 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'b' && c <= 'z': - goto yystate31 - } - -yystate184: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'n': - goto yystate185 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'm' || c >= 'o' && c <= 'z': - goto yystate31 - } - -yystate185: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'e': - goto yystate186 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'd' || c >= 'f' && c <= 'z': - goto yystate31 - } - -yystate186: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'n': - goto yystate187 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'm' || c >= 'o' && c <= 'z': - goto yystate31 - } - -yystate187: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 't': - goto yystate110 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 's' || c >= 'u' && c <= 'z': - goto yystate31 - } - -yystate188: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'u': - goto yystate189 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 't' || c >= 'v' && c <= 'z': - goto yystate31 - } - -yystate189: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'm': - goto yystate190 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'l' || c >= 'n' && c <= 'z': - goto yystate31 - } - -yystate190: - c = lexer.getChar() - switch { - default: - goto yyrule19 - case c == ':': - goto yystate27 - case c == 'm': - goto yystate191 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'l' || c >= 'n' && c <= 'z': - goto yystate31 - } - -yystate191: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'a': - goto yystate192 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'b' && c <= 'z': - goto yystate31 - } - -yystate192: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'r': - goto yystate193 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'q' || c >= 's' && c <= 'z': - goto yystate31 - } - -yystate193: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'y': - goto yystate117 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'x' || c == 'z': - goto yystate31 - } - -yystate194: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'i': - goto yystate195 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'h' || c >= 'j' && c <= 'z': - goto yystate31 - } - -yystate195: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 't': - goto yystate196 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 's' || c >= 'u' && c <= 'z': - goto yystate31 - } - -yystate196: - c = lexer.getChar() - switch { - default: - goto yyrule26 - case c == ':': - goto yystate27 - case c == 'h': - goto yystate121 - case c >= '0' && c <= '9' || c >= 'A' && c <= 'Z' || c == '_' || c >= 'a' && c <= 'g' || c >= 'i' && c <= 'z': - goto yystate31 - } - - goto yystate197 // silence unused label error -yystate197: - c = lexer.getChar() -yystart197: - switch { - default: - goto yyabort - case c == '*': - goto yystate199 - case c >= '\x01' && c <= ')' || c >= '+' && c <= 'ÿ': - goto yystate198 - } - -yystate198: - c = lexer.getChar() - goto yyrule3 - -yystate199: - c = lexer.getChar() - switch { - default: - goto yyrule3 - case c == '/': - goto yystate200 - } - -yystate200: - c = lexer.getChar() - goto yyrule2 - -yyrule1: // "/*" - { - currentState = S_COMMENTS - goto yystate0 - } -yyrule2: // "*/" - { - currentState = S_INITIAL - goto yystate0 - } -yyrule3: // .|\n - { - /* ignore chars within multi-line comments */ - goto yystate0 - } -yyrule4: // \/\/[^\r\n]*\n - { - /* gobble up one-line comments */ - goto yystate0 - } -yyrule5: // ALERT|alert - { - return ALERT - } -yyrule6: // IF|if - { - return IF - } -yyrule7: // FOR|for - { - return FOR - } -yyrule8: // WITH|with - { - return WITH - } -yyrule9: // SUMMARY|summary - { - return SUMMARY - } -yyrule10: // DESCRIPTION|description - { - return DESCRIPTION - } -yyrule11: // PERMANENT|permanent - { - return PERMANENT - } -yyrule12: // BY|by - { - return GROUP_OP - } -yyrule13: // ON|on - { - return MATCH_OP - } -yyrule14: // GROUP_LEFT|GROUP_RIGHT - { - lval.str = lexer.token() - return MATCH_MOD - goto yystate0 - } -yyrule15: // group_left|group_right - { - lval.str = strings.ToUpper(lexer.token()) - return MATCH_MOD - goto yystate0 - } -yyrule16: // KEEPING_EXTRA|keeping_extra - { - return KEEPING_EXTRA - } -yyrule17: // OFFSET|offset - { - return OFFSET - } -yyrule18: // AVG|SUM|MAX|MIN|COUNT - { - lval.str = lexer.token() - return AGGR_OP - goto yystate0 - } -yyrule19: // avg|sum|max|min|count - { - lval.str = strings.ToUpper(lexer.token()) - return AGGR_OP - goto yystate0 - } -yyrule20: // \<|>|AND|OR|and|or - { - lval.str = strings.ToUpper(lexer.token()) - return CMP_OP - goto yystate0 - } -yyrule21: // ==|!=|>=|<=|=~|!~ - { - lval.str = lexer.token() - return CMP_OP - goto yystate0 - } -yyrule22: // [+\-] - { - lval.str = lexer.token() - return ADDITIVE_OP - goto yystate0 - } -yyrule23: // [*/%] - { - lval.str = lexer.token() - return MULT_OP - goto yystate0 - } -yyrule24: // {FLOAT} - { - num, err := strconv.ParseFloat(lexer.token(), 64) - if err != nil && err.(*strconv.NumError).Err == strconv.ErrSyntax { - panic("Invalid float") - } - lval.num = clientmodel.SampleValue(num) - return NUMBER - } -yyrule25: // {D}+{U} - { - lval.str = lexer.token() - return DURATION - goto yystate0 - } -yyrule26: // {L}({L}|{D})* - { - lval.str = lexer.token() - return IDENTIFIER - goto yystate0 - } -yyrule27: // {M}({M}|{D})* - { - lval.str = lexer.token() - return METRICNAME - goto yystate0 - } -yyrule28: // {STR} - { - lval.str = lexer.token()[1 : len(lexer.token())-1] - return STRING - goto yystate0 - } -yyrule29: // [{}\[\]()=,] - { - return int(lexer.buf[0]) - } -yyrule30: // [\t\n\r ] - { - /* gobble up any whitespace */ - goto yystate0 - } - panic("unreachable") - - goto yyabort // silence unused label error - -yyabort: // no lexem recognized - - lexer.empty = true - return int(c) -} diff --git a/rules/load.go b/rules/load.go deleted file mode 100644 index c165b7d2a..000000000 --- a/rules/load.go +++ /dev/null @@ -1,164 +0,0 @@ -// 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 rules - -import ( - "bufio" - "errors" - "fmt" - "io" - "os" - "strings" - - "github.com/golang/glog" - - "github.com/prometheus/prometheus/rules/ast" -) - -// RulesLexer is the lexer for rule expressions. -type RulesLexer struct { - // Errors encountered during parsing. - errors []string - // Dummy token to simulate multiple start symbols (see below). - startToken int - // Parsed full rules. - parsedRules []Rule - // Parsed single expression. - parsedExpr ast.Node - - // Current character. - current byte - // Current token buffer. - buf []byte - // Input text. - src *bufio.Reader - // Whether we have a current char. - empty bool - - // Current input line. - line int - // Current character position within the current input line. - pos int -} - -func (lexer *RulesLexer) Error(errorStr string) { - err := fmt.Sprintf("Error parsing rules at line %v, char %v: %v", lexer.line, lexer.pos, errorStr) - lexer.errors = append(lexer.errors, err) -} - -func (lexer *RulesLexer) getChar() byte { - if lexer.current != 0 { - lexer.buf = append(lexer.buf, lexer.current) - } - lexer.current = 0 - if b, err := lexer.src.ReadByte(); err == nil { - if b == '\n' { - lexer.line++ - lexer.pos = 0 - } else { - lexer.pos++ - } - lexer.current = b - } else if err != io.EOF { - glog.Fatal(err) - } - return lexer.current -} - -func (lexer *RulesLexer) token() string { - return string(lexer.buf) -} - -func newRulesLexer(src io.Reader, singleExpr bool) *RulesLexer { - lexer := &RulesLexer{ - startToken: START_RULES, - src: bufio.NewReader(src), - pos: 1, - line: 1, - } - - if singleExpr { - lexer.startToken = START_EXPRESSION - } - lexer.getChar() - return lexer -} - -func lexAndParse(rulesReader io.Reader, singleExpr bool) (*RulesLexer, error) { - lexer := newRulesLexer(rulesReader, singleExpr) - ret := yyParse(lexer) - if ret != 0 && len(lexer.errors) == 0 { - lexer.Error("unknown parser error") - } - - if len(lexer.errors) > 0 { - err := errors.New(strings.Join(lexer.errors, "\n")) - return nil, err - } - return lexer, nil -} - -// LoadRulesFromReader parses rules from the provided reader and returns them. -func LoadRulesFromReader(rulesReader io.Reader) ([]Rule, error) { - lexer, err := lexAndParse(rulesReader, false) - if err != nil { - return nil, err - } - return lexer.parsedRules, err -} - -// LoadRulesFromString parses rules from the provided string returns them. -func LoadRulesFromString(rulesString string) ([]Rule, error) { - rulesReader := strings.NewReader(rulesString) - return LoadRulesFromReader(rulesReader) -} - -// LoadRulesFromFile parses rules from the file of the provided name and returns -// them. -func LoadRulesFromFile(fileName string) ([]Rule, error) { - rulesReader, err := os.Open(fileName) - if err != nil { - return []Rule{}, err - } - defer rulesReader.Close() - return LoadRulesFromReader(rulesReader) -} - -// LoadExprFromReader parses a single expression from the provided reader and -// returns it as an AST node. -func LoadExprFromReader(exprReader io.Reader) (ast.Node, error) { - lexer, err := lexAndParse(exprReader, true) - if err != nil { - return nil, err - } - return lexer.parsedExpr, err -} - -// LoadExprFromString parses a single expression from the provided string and -// returns it as an AST node. -func LoadExprFromString(exprString string) (ast.Node, error) { - exprReader := strings.NewReader(exprString) - return LoadExprFromReader(exprReader) -} - -// LoadExprFromFile parses a single expression from the file of the provided -// name and returns it as an AST node. -func LoadExprFromFile(fileName string) (ast.Node, error) { - exprReader, err := os.Open(fileName) - if err != nil { - return nil, err - } - defer exprReader.Close() - return LoadExprFromReader(exprReader) -} diff --git a/rules/manager/manager.go b/rules/manager.go similarity index 78% rename from rules/manager/manager.go rename to rules/manager.go index 0b1a386a5..26d12f95d 100644 --- a/rules/manager/manager.go +++ b/rules/manager.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package manager +package rules import ( "fmt" @@ -20,15 +20,15 @@ import ( "github.com/golang/glog" "github.com/prometheus/client_golang/prometheus" + "golang.org/x/net/context" clientmodel "github.com/prometheus/client_golang/model" - "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/notification" - "github.com/prometheus/prometheus/rules" + "github.com/prometheus/prometheus/promql" "github.com/prometheus/prometheus/storage" - "github.com/prometheus/prometheus/storage/local" "github.com/prometheus/prometheus/templates" + "github.com/prometheus/prometheus/utility" ) // Constants for instrumentation. @@ -73,60 +73,61 @@ func init() { // A RuleManager manages recording and alerting rules. Create instances with // NewRuleManager. type RuleManager interface { - // Load and add rules from rule files specified in the configuration. - AddRulesFromConfig(config config.Config) error // Start the rule manager's periodic rule evaluation. Run() // Stop the rule manager's rule evaluation cycles. Stop() // Return all rules. - Rules() []rules.Rule + Rules() []Rule // Return all alerting rules. - AlertingRules() []*rules.AlertingRule + AlertingRules() []*AlertingRule } type ruleManager struct { // Protects the rules list. sync.Mutex - rules []rules.Rule + rules []Rule done chan bool - interval time.Duration - storage local.Storage + interval time.Duration + queryEngine *promql.Engine sampleAppender storage.SampleAppender notificationHandler *notification.NotificationHandler prometheusURL string - pathPrefix string + pathPrefix string } // RuleManagerOptions bundles options for the RuleManager. type RuleManagerOptions struct { EvaluationInterval time.Duration - Storage local.Storage + QueryEngine *promql.Engine NotificationHandler *notification.NotificationHandler SampleAppender storage.SampleAppender PrometheusURL string - PathPrefix string + PathPrefix string } // NewRuleManager returns an implementation of RuleManager, ready to be started // by calling the Run method. func NewRuleManager(o *RuleManagerOptions) RuleManager { manager := &ruleManager{ - rules: []rules.Rule{}, + rules: []Rule{}, done: make(chan bool), interval: o.EvaluationInterval, - storage: o.Storage, sampleAppender: o.SampleAppender, + queryEngine: o.QueryEngine, notificationHandler: o.NotificationHandler, prometheusURL: o.PrometheusURL, } + manager.queryEngine.RegisterAlertHandler("rule_manager", manager.AddAlertingRule) + manager.queryEngine.RegisterRecordHandler("rule_manager", manager.AddRecordingRule) + return manager } @@ -162,7 +163,7 @@ func (m *ruleManager) Stop() { m.done <- true } -func (m *ruleManager) queueAlertNotifications(rule *rules.AlertingRule, timestamp clientmodel.Timestamp) { +func (m *ruleManager) queueAlertNotifications(rule *AlertingRule, timestamp clientmodel.Timestamp) { activeAlerts := rule.ActiveAlerts() if len(activeAlerts) == 0 { return @@ -170,7 +171,7 @@ func (m *ruleManager) queueAlertNotifications(rule *rules.AlertingRule, timestam notifications := make(notification.NotificationReqs, 0, len(activeAlerts)) for _, aa := range activeAlerts { - if aa.State != rules.Firing { + if aa.State != Firing { // BUG: In the future, make AlertManager support pending alerts? continue } @@ -192,7 +193,7 @@ func (m *ruleManager) queueAlertNotifications(rule *rules.AlertingRule, timestam defs := "{{$labels := .Labels}}{{$value := .Value}}" expand := func(text string) string { - template := templates.NewTemplateExpander(defs+text, "__alert_"+rule.Name(), tmplData, timestamp, m.storage, m.pathPrefix) + template := templates.NewTemplateExpander(defs+text, "__alert_"+rule.Name(), tmplData, timestamp, m.queryEngine, m.pathPrefix) result, err := template.Expand() if err != nil { result = err.Error() @@ -205,12 +206,12 @@ func (m *ruleManager) queueAlertNotifications(rule *rules.AlertingRule, timestam Summary: expand(rule.Summary), Description: expand(rule.Description), Labels: aa.Labels.Merge(clientmodel.LabelSet{ - rules.AlertNameLabel: clientmodel.LabelValue(rule.Name()), + AlertNameLabel: clientmodel.LabelValue(rule.Name()), }), Value: aa.Value, ActiveSince: aa.ActiveSince.Time(), RuleString: rule.String(), - GeneratorURL: m.prometheusURL + rules.GraphLinkForExpression(rule.Vector.String()), + GeneratorURL: m.prometheusURL + utility.GraphLinkForExpression(rule.Vector.String()), }) } m.notificationHandler.SubmitReqs(notifications) @@ -221,18 +222,18 @@ func (m *ruleManager) runIteration() { wg := sync.WaitGroup{} m.Lock() - rulesSnapshot := make([]rules.Rule, len(m.rules)) + rulesSnapshot := make([]Rule, len(m.rules)) copy(rulesSnapshot, m.rules) m.Unlock() for _, rule := range rulesSnapshot { wg.Add(1) // BUG(julius): Look at fixing thundering herd. - go func(rule rules.Rule) { + go func(rule Rule) { defer wg.Done() start := time.Now() - vector, err := rule.Eval(now, m.storage) + vector, err := rule.Eval(now, m.queryEngine) duration := time.Since(start) if err != nil { @@ -242,17 +243,17 @@ func (m *ruleManager) runIteration() { } switch r := rule.(type) { - case *rules.AlertingRule: + case *AlertingRule: m.queueAlertNotifications(r, now) evalDuration.WithLabelValues(alertingRuleType).Observe( float64(duration / time.Millisecond), ) - case *rules.RecordingRule: + case *RecordingRule: evalDuration.WithLabelValues(recordingRuleType).Observe( float64(duration / time.Millisecond), ) default: - panic(fmt.Sprintf("Unknown rule type: %T", rule)) + panic(fmt.Errorf("Unknown rule type: %T", rule)) } for _, s := range vector { @@ -267,35 +268,40 @@ func (m *ruleManager) runIteration() { wg.Wait() } -func (m *ruleManager) AddRulesFromConfig(config config.Config) error { - for _, ruleFile := range config.Global.RuleFile { - newRules, err := rules.LoadRulesFromFile(ruleFile) - if err != nil { - return fmt.Errorf("%s: %s", ruleFile, err) - } - m.Lock() - m.rules = append(m.rules, newRules...) - m.Unlock() - } +func (m *ruleManager) AddAlertingRule(ctx context.Context, r *promql.AlertStmt) error { + rule := NewAlertingRule(r.Name, r.Expr, r.Duration, r.Labels, r.Summary, r.Description) + + m.Lock() + m.rules = append(m.rules, rule) + m.Unlock() return nil } -func (m *ruleManager) Rules() []rules.Rule { +func (m *ruleManager) AddRecordingRule(ctx context.Context, r *promql.RecordStmt) error { + rule := &RecordingRule{r.Name, r.Expr, r.Labels} + + m.Lock() + m.rules = append(m.rules, rule) + m.Unlock() + return nil +} + +func (m *ruleManager) Rules() []Rule { m.Lock() defer m.Unlock() - rules := make([]rules.Rule, len(m.rules)) + rules := make([]Rule, len(m.rules)) copy(rules, m.rules) return rules } -func (m *ruleManager) AlertingRules() []*rules.AlertingRule { +func (m *ruleManager) AlertingRules() []*AlertingRule { m.Lock() defer m.Unlock() - alerts := []*rules.AlertingRule{} + alerts := []*AlertingRule{} for _, rule := range m.rules { - if alertingRule, ok := rule.(*rules.AlertingRule); ok { + if alertingRule, ok := rule.(*AlertingRule); ok { alerts = append(alerts, alertingRule) } } diff --git a/rules/parser.y b/rules/parser.y deleted file mode 100644 index feba0fc0b..000000000 --- a/rules/parser.y +++ /dev/null @@ -1,281 +0,0 @@ -// 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 rules - - import ( - clientmodel "github.com/prometheus/client_golang/model" - - "github.com/prometheus/prometheus/rules/ast" - "github.com/prometheus/prometheus/storage/metric" - ) -%} - -%union { - num clientmodel.SampleValue - str string - ruleNode ast.Node - ruleNodeSlice []ast.Node - boolean bool - labelNameSlice clientmodel.LabelNames - labelSet clientmodel.LabelSet - labelMatcher *metric.LabelMatcher - labelMatchers metric.LabelMatchers - vectorMatching *vectorMatching -} - -/* We simulate multiple start symbols for closely-related grammars via dummy tokens. See - http://www.gnu.org/software/bison/manual/html_node/Multiple-start_002dsymbols.html - Reason: we want to be able to parse lists of named rules as well as single expressions. - */ -%token START_RULES START_EXPRESSION - -%token IDENTIFIER STRING DURATION METRICNAME -%token NUMBER -%token PERMANENT GROUP_OP KEEPING_EXTRA OFFSET MATCH_OP -%token AGGR_OP CMP_OP ADDITIVE_OP MULT_OP MATCH_MOD -%token ALERT IF FOR WITH SUMMARY DESCRIPTION - -%type func_arg_list -%type label_list grouping_opts -%type label_assign label_assign_list rule_labels -%type label_match -%type label_match_list label_matches -%type vector_matching -%type rule_expr func_arg -%type qualifier extra_labels_opts -%type for_duration metric_name label_match_type offset_opts - -%right '=' -%left CMP_OP -%left ADDITIVE_OP -%left MULT_OP -%start start - -%% -start : START_RULES rules_stat_list - | START_EXPRESSION saved_rule_expr - ; - -rules_stat_list : /* empty */ - | rules_stat_list rules_stat - ; - -saved_rule_expr : rule_expr - { yylex.(*RulesLexer).parsedExpr = $1 } - ; - - -rules_stat : qualifier metric_name rule_labels '=' rule_expr - { - rule, err := CreateRecordingRule($2, $3, $5, $1) - if err != nil { yylex.Error(err.Error()); return 1 } - yylex.(*RulesLexer).parsedRules = append(yylex.(*RulesLexer).parsedRules, rule) - } - | ALERT IDENTIFIER IF rule_expr for_duration WITH rule_labels SUMMARY STRING DESCRIPTION STRING - { - rule, err := CreateAlertingRule($2, $4, $5, $7, $9, $11) - if err != nil { yylex.Error(err.Error()); return 1 } - yylex.(*RulesLexer).parsedRules = append(yylex.(*RulesLexer).parsedRules, rule) - } - ; - -for_duration : /* empty */ - { $$ = "0s" } - | FOR DURATION - { $$ = $2 } - ; - -qualifier : /* empty */ - { $$ = false } - | PERMANENT - { $$ = true } - ; - -metric_name : METRICNAME - { $$ = $1 } - | IDENTIFIER - { $$ = $1 } - ; - -rule_labels : /* empty */ - { $$ = clientmodel.LabelSet{} } - | '{' label_assign_list '}' - { $$ = $2 } - | '{' '}' - { $$ = clientmodel.LabelSet{} } - -label_assign_list : label_assign - { $$ = $1 } - | label_assign_list ',' label_assign - { for k, v := range $3 { $$[k] = v } } - ; - -label_assign : IDENTIFIER '=' STRING - { $$ = clientmodel.LabelSet{ clientmodel.LabelName($1): clientmodel.LabelValue($3) } } - ; - -label_matches : /* empty */ - { $$ = metric.LabelMatchers{} } - | '{' '}' - { $$ = metric.LabelMatchers{} } - | '{' label_match_list '}' - { $$ = $2 } - ; - -label_match_list : label_match - { $$ = metric.LabelMatchers{$1} } - | label_match_list ',' label_match - { $$ = append($$, $3) } - ; - -label_match : IDENTIFIER label_match_type STRING - { - var err error - $$, err = newLabelMatcher($2, clientmodel.LabelName($1), clientmodel.LabelValue($3)) - if err != nil { yylex.Error(err.Error()); return 1 } - } - ; - -label_match_type : '=' - { $$ = "=" } - | CMP_OP - { $$ = $1 } - ; - -offset_opts : /* empty */ - { $$ = "0s" } - | OFFSET DURATION - { $$ = $2 } - ; - -rule_expr : '(' rule_expr ')' - { $$ = $2 } - | '{' label_match_list '}' offset_opts - { - var err error - $$, err = NewVectorSelector($2, $4) - if err != nil { yylex.Error(err.Error()); return 1 } - } - | metric_name label_matches offset_opts - { - var err error - m, err := metric.NewLabelMatcher(metric.Equal, clientmodel.MetricNameLabel, clientmodel.LabelValue($1)) - if err != nil { yylex.Error(err.Error()); return 1 } - $2 = append($2, m) - $$, err = NewVectorSelector($2, $3) - if err != nil { yylex.Error(err.Error()); return 1 } - } - | IDENTIFIER '(' func_arg_list ')' - { - var err error - $$, err = NewFunctionCall($1, $3) - if err != nil { yylex.Error(err.Error()); return 1 } - } - | IDENTIFIER '(' ')' - { - var err error - $$, err = NewFunctionCall($1, []ast.Node{}) - if err != nil { yylex.Error(err.Error()); return 1 } - } - | rule_expr '[' DURATION ']' offset_opts - { - var err error - $$, err = NewMatrixSelector($1, $3, $5) - if err != nil { yylex.Error(err.Error()); return 1 } - } - | AGGR_OP '(' rule_expr ')' grouping_opts extra_labels_opts - { - var err error - $$, err = NewVectorAggregation($1, $3, $5, $6) - if err != nil { yylex.Error(err.Error()); return 1 } - } - | AGGR_OP grouping_opts extra_labels_opts '(' rule_expr ')' - { - var err error - $$, err = NewVectorAggregation($1, $5, $2, $3) - if err != nil { yylex.Error(err.Error()); return 1 } - } - /* Yacc can only attach associativity to terminals, so we - * have to list all operators here. */ - | rule_expr ADDITIVE_OP vector_matching rule_expr - { - var err error - $$, err = NewArithExpr($2, $1, $4, $3) - if err != nil { yylex.Error(err.Error()); return 1 } - } - | rule_expr MULT_OP vector_matching rule_expr - { - var err error - $$, err = NewArithExpr($2, $1, $4, $3) - if err != nil { yylex.Error(err.Error()); return 1 } - } - | rule_expr CMP_OP vector_matching rule_expr - { - var err error - $$, err = NewArithExpr($2, $1, $4, $3) - if err != nil { yylex.Error(err.Error()); return 1 } - } - | NUMBER - { $$ = NewScalarLiteral($1, "+")} - | ADDITIVE_OP NUMBER - { $$ = NewScalarLiteral($2, $1)} - ; - -extra_labels_opts : /* empty */ - { $$ = false } - | KEEPING_EXTRA - { $$ = true } - ; - -vector_matching : /* empty */ - { $$ = nil } - | MATCH_OP '(' label_list ')' - { - var err error - $$, err = newVectorMatching("", $3, nil) - if err != nil { yylex.Error(err.Error()); return 1 } - } - | MATCH_OP '(' label_list ')' MATCH_MOD '(' label_list ')' - { - var err error - $$, err = newVectorMatching($5, $3, $7) - if err != nil { yylex.Error(err.Error()); return 1 } - } - ; - -grouping_opts : - { $$ = clientmodel.LabelNames{} } - | GROUP_OP '(' label_list ')' - { $$ = $3 } - ; - -label_list : IDENTIFIER - { $$ = clientmodel.LabelNames{clientmodel.LabelName($1)} } - | label_list ',' IDENTIFIER - { $$ = append($$, clientmodel.LabelName($3)) } - ; - -func_arg_list : func_arg - { $$ = []ast.Node{$1} } - | func_arg_list ',' func_arg - { $$ = append($$, $3) } - ; - -func_arg : rule_expr - { $$ = $1 } - | STRING - { $$ = ast.NewStringLiteral($1) } - ; -%% diff --git a/rules/parser.y.go b/rules/parser.y.go deleted file mode 100644 index e066ff3a5..000000000 --- a/rules/parser.y.go +++ /dev/null @@ -1,784 +0,0 @@ -//line parser.y:15 -package rules - -import __yyfmt__ "fmt" - -//line parser.y:15 -import ( - clientmodel "github.com/prometheus/client_golang/model" - - "github.com/prometheus/prometheus/rules/ast" - "github.com/prometheus/prometheus/storage/metric" -) - -//line parser.y:25 -type yySymType struct { - yys int - num clientmodel.SampleValue - str string - ruleNode ast.Node - ruleNodeSlice []ast.Node - boolean bool - labelNameSlice clientmodel.LabelNames - labelSet clientmodel.LabelSet - labelMatcher *metric.LabelMatcher - labelMatchers metric.LabelMatchers - vectorMatching *vectorMatching -} - -const START_RULES = 57346 -const START_EXPRESSION = 57347 -const IDENTIFIER = 57348 -const STRING = 57349 -const DURATION = 57350 -const METRICNAME = 57351 -const NUMBER = 57352 -const PERMANENT = 57353 -const GROUP_OP = 57354 -const KEEPING_EXTRA = 57355 -const OFFSET = 57356 -const MATCH_OP = 57357 -const AGGR_OP = 57358 -const CMP_OP = 57359 -const ADDITIVE_OP = 57360 -const MULT_OP = 57361 -const MATCH_MOD = 57362 -const ALERT = 57363 -const IF = 57364 -const FOR = 57365 -const WITH = 57366 -const SUMMARY = 57367 -const DESCRIPTION = 57368 - -var yyToknames = []string{ - "START_RULES", - "START_EXPRESSION", - "IDENTIFIER", - "STRING", - "DURATION", - "METRICNAME", - "NUMBER", - "PERMANENT", - "GROUP_OP", - "KEEPING_EXTRA", - "OFFSET", - "MATCH_OP", - "AGGR_OP", - "CMP_OP", - "ADDITIVE_OP", - "MULT_OP", - "MATCH_MOD", - "ALERT", - "IF", - "FOR", - "WITH", - "SUMMARY", - "DESCRIPTION", - "'='", -} -var yyStatenames = []string{} - -const yyEofCode = 1 -const yyErrCode = 2 -const yyMaxDepth = 200 - -//line parser.y:281 - -//line yacctab:1 -var yyExca = []int{ - -1, 1, - 1, -1, - -2, 0, - -1, 4, - 1, 1, - -2, 10, -} - -const yyNprod = 56 -const yyPrivate = 57344 - -var yyTokenNames []string -var yyStates []string - -const yyLast = 159 - -var yyAct = []int{ - - 78, 61, 83, 58, 55, 54, 31, 48, 6, 25, - 20, 21, 23, 21, 10, 56, 64, 14, 12, 10, - 56, 19, 14, 12, 11, 19, 13, 19, 92, 11, - 113, 13, 22, 20, 21, 57, 8, 32, 109, 7, - 53, 8, 77, 65, 7, 67, 68, 101, 19, 22, - 20, 21, 70, 69, 10, 98, 30, 14, 12, 22, - 20, 21, 94, 95, 11, 19, 13, 87, 85, 92, - 96, 99, 86, 84, 76, 19, 8, 66, 60, 7, - 29, 88, 90, 89, 24, 93, 22, 20, 21, 22, - 20, 21, 92, 100, 91, 75, 82, 74, 103, 73, - 43, 42, 19, 44, 43, 19, 26, 108, 62, 47, - 111, 28, 80, 51, 114, 110, 38, 105, 63, 46, - 18, 107, 39, 9, 49, 59, 32, 33, 35, 50, - 17, 14, 106, 72, 37, 115, 112, 104, 40, 41, - 34, 71, 79, 84, 102, 26, 36, 2, 3, 15, - 5, 4, 1, 45, 97, 16, 27, 81, 52, -} -var yyPact = []int{ - - 143, -1000, -1000, 48, 109, -1000, 72, 48, 139, 83, - 49, 25, -1000, 117, -1000, -1000, 122, 140, -1000, 126, - 107, 107, 107, 69, 74, -1000, 92, 110, 100, 8, - 48, 112, 47, -1000, 80, -1000, 96, -18, 48, 46, - 48, 48, -1000, 139, 110, 134, -1000, -1000, -1000, 125, - -1000, 70, 65, -1000, -1000, 72, -1000, 42, 11, -1000, - 136, 85, 67, 48, 110, -6, 136, -12, -8, -1000, - -1000, -1000, -1000, -1000, -1000, 13, 114, 48, 62, -1000, - 48, 33, -1000, -1000, 43, 32, -1000, 39, -1000, 112, - 15, -1000, 138, 72, -1000, 137, 130, 93, 124, 101, - -1000, -1000, -1000, -1000, -1000, 80, -1000, 7, 90, 136, - 129, -2, 88, -1000, 128, -1000, -} -var yyPgo = []int{ - - 0, 158, 0, 6, 2, 157, 1, 9, 84, 156, - 116, 4, 5, 155, 3, 154, 123, 153, 7, 152, - 151, 150, 149, -} -var yyR1 = []int{ - - 0, 19, 19, 20, 20, 21, 22, 22, 15, 15, - 13, 13, 16, 16, 6, 6, 6, 5, 5, 4, - 9, 9, 9, 8, 8, 7, 17, 17, 18, 18, - 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, - 11, 11, 11, 14, 14, 10, 10, 10, 3, 3, - 2, 2, 1, 1, 12, 12, -} -var yyR2 = []int{ - - 0, 2, 2, 0, 2, 1, 5, 11, 0, 2, - 0, 1, 1, 1, 0, 3, 2, 1, 3, 3, - 0, 2, 3, 1, 3, 3, 1, 1, 0, 2, - 3, 4, 3, 4, 3, 5, 6, 6, 4, 4, - 4, 1, 2, 0, 1, 0, 4, 8, 0, 4, - 1, 3, 1, 3, 1, 1, -} -var yyChk = []int{ - - -1000, -19, 4, 5, -20, -21, -11, 31, 28, -16, - 6, 16, 10, 18, 9, -22, -13, 21, 11, 33, - 18, 19, 17, -11, -8, -7, 6, -9, 28, 31, - 31, -3, 12, 10, -16, 6, 6, 8, -10, 15, - -10, -10, 32, 30, 29, -17, 27, 17, -18, 14, - 29, -8, -1, 32, -12, -11, 7, -11, -14, 13, - 31, -6, 28, 22, 34, -11, 31, -11, -11, -7, - -18, 7, 8, 29, 32, 30, 32, 31, -2, 6, - 27, -5, 29, -4, 6, -11, -18, -2, -12, -3, - -11, 32, 30, -11, 29, 30, 27, -15, 23, 32, - -14, 32, 6, -4, 7, 24, 8, 20, -6, 31, - 25, -2, 7, 32, 26, 7, -} -var yyDef = []int{ - - 0, -2, 3, 0, -2, 2, 5, 0, 0, 20, - 13, 48, 41, 0, 12, 4, 0, 0, 11, 0, - 45, 45, 45, 0, 0, 23, 0, 28, 0, 0, - 0, 43, 0, 42, 14, 13, 0, 0, 0, 0, - 0, 0, 30, 0, 28, 0, 26, 27, 32, 0, - 21, 0, 0, 34, 52, 54, 55, 0, 0, 44, - 0, 0, 0, 0, 28, 38, 0, 39, 40, 24, - 31, 25, 29, 22, 33, 0, 48, 0, 0, 50, - 0, 0, 16, 17, 0, 8, 35, 0, 53, 43, - 0, 49, 0, 6, 15, 0, 0, 0, 0, 46, - 36, 37, 51, 18, 19, 14, 9, 0, 0, 0, - 0, 0, 0, 47, 0, 7, -} -var yyTok1 = []int{ - - 1, 3, 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, - 31, 32, 3, 3, 30, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, - 3, 27, 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, - 3, 33, 3, 34, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 28, 3, 29, -} -var yyTok2 = []int{ - - 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, - 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, - 22, 23, 24, 25, 26, -} -var yyTok3 = []int{ - 0, -} - -//line yaccpar:1 - -/* parser for yacc output */ - -var yyDebug = 0 - -type yyLexer interface { - Lex(lval *yySymType) int - Error(s string) -} - -const yyFlag = -1000 - -func yyTokname(c int) string { - // 4 is TOKSTART above - if c >= 4 && c-4 < len(yyToknames) { - if yyToknames[c-4] != "" { - return yyToknames[c-4] - } - } - return __yyfmt__.Sprintf("tok-%v", c) -} - -func yyStatname(s int) string { - if s >= 0 && s < len(yyStatenames) { - if yyStatenames[s] != "" { - return yyStatenames[s] - } - } - return __yyfmt__.Sprintf("state-%v", s) -} - -func yylex1(lex yyLexer, lval *yySymType) int { - c := 0 - char := lex.Lex(lval) - if char <= 0 { - c = yyTok1[0] - goto out - } - if char < len(yyTok1) { - c = yyTok1[char] - goto out - } - if char >= yyPrivate { - if char < yyPrivate+len(yyTok2) { - c = yyTok2[char-yyPrivate] - goto out - } - } - for i := 0; i < len(yyTok3); i += 2 { - c = yyTok3[i+0] - if c == char { - c = yyTok3[i+1] - goto out - } - } - -out: - if c == 0 { - c = yyTok2[1] /* unknown char */ - } - if yyDebug >= 3 { - __yyfmt__.Printf("lex %s(%d)\n", yyTokname(c), uint(char)) - } - return c -} - -func yyParse(yylex yyLexer) int { - var yyn int - var yylval yySymType - var yyVAL yySymType - yyS := make([]yySymType, yyMaxDepth) - - Nerrs := 0 /* number of errors */ - Errflag := 0 /* error recovery flag */ - yystate := 0 - yychar := -1 - yyp := -1 - goto yystack - -ret0: - return 0 - -ret1: - return 1 - -yystack: - /* put a state and value onto the stack */ - if yyDebug >= 4 { - __yyfmt__.Printf("char %v in %v\n", yyTokname(yychar), yyStatname(yystate)) - } - - yyp++ - if yyp >= len(yyS) { - nyys := make([]yySymType, len(yyS)*2) - copy(nyys, yyS) - yyS = nyys - } - yyS[yyp] = yyVAL - yyS[yyp].yys = yystate - -yynewstate: - yyn = yyPact[yystate] - if yyn <= yyFlag { - goto yydefault /* simple state */ - } - if yychar < 0 { - yychar = yylex1(yylex, &yylval) - } - yyn += yychar - if yyn < 0 || yyn >= yyLast { - goto yydefault - } - yyn = yyAct[yyn] - if yyChk[yyn] == yychar { /* valid shift */ - yychar = -1 - yyVAL = yylval - yystate = yyn - if Errflag > 0 { - Errflag-- - } - goto yystack - } - -yydefault: - /* default state action */ - yyn = yyDef[yystate] - if yyn == -2 { - if yychar < 0 { - yychar = yylex1(yylex, &yylval) - } - - /* look through exception table */ - xi := 0 - for { - if yyExca[xi+0] == -1 && yyExca[xi+1] == yystate { - break - } - xi += 2 - } - for xi += 2; ; xi += 2 { - yyn = yyExca[xi+0] - if yyn < 0 || yyn == yychar { - break - } - } - yyn = yyExca[xi+1] - if yyn < 0 { - goto ret0 - } - } - if yyn == 0 { - /* error ... attempt to resume parsing */ - switch Errflag { - case 0: /* brand new error */ - yylex.Error("syntax error") - Nerrs++ - if yyDebug >= 1 { - __yyfmt__.Printf("%s", yyStatname(yystate)) - __yyfmt__.Printf(" saw %s\n", yyTokname(yychar)) - } - fallthrough - - case 1, 2: /* incompletely recovered error ... try again */ - Errflag = 3 - - /* find a state where "error" is a legal shift action */ - for yyp >= 0 { - yyn = yyPact[yyS[yyp].yys] + yyErrCode - if yyn >= 0 && yyn < yyLast { - yystate = yyAct[yyn] /* simulate a shift of "error" */ - if yyChk[yystate] == yyErrCode { - goto yystack - } - } - - /* the current p has no shift on "error", pop stack */ - if yyDebug >= 2 { - __yyfmt__.Printf("error recovery pops state %d\n", yyS[yyp].yys) - } - yyp-- - } - /* there is no state on the stack with an error shift ... abort */ - goto ret1 - - case 3: /* no shift yet; clobber input char */ - if yyDebug >= 2 { - __yyfmt__.Printf("error recovery discards %s\n", yyTokname(yychar)) - } - if yychar == yyEofCode { - goto ret1 - } - yychar = -1 - goto yynewstate /* try again in the same state */ - } - } - - /* reduction by production yyn */ - if yyDebug >= 2 { - __yyfmt__.Printf("reduce %v in:\n\t%v\n", yyn, yyStatname(yystate)) - } - - yynt := yyn - yypt := yyp - _ = yypt // guard against "declared and not used" - - yyp -= yyR2[yyn] - yyVAL = yyS[yyp+1] - - /* consult goto table to find next state */ - yyn = yyR1[yyn] - yyg := yyPgo[yyn] - yyj := yyg + yyS[yyp].yys + 1 - - if yyj >= yyLast { - yystate = yyAct[yyg] - } else { - yystate = yyAct[yyj] - if yyChk[yystate] != -yyn { - yystate = yyAct[yyg] - } - } - // dummy call; replaced with literal code - switch yynt { - - case 5: - //line parser.y:76 - { - yylex.(*RulesLexer).parsedExpr = yyS[yypt-0].ruleNode - } - case 6: - //line parser.y:81 - { - rule, err := CreateRecordingRule(yyS[yypt-3].str, yyS[yypt-2].labelSet, yyS[yypt-0].ruleNode, yyS[yypt-4].boolean) - if err != nil { - yylex.Error(err.Error()) - return 1 - } - yylex.(*RulesLexer).parsedRules = append(yylex.(*RulesLexer).parsedRules, rule) - } - case 7: - //line parser.y:87 - { - rule, err := CreateAlertingRule(yyS[yypt-9].str, yyS[yypt-7].ruleNode, yyS[yypt-6].str, yyS[yypt-4].labelSet, yyS[yypt-2].str, yyS[yypt-0].str) - if err != nil { - yylex.Error(err.Error()) - return 1 - } - yylex.(*RulesLexer).parsedRules = append(yylex.(*RulesLexer).parsedRules, rule) - } - case 8: - //line parser.y:95 - { - yyVAL.str = "0s" - } - case 9: - //line parser.y:97 - { - yyVAL.str = yyS[yypt-0].str - } - case 10: - //line parser.y:101 - { - yyVAL.boolean = false - } - case 11: - //line parser.y:103 - { - yyVAL.boolean = true - } - case 12: - //line parser.y:107 - { - yyVAL.str = yyS[yypt-0].str - } - case 13: - //line parser.y:109 - { - yyVAL.str = yyS[yypt-0].str - } - case 14: - //line parser.y:113 - { - yyVAL.labelSet = clientmodel.LabelSet{} - } - case 15: - //line parser.y:115 - { - yyVAL.labelSet = yyS[yypt-1].labelSet - } - case 16: - //line parser.y:117 - { - yyVAL.labelSet = clientmodel.LabelSet{} - } - case 17: - //line parser.y:120 - { - yyVAL.labelSet = yyS[yypt-0].labelSet - } - case 18: - //line parser.y:122 - { - for k, v := range yyS[yypt-0].labelSet { - yyVAL.labelSet[k] = v - } - } - case 19: - //line parser.y:126 - { - yyVAL.labelSet = clientmodel.LabelSet{clientmodel.LabelName(yyS[yypt-2].str): clientmodel.LabelValue(yyS[yypt-0].str)} - } - case 20: - //line parser.y:130 - { - yyVAL.labelMatchers = metric.LabelMatchers{} - } - case 21: - //line parser.y:132 - { - yyVAL.labelMatchers = metric.LabelMatchers{} - } - case 22: - //line parser.y:134 - { - yyVAL.labelMatchers = yyS[yypt-1].labelMatchers - } - case 23: - //line parser.y:138 - { - yyVAL.labelMatchers = metric.LabelMatchers{yyS[yypt-0].labelMatcher} - } - case 24: - //line parser.y:140 - { - yyVAL.labelMatchers = append(yyVAL.labelMatchers, yyS[yypt-0].labelMatcher) - } - case 25: - //line parser.y:144 - { - var err error - yyVAL.labelMatcher, err = newLabelMatcher(yyS[yypt-1].str, clientmodel.LabelName(yyS[yypt-2].str), clientmodel.LabelValue(yyS[yypt-0].str)) - if err != nil { - yylex.Error(err.Error()) - return 1 - } - } - case 26: - //line parser.y:152 - { - yyVAL.str = "=" - } - case 27: - //line parser.y:154 - { - yyVAL.str = yyS[yypt-0].str - } - case 28: - //line parser.y:158 - { - yyVAL.str = "0s" - } - case 29: - //line parser.y:160 - { - yyVAL.str = yyS[yypt-0].str - } - case 30: - //line parser.y:164 - { - yyVAL.ruleNode = yyS[yypt-1].ruleNode - } - case 31: - //line parser.y:166 - { - var err error - yyVAL.ruleNode, err = NewVectorSelector(yyS[yypt-2].labelMatchers, yyS[yypt-0].str) - if err != nil { - yylex.Error(err.Error()) - return 1 - } - } - case 32: - //line parser.y:172 - { - var err error - m, err := metric.NewLabelMatcher(metric.Equal, clientmodel.MetricNameLabel, clientmodel.LabelValue(yyS[yypt-2].str)) - if err != nil { - yylex.Error(err.Error()) - return 1 - } - yyS[yypt-1].labelMatchers = append(yyS[yypt-1].labelMatchers, m) - yyVAL.ruleNode, err = NewVectorSelector(yyS[yypt-1].labelMatchers, yyS[yypt-0].str) - if err != nil { - yylex.Error(err.Error()) - return 1 - } - } - case 33: - //line parser.y:181 - { - var err error - yyVAL.ruleNode, err = NewFunctionCall(yyS[yypt-3].str, yyS[yypt-1].ruleNodeSlice) - if err != nil { - yylex.Error(err.Error()) - return 1 - } - } - case 34: - //line parser.y:187 - { - var err error - yyVAL.ruleNode, err = NewFunctionCall(yyS[yypt-2].str, []ast.Node{}) - if err != nil { - yylex.Error(err.Error()) - return 1 - } - } - case 35: - //line parser.y:193 - { - var err error - yyVAL.ruleNode, err = NewMatrixSelector(yyS[yypt-4].ruleNode, yyS[yypt-2].str, yyS[yypt-0].str) - if err != nil { - yylex.Error(err.Error()) - return 1 - } - } - case 36: - //line parser.y:199 - { - var err error - yyVAL.ruleNode, err = NewVectorAggregation(yyS[yypt-5].str, yyS[yypt-3].ruleNode, yyS[yypt-1].labelNameSlice, yyS[yypt-0].boolean) - if err != nil { - yylex.Error(err.Error()) - return 1 - } - } - case 37: - //line parser.y:205 - { - var err error - yyVAL.ruleNode, err = NewVectorAggregation(yyS[yypt-5].str, yyS[yypt-1].ruleNode, yyS[yypt-4].labelNameSlice, yyS[yypt-3].boolean) - if err != nil { - yylex.Error(err.Error()) - return 1 - } - } - case 38: - //line parser.y:213 - { - var err error - yyVAL.ruleNode, err = NewArithExpr(yyS[yypt-2].str, yyS[yypt-3].ruleNode, yyS[yypt-0].ruleNode, yyS[yypt-1].vectorMatching) - if err != nil { - yylex.Error(err.Error()) - return 1 - } - } - case 39: - //line parser.y:219 - { - var err error - yyVAL.ruleNode, err = NewArithExpr(yyS[yypt-2].str, yyS[yypt-3].ruleNode, yyS[yypt-0].ruleNode, yyS[yypt-1].vectorMatching) - if err != nil { - yylex.Error(err.Error()) - return 1 - } - } - case 40: - //line parser.y:225 - { - var err error - yyVAL.ruleNode, err = NewArithExpr(yyS[yypt-2].str, yyS[yypt-3].ruleNode, yyS[yypt-0].ruleNode, yyS[yypt-1].vectorMatching) - if err != nil { - yylex.Error(err.Error()) - return 1 - } - } - case 41: - //line parser.y:231 - { - yyVAL.ruleNode = NewScalarLiteral(yyS[yypt-0].num, "+") - } - case 42: - //line parser.y:233 - { - yyVAL.ruleNode = NewScalarLiteral(yyS[yypt-0].num, yyS[yypt-1].str) - } - case 43: - //line parser.y:237 - { - yyVAL.boolean = false - } - case 44: - //line parser.y:239 - { - yyVAL.boolean = true - } - case 45: - //line parser.y:243 - { - yyVAL.vectorMatching = nil - } - case 46: - //line parser.y:245 - { - var err error - yyVAL.vectorMatching, err = newVectorMatching("", yyS[yypt-1].labelNameSlice, nil) - if err != nil { - yylex.Error(err.Error()) - return 1 - } - } - case 47: - //line parser.y:251 - { - var err error - yyVAL.vectorMatching, err = newVectorMatching(yyS[yypt-3].str, yyS[yypt-5].labelNameSlice, yyS[yypt-1].labelNameSlice) - if err != nil { - yylex.Error(err.Error()) - return 1 - } - } - case 48: - //line parser.y:259 - { - yyVAL.labelNameSlice = clientmodel.LabelNames{} - } - case 49: - //line parser.y:261 - { - yyVAL.labelNameSlice = yyS[yypt-1].labelNameSlice - } - case 50: - //line parser.y:265 - { - yyVAL.labelNameSlice = clientmodel.LabelNames{clientmodel.LabelName(yyS[yypt-0].str)} - } - case 51: - //line parser.y:267 - { - yyVAL.labelNameSlice = append(yyVAL.labelNameSlice, clientmodel.LabelName(yyS[yypt-0].str)) - } - case 52: - //line parser.y:271 - { - yyVAL.ruleNodeSlice = []ast.Node{yyS[yypt-0].ruleNode} - } - case 53: - //line parser.y:273 - { - yyVAL.ruleNodeSlice = append(yyVAL.ruleNodeSlice, yyS[yypt-0].ruleNode) - } - case 54: - //line parser.y:277 - { - yyVAL.ruleNode = yyS[yypt-0].ruleNode - } - case 55: - //line parser.y:279 - { - yyVAL.ruleNode = ast.NewStringLiteral(yyS[yypt-0].str) - } - } - goto yystack /* stack new state and value */ -} diff --git a/rules/recording.go b/rules/recording.go index e69356689..2402fa84f 100644 --- a/rules/recording.go +++ b/rules/recording.go @@ -20,30 +20,32 @@ import ( clientmodel "github.com/prometheus/client_golang/model" - "github.com/prometheus/prometheus/rules/ast" - "github.com/prometheus/prometheus/stats" - "github.com/prometheus/prometheus/storage/local" + "github.com/prometheus/prometheus/promql" + "github.com/prometheus/prometheus/utility" ) // A RecordingRule records its vector expression into new timeseries. type RecordingRule struct { - name string - vector ast.VectorNode - labels clientmodel.LabelSet - permanent bool + name string + vector promql.Expr + labels clientmodel.LabelSet } // Name returns the rule name. func (rule RecordingRule) Name() string { return rule.name } // EvalRaw returns the raw value of the rule expression. -func (rule RecordingRule) EvalRaw(timestamp clientmodel.Timestamp, storage local.Storage) (ast.Vector, error) { - return ast.EvalVectorInstant(rule.vector, timestamp, storage, stats.NewTimerGroup()) +func (rule RecordingRule) EvalRaw(timestamp clientmodel.Timestamp, engine *promql.Engine) (promql.Vector, error) { + query, err := engine.NewInstantQuery(rule.vector.String(), timestamp) + if err != nil { + return nil, err + } + return query.Exec().Vector() } // Eval evaluates the rule and then overrides the metric names and labels accordingly. -func (rule RecordingRule) Eval(timestamp clientmodel.Timestamp, storage local.Storage) (ast.Vector, error) { - vector, err := rule.EvalRaw(timestamp, storage) +func (rule RecordingRule) Eval(timestamp clientmodel.Timestamp, engine *promql.Engine) (promql.Vector, error) { + vector, err := rule.EvalRaw(timestamp, engine) if err != nil { return nil, err } @@ -63,8 +65,8 @@ func (rule RecordingRule) Eval(timestamp clientmodel.Timestamp, storage local.St return vector, nil } -// ToDotGraph returns the text representation of a dot graph. -func (rule RecordingRule) ToDotGraph() string { +// DotGraph returns the text representation of a dot graph. +func (rule RecordingRule) DotGraph() string { graph := fmt.Sprintf( `digraph "Rules" { %#p[shape="box",label="%s = "]; @@ -73,7 +75,7 @@ func (rule RecordingRule) ToDotGraph() string { }`, &rule, rule.name, &rule, reflect.ValueOf(rule.vector).Pointer(), - rule.vector.NodeTreeToDotGraph(), + rule.vector.DotGraph(), ) return graph } @@ -87,9 +89,9 @@ func (rule RecordingRule) HTMLSnippet() template.HTML { ruleExpr := rule.vector.String() return template.HTML(fmt.Sprintf( `%s%s = %s`, - GraphLinkForExpression(rule.name), + utility.GraphLinkForExpression(rule.name), rule.name, rule.labels, - GraphLinkForExpression(ruleExpr), + utility.GraphLinkForExpression(ruleExpr), ruleExpr)) } diff --git a/rules/rules.go b/rules/rules.go index 61d774e99..9485bc533 100644 --- a/rules/rules.go +++ b/rules/rules.go @@ -18,8 +18,7 @@ import ( clientmodel "github.com/prometheus/client_golang/model" - "github.com/prometheus/prometheus/rules/ast" - "github.com/prometheus/prometheus/storage/local" + "github.com/prometheus/prometheus/promql" ) // A Rule encapsulates a vector expression which is evaluated at a specified @@ -29,11 +28,11 @@ type Rule interface { Name() string // EvalRaw evaluates the rule's vector expression without triggering any // other actions, like recording or alerting. - EvalRaw(timestamp clientmodel.Timestamp, storage local.Storage) (ast.Vector, error) + EvalRaw(clientmodel.Timestamp, *promql.Engine) (promql.Vector, error) // Eval evaluates the rule, including any associated recording or alerting actions. - Eval(timestamp clientmodel.Timestamp, storage local.Storage) (ast.Vector, error) - // ToDotGraph returns a Graphviz dot graph of the rule. - ToDotGraph() string + Eval(clientmodel.Timestamp, *promql.Engine) (promql.Vector, error) + // DotGraph returns a Graphviz dot graph of the rule. + DotGraph() string // String returns a human-readable string representation of the rule. String() string // HTMLSnippet returns a human-readable string representation of the rule, diff --git a/rules/rules_test.go b/rules/rules_test.go index 5966c21ec..113d200b9 100644 --- a/rules/rules_test.go +++ b/rules/rules_test.go @@ -15,41 +15,63 @@ package rules import ( "fmt" - "math" - "path" - "regexp" - "strconv" "strings" "testing" "time" clientmodel "github.com/prometheus/client_golang/model" - "github.com/prometheus/prometheus/rules/ast" - "github.com/prometheus/prometheus/stats" + "github.com/prometheus/prometheus/promql" "github.com/prometheus/prometheus/storage/local" "github.com/prometheus/prometheus/storage/metric" - "github.com/prometheus/prometheus/utility/test" ) var ( - testEvalTime = testStartTime.Add(testSampleInterval * 10) - fixturesPath = "fixtures" - - reSample = regexp.MustCompile(`^(.*)(?: \=\>|:) (\-?\d+\.?\d*(?:e-?\d*)?|[+-]Inf|NaN) \@\[(\d+)\]$`) - minNormal = math.Float64frombits(0x0010000000000000) // The smallest positive normal value of type float64. + testSampleInterval = time.Duration(5) * time.Minute + testStartTime = clientmodel.Timestamp(0) ) -const ( - epsilon = 0.000001 // Relative error allowed for sample values. -) - -func annotateWithTime(lines []string, timestamp clientmodel.Timestamp) []string { - annotatedLines := []string{} - for _, line := range lines { - annotatedLines = append(annotatedLines, fmt.Sprintf(line, timestamp)) +func getTestValueStream(startVal clientmodel.SampleValue, endVal clientmodel.SampleValue, stepVal clientmodel.SampleValue, startTime clientmodel.Timestamp) (resultValues metric.Values) { + currentTime := startTime + for currentVal := startVal; currentVal <= endVal; currentVal += stepVal { + sample := metric.SamplePair{ + Value: currentVal, + Timestamp: currentTime, + } + resultValues = append(resultValues, sample) + currentTime = currentTime.Add(testSampleInterval) } - return annotatedLines + return resultValues +} + +func getTestVectorFromTestMatrix(matrix promql.Matrix) promql.Vector { + vector := promql.Vector{} + for _, sampleStream := range matrix { + lastSample := sampleStream.Values[len(sampleStream.Values)-1] + vector = append(vector, &promql.Sample{ + Metric: sampleStream.Metric, + Value: lastSample.Value, + Timestamp: lastSample.Timestamp, + }) + } + return vector +} + +func storeMatrix(storage local.Storage, matrix promql.Matrix) { + pendingSamples := clientmodel.Samples{} + for _, sampleStream := range matrix { + for _, sample := range sampleStream.Values { + pendingSamples = append(pendingSamples, &clientmodel.Sample{ + Metric: sampleStream.Metric.Metric, + Value: sample.Value, + Timestamp: sample.Timestamp, + }) + } + } + for _, s := range pendingSamples { + storage.Append(s) + } + storage.WaitForIndexing() } func vectorComparisonString(expected []string, actual []string) string { @@ -63,1604 +85,59 @@ func vectorComparisonString(expected []string, actual []string) string { separator) } -// samplesAlmostEqual returns true if the two sample lines only differ by a -// small relative error in their sample value. -func samplesAlmostEqual(a, b string) bool { - if a == b { - // Fast path if strings are equal. - return true +func annotateWithTime(lines []string, timestamp clientmodel.Timestamp) []string { + annotatedLines := []string{} + for _, line := range lines { + annotatedLines = append(annotatedLines, fmt.Sprintf(line, timestamp)) } - aMatches := reSample.FindStringSubmatch(a) - if aMatches == nil { - panic(fmt.Errorf("sample %q did not match regular expression", a)) - } - bMatches := reSample.FindStringSubmatch(b) - if bMatches == nil { - panic(fmt.Errorf("sample %q did not match regular expression", b)) - } - if aMatches[1] != bMatches[1] { - return false // Labels don't match. - } - if aMatches[3] != bMatches[3] { - return false // Timestamps don't match. - } - // If we are here, we have the diff in the floats. - // We have to check if they are almost equal. - aVal, err := strconv.ParseFloat(aMatches[2], 64) - if err != nil { - panic(err) - } - bVal, err := strconv.ParseFloat(bMatches[2], 64) - if err != nil { - panic(err) - } - - // Cf. http://floating-point-gui.de/errors/comparison/ - if aVal == bVal { - return true - } - - diff := math.Abs(aVal - bVal) - - if aVal == 0 || bVal == 0 || diff < minNormal { - return diff < epsilon*minNormal - } - return diff/(math.Abs(aVal)+math.Abs(bVal)) < epsilon + return annotatedLines } -func newTestStorage(t testing.TB) (storage local.Storage, closer test.Closer) { - storage, closer = local.NewTestStorage(t, 1) - storeMatrix(storage, testMatrix) - return storage, closer -} - -func TestExpressions(t *testing.T) { - // Labels in expected output need to be alphabetically sorted. - expressionTests := []struct { - expr string - output []string - shouldFail bool - checkOrder bool - }{ - { - expr: `SUM(http_requests)`, - output: []string{`{} => 3600 @[%v]`}, - }, { - expr: `SUM(http_requests{instance="0"}) BY(job)`, - output: []string{ - `{job="api-server"} => 400 @[%v]`, - `{job="app-server"} => 1200 @[%v]`, - }, - }, { - expr: `SUM(http_requests{instance="0"}) BY(job) KEEPING_EXTRA`, - output: []string{ - `{instance="0", job="api-server"} => 400 @[%v]`, - `{instance="0", job="app-server"} => 1200 @[%v]`, - }, - }, { - expr: `SUM(http_requests) BY (job)`, - output: []string{ - `{job="api-server"} => 1000 @[%v]`, - `{job="app-server"} => 2600 @[%v]`, - }, - }, { - // Non-existent labels mentioned in BY-clauses shouldn't propagate to output. - expr: `SUM(http_requests) BY (job, nonexistent)`, - output: []string{ - `{job="api-server"} => 1000 @[%v]`, - `{job="app-server"} => 2600 @[%v]`, - }, - }, { - expr: ` - // Test comment. - SUM(http_requests) BY /* comments shouldn't - have any effect */ (job) // another comment`, - output: []string{ - `{job="api-server"} => 1000 @[%v]`, - `{job="app-server"} => 2600 @[%v]`, - }, - }, { - expr: `COUNT(http_requests) BY (job)`, - output: []string{ - `{job="api-server"} => 4 @[%v]`, - `{job="app-server"} => 4 @[%v]`, - }, - }, { - expr: `SUM(http_requests) BY (job, group)`, - output: []string{ - `{group="canary", job="api-server"} => 700 @[%v]`, - `{group="canary", job="app-server"} => 1500 @[%v]`, - `{group="production", job="api-server"} => 300 @[%v]`, - `{group="production", job="app-server"} => 1100 @[%v]`, - }, - }, { - expr: `AVG(http_requests) BY (job)`, - output: []string{ - `{job="api-server"} => 250 @[%v]`, - `{job="app-server"} => 650 @[%v]`, - }, - }, { - expr: `MIN(http_requests) BY (job)`, - output: []string{ - `{job="api-server"} => 100 @[%v]`, - `{job="app-server"} => 500 @[%v]`, - }, - }, { - expr: `MAX(http_requests) BY (job)`, - output: []string{ - `{job="api-server"} => 400 @[%v]`, - `{job="app-server"} => 800 @[%v]`, - }, - }, { - expr: `SUM(http_requests) BY (job) - COUNT(http_requests) BY (job)`, - output: []string{ - `{job="api-server"} => 996 @[%v]`, - `{job="app-server"} => 2596 @[%v]`, - }, - }, { - expr: `2 - SUM(http_requests) BY (job)`, - output: []string{ - `{job="api-server"} => -998 @[%v]`, - `{job="app-server"} => -2598 @[%v]`, - }, - }, { - expr: `1000 / SUM(http_requests) BY (job)`, - output: []string{ - `{job="api-server"} => 1 @[%v]`, - `{job="app-server"} => 0.38461538461538464 @[%v]`, - }, - }, { - expr: `SUM(http_requests) BY (job) - 2`, - output: []string{ - `{job="api-server"} => 998 @[%v]`, - `{job="app-server"} => 2598 @[%v]`, - }, - }, { - expr: `SUM(http_requests) BY (job) % 3`, - output: []string{ - `{job="api-server"} => 1 @[%v]`, - `{job="app-server"} => 2 @[%v]`, - }, - }, { - expr: `SUM(http_requests) BY (job) / 0`, - output: []string{ - `{job="api-server"} => +Inf @[%v]`, - `{job="app-server"} => +Inf @[%v]`, - }, - }, { - expr: `SUM(http_requests) BY (job) > 1000`, - output: []string{ - `{job="app-server"} => 2600 @[%v]`, - }, - }, { - expr: `1000 < SUM(http_requests) BY (job)`, - output: []string{ - `{job="app-server"} => 1000 @[%v]`, - }, - }, { - expr: `SUM(http_requests) BY (job) <= 1000`, - output: []string{ - `{job="api-server"} => 1000 @[%v]`, - }, - }, { - expr: `SUM(http_requests) BY (job) != 1000`, - output: []string{ - `{job="app-server"} => 2600 @[%v]`, - }, - }, { - expr: `SUM(http_requests) BY (job) == 1000`, - output: []string{ - `{job="api-server"} => 1000 @[%v]`, - }, - }, { - expr: `SUM(http_requests) BY (job) + SUM(http_requests) BY (job)`, - output: []string{ - `{job="api-server"} => 2000 @[%v]`, - `{job="app-server"} => 5200 @[%v]`, - }, - }, { - expr: `http_requests{job="api-server", group="canary"}`, - output: []string{ - `http_requests{group="canary", instance="0", job="api-server"} => 300 @[%v]`, - `http_requests{group="canary", instance="1", job="api-server"} => 400 @[%v]`, - }, - }, { - expr: `http_requests{job="api-server", group="canary"} + rate(http_requests{job="api-server"}[5m]) * 5 * 60`, - output: []string{ - `{group="canary", instance="0", job="api-server"} => 330 @[%v]`, - `{group="canary", instance="1", job="api-server"} => 440 @[%v]`, - }, - }, { - expr: `rate(http_requests[25m]) * 25 * 60`, - output: []string{ - `{group="canary", instance="0", job="api-server"} => 150 @[%v]`, - `{group="canary", instance="0", job="app-server"} => 350 @[%v]`, - `{group="canary", instance="1", job="api-server"} => 200 @[%v]`, - `{group="canary", instance="1", job="app-server"} => 400 @[%v]`, - `{group="production", instance="0", job="api-server"} => 50 @[%v]`, - `{group="production", instance="0", job="app-server"} => 249.99999999999997 @[%v]`, - `{group="production", instance="1", job="api-server"} => 100 @[%v]`, - `{group="production", instance="1", job="app-server"} => 300 @[%v]`, - }, - }, { - expr: `delta(http_requests[25m], 1)`, - output: []string{ - `{group="canary", instance="0", job="api-server"} => 150 @[%v]`, - `{group="canary", instance="0", job="app-server"} => 350 @[%v]`, - `{group="canary", instance="1", job="api-server"} => 200 @[%v]`, - `{group="canary", instance="1", job="app-server"} => 400 @[%v]`, - `{group="production", instance="0", job="api-server"} => 50 @[%v]`, - `{group="production", instance="0", job="app-server"} => 250 @[%v]`, - `{group="production", instance="1", job="api-server"} => 100 @[%v]`, - `{group="production", instance="1", job="app-server"} => 300 @[%v]`, - }, - }, { - expr: `sort(http_requests)`, - output: []string{ - `http_requests{group="production", instance="0", job="api-server"} => 100 @[%v]`, - `http_requests{group="production", instance="1", job="api-server"} => 200 @[%v]`, - `http_requests{group="canary", instance="0", job="api-server"} => 300 @[%v]`, - `http_requests{group="canary", instance="1", job="api-server"} => 400 @[%v]`, - `http_requests{group="production", instance="0", job="app-server"} => 500 @[%v]`, - `http_requests{group="production", instance="1", job="app-server"} => 600 @[%v]`, - `http_requests{group="canary", instance="0", job="app-server"} => 700 @[%v]`, - `http_requests{group="canary", instance="1", job="app-server"} => 800 @[%v]`, - }, - checkOrder: true, - }, { - expr: `sort_desc(http_requests)`, - output: []string{ - `http_requests{group="canary", instance="1", job="app-server"} => 800 @[%v]`, - `http_requests{group="canary", instance="0", job="app-server"} => 700 @[%v]`, - `http_requests{group="production", instance="1", job="app-server"} => 600 @[%v]`, - `http_requests{group="production", instance="0", job="app-server"} => 500 @[%v]`, - `http_requests{group="canary", instance="1", job="api-server"} => 400 @[%v]`, - `http_requests{group="canary", instance="0", job="api-server"} => 300 @[%v]`, - `http_requests{group="production", instance="1", job="api-server"} => 200 @[%v]`, - `http_requests{group="production", instance="0", job="api-server"} => 100 @[%v]`, - }, - checkOrder: true, - }, { - expr: `topk(3, http_requests)`, - output: []string{ - `http_requests{group="canary", instance="1", job="app-server"} => 800 @[%v]`, - `http_requests{group="canary", instance="0", job="app-server"} => 700 @[%v]`, - `http_requests{group="production", instance="1", job="app-server"} => 600 @[%v]`, - }, - checkOrder: true, - }, { - expr: `topk(5, http_requests{group="canary",job="app-server"})`, - output: []string{ - `http_requests{group="canary", instance="1", job="app-server"} => 800 @[%v]`, - `http_requests{group="canary", instance="0", job="app-server"} => 700 @[%v]`, - }, - checkOrder: true, - }, { - expr: `bottomk(3, http_requests)`, - output: []string{ - `http_requests{group="production", instance="0", job="api-server"} => 100 @[%v]`, - `http_requests{group="production", instance="1", job="api-server"} => 200 @[%v]`, - `http_requests{group="canary", instance="0", job="api-server"} => 300 @[%v]`, - }, - checkOrder: true, - }, { - expr: `bottomk(5, http_requests{group="canary",job="app-server"})`, - output: []string{ - `http_requests{group="canary", instance="0", job="app-server"} => 700 @[%v]`, - `http_requests{group="canary", instance="1", job="app-server"} => 800 @[%v]`, - }, - checkOrder: true, - }, { - // Single-letter label names and values. - expr: `x{y="testvalue"}`, - output: []string{ - `x{y="testvalue"} => 100 @[%v]`, - }, - }, { - // Lower-cased aggregation operators should work too. - expr: `sum(http_requests) by (job) + min(http_requests) by (job) + max(http_requests) by (job) + avg(http_requests) by (job)`, - output: []string{ - `{job="app-server"} => 4550 @[%v]`, - `{job="api-server"} => 1750 @[%v]`, - }, - }, { - // Deltas should be adjusted for target interval vs. samples under target interval. - expr: `delta(http_requests{group="canary", instance="1", job="app-server"}[18m])`, - output: []string{`{group="canary", instance="1", job="app-server"} => 288 @[%v]`}, - }, { - // Deltas should perform the same operation when 2nd argument is 0. - expr: `delta(http_requests{group="canary", instance="1", job="app-server"}[18m], 0)`, - output: []string{`{group="canary", instance="1", job="app-server"} => 288 @[%v]`}, - }, { - // Rates should calculate per-second rates. - expr: `rate(http_requests{group="canary", instance="1", job="app-server"}[60m])`, - output: []string{`{group="canary", instance="1", job="app-server"} => 0.26666666666666666 @[%v]`}, - }, { - // Deriv should return the same as rate in simple cases. - expr: `deriv(http_requests{group="canary", instance="1", job="app-server"}[60m])`, - output: []string{`{group="canary", instance="1", job="app-server"} => 0.26666666666666666 @[%v]`}, - }, { - // Counter resets at in the middle of range are handled correctly by rate(). - expr: `rate(testcounter_reset_middle[60m])`, - output: []string{`{} => 0.03 @[%v]`}, - }, { - // Counter resets at end of range are ignored by rate(). - expr: `rate(testcounter_reset_end[5m])`, - output: []string{`{} => 0 @[%v]`}, - }, { - // Deriv should return correct result. - expr: `deriv(testcounter_reset_middle[100m])`, - output: []string{`{} => 0.010606060606060607 @[%v]`}, - }, { - // count_scalar for a non-empty vector should return scalar element count. - expr: `count_scalar(http_requests)`, - output: []string{`scalar: 8 @[%v]`}, - }, { - // count_scalar for an empty vector should return scalar 0. - expr: `count_scalar(nonexistent)`, - output: []string{`scalar: 0 @[%v]`}, - }, { - // Empty expressions shouldn't parse. - expr: ``, - shouldFail: true, - }, { - // Interval durations can't be in quotes. - expr: `http_requests["1m"]`, - shouldFail: true, - }, { - // Binop arguments need to be scalar or vector. - expr: `http_requests - http_requests[1m]`, - shouldFail: true, - }, { - expr: `http_requests{group!="canary"}`, - output: []string{ - `http_requests{group="production", instance="1", job="app-server"} => 600 @[%v]`, - `http_requests{group="production", instance="0", job="app-server"} => 500 @[%v]`, - `http_requests{group="production", instance="1", job="api-server"} => 200 @[%v]`, - `http_requests{group="production", instance="0", job="api-server"} => 100 @[%v]`, - }, - }, { - expr: `http_requests{job=~"server",group!="canary"}`, - output: []string{ - `http_requests{group="production", instance="1", job="app-server"} => 600 @[%v]`, - `http_requests{group="production", instance="0", job="app-server"} => 500 @[%v]`, - `http_requests{group="production", instance="1", job="api-server"} => 200 @[%v]`, - `http_requests{group="production", instance="0", job="api-server"} => 100 @[%v]`, - }, - }, { - expr: `http_requests{job!~"api",group!="canary"}`, - output: []string{ - `http_requests{group="production", instance="1", job="app-server"} => 600 @[%v]`, - `http_requests{group="production", instance="0", job="app-server"} => 500 @[%v]`, - }, - }, { - expr: `count_scalar(http_requests{job=~"^server$"})`, - output: []string{`scalar: 0 @[%v]`}, - }, { - expr: `http_requests{group="production",job=~"^api"}`, - output: []string{ - `http_requests{group="production", instance="0", job="api-server"} => 100 @[%v]`, - `http_requests{group="production", instance="1", job="api-server"} => 200 @[%v]`, - }, - }, - { - expr: `abs(-1 * http_requests{group="production",job="api-server"})`, - output: []string{ - `{group="production", instance="0", job="api-server"} => 100 @[%v]`, - `{group="production", instance="1", job="api-server"} => 200 @[%v]`, - }, - }, - { - expr: `floor(0.004 * http_requests{group="production",job="api-server"})`, - output: []string{ - `{group="production", instance="0", job="api-server"} => 0 @[%v]`, - `{group="production", instance="1", job="api-server"} => 0 @[%v]`, - }, - }, - { - expr: `ceil(0.004 * http_requests{group="production",job="api-server"})`, - output: []string{ - `{group="production", instance="0", job="api-server"} => 1 @[%v]`, - `{group="production", instance="1", job="api-server"} => 1 @[%v]`, - }, - }, - { - expr: `round(0.004 * http_requests{group="production",job="api-server"})`, - output: []string{ - `{group="production", instance="0", job="api-server"} => 0 @[%v]`, - `{group="production", instance="1", job="api-server"} => 1 @[%v]`, - }, - }, - { // Round should correctly handle negative numbers. - expr: `round(-1 * (0.004 * http_requests{group="production",job="api-server"}))`, - output: []string{ - `{group="production", instance="0", job="api-server"} => 0 @[%v]`, - `{group="production", instance="1", job="api-server"} => -1 @[%v]`, - }, - }, - { // Round should round half up. - expr: `round(0.005 * http_requests{group="production",job="api-server"})`, - output: []string{ - `{group="production", instance="0", job="api-server"} => 1 @[%v]`, - `{group="production", instance="1", job="api-server"} => 1 @[%v]`, - }, - }, - { - expr: `round(-1 * (0.005 * http_requests{group="production",job="api-server"}))`, - output: []string{ - `{group="production", instance="0", job="api-server"} => 0 @[%v]`, - `{group="production", instance="1", job="api-server"} => -1 @[%v]`, - }, - }, - { - expr: `round(1 + 0.005 * http_requests{group="production",job="api-server"})`, - output: []string{ - `{group="production", instance="0", job="api-server"} => 2 @[%v]`, - `{group="production", instance="1", job="api-server"} => 2 @[%v]`, - }, - }, - { - expr: `round(-1 * (1 + 0.005 * http_requests{group="production",job="api-server"}))`, - output: []string{ - `{group="production", instance="0", job="api-server"} => -1 @[%v]`, - `{group="production", instance="1", job="api-server"} => -2 @[%v]`, - }, - }, - { // Round should accept the number to round nearest to. - expr: `round(0.0005 * http_requests{group="production",job="api-server"}, 0.1)`, - output: []string{ - `{group="production", instance="0", job="api-server"} => 0.1 @[%v]`, - `{group="production", instance="1", job="api-server"} => 0.1 @[%v]`, - }, - }, - { - expr: `round(2.1 + 0.0005 * http_requests{group="production",job="api-server"}, 0.1)`, - output: []string{ - `{group="production", instance="0", job="api-server"} => 2.2 @[%v]`, - `{group="production", instance="1", job="api-server"} => 2.2 @[%v]`, - }, - }, - { - expr: `round(5.2 + 0.0005 * http_requests{group="production",job="api-server"}, 0.1)`, - output: []string{ - `{group="production", instance="0", job="api-server"} => 5.3 @[%v]`, - `{group="production", instance="1", job="api-server"} => 5.3 @[%v]`, - }, - }, - { // Round should work correctly with negative numbers and multiple decimal places. - expr: `round(-1 * (5.2 + 0.0005 * http_requests{group="production",job="api-server"}), 0.1)`, - output: []string{ - `{group="production", instance="0", job="api-server"} => -5.2 @[%v]`, - `{group="production", instance="1", job="api-server"} => -5.3 @[%v]`, - }, - }, - { // Round should work correctly with big toNearests. - expr: `round(0.025 * http_requests{group="production",job="api-server"}, 5)`, - output: []string{ - `{group="production", instance="0", job="api-server"} => 5 @[%v]`, - `{group="production", instance="1", job="api-server"} => 5 @[%v]`, - }, - }, - { - expr: `round(0.045 * http_requests{group="production",job="api-server"}, 5)`, - output: []string{ - `{group="production", instance="0", job="api-server"} => 5 @[%v]`, - `{group="production", instance="1", job="api-server"} => 10 @[%v]`, - }, - }, - { - expr: `avg_over_time(http_requests{group="production",job="api-server"}[1h])`, - output: []string{ - `{group="production", instance="0", job="api-server"} => 50 @[%v]`, - `{group="production", instance="1", job="api-server"} => 100 @[%v]`, - }, - }, - { - expr: `count_over_time(http_requests{group="production",job="api-server"}[1h])`, - output: []string{ - `{group="production", instance="0", job="api-server"} => 11 @[%v]`, - `{group="production", instance="1", job="api-server"} => 11 @[%v]`, - }, - }, - { - expr: `max_over_time(http_requests{group="production",job="api-server"}[1h])`, - output: []string{ - `{group="production", instance="0", job="api-server"} => 100 @[%v]`, - `{group="production", instance="1", job="api-server"} => 200 @[%v]`, - }, - }, - { - expr: `min_over_time(http_requests{group="production",job="api-server"}[1h])`, - output: []string{ - `{group="production", instance="0", job="api-server"} => 0 @[%v]`, - `{group="production", instance="1", job="api-server"} => 0 @[%v]`, - }, - }, - { - expr: `sum_over_time(http_requests{group="production",job="api-server"}[1h])`, - output: []string{ - `{group="production", instance="0", job="api-server"} => 550 @[%v]`, - `{group="production", instance="1", job="api-server"} => 1100 @[%v]`, - }, - }, - { - expr: `time()`, - output: []string{`scalar: 3000 @[%v]`}, - }, - { - expr: `drop_common_labels(http_requests{group="production",job="api-server"})`, - output: []string{ - `http_requests{instance="0"} => 100 @[%v]`, - `http_requests{instance="1"} => 200 @[%v]`, - }, - }, - { - expr: `{` + string(clientmodel.MetricNameLabel) + `=~".*"}`, - output: []string{ - `http_requests{group="canary", instance="0", job="api-server"} => 300 @[%v]`, - `http_requests{group="canary", instance="0", job="app-server"} => 700 @[%v]`, - `http_requests{group="canary", instance="1", job="api-server"} => 400 @[%v]`, - `http_requests{group="canary", instance="1", job="app-server"} => 800 @[%v]`, - `http_requests{group="production", instance="0", job="api-server"} => 100 @[%v]`, - `http_requests{group="production", instance="0", job="app-server"} => 500 @[%v]`, - `http_requests{group="production", instance="1", job="api-server"} => 200 @[%v]`, - `http_requests{group="production", instance="1", job="app-server"} => 600 @[%v]`, - `testcounter_reset_end => 0 @[%v]`, - `testcounter_reset_middle => 50 @[%v]`, - `x{y="testvalue"} => 100 @[%v]`, - `label_grouping_test{a="a", b="abb"} => 200 @[%v]`, - `label_grouping_test{a="aa", b="bb"} => 100 @[%v]`, - `testhistogram_bucket{le="0.1", start="positive"} => 50 @[%v]`, - `testhistogram_bucket{le=".2", start="positive"} => 70 @[%v]`, - `testhistogram_bucket{le="1e0", start="positive"} => 110 @[%v]`, - `testhistogram_bucket{le="+Inf", start="positive"} => 120 @[%v]`, - `testhistogram_bucket{le="-.2", start="negative"} => 10 @[%v]`, - `testhistogram_bucket{le="-0.1", start="negative"} => 20 @[%v]`, - `testhistogram_bucket{le="0.3", start="negative"} => 20 @[%v]`, - `testhistogram_bucket{le="+Inf", start="negative"} => 30 @[%v]`, - `request_duration_seconds_bucket{instance="ins1", job="job1", le="0.1"} => 10 @[%v]`, - `request_duration_seconds_bucket{instance="ins1", job="job1", le="0.2"} => 30 @[%v]`, - `request_duration_seconds_bucket{instance="ins1", job="job1", le="+Inf"} => 40 @[%v]`, - `request_duration_seconds_bucket{instance="ins2", job="job1", le="0.1"} => 20 @[%v]`, - `request_duration_seconds_bucket{instance="ins2", job="job1", le="0.2"} => 50 @[%v]`, - `request_duration_seconds_bucket{instance="ins2", job="job1", le="+Inf"} => 60 @[%v]`, - `request_duration_seconds_bucket{instance="ins1", job="job2", le="0.1"} => 30 @[%v]`, - `request_duration_seconds_bucket{instance="ins1", job="job2", le="0.2"} => 40 @[%v]`, - `request_duration_seconds_bucket{instance="ins1", job="job2", le="+Inf"} => 60 @[%v]`, - `request_duration_seconds_bucket{instance="ins2", job="job2", le="0.1"} => 40 @[%v]`, - `request_duration_seconds_bucket{instance="ins2", job="job2", le="0.2"} => 70 @[%v]`, - `request_duration_seconds_bucket{instance="ins2", job="job2", le="+Inf"} => 90 @[%v]`, - `vector_matching_a{l="x"} => 10 @[%v]`, - `vector_matching_a{l="y"} => 20 @[%v]`, - `vector_matching_b{l="x"} => 40 @[%v]`, - `cpu_count{instance="1", type="smp"} => 200 @[%v]`, - `cpu_count{instance="0", type="smp"} => 100 @[%v]`, - `cpu_count{instance="0", type="numa"} => 300 @[%v]`, - }, - }, - { - expr: `{job=~"server", job!~"api"}`, - output: []string{ - `http_requests{group="canary", instance="0", job="app-server"} => 700 @[%v]`, - `http_requests{group="canary", instance="1", job="app-server"} => 800 @[%v]`, - `http_requests{group="production", instance="0", job="app-server"} => 500 @[%v]`, - `http_requests{group="production", instance="1", job="app-server"} => 600 @[%v]`, - }, - }, - { - // Test alternative "by"-clause order. - expr: `sum by (group) (http_requests{job="api-server"})`, - output: []string{ - `{group="canary"} => 700 @[%v]`, - `{group="production"} => 300 @[%v]`, - }, - }, - { - // Test alternative "by"-clause order with "keeping_extra". - expr: `sum by (group) keeping_extra (http_requests{job="api-server"})`, - output: []string{ - `{group="canary", job="api-server"} => 700 @[%v]`, - `{group="production", job="api-server"} => 300 @[%v]`, - }, - }, - { - // 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. - expr: `sum(sum by (group) keeping_extra (http_requests{job="api-server"})) by (job)`, - output: []string{ - `{job="api-server"} => 1000 @[%v]`, - }, - }, - { - expr: `http_requests{group="canary"} and http_requests{instance="0"}`, - output: []string{ - `http_requests{group="canary", instance="0", job="api-server"} => 300 @[%v]`, - `http_requests{group="canary", instance="0", job="app-server"} => 700 @[%v]`, - }, - }, - { - expr: `(http_requests{group="canary"} + 1) and http_requests{instance="0"}`, - output: []string{ - `{group="canary", instance="0", job="api-server"} => 301 @[%v]`, - `{group="canary", instance="0", job="app-server"} => 701 @[%v]`, - }, - }, - { - expr: `(http_requests{group="canary"} + 1) and on(instance, job) http_requests{instance="0", group="production"}`, - output: []string{ - `{group="canary", instance="0", job="api-server"} => 301 @[%v]`, - `{group="canary", instance="0", job="app-server"} => 701 @[%v]`, - }, - }, - { - expr: `(http_requests{group="canary"} + 1) and on(instance) http_requests{instance="0", group="production"}`, - output: []string{ - `{group="canary", instance="0", job="api-server"} => 301 @[%v]`, - `{group="canary", instance="0", job="app-server"} => 701 @[%v]`, - }, - }, - { - expr: `http_requests{group="canary"} or http_requests{group="production"}`, - output: []string{ - `http_requests{group="canary", instance="0", job="api-server"} => 300 @[%v]`, - `http_requests{group="canary", instance="0", job="app-server"} => 700 @[%v]`, - `http_requests{group="canary", instance="1", job="api-server"} => 400 @[%v]`, - `http_requests{group="canary", instance="1", job="app-server"} => 800 @[%v]`, - `http_requests{group="production", instance="0", job="api-server"} => 100 @[%v]`, - `http_requests{group="production", instance="0", job="app-server"} => 500 @[%v]`, - `http_requests{group="production", instance="1", job="api-server"} => 200 @[%v]`, - `http_requests{group="production", instance="1", job="app-server"} => 600 @[%v]`, - }, - }, - { - // On overlap the rhs samples must be dropped. - expr: `(http_requests{group="canary"} + 1) or http_requests{instance="1"}`, - output: []string{ - `{group="canary", instance="0", job="api-server"} => 301 @[%v]`, - `{group="canary", instance="0", job="app-server"} => 701 @[%v]`, - `{group="canary", instance="1", job="api-server"} => 401 @[%v]`, - `{group="canary", instance="1", job="app-server"} => 801 @[%v]`, - `http_requests{group="production", instance="1", job="api-server"} => 200 @[%v]`, - `http_requests{group="production", instance="1", job="app-server"} => 600 @[%v]`, - }, - }, - { - // Matching only on instance excludes everything that has instance=0/1 but includes - // entries without the instance label. - expr: `(http_requests{group="canary"} + 1) or on(instance) (http_requests or cpu_count or vector_matching_a)`, - output: []string{ - `{group="canary", instance="0", job="api-server"} => 301 @[%v]`, - `{group="canary", instance="0", job="app-server"} => 701 @[%v]`, - `{group="canary", instance="1", job="api-server"} => 401 @[%v]`, - `{group="canary", instance="1", job="app-server"} => 801 @[%v]`, - `vector_matching_a{l="x"} => 10 @[%v]`, - `vector_matching_a{l="y"} => 20 @[%v]`, - }, - }, - { - expr: `http_requests{group="canary"} / on(instance,job) http_requests{group="production"}`, - output: []string{ - `{instance="0", job="api-server"} => 3 @[%v]`, - `{instance="0", job="app-server"} => 1.4 @[%v]`, - `{instance="1", job="api-server"} => 2 @[%v]`, - `{instance="1", job="app-server"} => 1.3333333333333333 @[%v]`, - }, - }, - { - // Include labels must guarantee uniquely identifiable time series. - expr: `http_requests{group="production"} / on(instance) group_left(group) cpu_count{type="smp"}`, - output: []string{}, // Empty result returned on error (see TODOs). - }, - { - // Many-to-many matching is not allowed. - expr: `http_requests{group="production"} / on(instance) group_left(job,type) cpu_count`, - output: []string{}, // Empty result returned on error (see TODOs). - }, - { - // Many-to-one matching must be explicit. - expr: `http_requests{group="production"} / on(instance) cpu_count{type="smp"}`, - output: []string{}, // Empty result returned on error (see TODOs). - }, - { - expr: `http_requests{group="production"} / on(instance) group_left(job) cpu_count{type="smp"}`, - output: []string{ - `{instance="1", job="api-server"} => 1 @[%v]`, - `{instance="0", job="app-server"} => 5 @[%v]`, - `{instance="1", job="app-server"} => 3 @[%v]`, - `{instance="0", job="api-server"} => 1 @[%v]`, - }, - }, - { - // Ensure sidedness of grouping preserves operand sides. - expr: `cpu_count{type="smp"} / on(instance) group_right(job) http_requests{group="production"}`, - output: []string{ - `{instance="1", job="app-server"} => 0.3333333333333333 @[%v]`, - `{instance="0", job="app-server"} => 0.2 @[%v]`, - `{instance="1", job="api-server"} => 1 @[%v]`, - `{instance="0", job="api-server"} => 1 @[%v]`, - }, - }, - { - // Include labels from both sides. - expr: `http_requests{group="production"} / on(instance) group_left(job) cpu_count{type="smp"}`, - output: []string{ - `{instance="1", job="api-server"} => 1 @[%v]`, - `{instance="0", job="app-server"} => 5 @[%v]`, - `{instance="1", job="app-server"} => 3 @[%v]`, - `{instance="0", job="api-server"} => 1 @[%v]`, - }, - }, - { - expr: `http_requests{group="production"} < on(instance,job) http_requests{group="canary"}`, - output: []string{ - `{instance="1", job="app-server"} => 600 @[%v]`, - `{instance="0", job="app-server"} => 500 @[%v]`, - `{instance="1", job="api-server"} => 200 @[%v]`, - `{instance="0", job="api-server"} => 100 @[%v]`, - }, - }, - { - expr: `http_requests{group="production"} > on(instance,job) http_requests{group="canary"}`, - output: []string{}, - }, - { - expr: `http_requests{group="production"} == on(instance,job) http_requests{group="canary"}`, - output: []string{}, - }, - { - expr: `http_requests > on(instance) group_left(group,job) cpu_count{type="smp"}`, - output: []string{ - `{group="canary", instance="0", job="app-server"} => 700 @[%v]`, - `{group="canary", instance="1", job="app-server"} => 800 @[%v]`, - `{group="canary", instance="0", job="api-server"} => 300 @[%v]`, - `{group="canary", instance="1", job="api-server"} => 400 @[%v]`, - `{group="production", instance="0", job="app-server"} => 500 @[%v]`, - `{group="production", instance="1", job="app-server"} => 600 @[%v]`, - }, - }, - { - expr: `http_requests / on(instance) 3`, - shouldFail: true, - }, - { - expr: `3 / on(instance) http_requests_total`, - shouldFail: true, - }, - { - expr: `3 / on(instance) 3`, - shouldFail: true, - }, - { - // Missing label list for grouping mod. - expr: `http_requests{group="production"} / on(instance) group_left cpu_count{type="smp"}`, - shouldFail: true, - }, - { - // No group mod allowed for logical operations. - expr: `http_requests{group="production"} or on(instance) group_left(type) cpu_count{type="smp"}`, - shouldFail: true, - }, - { - // No group mod allowed for logical operations. - expr: `http_requests{group="production"} and on(instance) group_left(type) cpu_count{type="smp"}`, - shouldFail: true, - }, - { - // No duplicate use of label. - expr: `http_requests{group="production"} + on(instance) group_left(job,instance) cpu_count{type="smp"}`, - shouldFail: true, - }, - { - expr: `{l="x"} + on(__name__) {l="y"}`, - output: []string{ - `vector_matching_a => 30 @[%v]`, - }, - }, - { - expr: `absent(nonexistent)`, - output: []string{ - `{} => 1 @[%v]`, - }, - }, - { - expr: `absent(nonexistent{job="testjob", instance="testinstance", method=~".*"})`, - output: []string{ - `{instance="testinstance", job="testjob"} => 1 @[%v]`, - }, - }, - { - expr: `count_scalar(absent(http_requests))`, - output: []string{ - `scalar: 0 @[%v]`, - }, - }, - { - expr: `count_scalar(absent(sum(http_requests)))`, - output: []string{ - `scalar: 0 @[%v]`, - }, - }, - { - expr: `absent(sum(nonexistent{job="testjob", instance="testinstance"}))`, - output: []string{ - `{} => 1 @[%v]`, - }, - }, - { - expr: `http_requests{group="production",job="api-server"} offset 5m`, - output: []string{ - `http_requests{group="production", instance="0", job="api-server"} => 90 @[%v]`, - `http_requests{group="production", instance="1", job="api-server"} => 180 @[%v]`, - }, - }, - { - expr: `rate(http_requests{group="production",job="api-server"}[10m] offset 5m)`, - output: []string{ - `{group="production", instance="0", job="api-server"} => 0.03333333333333333 @[%v]`, - `{group="production", instance="1", job="api-server"} => 0.06666666666666667 @[%v]`, - }, - }, - { - expr: `rate(http_requests[10m]) offset 5m`, - shouldFail: true, - }, - { - expr: `sum(http_requests) offset 5m`, - shouldFail: true, - }, - // Regression test for missing separator byte in labelsToGroupingKey. - { - expr: `sum(label_grouping_test) by (a, b)`, - output: []string{ - `{a="a", b="abb"} => 200 @[%v]`, - `{a="aa", b="bb"} => 100 @[%v]`, - }, - }, - // Quantile too low. - { - expr: `histogram_quantile(-0.1, testhistogram_bucket)`, - output: []string{ - `{start="positive"} => -Inf @[%v]`, - `{start="negative"} => -Inf @[%v]`, - }, - }, - // Quantile too high. - { - expr: `histogram_quantile(1.01, testhistogram_bucket)`, - output: []string{ - `{start="positive"} => +Inf @[%v]`, - `{start="negative"} => +Inf @[%v]`, - }, - }, - // Quantile value in lowest bucket, which is positive. - { - expr: `histogram_quantile(0, testhistogram_bucket{start="positive"})`, - output: []string{ - `{start="positive"} => 0 @[%v]`, - }, - }, - // Quantile value in lowest bucket, which is negative. - { - expr: `histogram_quantile(0, testhistogram_bucket{start="negative"})`, - output: []string{ - `{start="negative"} => -0.2 @[%v]`, - }, - }, - // Quantile value in highest bucket. - { - expr: `histogram_quantile(1, testhistogram_bucket)`, - output: []string{ - `{start="positive"} => 1 @[%v]`, - `{start="negative"} => 0.3 @[%v]`, - }, - }, - // Finally some useful quantiles. - { - expr: `histogram_quantile(0.2, testhistogram_bucket)`, - output: []string{ - `{start="positive"} => 0.048 @[%v]`, - `{start="negative"} => -0.2 @[%v]`, - }, - }, - { - expr: `histogram_quantile(0.5, testhistogram_bucket)`, - output: []string{ - `{start="positive"} => 0.15 @[%v]`, - `{start="negative"} => -0.15 @[%v]`, - }, - }, - { - expr: `histogram_quantile(0.8, testhistogram_bucket)`, - output: []string{ - `{start="positive"} => 0.72 @[%v]`, - `{start="negative"} => 0.3 @[%v]`, - }, - }, - // More realistic with rates. - { - expr: `histogram_quantile(0.2, rate(testhistogram_bucket[5m]))`, - output: []string{ - `{start="positive"} => 0.048 @[%v]`, - `{start="negative"} => -0.2 @[%v]`, - }, - }, - { - expr: `histogram_quantile(0.5, rate(testhistogram_bucket[5m]))`, - output: []string{ - `{start="positive"} => 0.15 @[%v]`, - `{start="negative"} => -0.15 @[%v]`, - }, - }, - { - expr: `histogram_quantile(0.8, rate(testhistogram_bucket[5m]))`, - output: []string{ - `{start="positive"} => 0.72 @[%v]`, - `{start="negative"} => 0.3 @[%v]`, - }, - }, - // Aggregated histogram: Everything in one. - { - expr: `histogram_quantile(0.3, sum(rate(request_duration_seconds_bucket[5m])) by (le))`, - output: []string{ - `{} => 0.075 @[%v]`, - }, - }, - { - expr: `histogram_quantile(0.5, sum(rate(request_duration_seconds_bucket[5m])) by (le))`, - output: []string{ - `{} => 0.1277777777777778 @[%v]`, - }, - }, - // Aggregated histogram: Everything in one. Now with avg, which does not change anything. - { - expr: `histogram_quantile(0.3, avg(rate(request_duration_seconds_bucket[5m])) by (le))`, - output: []string{ - `{} => 0.075 @[%v]`, - }, - }, - { - expr: `histogram_quantile(0.5, avg(rate(request_duration_seconds_bucket[5m])) by (le))`, - output: []string{ - `{} => 0.12777777777777778 @[%v]`, - }, - }, - // Aggregated histogram: By job. - { - expr: `histogram_quantile(0.3, sum(rate(request_duration_seconds_bucket[5m])) by (le, instance))`, - output: []string{ - `{instance="ins1"} => 0.075 @[%v]`, - `{instance="ins2"} => 0.075 @[%v]`, - }, - }, - { - expr: `histogram_quantile(0.5, sum(rate(request_duration_seconds_bucket[5m])) by (le, instance))`, - output: []string{ - `{instance="ins1"} => 0.1333333333 @[%v]`, - `{instance="ins2"} => 0.125 @[%v]`, - }, - }, - // Aggregated histogram: By instance. - { - expr: `histogram_quantile(0.3, sum(rate(request_duration_seconds_bucket[5m])) by (le, job))`, - output: []string{ - `{job="job1"} => 0.1 @[%v]`, - `{job="job2"} => 0.0642857142857143 @[%v]`, - }, - }, - { - expr: `histogram_quantile(0.5, sum(rate(request_duration_seconds_bucket[5m])) by (le, job))`, - output: []string{ - `{job="job1"} => 0.14 @[%v]`, - `{job="job2"} => 0.1125 @[%v]`, - }, - }, - // Aggregated histogram: By job and instance. - { - expr: `histogram_quantile(0.3, sum(rate(request_duration_seconds_bucket[5m])) by (le, job, instance))`, - output: []string{ - `{instance="ins1", job="job1"} => 0.11 @[%v]`, - `{instance="ins2", job="job1"} => 0.09 @[%v]`, - `{instance="ins1", job="job2"} => 0.06 @[%v]`, - `{instance="ins2", job="job2"} => 0.0675 @[%v]`, - }, - }, - { - expr: `histogram_quantile(0.5, sum(rate(request_duration_seconds_bucket[5m])) by (le, job, instance))`, - output: []string{ - `{instance="ins1", job="job1"} => 0.15 @[%v]`, - `{instance="ins2", job="job1"} => 0.1333333333333333 @[%v]`, - `{instance="ins1", job="job2"} => 0.1 @[%v]`, - `{instance="ins2", job="job2"} => 0.1166666666666667 @[%v]`, - }, - }, - // The unaggregated histogram for comparison. Same result as the previous one. - { - expr: `histogram_quantile(0.3, rate(request_duration_seconds_bucket[5m]))`, - output: []string{ - `{instance="ins1", job="job1"} => 0.11 @[%v]`, - `{instance="ins2", job="job1"} => 0.09 @[%v]`, - `{instance="ins1", job="job2"} => 0.06 @[%v]`, - `{instance="ins2", job="job2"} => 0.0675 @[%v]`, - }, - }, - { - expr: `histogram_quantile(0.5, rate(request_duration_seconds_bucket[5m]))`, - output: []string{ - `{instance="ins1", job="job1"} => 0.15 @[%v]`, - `{instance="ins2", job="job1"} => 0.13333333333333333 @[%v]`, - `{instance="ins1", job="job2"} => 0.1 @[%v]`, - `{instance="ins2", job="job2"} => 0.11666666666666667 @[%v]`, - }, - }, - { - expr: `12.34e6`, - output: []string{`scalar: 12340000 @[%v]`}, - }, - { - expr: `12.34e+6`, - output: []string{`scalar: 12340000 @[%v]`}, - }, - { - expr: `12.34e-6`, - output: []string{`scalar: 0.00001234 @[%v]`}, - }, - { - expr: `1+1`, - output: []string{`scalar: 2 @[%v]`}, - }, - { - expr: `1-1`, - output: []string{`scalar: 0 @[%v]`}, - }, - { - expr: `1 - -1`, - output: []string{`scalar: 2 @[%v]`}, - }, - { - expr: `.2`, - output: []string{`scalar: 0.2 @[%v]`}, - }, - { - expr: `+0.2`, - output: []string{`scalar: 0.2 @[%v]`}, - }, - { - expr: `-0.2e-6`, - output: []string{`scalar: -0.0000002 @[%v]`}, - }, - { - expr: `+Inf`, - output: []string{`scalar: +Inf @[%v]`}, - }, - { - expr: `inF`, - output: []string{`scalar: +Inf @[%v]`}, - }, - { - expr: `-inf`, - output: []string{`scalar: -Inf @[%v]`}, - }, - { - expr: `NaN`, - output: []string{`scalar: NaN @[%v]`}, - }, - { - expr: `nan`, - output: []string{`scalar: NaN @[%v]`}, - }, - { - expr: `2.`, - output: []string{`scalar: 2 @[%v]`}, - }, - { - expr: `999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999999`, - output: []string{`scalar: +Inf @[%v]`}, - }, - { - expr: `1 / 0`, - output: []string{`scalar: +Inf @[%v]`}, - }, - { - expr: `-1 / 0`, - output: []string{`scalar: -Inf @[%v]`}, - }, - { - expr: `0 / 0`, - output: []string{`scalar: NaN @[%v]`}, - }, - { - expr: `1 % 0`, - output: []string{`scalar: NaN @[%v]`}, - }, - { - expr: `http_requests{group="canary", instance="0", job="api-server"} / 0`, - output: []string{ - `{group="canary", instance="0", job="api-server"} => +Inf @[%v]`, - }, - }, - { - expr: `-1 * http_requests{group="canary", instance="0", job="api-server"} / 0`, - output: []string{ - `{group="canary", instance="0", job="api-server"} => -Inf @[%v]`, - }, - }, - { - expr: `0 * http_requests{group="canary", instance="0", job="api-server"} / 0`, - output: []string{ - `{group="canary", instance="0", job="api-server"} => NaN @[%v]`, - }, - }, - { - expr: `0 * http_requests{group="canary", instance="0", job="api-server"} % 0`, - output: []string{ - `{group="canary", instance="0", job="api-server"} => NaN @[%v]`, - }, - }, - { - expr: `exp(vector_matching_a)`, - output: []string{ - `{l="x"} => 22026.465794806718 @[%v]`, - `{l="y"} => 485165195.4097903 @[%v]`, - }, - }, - { - expr: `exp(vector_matching_a - 10)`, - output: []string{ - `{l="y"} => 22026.465794806718 @[%v]`, - `{l="x"} => 1 @[%v]`, - }, - }, - { - expr: `exp(vector_matching_a - 20)`, - output: []string{ - `{l="x"} => 4.5399929762484854e-05 @[%v]`, - `{l="y"} => 1 @[%v]`, - }, - }, - { - expr: `ln(vector_matching_a)`, - output: []string{ - `{l="x"} => 2.302585092994046 @[%v]`, - `{l="y"} => 2.995732273553991 @[%v]`, - }, - }, - { - expr: `ln(vector_matching_a - 10)`, - output: []string{ - `{l="y"} => 2.302585092994046 @[%v]`, - `{l="x"} => -Inf @[%v]`, - }, - }, - { - expr: `ln(vector_matching_a - 20)`, - output: []string{ - `{l="y"} => -Inf @[%v]`, - `{l="x"} => NaN @[%v]`, - }, - }, - { - expr: `exp(ln(vector_matching_a))`, - output: []string{ - `{l="y"} => 20 @[%v]`, - `{l="x"} => 10 @[%v]`, - }, - }, - { - expr: `log2(vector_matching_a)`, - output: []string{ - `{l="x"} => 3.3219280948873626 @[%v]`, - `{l="y"} => 4.321928094887363 @[%v]`, - }, - }, - { - expr: `log2(vector_matching_a - 10)`, - output: []string{ - `{l="y"} => 3.3219280948873626 @[%v]`, - `{l="x"} => -Inf @[%v]`, - }, - }, - { - expr: `log2(vector_matching_a - 20)`, - output: []string{ - `{l="x"} => NaN @[%v]`, - `{l="y"} => -Inf @[%v]`, - }, - }, - { - expr: `log10(vector_matching_a)`, - output: []string{ - `{l="x"} => 1 @[%v]`, - `{l="y"} => 1.301029995663981 @[%v]`, - }, - }, - { - expr: `log10(vector_matching_a - 10)`, - output: []string{ - `{l="y"} => 1 @[%v]`, - `{l="x"} => -Inf @[%v]`, - }, - }, - { - expr: `log10(vector_matching_a - 20)`, - output: []string{ - `{l="x"} => NaN @[%v]`, - `{l="y"} => -Inf @[%v]`, - }, - }, - } - - storage, closer := newTestStorage(t) - defer closer.Close() - - for i, exprTest := range expressionTests { - expectedLines := annotateWithTime(exprTest.output, testEvalTime) - - testExpr, err := LoadExprFromString(exprTest.expr) - - if err != nil { - if exprTest.shouldFail { - continue - } - t.Errorf("%d. Error during parsing: %v", i, err) - t.Errorf("%d. Expression: %v", i, exprTest.expr) - } else { - if exprTest.shouldFail { - t.Errorf("%d. Test should fail, but didn't", i) - } - failed := false - - resultStr := ast.EvalToString(testExpr, testEvalTime, ast.Text, storage, stats.NewTimerGroup()) - resultLines := strings.Split(resultStr, "\n") - - if len(exprTest.output) == 0 && strings.Trim(resultStr, "\n") == "" { - // expected and received empty vector, everything is fine - continue - } else if len(exprTest.output) != len(resultLines) { - t.Errorf("%d. Number of samples in expected and actual output don't match", i) - failed = true - } - - if exprTest.checkOrder { - for j, expectedSample := range expectedLines { - if resultLines[j] != expectedSample { - t.Errorf("%d.%d. Expected sample '%v', got '%v'", i, j, resultLines[j], expectedSample) - failed = true - } - } - } else { - for j, expectedSample := range expectedLines { - found := false - for _, actualSample := range resultLines { - if samplesAlmostEqual(actualSample, expectedSample) { - found = true - } - } - if !found { - t.Errorf("%d.%d. Couldn't find expected sample in output: '%v'", i, j, expectedSample) - failed = true - } - } - } - - if failed { - t.Errorf("%d. Expression: %v\n%v", i, exprTest.expr, vectorComparisonString(expectedLines, resultLines)) - } - } - } -} - -func TestRangedEvaluationRegressions(t *testing.T) { - scenarios := []struct { - in ast.Matrix - out ast.Matrix - expr string - }{ - { - // Testing COWMetric behavior in drop_common_labels. - in: ast.Matrix{ - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "testmetric", - "testlabel": "1", - }, - }, - Values: metric.Values{ - { - Timestamp: testStartTime, - Value: 1, - }, - { - Timestamp: testStartTime.Add(time.Hour), - Value: 1, - }, - }, - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "testmetric", - "testlabel": "2", - }, - }, - Values: metric.Values{ - { - Timestamp: testStartTime.Add(time.Hour), - Value: 2, - }, - }, - }, - }, - out: ast.Matrix{ - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "testmetric", - }, - }, - Values: metric.Values{ - { - Timestamp: testStartTime, - Value: 1, - }, - }, - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "testmetric", - "testlabel": "1", - }, - }, - Values: metric.Values{ - { - Timestamp: testStartTime.Add(time.Hour), - Value: 1, - }, - }, - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "testmetric", - "testlabel": "2", - }, - }, - Values: metric.Values{ - { - Timestamp: testStartTime.Add(time.Hour), - Value: 2, - }, - }, - }, - }, - expr: "drop_common_labels(testmetric)", - }, - { - // Testing COWMetric behavior in vector aggregation. - in: ast.Matrix{ - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "testmetric", - "testlabel": "1", - }, - }, - Values: metric.Values{ - { - Timestamp: testStartTime, - Value: 1, - }, - { - Timestamp: testStartTime.Add(time.Hour), - Value: 1, - }, - }, - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "testmetric", - "testlabel": "2", - }, - }, - Values: metric.Values{ - { - Timestamp: testStartTime, - Value: 2, - }, - }, - }, - }, - out: ast.Matrix{ - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{}, - }, - Values: metric.Values{ - { - Timestamp: testStartTime, - Value: 3, - }, - }, - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - "testlabel": "1", - }, - }, - Values: metric.Values{ - { - Timestamp: testStartTime.Add(time.Hour), - Value: 1, - }, - }, - }, - }, - expr: "sum(testmetric) keeping_extra", - }, - { - // Testing metric fingerprint grouping behavior. - in: ast.Matrix{ - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "testmetric", - "aa": "bb", - }, - }, - Values: metric.Values{ - { - Timestamp: testStartTime, - Value: 1, - }, - }, - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "testmetric", - "a": "abb", - }, - }, - Values: metric.Values{ - { - Timestamp: testStartTime, - Value: 2, - }, - }, - }, - }, - out: ast.Matrix{ - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "testmetric", - "aa": "bb", - }, - }, - Values: metric.Values{ - { - Timestamp: testStartTime, - Value: 1, - }, - }, - }, - { - Metric: clientmodel.COWMetric{ - Metric: clientmodel.Metric{ - clientmodel.MetricNameLabel: "testmetric", - "a": "abb", - }, - }, - Values: metric.Values{ - { - Timestamp: testStartTime, - Value: 2, - }, - }, - }, - }, - expr: "testmetric", - }, - } - - for i, s := range scenarios { - storage, closer := local.NewTestStorage(t, 1) - storeMatrix(storage, s.in) - - expr, err := LoadExprFromString(s.expr) - if err != nil { - t.Fatalf("%d. Error parsing expression: %v", i, err) - } - - got, err := ast.EvalVectorRange( - expr.(ast.VectorNode), - testStartTime, - testStartTime.Add(time.Hour), - time.Hour, - storage, - stats.NewTimerGroup(), - ) - if err != nil { - t.Fatalf("%d. Error evaluating expression: %v", i, err) - } - - if got.String() != s.out.String() { - t.Fatalf("%d. Expression: %s\n\ngot:\n=====\n%v\n====\n\nwant:\n=====\n%v\n=====\n", i, s.expr, got.String(), s.out.String()) - } - - closer.Close() - } -} - -var ruleTests = []struct { - inputFile string - shouldFail bool - errContains string - numRecordingRules int - numAlertingRules int -}{ +var testMatrix = promql.Matrix{ { - inputFile: "empty.rules", - numRecordingRules: 0, - numAlertingRules: 0, - }, { - inputFile: "mixed.rules", - numRecordingRules: 2, - numAlertingRules: 2, + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "http_requests", + clientmodel.JobLabel: "api-server", + "instance": "0", + "group": "canary", + }, + }, + Values: getTestValueStream(0, 300, 30, testStartTime), }, { - inputFile: "syntax_error.rules", - shouldFail: true, - errContains: "Error parsing rules at line 5", + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "http_requests", + clientmodel.JobLabel: "api-server", + "instance": "1", + "group": "canary", + }, + }, + Values: getTestValueStream(0, 400, 40, testStartTime), }, { - inputFile: "non_vector.rules", - shouldFail: true, - errContains: "does not evaluate to vector type", + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "http_requests", + clientmodel.JobLabel: "app-server", + "instance": "0", + "group": "canary", + }, + }, + Values: getTestValueStream(0, 700, 70, testStartTime), + }, + { + Metric: clientmodel.COWMetric{ + Metric: clientmodel.Metric{ + clientmodel.MetricNameLabel: "http_requests", + clientmodel.JobLabel: "app-server", + "instance": "1", + "group": "canary", + }, + }, + Values: getTestValueStream(0, 800, 80, testStartTime), }, -} - -func TestRules(t *testing.T) { - for i, ruleTest := range ruleTests { - testRules, err := LoadRulesFromFile(path.Join(fixturesPath, ruleTest.inputFile)) - - if err != nil { - if !ruleTest.shouldFail { - t.Fatalf("%d. Error parsing rules file %v: %v", i, ruleTest.inputFile, err) - } else { - if !strings.Contains(err.Error(), ruleTest.errContains) { - t.Fatalf("%d. Expected error containing '%v', got: %v", i, ruleTest.errContains, err) - } - } - } else { - numRecordingRules := 0 - numAlertingRules := 0 - - for j, rule := range testRules { - switch rule.(type) { - case *RecordingRule: - numRecordingRules++ - case *AlertingRule: - numAlertingRules++ - default: - t.Fatalf("%d.%d. Unknown rule type!", i, j) - } - } - - if numRecordingRules != ruleTest.numRecordingRules { - t.Fatalf("%d. Expected %d recording rules, got %d", i, ruleTest.numRecordingRules, numRecordingRules) - } - if numAlertingRules != ruleTest.numAlertingRules { - t.Fatalf("%d. Expected %d alerting rules, got %d", i, ruleTest.numAlertingRules, numAlertingRules) - } - - // TODO(julius): add more complex checks on the parsed rules here. - } - } } func TestAlertingRule(t *testing.T) { @@ -1688,27 +165,34 @@ func TestAlertingRule(t *testing.T) { }, } - storage, closer := newTestStorage(t) + storage, closer := local.NewTestStorage(t, 1) defer closer.Close() - alertExpr, err := LoadExprFromString(`http_requests{group="canary", job="app-server"} < 100`) + storeMatrix(storage, testMatrix) + + engine := promql.NewEngine(storage) + defer engine.Stop() + + expr, err := promql.ParseExpr("test", `http_requests{group="canary", job="app-server"} < 100`) if err != nil { t.Fatalf("Unable to parse alert expression: %s", err) } - alertName := "HttpRequestRateLow" + alertLabels := clientmodel.LabelSet{ "severity": "critical", } - rule := NewAlertingRule(alertName, alertExpr.(ast.VectorNode), time.Minute, alertLabels, "summary", "description") + rule := NewAlertingRule("HttpRequestRateLow", expr, time.Minute, alertLabels, "summary", "description") - for i, expected := range evalOutputs { + for i, expectedLines := range evalOutputs { evalTime := testStartTime.Add(testSampleInterval * time.Duration(i)) - actual, err := rule.Eval(evalTime, storage) + + res, err := rule.Eval(evalTime, engine) if err != nil { t.Fatalf("Error during alerting rule evaluation: %s", err) } - actualLines := strings.Split(actual.String(), "\n") - expectedLines := annotateWithTime(expected, evalTime) + + actualLines := strings.Split(res.String(), "\n") + expectedLines := annotateWithTime(expectedLines, evalTime) if actualLines[0] == "" { actualLines = []string{} } diff --git a/templates/templates.go b/templates/templates.go index b07258482..aca5bff75 100644 --- a/templates/templates.go +++ b/templates/templates.go @@ -27,10 +27,8 @@ import ( clientmodel "github.com/prometheus/client_golang/model" - "github.com/prometheus/prometheus/rules" - "github.com/prometheus/prometheus/rules/ast" - "github.com/prometheus/prometheus/stats" - "github.com/prometheus/prometheus/storage/local" + "github.com/prometheus/prometheus/promql" + "github.com/prometheus/prometheus/utility" ) // A version of vector that's easier to use from templates. @@ -57,18 +55,17 @@ func (q queryResultByLabelSorter) Swap(i, j int) { q.results[i], q.results[j] = q.results[j], q.results[i] } -func query(q string, timestamp clientmodel.Timestamp, storage local.Storage) (queryResult, error) { - exprNode, err := rules.LoadExprFromString(q) +func query(q string, timestamp clientmodel.Timestamp, queryEngine *promql.Engine) (queryResult, error) { + query, err := queryEngine.NewInstantQuery(q, timestamp) if err != nil { return nil, err } - queryStats := stats.NewTimerGroup() - vector, err := ast.EvalToVector(exprNode, timestamp, storage, queryStats) + vector, err := query.Exec().Vector() if err != nil { return nil, err } - // ast.Vector is hard to work with in templates, so convert to + // promql.Vector is hard to work with in templates, so convert to // base data types. var result = make(queryResult, len(vector)) for n, v := range vector { @@ -92,14 +89,14 @@ type templateExpander struct { } // NewTemplateExpander returns a template expander ready to use. -func NewTemplateExpander(text string, name string, data interface{}, timestamp clientmodel.Timestamp, storage local.Storage, pathPrefix string) *templateExpander { +func NewTemplateExpander(text string, name string, data interface{}, timestamp clientmodel.Timestamp, queryEngine *promql.Engine, pathPrefix string) *templateExpander { return &templateExpander{ text: text, name: name, data: data, funcMap: text_template.FuncMap{ "query": func(q string) (queryResult, error) { - return query(q, timestamp, storage) + return query(q, timestamp, queryEngine) }, "first": func(v queryResult) (*sample, error) { if len(v) > 0 { @@ -132,8 +129,8 @@ func NewTemplateExpander(text string, name string, data interface{}, timestamp c }, "match": regexp.MatchString, "title": strings.Title, - "graphLink": rules.GraphLinkForExpression, - "tableLink": rules.TableLinkForExpression, + "graphLink": utility.GraphLinkForExpression, + "tableLink": utility.TableLinkForExpression, "sortByLabel": func(label string, v queryResult) queryResult { sorter := queryResultByLabelSorter{v[:], label} sort.Stable(sorter) @@ -219,7 +216,7 @@ func NewTemplateExpander(text string, name string, data interface{}, timestamp c return fmt.Sprintf("%.4g%ss", v, prefix) }, "pathPrefix": func() string { - return pathPrefix; + return pathPrefix }, }, } diff --git a/templates/templates_test.go b/templates/templates_test.go index f4c769b50..b65e237d7 100644 --- a/templates/templates_test.go +++ b/templates/templates_test.go @@ -19,6 +19,7 @@ import ( clientmodel "github.com/prometheus/client_golang/model" + "github.com/prometheus/prometheus/promql" "github.com/prometheus/prometheus/storage/local" ) @@ -175,10 +176,12 @@ func TestTemplateExpansion(t *testing.T) { }) storage.WaitForIndexing() + engine := promql.NewEngine(storage) + for i, s := range scenarios { var result string var err error - expander := NewTemplateExpander(s.text, "test", s.input, time, storage, "/") + expander := NewTemplateExpander(s.text, "test", s.input, time, engine, "/") if s.html { result, err = expander.ExpandHTML(nil) } else { diff --git a/tools/rule_checker/main.go b/tools/rule_checker/main.go index a4ecf90a5..f53991d5f 100644 --- a/tools/rule_checker/main.go +++ b/tools/rule_checker/main.go @@ -20,9 +20,10 @@ import ( "flag" "fmt" "io" + "io/ioutil" "os" - "github.com/prometheus/prometheus/rules" + "github.com/prometheus/prometheus/promql" ) var ( @@ -33,7 +34,12 @@ var ( // checkRules reads rules from in. Sucessfully read rules // are printed to out. func checkRules(filename string, in io.Reader, out io.Writer) error { - rules, err := rules.LoadRulesFromReader(in) + content, err := ioutil.ReadAll(in) + if err != nil { + return err + } + + rules, err := promql.ParseStmts(filename, string(content)) if err != nil { return err } diff --git a/utility/strconv.go b/utility/strconv.go index 5e2dd8d75..0ebfa1587 100644 --- a/utility/strconv.go +++ b/utility/strconv.go @@ -15,8 +15,10 @@ package utility import ( "fmt" + "net/url" "regexp" "strconv" + "strings" "time" ) @@ -80,3 +82,24 @@ func StringToDuration(durationStr string) (duration time.Duration, err error) { } return } + +// TableLinkForExpression creates an escaped relative link to the table view of +// the provided expression. +func TableLinkForExpression(expr string) string { + // url.QueryEscape percent-escapes everything except spaces, for which it + // uses "+". However, in the non-query part of a URI, only percent-escaped + // spaces are legal, so we need to manually replace "+" with "%20" after + // query-escaping the string. + // + // See also: + // http://stackoverflow.com/questions/1634271/url-encoding-the-space-character-or-20. + urlData := url.QueryEscape(fmt.Sprintf(`[{"expr":%q,"tab":1}]`, expr)) + return fmt.Sprintf("/graph#%s", strings.Replace(urlData, "+", "%20", -1)) +} + +// GraphLinkForExpression creates an escaped relative link to the graph view of +// the provided expression. +func GraphLinkForExpression(expr string) string { + urlData := url.QueryEscape(fmt.Sprintf(`[{"expr":%q,"tab":0}]`, expr)) + return fmt.Sprintf("/graph#%s", strings.Replace(urlData, "+", "%20", -1)) +} diff --git a/web/alerts.go b/web/alerts.go index 864e289fe..5744b074d 100644 --- a/web/alerts.go +++ b/web/alerts.go @@ -19,7 +19,6 @@ import ( "sync" "github.com/prometheus/prometheus/rules" - "github.com/prometheus/prometheus/rules/manager" ) // AlertStatus bundles alerting rules and the mapping of alert states to row @@ -47,9 +46,10 @@ func (s byAlertStateSorter) Swap(i, j int) { // AlertsHandler implements http.Handler. type AlertsHandler struct { - mutex sync.Mutex - RuleManager manager.RuleManager + RuleManager rules.RuleManager PathPrefix string + + mutex sync.Mutex } func (h *AlertsHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { diff --git a/web/api/api.go b/web/api/api.go index 90d1c1453..40da8d83e 100644 --- a/web/api/api.go +++ b/web/api/api.go @@ -20,14 +20,16 @@ import ( clientmodel "github.com/prometheus/client_golang/model" + "github.com/prometheus/prometheus/promql" "github.com/prometheus/prometheus/storage/local" "github.com/prometheus/prometheus/web/httputils" ) // MetricsService manages the /api HTTP endpoint. type MetricsService struct { - Now func() clientmodel.Timestamp - Storage local.Storage + Now func() clientmodel.Timestamp + Storage local.Storage + QueryEngine *promql.Engine } // RegisterHandler registers the handler for the various endpoints below /api. @@ -37,13 +39,13 @@ func (msrv *MetricsService) RegisterHandler(pathPrefix string) { Handler: http.HandlerFunc(h), } } - http.Handle(pathPrefix + "api/query", prometheus.InstrumentHandler( - pathPrefix + "api/query", handler(msrv.Query), + http.Handle(pathPrefix+"api/query", prometheus.InstrumentHandler( + pathPrefix+"api/query", handler(msrv.Query), )) - http.Handle(pathPrefix + "api/query_range", prometheus.InstrumentHandler( - pathPrefix + "api/query_range", handler(msrv.QueryRange), + http.Handle(pathPrefix+"api/query_range", prometheus.InstrumentHandler( + pathPrefix+"api/query_range", handler(msrv.QueryRange), )) - http.Handle(pathPrefix + "api/metrics", prometheus.InstrumentHandler( - pathPrefix + "api/metrics", handler(msrv.Metrics), + http.Handle(pathPrefix+"api/metrics", prometheus.InstrumentHandler( + pathPrefix+"api/metrics", handler(msrv.Metrics), )) } diff --git a/web/api/api_test.go b/web/api/api_test.go index e185fbb9d..44f52d8f0 100644 --- a/web/api/api_test.go +++ b/web/api/api_test.go @@ -23,6 +23,7 @@ import ( clientmodel "github.com/prometheus/client_golang/model" + "github.com/prometheus/prometheus/promql" "github.com/prometheus/prometheus/storage/local" ) @@ -51,7 +52,7 @@ func TestQuery(t *testing.T) { { queryStr: "", status: http.StatusOK, - bodyRe: "syntax error", + bodyRe: `{"type":"error","value":"query:1,1 no expression found in input","version":1}`, }, { queryStr: "expr=testmetric", @@ -76,7 +77,7 @@ func TestQuery(t *testing.T) { { queryStr: "expr=(badexpression", status: http.StatusOK, - bodyRe: "syntax error", + bodyRe: `{"type":"error","value":"query:1,15 unexpected unclosed left parenthesis in paren expression","version":1}`, }, } @@ -92,8 +93,9 @@ func TestQuery(t *testing.T) { storage.WaitForIndexing() api := MetricsService{ - Now: testNow, - Storage: storage, + Now: testNow, + Storage: storage, + QueryEngine: promql.NewEngine(storage), } api.RegisterHandler("/") diff --git a/web/api/query.go b/web/api/query.go index c4e8f4c81..ef8612851 100644 --- a/web/api/query.go +++ b/web/api/query.go @@ -26,9 +26,6 @@ import ( clientmodel "github.com/prometheus/client_golang/model" - "github.com/prometheus/prometheus/rules" - "github.com/prometheus/prometheus/rules/ast" - "github.com/prometheus/prometheus/stats" "github.com/prometheus/prometheus/web/httputils" ) @@ -41,9 +38,8 @@ func setAccessControlHeaders(w http.ResponseWriter) { } func httpJSONError(w http.ResponseWriter, err error, code int) { - w.Header().Set("Content-Type", "application/json") w.WriteHeader(code) - fmt.Fprintln(w, ast.ErrorToJSON(err)) + httputils.ErrorJSON(w, err) } func parseTimestampOrNow(t string, now clientmodel.Timestamp) (clientmodel.Timestamp, error) { @@ -80,16 +76,19 @@ func (serv MetricsService) Query(w http.ResponseWriter, r *http.Request) { return } - exprNode, err := rules.LoadExprFromString(expr) + query, err := serv.QueryEngine.NewInstantQuery(expr, timestamp) if err != nil { - fmt.Fprint(w, ast.ErrorToJSON(err)) + httpJSONError(w, err, http.StatusOK) return } + res := query.Exec() + if res.Err != nil { + httpJSONError(w, res.Err, http.StatusOK) + return + } + glog.V(1).Infof("Instant query: %s\nQuery stats:\n%s\n", expr, query.Stats()) - queryStats := stats.NewTimerGroup() - result := ast.EvalToString(exprNode, timestamp, ast.JSON, serv.Storage, queryStats) - glog.V(1).Infof("Instant query: %s\nQuery stats:\n%s\n", expr, queryStats) - fmt.Fprint(w, result) + httputils.RespondJSON(w, res.Value) } // QueryRange handles the /api/query_range endpoint. @@ -125,50 +124,31 @@ func (serv MetricsService) QueryRange(w http.ResponseWriter, r *http.Request) { end = serv.Now() } - exprNode, err := rules.LoadExprFromString(expr) - if err != nil { - fmt.Fprint(w, ast.ErrorToJSON(err)) - return - } - if exprNode.Type() != ast.VectorType { - fmt.Fprint(w, ast.ErrorToJSON(errors.New("expression does not evaluate to vector type"))) - return - } - // For safety, limit the number of returned points per timeseries. // This is sufficient for 60s resolution for a week or 1h resolution for a year. if duration/step > 11000 { - fmt.Fprint(w, ast.ErrorToJSON(errors.New("exceeded maximum resolution of 11,000 points per timeseries. Try decreasing the query resolution (?step=XX)"))) + err := errors.New("exceeded maximum resolution of 11,000 points per timeseries. Try decreasing the query resolution (?step=XX)") + httpJSONError(w, err, http.StatusBadRequest) return } // Align the start to step "tick" boundary. end = end.Add(-time.Duration(end.UnixNano() % int64(step))) + start := end.Add(-duration) - queryStats := stats.NewTimerGroup() - - matrix, err := ast.EvalVectorRange( - exprNode.(ast.VectorNode), - end.Add(-duration), - end, - step, - serv.Storage, - queryStats) + query, err := serv.QueryEngine.NewRangeQuery(expr, start, end, step) if err != nil { - fmt.Fprint(w, ast.ErrorToJSON(err)) + httpJSONError(w, err, http.StatusOK) + return + } + matrix, err := query.Exec().Matrix() + if err != nil { + httpJSONError(w, err, http.StatusOK) return } - sortTimer := queryStats.GetTimer(stats.ResultSortTime).Start() - sort.Sort(matrix) - sortTimer.Stop() - - jsonTimer := queryStats.GetTimer(stats.JSONEncodeTime).Start() - result := ast.TypedValueToJSON(matrix, "matrix") - jsonTimer.Stop() - - glog.V(1).Infof("Range query: %s\nQuery stats:\n%s\n", expr, queryStats) - fmt.Fprint(w, result) + glog.V(1).Infof("Range query: %s\nQuery stats:\n%s\n", expr, query.Stats()) + httputils.RespondJSON(w, matrix) } // Metrics handles the /api/metrics endpoint. diff --git a/web/consoles.go b/web/consoles.go index d8874aaae..9c2da52cf 100644 --- a/web/consoles.go +++ b/web/consoles.go @@ -22,7 +22,7 @@ import ( "path/filepath" clientmodel "github.com/prometheus/client_golang/model" - "github.com/prometheus/prometheus/storage/local" + "github.com/prometheus/prometheus/promql" "github.com/prometheus/prometheus/templates" ) @@ -33,8 +33,8 @@ var ( // ConsolesHandler implements http.Handler. type ConsolesHandler struct { - Storage local.Storage - PathPrefix string + QueryEngine *promql.Engine + PathPrefix string } func (h *ConsolesHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { @@ -70,7 +70,7 @@ func (h *ConsolesHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { Path: r.URL.Path, } - template := templates.NewTemplateExpander(string(text), "__console_"+r.URL.Path, data, clientmodel.Now(), h.Storage, h.PathPrefix) + template := templates.NewTemplateExpander(string(text), "__console_"+r.URL.Path, data, clientmodel.Now(), h.QueryEngine, h.PathPrefix) filenames, err := filepath.Glob(*consoleLibrariesPath + "/*.lib") if err != nil { http.Error(w, err.Error(), http.StatusInternalServerError) diff --git a/web/httputils/httputils.go b/web/httputils/httputils.go index d110de8ef..c0141d0ec 100644 --- a/web/httputils/httputils.go +++ b/web/httputils/httputils.go @@ -14,8 +14,12 @@ package httputils import ( + "encoding/json" + "io" "net/http" "net/url" + + "github.com/prometheus/prometheus/promql" ) // GetQueryParams calls r.ParseForm and returns r.Form. @@ -23,3 +27,39 @@ func GetQueryParams(r *http.Request) url.Values { r.ParseForm() return r.Form } + +var jsonFormatVersion = 1 + +// ErrorJSON writes the given error JSON-formatted to w. +func ErrorJSON(w io.Writer, err error) error { + data := struct { + Type string `json:"type"` + Value string `json:"value"` + Version int `json:"version"` + }{ + Type: "error", + Value: err.Error(), + Version: jsonFormatVersion, + } + enc := json.NewEncoder(w) + return enc.Encode(data) +} + +// RespondJSON converts the given data value to JSON and writes it to w. +func RespondJSON(w io.Writer, val promql.Value) error { + data := struct { + Type string `json:"type"` + Value interface{} `json:"value"` + Version int `json:"version"` + }{ + Type: val.Type().String(), + Value: val, + Version: jsonFormatVersion, + } + // TODO(fabxc): Adding MarshalJSON to promql.Values might be a good idea. + if sc, ok := val.(*promql.Scalar); ok { + data.Value = sc.Value + } + enc := json.NewEncoder(w) + return enc.Encode(data) +} diff --git a/web/status.go b/web/status.go index 2e905dd2b..4180874ed 100644 --- a/web/status.go +++ b/web/status.go @@ -19,7 +19,7 @@ import ( "time" "github.com/prometheus/prometheus/retrieval" - "github.com/prometheus/prometheus/rules/manager" + "github.com/prometheus/prometheus/rules" ) // PrometheusStatusHandler implements http.Handler. @@ -29,7 +29,7 @@ type PrometheusStatusHandler struct { BuildInfo map[string]string Config string Flags map[string]string - RuleManager manager.RuleManager + RuleManager rules.RuleManager TargetPools map[string]*retrieval.TargetPool Birth time.Time From 25cdff3527ae3bc382c02cbf33b1e4358e2a71f3 Mon Sep 17 00:00:00 2001 From: Fabian Reinartz Date: Wed, 29 Apr 2015 11:36:41 +0200 Subject: [PATCH 04/10] Remove `name` arg from `Parse*` functions, enhance parsing errors. --- promql/engine.go | 4 ++-- promql/lex.go | 12 +++++------ promql/lex_test.go | 12 +++++------ promql/parse.go | 43 ++++++++++++++++++++++++++++---------- promql/parse_test.go | 6 +++--- rules/rules_test.go | 2 +- tools/rule_checker/main.go | 2 +- web/api/api_test.go | 4 ++-- 8 files changed, 51 insertions(+), 34 deletions(-) diff --git a/promql/engine.go b/promql/engine.go index dc3714ab0..a149de0ad 100644 --- a/promql/engine.go +++ b/promql/engine.go @@ -294,7 +294,7 @@ func (ng *Engine) Stop() { // NewQuery returns a new query of the given query string. func (ng *Engine) NewQuery(qs string) (Query, error) { - stmts, err := ParseStmts("query", qs) + stmts, err := ParseStmts(qs) if err != nil { return nil, err } @@ -325,7 +325,7 @@ func (ng *Engine) NewInstantQuery(es string, ts clientmodel.Timestamp) (Query, e // NewRangeQuery returns an evaluation query for the given time range and with // the resolution set by the interval. func (ng *Engine) NewRangeQuery(qs string, start, end clientmodel.Timestamp, interval time.Duration) (Query, error) { - expr, err := ParseExpr("query", qs) + expr, err := ParseExpr(qs) if err != nil { return nil, err } diff --git a/promql/lex.go b/promql/lex.go index 4a27bbcc5..d147b39f2 100644 --- a/promql/lex.go +++ b/promql/lex.go @@ -222,7 +222,6 @@ type Pos int // lexer holds the state of the scanner. type lexer struct { - name string // The name of the input; used only for error reports. input string // The string being scanned. state stateFn // The next lexing function to enter. pos Pos // Current position in the input. @@ -298,12 +297,12 @@ func (l *lexer) lineNumber() int { // linePosition reports at which character in the current line // we are on. -func (l *lexer) linePosition() Pos { - lb := Pos(strings.LastIndex(l.input[:l.lastPos], "\n")) +func (l *lexer) linePosition() int { + lb := strings.LastIndex(l.input[:l.lastPos], "\n") if lb == -1 { - return 1 + l.lastPos + return 1 + int(l.lastPos) } - return 1 + l.lastPos - lb + return 1 + int(l.lastPos) - lb } // errorf returns an error token and terminates the scan by passing @@ -321,9 +320,8 @@ func (l *lexer) nextItem() item { } // lex creates a new scanner for the input string. -func lex(name, input string) *lexer { +func lex(input string) *lexer { l := &lexer{ - name: name, input: input, items: make(chan item), } diff --git a/promql/lex_test.go b/promql/lex_test.go index c9c45b58c..8d3425659 100644 --- a/promql/lex_test.go +++ b/promql/lex_test.go @@ -14,7 +14,6 @@ package promql import ( - "fmt" "reflect" "testing" ) @@ -328,8 +327,7 @@ var tests = []struct { // for the parser to avoid duplicated effort. func TestLexer(t *testing.T) { for i, test := range tests { - tn := fmt.Sprintf("test.%d \"%s\"", i, test.input) - l := lex(tn, test.input) + l := lex(test.input) out := []item{} for it := range l.items { @@ -339,20 +337,20 @@ func TestLexer(t *testing.T) { lastItem := out[len(out)-1] if test.fail { if lastItem.typ != itemError { - t.Fatalf("%s: expected lexing error but did not fail", tn) + t.Fatalf("%d: expected lexing error but did not fail", i) } continue } if lastItem.typ == itemError { - t.Fatalf("%s: unexpected lexing error: %s", tn, lastItem) + t.Fatalf("%d: unexpected lexing error: %s", i, lastItem) } if !reflect.DeepEqual(lastItem, item{itemEOF, Pos(len(test.input)), ""}) { - t.Fatalf("%s: lexing error: expected output to end with EOF item", tn) + t.Fatalf("%d: lexing error: expected output to end with EOF item", i) } out = out[:len(out)-1] if !reflect.DeepEqual(out, test.expected) { - t.Errorf("%s: lexing mismatch:\nexpected: %#v\n-----\ngot: %#v", tn, test.expected, out) + t.Errorf("%d: lexing mismatch:\nexpected: %#v\n-----\ngot: %#v", i, test.expected, out) } } } diff --git a/promql/parse.go b/promql/parse.go index ec388c450..f7ac25f9e 100644 --- a/promql/parse.go +++ b/promql/parse.go @@ -17,6 +17,7 @@ import ( "fmt" "runtime" "strconv" + "strings" "time" clientmodel "github.com/prometheus/client_golang/model" @@ -25,15 +26,29 @@ import ( ) type parser struct { - name string lex *lexer token [3]item peekCount int } +// ParseErr wraps a parsing error with line and position context. +// If the parsing input was a single line, line will be 0 and omitted +// from the error string. +type ParseErr struct { + Line, Pos int + Err error +} + +func (e *ParseErr) Error() string { + if e.Line == 0 { + return fmt.Sprintf("Parse error at char %d: %s", e.Pos, e.Err) + } + return fmt.Sprintf("Parse error at line %d, char %d: %s", e.Line, e.Pos, e.Err) +} + // ParseStmts parses the input and returns the resulting statements or any ocurring error. -func ParseStmts(name, input string) (Statements, error) { - p := newParser(name, input) +func ParseStmts(input string) (Statements, error) { + p := newParser(input) stmts, err := p.parseStmts() if err != nil { @@ -44,8 +59,8 @@ func ParseStmts(name, input string) (Statements, error) { } // ParseExpr returns the expression parsed from the input. -func ParseExpr(name, input string) (Expr, error) { - p := newParser(name, input) +func ParseExpr(input string) (Expr, error) { + p := newParser(input) expr, err := p.parseExpr() if err != nil { @@ -56,10 +71,9 @@ func ParseExpr(name, input string) (Expr, error) { } // newParser returns a new parser. -func newParser(name, input string) *parser { +func newParser(input string) *parser { p := &parser{ - name: name, - lex: lex(name, input), + lex: lex(input), } return p } @@ -144,13 +158,20 @@ func (p *parser) backup() { // errorf formats the error and terminates processing. func (p *parser) errorf(format string, args ...interface{}) { - format = fmt.Sprintf("%s:%d,%d %s", p.name, p.lex.lineNumber(), p.lex.linePosition(), format) - panic(fmt.Errorf(format, args...)) + p.error(fmt.Errorf(format, args...)) } // error terminates processing. func (p *parser) error(err error) { - p.errorf("%s", err) + perr := &ParseErr{ + Line: p.lex.lineNumber(), + Pos: p.lex.linePosition(), + Err: err, + } + if strings.Count(strings.TrimSpace(p.lex.input), "\n") == 0 { + perr.Line = 0 + } + panic(perr) } // expect consumes the next token and guarantees it has the required type. diff --git a/promql/parse_test.go b/promql/parse_test.go index 49750859a..02d350b4b 100644 --- a/promql/parse_test.go +++ b/promql/parse_test.go @@ -785,7 +785,7 @@ var testExpr = []struct { func TestParseExpressions(t *testing.T) { for _, test := range testExpr { - parser := newParser("test", test.input) + parser := newParser(test.input) expr, err := parser.parseExpr() if !test.fail && err != nil { @@ -819,7 +819,7 @@ func TestParseExpressions(t *testing.T) { // NaN has no equality. Thus, we need a separate test for it. func TestNaNExpression(t *testing.T) { - parser := newParser("test", "NaN") + parser := newParser("NaN") expr, err := parser.parseExpr() if err != nil { @@ -1028,7 +1028,7 @@ var testStatement = []struct { func TestParseStatements(t *testing.T) { for _, test := range testStatement { - parser := newParser("test", test.input) + parser := newParser(test.input) stmts, err := parser.parseStmts() if !test.fail && err != nil { diff --git a/rules/rules_test.go b/rules/rules_test.go index 113d200b9..9511f9682 100644 --- a/rules/rules_test.go +++ b/rules/rules_test.go @@ -173,7 +173,7 @@ func TestAlertingRule(t *testing.T) { engine := promql.NewEngine(storage) defer engine.Stop() - expr, err := promql.ParseExpr("test", `http_requests{group="canary", job="app-server"} < 100`) + expr, err := promql.ParseExpr(`http_requests{group="canary", job="app-server"} < 100`) if err != nil { t.Fatalf("Unable to parse alert expression: %s", err) } diff --git a/tools/rule_checker/main.go b/tools/rule_checker/main.go index f53991d5f..c072bd631 100644 --- a/tools/rule_checker/main.go +++ b/tools/rule_checker/main.go @@ -39,7 +39,7 @@ func checkRules(filename string, in io.Reader, out io.Writer) error { return err } - rules, err := promql.ParseStmts(filename, string(content)) + rules, err := promql.ParseStmts(string(content)) if err != nil { return err } diff --git a/web/api/api_test.go b/web/api/api_test.go index 44f52d8f0..5eabf4ed2 100644 --- a/web/api/api_test.go +++ b/web/api/api_test.go @@ -52,7 +52,7 @@ func TestQuery(t *testing.T) { { queryStr: "", status: http.StatusOK, - bodyRe: `{"type":"error","value":"query:1,1 no expression found in input","version":1}`, + bodyRe: `{"type":"error","value":"Parse error at char 1: no expression found in input","version":1}`, }, { queryStr: "expr=testmetric", @@ -77,7 +77,7 @@ func TestQuery(t *testing.T) { { queryStr: "expr=(badexpression", status: http.StatusOK, - bodyRe: `{"type":"error","value":"query:1,15 unexpected unclosed left parenthesis in paren expression","version":1}`, + bodyRe: `{"type":"error","value":"Parse error at char 15: unexpected unclosed left parenthesis in paren expression","version":1}`, }, } From 479891c9be688e35de03f2db4619eeceeacaed68 Mon Sep 17 00:00:00 2001 From: Fabian Reinartz Date: Wed, 29 Apr 2015 10:26:49 +0200 Subject: [PATCH 05/10] Rename RuleManager to Manager, remove interface. This commits renames the RuleManager to Manager as the package name is 'rules' now. The unused layer of abstraction of the RuleManager interface is removed. --- main.go | 4 ++-- rules/manager.go | 44 +++++++++++++++++--------------------------- web/alerts.go | 2 +- web/status.go | 4 ++-- 4 files changed, 22 insertions(+), 32 deletions(-) diff --git a/main.go b/main.go index 3e6f77550..bc092239d 100644 --- a/main.go +++ b/main.go @@ -78,7 +78,7 @@ var ( type prometheus struct { queryEngine *promql.Engine - ruleManager rules.RuleManager + ruleManager *rules.Manager targetManager retrieval.TargetManager notificationHandler *notification.NotificationHandler storage local.Storage @@ -159,7 +159,7 @@ func NewPrometheus() *prometheus { queryEngine := promql.NewEngine(memStorage) - ruleManager := rules.NewRuleManager(&rules.RuleManagerOptions{ + ruleManager := rules.NewManager(&rules.ManagerOptions{ SampleAppender: sampleAppender, NotificationHandler: notificationHandler, EvaluationInterval: conf.EvaluationInterval(), diff --git a/rules/manager.go b/rules/manager.go index 26d12f95d..3b2bed6ee 100644 --- a/rules/manager.go +++ b/rules/manager.go @@ -70,20 +70,8 @@ func init() { prometheus.MustRegister(evalDuration) } -// A RuleManager manages recording and alerting rules. Create instances with -// NewRuleManager. -type RuleManager interface { - // Start the rule manager's periodic rule evaluation. - Run() - // Stop the rule manager's rule evaluation cycles. - Stop() - // Return all rules. - Rules() []Rule - // Return all alerting rules. - AlertingRules() []*AlertingRule -} - -type ruleManager struct { +// The Manager manages recording and alerting rules. +type Manager struct { // Protects the rules list. sync.Mutex rules []Rule @@ -100,8 +88,8 @@ type ruleManager struct { pathPrefix string } -// RuleManagerOptions bundles options for the RuleManager. -type RuleManagerOptions struct { +// ManagerOptions bundles options for the Manager. +type ManagerOptions struct { EvaluationInterval time.Duration QueryEngine *promql.Engine @@ -112,10 +100,10 @@ type RuleManagerOptions struct { PathPrefix string } -// NewRuleManager returns an implementation of RuleManager, ready to be started +// NewManager returns an implementation of Manager, ready to be started // by calling the Run method. -func NewRuleManager(o *RuleManagerOptions) RuleManager { - manager := &ruleManager{ +func NewManager(o *ManagerOptions) *Manager { + manager := &Manager{ rules: []Rule{}, done: make(chan bool), @@ -131,7 +119,8 @@ func NewRuleManager(o *RuleManagerOptions) RuleManager { return manager } -func (m *ruleManager) Run() { +// Run the rule manager's periodic rule evaluation. +func (m *Manager) Run() { defer glog.Info("Rule manager stopped.") ticker := time.NewTicker(m.interval) @@ -158,12 +147,13 @@ func (m *ruleManager) Run() { } } -func (m *ruleManager) Stop() { +// Stop the rule manager's rule evaluation cycles. +func (m *Manager) Stop() { glog.Info("Stopping rule manager...") m.done <- true } -func (m *ruleManager) queueAlertNotifications(rule *AlertingRule, timestamp clientmodel.Timestamp) { +func (m *Manager) queueAlertNotifications(rule *AlertingRule, timestamp clientmodel.Timestamp) { activeAlerts := rule.ActiveAlerts() if len(activeAlerts) == 0 { return @@ -217,7 +207,7 @@ func (m *ruleManager) queueAlertNotifications(rule *AlertingRule, timestamp clie m.notificationHandler.SubmitReqs(notifications) } -func (m *ruleManager) runIteration() { +func (m *Manager) runIteration() { now := clientmodel.Now() wg := sync.WaitGroup{} @@ -268,7 +258,7 @@ func (m *ruleManager) runIteration() { wg.Wait() } -func (m *ruleManager) AddAlertingRule(ctx context.Context, r *promql.AlertStmt) error { +func (m *Manager) AddAlertingRule(ctx context.Context, r *promql.AlertStmt) error { rule := NewAlertingRule(r.Name, r.Expr, r.Duration, r.Labels, r.Summary, r.Description) m.Lock() @@ -277,7 +267,7 @@ func (m *ruleManager) AddAlertingRule(ctx context.Context, r *promql.AlertStmt) return nil } -func (m *ruleManager) AddRecordingRule(ctx context.Context, r *promql.RecordStmt) error { +func (m *Manager) AddRecordingRule(ctx context.Context, r *promql.RecordStmt) error { rule := &RecordingRule{r.Name, r.Expr, r.Labels} m.Lock() @@ -286,7 +276,7 @@ func (m *ruleManager) AddRecordingRule(ctx context.Context, r *promql.RecordStmt return nil } -func (m *ruleManager) Rules() []Rule { +func (m *Manager) Rules() []Rule { m.Lock() defer m.Unlock() @@ -295,7 +285,7 @@ func (m *ruleManager) Rules() []Rule { return rules } -func (m *ruleManager) AlertingRules() []*AlertingRule { +func (m *Manager) AlertingRules() []*AlertingRule { m.Lock() defer m.Unlock() diff --git a/web/alerts.go b/web/alerts.go index 5744b074d..0e02965f2 100644 --- a/web/alerts.go +++ b/web/alerts.go @@ -46,7 +46,7 @@ func (s byAlertStateSorter) Swap(i, j int) { // AlertsHandler implements http.Handler. type AlertsHandler struct { - RuleManager rules.RuleManager + RuleManager *rules.Manager PathPrefix string mutex sync.Mutex diff --git a/web/status.go b/web/status.go index 4180874ed..6b6dd0a65 100644 --- a/web/status.go +++ b/web/status.go @@ -29,10 +29,10 @@ type PrometheusStatusHandler struct { BuildInfo map[string]string Config string Flags map[string]string - RuleManager rules.RuleManager + RuleManager *rules.Manager TargetPools map[string]*retrieval.TargetPool - Birth time.Time + Birth time.Time PathPrefix string } From fe935179cdc6ef415d581aa06e4e5405a5cac09f Mon Sep 17 00:00:00 2001 From: Fabian Reinartz Date: Wed, 29 Apr 2015 11:08:56 +0200 Subject: [PATCH 06/10] Stop routing rule statements through the engine. --- main.go | 13 +-- promql/engine.go | 172 +++++++++------------------------------ promql/engine_test.go | 185 +++++------------------------------------- rules/manager.go | 47 +++++++---- 4 files changed, 94 insertions(+), 323 deletions(-) diff --git a/main.go b/main.go index bc092239d..4ac6ecb7a 100644 --- a/main.go +++ b/main.go @@ -167,16 +167,9 @@ func NewPrometheus() *prometheus { PrometheusURL: web.MustBuildServerURL(*pathPrefix), PathPrefix: *pathPrefix, }) - for _, rf := range conf.Global.GetRuleFile() { - query, err := queryEngine.NewQueryFromFile(rf) - if err != nil { - glog.Errorf("Error loading rule file %q: %s", rf, err) - os.Exit(1) - } - if res := query.Exec(); res.Err != nil { - glog.Errorf("Error initializing rules: %s", res.Err) - os.Exit(1) - } + if err := ruleManager.LoadRuleFiles(conf.Global.GetRuleFile()...); err != nil { + glog.Errorf("Error loading rule files: %s", err) + os.Exit(1) } flags := map[string]string{} diff --git a/promql/engine.go b/promql/engine.go index a149de0ad..945eca6c7 100644 --- a/promql/engine.go +++ b/promql/engine.go @@ -16,11 +16,9 @@ package promql import ( "flag" "fmt" - "io/ioutil" "math" "runtime" "sort" - "sync" "time" "golang.org/x/net/context" @@ -165,14 +163,10 @@ type ( ErrQueryTimeout string // ErrQueryCanceled is returned if a query was canceled during processing. ErrQueryCanceled string - // ErrNoHandlers is returned if no handlers were registered for the - // execution of a statement. - ErrNoHandlers string ) func (e ErrQueryTimeout) Error() string { return fmt.Sprintf("query timed out in %s", e) } func (e ErrQueryCanceled) Error() string { return fmt.Sprintf("query was canceled in %s", e) } -func (e ErrNoHandlers) Error() string { return fmt.Sprintf("no handlers registered to process %s", e) } // A Query is derived from an a raw query string and can be run against an engine // it is associated with. @@ -193,9 +187,6 @@ type query struct { q string // Statements of the parsed query. stmts Statements - // On finished execution two bools indicating success of the execution - // are sent on the channel. - done chan bool // Timer stats for the query execution. stats *stats.TimerGroup // Cancelation function for the query. @@ -231,15 +222,6 @@ func (q *query) Exec() *Result { return &Result{Err: err, Value: res} } -type ( - // AlertHandlers can be registered with an engine and are called on - // each executed alert statement. - AlertHandler func(context.Context, *AlertStmt) error - // RecordHandlers can be registered with an engine and are called on - // each executed record statement. - RecordHandler func(context.Context, *RecordStmt) error -) - // contextDone returns an error if the context was canceled or timed out. func contextDone(ctx context.Context, env string) error { select { @@ -258,32 +240,24 @@ func contextDone(ctx context.Context, env string) error { } } -// Engine handles the liftetime of queries from beginning to end. It is connected -// to a storage. +// Engine handles the liftetime of queries from beginning to end. +// It is connected to a storage. type Engine struct { - sync.RWMutex - // The storage on which the engine operates. storage local.Storage // The base context for all queries and its cancellation function. baseCtx context.Context cancelQueries func() - - // Handlers for the statements. - alertHandlers map[string]AlertHandler - recordHandlers map[string]RecordHandler } // NewEngine returns a new engine. func NewEngine(storage local.Storage) *Engine { ctx, cancel := context.WithCancel(context.Background()) return &Engine{ - storage: storage, - baseCtx: ctx, - cancelQueries: cancel, - alertHandlers: map[string]AlertHandler{}, - recordHandlers: map[string]RecordHandler{}, + storage: storage, + baseCtx: ctx, + cancelQueries: cancel, } } @@ -292,31 +266,6 @@ func (ng *Engine) Stop() { ng.cancelQueries() } -// NewQuery returns a new query of the given query string. -func (ng *Engine) NewQuery(qs string) (Query, error) { - stmts, err := ParseStmts(qs) - if err != nil { - return nil, err - } - query := &query{ - q: qs, - stmts: stmts, - ng: ng, - done: make(chan bool, 2), - stats: stats.NewTimerGroup(), - } - return query, nil -} - -// NewQueryFromFile reads a file and returns a query of statements it contains. -func (ng *Engine) NewQueryFromFile(filename string) (Query, error) { - content, err := ioutil.ReadFile(filename) - if err != nil { - return nil, err - } - return ng.NewQuery(string(content)) -} - // NewInstantQuery returns an evaluation query for the given expression at the given time. func (ng *Engine) NewInstantQuery(es string, ts clientmodel.Timestamp) (Query, error) { return ng.NewRangeQuery(es, ts, ts, 0) @@ -336,77 +285,64 @@ func (ng *Engine) NewRangeQuery(qs string, start, end clientmodel.Timestamp, int Interval: interval, } - query := &query{ + qry := &query{ q: qs, stmts: Statements{es}, ng: ng, - done: make(chan bool, 2), stats: stats.NewTimerGroup(), } - return query, nil + return qry, nil } -// exec executes all statements in the query. For evaluation statements only -// one statement per query is allowed, after which the execution returns. +// testStmt is an internal helper statement that allows execution +// of an arbitrary function during handling. It is used to test the Engine. +type testStmt func(context.Context) error + +func (testStmt) String() string { return "test statement" } +func (testStmt) DotGraph() string { return "test statement" } +func (testStmt) stmt() {} + +func (ng *Engine) newTestQuery(stmts ...Statement) Query { + qry := &query{ + q: "test statement", + stmts: Statements(stmts), + ng: ng, + stats: stats.NewTimerGroup(), + } + return qry +} + +// exec executes the query. +// +// At this point per query only one EvalStmt is evaluated. Alert and record +// statements are not handled by the Engine. func (ng *Engine) exec(ctx context.Context, q *query) (Value, error) { const env = "query execution" // Cancel when execution is done or an error was raised. defer q.cancel() - // The base context might already be canceled (e.g. during shutdown). - if err := contextDone(ctx, env); err != nil { - return nil, err - } - evalTimer := q.stats.GetTimer(stats.TotalEvalTime).Start() defer evalTimer.Stop() - ng.RLock() - alertHandlers := []AlertHandler{} - for _, h := range ng.alertHandlers { - alertHandlers = append(alertHandlers, h) - } - recordHandlers := []RecordHandler{} - for _, h := range ng.recordHandlers { - recordHandlers = append(recordHandlers, h) - } - ng.RUnlock() - for _, stmt := range q.stmts { + // The base context might already be canceled on the first iteration (e.g. during shutdown). + if err := contextDone(ctx, env); err != nil { + return nil, err + } + switch s := stmt.(type) { - case *AlertStmt: - if len(alertHandlers) == 0 { - return nil, ErrNoHandlers("alert statement") - } - for _, h := range alertHandlers { - if err := contextDone(ctx, env); err != nil { - return nil, err - } - err := h(ctx, s) - if err != nil { - return nil, err - } - } - case *RecordStmt: - if len(recordHandlers) == 0 { - return nil, ErrNoHandlers("record statement") - } - for _, h := range recordHandlers { - if err := contextDone(ctx, env); err != nil { - return nil, err - } - err := h(ctx, s) - if err != nil { - return nil, err - } - } case *EvalStmt: // Currently, only one execution statement per query is allowed. return ng.execEvalStmt(ctx, q, s) + case testStmt: + if err := s(ctx); err != nil { + return nil, err + } + default: - panic(fmt.Errorf("statement of unknown type %T", stmt)) + panic(fmt.Errorf("promql.Engine.exec: unhandled statement of type %T", stmt)) } } return nil, nil @@ -1050,34 +986,6 @@ func (ev *evaluator) aggregation(op itemType, grouping clientmodel.LabelNames, k return resultVector } -// RegisterAlertHandler registers a new alert handler of the given name. -func (ng *Engine) RegisterAlertHandler(name string, h AlertHandler) { - ng.Lock() - ng.alertHandlers[name] = h - ng.Unlock() -} - -// RegisterRecordHandler registers a new record handler of the given name. -func (ng *Engine) RegisterRecordHandler(name string, h RecordHandler) { - ng.Lock() - ng.recordHandlers[name] = h - ng.Unlock() -} - -// UnregisterAlertHandler removes the alert handler with the given name. -func (ng *Engine) UnregisterAlertHandler(name string) { - ng.Lock() - delete(ng.alertHandlers, name) - ng.Unlock() -} - -// UnregisterRecordHandler removes the record handler with the given name. -func (ng *Engine) UnregisterRecordHandler(name string) { - ng.Lock() - delete(ng.recordHandlers, name) - ng.Unlock() -} - // btos returns 1 if b is true, 0 otherwise. func btos(b bool) clientmodel.SampleValue { if b { diff --git a/promql/engine_test.go b/promql/engine_test.go index 35e9188af..09b175385 100644 --- a/promql/engine_test.go +++ b/promql/engine_test.go @@ -1,7 +1,6 @@ package promql import ( - "reflect" "sync" "testing" "time" @@ -11,6 +10,10 @@ import ( "github.com/prometheus/prometheus/storage/local" ) +var noop = testStmt(func(context.Context) error { + return nil +}) + func TestQueryTimeout(t *testing.T) { *defaultQueryTimeout = 5 * time.Millisecond defer func() { @@ -24,23 +27,14 @@ func TestQueryTimeout(t *testing.T) { engine := NewEngine(storage) defer engine.Stop() - query, err := engine.NewQuery("foo = bar") - if err != nil { - t.Fatalf("error parsing query: %s", err) - } + f1 := testStmt(func(context.Context) error { + time.Sleep(10 * time.Millisecond) + return nil + }) // Timeouts are not exact but checked in designated places. For example between - // invoking handlers. Thus, we reigster two handlers that take some time to ensure we check - // after exceeding the timeout. - // Should the implementation of this area change, the test might have to be adjusted. - engine.RegisterRecordHandler("test", func(context.Context, *RecordStmt) error { - time.Sleep(10 * time.Millisecond) - return nil - }) - engine.RegisterRecordHandler("test2", func(context.Context, *RecordStmt) error { - time.Sleep(10 * time.Millisecond) - return nil - }) + // invoking test statements. + query := engine.newTestQuery(f1, f1) res := query.Exec() if res.Err == nil { @@ -58,26 +52,16 @@ func TestQueryCancel(t *testing.T) { engine := NewEngine(storage) defer engine.Stop() - query1, err := engine.NewQuery("foo = bar") - if err != nil { - t.Fatalf("error parsing query: %s", err) - } - query2, err := engine.NewQuery("foo = baz") - if err != nil { - t.Fatalf("error parsing query: %s", err) - } - // As for timeouts, cancellation is only checked at designated points. We ensure // that we reach one of those points using the same method. - engine.RegisterRecordHandler("test1", func(context.Context, *RecordStmt) error { - <-time.After(2 * time.Millisecond) - return nil - }) - engine.RegisterRecordHandler("test2", func(context.Context, *RecordStmt) error { - <-time.After(2 * time.Millisecond) + f1 := testStmt(func(context.Context) error { + time.Sleep(2 * time.Millisecond) return nil }) + query1 := engine.newTestQuery(f1, f1) + query2 := engine.newTestQuery(f1, f1) + // Cancel query after starting it. var wg sync.WaitGroup var res *Result @@ -87,7 +71,7 @@ func TestQueryCancel(t *testing.T) { res = query1.Exec() wg.Done() }() - <-time.After(1 * time.Millisecond) + time.Sleep(1 * time.Millisecond) query1.Cancel() wg.Wait() @@ -112,34 +96,20 @@ func TestEngineShutdown(t *testing.T) { engine := NewEngine(storage) - query1, err := engine.NewQuery("foo = bar") - if err != nil { - t.Fatalf("error parsing query: %s", err) - } - query2, err := engine.NewQuery("foo = baz") - if err != nil { - t.Fatalf("error parsing query: %s", err) - } - handlerExecutions := 0 - // Shutdown engine on first handler execution. Should handler execution ever become // concurrent this test has to be adjusted accordingly. - engine.RegisterRecordHandler("test", func(context.Context, *RecordStmt) error { - handlerExecutions++ - engine.Stop() - time.Sleep(10 * time.Millisecond) - return nil - }) - engine.RegisterRecordHandler("test2", func(context.Context, *RecordStmt) error { + f1 := testStmt(func(context.Context) error { handlerExecutions++ engine.Stop() time.Sleep(10 * time.Millisecond) return nil }) + query1 := engine.newTestQuery(f1, f1) + query2 := engine.newTestQuery(f1, f1) - // Stopping the engine should cancel the base context. While setting up queries is - // still possible their context is canceled from the beginning and execution should + // 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. res := query1.Exec() @@ -147,7 +117,7 @@ func TestEngineShutdown(t *testing.T) { t.Fatalf("expected error on shutdown during query but got none") } if handlerExecutions != 1 { - t.Fatalf("expected only one handler to be executed before query cancellation but got %d executons", handlerExecutions) + t.Fatalf("expected only one handler to be executed before query cancellation but got %d executions", handlerExecutions) } res2 := query2.Exec() @@ -159,114 +129,3 @@ func TestEngineShutdown(t *testing.T) { } } - -func TestAlertHandler(t *testing.T) { - storage, closer := local.NewTestStorage(t, 1) - defer closer.Close() - - engine := NewEngine(storage) - defer engine.Stop() - - qs := `ALERT Foo IF bar FOR 5m WITH {a="b"} SUMMARY "sum" DESCRIPTION "desc"` - - doQuery := func(expectFailure bool) *AlertStmt { - query, err := engine.NewQuery(qs) - if err != nil { - t.Fatalf("error parsing query: %s", err) - } - res := query.Exec() - if expectFailure && res.Err == nil { - t.Fatalf("expected error but got none.") - } - if res.Err != nil && !expectFailure { - t.Fatalf("error on executing alert query: %s", res.Err) - } - // That this alert statement is correct is tested elsewhere. - return query.Statements()[0].(*AlertStmt) - } - - // We expect an error if nothing is registered to handle the query. - alertStmt := doQuery(true) - - receivedCalls := 0 - - // Ensure that we receive the correct statement. - engine.RegisterAlertHandler("test", func(ctx context.Context, as *AlertStmt) error { - if !reflect.DeepEqual(alertStmt, as) { - t.Errorf("received alert statement did not match input: %q", qs) - t.Fatalf("no match\n\nexpected:\n%s\ngot: \n%s\n", Tree(alertStmt), Tree(as)) - } - receivedCalls++ - return nil - }) - - for i := 0; i < 10; i++ { - doQuery(false) - if receivedCalls != i+1 { - t.Fatalf("alert handler was not called on query execution") - } - } - - engine.UnregisterAlertHandler("test") - - // We must receive no further calls after unregistering. - doQuery(true) - if receivedCalls != 10 { - t.Fatalf("received calls after unregistering alert handler") - } -} - -func TestRecordHandler(t *testing.T) { - storage, closer := local.NewTestStorage(t, 1) - defer closer.Close() - - engine := NewEngine(storage) - defer engine.Stop() - - qs := `foo = bar` - - doQuery := func(expectFailure bool) *RecordStmt { - query, err := engine.NewQuery(qs) - if err != nil { - t.Fatalf("error parsing query: %s", err) - } - res := query.Exec() - if expectFailure && res.Err == nil { - t.Fatalf("expected error but got none.") - } - if res.Err != nil && !expectFailure { - t.Fatalf("error on executing record query: %s", res.Err) - } - return query.Statements()[0].(*RecordStmt) - } - - // We expect an error if nothing is registered to handle the query. - recordStmt := doQuery(true) - - receivedCalls := 0 - - // Ensure that we receive the correct statement. - engine.RegisterRecordHandler("test", func(ctx context.Context, rs *RecordStmt) error { - if !reflect.DeepEqual(recordStmt, rs) { - t.Errorf("received record statement did not match input: %q", qs) - t.Fatalf("no match\n\nexpected:\n%s\ngot: \n%s\n", Tree(recordStmt), Tree(rs)) - } - receivedCalls++ - return nil - }) - - for i := 0; i < 10; i++ { - doQuery(false) - if receivedCalls != i+1 { - t.Fatalf("record handler was not called on query execution") - } - } - - engine.UnregisterRecordHandler("test") - - // We must receive no further calls after unregistering. - doQuery(true) - if receivedCalls != 10 { - t.Fatalf("received calls after unregistering record handler") - } -} diff --git a/rules/manager.go b/rules/manager.go index 3b2bed6ee..fe6b12b00 100644 --- a/rules/manager.go +++ b/rules/manager.go @@ -15,12 +15,12 @@ package rules import ( "fmt" + "io/ioutil" "sync" "time" "github.com/golang/glog" "github.com/prometheus/client_golang/prometheus" - "golang.org/x/net/context" clientmodel "github.com/prometheus/client_golang/model" @@ -113,9 +113,6 @@ func NewManager(o *ManagerOptions) *Manager { notificationHandler: o.NotificationHandler, prometheusURL: o.PrometheusURL, } - manager.queryEngine.RegisterAlertHandler("rule_manager", manager.AddAlertingRule) - manager.queryEngine.RegisterRecordHandler("rule_manager", manager.AddRecordingRule) - return manager } @@ -258,24 +255,37 @@ func (m *Manager) runIteration() { wg.Wait() } -func (m *Manager) AddAlertingRule(ctx context.Context, r *promql.AlertStmt) error { - rule := NewAlertingRule(r.Name, r.Expr, r.Duration, r.Labels, r.Summary, r.Description) - +// LoadRuleFiles loads alerting and recording rules from the given files. +func (m *Manager) LoadRuleFiles(filenames ...string) error { m.Lock() - m.rules = append(m.rules, rule) - m.Unlock() - return nil -} - -func (m *Manager) AddRecordingRule(ctx context.Context, r *promql.RecordStmt) error { - rule := &RecordingRule{r.Name, r.Expr, r.Labels} - - m.Lock() - m.rules = append(m.rules, rule) - m.Unlock() + defer m.Unlock() + + for _, fn := range filenames { + content, err := ioutil.ReadFile(fn) + if err != nil { + return err + } + stmts, err := promql.ParseStmts(string(content)) + if err != nil { + return fmt.Errorf("error parsing %s: %s", fn, err) + } + for _, stmt := range stmts { + switch r := stmt.(type) { + case *promql.AlertStmt: + rule := NewAlertingRule(r.Name, r.Expr, r.Duration, r.Labels, r.Summary, r.Description) + m.rules = append(m.rules, rule) + case *promql.RecordStmt: + rule := &RecordingRule{r.Name, r.Expr, r.Labels} + m.rules = append(m.rules, rule) + default: + panic("retrieval.Manager.LoadRuleFiles: unknown statement type") + } + } + } return nil } +// Rules returns the list of the manager's rules. func (m *Manager) Rules() []Rule { m.Lock() defer m.Unlock() @@ -285,6 +295,7 @@ func (m *Manager) Rules() []Rule { return rules } +// AlertingRules returns the list of the manager's alerting rules. func (m *Manager) AlertingRules() []*AlertingRule { m.Lock() defer m.Unlock() From 279831cdf125e37dc56131bb9d6ccd5ca56a10a0 Mon Sep 17 00:00:00 2001 From: Fabian Reinartz Date: Wed, 29 Apr 2015 16:35:18 +0200 Subject: [PATCH 07/10] Fix and improve parsing error output. --- promql/lex.go | 90 +++++++++++--- promql/lex_test.go | 3 + promql/parse.go | 39 +++--- promql/parse_test.go | 277 ++++++++++++++++++++++++++++++++----------- promql/printer.go | 3 + web/api/api_test.go | 2 +- 6 files changed, 307 insertions(+), 107 deletions(-) diff --git a/promql/lex.go b/promql/lex.go index d147b39f2..41dee517c 100644 --- a/promql/lex.go +++ b/promql/lex.go @@ -15,7 +15,6 @@ package promql import ( "fmt" - "reflect" "strings" "unicode" "unicode/utf8" @@ -35,6 +34,8 @@ func (i item) String() string { 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(): @@ -183,6 +184,16 @@ var key = map[string]itemType{ // 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: ";", + itemSUB: "-", itemADD: "+", itemMUL: "*", @@ -209,7 +220,39 @@ func (t itemType) String() string { if s, ok := itemTypeStr[t]; ok { return s } - return reflect.TypeOf(t).Name() + return fmt.Sprintf("", 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 @@ -377,7 +420,7 @@ func lexStatements(l *lexer) stateFn { l.next() l.emit(itemEQL) } else if t == '~' { - return l.errorf("unrecognized character after '=': %#U", t) + return l.errorf("unexpected character after '=': %q", t) } else { l.emit(itemAssign) } @@ -385,7 +428,7 @@ func lexStatements(l *lexer) stateFn { if t := l.next(); t == '=' { l.emit(itemNEQ) } else { - return l.errorf("unrecognized character after '!': %#U", t) + return l.errorf("unexpected character after '!': %q", t) } case r == '<': if t := l.peek(); t == '=' { @@ -401,7 +444,7 @@ func lexStatements(l *lexer) stateFn { } else { l.emit(itemGTR) } - case '0' <= r && r <= '9' || r == '.': + case unicode.IsDigit(r) || (r == '.' && unicode.IsDigit(l.peek())): l.backup() return lexNumberOrDuration case r == '"' || r == '\'': @@ -422,7 +465,7 @@ func lexStatements(l *lexer) stateFn { } } fallthrough - case isAlphaNumeric(r): + case isAlphaNumeric(r) || r == ':': l.backup() return lexKeywordOrIdentifier case r == '(': @@ -433,7 +476,7 @@ func lexStatements(l *lexer) stateFn { l.emit(itemRightParen) l.parenDepth-- if l.parenDepth < 0 { - return l.errorf("unexpected right parenthesis %#U", r) + return l.errorf("unexpected right parenthesis %q", r) } return lexStatements case r == '{': @@ -442,20 +485,20 @@ func lexStatements(l *lexer) stateFn { return lexInsideBraces(l) case r == '[': if l.bracketOpen { - return l.errorf("unexpected left bracket %#U", r) + 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 %#U", r) + return l.errorf("unexpected right bracket %q", r) } l.emit(itemRightBracket) l.bracketOpen = false default: - return l.errorf("unrecognized character in statement: %#U", r) + return l.errorf("unexpected character: %q", r) } return lexStatements } @@ -469,10 +512,10 @@ func lexInsideBraces(l *lexer) stateFn { switch r := l.next(); { case r == eof: - return l.errorf("unexpected EOF inside braces") + return l.errorf("unexpected end of input inside braces") case isSpace(r): return lexSpace - case isAlphaNumeric(r): + case unicode.IsLetter(r) || r == '_': l.backup() return lexIdentifier case r == ',': @@ -494,16 +537,16 @@ func lexInsideBraces(l *lexer) stateFn { case nr == '=': l.emit(itemNEQ) default: - return l.errorf("unrecognized character after '!' inside braces: %#U", nr) + return l.errorf("unexpected character after '!' inside braces: %q", nr) } case r == '{': - return l.errorf("unexpected left brace %#U", r) + return l.errorf("unexpected left brace %q", r) case r == '}': l.emit(itemRightBrace) l.braceOpen = false return lexStatements default: - return l.errorf("unrecognized character inside braces: %#U", r) + return l.errorf("unexpected character inside braces: %q", r) } return lexInsideBraces } @@ -553,7 +596,11 @@ func lexDuration(l *lexer) stateFn { return l.errorf("missing unit character in duration") } // Next two chars must be a valid unit and a non-alphanumeric. - if l.accept("smhdwy") && !isAlphaNumeric(l.peek()) { + 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 } @@ -576,7 +623,11 @@ func lexNumberOrDuration(l *lexer) stateFn { return lexStatements } // Next two chars must be a valid unit and a non-alphanumeric. - if l.accept("smhdwy") && !isAlphaNumeric(l.peek()) { + 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 } @@ -605,7 +656,8 @@ func (l *lexer) scanNumber() bool { return true } -// lexIdentifier scans an alphanumeric identifier. +// lexIdentifier scans an alphanumeric identifier. The next character +// is known to be a letter. func lexIdentifier(l *lexer) stateFn { for isAlphaNumeric(l.next()) { // absorb @@ -651,5 +703,5 @@ func isEndOfLine(r rune) bool { // isAlphaNumeric reports whether r is an alphabetic, digit, or underscore. func isAlphaNumeric(r rune) bool { - return r == '_' || unicode.IsLetter(r) || unicode.IsDigit(r) + return r == '_' || ('a' <= r && r <= 'z') || ('A' <= r && r <= 'Z') || unicode.IsDigit(r) } diff --git a/promql/lex_test.go b/promql/lex_test.go index 8d3425659..bcdec6242 100644 --- a/promql/lex_test.go +++ b/promql/lex_test.go @@ -245,6 +245,9 @@ var tests = []struct { }, // Test Selector. { + input: `台北`, + fail: true, + }, { input: `{foo="bar"}`, expected: []item{ {itemLeftBrace, 0, `{`}, diff --git a/promql/parse.go b/promql/parse.go index f7ac25f9e..8313cee2a 100644 --- a/promql/parse.go +++ b/promql/parse.go @@ -101,7 +101,7 @@ func (p *parser) parseExpr() (expr Expr, err error) { continue } if expr != nil { - p.errorf("expression read but input remaining") + p.errorf("could not parse remaining input %.15q...", p.lex.input[p.lex.lastPos:]) } expr = p.expr() } @@ -132,6 +132,9 @@ func (p *parser) next() item { } p.token[0] = t } + if p.token[p.peekCount].typ == itemError { + p.errorf("%s", p.token[p.peekCount].val) + } return p.token[p.peekCount] } @@ -175,28 +178,23 @@ func (p *parser) error(err error) { } // expect consumes the next token and guarantees it has the required type. -func (p *parser) expect(expected itemType, context string) item { +func (p *parser) expect(exp itemType, context string) item { token := p.next() - if token.typ != expected { - p.unexpected(token, context) + if token.typ != exp { + p.errorf("unexpected %s in %s, expected %s", token.desc(), context, exp.desc()) } return token } // expectOneOf consumes the next token and guarantees it has one of the required types. -func (p *parser) expectOneOf(expected1, expected2 itemType, context string) item { +func (p *parser) expectOneOf(exp1, exp2 itemType, context string) item { token := p.next() - if token.typ != expected1 && token.typ != expected2 { - p.unexpected(token, context) + if token.typ != exp1 && token.typ != exp2 { + p.errorf("unexpected %s in %s, expected %s or %s", token.desc(), context, exp1.desc(), exp2.desc()) } return token } -// unexpected complains about the token and terminates processing. -func (p *parser) unexpected(token item, context string) { - p.errorf("unexpected %s in %s", token, context) -} - // recover is the handler that turns panics into returns from the top level of Parse. func (p *parser) recover(errp *error) { e := recover() @@ -303,10 +301,11 @@ func (p *parser) recordStmt() *RecordStmt { // expr parses any expression. func (p *parser) expr() Expr { - const ctx = "binary expression" - // Parse the starting expression. expr := p.unaryExpr() + if expr == nil { + p.errorf("no valid expression found") + } // Loop through the operations and construct a binary operation tree based // on the operators' precedence. @@ -354,6 +353,9 @@ func (p *parser) expr() Expr { // Parse the next operand. rhs := p.unaryExpr() + if rhs == nil { + p.errorf("missing right-hand side in binary expression") + } // Assign the new root based on the precendence of the LHS and RHS operators. if lhs, ok := expr.(*BinaryExpr); ok && lhs.Op.precedence() < op.precedence() { @@ -497,7 +499,6 @@ func (p *parser) primaryExpr() Expr { p.backup() return p.aggrExpr() } - p.errorf("invalid primary expression") return nil } @@ -535,7 +536,7 @@ func (p *parser) aggrExpr() *AggregateExpr { agop := p.next() if !agop.typ.isAggregator() { - p.errorf("%s is not an aggregation operator", agop) + p.errorf("expected aggregation operator but got %s", agop) } var grouping clientmodel.LabelNames var keepExtra bool @@ -650,7 +651,7 @@ func (p *parser) labelMatchers(operators ...itemType) metric.LabelMatchers { op := p.next().typ if !op.isOperator() { - p.errorf("item %s is not a valid operator for label matching", op) + p.errorf("expected label matching operator but got %s", op) } var validOp = false for _, allowedOp := range operators { @@ -849,10 +850,10 @@ func (p *parser) checkType(node Node) (typ ExprType) { case *Call: nargs := len(n.Func.ArgTypes) if na := nargs - n.Func.OptionalArgs; na > len(n.Args) { - p.errorf("expected at least %d arguments in call to %q, got %d", na, n.Func.Name, len(n.Args)) + p.errorf("expected at least %d argument(s) in call to %q, got %d", na, n.Func.Name, len(n.Args)) } if nargs < len(n.Args) { - p.errorf("expected at most %d arguments in call to %q, got %d", nargs, n.Func.Name, len(n.Args)) + p.errorf("expected at most %d argument(s) in call to %q, got %d", nargs, n.Func.Name, len(n.Args)) } for i, arg := range n.Args { p.expectType(arg, n.Func.ArgTypes[i], fmt.Sprintf("call to function %q", n.Func.Name)) diff --git a/promql/parse_test.go b/promql/parse_test.go index 02d350b4b..8cb1da1d8 100644 --- a/promql/parse_test.go +++ b/promql/parse_test.go @@ -17,6 +17,7 @@ import ( "fmt" "math" "reflect" + "strings" "testing" "time" @@ -25,9 +26,10 @@ import ( ) var testExpr = []struct { - input string - expected Expr - fail bool + input string // The input to be parsed. + expected Expr // The expected expression AST. + fail bool // Whether parsing is supposed to fail. + errMsg string // If not empty the parsing error has to contain this string. }{ // Scalars and scalar-to-scalar operations. { @@ -122,39 +124,77 @@ var testExpr = []struct { }, }, }, { - input: "", fail: true, + input: "", + fail: true, + errMsg: "no expression found in input", }, { - input: "# just a comment\n\n", fail: true, + input: "# just a comment\n\n", + fail: true, + errMsg: "no expression found in input", }, { - input: "1+", fail: true, + input: "1+", + fail: true, + errMsg: "missing right-hand side in binary expression", }, { - input: "2.5.", fail: true, + input: ".", + fail: true, + errMsg: "unexpected character: '.'", }, { - input: "100..4", fail: true, + input: "2.5.", + fail: true, + errMsg: "could not parse remaining input \".\"...", }, { - input: "0deadbeef", fail: true, + input: "100..4", + fail: true, + errMsg: "could not parse remaining input \".4\"...", }, { - input: "1 /", fail: true, + input: "0deadbeef", + fail: true, + errMsg: "bad number or duration syntax: \"0de\"", }, { - input: "*1", fail: true, + input: "1 /", + fail: true, + errMsg: "missing right-hand side in binary expression", }, { - input: "(1))", fail: true, + input: "*1", + fail: true, + errMsg: "no valid expression found", }, { - input: "((1)", fail: true, + input: "(1))", + fail: true, + errMsg: "could not parse remaining input \")\"...", }, { - input: "(", fail: true, + input: "((1)", + fail: true, + errMsg: "unclosed left parenthesis", }, { - input: "1 and 1", fail: true, + input: "(", + fail: true, + errMsg: "unclosed left parenthesis", }, { - input: "1 or 1", fail: true, + input: "1 and 1", + fail: true, + errMsg: "AND and OR not allowed in binary scalar expression", }, { - input: "1 !~ 1", fail: true, + input: "1 or 1", + fail: true, + errMsg: "AND and OR not allowed in binary scalar expression", }, { - input: "1 =~ 1", fail: true, + input: "1 !~ 1", + fail: true, + errMsg: "could not parse remaining input \"!~ 1\"...", }, { - input: "-some_metric", fail: true, + input: "1 =~ 1", + fail: true, + errMsg: "could not parse remaining input \"=~ 1\"...", }, { - input: `-"string"`, fail: true, + input: "-some_metric", + fail: true, + errMsg: "expected type scalar in unary expression, got vector", + }, { + input: `-"string"`, + fail: true, + errMsg: "expected type scalar in unary expression, got string", }, // Vector binary operations. { @@ -397,25 +437,45 @@ var testExpr = []struct { }, }, }, { - input: "foo and 1", fail: true, + input: "foo and 1", + fail: true, + errMsg: "AND and OR not allowed in binary scalar expression", }, { - input: "1 and foo", fail: true, + input: "1 and foo", + fail: true, + errMsg: "AND and OR not allowed in binary scalar expression", }, { - input: "foo or 1", fail: true, + input: "foo or 1", + fail: true, + errMsg: "AND and OR not allowed in binary scalar expression", }, { - input: "1 or foo", fail: true, + input: "1 or foo", + fail: true, + errMsg: "AND and OR not allowed in binary scalar expression", }, { - input: "1 or on(bar) foo", fail: true, + input: "1 or on(bar) foo", + fail: true, + errMsg: "vector matching only allowed between vectors", }, { - input: "foo == on(bar) 10", fail: true, + input: "foo == on(bar) 10", + fail: true, + errMsg: "vector matching only allowed between vectors", }, { - input: "foo and on(bar) group_left(baz) bar", fail: true, + input: "foo and on(bar) group_left(baz) bar", + fail: true, + errMsg: "no grouping allowed for AND and OR operations", }, { - input: "foo and on(bar) group_right(baz) bar", fail: true, + input: "foo and on(bar) group_right(baz) bar", + fail: true, + errMsg: "no grouping allowed for AND and OR operations", }, { - input: "foo or on(bar) group_left(baz) bar", fail: true, + input: "foo or on(bar) group_left(baz) bar", + fail: true, + errMsg: "no grouping allowed for AND and OR operations", }, { - input: "foo or on(bar) group_right(baz) bar", fail: true, + input: "foo or on(bar) group_right(baz) bar", + fail: true, + errMsg: "no grouping allowed for AND and OR operations", }, // Test vector selector. { @@ -470,31 +530,59 @@ var testExpr = []struct { }, }, }, { - input: `{`, fail: true, + input: `{`, + fail: true, + errMsg: "unexpected end of input inside braces", }, { - input: `}`, fail: true, + input: `}`, + fail: true, + errMsg: "unexpected character: '}'", }, { - input: `some{`, fail: true, + input: `some{`, + fail: true, + errMsg: "unexpected end of input inside braces", }, { - input: `some}`, fail: true, + input: `some}`, + fail: true, + errMsg: "could not parse remaining input \"}\"...", }, { - input: `some_metric{a=b}`, fail: true, + input: `some_metric{a=b}`, + fail: true, + errMsg: "unexpected identifier \"b\" in label matching, expected string", }, { - input: `some_metric{a:b="b"}`, fail: true, + input: `some_metric{a:b="b"}`, + fail: true, + errMsg: "unexpected character inside braces: ':'", }, { - input: `foo{a*"b"}`, fail: true, + input: `foo{a*"b"}`, + fail: true, + errMsg: "unexpected character inside braces: '*'", }, { - input: `foo{a>="b"}`, fail: true, + input: `foo{a>="b"}`, + fail: true, + // TODO(fabxc): willingly lexing wrong tokens allows for more precrise error + // messages from the parser - consider if this is an option. + errMsg: "unexpected character inside braces: '>'", }, { - input: `foo{gibberish}`, fail: true, + input: `foo{gibberish}`, + fail: true, + errMsg: "expected label matching operator but got }", }, { - input: `foo{1}`, fail: true, + input: `foo{1}`, + fail: true, + errMsg: "unexpected character inside braces: '1'", }, { - input: `{}`, fail: true, + input: `{}`, + fail: true, + errMsg: "vector selector must contain label matchers or metric name", }, { - input: `foo{__name__="bar"}`, fail: true, - }, { - input: `:foo`, fail: true, + input: `foo{__name__="bar"}`, + fail: true, + errMsg: "metric name must not be set twice: \"foo\" or \"bar\"", + // }, { + // input: `:foo`, + // fail: true, + // errMsg: "bla", }, // Test matrix selector. { @@ -559,25 +647,45 @@ var testExpr = []struct { }, }, }, { - input: `foo[5mm]`, fail: true, + input: `foo[5mm]`, + fail: true, + errMsg: "bad duration syntax: \"5mm\"", }, { - input: `foo[0m]`, fail: true, + input: `foo[0m]`, + fail: true, + errMsg: "duration must be greater than 0", }, { - input: `foo[5m30s]`, fail: true, + input: `foo[5m30s]`, + fail: true, + errMsg: "bad duration syntax: \"5m3\"", }, { - input: `foo[5m] OFFSET 1h30m`, fail: true, + input: `foo[5m] OFFSET 1h30m`, + fail: true, + errMsg: "bad number or duration syntax: \"1h3\"", }, { - input: `foo[]`, fail: true, + input: `foo[]`, + fail: true, + errMsg: "missing unit character in duration", }, { - input: `foo[1]`, fail: true, + input: `foo[1]`, + fail: true, + errMsg: "missing unit character in duration", }, { - input: `some_metric[5m] OFFSET 1`, fail: true, + input: `some_metric[5m] OFFSET 1`, + fail: true, + errMsg: "unexpected number \"1\" in matrix selector, expected duration", }, { - input: `some_metric[5m] OFFSET 1mm`, fail: true, + input: `some_metric[5m] OFFSET 1mm`, + fail: true, + errMsg: "bad number or duration syntax: \"1mm\"", }, { - input: `some_metric[5m] OFFSET`, fail: true, + input: `some_metric[5m] OFFSET`, + fail: true, + errMsg: "unexpected end of input in matrix selector, expected duration", }, { - input: `(foo + bar)[5m]`, fail: true, + input: `(foo + bar)[5m]`, + fail: true, + errMsg: "could not parse remaining input \"[5m]\"...", }, // Test aggregation. { @@ -692,21 +800,37 @@ var testExpr = []struct { Grouping: clientmodel.LabelNames{"foo"}, }, }, { - input: `sum some_metric by (test)`, fail: true, + input: `sum some_metric by (test)`, + fail: true, + errMsg: "unexpected identifier \"some_metric\" in aggregation, expected \"(\"", }, { - input: `sum (some_metric) by test`, fail: true, + input: `sum (some_metric) by test`, + fail: true, + errMsg: "unexpected identifier \"test\" in grouping opts, expected \"(\"", }, { - input: `sum (some_metric) by ()`, fail: true, + input: `sum (some_metric) by ()`, + fail: true, + errMsg: "unexpected \")\" in grouping opts, expected identifier", }, { - input: `sum (some_metric) by test`, fail: true, + input: `sum (some_metric) by test`, + fail: true, + errMsg: "unexpected identifier \"test\" in grouping opts, expected \"(\"", }, { - input: `some_metric[5m] OFFSET`, fail: true, + input: `some_metric[5m] OFFSET`, + fail: true, + errMsg: "unexpected end of input in matrix selector, expected duration", }, { - input: `sum () by (test)`, fail: true, + input: `sum () by (test)`, + fail: true, + errMsg: "no valid expression found", }, { - input: "MIN keeping_extra (some_metric) by (foo)", fail: true, + input: "MIN keeping_extra (some_metric) by (foo)", + fail: true, + errMsg: "could not parse remaining input \"by (foo)\"...", }, { - input: "MIN by(test) (some_metric) keeping_extra", fail: true, + input: "MIN by(test) (some_metric) keeping_extra", + fail: true, + errMsg: "could not parse remaining input \"keeping_extra\"...", }, // Test function calls. { @@ -770,21 +894,30 @@ var testExpr = []struct { }, }, }, { - input: "floor()", fail: true, + input: "floor()", + fail: true, + errMsg: "expected at least 1 argument(s) in call to \"floor\", got 0", }, { - input: "floor(some_metric, other_metric)", fail: true, + input: "floor(some_metric, other_metric)", + fail: true, + errMsg: "expected at most 1 argument(s) in call to \"floor\", got 2", }, { - input: "floor(1)", fail: true, + input: "floor(1)", + fail: true, + errMsg: "expected type vector in call to function \"floor\", got scalar", }, { - input: "non_existant_function_far_bar()", fail: true, + input: "non_existant_function_far_bar()", + fail: true, + errMsg: "unknown function with name \"non_existant_function_far_bar\"", }, { - input: "rate(some_metric)", fail: true, + input: "rate(some_metric)", + fail: true, + errMsg: "expected type matrix in call to function \"rate\", got vector", }, } func TestParseExpressions(t *testing.T) { for _, test := range testExpr { - parser := newParser(test.input) expr, err := parser.parseExpr() @@ -793,6 +926,10 @@ func TestParseExpressions(t *testing.T) { t.Fatalf("could not parse: %s", err) } if test.fail && err != nil { + if !strings.Contains(err.Error(), test.errMsg) { + t.Errorf("unexpected error on input '%s'", test.input) + t.Fatalf("expected error to contain %q but got %q", test.errMsg, err) + } continue } @@ -804,6 +941,10 @@ func TestParseExpressions(t *testing.T) { if test.fail { if err != nil { + if !strings.Contains(err.Error(), test.errMsg) { + t.Errorf("unexpected error on input '%s'", test.input) + t.Fatalf("expected error to contain %q but got %q", test.errMsg, err) + } continue } t.Errorf("error on input '%s'", test.input) diff --git a/promql/printer.go b/promql/printer.go index 6576d2332..3e4f29b77 100644 --- a/promql/printer.go +++ b/promql/printer.go @@ -72,6 +72,9 @@ func Tree(node Node) string { } 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 diff --git a/web/api/api_test.go b/web/api/api_test.go index 5eabf4ed2..e16a149ff 100644 --- a/web/api/api_test.go +++ b/web/api/api_test.go @@ -77,7 +77,7 @@ func TestQuery(t *testing.T) { { queryStr: "expr=(badexpression", status: http.StatusOK, - bodyRe: `{"type":"error","value":"Parse error at char 15: unexpected unclosed left parenthesis in paren expression","version":1}`, + bodyRe: `{"type":"error","value":"Parse error at char 15: unclosed left parenthesis","version":1}`, }, } From d59d1cb2c1be4c3d8649eaec9ad9b2a4d700d2d3 Mon Sep 17 00:00:00 2001 From: Fabian Reinartz Date: Fri, 1 May 2015 17:58:58 +0200 Subject: [PATCH 08/10] Fix Error() methods. --- promql/engine.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/promql/engine.go b/promql/engine.go index 945eca6c7..7d3cc58dd 100644 --- a/promql/engine.go +++ b/promql/engine.go @@ -165,8 +165,8 @@ type ( ErrQueryCanceled string ) -func (e ErrQueryTimeout) Error() string { return fmt.Sprintf("query timed out in %s", e) } -func (e ErrQueryCanceled) Error() string { return fmt.Sprintf("query was canceled in %s", e) } +func (e ErrQueryTimeout) Error() string { return fmt.Sprintf("query timed out in %s", string(e)) } +func (e ErrQueryCanceled) Error() string { return fmt.Sprintf("query was canceled in %s", string(e)) } // A Query is derived from an a raw query string and can be run against an engine // it is associated with. From 9ab1f6c690940187b227cdd6ec591349abf0f5aa Mon Sep 17 00:00:00 2001 From: Fabian Reinartz Date: Fri, 1 May 2015 00:49:19 +0200 Subject: [PATCH 09/10] Limit maximum number of concurrent queries. A high number of concurrent queries can slow each other down so that none of them is reasonbly responsive. This commit limits the number of queries being concurrently executed. --- promql/engine.go | 59 +++++++++++++++++++++++++++++++++----- promql/engine_test.go | 66 ++++++++++++++++++++++++++++++++++--------- stats/query_stats.go | 6 ++-- 3 files changed, 107 insertions(+), 24 deletions(-) diff --git a/promql/engine.go b/promql/engine.go index 7d3cc58dd..7b53f661d 100644 --- a/promql/engine.go +++ b/promql/engine.go @@ -31,8 +31,9 @@ import ( ) var ( - stalenessDelta = flag.Duration("query.staleness-delta", 300*time.Second, "Staleness delta allowance during expression evaluations.") - defaultQueryTimeout = flag.Duration("query.timeout", 2*time.Minute, "Maximum time a query may take before being aborted.") + stalenessDelta = flag.Duration("query.staleness-delta", 300*time.Second, "Staleness delta allowance during expression evaluations.") + defaultQueryTimeout = flag.Duration("query.timeout", 2*time.Minute, "Maximum time a query may take before being aborted.") + maxConcurrentQueries = flag.Int("query.max-concurrency", 20, "Maximum number of queries executed concurrently.") ) // SampleStream is a stream of Values belonging to an attached COWMetric. @@ -215,10 +216,7 @@ func (q *query) Cancel() { // Exec implements the Query interface. func (q *query) Exec() *Result { - ctx, cancel := context.WithTimeout(q.ng.baseCtx, *defaultQueryTimeout) - q.cancel = cancel - - res, err := q.ng.exec(ctx, q) + res, err := q.ng.exec(q) return &Result{Err: err, Value: res} } @@ -249,6 +247,8 @@ type Engine struct { // The base context for all queries and its cancellation function. baseCtx context.Context cancelQueries func() + // The gate limiting the maximum number of concurrent and waiting queries. + gate *queryGate } // NewEngine returns a new engine. @@ -258,6 +258,7 @@ func NewEngine(storage local.Storage) *Engine { storage: storage, baseCtx: ctx, cancelQueries: cancel, + gate: newQueryGate(*maxConcurrentQueries), } } @@ -316,9 +317,21 @@ func (ng *Engine) newTestQuery(stmts ...Statement) Query { // // At this point per query only one EvalStmt is evaluated. Alert and record // statements are not handled by the Engine. -func (ng *Engine) exec(ctx context.Context, q *query) (Value, error) { +func (ng *Engine) exec(q *query) (Value, error) { const env = "query execution" + ctx, cancel := context.WithTimeout(q.ng.baseCtx, *defaultQueryTimeout) + q.cancel = cancel + + queueTimer := q.stats.GetTimer(stats.ExecQueueTime).Start() + + if err := ng.gate.Start(ctx); err != nil { + return nil, err + } + defer ng.gate.Done() + + queueTimer.Stop() + // Cancel when execution is done or an error was raised. defer q.cancel() @@ -1125,3 +1138,35 @@ func interpolateSamples(first, second *metric.SamplePair, timestamp clientmodel. Timestamp: timestamp, } } + +// A queryGate controls the maximum number of concurrently running and waiting queries. +type queryGate struct { + ch chan struct{} +} + +// newQueryGate returns a query gate that limits the number of queries +// being concurrently executed. +func newQueryGate(length int) *queryGate { + return &queryGate{ + ch: make(chan struct{}, length), + } +} + +// Start blocks until the gate has a free spot or the context is done. +func (g *queryGate) Start(ctx context.Context) error { + select { + case <-ctx.Done(): + return contextDone(ctx, "query queue") + case g.ch <- struct{}{}: + return nil + } +} + +// Done releases a single spot in the gate. +func (g *queryGate) Done() { + select { + case <-g.ch: + default: + panic("engine.queryGate.Done: more operations done than started") + } +} diff --git a/promql/engine_test.go b/promql/engine_test.go index 09b175385..ab86afba1 100644 --- a/promql/engine_test.go +++ b/promql/engine_test.go @@ -6,14 +6,61 @@ import ( "time" "golang.org/x/net/context" - - "github.com/prometheus/prometheus/storage/local" ) var noop = testStmt(func(context.Context) error { return nil }) +func TestQueryConcurreny(t *testing.T) { + engine := NewEngine(nil) + defer engine.Stop() + + block := make(chan struct{}) + processing := make(chan struct{}) + f1 := testStmt(func(context.Context) error { + processing <- struct{}{} + <-block + return nil + }) + + for i := 0; i < *maxConcurrentQueries; i++ { + q := engine.newTestQuery(f1) + 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(f1) + 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 < *maxConcurrentQueries; i++ { + block <- struct{}{} + } +} + func TestQueryTimeout(t *testing.T) { *defaultQueryTimeout = 5 * time.Millisecond defer func() { @@ -21,10 +68,7 @@ func TestQueryTimeout(t *testing.T) { *defaultQueryTimeout = 2 * time.Minute }() - storage, closer := local.NewTestStorage(t, 1) - defer closer.Close() - - engine := NewEngine(storage) + engine := NewEngine(nil) defer engine.Stop() f1 := testStmt(func(context.Context) error { @@ -46,10 +90,7 @@ func TestQueryTimeout(t *testing.T) { } func TestQueryCancel(t *testing.T) { - storage, closer := local.NewTestStorage(t, 1) - defer closer.Close() - - engine := NewEngine(storage) + engine := NewEngine(nil) defer engine.Stop() // As for timeouts, cancellation is only checked at designated points. We ensure @@ -91,10 +132,7 @@ func TestQueryCancel(t *testing.T) { } func TestEngineShutdown(t *testing.T) { - storage, closer := local.NewTestStorage(t, 1) - defer closer.Close() - - engine := NewEngine(storage) + engine := NewEngine(nil) handlerExecutions := 0 // Shutdown engine on first handler execution. Should handler execution ever become diff --git a/stats/query_stats.go b/stats/query_stats.go index 8ed7c5f46..1f5cb7d52 100644 --- a/stats/query_stats.go +++ b/stats/query_stats.go @@ -31,7 +31,7 @@ const ( GetValueAtTimeTime GetBoundaryValuesTime GetRangeValuesTime - ViewQueueTime + ExecQueueTime ViewDiskPreparationTime ViewDataExtractionTime ViewDiskExtractionTime @@ -64,8 +64,8 @@ func (s QueryTiming) String() string { return "GetBoundaryValues() time" case GetRangeValuesTime: return "GetRangeValues() time" - case ViewQueueTime: - return "View queue wait time" + case ExecQueueTime: + return "Exec queue wait time" case ViewDiskPreparationTime: return "View building disk preparation time" case ViewDataExtractionTime: From 1dd0b772cbef5c9025c9250da5572a1afeaa142d Mon Sep 17 00:00:00 2001 From: Fabian Reinartz Date: Wed, 6 May 2015 13:21:28 +0200 Subject: [PATCH 10/10] Fix missing Godep entry. --- Godeps/Godeps.json | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/Godeps/Godeps.json b/Godeps/Godeps.json index c844b2a86..acd7a675f 100644 --- a/Godeps/Godeps.json +++ b/Godeps/Godeps.json @@ -7,10 +7,6 @@ "Comment": "null-5", "Rev": "75cd24fc2f2c2a2088577d12123ddee5f54e0675" }, - { - "ImportPath": "github.com/prometheus/procfs", - "Rev": "92faa308558161acab0ada1db048e9996ecec160" - }, { "ImportPath": "github.com/beorn7/perks/quantile", "Rev": "b965b613227fddccbfffe13eae360ed3fa822f8d" @@ -67,6 +63,10 @@ { "ImportPath": "github.com/syndtr/gosnappy/snappy", "Rev": "156a073208e131d7d2e212cb749feae7c339e846" + }, + { + "ImportPath": "golang.org/x/net/context", + "Rev": "b6fdb7d8a4ccefede406f8fe0f017fb58265054c" } ] }