promql: migrate model types, use tsdb interfaces

pull/2643/head
Fabian Reinartz 2016-12-23 13:51:59 +01:00
parent 8b84ee5ee6
commit 15a931dbdb
10 changed files with 1335 additions and 1113 deletions

View File

@ -24,6 +24,7 @@ import (
"syscall"
"time"
"github.com/fabxc/tsdb"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/common/log"
"github.com/prometheus/common/model"
@ -82,6 +83,10 @@ func Main() int {
reloadables []Reloadable
)
_, err := tsdb.Open(cfg.localStoragePath, nil, nil)
if err != nil {
log.Errorf("Opening storage failed: %s", err)
}
var localStorage local.Storage
reloadableRemoteStorage := remote.New()

View File

@ -15,12 +15,11 @@ package promql
import (
"fmt"
"regexp"
"time"
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/storage/local"
"github.com/prometheus/prometheus/storage/metric"
"github.com/fabxc/tsdb"
"github.com/fabxc/tsdb/labels"
)
// Node is a generic interface for all nodes in an AST.
@ -58,8 +57,8 @@ type AlertStmt struct {
Name string
Expr Expr
Duration time.Duration
Labels model.LabelSet
Annotations model.LabelSet
Labels labels.Labels
Annotations labels.Labels
}
// 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
// is evaluated.
Start, End model.Time
Start, End time.Time
// Time between two evaluated instants for the range [Start:End].
Interval time.Duration
}
@ -78,7 +77,7 @@ type EvalStmt struct {
type RecordStmt struct {
Name string
Expr Expr
Labels model.LabelSet
Labels labels.Labels
}
func (*AlertStmt) stmt() {}
@ -91,7 +90,7 @@ type Expr interface {
// Type returns the type the expression evaluates to. It does not perform
// 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()
}
@ -101,12 +100,12 @@ 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.
Param Expr // Parameter used by some aggregators.
Grouping model.LabelNames // The labels by which to group the vector.
Without bool // Whether to drop the given labels rather than keep them.
KeepCommonLabels bool // Whether to keep common labels among result elements.
Op itemType // The used aggregation operation.
Expr Expr // The vector expression over which is aggregated.
Param Expr // Parameter used by some aggregators.
Grouping []string // The labels by which to group the vector.
Without bool // Whether to drop the given labels rather than keep them.
KeepCommonLabels bool // Whether to keep common labels among result elements.
}
// BinaryExpr represents a binary expression between two child expressions.
@ -133,15 +132,16 @@ type MatrixSelector struct {
Name string
Range time.Duration
Offset time.Duration
LabelMatchers metric.LabelMatchers
LabelMatchers []*LabelMatcher
// The series iterators are populated at query preparation time.
iterators []local.SeriesIterator
series []tsdb.Series
iterators []*tsdb.BufferedSeriesIterator
}
// NumberLiteral represents a number.
type NumberLiteral struct {
Val model.SampleValue
Val float64
}
// ParenExpr wraps an expression so it cannot be disassembled as a consequence
@ -166,25 +166,26 @@ type UnaryExpr struct {
type VectorSelector struct {
Name string
Offset time.Duration
LabelMatchers metric.LabelMatchers
LabelMatchers []*LabelMatcher
// 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 *Call) Type() model.ValueType { return e.Func.ReturnType }
func (e *MatrixSelector) Type() model.ValueType { return model.ValMatrix }
func (e *NumberLiteral) Type() model.ValueType { return model.ValScalar }
func (e *ParenExpr) Type() model.ValueType { return e.Expr.Type() }
func (e *StringLiteral) Type() model.ValueType { return model.ValString }
func (e *UnaryExpr) Type() model.ValueType { return e.Expr.Type() }
func (e *VectorSelector) Type() model.ValueType { return model.ValVector }
func (e *BinaryExpr) Type() model.ValueType {
if e.LHS.Type() == model.ValScalar && e.RHS.Type() == model.ValScalar {
return model.ValScalar
func (e *AggregateExpr) Type() ValueType { return ValueTypeVector }
func (e *Call) Type() ValueType { return e.Func.ReturnType }
func (e *MatrixSelector) Type() ValueType { return ValueTypeMatrix }
func (e *NumberLiteral) Type() ValueType { return ValueTypeScalar }
func (e *ParenExpr) Type() ValueType { return e.Expr.Type() }
func (e *StringLiteral) Type() ValueType { return ValueTypeString }
func (e *UnaryExpr) Type() ValueType { return e.Expr.Type() }
func (e *VectorSelector) Type() ValueType { return ValueTypeVector }
func (e *BinaryExpr) Type() ValueType {
if e.LHS.Type() == ValueTypeScalar && e.RHS.Type() == ValueTypeScalar {
return ValueTypeScalar
}
return model.ValVector
return ValueTypeScalar
}
func (*AggregateExpr) expr() {}
@ -229,13 +230,13 @@ type VectorMatching struct {
Card VectorMatchCardinality
// MatchingLabels contains the labels which define equality of a pair of
// elements from the vectors.
MatchingLabels model.LabelNames
MatchingLabels []string
// On includes the given label names from matching,
// rather than excluding them.
On bool
// Include contains additional labels that should be included in
// 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
@ -315,3 +316,83 @@ func (f inspector) Visit(node Node) Visitor {
func Inspect(node Node, f func(Node) bool) {
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")
}

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

View File

@ -16,14 +16,15 @@ package promql
import (
"fmt"
"runtime"
"sort"
"strconv"
"strings"
"time"
"github.com/fabxc/tsdb/labels"
"github.com/prometheus/common/log"
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/storage/metric"
"github.com/prometheus/prometheus/util/strutil"
)
@ -73,7 +74,7 @@ func ParseExpr(input string) (Expr, error) {
}
// 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)
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
// label matchers.
func ParseMetricSelector(input string) (m metric.LabelMatchers, err error) {
func ParseMetricSelector(input string) (m []*LabelMatcher, err error) {
p := newParser(input)
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.
func parseSeriesDesc(input string) (model.Metric, []sequenceValue, error) {
func parseSeriesDesc(input string) (labels.Labels, []sequenceValue, error) {
p := newParser(input)
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.
type sequenceValue struct {
value model.SampleValue
value float64
omitted bool
}
@ -161,11 +162,11 @@ func (v sequenceValue) String() string {
if v.omitted {
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.
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)
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)
vals = append(vals, sequenceValue{
value: model.SampleValue(k),
value: k,
})
// 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++ {
k += offset
vals = append(vals, sequenceValue{
value: model.SampleValue(k),
value: k,
})
}
}
@ -385,12 +386,12 @@ func (p *parser) alertStmt() *AlertStmt {
}
var (
labels = model.LabelSet{}
annotations = model.LabelSet{}
lset labels.Labels
annotations labels.Labels
)
if p.peek().typ == itemLabels {
p.expect(itemLabels, ctx)
labels = p.labelSet()
lset = p.labelSet()
}
if p.peek().typ == itemAnnotations {
p.expect(itemAnnotations, ctx)
@ -401,7 +402,7 @@ func (p *parser) alertStmt() *AlertStmt {
Name: name.val,
Expr: expr,
Duration: duration,
Labels: labels,
Labels: lset,
Annotations: annotations,
}
}
@ -412,7 +413,7 @@ func (p *parser) recordStmt() *RecordStmt {
name := p.expectOneOf(itemIdentifier, itemMetricIdentifier, ctx).val
var lset model.LabelSet
var lset labels.Labels
if p.peek().typ == itemLeftBrace {
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()
if (precd < 0) || (precd == 0 && op.isRightAssociative()) {
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")
}
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")
}
return &BinaryExpr{
@ -631,7 +632,7 @@ func (p *parser) primaryExpr() Expr {
switch t := p.next(); {
case t.typ == itemNumber:
f := p.number(t.val)
return &NumberLiteral{model.SampleValue(f)}
return &NumberLiteral{f}
case t.typ == itemString:
return &StringLiteral{p.unquoteString(t.val)}
@ -665,19 +666,19 @@ func (p *parser) primaryExpr() Expr {
//
// '(' <label_name>, ... ')'
//
func (p *parser) labels() model.LabelNames {
func (p *parser) labels() []string {
const ctx = "grouping opts"
p.expect(itemLeftParen, ctx)
labels := model.LabelNames{}
labels := []string{}
if p.peek().typ != itemRightParen {
for {
id := p.next()
if !isLabel(id.val) {
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 {
break
@ -702,7 +703,7 @@ func (p *parser) aggrExpr() *AggregateExpr {
if !agop.typ.isAggregator() {
p.errorf("expected aggregation operator but got %s", agop)
}
var grouping model.LabelNames
var grouping []string
var keepCommon, without bool
modifiersFirst := false
@ -802,22 +803,22 @@ func (p *parser) call(name string) *Call {
//
// '{' [ <labelname> '=' <match_string>, ... ] '}'
//
func (p *parser) labelSet() model.LabelSet {
set := model.LabelSet{}
func (p *parser) labelSet() labels.Labels {
set := []labels.Label{}
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.
//
// '{' [ <labelname> <match_op> <match_string>, ... ] '}'
//
func (p *parser) labelMatchers(operators ...itemType) metric.LabelMatchers {
func (p *parser) labelMatchers(operators ...itemType) []*LabelMatcher {
const ctx = "label matching"
matchers := metric.LabelMatchers{}
matchers := []*LabelMatcher{}
p.expect(itemLeftBrace, ctx)
@ -847,25 +848,21 @@ func (p *parser) labelMatchers(operators ...itemType) metric.LabelMatchers {
val := p.unquoteString(p.expect(itemString, ctx).val)
// Map the item to the respective match type.
var matchType metric.MatchType
var matchType MatchType
switch op {
case itemEQL:
matchType = metric.Equal
matchType = MatchEqual
case itemNEQ:
matchType = metric.NotEqual
matchType = MatchNotEqual
case itemEQLRegex:
matchType = metric.RegexMatch
matchType = MatchRegexp
case itemNEQRegex:
matchType = metric.RegexNoMatch
matchType = MatchNotRegexp
default:
p.errorf("item %q is not a metric match type", op)
}
m, err := metric.NewLabelMatcher(
matchType,
model.LabelName(label.val),
model.LabelValue(val),
)
m, err := NewLabelMatcher(matchType, label.val, val)
if err != nil {
p.error(err)
}
@ -898,9 +895,9 @@ func (p *parser) labelMatchers(operators ...itemType) metric.LabelMatchers {
// <label_set>
// <metric_identifier> [<label_set>]
//
func (p *parser) metric() model.Metric {
func (p *parser) metric() labels.Labels {
name := ""
m := model.Metric{}
var m labels.Labels
t := p.peek().typ
if t == itemIdentifier || t == itemMetricIdentifier {
@ -911,10 +908,11 @@ func (p *parser) metric() model.Metric {
p.errorf("missing metric name or metric selector")
}
if t == itemLeftBrace {
m = model.Metric(p.labelSet())
m = p.labelSet()
}
if name != "" {
m[model.MetricNameLabel] = model.LabelValue(name)
m = append(m, labels.Label{Name: MetricNameLabel, Value: name})
sort.Sort(m)
}
return m
}
@ -943,7 +941,7 @@ func (p *parser) offset() time.Duration {
// [<metric_identifier>] <label_matchers>
//
func (p *parser) vectorSelector(name string) *VectorSelector {
var matchers metric.LabelMatchers
var matchers []*LabelMatcher
// Parse label matching if any.
if t := p.peek(); t.typ == itemLeftBrace {
matchers = p.labelMatchers(itemEQL, itemNEQ, itemEQLRegex, itemNEQRegex)
@ -956,7 +954,7 @@ func (p *parser) vectorSelector(name string) *VectorSelector {
}
}
// Set name label matching.
m, err := metric.NewLabelMatcher(metric.Equal, model.MetricNameLabel, model.LabelValue(name))
m, err := NewLabelMatcher(MatchEqual, MetricNameLabel, name)
if err != nil {
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).
notEmpty := false
for _, lm := range matchers {
if !lm.MatchesEmptyString() {
if !lm.matcher().Matches("") {
notEmpty = true
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
// 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)
if t != want {
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
// 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.
// Statements and lists do not have a type but are not invalid either.
switch n := node.(type) {
case Statements, Expressions, Statement:
typ = model.ValNone
typ = ValueTypeNone
case Expr:
typ = n.Type()
default:
@ -1016,27 +1014,27 @@ func (p *parser) checkType(node Node) (typ model.ValueType) {
switch n := node.(type) {
case Statements:
for _, s := range n {
p.expectType(s, model.ValNone, "statement list")
p.expectType(s, ValueTypeNone, "statement list")
}
case *AlertStmt:
p.expectType(n.Expr, model.ValVector, "alert statement")
p.expectType(n.Expr, ValueTypeVector, "alert statement")
case *EvalStmt:
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))
}
case *RecordStmt:
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))
}
case Expressions:
for _, e := range n {
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))
}
}
@ -1044,12 +1042,12 @@ func (p *parser) checkType(node Node) (typ model.ValueType) {
if !n.Op.isAggregator() {
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 {
p.expectType(n.Param, model.ValScalar, "aggregation parameter")
p.expectType(n.Param, ValueTypeScalar, "aggregation parameter")
}
if n.Op == itemCountValues {
p.expectType(n.Param, model.ValString, "aggregation parameter")
p.expectType(n.Param, ValueTypeString, "aggregation parameter")
}
case *BinaryExpr:
@ -1059,11 +1057,11 @@ func (p *parser) checkType(node Node) (typ model.ValueType) {
if !n.Op.isOperator() {
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")
}
if (lt != model.ValVector || rt != model.ValVector) && n.VectorMatching != nil {
if (lt != ValueTypeVector || rt != ValueTypeVector) && n.VectorMatching != nil {
if len(n.VectorMatching.MatchingLabels) > 0 {
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)
}
@ -1103,7 +1101,7 @@ func (p *parser) checkType(node Node) (typ model.ValueType) {
if n.Op != itemADD && n.Op != itemSUB {
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))
}

File diff suppressed because it is too large Load Diff

View File

@ -20,8 +20,6 @@ import (
"time"
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/storage/metric"
)
// 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)
for _, matcher := range node.LabelMatchers {
// 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
}
labelStrings = append(labelStrings, matcher.String())

View File

@ -17,8 +17,7 @@ import (
"testing"
"time"
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/storage/metric"
"github.com/fabxc/tsdb/labels"
)
func TestStatementString(t *testing.T) {
@ -28,17 +27,15 @@ func TestStatementString(t *testing.T) {
Op: itemGTR,
LHS: &VectorSelector{
Name: "foo",
LabelMatchers: metric.LabelMatchers{
{Type: metric.Equal, Name: model.MetricNameLabel, Value: "bar"},
LabelMatchers: []*LabelMatcher{
{Type: MatchEqual, Name: MetricNameLabel, Value: "bar"},
},
},
RHS: &NumberLiteral{10},
},
Duration: 5 * time.Minute,
Labels: model.LabelSet{"foo": "bar"},
Annotations: model.LabelSet{
"notify": "team-a",
},
Duration: 5 * time.Minute,
Labels: labels.FromStrings("foo", "bar"),
Annotations: labels.FromStrings("notify", "team-a"),
}
expected := `ALERT FooAlert

View File

@ -17,23 +17,22 @@ import (
"math"
"sort"
"github.com/fabxc/tsdb/labels"
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/storage/metric"
)
// Helpers to calculate quantiles.
// excludedLabels are the labels to exclude from signature calculation for
// quantiles.
var excludedLabels = map[model.LabelName]struct{}{
model.MetricNameLabel: {},
model.BucketLabel: {},
var excludedLabels = []string{
model.MetricNameLabel,
model.BucketLabel,
}
type bucket struct {
upperBound float64
count model.SampleValue
count float64
}
// 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 }
type metricWithBuckets struct {
metric metric.Metric
metric labels.Labels
buckets buckets
}
@ -70,7 +69,7 @@ type metricWithBuckets struct {
// If q<0, -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 {
return math.Inf(-1)
}

View File

@ -22,6 +22,7 @@ import (
"strings"
"time"
"github.com/fabxc/tsdb/labels"
"github.com/prometheus/common/model"
"golang.org/x/net/context"
@ -39,10 +40,11 @@ var (
)
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
// against a test storage.
type Test struct {
@ -170,7 +172,7 @@ func (t *Test) parseEval(lines []string, i int) (int, *evalCmd, error) {
break
}
if f, err := parseNumber(defLine); err == nil {
cmd.expect(0, nil, sequenceValue{value: model.SampleValue(f)})
cmd.expect(0, nil, sequenceValue{value: f})
break
}
metric, vals, err := parseSeriesDesc(defLine)
@ -243,15 +245,15 @@ func (*evalCmd) testCmd() {}
// metrics into the storage.
type loadCmd struct {
gap time.Duration
metrics map[model.Fingerprint]model.Metric
defs map[model.Fingerprint][]model.SamplePair
metrics map[uint64]labels.Labels
defs map[uint64][]samplePair
}
func newLoadCmd(gap time.Duration) *loadCmd {
return &loadCmd{
gap: gap,
metrics: map[model.Fingerprint]model.Metric{},
defs: map[model.Fingerprint][]model.SamplePair{},
metrics: map[uint64]labels.Labels{},
defs: map[uint64][]samplePair{},
}
}
@ -260,51 +262,52 @@ func (cmd loadCmd) String() string {
}
// set a sequence of sample values for the given metric.
func (cmd *loadCmd) set(m model.Metric, vals ...sequenceValue) {
fp := m.Fingerprint()
func (cmd *loadCmd) set(m labels.Labels, vals ...sequenceValue) {
h := m.Hash()
samples := make([]model.SamplePair, 0, len(vals))
samples := make([]samplePair, 0, len(vals))
ts := testStartTime
for _, v := range vals {
if !v.omitted {
samples = append(samples, model.SamplePair{
Timestamp: ts,
Value: v.value,
samples = append(samples, samplePair{
t: ts.UnixNano() / int64(time.Millisecond/time.Nanosecond),
v: v.value,
})
}
ts = ts.Add(cmd.gap)
}
cmd.defs[fp] = samples
cmd.metrics[fp] = m
cmd.defs[h] = samples
cmd.metrics[h] = m
}
// append the defined time series to the storage.
func (cmd *loadCmd) append(a storage.SampleAppender) {
for fp, samples := range cmd.defs {
met := cmd.metrics[fp]
for _, smpl := range samples {
s := &model.Sample{
Metric: met,
Value: smpl.Value,
Timestamp: smpl.Timestamp,
}
a.Append(s)
}
}
// TODO(fabxc): commented out until Appender refactoring.
// for fp, samples := range cmd.defs {
// met := cmd.metrics[fp]
// for _, smpl := range samples {
// s := &model.Sample{
// Metric: met,
// Value: smpl.Value,
// Timestamp: smpl.Timestamp,
// }
// a.Append(s)
// }
// }
}
// evalCmd is a command that evaluates an expression for the given time (range)
// and expects a specific result.
type evalCmd struct {
expr Expr
start, end model.Time
start, end time.Time
interval time.Duration
instant bool
fail, ordered bool
metrics map[model.Fingerprint]model.Metric
expected map[model.Fingerprint]entry
metrics map[uint64]labels.Labels
expected map[uint64]entry
}
type entry struct {
@ -316,7 +319,7 @@ func (e entry) String() string {
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{
expr: expr,
start: start,
@ -324,8 +327,8 @@ func newEvalCmd(expr Expr, start, end model.Time, interval time.Duration) *evalC
interval: interval,
instant: start == end && interval == 0,
metrics: map[model.Fingerprint]model.Metric{},
expected: map[model.Fingerprint]entry{},
metrics: map[uint64]labels.Labels{},
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
// 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 {
ev.expected[0] = entry{pos: pos, vals: vals}
return
}
fp := m.Fingerprint()
ev.metrics[fp] = m
ev.expected[fp] = entry{pos: pos, vals: vals}
h := m.Hash()
ev.metrics[h] = m
ev.expected[h] = entry{pos: pos, vals: vals}
}
// 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) {
case model.Matrix:
case matrix:
if ev.instant {
return fmt.Errorf("received range result on instant evaluation")
}
seen := map[model.Fingerprint]bool{}
seen := map[uint64]bool{}
for pos, v := range val {
fp := v.Metric.Fingerprint()
fp := v.Metric.Hash()
if _, ok := ev.metrics[fp]; !ok {
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)
}
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)
}
}
@ -375,13 +378,13 @@ func (ev *evalCmd) compareResult(result model.Value) error {
}
}
case model.Vector:
case vector:
if !ev.instant {
return fmt.Errorf("received instant result on range evaluation")
}
seen := map[model.Fingerprint]bool{}
seen := map[uint64]bool{}
for pos, v := range val {
fp := v.Metric.Fingerprint()
fp := v.Metric.Hash()
if _, ok := ev.metrics[fp]; !ok {
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:
if !almostEqual(float64(ev.expected[0].vals[0].value), float64(val.Value)) {
return fmt.Errorf("expected scalar %v but got %v", val.Value, ev.expected[0].vals[0].value)
case scalar:
if !almostEqual(ev.expected[0].vals[0].value, val.v) {
return fmt.Errorf("expected scalar %v but got %v", val.v, ev.expected[0].vals[0].value)
}
default:
@ -506,7 +509,8 @@ func (t *Test) clear() {
t.storage, closer = local.NewTestStorage(t, 2)
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())
}