promql: scalar T/V and Point

This commit is contained in:
Fabian Reinartz 2016-12-24 11:23:06 +01:00
parent 09666e2e2a
commit ac5d3bc05e
5 changed files with 190 additions and 198 deletions

View file

@ -36,6 +36,7 @@ const (
// The smallest SampleValue that can be converted to an int64 without underflow. // The smallest SampleValue that can be converted to an int64 without underflow.
minInt64 = -9223372036854775808 minInt64 = -9223372036854775808
// MetricNameLabel is the name of the label containing the metric name.
MetricNameLabel = "__name__" MetricNameLabel = "__name__"
) )
@ -76,9 +77,10 @@ func (s stringVal) String() string {
return s.s return s.s
} }
// Scalar is a data point that's explicitly not associated with a metric.
type Scalar struct { type Scalar struct {
t int64 T int64
v float64 V float64
} }
func (s Scalar) String() string { func (s Scalar) String() string {
@ -88,27 +90,28 @@ func (s Scalar) String() string {
// sampleStream is a stream of Values belonging to an attached COWMetric. // sampleStream is a stream of Values belonging to an attached COWMetric.
type sampleStream struct { type sampleStream struct {
Metric labels.Labels Metric labels.Labels
Values []samplePair Values []Point
} }
func (s sampleStream) String() string { func (s sampleStream) String() string {
return "" return ""
} }
type samplePair struct { // Point represents a single data point for a given timestamp.
t int64 type Point struct {
v float64 T int64
V float64
} }
func (s samplePair) String() string { func (s Point) String() string {
return "" return ""
} }
// sample is a single sample belonging to a COWMetric. // sample is a single sample belonging to a COWMetric.
type sample struct { type sample struct {
Metric labels.Labels Point
Value float64
Timestamp int64 Metric labels.Labels
} }
func (s sample) String() string { func (s sample) String() string {
@ -475,13 +478,10 @@ func (ng *Engine) execEvalStmt(ctx context.Context, query *query, s *EvalStmt) (
// as the fingerprint for Scalar expressions. // as the fingerprint for Scalar expressions.
ss, ok := sampleStreams[0] ss, ok := sampleStreams[0]
if !ok { if !ok {
ss = sampleStream{Values: make([]samplePair, 0, numSteps)} ss = sampleStream{Values: make([]Point, 0, numSteps)}
sampleStreams[0] = ss sampleStreams[0] = ss
} }
ss.Values = append(ss.Values, samplePair{ ss.Values = append(ss.Values, Point(v))
v: v.v,
t: v.t,
})
case Vector: case Vector:
for _, sample := range v { for _, sample := range v {
h := sample.Metric.Hash() h := sample.Metric.Hash()
@ -489,14 +489,11 @@ func (ng *Engine) execEvalStmt(ctx context.Context, query *query, s *EvalStmt) (
if !ok { if !ok {
ss = sampleStream{ ss = sampleStream{
Metric: sample.Metric, Metric: sample.Metric,
Values: make([]samplePair, 0, numSteps), Values: make([]Point, 0, numSteps),
} }
sampleStreams[h] = ss sampleStreams[h] = ss
} }
ss.Values = append(ss.Values, samplePair{ ss.Values = append(ss.Values, sample.Point)
v: sample.Value,
t: sample.Timestamp,
})
} }
default: default:
panic(fmt.Errorf("promql.Engine.exec: invalid expression type %q", val.Type())) panic(fmt.Errorf("promql.Engine.exec: invalid expression type %q", val.Type()))
@ -656,16 +653,16 @@ func (ev *evaluator) evalVector(e Expr) Vector {
// evalInt attempts to evaluate e into an integer and errors otherwise. // evalInt attempts to evaluate e into an integer and errors otherwise.
func (ev *evaluator) evalInt(e Expr) int64 { func (ev *evaluator) evalInt(e Expr) int64 {
sc := ev.evalScalar(e) sc := ev.evalScalar(e)
if !convertibleToInt64(sc.v) { if !convertibleToInt64(sc.V) {
ev.errorf("Scalar value %v overflows int64", sc.v) ev.errorf("Scalar value %v overflows int64", sc.V)
} }
return int64(sc.v) return int64(sc.V)
} }
// evalFloat attempts to evaluate e into a float and errors otherwise. // evalFloat attempts to evaluate e into a float and errors otherwise.
func (ev *evaluator) evalFloat(e Expr) float64 { func (ev *evaluator) evalFloat(e Expr) float64 {
sc := ev.evalScalar(e) sc := ev.evalScalar(e)
return float64(sc.v) return float64(sc.V)
} }
// evalMatrix attempts to evaluate e into a Matrix and errors otherwise. // evalMatrix attempts to evaluate e into a Matrix and errors otherwise.
@ -724,8 +721,8 @@ func (ev *evaluator) eval(expr Expr) Value {
switch lt, rt := lhs.Type(), rhs.Type(); { switch lt, rt := lhs.Type(), rhs.Type(); {
case lt == ValueTypeScalar && rt == ValueTypeScalar: case lt == ValueTypeScalar && rt == ValueTypeScalar:
return Scalar{ return Scalar{
v: ScalarBinop(e.Op, lhs.(Scalar).v, rhs.(Scalar).v), V: ScalarBinop(e.Op, lhs.(Scalar).V, rhs.(Scalar).V),
t: ev.Timestamp, T: ev.Timestamp,
} }
case lt == ValueTypeVector && rt == ValueTypeVector: case lt == ValueTypeVector && rt == ValueTypeVector:
@ -753,7 +750,7 @@ func (ev *evaluator) eval(expr Expr) Value {
return ev.MatrixSelector(e) return ev.MatrixSelector(e)
case *NumberLiteral: case *NumberLiteral:
return Scalar{v: e.Val, t: ev.Timestamp} return Scalar{V: e.Val, T: ev.Timestamp}
case *ParenExpr: case *ParenExpr:
return ev.eval(e.Expr) return ev.eval(e.Expr)
@ -767,10 +764,10 @@ func (ev *evaluator) eval(expr Expr) Value {
if e.Op == itemSUB { if e.Op == itemSUB {
switch v := se.(type) { switch v := se.(type) {
case Scalar: case Scalar:
v.v = -v.v v.V = -v.V
case Vector: case Vector:
for i, sv := range v { for i, sv := range v {
v[i].Value = -sv.Value v[i].V = -sv.V
} }
} }
} }
@ -807,9 +804,8 @@ func (ev *evaluator) VectorSelector(node *VectorSelector) Vector {
} }
vec = append(vec, sample{ vec = append(vec, sample{
Metric: node.series[i].Labels(), Metric: node.series[i].Labels(),
Value: v, Point: Point{V: v, T: ev.Timestamp},
Timestamp: int64(ev.Timestamp),
}) })
} }
return vec return vec
@ -827,7 +823,7 @@ func (ev *evaluator) MatrixSelector(node *MatrixSelector) Matrix {
for i, it := range node.iterators { for i, it := range node.iterators {
ss := sampleStream{ ss := sampleStream{
Metric: node.series[i].Labels(), Metric: node.series[i].Labels(),
Values: make([]samplePair, 0, 16), Values: make([]Point, 0, 16),
} }
if !it.Seek(maxt) { if !it.Seek(maxt) {
@ -842,13 +838,13 @@ func (ev *evaluator) MatrixSelector(node *MatrixSelector) Matrix {
t, v := buf.Values() t, v := buf.Values()
// Values in the buffer are guaranteed to be smaller than maxt. // Values in the buffer are guaranteed to be smaller than maxt.
if t >= mint { if t >= mint {
ss.Values = append(ss.Values, samplePair{t: t + offset, v: v}) ss.Values = append(ss.Values, Point{T: t + offset, V: v})
} }
} }
// The seeked sample might also be in the range. // The seeked sample might also be in the range.
t, v := it.Values() t, v := it.Values()
if t == maxt { if t == maxt {
ss.Values = append(ss.Values, samplePair{t: t + offset, v: v}) ss.Values = append(ss.Values, Point{T: t + offset, V: v})
} }
Matrix = append(Matrix, ss) Matrix = append(Matrix, ss)
@ -968,7 +964,7 @@ func (ev *evaluator) VectorBinop(op itemType, lhs, rhs Vector, matching *VectorM
} }
// Account for potentially swapped sidedness. // Account for potentially swapped sidedness.
vl, vr := ls.Value, rs.Value vl, vr := ls.V, rs.V
if matching.Card == CardOneToMany { if matching.Card == CardOneToMany {
vl, vr = vr, vl vl, vr = vr, vl
} }
@ -1006,9 +1002,8 @@ func (ev *evaluator) VectorBinop(op itemType, lhs, rhs Vector, matching *VectorM
} }
result = append(result, sample{ result = append(result, sample{
Metric: metric, Metric: metric,
Value: value, Point: Point{V: value, T: ev.Timestamp},
Timestamp: ev.Timestamp,
}) })
} }
return result return result
@ -1121,7 +1116,7 @@ func (ev *evaluator) VectorScalarBinop(op itemType, lhs Vector, rhs Scalar, swap
vec := make(Vector, 0, len(lhs)) vec := make(Vector, 0, len(lhs))
for _, lhsSample := range lhs { for _, lhsSample := range lhs {
lv, rv := lhsSample.Value, rhs.v lv, rv := lhsSample.V, rhs.V
// lhs always contains the Vector. If the original position was different // lhs always contains the Vector. If the original position was different
// swap for calculating the value. // swap for calculating the value.
if swap { if swap {
@ -1137,7 +1132,7 @@ func (ev *evaluator) VectorScalarBinop(op itemType, lhs Vector, rhs Scalar, swap
keep = true keep = true
} }
if keep { if keep {
lhsSample.Value = value lhsSample.V = value
lhsSample.Metric = copyLabels(lhsSample.Metric, shouldDropMetricName(op)) lhsSample.Metric = copyLabels(lhsSample.Metric, shouldDropMetricName(op))
vec = append(vec, lhsSample) vec = append(vec, lhsSample)
@ -1280,7 +1275,7 @@ func (ev *evaluator) aggregation(op itemType, grouping []string, without bool, k
} }
if op == itemCountValues { if op == itemCountValues {
del = append(del, valueLabel) del = append(del, valueLabel)
add = append(add, labels.Label{Name: valueLabel, Value: fmt.Sprintf("%f", s.Value)}) add = append(add, labels.Label{Name: valueLabel, Value: fmt.Sprintf("%f", s.V)})
} }
var ( var (
@ -1308,16 +1303,22 @@ func (ev *evaluator) aggregation(op itemType, grouping []string, without bool, k
} }
result[groupingKey] = &groupedAggregation{ result[groupingKey] = &groupedAggregation{
labels: m, labels: m,
value: s.Value, value: s.V,
valuesSquaredSum: s.Value * s.Value, valuesSquaredSum: s.V * s.V,
groupCount: 1, groupCount: 1,
} }
if op == itemTopK || op == itemQuantile { if op == itemTopK || op == itemQuantile {
result[groupingKey].heap = make(VectorByValueHeap, 0, k) result[groupingKey].heap = make(VectorByValueHeap, 0, k)
heap.Push(&result[groupingKey].heap, &sample{Value: s.Value, Metric: s.Metric}) heap.Push(&result[groupingKey].heap, &sample{
Point: Point{V: s.V},
Metric: s.Metric,
})
} else if op == itemBottomK { } else if op == itemBottomK {
result[groupingKey].reverseHeap = make(VectorByReverseValueHeap, 0, k) result[groupingKey].reverseHeap = make(VectorByReverseValueHeap, 0, k)
heap.Push(&result[groupingKey].reverseHeap, &sample{Value: s.Value, Metric: s.Metric}) heap.Push(&result[groupingKey].reverseHeap, &sample{
Point: Point{V: s.V},
Metric: s.Metric,
})
} }
continue continue
} }
@ -1328,44 +1329,50 @@ func (ev *evaluator) aggregation(op itemType, grouping []string, without bool, k
switch op { switch op {
case itemSum: case itemSum:
group.value += s.Value group.value += s.V
case itemAvg: case itemAvg:
group.value += s.Value group.value += s.V
group.groupCount++ group.groupCount++
case itemMax: case itemMax:
if group.value < s.Value || math.IsNaN(float64(group.value)) { if group.value < s.V || math.IsNaN(float64(group.value)) {
group.value = s.Value group.value = s.V
} }
case itemMin: case itemMin:
if group.value > s.Value || math.IsNaN(float64(group.value)) { if group.value > s.V || math.IsNaN(float64(group.value)) {
group.value = s.Value group.value = s.V
} }
case itemCount, itemCountValues: case itemCount, itemCountValues:
group.groupCount++ group.groupCount++
case itemStdvar, itemStddev: case itemStdvar, itemStddev:
group.value += s.Value group.value += s.V
group.valuesSquaredSum += s.Value * s.Value group.valuesSquaredSum += s.V * s.V
group.groupCount++ group.groupCount++
case itemTopK: case itemTopK:
if int64(len(group.heap)) < k || group.heap[0].Value < s.Value || math.IsNaN(float64(group.heap[0].Value)) { if int64(len(group.heap)) < k || group.heap[0].V < s.V || math.IsNaN(float64(group.heap[0].V)) {
if int64(len(group.heap)) == k { if int64(len(group.heap)) == k {
heap.Pop(&group.heap) heap.Pop(&group.heap)
} }
heap.Push(&group.heap, &sample{Value: s.Value, Metric: s.Metric}) heap.Push(&group.heap, &sample{
Point: Point{V: s.V},
Metric: s.Metric,
})
} }
case itemBottomK: case itemBottomK:
if int64(len(group.reverseHeap)) < k || group.reverseHeap[0].Value > s.Value || math.IsNaN(float64(group.reverseHeap[0].Value)) { if int64(len(group.reverseHeap)) < k || group.reverseHeap[0].V > s.V || math.IsNaN(float64(group.reverseHeap[0].V)) {
if int64(len(group.reverseHeap)) == k { if int64(len(group.reverseHeap)) == k {
heap.Pop(&group.reverseHeap) heap.Pop(&group.reverseHeap)
} }
heap.Push(&group.reverseHeap, &sample{Value: s.Value, Metric: s.Metric}) heap.Push(&group.reverseHeap, &sample{
Point: Point{V: s.V},
Metric: s.Metric,
})
} }
case itemQuantile: case itemQuantile:
@ -1400,9 +1407,8 @@ func (ev *evaluator) aggregation(op itemType, grouping []string, without bool, k
sort.Sort(sort.Reverse(aggr.heap)) sort.Sort(sort.Reverse(aggr.heap))
for _, v := range aggr.heap { for _, v := range aggr.heap {
resultVector = append(resultVector, sample{ resultVector = append(resultVector, sample{
Metric: v.Metric, Metric: v.Metric,
Value: v.Value, Point: Point{V: v.V, T: ev.Timestamp},
Timestamp: ev.Timestamp,
}) })
} }
continue // Bypass default append. continue // Bypass default append.
@ -1412,9 +1418,8 @@ func (ev *evaluator) aggregation(op itemType, grouping []string, without bool, k
sort.Sort(sort.Reverse(aggr.reverseHeap)) sort.Sort(sort.Reverse(aggr.reverseHeap))
for _, v := range aggr.reverseHeap { for _, v := range aggr.reverseHeap {
resultVector = append(resultVector, sample{ resultVector = append(resultVector, sample{
Metric: v.Metric, Metric: v.Metric,
Value: v.Value, Point: Point{V: v.V, T: ev.Timestamp},
Timestamp: ev.Timestamp,
}) })
} }
continue // Bypass default append. continue // Bypass default append.
@ -1427,9 +1432,8 @@ func (ev *evaluator) aggregation(op itemType, grouping []string, without bool, k
} }
resultVector = append(resultVector, sample{ resultVector = append(resultVector, sample{
Metric: aggr.labels, Metric: aggr.labels,
Value: aggr.value, Point: Point{V: aggr.value, T: ev.Timestamp},
Timestamp: ev.Timestamp,
}) })
} }
return resultVector return resultVector

View file

@ -37,8 +37,8 @@ type Function struct {
// === time() float64 === // === time() float64 ===
func funcTime(ev *evaluator, args Expressions) Value { func funcTime(ev *evaluator, args Expressions) Value {
return Scalar{ return Scalar{
v: float64(ev.Timestamp / 1000), V: float64(ev.Timestamp / 1000),
t: ev.Timestamp, T: ev.Timestamp,
} }
} }
@ -66,21 +66,21 @@ func extrapolatedRate(ev *evaluator, arg Expr, isCounter bool, isRate bool) Valu
lastValue float64 lastValue float64
) )
for _, sample := range samples.Values { for _, sample := range samples.Values {
if isCounter && sample.v < lastValue { if isCounter && sample.V < lastValue {
counterCorrection += lastValue counterCorrection += lastValue
} }
lastValue = sample.v lastValue = sample.V
} }
resultValue := lastValue - samples.Values[0].v + counterCorrection resultValue := lastValue - samples.Values[0].V + counterCorrection
// Duration between first/last samples and boundary of range. // Duration between first/last samples and boundary of range.
durationToStart := float64(samples.Values[0].t - rangeStart) durationToStart := float64(samples.Values[0].T - rangeStart)
durationToEnd := float64(rangeEnd - samples.Values[len(samples.Values)-1].t) durationToEnd := float64(rangeEnd - samples.Values[len(samples.Values)-1].T)
sampledInterval := float64(samples.Values[len(samples.Values)-1].t - samples.Values[0].t) sampledInterval := float64(samples.Values[len(samples.Values)-1].T - samples.Values[0].T)
averageDurationBetweenSamples := float64(sampledInterval) / float64(len(samples.Values)-1) averageDurationBetweenSamples := float64(sampledInterval) / float64(len(samples.Values)-1)
if isCounter && resultValue > 0 && samples.Values[0].v >= 0 { if isCounter && resultValue > 0 && samples.Values[0].V >= 0 {
// Counters cannot be negative. If we have any slope at // Counters cannot be negative. If we have any slope at
// all (i.e. resultValue went up), we can extrapolate // all (i.e. resultValue went up), we can extrapolate
// the zero point of the counter. If the duration to the // the zero point of the counter. If the duration to the
@ -88,7 +88,7 @@ func extrapolatedRate(ev *evaluator, arg Expr, isCounter bool, isRate bool) Valu
// take the zero point as the start of the series, // take the zero point as the start of the series,
// thereby avoiding extrapolation to negative counter // thereby avoiding extrapolation to negative counter
// values. // values.
durationToZero := float64(sampledInterval) * float64(samples.Values[0].v/resultValue) durationToZero := float64(sampledInterval) * float64(samples.Values[0].V/resultValue)
if durationToZero < durationToStart { if durationToZero < durationToStart {
durationToStart = durationToZero durationToStart = durationToZero
} }
@ -117,9 +117,8 @@ func extrapolatedRate(ev *evaluator, arg Expr, isCounter bool, isRate bool) Valu
} }
resultVector = append(resultVector, sample{ resultVector = append(resultVector, sample{
Metric: copyLabels(samples.Metric, false), Metric: copyLabels(samples.Metric, false),
Value: resultValue, Point: Point{V: resultValue, T: ev.Timestamp},
Timestamp: ev.Timestamp,
}) })
} }
return resultVector return resultVector
@ -163,14 +162,14 @@ func instantValue(ev *evaluator, arg Expr, isRate bool) Value {
previousSample := samples.Values[len(samples.Values)-2] previousSample := samples.Values[len(samples.Values)-2]
var resultValue float64 var resultValue float64
if isRate && lastSample.v < previousSample.v { if isRate && lastSample.V < previousSample.V {
// Counter reset. // Counter reset.
resultValue = lastSample.v resultValue = lastSample.V
} else { } else {
resultValue = lastSample.v - previousSample.v resultValue = lastSample.V - previousSample.V
} }
sampledInterval := lastSample.t - previousSample.t sampledInterval := lastSample.T - previousSample.T
if sampledInterval == 0 { if sampledInterval == 0 {
// Avoid dividing by 0.float64 // Avoid dividing by 0.float64
} }
@ -181,9 +180,8 @@ func instantValue(ev *evaluator, arg Expr, isRate bool) Value {
} }
resultVector = append(resultVector, sample{ resultVector = append(resultVector, sample{
Metric: copyLabels(samples.Metric, false), Metric: copyLabels(samples.Metric, false),
Value: resultValue, Point: Point{V: resultValue, T: ev.Timestamp},
Timestamp: ev.Timestamp,
}) })
} }
return resultVector return resultVector
@ -224,10 +222,10 @@ func funcHoltWinters(ev *evaluator, args Expressions) Value {
// Sanity check the input. // Sanity check the input.
if sf <= 0 || sf >= 1 { if sf <= 0 || sf >= 1 {
ev.errorf("invalid smoothing factor. Expected: 0 < sf < 1 got: %f", sf) ev.errorf("invalid smoothing factor. Expected: 0 < sf < 1 goT: %f", sf)
} }
if tf <= 0 || tf >= 1 { if tf <= 0 || tf >= 1 {
ev.errorf("invalid trend factor. Expected: 0 < tf < 1 got: %f", sf) ev.errorf("invalid trend factor. Expected: 0 < tf < 1 goT: %f", sf)
} }
// Make an output Vector large enough to hold the entire result. // Make an output Vector large enough to hold the entire result.
@ -254,7 +252,7 @@ func funcHoltWinters(ev *evaluator, args Expressions) Value {
// Fill in the d values with the raw values from the input. // Fill in the d values with the raw values from the input.
for i, v := range samples.Values { for i, v := range samples.Values {
d[i] = v.v d[i] = v.V
} }
// Set initial values. // Set initial values.
@ -275,9 +273,8 @@ func funcHoltWinters(ev *evaluator, args Expressions) Value {
} }
resultVector = append(resultVector, sample{ resultVector = append(resultVector, sample{
Metric: copyLabels(samples.Metric, false), Metric: copyLabels(samples.Metric, false),
Value: s[len(s)-1], // The last value in the Vector is the smoothed result. Point: Point{V: s[len(s)-1], T: ev.Timestamp}, // The last value in the Vector is the smoothed result.
Timestamp: ev.Timestamp,
}) })
} }
@ -308,7 +305,7 @@ func funcClampMax(ev *evaluator, args Expressions) Value {
max := ev.evalFloat(args[1]) max := ev.evalFloat(args[1])
for _, el := range vec { for _, el := range vec {
el.Metric = copyLabels(el.Metric, false) el.Metric = copyLabels(el.Metric, false)
el.Value = math.Min(max, float64(el.Value)) el.V = math.Min(max, float64(el.V))
} }
return vec return vec
} }
@ -319,7 +316,7 @@ func funcClampMin(ev *evaluator, args Expressions) Value {
min := ev.evalFloat(args[1]) min := ev.evalFloat(args[1])
for _, el := range vec { for _, el := range vec {
el.Metric = copyLabels(el.Metric, false) el.Metric = copyLabels(el.Metric, false)
el.Value = math.Max(min, float64(el.Value)) el.V = math.Max(min, float64(el.V))
} }
return vec return vec
} }
@ -379,7 +376,7 @@ func funcRound(ev *evaluator, args Expressions) Value {
vec := ev.evalVector(args[0]) vec := ev.evalVector(args[0])
for _, el := range vec { for _, el := range vec {
el.Metric = copyLabels(el.Metric, false) el.Metric = copyLabels(el.Metric, false)
el.Value = math.Floor(float64(el.Value)*toNearestInverse+0.5) / toNearestInverse el.V = math.Floor(float64(el.V)*toNearestInverse+0.5) / toNearestInverse
} }
return vec return vec
} }
@ -389,25 +386,25 @@ func funcScalar(ev *evaluator, args Expressions) Value {
v := ev.evalVector(args[0]) v := ev.evalVector(args[0])
if len(v) != 1 { if len(v) != 1 {
return Scalar{ return Scalar{
v: math.NaN(), V: math.NaN(),
t: ev.Timestamp, T: ev.Timestamp,
} }
} }
return Scalar{ return Scalar{
v: v[0].Value, V: v[0].V,
t: ev.Timestamp, T: ev.Timestamp,
} }
} }
// === count_Scalar(Vector ValueTypeVector) float64 === // === count_Scalar(Vector ValueTypeVector) float64 ===
func funcCountScalar(ev *evaluator, args Expressions) Value { func funcCountScalar(ev *evaluator, args Expressions) Value {
return Scalar{ return Scalar{
v: float64(len(ev.evalVector(args[0]))), V: float64(len(ev.evalVector(args[0]))),
t: ev.Timestamp, T: ev.Timestamp,
} }
} }
func aggrOverTime(ev *evaluator, args Expressions, aggrFn func([]samplePair) float64) Value { func aggrOverTime(ev *evaluator, args Expressions, aggrFn func([]Point) float64) Value {
mat := ev.evalMatrix(args[0]) mat := ev.evalMatrix(args[0])
resultVector := Vector{} resultVector := Vector{}
@ -417,9 +414,8 @@ func aggrOverTime(ev *evaluator, args Expressions, aggrFn func([]samplePair) flo
} }
resultVector = append(resultVector, sample{ resultVector = append(resultVector, sample{
Metric: copyLabels(el.Metric, false), Metric: copyLabels(el.Metric, false),
Value: aggrFn(el.Values), Point: Point{V: aggrFn(el.Values), T: ev.Timestamp},
Timestamp: ev.Timestamp,
}) })
} }
return resultVector return resultVector
@ -427,10 +423,10 @@ func aggrOverTime(ev *evaluator, args Expressions, aggrFn func([]samplePair) flo
// === avg_over_time(Matrix ValueTypeMatrix) Vector === // === avg_over_time(Matrix ValueTypeMatrix) Vector ===
func funcAvgOverTime(ev *evaluator, args Expressions) Value { func funcAvgOverTime(ev *evaluator, args Expressions) Value {
return aggrOverTime(ev, args, func(values []samplePair) float64 { return aggrOverTime(ev, args, func(values []Point) float64 {
var sum float64 var sum float64
for _, v := range values { for _, v := range values {
sum += v.v sum += v.V
} }
return sum / float64(len(values)) return sum / float64(len(values))
}) })
@ -438,7 +434,7 @@ func funcAvgOverTime(ev *evaluator, args Expressions) Value {
// === count_over_time(Matrix ValueTypeMatrix) Vector === // === count_over_time(Matrix ValueTypeMatrix) Vector ===
func funcCountOverTime(ev *evaluator, args Expressions) Value { func funcCountOverTime(ev *evaluator, args Expressions) Value {
return aggrOverTime(ev, args, func(values []samplePair) float64 { return aggrOverTime(ev, args, func(values []Point) float64 {
return float64(len(values)) return float64(len(values))
}) })
} }
@ -448,17 +444,17 @@ func funcFloor(ev *evaluator, args Expressions) Value {
Vector := ev.evalVector(args[0]) Vector := ev.evalVector(args[0])
for _, el := range Vector { for _, el := range Vector {
el.Metric = copyLabels(el.Metric, false) el.Metric = copyLabels(el.Metric, false)
el.Value = math.Floor(float64(el.Value)) el.V = math.Floor(float64(el.V))
} }
return Vector return Vector
} }
// === max_over_time(Matrix ValueTypeMatrix) Vector === // === max_over_time(Matrix ValueTypeMatrix) Vector ===
func funcMaxOverTime(ev *evaluator, args Expressions) Value { func funcMaxOverTime(ev *evaluator, args Expressions) Value {
return aggrOverTime(ev, args, func(values []samplePair) float64 { return aggrOverTime(ev, args, func(values []Point) float64 {
max := math.Inf(-1) max := math.Inf(-1)
for _, v := range values { for _, v := range values {
max = math.Max(max, float64(v.v)) max = math.Max(max, float64(v.V))
} }
return max return max
}) })
@ -466,10 +462,10 @@ func funcMaxOverTime(ev *evaluator, args Expressions) Value {
// === min_over_time(Matrix ValueTypeMatrix) Vector === // === min_over_time(Matrix ValueTypeMatrix) Vector ===
func funcMinOverTime(ev *evaluator, args Expressions) Value { func funcMinOverTime(ev *evaluator, args Expressions) Value {
return aggrOverTime(ev, args, func(values []samplePair) float64 { return aggrOverTime(ev, args, func(values []Point) float64 {
min := math.Inf(1) min := math.Inf(1)
for _, v := range values { for _, v := range values {
min = math.Min(min, float64(v.v)) min = math.Min(min, float64(v.V))
} }
return min return min
}) })
@ -477,10 +473,10 @@ func funcMinOverTime(ev *evaluator, args Expressions) Value {
// === sum_over_time(Matrix ValueTypeMatrix) Vector === // === sum_over_time(Matrix ValueTypeMatrix) Vector ===
func funcSumOverTime(ev *evaluator, args Expressions) Value { func funcSumOverTime(ev *evaluator, args Expressions) Value {
return aggrOverTime(ev, args, func(values []samplePair) float64 { return aggrOverTime(ev, args, func(values []Point) float64 {
var sum float64 var sum float64
for _, v := range values { for _, v := range values {
sum += v.v sum += v.V
} }
return sum return sum
}) })
@ -500,12 +496,11 @@ func funcQuantileOverTime(ev *evaluator, args Expressions) Value {
el.Metric = copyLabels(el.Metric, false) el.Metric = copyLabels(el.Metric, false)
values := make(VectorByValueHeap, 0, len(el.Values)) values := make(VectorByValueHeap, 0, len(el.Values))
for _, v := range el.Values { for _, v := range el.Values {
values = append(values, sample{Value: v.v}) values = append(values, sample{Point: Point{V: v.V}})
} }
resultVector = append(resultVector, sample{ resultVector = append(resultVector, sample{
Metric: el.Metric, Metric: el.Metric,
Value: quantile(q, values), Point: Point{V: quantile(q, values), T: ev.Timestamp},
Timestamp: ev.Timestamp,
}) })
} }
return resultVector return resultVector
@ -513,11 +508,11 @@ func funcQuantileOverTime(ev *evaluator, args Expressions) Value {
// === stddev_over_time(Matrix ValueTypeMatrix) Vector === // === stddev_over_time(Matrix ValueTypeMatrix) Vector ===
func funcStddevOverTime(ev *evaluator, args Expressions) Value { func funcStddevOverTime(ev *evaluator, args Expressions) Value {
return aggrOverTime(ev, args, func(values []samplePair) float64 { return aggrOverTime(ev, args, func(values []Point) float64 {
var sum, squaredSum, count float64 var sum, squaredSum, count float64
for _, v := range values { for _, v := range values {
sum += v.v sum += v.V
squaredSum += v.v * v.v squaredSum += v.V * v.V
count++ count++
} }
avg := sum / count avg := sum / count
@ -527,11 +522,11 @@ func funcStddevOverTime(ev *evaluator, args Expressions) Value {
// === stdvar_over_time(Matrix ValueTypeMatrix) Vector === // === stdvar_over_time(Matrix ValueTypeMatrix) Vector ===
func funcStdvarOverTime(ev *evaluator, args Expressions) Value { func funcStdvarOverTime(ev *evaluator, args Expressions) Value {
return aggrOverTime(ev, args, func(values []samplePair) float64 { return aggrOverTime(ev, args, func(values []Point) float64 {
var sum, squaredSum, count float64 var sum, squaredSum, count float64
for _, v := range values { for _, v := range values {
sum += v.v sum += v.V
squaredSum += v.v * v.v squaredSum += v.V * v.V
count++ count++
} }
avg := sum / count avg := sum / count
@ -544,7 +539,7 @@ func funcAbs(ev *evaluator, args Expressions) Value {
Vector := ev.evalVector(args[0]) Vector := ev.evalVector(args[0])
for _, el := range Vector { for _, el := range Vector {
el.Metric = copyLabels(el.Metric, false) el.Metric = copyLabels(el.Metric, false)
el.Value = math.Abs(float64(el.Value)) el.V = math.Abs(float64(el.V))
} }
return Vector return Vector
} }
@ -565,9 +560,8 @@ func funcAbsent(ev *evaluator, args Expressions) Value {
} }
return Vector{ return Vector{
sample{ sample{
Metric: labels.New(m...), Metric: labels.New(m...),
Value: 1, Point: Point{V: 1, T: ev.Timestamp},
Timestamp: ev.Timestamp,
}, },
} }
} }
@ -577,7 +571,7 @@ func funcCeil(ev *evaluator, args Expressions) Value {
Vector := ev.evalVector(args[0]) Vector := ev.evalVector(args[0])
for _, el := range Vector { for _, el := range Vector {
el.Metric = copyLabels(el.Metric, false) el.Metric = copyLabels(el.Metric, false)
el.Value = math.Ceil(float64(el.Value)) el.V = math.Ceil(float64(el.V))
} }
return Vector return Vector
} }
@ -587,7 +581,7 @@ func funcExp(ev *evaluator, args Expressions) Value {
Vector := ev.evalVector(args[0]) Vector := ev.evalVector(args[0])
for _, el := range Vector { for _, el := range Vector {
el.Metric = copyLabels(el.Metric, false) el.Metric = copyLabels(el.Metric, false)
el.Value = math.Exp(float64(el.Value)) el.V = math.Exp(float64(el.V))
} }
return Vector return Vector
} }
@ -597,7 +591,7 @@ func funcSqrt(ev *evaluator, args Expressions) Value {
Vector := ev.evalVector(args[0]) Vector := ev.evalVector(args[0])
for _, el := range Vector { for _, el := range Vector {
el.Metric = copyLabels(el.Metric, false) el.Metric = copyLabels(el.Metric, false)
el.Value = math.Sqrt(float64(el.Value)) el.V = math.Sqrt(float64(el.V))
} }
return Vector return Vector
} }
@ -607,7 +601,7 @@ func funcLn(ev *evaluator, args Expressions) Value {
Vector := ev.evalVector(args[0]) Vector := ev.evalVector(args[0])
for _, el := range Vector { for _, el := range Vector {
el.Metric = copyLabels(el.Metric, false) el.Metric = copyLabels(el.Metric, false)
el.Value = math.Log(float64(el.Value)) el.V = math.Log(float64(el.V))
} }
return Vector return Vector
} }
@ -617,7 +611,7 @@ func funcLog2(ev *evaluator, args Expressions) Value {
Vector := ev.evalVector(args[0]) Vector := ev.evalVector(args[0])
for _, el := range Vector { for _, el := range Vector {
el.Metric = copyLabels(el.Metric, false) el.Metric = copyLabels(el.Metric, false)
el.Value = math.Log2(float64(el.Value)) el.V = math.Log2(float64(el.V))
} }
return Vector return Vector
} }
@ -627,7 +621,7 @@ func funcLog10(ev *evaluator, args Expressions) Value {
Vector := ev.evalVector(args[0]) Vector := ev.evalVector(args[0])
for _, el := range Vector { for _, el := range Vector {
el.Metric = copyLabels(el.Metric, false) el.Metric = copyLabels(el.Metric, false)
el.Value = math.Log10(float64(el.Value)) el.V = math.Log10(float64(el.V))
} }
return Vector return Vector
} }
@ -635,18 +629,18 @@ func funcLog10(ev *evaluator, args Expressions) Value {
// linearRegression performs a least-square linear regression analysis on the // linearRegression performs a least-square linear regression analysis on the
// provided SamplePairs. It returns the slope, and the intercept value at the // provided SamplePairs. It returns the slope, and the intercept value at the
// provided time. // provided time.
func linearRegression(samples []samplePair, interceptTime int64) (slope, intercept float64) { func linearRegression(samples []Point, interceptTime int64) (slope, intercept float64) {
var ( var (
n float64 n float64
sumX, sumY float64 sumX, sumY float64
sumXY, sumX2 float64 sumXY, sumX2 float64
) )
for _, sample := range samples { for _, sample := range samples {
x := float64(sample.t-interceptTime) / 1e6 x := float64(sample.T-interceptTime) / 1e6
n += 1.0 n += 1.0
sumY += sample.v sumY += sample.V
sumX += x sumX += x
sumXY += x * sample.v sumXY += x * sample.V
sumX2 += x * x sumX2 += x * x
} }
covXY := sumXY - sumX*sumY/n covXY := sumXY - sumX*sumY/n
@ -670,9 +664,8 @@ func funcDeriv(ev *evaluator, args Expressions) Value {
} }
slope, _ := linearRegression(samples.Values, 0) slope, _ := linearRegression(samples.Values, 0)
resultSample := sample{ resultSample := sample{
Metric: copyLabels(samples.Metric, false), Metric: copyLabels(samples.Metric, false),
Value: slope, Point: Point{V: slope, T: ev.Timestamp},
Timestamp: ev.Timestamp,
} }
resultVector = append(resultVector, resultSample) resultVector = append(resultVector, resultSample)
@ -695,9 +688,8 @@ func funcPredictLinear(ev *evaluator, args Expressions) Value {
slope, intercept := linearRegression(samples.Values, ev.Timestamp) slope, intercept := linearRegression(samples.Values, ev.Timestamp)
resultVector = append(resultVector, sample{ resultVector = append(resultVector, sample{
Metric: copyLabels(samples.Metric, false), Metric: copyLabels(samples.Metric, false),
Value: slope*duration + intercept, Point: Point{V: slope*duration + intercept, T: ev.Timestamp},
Timestamp: ev.Timestamp,
}) })
} }
return resultVector return resultVector
@ -731,14 +723,13 @@ func funcHistogramQuantile(ev *evaluator, args Expressions) Value {
mb = &metricWithBuckets{el.Metric, nil} mb = &metricWithBuckets{el.Metric, nil}
signatureToMetricWithBuckets[hash] = mb signatureToMetricWithBuckets[hash] = mb
} }
mb.buckets = append(mb.buckets, bucket{upperBound, el.Value}) mb.buckets = append(mb.buckets, bucket{upperBound, el.V})
} }
for _, mb := range signatureToMetricWithBuckets { for _, mb := range signatureToMetricWithBuckets {
outVec = append(outVec, sample{ outVec = append(outVec, sample{
Metric: mb.metric, Metric: mb.metric,
Value: bucketQuantile(q, mb.buckets), Point: Point{V: bucketQuantile(q, mb.buckets), T: ev.Timestamp},
Timestamp: ev.Timestamp,
}) })
} }
@ -752,9 +743,9 @@ func funcResets(ev *evaluator, args Expressions) Value {
for _, samples := range in { for _, samples := range in {
resets := 0 resets := 0
prev := samples.Values[0].v prev := samples.Values[0].V
for _, sample := range samples.Values[1:] { for _, sample := range samples.Values[1:] {
current := sample.v current := sample.V
if current < prev { if current < prev {
resets++ resets++
} }
@ -762,9 +753,8 @@ func funcResets(ev *evaluator, args Expressions) Value {
} }
out = append(out, sample{ out = append(out, sample{
Metric: copyLabels(samples.Metric, false), Metric: copyLabels(samples.Metric, false),
Value: float64(resets), Point: Point{V: float64(resets), T: ev.Timestamp},
Timestamp: ev.Timestamp,
}) })
} }
return out return out
@ -777,9 +767,9 @@ func funcChanges(ev *evaluator, args Expressions) Value {
for _, samples := range in { for _, samples := range in {
changes := 0 changes := 0
prev := samples.Values[0].v prev := samples.Values[0].V
for _, sample := range samples.Values[1:] { for _, sample := range samples.Values[1:] {
current := sample.v current := sample.V
if current != prev && !(math.IsNaN(float64(current)) && math.IsNaN(float64(prev))) { if current != prev && !(math.IsNaN(float64(current)) && math.IsNaN(float64(prev))) {
changes++ changes++
} }
@ -787,9 +777,8 @@ func funcChanges(ev *evaluator, args Expressions) Value {
} }
out = append(out, sample{ out = append(out, sample{
Metric: copyLabels(samples.Metric, false), Metric: copyLabels(samples.Metric, false),
Value: float64(changes), Point: Point{V: float64(changes), T: ev.Timestamp},
Timestamp: ev.Timestamp,
}) })
} }
return out return out
@ -844,9 +833,8 @@ func funcLabelReplace(ev *evaluator, args Expressions) Value {
func funcVector(ev *evaluator, args Expressions) Value { func funcVector(ev *evaluator, args Expressions) Value {
return Vector{ return Vector{
sample{ sample{
Metric: labels.Labels{}, Metric: labels.Labels{},
Value: ev.evalFloat(args[0]), Point: Point{V: ev.evalFloat(args[0]), T: ev.Timestamp},
Timestamp: ev.Timestamp,
}, },
} }
} }
@ -858,7 +846,7 @@ func dateWrapper(ev *evaluator, args Expressions, f func(time.Time) float64) Val
v = Vector{ v = Vector{
sample{ sample{
Metric: labels.Labels{}, Metric: labels.Labels{},
Value: float64(ev.Timestamp) / 1000, Point: Point{V: float64(ev.Timestamp) / 1000},
}, },
} }
} else { } else {
@ -866,8 +854,8 @@ func dateWrapper(ev *evaluator, args Expressions, f func(time.Time) float64) Val
} }
for _, el := range v { for _, el := range v {
el.Metric = copyLabels(el.Metric, false) el.Metric = copyLabels(el.Metric, false)
t := time.Unix(int64(el.Value), 0).UTC() t := time.Unix(int64(el.V), 0).UTC()
el.Value = f(t) el.V = f(t)
} }
return v return v
} }
@ -1221,10 +1209,10 @@ func (s VectorByValueHeap) Len() int {
} }
func (s VectorByValueHeap) Less(i, j int) bool { func (s VectorByValueHeap) Less(i, j int) bool {
if math.IsNaN(float64(s[i].Value)) { if math.IsNaN(float64(s[i].V)) {
return true return true
} }
return s[i].Value < s[j].Value return s[i].V < s[j].V
} }
func (s VectorByValueHeap) Swap(i, j int) { func (s VectorByValueHeap) Swap(i, j int) {
@ -1250,10 +1238,10 @@ func (s VectorByReverseValueHeap) Len() int {
} }
func (s VectorByReverseValueHeap) Less(i, j int) bool { func (s VectorByReverseValueHeap) Less(i, j int) bool {
if math.IsNaN(float64(s[i].Value)) { if math.IsNaN(float64(s[i].V)) {
return true return true
} }
return s[i].Value > s[j].Value return s[i].V > s[j].V
} }
func (s VectorByReverseValueHeap) Swap(i, j int) { func (s VectorByReverseValueHeap) Swap(i, j int) {

View file

@ -31,7 +31,7 @@ var testExpr = []struct {
fail bool // Whether parsing is supposed to fail. fail bool // Whether parsing is supposed to fail.
errMsg string // If not empty the parsing error has to contain this string. errMsg string // If not empty the parsing error has to contain this string.
}{ }{
// Scalars and scalar-to-scalar operations. // Scalars and Scalar-to-Scalar operations.
{ {
input: "1", input: "1",
expected: &NumberLiteral{1}, expected: &NumberLiteral{1},
@ -212,19 +212,19 @@ var testExpr = []struct {
}, { }, {
input: "1 and 1", input: "1 and 1",
fail: true, fail: true,
errMsg: "set operator \"and\" not allowed in binary scalar expression", errMsg: "set operator \"and\" not allowed in binary Scalar expression",
}, { }, {
input: "1 == 1", input: "1 == 1",
fail: true, fail: true,
errMsg: "parse error at char 7: comparisons between scalars must use BOOL modifier", errMsg: "parse error at char 7: comparisons between Scalars must use BOOL modifier",
}, { }, {
input: "1 or 1", input: "1 or 1",
fail: true, fail: true,
errMsg: "set operator \"or\" not allowed in binary scalar expression", errMsg: "set operator \"or\" not allowed in binary Scalar expression",
}, { }, {
input: "1 unless 1", input: "1 unless 1",
fail: true, fail: true,
errMsg: "set operator \"unless\" not allowed in binary scalar expression", errMsg: "set operator \"unless\" not allowed in binary Scalar expression",
}, { }, {
input: "1 !~ 1", input: "1 !~ 1",
fail: true, fail: true,
@ -236,11 +236,11 @@ var testExpr = []struct {
}, { }, {
input: `-"string"`, input: `-"string"`,
fail: true, fail: true,
errMsg: `unary expression only allowed on expressions of type scalar or instant Vector, got "string"`, errMsg: `unary expression only allowed on expressions of type Scalar or instant Vector, got "string"`,
}, { }, {
input: `-test[5m]`, input: `-test[5m]`,
fail: true, fail: true,
errMsg: `unary expression only allowed on expressions of type scalar or instant Vector, got "range Vector"`, errMsg: `unary expression only allowed on expressions of type Scalar or instant Vector, got "range Vector"`,
}, { }, {
input: `*test`, input: `*test`,
fail: true, fail: true,
@ -747,27 +747,27 @@ var testExpr = []struct {
}, { }, {
input: "foo and 1", input: "foo and 1",
fail: true, fail: true,
errMsg: "set operator \"and\" not allowed in binary scalar expression", errMsg: "set operator \"and\" not allowed in binary Scalar expression",
}, { }, {
input: "1 and foo", input: "1 and foo",
fail: true, fail: true,
errMsg: "set operator \"and\" not allowed in binary scalar expression", errMsg: "set operator \"and\" not allowed in binary Scalar expression",
}, { }, {
input: "foo or 1", input: "foo or 1",
fail: true, fail: true,
errMsg: "set operator \"or\" not allowed in binary scalar expression", errMsg: "set operator \"or\" not allowed in binary Scalar expression",
}, { }, {
input: "1 or foo", input: "1 or foo",
fail: true, fail: true,
errMsg: "set operator \"or\" not allowed in binary scalar expression", errMsg: "set operator \"or\" not allowed in binary Scalar expression",
}, { }, {
input: "foo unless 1", input: "foo unless 1",
fail: true, fail: true,
errMsg: "set operator \"unless\" not allowed in binary scalar expression", errMsg: "set operator \"unless\" not allowed in binary Scalar expression",
}, { }, {
input: "1 unless foo", input: "1 unless foo",
fail: true, fail: true,
errMsg: "set operator \"unless\" not allowed in binary scalar expression", errMsg: "set operator \"unless\" not allowed in binary Scalar expression",
}, { }, {
input: "1 or on(bar) foo", input: "1 or on(bar) foo",
fail: true, fail: true,
@ -1285,11 +1285,11 @@ var testExpr = []struct {
}, { }, {
input: `topk(some_metric, other_metric)`, input: `topk(some_metric, other_metric)`,
fail: true, fail: true,
errMsg: "parse error at char 32: expected type scalar in aggregation parameter, got instant Vector", errMsg: "parse error at char 32: expected type Scalar in aggregation parameter, got instant Vector",
}, { }, {
input: `count_values(5, other_metric)`, input: `count_values(5, other_metric)`,
fail: true, fail: true,
errMsg: "parse error at char 30: expected type string in aggregation parameter, got scalar", errMsg: "parse error at char 30: expected type string in aggregation parameter, got Scalar",
}, },
// Test function calls. // Test function calls.
{ {
@ -1363,7 +1363,7 @@ var testExpr = []struct {
}, { }, {
input: "floor(1)", input: "floor(1)",
fail: true, fail: true,
errMsg: "expected type instant Vector in call to function \"floor\", got scalar", errMsg: "expected type instant Vector in call to function \"floor\", got Scalar",
}, { }, {
input: "non_existent_function_far_bar()", input: "non_existent_function_far_bar()",
fail: true, fail: true,

View file

@ -133,5 +133,5 @@ func quantile(q float64, values VectorByValueHeap) float64 {
upperIndex := math.Min(n-1, lowerIndex+1) upperIndex := math.Min(n-1, lowerIndex+1)
weight := rank - math.Floor(rank) weight := rank - math.Floor(rank)
return float64(values[int(lowerIndex)].Value)*(1-weight) + float64(values[int(upperIndex)].Value)*weight return float64(values[int(lowerIndex)].V)*(1-weight) + float64(values[int(upperIndex)].V)*weight
} }

View file

@ -246,14 +246,14 @@ func (*evalCmd) testCmd() {}
type loadCmd struct { type loadCmd struct {
gap time.Duration gap time.Duration
metrics map[uint64]labels.Labels metrics map[uint64]labels.Labels
defs map[uint64][]samplePair defs map[uint64][]Point
} }
func newLoadCmd(gap time.Duration) *loadCmd { func newLoadCmd(gap time.Duration) *loadCmd {
return &loadCmd{ return &loadCmd{
gap: gap, gap: gap,
metrics: map[uint64]labels.Labels{}, metrics: map[uint64]labels.Labels{},
defs: map[uint64][]samplePair{}, defs: map[uint64][]Point{},
} }
} }
@ -265,13 +265,13 @@ func (cmd loadCmd) String() string {
func (cmd *loadCmd) set(m labels.Labels, vals ...sequenceValue) { func (cmd *loadCmd) set(m labels.Labels, vals ...sequenceValue) {
h := m.Hash() h := m.Hash()
samples := make([]samplePair, 0, len(vals)) samples := make([]Point, 0, len(vals))
ts := testStartTime ts := testStartTime
for _, v := range vals { for _, v := range vals {
if !v.omitted { if !v.omitted {
samples = append(samples, samplePair{ samples = append(samples, Point{
t: ts.UnixNano() / int64(time.Millisecond/time.Nanosecond), T: ts.UnixNano() / int64(time.Millisecond/time.Nanosecond),
v: v.value, V: v.value,
}) })
} }
ts = ts.Add(cmd.gap) ts = ts.Add(cmd.gap)
@ -366,7 +366,7 @@ func (ev *evalCmd) compareResult(result 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(expVal.value, v.Values[i].v) { 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)
} }
} }
@ -392,8 +392,8 @@ func (ev *evalCmd) compareResult(result Value) error {
if ev.ordered && exp.pos != pos+1 { if ev.ordered && 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) return fmt.Errorf("expected metric %s with %v at position %d but was at %d", v.Metric, exp.vals, exp.pos, pos+1)
} }
if !almostEqual(float64(exp.vals[0].value), float64(v.Value)) { if !almostEqual(float64(exp.vals[0].value), float64(v.V)) {
return fmt.Errorf("expected %v for %s but got %v", exp.vals[0].value, v.Metric, v.Value) return fmt.Errorf("expected %v for %s but got %v", exp.vals[0].value, v.Metric, v.V)
} }
seen[fp] = true seen[fp] = true
@ -405,8 +405,8 @@ func (ev *evalCmd) compareResult(result Value) error {
} }
case Scalar: case Scalar:
if !almostEqual(ev.expected[0].vals[0].value, val.v) { 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) return fmt.Errorf("expected Scalar %v but got %v", val.V, ev.expected[0].vals[0].value)
} }
default: default: