mirror of https://github.com/grafana/loki
Inject downstream accumulator. (#11863)
**What this PR does / why we need it**: We inject the downstream accumulator into the downstreamer instead of constructing it on the first arrival of the first result. Since we know the query type before executing the first query we can pass the correct accumulator. This will allow us to define special `topk` or `sum by` accumulators in the future. **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](pull/11902/headd10549e3ec
) - [ ] 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](0d4416a4b0
) --------- Co-authored-by: Christian Haudum <christian.haudum@gmail.com>
parent
b26bd6d77b
commit
483ee56139
@ -0,0 +1,379 @@ |
||||
package logql |
||||
|
||||
import ( |
||||
"container/heap" |
||||
"context" |
||||
"fmt" |
||||
"sort" |
||||
"time" |
||||
|
||||
"github.com/grafana/loki/pkg/logproto" |
||||
"github.com/grafana/loki/pkg/logqlmodel" |
||||
"github.com/grafana/loki/pkg/logqlmodel/metadata" |
||||
"github.com/grafana/loki/pkg/logqlmodel/stats" |
||||
"github.com/grafana/loki/pkg/querier/queryrange/queryrangebase/definitions" |
||||
"github.com/grafana/loki/pkg/util/math" |
||||
) |
||||
|
||||
// NewBufferedAccumulator returns an accumulator which aggregates all query
|
||||
// results in a slice. This is useful for metric queries, which are generally
|
||||
// small payloads and the memory overhead for buffering is negligible.
|
||||
func NewBufferedAccumulator(n int) *BufferedAccumulator { |
||||
return &BufferedAccumulator{ |
||||
results: make([]logqlmodel.Result, n), |
||||
} |
||||
} |
||||
|
||||
type BufferedAccumulator struct { |
||||
results []logqlmodel.Result |
||||
} |
||||
|
||||
func (a *BufferedAccumulator) Accumulate(_ context.Context, acc logqlmodel.Result, i int) error { |
||||
a.results[i] = acc |
||||
return nil |
||||
} |
||||
|
||||
func (a *BufferedAccumulator) Result() []logqlmodel.Result { |
||||
return a.results |
||||
} |
||||
|
||||
type QuantileSketchAccumulator struct { |
||||
matrix ProbabilisticQuantileMatrix |
||||
} |
||||
|
||||
// newQuantileSketchAccumulator returns an accumulator for sharded
|
||||
// probabilistic quantile queries that merges results as they come in.
|
||||
func newQuantileSketchAccumulator() *QuantileSketchAccumulator { |
||||
return &QuantileSketchAccumulator{} |
||||
} |
||||
|
||||
func (a *QuantileSketchAccumulator) Accumulate(_ context.Context, res logqlmodel.Result, _ int) error { |
||||
if res.Data.Type() != QuantileSketchMatrixType { |
||||
return fmt.Errorf("unexpected matrix data type: got (%s), want (%s)", res.Data.Type(), QuantileSketchMatrixType) |
||||
} |
||||
data, ok := res.Data.(ProbabilisticQuantileMatrix) |
||||
if !ok { |
||||
return fmt.Errorf("unexpected matrix type: got (%T), want (ProbabilisticQuantileMatrix)", res.Data) |
||||
} |
||||
if a.matrix == nil { |
||||
a.matrix = data |
||||
return nil |
||||
} |
||||
|
||||
var err error |
||||
a.matrix, err = a.matrix.Merge(data) |
||||
return err |
||||
} |
||||
|
||||
func (a *QuantileSketchAccumulator) Result() []logqlmodel.Result { |
||||
return []logqlmodel.Result{{Data: a.matrix}} |
||||
} |
||||
|
||||
// heap impl for keeping only the top n results across m streams
|
||||
// importantly, AccumulatedStreams is _bounded_, so it will only
|
||||
// store the top `limit` results across all streams.
|
||||
// To implement this, we use a min-heap when looking
|
||||
// for the max values (logproto.FORWARD)
|
||||
// and vice versa for logproto.BACKWARD.
|
||||
// This allows us to easily find the 'worst' value
|
||||
// and replace it with a better one.
|
||||
// Once we've fully processed all log lines,
|
||||
// we return the heap in opposite order and then reverse it
|
||||
// to get the correct order.
|
||||
// Heap implements container/heap.Interface
|
||||
// solely to use heap.Interface as a library.
|
||||
// It is not intended for the heap pkg functions
|
||||
// to otherwise call this type.
|
||||
type AccumulatedStreams struct { |
||||
count, limit int |
||||
labelmap map[string]int |
||||
streams []*logproto.Stream |
||||
order logproto.Direction |
||||
|
||||
stats stats.Result // for accumulating statistics from downstream requests
|
||||
headers map[string][]string // for accumulating headers from downstream requests
|
||||
} |
||||
|
||||
// NewStreamAccumulator returns an accumulator for limited log queries.
|
||||
// Log queries, sharded thousands of times and each returning <limit>
|
||||
// results, can be _considerably_ larger. In this case, we eagerly
|
||||
// accumulate the results into a logsAccumulator, discarding values
|
||||
// over the limit to keep memory pressure down while other subqueries
|
||||
// are executing.
|
||||
func NewStreamAccumulator(params Params) *AccumulatedStreams { |
||||
// the stream accumulator stores a heap with reversed order
|
||||
// from the results we expect, so we need to reverse the direction
|
||||
order := logproto.FORWARD |
||||
if params.Direction() == logproto.FORWARD { |
||||
order = logproto.BACKWARD |
||||
} |
||||
|
||||
return &AccumulatedStreams{ |
||||
labelmap: make(map[string]int), |
||||
order: order, |
||||
limit: int(params.Limit()), |
||||
|
||||
headers: make(map[string][]string), |
||||
} |
||||
} |
||||
|
||||
// returns the top priority
|
||||
func (acc *AccumulatedStreams) top() (time.Time, bool) { |
||||
if len(acc.streams) > 0 && len(acc.streams[0].Entries) > 0 { |
||||
return acc.streams[0].Entries[len(acc.streams[0].Entries)-1].Timestamp, true |
||||
} |
||||
return time.Time{}, false |
||||
} |
||||
|
||||
func (acc *AccumulatedStreams) Find(labels string) (int, bool) { |
||||
i, ok := acc.labelmap[labels] |
||||
return i, ok |
||||
} |
||||
|
||||
// number of streams
|
||||
func (acc *AccumulatedStreams) Len() int { return len(acc.streams) } |
||||
|
||||
func (acc *AccumulatedStreams) Swap(i, j int) { |
||||
// for i=0, j=1
|
||||
|
||||
// {'a': 0, 'b': 1}
|
||||
// [a, b]
|
||||
acc.streams[i], acc.streams[j] = acc.streams[j], acc.streams[i] |
||||
// {'a': 0, 'b': 1}
|
||||
// [b, a]
|
||||
acc.labelmap[acc.streams[i].Labels] = i |
||||
acc.labelmap[acc.streams[j].Labels] = j |
||||
// {'a': 1, 'b': 0}
|
||||
// [b, a]
|
||||
} |
||||
|
||||
// first order by timestamp, then by labels
|
||||
func (acc *AccumulatedStreams) Less(i, j int) bool { |
||||
// order by the 'oldest' entry in the stream
|
||||
if a, b := acc.streams[i].Entries[len(acc.streams[i].Entries)-1].Timestamp, acc.streams[j].Entries[len(acc.streams[j].Entries)-1].Timestamp; !a.Equal(b) { |
||||
return acc.less(a, b) |
||||
} |
||||
return acc.streams[i].Labels <= acc.streams[j].Labels |
||||
} |
||||
|
||||
func (acc *AccumulatedStreams) less(a, b time.Time) bool { |
||||
// use after for stable sort
|
||||
if acc.order == logproto.FORWARD { |
||||
return !a.After(b) |
||||
} |
||||
return !b.After(a) |
||||
} |
||||
|
||||
func (acc *AccumulatedStreams) Push(x any) { |
||||
s := x.(*logproto.Stream) |
||||
if len(s.Entries) == 0 { |
||||
return |
||||
} |
||||
|
||||
if room := acc.limit - acc.count; room >= len(s.Entries) { |
||||
if i, ok := acc.Find(s.Labels); ok { |
||||
// stream already exists, append entries
|
||||
|
||||
// these are already guaranteed to be sorted
|
||||
// Reasoning: we shard subrequests so each stream exists on only one
|
||||
// shard. Therefore, the only time a stream should already exist
|
||||
// is in successive splits, which are already guaranteed to be ordered
|
||||
// and we can just append.
|
||||
acc.appendTo(acc.streams[i], s) |
||||
|
||||
return |
||||
} |
||||
|
||||
// new stream
|
||||
acc.addStream(s) |
||||
return |
||||
} |
||||
|
||||
// there's not enough room for all the entries,
|
||||
// so we need to
|
||||
acc.push(s) |
||||
} |
||||
|
||||
// there's not enough room for all the entries.
|
||||
// since we store them in a reverse heap relative to what we _want_
|
||||
// (i.e. the max value for FORWARD, the min value for BACKWARD),
|
||||
// we test if the new entry is better than the worst entry,
|
||||
// swapping them if so.
|
||||
func (acc *AccumulatedStreams) push(s *logproto.Stream) { |
||||
worst, ok := acc.top() |
||||
room := math.Min(acc.limit-acc.count, len(s.Entries)) |
||||
|
||||
if !ok { |
||||
if room == 0 { |
||||
// special case: limit must be zero since there's no room and no worst entry
|
||||
return |
||||
} |
||||
s.Entries = s.Entries[:room] |
||||
// special case: there are no entries in the heap. Push entries up to the limit
|
||||
acc.addStream(s) |
||||
return |
||||
} |
||||
|
||||
// since entries are sorted by timestamp from best -> worst,
|
||||
// we can discard the entire stream if the incoming best entry
|
||||
// is worse than the worst entry in the heap.
|
||||
cutoff := sort.Search(len(s.Entries), func(i int) bool { |
||||
// TODO(refactor label comparison -- should be in another fn)
|
||||
if worst.Equal(s.Entries[i].Timestamp) { |
||||
return acc.streams[0].Labels < s.Labels |
||||
} |
||||
return acc.less(s.Entries[i].Timestamp, worst) |
||||
}) |
||||
s.Entries = s.Entries[:cutoff] |
||||
|
||||
for i := 0; i < len(s.Entries) && acc.less(worst, s.Entries[i].Timestamp); i++ { |
||||
|
||||
// push one entry at a time
|
||||
room = acc.limit - acc.count |
||||
// pop if there's no room to make the heap small enough for an append;
|
||||
// in the short path of Push() we know that there's room for at least one entry
|
||||
if room == 0 { |
||||
acc.Pop() |
||||
} |
||||
|
||||
cpy := *s |
||||
cpy.Entries = []logproto.Entry{s.Entries[i]} |
||||
acc.Push(&cpy) |
||||
|
||||
// update worst
|
||||
worst, _ = acc.top() |
||||
} |
||||
} |
||||
|
||||
func (acc *AccumulatedStreams) addStream(s *logproto.Stream) { |
||||
// ensure entries conform to order we expect
|
||||
// TODO(owen-d): remove? should be unnecessary since we insert in appropriate order
|
||||
// but it's nice to have the safeguard
|
||||
sort.Slice(s.Entries, func(i, j int) bool { |
||||
return acc.less(s.Entries[j].Timestamp, s.Entries[i].Timestamp) |
||||
}) |
||||
|
||||
acc.streams = append(acc.streams, s) |
||||
i := len(acc.streams) - 1 |
||||
acc.labelmap[s.Labels] = i |
||||
acc.count += len(s.Entries) |
||||
heap.Fix(acc, i) |
||||
} |
||||
|
||||
// dst must already exist in acc
|
||||
func (acc *AccumulatedStreams) appendTo(dst, src *logproto.Stream) { |
||||
// these are already guaranteed to be sorted
|
||||
// Reasoning: we shard subrequests so each stream exists on only one
|
||||
// shard. Therefore, the only time a stream should already exist
|
||||
// is in successive splits, which are already guaranteed to be ordered
|
||||
// and we can just append.
|
||||
|
||||
var needsSort bool |
||||
for _, e := range src.Entries { |
||||
// sort if order has broken
|
||||
if len(dst.Entries) > 0 && acc.less(dst.Entries[len(dst.Entries)-1].Timestamp, e.Timestamp) { |
||||
needsSort = true |
||||
} |
||||
dst.Entries = append(dst.Entries, e) |
||||
} |
||||
|
||||
if needsSort { |
||||
sort.Slice(dst.Entries, func(i, j int) bool { |
||||
// store in reverse order so we can more reliably insert without sorting and pop from end
|
||||
return acc.less(dst.Entries[j].Timestamp, dst.Entries[i].Timestamp) |
||||
}) |
||||
} |
||||
|
||||
acc.count += len(src.Entries) |
||||
heap.Fix(acc, acc.labelmap[dst.Labels]) |
||||
|
||||
} |
||||
|
||||
// Pop returns a stream with one entry. It pops the first entry of the first stream
|
||||
func (acc *AccumulatedStreams) Pop() any { |
||||
n := acc.Len() |
||||
if n == 0 { |
||||
return nil |
||||
} |
||||
|
||||
stream := acc.streams[0] |
||||
cpy := *stream |
||||
cpy.Entries = []logproto.Entry{cpy.Entries[len(stream.Entries)-1]} |
||||
stream.Entries = stream.Entries[:len(stream.Entries)-1] |
||||
|
||||
acc.count-- |
||||
|
||||
if len(stream.Entries) == 0 { |
||||
// remove stream
|
||||
acc.Swap(0, n-1) |
||||
acc.streams[n-1] = nil // avoid leaking reference
|
||||
delete(acc.labelmap, stream.Labels) |
||||
acc.streams = acc.streams[:n-1] |
||||
|
||||
} |
||||
|
||||
if acc.Len() > 0 { |
||||
heap.Fix(acc, 0) |
||||
} |
||||
|
||||
return &cpy |
||||
} |
||||
|
||||
// Note: can only be called once as it will alter stream ordreing.
|
||||
func (acc *AccumulatedStreams) Result() []logqlmodel.Result { |
||||
// sort streams by label
|
||||
sort.Slice(acc.streams, func(i, j int) bool { |
||||
return acc.streams[i].Labels < acc.streams[j].Labels |
||||
}) |
||||
|
||||
streams := make(logqlmodel.Streams, 0, len(acc.streams)) |
||||
|
||||
for _, s := range acc.streams { |
||||
// sort entries by timestamp, inversely based on direction
|
||||
sort.Slice(s.Entries, func(i, j int) bool { |
||||
return acc.less(s.Entries[j].Timestamp, s.Entries[i].Timestamp) |
||||
}) |
||||
streams = append(streams, *s) |
||||
} |
||||
|
||||
res := logqlmodel.Result{ |
||||
// stats & headers are already aggregated in the context
|
||||
Data: streams, |
||||
Statistics: acc.stats, |
||||
Headers: make([]*definitions.PrometheusResponseHeader, 0, len(acc.headers)), |
||||
} |
||||
|
||||
for name, vals := range acc.headers { |
||||
res.Headers = append( |
||||
res.Headers, |
||||
&definitions.PrometheusResponseHeader{ |
||||
Name: name, |
||||
Values: vals, |
||||
}, |
||||
) |
||||
} |
||||
|
||||
return []logqlmodel.Result{res} |
||||
} |
||||
|
||||
func (acc *AccumulatedStreams) Accumulate(_ context.Context, x logqlmodel.Result, _ int) error { |
||||
// TODO(owen-d/ewelch): Shard counts should be set by the querier
|
||||
// so we don't have to do it in tricky ways in multiple places.
|
||||
// See pkg/logql/downstream.go:DownstreamEvaluator.Downstream
|
||||
// for another example.
|
||||
if x.Statistics.Summary.Shards == 0 { |
||||
x.Statistics.Summary.Shards = 1 |
||||
} |
||||
acc.stats.Merge(x.Statistics) |
||||
metadata.ExtendHeaders(acc.headers, x.Headers) |
||||
|
||||
switch got := x.Data.(type) { |
||||
case logqlmodel.Streams: |
||||
for i := range got { |
||||
acc.Push(&got[i]) |
||||
} |
||||
default: |
||||
return fmt.Errorf("unexpected response type during response result accumulation. Got (%T), wanted %s", got, logqlmodel.ValueTypeStreams) |
||||
} |
||||
return nil |
||||
} |
@ -0,0 +1,273 @@ |
||||
package logql |
||||
|
||||
import ( |
||||
"context" |
||||
"fmt" |
||||
"math/rand" |
||||
"testing" |
||||
"time" |
||||
|
||||
"github.com/prometheus/prometheus/model/labels" |
||||
"github.com/stretchr/testify/require" |
||||
|
||||
"github.com/grafana/loki/pkg/logproto" |
||||
"github.com/grafana/loki/pkg/logql/sketch" |
||||
"github.com/grafana/loki/pkg/logqlmodel" |
||||
) |
||||
|
||||
func TestAccumulatedStreams(t *testing.T) { |
||||
lim := 30 |
||||
nStreams := 10 |
||||
start, end := 0, 10 |
||||
// for a logproto.BACKWARD query, we use a min heap based on FORWARD
|
||||
// to store the _earliest_ timestamp of the _latest_ entries, up to `limit`
|
||||
xs := newStreams(time.Unix(int64(start), 0), time.Unix(int64(end), 0), time.Second, nStreams, logproto.BACKWARD) |
||||
acc := NewStreamAccumulator(LiteralParams{ |
||||
direction: logproto.BACKWARD, |
||||
limit: uint32(lim), |
||||
}) |
||||
for _, x := range xs { |
||||
acc.Push(x) |
||||
} |
||||
|
||||
for i := 0; i < lim; i++ { |
||||
got := acc.Pop().(*logproto.Stream) |
||||
require.Equal(t, fmt.Sprintf(`{n="%d"}`, i%nStreams), got.Labels) |
||||
exp := (nStreams*(end-start) - lim + i) / nStreams |
||||
require.Equal(t, time.Unix(int64(exp), 0), got.Entries[0].Timestamp) |
||||
} |
||||
|
||||
} |
||||
|
||||
func TestDownstreamAccumulatorSimple(t *testing.T) { |
||||
lim := 30 |
||||
start, end := 0, 10 |
||||
direction := logproto.BACKWARD |
||||
|
||||
streams := newStreams(time.Unix(int64(start), 0), time.Unix(int64(end), 0), time.Second, 10, direction) |
||||
x := make(logqlmodel.Streams, 0, len(streams)) |
||||
for _, s := range streams { |
||||
x = append(x, *s) |
||||
} |
||||
// dummy params. Only need to populate direction & limit
|
||||
params, err := NewLiteralParams( |
||||
`{app="foo"}`, time.Time{}, time.Time{}, 0, 0, direction, uint32(lim), nil, |
||||
) |
||||
require.NoError(t, err) |
||||
|
||||
acc := NewStreamAccumulator(params) |
||||
result := logqlmodel.Result{ |
||||
Data: x, |
||||
} |
||||
|
||||
require.Nil(t, acc.Accumulate(context.Background(), result, 0)) |
||||
|
||||
res := acc.Result()[0] |
||||
got, ok := res.Data.(logqlmodel.Streams) |
||||
require.Equal(t, true, ok) |
||||
require.Equal(t, 10, len(got), "correct number of streams") |
||||
|
||||
// each stream should have the top 3 entries
|
||||
for i := 0; i < 10; i++ { |
||||
require.Equal(t, 3, len(got[i].Entries), "correct number of entries in stream") |
||||
for j := 0; j < 3; j++ { |
||||
require.Equal(t, time.Unix(int64(9-j), 0), got[i].Entries[j].Timestamp, "correct timestamp") |
||||
} |
||||
} |
||||
} |
||||
|
||||
// TestDownstreamAccumulatorMultiMerge simulates merging multiple
|
||||
// sub-results from different queries.
|
||||
func TestDownstreamAccumulatorMultiMerge(t *testing.T) { |
||||
for _, direction := range []logproto.Direction{logproto.BACKWARD, logproto.FORWARD} { |
||||
t.Run(direction.String(), func(t *testing.T) { |
||||
nQueries := 10 |
||||
delta := 10 // 10 entries per stream, 1s apart
|
||||
streamsPerQuery := 10 |
||||
lim := 30 |
||||
|
||||
payloads := make([]logqlmodel.Streams, 0, nQueries) |
||||
for i := 0; i < nQueries; i++ { |
||||
start := i * delta |
||||
end := start + delta |
||||
streams := newStreams(time.Unix(int64(start), 0), time.Unix(int64(end), 0), time.Second, streamsPerQuery, direction) |
||||
var res logqlmodel.Streams |
||||
for i := range streams { |
||||
res = append(res, *streams[i]) |
||||
} |
||||
payloads = append(payloads, res) |
||||
|
||||
} |
||||
|
||||
// queries are always dispatched in the correct order.
|
||||
// oldest time ranges first in the case of logproto.FORWARD
|
||||
// and newest time ranges first in the case of logproto.BACKWARD
|
||||
if direction == logproto.BACKWARD { |
||||
for i, j := 0, len(payloads)-1; i < j; i, j = i+1, j-1 { |
||||
payloads[i], payloads[j] = payloads[j], payloads[i] |
||||
} |
||||
} |
||||
|
||||
// dummy params. Only need to populate direction & limit
|
||||
params, err := NewLiteralParams( |
||||
`{app="foo"}`, time.Time{}, time.Time{}, 0, 0, direction, uint32(lim), nil, |
||||
) |
||||
require.NoError(t, err) |
||||
|
||||
acc := NewStreamAccumulator(params) |
||||
for i := 0; i < nQueries; i++ { |
||||
err := acc.Accumulate(context.Background(), logqlmodel.Result{ |
||||
Data: payloads[i], |
||||
}, i) |
||||
require.Nil(t, err) |
||||
} |
||||
|
||||
got, ok := acc.Result()[0].Data.(logqlmodel.Streams) |
||||
require.Equal(t, true, ok) |
||||
require.Equal(t, int64(nQueries), acc.Result()[0].Statistics.Summary.Shards) |
||||
|
||||
// each stream should have the top 3 entries
|
||||
for i := 0; i < streamsPerQuery; i++ { |
||||
stream := got[i] |
||||
require.Equal(t, fmt.Sprintf(`{n="%d"}`, i), stream.Labels, "correct labels") |
||||
ln := lim / streamsPerQuery |
||||
require.Equal(t, ln, len(stream.Entries), "correct number of entries in stream") |
||||
switch direction { |
||||
case logproto.BACKWARD: |
||||
for i := 0; i < ln; i++ { |
||||
offset := delta*nQueries - 1 - i |
||||
require.Equal(t, time.Unix(int64(offset), 0), stream.Entries[i].Timestamp, "correct timestamp") |
||||
} |
||||
default: |
||||
for i := 0; i < ln; i++ { |
||||
offset := i |
||||
require.Equal(t, time.Unix(int64(offset), 0), stream.Entries[i].Timestamp, "correct timestamp") |
||||
} |
||||
} |
||||
} |
||||
}) |
||||
} |
||||
} |
||||
|
||||
func BenchmarkAccumulator(b *testing.B) { |
||||
|
||||
// dummy params. Only need to populate direction & limit
|
||||
lim := 30 |
||||
params, err := NewLiteralParams( |
||||
`{app="foo"}`, time.Time{}, time.Time{}, 0, 0, logproto.BACKWARD, uint32(lim), nil, |
||||
) |
||||
require.NoError(b, err) |
||||
|
||||
for acc, tc := range map[string]struct { |
||||
results []logqlmodel.Result |
||||
newAcc func(Params, []logqlmodel.Result) Accumulator |
||||
params Params |
||||
}{ |
||||
"streams": { |
||||
newStreamResults(), |
||||
func(p Params, _ []logqlmodel.Result) Accumulator { |
||||
return NewStreamAccumulator(p) |
||||
}, |
||||
params, |
||||
}, |
||||
"quantile sketches": { |
||||
newQuantileSketchResults(), |
||||
func(p Params, _ []logqlmodel.Result) Accumulator { |
||||
return newQuantileSketchAccumulator() |
||||
}, |
||||
params, |
||||
}, |
||||
} { |
||||
b.Run(acc, func(b *testing.B) { |
||||
b.ResetTimer() |
||||
b.ReportAllocs() |
||||
for n := 0; n < b.N; n++ { |
||||
|
||||
acc := tc.newAcc(params, tc.results) |
||||
for i, r := range tc.results { |
||||
err := acc.Accumulate(context.Background(), r, i) |
||||
require.Nil(b, err) |
||||
} |
||||
|
||||
acc.Result() |
||||
} |
||||
}) |
||||
} |
||||
} |
||||
|
||||
func newStreamResults() []logqlmodel.Result { |
||||
nQueries := 50 |
||||
delta := 100 // 10 entries per stream, 1s apart
|
||||
streamsPerQuery := 50 |
||||
|
||||
results := make([]logqlmodel.Result, nQueries) |
||||
for i := 0; i < nQueries; i++ { |
||||
start := i * delta |
||||
end := start + delta |
||||
streams := newStreams(time.Unix(int64(start), 0), time.Unix(int64(end), 0), time.Second, streamsPerQuery, logproto.BACKWARD) |
||||
var res logqlmodel.Streams |
||||
for i := range streams { |
||||
res = append(res, *streams[i]) |
||||
} |
||||
results[i] = logqlmodel.Result{Data: res} |
||||
|
||||
} |
||||
|
||||
return results |
||||
} |
||||
|
||||
func newQuantileSketchResults() []logqlmodel.Result { |
||||
results := make([]logqlmodel.Result, 100) |
||||
|
||||
for r := range results { |
||||
vectors := make([]ProbabilisticQuantileVector, 10) |
||||
for i := range vectors { |
||||
vectors[i] = make(ProbabilisticQuantileVector, 10) |
||||
for j := range vectors[i] { |
||||
vectors[i][j] = ProbabilisticQuantileSample{ |
||||
T: int64(i), |
||||
F: newRandomSketch(), |
||||
Metric: []labels.Label{{Name: "foo", Value: fmt.Sprintf("bar-%d", j)}}, |
||||
} |
||||
} |
||||
} |
||||
results[r] = logqlmodel.Result{Data: ProbabilisticQuantileMatrix(vectors)} |
||||
} |
||||
|
||||
return results |
||||
} |
||||
|
||||
func newStreamWithDirection(start, end time.Time, delta time.Duration, ls string, direction logproto.Direction) *logproto.Stream { |
||||
s := &logproto.Stream{ |
||||
Labels: ls, |
||||
} |
||||
for t := start; t.Before(end); t = t.Add(delta) { |
||||
s.Entries = append(s.Entries, logproto.Entry{ |
||||
Timestamp: t, |
||||
Line: fmt.Sprintf("%d", t.Unix()), |
||||
}) |
||||
} |
||||
if direction == logproto.BACKWARD { |
||||
// simulate data coming in reverse order (logproto.BACKWARD)
|
||||
for i, j := 0, len(s.Entries)-1; i < j; i, j = i+1, j-1 { |
||||
s.Entries[i], s.Entries[j] = s.Entries[j], s.Entries[i] |
||||
} |
||||
} |
||||
return s |
||||
} |
||||
|
||||
func newStreams(start, end time.Time, delta time.Duration, n int, direction logproto.Direction) (res []*logproto.Stream) { |
||||
for i := 0; i < n; i++ { |
||||
res = append(res, newStreamWithDirection(start, end, delta, fmt.Sprintf(`{n="%d"}`, i), direction)) |
||||
} |
||||
return res |
||||
} |
||||
|
||||
func newRandomSketch() sketch.QuantileSketch { |
||||
r := rand.New(rand.NewSource(42)) |
||||
s := sketch.NewDDSketch() |
||||
for i := 0; i < 1000; i++ { |
||||
_ = s.Add(r.Float64()) |
||||
} |
||||
return s |
||||
} |
Loading…
Reference in new issue