mirror of https://github.com/prometheus/prometheus
promql: migrate model types, use tsdb interfaces
parent
8b84ee5ee6
commit
15a931dbdb
|
@ -24,6 +24,7 @@ import (
|
||||||
"syscall"
|
"syscall"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/fabxc/tsdb"
|
||||||
"github.com/prometheus/client_golang/prometheus"
|
"github.com/prometheus/client_golang/prometheus"
|
||||||
"github.com/prometheus/common/log"
|
"github.com/prometheus/common/log"
|
||||||
"github.com/prometheus/common/model"
|
"github.com/prometheus/common/model"
|
||||||
|
@ -82,6 +83,10 @@ func Main() int {
|
||||||
reloadables []Reloadable
|
reloadables []Reloadable
|
||||||
)
|
)
|
||||||
|
|
||||||
|
_, err := tsdb.Open(cfg.localStoragePath, nil, nil)
|
||||||
|
if err != nil {
|
||||||
|
log.Errorf("Opening storage failed: %s", err)
|
||||||
|
}
|
||||||
var localStorage local.Storage
|
var localStorage local.Storage
|
||||||
|
|
||||||
reloadableRemoteStorage := remote.New()
|
reloadableRemoteStorage := remote.New()
|
||||||
|
|
149
promql/ast.go
149
promql/ast.go
|
@ -15,12 +15,11 @@ package promql
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"fmt"
|
"fmt"
|
||||||
|
"regexp"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/prometheus/common/model"
|
"github.com/fabxc/tsdb"
|
||||||
|
"github.com/fabxc/tsdb/labels"
|
||||||
"github.com/prometheus/prometheus/storage/local"
|
|
||||||
"github.com/prometheus/prometheus/storage/metric"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
// Node is a generic interface for all nodes in an AST.
|
// Node is a generic interface for all nodes in an AST.
|
||||||
|
@ -58,8 +57,8 @@ type AlertStmt struct {
|
||||||
Name string
|
Name string
|
||||||
Expr Expr
|
Expr Expr
|
||||||
Duration time.Duration
|
Duration time.Duration
|
||||||
Labels model.LabelSet
|
Labels labels.Labels
|
||||||
Annotations model.LabelSet
|
Annotations labels.Labels
|
||||||
}
|
}
|
||||||
|
|
||||||
// EvalStmt holds an expression and information on the range it should
|
// EvalStmt holds an expression and information on the range it should
|
||||||
|
@ -69,7 +68,7 @@ type EvalStmt struct {
|
||||||
|
|
||||||
// The time boundaries for the evaluation. If Start equals End an instant
|
// The time boundaries for the evaluation. If Start equals End an instant
|
||||||
// is evaluated.
|
// is evaluated.
|
||||||
Start, End model.Time
|
Start, End time.Time
|
||||||
// Time between two evaluated instants for the range [Start:End].
|
// Time between two evaluated instants for the range [Start:End].
|
||||||
Interval time.Duration
|
Interval time.Duration
|
||||||
}
|
}
|
||||||
|
@ -78,7 +77,7 @@ type EvalStmt struct {
|
||||||
type RecordStmt struct {
|
type RecordStmt struct {
|
||||||
Name string
|
Name string
|
||||||
Expr Expr
|
Expr Expr
|
||||||
Labels model.LabelSet
|
Labels labels.Labels
|
||||||
}
|
}
|
||||||
|
|
||||||
func (*AlertStmt) stmt() {}
|
func (*AlertStmt) stmt() {}
|
||||||
|
@ -91,7 +90,7 @@ type Expr interface {
|
||||||
|
|
||||||
// Type returns the type the expression evaluates to. It does not perform
|
// Type returns the type the expression evaluates to. It does not perform
|
||||||
// in-depth checks as this is done at parsing-time.
|
// in-depth checks as this is done at parsing-time.
|
||||||
Type() model.ValueType
|
Type() ValueType
|
||||||
// expr ensures that no other types accidentally implement the interface.
|
// expr ensures that no other types accidentally implement the interface.
|
||||||
expr()
|
expr()
|
||||||
}
|
}
|
||||||
|
@ -101,12 +100,12 @@ type Expressions []Expr
|
||||||
|
|
||||||
// AggregateExpr represents an aggregation operation on a vector.
|
// AggregateExpr represents an aggregation operation on a vector.
|
||||||
type AggregateExpr struct {
|
type AggregateExpr struct {
|
||||||
Op itemType // The used aggregation operation.
|
Op itemType // The used aggregation operation.
|
||||||
Expr Expr // The vector expression over which is aggregated.
|
Expr Expr // The vector expression over which is aggregated.
|
||||||
Param Expr // Parameter used by some aggregators.
|
Param Expr // Parameter used by some aggregators.
|
||||||
Grouping model.LabelNames // The labels by which to group the vector.
|
Grouping []string // The labels by which to group the vector.
|
||||||
Without bool // Whether to drop the given labels rather than keep them.
|
Without bool // Whether to drop the given labels rather than keep them.
|
||||||
KeepCommonLabels bool // Whether to keep common labels among result elements.
|
KeepCommonLabels bool // Whether to keep common labels among result elements.
|
||||||
}
|
}
|
||||||
|
|
||||||
// BinaryExpr represents a binary expression between two child expressions.
|
// BinaryExpr represents a binary expression between two child expressions.
|
||||||
|
@ -133,15 +132,16 @@ type MatrixSelector struct {
|
||||||
Name string
|
Name string
|
||||||
Range time.Duration
|
Range time.Duration
|
||||||
Offset time.Duration
|
Offset time.Duration
|
||||||
LabelMatchers metric.LabelMatchers
|
LabelMatchers []*LabelMatcher
|
||||||
|
|
||||||
// The series iterators are populated at query preparation time.
|
// The series iterators are populated at query preparation time.
|
||||||
iterators []local.SeriesIterator
|
series []tsdb.Series
|
||||||
|
iterators []*tsdb.BufferedSeriesIterator
|
||||||
}
|
}
|
||||||
|
|
||||||
// NumberLiteral represents a number.
|
// NumberLiteral represents a number.
|
||||||
type NumberLiteral struct {
|
type NumberLiteral struct {
|
||||||
Val model.SampleValue
|
Val float64
|
||||||
}
|
}
|
||||||
|
|
||||||
// ParenExpr wraps an expression so it cannot be disassembled as a consequence
|
// ParenExpr wraps an expression so it cannot be disassembled as a consequence
|
||||||
|
@ -166,25 +166,26 @@ type UnaryExpr struct {
|
||||||
type VectorSelector struct {
|
type VectorSelector struct {
|
||||||
Name string
|
Name string
|
||||||
Offset time.Duration
|
Offset time.Duration
|
||||||
LabelMatchers metric.LabelMatchers
|
LabelMatchers []*LabelMatcher
|
||||||
|
|
||||||
// The series iterators are populated at query preparation time.
|
// The series iterators are populated at query preparation time.
|
||||||
iterators []local.SeriesIterator
|
series []tsdb.Series
|
||||||
|
iterators []*tsdb.BufferedSeriesIterator
|
||||||
}
|
}
|
||||||
|
|
||||||
func (e *AggregateExpr) Type() model.ValueType { return model.ValVector }
|
func (e *AggregateExpr) Type() ValueType { return ValueTypeVector }
|
||||||
func (e *Call) Type() model.ValueType { return e.Func.ReturnType }
|
func (e *Call) Type() ValueType { return e.Func.ReturnType }
|
||||||
func (e *MatrixSelector) Type() model.ValueType { return model.ValMatrix }
|
func (e *MatrixSelector) Type() ValueType { return ValueTypeMatrix }
|
||||||
func (e *NumberLiteral) Type() model.ValueType { return model.ValScalar }
|
func (e *NumberLiteral) Type() ValueType { return ValueTypeScalar }
|
||||||
func (e *ParenExpr) Type() model.ValueType { return e.Expr.Type() }
|
func (e *ParenExpr) Type() ValueType { return e.Expr.Type() }
|
||||||
func (e *StringLiteral) Type() model.ValueType { return model.ValString }
|
func (e *StringLiteral) Type() ValueType { return ValueTypeString }
|
||||||
func (e *UnaryExpr) Type() model.ValueType { return e.Expr.Type() }
|
func (e *UnaryExpr) Type() ValueType { return e.Expr.Type() }
|
||||||
func (e *VectorSelector) Type() model.ValueType { return model.ValVector }
|
func (e *VectorSelector) Type() ValueType { return ValueTypeVector }
|
||||||
func (e *BinaryExpr) Type() model.ValueType {
|
func (e *BinaryExpr) Type() ValueType {
|
||||||
if e.LHS.Type() == model.ValScalar && e.RHS.Type() == model.ValScalar {
|
if e.LHS.Type() == ValueTypeScalar && e.RHS.Type() == ValueTypeScalar {
|
||||||
return model.ValScalar
|
return ValueTypeScalar
|
||||||
}
|
}
|
||||||
return model.ValVector
|
return ValueTypeScalar
|
||||||
}
|
}
|
||||||
|
|
||||||
func (*AggregateExpr) expr() {}
|
func (*AggregateExpr) expr() {}
|
||||||
|
@ -229,13 +230,13 @@ type VectorMatching struct {
|
||||||
Card VectorMatchCardinality
|
Card VectorMatchCardinality
|
||||||
// MatchingLabels contains the labels which define equality of a pair of
|
// MatchingLabels contains the labels which define equality of a pair of
|
||||||
// elements from the vectors.
|
// elements from the vectors.
|
||||||
MatchingLabels model.LabelNames
|
MatchingLabels []string
|
||||||
// On includes the given label names from matching,
|
// On includes the given label names from matching,
|
||||||
// rather than excluding them.
|
// rather than excluding them.
|
||||||
On bool
|
On bool
|
||||||
// Include contains additional labels that should be included in
|
// Include contains additional labels that should be included in
|
||||||
// the result from the side with the lower cardinality.
|
// the result from the side with the lower cardinality.
|
||||||
Include model.LabelNames
|
Include []string
|
||||||
}
|
}
|
||||||
|
|
||||||
// Visitor allows visiting a Node and its child nodes. The Visit method is
|
// Visitor allows visiting a Node and its child nodes. The Visit method is
|
||||||
|
@ -315,3 +316,83 @@ func (f inspector) Visit(node Node) Visitor {
|
||||||
func Inspect(node Node, f func(Node) bool) {
|
func Inspect(node Node, f func(Node) bool) {
|
||||||
Walk(inspector(f), node)
|
Walk(inspector(f), node)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// MatchType is an enum for label matching types.
|
||||||
|
type MatchType int
|
||||||
|
|
||||||
|
// Possible MatchTypes.
|
||||||
|
const (
|
||||||
|
MatchEqual MatchType = iota
|
||||||
|
MatchNotEqual
|
||||||
|
MatchRegexp
|
||||||
|
MatchNotRegexp
|
||||||
|
)
|
||||||
|
|
||||||
|
func (m MatchType) String() string {
|
||||||
|
typeToStr := map[MatchType]string{
|
||||||
|
MatchEqual: "=",
|
||||||
|
MatchNotEqual: "!=",
|
||||||
|
MatchRegexp: "=~",
|
||||||
|
MatchNotRegexp: "!~",
|
||||||
|
}
|
||||||
|
if str, ok := typeToStr[m]; ok {
|
||||||
|
return str
|
||||||
|
}
|
||||||
|
panic("unknown match type")
|
||||||
|
}
|
||||||
|
|
||||||
|
// LabelMatcher models the matching of a label.
|
||||||
|
type LabelMatcher struct {
|
||||||
|
Type MatchType
|
||||||
|
Name string
|
||||||
|
Value string
|
||||||
|
|
||||||
|
re *regexp.Regexp
|
||||||
|
}
|
||||||
|
|
||||||
|
// NewLabelMatcher returns a LabelMatcher object ready to use.
|
||||||
|
func NewLabelMatcher(t MatchType, n, v string) (*LabelMatcher, error) {
|
||||||
|
m := &LabelMatcher{
|
||||||
|
Type: t,
|
||||||
|
Name: n,
|
||||||
|
Value: v,
|
||||||
|
}
|
||||||
|
if t == MatchRegexp || t == MatchNotRegexp {
|
||||||
|
m.Value = "^(?:" + v + ")$"
|
||||||
|
re, err := regexp.Compile(m.Value)
|
||||||
|
if err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
m.re = re
|
||||||
|
}
|
||||||
|
return m, nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (m *LabelMatcher) String() string {
|
||||||
|
return fmt.Sprintf("%s%s%q", m.Name, m.Type, m.Value)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (m *LabelMatcher) matcher() labels.Matcher {
|
||||||
|
switch m.Type {
|
||||||
|
case MatchEqual:
|
||||||
|
return labels.NewEqualMatcher(m.Name, m.Value)
|
||||||
|
|
||||||
|
case MatchNotEqual:
|
||||||
|
return labels.Not(labels.NewEqualMatcher(m.Name, m.Value))
|
||||||
|
|
||||||
|
case MatchRegexp:
|
||||||
|
res, err := labels.NewRegexpMatcher(m.Name, m.Value)
|
||||||
|
if err != nil {
|
||||||
|
panic(err)
|
||||||
|
}
|
||||||
|
return res
|
||||||
|
|
||||||
|
case MatchNotRegexp:
|
||||||
|
res, err := labels.NewRegexpMatcher(m.Name, m.Value)
|
||||||
|
if err != nil {
|
||||||
|
panic(err)
|
||||||
|
}
|
||||||
|
return labels.Not(res)
|
||||||
|
}
|
||||||
|
panic("promql.LabelMatcher.matcher: invalid matcher type")
|
||||||
|
}
|
||||||
|
|
802
promql/engine.go
802
promql/engine.go
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
116
promql/parse.go
116
promql/parse.go
|
@ -16,14 +16,15 @@ package promql
|
||||||
import (
|
import (
|
||||||
"fmt"
|
"fmt"
|
||||||
"runtime"
|
"runtime"
|
||||||
|
"sort"
|
||||||
"strconv"
|
"strconv"
|
||||||
"strings"
|
"strings"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/fabxc/tsdb/labels"
|
||||||
"github.com/prometheus/common/log"
|
"github.com/prometheus/common/log"
|
||||||
"github.com/prometheus/common/model"
|
"github.com/prometheus/common/model"
|
||||||
|
|
||||||
"github.com/prometheus/prometheus/storage/metric"
|
|
||||||
"github.com/prometheus/prometheus/util/strutil"
|
"github.com/prometheus/prometheus/util/strutil"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
@ -73,7 +74,7 @@ func ParseExpr(input string) (Expr, error) {
|
||||||
}
|
}
|
||||||
|
|
||||||
// ParseMetric parses the input into a metric
|
// ParseMetric parses the input into a metric
|
||||||
func ParseMetric(input string) (m model.Metric, err error) {
|
func ParseMetric(input string) (m labels.Labels, err error) {
|
||||||
p := newParser(input)
|
p := newParser(input)
|
||||||
defer p.recover(&err)
|
defer p.recover(&err)
|
||||||
|
|
||||||
|
@ -86,7 +87,7 @@ func ParseMetric(input string) (m model.Metric, err error) {
|
||||||
|
|
||||||
// ParseMetricSelector parses the provided textual metric selector into a list of
|
// ParseMetricSelector parses the provided textual metric selector into a list of
|
||||||
// label matchers.
|
// label matchers.
|
||||||
func ParseMetricSelector(input string) (m metric.LabelMatchers, err error) {
|
func ParseMetricSelector(input string) (m []*LabelMatcher, err error) {
|
||||||
p := newParser(input)
|
p := newParser(input)
|
||||||
defer p.recover(&err)
|
defer p.recover(&err)
|
||||||
|
|
||||||
|
@ -102,7 +103,7 @@ func ParseMetricSelector(input string) (m metric.LabelMatchers, err error) {
|
||||||
}
|
}
|
||||||
|
|
||||||
// parseSeriesDesc parses the description of a time series.
|
// parseSeriesDesc parses the description of a time series.
|
||||||
func parseSeriesDesc(input string) (model.Metric, []sequenceValue, error) {
|
func parseSeriesDesc(input string) (labels.Labels, []sequenceValue, error) {
|
||||||
p := newParser(input)
|
p := newParser(input)
|
||||||
p.lex.seriesDesc = true
|
p.lex.seriesDesc = true
|
||||||
|
|
||||||
|
@ -153,7 +154,7 @@ func (p *parser) parseExpr() (expr Expr, err error) {
|
||||||
|
|
||||||
// sequenceValue is an omittable value in a sequence of time series values.
|
// sequenceValue is an omittable value in a sequence of time series values.
|
||||||
type sequenceValue struct {
|
type sequenceValue struct {
|
||||||
value model.SampleValue
|
value float64
|
||||||
omitted bool
|
omitted bool
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -161,11 +162,11 @@ func (v sequenceValue) String() string {
|
||||||
if v.omitted {
|
if v.omitted {
|
||||||
return "_"
|
return "_"
|
||||||
}
|
}
|
||||||
return v.value.String()
|
return fmt.Sprintf("%f", v.value)
|
||||||
}
|
}
|
||||||
|
|
||||||
// parseSeriesDesc parses a description of a time series into its metric and value sequence.
|
// parseSeriesDesc parses a description of a time series into its metric and value sequence.
|
||||||
func (p *parser) parseSeriesDesc() (m model.Metric, vals []sequenceValue, err error) {
|
func (p *parser) parseSeriesDesc() (m labels.Labels, vals []sequenceValue, err error) {
|
||||||
defer p.recover(&err)
|
defer p.recover(&err)
|
||||||
|
|
||||||
m = p.metric()
|
m = p.metric()
|
||||||
|
@ -202,7 +203,7 @@ func (p *parser) parseSeriesDesc() (m model.Metric, vals []sequenceValue, err er
|
||||||
}
|
}
|
||||||
k := sign * p.number(p.expect(itemNumber, ctx).val)
|
k := sign * p.number(p.expect(itemNumber, ctx).val)
|
||||||
vals = append(vals, sequenceValue{
|
vals = append(vals, sequenceValue{
|
||||||
value: model.SampleValue(k),
|
value: k,
|
||||||
})
|
})
|
||||||
|
|
||||||
// If there are no offset repetitions specified, proceed with the next value.
|
// If there are no offset repetitions specified, proceed with the next value.
|
||||||
|
@ -230,7 +231,7 @@ func (p *parser) parseSeriesDesc() (m model.Metric, vals []sequenceValue, err er
|
||||||
for i := uint64(0); i < times; i++ {
|
for i := uint64(0); i < times; i++ {
|
||||||
k += offset
|
k += offset
|
||||||
vals = append(vals, sequenceValue{
|
vals = append(vals, sequenceValue{
|
||||||
value: model.SampleValue(k),
|
value: k,
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -385,12 +386,12 @@ func (p *parser) alertStmt() *AlertStmt {
|
||||||
}
|
}
|
||||||
|
|
||||||
var (
|
var (
|
||||||
labels = model.LabelSet{}
|
lset labels.Labels
|
||||||
annotations = model.LabelSet{}
|
annotations labels.Labels
|
||||||
)
|
)
|
||||||
if p.peek().typ == itemLabels {
|
if p.peek().typ == itemLabels {
|
||||||
p.expect(itemLabels, ctx)
|
p.expect(itemLabels, ctx)
|
||||||
labels = p.labelSet()
|
lset = p.labelSet()
|
||||||
}
|
}
|
||||||
if p.peek().typ == itemAnnotations {
|
if p.peek().typ == itemAnnotations {
|
||||||
p.expect(itemAnnotations, ctx)
|
p.expect(itemAnnotations, ctx)
|
||||||
|
@ -401,7 +402,7 @@ func (p *parser) alertStmt() *AlertStmt {
|
||||||
Name: name.val,
|
Name: name.val,
|
||||||
Expr: expr,
|
Expr: expr,
|
||||||
Duration: duration,
|
Duration: duration,
|
||||||
Labels: labels,
|
Labels: lset,
|
||||||
Annotations: annotations,
|
Annotations: annotations,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -412,7 +413,7 @@ func (p *parser) recordStmt() *RecordStmt {
|
||||||
|
|
||||||
name := p.expectOneOf(itemIdentifier, itemMetricIdentifier, ctx).val
|
name := p.expectOneOf(itemIdentifier, itemMetricIdentifier, ctx).val
|
||||||
|
|
||||||
var lset model.LabelSet
|
var lset labels.Labels
|
||||||
if p.peek().typ == itemLeftBrace {
|
if p.peek().typ == itemLeftBrace {
|
||||||
lset = p.labelSet()
|
lset = p.labelSet()
|
||||||
}
|
}
|
||||||
|
@ -504,7 +505,7 @@ func (p *parser) balance(lhs Expr, op itemType, rhs Expr, vecMatching *VectorMat
|
||||||
precd := lhsBE.Op.precedence() - op.precedence()
|
precd := lhsBE.Op.precedence() - op.precedence()
|
||||||
if (precd < 0) || (precd == 0 && op.isRightAssociative()) {
|
if (precd < 0) || (precd == 0 && op.isRightAssociative()) {
|
||||||
balanced := p.balance(lhsBE.RHS, op, rhs, vecMatching, returnBool)
|
balanced := p.balance(lhsBE.RHS, op, rhs, vecMatching, returnBool)
|
||||||
if lhsBE.Op.isComparisonOperator() && !lhsBE.ReturnBool && balanced.Type() == model.ValScalar && lhsBE.LHS.Type() == model.ValScalar {
|
if lhsBE.Op.isComparisonOperator() && !lhsBE.ReturnBool && balanced.Type() == ValueTypeScalar && lhsBE.LHS.Type() == ValueTypeScalar {
|
||||||
p.errorf("comparisons between scalars must use BOOL modifier")
|
p.errorf("comparisons between scalars must use BOOL modifier")
|
||||||
}
|
}
|
||||||
return &BinaryExpr{
|
return &BinaryExpr{
|
||||||
|
@ -516,7 +517,7 @@ func (p *parser) balance(lhs Expr, op itemType, rhs Expr, vecMatching *VectorMat
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if op.isComparisonOperator() && !returnBool && rhs.Type() == model.ValScalar && lhs.Type() == model.ValScalar {
|
if op.isComparisonOperator() && !returnBool && rhs.Type() == ValueTypeScalar && lhs.Type() == ValueTypeScalar {
|
||||||
p.errorf("comparisons between scalars must use BOOL modifier")
|
p.errorf("comparisons between scalars must use BOOL modifier")
|
||||||
}
|
}
|
||||||
return &BinaryExpr{
|
return &BinaryExpr{
|
||||||
|
@ -631,7 +632,7 @@ func (p *parser) primaryExpr() Expr {
|
||||||
switch t := p.next(); {
|
switch t := p.next(); {
|
||||||
case t.typ == itemNumber:
|
case t.typ == itemNumber:
|
||||||
f := p.number(t.val)
|
f := p.number(t.val)
|
||||||
return &NumberLiteral{model.SampleValue(f)}
|
return &NumberLiteral{f}
|
||||||
|
|
||||||
case t.typ == itemString:
|
case t.typ == itemString:
|
||||||
return &StringLiteral{p.unquoteString(t.val)}
|
return &StringLiteral{p.unquoteString(t.val)}
|
||||||
|
@ -665,19 +666,19 @@ func (p *parser) primaryExpr() Expr {
|
||||||
//
|
//
|
||||||
// '(' <label_name>, ... ')'
|
// '(' <label_name>, ... ')'
|
||||||
//
|
//
|
||||||
func (p *parser) labels() model.LabelNames {
|
func (p *parser) labels() []string {
|
||||||
const ctx = "grouping opts"
|
const ctx = "grouping opts"
|
||||||
|
|
||||||
p.expect(itemLeftParen, ctx)
|
p.expect(itemLeftParen, ctx)
|
||||||
|
|
||||||
labels := model.LabelNames{}
|
labels := []string{}
|
||||||
if p.peek().typ != itemRightParen {
|
if p.peek().typ != itemRightParen {
|
||||||
for {
|
for {
|
||||||
id := p.next()
|
id := p.next()
|
||||||
if !isLabel(id.val) {
|
if !isLabel(id.val) {
|
||||||
p.errorf("unexpected %s in %s, expected label", id.desc(), ctx)
|
p.errorf("unexpected %s in %s, expected label", id.desc(), ctx)
|
||||||
}
|
}
|
||||||
labels = append(labels, model.LabelName(id.val))
|
labels = append(labels, id.val)
|
||||||
|
|
||||||
if p.peek().typ != itemComma {
|
if p.peek().typ != itemComma {
|
||||||
break
|
break
|
||||||
|
@ -702,7 +703,7 @@ func (p *parser) aggrExpr() *AggregateExpr {
|
||||||
if !agop.typ.isAggregator() {
|
if !agop.typ.isAggregator() {
|
||||||
p.errorf("expected aggregation operator but got %s", agop)
|
p.errorf("expected aggregation operator but got %s", agop)
|
||||||
}
|
}
|
||||||
var grouping model.LabelNames
|
var grouping []string
|
||||||
var keepCommon, without bool
|
var keepCommon, without bool
|
||||||
|
|
||||||
modifiersFirst := false
|
modifiersFirst := false
|
||||||
|
@ -802,22 +803,22 @@ func (p *parser) call(name string) *Call {
|
||||||
//
|
//
|
||||||
// '{' [ <labelname> '=' <match_string>, ... ] '}'
|
// '{' [ <labelname> '=' <match_string>, ... ] '}'
|
||||||
//
|
//
|
||||||
func (p *parser) labelSet() model.LabelSet {
|
func (p *parser) labelSet() labels.Labels {
|
||||||
set := model.LabelSet{}
|
set := []labels.Label{}
|
||||||
for _, lm := range p.labelMatchers(itemEQL) {
|
for _, lm := range p.labelMatchers(itemEQL) {
|
||||||
set[lm.Name] = lm.Value
|
set = append(set, labels.Label{Name: lm.Name, Value: lm.Value})
|
||||||
}
|
}
|
||||||
return set
|
return labels.New(set...)
|
||||||
}
|
}
|
||||||
|
|
||||||
// labelMatchers parses a set of label matchers.
|
// labelMatchers parses a set of label matchers.
|
||||||
//
|
//
|
||||||
// '{' [ <labelname> <match_op> <match_string>, ... ] '}'
|
// '{' [ <labelname> <match_op> <match_string>, ... ] '}'
|
||||||
//
|
//
|
||||||
func (p *parser) labelMatchers(operators ...itemType) metric.LabelMatchers {
|
func (p *parser) labelMatchers(operators ...itemType) []*LabelMatcher {
|
||||||
const ctx = "label matching"
|
const ctx = "label matching"
|
||||||
|
|
||||||
matchers := metric.LabelMatchers{}
|
matchers := []*LabelMatcher{}
|
||||||
|
|
||||||
p.expect(itemLeftBrace, ctx)
|
p.expect(itemLeftBrace, ctx)
|
||||||
|
|
||||||
|
@ -847,25 +848,21 @@ func (p *parser) labelMatchers(operators ...itemType) metric.LabelMatchers {
|
||||||
val := p.unquoteString(p.expect(itemString, ctx).val)
|
val := p.unquoteString(p.expect(itemString, ctx).val)
|
||||||
|
|
||||||
// Map the item to the respective match type.
|
// Map the item to the respective match type.
|
||||||
var matchType metric.MatchType
|
var matchType MatchType
|
||||||
switch op {
|
switch op {
|
||||||
case itemEQL:
|
case itemEQL:
|
||||||
matchType = metric.Equal
|
matchType = MatchEqual
|
||||||
case itemNEQ:
|
case itemNEQ:
|
||||||
matchType = metric.NotEqual
|
matchType = MatchNotEqual
|
||||||
case itemEQLRegex:
|
case itemEQLRegex:
|
||||||
matchType = metric.RegexMatch
|
matchType = MatchRegexp
|
||||||
case itemNEQRegex:
|
case itemNEQRegex:
|
||||||
matchType = metric.RegexNoMatch
|
matchType = MatchNotRegexp
|
||||||
default:
|
default:
|
||||||
p.errorf("item %q is not a metric match type", op)
|
p.errorf("item %q is not a metric match type", op)
|
||||||
}
|
}
|
||||||
|
|
||||||
m, err := metric.NewLabelMatcher(
|
m, err := NewLabelMatcher(matchType, label.val, val)
|
||||||
matchType,
|
|
||||||
model.LabelName(label.val),
|
|
||||||
model.LabelValue(val),
|
|
||||||
)
|
|
||||||
if err != nil {
|
if err != nil {
|
||||||
p.error(err)
|
p.error(err)
|
||||||
}
|
}
|
||||||
|
@ -898,9 +895,9 @@ func (p *parser) labelMatchers(operators ...itemType) metric.LabelMatchers {
|
||||||
// <label_set>
|
// <label_set>
|
||||||
// <metric_identifier> [<label_set>]
|
// <metric_identifier> [<label_set>]
|
||||||
//
|
//
|
||||||
func (p *parser) metric() model.Metric {
|
func (p *parser) metric() labels.Labels {
|
||||||
name := ""
|
name := ""
|
||||||
m := model.Metric{}
|
var m labels.Labels
|
||||||
|
|
||||||
t := p.peek().typ
|
t := p.peek().typ
|
||||||
if t == itemIdentifier || t == itemMetricIdentifier {
|
if t == itemIdentifier || t == itemMetricIdentifier {
|
||||||
|
@ -911,10 +908,11 @@ func (p *parser) metric() model.Metric {
|
||||||
p.errorf("missing metric name or metric selector")
|
p.errorf("missing metric name or metric selector")
|
||||||
}
|
}
|
||||||
if t == itemLeftBrace {
|
if t == itemLeftBrace {
|
||||||
m = model.Metric(p.labelSet())
|
m = p.labelSet()
|
||||||
}
|
}
|
||||||
if name != "" {
|
if name != "" {
|
||||||
m[model.MetricNameLabel] = model.LabelValue(name)
|
m = append(m, labels.Label{Name: MetricNameLabel, Value: name})
|
||||||
|
sort.Sort(m)
|
||||||
}
|
}
|
||||||
return m
|
return m
|
||||||
}
|
}
|
||||||
|
@ -943,7 +941,7 @@ func (p *parser) offset() time.Duration {
|
||||||
// [<metric_identifier>] <label_matchers>
|
// [<metric_identifier>] <label_matchers>
|
||||||
//
|
//
|
||||||
func (p *parser) vectorSelector(name string) *VectorSelector {
|
func (p *parser) vectorSelector(name string) *VectorSelector {
|
||||||
var matchers metric.LabelMatchers
|
var matchers []*LabelMatcher
|
||||||
// Parse label matching if any.
|
// Parse label matching if any.
|
||||||
if t := p.peek(); t.typ == itemLeftBrace {
|
if t := p.peek(); t.typ == itemLeftBrace {
|
||||||
matchers = p.labelMatchers(itemEQL, itemNEQ, itemEQLRegex, itemNEQRegex)
|
matchers = p.labelMatchers(itemEQL, itemNEQ, itemEQLRegex, itemNEQRegex)
|
||||||
|
@ -956,7 +954,7 @@ func (p *parser) vectorSelector(name string) *VectorSelector {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
// Set name label matching.
|
// Set name label matching.
|
||||||
m, err := metric.NewLabelMatcher(metric.Equal, model.MetricNameLabel, model.LabelValue(name))
|
m, err := NewLabelMatcher(MatchEqual, MetricNameLabel, name)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
panic(err) // Must not happen with metric.Equal.
|
panic(err) // Must not happen with metric.Equal.
|
||||||
}
|
}
|
||||||
|
@ -970,7 +968,7 @@ func (p *parser) vectorSelector(name string) *VectorSelector {
|
||||||
// implicit selection of all metrics (e.g. by a typo).
|
// implicit selection of all metrics (e.g. by a typo).
|
||||||
notEmpty := false
|
notEmpty := false
|
||||||
for _, lm := range matchers {
|
for _, lm := range matchers {
|
||||||
if !lm.MatchesEmptyString() {
|
if !lm.matcher().Matches("") {
|
||||||
notEmpty = true
|
notEmpty = true
|
||||||
break
|
break
|
||||||
}
|
}
|
||||||
|
@ -987,7 +985,7 @@ func (p *parser) vectorSelector(name string) *VectorSelector {
|
||||||
|
|
||||||
// expectType checks the type of the node and raises an error if it
|
// expectType checks the type of the node and raises an error if it
|
||||||
// is not of the expected type.
|
// is not of the expected type.
|
||||||
func (p *parser) expectType(node Node, want model.ValueType, context string) {
|
func (p *parser) expectType(node Node, want ValueType, context string) {
|
||||||
t := p.checkType(node)
|
t := p.checkType(node)
|
||||||
if t != want {
|
if t != want {
|
||||||
p.errorf("expected type %s in %s, got %s", documentedType(want), context, documentedType(t))
|
p.errorf("expected type %s in %s, got %s", documentedType(want), context, documentedType(t))
|
||||||
|
@ -999,12 +997,12 @@ func (p *parser) expectType(node Node, want model.ValueType, context string) {
|
||||||
//
|
//
|
||||||
// Some of these checks are redundant as the the parsing stage does not allow
|
// 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.
|
// them, but the costs are small and might reveal errors when making changes.
|
||||||
func (p *parser) checkType(node Node) (typ model.ValueType) {
|
func (p *parser) checkType(node Node) (typ ValueType) {
|
||||||
// For expressions the type is determined by their Type function.
|
// For expressions the type is determined by their Type function.
|
||||||
// Statements and lists do not have a type but are not invalid either.
|
// Statements and lists do not have a type but are not invalid either.
|
||||||
switch n := node.(type) {
|
switch n := node.(type) {
|
||||||
case Statements, Expressions, Statement:
|
case Statements, Expressions, Statement:
|
||||||
typ = model.ValNone
|
typ = ValueTypeNone
|
||||||
case Expr:
|
case Expr:
|
||||||
typ = n.Type()
|
typ = n.Type()
|
||||||
default:
|
default:
|
||||||
|
@ -1016,27 +1014,27 @@ func (p *parser) checkType(node Node) (typ model.ValueType) {
|
||||||
switch n := node.(type) {
|
switch n := node.(type) {
|
||||||
case Statements:
|
case Statements:
|
||||||
for _, s := range n {
|
for _, s := range n {
|
||||||
p.expectType(s, model.ValNone, "statement list")
|
p.expectType(s, ValueTypeNone, "statement list")
|
||||||
}
|
}
|
||||||
case *AlertStmt:
|
case *AlertStmt:
|
||||||
p.expectType(n.Expr, model.ValVector, "alert statement")
|
p.expectType(n.Expr, ValueTypeVector, "alert statement")
|
||||||
|
|
||||||
case *EvalStmt:
|
case *EvalStmt:
|
||||||
ty := p.checkType(n.Expr)
|
ty := p.checkType(n.Expr)
|
||||||
if ty == model.ValNone {
|
if ty == ValueTypeNone {
|
||||||
p.errorf("evaluation statement must have a valid expression type but got %s", documentedType(ty))
|
p.errorf("evaluation statement must have a valid expression type but got %s", documentedType(ty))
|
||||||
}
|
}
|
||||||
|
|
||||||
case *RecordStmt:
|
case *RecordStmt:
|
||||||
ty := p.checkType(n.Expr)
|
ty := p.checkType(n.Expr)
|
||||||
if ty != model.ValVector && ty != model.ValScalar {
|
if ty != ValueTypeVector && ty != ValueTypeScalar {
|
||||||
p.errorf("record statement must have a valid expression of type instant vector or scalar but got %s", documentedType(ty))
|
p.errorf("record statement must have a valid expression of type instant vector or scalar but got %s", documentedType(ty))
|
||||||
}
|
}
|
||||||
|
|
||||||
case Expressions:
|
case Expressions:
|
||||||
for _, e := range n {
|
for _, e := range n {
|
||||||
ty := p.checkType(e)
|
ty := p.checkType(e)
|
||||||
if ty == model.ValNone {
|
if ty == ValueTypeNone {
|
||||||
p.errorf("expression must have a valid expression type but got %s", documentedType(ty))
|
p.errorf("expression must have a valid expression type but got %s", documentedType(ty))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -1044,12 +1042,12 @@ func (p *parser) checkType(node Node) (typ model.ValueType) {
|
||||||
if !n.Op.isAggregator() {
|
if !n.Op.isAggregator() {
|
||||||
p.errorf("aggregation operator expected in aggregation expression but got %q", n.Op)
|
p.errorf("aggregation operator expected in aggregation expression but got %q", n.Op)
|
||||||
}
|
}
|
||||||
p.expectType(n.Expr, model.ValVector, "aggregation expression")
|
p.expectType(n.Expr, ValueTypeVector, "aggregation expression")
|
||||||
if n.Op == itemTopK || n.Op == itemBottomK || n.Op == itemQuantile {
|
if n.Op == itemTopK || n.Op == itemBottomK || n.Op == itemQuantile {
|
||||||
p.expectType(n.Param, model.ValScalar, "aggregation parameter")
|
p.expectType(n.Param, ValueTypeScalar, "aggregation parameter")
|
||||||
}
|
}
|
||||||
if n.Op == itemCountValues {
|
if n.Op == itemCountValues {
|
||||||
p.expectType(n.Param, model.ValString, "aggregation parameter")
|
p.expectType(n.Param, ValueTypeString, "aggregation parameter")
|
||||||
}
|
}
|
||||||
|
|
||||||
case *BinaryExpr:
|
case *BinaryExpr:
|
||||||
|
@ -1059,11 +1057,11 @@ func (p *parser) checkType(node Node) (typ model.ValueType) {
|
||||||
if !n.Op.isOperator() {
|
if !n.Op.isOperator() {
|
||||||
p.errorf("binary expression does not support operator %q", n.Op)
|
p.errorf("binary expression does not support operator %q", n.Op)
|
||||||
}
|
}
|
||||||
if (lt != model.ValScalar && lt != model.ValVector) || (rt != model.ValScalar && rt != model.ValVector) {
|
if (lt != ValueTypeScalar && lt != ValueTypeVector) || (rt != ValueTypeScalar && rt != ValueTypeVector) {
|
||||||
p.errorf("binary expression must contain only scalar and instant vector types")
|
p.errorf("binary expression must contain only scalar and instant vector types")
|
||||||
}
|
}
|
||||||
|
|
||||||
if (lt != model.ValVector || rt != model.ValVector) && n.VectorMatching != nil {
|
if (lt != ValueTypeVector || rt != ValueTypeVector) && n.VectorMatching != nil {
|
||||||
if len(n.VectorMatching.MatchingLabels) > 0 {
|
if len(n.VectorMatching.MatchingLabels) > 0 {
|
||||||
p.errorf("vector matching only allowed between instant vectors")
|
p.errorf("vector matching only allowed between instant vectors")
|
||||||
}
|
}
|
||||||
|
@ -1080,7 +1078,7 @@ func (p *parser) checkType(node Node) (typ model.ValueType) {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (lt == model.ValScalar || rt == model.ValScalar) && n.Op.isSetOperator() {
|
if (lt == ValueTypeScalar || rt == ValueTypeScalar) && n.Op.isSetOperator() {
|
||||||
p.errorf("set operator %q not allowed in binary scalar expression", n.Op)
|
p.errorf("set operator %q not allowed in binary scalar expression", n.Op)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1103,7 +1101,7 @@ func (p *parser) checkType(node Node) (typ model.ValueType) {
|
||||||
if n.Op != itemADD && n.Op != itemSUB {
|
if n.Op != itemADD && n.Op != itemSUB {
|
||||||
p.errorf("only + and - operators allowed for unary expressions")
|
p.errorf("only + and - operators allowed for unary expressions")
|
||||||
}
|
}
|
||||||
if t := p.checkType(n.Expr); t != model.ValScalar && t != model.ValVector {
|
if t := p.checkType(n.Expr); t != ValueTypeScalar && t != ValueTypeVector {
|
||||||
p.errorf("unary expression only allowed on expressions of type scalar or instant vector, got %q", documentedType(t))
|
p.errorf("unary expression only allowed on expressions of type scalar or instant vector, got %q", documentedType(t))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -20,8 +20,6 @@ import (
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/prometheus/common/model"
|
"github.com/prometheus/common/model"
|
||||||
|
|
||||||
"github.com/prometheus/prometheus/storage/metric"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
// Tree returns a string of the tree structure of the given node.
|
// Tree returns a string of the tree structure of the given node.
|
||||||
|
@ -218,7 +216,7 @@ func (node *VectorSelector) String() string {
|
||||||
labelStrings := make([]string, 0, len(node.LabelMatchers)-1)
|
labelStrings := make([]string, 0, len(node.LabelMatchers)-1)
|
||||||
for _, matcher := range node.LabelMatchers {
|
for _, matcher := range node.LabelMatchers {
|
||||||
// Only include the __name__ label if its no equality matching.
|
// Only include the __name__ label if its no equality matching.
|
||||||
if matcher.Name == model.MetricNameLabel && matcher.Type == metric.Equal {
|
if matcher.Name == MetricNameLabel && matcher.Type == MatchEqual {
|
||||||
continue
|
continue
|
||||||
}
|
}
|
||||||
labelStrings = append(labelStrings, matcher.String())
|
labelStrings = append(labelStrings, matcher.String())
|
||||||
|
|
|
@ -17,8 +17,7 @@ import (
|
||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/prometheus/common/model"
|
"github.com/fabxc/tsdb/labels"
|
||||||
"github.com/prometheus/prometheus/storage/metric"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
func TestStatementString(t *testing.T) {
|
func TestStatementString(t *testing.T) {
|
||||||
|
@ -28,17 +27,15 @@ func TestStatementString(t *testing.T) {
|
||||||
Op: itemGTR,
|
Op: itemGTR,
|
||||||
LHS: &VectorSelector{
|
LHS: &VectorSelector{
|
||||||
Name: "foo",
|
Name: "foo",
|
||||||
LabelMatchers: metric.LabelMatchers{
|
LabelMatchers: []*LabelMatcher{
|
||||||
{Type: metric.Equal, Name: model.MetricNameLabel, Value: "bar"},
|
{Type: MatchEqual, Name: MetricNameLabel, Value: "bar"},
|
||||||
},
|
},
|
||||||
},
|
},
|
||||||
RHS: &NumberLiteral{10},
|
RHS: &NumberLiteral{10},
|
||||||
},
|
},
|
||||||
Duration: 5 * time.Minute,
|
Duration: 5 * time.Minute,
|
||||||
Labels: model.LabelSet{"foo": "bar"},
|
Labels: labels.FromStrings("foo", "bar"),
|
||||||
Annotations: model.LabelSet{
|
Annotations: labels.FromStrings("notify", "team-a"),
|
||||||
"notify": "team-a",
|
|
||||||
},
|
|
||||||
}
|
}
|
||||||
|
|
||||||
expected := `ALERT FooAlert
|
expected := `ALERT FooAlert
|
||||||
|
|
|
@ -17,23 +17,22 @@ import (
|
||||||
"math"
|
"math"
|
||||||
"sort"
|
"sort"
|
||||||
|
|
||||||
|
"github.com/fabxc/tsdb/labels"
|
||||||
"github.com/prometheus/common/model"
|
"github.com/prometheus/common/model"
|
||||||
|
|
||||||
"github.com/prometheus/prometheus/storage/metric"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
// Helpers to calculate quantiles.
|
// Helpers to calculate quantiles.
|
||||||
|
|
||||||
// excludedLabels are the labels to exclude from signature calculation for
|
// excludedLabels are the labels to exclude from signature calculation for
|
||||||
// quantiles.
|
// quantiles.
|
||||||
var excludedLabels = map[model.LabelName]struct{}{
|
var excludedLabels = []string{
|
||||||
model.MetricNameLabel: {},
|
model.MetricNameLabel,
|
||||||
model.BucketLabel: {},
|
model.BucketLabel,
|
||||||
}
|
}
|
||||||
|
|
||||||
type bucket struct {
|
type bucket struct {
|
||||||
upperBound float64
|
upperBound float64
|
||||||
count model.SampleValue
|
count float64
|
||||||
}
|
}
|
||||||
|
|
||||||
// buckets implements sort.Interface.
|
// buckets implements sort.Interface.
|
||||||
|
@ -44,7 +43,7 @@ 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 }
|
func (b buckets) Less(i, j int) bool { return b[i].upperBound < b[j].upperBound }
|
||||||
|
|
||||||
type metricWithBuckets struct {
|
type metricWithBuckets struct {
|
||||||
metric metric.Metric
|
metric labels.Labels
|
||||||
buckets buckets
|
buckets buckets
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -70,7 +69,7 @@ type metricWithBuckets struct {
|
||||||
// If q<0, -Inf is returned.
|
// If q<0, -Inf is returned.
|
||||||
//
|
//
|
||||||
// If q>1, +Inf is returned.
|
// If q>1, +Inf is returned.
|
||||||
func bucketQuantile(q model.SampleValue, buckets buckets) float64 {
|
func bucketQuantile(q float64, buckets buckets) float64 {
|
||||||
if q < 0 {
|
if q < 0 {
|
||||||
return math.Inf(-1)
|
return math.Inf(-1)
|
||||||
}
|
}
|
||||||
|
|
100
promql/test.go
100
promql/test.go
|
@ -22,6 +22,7 @@ import (
|
||||||
"strings"
|
"strings"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
"github.com/fabxc/tsdb/labels"
|
||||||
"github.com/prometheus/common/model"
|
"github.com/prometheus/common/model"
|
||||||
"golang.org/x/net/context"
|
"golang.org/x/net/context"
|
||||||
|
|
||||||
|
@ -39,10 +40,11 @@ var (
|
||||||
)
|
)
|
||||||
|
|
||||||
const (
|
const (
|
||||||
testStartTime = model.Time(0)
|
epsilon = 0.000001 // Relative error allowed for sample values.
|
||||||
epsilon = 0.000001 // Relative error allowed for sample values.
|
|
||||||
)
|
)
|
||||||
|
|
||||||
|
var testStartTime = time.Time{}
|
||||||
|
|
||||||
// Test is a sequence of read and write commands that are run
|
// Test is a sequence of read and write commands that are run
|
||||||
// against a test storage.
|
// against a test storage.
|
||||||
type Test struct {
|
type Test struct {
|
||||||
|
@ -170,7 +172,7 @@ func (t *Test) parseEval(lines []string, i int) (int, *evalCmd, error) {
|
||||||
break
|
break
|
||||||
}
|
}
|
||||||
if f, err := parseNumber(defLine); err == nil {
|
if f, err := parseNumber(defLine); err == nil {
|
||||||
cmd.expect(0, nil, sequenceValue{value: model.SampleValue(f)})
|
cmd.expect(0, nil, sequenceValue{value: f})
|
||||||
break
|
break
|
||||||
}
|
}
|
||||||
metric, vals, err := parseSeriesDesc(defLine)
|
metric, vals, err := parseSeriesDesc(defLine)
|
||||||
|
@ -243,15 +245,15 @@ func (*evalCmd) testCmd() {}
|
||||||
// metrics into the storage.
|
// metrics into the storage.
|
||||||
type loadCmd struct {
|
type loadCmd struct {
|
||||||
gap time.Duration
|
gap time.Duration
|
||||||
metrics map[model.Fingerprint]model.Metric
|
metrics map[uint64]labels.Labels
|
||||||
defs map[model.Fingerprint][]model.SamplePair
|
defs map[uint64][]samplePair
|
||||||
}
|
}
|
||||||
|
|
||||||
func newLoadCmd(gap time.Duration) *loadCmd {
|
func newLoadCmd(gap time.Duration) *loadCmd {
|
||||||
return &loadCmd{
|
return &loadCmd{
|
||||||
gap: gap,
|
gap: gap,
|
||||||
metrics: map[model.Fingerprint]model.Metric{},
|
metrics: map[uint64]labels.Labels{},
|
||||||
defs: map[model.Fingerprint][]model.SamplePair{},
|
defs: map[uint64][]samplePair{},
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -260,51 +262,52 @@ func (cmd loadCmd) String() string {
|
||||||
}
|
}
|
||||||
|
|
||||||
// set a sequence of sample values for the given metric.
|
// set a sequence of sample values for the given metric.
|
||||||
func (cmd *loadCmd) set(m model.Metric, vals ...sequenceValue) {
|
func (cmd *loadCmd) set(m labels.Labels, vals ...sequenceValue) {
|
||||||
fp := m.Fingerprint()
|
h := m.Hash()
|
||||||
|
|
||||||
samples := make([]model.SamplePair, 0, len(vals))
|
samples := make([]samplePair, 0, len(vals))
|
||||||
ts := testStartTime
|
ts := testStartTime
|
||||||
for _, v := range vals {
|
for _, v := range vals {
|
||||||
if !v.omitted {
|
if !v.omitted {
|
||||||
samples = append(samples, model.SamplePair{
|
samples = append(samples, samplePair{
|
||||||
Timestamp: ts,
|
t: ts.UnixNano() / int64(time.Millisecond/time.Nanosecond),
|
||||||
Value: v.value,
|
v: v.value,
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
ts = ts.Add(cmd.gap)
|
ts = ts.Add(cmd.gap)
|
||||||
}
|
}
|
||||||
cmd.defs[fp] = samples
|
cmd.defs[h] = samples
|
||||||
cmd.metrics[fp] = m
|
cmd.metrics[h] = m
|
||||||
}
|
}
|
||||||
|
|
||||||
// append the defined time series to the storage.
|
// append the defined time series to the storage.
|
||||||
func (cmd *loadCmd) append(a storage.SampleAppender) {
|
func (cmd *loadCmd) append(a storage.SampleAppender) {
|
||||||
for fp, samples := range cmd.defs {
|
// TODO(fabxc): commented out until Appender refactoring.
|
||||||
met := cmd.metrics[fp]
|
// for fp, samples := range cmd.defs {
|
||||||
for _, smpl := range samples {
|
// met := cmd.metrics[fp]
|
||||||
s := &model.Sample{
|
// for _, smpl := range samples {
|
||||||
Metric: met,
|
// s := &model.Sample{
|
||||||
Value: smpl.Value,
|
// Metric: met,
|
||||||
Timestamp: smpl.Timestamp,
|
// Value: smpl.Value,
|
||||||
}
|
// Timestamp: smpl.Timestamp,
|
||||||
a.Append(s)
|
// }
|
||||||
}
|
// a.Append(s)
|
||||||
}
|
// }
|
||||||
|
// }
|
||||||
}
|
}
|
||||||
|
|
||||||
// evalCmd is a command that evaluates an expression for the given time (range)
|
// evalCmd is a command that evaluates an expression for the given time (range)
|
||||||
// and expects a specific result.
|
// and expects a specific result.
|
||||||
type evalCmd struct {
|
type evalCmd struct {
|
||||||
expr Expr
|
expr Expr
|
||||||
start, end model.Time
|
start, end time.Time
|
||||||
interval time.Duration
|
interval time.Duration
|
||||||
|
|
||||||
instant bool
|
instant bool
|
||||||
fail, ordered bool
|
fail, ordered bool
|
||||||
|
|
||||||
metrics map[model.Fingerprint]model.Metric
|
metrics map[uint64]labels.Labels
|
||||||
expected map[model.Fingerprint]entry
|
expected map[uint64]entry
|
||||||
}
|
}
|
||||||
|
|
||||||
type entry struct {
|
type entry struct {
|
||||||
|
@ -316,7 +319,7 @@ func (e entry) String() string {
|
||||||
return fmt.Sprintf("%d: %s", e.pos, e.vals)
|
return fmt.Sprintf("%d: %s", e.pos, e.vals)
|
||||||
}
|
}
|
||||||
|
|
||||||
func newEvalCmd(expr Expr, start, end model.Time, interval time.Duration) *evalCmd {
|
func newEvalCmd(expr Expr, start, end time.Time, interval time.Duration) *evalCmd {
|
||||||
return &evalCmd{
|
return &evalCmd{
|
||||||
expr: expr,
|
expr: expr,
|
||||||
start: start,
|
start: start,
|
||||||
|
@ -324,8 +327,8 @@ func newEvalCmd(expr Expr, start, end model.Time, interval time.Duration) *evalC
|
||||||
interval: interval,
|
interval: interval,
|
||||||
instant: start == end && interval == 0,
|
instant: start == end && interval == 0,
|
||||||
|
|
||||||
metrics: map[model.Fingerprint]model.Metric{},
|
metrics: map[uint64]labels.Labels{},
|
||||||
expected: map[model.Fingerprint]entry{},
|
expected: map[uint64]entry{},
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -335,26 +338,26 @@ func (ev *evalCmd) String() string {
|
||||||
|
|
||||||
// expect adds a new metric with a sequence of values to the set of expected
|
// expect adds a new metric with a sequence of values to the set of expected
|
||||||
// results for the query.
|
// results for the query.
|
||||||
func (ev *evalCmd) expect(pos int, m model.Metric, vals ...sequenceValue) {
|
func (ev *evalCmd) expect(pos int, m labels.Labels, vals ...sequenceValue) {
|
||||||
if m == nil {
|
if m == nil {
|
||||||
ev.expected[0] = entry{pos: pos, vals: vals}
|
ev.expected[0] = entry{pos: pos, vals: vals}
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
fp := m.Fingerprint()
|
h := m.Hash()
|
||||||
ev.metrics[fp] = m
|
ev.metrics[h] = m
|
||||||
ev.expected[fp] = entry{pos: pos, vals: vals}
|
ev.expected[h] = entry{pos: pos, vals: vals}
|
||||||
}
|
}
|
||||||
|
|
||||||
// compareResult compares the result value with the defined expectation.
|
// compareResult compares the result value with the defined expectation.
|
||||||
func (ev *evalCmd) compareResult(result model.Value) error {
|
func (ev *evalCmd) compareResult(result Value) error {
|
||||||
switch val := result.(type) {
|
switch val := result.(type) {
|
||||||
case model.Matrix:
|
case matrix:
|
||||||
if ev.instant {
|
if ev.instant {
|
||||||
return fmt.Errorf("received range result on instant evaluation")
|
return fmt.Errorf("received range result on instant evaluation")
|
||||||
}
|
}
|
||||||
seen := map[model.Fingerprint]bool{}
|
seen := map[uint64]bool{}
|
||||||
for pos, v := range val {
|
for pos, v := range val {
|
||||||
fp := v.Metric.Fingerprint()
|
fp := v.Metric.Hash()
|
||||||
if _, ok := ev.metrics[fp]; !ok {
|
if _, ok := ev.metrics[fp]; !ok {
|
||||||
return fmt.Errorf("unexpected metric %s in result", v.Metric)
|
return fmt.Errorf("unexpected metric %s in result", v.Metric)
|
||||||
}
|
}
|
||||||
|
@ -363,7 +366,7 @@ func (ev *evalCmd) compareResult(result model.Value) error {
|
||||||
return fmt.Errorf("expected metric %s with %v at position %d but was at %d", v.Metric, exp.vals, exp.pos, pos+1)
|
return fmt.Errorf("expected metric %s with %v at position %d but was at %d", v.Metric, exp.vals, exp.pos, pos+1)
|
||||||
}
|
}
|
||||||
for i, expVal := range exp.vals {
|
for i, expVal := range exp.vals {
|
||||||
if !almostEqual(float64(expVal.value), float64(v.Values[i].Value)) {
|
if !almostEqual(expVal.value, v.Values[i].v) {
|
||||||
return fmt.Errorf("expected %v for %s but got %v", expVal, v.Metric, v.Values)
|
return fmt.Errorf("expected %v for %s but got %v", expVal, v.Metric, v.Values)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -375,13 +378,13 @@ func (ev *evalCmd) compareResult(result model.Value) error {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
case model.Vector:
|
case vector:
|
||||||
if !ev.instant {
|
if !ev.instant {
|
||||||
return fmt.Errorf("received instant result on range evaluation")
|
return fmt.Errorf("received instant result on range evaluation")
|
||||||
}
|
}
|
||||||
seen := map[model.Fingerprint]bool{}
|
seen := map[uint64]bool{}
|
||||||
for pos, v := range val {
|
for pos, v := range val {
|
||||||
fp := v.Metric.Fingerprint()
|
fp := v.Metric.Hash()
|
||||||
if _, ok := ev.metrics[fp]; !ok {
|
if _, ok := ev.metrics[fp]; !ok {
|
||||||
return fmt.Errorf("unexpected metric %s in result", v.Metric)
|
return fmt.Errorf("unexpected metric %s in result", v.Metric)
|
||||||
}
|
}
|
||||||
|
@ -401,9 +404,9 @@ func (ev *evalCmd) compareResult(result model.Value) error {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
case *model.Scalar:
|
case scalar:
|
||||||
if !almostEqual(float64(ev.expected[0].vals[0].value), float64(val.Value)) {
|
if !almostEqual(ev.expected[0].vals[0].value, val.v) {
|
||||||
return fmt.Errorf("expected scalar %v but got %v", val.Value, ev.expected[0].vals[0].value)
|
return fmt.Errorf("expected scalar %v but got %v", val.v, ev.expected[0].vals[0].value)
|
||||||
}
|
}
|
||||||
|
|
||||||
default:
|
default:
|
||||||
|
@ -506,7 +509,8 @@ func (t *Test) clear() {
|
||||||
t.storage, closer = local.NewTestStorage(t, 2)
|
t.storage, closer = local.NewTestStorage(t, 2)
|
||||||
|
|
||||||
t.closeStorage = closer.Close
|
t.closeStorage = closer.Close
|
||||||
t.queryEngine = NewEngine(t.storage, nil)
|
// TODO(fabxc): add back
|
||||||
|
// t.queryEngine = NewEngine(t.storage, nil)
|
||||||
t.context, t.cancelCtx = context.WithCancel(context.Background())
|
t.context, t.cancelCtx = context.WithCancel(context.Background())
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue