Like Prometheus, but for logs.
You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
loki/pkg/logql/quantile_over_time_sketch.go

453 lines
12 KiB

package logql
import (
"fmt"
Benchmark and improve sketch join (#11534) **What this PR does / why we need it**: We saw that `JoinQuantileSketchVector` was using too much memory. This is a small step to improve the allocated memory. ``` pkg: github.com/grafana/loki/pkg/logql │ main.log │ pool.log │ │ sec/op │ sec/op vs base │ JoinQuantileSketchVector-10 3.603µ ± 4% 2.631µ ± 0% -26.99% (p=0.002 n=6) │ main.log │ pool.log │ │ B/op │ B/op vs base │ JoinQuantileSketchVector-10 8.344Ki ± 0% 2.539Ki ± 0% -69.57% (p=0.002 n=6) │ main.log │ pool.log │ │ allocs/op │ allocs/op vs base │ JoinQuantileSketchVector-10 109.0 ± 0% 104.0 ± 0% -4.59% (p=0.002 n=6) ``` **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Callum Styan <callumstyan@gmail.com> Co-authored-by: Callum Styan <callumstyan@gmail.com>
1 year ago
"math"
"sync"
"time"
"github.com/prometheus/prometheus/model/labels"
"github.com/prometheus/prometheus/promql"
promql_parser "github.com/prometheus/prometheus/promql/parser"
"github.com/grafana/loki/pkg/iter"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logql/sketch"
"github.com/grafana/loki/pkg/logqlmodel"
Benchmark and improve sketch join (#11534) **What this PR does / why we need it**: We saw that `JoinQuantileSketchVector` was using too much memory. This is a small step to improve the allocated memory. ``` pkg: github.com/grafana/loki/pkg/logql │ main.log │ pool.log │ │ sec/op │ sec/op vs base │ JoinQuantileSketchVector-10 3.603µ ± 4% 2.631µ ± 0% -26.99% (p=0.002 n=6) │ main.log │ pool.log │ │ B/op │ B/op vs base │ JoinQuantileSketchVector-10 8.344Ki ± 0% 2.539Ki ± 0% -69.57% (p=0.002 n=6) │ main.log │ pool.log │ │ allocs/op │ allocs/op vs base │ JoinQuantileSketchVector-10 109.0 ± 0% 104.0 ± 0% -4.59% (p=0.002 n=6) ``` **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Callum Styan <callumstyan@gmail.com> Co-authored-by: Callum Styan <callumstyan@gmail.com>
1 year ago
"github.com/grafana/loki/pkg/queue"
)
const (
QuantileSketchMatrixType = "QuantileSketchMatrix"
)
type ProbabilisticQuantileVector []ProbabilisticQuantileSample
type ProbabilisticQuantileMatrix []ProbabilisticQuantileVector
var streamHashPool = sync.Pool{
New: func() interface{} { return make(map[uint64]int) },
}
func (q ProbabilisticQuantileVector) Merge(right ProbabilisticQuantileVector) (ProbabilisticQuantileVector, error) {
// labels hash to vector index map
groups := streamHashPool.Get().(map[uint64]int)
defer func() {
clear(groups)
streamHashPool.Put(groups)
}()
for i, sample := range q {
groups[sample.Metric.Hash()] = i
}
for _, sample := range right {
i, ok := groups[sample.Metric.Hash()]
if !ok {
q = append(q, sample)
continue
}
_, err := q[i].F.Merge(sample.F)
if err != nil {
return q, err
}
}
return q, nil
}
func (ProbabilisticQuantileVector) SampleVector() promql.Vector {
return promql.Vector{}
}
func (q ProbabilisticQuantileVector) QuantileSketchVec() ProbabilisticQuantileVector {
return q
}
func (q ProbabilisticQuantileVector) ToProto() *logproto.QuantileSketchVector {
samples := make([]*logproto.QuantileSketchSample, len(q))
for i, sample := range q {
samples[i] = sample.ToProto()
}
return &logproto.QuantileSketchVector{Samples: samples}
}
func ProbabilisticQuantileVectorFromProto(proto *logproto.QuantileSketchVector) (ProbabilisticQuantileVector, error) {
out := make([]ProbabilisticQuantileSample, len(proto.Samples))
var s ProbabilisticQuantileSample
var err error
for i, sample := range proto.Samples {
s, err = probabilisticQuantileSampleFromProto(sample)
if err != nil {
return ProbabilisticQuantileVector{}, err
}
out[i] = s
}
return out, nil
}
func (ProbabilisticQuantileMatrix) String() string {
return "QuantileSketchMatrix()"
}
func (m ProbabilisticQuantileMatrix) Merge(right ProbabilisticQuantileMatrix) (ProbabilisticQuantileMatrix, error) {
if len(m) != len(right) {
return nil, fmt.Errorf("failed to merge probabilistic quantile matrix: lengths differ %d!=%d", len(m), len(right))
}
var err error
for i, vec := range m {
m[i], err = vec.Merge(right[i])
if err != nil {
return nil, fmt.Errorf("failed to merge probabilistic quantile matrix: %w", err)
}
}
return m, nil
}
func (ProbabilisticQuantileMatrix) Type() promql_parser.ValueType { return QuantileSketchMatrixType }
Benchmark and improve sketch join (#11534) **What this PR does / why we need it**: We saw that `JoinQuantileSketchVector` was using too much memory. This is a small step to improve the allocated memory. ``` pkg: github.com/grafana/loki/pkg/logql │ main.log │ pool.log │ │ sec/op │ sec/op vs base │ JoinQuantileSketchVector-10 3.603µ ± 4% 2.631µ ± 0% -26.99% (p=0.002 n=6) │ main.log │ pool.log │ │ B/op │ B/op vs base │ JoinQuantileSketchVector-10 8.344Ki ± 0% 2.539Ki ± 0% -69.57% (p=0.002 n=6) │ main.log │ pool.log │ │ allocs/op │ allocs/op vs base │ JoinQuantileSketchVector-10 109.0 ± 0% 104.0 ± 0% -4.59% (p=0.002 n=6) ``` **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Callum Styan <callumstyan@gmail.com> Co-authored-by: Callum Styan <callumstyan@gmail.com>
1 year ago
func (m ProbabilisticQuantileMatrix) Release() {
quantileVectorPool.Put(m)
}
func (m ProbabilisticQuantileMatrix) ToProto() *logproto.QuantileSketchMatrix {
values := make([]*logproto.QuantileSketchVector, len(m))
for i, vec := range m {
values[i] = vec.ToProto()
}
return &logproto.QuantileSketchMatrix{Values: values}
}
func ProbabilisticQuantileMatrixFromProto(proto *logproto.QuantileSketchMatrix) (ProbabilisticQuantileMatrix, error) {
out := make([]ProbabilisticQuantileVector, len(proto.Values))
var s ProbabilisticQuantileVector
var err error
for i, v := range proto.Values {
s, err = ProbabilisticQuantileVectorFromProto(v)
if err != nil {
return ProbabilisticQuantileMatrix{}, err
}
out[i] = s
}
return out, nil
}
type QuantileSketchStepEvaluator struct {
iter RangeVectorIterator
err error
}
func (e *QuantileSketchStepEvaluator) Next() (bool, int64, StepResult) {
next := e.iter.Next()
if !next {
return false, 0, ProbabilisticQuantileVector{}
}
ts, r := e.iter.At()
vec := r.QuantileSketchVec()
for _, s := range vec {
// Errors are not allowed in metrics unless they've been specifically requested.
if s.Metric.Has(logqlmodel.ErrorLabel) && s.Metric.Get(logqlmodel.PreserveErrorLabel) != "true" {
e.err = logqlmodel.NewPipelineErr(s.Metric)
return false, 0, ProbabilisticQuantileVector{}
}
}
return true, ts, vec
}
func (e *QuantileSketchStepEvaluator) Close() error { return e.iter.Close() }
func (e *QuantileSketchStepEvaluator) Error() error {
if e.err != nil {
return e.err
}
return e.iter.Error()
}
func (e *QuantileSketchStepEvaluator) Explain(parent Node) {
parent.Child("QuantileSketch")
}
func newQuantileSketchIterator(
it iter.PeekingSampleIterator,
selRange, step, start, end, offset int64) RangeVectorIterator {
inner := &batchRangeVectorIterator{
iter: it,
step: step,
end: end,
selRange: selRange,
metrics: map[string]labels.Labels{},
window: map[string]*promql.Series{},
agg: nil,
current: start - step, // first loop iteration will set it to start
offset: offset,
}
return &quantileSketchBatchRangeVectorIterator{
batchRangeVectorIterator: inner,
}
}
type ProbabilisticQuantileSample struct {
T int64
F sketch.QuantileSketch
Metric labels.Labels
}
func (q ProbabilisticQuantileSample) ToProto() *logproto.QuantileSketchSample {
metric := make([]*logproto.LabelPair, len(q.Metric))
for i, m := range q.Metric {
metric[i] = &logproto.LabelPair{Name: m.Name, Value: m.Value}
}
sketch := q.F.ToProto()
return &logproto.QuantileSketchSample{
F: sketch,
TimestampMs: q.T,
Metric: metric,
}
}
func probabilisticQuantileSampleFromProto(proto *logproto.QuantileSketchSample) (ProbabilisticQuantileSample, error) {
s, err := sketch.QuantileSketchFromProto(proto.F)
if err != nil {
return ProbabilisticQuantileSample{}, err
}
out := ProbabilisticQuantileSample{
T: proto.TimestampMs,
F: s,
Metric: make(labels.Labels, len(proto.Metric)),
}
for i, p := range proto.Metric {
out.Metric[i] = labels.Label{Name: p.Name, Value: p.Value}
}
return out, nil
}
type quantileSketchBatchRangeVectorIterator struct {
*batchRangeVectorIterator
at []ProbabilisticQuantileSample
}
func (r *quantileSketchBatchRangeVectorIterator) At() (int64, StepResult) {
if r.at == nil {
r.at = make([]ProbabilisticQuantileSample, 0, len(r.window))
}
r.at = r.at[:0]
// convert ts from nano to milli seconds as the iterator work with nanoseconds
ts := r.current/1e+6 + r.offset/1e+6
for _, series := range r.window {
r.at = append(r.at, ProbabilisticQuantileSample{
F: r.agg(series.Floats),
T: ts,
Metric: series.Metric,
})
}
return ts, ProbabilisticQuantileVector(r.at)
}
func (r *quantileSketchBatchRangeVectorIterator) agg(samples []promql.FPoint) sketch.QuantileSketch {
s := sketch.NewDDSketch()
for _, v := range samples {
// The sketch from the underlying sketch package we are using
// cannot return an error when calling Add.
s.Add(v.F) //nolint:errcheck
}
return s
}
Benchmark and improve sketch join (#11534) **What this PR does / why we need it**: We saw that `JoinQuantileSketchVector` was using too much memory. This is a small step to improve the allocated memory. ``` pkg: github.com/grafana/loki/pkg/logql │ main.log │ pool.log │ │ sec/op │ sec/op vs base │ JoinQuantileSketchVector-10 3.603µ ± 4% 2.631µ ± 0% -26.99% (p=0.002 n=6) │ main.log │ pool.log │ │ B/op │ B/op vs base │ JoinQuantileSketchVector-10 8.344Ki ± 0% 2.539Ki ± 0% -69.57% (p=0.002 n=6) │ main.log │ pool.log │ │ allocs/op │ allocs/op vs base │ JoinQuantileSketchVector-10 109.0 ± 0% 104.0 ± 0% -4.59% (p=0.002 n=6) ``` **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Callum Styan <callumstyan@gmail.com> Co-authored-by: Callum Styan <callumstyan@gmail.com>
1 year ago
// quantileVectorPool slice of ProbabilisticQuantileVector [64, 128, 256, ..., 65536]
var quantileVectorPool = queue.NewSlicePool[ProbabilisticQuantileVector](1<<6, 1<<16, 2)
// JoinQuantileSketchVector joins the results from stepEvaluator into a ProbabilisticQuantileMatrix.
Benchmark and improve sketch join (#11534) **What this PR does / why we need it**: We saw that `JoinQuantileSketchVector` was using too much memory. This is a small step to improve the allocated memory. ``` pkg: github.com/grafana/loki/pkg/logql │ main.log │ pool.log │ │ sec/op │ sec/op vs base │ JoinQuantileSketchVector-10 3.603µ ± 4% 2.631µ ± 0% -26.99% (p=0.002 n=6) │ main.log │ pool.log │ │ B/op │ B/op vs base │ JoinQuantileSketchVector-10 8.344Ki ± 0% 2.539Ki ± 0% -69.57% (p=0.002 n=6) │ main.log │ pool.log │ │ allocs/op │ allocs/op vs base │ JoinQuantileSketchVector-10 109.0 ± 0% 104.0 ± 0% -4.59% (p=0.002 n=6) ``` **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Callum Styan <callumstyan@gmail.com> Co-authored-by: Callum Styan <callumstyan@gmail.com>
1 year ago
func JoinQuantileSketchVector(next bool, r StepResult, stepEvaluator StepEvaluator, params Params) (promql_parser.Value, error) {
vec := r.QuantileSketchVec()
if stepEvaluator.Error() != nil {
return nil, stepEvaluator.Error()
}
Benchmark and improve sketch join (#11534) **What this PR does / why we need it**: We saw that `JoinQuantileSketchVector` was using too much memory. This is a small step to improve the allocated memory. ``` pkg: github.com/grafana/loki/pkg/logql │ main.log │ pool.log │ │ sec/op │ sec/op vs base │ JoinQuantileSketchVector-10 3.603µ ± 4% 2.631µ ± 0% -26.99% (p=0.002 n=6) │ main.log │ pool.log │ │ B/op │ B/op vs base │ JoinQuantileSketchVector-10 8.344Ki ± 0% 2.539Ki ± 0% -69.57% (p=0.002 n=6) │ main.log │ pool.log │ │ allocs/op │ allocs/op vs base │ JoinQuantileSketchVector-10 109.0 ± 0% 104.0 ± 0% -4.59% (p=0.002 n=6) ``` **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --------- Signed-off-by: Callum Styan <callumstyan@gmail.com> Co-authored-by: Callum Styan <callumstyan@gmail.com>
1 year ago
stepCount := int(math.Ceil(float64(params.End().Sub(params.Start()).Nanoseconds()) / float64(params.Step().Nanoseconds())))
if stepCount <= 0 {
stepCount = 1
}
// The result is released to the pool when the matrix is serialized.
result := quantileVectorPool.Get(stepCount)
for next {
result = append(result, vec)
next, _, r = stepEvaluator.Next()
vec = r.QuantileSketchVec()
if stepEvaluator.Error() != nil {
return nil, stepEvaluator.Error()
}
}
return ProbabilisticQuantileMatrix(result), stepEvaluator.Error()
}
// QuantileSketchMatrixStepEvaluator steps through a matrix of quantile sketch
// vectors, ie t-digest or DDSketch structures per time step.
type QuantileSketchMatrixStepEvaluator struct {
start, end, ts time.Time
step time.Duration
m ProbabilisticQuantileMatrix
}
func NewQuantileSketchMatrixStepEvaluator(m ProbabilisticQuantileMatrix, params Params) *QuantileSketchMatrixStepEvaluator {
var (
start = params.Start()
end = params.End()
step = params.Step()
)
return &QuantileSketchMatrixStepEvaluator{
start: start,
end: end,
ts: start.Add(-step), // will be corrected on first Next() call
step: step,
m: m,
}
}
func (m *QuantileSketchMatrixStepEvaluator) Next() (bool, int64, StepResult) {
m.ts = m.ts.Add(m.step)
if m.ts.After(m.end) {
return false, 0, nil
}
ts := m.ts.UnixNano() / int64(time.Millisecond)
if len(m.m) == 0 {
return false, 0, nil
}
vec := m.m[0]
// Reset for next step
m.m = m.m[1:]
return true, ts, vec
}
func (*QuantileSketchMatrixStepEvaluator) Close() error { return nil }
func (*QuantileSketchMatrixStepEvaluator) Error() error { return nil }
func (*QuantileSketchMatrixStepEvaluator) Explain(parent Node) {
parent.Child("QuantileSketchMatrix")
}
// QuantileSketchMergeStepEvaluator merges multiple quantile sketches into one for each
// step.
type QuantileSketchMergeStepEvaluator struct {
evaluators []StepEvaluator
err error
}
func NewQuantileSketchMergeStepEvaluator(evaluators []StepEvaluator) *QuantileSketchMergeStepEvaluator {
return &QuantileSketchMergeStepEvaluator{
evaluators: evaluators,
err: nil,
}
}
func (e *QuantileSketchMergeStepEvaluator) Next() (bool, int64, StepResult) {
ok, ts, r := e.evaluators[0].Next()
var cur ProbabilisticQuantileVector
if ok {
cur = r.QuantileSketchVec()
}
if len(e.evaluators) == 1 {
return ok, ts, cur
}
for _, eval := range e.evaluators[1:] {
ok, nextTs, vec := eval.Next()
if ok {
if cur == nil {
cur = vec.QuantileSketchVec()
} else {
if ts != nextTs {
e.err = fmt.Errorf("timestamps of sketches differ: %d!=%d", ts, nextTs)
return false, 0, nil
}
_, e.err = cur.Merge(vec.QuantileSketchVec())
if e.err != nil {
return false, 0, nil
}
}
}
}
return ok, ts, cur
}
func (*QuantileSketchMergeStepEvaluator) Close() error { return nil }
func (e *QuantileSketchMergeStepEvaluator) Error() error { return e.err }
func (e *QuantileSketchMergeStepEvaluator) Explain(parent Node) {
b := parent.Child("QuantileSketchMerge")
if len(e.evaluators) < MaxChildrenDisplay {
for _, child := range e.evaluators {
child.Explain(b)
}
} else {
e.evaluators[0].Explain(b)
b.Child("...")
e.evaluators[len(e.evaluators)-1].Explain(b)
}
}
// QuantileSketchVectorStepEvaluator evaluates a quantile sketch into a
// promql.Vector.
type QuantileSketchVectorStepEvaluator struct {
inner StepEvaluator
quantile float64
}
var _ StepEvaluator = NewQuantileSketchVectorStepEvaluator(nil, 0)
func NewQuantileSketchVectorStepEvaluator(inner StepEvaluator, quantile float64) *QuantileSketchVectorStepEvaluator {
return &QuantileSketchVectorStepEvaluator{
inner: inner,
quantile: quantile,
}
}
func (e *QuantileSketchVectorStepEvaluator) Next() (bool, int64, StepResult) {
ok, ts, r := e.inner.Next()
if !ok {
return false, 0, SampleVector{}
}
quantileSketchVec := r.QuantileSketchVec()
vec := make(promql.Vector, len(quantileSketchVec))
for i, quantileSketch := range quantileSketchVec {
f, _ := quantileSketch.F.Quantile(e.quantile)
vec[i] = promql.Sample{
T: quantileSketch.T,
F: f,
Metric: quantileSketch.Metric,
}
}
return ok, ts, SampleVector(vec)
}
func (*QuantileSketchVectorStepEvaluator) Close() error { return nil }
func (*QuantileSketchVectorStepEvaluator) Error() error { return nil }
func (e *QuantileSketchVectorStepEvaluator) Explain(parent Node) {
b := parent.Child("QuantileSketchVector")
e.inner.Explain(b)
}