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).
This commit is contained in:
Fabian Reinartz 2015-03-30 18:12:51 +02:00
parent 54f5c524e5
commit 32b7595c47
8 changed files with 3855 additions and 0 deletions

345
promql/ast.go Normal file
View File

@ -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 "<ExprNone>"
case ExprScalar:
return "scalar"
case ExprVector:
return "vector"
case ExprMatrix:
return "matrix"
case ExprString:
return "string"
}
panic("promql.ExprType.String: unhandled expression type")
}
// Expr is a generic interface for all expression types.
type Expr interface {
Node
// Type returns the type the expression evaluates to. It does not perform
// in-depth checks as this is done at parsing-time.
Type() ExprType
// expr ensures that no other types accidentally implement the interface.
expr()
}
// Expressions is a list of expression nodes that implements Node.
type Expressions []Expr
// AggregateExpr represents an aggregation operation on a vector.
type AggregateExpr struct {
Op itemType // The used aggregation operation.
Expr Expr // The vector expression over which is aggregated.
Grouping clientmodel.LabelNames // The labels by which to group the vector.
KeepExtraLabels bool // Whether to keep extra labels common among result elements.
}
// BinaryExpr represents a binary expression between two child expressions.
type BinaryExpr struct {
Op itemType // The operation of the expression.
LHS, RHS Expr // The operands on the respective sides of the operator.
// The matching behavior for the operation if both operands are vectors.
// If they are not this field is nil.
VectorMatching *VectorMatching
}
// Call represents a function call.
type Call struct {
Func *Function // The function that was called.
Args Expressions // Arguments used in the call.
}
// MatrixSelector represents a matrix selection.
type MatrixSelector struct {
Name string
Range time.Duration
Offset time.Duration
LabelMatchers metric.LabelMatchers
// The series iterators are populated at query analysis time.
iterators map[clientmodel.Fingerprint]local.SeriesIterator
metrics map[clientmodel.Fingerprint]clientmodel.COWMetric
// 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)
}

191
promql/functions.go Normal file
View File

@ -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
}

657
promql/lex.go Normal file
View File

@ -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("<op:%s>", i.val)
case i.typ.isAggregator():
return fmt.Sprintf("<aggr:%s>", i.val)
case len(i.val) > 10:
return fmt.Sprintf("%.10q...", i.val)
}
return fmt.Sprintf("%q", i.val)
}
// isOperator returns true if the item corresponds to a logical or arithmetic operator.
// Returns false otherwise.
func (i itemType) isOperator() bool { return i > operatorsStart && i < operatorsEnd }
// isAggregator returns true if the item belongs to the aggregator functions.
// Returns false otherwise
func (i itemType) isAggregator() bool { return i > aggregatorsStart && i < aggregatorsEnd }
// isKeyword returns true if the item corresponds to a keyword.
// Returns false otherwise.
func (i itemType) isKeyword() bool { return i > keywordsStart && i < keywordsEnd }
// Constants for operator precedence in expressions.
//
const LowestPrec = 0 // Non-operators.
// Precedence returns the operator precedence of the binary
// operator op. If op is not a binary operator, the result
// is LowestPrec.
func (i itemType) precedence() int {
switch i {
case itemLOR:
return 1
case itemLAND:
return 2
case itemEQL, itemNEQ, itemLTE, itemLSS, itemGTE, itemGTR:
return 3
case itemADD, itemSUB:
return 4
case itemMUL, itemDIV, itemMOD:
return 5
default:
return LowestPrec
}
}
type itemType int
const (
itemError itemType = iota // Error occurred, value is error message
itemEOF
itemComment
itemIdentifier
itemMetricIdentifier
itemLeftParen
itemRightParen
itemLeftBrace
itemRightBrace
itemLeftBracket
itemRightBracket
itemComma
itemAssign
itemSemicolon
itemString
itemNumber
itemDuration
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)
}

358
promql/lex_test.go Normal file
View File

@ -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)
}
}
}

867
promql/parse.go Normal file
View File

@ -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.
//
// <vector_selector> | <matrix_selector> | (+|-) <number_literal> | '(' <expr> ')'
//
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.
//
// <vector_selector> '[' <duration> ']'
//
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.
//
// <metric_name> | <function_call> | <vector_aggregation> | <literal>
//
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.
//
// '(' <label_name>, ... ')'
//
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.
//
// <aggr_op> (<vector_expr>) [by <labels>] [keeping_extra]
// <aggr_op> [by <labels>] [keeping_extra] (<vector_expr>)
//
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_name> '(' [ <arg_expr>, ...] ')'
//
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
//
// '{' [ <labelname> '=' <match_string>, ... ] '}'
//
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.
//
// '{' [ <labelname> <match_op> <match_string>, ... ] '}'
//
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.
//
// <metric_identifier> [<label_matchers>] [ offset <duration> ]
// [<metric_identifier>] <label_matchers> [ offset <duration> ]
//
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
}

1077
promql/parse_test.go Normal file

File diff suppressed because it is too large Load Diff

355
promql/printer.go Normal file
View File

@ -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
}

View File

@ -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 {