mirror of
https://github.com/prometheus/prometheus.git
synced 2025-03-05 20:59:13 -08:00
promql: scalar T/V and Point
This commit is contained in:
parent
09666e2e2a
commit
ac5d3bc05e
138
promql/engine.go
138
promql/engine.go
|
@ -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
|
||||||
|
|
|
@ -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) {
|
||||||
|
|
|
@ -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,
|
||||||
|
|
|
@ -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
|
||||||
}
|
}
|
||||||
|
|
|
@ -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:
|
||||||
|
|
Loading…
Reference in a new issue