From 483ee5613901c62d55f7b43eab07cea8c6ab6043 Mon Sep 17 00:00:00 2001 From: Karsten Jeschkies Date: Thu, 8 Feb 2024 16:22:54 +0100 Subject: [PATCH 01/22] 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](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) --------- Co-authored-by: Christian Haudum --- pkg/logql/accumulator.go | 379 ++++++++++++++++++ pkg/logql/accumulator_test.go | 273 +++++++++++++ pkg/logql/downstream.go | 27 +- pkg/logql/test_utils.go | 2 +- pkg/querier/queryrange/downstreamer.go | 419 +------------------- pkg/querier/queryrange/downstreamer_test.go | 262 +----------- pkg/querier/queryrange/shard_resolver.go | 12 +- 7 files changed, 687 insertions(+), 687 deletions(-) create mode 100644 pkg/logql/accumulator.go create mode 100644 pkg/logql/accumulator_test.go diff --git a/pkg/logql/accumulator.go b/pkg/logql/accumulator.go new file mode 100644 index 0000000000000..9e9784cb037ef --- /dev/null +++ b/pkg/logql/accumulator.go @@ -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 +// 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 +} diff --git a/pkg/logql/accumulator_test.go b/pkg/logql/accumulator_test.go new file mode 100644 index 0000000000000..d827e3ea02e71 --- /dev/null +++ b/pkg/logql/accumulator_test.go @@ -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 +} diff --git a/pkg/logql/downstream.go b/pkg/logql/downstream.go index 76594dc040c22..e29b47054fea6 100644 --- a/pkg/logql/downstream.go +++ b/pkg/logql/downstream.go @@ -244,7 +244,13 @@ type Resp struct { // Downstreamer is an interface for deferring responsibility for query execution. // It is decoupled from but consumed by a downStreamEvaluator to dispatch ASTs. type Downstreamer interface { - Downstream(context.Context, []DownstreamQuery) ([]logqlmodel.Result, error) + Downstream(context.Context, []DownstreamQuery, Accumulator) ([]logqlmodel.Result, error) +} + +// Accumulator is an interface for accumulating query results. +type Accumulator interface { + Accumulate(context.Context, logqlmodel.Result, int) error + Result() []logqlmodel.Result } // DownstreamEvaluator is an evaluator which handles shard aware AST nodes @@ -254,8 +260,8 @@ type DownstreamEvaluator struct { } // Downstream runs queries and collects stats from the embedded Downstreamer -func (ev DownstreamEvaluator) Downstream(ctx context.Context, queries []DownstreamQuery) ([]logqlmodel.Result, error) { - results, err := ev.Downstreamer.Downstream(ctx, queries) +func (ev DownstreamEvaluator) Downstream(ctx context.Context, queries []DownstreamQuery, acc Accumulator) ([]logqlmodel.Result, error) { + results, err := ev.Downstreamer.Downstream(ctx, queries, acc) if err != nil { return nil, err } @@ -314,12 +320,13 @@ func (ev *DownstreamEvaluator) NewStepEvaluator( if e.shard != nil { shards = append(shards, *e.shard) } + acc := NewBufferedAccumulator(1) results, err := ev.Downstream(ctx, []DownstreamQuery{{ Params: ParamsWithShardsOverride{ Params: ParamsWithExpressionOverride{Params: params, ExpressionOverride: e.SampleExpr}, ShardsOverride: Shards(shards).Encode(), }, - }}) + }}, acc) if err != nil { return nil, err } @@ -339,7 +346,8 @@ func (ev *DownstreamEvaluator) NewStepEvaluator( cur = cur.next } - results, err := ev.Downstream(ctx, queries) + acc := NewBufferedAccumulator(len(queries)) + results, err := ev.Downstream(ctx, queries, acc) if err != nil { return nil, err } @@ -379,7 +387,8 @@ func (ev *DownstreamEvaluator) NewStepEvaluator( } } - results, err := ev.Downstream(ctx, queries) + acc := newQuantileSketchAccumulator() + results, err := ev.Downstream(ctx, queries, acc) if err != nil { return nil, err } @@ -413,12 +422,13 @@ func (ev *DownstreamEvaluator) NewIterator( if e.shard != nil { shards = append(shards, *e.shard) } + acc := NewStreamAccumulator(params) results, err := ev.Downstream(ctx, []DownstreamQuery{{ Params: ParamsWithShardsOverride{ Params: ParamsWithExpressionOverride{Params: params, ExpressionOverride: e.LogSelectorExpr}, ShardsOverride: shards.Encode(), }, - }}) + }}, acc) if err != nil { return nil, err } @@ -438,7 +448,8 @@ func (ev *DownstreamEvaluator) NewIterator( cur = cur.next } - results, err := ev.Downstream(ctx, queries) + acc := NewStreamAccumulator(params) + results, err := ev.Downstream(ctx, queries, acc) if err != nil { return nil, err } diff --git a/pkg/logql/test_utils.go b/pkg/logql/test_utils.go index 82442e09bf60d..72b8429e11bf9 100644 --- a/pkg/logql/test_utils.go +++ b/pkg/logql/test_utils.go @@ -215,7 +215,7 @@ type MockDownstreamer struct { func (m MockDownstreamer) Downstreamer(_ context.Context) Downstreamer { return m } -func (m MockDownstreamer) Downstream(ctx context.Context, queries []DownstreamQuery) ([]logqlmodel.Result, error) { +func (m MockDownstreamer) Downstream(ctx context.Context, queries []DownstreamQuery, _ Accumulator) ([]logqlmodel.Result, error) { results := make([]logqlmodel.Result, 0, len(queries)) for _, query := range queries { res, err := m.Query(query.Params).Exec(ctx) diff --git a/pkg/querier/queryrange/downstreamer.go b/pkg/querier/queryrange/downstreamer.go index d8514e8a4ee75..31f8997ed767e 100644 --- a/pkg/querier/queryrange/downstreamer.go +++ b/pkg/querier/queryrange/downstreamer.go @@ -1,12 +1,9 @@ package queryrange import ( - "container/heap" "context" "fmt" "reflect" - "sort" - "time" "github.com/go-kit/log/level" "github.com/grafana/dskit/concurrency" @@ -16,14 +13,10 @@ import ( "github.com/prometheus/prometheus/promql" "github.com/prometheus/prometheus/promql/parser" - "github.com/grafana/loki/pkg/logproto" "github.com/grafana/loki/pkg/logql" "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/plan" "github.com/grafana/loki/pkg/querier/queryrange/queryrangebase" - "github.com/grafana/loki/pkg/querier/queryrange/queryrangebase/definitions" "github.com/grafana/loki/pkg/util/spanlogger" ) @@ -103,8 +96,8 @@ type instance struct { handler queryrangebase.Handler } -func (in instance) Downstream(ctx context.Context, queries []logql.DownstreamQuery) ([]logqlmodel.Result, error) { - return in.For(ctx, queries, func(qry logql.DownstreamQuery) (logqlmodel.Result, error) { +func (in instance) Downstream(ctx context.Context, queries []logql.DownstreamQuery, acc logql.Accumulator) ([]logqlmodel.Result, error) { + return in.For(ctx, queries, acc, func(qry logql.DownstreamQuery) (logqlmodel.Result, error) { req := ParamsToLokiRequest(qry.Params).WithQuery(qry.Params.GetExpression().String()) sp, ctx := opentracing.StartSpanFromContext(ctx, "DownstreamHandler.instance") defer sp.Finish() @@ -124,6 +117,7 @@ func (in instance) Downstream(ctx context.Context, queries []logql.DownstreamQue func (in instance) For( ctx context.Context, queries []logql.DownstreamQuery, + acc logql.Accumulator, fn func(logql.DownstreamQuery) (logqlmodel.Result, error), ) ([]logqlmodel.Result, error) { ctx, cancel := context.WithCancel(ctx) @@ -159,12 +153,11 @@ func (in instance) For( close(ch) }() - acc := newDownstreamAccumulator(queries[0].Params, len(queries)) for resp := range ch { if resp.Err != nil { return nil, resp.Err } - if err := acc.Accumulate(ctx, resp.I, resp.Res); err != nil { + if err := acc.Accumulate(ctx, resp.Res, resp.I); err != nil { return nil, err } } @@ -210,407 +203,3 @@ func sampleStreamToVector(streams []queryrangebase.SampleStream) parser.Value { } return xs } - -// downstreamAccumulator is one of three variants: -// a logsAccumulator, a bufferedAccumulator, or a quantileSketchAccumulator. -// Which variant is detected on the first call to Accumulate. -// Metric queries, which are generally small payloads, are buffered -// since the memory overhead is negligible. -// Log queries, sharded thousands of times and each returning -// 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. -// Sharded probabilistic quantile query results are merged as they come in. -type downstreamAccumulator struct { - acc resultAccumulator - params logql.Params - n int // number of queries, used to build slice size -} - -type resultAccumulator interface { - Accumulate(logqlmodel.Result, int) error - Result() []logqlmodel.Result -} - -func newDownstreamAccumulator(params logql.Params, nQueries int) *downstreamAccumulator { - return &downstreamAccumulator{params: params, n: nQueries} -} - -func (a *downstreamAccumulator) build(acc logqlmodel.Result) { - switch acc.Data.Type() { - case logqlmodel.ValueTypeStreams: - - // the stream accumulator stores a heap with reversed order - // from the results we expect, so we need to reverse the direction - direction := logproto.FORWARD - if a.params.Direction() == logproto.FORWARD { - direction = logproto.BACKWARD - } - - a.acc = newStreamAccumulator(direction, int(a.params.Limit())) - case logql.QuantileSketchMatrixType: - a.acc = newQuantileSketchAccumulator() - default: - a.acc = &bufferedAccumulator{ - results: make([]logqlmodel.Result, a.n), - } - - } -} - -func (a *downstreamAccumulator) Accumulate(_ context.Context, index int, acc logqlmodel.Result) error { - // on first pass, determine which accumulator to use - if a.acc == nil { - a.build(acc) - } - - return a.acc.Accumulate(acc, index) -} - -func (a *downstreamAccumulator) Result() []logqlmodel.Result { - if a.acc == nil { - return nil - } - return a.acc.Result() - -} - -type bufferedAccumulator struct { - results []logqlmodel.Result -} - -func (a *bufferedAccumulator) Accumulate(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 logql.ProbabilisticQuantileMatrix -} - -func newQuantileSketchAccumulator() *quantileSketchAccumulator { - return &quantileSketchAccumulator{} -} - -func (a *quantileSketchAccumulator) Accumulate(res logqlmodel.Result, _ int) error { - if res.Data.Type() != logql.QuantileSketchMatrixType { - return fmt.Errorf("unexpected matrix data type: got (%s), want (%s)", res.Data.Type(), logql.QuantileSketchMatrixType) - } - data, ok := res.Data.(logql.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 -} - -func newStreamAccumulator(order logproto.Direction, limit int) *accumulatedStreams { - return &accumulatedStreams{ - labelmap: make(map[string]int), - order: order, - limit: 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 := 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(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 -} diff --git a/pkg/querier/queryrange/downstreamer_test.go b/pkg/querier/queryrange/downstreamer_test.go index 007166c30c305..a23f2a381b007 100644 --- a/pkg/querier/queryrange/downstreamer_test.go +++ b/pkg/querier/queryrange/downstreamer_test.go @@ -3,8 +3,6 @@ package queryrange import ( "context" "errors" - "fmt" - "math/rand" "strconv" "strings" "sync" @@ -19,7 +17,6 @@ import ( "github.com/grafana/loki/pkg/logproto" "github.com/grafana/loki/pkg/logql" - "github.com/grafana/loki/pkg/logql/sketch" "github.com/grafana/loki/pkg/logql/syntax" "github.com/grafana/loki/pkg/logqlmodel" "github.com/grafana/loki/pkg/logqlmodel/stats" @@ -250,8 +247,10 @@ func TestInstanceFor(t *testing.T) { var mtx sync.Mutex var ct int + acc := logql.NewBufferedAccumulator(len(queries)) + // ensure we can execute queries that number more than the parallelism parameter - _, err := in.For(context.TODO(), queries, func(_ logql.DownstreamQuery) (logqlmodel.Result, error) { + _, err := in.For(context.TODO(), queries, acc, func(_ logql.DownstreamQuery) (logqlmodel.Result, error) { mtx.Lock() defer mtx.Unlock() ct++ @@ -266,7 +265,7 @@ func TestInstanceFor(t *testing.T) { // ensure an early error abandons the other queues queries in = mkIn() ct = 0 - _, err = in.For(context.TODO(), queries, func(_ logql.DownstreamQuery) (logqlmodel.Result, error) { + _, err = in.For(context.TODO(), queries, acc, func(_ logql.DownstreamQuery) (logqlmodel.Result, error) { mtx.Lock() defer mtx.Unlock() ct++ @@ -302,6 +301,7 @@ func TestInstanceFor(t *testing.T) { }, }, }, + logql.NewBufferedAccumulator(2), func(qry logql.DownstreamQuery) (logqlmodel.Result, error) { // Decode shard s := strings.Split(qry.Params.Shards()[0], "_") @@ -383,7 +383,7 @@ func TestInstanceDownstream(t *testing.T) { results, err := DownstreamHandler{ limits: fakeLimits{}, next: handler, - }.Downstreamer(context.Background()).Downstream(context.Background(), queries) + }.Downstreamer(context.Background()).Downstream(context.Background(), queries, logql.NewBufferedAccumulator(len(queries))) require.Equal(t, want, got) @@ -402,6 +402,7 @@ func TestCancelWhileWaitingResponse(t *testing.T) { in := mkIn() queries := make([]logql.DownstreamQuery, in.parallelism+1) + acc := logql.NewBufferedAccumulator(len(queries)) ctx, cancel := context.WithCancel(context.Background()) @@ -409,7 +410,7 @@ func TestCancelWhileWaitingResponse(t *testing.T) { // to prove it will exit when the context is canceled. b := atomic.NewBool(false) go func() { - _, _ = in.For(ctx, queries, func(_ logql.DownstreamQuery) (logqlmodel.Result, error) { + _, _ = in.For(ctx, queries, acc, func(_ logql.DownstreamQuery) (logqlmodel.Result, error) { // Intended to keep the For method from returning unless the context is canceled. time.Sleep(100 * time.Second) return logqlmodel.Result{}, nil @@ -443,250 +444,3 @@ func TestDownstreamerUsesCorrectParallelism(t *testing.T) { } require.Equal(t, l.maxQueryParallelism, ct) } - -func newStream(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, newStream(start, end, delta, fmt.Sprintf(`{n="%d"}`, i), direction)) - } - return res -} - -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(logproto.FORWARD, 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 := logql.NewLiteralParams( - `{app="foo"}`, time.Time{}, time.Time{}, 0, 0, direction, uint32(lim), nil, - ) - require.NoError(t, err) - - acc := newDownstreamAccumulator(params, 1) - result := logqlmodel.Result{ - Data: x, - } - - require.Nil(t, acc.Accumulate(context.Background(), 0, result)) - - 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 := logql.NewLiteralParams( - `{app="foo"}`, time.Time{}, time.Time{}, 0, 0, direction, uint32(lim), nil, - ) - require.NoError(t, err) - - acc := newDownstreamAccumulator(params, 1) - for i := 0; i < nQueries; i++ { - err := acc.Accumulate(context.Background(), i, logqlmodel.Result{ - Data: payloads[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 := logql.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 - params logql.Params - }{ - "streams": { - newStreamResults(), - params, - }, - "quantile sketches": { - newQuantileSketchResults(), - params, - }, - } { - b.Run(acc, func(b *testing.B) { - b.ResetTimer() - b.ReportAllocs() - for n := 0; n < b.N; n++ { - - acc := newDownstreamAccumulator(params, len(tc.results)) - for i, r := range tc.results { - err := acc.Accumulate(context.Background(), i, r) - 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([]logql.ProbabilisticQuantileVector, 10) - for i := range vectors { - vectors[i] = make(logql.ProbabilisticQuantileVector, 10) - for j := range vectors[i] { - vectors[i][j] = logql.ProbabilisticQuantileSample{ - T: int64(i), - F: newRandomSketch(), - Metric: []labels.Label{{Name: "foo", Value: fmt.Sprintf("bar-%d", j)}}, - } - } - } - results[r] = logqlmodel.Result{Data: logql.ProbabilisticQuantileMatrix(vectors)} - } - - return results -} - -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 -} diff --git a/pkg/querier/queryrange/shard_resolver.go b/pkg/querier/queryrange/shard_resolver.go index aed0e96e2b47d..652637a724655 100644 --- a/pkg/querier/queryrange/shard_resolver.go +++ b/pkg/querier/queryrange/shard_resolver.go @@ -3,7 +3,7 @@ package queryrange import ( "context" "fmt" - math "math" + "math" strings "strings" "time" @@ -21,6 +21,7 @@ import ( "github.com/grafana/loki/pkg/querier/queryrange/queryrangebase" "github.com/grafana/loki/pkg/storage/config" "github.com/grafana/loki/pkg/storage/stores/index/stats" + utilMath "github.com/grafana/loki/pkg/util/math" "github.com/grafana/loki/pkg/util/spanlogger" "github.com/grafana/loki/pkg/util/validation" valid "github.com/grafana/loki/pkg/validation" @@ -231,7 +232,7 @@ func guessShardFactor(stats stats.Stats, maxBytesPerShard, maxShards int) int { // reset this edge case manually factor := int(math.Pow(2, power)) if maxShards > 0 { - factor = min(factor, maxShards) + factor = utilMath.Min(factor, maxShards) } // shortcut: no need to run any sharding logic when factor=1 @@ -241,10 +242,3 @@ func guessShardFactor(stats stats.Stats, maxBytesPerShard, maxShards int) int { } return factor } - -func min(a, b int) int { - if a < b { - return a - } - return b -} From 9128036145967cdd75440d956bb88b9506752ee4 Mon Sep 17 00:00:00 2001 From: Christian Haudum Date: Thu, 8 Feb 2024 16:24:35 +0100 Subject: [PATCH 02/22] Bloom gateway: Integrate processor into worker (#11895) This PR replaces the iteration of the bloom gateway worker with the code encapsulated in the previously established processor. Since the processor only relies on the BloomStore, rather than on the BloomShipper, the PR also removes unused code from the shipper. Signed-off-by: Christian Haudum --- pkg/bloomgateway/bloomgateway.go | 27 ++- pkg/bloomgateway/bloomgateway_test.go | 27 +-- pkg/bloomgateway/processor.go | 7 + pkg/bloomgateway/processor_test.go | 82 ++++++++- pkg/bloomgateway/util_test.go | 54 ------ pkg/bloomgateway/worker.go | 160 ++---------------- .../stores/shipper/bloomshipper/shipper.go | 62 +------ .../shipper/bloomshipper/shipper_test.go | 34 +++- 8 files changed, 160 insertions(+), 293 deletions(-) diff --git a/pkg/bloomgateway/bloomgateway.go b/pkg/bloomgateway/bloomgateway.go index 1e7a54f1d1e33..f70f879ea8797 100644 --- a/pkg/bloomgateway/bloomgateway.go +++ b/pkg/bloomgateway/bloomgateway.go @@ -23,13 +23,15 @@ of line filter expressions. | bloomgateway.Gateway | - queue.RequestQueue + queue.RequestQueue | - bloomgateway.Worker + bloomgateway.Worker | - bloomshipper.Shipper + bloomgateway.Processor | - bloomshipper.BloomFileClient + bloomshipper.Store + | + bloomshipper.Client | ObjectClient | @@ -170,9 +172,9 @@ type Gateway struct { workerMetrics *workerMetrics queueMetrics *queue.Metrics - queue *queue.RequestQueue - activeUsers *util.ActiveUsersCleanupService - bloomShipper bloomshipper.Interface + queue *queue.RequestQueue + activeUsers *util.ActiveUsersCleanupService + bloomStore bloomshipper.Store sharding ShardingStrategy @@ -218,13 +220,8 @@ func New(cfg Config, schemaCfg config.SchemaConfig, storageCfg storage.Config, o return nil, err } - bloomShipper, err := bloomshipper.NewShipper(store, storageCfg.BloomShipperConfig, overrides, logger, reg) - if err != nil { - return nil, err - } - // We need to keep a reference to be able to call Stop() on shutdown of the gateway. - g.bloomShipper = bloomShipper + g.bloomStore = store if err := g.initServices(); err != nil { return nil, err @@ -239,7 +236,7 @@ func (g *Gateway) initServices() error { svcs := []services.Service{g.queue, g.activeUsers} for i := 0; i < g.cfg.WorkerConcurrency; i++ { id := fmt.Sprintf("bloom-query-worker-%d", i) - w := newWorker(id, g.workerConfig, g.queue, g.bloomShipper, g.pendingTasks, g.logger, g.workerMetrics) + w := newWorker(id, g.workerConfig, g.queue, g.bloomStore, g.pendingTasks, g.logger, g.workerMetrics) svcs = append(svcs, w) } g.serviceMngr, err = services.NewManager(svcs...) @@ -291,7 +288,7 @@ func (g *Gateway) running(ctx context.Context) error { } func (g *Gateway) stopping(_ error) error { - g.bloomShipper.Stop() + g.bloomStore.Stop() return services.StopManagerAndAwaitStopped(context.Background(), g.serviceMngr) } diff --git a/pkg/bloomgateway/bloomgateway_test.go b/pkg/bloomgateway/bloomgateway_test.go index c8da44a7c719b..329c7b38a33b5 100644 --- a/pkg/bloomgateway/bloomgateway_test.go +++ b/pkg/bloomgateway/bloomgateway_test.go @@ -170,10 +170,12 @@ func TestBloomGateway_FilterChunkRefs(t *testing.T) { now := mktime("2023-10-03 10:00") - bqs, data := createBlockQueriers(t, 10, now.Add(-24*time.Hour), now, 0, 1000) - mockStore := newMockBloomStore(bqs) - mockStore.err = errors.New("failed to fetch block") - gw.bloomShipper = mockStore + // replace store implementation and re-initialize workers and sub-services + _, metas, queriers, data := createBlocks(t, tenantID, 10, now.Add(-1*time.Hour), now, 0x0000, 0x0fff) + + mockStore := newMockBloomStore(queriers, metas) + mockStore.err = errors.New("request failed") + gw.bloomStore = mockStore err = gw.initServices() require.NoError(t, err) @@ -204,7 +206,7 @@ func TestBloomGateway_FilterChunkRefs(t *testing.T) { t.Cleanup(cancelFn) res, err := gw.FilterChunkRefs(ctx, req) - require.ErrorContainsf(t, err, "request failed: failed to fetch block", "%+v", res) + require.ErrorContainsf(t, err, "request failed", "%+v", res) } }) @@ -215,10 +217,12 @@ func TestBloomGateway_FilterChunkRefs(t *testing.T) { now := mktime("2024-01-25 10:00") - bqs, data := createBlockQueriers(t, 50, now.Add(-24*time.Hour), now, 0, 1024) - mockStore := newMockBloomStore(bqs) - mockStore.delay = 50 * time.Millisecond // delay for each block - 50x50=2500ms - gw.bloomShipper = mockStore + // replace store implementation and re-initialize workers and sub-services + _, metas, queriers, data := createBlocks(t, tenantID, 10, now.Add(-1*time.Hour), now, 0x0000, 0x0fff) + + mockStore := newMockBloomStore(queriers, metas) + mockStore.delay = 2000 * time.Millisecond + gw.bloomStore = mockStore err = gw.initServices() require.NoError(t, err) @@ -346,8 +350,9 @@ func TestBloomGateway_FilterChunkRefs(t *testing.T) { now := mktime("2023-10-03 10:00") // replace store implementation and re-initialize workers and sub-services - bqs, data := createBlockQueriers(t, 5, now.Add(-8*time.Hour), now, 0, 1024) - gw.bloomShipper = newMockBloomStore(bqs) + _, metas, queriers, data := createBlocks(t, tenantID, 10, now.Add(-1*time.Hour), now, 0x0000, 0x0fff) + + gw.bloomStore = newMockBloomStore(queriers, metas) err = gw.initServices() require.NoError(t, err) diff --git a/pkg/bloomgateway/processor.go b/pkg/bloomgateway/processor.go index 26895bc43eda5..4b5edf9e18762 100644 --- a/pkg/bloomgateway/processor.go +++ b/pkg/bloomgateway/processor.go @@ -17,6 +17,13 @@ type tasksForBlock struct { tasks []Task } +func newProcessor(store bloomshipper.Store, logger log.Logger) *processor { + return &processor{ + store: store, + logger: logger, + } +} + type processor struct { store bloomshipper.Store logger log.Logger diff --git a/pkg/bloomgateway/processor_test.go b/pkg/bloomgateway/processor_test.go index d39ba61a89613..b619ec56f7280 100644 --- a/pkg/bloomgateway/processor_test.go +++ b/pkg/bloomgateway/processor_test.go @@ -7,6 +7,8 @@ import ( "testing" "time" + "github.com/go-kit/log" + "github.com/pkg/errors" "github.com/prometheus/common/model" "github.com/stretchr/testify/require" "go.uber.org/atomic" @@ -17,13 +19,26 @@ import ( var _ bloomshipper.Store = &dummyStore{} +func newMockBloomStore(bqs []*bloomshipper.CloseableBlockQuerier, metas []bloomshipper.Meta) *dummyStore { + return &dummyStore{ + querieres: bqs, + metas: metas, + } +} + type dummyStore struct { metas []bloomshipper.Meta - blocks []bloomshipper.BlockRef querieres []*bloomshipper.CloseableBlockQuerier + + // mock how long it takes to serve block queriers + delay time.Duration + // mock response error when serving block queriers in ForEach + err error } func (s *dummyStore) ResolveMetas(_ context.Context, _ bloomshipper.MetaSearchParams) ([][]bloomshipper.MetaRef, []*bloomshipper.Fetcher, error) { + time.Sleep(s.delay) + //TODO(chaudum) Filter metas based on search params refs := make([]bloomshipper.MetaRef, 0, len(s.metas)) for _, meta := range s.metas { @@ -51,6 +66,11 @@ func (s *dummyStore) Stop() { func (s *dummyStore) FetchBlocks(_ context.Context, refs []bloomshipper.BlockRef) ([]*bloomshipper.CloseableBlockQuerier, error) { result := make([]*bloomshipper.CloseableBlockQuerier, 0, len(s.querieres)) + if s.err != nil { + time.Sleep(s.delay) + return result, s.err + } + for _, ref := range refs { for _, bq := range s.querieres { if ref.Bounds.Equal(bq.Bounds) { @@ -63,6 +83,8 @@ func (s *dummyStore) FetchBlocks(_ context.Context, refs []bloomshipper.BlockRef result[i], result[j] = result[j], result[i] }) + time.Sleep(s.delay) + return result, nil } @@ -71,14 +93,11 @@ func TestProcessor(t *testing.T) { tenant := "fake" now := mktime("2024-01-27 12:00") - t.Run("dummy", func(t *testing.T) { - blocks, metas, queriers, data := createBlocks(t, tenant, 10, now.Add(-1*time.Hour), now, 0x0000, 0x1000) + t.Run("success case", func(t *testing.T) { + _, metas, queriers, data := createBlocks(t, tenant, 10, now.Add(-1*time.Hour), now, 0x0000, 0x0fff) p := &processor{ - store: &dummyStore{ - querieres: queriers, - metas: metas, - blocks: blocks, - }, + store: newMockBloomStore(queriers, metas), + logger: log.NewNopLogger(), } chunkRefs := createQueryInputFromBlockData(t, tenant, data, 10) @@ -116,4 +135,51 @@ func TestProcessor(t *testing.T) { require.NoError(t, err) require.Equal(t, int64(len(swb.series)), results.Load()) }) + + t.Run("failure case", func(t *testing.T) { + _, metas, queriers, data := createBlocks(t, tenant, 10, now.Add(-1*time.Hour), now, 0x0000, 0x0fff) + + mockStore := newMockBloomStore(queriers, metas) + mockStore.err = errors.New("store failed") + + p := &processor{ + store: mockStore, + logger: log.NewNopLogger(), + } + + chunkRefs := createQueryInputFromBlockData(t, tenant, data, 10) + swb := seriesWithBounds{ + series: groupRefs(t, chunkRefs), + bounds: model.Interval{ + Start: now.Add(-1 * time.Hour), + End: now, + }, + day: truncateDay(now), + } + filters := []syntax.LineFilter{ + {Ty: 0, Match: "no match"}, + } + + t.Log("series", len(swb.series)) + task, _ := NewTask(ctx, "fake", swb, filters) + tasks := []Task{task} + + results := atomic.NewInt64(0) + var wg sync.WaitGroup + for i := range tasks { + wg.Add(1) + go func(ta Task) { + defer wg.Done() + for range ta.resCh { + results.Inc() + } + t.Log("done", results.Load()) + }(tasks[i]) + } + + err := p.run(ctx, tasks) + wg.Wait() + require.Errorf(t, err, "store failed") + require.Equal(t, int64(0), results.Load()) + }) } diff --git a/pkg/bloomgateway/util_test.go b/pkg/bloomgateway/util_test.go index f19564b43ef59..156b05cf4ca41 100644 --- a/pkg/bloomgateway/util_test.go +++ b/pkg/bloomgateway/util_test.go @@ -1,8 +1,6 @@ package bloomgateway import ( - "context" - "math/rand" "testing" "time" @@ -370,58 +368,6 @@ func createBlocks(t *testing.T, tenant string, n int, from, through model.Time, return blocks, metas, queriers, series } -func newMockBloomStore(bqs []*bloomshipper.CloseableBlockQuerier) *mockBloomStore { - return &mockBloomStore{bqs: bqs} -} - -type mockBloomStore struct { - bqs []*bloomshipper.CloseableBlockQuerier - // mock how long it takes to serve block queriers - delay time.Duration - // mock response error when serving block queriers in ForEach - err error -} - -var _ bloomshipper.Interface = &mockBloomStore{} - -// GetBlockRefs implements bloomshipper.Interface -func (s *mockBloomStore) GetBlockRefs(_ context.Context, _ string, _ bloomshipper.Interval) ([]bloomshipper.BlockRef, error) { - time.Sleep(s.delay) - blocks := make([]bloomshipper.BlockRef, 0, len(s.bqs)) - for i := range s.bqs { - blocks = append(blocks, s.bqs[i].BlockRef) - } - return blocks, nil -} - -// Stop implements bloomshipper.Interface -func (s *mockBloomStore) Stop() {} - -// ForEach implements bloomshipper.Interface -func (s *mockBloomStore) ForEach(_ context.Context, _ string, _ []bloomshipper.BlockRef, callback bloomshipper.ForEachBlockCallback) error { - if s.err != nil { - time.Sleep(s.delay) - return s.err - } - - shuffled := make([]*bloomshipper.CloseableBlockQuerier, len(s.bqs)) - _ = copy(shuffled, s.bqs) - - rand.Shuffle(len(shuffled), func(i, j int) { - shuffled[i], shuffled[j] = shuffled[j], shuffled[i] - }) - - for _, bq := range shuffled { - // ignore errors in the mock - time.Sleep(s.delay) - err := callback(bq.BlockQuerier, bq.Bounds) - if err != nil { - return err - } - } - return nil -} - func createQueryInputFromBlockData(t *testing.T, tenant string, data [][]v1.SeriesWithBloom, nthSeries int) []*logproto.ChunkRef { t.Helper() n := 0 diff --git a/pkg/bloomgateway/worker.go b/pkg/bloomgateway/worker.go index 0f7db8a9ca586..3063d8e7d4f26 100644 --- a/pkg/bloomgateway/worker.go +++ b/pkg/bloomgateway/worker.go @@ -10,11 +10,8 @@ import ( "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" - "github.com/prometheus/common/model" - "golang.org/x/exp/slices" "github.com/grafana/loki/pkg/queue" - v1 "github.com/grafana/loki/pkg/storage/bloom/v1" "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper" ) @@ -23,11 +20,9 @@ type workerConfig struct { } type workerMetrics struct { - dequeuedTasks *prometheus.CounterVec - dequeueErrors *prometheus.CounterVec - dequeueWaitTime *prometheus.SummaryVec - storeAccessLatency *prometheus.HistogramVec - bloomQueryLatency *prometheus.HistogramVec + dequeuedTasks *prometheus.CounterVec + dequeueErrors *prometheus.CounterVec + dequeueWaitTime *prometheus.SummaryVec } func newWorkerMetrics(registerer prometheus.Registerer, namespace, subsystem string) *workerMetrics { @@ -51,19 +46,6 @@ func newWorkerMetrics(registerer prometheus.Registerer, namespace, subsystem str Name: "dequeue_wait_time", Help: "Time spent waiting for dequeuing tasks from queue", }, labels), - bloomQueryLatency: promauto.With(registerer).NewHistogramVec(prometheus.HistogramOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "bloom_query_latency", - Help: "Latency in seconds of processing bloom blocks", - }, append(labels, "status")), - // TODO(chaudum): Move this metric into the bloomshipper - storeAccessLatency: promauto.With(registerer).NewHistogramVec(prometheus.HistogramOpts{ - Namespace: namespace, - Subsystem: subsystem, - Name: "store_latency", - Help: "Latency in seconds of accessing the bloom store component", - }, append(labels, "operation")), } } @@ -78,18 +60,18 @@ type worker struct { id string cfg workerConfig queue *queue.RequestQueue - shipper bloomshipper.Interface + store bloomshipper.Store pending *pendingTasks logger log.Logger metrics *workerMetrics } -func newWorker(id string, cfg workerConfig, queue *queue.RequestQueue, shipper bloomshipper.Interface, pending *pendingTasks, logger log.Logger, metrics *workerMetrics) *worker { +func newWorker(id string, cfg workerConfig, queue *queue.RequestQueue, store bloomshipper.Store, pending *pendingTasks, logger log.Logger, metrics *workerMetrics) *worker { w := &worker{ id: id, cfg: cfg, queue: queue, - shipper: shipper, + store: store, pending: pending, logger: log.With(logger, "worker", id), metrics: metrics, @@ -107,6 +89,8 @@ func (w *worker) starting(_ context.Context) error { func (w *worker) running(_ context.Context) error { idx := queue.StartIndexWithLocalQueue + p := processor{store: w.store, logger: w.logger} + for st := w.State(); st == services.Running || st == services.Stopping; { taskCtx := context.Background() dequeueStart := time.Now() @@ -128,8 +112,7 @@ func (w *worker) running(_ context.Context) error { } w.metrics.dequeuedTasks.WithLabelValues(w.id).Add(float64(len(items))) - tasksPerDay := make(map[model.Time][]Task) - + tasks := make([]Task, 0, len(items)) for _, item := range items { task, ok := item.(Task) if !ok { @@ -139,91 +122,12 @@ func (w *worker) running(_ context.Context) error { } level.Debug(w.logger).Log("msg", "dequeued task", "task", task.ID) w.pending.Delete(task.ID) - - tasksPerDay[task.day] = append(tasksPerDay[task.day], task) + tasks = append(tasks, task) } - for day, tasks := range tasksPerDay { - - // Remove tasks that are already cancelled - tasks = slices.DeleteFunc(tasks, func(t Task) bool { - if res := t.ctx.Err(); res != nil { - t.CloseWithError(res) - return true - } - return false - }) - // no tasks to process - // continue with tasks of next day - if len(tasks) == 0 { - continue - } - - // interval is [Start, End) - interval := bloomshipper.NewInterval(day, day.Add(Day)) - logger := log.With(w.logger, "day", day.Time(), "tenant", tasks[0].Tenant) - level.Debug(logger).Log("msg", "process tasks", "tasks", len(tasks)) - - storeFetchStart := time.Now() - blockRefs, err := w.shipper.GetBlockRefs(taskCtx, tasks[0].Tenant, interval) - w.metrics.storeAccessLatency.WithLabelValues(w.id, "GetBlockRefs").Observe(time.Since(storeFetchStart).Seconds()) - if err != nil { - for _, t := range tasks { - t.CloseWithError(err) - } - // continue with tasks of next day - continue - } - if len(tasks) == 0 { - continue - } - - // No blocks found. - // Since there are no blocks for the given tasks, we need to return the - // unfiltered list of chunk refs. - if len(blockRefs) == 0 { - level.Warn(logger).Log("msg", "no blocks found") - for _, t := range tasks { - t.Close() - } - // continue with tasks of next day - continue - } - - // Remove tasks that are already cancelled - tasks = slices.DeleteFunc(tasks, func(t Task) bool { - if res := t.ctx.Err(); res != nil { - t.CloseWithError(res) - return true - } - return false - }) - // no tasks to process - // continue with tasks of next day - if len(tasks) == 0 { - continue - } - - tasksForBlocks := partitionFingerprintRange(tasks, blockRefs) - blockRefs = blockRefs[:0] - for _, b := range tasksForBlocks { - blockRefs = append(blockRefs, b.blockRef) - } - - err = w.processBlocksWithCallback(taskCtx, tasks[0].Tenant, blockRefs, tasksForBlocks) - if err != nil { - for _, t := range tasks { - t.CloseWithError(err) - } - // continue with tasks of next day - continue - } - - // all tasks for this day are done. - // close them to notify the request handler - for _, task := range tasks { - task.Close() - } + err = p.run(taskCtx, tasks) + if err != nil { + level.Error(w.logger).Log("msg", "failed to process tasks", "err", err) } // return dequeued items back to the pool @@ -238,41 +142,3 @@ func (w *worker) stopping(err error) error { w.queue.UnregisterConsumerConnection(w.id) return nil } - -func (w *worker) processBlocksWithCallback(taskCtx context.Context, tenant string, blockRefs []bloomshipper.BlockRef, boundedRefs []boundedTasks) error { - return w.shipper.ForEach(taskCtx, tenant, blockRefs, func(bq *v1.BlockQuerier, bounds v1.FingerprintBounds) error { - for _, b := range boundedRefs { - if b.blockRef.Bounds.Equal(bounds) { - return w.processBlock(bq, b.tasks) - } - } - return nil - }) -} - -func (w *worker) processBlock(blockQuerier *v1.BlockQuerier, tasks []Task) error { - schema, err := blockQuerier.Schema() - if err != nil { - return err - } - - tokenizer := v1.NewNGramTokenizer(schema.NGramLen(), 0) - iters := make([]v1.PeekingIterator[v1.Request], 0, len(tasks)) - for _, task := range tasks { - it := v1.NewPeekingIter(task.RequestIter(tokenizer)) - iters = append(iters, it) - } - fq := blockQuerier.Fuse(iters) - - start := time.Now() - err = fq.Run() - duration := time.Since(start).Seconds() - - if err != nil { - w.metrics.bloomQueryLatency.WithLabelValues(w.id, "failure").Observe(duration) - return err - } - - w.metrics.bloomQueryLatency.WithLabelValues(w.id, "success").Observe(duration) - return nil -} diff --git a/pkg/storage/stores/shipper/bloomshipper/shipper.go b/pkg/storage/stores/shipper/bloomshipper/shipper.go index 6d6322c91382b..dee8f2464bcbb 100644 --- a/pkg/storage/stores/shipper/bloomshipper/shipper.go +++ b/pkg/storage/stores/shipper/bloomshipper/shipper.go @@ -3,59 +3,34 @@ package bloomshipper import ( "context" "fmt" - "math" "sort" - "github.com/go-kit/log" - "github.com/go-kit/log/level" - "github.com/prometheus/client_golang/prometheus" - v1 "github.com/grafana/loki/pkg/storage/bloom/v1" - "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper/config" ) type ForEachBlockCallback func(bq *v1.BlockQuerier, bounds v1.FingerprintBounds) error type Interface interface { - GetBlockRefs(ctx context.Context, tenant string, interval Interval) ([]BlockRef, error) ForEach(ctx context.Context, tenant string, blocks []BlockRef, callback ForEachBlockCallback) error Stop() } type Shipper struct { - store Store - config config.Config - logger log.Logger + store Store } type Limits interface { BloomGatewayBlocksDownloadingParallelism(tenantID string) int } -func NewShipper(client Store, config config.Config, _ Limits, logger log.Logger, _ prometheus.Registerer) (*Shipper, error) { - logger = log.With(logger, "component", "bloom-shipper") - return &Shipper{ - store: client, - config: config, - logger: logger, - }, nil -} - -func (s *Shipper) GetBlockRefs(ctx context.Context, tenantID string, interval Interval) ([]BlockRef, error) { - level.Debug(s.logger).Log("msg", "GetBlockRefs", "tenant", tenantID, "[", interval.Start, "", interval.End) - - // TODO(chaudum): The bloom gateway should not fetch blocks for the complete key space - bounds := []v1.FingerprintBounds{v1.NewBounds(0, math.MaxUint64)} - blockRefs, err := s.getActiveBlockRefs(ctx, tenantID, interval, bounds) - if err != nil { - return nil, fmt.Errorf("error fetching active block references : %w", err) - } - return blockRefs, nil +func NewShipper(client Store) *Shipper { + return &Shipper{store: client} } -func (s *Shipper) ForEach(ctx context.Context, _ string, refs []BlockRef, callback ForEachBlockCallback) error { +// ForEach is a convenience function that wraps the store's FetchBlocks function +// and automatically closes the block querier once the callback was run. +func (s *Shipper) ForEach(ctx context.Context, refs []BlockRef, callback ForEachBlockCallback) error { bqs, err := s.store.FetchBlocks(ctx, refs) - if err != nil { return err } @@ -79,31 +54,6 @@ func (s *Shipper) Stop() { s.store.Stop() } -// getFirstLast returns the first and last item of a fingerprint slice -// It assumes an ascending sorted list of fingerprints. -func getFirstLast[T any](s []T) (T, T) { - var zero T - if len(s) == 0 { - return zero, zero - } - return s[0], s[len(s)-1] -} - -func (s *Shipper) getActiveBlockRefs(ctx context.Context, tenantID string, interval Interval, bounds []v1.FingerprintBounds) ([]BlockRef, error) { - minFpRange, maxFpRange := getFirstLast(bounds) - metas, err := s.store.FetchMetas(ctx, MetaSearchParams{ - TenantID: tenantID, - Keyspace: v1.NewBounds(minFpRange.Min, maxFpRange.Max), - Interval: interval, - }) - if err != nil { - return []BlockRef{}, fmt.Errorf("error fetching meta.json files: %w", err) - } - level.Debug(s.logger).Log("msg", "dowloaded metas", "count", len(metas)) - - return BlocksForMetas(metas, interval, bounds), nil -} - // BlocksForMetas returns all the blocks from all the metas listed that are within the requested bounds // and not tombstoned in any of the metas func BlocksForMetas(metas []Meta, interval Interval, keyspaces []v1.FingerprintBounds) []BlockRef { diff --git a/pkg/storage/stores/shipper/bloomshipper/shipper_test.go b/pkg/storage/stores/shipper/bloomshipper/shipper_test.go index 8ad776d4164aa..4c755e1baa6d3 100644 --- a/pkg/storage/stores/shipper/bloomshipper/shipper_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/shipper_test.go @@ -1,6 +1,7 @@ package bloomshipper import ( + "context" "fmt" "math" "testing" @@ -12,7 +13,7 @@ import ( v1 "github.com/grafana/loki/pkg/storage/bloom/v1" ) -func Test_Shipper_findBlocks(t *testing.T) { +func TestBloomShipper_findBlocks(t *testing.T) { t.Run("expected block that are specified in tombstones to be filtered out", func(t *testing.T) { metas := []Meta{ { @@ -110,7 +111,7 @@ func Test_Shipper_findBlocks(t *testing.T) { } } -func TestIsOutsideRange(t *testing.T) { +func TestBloomShipper_IsOutsideRange(t *testing.T) { startTs := model.Time(1000) endTs := model.Time(2000) @@ -181,6 +182,35 @@ func TestIsOutsideRange(t *testing.T) { }) } +func TestBloomShipper_ForEach(t *testing.T) { + blockRefs := make([]BlockRef, 0, 3) + + store, _ := newMockBloomStore(t) + for i := 0; i < len(blockRefs); i++ { + block, err := createBlockInStorage(t, store, "tenant", model.Time(i*24*int(time.Hour)), 0x0000, 0x00ff) + require.NoError(t, err) + blockRefs = append(blockRefs, block.BlockRef) + } + shipper := NewShipper(store) + + var count int + err := shipper.ForEach(context.Background(), blockRefs, func(_ *v1.BlockQuerier, _ v1.FingerprintBounds) error { + count++ + return nil + }) + require.NoError(t, err) + require.Equal(t, len(blockRefs), count) + + // check that the BlockDirectory ref counter is 0 + for i := 0; i < len(blockRefs); i++ { + s := store.stores[0] + key := s.Block(blockRefs[i]).Addr() + dir, found := s.fetcher.blocksCache.Get(context.Background(), key) + require.True(t, found) + require.Equal(t, int32(0), dir.refCount.Load()) + } +} + func createMatchingBlockRef(checksum uint32) BlockRef { block := createBlockRef(0, math.MaxUint64, model.Time(0), model.Time(math.MaxInt64)) block.Checksum = checksum From 3b122e4be173d051f2e3918a66f8b6a7e9cc84db Mon Sep 17 00:00:00 2001 From: Christian Haudum Date: Thu, 8 Feb 2024 16:27:28 +0100 Subject: [PATCH 03/22] Bloom gateway: Wire up cache config with implementation (#11898) Until now, the cache configuration in the bloom gateway did not initialise the cache. This PR wires the config with the actual implementation and passes them to the bloom store. Signed-off-by: Christian Haudum --- docs/sources/configure/_index.md | 28 ++++++------- pkg/bloomgateway/bloomgateway.go | 18 ++++++++- pkg/logqlmodel/stats/context.go | 21 +++++----- pkg/storage/chunk/cache/embeddedcache.go | 39 +++++++++++++++++-- .../stores/shipper/bloomshipper/cache.go | 5 +-- .../shipper/bloomshipper/config/config.go | 21 +++------- .../stores/shipper/bloomshipper/fetcher.go | 4 +- .../stores/shipper/bloomshipper/store.go | 6 ++- .../stores/shipper/bloomshipper/store_test.go | 10 ++--- 9 files changed, 95 insertions(+), 57 deletions(-) diff --git a/docs/sources/configure/_index.md b/docs/sources/configure/_index.md index b675f85157423..5c225920ef589 100644 --- a/docs/sources/configure/_index.md +++ b/docs/sources/configure/_index.md @@ -2327,27 +2327,26 @@ bloom_shipper: [max_tasks_enqueued_per_tenant: | default = 10000] blocks_cache: - # Whether embedded cache is enabled. - # CLI flag: -blocks-cache.enabled + # Cache for bloom blocks. Whether embedded cache is enabled. + # CLI flag: -bloom.blocks-cache.enabled [enabled: | default = false] - # Maximum memory size of the cache in MB. - # CLI flag: -blocks-cache.max-size-mb + # Cache for bloom blocks. Maximum memory size of the cache in MB. + # CLI flag: -bloom.blocks-cache.max-size-mb [max_size_mb: | default = 100] - # Maximum number of entries in the cache. - # CLI flag: -blocks-cache.max-size-items + # Cache for bloom blocks. Maximum number of entries in the cache. + # CLI flag: -bloom.blocks-cache.max-size-items [max_size_items: | default = 0] - # The time to live for items in the cache before they get purged. - # CLI flag: -blocks-cache.ttl - [ttl: | default = 0s] + # Cache for bloom blocks. The time to live for items in the cache before + # they get purged. + # CLI flag: -bloom.blocks-cache.ttl + [ttl: | default = 24h] - # During this period the process waits until the directory becomes not used - # and only after this it will be deleted. If the timeout is reached, the - # directory is force deleted. - # CLI flag: -blocks-cache.remove-directory-graceful-period - [remove_directory_graceful_period: | default = 5m] + # The cache block configures the cache backend. + # The CLI flags prefix for this block configuration is: bloom.metas-cache + [metas_cache: ] ``` ### chunk_store_config @@ -4354,6 +4353,7 @@ The TLS configuration. The cache block configures the cache backend. The supported CLI flags `` used to reference this configuration block are: - `bloom-gateway-client.cache` +- `bloom.metas-cache` - `frontend` - `frontend.index-stats-results-cache` - `frontend.label-results-cache` diff --git a/pkg/bloomgateway/bloomgateway.go b/pkg/bloomgateway/bloomgateway.go index f70f879ea8797..17bd9b395b9a9 100644 --- a/pkg/bloomgateway/bloomgateway.go +++ b/pkg/bloomgateway/bloomgateway.go @@ -58,6 +58,7 @@ import ( "github.com/prometheus/client_golang/prometheus/promauto" "github.com/grafana/loki/pkg/logproto" + "github.com/grafana/loki/pkg/logqlmodel/stats" "github.com/grafana/loki/pkg/queue" "github.com/grafana/loki/pkg/storage" v1 "github.com/grafana/loki/pkg/storage/bloom/v1" @@ -208,13 +209,26 @@ func New(cfg Config, schemaCfg config.SchemaConfig, storageCfg storage.Config, o workerMetrics: newWorkerMetrics(reg, constants.Loki, metricsSubsystem), queueMetrics: queue.NewMetrics(reg, constants.Loki, metricsSubsystem), } + var err error g.queue = queue.NewRequestQueue(cfg.MaxOutstandingPerTenant, time.Minute, &fixedQueueLimits{0}, g.queueMetrics) g.activeUsers = util.NewActiveUsersCleanupWithDefaultValues(g.queueMetrics.Cleanup) - // TODO(chaudum): Plug in cache var metasCache cache.Cache - var blocksCache *cache.EmbeddedCache[string, bloomshipper.BlockDirectory] + mcCfg := storageCfg.BloomShipperConfig.MetasCache + if cache.IsCacheConfigured(mcCfg) { + metasCache, err = cache.New(mcCfg, reg, logger, stats.BloomMetasCache, constants.Loki) + if err != nil { + return nil, err + } + } + + var blocksCache cache.TypedCache[string, bloomshipper.BlockDirectory] + bcCfg := storageCfg.BloomShipperConfig.BlocksCache + if bcCfg.IsEnabled() { + blocksCache = bloomshipper.NewBlocksCache(bcCfg, reg, logger) + } + store, err := bloomshipper.NewBloomStore(schemaCfg.Configs, storageCfg, cm, metasCache, blocksCache, logger) if err != nil { return nil, err diff --git a/pkg/logqlmodel/stats/context.go b/pkg/logqlmodel/stats/context.go index 5b25d900c37a8..4fbddc790b8b2 100644 --- a/pkg/logqlmodel/stats/context.go +++ b/pkg/logqlmodel/stats/context.go @@ -55,16 +55,17 @@ type Context struct { type CacheType string const ( - ChunkCache CacheType = "chunk" //nolint:staticcheck - IndexCache = "index" - ResultCache = "result" - StatsResultCache = "stats-result" - VolumeResultCache = "volume-result" - WriteDedupeCache = "write-dedupe" - SeriesResultCache = "series-result" - LabelResultCache = "label-result" - BloomFilterCache = "bloom-filter" - BloomBlocksCache = "bloom-blocks" + ChunkCache CacheType = "chunk" //nolint:staticcheck + IndexCache CacheType = "index" //nolint:staticcheck + ResultCache CacheType = "result" //nolint:staticcheck + StatsResultCache CacheType = "stats-result" //nolint:staticcheck + VolumeResultCache CacheType = "volume-result" //nolint:staticcheck + WriteDedupeCache CacheType = "write-dedupe" //nolint:staticcheck + SeriesResultCache CacheType = "series-result" //nolint:staticcheck + LabelResultCache CacheType = "label-result" //nolint:staticcheck + BloomFilterCache CacheType = "bloom-filter" //nolint:staticcheck + BloomBlocksCache CacheType = "bloom-blocks" //nolint:staticcheck + BloomMetasCache CacheType = "bloom-metas" //nolint:staticcheck ) // NewContext creates a new statistics context diff --git a/pkg/storage/chunk/cache/embeddedcache.go b/pkg/storage/chunk/cache/embeddedcache.go index 871c1ef4e1e78..3c1fae8fb4d2e 100644 --- a/pkg/storage/chunk/cache/embeddedcache.go +++ b/pkg/storage/chunk/cache/embeddedcache.go @@ -23,12 +23,22 @@ const ( defaultPurgeInterval = 1 * time.Minute - expiredReason string = "expired" //nolint:staticcheck - fullReason = "full" - tooBigReason = "object too big" - replacedReason = "replaced" + expiredReason = "expired" + fullReason = "full" + tooBigReason = "object too big" + replacedReason = "replaced" ) +// Interface for EmbeddedCache +// Matches the interface from cache.Cache but has generics +type TypedCache[K comparable, V any] interface { + Store(ctx context.Context, keys []K, values []V) error + Fetch(ctx context.Context, keys []K) (found []K, values []V, missing []K, err error) + Stop() + // GetCacheType returns a string indicating the cache "type" for the purpose of grouping cache usage statistics + GetCacheType() stats.CacheType +} + // EmbeddedCache is a simple (comparable -> any) cache which uses a fifo slide to // manage evictions. O(1) inserts and updates, O(1) gets. // @@ -324,3 +334,24 @@ func sizeOf(item *Entry[string, []byte]) uint64 { elementSize + // size of the element in linked list elementPrtSize) // size of the pointer to an element in the map } + +func NewNoopTypedCache[K comparable, V any]() TypedCache[K, V] { + return &noopEmbeddedCache[K, V]{} +} + +type noopEmbeddedCache[K comparable, V any] struct{} + +func (noopEmbeddedCache[K, V]) Store(_ context.Context, _ []K, _ []V) error { + return nil +} + +func (noopEmbeddedCache[K, V]) Fetch(_ context.Context, keys []K) ([]K, []V, []K, error) { + return []K{}, []V{}, keys, nil +} + +func (noopEmbeddedCache[K, V]) Stop() { +} + +func (noopEmbeddedCache[K, V]) GetCacheType() stats.CacheType { + return "noop" +} diff --git a/pkg/storage/stores/shipper/bloomshipper/cache.go b/pkg/storage/stores/shipper/bloomshipper/cache.go index 52899a03fea0a..f5ceb930017f7 100644 --- a/pkg/storage/stores/shipper/bloomshipper/cache.go +++ b/pkg/storage/stores/shipper/bloomshipper/cache.go @@ -14,7 +14,6 @@ import ( "github.com/grafana/loki/pkg/logqlmodel/stats" v1 "github.com/grafana/loki/pkg/storage/bloom/v1" "github.com/grafana/loki/pkg/storage/chunk/cache" - "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper/config" ) type CloseableBlockQuerier struct { @@ -30,10 +29,10 @@ func (c *CloseableBlockQuerier) Close() error { return nil } -func NewBlocksCache(config config.Config, reg prometheus.Registerer, logger log.Logger) *cache.EmbeddedCache[string, BlockDirectory] { +func NewBlocksCache(cfg cache.EmbeddedCacheConfig, reg prometheus.Registerer, logger log.Logger) *cache.EmbeddedCache[string, BlockDirectory] { return cache.NewTypedEmbeddedCache[string, BlockDirectory]( "bloom-blocks-cache", - config.BlocksCache.EmbeddedCacheConfig, + cfg, reg, logger, stats.BloomBlocksCache, diff --git a/pkg/storage/stores/shipper/bloomshipper/config/config.go b/pkg/storage/stores/shipper/bloomshipper/config/config.go index fbfe5f7803516..8b9eb7d9c706d 100644 --- a/pkg/storage/stores/shipper/bloomshipper/config/config.go +++ b/pkg/storage/stores/shipper/bloomshipper/config/config.go @@ -11,20 +11,10 @@ import ( ) type Config struct { - WorkingDirectory string `yaml:"working_directory"` - BlocksDownloadingQueue DownloadingQueueConfig `yaml:"blocks_downloading_queue"` - BlocksCache BlocksCacheConfig `yaml:"blocks_cache"` -} - -type BlocksCacheConfig struct { - EmbeddedCacheConfig cache.EmbeddedCacheConfig `yaml:",inline"` - RemoveDirectoryGracefulPeriod time.Duration `yaml:"remove_directory_graceful_period"` -} - -func (c *BlocksCacheConfig) RegisterFlagsWithPrefixAndDefaults(prefix string, f *flag.FlagSet) { - c.EmbeddedCacheConfig.RegisterFlagsWithPrefixAndDefaults(prefix, "", f, 0) - f.DurationVar(&c.RemoveDirectoryGracefulPeriod, prefix+"remove-directory-graceful-period", 5*time.Minute, - "During this period the process waits until the directory becomes not used and only after this it will be deleted. If the timeout is reached, the directory is force deleted.") + WorkingDirectory string `yaml:"working_directory"` + BlocksDownloadingQueue DownloadingQueueConfig `yaml:"blocks_downloading_queue"` + BlocksCache cache.EmbeddedCacheConfig `yaml:"blocks_cache"` + MetasCache cache.Config `yaml:"metas_cache"` } type DownloadingQueueConfig struct { @@ -40,7 +30,8 @@ func (cfg *DownloadingQueueConfig) RegisterFlagsWithPrefix(prefix string, f *fla func (c *Config) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) { f.StringVar(&c.WorkingDirectory, prefix+"shipper.working-directory", "bloom-shipper", "Working directory to store downloaded Bloom Blocks.") c.BlocksDownloadingQueue.RegisterFlagsWithPrefix(prefix+"shipper.blocks-downloading-queue.", f) - c.BlocksCache.RegisterFlagsWithPrefixAndDefaults("blocks-cache.", f) + c.BlocksCache.RegisterFlagsWithPrefixAndDefaults(prefix+"blocks-cache.", "Cache for bloom blocks. ", f, 24*time.Hour) + c.MetasCache.RegisterFlagsWithPrefix(prefix+"metas-cache.", "Cache for bloom metas. ", f) } func (c *Config) Validate() error { diff --git a/pkg/storage/stores/shipper/bloomshipper/fetcher.go b/pkg/storage/stores/shipper/bloomshipper/fetcher.go index bb9a70644f5e6..61dac17b21d87 100644 --- a/pkg/storage/stores/shipper/bloomshipper/fetcher.go +++ b/pkg/storage/stores/shipper/bloomshipper/fetcher.go @@ -32,7 +32,7 @@ type Fetcher struct { client Client metasCache cache.Cache - blocksCache *cache.EmbeddedCache[string, BlockDirectory] + blocksCache cache.TypedCache[string, BlockDirectory] localFSResolver KeyResolver q *downloadQueue[BlockRef, BlockDirectory] @@ -41,7 +41,7 @@ type Fetcher struct { logger log.Logger } -func NewFetcher(cfg bloomStoreConfig, client Client, metasCache cache.Cache, blocksCache *cache.EmbeddedCache[string, BlockDirectory], logger log.Logger) (*Fetcher, error) { +func NewFetcher(cfg bloomStoreConfig, client Client, metasCache cache.Cache, blocksCache cache.TypedCache[string, BlockDirectory], logger log.Logger) (*Fetcher, error) { fetcher := &Fetcher{ client: client, metasCache: metasCache, diff --git a/pkg/storage/stores/shipper/bloomshipper/store.go b/pkg/storage/stores/shipper/bloomshipper/store.go index c95d04122117f..d5cfa24b11ed5 100644 --- a/pkg/storage/stores/shipper/bloomshipper/store.go +++ b/pkg/storage/stores/shipper/bloomshipper/store.go @@ -146,7 +146,7 @@ func NewBloomStore( storageConfig storage.Config, clientMetrics storage.ClientMetrics, metasCache cache.Cache, - blocksCache *cache.EmbeddedCache[string, BlockDirectory], + blocksCache cache.TypedCache[string, BlockDirectory], logger log.Logger, ) (*BloomStore, error) { store := &BloomStore{ @@ -157,6 +157,10 @@ func NewBloomStore( metasCache = cache.NewNoopCache() } + if blocksCache == nil { + blocksCache = cache.NewNoopTypedCache[string, BlockDirectory]() + } + // sort by From time sort.Slice(periodicConfigs, func(i, j int) bool { return periodicConfigs[i].From.Time.Before(periodicConfigs[i].From.Time) diff --git a/pkg/storage/stores/shipper/bloomshipper/store_test.go b/pkg/storage/stores/shipper/bloomshipper/store_test.go index 42b35111688b2..56179cde97c36 100644 --- a/pkg/storage/stores/shipper/bloomshipper/store_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/store_test.go @@ -52,11 +52,9 @@ func newMockBloomStore(t *testing.T) (*BloomStore, string) { BlocksDownloadingQueue: config.DownloadingQueueConfig{ WorkersCount: 1, }, - BlocksCache: config.BlocksCacheConfig{ - EmbeddedCacheConfig: cache.EmbeddedCacheConfig{ - MaxSizeItems: 1000, - TTL: 1 * time.Hour, - }, + BlocksCache: cache.EmbeddedCacheConfig{ + MaxSizeItems: 1000, + TTL: 1 * time.Hour, }, }, } @@ -66,7 +64,7 @@ func newMockBloomStore(t *testing.T) (*BloomStore, string) { logger := log.NewLogfmtLogger(os.Stderr) metasCache := cache.NewMockCache() - blocksCache := NewBlocksCache(storageConfig.BloomShipperConfig, prometheus.NewPedanticRegistry(), logger) + blocksCache := NewBlocksCache(storageConfig.BloomShipperConfig.BlocksCache, prometheus.NewPedanticRegistry(), logger) store, err := NewBloomStore(periodicConfigs, storageConfig, metrics, metasCache, blocksCache, logger) require.NoError(t, err) t.Cleanup(store.Stop) From 3a1bd41ef56f32163e146aa0571922c202f054d8 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 8 Feb 2024 07:29:24 -0800 Subject: [PATCH 04/22] [Blooms] Routing & wiring for bloom building (#11894) A few changes wiring up bloom building logic: * A `Router` as the controlling mechanism for iterating tenants+periods, queueing work for parallel computation, determining which fingerprint ranges and tenants are owned by a replica, and waiting for it's completion * Updates the `SimplerBloomController` to accept ownership ranges, periods, and tenants as arguments for use by the `Router` * Builds a `BloomTSDBStore` struct around the `indexshipper/storage.Client` interface in order to iterate relevant TSDBs/etc --- pkg/bloomcompactor/controller.go | 107 ++++---- pkg/bloomcompactor/meta.go | 16 -- pkg/bloomcompactor/router.go | 229 ++++++++++++++++++ pkg/bloomcompactor/spec.go | 5 +- pkg/bloomcompactor/tsdb.go | 91 +++++++ .../shipper/indexshipper/tsdb/identifier.go | 4 +- .../indexshipper/tsdb/identifier_test.go | 2 +- 7 files changed, 381 insertions(+), 73 deletions(-) delete mode 100644 pkg/bloomcompactor/meta.go create mode 100644 pkg/bloomcompactor/router.go diff --git a/pkg/bloomcompactor/controller.go b/pkg/bloomcompactor/controller.go index 2002d8ce2a8bc..f3db6edb0636c 100644 --- a/pkg/bloomcompactor/controller.go +++ b/pkg/bloomcompactor/controller.go @@ -14,57 +14,47 @@ import ( "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb" ) -type uploader interface { - PutBlock(ctx context.Context, block bloomshipper.Block) error - PutMeta(ctx context.Context, meta bloomshipper.Meta) error -} - type SimpleBloomController struct { - // TODO(owen-d): consider making tenant+table dynamic (not 1 struct per combination) - tenant string - table string - ownershipRange v1.FingerprintBounds // ownership range of this controller - tsdbStore TSDBStore - bloomStore bloomshipper.Store - uploader uploader - chunkLoader ChunkLoader - rwFn func() (v1.BlockWriter, v1.BlockReader) - metrics *Metrics + tsdbStore TSDBStore + bloomStore bloomshipper.Store + chunkLoader ChunkLoader + rwFn func() (v1.BlockWriter, v1.BlockReader) + metrics *Metrics // TODO(owen-d): add metrics logger log.Logger } func NewSimpleBloomController( - tenant, table string, - ownershipRange v1.FingerprintBounds, tsdbStore TSDBStore, blockStore bloomshipper.Store, - uploader uploader, chunkLoader ChunkLoader, rwFn func() (v1.BlockWriter, v1.BlockReader), metrics *Metrics, logger log.Logger, ) *SimpleBloomController { return &SimpleBloomController{ - tenant: tenant, - table: table, - ownershipRange: ownershipRange, - tsdbStore: tsdbStore, - bloomStore: blockStore, - uploader: uploader, - chunkLoader: chunkLoader, - rwFn: rwFn, - metrics: metrics, - logger: log.With(logger, "ownership", ownershipRange), + tsdbStore: tsdbStore, + bloomStore: blockStore, + chunkLoader: chunkLoader, + rwFn: rwFn, + metrics: metrics, + logger: logger, } } -func (s *SimpleBloomController) do(ctx context.Context) error { +func (s *SimpleBloomController) buildBlocks( + ctx context.Context, + table DayTable, + tenant string, + ownershipRange v1.FingerprintBounds, +) error { + logger := log.With(s.logger, "ownership", ownershipRange, "org_id", tenant, "table", table) + // 1. Resolve TSDBs - tsdbs, err := s.tsdbStore.ResolveTSDBs() + tsdbs, err := s.tsdbStore.ResolveTSDBs(ctx, table.String(), tenant) if err != nil { - level.Error(s.logger).Log("msg", "failed to resolve tsdbs", "err", err) + level.Error(logger).Log("msg", "failed to resolve tsdbs", "err", err) return errors.Wrap(err, "failed to resolve tsdbs") } @@ -78,35 +68,39 @@ func (s *SimpleBloomController) do(ctx context.Context) error { } // 2. Fetch metas + bounds := table.Bounds() metas, err := s.bloomStore.FetchMetas( ctx, bloomshipper.MetaSearchParams{ - TenantID: s.tenant, - Interval: bloomshipper.Interval{}, // TODO(owen-d): gen interval - Keyspace: s.ownershipRange, + TenantID: tenant, + Interval: bloomshipper.Interval{ + Start: bounds.Start, + End: bounds.End, + }, + Keyspace: ownershipRange, }, ) if err != nil { - level.Error(s.logger).Log("msg", "failed to get metas", "err", err) + level.Error(logger).Log("msg", "failed to get metas", "err", err) return errors.Wrap(err, "failed to get metas") } // 3. Determine which TSDBs have gaps in the ownership range and need to // be processed. - tsdbsWithGaps, err := gapsBetweenTSDBsAndMetas(s.ownershipRange, ids, metas) + tsdbsWithGaps, err := gapsBetweenTSDBsAndMetas(ownershipRange, ids, metas) if err != nil { - level.Error(s.logger).Log("msg", "failed to find gaps", "err", err) + level.Error(logger).Log("msg", "failed to find gaps", "err", err) return errors.Wrap(err, "failed to find gaps") } if len(tsdbsWithGaps) == 0 { - level.Debug(s.logger).Log("msg", "blooms exist for all tsdbs") + level.Debug(logger).Log("msg", "blooms exist for all tsdbs") return nil } work, err := blockPlansForGaps(tsdbsWithGaps, metas) if err != nil { - level.Error(s.logger).Log("msg", "failed to create plan", "err", err) + level.Error(logger).Log("msg", "failed to create plan", "err", err) return errors.Wrap(err, "failed to create plan") } @@ -130,9 +124,9 @@ func (s *SimpleBloomController) do(ctx context.Context) error { for _, gap := range plan.gaps { // Fetch blocks that aren't up to date but are in the desired fingerprint range // to try and accelerate bloom creation - seriesItr, preExistingBlocks, err := s.loadWorkForGap(ctx, plan.tsdb, gap) + seriesItr, preExistingBlocks, err := s.loadWorkForGap(ctx, table.String(), tenant, plan.tsdb, gap) if err != nil { - level.Error(s.logger).Log("msg", "failed to get series and blocks", "err", err) + level.Error(logger).Log("msg", "failed to get series and blocks", "err", err) return errors.Wrap(err, "failed to get series and blocks") } @@ -143,33 +137,38 @@ func (s *SimpleBloomController) do(ctx context.Context) error { preExistingBlocks, s.rwFn, s.metrics, - log.With(s.logger, "tsdb", plan.tsdb.Name(), "ownership", gap, "blocks", len(preExistingBlocks)), + log.With(logger, "tsdb", plan.tsdb.Name(), "ownership", gap, "blocks", len(preExistingBlocks)), ) _, newBlocks, err := gen.Generate(ctx) if err != nil { // TODO(owen-d): metrics - level.Error(s.logger).Log("msg", "failed to generate bloom", "err", err) + level.Error(logger).Log("msg", "failed to generate bloom", "err", err) return errors.Wrap(err, "failed to generate bloom") } - // TODO(owen-d): dispatch this to a queue for writing, handling retries/backpressure, etc? + client, err := s.bloomStore.Client(table.ModelTime()) + + if err != nil { + level.Error(logger).Log("msg", "failed to get client", "err", err) + return errors.Wrap(err, "failed to get client") + } for newBlocks.Next() { blockCt++ blk := newBlocks.At() - if err := s.uploader.PutBlock( + if err := client.PutBlock( ctx, - bloomshipper.BlockFrom(s.tenant, s.table, blk), + bloomshipper.BlockFrom(tenant, table.String(), blk), ); err != nil { - level.Error(s.logger).Log("msg", "failed to write block", "err", err) + level.Error(logger).Log("msg", "failed to write block", "err", err) return errors.Wrap(err, "failed to write block") } } if err := newBlocks.Err(); err != nil { // TODO(owen-d): metrics - level.Error(s.logger).Log("msg", "failed to generate bloom", "err", err) + level.Error(logger).Log("msg", "failed to generate bloom", "err", err) return errors.Wrap(err, "failed to generate bloom") } @@ -179,14 +178,20 @@ func (s *SimpleBloomController) do(ctx context.Context) error { // TODO(owen-d): build meta from blocks // TODO(owen-d): reap tombstones, old metas - level.Debug(s.logger).Log("msg", "finished bloom generation", "blocks", blockCt, "tsdbs", tsdbCt) + level.Debug(logger).Log("msg", "finished bloom generation", "blocks", blockCt, "tsdbs", tsdbCt) return nil } -func (s *SimpleBloomController) loadWorkForGap(ctx context.Context, id tsdb.Identifier, gap gapWithBlocks) (v1.CloseableIterator[*v1.Series], []*bloomshipper.CloseableBlockQuerier, error) { +func (s *SimpleBloomController) loadWorkForGap( + ctx context.Context, + table, + tenant string, + id tsdb.Identifier, + gap gapWithBlocks, +) (v1.CloseableIterator[*v1.Series], []*bloomshipper.CloseableBlockQuerier, error) { // load a series iterator for the gap - seriesItr, err := s.tsdbStore.LoadTSDB(id, gap.bounds) + seriesItr, err := s.tsdbStore.LoadTSDB(ctx, table, tenant, id, gap.bounds) if err != nil { return nil, nil, errors.Wrap(err, "failed to load tsdb") } diff --git a/pkg/bloomcompactor/meta.go b/pkg/bloomcompactor/meta.go deleted file mode 100644 index 2f2c2cd9de16e..0000000000000 --- a/pkg/bloomcompactor/meta.go +++ /dev/null @@ -1,16 +0,0 @@ -package bloomcompactor - -import ( - v1 "github.com/grafana/loki/pkg/storage/bloom/v1" - "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb" -) - -const ( - BloomPrefix = "bloom" - MetasPrefix = "metas" -) - -type TSDBStore interface { - ResolveTSDBs() ([]*tsdb.SingleTenantTSDBIdentifier, error) - LoadTSDB(id tsdb.Identifier, bounds v1.FingerprintBounds) (v1.CloseableIterator[*v1.Series], error) -} diff --git a/pkg/bloomcompactor/router.go b/pkg/bloomcompactor/router.go new file mode 100644 index 0000000000000..e2c8b6545260f --- /dev/null +++ b/pkg/bloomcompactor/router.go @@ -0,0 +1,229 @@ +package bloomcompactor + +import ( + "context" + "fmt" + "math" + "sync" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/grafana/dskit/concurrency" + "github.com/grafana/dskit/multierror" + "github.com/pkg/errors" + "github.com/prometheus/common/model" + + v1 "github.com/grafana/loki/pkg/storage/bloom/v1" + "github.com/grafana/loki/pkg/storage/config" + "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper" +) + +type DayTable model.Time + +func (d DayTable) String() string { + return fmt.Sprintf("%d", d.ModelTime().Time().UnixNano()/int64(config.ObjectStorageIndexRequiredPeriod)) +} + +func (d DayTable) Inc() DayTable { + return DayTable(d.ModelTime().Add(config.ObjectStorageIndexRequiredPeriod)) +} + +func (d DayTable) Dec() DayTable { + return DayTable(d.ModelTime().Add(-config.ObjectStorageIndexRequiredPeriod)) +} + +func (d DayTable) Before(other DayTable) bool { + return d.ModelTime().Before(model.Time(other)) +} + +func (d DayTable) After(other DayTable) bool { + return d.ModelTime().After(model.Time(other)) +} + +func (d DayTable) ModelTime() model.Time { + return model.Time(d) +} + +func (d DayTable) Bounds() bloomshipper.Interval { + return bloomshipper.Interval{ + Start: model.Time(d), + End: model.Time(d.Inc()), + } +} + +type router struct { + // TODO(owen-d): configure these w/ limits + interval time.Duration // how often to run compaction loops + minTable, maxTable DayTable + + controller *SimpleBloomController + tsdbStore TSDBStore + + // we can parallelize by (tenant, table) tuples and we run `parallelism` workers + parallelism int + logger log.Logger +} + +type tenantTable struct { + tenant string + table DayTable + ownershipRange v1.FingerprintBounds +} + +func (r *router) tenants(ctx context.Context, table string) (v1.Iterator[string], error) { + tenants, err := r.tsdbStore.UsersForPeriod(ctx, table) + if err != nil { + return nil, errors.Wrap(err, "getting tenants") + } + + return v1.NewSliceIter(tenants), nil +} + +// TODO(owen-d): implement w/ subrings +func (r *router) ownsTenant(_ string) (ownershipRange v1.FingerprintBounds, owns bool) { + return v1.NewBounds(0, math.MaxUint64), true +} + +// TODO(owen-d): parameterize via limits +func (r *router) tables() (v1.Iterator[DayTable], error) { + return newDayRangeIterator(r.minTable, r.maxTable), nil +} + +func (r *router) run(ctx context.Context) error { + // run once at beginning + if err := r.runOne(ctx); err != nil { + return err + } + + ticker := time.NewTicker(r.interval) + defer ticker.Stop() + for { + select { + case <-ctx.Done(): + return ctx.Err() + + case <-ticker.C: + if err := r.runOne(ctx); err != nil { + level.Error(r.logger).Log("msg", "compaction iteration failed", "err", err) + return err + } + } + } +} + +// runs a single round of compaction for all relevant tenants and tables +func (r *router) runOne(ctx context.Context) error { + var workersErr error + var wg sync.WaitGroup + ch := make(chan tenantTable) + wg.Add(1) + go func() { + workersErr = r.runWorkers(ctx, ch) + wg.Done() + }() + + err := r.loadWork(ctx, ch) + + wg.Wait() + return multierror.New(workersErr, err, ctx.Err()).Err() +} + +func (r *router) loadWork(ctx context.Context, ch chan<- tenantTable) error { + tables, err := r.tables() + if err != nil { + return errors.Wrap(err, "getting tables") + } + + for tables.Next() && tables.Err() == nil && ctx.Err() == nil { + + table := tables.At() + tablestr := fmt.Sprintf("%d", table) + tenants, err := r.tenants(ctx, tablestr) + if err != nil { + return errors.Wrap(err, "getting tenants") + } + + for tenants.Next() && tenants.Err() == nil && ctx.Err() == nil { + tenant := tenants.At() + ownershipRange, owns := r.ownsTenant(tenant) + if !owns { + continue + } + + select { + case ch <- tenantTable{tenant: tenant, table: table, ownershipRange: ownershipRange}: + case <-ctx.Done(): + return ctx.Err() + } + } + + if err := tenants.Err(); err != nil { + return errors.Wrap(err, "iterating tenants") + } + + } + + if err := tables.Err(); err != nil { + return errors.Wrap(err, "iterating tables") + } + + close(ch) + return ctx.Err() +} + +func (r *router) runWorkers(ctx context.Context, ch <-chan tenantTable) error { + + return concurrency.ForEachJob(ctx, r.parallelism, r.parallelism, func(ctx context.Context, idx int) error { + + for { + select { + case <-ctx.Done(): + return ctx.Err() + + case tt, ok := <-ch: + if !ok { + return nil + } + + if err := r.compactTenantTable(ctx, tt); err != nil { + return errors.Wrapf( + err, + "compacting tenant table (%s) for tenant (%s) with ownership (%s)", + tt.table, + tt.tenant, + tt.ownershipRange, + ) + } + } + } + + }) + +} + +func (r *router) compactTenantTable(ctx context.Context, tt tenantTable) error { + level.Info(r.logger).Log("msg", "compacting", "org_id", tt.tenant, "table", tt.table, "ownership", tt.ownershipRange) + return r.controller.buildBlocks(ctx, tt.table, tt.tenant, tt.ownershipRange) +} + +type dayRangeIterator struct { + min, max, cur DayTable +} + +func newDayRangeIterator(min, max DayTable) *dayRangeIterator { + return &dayRangeIterator{min: min, max: max, cur: min.Dec()} +} + +func (r *dayRangeIterator) Next() bool { + r.cur = r.cur.Inc() + return r.cur.Before(r.max) +} + +func (r *dayRangeIterator) At() DayTable { + return r.cur +} + +func (r *dayRangeIterator) Err() error { + return nil +} diff --git a/pkg/bloomcompactor/spec.go b/pkg/bloomcompactor/spec.go index bf9a0a02387b4..088c5b823a1b5 100644 --- a/pkg/bloomcompactor/spec.go +++ b/pkg/bloomcompactor/spec.go @@ -148,8 +148,7 @@ func (s *SimpleBloomGenerator) Generate(ctx context.Context) (skippedBlocks []v1 }() for _, block := range s.blocks { - // TODO(owen-d): implement block naming so we can log the affected block in all these calls - logger := log.With(s.logger, "block", fmt.Sprintf("%+v", block)) + logger := log.With(s.logger, "block", block.BlockRef) md, err := block.Metadata() schema := md.Options.Schema if err != nil { @@ -331,7 +330,7 @@ func (b *batchedLoader) format(c chunk.Chunk) (v1.ChunkRefWithIter, error) { b.metrics.chunkSize.Observe(float64(chk.UncompressedSize())) itr, err := chk.Iterator( b.ctx, - time.Unix(0, 0), // TODO: Parameterize/better handle the timestamps? + time.Unix(0, 0), time.Unix(0, math.MaxInt64), logproto.FORWARD, logql_log.NewNoopPipeline().ForStream(c.Metric), diff --git a/pkg/bloomcompactor/tsdb.go b/pkg/bloomcompactor/tsdb.go index bb4383cc84f60..5ade52f8ab1cd 100644 --- a/pkg/bloomcompactor/tsdb.go +++ b/pkg/bloomcompactor/tsdb.go @@ -2,15 +2,106 @@ package bloomcompactor import ( "context" + "io" "math" + "path" + "strings" + "github.com/pkg/errors" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" v1 "github.com/grafana/loki/pkg/storage/bloom/v1" + "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/storage" + "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb" "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/index" ) +const ( + gzipExtension = ".gz" +) + +type TSDBStore interface { + UsersForPeriod(ctx context.Context, table string) ([]string, error) + ResolveTSDBs(ctx context.Context, table, tenant string) ([]tsdb.SingleTenantTSDBIdentifier, error) + LoadTSDB( + ctx context.Context, + table, + tenant string, + id tsdb.Identifier, + bounds v1.FingerprintBounds, + ) (v1.CloseableIterator[*v1.Series], error) +} + +// BloomTSDBStore is a wrapper around the storage.Client interface which +// implements the TSDBStore interface for this pkg. +type BloomTSDBStore struct { + storage storage.Client +} + +func NewBloomTSDBStore(storage storage.Client) *BloomTSDBStore { + return &BloomTSDBStore{ + storage: storage, + } +} + +func (b *BloomTSDBStore) UsersForPeriod(ctx context.Context, table string) ([]string, error) { + _, users, err := b.storage.ListFiles(ctx, table, false) + return users, err +} + +func (b *BloomTSDBStore) ResolveTSDBs(ctx context.Context, table, tenant string) ([]tsdb.SingleTenantTSDBIdentifier, error) { + indices, err := b.storage.ListUserFiles(ctx, table, tenant, false) + if err != nil { + return nil, errors.Wrap(err, "failed to list user files") + } + + ids := make([]tsdb.SingleTenantTSDBIdentifier, 0, len(indices)) + for _, index := range indices { + key := index.Name + if decompress := storage.IsCompressedFile(index.Name); decompress { + key = strings.TrimSuffix(key, gzipExtension) + } + + id, ok := tsdb.ParseSingleTenantTSDBPath(path.Base(key)) + if !ok { + return nil, errors.Errorf("failed to parse single tenant tsdb path: %s", key) + } + + ids = append(ids, id) + + } + return ids, nil +} + +func (b *BloomTSDBStore) LoadTSDB( + ctx context.Context, + table, + tenant string, + id tsdb.Identifier, + bounds v1.FingerprintBounds, +) (v1.CloseableIterator[*v1.Series], error) { + data, err := b.storage.GetUserFile(ctx, table, tenant, id.Name()) + if err != nil { + return nil, errors.Wrap(err, "failed to get file") + } + + buf, err := io.ReadAll(data) + if err != nil { + return nil, errors.Wrap(err, "failed to read file") + } + _ = data.Close() + + reader, err := index.NewReader(index.RealByteSlice(buf)) + if err != nil { + return nil, errors.Wrap(err, "failed to create index reader") + } + + idx := tsdb.NewTSDBIndex(reader) + + return NewTSDBSeriesIter(ctx, idx, bounds), nil +} + // TSDBStore is an interface for interacting with the TSDB, // modeled off a relevant subset of the `tsdb.TSDBIndex` struct type forSeries interface { diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/identifier.go b/pkg/storage/stores/shipper/indexshipper/tsdb/identifier.go index 7bfd10ff4d4dc..451688d040e36 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/identifier.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/identifier.go @@ -25,7 +25,7 @@ type Identifier interface { // identifierFromPath will detect whether this is a single or multitenant TSDB func identifierFromPath(p string) (Identifier, error) { // try parsing as single tenant since the filename is more deterministic without an arbitrary nodename for uploader - id, ok := parseSingleTenantTSDBPath(p) + id, ok := ParseSingleTenantTSDBPath(p) if ok { return NewPrefixedIdentifier(id, filepath.Dir(p), ""), nil } @@ -95,7 +95,7 @@ func (i SingleTenantTSDBIdentifier) Path() string { return i.str() } -func parseSingleTenantTSDBPath(p string) (id SingleTenantTSDBIdentifier, ok bool) { +func ParseSingleTenantTSDBPath(p string) (id SingleTenantTSDBIdentifier, ok bool) { // parsing as multitenant didn't work, so try single tenant // incorrect suffix diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/identifier_test.go b/pkg/storage/stores/shipper/indexshipper/tsdb/identifier_test.go index 635a14a2eaa95..7fcd56f89b0eb 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/identifier_test.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/identifier_test.go @@ -44,7 +44,7 @@ func TestParseSingleTenantTSDBPath(t *testing.T) { }, } { t.Run(tc.desc, func(t *testing.T) { - id, ok := parseSingleTenantTSDBPath(tc.input) + id, ok := ParseSingleTenantTSDBPath(tc.input) require.Equal(t, tc.id, id) require.Equal(t, tc.ok, ok) }) From fe53eb6817756c54d3447d5f1265bbac7f8a492b Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 8 Feb 2024 10:08:11 -0800 Subject: [PATCH 05/22] fix type errors after introduction of typedcache into bloomshipper (#11902) Recent changes causing compilation failures -- this fixes type signatures after refactoring. Also removes test coverage comparisons because they prevent us from fixing `main` --- .drone/drone.jsonnet | 17 -------- .drone/drone.yml | 43 +------------------ .../shipper/bloomshipper/shipper_test.go | 8 ++-- 3 files changed, 6 insertions(+), 62 deletions(-) diff --git a/.drone/drone.jsonnet b/.drone/drone.jsonnet index 49f67f06861a3..9351f2f693c06 100644 --- a/.drone/drone.jsonnet +++ b/.drone/drone.jsonnet @@ -610,23 +610,6 @@ local build_image_tag = '0.33.0'; 'cd -', ]) { depends_on: ['clone'], when: onPRs }, make('test', container=false) { depends_on: ['clone-target-branch', 'check-generated-files'] }, - run('test-target-branch', commands=['cd ../loki-target-branch && BUILD_IN_CONTAINER=false make test']) { depends_on: ['clone-target-branch'], when: onPRs }, - make('compare-coverage', container=false, args=[ - 'old=../loki-target-branch/test_results.txt', - 'new=test_results.txt', - 'packages=ingester,distributor,querier,querier/queryrange,iter,storage,chunkenc,logql,loki', - '> diff.txt', - ]) { depends_on: ['test', 'test-target-branch'], when: onPRs }, - run('report-coverage', commands=[ - "total_diff=$(sed 's/%//' diff.txt | awk '{sum+=$3;}END{print sum;}')", - 'if [ $total_diff = 0 ]; then exit 0; fi', - "pull=$(echo $CI_COMMIT_REF | awk -F '/' '{print $3}')", - "body=$(jq -Rs '{body: . }' diff.txt)", - 'curl -X POST -u $USER:$TOKEN -H "Accept: application/vnd.github.v3+json" https://api.github.com/repos/grafana/loki/issues/$pull/comments -d "$body" > /dev/null', - ], env={ - USER: 'grafanabot', - TOKEN: { from_secret: github_secret.name }, - }) { depends_on: ['compare-coverage'], when: onPRs }, make('lint', container=false) { depends_on: ['check-generated-files'] }, make('check-mod', container=false) { depends_on: ['test', 'lint'] }, { diff --git a/.drone/drone.yml b/.drone/drone.yml index 7a62b621262a8..c33a66998e71c 100644 --- a/.drone/drone.yml +++ b/.drone/drone.yml @@ -212,47 +212,6 @@ steps: environment: {} image: grafana/loki-build-image:0.33.0 name: test -- commands: - - cd ../loki-target-branch && BUILD_IN_CONTAINER=false make test - depends_on: - - clone-target-branch - environment: {} - image: grafana/loki-build-image:0.33.0 - name: test-target-branch - when: - event: - - pull_request -- commands: - - make BUILD_IN_CONTAINER=false compare-coverage old=../loki-target-branch/test_results.txt - new=test_results.txt packages=ingester,distributor,querier,querier/queryrange,iter,storage,chunkenc,logql,loki - > diff.txt - depends_on: - - test - - test-target-branch - environment: {} - image: grafana/loki-build-image:0.33.0 - name: compare-coverage - when: - event: - - pull_request -- commands: - - total_diff=$(sed 's/%//' diff.txt | awk '{sum+=$3;}END{print sum;}') - - if [ $total_diff = 0 ]; then exit 0; fi - - pull=$(echo $CI_COMMIT_REF | awk -F '/' '{print $3}') - - 'body=$(jq -Rs ''{body: . }'' diff.txt)' - - 'curl -X POST -u $USER:$TOKEN -H "Accept: application/vnd.github.v3+json" https://api.github.com/repos/grafana/loki/issues/$pull/comments - -d "$body" > /dev/null' - depends_on: - - compare-coverage - environment: - TOKEN: - from_secret: github_token - USER: grafanabot - image: grafana/loki-build-image:0.33.0 - name: report-coverage - when: - event: - - pull_request - commands: - make BUILD_IN_CONTAINER=false lint depends_on: @@ -2113,6 +2072,6 @@ kind: secret name: gpg_private_key --- kind: signature -hmac: 457592d17208477ceb480f81dbdb88f7b95a5ad015c88d9d6fed06c2422a52f9 +hmac: 51861919f0ba5370a152bdb9267828c742f2042819fb01388c6d23bf44e3cbb7 ... diff --git a/pkg/storage/stores/shipper/bloomshipper/shipper_test.go b/pkg/storage/stores/shipper/bloomshipper/shipper_test.go index 4c755e1baa6d3..a3a9442ddfb03 100644 --- a/pkg/storage/stores/shipper/bloomshipper/shipper_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/shipper_test.go @@ -205,9 +205,11 @@ func TestBloomShipper_ForEach(t *testing.T) { for i := 0; i < len(blockRefs); i++ { s := store.stores[0] key := s.Block(blockRefs[i]).Addr() - dir, found := s.fetcher.blocksCache.Get(context.Background(), key) - require.True(t, found) - require.Equal(t, int32(0), dir.refCount.Load()) + found, dirs, missing, err := s.fetcher.blocksCache.Fetch(context.Background(), []string{key}) + require.NoError(t, err) + require.Equal(t, 1, len(found)) + require.Equal(t, 0, len(missing)) + require.Equal(t, int32(0), dirs[0].refCount.Load()) } } From 52167c4d32cc1ed9b2cb51d48e443b61a52e4a3f Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 8 Feb 2024 10:21:27 -0800 Subject: [PATCH 06/22] logs query & hashedquery separately to minimize stacktraces (#11901) Rather than call all our values together, this PR precomputes a few in `metrics.go` so any issues will result in smaller stack traces. --- pkg/logql/metrics.go | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/pkg/logql/metrics.go b/pkg/logql/metrics.go index 63051e362eae6..40fbece82d87d 100644 --- a/pkg/logql/metrics.go +++ b/pkg/logql/metrics.go @@ -114,13 +114,17 @@ func RecordRangeAndInstantQueryMetrics( } queryTags, _ := ctx.Value(httpreq.QueryTagsHTTPHeader).(string) // it's ok to be empty. + var ( + query = p.QueryString() + hashedQuery = util.HashedQuery(query) + ) logValues := make([]interface{}, 0, 50) logValues = append(logValues, []interface{}{ "latency", latencyType, // this can be used to filter log lines. - "query", p.QueryString(), - "query_hash", util.HashedQuery(p.QueryString()), + "query", query, + "query_hash", hashedQuery, "query_type", queryType, "range_type", rt, "length", p.End().Sub(p.Start()), From 5914df77b961f44563b2827357cba87df84f3778 Mon Sep 17 00:00:00 2001 From: Periklis Tsirakidis Date: Thu, 8 Feb 2024 19:35:37 +0100 Subject: [PATCH 07/22] operator: Add support for running with Google Workload Identity (#11869) Co-authored-by: Robert Jacob --- operator/CHANGELOG.md | 1 + .../handlers/internal/storage/secrets.go | 36 +++- .../handlers/internal/storage/secrets_test.go | 36 +++- .../internal/manifests/storage/configure.go | 10 ++ .../manifests/storage/configure_test.go | 162 ++++++++++++++++++ .../internal/manifests/storage/options.go | 4 +- operator/internal/manifests/storage/var.go | 5 + 7 files changed, 251 insertions(+), 3 deletions(-) diff --git a/operator/CHANGELOG.md b/operator/CHANGELOG.md index d978c0c8f423d..2a1ebc2f5d362 100644 --- a/operator/CHANGELOG.md +++ b/operator/CHANGELOG.md @@ -1,5 +1,6 @@ ## Main +- [11869](https://github.com/grafana/loki/pull/11869) **periklis**: Add support for running with Google Workload Identity - [11868](https://github.com/grafana/loki/pull/11868) **xperimental**: Integrate support for OpenShift-managed credentials in Azure - [11854](https://github.com/grafana/loki/pull/11854) **periklis**: Allow custom audience for managed-auth on STS - [11802](https://github.com/grafana/loki/pull/11802) **xperimental**: Add support for running with Azure Workload Identity diff --git a/operator/internal/handlers/internal/storage/secrets.go b/operator/internal/handlers/internal/storage/secrets.go index 6b8275d2d28ae..21cd58b7c3c25 100644 --- a/operator/internal/handlers/internal/storage/secrets.go +++ b/operator/internal/handlers/internal/storage/secrets.go @@ -3,6 +3,7 @@ package storage import ( "context" "crypto/sha1" + "encoding/json" "errors" "fmt" "sort" @@ -32,8 +33,13 @@ var ( errAzureNoCredentials = errors.New("azure storage secret does contain neither account_key or client_id") errAzureMixedCredentials = errors.New("azure storage secret can not contain both account_key and client_id") errAzureManagedIdentityNoOverride = errors.New("when in managed mode, storage secret can not contain credentials") + + errGCPParseCredentialsFile = errors.New("gcp storage secret cannot be parsed from JSON content") + errGCPWrongCredentialSourceFile = errors.New("credential source in secret needs to point to token file") ) +const gcpAccountTypeExternal = "external_account" + func getSecrets(ctx context.Context, k k8s.Client, stack *lokiv1.LokiStack, fg configv1.FeatureGates) (*corev1.Secret, *corev1.Secret, error) { var ( storageSecret corev1.Secret @@ -255,8 +261,36 @@ func extractGCSConfigSecret(s *corev1.Secret) (*storage.GCSStorageConfig, error) return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyGCPServiceAccountKeyFilename) } + credentialsFile := struct { + CredentialsType string `json:"type"` + CredentialsSource struct { + File string `json:"file"` + } `json:"credential_source"` + }{} + + err := json.Unmarshal(keyJSON, &credentialsFile) + if err != nil { + return nil, errGCPParseCredentialsFile + } + + var ( + audience = s.Data[storage.KeyGCPWorkloadIdentityProviderAudience] + isWorkloadIdentity = credentialsFile.CredentialsType == gcpAccountTypeExternal + ) + if isWorkloadIdentity { + if len(audience) == 0 { + return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyGCPWorkloadIdentityProviderAudience) + } + + if credentialsFile.CredentialsSource.File != storage.GCPDefautCredentialsFile { + return nil, fmt.Errorf("%w: %s", errGCPWrongCredentialSourceFile, storage.GCPDefautCredentialsFile) + } + } + return &storage.GCSStorageConfig{ - Bucket: string(bucket), + Bucket: string(bucket), + WorkloadIdentity: isWorkloadIdentity, + Audience: string(audience), }, nil } diff --git a/operator/internal/handlers/internal/storage/secrets_test.go b/operator/internal/handlers/internal/storage/secrets_test.go index 94b6ae2e3aaa1..cc18360232315 100644 --- a/operator/internal/handlers/internal/storage/secrets_test.go +++ b/operator/internal/handlers/internal/storage/secrets_test.go @@ -314,13 +314,47 @@ func TestGCSExtract(t *testing.T) { }, wantError: "missing secret field: key.json", }, + { + name: "missing audience", + secret: &corev1.Secret{ + ObjectMeta: metav1.ObjectMeta{Name: "test"}, + Data: map[string][]byte{ + "bucketname": []byte("here"), + "key.json": []byte("{\"type\": \"external_account\"}"), + }, + }, + wantError: "missing secret field: audience", + }, + { + name: "credential_source file no override", + secret: &corev1.Secret{ + ObjectMeta: metav1.ObjectMeta{Name: "test"}, + Data: map[string][]byte{ + "bucketname": []byte("here"), + "audience": []byte("test"), + "key.json": []byte("{\"type\": \"external_account\", \"credential_source\": {\"file\": \"/custom/path/to/secret/gcp/serviceaccount/token\"}}"), + }, + }, + wantError: "credential source in secret needs to point to token file: /var/run/secrets/gcp/serviceaccount/token", + }, { name: "all set", secret: &corev1.Secret{ ObjectMeta: metav1.ObjectMeta{Name: "test"}, Data: map[string][]byte{ "bucketname": []byte("here"), - "key.json": []byte("{\"type\": \"SA\"}"), + "key.json": []byte("{\"type\": \"service_account\"}"), + }, + }, + }, + { + name: "mandatory for workload-identity set", + secret: &corev1.Secret{ + ObjectMeta: metav1.ObjectMeta{Name: "test"}, + Data: map[string][]byte{ + "bucketname": []byte("here"), + "audience": []byte("test"), + "key.json": []byte("{\"type\": \"external_account\", \"credential_source\": {\"file\": \"/var/run/secrets/gcp/serviceaccount/token\"}}"), }, }, }, diff --git a/operator/internal/manifests/storage/configure.go b/operator/internal/manifests/storage/configure.go index f3fd86ebbaa1c..49958ebec7b9c 100644 --- a/operator/internal/manifests/storage/configure.go +++ b/operator/internal/manifests/storage/configure.go @@ -210,6 +210,10 @@ func managedAuthCredentials(opts Options) []corev1.EnvVar { envVarFromSecret(EnvAzureSubscriptionID, opts.SecretName, KeyAzureStorageSubscriptionID), envVarFromValue(EnvAzureFederatedTokenFile, path.Join(azureTokenVolumeDirectory, "token")), } + case lokiv1.ObjectStorageSecretGCS: + return []corev1.EnvVar{ + envVarFromValue(EnvGoogleApplicationCredentials, path.Join(secretDirectory, KeyGCPServiceAccountKeyFilename)), + } default: return []corev1.EnvVar{} } @@ -290,6 +294,8 @@ func managedAuthEnabled(opts Options) bool { return opts.S3 != nil && opts.S3.STS case lokiv1.ObjectStorageSecretAzure: return opts.Azure != nil && opts.Azure.WorkloadIdentity + case lokiv1.ObjectStorageSecretGCS: + return opts.GCS != nil && opts.GCS.WorkloadIdentity default: return false } @@ -302,6 +308,8 @@ func saTokenVolumeMount(opts Options) corev1.VolumeMount { tokenPath = AWSTokenVolumeDirectory case lokiv1.ObjectStorageSecretAzure: tokenPath = azureTokenVolumeDirectory + case lokiv1.ObjectStorageSecretGCS: + tokenPath = gcpTokenVolumeDirectory } return corev1.VolumeMount{ Name: saTokenVolumeName, @@ -323,6 +331,8 @@ func saTokenVolume(opts Options) corev1.Volume { if opts.Azure.Audience != "" { audience = opts.Azure.Audience } + case lokiv1.ObjectStorageSecretGCS: + audience = opts.GCS.Audience } return corev1.Volume{ Name: saTokenVolumeName, diff --git a/operator/internal/manifests/storage/configure_test.go b/operator/internal/manifests/storage/configure_test.go index 03e22682f4028..f17a9af6c3524 100644 --- a/operator/internal/manifests/storage/configure_test.go +++ b/operator/internal/manifests/storage/configure_test.go @@ -616,6 +616,87 @@ func TestConfigureDeploymentForStorageType(t *testing.T) { }, }, }, + { + desc: "object storage GCS with Workload Identity", + opts: Options{ + SecretName: "test", + SharedStore: lokiv1.ObjectStorageSecretGCS, + GCS: &GCSStorageConfig{ + Audience: "test", + WorkloadIdentity: true, + }, + }, + dpl: &appsv1.Deployment{ + Spec: appsv1.DeploymentSpec{ + Template: corev1.PodTemplateSpec{ + Spec: corev1.PodSpec{ + Containers: []corev1.Container{ + { + Name: "loki-ingester", + }, + }, + }, + }, + }, + }, + want: &appsv1.Deployment{ + Spec: appsv1.DeploymentSpec{ + Template: corev1.PodTemplateSpec{ + Spec: corev1.PodSpec{ + Containers: []corev1.Container{ + { + Name: "loki-ingester", + VolumeMounts: []corev1.VolumeMount{ + { + Name: "test", + ReadOnly: false, + MountPath: "/etc/storage/secrets", + }, + { + Name: saTokenVolumeName, + ReadOnly: false, + MountPath: "/var/run/secrets/gcp/serviceaccount", + }, + }, + Env: []corev1.EnvVar{ + { + Name: EnvGoogleApplicationCredentials, + Value: "/etc/storage/secrets/key.json", + }, + }, + }, + }, + Volumes: []corev1.Volume{ + { + Name: "test", + VolumeSource: corev1.VolumeSource{ + Secret: &corev1.SecretVolumeSource{ + SecretName: "test", + }, + }, + }, + { + Name: saTokenVolumeName, + VolumeSource: corev1.VolumeSource{ + Projected: &corev1.ProjectedVolumeSource{ + Sources: []corev1.VolumeProjection{ + { + ServiceAccountToken: &corev1.ServiceAccountTokenProjection{ + Audience: "test", + ExpirationSeconds: ptr.To[int64](3600), + Path: corev1.ServiceAccountTokenKey, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, { desc: "object storage S3", opts: Options{ @@ -1669,6 +1750,87 @@ func TestConfigureStatefulSetForStorageType(t *testing.T) { }, }, }, + { + desc: "object storage GCS with Workload Identity", + opts: Options{ + SecretName: "test", + SharedStore: lokiv1.ObjectStorageSecretGCS, + GCS: &GCSStorageConfig{ + Audience: "test", + WorkloadIdentity: true, + }, + }, + sts: &appsv1.StatefulSet{ + Spec: appsv1.StatefulSetSpec{ + Template: corev1.PodTemplateSpec{ + Spec: corev1.PodSpec{ + Containers: []corev1.Container{ + { + Name: "loki-ingester", + }, + }, + }, + }, + }, + }, + want: &appsv1.StatefulSet{ + Spec: appsv1.StatefulSetSpec{ + Template: corev1.PodTemplateSpec{ + Spec: corev1.PodSpec{ + Containers: []corev1.Container{ + { + Name: "loki-ingester", + VolumeMounts: []corev1.VolumeMount{ + { + Name: "test", + ReadOnly: false, + MountPath: "/etc/storage/secrets", + }, + { + Name: saTokenVolumeName, + ReadOnly: false, + MountPath: "/var/run/secrets/gcp/serviceaccount", + }, + }, + Env: []corev1.EnvVar{ + { + Name: EnvGoogleApplicationCredentials, + Value: "/etc/storage/secrets/key.json", + }, + }, + }, + }, + Volumes: []corev1.Volume{ + { + Name: "test", + VolumeSource: corev1.VolumeSource{ + Secret: &corev1.SecretVolumeSource{ + SecretName: "test", + }, + }, + }, + { + Name: saTokenVolumeName, + VolumeSource: corev1.VolumeSource{ + Projected: &corev1.ProjectedVolumeSource{ + Sources: []corev1.VolumeProjection{ + { + ServiceAccountToken: &corev1.ServiceAccountTokenProjection{ + Audience: "test", + ExpirationSeconds: ptr.To[int64](3600), + Path: corev1.ServiceAccountTokenKey, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, { desc: "object storage S3", opts: Options{ diff --git a/operator/internal/manifests/storage/options.go b/operator/internal/manifests/storage/options.go index 4c62ce7513755..e525640da6c0c 100644 --- a/operator/internal/manifests/storage/options.go +++ b/operator/internal/manifests/storage/options.go @@ -34,7 +34,9 @@ type AzureStorageConfig struct { // GCSStorageConfig for GCS storage config type GCSStorageConfig struct { - Bucket string + Bucket string + Audience string + WorkloadIdentity bool } // S3StorageConfig for S3 storage config diff --git a/operator/internal/manifests/storage/var.go b/operator/internal/manifests/storage/var.go index 418fb27152bd3..49ec0b0a16ae1 100644 --- a/operator/internal/manifests/storage/var.go +++ b/operator/internal/manifests/storage/var.go @@ -91,6 +91,8 @@ const ( // KeyAzureAudience is the secret data key for customizing the audience used for the ServiceAccount token. KeyAzureAudience = "audience" + // KeyGCPWorkloadIdentityProviderAudience is the secret data key for the GCP Workload Identity Provider audience. + KeyGCPWorkloadIdentityProviderAudience = "audience" // KeyGCPStorageBucketName is the secret data key for the GCS bucket name. KeyGCPStorageBucketName = "bucketname" // KeyGCPServiceAccountKeyFilename is the service account key filename containing the Google authentication credentials. @@ -144,5 +146,8 @@ const ( azureManagedCredentialKeyTenantID = "azure_tenant_id" azureManagedCredentialKeySubscriptionID = "azure_subscription_id" + gcpTokenVolumeDirectory = "/var/run/secrets/gcp/serviceaccount" + GCPDefautCredentialsFile = gcpTokenVolumeDirectory + "/token" + AnnotationCredentialsRequestsSecretRef = "loki.grafana.com/credentials-request-secret-ref" ) From 60551dacc4f4e77baef026fd30b632cd39b4eef3 Mon Sep 17 00:00:00 2001 From: Christian Haudum Date: Fri, 9 Feb 2024 10:15:04 +0100 Subject: [PATCH 08/22] Remove unused sharding code from bloom gateway (#11906) The sharding by fingerprint range is done on the client side, not on the server side. The sharding related code was not used in the gateway. Signed-off-by: Christian Haudum --- pkg/bloomgateway/bloomgateway.go | 5 +- pkg/bloomgateway/bloomgateway_test.go | 16 +-- pkg/bloomgateway/client.go | 37 ++++-- pkg/bloomgateway/client_test.go | 16 +-- pkg/bloomgateway/sharding.go | 156 -------------------------- pkg/loki/modules.go | 4 +- 6 files changed, 38 insertions(+), 196 deletions(-) delete mode 100644 pkg/bloomgateway/sharding.go diff --git a/pkg/bloomgateway/bloomgateway.go b/pkg/bloomgateway/bloomgateway.go index 17bd9b395b9a9..abecbf6773fd3 100644 --- a/pkg/bloomgateway/bloomgateway.go +++ b/pkg/bloomgateway/bloomgateway.go @@ -177,8 +177,6 @@ type Gateway struct { activeUsers *util.ActiveUsersCleanupService bloomStore bloomshipper.Store - sharding ShardingStrategy - pendingTasks *pendingTasks serviceMngr *services.Manager @@ -196,12 +194,11 @@ func (l *fixedQueueLimits) MaxConsumers(_ string, _ int) int { } // New returns a new instance of the Bloom Gateway. -func New(cfg Config, schemaCfg config.SchemaConfig, storageCfg storage.Config, overrides Limits, shardingStrategy ShardingStrategy, cm storage.ClientMetrics, logger log.Logger, reg prometheus.Registerer) (*Gateway, error) { +func New(cfg Config, schemaCfg config.SchemaConfig, storageCfg storage.Config, overrides Limits, cm storage.ClientMetrics, logger log.Logger, reg prometheus.Registerer) (*Gateway, error) { g := &Gateway{ cfg: cfg, logger: logger, metrics: newMetrics(reg, constants.Loki, metricsSubsystem), - sharding: shardingStrategy, pendingTasks: makePendingTasks(pendingTasksInitialCap), workerConfig: workerConfig{ maxItems: 100, diff --git a/pkg/bloomgateway/bloomgateway_test.go b/pkg/bloomgateway/bloomgateway_test.go index 329c7b38a33b5..1b1c8b7d56a38 100644 --- a/pkg/bloomgateway/bloomgateway_test.go +++ b/pkg/bloomgateway/bloomgateway_test.go @@ -45,8 +45,6 @@ func newLimits() *validation.Overrides { } func TestBloomGateway_StartStopService(t *testing.T) { - - ss := NewNoopStrategy() logger := log.NewNopLogger() reg := prometheus.NewRegistry() limits := newLimits() @@ -96,7 +94,7 @@ func TestBloomGateway_StartStopService(t *testing.T) { MaxOutstandingPerTenant: 1024, } - gw, err := New(cfg, schemaCfg, storageCfg, limits, ss, cm, logger, reg) + gw, err := New(cfg, schemaCfg, storageCfg, limits, cm, logger, reg) require.NoError(t, err) err = services.StartAndAwaitRunning(context.Background(), gw) @@ -113,8 +111,6 @@ func TestBloomGateway_StartStopService(t *testing.T) { func TestBloomGateway_FilterChunkRefs(t *testing.T) { tenantID := "test" - - ss := NewNoopStrategy() logger := log.NewLogfmtLogger(os.Stderr) reg := prometheus.NewRegistry() limits := newLimits() @@ -165,7 +161,7 @@ func TestBloomGateway_FilterChunkRefs(t *testing.T) { t.Run("shipper error is propagated", func(t *testing.T) { reg := prometheus.NewRegistry() - gw, err := New(cfg, schemaCfg, storageCfg, limits, ss, cm, logger, reg) + gw, err := New(cfg, schemaCfg, storageCfg, limits, cm, logger, reg) require.NoError(t, err) now := mktime("2023-10-03 10:00") @@ -212,7 +208,7 @@ func TestBloomGateway_FilterChunkRefs(t *testing.T) { t.Run("request cancellation does not result in channel locking", func(t *testing.T) { reg := prometheus.NewRegistry() - gw, err := New(cfg, schemaCfg, storageCfg, limits, ss, cm, logger, reg) + gw, err := New(cfg, schemaCfg, storageCfg, limits, cm, logger, reg) require.NoError(t, err) now := mktime("2024-01-25 10:00") @@ -259,7 +255,7 @@ func TestBloomGateway_FilterChunkRefs(t *testing.T) { t.Run("returns unfiltered chunk refs if no filters provided", func(t *testing.T) { reg := prometheus.NewRegistry() - gw, err := New(cfg, schemaCfg, storageCfg, limits, ss, cm, logger, reg) + gw, err := New(cfg, schemaCfg, storageCfg, limits, cm, logger, reg) require.NoError(t, err) err = services.StartAndAwaitRunning(context.Background(), gw) @@ -304,7 +300,7 @@ func TestBloomGateway_FilterChunkRefs(t *testing.T) { t.Run("gateway tracks active users", func(t *testing.T) { reg := prometheus.NewRegistry() - gw, err := New(cfg, schemaCfg, storageCfg, limits, ss, cm, logger, reg) + gw, err := New(cfg, schemaCfg, storageCfg, limits, cm, logger, reg) require.NoError(t, err) err = services.StartAndAwaitRunning(context.Background(), gw) @@ -344,7 +340,7 @@ func TestBloomGateway_FilterChunkRefs(t *testing.T) { t.Run("use fuse queriers to filter chunks", func(t *testing.T) { reg := prometheus.NewRegistry() - gw, err := New(cfg, schemaCfg, storageCfg, limits, ss, cm, logger, reg) + gw, err := New(cfg, schemaCfg, storageCfg, limits, cm, logger, reg) require.NoError(t, err) now := mktime("2023-10-03 10:00") diff --git a/pkg/bloomgateway/client.go b/pkg/bloomgateway/client.go index 6453987b91683..9a75e4e87c26b 100644 --- a/pkg/bloomgateway/client.go +++ b/pkg/bloomgateway/client.go @@ -36,6 +36,10 @@ import ( ) var ( + // BlocksOwnerRead is the operation used to check the authoritative owners of a block + // (replicas included) that are available for queries (a bloom gateway is available for + // queries only when ACTIVE). + BlocksOwnerRead = ring.NewOp([]ring.InstanceState{ring.ACTIVE}, nil) // groupedChunksRefPool pooling slice of logproto.GroupedChunkRefs [64, 128, 256, ..., 65536] groupedChunksRefPool = queue.NewSlicePool[*logproto.GroupedChunkRefs](1<<6, 1<<16, 2) // ringGetBuffersPool pooling for ringGetBuffers to avoid calling ring.MakeBuffersForGet() for each request @@ -226,15 +230,16 @@ func (c *GatewayClient) FilterChunks(ctx context.Context, tenant string, from, t } subRing := GetShuffleShardingSubring(c.ring, tenant, c.limits) - rs, err := subRing.GetAllHealthy(BlocksRead) + rs, err := subRing.GetAllHealthy(BlocksOwnerRead) if err != nil { return nil, errors.Wrap(err, "bloom gateway get healthy instances") } - streamsByInst, err := c.groupFingerprintsByServer(groups, subRing, rs.Instances) + servers, err := serverAddressesWithTokenRanges(subRing, rs.Instances) if err != nil { return nil, err } + streamsByInst := groupFingerprintsByServer(groups, servers) filteredChunkRefs := groupedChunksRefPool.Get(len(groups)) defer groupedChunksRefPool.Put(filteredChunkRefs) @@ -286,13 +291,9 @@ func (c *GatewayClient) doForAddrs(addrs []string, fn func(logproto.BloomGateway return err } -func (c *GatewayClient) groupFingerprintsByServer(groups []*logproto.GroupedChunkRefs, subRing ring.ReadRing, instances []ring.InstanceDesc) ([]instanceWithFingerprints, error) { - servers, err := serverAddressesWithTokenRanges(subRing, instances) - if err != nil { - return nil, err - } +func groupFingerprintsByServer(groups []*logproto.GroupedChunkRefs, servers []addrsWithTokenRange) []instanceWithFingerprints { boundedFingerprints := partitionFingerprintsByAddresses(groups, servers) - return groupByInstance(boundedFingerprints), nil + return groupByInstance(boundedFingerprints) } func serverAddressesWithTokenRanges(subRing ring.ReadRing, instances []ring.InstanceDesc) ([]addrsWithTokenRange, error) { @@ -303,7 +304,7 @@ func serverAddressesWithTokenRanges(subRing ring.ReadRing, instances []ring.Inst for it.Next() { // We can use on of the tokens from the token range // to obtain all addresses for that token. - rs, err := subRing.Get(it.At().MaxToken, BlocksRead, bufDescs, bufHosts, bufZones) + rs, err := subRing.Get(it.At().MaxToken, BlocksOwnerRead, bufDescs, bufHosts, bufZones) if err != nil { return nil, errors.Wrap(err, "bloom gateway get ring") } @@ -410,3 +411,21 @@ func groupByInstance(boundedFingerprints []instanceWithFingerprints) []instanceW return result } + +// GetShuffleShardingSubring returns the subring to be used for a given user. +// This function should be used both by index gateway servers and clients in +// order to guarantee the same logic is used. +func GetShuffleShardingSubring(ring ring.ReadRing, tenantID string, limits Limits) ring.ReadRing { + shardSize := limits.BloomGatewayShardSize(tenantID) + + // A shard size of 0 means shuffle sharding is disabled for this specific user, + // so we just return the full ring so that indexes will be sharded across all index gateways. + // Since we set the shard size to replication factor if shard size is 0, this + // can only happen if both the shard size and the replication factor are set + // to 0. + if shardSize <= 0 { + return ring + } + + return ring.ShuffleShard(tenantID, shardSize) +} diff --git a/pkg/bloomgateway/client_test.go b/pkg/bloomgateway/client_test.go index e59fff2306ab9..b1716de8150ea 100644 --- a/pkg/bloomgateway/client_test.go +++ b/pkg/bloomgateway/client_test.go @@ -207,19 +207,6 @@ func TestBloomGatewayClient_ServerAddressesWithTokenRanges(t *testing.T) { } func TestBloomGatewayClient_GroupFingerprintsByServer(t *testing.T) { - - logger := log.NewNopLogger() - reg := prometheus.NewRegistry() - - l, err := validation.NewOverrides(validation.Limits{BloomGatewayShardSize: 1}, nil) - require.NoError(t, err) - - cfg := ClientConfig{} - flagext.DefaultValues(&cfg) - - c, err := NewClient(cfg, nil, l, reg, logger, "loki", nil, false) - require.NoError(t, err) - instances := []ring.InstanceDesc{ {Id: "instance-1", Addr: "10.0.0.1", Tokens: []uint32{2146405214, 1029997044, 678878693}}, {Id: "instance-2", Addr: "10.0.0.2", Tokens: []uint32{296463531, 1697323986, 800258284}}, @@ -339,8 +326,9 @@ func TestBloomGatewayClient_GroupFingerprintsByServer(t *testing.T) { return tc.chunks[i].Fingerprint < tc.chunks[j].Fingerprint }) - res, err := c.groupFingerprintsByServer(tc.chunks, subRing, instances) + servers, err := serverAddressesWithTokenRanges(subRing, instances) require.NoError(t, err) + res := groupFingerprintsByServer(tc.chunks, servers) require.Equal(t, tc.expected, res) }) } diff --git a/pkg/bloomgateway/sharding.go b/pkg/bloomgateway/sharding.go deleted file mode 100644 index 5dfb9f11732a0..0000000000000 --- a/pkg/bloomgateway/sharding.go +++ /dev/null @@ -1,156 +0,0 @@ -package bloomgateway - -import ( - "context" - - "github.com/go-kit/log" - "github.com/grafana/dskit/ring" - - util_ring "github.com/grafana/loki/pkg/util/ring" -) - -// TODO(chaudum): Replace this placeholder with actual BlockRef struct. -type BlockRef struct { - FromFp, ThroughFp uint64 - FromTs, ThroughTs int64 -} - -var ( - // BlocksOwnerSync is the operation used to check the authoritative owners of a block - // (replicas included). - BlocksOwnerSync = ring.NewOp([]ring.InstanceState{ring.JOINING, ring.ACTIVE, ring.LEAVING}, nil) - - // BlocksOwnerRead is the operation used to check the authoritative owners of a block - // (replicas included) that are available for queries (a bloom gateway is available for - // queries only when ACTIVE). - BlocksOwnerRead = ring.NewOp([]ring.InstanceState{ring.ACTIVE}, nil) - - // BlocksRead is the operation run by the querier to query blocks via the bloom gateway. - BlocksRead = ring.NewOp([]ring.InstanceState{ring.ACTIVE}, func(s ring.InstanceState) bool { - // Blocks can only be queried from ACTIVE instances. However, if the block belongs to - // a non-active instance, then we should extend the replication set and try to query it - // from the next ACTIVE instance in the ring (which is expected to have it because a - // bloom gateway keeps their previously owned blocks until new owners are ACTIVE). - return s != ring.ACTIVE - }) -) - -type ShardingStrategy interface { - // FilterTenants whose indexes should be loaded by the index gateway. - // Returns the list of user IDs that should be synced by the index gateway. - FilterTenants(ctx context.Context, tenantIDs []string) ([]string, error) - FilterBlocks(ctx context.Context, tenantID string, blockRefs []BlockRef) ([]BlockRef, error) -} - -type ShuffleShardingStrategy struct { - util_ring.TenantSharding - r ring.ReadRing - ringLifeCycler *ring.BasicLifecycler - logger log.Logger -} - -func NewShuffleShardingStrategy(r ring.ReadRing, ringLifecycler *ring.BasicLifecycler, limits Limits, logger log.Logger) *ShuffleShardingStrategy { - return &ShuffleShardingStrategy{ - TenantSharding: util_ring.NewTenantShuffleSharding(r, ringLifecycler, limits.BloomGatewayShardSize), - ringLifeCycler: ringLifecycler, - logger: logger, - } -} - -// FilterTenants implements ShardingStrategy. -func (s *ShuffleShardingStrategy) FilterTenants(_ context.Context, tenantIDs []string) ([]string, error) { - // As a protection, ensure the bloom gateway instance is healthy in the ring. It could also be missing - // in the ring if it was failing to heartbeat the ring and it got remove from another healthy bloom gateway - // instance, because of the auto-forget feature. - if set, err := s.r.GetAllHealthy(BlocksOwnerSync); err != nil { - return nil, err - } else if !set.Includes(s.ringLifeCycler.GetInstanceID()) { - return nil, errGatewayUnhealthy - } - - var filteredIDs []string - - for _, tenantID := range tenantIDs { - // Include the user only if it belongs to this bloom gateway shard. - if s.OwnsTenant(tenantID) { - filteredIDs = append(filteredIDs, tenantID) - } - } - - return filteredIDs, nil -} - -// nolint:revive -func getBucket(rangeMin, rangeMax, pos uint64) int { - return 0 -} - -// FilterBlocks implements ShardingStrategy. -func (s *ShuffleShardingStrategy) FilterBlocks(_ context.Context, tenantID string, blockRefs []BlockRef) ([]BlockRef, error) { - if !s.OwnsTenant(tenantID) { - return nil, nil - } - - filteredBlockRefs := make([]BlockRef, 0, len(blockRefs)) - - tenantRing := s.GetTenantSubRing(tenantID) - - fpSharding := util_ring.NewFingerprintShuffleSharding(tenantRing, s.ringLifeCycler, BlocksOwnerSync) - for _, blockRef := range blockRefs { - owns, err := fpSharding.OwnsFingerprint(blockRef.FromFp) - if err != nil { - return nil, err - } - if owns { - filteredBlockRefs = append(filteredBlockRefs, blockRef) - continue - } - - owns, err = fpSharding.OwnsFingerprint(blockRef.ThroughFp) - if err != nil { - return nil, err - } - if owns { - filteredBlockRefs = append(filteredBlockRefs, blockRef) - continue - } - } - - return filteredBlockRefs, nil -} - -// GetShuffleShardingSubring returns the subring to be used for a given user. -// This function should be used both by index gateway servers and clients in -// order to guarantee the same logic is used. -func GetShuffleShardingSubring(ring ring.ReadRing, tenantID string, limits Limits) ring.ReadRing { - shardSize := limits.BloomGatewayShardSize(tenantID) - - // A shard size of 0 means shuffle sharding is disabled for this specific user, - // so we just return the full ring so that indexes will be sharded across all index gateways. - // Since we set the shard size to replication factor if shard size is 0, this - // can only happen if both the shard size and the replication factor are set - // to 0. - if shardSize <= 0 { - return ring - } - - return ring.ShuffleShard(tenantID, shardSize) -} - -// NoopStrategy is an implementation of the ShardingStrategy that does not -// filter anything. -type NoopStrategy struct{} - -func NewNoopStrategy() *NoopStrategy { - return &NoopStrategy{} -} - -// FilterTenants implements ShardingStrategy. -func (s *NoopStrategy) FilterTenants(_ context.Context, tenantIDs []string) ([]string, error) { - return tenantIDs, nil -} - -// FilterBlocks implements ShardingStrategy. -func (s *NoopStrategy) FilterBlocks(_ context.Context, _ string, blockRefs []BlockRef) ([]BlockRef, error) { - return blockRefs, nil -} diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index 4bf5282746077..592aa12b351cf 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -1271,9 +1271,7 @@ func (t *Loki) addCompactorMiddleware(h http.HandlerFunc) http.Handler { func (t *Loki) initBloomGateway() (services.Service, error) { logger := log.With(util_log.Logger, "component", "bloom-gateway") - shuffleSharding := bloomgateway.NewShuffleShardingStrategy(t.bloomGatewayRingManager.Ring, t.bloomGatewayRingManager.RingLifecycler, t.Overrides, logger) - - gateway, err := bloomgateway.New(t.Cfg.BloomGateway, t.Cfg.SchemaConfig, t.Cfg.StorageConfig, t.Overrides, shuffleSharding, t.clientMetrics, logger, prometheus.DefaultRegisterer) + gateway, err := bloomgateway.New(t.Cfg.BloomGateway, t.Cfg.SchemaConfig, t.Cfg.StorageConfig, t.Overrides, t.clientMetrics, logger, prometheus.DefaultRegisterer) if err != nil { return nil, err } From ddaa497554dd1f9b7e20713384c034d66f024f42 Mon Sep 17 00:00:00 2001 From: Karsten Jeschkies Date: Fri, 9 Feb 2024 14:03:57 +0100 Subject: [PATCH 09/22] Prettify concat and downstream expressions. (#11877) **What this PR does / why we need it**: Prettifying `ConcatSampleExpr` et al will simplify debugging the shard mapper. **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [x] 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) --- pkg/logql/downstream.go | 112 ++++++++++++++++++++++-- pkg/logql/downstream_test.go | 141 ++++++++++++++++++++++++++++++ pkg/logql/shardmapper_test.go | 29 ++++++ pkg/logql/syntax/prettier.go | 64 +++++++------- pkg/logql/syntax/prettier_test.go | 8 +- 5 files changed, 313 insertions(+), 41 deletions(-) diff --git a/pkg/logql/downstream.go b/pkg/logql/downstream.go index e29b47054fea6..33d945f11b923 100644 --- a/pkg/logql/downstream.go +++ b/pkg/logql/downstream.go @@ -83,6 +83,29 @@ func (d DownstreamSampleExpr) String() string { return fmt.Sprintf("downstream<%s, shard=%s>", d.SampleExpr.String(), d.shard) } +// The DownstreamSampleExpr is not part of LogQL. In the prettified version it's +// represented as e.g. `downstream` +func (d DownstreamSampleExpr) Pretty(level int) string { + s := syntax.Indent(level) + if !syntax.NeedSplit(d) { + return s + d.String() + } + + s += "downstream<\n" + + s += d.SampleExpr.Pretty(level + 1) + s += ",\n" + s += syntax.Indent(level+1) + "shard=" + if d.shard != nil { + s += d.shard.String() + "\n" + } else { + s += "nil\n" + } + + s += syntax.Indent(level) + ">" + return s +} + // DownstreamLogSelectorExpr is a LogSelectorExpr which signals downstream computation type DownstreamLogSelectorExpr struct { shard *astmapper.ShardAnnotation @@ -93,6 +116,29 @@ func (d DownstreamLogSelectorExpr) String() string { return fmt.Sprintf("downstream<%s, shard=%s>", d.LogSelectorExpr.String(), d.shard) } +// The DownstreamLogSelectorExpr is not part of LogQL. In the prettified version it's +// represented as e.g. `downstream<{foo="bar"} |= "error", shard=1_of_3>` +func (d DownstreamLogSelectorExpr) Pretty(level int) string { + s := syntax.Indent(level) + if !syntax.NeedSplit(d) { + return s + d.String() + } + + s += "downstream<\n" + + s += d.LogSelectorExpr.Pretty(level + 1) + s += ",\n" + s += syntax.Indent(level+1) + "shard=" + if d.shard != nil { + s += d.shard.String() + "\n" + } else { + s += "nil\n" + } + + s += syntax.Indent(level) + ">" + return s +} + func (d DownstreamSampleExpr) Walk(f syntax.WalkFn) { f(d) } var defaultMaxDepth = 4 @@ -105,7 +151,7 @@ type ConcatSampleExpr struct { next *ConcatSampleExpr } -func (c ConcatSampleExpr) String() string { +func (c *ConcatSampleExpr) String() string { if c.next == nil { return c.DownstreamSampleExpr.String() } @@ -115,7 +161,7 @@ func (c ConcatSampleExpr) String() string { // in order to not display huge queries with thousands of shards, // we can limit the number of stringified subqueries. -func (c ConcatSampleExpr) string(maxDepth int) string { +func (c *ConcatSampleExpr) string(maxDepth int) string { if c.next == nil { return c.DownstreamSampleExpr.String() } @@ -125,18 +171,46 @@ func (c ConcatSampleExpr) string(maxDepth int) string { return fmt.Sprintf("%s ++ %s", c.DownstreamSampleExpr.String(), c.next.string(maxDepth-1)) } -func (c ConcatSampleExpr) Walk(f syntax.WalkFn) { +func (c *ConcatSampleExpr) Walk(f syntax.WalkFn) { f(c) f(c.next) } +// ConcatSampleExpr has no LogQL repretenstation. It is expressed in in the +// prettified version as e.g. `concat(downstream ++ )` +func (c *ConcatSampleExpr) Pretty(level int) string { + s := syntax.Indent(level) + if !syntax.NeedSplit(c) { + return s + c.String() + } + + s += "concat(\n" + + head := c + for i := 0; i < defaultMaxDepth && head != nil; i++ { + if i > 0 { + s += syntax.Indent(level+1) + "++\n" + } + s += head.DownstreamSampleExpr.Pretty(level + 1) + s += "\n" + head = head.next + } + // There are more downstream samples... + if head != nil { + s += syntax.Indent(level+1) + "++ ...\n" + } + s += syntax.Indent(level) + ")" + + return s +} + // ConcatLogSelectorExpr is an expr for concatenating multiple LogSelectorExpr type ConcatLogSelectorExpr struct { DownstreamLogSelectorExpr next *ConcatLogSelectorExpr } -func (c ConcatLogSelectorExpr) String() string { +func (c *ConcatLogSelectorExpr) String() string { if c.next == nil { return c.DownstreamLogSelectorExpr.String() } @@ -146,7 +220,7 @@ func (c ConcatLogSelectorExpr) String() string { // in order to not display huge queries with thousands of shards, // we can limit the number of stringified subqueries. -func (c ConcatLogSelectorExpr) string(maxDepth int) string { +func (c *ConcatLogSelectorExpr) string(maxDepth int) string { if c.next == nil { return c.DownstreamLogSelectorExpr.String() } @@ -156,6 +230,34 @@ func (c ConcatLogSelectorExpr) string(maxDepth int) string { return fmt.Sprintf("%s ++ %s", c.DownstreamLogSelectorExpr.String(), c.next.string(maxDepth-1)) } +// ConcatLogSelectorExpr has no representation in LogQL. Its prettified version +// is e.g. `concat(downstream<{foo="bar"} |= "error", shard=1_of_3>)` +func (c *ConcatLogSelectorExpr) Pretty(level int) string { + s := syntax.Indent(level) + if !syntax.NeedSplit(c) { + return s + c.String() + } + + s += "concat(\n" + + head := c + for i := 0; i < defaultMaxDepth && head != nil; i++ { + if i > 0 { + s += syntax.Indent(level+1) + "++\n" + } + s += head.DownstreamLogSelectorExpr.Pretty(level + 1) + s += "\n" + head = head.next + } + // There are more downstream samples... + if head != nil { + s += syntax.Indent(level+1) + "++ ...\n" + } + s += ")" + + return s +} + // QuantileSketchEvalExpr evaluates a quantile sketch to the actual quantile. type QuantileSketchEvalExpr struct { syntax.SampleExpr diff --git a/pkg/logql/downstream_test.go b/pkg/logql/downstream_test.go index 426722a554594..ec5f3170468d0 100644 --- a/pkg/logql/downstream_test.go +++ b/pkg/logql/downstream_test.go @@ -8,12 +8,14 @@ import ( "github.com/go-kit/log" "github.com/grafana/dskit/user" + "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/promql" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/grafana/loki/pkg/logproto" "github.com/grafana/loki/pkg/logql/syntax" + "github.com/grafana/loki/pkg/querier/astmapper" ) var nilShardMetrics = NewShardMapperMetrics(nil) @@ -543,3 +545,142 @@ func relativeError(t *testing.T, expected, actual promql.Matrix, alpha float64) require.InEpsilonSlice(t, e, a, alpha) } } + +func TestFormat_ShardedExpr(t *testing.T) { + oldMax := syntax.MaxCharsPerLine + syntax.MaxCharsPerLine = 20 + + oldDefaultDepth := defaultMaxDepth + defaultMaxDepth = 2 + defer func() { + syntax.MaxCharsPerLine = oldMax + defaultMaxDepth = oldDefaultDepth + }() + + cases := []struct { + name string + in syntax.Expr + exp string + }{ + { + name: "ConcatSampleExpr", + in: &ConcatSampleExpr{ + DownstreamSampleExpr: DownstreamSampleExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 0, + Of: 3, + }, + SampleExpr: &syntax.RangeAggregationExpr{ + Operation: syntax.OpRangeTypeRate, + Left: &syntax.LogRange{ + Left: &syntax.MatchersExpr{ + Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}, + }, + Interval: time.Minute, + }, + }, + }, + next: &ConcatSampleExpr{ + DownstreamSampleExpr: DownstreamSampleExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 1, + Of: 3, + }, + SampleExpr: &syntax.RangeAggregationExpr{ + Operation: syntax.OpRangeTypeRate, + Left: &syntax.LogRange{ + Left: &syntax.MatchersExpr{ + Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}, + }, + Interval: time.Minute, + }, + }, + }, + next: &ConcatSampleExpr{ + DownstreamSampleExpr: DownstreamSampleExpr{ + shard: &astmapper.ShardAnnotation{ + Shard: 1, + Of: 3, + }, + SampleExpr: &syntax.RangeAggregationExpr{ + Operation: syntax.OpRangeTypeRate, + Left: &syntax.LogRange{ + Left: &syntax.MatchersExpr{ + Mts: []*labels.Matcher{mustNewMatcher(labels.MatchEqual, "foo", "bar")}, + }, + Interval: time.Minute, + }, + }, + }, + next: nil, + }, + }, + }, + exp: `concat( + downstream< + rate( + {foo="bar"} [1m] + ), + shard=0_of_3 + > + ++ + downstream< + rate( + {foo="bar"} [1m] + ), + shard=1_of_3 + > + ++ ... +)`, + }, + } + + for _, c := range cases { + t.Run(c.name, func(t *testing.T) { + got := syntax.Prettify(c.in) + assert.Equal(t, c.exp, got) + }) + } +} + +func TestPrettierWithoutShards(t *testing.T) { + q := `((quantile_over_time(0.5,{foo="bar"} | json | unwrap bytes[1d]) by (cluster) > 42) and (count by (cluster)(max_over_time({foo="baz"} |= "error" | json | unwrap bytes[1d]) by (cluster,namespace)) > 10))` + e := syntax.MustParseExpr(q) + + mapper := NewShardMapper(ConstantShards(4), nilShardMetrics, []string{}) + _, _, mapped, err := mapper.Parse(e) + require.NoError(t, err) + got := syntax.Prettify(mapped) + expected := ` downstream> + > + 42 +and + count by (cluster)( + max by (cluster, namespace)( + concat( + downstream< + max_over_time({foo="baz"} |= "error" | json | unwrap bytes[1d]) by (cluster,namespace), + shard=0_of_4 + > + ++ + downstream< + max_over_time({foo="baz"} |= "error" | json | unwrap bytes[1d]) by (cluster,namespace), + shard=1_of_4 + > + ++ + downstream< + max_over_time({foo="baz"} |= "error" | json | unwrap bytes[1d]) by (cluster,namespace), + shard=2_of_4 + > + ++ + downstream< + max_over_time({foo="baz"} |= "error" | json | unwrap bytes[1d]) by (cluster,namespace), + shard=3_of_4 + > + ) + ) + ) + > + 10` + assert.Equal(t, expected, got) +} diff --git a/pkg/logql/shardmapper_test.go b/pkg/logql/shardmapper_test.go index 96955109a9413..0e345291eed3b 100644 --- a/pkg/logql/shardmapper_test.go +++ b/pkg/logql/shardmapper_test.go @@ -1598,3 +1598,32 @@ func TestStringTrimming(t *testing.T) { func float64p(v float64) *float64 { return &v } + +func TestShardTopk(t *testing.T) { + expr := `topk( + 10, + sum by (ip) ( + sum_over_time({job="foo"} | json | unwrap bytes(bytes)[1m]) + ) + )` + m := NewShardMapper(ConstantShards(5), nilShardMetrics, []string{ShardQuantileOverTime}) + _, _, mappedExpr, err := m.Parse(syntax.MustParseExpr(expr)) + require.NoError(t, err) + + expected := `topk( + 10, + sum by (ip)( + concat( + downstream + ++ + downstream + ++ + downstream + ++ + downstream + ++ ... + ) + ) +)` + require.Equal(t, expected, mappedExpr.Pretty(0)) +} diff --git a/pkg/logql/syntax/prettier.go b/pkg/logql/syntax/prettier.go index cf346e26c562f..1b407453858f7 100644 --- a/pkg/logql/syntax/prettier.go +++ b/pkg/logql/syntax/prettier.go @@ -35,8 +35,8 @@ import ( // var ( - // maxCharsPerLine is used to qualify whether some LogQL expressions are worth `splitting` into new lines. - maxCharsPerLine = 100 + // MaxCharsPerLine is used to qualify whether some LogQL expressions are worth `splitting` into new lines. + MaxCharsPerLine = 100 ) func Prettify(e Expr) string { @@ -51,8 +51,8 @@ func (e *MatchersExpr) Pretty(level int) string { // e.g: `{foo="bar"} | logfmt | level="error"` // Here, left = `{foo="bar"}` and multistages would collection of each stage in pipeline, here `logfmt` and `level="error"` func (e *PipelineExpr) Pretty(level int) string { - if !needSplit(e) { - return indent(level) + e.String() + if !NeedSplit(e) { + return Indent(level) + e.String() } s := fmt.Sprintf("%s\n", e.Left.Pretty(level)) @@ -73,8 +73,8 @@ func (e *PipelineExpr) Pretty(level int) string { // e.g: `|= "error" != "memcache" |= ip("192.168.0.1")` // NOTE: here `ip` is Op in this expression. func (e *LineFilterExpr) Pretty(level int) string { - if !needSplit(e) { - return indent(level) + e.String() + if !NeedSplit(e) { + return Indent(level) + e.String() } var s string @@ -90,7 +90,7 @@ func (e *LineFilterExpr) Pretty(level int) string { s += "\n" } - s += indent(level) + s += Indent(level) // We re-use LineFilterExpr's String() implementation to avoid duplication. // We create new LineFilterExpr without `Left`. @@ -153,7 +153,7 @@ func (e *LogfmtExpressionParser) Pretty(level int) string { // e.g: sum_over_time({foo="bar"} | logfmt | unwrap bytes_processed [5m]) func (e *UnwrapExpr) Pretty(level int) string { - s := indent(level) + s := Indent(level) if e.Operation != "" { s += fmt.Sprintf("%s %s %s(%s)", OpPipe, OpUnwrap, e.Operation, e.Identifier) @@ -161,7 +161,7 @@ func (e *UnwrapExpr) Pretty(level int) string { s += fmt.Sprintf("%s %s %s", OpPipe, OpUnwrap, e.Identifier) } for _, f := range e.PostFilters { - s += fmt.Sprintf("\n%s%s %s", indent(level), OpPipe, f) + s += fmt.Sprintf("\n%s%s %s", Indent(level), OpPipe, f) } return s } @@ -200,8 +200,8 @@ func (e *OffsetExpr) Pretty(_ int) string { // e.g: count_over_time({foo="bar"}[5m]) func (e *RangeAggregationExpr) Pretty(level int) string { - s := indent(level) - if !needSplit(e) { + s := Indent(level) + if !NeedSplit(e) { return s + e.String() } @@ -211,13 +211,13 @@ func (e *RangeAggregationExpr) Pretty(level int) string { // print args to the function. if e.Params != nil { - s = fmt.Sprintf("%s%s%s,", s, indent(level+1), fmt.Sprint(*e.Params)) + s = fmt.Sprintf("%s%s%s,", s, Indent(level+1), fmt.Sprint(*e.Params)) s += "\n" } s += e.Left.Pretty(level + 1) - s += "\n" + indent(level) + ")" + s += "\n" + Indent(level) + ")" if e.Grouping != nil { s += e.Grouping.Pretty(level) @@ -236,9 +236,9 @@ func (e *RangeAggregationExpr) Pretty(level int) string { // - vector on which aggregation is done. // [without|by (