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 (", ">=", "<", "<=" (comparison)
func (e *BinOpExpr) Pretty(level int) string {
- s := indent(level)
- if !needSplit(e) {
+ s := Indent(level)
+ if !NeedSplit(e) {
return s + e.String()
}
s = e.SampleExpr.Pretty(level+1) + "\n"
op := formatBinaryOp(e.Op, e.Opts)
- s += indent(level) + op + "\n"
+ s += Indent(level) + op + "\n"
s += e.RHS.Pretty(level + 1)
return s
@@ -306,9 +306,9 @@ func (e *LiteralExpr) Pretty(level int) string {
// e.g: label_replace(rate({job="api-server",service="a:c"}[5m]), "foo", "$1", "service", "(.*):.*")
func (e *LabelReplaceExpr) Pretty(level int) string {
- s := indent(level)
+ s := Indent(level)
- if !needSplit(e) {
+ if !NeedSplit(e) {
return s + e.String()
}
@@ -318,10 +318,10 @@ func (e *LabelReplaceExpr) Pretty(level int) string {
params := []string{
e.Left.Pretty(level + 1),
- indent(level+1) + strconv.Quote(e.Dst),
- indent(level+1) + strconv.Quote(e.Replacement),
- indent(level+1) + strconv.Quote(e.Src),
- indent(level+1) + strconv.Quote(e.Regex),
+ Indent(level+1) + strconv.Quote(e.Dst),
+ Indent(level+1) + strconv.Quote(e.Replacement),
+ Indent(level+1) + strconv.Quote(e.Src),
+ Indent(level+1) + strconv.Quote(e.Regex),
}
for i, v := range params {
@@ -333,7 +333,7 @@ func (e *LabelReplaceExpr) Pretty(level int) string {
s += "\n"
}
- s += indent(level) + ")"
+ s += Indent(level) + ")"
return s
}
@@ -369,19 +369,19 @@ func (g *Grouping) Pretty(_ int) string {
// Helpers
func commonPrefixIndent(level int, current Expr) string {
- return fmt.Sprintf("%s%s", indent(level), current.String())
+ return fmt.Sprintf("%s%s", Indent(level), current.String())
}
-func needSplit(e Expr) bool {
+func NeedSplit(e Expr) bool {
if e == nil {
return false
}
- return len(e.String()) > maxCharsPerLine
+ return len(e.String()) > MaxCharsPerLine
}
const indentString = " "
-func indent(level int) string {
+func Indent(level int) string {
return strings.Repeat(indentString, level)
}
diff --git a/pkg/logql/syntax/prettier_test.go b/pkg/logql/syntax/prettier_test.go
index 5532c0e54403f..4a850f1fd2596 100644
--- a/pkg/logql/syntax/prettier_test.go
+++ b/pkg/logql/syntax/prettier_test.go
@@ -8,7 +8,7 @@ import (
)
func TestFormat(t *testing.T) {
- maxCharsPerLine = 20
+ MaxCharsPerLine = 20
cases := []struct {
name string
@@ -108,7 +108,7 @@ func TestFormat(t *testing.T) {
}
func TestFormat_VectorAggregation(t *testing.T) {
- maxCharsPerLine = 20
+ MaxCharsPerLine = 20
cases := []struct {
name string
@@ -147,7 +147,7 @@ func TestFormat_VectorAggregation(t *testing.T) {
}
func TestFormat_LabelReplace(t *testing.T) {
- maxCharsPerLine = 20
+ MaxCharsPerLine = 20
cases := []struct {
name string
@@ -201,7 +201,7 @@ func TestFormat_LabelReplace(t *testing.T) {
}
func TestFormat_BinOp(t *testing.T) {
- maxCharsPerLine = 20
+ MaxCharsPerLine = 20
cases := []struct {
name string
From c1d5ec14fde527a7d4eded041d7ace08a21a57b1 Mon Sep 17 00:00:00 2001
From: Christian Haudum
Date: Fri, 9 Feb 2024 14:07:27 +0100
Subject: [PATCH 10/22] Update metrics of bloomgateway worker (#11907)
**What this PR does / why we need it**:
Update metrics for bloomgatway worker to improve observability of the query path.
```
dequeueDuration : Summary
processDuration : Summary
metasFetched : Summary
blocksFetched : Summary
tasksDequeued : Counter
tasksProcessed : Counter
blockQueryLatency : Histogram
```
Signed-off-by: Christian Haudum
---
pkg/bloomgateway/processor.go | 29 +++++++++---
pkg/bloomgateway/processor_test.go | 15 +++---
pkg/bloomgateway/worker.go | 75 +++++++++++++++++++++++-------
3 files changed, 88 insertions(+), 31 deletions(-)
diff --git a/pkg/bloomgateway/processor.go b/pkg/bloomgateway/processor.go
index 4b5edf9e18762..4fe9c38483cbe 100644
--- a/pkg/bloomgateway/processor.go
+++ b/pkg/bloomgateway/processor.go
@@ -4,6 +4,7 @@ import (
"context"
"math"
"sort"
+ "time"
"github.com/go-kit/log"
"github.com/prometheus/common/model"
@@ -17,16 +18,20 @@ type tasksForBlock struct {
tasks []Task
}
-func newProcessor(store bloomshipper.Store, logger log.Logger) *processor {
+func newProcessor(id string, store bloomshipper.Store, logger log.Logger, metrics *workerMetrics) *processor {
return &processor{
- store: store,
- logger: logger,
+ id: id,
+ store: store,
+ logger: logger,
+ metrics: metrics,
}
}
type processor struct {
- store bloomshipper.Store
- logger log.Logger
+ id string
+ store bloomshipper.Store
+ logger log.Logger
+ metrics *workerMetrics
}
func (p *processor) run(ctx context.Context, tasks []Task) error {
@@ -58,6 +63,8 @@ func (p *processor) processTasks(ctx context.Context, tenant string, interval bl
if err != nil {
return err
}
+ p.metrics.metasFetched.WithLabelValues(p.id).Observe(float64(len(metas)))
+
blocksRefs := bloomshipper.BlocksForMetas(metas, interval, keyspaces)
return p.processBlocks(ctx, partition(tasks, blocksRefs))
}
@@ -72,6 +79,7 @@ func (p *processor) processBlocks(ctx context.Context, data []tasksForBlock) err
if err != nil {
return err
}
+ p.metrics.metasFetched.WithLabelValues(p.id).Observe(float64(len(bqs)))
blockIter := v1.NewSliceIter(bqs)
@@ -109,7 +117,16 @@ func (p *processor) processBlock(_ context.Context, blockQuerier *v1.BlockQuerie
}
fq := blockQuerier.Fuse(iters)
- return fq.Run()
+
+ start := time.Now()
+ err = fq.Run()
+ if err != nil {
+ p.metrics.blockQueryLatency.WithLabelValues(p.id, labelFailure).Observe(time.Since(start).Seconds())
+ } else {
+ p.metrics.blockQueryLatency.WithLabelValues(p.id, labelSuccess).Observe(time.Since(start).Seconds())
+ }
+
+ return err
}
// getFirstLast returns the first and last item of a fingerprint slice
diff --git a/pkg/bloomgateway/processor_test.go b/pkg/bloomgateway/processor_test.go
index b619ec56f7280..c4c8f8457b3a1 100644
--- a/pkg/bloomgateway/processor_test.go
+++ b/pkg/bloomgateway/processor_test.go
@@ -9,12 +9,14 @@ import (
"github.com/go-kit/log"
"github.com/pkg/errors"
+ "github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/common/model"
"github.com/stretchr/testify/require"
"go.uber.org/atomic"
"github.com/grafana/loki/pkg/logql/syntax"
"github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper"
+ "github.com/grafana/loki/pkg/util/constants"
)
var _ bloomshipper.Store = &dummyStore{}
@@ -92,13 +94,13 @@ func TestProcessor(t *testing.T) {
ctx := context.Background()
tenant := "fake"
now := mktime("2024-01-27 12:00")
+ metrics := newWorkerMetrics(prometheus.NewPedanticRegistry(), constants.Loki, "bloom_gatway")
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: newMockBloomStore(queriers, metas),
- logger: log.NewNopLogger(),
- }
+
+ mockStore := newMockBloomStore(queriers, metas)
+ p := newProcessor("worker", mockStore, log.NewNopLogger(), metrics)
chunkRefs := createQueryInputFromBlockData(t, tenant, data, 10)
swb := seriesWithBounds{
@@ -142,10 +144,7 @@ func TestProcessor(t *testing.T) {
mockStore := newMockBloomStore(queriers, metas)
mockStore.err = errors.New("store failed")
- p := &processor{
- store: mockStore,
- logger: log.NewNopLogger(),
- }
+ p := newProcessor("worker", mockStore, log.NewNopLogger(), metrics)
chunkRefs := createQueryInputFromBlockData(t, tenant, data, 10)
swb := seriesWithBounds{
diff --git a/pkg/bloomgateway/worker.go b/pkg/bloomgateway/worker.go
index 3063d8e7d4f26..aed42557241b2 100644
--- a/pkg/bloomgateway/worker.go
+++ b/pkg/bloomgateway/worker.go
@@ -15,37 +15,71 @@ import (
"github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper"
)
+const (
+ labelSuccess = "success"
+ labelFailure = "failure"
+)
+
type workerConfig struct {
maxItems int
}
type workerMetrics struct {
- dequeuedTasks *prometheus.CounterVec
- dequeueErrors *prometheus.CounterVec
- dequeueWaitTime *prometheus.SummaryVec
+ dequeueDuration *prometheus.SummaryVec
+ processDuration *prometheus.SummaryVec
+ metasFetched *prometheus.SummaryVec
+ blocksFetched *prometheus.SummaryVec
+ tasksDequeued *prometheus.CounterVec
+ tasksProcessed *prometheus.CounterVec
+ blockQueryLatency *prometheus.HistogramVec
}
func newWorkerMetrics(registerer prometheus.Registerer, namespace, subsystem string) *workerMetrics {
labels := []string{"worker"}
+ r := promauto.With(registerer)
return &workerMetrics{
- dequeuedTasks: promauto.With(registerer).NewCounterVec(prometheus.CounterOpts{
+ dequeueDuration: r.NewSummaryVec(prometheus.SummaryOpts{
Namespace: namespace,
Subsystem: subsystem,
- Name: "dequeued_tasks_total",
- Help: "Total amount of tasks that the worker dequeued from the bloom query queue",
+ Name: "dequeue_duration_seconds",
+ Help: "Time spent dequeuing tasks from queue in seconds",
}, labels),
- dequeueErrors: promauto.With(registerer).NewCounterVec(prometheus.CounterOpts{
+ processDuration: r.NewSummaryVec(prometheus.SummaryOpts{
+ Namespace: namespace,
+ Subsystem: subsystem,
+ Name: "process_duration_seconds",
+ Help: "Time spent processing tasks in seconds",
+ }, append(labels, "status")),
+ metasFetched: r.NewSummaryVec(prometheus.SummaryOpts{
Namespace: namespace,
Subsystem: subsystem,
- Name: "dequeue_errors_total",
- Help: "Total amount of failed dequeue operations",
+ Name: "metas_fetched",
+ Help: "Amount of metas fetched",
}, labels),
- dequeueWaitTime: promauto.With(registerer).NewSummaryVec(prometheus.SummaryOpts{
+ blocksFetched: r.NewSummaryVec(prometheus.SummaryOpts{
Namespace: namespace,
Subsystem: subsystem,
- Name: "dequeue_wait_time",
- Help: "Time spent waiting for dequeuing tasks from queue",
+ Name: "blocks_fetched",
+ Help: "Amount of blocks fetched",
}, labels),
+ tasksDequeued: r.NewCounterVec(prometheus.CounterOpts{
+ Namespace: namespace,
+ Subsystem: subsystem,
+ Name: "tasks_dequeued_total",
+ Help: "Total amount of tasks that the worker dequeued from the queue",
+ }, append(labels, "status")),
+ tasksProcessed: r.NewCounterVec(prometheus.CounterOpts{
+ Namespace: namespace,
+ Subsystem: subsystem,
+ Name: "tasks_processed_total",
+ Help: "Total amount of tasks that the worker processed",
+ }, append(labels, "status")),
+ blockQueryLatency: r.NewHistogramVec(prometheus.HistogramOpts{
+ Namespace: namespace,
+ Subsystem: subsystem,
+ Name: "block_query_latency",
+ Help: "Time spent running searches against a bloom block",
+ }, append(labels, "status")),
}
}
@@ -89,19 +123,19 @@ 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}
+ p := newProcessor(w.id, w.store, w.logger, w.metrics)
for st := w.State(); st == services.Running || st == services.Stopping; {
taskCtx := context.Background()
- dequeueStart := time.Now()
+ start := time.Now()
items, newIdx, err := w.queue.DequeueMany(taskCtx, idx, w.id, w.cfg.maxItems)
- w.metrics.dequeueWaitTime.WithLabelValues(w.id).Observe(time.Since(dequeueStart).Seconds())
+ w.metrics.dequeueDuration.WithLabelValues(w.id).Observe(time.Since(start).Seconds())
if err != nil {
// We only return an error if the queue is stopped and dequeuing did not yield any items
if err == queue.ErrStopped && len(items) == 0 {
return err
}
- w.metrics.dequeueErrors.WithLabelValues(w.id).Inc()
+ w.metrics.tasksDequeued.WithLabelValues(w.id, labelFailure).Inc()
level.Error(w.logger).Log("msg", "failed to dequeue tasks", "err", err, "items", len(items))
}
idx = newIdx
@@ -110,7 +144,7 @@ func (w *worker) running(_ context.Context) error {
w.queue.ReleaseRequests(items)
continue
}
- w.metrics.dequeuedTasks.WithLabelValues(w.id).Add(float64(len(items)))
+ w.metrics.tasksDequeued.WithLabelValues(w.id, labelSuccess).Add(float64(len(items)))
tasks := make([]Task, 0, len(items))
for _, item := range items {
@@ -125,9 +159,16 @@ func (w *worker) running(_ context.Context) error {
tasks = append(tasks, task)
}
+ start = time.Now()
err = p.run(taskCtx, tasks)
+
if err != nil {
+ w.metrics.processDuration.WithLabelValues(w.id, labelSuccess).Observe(time.Since(start).Seconds())
+ w.metrics.tasksProcessed.WithLabelValues(w.id, labelFailure).Add(float64(len(tasks)))
level.Error(w.logger).Log("msg", "failed to process tasks", "err", err)
+ } else {
+ w.metrics.processDuration.WithLabelValues(w.id, labelSuccess).Observe(time.Since(start).Seconds())
+ w.metrics.tasksProcessed.WithLabelValues(w.id, labelSuccess).Add(float64(len(tasks)))
}
// return dequeued items back to the pool
From 3efd201bffcd02ed230bcf16b9c7153d6cce428d Mon Sep 17 00:00:00 2001
From: Christian Haudum
Date: Fri, 9 Feb 2024 19:28:52 +0100
Subject: [PATCH 11/22] Fix test utility function `MkBasicSeriesWithBlooms`
(#11909)
The utility correctly populated the blooms, but returned incorrect
values for `keys`, which are the indexed keys of a bloom.
The values need to be copied when appending to the result slice.
Additionally, the source for the tokenizer changed to be avoid duplicate
keys that are present in all blooms.
---------
Signed-off-by: Christian Haudum
---
pkg/bloomgateway/bloomgateway_test.go | 33 +++++++++++------
pkg/bloomgateway/util_test.go | 52 ++++++++-------------------
pkg/storage/bloom/v1/archive_test.go | 2 +-
pkg/storage/bloom/v1/builder_test.go | 2 +-
pkg/storage/bloom/v1/fuse_test.go | 12 +++----
pkg/storage/bloom/v1/test_util.go | 39 ++++++++++----------
6 files changed, 66 insertions(+), 74 deletions(-)
diff --git a/pkg/bloomgateway/bloomgateway_test.go b/pkg/bloomgateway/bloomgateway_test.go
index 1b1c8b7d56a38..f07e014b84dc3 100644
--- a/pkg/bloomgateway/bloomgateway_test.go
+++ b/pkg/bloomgateway/bloomgateway_test.go
@@ -3,6 +3,7 @@ package bloomgateway
import (
"context"
"fmt"
+ "math/rand"
"os"
"testing"
"time"
@@ -183,7 +184,7 @@ func TestBloomGateway_FilterChunkRefs(t *testing.T) {
require.NoError(t, err)
})
- chunkRefs := createQueryInputFromBlockData(t, tenantID, data, 10)
+ chunkRefs := createQueryInputFromBlockData(t, tenantID, data, 100)
// saturate workers
// then send additional request
@@ -359,7 +360,7 @@ func TestBloomGateway_FilterChunkRefs(t *testing.T) {
require.NoError(t, err)
})
- chunkRefs := createQueryInputFromBlockData(t, tenantID, data, 100)
+ chunkRefs := createQueryInputFromBlockData(t, tenantID, data, 10)
t.Run("no match - return empty response", func(t *testing.T) {
inputChunkRefs := groupRefs(t, chunkRefs)
@@ -383,27 +384,37 @@ func TestBloomGateway_FilterChunkRefs(t *testing.T) {
t.Run("match - return filtered", func(t *testing.T) {
inputChunkRefs := groupRefs(t, chunkRefs)
- // hack to get indexed key for a specific series
- // the indexed key range for a series is defined as
- // i * keysPerSeries ... i * keysPerSeries + keysPerSeries - 1
- // where i is the nth series in a block
- // fortunately, i is also used as Checksum for the single chunk of a series
- // see mkBasicSeriesWithBlooms() in pkg/storage/bloom/v1/test_util.go
- key := inputChunkRefs[0].Refs[0].Checksum*1000 + 500
+ // Hack to get search string for a specific series
+ // see MkBasicSeriesWithBlooms() in pkg/storage/bloom/v1/test_util.go
+ // each series has 1 chunk
+ // each chunk has multiple strings, from int(fp) to int(nextFp)-1
+ x := rand.Intn(len(inputChunkRefs))
+ fp := inputChunkRefs[x].Fingerprint
+ chks := inputChunkRefs[x].Refs
+ line := fmt.Sprintf("%04x:%04x", int(fp), 0) // first line
+
+ t.Log("x=", x, "fp=", fp, "line=", line)
req := &logproto.FilterChunkRefRequest{
From: now.Add(-8 * time.Hour),
Through: now,
Refs: inputChunkRefs,
Filters: []syntax.LineFilter{
- {Ty: labels.MatchEqual, Match: fmt.Sprintf("series %d", key)},
+ {Ty: labels.MatchEqual, Match: line},
},
}
ctx := user.InjectOrgID(context.Background(), tenantID)
res, err := gw.FilterChunkRefs(ctx, req)
require.NoError(t, err)
+
expectedResponse := &logproto.FilterChunkRefResponse{
- ChunkRefs: inputChunkRefs[:1],
+ ChunkRefs: []*logproto.GroupedChunkRefs{
+ {
+ Fingerprint: fp,
+ Refs: chks,
+ Tenant: tenantID,
+ },
+ },
}
require.Equal(t, expectedResponse, res)
})
diff --git a/pkg/bloomgateway/util_test.go b/pkg/bloomgateway/util_test.go
index 156b05cf4ca41..8fc37f20bac8e 100644
--- a/pkg/bloomgateway/util_test.go
+++ b/pkg/bloomgateway/util_test.go
@@ -293,39 +293,10 @@ func TestPartitionRequest(t *testing.T) {
}
-func createBlockQueriers(t *testing.T, numBlocks int, from, through model.Time, minFp, maxFp model.Fingerprint) ([]*bloomshipper.CloseableBlockQuerier, [][]v1.SeriesWithBloom) {
- t.Helper()
- step := (maxFp - minFp) / model.Fingerprint(numBlocks)
- bqs := make([]*bloomshipper.CloseableBlockQuerier, 0, numBlocks)
- series := make([][]v1.SeriesWithBloom, 0, numBlocks)
- for i := 0; i < numBlocks; i++ {
- fromFp := minFp + (step * model.Fingerprint(i))
- throughFp := fromFp + step - 1
- // last block needs to include maxFp
- if i == numBlocks-1 {
- throughFp = maxFp
- }
- blockQuerier, data := v1.MakeBlockQuerier(t, fromFp, throughFp, from, through)
- bq := &bloomshipper.CloseableBlockQuerier{
- BlockQuerier: blockQuerier,
- BlockRef: bloomshipper.BlockRef{
- Ref: bloomshipper.Ref{
- Bounds: v1.NewBounds(fromFp, throughFp),
- StartTimestamp: from,
- EndTimestamp: through,
- },
- },
- }
- bqs = append(bqs, bq)
- series = append(series, data)
- }
- return bqs, series
-}
-
func createBlocks(t *testing.T, tenant string, n int, from, through model.Time, minFp, maxFp model.Fingerprint) ([]bloomshipper.BlockRef, []bloomshipper.Meta, []*bloomshipper.CloseableBlockQuerier, [][]v1.SeriesWithBloom) {
t.Helper()
- blocks := make([]bloomshipper.BlockRef, 0, n)
+ blockRefs := make([]bloomshipper.BlockRef, 0, n)
metas := make([]bloomshipper.Meta, 0, n)
queriers := make([]*bloomshipper.CloseableBlockQuerier, 0, n)
series := make([][]v1.SeriesWithBloom, 0, n)
@@ -345,7 +316,7 @@ func createBlocks(t *testing.T, tenant string, n int, from, through model.Time,
StartTimestamp: from,
EndTimestamp: through,
}
- block := bloomshipper.BlockRef{
+ blockRef := bloomshipper.BlockRef{
Ref: ref,
}
meta := bloomshipper.Meta{
@@ -353,19 +324,26 @@ func createBlocks(t *testing.T, tenant string, n int, from, through model.Time,
Ref: ref,
},
Tombstones: []bloomshipper.BlockRef{},
- Blocks: []bloomshipper.BlockRef{block},
+ Blocks: []bloomshipper.BlockRef{blockRef},
}
- blockQuerier, data := v1.MakeBlockQuerier(t, fromFp, throughFp, from, through)
+ block, data, _ := v1.MakeBlock(t, n, fromFp, throughFp, from, through)
+ // Printing fingerprints and the log lines of its chunks comes handy for debugging...
+ // for i := range keys {
+ // t.Log(data[i].Series.Fingerprint)
+ // for j := range keys[i] {
+ // t.Log(i, j, string(keys[i][j]))
+ // }
+ // }
querier := &bloomshipper.CloseableBlockQuerier{
- BlockQuerier: blockQuerier,
- BlockRef: block,
+ BlockQuerier: v1.NewBlockQuerier(block),
+ BlockRef: blockRef,
}
queriers = append(queriers, querier)
metas = append(metas, meta)
- blocks = append(blocks, block)
+ blockRefs = append(blockRefs, blockRef)
series = append(series, data)
}
- return blocks, metas, queriers, series
+ return blockRefs, metas, queriers, series
}
func createQueryInputFromBlockData(t *testing.T, tenant string, data [][]v1.SeriesWithBloom, nthSeries int) []*logproto.ChunkRef {
diff --git a/pkg/storage/bloom/v1/archive_test.go b/pkg/storage/bloom/v1/archive_test.go
index 7dac04e4c8f1a..c77fbc69f122b 100644
--- a/pkg/storage/bloom/v1/archive_test.go
+++ b/pkg/storage/bloom/v1/archive_test.go
@@ -18,7 +18,7 @@ func TestArchive(t *testing.T) {
numSeries := 100
numKeysPerSeries := 10000
- data, _ := MkBasicSeriesWithBlooms(numSeries, numKeysPerSeries, 0, 0xffff, 0, 10000)
+ data, _ := MkBasicSeriesWithBlooms(numSeries, numKeysPerSeries, 0x0000, 0xffff, 0, 10000)
builder, err := NewBlockBuilder(
BlockOptions{
diff --git a/pkg/storage/bloom/v1/builder_test.go b/pkg/storage/bloom/v1/builder_test.go
index 1b988871143ae..36db0e1f193ce 100644
--- a/pkg/storage/bloom/v1/builder_test.go
+++ b/pkg/storage/bloom/v1/builder_test.go
@@ -362,7 +362,7 @@ func TestMergeBuilder_Roundtrip(t *testing.T) {
checksum, err := mb.Build(builder)
require.Nil(t, err)
- require.Equal(t, uint32(0xe306ec6e), checksum)
+ require.Equal(t, uint32(0x30712486), checksum)
// ensure the new block contains one copy of all the data
// by comparing it against an iterator over the source data
diff --git a/pkg/storage/bloom/v1/fuse_test.go b/pkg/storage/bloom/v1/fuse_test.go
index be0b6d30ed7df..7b23e1c133361 100644
--- a/pkg/storage/bloom/v1/fuse_test.go
+++ b/pkg/storage/bloom/v1/fuse_test.go
@@ -19,9 +19,8 @@ func TestFusedQuerier(t *testing.T) {
bloomsBuf := bytes.NewBuffer(nil)
writer := NewMemoryBlockWriter(indexBuf, bloomsBuf)
reader := NewByteReader(indexBuf, bloomsBuf)
- numSeries := 100
- numKeysPerSeries := 10000
- data, keys := MkBasicSeriesWithBlooms(numSeries, numKeysPerSeries, 0, 0xffff, 0, 10000)
+ numSeries := 1000
+ data, keys := MkBasicSeriesWithBlooms(numSeries, 0, 0x0000, 0xffff, 0, 10000)
builder, err := NewBlockBuilder(
BlockOptions{
@@ -41,14 +40,15 @@ func TestFusedQuerier(t *testing.T) {
block := NewBlock(reader)
querier := NewBlockQuerier(block)
- nReqs := 10
+ n := 2
+ nReqs := numSeries / n
var inputs [][]Request
var resChans []chan Output
for i := 0; i < nReqs; i++ {
ch := make(chan Output)
var reqs []Request
- // find 2 series for each
- for j := 0; j < 2; j++ {
+ // find n series for each
+ for j := 0; j < n; j++ {
idx := numSeries/nReqs*i + j
reqs = append(reqs, Request{
Fp: data[idx].Series.Fingerprint,
diff --git a/pkg/storage/bloom/v1/test_util.go b/pkg/storage/bloom/v1/test_util.go
index ada8cd7319e90..3a5d37975a4ba 100644
--- a/pkg/storage/bloom/v1/test_util.go
+++ b/pkg/storage/bloom/v1/test_util.go
@@ -15,15 +15,14 @@ import (
// TODO(owen-d): this should probably be in it's own testing-util package
-func MakeBlockQuerier(t testing.TB, fromFp, throughFp model.Fingerprint, fromTs, throughTs model.Time) (*BlockQuerier, []SeriesWithBloom) {
+func MakeBlock(t testing.TB, nth int, fromFp, throughFp model.Fingerprint, fromTs, throughTs model.Time) (*Block, []SeriesWithBloom, [][][]byte) {
// references for linking in memory reader+writer
indexBuf := bytes.NewBuffer(nil)
bloomsBuf := bytes.NewBuffer(nil)
writer := NewMemoryBlockWriter(indexBuf, bloomsBuf)
reader := NewByteReader(indexBuf, bloomsBuf)
- numSeries := int(throughFp - fromFp)
- numKeysPerSeries := 1000
- data, _ := MkBasicSeriesWithBlooms(numSeries, numKeysPerSeries, fromFp, throughFp, fromTs, throughTs)
+ numSeries := int(throughFp-fromFp) / nth
+ data, keys := MkBasicSeriesWithBlooms(numSeries, nth, fromFp, throughFp, fromTs, throughTs)
builder, err := NewBlockBuilder(
BlockOptions{
@@ -43,10 +42,10 @@ func MakeBlockQuerier(t testing.TB, fromFp, throughFp model.Fingerprint, fromTs,
_, err = builder.BuildFrom(itr)
require.Nil(t, err)
block := NewBlock(reader)
- return NewBlockQuerier(block), data
+ return block, data, keys
}
-func MkBasicSeriesWithBlooms(nSeries, keysPerSeries int, fromFp, throughFp model.Fingerprint, fromTs, throughTs model.Time) (seriesList []SeriesWithBloom, keysList [][][]byte) {
+func MkBasicSeriesWithBlooms(nSeries, _ int, fromFp, throughFp model.Fingerprint, fromTs, throughTs model.Time) (seriesList []SeriesWithBloom, keysList [][][]byte) {
const nGramLen = 4
seriesList = make([]SeriesWithBloom, 0, nSeries)
keysList = make([][][]byte, 0, nSeries)
@@ -70,20 +69,24 @@ func MkBasicSeriesWithBlooms(nSeries, keysPerSeries int, fromFp, throughFp model
var bloom Bloom
bloom.ScalableBloomFilter = *filter.NewScalableBloomFilter(1024, 0.01, 0.8)
- keys := make([][]byte, 0, keysPerSeries)
- for j := 0; j < keysPerSeries; j++ {
- it := tokenizer.Tokens(fmt.Sprintf("series %d", i*keysPerSeries+j))
- for it.Next() {
- key := it.At()
- // series-level key
- bloom.Add(key)
- keys = append(keys, key)
-
- // chunk-level key
- for _, chk := range series.Chunks {
- tokenBuf, prefixLen := prefixedToken(nGramLen, chk, nil)
+ keys := make([][]byte, 0, int(step))
+ for _, chk := range series.Chunks {
+ tokenBuf, prefixLen := prefixedToken(nGramLen, chk, nil)
+
+ for j := 0; j < int(step); j++ {
+ line := fmt.Sprintf("%04x:%04x", int(series.Fingerprint), j)
+ it := tokenizer.Tokens(line)
+ for it.Next() {
+ key := it.At()
+ // series-level key
+ bloom.Add(key)
+
+ // chunk-level key
tokenBuf = append(tokenBuf[:prefixLen], key...)
bloom.Add(tokenBuf)
+
+ keyCopy := key
+ keys = append(keys, keyCopy)
}
}
}
From a24bd1ee1c3d95e7307cb643cf3f03e6f1bc3d58 Mon Sep 17 00:00:00 2001
From: Rafael Pissolatto Nunes
Date: Sat, 10 Feb 2024 04:27:18 -0300
Subject: [PATCH 12/22] =?UTF-8?q?update=20golang=20to=201.22=20of=20fluent?=
=?UTF-8?q?-bit=20base=20image=20to=20fix=20morestack=20on=20g0=E2=80=A6?=
=?UTF-8?q?=20(#11904)?=
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit
**Issue**:
Since version `grafana/fluent-bit-grafana-loki` >= `2.9.2` we could not
run the client due to an error during the startup/forward process where
the container/service crashes and exits with an error message:
`fatal: morestack on g0`
refs:
- https://github.com/grafana/loki/issues/10944
- https://github.com/golang/go/issues/62440
**What this PR does / why we need it**:
A fix was released through the Golang version 1.22.0 and after some
tests (updating the Go version used to the fluent-bit base image), it
seems working fine now.
**Which issue(s) this PR fixes**:
Fixes #10944
---
clients/cmd/fluent-bit/Dockerfile | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/clients/cmd/fluent-bit/Dockerfile b/clients/cmd/fluent-bit/Dockerfile
index 563614a75f52e..f0dfbc90c36a3 100644
--- a/clients/cmd/fluent-bit/Dockerfile
+++ b/clients/cmd/fluent-bit/Dockerfile
@@ -1,4 +1,4 @@
-FROM golang:1.21.3-bullseye AS builder
+FROM golang:1.22.0-bullseye AS builder
COPY . /src
From d58031e853eb7870db2ac60047a26a7c92e90a49 Mon Sep 17 00:00:00 2001
From: Owen Diehl
Date: Sun, 11 Feb 2024 15:00:41 -0800
Subject: [PATCH 13/22] Blooms/bloomcompactor wiring (#11914)
* Moves `router` struct into the `Compactor` struct
* Implements period aware `TSDBStore`
* Integrates everything together, hooks into `modules.go`
* Removes some now unused compactor code
* Light refactoring to make types align where appropriate
---
docs/sources/configure/_index.md | 12 +
pkg/bloomcompactor/bloomcompactor.go | 356 +++++++++++++++------------
pkg/bloomcompactor/config.go | 58 ++++-
pkg/bloomcompactor/controller.go | 18 +-
pkg/bloomcompactor/metrics.go | 17 +-
pkg/bloomcompactor/router.go | 229 -----------------
pkg/bloomcompactor/spec.go | 63 ++---
pkg/bloomcompactor/spec_test.go | 3 +-
pkg/bloomcompactor/tsdb.go | 111 ++++++++-
pkg/loki/loki.go | 2 +-
pkg/loki/modules.go | 8 +-
11 files changed, 427 insertions(+), 450 deletions(-)
delete mode 100644 pkg/bloomcompactor/router.go
diff --git a/docs/sources/configure/_index.md b/docs/sources/configure/_index.md
index 5c225920ef589..c3854e434f4da 100644
--- a/docs/sources/configure/_index.md
+++ b/docs/sources/configure/_index.md
@@ -2649,6 +2649,18 @@ ring:
# CLI flag: -bloom-compactor.compaction-interval
[compaction_interval: | default = 10m]
+# Minimum age of a table before it is considered for compaction.
+# CLI flag: -bloom-compactor.min-compaction-age
+[min_compaction_age: | default = 24h]
+
+# Maximum age of a table before it is considered for compaction.
+# CLI flag: -bloom-compactor.max-compaction-age
+[max_compaction_age: | default = 168h]
+
+# Number of workers to run in parallel for compaction.
+# CLI flag: -bloom-compactor.worker-parallelism
+[worker_parallelism: | default = 1]
+
# Minimum backoff time between retries.
# CLI flag: -bloom-compactor.compaction-retries-min-backoff
[compaction_retries_min_backoff: | default = 10s]
diff --git a/pkg/bloomcompactor/bloomcompactor.go b/pkg/bloomcompactor/bloomcompactor.go
index cf3b3fafcb6d1..a91b6796e5360 100644
--- a/pkg/bloomcompactor/bloomcompactor.go
+++ b/pkg/bloomcompactor/bloomcompactor.go
@@ -2,23 +2,25 @@ package bloomcompactor
import (
"context"
- "fmt"
+ "math"
+ "sync"
"time"
"github.com/go-kit/log"
"github.com/go-kit/log/level"
"github.com/grafana/dskit/backoff"
+ "github.com/grafana/dskit/concurrency"
"github.com/grafana/dskit/multierror"
"github.com/grafana/dskit/services"
"github.com/pkg/errors"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/common/model"
- "github.com/grafana/loki/pkg/bloomutils"
- "github.com/grafana/loki/pkg/compactor"
+ "github.com/grafana/loki/pkg/storage"
v1 "github.com/grafana/loki/pkg/storage/bloom/v1"
+ "github.com/grafana/loki/pkg/storage/config"
+ "github.com/grafana/loki/pkg/storage/stores"
"github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper"
- "github.com/grafana/loki/pkg/util"
)
/*
@@ -33,38 +35,73 @@ Bloom-compactor regularly runs to check for changes in meta.jsons and runs compa
type Compactor struct {
services.Service
- cfg Config
- logger log.Logger
- limits Limits
+ cfg Config
+ schemaCfg config.SchemaConfig
+ logger log.Logger
+ limits Limits
- // temporary workaround until store has implemented read/write shipper interface
- store bloomshipper.Store
+ tsdbStore TSDBStore
+ // TODO(owen-d): ShardingStrategy
+ controller *SimpleBloomController
+
+ // temporary workaround until bloomStore has implemented read/write shipper interface
+ bloomStore bloomshipper.Store
sharding ShardingStrategy
- metrics *metrics
+ metrics *Metrics
btMetrics *v1.Metrics
}
func New(
cfg Config,
- store bloomshipper.Store,
+ schemaCfg config.SchemaConfig,
+ storeCfg storage.Config,
+ clientMetrics storage.ClientMetrics,
+ fetcherProvider stores.ChunkFetcherProvider,
sharding ShardingStrategy,
limits Limits,
logger log.Logger,
r prometheus.Registerer,
) (*Compactor, error) {
c := &Compactor{
- cfg: cfg,
- store: store,
- logger: logger,
- sharding: sharding,
- limits: limits,
+ cfg: cfg,
+ schemaCfg: schemaCfg,
+ logger: logger,
+ sharding: sharding,
+ limits: limits,
+ }
+
+ tsdbStore, err := NewTSDBStores(schemaCfg, storeCfg, clientMetrics)
+ if err != nil {
+ return nil, errors.Wrap(err, "failed to create TSDB store")
+ }
+ c.tsdbStore = tsdbStore
+
+ // TODO(owen-d): export bloomstore as a dependency that can be reused by the compactor & gateway rather that
+ bloomStore, err := bloomshipper.NewBloomStore(schemaCfg.Configs, storeCfg, clientMetrics, nil, nil, logger)
+ if err != nil {
+ return nil, errors.Wrap(err, "failed to create bloom store")
}
+ c.bloomStore = bloomStore
// initialize metrics
c.btMetrics = v1.NewMetrics(prometheus.WrapRegistererWithPrefix("loki_bloom_tokenizer", r))
- c.metrics = newMetrics(r)
+ c.metrics = NewMetrics(r, c.btMetrics)
+
+ chunkLoader := NewStoreChunkLoader(
+ NewFetcherProviderAdapter(fetcherProvider),
+ c.metrics,
+ )
+
+ c.controller = NewSimpleBloomController(
+ c.tsdbStore,
+ c.bloomStore,
+ chunkLoader,
+ c.metrics,
+ c.logger,
+ )
+
c.metrics.compactionRunInterval.Set(cfg.CompactionInterval.Seconds())
c.Service = services.NewBasicService(c.starting, c.running, c.stopping)
@@ -76,192 +113,193 @@ func (c *Compactor) starting(_ context.Context) (err error) {
return err
}
+func (c *Compactor) stopping(_ error) error {
+ c.metrics.compactorRunning.Set(0)
+ return nil
+}
+
func (c *Compactor) running(ctx context.Context) error {
- // Run an initial compaction before starting the interval.
- if err := c.runCompaction(ctx); err != nil {
- level.Error(c.logger).Log("msg", "failed to run compaction", "err", err)
+ // run once at beginning
+ if err := c.runOne(ctx); err != nil {
+ return err
}
- ticker := time.NewTicker(util.DurationWithJitter(c.cfg.CompactionInterval, 0.05))
+ ticker := time.NewTicker(c.cfg.CompactionInterval)
defer ticker.Stop()
-
for {
select {
- case start := <-ticker.C:
- c.metrics.compactionRunsStarted.Inc()
- if err := c.runCompaction(ctx); err != nil {
- c.metrics.compactionRunsCompleted.WithLabelValues(statusFailure).Inc()
- c.metrics.compactionRunTime.WithLabelValues(statusFailure).Observe(time.Since(start).Seconds())
- level.Error(c.logger).Log("msg", "failed to run compaction", "err", err)
- continue
- }
- c.metrics.compactionRunsCompleted.WithLabelValues(statusSuccess).Inc()
- c.metrics.compactionRunTime.WithLabelValues(statusSuccess).Observe(time.Since(start).Seconds())
case <-ctx.Done():
- return nil
+ return ctx.Err()
+
+ case <-ticker.C:
+ if err := c.runOne(ctx); err != nil {
+ level.Error(c.logger).Log("msg", "compaction iteration failed", "err", err)
+ return err
+ }
}
}
}
-func (c *Compactor) stopping(_ error) error {
- c.metrics.compactorRunning.Set(0)
- return nil
-}
-
-func (c *Compactor) runCompaction(ctx context.Context) error {
- var tables []string
- // TODO(owen-d): resolve tables
+func runWithRetries(
+ ctx context.Context,
+ minBackoff, maxBackoff time.Duration,
+ maxRetries int,
+ f func(ctx context.Context) error,
+) error {
+ var lastErr error
- // process most recent tables first
- tablesIntervals := getIntervalsForTables(tables)
- compactor.SortTablesByRange(tables)
+ retries := backoff.New(ctx, backoff.Config{
+ MinBackoff: minBackoff,
+ MaxBackoff: maxBackoff,
+ MaxRetries: maxRetries,
+ })
- // TODO(owen-d): parallelize at the bottom level, not the top level.
- // Can dispatch to a queue & wait.
- for _, table := range tables {
- logger := log.With(c.logger, "table", table)
- err := c.compactTable(ctx, logger, table, tablesIntervals[table])
- if err != nil {
- level.Error(logger).Log("msg", "failed to compact table", "err", err)
- return errors.Wrapf(err, "failed to compact table %s", table)
+ for retries.Ongoing() {
+ lastErr = f(ctx)
+ if lastErr == nil {
+ return nil
}
+
+ retries.Wait()
}
- return nil
+
+ return lastErr
+}
+
+type tenantTable struct {
+ tenant string
+ table DayTable
+ ownershipRange v1.FingerprintBounds
}
-func (c *Compactor) compactTable(ctx context.Context, logger log.Logger, tableName string, tableInterval model.Interval) error {
- // Ensure the context has not been canceled (ie. compactor shutdown has been triggered).
- if err := ctx.Err(); err != nil {
- return fmt.Errorf("interrupting compaction of table: %w", err)
+func (c *Compactor) tenants(ctx context.Context, table DayTable) (v1.Iterator[string], error) {
+ tenants, err := c.tsdbStore.UsersForPeriod(ctx, table)
+ if err != nil {
+ return nil, errors.Wrap(err, "getting tenants")
}
- var tenants []string
+ return v1.NewSliceIter(tenants), nil
+}
- level.Info(logger).Log("msg", "discovered tenants from bucket", "users", len(tenants))
- return c.compactUsers(ctx, logger, tableName, tableInterval, tenants)
+// TODO(owen-d): implement w/ subrings
+func (c *Compactor) ownsTenant(_ string) (ownershipRange v1.FingerprintBounds, owns bool) {
+ return v1.NewBounds(0, math.MaxUint64), true
}
-func (c *Compactor) compactUsers(ctx context.Context, logger log.Logger, tableName string, tableInterval model.Interval, tenants []string) error {
- // Keep track of tenants owned by this shard, so that we can delete the local files for all other users.
- errs := multierror.New()
- ownedTenants := make(map[string]struct{}, len(tenants))
- for _, tenant := range tenants {
- tenantLogger := log.With(logger, "tenant", tenant)
+// runs a single round of compaction for all relevant tenants and tables
+func (c *Compactor) runOne(ctx context.Context) error {
+ var workersErr error
+ var wg sync.WaitGroup
+ ch := make(chan tenantTable)
+ wg.Add(1)
+ go func() {
+ workersErr = c.runWorkers(ctx, ch)
+ wg.Done()
+ }()
+
+ err := c.loadWork(ctx, ch)
+
+ wg.Wait()
+ return multierror.New(workersErr, err, ctx.Err()).Err()
+}
- // Ensure the context has not been canceled (ie. compactor shutdown has been triggered).
- if err := ctx.Err(); err != nil {
- return fmt.Errorf("interrupting compaction of tenants: %w", err)
- }
+func (c *Compactor) tables(ts time.Time) *dayRangeIterator {
+ from := model.TimeFromUnixNano(ts.Add(-c.cfg.MaxCompactionAge).UnixNano() / int64(config.ObjectStorageIndexRequiredPeriod))
+ through := model.TimeFromUnixNano(ts.Add(-c.cfg.MinCompactionAge).UnixNano() / int64(config.ObjectStorageIndexRequiredPeriod))
+ return newDayRangeIterator(DayTable(from), DayTable(through))
+}
- // Skip tenant if compaction is not enabled
- if !c.limits.BloomCompactorEnabled(tenant) {
- level.Info(tenantLogger).Log("msg", "compaction disabled for tenant. Skipping.")
- continue
- }
+func (c *Compactor) loadWork(ctx context.Context, ch chan<- tenantTable) error {
+ tables := c.tables(time.Now())
- // Skip this table if it is too old for the tenant limits.
- now := model.Now()
- tableMaxAge := c.limits.BloomCompactorMaxTableAge(tenant)
- if tableMaxAge > 0 && tableInterval.Start.Before(now.Add(-tableMaxAge)) {
- level.Debug(tenantLogger).Log("msg", "skipping tenant because table is too old", "table-max-age", tableMaxAge, "table-start", tableInterval.Start, "now", now)
- continue
- }
+ for tables.Next() && tables.Err() == nil && ctx.Err() == nil {
- // Ensure the tenant ID belongs to our shard.
- if !c.sharding.OwnsTenant(tenant) {
- c.metrics.compactionRunSkippedTenants.Inc()
- level.Debug(tenantLogger).Log("msg", "skipping tenant because it is not owned by this shard")
- continue
+ table := tables.At()
+ tenants, err := c.tenants(ctx, table)
+ if err != nil {
+ return errors.Wrap(err, "getting tenants")
}
- ownedTenants[tenant] = struct{}{}
-
- start := time.Now()
- if err := c.compactTenantWithRetries(ctx, tenantLogger, tableName, tenant); err != nil {
- switch {
- case errors.Is(err, context.Canceled):
- // We don't want to count shutdowns as failed compactions because we will pick up with the rest of the compaction after the restart.
- level.Info(tenantLogger).Log("msg", "compaction for tenant was interrupted by a shutdown")
- return nil
- default:
- c.metrics.compactionRunTenantsCompleted.WithLabelValues(statusFailure).Inc()
- c.metrics.compactionRunTenantsTime.WithLabelValues(statusFailure).Observe(time.Since(start).Seconds())
- level.Error(tenantLogger).Log("msg", "failed to compact tenant", "err", err)
- errs.Add(err)
+ for tenants.Next() && tenants.Err() == nil && ctx.Err() == nil {
+ tenant := tenants.At()
+ ownershipRange, owns := c.ownsTenant(tenant)
+ if !owns {
+ continue
+ }
+
+ select {
+ case ch <- tenantTable{tenant: tenant, table: table, ownershipRange: ownershipRange}:
+ case <-ctx.Done():
+ return ctx.Err()
}
- continue
}
- c.metrics.compactionRunTenantsCompleted.WithLabelValues(statusSuccess).Inc()
- c.metrics.compactionRunTenantsTime.WithLabelValues(statusSuccess).Observe(time.Since(start).Seconds())
- level.Info(tenantLogger).Log("msg", "successfully compacted tenant")
+ if err := tenants.Err(); err != nil {
+ return errors.Wrap(err, "iterating tenants")
+ }
+
}
- return errs.Err()
+ if err := tables.Err(); err != nil {
+ return errors.Wrap(err, "iterating tables")
+ }
- // TODO: Delete local files for unowned tenants, if there are any.
+ close(ch)
+ return ctx.Err()
}
-func (c *Compactor) compactTenant(ctx context.Context, logger log.Logger, _ string, tenant string) error {
- level.Info(logger).Log("msg", "starting compaction of tenant")
-
- // Ensure the context has not been canceled (ie. compactor shutdown has been triggered).
- if err := ctx.Err(); err != nil {
- return err
- }
-
- // Tokenizer is not thread-safe so we need one per goroutine.
- nGramLen := c.limits.BloomNGramLength(tenant)
- nGramSkip := c.limits.BloomNGramSkip(tenant)
- _ = v1.NewBloomTokenizer(nGramLen, nGramSkip, c.btMetrics)
+func (c *Compactor) runWorkers(ctx context.Context, ch <-chan tenantTable) error {
+
+ return concurrency.ForEachJob(ctx, c.cfg.WorkerParallelism, c.cfg.WorkerParallelism, 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 := c.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,
+ )
+ }
+ }
+ }
- rs, err := c.sharding.GetTenantSubRing(tenant).GetAllHealthy(RingOp)
- if err != nil {
- return err
- }
- tokenRanges := bloomutils.GetInstanceWithTokenRange(c.cfg.Ring.InstanceID, rs.Instances)
- for _, tr := range tokenRanges {
- level.Debug(logger).Log("msg", "got token range for instance", "id", tr.Instance.Id, "min", tr.MinToken, "max", tr.MaxToken)
- }
+ })
- // TODO(owen-d): impl
- return nil
}
-func runWithRetries(
- ctx context.Context,
- minBackoff, maxBackoff time.Duration,
- maxRetries int,
- f func(ctx context.Context) error,
-) error {
- var lastErr error
+func (c *Compactor) compactTenantTable(ctx context.Context, tt tenantTable) error {
+ level.Info(c.logger).Log("msg", "compacting", "org_id", tt.tenant, "table", tt.table, "ownership", tt.ownershipRange)
+ return c.controller.buildBlocks(ctx, tt.table, tt.tenant, tt.ownershipRange)
+}
- retries := backoff.New(ctx, backoff.Config{
- MinBackoff: minBackoff,
- MaxBackoff: maxBackoff,
- MaxRetries: maxRetries,
- })
+type dayRangeIterator struct {
+ min, max, cur DayTable
+}
- for retries.Ongoing() {
- lastErr = f(ctx)
- if lastErr == nil {
- return nil
- }
+func newDayRangeIterator(min, max DayTable) *dayRangeIterator {
+ return &dayRangeIterator{min: min, max: max, cur: min.Dec()}
+}
- retries.Wait()
- }
+func (r *dayRangeIterator) Next() bool {
+ r.cur = r.cur.Inc()
+ return r.cur.Before(r.max)
+}
- return lastErr
+func (r *dayRangeIterator) At() DayTable {
+ return r.cur
}
-func (c *Compactor) compactTenantWithRetries(ctx context.Context, logger log.Logger, tableName string, tenant string) error {
- return runWithRetries(
- ctx,
- c.cfg.RetryMinBackoff,
- c.cfg.RetryMaxBackoff,
- c.cfg.CompactionRetries,
- func(ctx context.Context) error {
- return c.compactTenant(ctx, logger, tableName, tenant)
- },
- )
+func (r *dayRangeIterator) Err() error {
+ return nil
}
diff --git a/pkg/bloomcompactor/config.go b/pkg/bloomcompactor/config.go
index 884034fdd043d..37c8844378465 100644
--- a/pkg/bloomcompactor/config.go
+++ b/pkg/bloomcompactor/config.go
@@ -2,8 +2,13 @@ package bloomcompactor
import (
"flag"
+ "fmt"
"time"
+ "github.com/prometheus/common/model"
+
+ "github.com/grafana/loki/pkg/storage/config"
+ "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper"
"github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/downloads"
"github.com/grafana/loki/pkg/util/ring"
)
@@ -18,10 +23,12 @@ type Config struct {
Enabled bool `yaml:"enabled"`
WorkingDirectory string `yaml:"working_directory"`
CompactionInterval time.Duration `yaml:"compaction_interval"`
-
- RetryMinBackoff time.Duration `yaml:"compaction_retries_min_backoff"`
- RetryMaxBackoff time.Duration `yaml:"compaction_retries_max_backoff"`
- CompactionRetries int `yaml:"compaction_retries"`
+ MinCompactionAge time.Duration `yaml:"min_compaction_age"`
+ MaxCompactionAge time.Duration `yaml:"max_compaction_age"`
+ WorkerParallelism int `yaml:"worker_parallelism"`
+ RetryMinBackoff time.Duration `yaml:"compaction_retries_min_backoff"`
+ RetryMaxBackoff time.Duration `yaml:"compaction_retries_max_backoff"`
+ CompactionRetries int `yaml:"compaction_retries"`
MaxCompactionParallelism int `yaml:"max_compaction_parallelism"`
}
@@ -32,6 +39,15 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) {
f.BoolVar(&cfg.Enabled, "bloom-compactor.enabled", false, "Flag to enable or disable the usage of the bloom-compactor component.")
f.StringVar(&cfg.WorkingDirectory, "bloom-compactor.working-directory", "", "Directory where files can be downloaded for compaction.")
f.DurationVar(&cfg.CompactionInterval, "bloom-compactor.compaction-interval", 10*time.Minute, "Interval at which to re-run the compaction operation.")
+ f.IntVar(&cfg.WorkerParallelism, "bloom-compactor.worker-parallelism", 1, "Number of workers to run in parallel for compaction.")
+ f.DurationVar(&cfg.MinCompactionAge, "bloom-compactor.min-compaction-age", 24*time.Hour, "Minimum age of a table before it is considered for compaction.")
+ // TODO(owen-d): ideally we'd set this per tenant based on their `reject_old_samples_max_age` setting,
+ // but due to how we need to discover tenants, we can't do that yet. Tenant+Period discovery is done by
+ // iterating the table periods in object storage and looking for tenants within that period.
+ // In order to have this done dynamically, we'd need to account for tenant specific overrides, which are also
+ // dynamically reloaded.
+ // I'm doing it the simple way for now.
+ f.DurationVar(&cfg.MaxCompactionAge, "bloom-compactor.max-compaction-age", 7*24*time.Hour, "Maximum age of a table before it is considered for compaction.")
f.DurationVar(&cfg.RetryMinBackoff, "bloom-compactor.compaction-retries-min-backoff", 10*time.Second, "Minimum backoff time between retries.")
f.DurationVar(&cfg.RetryMaxBackoff, "bloom-compactor.compaction-retries-max-backoff", time.Minute, "Maximum backoff time between retries.")
f.IntVar(&cfg.CompactionRetries, "bloom-compactor.compaction-retries", 3, "Number of retries to perform when compaction fails.")
@@ -48,3 +64,37 @@ type Limits interface {
BloomNGramSkip(tenantID string) int
BloomFalsePositiveRate(tenantID string) float64
}
+
+// TODO(owen-d): Remove this type in favor of config.DayTime
+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()),
+ }
+}
diff --git a/pkg/bloomcompactor/controller.go b/pkg/bloomcompactor/controller.go
index f3db6edb0636c..dd48351a98ec8 100644
--- a/pkg/bloomcompactor/controller.go
+++ b/pkg/bloomcompactor/controller.go
@@ -1,6 +1,7 @@
package bloomcompactor
import (
+ "bytes"
"context"
"fmt"
"sort"
@@ -18,7 +19,6 @@ type SimpleBloomController struct {
tsdbStore TSDBStore
bloomStore bloomshipper.Store
chunkLoader ChunkLoader
- rwFn func() (v1.BlockWriter, v1.BlockReader)
metrics *Metrics
// TODO(owen-d): add metrics
@@ -29,7 +29,6 @@ func NewSimpleBloomController(
tsdbStore TSDBStore,
blockStore bloomshipper.Store,
chunkLoader ChunkLoader,
- rwFn func() (v1.BlockWriter, v1.BlockReader),
metrics *Metrics,
logger log.Logger,
) *SimpleBloomController {
@@ -37,12 +36,18 @@ func NewSimpleBloomController(
tsdbStore: tsdbStore,
bloomStore: blockStore,
chunkLoader: chunkLoader,
- rwFn: rwFn,
metrics: metrics,
logger: logger,
}
}
+// TODO(owen-d): pool, evaluate if memory-only is the best choice
+func (s *SimpleBloomController) rwFn() (v1.BlockWriter, v1.BlockReader) {
+ indexBuf := bytes.NewBuffer(nil)
+ bloomsBuf := bytes.NewBuffer(nil)
+ return v1.NewMemoryBlockWriter(indexBuf, bloomsBuf), v1.NewByteReader(indexBuf, bloomsBuf)
+}
+
func (s *SimpleBloomController) buildBlocks(
ctx context.Context,
table DayTable,
@@ -52,7 +57,7 @@ func (s *SimpleBloomController) buildBlocks(
logger := log.With(s.logger, "ownership", ownershipRange, "org_id", tenant, "table", table)
// 1. Resolve TSDBs
- tsdbs, err := s.tsdbStore.ResolveTSDBs(ctx, table.String(), tenant)
+ tsdbs, err := s.tsdbStore.ResolveTSDBs(ctx, table, tenant)
if err != nil {
level.Error(logger).Log("msg", "failed to resolve tsdbs", "err", err)
return errors.Wrap(err, "failed to resolve tsdbs")
@@ -124,13 +129,14 @@ func (s *SimpleBloomController) buildBlocks(
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, table.String(), tenant, plan.tsdb, gap)
+ seriesItr, preExistingBlocks, err := s.loadWorkForGap(ctx, table, tenant, plan.tsdb, gap)
if err != nil {
level.Error(logger).Log("msg", "failed to get series and blocks", "err", err)
return errors.Wrap(err, "failed to get series and blocks")
}
gen := NewSimpleBloomGenerator(
+ tenant,
v1.DefaultBlockOptions,
seriesItr,
s.chunkLoader,
@@ -185,7 +191,7 @@ func (s *SimpleBloomController) buildBlocks(
func (s *SimpleBloomController) loadWorkForGap(
ctx context.Context,
- table,
+ table DayTable,
tenant string,
id tsdb.Identifier,
gap gapWithBlocks,
diff --git a/pkg/bloomcompactor/metrics.go b/pkg/bloomcompactor/metrics.go
index ee2f1630ab5ec..b02ac32aca727 100644
--- a/pkg/bloomcompactor/metrics.go
+++ b/pkg/bloomcompactor/metrics.go
@@ -3,6 +3,8 @@ package bloomcompactor
import (
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/client_golang/prometheus/promauto"
+
+ v1 "github.com/grafana/loki/pkg/storage/bloom/v1"
)
const (
@@ -13,7 +15,10 @@ const (
statusFailure = "failure"
)
-type metrics struct {
+type Metrics struct {
+ bloomMetrics *v1.Metrics
+ chunkSize prometheus.Histogram // uncompressed size of all chunks summed per series
+
compactionRunsStarted prometheus.Counter
compactionRunsCompleted *prometheus.CounterVec
compactionRunTime *prometheus.HistogramVec
@@ -28,8 +33,14 @@ type metrics struct {
compactorRunning prometheus.Gauge
}
-func newMetrics(r prometheus.Registerer) *metrics {
- m := metrics{
+func NewMetrics(r prometheus.Registerer, bloomMetrics *v1.Metrics) *Metrics {
+ m := Metrics{
+ bloomMetrics: bloomMetrics,
+ chunkSize: promauto.With(r).NewHistogram(prometheus.HistogramOpts{
+ Name: "bloom_chunk_series_size",
+ Help: "Uncompressed size of chunks in a series",
+ Buckets: prometheus.ExponentialBucketsRange(1024, 1073741824, 10),
+ }),
compactionRunsStarted: promauto.With(r).NewCounter(prometheus.CounterOpts{
Namespace: metricsNamespace,
Subsystem: metricsSubsystem,
diff --git a/pkg/bloomcompactor/router.go b/pkg/bloomcompactor/router.go
deleted file mode 100644
index e2c8b6545260f..0000000000000
--- a/pkg/bloomcompactor/router.go
+++ /dev/null
@@ -1,229 +0,0 @@
-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 088c5b823a1b5..7353ca9708121 100644
--- a/pkg/bloomcompactor/spec.go
+++ b/pkg/bloomcompactor/spec.go
@@ -9,8 +9,6 @@ import (
"github.com/go-kit/log"
"github.com/go-kit/log/level"
"github.com/pkg/errors"
- "github.com/prometheus/client_golang/prometheus"
- "github.com/prometheus/client_golang/prometheus/promauto"
"github.com/prometheus/common/model"
"github.com/grafana/dskit/multierror"
@@ -20,34 +18,11 @@ import (
logql_log "github.com/grafana/loki/pkg/logql/log"
v1 "github.com/grafana/loki/pkg/storage/bloom/v1"
"github.com/grafana/loki/pkg/storage/chunk"
+ "github.com/grafana/loki/pkg/storage/stores"
"github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper"
"github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb"
)
-/*
-This file maintains a number of things supporting bloom generation. Most notably, the `BloomGenerator` interface/implementation which builds bloom filters.
-
-- `BloomGenerator`: Builds blooms. Most other things in this file are supporting this in various ways.
-- `SimpleBloomGenerator`: A foundational implementation of `BloomGenerator` which wires up a few different components to generate bloom filters for a set of blocks and handles schema compatibility:
-- `chunkLoader`: Loads chunks w/ a specific fingerprint from the store, returns an iterator of chunk iterators. We return iterators rather than chunk implementations mainly for ease of testing. In practice, this will just be an iterator over `MemChunk`s.
-*/
-
-type Metrics struct {
- bloomMetrics *v1.Metrics
- chunkSize prometheus.Histogram // uncompressed size of all chunks summed per series
-}
-
-func NewMetrics(r prometheus.Registerer, bloomMetrics *v1.Metrics) *Metrics {
- return &Metrics{
- bloomMetrics: bloomMetrics,
- chunkSize: promauto.With(r).NewHistogram(prometheus.HistogramOpts{
- Name: "bloom_chunk_series_size",
- Help: "Uncompressed size of chunks in a series",
- Buckets: prometheus.ExponentialBucketsRange(1024, 1073741824, 10),
- }),
- }
-}
-
// inclusive range
type Keyspace struct {
min, max model.Fingerprint
@@ -70,6 +45,7 @@ type BloomGenerator interface {
// Simple implementation of a BloomGenerator.
type SimpleBloomGenerator struct {
+ userID string
store v1.Iterator[*v1.Series]
chunkLoader ChunkLoader
// TODO(owen-d): blocks need not be all downloaded prior. Consider implementing
@@ -92,6 +68,7 @@ type SimpleBloomGenerator struct {
// and handles schema compatibility:
// Blocks which are incompatible with the schema are skipped and will have their chunks reindexed
func NewSimpleBloomGenerator(
+ userID string,
opts v1.BlockOptions,
store v1.Iterator[*v1.Series],
chunkLoader ChunkLoader,
@@ -101,8 +78,8 @@ func NewSimpleBloomGenerator(
logger log.Logger,
) *SimpleBloomGenerator {
return &SimpleBloomGenerator{
- opts: opts,
- // TODO(owen-d): implement Iterator[Series] against TSDB files to hook in here.
+ userID: userID,
+ opts: opts,
store: store,
chunkLoader: chunkLoader,
blocks: blocks,
@@ -116,7 +93,7 @@ func NewSimpleBloomGenerator(
func (s *SimpleBloomGenerator) populator(ctx context.Context) func(series *v1.Series, bloom *v1.Bloom) error {
return func(series *v1.Series, bloom *v1.Bloom) error {
- chunkItersWithFP, err := s.chunkLoader.Load(ctx, series)
+ chunkItersWithFP, err := s.chunkLoader.Load(ctx, s.userID, series)
if err != nil {
return errors.Wrapf(err, "failed to load chunks for series: %+v", series)
}
@@ -210,7 +187,7 @@ type ChunkItersByFingerprint struct {
// ChunkLoader loads chunks from a store
type ChunkLoader interface {
- Load(context.Context, *v1.Series) (*ChunkItersByFingerprint, error)
+ Load(ctx context.Context, userID string, series *v1.Series) (*ChunkItersByFingerprint, error)
}
// interface modeled from `pkg/storage/stores/composite_store.ChunkFetcherProvider`
@@ -223,23 +200,37 @@ type chunkFetcher interface {
FetchChunks(ctx context.Context, chunks []chunk.Chunk) ([]chunk.Chunk, error)
}
+// Adapter turning `stores.ChunkFetcherProvider` into `fetcherProvider`
+// The former returns a concrete type and is heavily used externally
+// while the latter returns an interface for better testing and
+// is used internally
+type FetcherProviderAdapter struct {
+ root stores.ChunkFetcherProvider
+}
+
+func NewFetcherProviderAdapter(root stores.ChunkFetcherProvider) *FetcherProviderAdapter {
+ return &FetcherProviderAdapter{root: root}
+}
+
+func (f *FetcherProviderAdapter) GetChunkFetcher(t model.Time) chunkFetcher {
+ return f.root.GetChunkFetcher(t)
+}
+
// StoreChunkLoader loads chunks from a store
type StoreChunkLoader struct {
- userID string
fetcherProvider fetcherProvider
metrics *Metrics
}
-func NewStoreChunkLoader(userID string, fetcherProvider fetcherProvider, metrics *Metrics) *StoreChunkLoader {
+func NewStoreChunkLoader(fetcherProvider fetcherProvider, metrics *Metrics) *StoreChunkLoader {
return &StoreChunkLoader{
- userID: userID,
fetcherProvider: fetcherProvider,
metrics: metrics,
}
}
-func (s *StoreChunkLoader) Load(ctx context.Context, series *v1.Series) (*ChunkItersByFingerprint, error) {
- // TODO(owen-d): This is probalby unnecessary as we should only have one fetcher
+func (s *StoreChunkLoader) Load(ctx context.Context, userID string, series *v1.Series) (*ChunkItersByFingerprint, error) {
+ // NB(owen-d): This is probalby unnecessary as we should only have one fetcher
// because we'll only be working on a single index period at a time, but this should protect
// us in the case of refactoring/changing this and likely isn't a perf bottleneck.
chksByFetcher := make(map[chunkFetcher][]chunk.Chunk)
@@ -248,7 +239,7 @@ func (s *StoreChunkLoader) Load(ctx context.Context, series *v1.Series) (*ChunkI
chksByFetcher[fetcher] = append(chksByFetcher[fetcher], chunk.Chunk{
ChunkRef: logproto.ChunkRef{
Fingerprint: uint64(series.Fingerprint),
- UserID: s.userID,
+ UserID: userID,
From: chk.Start,
Through: chk.End,
Checksum: chk.Checksum,
diff --git a/pkg/bloomcompactor/spec_test.go b/pkg/bloomcompactor/spec_test.go
index c43a4b715a1e7..443d491ea25be 100644
--- a/pkg/bloomcompactor/spec_test.go
+++ b/pkg/bloomcompactor/spec_test.go
@@ -56,7 +56,7 @@ func blocksFromSchemaWithRange(t *testing.T, n int, options v1.BlockOptions, fro
// doesn't actually load any chunks
type dummyChunkLoader struct{}
-func (dummyChunkLoader) Load(_ context.Context, series *v1.Series) (*ChunkItersByFingerprint, error) {
+func (dummyChunkLoader) Load(_ context.Context, _ string, series *v1.Series) (*ChunkItersByFingerprint, error) {
return &ChunkItersByFingerprint{
fp: series.Fingerprint,
itr: v1.NewEmptyIter[v1.ChunkRefWithIter](),
@@ -72,6 +72,7 @@ func dummyBloomGen(opts v1.BlockOptions, store v1.Iterator[*v1.Series], blocks [
}
return NewSimpleBloomGenerator(
+ "fake",
opts,
store,
dummyChunkLoader{},
diff --git a/pkg/bloomcompactor/tsdb.go b/pkg/bloomcompactor/tsdb.go
index 5ade52f8ab1cd..be45d293f6286 100644
--- a/pkg/bloomcompactor/tsdb.go
+++ b/pkg/bloomcompactor/tsdb.go
@@ -2,6 +2,7 @@ package bloomcompactor
import (
"context"
+ "fmt"
"io"
"math"
"path"
@@ -11,7 +12,9 @@ import (
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/model/labels"
+ baseStore "github.com/grafana/loki/pkg/storage"
v1 "github.com/grafana/loki/pkg/storage/bloom/v1"
+ "github.com/grafana/loki/pkg/storage/config"
"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"
@@ -22,11 +25,11 @@ const (
)
type TSDBStore interface {
- UsersForPeriod(ctx context.Context, table string) ([]string, error)
- ResolveTSDBs(ctx context.Context, table, tenant string) ([]tsdb.SingleTenantTSDBIdentifier, error)
+ UsersForPeriod(ctx context.Context, table DayTable) ([]string, error)
+ ResolveTSDBs(ctx context.Context, table DayTable, tenant string) ([]tsdb.SingleTenantTSDBIdentifier, error)
LoadTSDB(
ctx context.Context,
- table,
+ table DayTable,
tenant string,
id tsdb.Identifier,
bounds v1.FingerprintBounds,
@@ -45,13 +48,13 @@ func NewBloomTSDBStore(storage storage.Client) *BloomTSDBStore {
}
}
-func (b *BloomTSDBStore) UsersForPeriod(ctx context.Context, table string) ([]string, error) {
- _, users, err := b.storage.ListFiles(ctx, table, false)
+func (b *BloomTSDBStore) UsersForPeriod(ctx context.Context, table DayTable) ([]string, error) {
+ _, users, err := b.storage.ListFiles(ctx, table.String(), 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)
+func (b *BloomTSDBStore) ResolveTSDBs(ctx context.Context, table DayTable, tenant string) ([]tsdb.SingleTenantTSDBIdentifier, error) {
+ indices, err := b.storage.ListUserFiles(ctx, table.String(), tenant, false)
if err != nil {
return nil, errors.Wrap(err, "failed to list user files")
}
@@ -76,12 +79,12 @@ func (b *BloomTSDBStore) ResolveTSDBs(ctx context.Context, table, tenant string)
func (b *BloomTSDBStore) LoadTSDB(
ctx context.Context,
- table,
+ table DayTable,
tenant string,
id tsdb.Identifier,
bounds v1.FingerprintBounds,
) (v1.CloseableIterator[*v1.Series], error) {
- data, err := b.storage.GetUserFile(ctx, table, tenant, id.Name())
+ data, err := b.storage.GetUserFile(ctx, table.String(), tenant, id.Name())
if err != nil {
return nil, errors.Wrap(err, "failed to get file")
}
@@ -200,3 +203,93 @@ func (t *TSDBSeriesIter) background() {
close(t.ch)
}()
}
+
+type TSDBStores struct {
+ schemaCfg config.SchemaConfig
+ stores []TSDBStore
+}
+
+func NewTSDBStores(
+ schemaCfg config.SchemaConfig,
+ storeCfg baseStore.Config,
+ clientMetrics baseStore.ClientMetrics,
+) (*TSDBStores, error) {
+ res := &TSDBStores{
+ schemaCfg: schemaCfg,
+ stores: make([]TSDBStore, len(schemaCfg.Configs)),
+ }
+
+ for i, cfg := range schemaCfg.Configs {
+ if cfg.IndexType == config.TSDBType {
+
+ c, err := baseStore.NewObjectClient(cfg.ObjectType, storeCfg, clientMetrics)
+ if err != nil {
+ return nil, errors.Wrap(err, "failed to create object client")
+ }
+ res.stores[i] = NewBloomTSDBStore(storage.NewIndexStorageClient(c, cfg.IndexTables.PathPrefix))
+ }
+ }
+
+ return res, nil
+}
+
+func (s *TSDBStores) storeForPeriod(table DayTable) (TSDBStore, error) {
+ for i := len(s.schemaCfg.Configs) - 1; i >= 0; i-- {
+ period := s.schemaCfg.Configs[i]
+
+ if !table.Before(DayTable(period.From.Time)) {
+ // we have the desired period config
+
+ if s.stores[i] != nil {
+ // valid: it's of tsdb type
+ return s.stores[i], nil
+ }
+
+ // invalid
+ return nil, errors.Errorf(
+ "store for period is not of TSDB type (%s) while looking up store for (%v)",
+ period.IndexType,
+ table.ModelTime().Time(),
+ )
+ }
+
+ }
+
+ return nil, fmt.Errorf(
+ "There is no store matching no matching period found for table (%v) -- too early",
+ table.ModelTime().Time(),
+ )
+}
+
+func (s *TSDBStores) UsersForPeriod(ctx context.Context, table DayTable) ([]string, error) {
+ store, err := s.storeForPeriod(table)
+ if err != nil {
+ return nil, err
+ }
+
+ return store.UsersForPeriod(ctx, table)
+}
+
+func (s *TSDBStores) ResolveTSDBs(ctx context.Context, table DayTable, tenant string) ([]tsdb.SingleTenantTSDBIdentifier, error) {
+ store, err := s.storeForPeriod(table)
+ if err != nil {
+ return nil, err
+ }
+
+ return store.ResolveTSDBs(ctx, table, tenant)
+}
+
+func (s *TSDBStores) LoadTSDB(
+ ctx context.Context,
+ table DayTable,
+ tenant string,
+ id tsdb.Identifier,
+ bounds v1.FingerprintBounds,
+) (v1.CloseableIterator[*v1.Series], error) {
+ store, err := s.storeForPeriod(table)
+ if err != nil {
+ return nil, err
+ }
+
+ return store.LoadTSDB(ctx, table, tenant, id, bounds)
+}
diff --git a/pkg/loki/loki.go b/pkg/loki/loki.go
index 6ef0572d6bad0..a83f8ba43394f 100644
--- a/pkg/loki/loki.go
+++ b/pkg/loki/loki.go
@@ -636,7 +636,7 @@ func (t *Loki) setupModuleManager() error {
Compactor: {Server, Overrides, MemberlistKV, Analytics},
IndexGateway: {Server, Store, IndexGatewayRing, IndexGatewayInterceptors, Analytics},
BloomGateway: {Server, BloomGatewayRing, Analytics},
- BloomCompactor: {Server, BloomCompactorRing, Analytics},
+ BloomCompactor: {Server, BloomCompactorRing, Analytics, Store},
IngesterQuerier: {Ring},
QuerySchedulerRing: {Overrides, MemberlistKV},
IndexGatewayRing: {Overrides, MemberlistKV},
diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go
index 592aa12b351cf..23e59e711f29b 100644
--- a/pkg/loki/modules.go
+++ b/pkg/loki/modules.go
@@ -1422,11 +1422,15 @@ func (t *Loki) initBloomCompactor() (services.Service, error) {
compactor, err := bloomcompactor.New(
t.Cfg.BloomCompactor,
- nil, // StoreAndClient placeholder. TODO(owen-d): remove this once we have a proper store and client
+ t.Cfg.SchemaConfig,
+ t.Cfg.StorageConfig,
+ t.clientMetrics,
+ t.Store,
shuffleSharding,
t.Overrides,
logger,
- prometheus.DefaultRegisterer)
+ prometheus.DefaultRegisterer,
+ )
if err != nil {
return nil, err
From 57619b76a4700d001d621212d6117120b18f060c Mon Sep 17 00:00:00 2001
From: Christian Haudum
Date: Mon, 12 Feb 2024 08:25:18 +0100
Subject: [PATCH 14/22] Bloom gateway: Change Summary metrics to Histogram
metrics (#11913)
Even though the main information from these metrics are the `sum` and
the `count`, histograms offer the ability to aggregate them.
Follow up to https://github.com/grafana/loki/pull/11907
Ref: https://prometheus.io/docs/practices/histograms/#quantiles
Signed-off-by: Christian Haudum
---
pkg/bloomgateway/worker.go | 16 ++++++++--------
1 file changed, 8 insertions(+), 8 deletions(-)
diff --git a/pkg/bloomgateway/worker.go b/pkg/bloomgateway/worker.go
index aed42557241b2..5c57c0a2e4952 100644
--- a/pkg/bloomgateway/worker.go
+++ b/pkg/bloomgateway/worker.go
@@ -25,10 +25,10 @@ type workerConfig struct {
}
type workerMetrics struct {
- dequeueDuration *prometheus.SummaryVec
- processDuration *prometheus.SummaryVec
- metasFetched *prometheus.SummaryVec
- blocksFetched *prometheus.SummaryVec
+ dequeueDuration *prometheus.HistogramVec
+ processDuration *prometheus.HistogramVec
+ metasFetched *prometheus.HistogramVec
+ blocksFetched *prometheus.HistogramVec
tasksDequeued *prometheus.CounterVec
tasksProcessed *prometheus.CounterVec
blockQueryLatency *prometheus.HistogramVec
@@ -38,25 +38,25 @@ func newWorkerMetrics(registerer prometheus.Registerer, namespace, subsystem str
labels := []string{"worker"}
r := promauto.With(registerer)
return &workerMetrics{
- dequeueDuration: r.NewSummaryVec(prometheus.SummaryOpts{
+ dequeueDuration: r.NewHistogramVec(prometheus.HistogramOpts{
Namespace: namespace,
Subsystem: subsystem,
Name: "dequeue_duration_seconds",
Help: "Time spent dequeuing tasks from queue in seconds",
}, labels),
- processDuration: r.NewSummaryVec(prometheus.SummaryOpts{
+ processDuration: r.NewHistogramVec(prometheus.HistogramOpts{
Namespace: namespace,
Subsystem: subsystem,
Name: "process_duration_seconds",
Help: "Time spent processing tasks in seconds",
}, append(labels, "status")),
- metasFetched: r.NewSummaryVec(prometheus.SummaryOpts{
+ metasFetched: r.NewHistogramVec(prometheus.HistogramOpts{
Namespace: namespace,
Subsystem: subsystem,
Name: "metas_fetched",
Help: "Amount of metas fetched",
}, labels),
- blocksFetched: r.NewSummaryVec(prometheus.SummaryOpts{
+ blocksFetched: r.NewHistogramVec(prometheus.HistogramOpts{
Namespace: namespace,
Subsystem: subsystem,
Name: "blocks_fetched",
From 681bb57971fed0ebab40a2a42b1e25970f61ac63 Mon Sep 17 00:00:00 2001
From: Salva Corts
Date: Mon, 12 Feb 2024 11:37:13 +0100
Subject: [PATCH 15/22] Limit bloom block size (#11878)
**What this PR does / why we need it**:
This PR limits the size of the blocks created by the compactor.
The block builder keeps adding series' blooms to a block until the limit
is exceeded, meaning that blocks may grow beyond the configured maximum.
This is needed so we avoid having tiny blocks which had space for small
blooms but later a bigger blooms didn't fit.
Blocks are built lazily: the generator returns an iterator that builds
one block at a time.
**Special notes for your reviewer**:
The maximum size is currently set to 50 MBs. We will make this
configurable on a followup PR.
**Checklist**
- [x] Reviewed the
[`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md)
guide (**required**)
- [x] 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)
---------
Co-authored-by: Christian Haudum
---
pkg/bloomcompactor/controller.go | 20 ++++-
pkg/bloomcompactor/spec.go | 113 +++++++++++++++++++--------
pkg/bloomcompactor/spec_test.go | 62 +++++++++------
pkg/storage/bloom/v1/block.go | 4 +
pkg/storage/bloom/v1/builder.go | 59 +++++++++++---
pkg/storage/bloom/v1/builder_test.go | 65 ++++++++++-----
pkg/storage/bloom/v1/fuse_test.go | 3 +-
pkg/storage/bloom/v1/util.go | 2 +-
8 files changed, 234 insertions(+), 94 deletions(-)
diff --git a/pkg/bloomcompactor/controller.go b/pkg/bloomcompactor/controller.go
index dd48351a98ec8..92a6f8ca5f7c5 100644
--- a/pkg/bloomcompactor/controller.go
+++ b/pkg/bloomcompactor/controller.go
@@ -8,6 +8,7 @@ import (
"github.com/go-kit/log"
"github.com/go-kit/log/level"
+ "github.com/grafana/dskit/multierror"
"github.com/pkg/errors"
v1 "github.com/grafana/loki/pkg/storage/bloom/v1"
@@ -134,10 +135,20 @@ func (s *SimpleBloomController) buildBlocks(
level.Error(logger).Log("msg", "failed to get series and blocks", "err", err)
return errors.Wrap(err, "failed to get series and blocks")
}
+ // Close all remaining blocks on exit
+ closePreExistingBlocks := func() {
+ var closeErrors multierror.MultiError
+ for _, block := range preExistingBlocks {
+ closeErrors.Add(block.Close())
+ }
+ if err := closeErrors.Err(); err != nil {
+ level.Error(s.logger).Log("msg", "failed to close blocks", "err", err)
+ }
+ }
gen := NewSimpleBloomGenerator(
tenant,
- v1.DefaultBlockOptions,
+ v1.DefaultBlockOptions, // TODO(salvacorts) make block options configurable
seriesItr,
s.chunkLoader,
preExistingBlocks,
@@ -150,13 +161,14 @@ func (s *SimpleBloomController) buildBlocks(
if err != nil {
// TODO(owen-d): metrics
level.Error(logger).Log("msg", "failed to generate bloom", "err", err)
+ closePreExistingBlocks()
return errors.Wrap(err, "failed to generate bloom")
}
client, err := s.bloomStore.Client(table.ModelTime())
-
if err != nil {
level.Error(logger).Log("msg", "failed to get client", "err", err)
+ closePreExistingBlocks()
return errors.Wrap(err, "failed to get client")
}
for newBlocks.Next() {
@@ -168,6 +180,7 @@ func (s *SimpleBloomController) buildBlocks(
bloomshipper.BlockFrom(tenant, table.String(), blk),
); err != nil {
level.Error(logger).Log("msg", "failed to write block", "err", err)
+ closePreExistingBlocks()
return errors.Wrap(err, "failed to write block")
}
}
@@ -175,9 +188,12 @@ func (s *SimpleBloomController) buildBlocks(
if err := newBlocks.Err(); err != nil {
// TODO(owen-d): metrics
level.Error(logger).Log("msg", "failed to generate bloom", "err", err)
+ closePreExistingBlocks()
return errors.Wrap(err, "failed to generate bloom")
}
+ // Close pre-existing blocks
+ closePreExistingBlocks()
}
}
diff --git a/pkg/bloomcompactor/spec.go b/pkg/bloomcompactor/spec.go
index 7353ca9708121..70ea71c4e605f 100644
--- a/pkg/bloomcompactor/spec.go
+++ b/pkg/bloomcompactor/spec.go
@@ -11,8 +11,6 @@ import (
"github.com/pkg/errors"
"github.com/prometheus/common/model"
- "github.com/grafana/dskit/multierror"
-
"github.com/grafana/loki/pkg/chunkenc"
"github.com/grafana/loki/pkg/logproto"
logql_log "github.com/grafana/loki/pkg/logql/log"
@@ -110,20 +108,7 @@ func (s *SimpleBloomGenerator) populator(ctx context.Context) func(series *v1.Se
}
func (s *SimpleBloomGenerator) Generate(ctx context.Context) (skippedBlocks []v1.BlockMetadata, results v1.Iterator[*v1.Block], err error) {
-
- var closeErrors multierror.MultiError
- blocksMatchingSchema := make([]v1.PeekingIterator[*v1.SeriesWithBloom], 0, len(s.blocks))
- toClose := make([]*bloomshipper.CloseableBlockQuerier, 0, len(s.blocks))
- // Close all remaining blocks on exit
- defer func() {
- for _, block := range toClose {
- closeErrors.Add(block.Close())
- }
- if err := closeErrors.Err(); err != nil {
- level.Error(s.logger).Log("msg", "failed to close blocks", "err", err)
- }
- }()
-
+ blocksMatchingSchema := make([]*bloomshipper.CloseableBlockQuerier, 0, len(s.blocks))
for _, block := range s.blocks {
logger := log.With(s.logger, "block", block.BlockRef)
md, err := block.Metadata()
@@ -131,46 +116,106 @@ func (s *SimpleBloomGenerator) Generate(ctx context.Context) (skippedBlocks []v1
if err != nil {
level.Warn(logger).Log("msg", "failed to get schema for block", "err", err)
skippedBlocks = append(skippedBlocks, md)
-
- // Close unneeded block
- closeErrors.Add(block.Close())
continue
}
if !s.opts.Schema.Compatible(schema) {
level.Warn(logger).Log("msg", "block schema incompatible with options", "generator_schema", fmt.Sprintf("%+v", s.opts.Schema), "block_schema", fmt.Sprintf("%+v", schema))
skippedBlocks = append(skippedBlocks, md)
-
- // Close unneeded block
- closeErrors.Add(block.Close())
continue
}
level.Debug(logger).Log("msg", "adding compatible block to bloom generation inputs")
- itr := v1.NewPeekingIter[*v1.SeriesWithBloom](block)
- blocksMatchingSchema = append(blocksMatchingSchema, itr)
- // append needed block to close list (when finished)
- toClose = append(toClose, block)
+ blocksMatchingSchema = append(blocksMatchingSchema, block)
}
level.Debug(s.logger).Log("msg", "generating bloom filters for blocks", "num_blocks", len(blocksMatchingSchema), "skipped_blocks", len(skippedBlocks), "schema", fmt.Sprintf("%+v", s.opts.Schema))
- // TODO(owen-d): implement bounded block sizes
- mergeBuilder := v1.NewMergeBuilder(blocksMatchingSchema, s.store, s.populator(ctx))
- writer, reader := s.readWriterFn()
+ series := v1.NewPeekingIter(s.store)
+ blockIter := NewLazyBlockBuilderIterator(ctx, s.opts, s.populator(ctx), s.readWriterFn, series, blocksMatchingSchema)
+ return skippedBlocks, blockIter, nil
+}
- blockBuilder, err := v1.NewBlockBuilder(v1.NewBlockOptionsFromSchema(s.opts.Schema), writer)
- if err != nil {
- return skippedBlocks, nil, errors.Wrap(err, "failed to create bloom block builder")
+// LazyBlockBuilderIterator is a lazy iterator over blocks that builds
+// each block by adding series to them until they are full.
+type LazyBlockBuilderIterator struct {
+ ctx context.Context
+ opts v1.BlockOptions
+ populate func(*v1.Series, *v1.Bloom) error
+ readWriterFn func() (v1.BlockWriter, v1.BlockReader)
+ series v1.PeekingIterator[*v1.Series]
+ blocks []*bloomshipper.CloseableBlockQuerier
+
+ blocksAsPeekingIter []v1.PeekingIterator[*v1.SeriesWithBloom]
+ curr *v1.Block
+ err error
+}
+
+func NewLazyBlockBuilderIterator(
+ ctx context.Context,
+ opts v1.BlockOptions,
+ populate func(*v1.Series, *v1.Bloom) error,
+ readWriterFn func() (v1.BlockWriter, v1.BlockReader),
+ series v1.PeekingIterator[*v1.Series],
+ blocks []*bloomshipper.CloseableBlockQuerier,
+) *LazyBlockBuilderIterator {
+ it := &LazyBlockBuilderIterator{
+ ctx: ctx,
+ opts: opts,
+ populate: populate,
+ readWriterFn: readWriterFn,
+ series: series,
+ blocks: blocks,
+
+ blocksAsPeekingIter: make([]v1.PeekingIterator[*v1.SeriesWithBloom], len(blocks)),
}
+ return it
+}
+
+func (b *LazyBlockBuilderIterator) Next() bool {
+ // No more series to process
+ if _, hasNext := b.series.Peek(); !hasNext {
+ return false
+ }
+
+ // reset all the blocks to the start
+ for i, block := range b.blocks {
+ if err := block.Reset(); err != nil {
+ b.err = errors.Wrapf(err, "failed to reset block iterator %d", i)
+ return false
+ }
+ b.blocksAsPeekingIter[i] = v1.NewPeekingIter[*v1.SeriesWithBloom](block)
+ }
+
+ if err := b.ctx.Err(); err != nil {
+ b.err = errors.Wrap(err, "context canceled")
+ return false
+ }
+
+ mergeBuilder := v1.NewMergeBuilder(b.blocksAsPeekingIter, b.series, b.populate)
+ writer, reader := b.readWriterFn()
+ blockBuilder, err := v1.NewBlockBuilder(b.opts, writer)
+ if err != nil {
+ b.err = errors.Wrap(err, "failed to create bloom block builder")
+ return false
+ }
_, err = mergeBuilder.Build(blockBuilder)
if err != nil {
- return skippedBlocks, nil, errors.Wrap(err, "failed to build bloom block")
+ b.err = errors.Wrap(err, "failed to build bloom block")
+ return false
}
- return skippedBlocks, v1.NewSliceIter[*v1.Block]([]*v1.Block{v1.NewBlock(reader)}), nil
+ b.curr = v1.NewBlock(reader)
+ return true
+}
+
+func (b *LazyBlockBuilderIterator) At() *v1.Block {
+ return b.curr
+}
+func (b *LazyBlockBuilderIterator) Err() error {
+ return b.err
}
// IndexLoader loads an index. This helps us do things like
diff --git a/pkg/bloomcompactor/spec_test.go b/pkg/bloomcompactor/spec_test.go
index 443d491ea25be..798d65e2f2bcd 100644
--- a/pkg/bloomcompactor/spec_test.go
+++ b/pkg/bloomcompactor/spec_test.go
@@ -28,7 +28,7 @@ func blocksFromSchemaWithRange(t *testing.T, n int, options v1.BlockOptions, fro
numKeysPerSeries := 10000
data, _ = v1.MkBasicSeriesWithBlooms(numSeries, numKeysPerSeries, fromFP, throughFp, 0, 10000)
- seriesPerBlock := 100 / n
+ seriesPerBlock := numSeries / n
for i := 0; i < n; i++ {
// references for linking in memory reader+writer
@@ -88,24 +88,35 @@ func dummyBloomGen(opts v1.BlockOptions, store v1.Iterator[*v1.Series], blocks [
}
func TestSimpleBloomGenerator(t *testing.T) {
+ const maxBlockSize = 100 << 20 // 100MB
for _, tc := range []struct {
- desc string
- fromSchema, toSchema v1.BlockOptions
- sourceBlocks, numSkipped int
+ desc string
+ fromSchema, toSchema v1.BlockOptions
+ sourceBlocks, numSkipped, outputBlocks int
}{
{
desc: "SkipsIncompatibleSchemas",
- fromSchema: v1.NewBlockOptions(3, 0),
- toSchema: v1.NewBlockOptions(4, 0),
+ fromSchema: v1.NewBlockOptions(3, 0, maxBlockSize),
+ toSchema: v1.NewBlockOptions(4, 0, maxBlockSize),
sourceBlocks: 2,
numSkipped: 2,
+ outputBlocks: 1,
},
{
desc: "CombinesBlocks",
- fromSchema: v1.NewBlockOptions(4, 0),
- toSchema: v1.NewBlockOptions(4, 0),
+ fromSchema: v1.NewBlockOptions(4, 0, maxBlockSize),
+ toSchema: v1.NewBlockOptions(4, 0, maxBlockSize),
sourceBlocks: 2,
numSkipped: 0,
+ outputBlocks: 1,
+ },
+ {
+ desc: "MaxBlockSize",
+ fromSchema: v1.NewBlockOptions(4, 0, maxBlockSize),
+ toSchema: v1.NewBlockOptions(4, 0, 1<<10), // 1KB
+ sourceBlocks: 2,
+ numSkipped: 0,
+ outputBlocks: 3,
},
} {
t.Run(tc.desc, func(t *testing.T) {
@@ -122,22 +133,25 @@ func TestSimpleBloomGenerator(t *testing.T) {
require.Nil(t, err)
require.Equal(t, tc.numSkipped, len(skipped))
- require.True(t, results.Next())
- block := results.At()
- require.False(t, results.Next())
-
- refs := v1.PointerSlice[v1.SeriesWithBloom](data)
-
- v1.EqualIterators[*v1.SeriesWithBloom](
- t,
- func(a, b *v1.SeriesWithBloom) {
- // TODO(owen-d): better equality check
- // once chunk fetching is implemented
- require.Equal(t, a.Series, b.Series)
- },
- v1.NewSliceIter[*v1.SeriesWithBloom](refs),
- block.Querier(),
- )
+ var outputBlocks []*v1.Block
+ for results.Next() {
+ outputBlocks = append(outputBlocks, results.At())
+ }
+ require.Equal(t, tc.outputBlocks, len(outputBlocks))
+
+ // Check all the input series are present in the output blocks.
+ expectedRefs := v1.PointerSlice(data)
+ outputRefs := make([]*v1.SeriesWithBloom, 0, len(data))
+ for _, block := range outputBlocks {
+ bq := block.Querier()
+ for bq.Next() {
+ outputRefs = append(outputRefs, bq.At())
+ }
+ }
+ require.Equal(t, len(expectedRefs), len(outputRefs))
+ for i := range expectedRefs {
+ require.Equal(t, expectedRefs[i].Series, outputRefs[i].Series)
+ }
})
}
}
diff --git a/pkg/storage/bloom/v1/block.go b/pkg/storage/bloom/v1/block.go
index 0b10b9d5b058e..09cc5fa4866e7 100644
--- a/pkg/storage/bloom/v1/block.go
+++ b/pkg/storage/bloom/v1/block.go
@@ -135,6 +135,10 @@ func (bq *BlockQuerier) Schema() (Schema, error) {
return bq.block.Schema()
}
+func (bq *BlockQuerier) Reset() error {
+ return bq.series.Seek(0)
+}
+
func (bq *BlockQuerier) Seek(fp model.Fingerprint) error {
return bq.series.Seek(fp)
}
diff --git a/pkg/storage/bloom/v1/builder.go b/pkg/storage/bloom/v1/builder.go
index ac7a83baad374..8821816958b6f 100644
--- a/pkg/storage/bloom/v1/builder.go
+++ b/pkg/storage/bloom/v1/builder.go
@@ -15,7 +15,7 @@ import (
)
var (
- DefaultBlockOptions = NewBlockOptions(4, 0)
+ DefaultBlockOptions = NewBlockOptions(4, 0, 50<<20) // 50MB
)
type BlockOptions struct {
@@ -65,16 +65,19 @@ func (b BlockOptions) Encode(enc *encoding.Encbuf) {
type BlockBuilder struct {
opts BlockOptions
+ writer BlockWriter
index *IndexBuilder
blooms *BloomBlockBuilder
}
-func NewBlockOptions(NGramLength, NGramSkip uint64) BlockOptions {
- return NewBlockOptionsFromSchema(Schema{
+func NewBlockOptions(NGramLength, NGramSkip, MaxBlockSizeBytes uint64) BlockOptions {
+ opts := NewBlockOptionsFromSchema(Schema{
version: byte(1),
nGramLength: NGramLength,
nGramSkip: NGramSkip,
})
+ opts.BlockSize = MaxBlockSizeBytes
+ return opts
}
func NewBlockOptionsFromSchema(s Schema) BlockOptions {
@@ -98,6 +101,7 @@ func NewBlockBuilder(opts BlockOptions, writer BlockWriter) (*BlockBuilder, erro
return &BlockBuilder{
opts: opts,
+ writer: writer,
index: NewIndexBuilder(opts, index),
blooms: NewBloomBlockBuilder(opts, blooms),
}, nil
@@ -110,10 +114,13 @@ type SeriesWithBloom struct {
func (b *BlockBuilder) BuildFrom(itr Iterator[SeriesWithBloom]) (uint32, error) {
for itr.Next() {
- if err := b.AddSeries(itr.At()); err != nil {
+ blockFull, err := b.AddSeries(itr.At())
+ if err != nil {
return 0, err
}
-
+ if blockFull {
+ break
+ }
}
if err := itr.Err(); err != nil {
@@ -135,20 +142,40 @@ func (b *BlockBuilder) Close() (uint32, error) {
return combineChecksums(indexCheckSum, bloomChecksum), nil
}
-func (b *BlockBuilder) AddSeries(series SeriesWithBloom) error {
+// AddSeries adds a series to the block. It returns true after adding the series, the block is full.
+func (b *BlockBuilder) AddSeries(series SeriesWithBloom) (bool, error) {
offset, err := b.blooms.Append(series)
if err != nil {
- return errors.Wrapf(err, "writing bloom for series %v", series.Series.Fingerprint)
+ return false, errors.Wrapf(err, "writing bloom for series %v", series.Series.Fingerprint)
}
if err := b.index.Append(SeriesWithOffset{
Offset: offset,
Series: *series.Series,
}); err != nil {
- return errors.Wrapf(err, "writing index for series %v", series.Series.Fingerprint)
+ return false, errors.Wrapf(err, "writing index for series %v", series.Series.Fingerprint)
}
- return nil
+ full, err := b.isBlockFull()
+ if err != nil {
+ return false, errors.Wrap(err, "checking if block is full")
+ }
+
+ return full, nil
+}
+
+func (b *BlockBuilder) isBlockFull() (bool, error) {
+ // if the block size is 0, the max size is unlimited
+ if b.opts.BlockSize == 0 {
+ return false, nil
+ }
+
+ size, err := b.writer.Size()
+ if err != nil {
+ return false, errors.Wrap(err, "getting block size")
+ }
+
+ return uint64(size) >= b.opts.BlockSize, nil
}
type BloomBlockBuilder struct {
@@ -505,7 +532,11 @@ type MergeBuilder struct {
// 1. merges multiple blocks into a single ordered querier,
// i) When two blocks have the same series, it will prefer the one with the most chunks already indexed
// 2. iterates through the store, adding chunks to the relevant blooms via the `populate` argument
-func NewMergeBuilder(blocks []PeekingIterator[*SeriesWithBloom], store Iterator[*Series], populate func(*Series, *Bloom) error) *MergeBuilder {
+func NewMergeBuilder(
+ blocks []PeekingIterator[*SeriesWithBloom],
+ store Iterator[*Series],
+ populate func(*Series, *Bloom) error,
+) *MergeBuilder {
return &MergeBuilder{
blocks: blocks,
store: store,
@@ -513,8 +544,6 @@ func NewMergeBuilder(blocks []PeekingIterator[*SeriesWithBloom], store Iterator[
}
}
-// NB: this will build one block. Ideally we would build multiple blocks once a target size threshold is met
-// but this gives us a good starting point.
func (mb *MergeBuilder) Build(builder *BlockBuilder) (uint32, error) {
var (
nextInBlocks *SeriesWithBloom
@@ -585,9 +614,13 @@ func (mb *MergeBuilder) Build(builder *BlockBuilder) (uint32, error) {
}
}
- if err := builder.AddSeries(*cur); err != nil {
+ blockFull, err := builder.AddSeries(*cur)
+ if err != nil {
return 0, errors.Wrap(err, "adding series to block")
}
+ if blockFull {
+ break
+ }
}
checksum, err := builder.Close()
diff --git a/pkg/storage/bloom/v1/builder_test.go b/pkg/storage/bloom/v1/builder_test.go
index 36db0e1f193ce..ac38cdeec427a 100644
--- a/pkg/storage/bloom/v1/builder_test.go
+++ b/pkg/storage/bloom/v1/builder_test.go
@@ -3,6 +3,7 @@ package v1
import (
"bytes"
"errors"
+ "sort"
"testing"
"github.com/prometheus/common/model"
@@ -48,9 +49,11 @@ func TestBlockBuilderRoundTrip(t *testing.T) {
tmpDir := t.TempDir()
for _, tc := range []struct {
- desc string
- writer BlockWriter
- reader BlockReader
+ desc string
+ writer BlockWriter
+ reader BlockReader
+ maxBlockSize uint64
+ iterHasPendingData bool
}{
{
desc: "in-memory",
@@ -62,6 +65,14 @@ func TestBlockBuilderRoundTrip(t *testing.T) {
writer: NewDirectoryBlockWriter(tmpDir),
reader: NewDirectoryBlockReader(tmpDir),
},
+ {
+ desc: "max block size",
+ writer: NewDirectoryBlockWriter(tmpDir),
+ reader: NewDirectoryBlockReader(tmpDir),
+ // Set max block big enough to fit a bunch of series but not all of them
+ maxBlockSize: 50 << 10,
+ iterHasPendingData: true,
+ },
} {
t.Run(tc.desc, func(t *testing.T) {
schema := Schema{
@@ -76,14 +87,27 @@ func TestBlockBuilderRoundTrip(t *testing.T) {
Schema: schema,
SeriesPageSize: 100,
BloomPageSize: 10 << 10,
+ BlockSize: tc.maxBlockSize,
},
tc.writer,
)
require.Nil(t, err)
- itr := NewSliceIter[SeriesWithBloom](data)
+ itr := NewPeekingIter[SeriesWithBloom](NewSliceIter[SeriesWithBloom](data))
_, err = builder.BuildFrom(itr)
require.Nil(t, err)
+
+ firstPendingSeries, iterHasPendingData := itr.Peek()
+ require.Equal(t, tc.iterHasPendingData, iterHasPendingData)
+
+ processedData := data
+ if iterHasPendingData {
+ lastProcessedIdx := sort.Search(len(data), func(i int) bool {
+ return data[i].Series.Fingerprint >= firstPendingSeries.Series.Fingerprint
+ })
+ processedData = data[:lastProcessedIdx]
+ }
+
block := NewBlock(tc.reader)
querier := NewBlockQuerier(block)
@@ -91,10 +115,11 @@ func TestBlockBuilderRoundTrip(t *testing.T) {
require.Nil(t, err)
require.Equal(t, block.blooms.schema, schema)
- for i := 0; i < len(data); i++ {
+ // Check processed data can be queried
+ for i := 0; i < len(processedData); i++ {
require.Equal(t, true, querier.Next(), "on iteration %d with error %v", i, querier.Err())
got := querier.At()
- require.Equal(t, data[i].Series, got.Series)
+ require.Equal(t, processedData[i].Series, got.Series)
for _, key := range keys[i] {
require.True(t, got.Bloom.Test(key))
}
@@ -104,20 +129,22 @@ func TestBlockBuilderRoundTrip(t *testing.T) {
require.False(t, querier.Next())
// test seek
- i := numSeries / 2
- halfData := data[i:]
- halfKeys := keys[i:]
- require.Nil(t, querier.Seek(halfData[0].Series.Fingerprint))
- for j := 0; j < len(halfData); j++ {
- require.Equal(t, true, querier.Next(), "on iteration %d", j)
- got := querier.At()
- require.Equal(t, halfData[j].Series, got.Series)
- for _, key := range halfKeys[j] {
- require.True(t, got.Bloom.Test(key))
+ if !iterHasPendingData {
+ i := numSeries / 2
+ halfData := data[i:]
+ halfKeys := keys[i:]
+ require.NoError(t, querier.Seek(halfData[0].Series.Fingerprint))
+ for j := 0; j < len(halfData); j++ {
+ require.Equal(t, true, querier.Next(), "on iteration %d", j)
+ got := querier.At()
+ require.Equal(t, halfData[j].Series, got.Series)
+ for _, key := range halfKeys[j] {
+ require.True(t, got.Bloom.Test(key))
+ }
+ require.NoError(t, querier.Err())
}
- require.NoError(t, querier.Err())
+ require.False(t, querier.Next())
}
- require.False(t, querier.Next())
})
}
@@ -357,7 +384,7 @@ func TestMergeBuilder_Roundtrip(t *testing.T) {
return nil
},
)
- builder, err := NewBlockBuilder(NewBlockOptions(4, 0), writer)
+ builder, err := NewBlockBuilder(DefaultBlockOptions, writer)
require.Nil(t, err)
checksum, err := mb.Build(builder)
diff --git a/pkg/storage/bloom/v1/fuse_test.go b/pkg/storage/bloom/v1/fuse_test.go
index 7b23e1c133361..2ab0a51bced2c 100644
--- a/pkg/storage/bloom/v1/fuse_test.go
+++ b/pkg/storage/bloom/v1/fuse_test.go
@@ -36,7 +36,8 @@ func TestFusedQuerier(t *testing.T) {
require.Nil(t, err)
itr := NewSliceIter[SeriesWithBloom](data)
_, err = builder.BuildFrom(itr)
- require.Nil(t, err)
+ require.NoError(t, err)
+ require.False(t, itr.Next())
block := NewBlock(reader)
querier := NewBlockQuerier(block)
diff --git a/pkg/storage/bloom/v1/util.go b/pkg/storage/bloom/v1/util.go
index 27d347bfb4d0f..d980a9ecc4df1 100644
--- a/pkg/storage/bloom/v1/util.go
+++ b/pkg/storage/bloom/v1/util.go
@@ -258,7 +258,7 @@ type PeekCloseIter[T any] struct {
}
func NewPeekCloseIter[T any](itr CloseableIterator[T]) *PeekCloseIter[T] {
- return &PeekCloseIter[T]{PeekIter: NewPeekingIter(itr), close: itr.Close}
+ return &PeekCloseIter[T]{PeekIter: NewPeekingIter[T](itr), close: itr.Close}
}
func (it *PeekCloseIter[T]) Close() error {
From 7a95cb86633215805a4b4b7d9399c71c38f43584 Mon Sep 17 00:00:00 2001
From: Salva Corts
Date: Mon, 12 Feb 2024 15:25:51 +0100
Subject: [PATCH 16/22] Configurable bloom tokenizer and block settings
(#11889)
**What this PR does / why we need it**:
This PR makes the max block size configurable and wires up the already
existing nGramLen and nGramSkip settings.
**Special notes for your reviewer**:
**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)
---
docs/sources/configure/_index.md | 6 ++++++
pkg/bloomcompactor/bloomcompactor.go | 1 +
pkg/bloomcompactor/config.go | 1 +
pkg/bloomcompactor/controller.go | 10 +++++++++-
pkg/validation/limits.go | 26 +++++++++++++++++---------
5 files changed, 34 insertions(+), 10 deletions(-)
diff --git a/docs/sources/configure/_index.md b/docs/sources/configure/_index.md
index c3854e434f4da..e79a2503176fc 100644
--- a/docs/sources/configure/_index.md
+++ b/docs/sources/configure/_index.md
@@ -3140,6 +3140,12 @@ shard_streams:
# CLI flag: -bloom-gateway.cache-key-interval
[bloom_gateway_cache_key_interval: | default = 15m]
+# The maximum bloom block size. A value of 0 sets an unlimited size. Default is
+# 200MB. The actual block size might exceed this limit since blooms will be
+# added to blocks until the block exceeds the maximum block size.
+# CLI flag: -bloom-compactor.max-block-size
+[bloom_compactor_max_block_size: | default = 200MB]
+
# Allow user to send structured metadata in push payload.
# CLI flag: -validation.allow-structured-metadata
[allow_structured_metadata: | default = false]
diff --git a/pkg/bloomcompactor/bloomcompactor.go b/pkg/bloomcompactor/bloomcompactor.go
index a91b6796e5360..8a3e7c6266c1d 100644
--- a/pkg/bloomcompactor/bloomcompactor.go
+++ b/pkg/bloomcompactor/bloomcompactor.go
@@ -98,6 +98,7 @@ func New(
c.tsdbStore,
c.bloomStore,
chunkLoader,
+ c.limits,
c.metrics,
c.logger,
)
diff --git a/pkg/bloomcompactor/config.go b/pkg/bloomcompactor/config.go
index 37c8844378465..37aac3310829a 100644
--- a/pkg/bloomcompactor/config.go
+++ b/pkg/bloomcompactor/config.go
@@ -63,6 +63,7 @@ type Limits interface {
BloomNGramLength(tenantID string) int
BloomNGramSkip(tenantID string) int
BloomFalsePositiveRate(tenantID string) float64
+ BloomCompactorMaxBlockSize(tenantID string) int
}
// TODO(owen-d): Remove this type in favor of config.DayTime
diff --git a/pkg/bloomcompactor/controller.go b/pkg/bloomcompactor/controller.go
index 92a6f8ca5f7c5..cf6fff090f0ae 100644
--- a/pkg/bloomcompactor/controller.go
+++ b/pkg/bloomcompactor/controller.go
@@ -21,6 +21,7 @@ type SimpleBloomController struct {
bloomStore bloomshipper.Store
chunkLoader ChunkLoader
metrics *Metrics
+ limits Limits
// TODO(owen-d): add metrics
logger log.Logger
@@ -30,6 +31,7 @@ func NewSimpleBloomController(
tsdbStore TSDBStore,
blockStore bloomshipper.Store,
chunkLoader ChunkLoader,
+ limits Limits,
metrics *Metrics,
logger log.Logger,
) *SimpleBloomController {
@@ -38,6 +40,7 @@ func NewSimpleBloomController(
bloomStore: blockStore,
chunkLoader: chunkLoader,
metrics: metrics,
+ limits: limits,
logger: logger,
}
}
@@ -110,6 +113,11 @@ func (s *SimpleBloomController) buildBlocks(
return errors.Wrap(err, "failed to create plan")
}
+ nGramSize := uint64(s.limits.BloomNGramLength(tenant))
+ nGramSkip := uint64(s.limits.BloomNGramSkip(tenant))
+ maxBlockSize := uint64(s.limits.BloomCompactorMaxBlockSize(tenant))
+ blockOpts := v1.NewBlockOptions(nGramSize, nGramSkip, maxBlockSize)
+
// 4. Generate Blooms
// Now that we have the gaps, we will generate a bloom block for each gap.
// We can accelerate this by using existing blocks which may already contain
@@ -148,7 +156,7 @@ func (s *SimpleBloomController) buildBlocks(
gen := NewSimpleBloomGenerator(
tenant,
- v1.DefaultBlockOptions, // TODO(salvacorts) make block options configurable
+ blockOpts,
seriesItr,
s.chunkLoader,
preExistingBlocks,
diff --git a/pkg/validation/limits.go b/pkg/validation/limits.go
index e3052c1781b89..262631643c723 100644
--- a/pkg/validation/limits.go
+++ b/pkg/validation/limits.go
@@ -56,6 +56,7 @@ const (
defaultMaxStructuredMetadataSize = "64kb"
defaultMaxStructuredMetadataCount = 128
+ defaultBloomCompactorMaxBlockSize = "200MB"
)
// Limits describe all the limits for users; can be used to describe global default
@@ -187,15 +188,16 @@ type Limits struct {
BloomGatewayShardSize int `yaml:"bloom_gateway_shard_size" json:"bloom_gateway_shard_size"`
BloomGatewayEnabled bool `yaml:"bloom_gateway_enable_filtering" json:"bloom_gateway_enable_filtering"`
- BloomCompactorShardSize int `yaml:"bloom_compactor_shard_size" json:"bloom_compactor_shard_size"`
- BloomCompactorMaxTableAge time.Duration `yaml:"bloom_compactor_max_table_age" json:"bloom_compactor_max_table_age"`
- BloomCompactorEnabled bool `yaml:"bloom_compactor_enable_compaction" json:"bloom_compactor_enable_compaction"`
- BloomCompactorChunksBatchSize int `yaml:"bloom_compactor_chunks_batch_size" json:"bloom_compactor_chunks_batch_size"`
- BloomNGramLength int `yaml:"bloom_ngram_length" json:"bloom_ngram_length"`
- BloomNGramSkip int `yaml:"bloom_ngram_skip" json:"bloom_ngram_skip"`
- BloomFalsePositiveRate float64 `yaml:"bloom_false_positive_rate" json:"bloom_false_positive_rate"`
- BloomGatewayBlocksDownloadingParallelism int `yaml:"bloom_gateway_blocks_downloading_parallelism" json:"bloom_gateway_blocks_downloading_parallelism"`
- BloomGatewayCacheKeyInterval time.Duration `yaml:"bloom_gateway_cache_key_interval" json:"bloom_gateway_cache_key_interval"`
+ BloomCompactorShardSize int `yaml:"bloom_compactor_shard_size" json:"bloom_compactor_shard_size"`
+ BloomCompactorMaxTableAge time.Duration `yaml:"bloom_compactor_max_table_age" json:"bloom_compactor_max_table_age"`
+ BloomCompactorEnabled bool `yaml:"bloom_compactor_enable_compaction" json:"bloom_compactor_enable_compaction"`
+ BloomCompactorChunksBatchSize int `yaml:"bloom_compactor_chunks_batch_size" json:"bloom_compactor_chunks_batch_size"`
+ BloomNGramLength int `yaml:"bloom_ngram_length" json:"bloom_ngram_length"`
+ BloomNGramSkip int `yaml:"bloom_ngram_skip" json:"bloom_ngram_skip"`
+ BloomFalsePositiveRate float64 `yaml:"bloom_false_positive_rate" json:"bloom_false_positive_rate"`
+ BloomGatewayBlocksDownloadingParallelism int `yaml:"bloom_gateway_blocks_downloading_parallelism" json:"bloom_gateway_blocks_downloading_parallelism"`
+ BloomGatewayCacheKeyInterval time.Duration `yaml:"bloom_gateway_cache_key_interval" json:"bloom_gateway_cache_key_interval"`
+ BloomCompactorMaxBlockSize flagext.ByteSize `yaml:"bloom_compactor_max_block_size" json:"bloom_compactor_max_block_size"`
AllowStructuredMetadata bool `yaml:"allow_structured_metadata,omitempty" json:"allow_structured_metadata,omitempty" doc:"description=Allow user to send structured metadata in push payload."`
MaxStructuredMetadataSize flagext.ByteSize `yaml:"max_structured_metadata_size" json:"max_structured_metadata_size" doc:"description=Maximum size accepted for structured metadata per log line."`
@@ -333,6 +335,8 @@ func (l *Limits) RegisterFlags(f *flag.FlagSet) {
f.Float64Var(&l.BloomFalsePositiveRate, "bloom-compactor.false-positive-rate", 0.01, "Scalable Bloom Filter desired false-positive rate.")
f.IntVar(&l.BloomGatewayBlocksDownloadingParallelism, "bloom-gateway.blocks-downloading-parallelism", 50, "Maximum number of blocks will be downloaded in parallel by the Bloom Gateway.")
f.DurationVar(&l.BloomGatewayCacheKeyInterval, "bloom-gateway.cache-key-interval", 15*time.Minute, "Interval for computing the cache key in the Bloom Gateway.")
+ _ = l.BloomCompactorMaxBlockSize.Set(defaultBloomCompactorMaxBlockSize)
+ f.Var(&l.BloomCompactorMaxBlockSize, "bloom-compactor.max-block-size", "The maximum bloom block size. A value of 0 sets an unlimited size. Default is 200MB. The actual block size might exceed this limit since blooms will be added to blocks until the block exceeds the maximum block size.")
l.ShardStreams = &shardstreams.Config{}
l.ShardStreams.RegisterFlagsWithPrefix("shard-streams", f)
@@ -882,6 +886,10 @@ func (o *Overrides) BloomNGramSkip(userID string) int {
return o.getOverridesForUser(userID).BloomNGramSkip
}
+func (o *Overrides) BloomCompactorMaxBlockSize(userID string) int {
+ return o.getOverridesForUser(userID).BloomCompactorMaxBlockSize.Val()
+}
+
func (o *Overrides) BloomFalsePositiveRate(userID string) float64 {
return o.getOverridesForUser(userID).BloomFalsePositiveRate
}
From d9d9ebd29d013ac37df414d364dd8b5c491313ca Mon Sep 17 00:00:00 2001
From: Zirko <64951262+QuantumEnigmaa@users.noreply.github.com>
Date: Mon, 12 Feb 2024 23:03:57 +0100
Subject: [PATCH 17/22] Helm: fix egress-dicovery netpols (#11838)
**What this PR does / why we need it**:
This PR fix an issue happening when deploying the `egress-discovery`
networkPolicy or ciliumNetworkPolicy. When one wants to deploy the
`egress-discovery` netpol, one only has to specify the
`networkPolicy.discovery.port` field in the values. However, with the
current state of the templates, when this field is specified, **both**
the networkPolicy and the ciliumNetworkPolicy are generated since the
only condition for their creation is `if
.Values.networkPolicy.discovery.port`.
I thus added an additional condition : `eq .Values.networkPolicy.flavor
"cilium"` (or `"kubernetes"` for the networkPolicy) so that only one of
these is generated according to the `flavor` specified in the values.
**Special notes for your reviewer**:
**Checklist**
- [x] Reviewed the
[`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md)
guide (**required**)
- [ ] Documentation added
- [ ] Tests updated
- [x] `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`
- [x] For Helm chart changes bump the Helm chart version in
`production/helm/loki/Chart.yaml` and update
`production/helm/loki/CHANGELOG.md` and
`production/helm/loki/README.md`. [Example
PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213)
- [ ] If the change is deprecating or removing a configuration option,
update the `deprecated-config.yaml` and `deleted-config.yaml` files
respectively in the `tools/deprecated-config-checker` directory.
[Example
PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15)
---------
Signed-off-by: QuantumEnigmaa
Co-authored-by: Michel Hollands <42814411+MichelHollands@users.noreply.github.com>
---
production/helm/loki/CHANGELOG.md | 1 +
production/helm/loki/templates/ciliumnetworkpolicy.yaml | 2 +-
production/helm/loki/templates/networkpolicy.yaml | 2 +-
3 files changed, 3 insertions(+), 2 deletions(-)
diff --git a/production/helm/loki/CHANGELOG.md b/production/helm/loki/CHANGELOG.md
index 9a5b2a3cb4f27..e849918585ea2 100644
--- a/production/helm/loki/CHANGELOG.md
+++ b/production/helm/loki/CHANGELOG.md
@@ -15,6 +15,7 @@ Entries should include a reference to the pull request that introduced the chang
## 5.42.2
+- [BUGFIX] Added condition for `egress-discovery` networkPolicies and ciliumNetworkPolicies.
- [BUGFIX] Remove trailing tab character in statefulset templates
## 5.42.1
diff --git a/production/helm/loki/templates/ciliumnetworkpolicy.yaml b/production/helm/loki/templates/ciliumnetworkpolicy.yaml
index 5633ae1945206..ddcef3b61a8ec 100644
--- a/production/helm/loki/templates/ciliumnetworkpolicy.yaml
+++ b/production/helm/loki/templates/ciliumnetworkpolicy.yaml
@@ -156,7 +156,7 @@ spec:
{{- end }}
-{{- if .Values.networkPolicy.discovery.port }}
+{{- if and .Values.networkPolicy.discovery.port (eq .Values.networkPolicy.flavor "cilium") }}
---
apiVersion: cilium.io/v2
kind: CiliumNetworkPolicy
diff --git a/production/helm/loki/templates/networkpolicy.yaml b/production/helm/loki/templates/networkpolicy.yaml
index 27c85280eb08c..5052e81162b3d 100644
--- a/production/helm/loki/templates/networkpolicy.yaml
+++ b/production/helm/loki/templates/networkpolicy.yaml
@@ -172,7 +172,7 @@ spec:
{{- end }}
-{{- if .Values.networkPolicy.discovery.port }}
+{{- if and .Values.networkPolicy.discovery.port (eq .Values.networkPolicy.flavor "kubernetes") }}
---
apiVersion: networking.k8s.io/v1
kind: NetworkPolicy
From c277158f515f0ba4ad27e7189dea4da9f50e7e4c Mon Sep 17 00:00:00 2001
From: Paul Rogers <129207811+paul1r@users.noreply.github.com>
Date: Mon, 12 Feb 2024 18:52:27 -0500
Subject: [PATCH 18/22] Cleanup data race associated with workerID var (#11922)
**What this PR does / why we need it**:
A data race existed with the workerID variable, as it could be modified
by multiple goroutines.
Relates to: https://github.com/grafana/loki/issues/8586
--
Before fix:
```
go test -count=1 -race ./pkg/querier/worker
==================
WARNING: DATA RACE
Read at 0x00c000494108 by goroutine 229:
github.com/grafana/loki/pkg/querier/worker.(*processorManager).concurrency.func1()
/Users/progers/dev/src/github.com/grafana/loki/pkg/querier/worker/processor_manager.go:81 +0x118
Previous write at 0x00c000494108 by goroutine 222:
github.com/grafana/loki/pkg/querier/worker.(*processorManager).concurrency()
/Users/progers/dev/src/github.com/grafana/loki/pkg/querier/worker/processor_manager.go:70 +0x108
github.com/grafana/loki/pkg/querier/worker.(*querierWorker).resetConcurrency()
/Users/progers/dev/src/github.com/grafana/loki/pkg/querier/worker/worker.go:267 +0x10c
github.com/grafana/loki/pkg/querier/worker.(*querierWorker).AddressAdded()
/Users/progers/dev/src/github.com/grafana/loki/pkg/querier/worker/worker.go:219 +0x868
github.com/grafana/loki/pkg/querier/worker.TestResetConcurrency.func1()
/Users/progers/dev/src/github.com/grafana/loki/pkg/querier/worker/worker_test.go:64 +0x1c8
testing.tRunner()
/opt/homebrew/Cellar/go/1.21.6/libexec/src/testing/testing.go:1595 +0x1b0
testing.(*T).Run.func1()
/opt/homebrew/Cellar/go/1.21.6/libexec/src/testing/testing.go:1648 +0x40
Goroutine 229 (running) created at:
github.com/grafana/loki/pkg/querier/worker.(*processorManager).concurrency()
/Users/progers/dev/src/github.com/grafana/loki/pkg/querier/worker/processor_manager.go:75 +0xcc
github.com/grafana/loki/pkg/querier/worker.(*querierWorker).resetConcurrency()
/Users/progers/dev/src/github.com/grafana/loki/pkg/querier/worker/worker.go:267 +0x10c
github.com/grafana/loki/pkg/querier/worker.(*querierWorker).AddressAdded()
/Users/progers/dev/src/github.com/grafana/loki/pkg/querier/worker/worker.go:219 +0x868
github.com/grafana/loki/pkg/querier/worker.TestResetConcurrency.func1()
/Users/progers/dev/src/github.com/grafana/loki/pkg/querier/worker/worker_test.go:64 +0x1c8
testing.tRunner()
/opt/homebrew/Cellar/go/1.21.6/libexec/src/testing/testing.go:1595 +0x1b0
testing.(*T).Run.func1()
/opt/homebrew/Cellar/go/1.21.6/libexec/src/testing/testing.go:1648 +0x40
Goroutine 222 (running) created at:
testing.(*T).Run()
/opt/homebrew/Cellar/go/1.21.6/libexec/src/testing/testing.go:1648 +0x5e8
github.com/grafana/loki/pkg/querier/worker.TestResetConcurrency()
/Users/progers/dev/src/github.com/grafana/loki/pkg/querier/worker/worker_test.go:52 +0x1b0
testing.tRunner()
/opt/homebrew/Cellar/go/1.21.6/libexec/src/testing/testing.go:1595 +0x1b0
testing.(*T).Run.func1()
/opt/homebrew/Cellar/go/1.21.6/libexec/src/testing/testing.go:1648 +0x40
==================
--- FAIL: TestResetConcurrency (0.02s)
--- FAIL: TestResetConcurrency/concurrency_is_correct_when_numTargets_does_not_divide_evenly_into_maxConcurrent (0.01s)
testing.go:1465: race detected during execution of test
testing.go:1465: race detected during execution of test
FAIL
FAIL github.com/grafana/loki/pkg/querier/worker 4.626s
FAIL
```
--
After fix:
```
go clean -testcache
go test -count=1 -race ./pkg/querier/worker
ok github.com/grafana/loki/pkg/querier/worker 6.034s
```
**Which issue(s) this PR fixes**:
Fixes #
**Special notes for your reviewer**:
**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)
---
pkg/querier/worker/processor_manager.go | 7 +++----
1 file changed, 3 insertions(+), 4 deletions(-)
diff --git a/pkg/querier/worker/processor_manager.go b/pkg/querier/worker/processor_manager.go
index 3a2c8c338865d..74c9517f86c24 100644
--- a/pkg/querier/worker/processor_manager.go
+++ b/pkg/querier/worker/processor_manager.go
@@ -65,21 +65,20 @@ func (pm *processorManager) concurrency(n int) {
n = 0
}
- workerID := 0
for len(pm.cancels) < n {
- workerID++
+ workerID := len(pm.cancels) + 1
ctx, cancel := context.WithCancel(pm.ctx)
pm.cancels = append(pm.cancels, cancel)
pm.wg.Add(1)
- go func() {
+ go func(workerID int) {
defer pm.wg.Done()
pm.currentProcessors.Inc()
defer pm.currentProcessors.Dec()
pm.p.processQueriesOnSingleStream(ctx, pm.conn, pm.address, strconv.Itoa(workerID))
- }()
+ }(workerID)
}
for len(pm.cancels) > n {
From 75600674397f270e9d4a57303de9dc468a58faaf Mon Sep 17 00:00:00 2001
From: Owen Diehl
Date: Mon, 12 Feb 2024 16:31:05 -0800
Subject: [PATCH 19/22] Bloom/running (#11918)
Includes a few small fixes in order to get the compactor running,
including
* Wires up i/o from `v1` pkg and the `bloomshipper` pkg in order to
archive+write blocks remotely after building
* config error handling
* fix an error iterating chunks
* handles gzipping when loading tsdbs from storage
* bypasses internal cache in `BloomTSDBStore`'s internal
`indexshipper/storage.Client` to get up to date results, which is easier
for prototyping
* `ClosableReadSeekerAdapter`
---------
Signed-off-by: Christian Haudum
Signed-off-by: Owen Diehl
Co-authored-by: Christian Haudum
---
docs/sources/configure/_index.md | 23 ++++++++-------
integration/cluster/cluster.go | 1 -
pkg/bloomcompactor/bloomcompactor.go | 16 ++++++++--
pkg/bloomcompactor/config.go | 29 +++++++++++--------
pkg/bloomcompactor/controller.go | 11 +++++--
pkg/bloomcompactor/spec.go | 23 ++++++++++++---
pkg/bloomcompactor/tsdb.go | 23 +++++++++++----
pkg/loki/loki.go | 5 +++-
pkg/loki/modules.go | 2 +-
pkg/storage/bloom/v1/block.go | 4 +++
pkg/storage/bloom/v1/bloom_tokenizer.go | 2 +-
.../stores/shipper/bloomshipper/client.go | 29 +++++++++++++++++--
12 files changed, 124 insertions(+), 44 deletions(-)
diff --git a/docs/sources/configure/_index.md b/docs/sources/configure/_index.md
index e79a2503176fc..d5dd9b43bd146 100644
--- a/docs/sources/configure/_index.md
+++ b/docs/sources/configure/_index.md
@@ -2641,21 +2641,22 @@ ring:
# CLI flag: -bloom-compactor.enabled
[enabled: | default = false]
-# Directory where files can be downloaded for compaction.
-# CLI flag: -bloom-compactor.working-directory
-[working_directory: | default = ""]
-
# Interval at which to re-run the compaction operation.
# CLI flag: -bloom-compactor.compaction-interval
[compaction_interval: | default = 10m]
-# Minimum age of a table before it is considered for compaction.
-# CLI flag: -bloom-compactor.min-compaction-age
-[min_compaction_age: | default = 24h]
-
-# Maximum age of a table before it is considered for compaction.
-# CLI flag: -bloom-compactor.max-compaction-age
-[max_compaction_age: | default = 168h]
+# How many index periods (days) to wait before compacting a table. This can be
+# used to lower cost by not re-writing data to object storage too frequently
+# since recent data changes more often.
+# CLI flag: -bloom-compactor.min-table-compaction-period
+[min_table_compaction_period: | default = 1]
+
+# How many index periods (days) to wait before compacting a table. This can be
+# used to lower cost by not trying to compact older data which doesn't change.
+# This can be optimized by aligning it with the maximum
+# `reject_old_samples_max_age` setting of any tenant.
+# CLI flag: -bloom-compactor.max-table-compaction-period
+[max_table_compaction_period: | default = 7]
# Number of workers to run in parallel for compaction.
# CLI flag: -bloom-compactor.worker-parallelism
diff --git a/integration/cluster/cluster.go b/integration/cluster/cluster.go
index 831da46f2cb99..7e978b84eb326 100644
--- a/integration/cluster/cluster.go
+++ b/integration/cluster/cluster.go
@@ -84,7 +84,6 @@ bloom_gateway:
bloom_compactor:
enabled: false
- working_directory: {{.dataPath}}/bloom-compactor
compactor:
working_directory: {{.dataPath}}/compactor
diff --git a/pkg/bloomcompactor/bloomcompactor.go b/pkg/bloomcompactor/bloomcompactor.go
index 8a3e7c6266c1d..5cece24172526 100644
--- a/pkg/bloomcompactor/bloomcompactor.go
+++ b/pkg/bloomcompactor/bloomcompactor.go
@@ -205,9 +205,19 @@ func (c *Compactor) runOne(ctx context.Context) error {
}
func (c *Compactor) tables(ts time.Time) *dayRangeIterator {
- from := model.TimeFromUnixNano(ts.Add(-c.cfg.MaxCompactionAge).UnixNano() / int64(config.ObjectStorageIndexRequiredPeriod))
- through := model.TimeFromUnixNano(ts.Add(-c.cfg.MinCompactionAge).UnixNano() / int64(config.ObjectStorageIndexRequiredPeriod))
- return newDayRangeIterator(DayTable(from), DayTable(through))
+ // adjust the minimum by one to make it inclusive, which is more intuitive
+ // for a configuration variable
+ adjustedMin := min(c.cfg.MinTableCompactionPeriod - 1)
+ minCompactionPeriod := time.Duration(adjustedMin) * config.ObjectStorageIndexRequiredPeriod
+ maxCompactionPeriod := time.Duration(c.cfg.MaxTableCompactionPeriod) * config.ObjectStorageIndexRequiredPeriod
+
+ from := ts.Add(-maxCompactionPeriod).UnixNano() / int64(config.ObjectStorageIndexRequiredPeriod) * int64(config.ObjectStorageIndexRequiredPeriod)
+ through := ts.Add(-minCompactionPeriod).UnixNano() / int64(config.ObjectStorageIndexRequiredPeriod) * int64(config.ObjectStorageIndexRequiredPeriod)
+
+ fromDay := DayTable(model.TimeFromUnixNano(from))
+ throughDay := DayTable(model.TimeFromUnixNano(through))
+ return newDayRangeIterator(fromDay, throughDay)
+
}
func (c *Compactor) loadWork(ctx context.Context, ch chan<- tenantTable) error {
diff --git a/pkg/bloomcompactor/config.go b/pkg/bloomcompactor/config.go
index 37aac3310829a..dd821d81c906b 100644
--- a/pkg/bloomcompactor/config.go
+++ b/pkg/bloomcompactor/config.go
@@ -20,15 +20,14 @@ type Config struct {
// section and the ingester configuration by default).
Ring ring.RingConfig `yaml:"ring,omitempty" doc:"description=Defines the ring to be used by the bloom-compactor servers. In case this isn't configured, this block supports inheriting configuration from the common ring section."`
// Enabled configures whether bloom-compactors should be used to compact index values into bloomfilters
- Enabled bool `yaml:"enabled"`
- WorkingDirectory string `yaml:"working_directory"`
- CompactionInterval time.Duration `yaml:"compaction_interval"`
- MinCompactionAge time.Duration `yaml:"min_compaction_age"`
- MaxCompactionAge time.Duration `yaml:"max_compaction_age"`
- WorkerParallelism int `yaml:"worker_parallelism"`
- RetryMinBackoff time.Duration `yaml:"compaction_retries_min_backoff"`
- RetryMaxBackoff time.Duration `yaml:"compaction_retries_max_backoff"`
- CompactionRetries int `yaml:"compaction_retries"`
+ Enabled bool `yaml:"enabled"`
+ CompactionInterval time.Duration `yaml:"compaction_interval"`
+ MinTableCompactionPeriod int `yaml:"min_table_compaction_period"`
+ MaxTableCompactionPeriod int `yaml:"max_table_compaction_period"`
+ WorkerParallelism int `yaml:"worker_parallelism"`
+ RetryMinBackoff time.Duration `yaml:"compaction_retries_min_backoff"`
+ RetryMaxBackoff time.Duration `yaml:"compaction_retries_max_backoff"`
+ CompactionRetries int `yaml:"compaction_retries"`
MaxCompactionParallelism int `yaml:"max_compaction_parallelism"`
}
@@ -37,23 +36,29 @@ type Config struct {
func (cfg *Config) RegisterFlags(f *flag.FlagSet) {
cfg.Ring.RegisterFlagsWithPrefix("bloom-compactor.", "collectors/", f)
f.BoolVar(&cfg.Enabled, "bloom-compactor.enabled", false, "Flag to enable or disable the usage of the bloom-compactor component.")
- f.StringVar(&cfg.WorkingDirectory, "bloom-compactor.working-directory", "", "Directory where files can be downloaded for compaction.")
f.DurationVar(&cfg.CompactionInterval, "bloom-compactor.compaction-interval", 10*time.Minute, "Interval at which to re-run the compaction operation.")
f.IntVar(&cfg.WorkerParallelism, "bloom-compactor.worker-parallelism", 1, "Number of workers to run in parallel for compaction.")
- f.DurationVar(&cfg.MinCompactionAge, "bloom-compactor.min-compaction-age", 24*time.Hour, "Minimum age of a table before it is considered for compaction.")
+ f.IntVar(&cfg.MinTableCompactionPeriod, "bloom-compactor.min-table-compaction-period", 1, "How many index periods (days) to wait before compacting a table. This can be used to lower cost by not re-writing data to object storage too frequently since recent data changes more often.")
// TODO(owen-d): ideally we'd set this per tenant based on their `reject_old_samples_max_age` setting,
// but due to how we need to discover tenants, we can't do that yet. Tenant+Period discovery is done by
// iterating the table periods in object storage and looking for tenants within that period.
// In order to have this done dynamically, we'd need to account for tenant specific overrides, which are also
// dynamically reloaded.
// I'm doing it the simple way for now.
- f.DurationVar(&cfg.MaxCompactionAge, "bloom-compactor.max-compaction-age", 7*24*time.Hour, "Maximum age of a table before it is considered for compaction.")
+ f.IntVar(&cfg.MaxTableCompactionPeriod, "bloom-compactor.max-table-compaction-period", 7, "How many index periods (days) to wait before compacting a table. This can be used to lower cost by not trying to compact older data which doesn't change. This can be optimized by aligning it with the maximum `reject_old_samples_max_age` setting of any tenant.")
f.DurationVar(&cfg.RetryMinBackoff, "bloom-compactor.compaction-retries-min-backoff", 10*time.Second, "Minimum backoff time between retries.")
f.DurationVar(&cfg.RetryMaxBackoff, "bloom-compactor.compaction-retries-max-backoff", time.Minute, "Maximum backoff time between retries.")
f.IntVar(&cfg.CompactionRetries, "bloom-compactor.compaction-retries", 3, "Number of retries to perform when compaction fails.")
f.IntVar(&cfg.MaxCompactionParallelism, "bloom-compactor.max-compaction-parallelism", 1, "Maximum number of tables to compact in parallel. While increasing this value, please make sure compactor has enough disk space allocated to be able to store and compact as many tables.")
}
+func (cfg *Config) Validate() error {
+ if cfg.MinTableCompactionPeriod > cfg.MaxTableCompactionPeriod {
+ return fmt.Errorf("min_compaction_age must be less than or equal to max_compaction_age")
+ }
+ return nil
+}
+
type Limits interface {
downloads.Limits
BloomCompactorShardSize(tenantID string) int
diff --git a/pkg/bloomcompactor/controller.go b/pkg/bloomcompactor/controller.go
index cf6fff090f0ae..38831ef932e69 100644
--- a/pkg/bloomcompactor/controller.go
+++ b/pkg/bloomcompactor/controller.go
@@ -179,13 +179,20 @@ func (s *SimpleBloomController) buildBlocks(
closePreExistingBlocks()
return errors.Wrap(err, "failed to get client")
}
- for newBlocks.Next() {
+
+ for newBlocks.Next() && newBlocks.Err() == nil {
blockCt++
blk := newBlocks.At()
+ built, err := bloomshipper.BlockFrom(tenant, table.String(), blk)
+ if err != nil {
+ level.Error(logger).Log("msg", "failed to build block", "err", err)
+ return errors.Wrap(err, "failed to build block")
+ }
+
if err := client.PutBlock(
ctx,
- bloomshipper.BlockFrom(tenant, table.String(), blk),
+ built,
); err != nil {
level.Error(logger).Log("msg", "failed to write block", "err", err)
closePreExistingBlocks()
diff --git a/pkg/bloomcompactor/spec.go b/pkg/bloomcompactor/spec.go
index 70ea71c4e605f..d9d9c68947a73 100644
--- a/pkg/bloomcompactor/spec.go
+++ b/pkg/bloomcompactor/spec.go
@@ -275,7 +275,7 @@ func NewStoreChunkLoader(fetcherProvider fetcherProvider, metrics *Metrics) *Sto
}
func (s *StoreChunkLoader) Load(ctx context.Context, userID string, series *v1.Series) (*ChunkItersByFingerprint, error) {
- // NB(owen-d): This is probalby unnecessary as we should only have one fetcher
+ // NB(owen-d): This is probably unnecessary as we should only have one fetcher
// because we'll only be working on a single index period at a time, but this should protect
// us in the case of refactoring/changing this and likely isn't a perf bottleneck.
chksByFetcher := make(map[chunkFetcher][]chunk.Chunk)
@@ -338,9 +338,7 @@ func newBatchedLoader(ctx context.Context, work []chunkWork, batchSize int, metr
func (b *batchedLoader) Next() bool {
if len(b.batch) > 0 {
- b.cur, b.err = b.format(b.batch[0])
- b.batch = b.batch[1:]
- return b.err == nil
+ return b.prepNext(false)
}
if len(b.work) == 0 {
@@ -357,7 +355,24 @@ func (b *batchedLoader) Next() bool {
b.work = b.work[1:]
}
+ if len(toFetch) == 0 {
+ return false
+ }
+
b.batch, b.err = next.fetcher.FetchChunks(b.ctx, toFetch)
+ if b.err != nil {
+ return false
+ }
+
+ return b.prepNext(true)
+}
+
+func (b *batchedLoader) prepNext(checkLen bool) bool {
+ if checkLen && len(b.batch) == 0 {
+ return false
+ }
+ b.cur, b.err = b.format(b.batch[0])
+ b.batch = b.batch[1:]
return b.err == nil
}
diff --git a/pkg/bloomcompactor/tsdb.go b/pkg/bloomcompactor/tsdb.go
index be45d293f6286..e6fd92961c46c 100644
--- a/pkg/bloomcompactor/tsdb.go
+++ b/pkg/bloomcompactor/tsdb.go
@@ -12,6 +12,7 @@ import (
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/model/labels"
+ "github.com/grafana/loki/pkg/chunkenc"
baseStore "github.com/grafana/loki/pkg/storage"
v1 "github.com/grafana/loki/pkg/storage/bloom/v1"
"github.com/grafana/loki/pkg/storage/config"
@@ -49,12 +50,12 @@ func NewBloomTSDBStore(storage storage.Client) *BloomTSDBStore {
}
func (b *BloomTSDBStore) UsersForPeriod(ctx context.Context, table DayTable) ([]string, error) {
- _, users, err := b.storage.ListFiles(ctx, table.String(), false)
+ _, users, err := b.storage.ListFiles(ctx, table.String(), true) // bypass cache for ease of testing
return users, err
}
func (b *BloomTSDBStore) ResolveTSDBs(ctx context.Context, table DayTable, tenant string) ([]tsdb.SingleTenantTSDBIdentifier, error) {
- indices, err := b.storage.ListUserFiles(ctx, table.String(), tenant, false)
+ indices, err := b.storage.ListUserFiles(ctx, table.String(), tenant, true) // bypass cache for ease of testing
if err != nil {
return nil, errors.Wrap(err, "failed to list user files")
}
@@ -84,16 +85,25 @@ func (b *BloomTSDBStore) LoadTSDB(
id tsdb.Identifier,
bounds v1.FingerprintBounds,
) (v1.CloseableIterator[*v1.Series], error) {
- data, err := b.storage.GetUserFile(ctx, table.String(), tenant, id.Name())
+ withCompression := id.Name() + gzipExtension
+
+ data, err := b.storage.GetUserFile(ctx, table.String(), tenant, withCompression)
if err != nil {
return nil, errors.Wrap(err, "failed to get file")
}
+ defer data.Close()
+
+ decompressorPool := chunkenc.GetReaderPool(chunkenc.EncGZIP)
+ decompressor, err := decompressorPool.GetReader(data)
+ if err != nil {
+ return nil, errors.Wrap(err, "failed to get decompressor")
+ }
+ defer decompressorPool.PutReader(decompressor)
- buf, err := io.ReadAll(data)
+ buf, err := io.ReadAll(decompressor)
if err != nil {
return nil, errors.Wrap(err, "failed to read file")
}
- _ = data.Close()
reader, err := index.NewReader(index.RealByteSlice(buf))
if err != nil {
@@ -226,7 +236,8 @@ func NewTSDBStores(
if err != nil {
return nil, errors.Wrap(err, "failed to create object client")
}
- res.stores[i] = NewBloomTSDBStore(storage.NewIndexStorageClient(c, cfg.IndexTables.PathPrefix))
+ prefix := path.Join(cfg.IndexTables.PathPrefix, cfg.IndexTables.Prefix)
+ res.stores[i] = NewBloomTSDBStore(storage.NewIndexStorageClient(c, prefix))
}
}
diff --git a/pkg/loki/loki.go b/pkg/loki/loki.go
index a83f8ba43394f..75401decb8fc0 100644
--- a/pkg/loki/loki.go
+++ b/pkg/loki/loki.go
@@ -248,6 +248,9 @@ func (c *Config) Validate() error {
if err := c.QueryRange.Validate(); err != nil {
return errors.Wrap(err, "invalid query_range config")
}
+ if err := c.BloomCompactor.Validate(); err != nil {
+ return errors.Wrap(err, "invalid bloom_compactor config")
+ }
if err := ValidateConfigCompatibility(*c); err != nil {
return err
@@ -648,7 +651,7 @@ func (t *Loki) setupModuleManager() error {
Write: {Ingester, Distributor},
Backend: {QueryScheduler, Ruler, Compactor, IndexGateway, BloomGateway, BloomCompactor},
- All: {QueryScheduler, QueryFrontend, Querier, Ingester, Distributor, Ruler, Compactor},
+ All: {QueryScheduler, QueryFrontend, Querier, Ingester, Distributor, Ruler, Compactor, BloomCompactor},
}
if t.Cfg.Querier.PerRequestLimitsEnabled {
diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go
index 23e59e711f29b..111d313956881 100644
--- a/pkg/loki/modules.go
+++ b/pkg/loki/modules.go
@@ -691,7 +691,7 @@ func (t *Loki) updateConfigForShipperStore() {
t.Cfg.StorageConfig.TSDBShipperConfig.Mode = indexshipper.ModeWriteOnly
t.Cfg.StorageConfig.TSDBShipperConfig.IngesterDBRetainPeriod = shipperQuerierIndexUpdateDelay(t.Cfg.StorageConfig.IndexCacheValidity, t.Cfg.StorageConfig.TSDBShipperConfig.ResyncInterval)
- case t.Cfg.isModuleEnabled(Querier), t.Cfg.isModuleEnabled(Ruler), t.Cfg.isModuleEnabled(Read), t.Cfg.isModuleEnabled(Backend), t.isModuleActive(IndexGateway), t.isModuleActive(BloomCompactor):
+ case t.Cfg.isModuleEnabled(Querier), t.Cfg.isModuleEnabled(Ruler), t.Cfg.isModuleEnabled(Read), t.Cfg.isModuleEnabled(Backend), t.isModuleActive(IndexGateway), t.Cfg.isModuleEnabled(BloomCompactor):
// We do not want query to do any updates to index
t.Cfg.StorageConfig.BoltDBShipperConfig.Mode = indexshipper.ModeReadOnly
t.Cfg.StorageConfig.TSDBShipperConfig.Mode = indexshipper.ModeReadOnly
diff --git a/pkg/storage/bloom/v1/block.go b/pkg/storage/bloom/v1/block.go
index 09cc5fa4866e7..84bc71a6b203c 100644
--- a/pkg/storage/bloom/v1/block.go
+++ b/pkg/storage/bloom/v1/block.go
@@ -32,6 +32,10 @@ func NewBlock(reader BlockReader) *Block {
}
}
+func (b *Block) Reader() BlockReader {
+ return b.reader
+}
+
func (b *Block) LoadHeaders() error {
// TODO(owen-d): better control over when to decode
if !b.initialized {
diff --git a/pkg/storage/bloom/v1/bloom_tokenizer.go b/pkg/storage/bloom/v1/bloom_tokenizer.go
index 7dd0d8ae44974..59bb2644f87e8 100644
--- a/pkg/storage/bloom/v1/bloom_tokenizer.go
+++ b/pkg/storage/bloom/v1/bloom_tokenizer.go
@@ -94,7 +94,7 @@ func (bt *BloomTokenizer) Populate(swb *SeriesWithBloom, chks Iterator[ChunkRefW
var tokenBuf []byte
var prefixLn int
- for chks.Err() == nil && chks.Next() {
+ for chks.Next() && chks.Err() == nil {
chk := chks.At()
itr := chk.Itr
tokenBuf, prefixLn = prefixedToken(bt.lineTokenizer.N, chk.Ref, tokenBuf)
diff --git a/pkg/storage/stores/shipper/bloomshipper/client.go b/pkg/storage/stores/shipper/bloomshipper/client.go
index 2e31106548d1a..80eba70d18cdb 100644
--- a/pkg/storage/stores/shipper/bloomshipper/client.go
+++ b/pkg/storage/stores/shipper/bloomshipper/client.go
@@ -149,7 +149,20 @@ type Block struct {
Data io.ReadSeekCloser
}
-func BlockFrom(tenant, table string, blk *v1.Block) Block {
+// CloseableReadSeekerAdapter is a wrapper around io.ReadSeeker to make it io.Closer
+// if it doesn't already implement it.
+type ClosableReadSeekerAdapter struct {
+ io.ReadSeeker
+}
+
+func (c ClosableReadSeekerAdapter) Close() error {
+ if closer, ok := c.ReadSeeker.(io.Closer); ok {
+ return closer.Close()
+ }
+ return nil
+}
+
+func BlockFrom(tenant, table string, blk *v1.Block) (Block, error) {
md, _ := blk.Metadata()
ref := Ref{
TenantID: tenant,
@@ -159,9 +172,21 @@ func BlockFrom(tenant, table string, blk *v1.Block) Block {
EndTimestamp: md.Series.ThroughTs,
Checksum: md.Checksum,
}
+
+ // TODO(owen-d): pool
+ buf := bytes.NewBuffer(nil)
+ err := v1.TarGz(buf, blk.Reader())
+
+ if err != nil {
+ return Block{}, errors.Wrap(err, "archiving+compressing block")
+ }
+
+ reader := bytes.NewReader(buf.Bytes())
+
return Block{
BlockRef: BlockRef{Ref: ref},
- }
+ Data: ClosableReadSeekerAdapter{reader},
+ }, nil
}
type BlockClient interface {
From 0bb257404029529e316f359454209ea3a72ef8bc Mon Sep 17 00:00:00 2001
From: Owen Diehl
Date: Mon, 12 Feb 2024 23:35:39 -0800
Subject: [PATCH 20/22] makes batchedLoader generic + removes unnecessary
interfaces & adapters (#11924)
While reviewing https://github.com/grafana/loki/pull/11919, I figured
it'd be nice to make `batchedLoader` generic so we can reuse it's logic.
This let me test it easier and remove a lot of now-unnecessary adapter
code (interfaces, types)
---
pkg/bloomcompactor/bloomcompactor.go | 2 +-
pkg/bloomcompactor/spec.go | 206 ++++++++++++++-------------
pkg/bloomcompactor/spec_test.go | 127 +++++++++++++++++
3 files changed, 235 insertions(+), 100 deletions(-)
diff --git a/pkg/bloomcompactor/bloomcompactor.go b/pkg/bloomcompactor/bloomcompactor.go
index 5cece24172526..ed1f50ae72582 100644
--- a/pkg/bloomcompactor/bloomcompactor.go
+++ b/pkg/bloomcompactor/bloomcompactor.go
@@ -90,7 +90,7 @@ func New(
c.metrics = NewMetrics(r, c.btMetrics)
chunkLoader := NewStoreChunkLoader(
- NewFetcherProviderAdapter(fetcherProvider),
+ fetcherProvider,
c.metrics,
)
diff --git a/pkg/bloomcompactor/spec.go b/pkg/bloomcompactor/spec.go
index d9d9c68947a73..58dd2674895ed 100644
--- a/pkg/bloomcompactor/spec.go
+++ b/pkg/bloomcompactor/spec.go
@@ -16,6 +16,7 @@ import (
logql_log "github.com/grafana/loki/pkg/logql/log"
v1 "github.com/grafana/loki/pkg/storage/bloom/v1"
"github.com/grafana/loki/pkg/storage/chunk"
+ "github.com/grafana/loki/pkg/storage/chunk/fetcher"
"github.com/grafana/loki/pkg/storage/stores"
"github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper"
"github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb"
@@ -235,39 +236,13 @@ type ChunkLoader interface {
Load(ctx context.Context, userID string, series *v1.Series) (*ChunkItersByFingerprint, error)
}
-// interface modeled from `pkg/storage/stores/composite_store.ChunkFetcherProvider`
-type fetcherProvider interface {
- GetChunkFetcher(model.Time) chunkFetcher
-}
-
-// interface modeled from `pkg/storage/chunk/fetcher.Fetcher`
-type chunkFetcher interface {
- FetchChunks(ctx context.Context, chunks []chunk.Chunk) ([]chunk.Chunk, error)
-}
-
-// Adapter turning `stores.ChunkFetcherProvider` into `fetcherProvider`
-// The former returns a concrete type and is heavily used externally
-// while the latter returns an interface for better testing and
-// is used internally
-type FetcherProviderAdapter struct {
- root stores.ChunkFetcherProvider
-}
-
-func NewFetcherProviderAdapter(root stores.ChunkFetcherProvider) *FetcherProviderAdapter {
- return &FetcherProviderAdapter{root: root}
-}
-
-func (f *FetcherProviderAdapter) GetChunkFetcher(t model.Time) chunkFetcher {
- return f.root.GetChunkFetcher(t)
-}
-
// StoreChunkLoader loads chunks from a store
type StoreChunkLoader struct {
- fetcherProvider fetcherProvider
+ fetcherProvider stores.ChunkFetcherProvider
metrics *Metrics
}
-func NewStoreChunkLoader(fetcherProvider fetcherProvider, metrics *Metrics) *StoreChunkLoader {
+func NewStoreChunkLoader(fetcherProvider stores.ChunkFetcherProvider, metrics *Metrics) *StoreChunkLoader {
return &StoreChunkLoader{
fetcherProvider: fetcherProvider,
metrics: metrics,
@@ -278,7 +253,7 @@ func (s *StoreChunkLoader) Load(ctx context.Context, userID string, series *v1.S
// NB(owen-d): This is probably unnecessary as we should only have one fetcher
// because we'll only be working on a single index period at a time, but this should protect
// us in the case of refactoring/changing this and likely isn't a perf bottleneck.
- chksByFetcher := make(map[chunkFetcher][]chunk.Chunk)
+ chksByFetcher := make(map[*fetcher.Fetcher][]chunk.Chunk)
for _, chk := range series.Chunks {
fetcher := s.fetcherProvider.GetChunkFetcher(chk.Start)
chksByFetcher[fetcher] = append(chksByFetcher[fetcher], chunk.Chunk{
@@ -292,119 +267,152 @@ func (s *StoreChunkLoader) Load(ctx context.Context, userID string, series *v1.S
})
}
- work := make([]chunkWork, 0, len(chksByFetcher))
+ var (
+ fetchers = make([]Fetcher[chunk.Chunk, chunk.Chunk], 0, len(chksByFetcher))
+ inputs = make([][]chunk.Chunk, 0, len(chksByFetcher))
+ )
for fetcher, chks := range chksByFetcher {
- work = append(work, chunkWork{
- fetcher: fetcher,
- chks: chks,
- })
+ fn := FetchFunc[chunk.Chunk, chunk.Chunk](fetcher.FetchChunks)
+ fetchers = append(fetchers, fn)
+ inputs = append(inputs, chks)
}
return &ChunkItersByFingerprint{
fp: series.Fingerprint,
- itr: newBatchedLoader(ctx, work, batchedLoaderDefaultBatchSize, s.metrics),
+ itr: newBatchedChunkLoader(ctx, fetchers, inputs, s.metrics, batchedLoaderDefaultBatchSize),
}, nil
}
-type chunkWork struct {
- fetcher chunkFetcher
- chks []chunk.Chunk
+type Fetcher[A, B any] interface {
+ Fetch(ctx context.Context, inputs []A) ([]B, error)
+}
+
+type FetchFunc[A, B any] func(ctx context.Context, inputs []A) ([]B, error)
+
+func (f FetchFunc[A, B]) Fetch(ctx context.Context, inputs []A) ([]B, error) {
+ return f(ctx, inputs)
}
// batchedLoader implements `v1.Iterator[v1.ChunkRefWithIter]` in batches
// to ensure memory is bounded while loading chunks
// TODO(owen-d): testware
-type batchedLoader struct {
+type batchedLoader[A, B, C any] struct {
metrics *Metrics
batchSize int
ctx context.Context
- work []chunkWork
+ fetchers []Fetcher[A, B]
+ work [][]A
- cur v1.ChunkRefWithIter
- batch []chunk.Chunk
- err error
+ mapper func(B) (C, error)
+ cur C
+ batch []B
+ err error
}
const batchedLoaderDefaultBatchSize = 50
-func newBatchedLoader(ctx context.Context, work []chunkWork, batchSize int, metrics *Metrics) *batchedLoader {
- return &batchedLoader{
- metrics: metrics,
- batchSize: batchSize,
+func newBatchedLoader[A, B, C any](
+ ctx context.Context,
+ fetchers []Fetcher[A, B],
+ inputs [][]A,
+ mapper func(B) (C, error),
+ batchSize int,
+) *batchedLoader[A, B, C] {
+ return &batchedLoader[A, B, C]{
+ batchSize: max(batchSize, 1),
ctx: ctx,
- work: work,
+ fetchers: fetchers,
+ work: inputs,
+ mapper: mapper,
}
}
-func (b *batchedLoader) Next() bool {
- if len(b.batch) > 0 {
- return b.prepNext(false)
- }
+func (b *batchedLoader[A, B, C]) Next() bool {
- if len(b.work) == 0 {
- return false
- }
+ // iterate work until we have non-zero length batch
+ for len(b.batch) == 0 {
- // setup next batch
- next := b.work[0]
- batchSize := min(b.batchSize, len(next.chks))
- toFetch := next.chks[:batchSize]
- // update work
- b.work[0].chks = next.chks[batchSize:]
- if len(b.work[0].chks) == 0 {
- b.work = b.work[1:]
- }
+ // empty batch + no work remaining = we're done
+ if len(b.work) == 0 {
+ return false
+ }
- if len(toFetch) == 0 {
- return false
- }
+ // setup next batch
+ next := b.work[0]
+ batchSize := min(b.batchSize, len(next))
+ toFetch := next[:batchSize]
+ fetcher := b.fetchers[0]
+
+ // update work
+ b.work[0] = b.work[0][batchSize:]
+ if len(b.work[0]) == 0 {
+ // if we've exhausted work from this set of inputs,
+ // set pointer to next set of inputs
+ // and their respective fetcher
+ b.work = b.work[1:]
+ b.fetchers = b.fetchers[1:]
+ }
- b.batch, b.err = next.fetcher.FetchChunks(b.ctx, toFetch)
- if b.err != nil {
- return false
+ // there was no work in this batch; continue (should not happen)
+ if len(toFetch) == 0 {
+ continue
+ }
+
+ b.batch, b.err = fetcher.Fetch(b.ctx, toFetch)
+ // error fetching, short-circuit iteration
+ if b.err != nil {
+ return false
+ }
}
- return b.prepNext(true)
+ return b.prepNext()
}
-func (b *batchedLoader) prepNext(checkLen bool) bool {
- if checkLen && len(b.batch) == 0 {
- return false
- }
- b.cur, b.err = b.format(b.batch[0])
+func (b *batchedLoader[_, B, C]) prepNext() bool {
+ b.cur, b.err = b.mapper(b.batch[0])
b.batch = b.batch[1:]
return b.err == nil
}
-func (b *batchedLoader) format(c chunk.Chunk) (v1.ChunkRefWithIter, error) {
- chk := c.Data.(*chunkenc.Facade).LokiChunk()
- b.metrics.chunkSize.Observe(float64(chk.UncompressedSize()))
- itr, err := chk.Iterator(
- b.ctx,
- time.Unix(0, 0),
- time.Unix(0, math.MaxInt64),
- logproto.FORWARD,
- logql_log.NewNoopPipeline().ForStream(c.Metric),
- )
+func newBatchedChunkLoader(
+ ctx context.Context,
+ fetchers []Fetcher[chunk.Chunk, chunk.Chunk],
+ inputs [][]chunk.Chunk,
+ metrics *Metrics,
+ batchSize int,
+) *batchedLoader[chunk.Chunk, chunk.Chunk, v1.ChunkRefWithIter] {
+
+ mapper := func(c chunk.Chunk) (v1.ChunkRefWithIter, error) {
+ chk := c.Data.(*chunkenc.Facade).LokiChunk()
+ metrics.chunkSize.Observe(float64(chk.UncompressedSize()))
+ itr, err := chk.Iterator(
+ ctx,
+ time.Unix(0, 0),
+ time.Unix(0, math.MaxInt64),
+ logproto.FORWARD,
+ logql_log.NewNoopPipeline().ForStream(c.Metric),
+ )
- if err != nil {
- return v1.ChunkRefWithIter{}, err
- }
+ if err != nil {
+ return v1.ChunkRefWithIter{}, err
+ }
- return v1.ChunkRefWithIter{
- Ref: v1.ChunkRef{
- Start: c.From,
- End: c.Through,
- Checksum: c.Checksum,
- },
- Itr: itr,
- }, nil
+ return v1.ChunkRefWithIter{
+ Ref: v1.ChunkRef{
+ Start: c.From,
+ End: c.Through,
+ Checksum: c.Checksum,
+ },
+ Itr: itr,
+ }, nil
+ }
+ return newBatchedLoader(ctx, fetchers, inputs, mapper, batchSize)
}
-func (b *batchedLoader) At() v1.ChunkRefWithIter {
+func (b *batchedLoader[_, _, C]) At() C {
return b.cur
}
-func (b *batchedLoader) Err() error {
+func (b *batchedLoader[_, _, _]) Err() error {
return b.err
}
diff --git a/pkg/bloomcompactor/spec_test.go b/pkg/bloomcompactor/spec_test.go
index 798d65e2f2bcd..44b1fa26a4d1f 100644
--- a/pkg/bloomcompactor/spec_test.go
+++ b/pkg/bloomcompactor/spec_test.go
@@ -3,6 +3,7 @@ package bloomcompactor
import (
"bytes"
"context"
+ "errors"
"testing"
"github.com/go-kit/log"
@@ -155,3 +156,129 @@ func TestSimpleBloomGenerator(t *testing.T) {
})
}
}
+
+func TestBatchedLoader(t *testing.T) {
+ errMapper := func(i int) (int, error) {
+ return 0, errors.New("bzzt")
+ }
+ successMapper := func(i int) (int, error) {
+ return i, nil
+ }
+
+ expired, cancel := context.WithCancel(context.Background())
+ cancel()
+
+ for _, tc := range []struct {
+ desc string
+ ctx context.Context
+ batchSize int
+ mapper func(int) (int, error)
+ err bool
+ inputs [][]int
+ exp []int
+ }{
+ {
+ desc: "OneBatch",
+ ctx: context.Background(),
+ batchSize: 2,
+ mapper: successMapper,
+ err: false,
+ inputs: [][]int{{0, 1}},
+ exp: []int{0, 1},
+ },
+ {
+ desc: "ZeroBatchSizeStillWorks",
+ ctx: context.Background(),
+ batchSize: 0,
+ mapper: successMapper,
+ err: false,
+ inputs: [][]int{{0, 1}},
+ exp: []int{0, 1},
+ },
+ {
+ desc: "OneBatchLessThanFull",
+ ctx: context.Background(),
+ batchSize: 2,
+ mapper: successMapper,
+ err: false,
+ inputs: [][]int{{0}},
+ exp: []int{0},
+ },
+ {
+ desc: "TwoBatches",
+ ctx: context.Background(),
+ batchSize: 2,
+ mapper: successMapper,
+ err: false,
+ inputs: [][]int{{0, 1, 2, 3}},
+ exp: []int{0, 1, 2, 3},
+ },
+ {
+ desc: "MultipleBatchesMultipleLoaders",
+ ctx: context.Background(),
+ batchSize: 2,
+ mapper: successMapper,
+ err: false,
+ inputs: [][]int{{0, 1}, {2}, {3, 4, 5}},
+ exp: []int{0, 1, 2, 3, 4, 5},
+ },
+ {
+ desc: "HandlesEmptyInputs",
+ ctx: context.Background(),
+ batchSize: 2,
+ mapper: successMapper,
+ err: false,
+ inputs: [][]int{{0, 1, 2, 3}, nil, {4}},
+ exp: []int{0, 1, 2, 3, 4},
+ },
+ {
+ desc: "Timeout",
+ ctx: expired,
+ batchSize: 2,
+ mapper: successMapper,
+ err: true,
+ inputs: [][]int{{0}},
+ },
+ {
+ desc: "MappingFailure",
+ ctx: context.Background(),
+ batchSize: 2,
+ mapper: errMapper,
+ err: true,
+ inputs: [][]int{{0}},
+ },
+ } {
+ t.Run(tc.desc, func(t *testing.T) {
+ fetchers := make([]Fetcher[int, int], 0, len(tc.inputs))
+ for range tc.inputs {
+ fetchers = append(
+ fetchers,
+ FetchFunc[int, int](func(ctx context.Context, xs []int) ([]int, error) {
+ if ctx.Err() != nil {
+ return nil, ctx.Err()
+ }
+ return xs, nil
+ }),
+ )
+ }
+
+ loader := newBatchedLoader[int, int, int](
+ tc.ctx,
+ fetchers,
+ tc.inputs,
+ tc.mapper,
+ tc.batchSize,
+ )
+
+ got, err := v1.Collect[int](loader)
+ if tc.err {
+ require.Error(t, err)
+ return
+ }
+ require.NoError(t, err)
+ require.Equal(t, tc.exp, got)
+
+ })
+ }
+
+}
From eb8464a64117a070d9e1baf784b9ee59fca1911f Mon Sep 17 00:00:00 2001
From: Christian Haudum
Date: Tue, 13 Feb 2024 09:10:56 +0100
Subject: [PATCH 21/22] Bloom compactor: Load blocks lazily in batches (#11919)
To avoid loading possibly lots of blocks upfront, this PR introduces
lazy loading of blocks in batches using an iterator that loads blocks on
demand.
Signed-off-by: Christian Haudum
---
pkg/bloomcompactor/batch.go | 95 ++++++++++++++++++++++++++++++++
pkg/bloomcompactor/batch_test.go | 37 +++++++++++++
pkg/bloomcompactor/controller.go | 68 +++++++++++++++--------
pkg/bloomcompactor/spec.go | 30 ++++++----
pkg/bloomcompactor/spec_test.go | 5 +-
pkg/storage/bloom/v1/util.go | 12 ++++
6 files changed, 212 insertions(+), 35 deletions(-)
create mode 100644 pkg/bloomcompactor/batch.go
create mode 100644 pkg/bloomcompactor/batch_test.go
diff --git a/pkg/bloomcompactor/batch.go b/pkg/bloomcompactor/batch.go
new file mode 100644
index 0000000000000..2d43f83219df9
--- /dev/null
+++ b/pkg/bloomcompactor/batch.go
@@ -0,0 +1,95 @@
+package bloomcompactor
+
+import (
+ "context"
+
+ "github.com/grafana/dskit/multierror"
+
+ "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper"
+)
+
+// interface modeled from `pkg/storage/stores/shipper/bloomshipper.Fetcher`
+type blocksFetcher interface {
+ FetchBlocks(context.Context, []bloomshipper.BlockRef) ([]*bloomshipper.CloseableBlockQuerier, error)
+}
+
+func newBatchedBlockLoader(ctx context.Context, fetcher blocksFetcher, blocks []bloomshipper.BlockRef) (*batchedBlockLoader, error) {
+ return &batchedBlockLoader{
+ ctx: ctx,
+ batchSize: 10, // make configurable?
+ source: blocks,
+ fetcher: fetcher,
+ }, nil
+}
+
+type batchedBlockLoader struct {
+ ctx context.Context
+ batchSize int
+
+ source []bloomshipper.BlockRef
+ fetcher blocksFetcher
+
+ batch []*bloomshipper.CloseableBlockQuerier
+ cur *bloomshipper.CloseableBlockQuerier
+ err error
+}
+
+// At implements v1.CloseableIterator.
+func (b *batchedBlockLoader) At() *bloomshipper.CloseableBlockQuerier {
+ return b.cur
+}
+
+// Close implements v1.CloseableIterator.
+func (b *batchedBlockLoader) Close() error {
+ if b.cur != nil {
+ return b.cur.Close()
+ }
+ return nil
+}
+
+// CloseBatch closes the remaining items from the current batch
+func (b *batchedBlockLoader) CloseBatch() error {
+ var err multierror.MultiError
+ for _, cur := range b.batch {
+ err.Add(cur.Close())
+ }
+ if len(b.batch) > 0 {
+ b.batch = b.batch[:0]
+ }
+ return err.Err()
+}
+
+// Err implements v1.CloseableIterator.
+func (b *batchedBlockLoader) Err() error {
+ return b.err
+}
+
+// Next implements v1.CloseableIterator.
+func (b *batchedBlockLoader) Next() bool {
+ if len(b.batch) > 0 {
+ return b.setNext()
+ }
+
+ if len(b.source) == 0 {
+ return false
+ }
+
+ // setup next batch
+ batchSize := min(b.batchSize, len(b.source))
+ toFetch := b.source[:batchSize]
+
+ // update source
+ b.source = b.source[batchSize:]
+
+ b.batch, b.err = b.fetcher.FetchBlocks(b.ctx, toFetch)
+ if b.err != nil {
+ return false
+ }
+ return b.setNext()
+}
+
+func (b *batchedBlockLoader) setNext() bool {
+ b.cur, b.err = b.batch[0], nil
+ b.batch = b.batch[1:]
+ return true
+}
diff --git a/pkg/bloomcompactor/batch_test.go b/pkg/bloomcompactor/batch_test.go
new file mode 100644
index 0000000000000..a1922bf931b86
--- /dev/null
+++ b/pkg/bloomcompactor/batch_test.go
@@ -0,0 +1,37 @@
+package bloomcompactor
+
+import (
+ "context"
+ "testing"
+
+ "github.com/stretchr/testify/require"
+ "go.uber.org/atomic"
+
+ "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper"
+)
+
+type dummyBlocksFetcher struct {
+ count *atomic.Int32
+}
+
+func (f *dummyBlocksFetcher) FetchBlocks(_ context.Context, blocks []bloomshipper.BlockRef) ([]*bloomshipper.CloseableBlockQuerier, error) {
+ f.count.Inc()
+ return make([]*bloomshipper.CloseableBlockQuerier, len(blocks)), nil
+}
+
+func TestBatchedBlockLoader(t *testing.T) {
+ ctx := context.Background()
+ f := &dummyBlocksFetcher{count: atomic.NewInt32(0)}
+
+ blocks := make([]bloomshipper.BlockRef, 25)
+ blocksIter, err := newBatchedBlockLoader(ctx, f, blocks)
+ require.NoError(t, err)
+
+ var count int
+ for blocksIter.Next() && blocksIter.Err() == nil {
+ count++
+ }
+
+ require.Equal(t, len(blocks), count)
+ require.Equal(t, int32(len(blocks)/blocksIter.batchSize+1), f.count.Load())
+}
diff --git a/pkg/bloomcompactor/controller.go b/pkg/bloomcompactor/controller.go
index 38831ef932e69..47d9627d92e1a 100644
--- a/pkg/bloomcompactor/controller.go
+++ b/pkg/bloomcompactor/controller.go
@@ -4,6 +4,7 @@ import (
"bytes"
"context"
"fmt"
+ "io"
"sort"
"github.com/go-kit/log"
@@ -138,45 +139,36 @@ func (s *SimpleBloomController) buildBlocks(
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, table, tenant, plan.tsdb, gap)
+ seriesItr, blocksIter, err := s.loadWorkForGap(ctx, table, tenant, plan.tsdb, gap)
if err != nil {
level.Error(logger).Log("msg", "failed to get series and blocks", "err", err)
return errors.Wrap(err, "failed to get series and blocks")
}
- // Close all remaining blocks on exit
- closePreExistingBlocks := func() {
- var closeErrors multierror.MultiError
- for _, block := range preExistingBlocks {
- closeErrors.Add(block.Close())
- }
- if err := closeErrors.Err(); err != nil {
- level.Error(s.logger).Log("msg", "failed to close blocks", "err", err)
- }
- }
gen := NewSimpleBloomGenerator(
tenant,
blockOpts,
seriesItr,
s.chunkLoader,
- preExistingBlocks,
+ blocksIter,
s.rwFn,
s.metrics,
- log.With(logger, "tsdb", plan.tsdb.Name(), "ownership", gap, "blocks", len(preExistingBlocks)),
+ log.With(logger, "tsdb", plan.tsdb.Name(), "ownership", gap),
)
- _, newBlocks, err := gen.Generate(ctx)
+ _, loaded, newBlocks, err := gen.Generate(ctx)
+
if err != nil {
// TODO(owen-d): metrics
level.Error(logger).Log("msg", "failed to generate bloom", "err", err)
- closePreExistingBlocks()
+ s.closeLoadedBlocks(loaded, blocksIter)
return errors.Wrap(err, "failed to generate bloom")
}
client, err := s.bloomStore.Client(table.ModelTime())
if err != nil {
level.Error(logger).Log("msg", "failed to get client", "err", err)
- closePreExistingBlocks()
+ s.closeLoadedBlocks(loaded, blocksIter)
return errors.Wrap(err, "failed to get client")
}
@@ -195,7 +187,7 @@ func (s *SimpleBloomController) buildBlocks(
built,
); err != nil {
level.Error(logger).Log("msg", "failed to write block", "err", err)
- closePreExistingBlocks()
+ s.closeLoadedBlocks(loaded, blocksIter)
return errors.Wrap(err, "failed to write block")
}
}
@@ -203,12 +195,12 @@ func (s *SimpleBloomController) buildBlocks(
if err := newBlocks.Err(); err != nil {
// TODO(owen-d): metrics
level.Error(logger).Log("msg", "failed to generate bloom", "err", err)
- closePreExistingBlocks()
+ s.closeLoadedBlocks(loaded, blocksIter)
return errors.Wrap(err, "failed to generate bloom")
}
// Close pre-existing blocks
- closePreExistingBlocks()
+ s.closeLoadedBlocks(loaded, blocksIter)
}
}
@@ -226,19 +218,49 @@ func (s *SimpleBloomController) loadWorkForGap(
tenant string,
id tsdb.Identifier,
gap gapWithBlocks,
-) (v1.CloseableIterator[*v1.Series], []*bloomshipper.CloseableBlockQuerier, error) {
+) (v1.CloseableIterator[*v1.Series], v1.CloseableIterator[*bloomshipper.CloseableBlockQuerier], error) {
// load a series iterator for the gap
seriesItr, err := s.tsdbStore.LoadTSDB(ctx, table, tenant, id, gap.bounds)
if err != nil {
return nil, nil, errors.Wrap(err, "failed to load tsdb")
}
- blocks, err := s.bloomStore.FetchBlocks(ctx, gap.blocks)
+ // load a blocks iterator for the gap
+ fetcher, err := s.bloomStore.Fetcher(table.ModelTime())
if err != nil {
- return nil, nil, errors.Wrap(err, "failed to get blocks")
+ return nil, nil, errors.Wrap(err, "failed to get fetcher")
}
- return seriesItr, blocks, nil
+ blocksIter, err := newBatchedBlockLoader(ctx, fetcher, gap.blocks)
+ if err != nil {
+ return nil, nil, errors.Wrap(err, "failed to load blocks")
+ }
+
+ return seriesItr, blocksIter, nil
+}
+
+func (s *SimpleBloomController) closeLoadedBlocks(toClose []io.Closer, it v1.CloseableIterator[*bloomshipper.CloseableBlockQuerier]) {
+ // close loaded blocks
+ var err multierror.MultiError
+ for _, closer := range toClose {
+ err.Add(closer.Close())
+ }
+
+ switch itr := it.(type) {
+ case *batchedBlockLoader:
+ // close remaining loaded blocks from batch
+ err.Add(itr.CloseBatch())
+ default:
+ // close remaining loaded blocks
+ for itr.Next() && itr.Err() == nil {
+ err.Add(itr.At().Close())
+ }
+ }
+
+ // log error
+ if err.Err() != nil {
+ level.Error(s.logger).Log("msg", "failed to close blocks", "err", err)
+ }
}
type gapWithBlocks struct {
diff --git a/pkg/bloomcompactor/spec.go b/pkg/bloomcompactor/spec.go
index 58dd2674895ed..4a1125082ca54 100644
--- a/pkg/bloomcompactor/spec.go
+++ b/pkg/bloomcompactor/spec.go
@@ -3,6 +3,7 @@ package bloomcompactor
import (
"context"
"fmt"
+ "io"
"math"
"time"
@@ -39,7 +40,7 @@ func (k Keyspace) Cmp(other Keyspace) v1.BoundsCheck {
// Store is likely bound within. This allows specifying impls like ShardedStore
// to only request the shard-range needed from the existing store.
type BloomGenerator interface {
- Generate(ctx context.Context) (skippedBlocks []*v1.Block, results v1.Iterator[*v1.Block], err error)
+ Generate(ctx context.Context) (skippedBlocks []v1.BlockMetadata, toClose []io.Closer, results v1.Iterator[*v1.Block], err error)
}
// Simple implementation of a BloomGenerator.
@@ -47,9 +48,7 @@ type SimpleBloomGenerator struct {
userID string
store v1.Iterator[*v1.Series]
chunkLoader ChunkLoader
- // TODO(owen-d): blocks need not be all downloaded prior. Consider implementing
- // as an iterator of iterators, where each iterator is a batch of overlapping blocks.
- blocks []*bloomshipper.CloseableBlockQuerier
+ blocksIter v1.CloseableIterator[*bloomshipper.CloseableBlockQuerier]
// options to build blocks with
opts v1.BlockOptions
@@ -71,7 +70,7 @@ func NewSimpleBloomGenerator(
opts v1.BlockOptions,
store v1.Iterator[*v1.Series],
chunkLoader ChunkLoader,
- blocks []*bloomshipper.CloseableBlockQuerier,
+ blocksIter v1.CloseableIterator[*bloomshipper.CloseableBlockQuerier],
readWriterFn func() (v1.BlockWriter, v1.BlockReader),
metrics *Metrics,
logger log.Logger,
@@ -81,7 +80,7 @@ func NewSimpleBloomGenerator(
opts: opts,
store: store,
chunkLoader: chunkLoader,
- blocks: blocks,
+ blocksIter: blocksIter,
logger: log.With(logger, "component", "bloom_generator"),
readWriterFn: readWriterFn,
metrics: metrics,
@@ -108,9 +107,15 @@ func (s *SimpleBloomGenerator) populator(ctx context.Context) func(series *v1.Se
}
-func (s *SimpleBloomGenerator) Generate(ctx context.Context) (skippedBlocks []v1.BlockMetadata, results v1.Iterator[*v1.Block], err error) {
- blocksMatchingSchema := make([]*bloomshipper.CloseableBlockQuerier, 0, len(s.blocks))
- for _, block := range s.blocks {
+func (s *SimpleBloomGenerator) Generate(ctx context.Context) ([]v1.BlockMetadata, []io.Closer, v1.Iterator[*v1.Block], error) {
+ skippedBlocks := make([]v1.BlockMetadata, 0)
+ toClose := make([]io.Closer, 0)
+ blocksMatchingSchema := make([]*bloomshipper.CloseableBlockQuerier, 0)
+
+ for s.blocksIter.Next() && s.blocksIter.Err() == nil {
+ block := s.blocksIter.At()
+ toClose = append(toClose, block)
+
logger := log.With(s.logger, "block", block.BlockRef)
md, err := block.Metadata()
schema := md.Options.Schema
@@ -130,11 +135,16 @@ func (s *SimpleBloomGenerator) Generate(ctx context.Context) (skippedBlocks []v1
blocksMatchingSchema = append(blocksMatchingSchema, block)
}
+ if s.blocksIter.Err() != nil {
+ // should we ignore the error and continue with the blocks we got?
+ return skippedBlocks, toClose, v1.NewSliceIter([]*v1.Block{}), s.blocksIter.Err()
+ }
+
level.Debug(s.logger).Log("msg", "generating bloom filters for blocks", "num_blocks", len(blocksMatchingSchema), "skipped_blocks", len(skippedBlocks), "schema", fmt.Sprintf("%+v", s.opts.Schema))
series := v1.NewPeekingIter(s.store)
blockIter := NewLazyBlockBuilderIterator(ctx, s.opts, s.populator(ctx), s.readWriterFn, series, blocksMatchingSchema)
- return skippedBlocks, blockIter, nil
+ return skippedBlocks, toClose, blockIter, nil
}
// LazyBlockBuilderIterator is a lazy iterator over blocks that builds
diff --git a/pkg/bloomcompactor/spec_test.go b/pkg/bloomcompactor/spec_test.go
index 44b1fa26a4d1f..bb4fde6cc2359 100644
--- a/pkg/bloomcompactor/spec_test.go
+++ b/pkg/bloomcompactor/spec_test.go
@@ -71,13 +71,14 @@ func dummyBloomGen(opts v1.BlockOptions, store v1.Iterator[*v1.Series], blocks [
BlockQuerier: v1.NewBlockQuerier(b),
})
}
+ blocksIter := v1.NewCloseableIterator(v1.NewSliceIter(bqs))
return NewSimpleBloomGenerator(
"fake",
opts,
store,
dummyChunkLoader{},
- bqs,
+ blocksIter,
func() (v1.BlockWriter, v1.BlockReader) {
indexBuf := bytes.NewBuffer(nil)
bloomsBuf := bytes.NewBuffer(nil)
@@ -130,7 +131,7 @@ func TestSimpleBloomGenerator(t *testing.T) {
)
gen := dummyBloomGen(tc.toSchema, storeItr, sourceBlocks)
- skipped, results, err := gen.Generate(context.Background())
+ skipped, _, results, err := gen.Generate(context.Background())
require.Nil(t, err)
require.Equal(t, tc.numSkipped, len(skipped))
diff --git a/pkg/storage/bloom/v1/util.go b/pkg/storage/bloom/v1/util.go
index d980a9ecc4df1..3b9e0631b715d 100644
--- a/pkg/storage/bloom/v1/util.go
+++ b/pkg/storage/bloom/v1/util.go
@@ -247,6 +247,18 @@ type CloseableIterator[T any] interface {
Close() error
}
+func NewCloseableIterator[T io.Closer](itr Iterator[T]) *CloseIter[T] {
+ return &CloseIter[T]{itr}
+}
+
+type CloseIter[T io.Closer] struct {
+ Iterator[T]
+}
+
+func (i *CloseIter[T]) Close() error {
+ return i.At().Close()
+}
+
type PeekingCloseableIterator[T any] interface {
PeekingIterator[T]
CloseableIterator[T]
From 6c5c347270e7f41ed4240945fcd567c7e4630921 Mon Sep 17 00:00:00 2001
From: Robert Jacob
Date: Tue, 13 Feb 2024 09:23:20 +0100
Subject: [PATCH 22/22] operator: Refactor handling of credentials in
managed-auth mode (#11920)
---
operator/CHANGELOG.md | 1 +
.../apis/config/v1/projectconfig_types.go | 9 +-
operator/apis/loki/v1/lokistack_types.go | 27 +++
.../loki-operator.clusterserviceversion.yaml | 3 +-
.../loki.grafana.com_lokistacks.yaml | 8 +
.../loki-operator.clusterserviceversion.yaml | 3 +-
.../loki.grafana.com_lokistacks.yaml | 8 +
.../loki-operator.clusterserviceversion.yaml | 3 +-
.../loki.grafana.com_lokistacks.yaml | 8 +
.../bases/loki.grafana.com_lokistacks.yaml | 8 +
operator/config/rbac/role.yaml | 1 +
.../loki/credentialsrequests_controller.go | 82 ---------
.../credentialsrequests_controller_test.go | 164 ------------------
.../lokistack/credentialsrequest_discovery.go | 30 ----
.../credentialsrequest_discovery_test.go | 98 -----------
.../controllers/loki/lokistack_controller.go | 60 ++-----
.../loki/lokistack_controller_test.go | 30 ++--
operator/docs/operator/api.md | 48 +++++
operator/docs/operator/feature-gates.md | 3 +-
operator/internal/config/managed_auth.go | 48 +++++
operator/internal/config/options.go | 13 +-
.../handlers/credentialsrequest_create.go | 80 ++++++---
.../credentialsrequest_create_test.go | 157 +++++++++++++----
.../handlers/credentialsrequest_delete.go | 43 -----
.../credentialsrequest_delete_test.go | 47 -----
.../handlers/internal/storage/secrets.go | 32 ++--
.../handlers/internal/storage/secrets_test.go | 72 +++++---
.../handlers/internal/storage/storage_test.go | 75 --------
.../handlers/lokistack_create_or_update.go | 32 ++--
.../lokistack_create_or_update_test.go | 18 +-
operator/internal/manifests/mutate.go | 10 ++
.../manifests/openshift/credentialsrequest.go | 62 +------
.../openshift/credentialsrequest_test.go | 42 ++---
.../internal/manifests/openshift/options.go | 19 +-
operator/internal/manifests/openshift/var.go | 2 -
.../internal/manifests/storage/configure.go | 55 +++---
.../manifests/storage/configure_test.go | 76 +++-----
.../internal/manifests/storage/options.go | 35 ++++
operator/internal/manifests/storage/var.go | 36 ++--
operator/internal/status/status.go | 3 +-
operator/internal/status/status_test.go | 8 +-
operator/main.go | 24 +--
42 files changed, 621 insertions(+), 962 deletions(-)
delete mode 100644 operator/controllers/loki/credentialsrequests_controller.go
delete mode 100644 operator/controllers/loki/credentialsrequests_controller_test.go
delete mode 100644 operator/controllers/loki/internal/lokistack/credentialsrequest_discovery.go
delete mode 100644 operator/controllers/loki/internal/lokistack/credentialsrequest_discovery_test.go
create mode 100644 operator/internal/config/managed_auth.go
delete mode 100644 operator/internal/handlers/credentialsrequest_delete.go
delete mode 100644 operator/internal/handlers/credentialsrequest_delete_test.go
diff --git a/operator/CHANGELOG.md b/operator/CHANGELOG.md
index 2a1ebc2f5d362..59afb29708782 100644
--- a/operator/CHANGELOG.md
+++ b/operator/CHANGELOG.md
@@ -1,5 +1,6 @@
## Main
+- [11920](https://github.com/grafana/loki/pull/11920) **xperimental**: Refactor handling of credentials in managed-auth mode
- [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
diff --git a/operator/apis/config/v1/projectconfig_types.go b/operator/apis/config/v1/projectconfig_types.go
index 06ff8cb090598..8e510b5d3ab79 100644
--- a/operator/apis/config/v1/projectconfig_types.go
+++ b/operator/apis/config/v1/projectconfig_types.go
@@ -52,16 +52,11 @@ type OpenShiftFeatureGates struct {
// Dashboards enables the loki-mixin dashboards into the OpenShift Console
Dashboards bool `json:"dashboards,omitempty"`
- // ManagedAuthEnv enabled when the operator installation is on OpenShift STS clusters.
+ // ManagedAuthEnv is true when OpenShift-functions are enabled and the operator has detected
+ // that it is running with some kind of "workload identity" (AWS STS, Azure WIF) enabled.
ManagedAuthEnv bool
}
-// ManagedAuthEnabled returns true when OpenShift-functions are enabled and the operator has detected that it is
-// running with some kind of "workload identity" (AWS STS, Azure WIF) enabled.
-func (o *OpenShiftFeatureGates) ManagedAuthEnabled() bool {
- return o.Enabled && o.ManagedAuthEnv
-}
-
// FeatureGates is the supported set of all operator feature gates.
type FeatureGates struct {
// ServiceMonitors enables creating a Prometheus-Operator managed ServiceMonitor
diff --git a/operator/apis/loki/v1/lokistack_types.go b/operator/apis/loki/v1/lokistack_types.go
index a50fb48b187ea..b652ba0c7a4d9 100644
--- a/operator/apis/loki/v1/lokistack_types.go
+++ b/operator/apis/loki/v1/lokistack_types.go
@@ -1174,6 +1174,27 @@ type LokiStackComponentStatus struct {
Ruler PodStatusMap `json:"ruler,omitempty"`
}
+// CredentialMode represents the type of authentication used for accessing the object storage.
+//
+// +kubebuilder:validation:Enum=static;token;managed
+type CredentialMode string
+
+const (
+ // CredentialModeStatic represents the usage of static, long-lived credentials stored in a Secret.
+ // This is the default authentication mode and available for all supported object storage types.
+ CredentialModeStatic CredentialMode = "static"
+ // CredentialModeToken represents the usage of short-lived tokens retrieved from a credential source.
+ // In this mode the static configuration does not contain credentials needed for the object storage.
+ // Instead, they are generated during runtime using a service, which allows for shorter-lived credentials and
+ // much more granular control. This authentication mode is not supported for all object storage types.
+ CredentialModeToken CredentialMode = "token"
+ // CredentialModeManaged represents the usage of short-lived tokens retrieved from a credential source.
+ // This mode is similar to CredentialModeToken,but instead of having a user-configured credential source,
+ // it is configured by the environment, for example the Cloud Credential Operator in OpenShift.
+ // This mode is only supported for certain object storage types in certain runtime environments.
+ CredentialModeManaged CredentialMode = "managed"
+)
+
// LokiStackStorageStatus defines the observed state of
// the Loki storage configuration.
type LokiStackStorageStatus struct {
@@ -1183,6 +1204,12 @@ type LokiStackStorageStatus struct {
// +optional
// +kubebuilder:validation:Optional
Schemas []ObjectStorageSchema `json:"schemas,omitempty"`
+
+ // CredentialMode contains the authentication mode used for accessing the object storage.
+ //
+ // +optional
+ // +kubebuilder:validation:Optional
+ CredentialMode CredentialMode `json:"credentialMode,omitempty"`
}
// LokiStackStatus defines the observed state of LokiStack
diff --git a/operator/bundle/community-openshift/manifests/loki-operator.clusterserviceversion.yaml b/operator/bundle/community-openshift/manifests/loki-operator.clusterserviceversion.yaml
index 6854bf38ff661..ad2b2e1bc93b4 100644
--- a/operator/bundle/community-openshift/manifests/loki-operator.clusterserviceversion.yaml
+++ b/operator/bundle/community-openshift/manifests/loki-operator.clusterserviceversion.yaml
@@ -150,7 +150,7 @@ metadata:
categories: OpenShift Optional, Logging & Tracing
certified: "false"
containerImage: docker.io/grafana/loki-operator:0.5.0
- createdAt: "2024-01-31T16:48:07Z"
+ createdAt: "2024-02-12T14:48:52Z"
description: The Community Loki Operator provides Kubernetes native deployment
and management of Loki and related logging components.
features.operators.openshift.io/disconnected: "true"
@@ -1472,6 +1472,7 @@ spec:
- delete
- get
- list
+ - update
- watch
- apiGroups:
- config.openshift.io
diff --git a/operator/bundle/community-openshift/manifests/loki.grafana.com_lokistacks.yaml b/operator/bundle/community-openshift/manifests/loki.grafana.com_lokistacks.yaml
index a8033e692214e..e1a7e5578965a 100644
--- a/operator/bundle/community-openshift/manifests/loki.grafana.com_lokistacks.yaml
+++ b/operator/bundle/community-openshift/manifests/loki.grafana.com_lokistacks.yaml
@@ -4064,6 +4064,14 @@ spec:
description: Storage provides summary of all changes that have occurred
to the storage configuration.
properties:
+ credentialMode:
+ description: CredentialMode contains the authentication mode used
+ for accessing the object storage.
+ enum:
+ - static
+ - token
+ - managed
+ type: string
schemas:
description: Schemas is a list of schemas which have been applied
to the LokiStack.
diff --git a/operator/bundle/community/manifests/loki-operator.clusterserviceversion.yaml b/operator/bundle/community/manifests/loki-operator.clusterserviceversion.yaml
index f8c37162b5a44..b372a29504e3a 100644
--- a/operator/bundle/community/manifests/loki-operator.clusterserviceversion.yaml
+++ b/operator/bundle/community/manifests/loki-operator.clusterserviceversion.yaml
@@ -150,7 +150,7 @@ metadata:
categories: OpenShift Optional, Logging & Tracing
certified: "false"
containerImage: docker.io/grafana/loki-operator:0.5.0
- createdAt: "2024-01-31T16:48:04Z"
+ createdAt: "2024-02-12T14:48:49Z"
description: The Community Loki Operator provides Kubernetes native deployment
and management of Loki and related logging components.
operators.operatorframework.io/builder: operator-sdk-unknown
@@ -1452,6 +1452,7 @@ spec:
- delete
- get
- list
+ - update
- watch
- apiGroups:
- config.openshift.io
diff --git a/operator/bundle/community/manifests/loki.grafana.com_lokistacks.yaml b/operator/bundle/community/manifests/loki.grafana.com_lokistacks.yaml
index 8b86ddfff8bbf..f92665f5095d2 100644
--- a/operator/bundle/community/manifests/loki.grafana.com_lokistacks.yaml
+++ b/operator/bundle/community/manifests/loki.grafana.com_lokistacks.yaml
@@ -4064,6 +4064,14 @@ spec:
description: Storage provides summary of all changes that have occurred
to the storage configuration.
properties:
+ credentialMode:
+ description: CredentialMode contains the authentication mode used
+ for accessing the object storage.
+ enum:
+ - static
+ - token
+ - managed
+ type: string
schemas:
description: Schemas is a list of schemas which have been applied
to the LokiStack.
diff --git a/operator/bundle/openshift/manifests/loki-operator.clusterserviceversion.yaml b/operator/bundle/openshift/manifests/loki-operator.clusterserviceversion.yaml
index 234ddb423a3aa..8026bbcd0fc4c 100644
--- a/operator/bundle/openshift/manifests/loki-operator.clusterserviceversion.yaml
+++ b/operator/bundle/openshift/manifests/loki-operator.clusterserviceversion.yaml
@@ -150,7 +150,7 @@ metadata:
categories: OpenShift Optional, Logging & Tracing
certified: "false"
containerImage: quay.io/openshift-logging/loki-operator:0.1.0
- createdAt: "2024-01-31T16:48:10Z"
+ createdAt: "2024-02-12T14:48:55Z"
description: |
The Loki Operator for OCP provides a means for configuring and managing a Loki stack for cluster logging.
## Prerequisites and Requirements
@@ -1457,6 +1457,7 @@ spec:
- delete
- get
- list
+ - update
- watch
- apiGroups:
- config.openshift.io
diff --git a/operator/bundle/openshift/manifests/loki.grafana.com_lokistacks.yaml b/operator/bundle/openshift/manifests/loki.grafana.com_lokistacks.yaml
index f121699ec6fb8..3163752ad36f0 100644
--- a/operator/bundle/openshift/manifests/loki.grafana.com_lokistacks.yaml
+++ b/operator/bundle/openshift/manifests/loki.grafana.com_lokistacks.yaml
@@ -4064,6 +4064,14 @@ spec:
description: Storage provides summary of all changes that have occurred
to the storage configuration.
properties:
+ credentialMode:
+ description: CredentialMode contains the authentication mode used
+ for accessing the object storage.
+ enum:
+ - static
+ - token
+ - managed
+ type: string
schemas:
description: Schemas is a list of schemas which have been applied
to the LokiStack.
diff --git a/operator/config/crd/bases/loki.grafana.com_lokistacks.yaml b/operator/config/crd/bases/loki.grafana.com_lokistacks.yaml
index 4661097811b75..d603ef2a9b644 100644
--- a/operator/config/crd/bases/loki.grafana.com_lokistacks.yaml
+++ b/operator/config/crd/bases/loki.grafana.com_lokistacks.yaml
@@ -4046,6 +4046,14 @@ spec:
description: Storage provides summary of all changes that have occurred
to the storage configuration.
properties:
+ credentialMode:
+ description: CredentialMode contains the authentication mode used
+ for accessing the object storage.
+ enum:
+ - static
+ - token
+ - managed
+ type: string
schemas:
description: Schemas is a list of schemas which have been applied
to the LokiStack.
diff --git a/operator/config/rbac/role.yaml b/operator/config/rbac/role.yaml
index 766a6d7d191e6..072efd5b99128 100644
--- a/operator/config/rbac/role.yaml
+++ b/operator/config/rbac/role.yaml
@@ -56,6 +56,7 @@ rules:
- delete
- get
- list
+ - update
- watch
- apiGroups:
- config.openshift.io
diff --git a/operator/controllers/loki/credentialsrequests_controller.go b/operator/controllers/loki/credentialsrequests_controller.go
deleted file mode 100644
index efd0226c6a340..0000000000000
--- a/operator/controllers/loki/credentialsrequests_controller.go
+++ /dev/null
@@ -1,82 +0,0 @@
-package controllers
-
-import (
- "context"
-
- "github.com/go-logr/logr"
- corev1 "k8s.io/api/core/v1"
- apierrors "k8s.io/apimachinery/pkg/api/errors"
- "k8s.io/apimachinery/pkg/runtime"
- ctrl "sigs.k8s.io/controller-runtime"
- "sigs.k8s.io/controller-runtime/pkg/client"
-
- lokiv1 "github.com/grafana/loki/operator/apis/loki/v1"
- "github.com/grafana/loki/operator/controllers/loki/internal/lokistack"
- "github.com/grafana/loki/operator/controllers/loki/internal/management/state"
- "github.com/grafana/loki/operator/internal/external/k8s"
- "github.com/grafana/loki/operator/internal/handlers"
-)
-
-// CredentialsRequestsReconciler reconciles a single CredentialsRequest resource for each LokiStack request.
-type CredentialsRequestsReconciler struct {
- client.Client
- Scheme *runtime.Scheme
- Log logr.Logger
-}
-
-// Reconcile creates a single CredentialsRequest per LokiStack for the OpenShift cloud-credentials-operator (CCO) to
-// provide a managed cloud credentials Secret. On successful creation, the LokiStack resource is annotated
-// with `loki.grafana.com/credentials-request-secret-ref` that refers to the secret provided by CCO. If the LokiStack
-// resource is not found its accompanying CredentialsRequest resource is deleted.
-func (r *CredentialsRequestsReconciler) Reconcile(ctx context.Context, req ctrl.Request) (ctrl.Result, error) {
- var stack lokiv1.LokiStack
- if err := r.Client.Get(ctx, req.NamespacedName, &stack); err != nil {
- if apierrors.IsNotFound(err) {
- return ctrl.Result{}, handlers.DeleteCredentialsRequest(ctx, r.Client, req.NamespacedName)
- }
- return ctrl.Result{}, err
- }
-
- managed, err := state.IsManaged(ctx, req, r.Client)
- if err != nil {
- return ctrl.Result{}, err
- }
- if !managed {
- r.Log.Info("Skipping reconciliation for unmanaged LokiStack resource", "name", req.String())
- // Stop requeueing for unmanaged LokiStack custom resources
- return ctrl.Result{}, nil
- }
-
- storageSecretName := client.ObjectKey{
- Namespace: req.Namespace,
- Name: stack.Spec.Storage.Secret.Name,
- }
- storageSecret := &corev1.Secret{}
- err = r.Client.Get(ctx, storageSecretName, storageSecret)
- if err != nil {
- return ctrl.Result{}, err
- }
-
- secretRef, err := handlers.CreateCredentialsRequest(ctx, r.Client, req.NamespacedName, storageSecret)
- if err != nil {
- return ctrl.Result{}, err
- }
-
- if err := lokistack.AnnotateForCredentialsRequest(ctx, r.Client, req.NamespacedName, secretRef); err != nil {
- return ctrl.Result{}, err
- }
-
- return ctrl.Result{}, nil
-}
-
-// SetupWithManager sets up the controller with the Manager.
-func (r *CredentialsRequestsReconciler) SetupWithManager(mgr ctrl.Manager) error {
- b := ctrl.NewControllerManagedBy(mgr)
- return r.buildController(k8s.NewCtrlBuilder(b))
-}
-
-func (r *CredentialsRequestsReconciler) buildController(bld k8s.Builder) error {
- return bld.
- For(&lokiv1.LokiStack{}).
- Complete(r)
-}
diff --git a/operator/controllers/loki/credentialsrequests_controller_test.go b/operator/controllers/loki/credentialsrequests_controller_test.go
deleted file mode 100644
index 3c91ee2275e97..0000000000000
--- a/operator/controllers/loki/credentialsrequests_controller_test.go
+++ /dev/null
@@ -1,164 +0,0 @@
-package controllers
-
-import (
- "context"
- "testing"
-
- cloudcredentialsv1 "github.com/openshift/cloud-credential-operator/pkg/apis/cloudcredential/v1"
- "github.com/stretchr/testify/require"
- corev1 "k8s.io/api/core/v1"
- apierrors "k8s.io/apimachinery/pkg/api/errors"
- metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
- "k8s.io/apimachinery/pkg/runtime/schema"
- "k8s.io/apimachinery/pkg/types"
- ctrl "sigs.k8s.io/controller-runtime"
- "sigs.k8s.io/controller-runtime/pkg/client"
-
- lokiv1 "github.com/grafana/loki/operator/apis/loki/v1"
- "github.com/grafana/loki/operator/internal/external/k8s/k8sfakes"
- "github.com/grafana/loki/operator/internal/manifests/storage"
-)
-
-func TestCredentialsRequestController_RegistersCustomResource_WithDefaultPredicates(t *testing.T) {
- b := &k8sfakes.FakeBuilder{}
- k := &k8sfakes.FakeClient{}
- c := &CredentialsRequestsReconciler{Client: k, Scheme: scheme}
-
- b.ForReturns(b)
- b.OwnsReturns(b)
-
- err := c.buildController(b)
- require.NoError(t, err)
-
- // Require only one For-Call for the custom resource
- require.Equal(t, 1, b.ForCallCount())
-
- // Require For-call with LokiStack resource
- obj, _ := b.ForArgsForCall(0)
- require.Equal(t, &lokiv1.LokiStack{}, obj)
-}
-
-func TestCredentialsRequestController_DeleteCredentialsRequest_WhenLokiStackNotFound(t *testing.T) {
- k := &k8sfakes.FakeClient{}
- c := &CredentialsRequestsReconciler{Client: k, Scheme: scheme}
- r := ctrl.Request{
- NamespacedName: types.NamespacedName{
- Name: "my-stack",
- Namespace: "ns",
- },
- }
-
- // Set managed auth environment
- t.Setenv("ROLEARN", "a-role-arn")
-
- k.GetStub = func(_ context.Context, key types.NamespacedName, _ client.Object, _ ...client.GetOption) error {
- if key.Name == r.Name && key.Namespace == r.Namespace {
- return apierrors.NewNotFound(schema.GroupResource{}, "lokistack not found")
- }
- return nil
- }
-
- res, err := c.Reconcile(context.Background(), r)
- require.NoError(t, err)
- require.Equal(t, ctrl.Result{}, res)
- require.Equal(t, 1, k.DeleteCallCount())
-}
-
-func TestCredentialsRequestController_CreateCredentialsRequest_WhenLokiStackNotAnnotated(t *testing.T) {
- k := &k8sfakes.FakeClient{}
- c := &CredentialsRequestsReconciler{Client: k, Scheme: scheme}
- r := ctrl.Request{
- NamespacedName: types.NamespacedName{
- Name: "my-stack",
- Namespace: "ns",
- },
- }
- s := lokiv1.LokiStack{
- ObjectMeta: metav1.ObjectMeta{
- Name: "my-stack",
- Namespace: "ns",
- },
- Spec: lokiv1.LokiStackSpec{
- ManagementState: lokiv1.ManagementStateManaged,
- },
- }
- secret := &corev1.Secret{}
-
- // Set managed auth environment
- t.Setenv("ROLEARN", "a-role-arn")
-
- k.GetStub = func(_ context.Context, key types.NamespacedName, out client.Object, _ ...client.GetOption) error {
- switch out.(type) {
- case *lokiv1.LokiStack:
- if key.Name == r.Name && key.Namespace == r.Namespace {
- k.SetClientObject(out, &s)
- return nil
- }
- return apierrors.NewNotFound(schema.GroupResource{}, "lokistack not found")
- case *corev1.Secret:
- k.SetClientObject(out, secret)
- return nil
- }
- return nil
- }
-
- k.CreateStub = func(_ context.Context, o client.Object, _ ...client.CreateOption) error {
- _, isCredReq := o.(*cloudcredentialsv1.CredentialsRequest)
- if !isCredReq {
- return apierrors.NewBadRequest("something went wrong creating a credentials request")
- }
- return nil
- }
-
- k.UpdateStub = func(_ context.Context, o client.Object, _ ...client.UpdateOption) error {
- stack, ok := o.(*lokiv1.LokiStack)
- if !ok {
- return apierrors.NewBadRequest("something went wrong creating a credentials request")
- }
-
- _, hasSecretRef := stack.Annotations[storage.AnnotationCredentialsRequestsSecretRef]
- if !hasSecretRef {
- return apierrors.NewBadRequest("something went updating the lokistack annotations")
- }
- return nil
- }
-
- res, err := c.Reconcile(context.Background(), r)
- require.NoError(t, err)
- require.Equal(t, ctrl.Result{}, res)
- require.Equal(t, 1, k.CreateCallCount())
- require.Equal(t, 1, k.UpdateCallCount())
-}
-
-func TestCredentialsRequestController_SkipsUnmanaged(t *testing.T) {
- k := &k8sfakes.FakeClient{}
- c := &CredentialsRequestsReconciler{Client: k, Scheme: scheme}
- r := ctrl.Request{
- NamespacedName: types.NamespacedName{
- Name: "my-stack",
- Namespace: "ns",
- },
- }
-
- s := lokiv1.LokiStack{
- ObjectMeta: metav1.ObjectMeta{
- Name: "my-stack",
- Namespace: "ns",
- },
- Spec: lokiv1.LokiStackSpec{
- ManagementState: lokiv1.ManagementStateUnmanaged,
- },
- }
-
- k.GetStub = func(_ context.Context, key types.NamespacedName, out client.Object, _ ...client.GetOption) error {
- if key.Name == s.Name && key.Namespace == s.Namespace {
- k.SetClientObject(out, &s)
- return nil
- }
- return apierrors.NewNotFound(schema.GroupResource{}, "something not found")
- }
-
- res, err := c.Reconcile(context.Background(), r)
- require.NoError(t, err)
- require.Equal(t, ctrl.Result{}, res)
-}
diff --git a/operator/controllers/loki/internal/lokistack/credentialsrequest_discovery.go b/operator/controllers/loki/internal/lokistack/credentialsrequest_discovery.go
deleted file mode 100644
index c911c1196eed4..0000000000000
--- a/operator/controllers/loki/internal/lokistack/credentialsrequest_discovery.go
+++ /dev/null
@@ -1,30 +0,0 @@
-package lokistack
-
-import (
- "context"
-
- "github.com/ViaQ/logerr/v2/kverrors"
- "sigs.k8s.io/controller-runtime/pkg/client"
-
- "github.com/grafana/loki/operator/internal/external/k8s"
- "github.com/grafana/loki/operator/internal/manifests/storage"
-)
-
-// AnnotateForCredentialsRequest adds the `loki.grafana.com/credentials-request-secret-ref` annotation
-// to the named Lokistack. If no LokiStack is found, then skip reconciliation. Or else return an error.
-func AnnotateForCredentialsRequest(ctx context.Context, k k8s.Client, key client.ObjectKey, secretRef string) error {
- stack, err := getLokiStack(ctx, k, key)
- if stack == nil || err != nil {
- return err
- }
-
- if val, ok := stack.Annotations[storage.AnnotationCredentialsRequestsSecretRef]; ok && val == secretRef {
- return nil
- }
-
- if err := updateAnnotation(ctx, k, stack, storage.AnnotationCredentialsRequestsSecretRef, secretRef); err != nil {
- return kverrors.Wrap(err, "failed to update lokistack `credentialsRequestSecretRef` annotation", "key", key)
- }
-
- return nil
-}
diff --git a/operator/controllers/loki/internal/lokistack/credentialsrequest_discovery_test.go b/operator/controllers/loki/internal/lokistack/credentialsrequest_discovery_test.go
deleted file mode 100644
index ef073ca853ba5..0000000000000
--- a/operator/controllers/loki/internal/lokistack/credentialsrequest_discovery_test.go
+++ /dev/null
@@ -1,98 +0,0 @@
-package lokistack
-
-import (
- "context"
- "testing"
-
- "github.com/stretchr/testify/require"
- apierrors "k8s.io/apimachinery/pkg/api/errors"
- metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
- "k8s.io/apimachinery/pkg/types"
- "sigs.k8s.io/controller-runtime/pkg/client"
-
- lokiv1 "github.com/grafana/loki/operator/apis/loki/v1"
- "github.com/grafana/loki/operator/internal/external/k8s/k8sfakes"
- "github.com/grafana/loki/operator/internal/manifests/storage"
-)
-
-func TestAnnotateForCredentialsRequest_ReturnError_WhenLokiStackMissing(t *testing.T) {
- k := &k8sfakes.FakeClient{}
- annotationVal := "ns-my-stack-aws-creds"
- stackKey := client.ObjectKey{Name: "my-stack", Namespace: "ns"}
-
- k.GetStub = func(_ context.Context, _ types.NamespacedName, out client.Object, _ ...client.GetOption) error {
- return apierrors.NewBadRequest("failed to get lokistack")
- }
-
- err := AnnotateForCredentialsRequest(context.Background(), k, stackKey, annotationVal)
- require.Error(t, err)
-}
-
-func TestAnnotateForCredentialsRequest_DoNothing_WhenAnnotationExists(t *testing.T) {
- k := &k8sfakes.FakeClient{}
-
- annotationVal := "ns-my-stack-aws-creds"
- s := &lokiv1.LokiStack{
- ObjectMeta: metav1.ObjectMeta{
- Name: "my-stack",
- Namespace: "ns",
- Annotations: map[string]string{
- storage.AnnotationCredentialsRequestsSecretRef: annotationVal,
- },
- },
- }
- stackKey := client.ObjectKeyFromObject(s)
-
- k.GetStub = func(_ context.Context, key types.NamespacedName, out client.Object, _ ...client.GetOption) error {
- if key.Name == stackKey.Name && key.Namespace == stackKey.Namespace {
- k.SetClientObject(out, s)
- return nil
- }
- return nil
- }
-
- err := AnnotateForCredentialsRequest(context.Background(), k, stackKey, annotationVal)
- require.NoError(t, err)
- require.Equal(t, 0, k.UpdateCallCount())
-}
-
-func TestAnnotateForCredentialsRequest_UpdateLokistack_WhenAnnotationMissing(t *testing.T) {
- k := &k8sfakes.FakeClient{}
-
- annotationVal := "ns-my-stack-aws-creds"
- s := &lokiv1.LokiStack{
- ObjectMeta: metav1.ObjectMeta{
- Name: "my-stack",
- Namespace: "ns",
- Annotations: map[string]string{},
- },
- }
- stackKey := client.ObjectKeyFromObject(s)
-
- k.GetStub = func(_ context.Context, key types.NamespacedName, out client.Object, _ ...client.GetOption) error {
- if key.Name == stackKey.Name && key.Namespace == stackKey.Namespace {
- k.SetClientObject(out, s)
- return nil
- }
- return nil
- }
-
- k.UpdateStub = func(_ context.Context, o client.Object, _ ...client.UpdateOption) error {
- stack, ok := o.(*lokiv1.LokiStack)
- if !ok {
- return apierrors.NewBadRequest("failed conversion to *lokiv1.LokiStack")
- }
- val, ok := stack.Annotations[storage.AnnotationCredentialsRequestsSecretRef]
- if !ok {
- return apierrors.NewBadRequest("missing annotation")
- }
- if val != annotationVal {
- return apierrors.NewBadRequest("annotations does not match input")
- }
- return nil
- }
-
- err := AnnotateForCredentialsRequest(context.Background(), k, stackKey, annotationVal)
- require.NoError(t, err)
- require.Equal(t, 1, k.UpdateCallCount())
-}
diff --git a/operator/controllers/loki/lokistack_controller.go b/operator/controllers/loki/lokistack_controller.go
index 40e7691bd1a2b..eb30a1a9bf555 100644
--- a/operator/controllers/loki/lokistack_controller.go
+++ b/operator/controllers/loki/lokistack_controller.go
@@ -3,7 +3,6 @@ package controllers
import (
"context"
"errors"
- "strings"
"time"
"github.com/go-logr/logr"
@@ -16,7 +15,6 @@ import (
corev1 "k8s.io/api/core/v1"
networkingv1 "k8s.io/api/networking/v1"
rbacv1 "k8s.io/api/rbac/v1"
- apierrors "k8s.io/apimachinery/pkg/api/errors"
"k8s.io/apimachinery/pkg/runtime"
"k8s.io/apimachinery/pkg/types"
ctrl "sigs.k8s.io/controller-runtime"
@@ -31,6 +29,7 @@ import (
configv1 "github.com/grafana/loki/operator/apis/config/v1"
lokiv1 "github.com/grafana/loki/operator/apis/loki/v1"
"github.com/grafana/loki/operator/controllers/loki/internal/management/state"
+ "github.com/grafana/loki/operator/internal/config"
"github.com/grafana/loki/operator/internal/external/k8s"
"github.com/grafana/loki/operator/internal/handlers"
manifestsocp "github.com/grafana/loki/operator/internal/manifests/openshift"
@@ -111,6 +110,7 @@ type LokiStackReconciler struct {
Log logr.Logger
Scheme *runtime.Scheme
FeatureGates configv1.FeatureGates
+ AuthConfig *config.ManagedAuthConfig
}
// +kubebuilder:rbac:groups=loki.grafana.com,resources=lokistacks,verbs=get;list;watch;create;update;patch;delete
@@ -128,7 +128,7 @@ type LokiStackReconciler struct {
// +kubebuilder:rbac:groups=policy,resources=poddisruptionbudgets,verbs=get;list;watch;create;update
// +kubebuilder:rbac:groups=config.openshift.io,resources=dnses;apiservers;proxies,verbs=get;list;watch
// +kubebuilder:rbac:groups=route.openshift.io,resources=routes,verbs=get;list;watch;create;update;delete
-// +kubebuilder:rbac:groups=cloudcredential.openshift.io,resources=credentialsrequests,verbs=get;list;watch;create;delete
+// +kubebuilder:rbac:groups=cloudcredential.openshift.io,resources=credentialsrequests,verbs=get;list;watch;create;update;delete
// Reconcile is part of the main kubernetes reconciliation loop which aims to
// move the current state of the cluster closer to the desired state.
@@ -150,7 +150,7 @@ func (r *LokiStackReconciler) Reconcile(ctx context.Context, req ctrl.Request) (
}
var degraded *status.DegradedError
- err = r.updateResources(ctx, req)
+ credentialMode, err := r.updateResources(ctx, req)
switch {
case errors.As(err, °raded):
// degraded errors are handled by status.Refresh below
@@ -158,7 +158,7 @@ func (r *LokiStackReconciler) Reconcile(ctx context.Context, req ctrl.Request) (
return ctrl.Result{}, err
}
- err = status.Refresh(ctx, r.Client, req, time.Now(), degraded)
+ err = status.Refresh(ctx, r.Client, req, time.Now(), credentialMode, degraded)
if err != nil {
return ctrl.Result{}, err
}
@@ -172,18 +172,25 @@ func (r *LokiStackReconciler) Reconcile(ctx context.Context, req ctrl.Request) (
return ctrl.Result{}, nil
}
-func (r *LokiStackReconciler) updateResources(ctx context.Context, req ctrl.Request) error {
+func (r *LokiStackReconciler) updateResources(ctx context.Context, req ctrl.Request) (lokiv1.CredentialMode, error) {
if r.FeatureGates.BuiltInCertManagement.Enabled {
if err := handlers.CreateOrRotateCertificates(ctx, r.Log, req, r.Client, r.Scheme, r.FeatureGates); err != nil {
- return err
+ return "", err
}
}
- if err := handlers.CreateOrUpdateLokiStack(ctx, r.Log, req, r.Client, r.Scheme, r.FeatureGates); err != nil {
- return err
+ if r.FeatureGates.OpenShift.ManagedAuthEnv {
+ if err := handlers.CreateCredentialsRequest(ctx, r.Log, r.Scheme, r.AuthConfig, r.Client, req); err != nil {
+ return "", err
+ }
+ }
+
+ credentialMode, err := handlers.CreateOrUpdateLokiStack(ctx, r.Log, req, r.Client, r.Scheme, r.FeatureGates)
+ if err != nil {
+ return "", err
}
- return nil
+ return credentialMode, nil
}
// SetupWithManager sets up the controller with the Manager.
@@ -216,7 +223,7 @@ func (r *LokiStackReconciler) buildController(bld k8s.Builder) error {
if r.FeatureGates.OpenShift.Enabled {
bld = bld.
Owns(&routev1.Route{}, updateOrDeleteOnlyPred).
- Watches(&cloudcredentialv1.CredentialsRequest{}, r.enqueueForCredentialsRequest(), updateOrDeleteOnlyPred)
+ Owns(&cloudcredentialv1.CredentialsRequest{}, updateOrDeleteOnlyPred)
if r.FeatureGates.OpenShift.ClusterTLSPolicy {
bld = bld.Watches(&openshiftconfigv1.APIServer{}, r.enqueueAllLokiStacksHandler(), updateOrDeleteOnlyPred)
@@ -358,34 +365,3 @@ func (r *LokiStackReconciler) enqueueForStorageCA() handler.EventHandler {
return requests
})
}
-
-func (r *LokiStackReconciler) enqueueForCredentialsRequest() handler.EventHandler {
- return handler.EnqueueRequestsFromMapFunc(func(ctx context.Context, obj client.Object) []reconcile.Request {
- a := obj.GetAnnotations()
- owner, ok := a[manifestsocp.AnnotationCredentialsRequestOwner]
- if !ok {
- return nil
- }
-
- var (
- ownerParts = strings.Split(owner, "/")
- namespace = ownerParts[0]
- name = ownerParts[1]
- key = client.ObjectKey{Namespace: namespace, Name: name}
- )
-
- var stack lokiv1.LokiStack
- if err := r.Client.Get(ctx, key, &stack); err != nil {
- if !apierrors.IsNotFound(err) {
- r.Log.Error(err, "failed retrieving CredentialsRequest owning Lokistack", "key", key)
- }
- return nil
- }
-
- return []reconcile.Request{
- {
- NamespacedName: key,
- },
- }
- })
-}
diff --git a/operator/controllers/loki/lokistack_controller_test.go b/operator/controllers/loki/lokistack_controller_test.go
index 515d829766aa1..6be22022c19db 100644
--- a/operator/controllers/loki/lokistack_controller_test.go
+++ b/operator/controllers/loki/lokistack_controller_test.go
@@ -161,7 +161,18 @@ func TestLokiStackController_RegisterOwnedResourcesForUpdateOrDeleteOnly(t *test
{
obj: &routev1.Route{},
index: 10,
- ownCallsCount: 11,
+ ownCallsCount: 12,
+ featureGates: configv1.FeatureGates{
+ OpenShift: configv1.OpenShiftFeatureGates{
+ Enabled: true,
+ },
+ },
+ pred: updateOrDeleteOnlyPred,
+ },
+ {
+ obj: &cloudcredentialv1.CredentialsRequest{},
+ index: 11,
+ ownCallsCount: 12,
featureGates: configv1.FeatureGates{
OpenShift: configv1.OpenShiftFeatureGates{
Enabled: true,
@@ -203,20 +214,9 @@ func TestLokiStackController_RegisterWatchedResources(t *testing.T) {
}
table := []test{
{
- src: &cloudcredentialv1.CredentialsRequest{},
+ src: &openshiftconfigv1.APIServer{},
index: 3,
watchesCallsCount: 4,
- featureGates: configv1.FeatureGates{
- OpenShift: configv1.OpenShiftFeatureGates{
- Enabled: true,
- },
- },
- pred: updateOrDeleteOnlyPred,
- },
- {
- src: &openshiftconfigv1.APIServer{},
- index: 4,
- watchesCallsCount: 5,
featureGates: configv1.FeatureGates{
OpenShift: configv1.OpenShiftFeatureGates{
Enabled: true,
@@ -227,8 +227,8 @@ func TestLokiStackController_RegisterWatchedResources(t *testing.T) {
},
{
src: &openshiftconfigv1.Proxy{},
- index: 4,
- watchesCallsCount: 5,
+ index: 3,
+ watchesCallsCount: 4,
featureGates: configv1.FeatureGates{
OpenShift: configv1.OpenShiftFeatureGates{
Enabled: true,
diff --git a/operator/docs/operator/api.md b/operator/docs/operator/api.md
index 92f93dd970224..48fbe0c8a7e48 100644
--- a/operator/docs/operator/api.md
+++ b/operator/docs/operator/api.md
@@ -1100,6 +1100,40 @@ string
+## CredentialMode { #loki-grafana-com-v1-CredentialMode }
+(string
alias)
+
+(Appears on: LokiStackStorageStatus )
+
+
+
CredentialMode represents the type of authentication used for accessing the object storage.
+
+
+
+
+Value
+Description
+
+
+"managed"
+CredentialModeManaged represents the usage of short-lived tokens retrieved from a credential source.
+This mode is similar to CredentialModeToken,but instead of having a user-configured credential source,
+it is configured by the environment, for example the Cloud Credential Operator in OpenShift.
+This mode is only supported for certain object storage types in certain runtime environments.
+
+"static"
+CredentialModeStatic represents the usage of static, long-lived credentials stored in a Secret.
+This is the default authentication mode and available for all supported object storage types.
+
+"token"
+CredentialModeToken represents the usage of short-lived tokens retrieved from a credential source.
+In this mode the static configuration does not contain credentials needed for the object storage.
+Instead, they are generated during runtime using a service, which allows for shorter-lived credentials and
+much more granular control. This authentication mode is not supported for all object storage types.
+
+
+
+
## HashRingSpec { #loki-grafana-com-v1-HashRingSpec }
(Appears on: LokiStackSpec )
@@ -2152,6 +2186,20 @@ the Loki storage configuration.
to the LokiStack.
+
+
+credentialMode
+
+
+CredentialMode
+
+
+
+
+(Optional)
+CredentialMode contains the authentication mode used for accessing the object storage.
+
+
diff --git a/operator/docs/operator/feature-gates.md b/operator/docs/operator/feature-gates.md
index 34fbdf4b69a4d..189b72e4ddb12 100644
--- a/operator/docs/operator/feature-gates.md
+++ b/operator/docs/operator/feature-gates.md
@@ -417,7 +417,8 @@ bool
-ManagedAuthEnv enabled when the operator installation is on OpenShift STS clusters.
+ManagedAuthEnv is true when OpenShift-functions are enabled and the operator has detected
+that it is running with some kind of “workload identity” (AWS STS, Azure WIF) enabled.
diff --git a/operator/internal/config/managed_auth.go b/operator/internal/config/managed_auth.go
new file mode 100644
index 0000000000000..73598e7032f8f
--- /dev/null
+++ b/operator/internal/config/managed_auth.go
@@ -0,0 +1,48 @@
+package config
+
+import "os"
+
+type AWSEnvironment struct {
+ RoleARN string
+}
+
+type AzureEnvironment struct {
+ ClientID string
+ SubscriptionID string
+ TenantID string
+ Region string
+}
+
+type ManagedAuthConfig struct {
+ AWS *AWSEnvironment
+ Azure *AzureEnvironment
+}
+
+func discoverManagedAuthConfig() *ManagedAuthConfig {
+ // AWS
+ roleARN := os.Getenv("ROLEARN")
+
+ // Azure
+ clientID := os.Getenv("CLIENTID")
+ tenantID := os.Getenv("TENANTID")
+ subscriptionID := os.Getenv("SUBSCRIPTIONID")
+
+ switch {
+ case roleARN != "":
+ return &ManagedAuthConfig{
+ AWS: &AWSEnvironment{
+ RoleARN: roleARN,
+ },
+ }
+ case clientID != "" && tenantID != "" && subscriptionID != "":
+ return &ManagedAuthConfig{
+ Azure: &AzureEnvironment{
+ ClientID: clientID,
+ SubscriptionID: subscriptionID,
+ TenantID: tenantID,
+ },
+ }
+ }
+
+ return nil
+}
diff --git a/operator/internal/config/options.go b/operator/internal/config/options.go
index 7ed9abb526a7b..dc54404f22450 100644
--- a/operator/internal/config/options.go
+++ b/operator/internal/config/options.go
@@ -17,19 +17,24 @@ import (
// LoadConfig initializes the controller configuration, optionally overriding the defaults
// from a provided configuration file.
-func LoadConfig(scheme *runtime.Scheme, configFile string) (*configv1.ProjectConfig, ctrl.Options, error) {
+func LoadConfig(scheme *runtime.Scheme, configFile string) (*configv1.ProjectConfig, *ManagedAuthConfig, ctrl.Options, error) {
options := ctrl.Options{Scheme: scheme}
if configFile == "" {
- return &configv1.ProjectConfig{}, options, nil
+ return &configv1.ProjectConfig{}, nil, options, nil
}
ctrlCfg, err := loadConfigFile(scheme, configFile)
if err != nil {
- return nil, options, fmt.Errorf("failed to parse controller manager config file: %w", err)
+ return nil, nil, options, fmt.Errorf("failed to parse controller manager config file: %w", err)
+ }
+
+ managedAuth := discoverManagedAuthConfig()
+ if ctrlCfg.Gates.OpenShift.Enabled && managedAuth != nil {
+ ctrlCfg.Gates.OpenShift.ManagedAuthEnv = true
}
options = mergeOptionsFromFile(options, ctrlCfg)
- return ctrlCfg, options, nil
+ return ctrlCfg, managedAuth, options, nil
}
func mergeOptionsFromFile(o manager.Options, cfg *configv1.ProjectConfig) manager.Options {
diff --git a/operator/internal/handlers/credentialsrequest_create.go b/operator/internal/handlers/credentialsrequest_create.go
index 6074e10b2d5af..50e06375ffd8b 100644
--- a/operator/internal/handlers/credentialsrequest_create.go
+++ b/operator/internal/handlers/credentialsrequest_create.go
@@ -3,64 +3,102 @@ package handlers
import (
"context"
"errors"
+ "fmt"
"github.com/ViaQ/logerr/v2/kverrors"
+ "github.com/go-logr/logr"
corev1 "k8s.io/api/core/v1"
apierrors "k8s.io/apimachinery/pkg/api/errors"
+ "k8s.io/apimachinery/pkg/runtime"
+ ctrl "sigs.k8s.io/controller-runtime"
"sigs.k8s.io/controller-runtime/pkg/client"
+ ctrlutil "sigs.k8s.io/controller-runtime/pkg/controller/controllerutil"
+ lokiv1 "github.com/grafana/loki/operator/apis/loki/v1"
+ "github.com/grafana/loki/operator/internal/config"
"github.com/grafana/loki/operator/internal/external/k8s"
+ "github.com/grafana/loki/operator/internal/manifests"
"github.com/grafana/loki/operator/internal/manifests/openshift"
"github.com/grafana/loki/operator/internal/manifests/storage"
)
-var (
- errAzureNoSecretFound = errors.New("can not create CredentialsRequest: no azure secret found")
- errAzureNoRegion = errors.New("can not create CredentialsRequest: missing secret field: region")
-)
+var errAzureNoRegion = errors.New("can not create CredentialsRequest: missing secret field: region")
// CreateCredentialsRequest creates a new CredentialsRequest resource for a Lokistack
// to request a cloud credentials Secret resource from the OpenShift cloud-credentials-operator.
-func CreateCredentialsRequest(ctx context.Context, k k8s.Client, stack client.ObjectKey, secret *corev1.Secret) (string, error) {
- managedAuthEnv := openshift.DiscoverManagedAuthEnv()
- if managedAuthEnv == nil {
- return "", nil
+func CreateCredentialsRequest(ctx context.Context, log logr.Logger, scheme *runtime.Scheme, managedAuth *config.ManagedAuthConfig, k k8s.Client, req ctrl.Request) error {
+ ll := log.WithValues("lokistack", req.NamespacedName, "event", "createCredentialsRequest")
+
+ var stack lokiv1.LokiStack
+ if err := k.Get(ctx, req.NamespacedName, &stack); err != nil {
+ if apierrors.IsNotFound(err) {
+ // maybe the user deleted it before we could react? Either way this isn't an issue
+ ll.Error(err, "could not find the requested LokiStack", "name", req.String())
+ return nil
+ }
+ return kverrors.Wrap(err, "failed to lookup LokiStack", "name", req.String())
}
- if managedAuthEnv.Azure != nil && managedAuthEnv.Azure.Region == "" {
+ if managedAuth.Azure != nil && managedAuth.Azure.Region == "" {
// Managed environment for Azure does not provide Region, but we need this for the CredentialsRequest.
// This looks like an oversight when creating the UI in OpenShift, but for now we need to pull this data
// from somewhere else -> the Azure Storage Secret
- if secret == nil {
- return "", errAzureNoSecretFound
+ storageSecretName := client.ObjectKey{
+ Namespace: stack.Namespace,
+ Name: stack.Spec.Storage.Secret.Name,
+ }
+ storageSecret := &corev1.Secret{}
+ if err := k.Get(ctx, storageSecretName, storageSecret); err != nil {
+ if apierrors.IsNotFound(err) {
+ // Skip this error here as it will be picked up by the LokiStack handler instead
+ ll.Error(err, "could not find secret for LokiStack", "name", req.String())
+ return nil
+ }
+ return err
}
- region := secret.Data[storage.KeyAzureRegion]
+ region := storageSecret.Data[storage.KeyAzureRegion]
if len(region) == 0 {
- return "", errAzureNoRegion
+ return errAzureNoRegion
}
- managedAuthEnv.Azure.Region = string(region)
+ managedAuth.Azure.Region = string(region)
}
opts := openshift.Options{
BuildOpts: openshift.BuildOptions{
LokiStackName: stack.Name,
LokiStackNamespace: stack.Namespace,
+ RulerName: manifests.RulerName(stack.Name),
},
- ManagedAuthEnv: managedAuthEnv,
+ ManagedAuth: managedAuth,
}
credReq, err := openshift.BuildCredentialsRequest(opts)
if err != nil {
- return "", err
+ return err
}
- if err := k.Create(ctx, credReq); err != nil {
- if !apierrors.IsAlreadyExists(err) {
- return "", kverrors.Wrap(err, "failed to create credentialsrequest", "key", client.ObjectKeyFromObject(credReq))
- }
+ err = ctrl.SetControllerReference(&stack, credReq, scheme)
+ if err != nil {
+ return kverrors.Wrap(err, "failed to set controller owner reference to resource")
+ }
+
+ desired := credReq.DeepCopyObject().(client.Object)
+ mutateFn := manifests.MutateFuncFor(credReq, desired, map[string]string{})
+
+ op, err := ctrl.CreateOrUpdate(ctx, k, credReq, mutateFn)
+ if err != nil {
+ return kverrors.Wrap(err, "failed to configure CredentialRequest")
+ }
+
+ msg := fmt.Sprintf("Resource has been %s", op)
+ switch op {
+ case ctrlutil.OperationResultNone:
+ ll.V(1).Info(msg)
+ default:
+ ll.Info(msg)
}
- return credReq.Spec.SecretRef.Name, nil
+ return nil
}
diff --git a/operator/internal/handlers/credentialsrequest_create_test.go b/operator/internal/handlers/credentialsrequest_create_test.go
index df903eaec662f..626302a113274 100644
--- a/operator/internal/handlers/credentialsrequest_create_test.go
+++ b/operator/internal/handlers/credentialsrequest_create_test.go
@@ -8,51 +8,108 @@ import (
"github.com/stretchr/testify/require"
corev1 "k8s.io/api/core/v1"
"k8s.io/apimachinery/pkg/api/errors"
+ metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
"k8s.io/apimachinery/pkg/runtime/schema"
+ "k8s.io/apimachinery/pkg/types"
+ ctrl "sigs.k8s.io/controller-runtime"
"sigs.k8s.io/controller-runtime/pkg/client"
+ lokiv1 "github.com/grafana/loki/operator/apis/loki/v1"
+ "github.com/grafana/loki/operator/internal/config"
"github.com/grafana/loki/operator/internal/external/k8s/k8sfakes"
)
-func TestCreateCredentialsRequest_DoNothing_WhenManagedAuthEnvMissing(t *testing.T) {
+func credentialsRequestFakeClient(cr *cloudcredentialv1.CredentialsRequest, lokistack *lokiv1.LokiStack, secret *corev1.Secret) *k8sfakes.FakeClient {
k := &k8sfakes.FakeClient{}
- key := client.ObjectKey{Name: "my-stack", Namespace: "ns"}
+ k.GetStub = func(_ context.Context, name types.NamespacedName, object client.Object, _ ...client.GetOption) error {
+ switch object.(type) {
+ case *cloudcredentialv1.CredentialsRequest:
+ if cr == nil {
+ return errors.NewNotFound(schema.GroupResource{}, name.Name)
+ }
+ k.SetClientObject(object, cr)
+ case *lokiv1.LokiStack:
+ if lokistack == nil {
+ return errors.NewNotFound(schema.GroupResource{}, name.Name)
+ }
+ k.SetClientObject(object, lokistack)
+ case *corev1.Secret:
+ if secret == nil {
+ return errors.NewNotFound(schema.GroupResource{}, name.Name)
+ }
+ k.SetClientObject(object, secret)
+ }
+ return nil
+ }
- secretRef, err := CreateCredentialsRequest(context.Background(), k, key, nil)
- require.NoError(t, err)
- require.Empty(t, secretRef)
+ return k
}
func TestCreateCredentialsRequest_CreateNewResource(t *testing.T) {
- k := &k8sfakes.FakeClient{}
- key := client.ObjectKey{Name: "my-stack", Namespace: "ns"}
+ wantServiceAccountNames := []string{
+ "my-stack",
+ "my-stack-ruler",
+ }
+
+ lokistack := &lokiv1.LokiStack{
+ ObjectMeta: metav1.ObjectMeta{
+ Name: "my-stack",
+ Namespace: "ns",
+ },
+ }
- t.Setenv("ROLEARN", "a-role-arn")
+ k := credentialsRequestFakeClient(nil, lokistack, nil)
+ req := ctrl.Request{
+ NamespacedName: client.ObjectKey{Name: "my-stack", Namespace: "ns"},
+ }
+
+ managedAuth := &config.ManagedAuthConfig{
+ AWS: &config.AWSEnvironment{
+ RoleARN: "a-role-arn",
+ },
+ }
- secretRef, err := CreateCredentialsRequest(context.Background(), k, key, nil)
+ err := CreateCredentialsRequest(context.Background(), logger, scheme, managedAuth, k, req)
require.NoError(t, err)
- require.NotEmpty(t, secretRef)
require.Equal(t, 1, k.CreateCallCount())
+
+ _, obj, _ := k.CreateArgsForCall(0)
+ credReq, ok := obj.(*cloudcredentialv1.CredentialsRequest)
+ require.True(t, ok)
+
+ require.Equal(t, wantServiceAccountNames, credReq.Spec.ServiceAccountNames)
}
func TestCreateCredentialsRequest_CreateNewResourceAzure(t *testing.T) {
wantRegion := "test-region"
- k := &k8sfakes.FakeClient{}
- key := client.ObjectKey{Name: "my-stack", Namespace: "ns"}
+ lokistack := &lokiv1.LokiStack{
+ ObjectMeta: metav1.ObjectMeta{
+ Name: "my-stack",
+ Namespace: "ns",
+ },
+ }
secret := &corev1.Secret{
Data: map[string][]byte{
"region": []byte(wantRegion),
},
}
- t.Setenv("CLIENTID", "test-client-id")
- t.Setenv("TENANTID", "test-tenant-id")
- t.Setenv("SUBSCRIPTIONID", "test-subscription-id")
+ k := credentialsRequestFakeClient(nil, lokistack, secret)
+ req := ctrl.Request{
+ NamespacedName: client.ObjectKey{Name: "my-stack", Namespace: "ns"},
+ }
- secretRef, err := CreateCredentialsRequest(context.Background(), k, key, secret)
+ managedAuth := &config.ManagedAuthConfig{
+ Azure: &config.AzureEnvironment{
+ ClientID: "test-client-id",
+ SubscriptionID: "test-tenant-id",
+ TenantID: "test-subscription-id",
+ },
+ }
+
+ err := CreateCredentialsRequest(context.Background(), logger, scheme, managedAuth, k, req)
require.NoError(t, err)
- require.NotEmpty(t, secretRef)
require.Equal(t, 1, k.CreateCallCount())
_, obj, _ := k.CreateArgsForCall(0)
@@ -66,17 +123,20 @@ func TestCreateCredentialsRequest_CreateNewResourceAzure(t *testing.T) {
}
func TestCreateCredentialsRequest_CreateNewResourceAzure_Errors(t *testing.T) {
- k := &k8sfakes.FakeClient{}
- key := client.ObjectKey{Name: "my-stack", Namespace: "ns"}
+ lokistack := &lokiv1.LokiStack{
+ ObjectMeta: metav1.ObjectMeta{
+ Name: "my-stack",
+ Namespace: "ns",
+ },
+ }
+ req := ctrl.Request{
+ NamespacedName: client.ObjectKey{Name: "my-stack", Namespace: "ns"},
+ }
tt := []struct {
secret *corev1.Secret
wantError string
}{
- {
- secret: nil,
- wantError: errAzureNoSecretFound.Error(),
- },
{
secret: &corev1.Secret{},
wantError: errAzureNoRegion.Error(),
@@ -86,29 +146,52 @@ func TestCreateCredentialsRequest_CreateNewResourceAzure_Errors(t *testing.T) {
for _, tc := range tt {
tc := tc
t.Run(tc.wantError, func(t *testing.T) {
- // Not parallel (environment variables)
- t.Setenv("CLIENTID", "test-client-id")
- t.Setenv("TENANTID", "test-tenant-id")
- t.Setenv("SUBSCRIPTIONID", "test-subscription-id")
-
- _, err := CreateCredentialsRequest(context.Background(), k, key, tc.secret)
+ t.Parallel()
+
+ managedAuth := &config.ManagedAuthConfig{
+ Azure: &config.AzureEnvironment{
+ ClientID: "test-client-id",
+ SubscriptionID: "test-tenant-id",
+ TenantID: "test-subscription-id",
+ },
+ }
+ k := credentialsRequestFakeClient(nil, lokistack, tc.secret)
+
+ err := CreateCredentialsRequest(context.Background(), logger, scheme, managedAuth, k, req)
require.EqualError(t, err, tc.wantError)
})
}
}
func TestCreateCredentialsRequest_DoNothing_WhenCredentialsRequestExist(t *testing.T) {
- k := &k8sfakes.FakeClient{}
- key := client.ObjectKey{Name: "my-stack", Namespace: "ns"}
+ req := ctrl.Request{
+ NamespacedName: client.ObjectKey{Name: "my-stack", Namespace: "ns"},
+ }
- t.Setenv("ROLEARN", "a-role-arn")
+ managedAuth := &config.ManagedAuthConfig{
+ AWS: &config.AWSEnvironment{
+ RoleARN: "a-role-arn",
+ },
+ }
- k.CreateStub = func(_ context.Context, _ client.Object, _ ...client.CreateOption) error {
- return errors.NewAlreadyExists(schema.GroupResource{}, "credentialsrequest exists")
+ cr := &cloudcredentialv1.CredentialsRequest{
+ ObjectMeta: metav1.ObjectMeta{
+ Name: "my-stack",
+ Namespace: "ns",
+ },
+ }
+ lokistack := &lokiv1.LokiStack{
+ ObjectMeta: metav1.ObjectMeta{
+ Name: "my-stack",
+ Namespace: "ns",
+ },
}
- secretRef, err := CreateCredentialsRequest(context.Background(), k, key, nil)
+ k := credentialsRequestFakeClient(cr, lokistack, nil)
+
+ err := CreateCredentialsRequest(context.Background(), logger, scheme, managedAuth, k, req)
require.NoError(t, err)
- require.NotEmpty(t, secretRef)
- require.Equal(t, 1, k.CreateCallCount())
+ require.Equal(t, 2, k.GetCallCount())
+ require.Equal(t, 0, k.CreateCallCount())
+ require.Equal(t, 1, k.UpdateCallCount())
}
diff --git a/operator/internal/handlers/credentialsrequest_delete.go b/operator/internal/handlers/credentialsrequest_delete.go
deleted file mode 100644
index edf05fcb205d0..0000000000000
--- a/operator/internal/handlers/credentialsrequest_delete.go
+++ /dev/null
@@ -1,43 +0,0 @@
-package handlers
-
-import (
- "context"
-
- "github.com/ViaQ/logerr/v2/kverrors"
- "k8s.io/apimachinery/pkg/api/errors"
- "sigs.k8s.io/controller-runtime/pkg/client"
-
- "github.com/grafana/loki/operator/internal/external/k8s"
- "github.com/grafana/loki/operator/internal/manifests/openshift"
-)
-
-// DeleteCredentialsRequest deletes a LokiStack's accompanying CredentialsRequest resource
-// to trigger the OpenShift cloud-credentials-operator to wipe out any credentials related
-// Secret resource on the LokiStack namespace.
-func DeleteCredentialsRequest(ctx context.Context, k k8s.Client, stack client.ObjectKey) error {
- managedAuthEnv := openshift.DiscoverManagedAuthEnv()
- if managedAuthEnv == nil {
- return nil
- }
-
- opts := openshift.Options{
- BuildOpts: openshift.BuildOptions{
- LokiStackName: stack.Name,
- LokiStackNamespace: stack.Namespace,
- },
- ManagedAuthEnv: managedAuthEnv,
- }
-
- credReq, err := openshift.BuildCredentialsRequest(opts)
- if err != nil {
- return kverrors.Wrap(err, "failed to build credentialsrequest", "key", stack)
- }
-
- if err := k.Delete(ctx, credReq); err != nil {
- if !errors.IsNotFound(err) {
- return kverrors.Wrap(err, "failed to delete credentialsrequest", "key", client.ObjectKeyFromObject(credReq))
- }
- }
-
- return nil
-}
diff --git a/operator/internal/handlers/credentialsrequest_delete_test.go b/operator/internal/handlers/credentialsrequest_delete_test.go
deleted file mode 100644
index 57f1c005ee706..0000000000000
--- a/operator/internal/handlers/credentialsrequest_delete_test.go
+++ /dev/null
@@ -1,47 +0,0 @@
-package handlers
-
-import (
- "context"
- "testing"
-
- "github.com/stretchr/testify/require"
- "k8s.io/apimachinery/pkg/api/errors"
- "k8s.io/apimachinery/pkg/runtime/schema"
- "sigs.k8s.io/controller-runtime/pkg/client"
-
- "github.com/grafana/loki/operator/internal/external/k8s/k8sfakes"
-)
-
-func TestDeleteCredentialsRequest_DoNothing_WhenManagedAuthEnvMissing(t *testing.T) {
- k := &k8sfakes.FakeClient{}
- key := client.ObjectKey{Name: "my-stack", Namespace: "ns"}
-
- err := DeleteCredentialsRequest(context.Background(), k, key)
- require.NoError(t, err)
-}
-
-func TestDeleteCredentialsRequest_DeleteExistingResource(t *testing.T) {
- k := &k8sfakes.FakeClient{}
- key := client.ObjectKey{Name: "my-stack", Namespace: "ns"}
-
- t.Setenv("ROLEARN", "a-role-arn")
-
- err := DeleteCredentialsRequest(context.Background(), k, key)
- require.NoError(t, err)
- require.Equal(t, 1, k.DeleteCallCount())
-}
-
-func TestDeleteCredentialsRequest_DoNothing_WhenCredentialsRequestNotExists(t *testing.T) {
- k := &k8sfakes.FakeClient{}
- key := client.ObjectKey{Name: "my-stack", Namespace: "ns"}
-
- t.Setenv("ROLEARN", "a-role-arn")
-
- k.DeleteStub = func(_ context.Context, _ client.Object, _ ...client.DeleteOption) error {
- return errors.NewNotFound(schema.GroupResource{}, "credentials request not found")
- }
-
- err := DeleteCredentialsRequest(context.Background(), k, key)
- require.NoError(t, err)
- require.Equal(t, 1, k.DeleteCallCount())
-}
diff --git a/operator/internal/handlers/internal/storage/secrets.go b/operator/internal/handlers/internal/storage/secrets.go
index 21cd58b7c3c25..99bafb911ec26 100644
--- a/operator/internal/handlers/internal/storage/secrets.go
+++ b/operator/internal/handlers/internal/storage/secrets.go
@@ -59,15 +59,7 @@ func getSecrets(ctx context.Context, k k8s.Client, stack *lokiv1.LokiStack, fg c
}
if fg.OpenShift.ManagedAuthEnv {
- secretName, ok := stack.Annotations[storage.AnnotationCredentialsRequestsSecretRef]
- if !ok {
- return nil, nil, &status.DegradedError{
- Message: "Missing OpenShift cloud credentials request",
- Reason: lokiv1.ReasonMissingCredentialsRequest,
- Requeue: true,
- }
- }
-
+ secretName := storage.ManagedCredentialsSecretName(stack.Name)
managedAuthCredsKey := client.ObjectKey{Name: secretName, Namespace: stack.Namespace}
if err := k.Get(ctx, managedAuthCredsKey, &managedAuthSecret); err != nil {
if apierrors.IsNotFound(err) {
@@ -100,7 +92,7 @@ func extractSecrets(secretType lokiv1.ObjectStorageSecretType, objStore, managed
SharedStore: secretType,
}
- if fg.OpenShift.ManagedAuthEnabled() {
+ if fg.OpenShift.ManagedAuthEnv {
var managedAuthHash string
managedAuthHash, err = hashSecretData(managedAuth)
if err != nil {
@@ -190,11 +182,18 @@ func extractAzureConfigSecret(s *corev1.Secret, fg configv1.FeatureGates) (*stor
// Extract and validate optional fields
endpointSuffix := s.Data[storage.KeyAzureStorageEndpointSuffix]
audience := s.Data[storage.KeyAzureAudience]
+ region := s.Data[storage.KeyAzureRegion]
if !workloadIdentity && len(audience) > 0 {
return nil, fmt.Errorf("%w: %s", errSecretFieldNotAllowed, storage.KeyAzureAudience)
}
+ if fg.OpenShift.ManagedAuthEnv {
+ if len(region) == 0 {
+ return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyAzureRegion)
+ }
+ }
+
return &storage.AzureStorageConfig{
Env: string(env),
Container: string(container),
@@ -210,12 +209,7 @@ func validateAzureCredentials(s *corev1.Secret, fg configv1.FeatureGates) (workl
tenantID := s.Data[storage.KeyAzureStorageTenantID]
subscriptionID := s.Data[storage.KeyAzureStorageSubscriptionID]
- if fg.OpenShift.ManagedAuthEnabled() {
- region := s.Data[storage.KeyAzureRegion]
- if len(region) == 0 {
- return false, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyAzureRegion)
- }
-
+ if fg.OpenShift.ManagedAuthEnv {
if len(accountKey) > 0 || len(clientID) > 0 || len(tenantID) > 0 || len(subscriptionID) > 0 {
return false, errAzureManagedIdentityNoOverride
}
@@ -282,8 +276,8 @@ func extractGCSConfigSecret(s *corev1.Secret) (*storage.GCSStorageConfig, error)
return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyGCPWorkloadIdentityProviderAudience)
}
- if credentialsFile.CredentialsSource.File != storage.GCPDefautCredentialsFile {
- return nil, fmt.Errorf("%w: %s", errGCPWrongCredentialSourceFile, storage.GCPDefautCredentialsFile)
+ if credentialsFile.CredentialsSource.File != storage.ServiceAccountTokenFilePath {
+ return nil, fmt.Errorf("%w: %s", errGCPWrongCredentialSourceFile, storage.ServiceAccountTokenFilePath)
}
}
@@ -330,7 +324,7 @@ func extractS3ConfigSecret(s *corev1.Secret, fg configv1.FeatureGates) (*storage
)
switch {
- case fg.OpenShift.ManagedAuthEnabled():
+ case fg.OpenShift.ManagedAuthEnv:
cfg.STS = true
cfg.Audience = string(audience)
// Do not allow users overriding the role arn provided on Loki Operator installation
diff --git a/operator/internal/handlers/internal/storage/secrets_test.go b/operator/internal/handlers/internal/storage/secrets_test.go
index cc18360232315..1363cd4a660a6 100644
--- a/operator/internal/handlers/internal/storage/secrets_test.go
+++ b/operator/internal/handlers/internal/storage/secrets_test.go
@@ -71,11 +71,12 @@ func TestUnknownType(t *testing.T) {
func TestAzureExtract(t *testing.T) {
type test struct {
- name string
- secret *corev1.Secret
- managedSecret *corev1.Secret
- featureGates configv1.FeatureGates
- wantError string
+ name string
+ secret *corev1.Secret
+ managedSecret *corev1.Secret
+ featureGates configv1.FeatureGates
+ wantError string
+ wantCredentialMode lokiv1.CredentialMode
}
table := []test{
{
@@ -224,6 +225,7 @@ func TestAzureExtract(t *testing.T) {
"account_key": []byte("secret"),
},
},
+ wantCredentialMode: lokiv1.CredentialModeStatic,
},
{
name: "mandatory for workload-identity set",
@@ -239,6 +241,7 @@ func TestAzureExtract(t *testing.T) {
"region": []byte("test-region"),
},
},
+ wantCredentialMode: lokiv1.CredentialModeToken,
},
{
name: "mandatory for managed workload-identity set",
@@ -252,7 +255,14 @@ func TestAzureExtract(t *testing.T) {
},
},
managedSecret: &corev1.Secret{
- Data: map[string][]byte{},
+ ObjectMeta: metav1.ObjectMeta{
+ Name: "managed-secret",
+ },
+ Data: map[string][]byte{
+ "azure_client_id": []byte("test-client-id"),
+ "azure_tenant_id": []byte("test-tenant-id"),
+ "azure_subscription_id": []byte("test-subscription-id"),
+ },
},
featureGates: configv1.FeatureGates{
OpenShift: configv1.OpenShiftFeatureGates{
@@ -260,6 +270,7 @@ func TestAzureExtract(t *testing.T) {
ManagedAuthEnv: true,
},
},
+ wantCredentialMode: lokiv1.CredentialModeManaged,
},
{
name: "all set including optional",
@@ -273,6 +284,7 @@ func TestAzureExtract(t *testing.T) {
"endpoint_suffix": []byte("suffix"),
},
},
+ wantCredentialMode: lokiv1.CredentialModeStatic,
},
}
for _, tst := range table {
@@ -285,7 +297,8 @@ func TestAzureExtract(t *testing.T) {
require.NoError(t, err)
require.NotEmpty(t, opts.SecretName)
require.NotEmpty(t, opts.SecretSHA1)
- require.Equal(t, opts.SharedStore, lokiv1.ObjectStorageSecretAzure)
+ require.Equal(t, lokiv1.ObjectStorageSecretAzure, opts.SharedStore)
+ require.Equal(t, tst.wantCredentialMode, opts.CredentialMode())
} else {
require.EqualError(t, err, tst.wantError)
}
@@ -295,9 +308,10 @@ func TestAzureExtract(t *testing.T) {
func TestGCSExtract(t *testing.T) {
type test struct {
- name string
- secret *corev1.Secret
- wantError string
+ name string
+ secret *corev1.Secret
+ wantError string
+ wantCredentialMode lokiv1.CredentialMode
}
table := []test{
{
@@ -332,10 +346,10 @@ func TestGCSExtract(t *testing.T) {
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\"}}"),
+ "key.json": []byte("{\"type\": \"external_account\", \"credential_source\": {\"file\": \"/custom/path/to/secret/storage/serviceaccount/token\"}}"),
},
},
- wantError: "credential source in secret needs to point to token file: /var/run/secrets/gcp/serviceaccount/token",
+ wantError: "credential source in secret needs to point to token file: /var/run/secrets/storage/serviceaccount/token",
},
{
name: "all set",
@@ -346,6 +360,7 @@ func TestGCSExtract(t *testing.T) {
"key.json": []byte("{\"type\": \"service_account\"}"),
},
},
+ wantCredentialMode: lokiv1.CredentialModeStatic,
},
{
name: "mandatory for workload-identity set",
@@ -354,9 +369,10 @@ func TestGCSExtract(t *testing.T) {
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\"}}"),
+ "key.json": []byte("{\"type\": \"external_account\", \"credential_source\": {\"file\": \"/var/run/secrets/storage/serviceaccount/token\"}}"),
},
},
+ wantCredentialMode: lokiv1.CredentialModeToken,
},
}
for _, tst := range table {
@@ -364,9 +380,10 @@ func TestGCSExtract(t *testing.T) {
t.Run(tst.name, func(t *testing.T) {
t.Parallel()
- _, err := extractSecrets(lokiv1.ObjectStorageSecretGCS, tst.secret, nil, configv1.FeatureGates{})
+ opts, err := extractSecrets(lokiv1.ObjectStorageSecretGCS, tst.secret, nil, configv1.FeatureGates{})
if tst.wantError == "" {
require.NoError(t, err)
+ require.Equal(t, tst.wantCredentialMode, opts.CredentialMode())
} else {
require.EqualError(t, err, tst.wantError)
}
@@ -376,9 +393,10 @@ func TestGCSExtract(t *testing.T) {
func TestS3Extract(t *testing.T) {
type test struct {
- name string
- secret *corev1.Secret
- wantError string
+ name string
+ secret *corev1.Secret
+ wantError string
+ wantCredentialMode lokiv1.CredentialMode
}
table := []test{
{
@@ -456,6 +474,7 @@ func TestS3Extract(t *testing.T) {
"sse_kms_key_id": []byte("kms-key-id"),
},
},
+ wantCredentialMode: lokiv1.CredentialModeStatic,
},
{
name: "all set with SSE-KMS with encryption context",
@@ -471,6 +490,7 @@ func TestS3Extract(t *testing.T) {
"sse_kms_encryption_context": []byte("kms-encryption-ctx"),
},
},
+ wantCredentialMode: lokiv1.CredentialModeStatic,
},
{
name: "all set with SSE-S3",
@@ -484,6 +504,7 @@ func TestS3Extract(t *testing.T) {
"sse_type": []byte("SSE-S3"),
},
},
+ wantCredentialMode: lokiv1.CredentialModeStatic,
},
{
name: "all set without SSE",
@@ -496,6 +517,7 @@ func TestS3Extract(t *testing.T) {
"access_key_secret": []byte("secret"),
},
},
+ wantCredentialMode: lokiv1.CredentialModeStatic,
},
{
name: "STS missing region",
@@ -518,6 +540,7 @@ func TestS3Extract(t *testing.T) {
"region": []byte("here"),
},
},
+ wantCredentialMode: lokiv1.CredentialModeToken,
},
{
name: "STS all set",
@@ -530,6 +553,7 @@ func TestS3Extract(t *testing.T) {
"audience": []byte("audience"),
},
},
+ wantCredentialMode: lokiv1.CredentialModeToken,
},
}
for _, tst := range table {
@@ -542,7 +566,8 @@ func TestS3Extract(t *testing.T) {
require.NoError(t, err)
require.NotEmpty(t, opts.SecretName)
require.NotEmpty(t, opts.SecretSHA1)
- require.Equal(t, opts.SharedStore, lokiv1.ObjectStorageSecretS3)
+ require.Equal(t, lokiv1.ObjectStorageSecretS3, opts.SharedStore)
+ require.Equal(t, tst.wantCredentialMode, opts.CredentialMode())
} else {
require.EqualError(t, err, tst.wantError)
}
@@ -616,10 +641,11 @@ func TestS3Extract_WithOpenShiftManagedAuth(t *testing.T) {
require.NoError(t, err)
require.NotEmpty(t, opts.SecretName)
require.NotEmpty(t, opts.SecretSHA1)
- require.Equal(t, opts.SharedStore, lokiv1.ObjectStorageSecretS3)
+ require.Equal(t, lokiv1.ObjectStorageSecretS3, opts.SharedStore)
require.True(t, opts.S3.STS)
- require.Equal(t, opts.OpenShift.CloudCredentials.SecretName, tst.managedAuthSecret.Name)
+ require.Equal(t, tst.managedAuthSecret.Name, opts.OpenShift.CloudCredentials.SecretName)
require.NotEmpty(t, opts.OpenShift.CloudCredentials.SHA1)
+ require.Equal(t, lokiv1.CredentialModeManaged, opts.CredentialMode())
} else {
require.EqualError(t, err, tst.wantError)
}
@@ -767,7 +793,8 @@ func TestSwiftExtract(t *testing.T) {
require.NoError(t, err)
require.NotEmpty(t, opts.SecretName)
require.NotEmpty(t, opts.SecretSHA1)
- require.Equal(t, opts.SharedStore, lokiv1.ObjectStorageSecretSwift)
+ require.Equal(t, lokiv1.ObjectStorageSecretSwift, opts.SharedStore)
+ require.Equal(t, lokiv1.CredentialModeStatic, opts.CredentialMode())
} else {
require.EqualError(t, err, tst.wantError)
}
@@ -840,7 +867,8 @@ func TestAlibabaCloudExtract(t *testing.T) {
require.NoError(t, err)
require.NotEmpty(t, opts.SecretName)
require.NotEmpty(t, opts.SecretSHA1)
- require.Equal(t, opts.SharedStore, lokiv1.ObjectStorageSecretAlibabaCloud)
+ require.Equal(t, lokiv1.ObjectStorageSecretAlibabaCloud, opts.SharedStore)
+ require.Equal(t, lokiv1.CredentialModeStatic, opts.CredentialMode())
} else {
require.EqualError(t, err, tst.wantError)
}
diff --git a/operator/internal/handlers/internal/storage/storage_test.go b/operator/internal/handlers/internal/storage/storage_test.go
index 9e041bf99a23a..45f5b0f2865ba 100644
--- a/operator/internal/handlers/internal/storage/storage_test.go
+++ b/operator/internal/handlers/internal/storage/storage_test.go
@@ -17,7 +17,6 @@ import (
configv1 "github.com/grafana/loki/operator/apis/config/v1"
lokiv1 "github.com/grafana/loki/operator/apis/loki/v1"
"github.com/grafana/loki/operator/internal/external/k8s/k8sfakes"
- "github.com/grafana/loki/operator/internal/manifests/storage"
"github.com/grafana/loki/operator/internal/status"
)
@@ -135,77 +134,6 @@ func TestBuildOptions_WhenMissingSecret_SetDegraded(t *testing.T) {
require.Equal(t, degradedErr, err)
}
-func TestBuildOptions_WhenMissingCloudCredentialsRequest_SetDegraded(t *testing.T) {
- sw := &k8sfakes.FakeStatusWriter{}
- k := &k8sfakes.FakeClient{}
- r := ctrl.Request{
- NamespacedName: types.NamespacedName{
- Name: "my-stack",
- Namespace: "some-ns",
- },
- }
-
- fg := configv1.FeatureGates{
- OpenShift: configv1.OpenShiftFeatureGates{
- ManagedAuthEnv: true,
- },
- }
-
- degradedErr := &status.DegradedError{
- Message: "Missing OpenShift cloud credentials request",
- Reason: lokiv1.ReasonMissingCredentialsRequest,
- Requeue: true,
- }
-
- stack := &lokiv1.LokiStack{
- TypeMeta: metav1.TypeMeta{
- Kind: "LokiStack",
- },
- ObjectMeta: metav1.ObjectMeta{
- Name: "my-stack",
- Namespace: "some-ns",
- UID: "b23f9a38-9672-499f-8c29-15ede74d3ece",
- Annotations: map[string]string{},
- },
- Spec: lokiv1.LokiStackSpec{
- Size: lokiv1.SizeOneXExtraSmall,
- Storage: lokiv1.ObjectStorageSpec{
- Schemas: []lokiv1.ObjectStorageSchema{
- {
- Version: lokiv1.ObjectStorageSchemaV11,
- EffectiveDate: "2020-10-11",
- },
- },
- Secret: lokiv1.ObjectStorageSecretSpec{
- Name: defaultManagedAuthSecret.Name,
- Type: lokiv1.ObjectStorageSecretS3,
- },
- },
- },
- }
-
- k.GetStub = func(_ context.Context, name types.NamespacedName, object client.Object, _ ...client.GetOption) error {
- _, isLokiStack := object.(*lokiv1.LokiStack)
- if r.Name == name.Name && r.Namespace == name.Namespace && isLokiStack {
- k.SetClientObject(object, stack)
- return nil
- }
- if name.Name == defaultManagedAuthSecret.Name {
- k.SetClientObject(object, &defaultManagedAuthSecret)
- return nil
- }
- return apierrors.NewNotFound(schema.GroupResource{}, "something is not found")
- }
-
- k.StatusStub = func() client.StatusWriter { return sw }
-
- _, err := BuildOptions(context.TODO(), k, stack, fg)
-
- // make sure error is returned
- require.Error(t, err)
- require.Equal(t, degradedErr, err)
-}
-
func TestBuildOptions_WhenMissingCloudCredentialsSecret_SetDegraded(t *testing.T) {
sw := &k8sfakes.FakeStatusWriter{}
k := &k8sfakes.FakeClient{}
@@ -236,9 +164,6 @@ func TestBuildOptions_WhenMissingCloudCredentialsSecret_SetDegraded(t *testing.T
Name: "my-stack",
Namespace: "some-ns",
UID: "b23f9a38-9672-499f-8c29-15ede74d3ece",
- Annotations: map[string]string{
- storage.AnnotationCredentialsRequestsSecretRef: "my-stack-aws-creds",
- },
},
Spec: lokiv1.LokiStackSpec{
Size: lokiv1.SizeOneXExtraSmall,
diff --git a/operator/internal/handlers/lokistack_create_or_update.go b/operator/internal/handlers/lokistack_create_or_update.go
index 2f78f75d02c5b..47e7a309bf8b9 100644
--- a/operator/internal/handlers/lokistack_create_or_update.go
+++ b/operator/internal/handlers/lokistack_create_or_update.go
@@ -36,7 +36,7 @@ func CreateOrUpdateLokiStack(
k k8s.Client,
s *runtime.Scheme,
fg configv1.FeatureGates,
-) error {
+) (lokiv1.CredentialMode, error) {
ll := log.WithValues("lokistack", req.NamespacedName, "event", "createOrUpdate")
var stack lokiv1.LokiStack
@@ -44,9 +44,9 @@ func CreateOrUpdateLokiStack(
if apierrors.IsNotFound(err) {
// maybe the user deleted it before we could react? Either way this isn't an issue
ll.Error(err, "could not find the requested loki stack", "name", req.NamespacedName)
- return nil
+ return "", nil
}
- return kverrors.Wrap(err, "failed to lookup lokistack", "name", req.NamespacedName)
+ return "", kverrors.Wrap(err, "failed to lookup lokistack", "name", req.NamespacedName)
}
img := os.Getenv(manifests.EnvRelatedImageLoki)
@@ -61,21 +61,21 @@ func CreateOrUpdateLokiStack(
objStore, err := storage.BuildOptions(ctx, k, &stack, fg)
if err != nil {
- return err
+ return "", err
}
baseDomain, tenants, err := gateway.BuildOptions(ctx, ll, k, &stack, fg)
if err != nil {
- return err
+ return "", err
}
if err = rules.Cleanup(ctx, ll, k, &stack); err != nil {
- return err
+ return "", err
}
alertingRules, recordingRules, ruler, ocpOptions, err := rules.BuildOptions(ctx, ll, k, &stack)
if err != nil {
- return err
+ return "", err
}
certRotationRequiredAt := ""
@@ -86,7 +86,7 @@ func CreateOrUpdateLokiStack(
timeoutConfig, err := manifests.NewTimeoutConfig(stack.Spec.Limits)
if err != nil {
ll.Error(err, "failed to parse query timeout")
- return &status.DegradedError{
+ return "", &status.DegradedError{
Message: fmt.Sprintf("Error parsing query timeout: %s", err),
Reason: lokiv1.ReasonQueryTimeoutInvalid,
Requeue: false,
@@ -116,13 +116,13 @@ func CreateOrUpdateLokiStack(
if optErr := manifests.ApplyDefaultSettings(&opts); optErr != nil {
ll.Error(optErr, "failed to conform options to build settings")
- return optErr
+ return "", optErr
}
if fg.LokiStackGateway {
if optErr := manifests.ApplyGatewayDefaultOptions(&opts); optErr != nil {
ll.Error(optErr, "failed to apply defaults options to gateway settings")
- return optErr
+ return "", optErr
}
}
@@ -140,13 +140,13 @@ func CreateOrUpdateLokiStack(
if optErr := manifests.ApplyTLSSettings(&opts, tlsProfile); optErr != nil {
ll.Error(optErr, "failed to conform options to tls profile settings")
- return optErr
+ return "", optErr
}
objects, err := manifests.BuildAll(opts)
if err != nil {
ll.Error(err, "failed to build manifests")
- return err
+ return "", err
}
ll.Info("manifests built", "count", len(objects))
@@ -158,7 +158,7 @@ func CreateOrUpdateLokiStack(
// a user possibly being unable to read logs.
if err := status.SetStorageSchemaStatus(ctx, k, req, objStore.Schemas); err != nil {
ll.Error(err, "failed to set storage schema status")
- return err
+ return "", err
}
var errCount int32
@@ -182,7 +182,7 @@ func CreateOrUpdateLokiStack(
depAnnotations, err := dependentAnnotations(ctx, k, obj)
if err != nil {
l.Error(err, "failed to set dependent annotations")
- return err
+ return "", err
}
desired := obj.DeepCopyObject().(client.Object)
@@ -205,7 +205,7 @@ func CreateOrUpdateLokiStack(
}
if errCount > 0 {
- return kverrors.New("failed to configure lokistack resources", "name", req.NamespacedName)
+ return "", kverrors.New("failed to configure lokistack resources", "name", req.NamespacedName)
}
// 1x.demo is used only for development, so the metrics will not
@@ -214,7 +214,7 @@ func CreateOrUpdateLokiStack(
metrics.Collect(&opts.Stack, opts.Name)
}
- return nil
+ return objStore.CredentialMode(), nil
}
func dependentAnnotations(ctx context.Context, k k8s.Client, obj client.Object) (map[string]string, error) {
diff --git a/operator/internal/handlers/lokistack_create_or_update_test.go b/operator/internal/handlers/lokistack_create_or_update_test.go
index 4ba9a9affc369..bef5ffc9efb70 100644
--- a/operator/internal/handlers/lokistack_create_or_update_test.go
+++ b/operator/internal/handlers/lokistack_create_or_update_test.go
@@ -108,7 +108,7 @@ func TestCreateOrUpdateLokiStack_WhenGetReturnsNotFound_DoesNotError(t *testing.
k.StatusStub = func() client.StatusWriter { return sw }
- err := CreateOrUpdateLokiStack(context.TODO(), logger, r, k, scheme, featureGates)
+ _, err := CreateOrUpdateLokiStack(context.TODO(), logger, r, k, scheme, featureGates)
require.NoError(t, err)
// make sure create was NOT called because the Get failed
@@ -132,7 +132,7 @@ func TestCreateOrUpdateLokiStack_WhenGetReturnsAnErrorOtherThanNotFound_ReturnsT
k.StatusStub = func() client.StatusWriter { return sw }
- err := CreateOrUpdateLokiStack(context.TODO(), logger, r, k, scheme, featureGates)
+ _, err := CreateOrUpdateLokiStack(context.TODO(), logger, r, k, scheme, featureGates)
require.Equal(t, badRequestErr, errors.Unwrap(err))
@@ -219,7 +219,7 @@ func TestCreateOrUpdateLokiStack_SetsNamespaceOnAllObjects(t *testing.T) {
k.StatusStub = func() client.StatusWriter { return sw }
- err := CreateOrUpdateLokiStack(context.TODO(), logger, r, k, scheme, featureGates)
+ _, err := CreateOrUpdateLokiStack(context.TODO(), logger, r, k, scheme, featureGates)
require.NoError(t, err)
// make sure create was called
@@ -327,7 +327,7 @@ func TestCreateOrUpdateLokiStack_SetsOwnerRefOnAllObjects(t *testing.T) {
k.StatusStub = func() client.StatusWriter { return sw }
- err := CreateOrUpdateLokiStack(context.TODO(), logger, r, k, scheme, featureGates)
+ _, err := CreateOrUpdateLokiStack(context.TODO(), logger, r, k, scheme, featureGates)
require.NoError(t, err)
// make sure create was called
@@ -387,7 +387,7 @@ func TestCreateOrUpdateLokiStack_WhenSetControllerRefInvalid_ContinueWithOtherOb
k.StatusStub = func() client.StatusWriter { return sw }
- err := CreateOrUpdateLokiStack(context.TODO(), logger, r, k, scheme, featureGates)
+ _, err := CreateOrUpdateLokiStack(context.TODO(), logger, r, k, scheme, featureGates)
// make sure error is returned to re-trigger reconciliation
require.Error(t, err)
@@ -490,7 +490,7 @@ func TestCreateOrUpdateLokiStack_WhenGetReturnsNoError_UpdateObjects(t *testing.
k.StatusStub = func() client.StatusWriter { return sw }
- err := CreateOrUpdateLokiStack(context.TODO(), logger, r, k, scheme, featureGates)
+ _, err := CreateOrUpdateLokiStack(context.TODO(), logger, r, k, scheme, featureGates)
require.NoError(t, err)
// make sure create not called
@@ -556,7 +556,7 @@ func TestCreateOrUpdateLokiStack_WhenCreateReturnsError_ContinueWithOtherObjects
k.StatusStub = func() client.StatusWriter { return sw }
- err := CreateOrUpdateLokiStack(context.TODO(), logger, r, k, scheme, featureGates)
+ _, err := CreateOrUpdateLokiStack(context.TODO(), logger, r, k, scheme, featureGates)
// make sure error is returned to re-trigger reconciliation
require.Error(t, err)
@@ -663,7 +663,7 @@ func TestCreateOrUpdateLokiStack_WhenUpdateReturnsError_ContinueWithOtherObjects
k.StatusStub = func() client.StatusWriter { return sw }
- err := CreateOrUpdateLokiStack(context.TODO(), logger, r, k, scheme, featureGates)
+ _, err := CreateOrUpdateLokiStack(context.TODO(), logger, r, k, scheme, featureGates)
// make sure error is returned to re-trigger reconciliation
require.Error(t, err)
@@ -734,7 +734,7 @@ func TestCreateOrUpdateLokiStack_WhenInvalidQueryTimeout_SetDegraded(t *testing.
k.StatusStub = func() client.StatusWriter { return sw }
- err := CreateOrUpdateLokiStack(context.TODO(), logger, r, k, scheme, featureGates)
+ _, err := CreateOrUpdateLokiStack(context.TODO(), logger, r, k, scheme, featureGates)
// make sure error is returned
require.Error(t, err)
diff --git a/operator/internal/manifests/mutate.go b/operator/internal/manifests/mutate.go
index 27421750bf2cc..63308bb9ceb62 100644
--- a/operator/internal/manifests/mutate.go
+++ b/operator/internal/manifests/mutate.go
@@ -6,6 +6,7 @@ import (
"github.com/ViaQ/logerr/v2/kverrors"
"github.com/imdario/mergo"
routev1 "github.com/openshift/api/route/v1"
+ cloudcredentialv1 "github.com/openshift/cloud-credential-operator/pkg/apis/cloudcredential/v1"
monitoringv1 "github.com/prometheus-operator/prometheus-operator/pkg/apis/monitoring/v1"
appsv1 "k8s.io/api/apps/v1"
corev1 "k8s.io/api/core/v1"
@@ -123,6 +124,11 @@ func MutateFuncFor(existing, desired client.Object, depAnnotations map[string]st
wantRt := desired.(*routev1.Route)
mutateRoute(rt, wantRt)
+ case *cloudcredentialv1.CredentialsRequest:
+ cr := existing.(*cloudcredentialv1.CredentialsRequest)
+ wantCr := desired.(*cloudcredentialv1.CredentialsRequest)
+ mutateCredentialRequest(cr, wantCr)
+
case *monitoringv1.PrometheusRule:
pr := existing.(*monitoringv1.PrometheusRule)
wantPr := desired.(*monitoringv1.PrometheusRule)
@@ -213,6 +219,10 @@ func mutateRoute(existing, desired *routev1.Route) {
existing.Spec = desired.Spec
}
+func mutateCredentialRequest(existing, desired *cloudcredentialv1.CredentialsRequest) {
+ existing.Spec = desired.Spec
+}
+
func mutatePrometheusRule(existing, desired *monitoringv1.PrometheusRule) {
existing.Annotations = desired.Annotations
existing.Labels = desired.Labels
diff --git a/operator/internal/manifests/openshift/credentialsrequest.go b/operator/internal/manifests/openshift/credentialsrequest.go
index 2962b61d0d1ef..0e97dd97c2b19 100644
--- a/operator/internal/manifests/openshift/credentialsrequest.go
+++ b/operator/internal/manifests/openshift/credentialsrequest.go
@@ -1,10 +1,6 @@
package openshift
import (
- "fmt"
- "os"
- "path"
-
"github.com/ViaQ/logerr/v2/kverrors"
cloudcredentialv1 "github.com/openshift/cloud-credential-operator/pkg/apis/cloudcredential/v1"
corev1 "k8s.io/api/core/v1"
@@ -12,32 +8,26 @@ import (
"k8s.io/apimachinery/pkg/runtime"
"sigs.k8s.io/controller-runtime/pkg/client"
+ "github.com/grafana/loki/operator/internal/config"
"github.com/grafana/loki/operator/internal/manifests/storage"
)
-const (
- ccoNamespace = "openshift-cloud-credential-operator"
-)
-
func BuildCredentialsRequest(opts Options) (*cloudcredentialv1.CredentialsRequest, error) {
stack := client.ObjectKey{Name: opts.BuildOpts.LokiStackName, Namespace: opts.BuildOpts.LokiStackNamespace}
- providerSpec, secretName, err := encodeProviderSpec(opts.BuildOpts.LokiStackName, opts.ManagedAuthEnv)
+ providerSpec, err := encodeProviderSpec(opts.ManagedAuth)
if err != nil {
return nil, kverrors.Wrap(err, "failed encoding credentialsrequest provider spec")
}
return &cloudcredentialv1.CredentialsRequest{
ObjectMeta: metav1.ObjectMeta{
- Name: fmt.Sprintf("%s-%s", stack.Namespace, secretName),
- Namespace: ccoNamespace,
- Annotations: map[string]string{
- AnnotationCredentialsRequestOwner: stack.String(),
- },
+ Name: stack.Name,
+ Namespace: stack.Namespace,
},
Spec: cloudcredentialv1.CredentialsRequestSpec{
SecretRef: corev1.ObjectReference{
- Name: secretName,
+ Name: storage.ManagedCredentialsSecretName(stack.Name),
Namespace: stack.Namespace,
},
ProviderSpec: providerSpec,
@@ -45,16 +35,13 @@ func BuildCredentialsRequest(opts Options) (*cloudcredentialv1.CredentialsReques
stack.Name,
rulerServiceAccountName(opts),
},
- CloudTokenPath: path.Join(storage.AWSTokenVolumeDirectory, "token"),
+ CloudTokenPath: storage.ServiceAccountTokenFilePath,
},
}, nil
}
-func encodeProviderSpec(stackName string, env *ManagedAuthEnv) (*runtime.RawExtension, string, error) {
- var (
- spec runtime.Object
- secretName string
- )
+func encodeProviderSpec(env *config.ManagedAuthConfig) (*runtime.RawExtension, error) {
+ var spec runtime.Object
switch {
case env.AWS != nil:
@@ -73,7 +60,6 @@ func encodeProviderSpec(stackName string, env *ManagedAuthEnv) (*runtime.RawExte
},
STSIAMRoleARN: env.AWS.RoleARN,
}
- secretName = fmt.Sprintf("%s-aws-creds", stackName)
case env.Azure != nil:
azure := env.Azure
@@ -101,38 +87,8 @@ func encodeProviderSpec(stackName string, env *ManagedAuthEnv) (*runtime.RawExte
AzureSubscriptionID: azure.SubscriptionID,
AzureTenantID: azure.TenantID,
}
- secretName = fmt.Sprintf("%s-azure-creds", stackName)
}
encodedSpec, err := cloudcredentialv1.Codec.EncodeProviderSpec(spec.DeepCopyObject())
- return encodedSpec, secretName, err
-}
-
-func DiscoverManagedAuthEnv() *ManagedAuthEnv {
- // AWS
- roleARN := os.Getenv("ROLEARN")
-
- // Azure
- clientID := os.Getenv("CLIENTID")
- tenantID := os.Getenv("TENANTID")
- subscriptionID := os.Getenv("SUBSCRIPTIONID")
-
- switch {
- case roleARN != "":
- return &ManagedAuthEnv{
- AWS: &AWSSTSEnv{
- RoleARN: roleARN,
- },
- }
- case clientID != "" && tenantID != "" && subscriptionID != "":
- return &ManagedAuthEnv{
- Azure: &AzureWIFEnvironment{
- ClientID: clientID,
- SubscriptionID: subscriptionID,
- TenantID: tenantID,
- },
- }
- }
-
- return nil
+ return encodedSpec, err
}
diff --git a/operator/internal/manifests/openshift/credentialsrequest_test.go b/operator/internal/manifests/openshift/credentialsrequest_test.go
index 21b193c8c7d7e..36c6e2331f7e5 100644
--- a/operator/internal/manifests/openshift/credentialsrequest_test.go
+++ b/operator/internal/manifests/openshift/credentialsrequest_test.go
@@ -1,40 +1,22 @@
package openshift
import (
- "strings"
"testing"
"github.com/stretchr/testify/require"
+ "github.com/grafana/loki/operator/internal/config"
"github.com/grafana/loki/operator/internal/manifests/storage"
)
-func TestBuildCredentialsRequest_HasOwnerAnnotation(t *testing.T) {
- opts := Options{
- BuildOpts: BuildOptions{
- LokiStackName: "a-stack",
- LokiStackNamespace: "ns",
- },
- ManagedAuthEnv: &ManagedAuthEnv{
- AWS: &AWSSTSEnv{
- RoleARN: "role-arn",
- },
- },
- }
-
- credReq, err := BuildCredentialsRequest(opts)
- require.NoError(t, err)
- require.Contains(t, credReq.Annotations, AnnotationCredentialsRequestOwner)
-}
-
func TestBuildCredentialsRequest_HasSecretRef_MatchingLokiStackNamespace(t *testing.T) {
opts := Options{
BuildOpts: BuildOptions{
LokiStackName: "a-stack",
LokiStackNamespace: "ns",
},
- ManagedAuthEnv: &ManagedAuthEnv{
- AWS: &AWSSTSEnv{
+ ManagedAuth: &config.ManagedAuthConfig{
+ AWS: &config.AWSEnvironment{
RoleARN: "role-arn",
},
},
@@ -51,8 +33,8 @@ func TestBuildCredentialsRequest_HasServiceAccountNames_ContainsAllLokiStackServ
LokiStackName: "a-stack",
LokiStackNamespace: "ns",
},
- ManagedAuthEnv: &ManagedAuthEnv{
- AWS: &AWSSTSEnv{
+ ManagedAuth: &config.ManagedAuthConfig{
+ AWS: &config.AWSEnvironment{
RoleARN: "role-arn",
},
},
@@ -70,8 +52,8 @@ func TestBuildCredentialsRequest_CloudTokenPath_MatchinOpenShiftSADirectory(t *t
LokiStackName: "a-stack",
LokiStackNamespace: "ns",
},
- ManagedAuthEnv: &ManagedAuthEnv{
- AWS: &AWSSTSEnv{
+ ManagedAuth: &config.ManagedAuthConfig{
+ AWS: &config.AWSEnvironment{
RoleARN: "role-arn",
},
},
@@ -79,7 +61,7 @@ func TestBuildCredentialsRequest_CloudTokenPath_MatchinOpenShiftSADirectory(t *t
credReq, err := BuildCredentialsRequest(opts)
require.NoError(t, err)
- require.True(t, strings.HasPrefix(credReq.Spec.CloudTokenPath, storage.AWSTokenVolumeDirectory))
+ require.Equal(t, storage.ServiceAccountTokenFilePath, credReq.Spec.CloudTokenPath)
}
func TestBuildCredentialsRequest_FollowsNamingConventions(t *testing.T) {
@@ -96,14 +78,14 @@ func TestBuildCredentialsRequest_FollowsNamingConventions(t *testing.T) {
LokiStackName: "a-stack",
LokiStackNamespace: "ns",
},
- ManagedAuthEnv: &ManagedAuthEnv{
- AWS: &AWSSTSEnv{
+ ManagedAuth: &config.ManagedAuthConfig{
+ AWS: &config.AWSEnvironment{
RoleARN: "role-arn",
},
},
},
- wantName: "ns-a-stack-aws-creds",
- wantSecretName: "a-stack-aws-creds",
+ wantName: "a-stack",
+ wantSecretName: "a-stack-managed-credentials",
},
}
for _, test := range tests {
diff --git a/operator/internal/manifests/openshift/options.go b/operator/internal/manifests/openshift/options.go
index 9bc2e4faae36e..572db7fe64453 100644
--- a/operator/internal/manifests/openshift/options.go
+++ b/operator/internal/manifests/openshift/options.go
@@ -6,6 +6,7 @@ import (
"time"
lokiv1 "github.com/grafana/loki/operator/apis/loki/v1"
+ "github.com/grafana/loki/operator/internal/config"
)
// Options is the set of internal template options for rendering
@@ -14,7 +15,7 @@ type Options struct {
BuildOpts BuildOptions
Authentication []AuthenticationSpec
Authorization AuthorizationSpec
- ManagedAuthEnv *ManagedAuthEnv
+ ManagedAuth *config.ManagedAuthConfig
}
// AuthenticationSpec describes the authentication specification
@@ -55,22 +56,6 @@ type TenantData struct {
CookieSecret string
}
-type AWSSTSEnv struct {
- RoleARN string
-}
-
-type AzureWIFEnvironment struct {
- ClientID string
- SubscriptionID string
- TenantID string
- Region string
-}
-
-type ManagedAuthEnv struct {
- AWS *AWSSTSEnv
- Azure *AzureWIFEnvironment
-}
-
// NewOptions returns an openshift options struct.
func NewOptions(
stackName, stackNamespace string,
diff --git a/operator/internal/manifests/openshift/var.go b/operator/internal/manifests/openshift/var.go
index 84928c48d7e28..5e3ac6300e3eb 100644
--- a/operator/internal/manifests/openshift/var.go
+++ b/operator/internal/manifests/openshift/var.go
@@ -48,8 +48,6 @@ var (
MonitoringSVCUserWorkload = "alertmanager-user-workload"
MonitoringUserWorkloadNS = "openshift-user-workload-monitoring"
-
- AnnotationCredentialsRequestOwner = "loki.grafana.com/credentialsrequest-owner"
)
func authorizerRbacName(componentName string) string {
diff --git a/operator/internal/manifests/storage/configure.go b/operator/internal/manifests/storage/configure.go
index 49958ebec7b9c..ede098425323d 100644
--- a/operator/internal/manifests/storage/configure.go
+++ b/operator/internal/manifests/storage/configure.go
@@ -13,6 +13,18 @@ import (
lokiv1 "github.com/grafana/loki/operator/apis/loki/v1"
)
+var (
+ managedAuthConfigVolumeMount = corev1.VolumeMount{
+ Name: managedAuthConfigVolumeName,
+ MountPath: managedAuthConfigDirectory,
+ }
+
+ saTokenVolumeMount = corev1.VolumeMount{
+ Name: saTokenVolumeName,
+ MountPath: saTokenVolumeMountPath,
+ }
+)
+
// ConfigureDeployment appends additional pod volumes and container env vars, args, volume mounts
// based on the object storage type. Currently supported amendments:
// - All: Ensure object storage secret mounted and auth projected as env vars.
@@ -127,11 +139,11 @@ func ensureObjectStoreCredentials(p *corev1.PodSpec, opts Options) corev1.PodSpe
if managedAuthEnabled(opts) {
container.Env = append(container.Env, managedAuthCredentials(opts)...)
volumes = append(volumes, saTokenVolume(opts))
- container.VolumeMounts = append(container.VolumeMounts, saTokenVolumeMount(opts))
+ container.VolumeMounts = append(container.VolumeMounts, saTokenVolumeMount)
- if opts.OpenShift.ManagedAuthEnabled() {
- volumes = append(volumes, managedAuthVolume(opts))
- container.VolumeMounts = append(container.VolumeMounts, managedAuthVolumeMount(opts))
+ if opts.OpenShift.ManagedAuthEnabled() && opts.S3 != nil && opts.S3.STS {
+ volumes = append(volumes, managedAuthConfigVolume(opts))
+ container.VolumeMounts = append(container.VolumeMounts, managedAuthConfigVolumeMount)
}
} else {
container.Env = append(container.Env, staticAuthCredentials(opts)...)
@@ -183,13 +195,13 @@ func managedAuthCredentials(opts Options) []corev1.EnvVar {
case lokiv1.ObjectStorageSecretS3:
if opts.OpenShift.ManagedAuthEnabled() {
return []corev1.EnvVar{
- envVarFromValue(EnvAWSCredentialsFile, path.Join(managedAuthSecretDirectory, KeyAWSCredentialsFilename)),
+ envVarFromValue(EnvAWSCredentialsFile, path.Join(managedAuthConfigDirectory, KeyAWSCredentialsFilename)),
envVarFromValue(EnvAWSSdkLoadConfig, "true"),
}
} else {
return []corev1.EnvVar{
envVarFromSecret(EnvAWSRoleArn, opts.SecretName, KeyAWSRoleArn),
- envVarFromValue(EnvAWSWebIdentityTokenFile, path.Join(AWSTokenVolumeDirectory, "token")),
+ envVarFromValue(EnvAWSWebIdentityTokenFile, ServiceAccountTokenFilePath),
}
}
case lokiv1.ObjectStorageSecretAzure:
@@ -199,7 +211,7 @@ func managedAuthCredentials(opts Options) []corev1.EnvVar {
envVarFromSecret(EnvAzureClientID, opts.OpenShift.CloudCredentials.SecretName, azureManagedCredentialKeyClientID),
envVarFromSecret(EnvAzureTenantID, opts.OpenShift.CloudCredentials.SecretName, azureManagedCredentialKeyTenantID),
envVarFromSecret(EnvAzureSubscriptionID, opts.OpenShift.CloudCredentials.SecretName, azureManagedCredentialKeySubscriptionID),
- envVarFromValue(EnvAzureFederatedTokenFile, path.Join(azureTokenVolumeDirectory, "token")),
+ envVarFromValue(EnvAzureFederatedTokenFile, ServiceAccountTokenFilePath),
}
}
@@ -208,7 +220,7 @@ func managedAuthCredentials(opts Options) []corev1.EnvVar {
envVarFromSecret(EnvAzureClientID, opts.SecretName, KeyAzureStorageClientID),
envVarFromSecret(EnvAzureTenantID, opts.SecretName, KeyAzureStorageTenantID),
envVarFromSecret(EnvAzureSubscriptionID, opts.SecretName, KeyAzureStorageSubscriptionID),
- envVarFromValue(EnvAzureFederatedTokenFile, path.Join(azureTokenVolumeDirectory, "token")),
+ envVarFromValue(EnvAzureFederatedTokenFile, ServiceAccountTokenFilePath),
}
case lokiv1.ObjectStorageSecretGCS:
return []corev1.EnvVar{
@@ -301,22 +313,6 @@ func managedAuthEnabled(opts Options) bool {
}
}
-func saTokenVolumeMount(opts Options) corev1.VolumeMount {
- var tokenPath string
- switch opts.SharedStore {
- case lokiv1.ObjectStorageSecretS3:
- tokenPath = AWSTokenVolumeDirectory
- case lokiv1.ObjectStorageSecretAzure:
- tokenPath = azureTokenVolumeDirectory
- case lokiv1.ObjectStorageSecretGCS:
- tokenPath = gcpTokenVolumeDirectory
- }
- return corev1.VolumeMount{
- Name: saTokenVolumeName,
- MountPath: tokenPath,
- }
-}
-
func saTokenVolume(opts Options) corev1.Volume {
var audience string
storeType := opts.SharedStore
@@ -352,16 +348,9 @@ func saTokenVolume(opts Options) corev1.Volume {
}
}
-func managedAuthVolumeMount(opts Options) corev1.VolumeMount {
- return corev1.VolumeMount{
- Name: opts.OpenShift.CloudCredentials.SecretName,
- MountPath: managedAuthSecretDirectory,
- }
-}
-
-func managedAuthVolume(opts Options) corev1.Volume {
+func managedAuthConfigVolume(opts Options) corev1.Volume {
return corev1.Volume{
- Name: opts.OpenShift.CloudCredentials.SecretName,
+ Name: managedAuthConfigVolumeName,
VolumeSource: corev1.VolumeSource{
Secret: &corev1.SecretVolumeSource{
SecretName: opts.OpenShift.CloudCredentials.SecretName,
diff --git a/operator/internal/manifests/storage/configure_test.go b/operator/internal/manifests/storage/configure_test.go
index f17a9af6c3524..2cd7b079a4b4a 100644
--- a/operator/internal/manifests/storage/configure_test.go
+++ b/operator/internal/manifests/storage/configure_test.go
@@ -206,7 +206,7 @@ func TestConfigureDeploymentForStorageType(t *testing.T) {
{
Name: saTokenVolumeName,
ReadOnly: false,
- MountPath: "/var/run/secrets/azure/serviceaccount",
+ MountPath: saTokenVolumeMountPath,
},
},
Env: []corev1.EnvVar{
@@ -256,7 +256,7 @@ func TestConfigureDeploymentForStorageType(t *testing.T) {
},
{
Name: EnvAzureFederatedTokenFile,
- Value: "/var/run/secrets/azure/serviceaccount/token",
+ Value: "/var/run/secrets/storage/serviceaccount/token",
},
},
},
@@ -331,7 +331,7 @@ func TestConfigureDeploymentForStorageType(t *testing.T) {
{
Name: saTokenVolumeName,
ReadOnly: false,
- MountPath: "/var/run/secrets/azure/serviceaccount",
+ MountPath: saTokenVolumeMountPath,
},
},
Env: []corev1.EnvVar{
@@ -381,7 +381,7 @@ func TestConfigureDeploymentForStorageType(t *testing.T) {
},
{
Name: EnvAzureFederatedTokenFile,
- Value: "/var/run/secrets/azure/serviceaccount/token",
+ Value: "/var/run/secrets/storage/serviceaccount/token",
},
},
},
@@ -462,11 +462,7 @@ func TestConfigureDeploymentForStorageType(t *testing.T) {
{
Name: saTokenVolumeName,
ReadOnly: false,
- MountPath: "/var/run/secrets/azure/serviceaccount",
- },
- {
- Name: "cloud-credentials",
- MountPath: managedAuthSecretDirectory,
+ MountPath: saTokenVolumeMountPath,
},
},
Env: []corev1.EnvVar{
@@ -516,7 +512,7 @@ func TestConfigureDeploymentForStorageType(t *testing.T) {
},
{
Name: EnvAzureFederatedTokenFile,
- Value: "/var/run/secrets/azure/serviceaccount/token",
+ Value: "/var/run/secrets/storage/serviceaccount/token",
},
},
},
@@ -546,14 +542,6 @@ func TestConfigureDeploymentForStorageType(t *testing.T) {
},
},
},
- {
- Name: "cloud-credentials",
- VolumeSource: corev1.VolumeSource{
- Secret: &corev1.SecretVolumeSource{
- SecretName: "cloud-credentials",
- },
- },
- },
},
},
},
@@ -655,7 +643,7 @@ func TestConfigureDeploymentForStorageType(t *testing.T) {
{
Name: saTokenVolumeName,
ReadOnly: false,
- MountPath: "/var/run/secrets/gcp/serviceaccount",
+ MountPath: saTokenVolumeMountPath,
},
},
Env: []corev1.EnvVar{
@@ -810,7 +798,7 @@ func TestConfigureDeploymentForStorageType(t *testing.T) {
{
Name: saTokenVolumeName,
ReadOnly: false,
- MountPath: "/var/run/secrets/aws/serviceaccount",
+ MountPath: saTokenVolumeMountPath,
},
},
Env: []corev1.EnvVar{
@@ -827,7 +815,7 @@ func TestConfigureDeploymentForStorageType(t *testing.T) {
},
{
Name: "AWS_WEB_IDENTITY_TOKEN_FILE",
- Value: "/var/run/secrets/aws/serviceaccount/token",
+ Value: "/var/run/secrets/storage/serviceaccount/token",
},
},
},
@@ -908,13 +896,9 @@ func TestConfigureDeploymentForStorageType(t *testing.T) {
{
Name: saTokenVolumeName,
ReadOnly: false,
- MountPath: "/var/run/secrets/aws/serviceaccount",
- },
- {
- Name: "cloud-credentials",
- ReadOnly: false,
- MountPath: "/etc/storage/managed-auth",
+ MountPath: saTokenVolumeMountPath,
},
+ managedAuthConfigVolumeMount,
},
Env: []corev1.EnvVar{
{
@@ -954,7 +938,7 @@ func TestConfigureDeploymentForStorageType(t *testing.T) {
},
},
{
- Name: "cloud-credentials",
+ Name: managedAuthConfigVolumeName,
VolumeSource: corev1.VolumeSource{
Secret: &corev1.SecretVolumeSource{
SecretName: "cloud-credentials",
@@ -1340,7 +1324,7 @@ func TestConfigureStatefulSetForStorageType(t *testing.T) {
{
Name: saTokenVolumeName,
ReadOnly: false,
- MountPath: "/var/run/secrets/azure/serviceaccount",
+ MountPath: saTokenVolumeMountPath,
},
},
Env: []corev1.EnvVar{
@@ -1390,7 +1374,7 @@ func TestConfigureStatefulSetForStorageType(t *testing.T) {
},
{
Name: EnvAzureFederatedTokenFile,
- Value: "/var/run/secrets/azure/serviceaccount/token",
+ Value: "/var/run/secrets/storage/serviceaccount/token",
},
},
},
@@ -1465,7 +1449,7 @@ func TestConfigureStatefulSetForStorageType(t *testing.T) {
{
Name: saTokenVolumeName,
ReadOnly: false,
- MountPath: "/var/run/secrets/azure/serviceaccount",
+ MountPath: saTokenVolumeMountPath,
},
},
Env: []corev1.EnvVar{
@@ -1515,7 +1499,7 @@ func TestConfigureStatefulSetForStorageType(t *testing.T) {
},
{
Name: EnvAzureFederatedTokenFile,
- Value: "/var/run/secrets/azure/serviceaccount/token",
+ Value: "/var/run/secrets/storage/serviceaccount/token",
},
},
},
@@ -1596,11 +1580,7 @@ func TestConfigureStatefulSetForStorageType(t *testing.T) {
{
Name: saTokenVolumeName,
ReadOnly: false,
- MountPath: "/var/run/secrets/azure/serviceaccount",
- },
- {
- Name: "cloud-credentials",
- MountPath: managedAuthSecretDirectory,
+ MountPath: saTokenVolumeMountPath,
},
},
Env: []corev1.EnvVar{
@@ -1650,7 +1630,7 @@ func TestConfigureStatefulSetForStorageType(t *testing.T) {
},
{
Name: EnvAzureFederatedTokenFile,
- Value: "/var/run/secrets/azure/serviceaccount/token",
+ Value: "/var/run/secrets/storage/serviceaccount/token",
},
},
},
@@ -1680,14 +1660,6 @@ func TestConfigureStatefulSetForStorageType(t *testing.T) {
},
},
},
- {
- Name: "cloud-credentials",
- VolumeSource: corev1.VolumeSource{
- Secret: &corev1.SecretVolumeSource{
- SecretName: "cloud-credentials",
- },
- },
- },
},
},
},
@@ -1789,7 +1761,7 @@ func TestConfigureStatefulSetForStorageType(t *testing.T) {
{
Name: saTokenVolumeName,
ReadOnly: false,
- MountPath: "/var/run/secrets/gcp/serviceaccount",
+ MountPath: saTokenVolumeMountPath,
},
},
Env: []corev1.EnvVar{
@@ -1950,13 +1922,9 @@ func TestConfigureStatefulSetForStorageType(t *testing.T) {
{
Name: saTokenVolumeName,
ReadOnly: false,
- MountPath: "/var/run/secrets/aws/serviceaccount",
- },
- {
- Name: "cloud-credentials",
- ReadOnly: false,
- MountPath: "/etc/storage/managed-auth",
+ MountPath: saTokenVolumeMountPath,
},
+ managedAuthConfigVolumeMount,
},
Env: []corev1.EnvVar{
{
@@ -1996,7 +1964,7 @@ func TestConfigureStatefulSetForStorageType(t *testing.T) {
},
},
{
- Name: "cloud-credentials",
+ Name: managedAuthConfigVolumeName,
VolumeSource: corev1.VolumeSource{
Secret: &corev1.SecretVolumeSource{
SecretName: "cloud-credentials",
diff --git a/operator/internal/manifests/storage/options.go b/operator/internal/manifests/storage/options.go
index e525640da6c0c..6693d2261e978 100644
--- a/operator/internal/manifests/storage/options.go
+++ b/operator/internal/manifests/storage/options.go
@@ -23,12 +23,47 @@ type Options struct {
OpenShift OpenShiftOptions
}
+// CredentialMode returns which mode is used by the current storage configuration.
+// This defaults to CredentialModeStatic, but can be CredentialModeToken
+// or CredentialModeManaged depending on the object storage provide, the provided
+// secret and whether the operator is running in a managed-auth cluster.
+func (o Options) CredentialMode() lokiv1.CredentialMode {
+ if o.Azure != nil {
+ if o.OpenShift.ManagedAuthEnabled() {
+ return lokiv1.CredentialModeManaged
+ }
+
+ if o.Azure.WorkloadIdentity {
+ return lokiv1.CredentialModeToken
+ }
+ }
+
+ if o.GCS != nil {
+ if o.GCS.WorkloadIdentity {
+ return lokiv1.CredentialModeToken
+ }
+ }
+
+ if o.S3 != nil {
+ if o.OpenShift.ManagedAuthEnabled() {
+ return lokiv1.CredentialModeManaged
+ }
+
+ if o.S3.STS {
+ return lokiv1.CredentialModeToken
+ }
+ }
+
+ return lokiv1.CredentialModeStatic
+}
+
// AzureStorageConfig for Azure storage config
type AzureStorageConfig struct {
Env string
Container string
EndpointSuffix string
Audience string
+ Region string
WorkloadIdentity bool
}
diff --git a/operator/internal/manifests/storage/var.go b/operator/internal/manifests/storage/var.go
index 49ec0b0a16ae1..cbd944a821c34 100644
--- a/operator/internal/manifests/storage/var.go
+++ b/operator/internal/manifests/storage/var.go
@@ -1,5 +1,7 @@
package storage
+import "fmt"
+
const (
// EnvAlibabaCloudAccessKeyID is the environment variable to specify the AlibabaCloud client id to access S3.
EnvAlibabaCloudAccessKeyID = "ALIBABA_CLOUD_ACCESS_KEY_ID"
@@ -127,27 +129,29 @@ const (
// KeySwiftUsername is the secret data key for the OpenStack Swift password.
KeySwiftUsername = "username"
- saTokenVolumeK8sDirectory = "/var/run/secrets/kubernetes.io/serviceaccount"
- saTokenVolumeName = "bound-sa-token"
- saTokenExpiration int64 = 3600
+ saTokenVolumeName = "bound-sa-token"
+ saTokenExpiration int64 = 3600
+ saTokenVolumeMountPath = "/var/run/secrets/storage/serviceaccount"
+
+ ServiceAccountTokenFilePath = saTokenVolumeMountPath + "/token"
+
+ secretDirectory = "/etc/storage/secrets"
+ storageTLSVolume = "storage-tls"
+ caDirectory = "/etc/storage/ca"
- secretDirectory = "/etc/storage/secrets"
- managedAuthSecretDirectory = "/etc/storage/managed-auth"
- storageTLSVolume = "storage-tls"
- caDirectory = "/etc/storage/ca"
+ managedAuthConfigVolumeName = "managed-auth-config"
+ managedAuthConfigDirectory = "/etc/storage/managed-auth"
- awsDefaultAudience = "sts.amazonaws.com"
- AWSTokenVolumeDirectory = "/var/run/secrets/aws/serviceaccount"
+ awsDefaultAudience = "sts.amazonaws.com"
- azureDefaultAudience = "api://AzureADTokenExchange"
- azureTokenVolumeDirectory = "/var/run/secrets/azure/serviceaccount"
+ azureDefaultAudience = "api://AzureADTokenExchange"
azureManagedCredentialKeyClientID = "azure_client_id"
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"
)
+
+// ManagedCredentialsSecretName returns the name of the secret holding the managed credentials.
+func ManagedCredentialsSecretName(stackName string) string {
+ return fmt.Sprintf("%s-managed-credentials", stackName)
+}
diff --git a/operator/internal/status/status.go b/operator/internal/status/status.go
index 281a167355c37..c544695d3d2ea 100644
--- a/operator/internal/status/status.go
+++ b/operator/internal/status/status.go
@@ -17,7 +17,7 @@ import (
// Refresh executes an aggregate update of the LokiStack Status struct, i.e.
// - It recreates the Status.Components pod status map per component.
// - It sets the appropriate Status.Condition to true that matches the pod status maps.
-func Refresh(ctx context.Context, k k8s.Client, req ctrl.Request, now time.Time, degradedErr *DegradedError) error {
+func Refresh(ctx context.Context, k k8s.Client, req ctrl.Request, now time.Time, credentialMode lokiv1.CredentialMode, degradedErr *DegradedError) error {
var stack lokiv1.LokiStack
if err := k.Get(ctx, req.NamespacedName, &stack); err != nil {
if apierrors.IsNotFound(err) {
@@ -45,6 +45,7 @@ func Refresh(ctx context.Context, k k8s.Client, req ctrl.Request, now time.Time,
statusUpdater := func(stack *lokiv1.LokiStack) {
stack.Status.Components = *cs
stack.Status.Conditions = mergeConditions(stack.Status.Conditions, activeConditions, metaTime)
+ stack.Status.Storage.CredentialMode = credentialMode
}
statusUpdater(&stack)
diff --git a/operator/internal/status/status_test.go b/operator/internal/status/status_test.go
index c7895cbe8020e..32ef892ed1bde 100644
--- a/operator/internal/status/status_test.go
+++ b/operator/internal/status/status_test.go
@@ -54,7 +54,9 @@ func TestRefreshSuccess(t *testing.T) {
Gateway: map[corev1.PodPhase][]string{corev1.PodRunning: {"lokistack-gateway-pod-0"}},
Ruler: map[corev1.PodPhase][]string{corev1.PodRunning: {"ruler-pod-0"}},
},
- Storage: lokiv1.LokiStackStorageStatus{},
+ Storage: lokiv1.LokiStackStorageStatus{
+ CredentialMode: lokiv1.CredentialModeStatic,
+ },
Conditions: []metav1.Condition{
{
Type: string(lokiv1.ConditionReady),
@@ -68,7 +70,7 @@ func TestRefreshSuccess(t *testing.T) {
k, sw := setupListClient(t, stack, componentPods)
- err := Refresh(context.Background(), k, req, now, nil)
+ err := Refresh(context.Background(), k, req, now, lokiv1.CredentialModeStatic, nil)
require.NoError(t, err)
require.Equal(t, 1, k.GetCallCount())
@@ -130,7 +132,7 @@ func TestRefreshSuccess_ZoneAwarePendingPod(t *testing.T) {
return nil
}
- err := Refresh(context.Background(), k, req, now, nil)
+ err := Refresh(context.Background(), k, req, now, lokiv1.CredentialModeStatic, nil)
require.NoError(t, err)
require.Equal(t, 1, k.GetCallCount())
diff --git a/operator/main.go b/operator/main.go
index a88a857bcee44..e212c268cbad8 100644
--- a/operator/main.go
+++ b/operator/main.go
@@ -21,7 +21,6 @@ import (
lokiv1beta1 "github.com/grafana/loki/operator/apis/loki/v1beta1"
lokictrl "github.com/grafana/loki/operator/controllers/loki"
"github.com/grafana/loki/operator/internal/config"
- manifestsocp "github.com/grafana/loki/operator/internal/manifests/openshift"
"github.com/grafana/loki/operator/internal/metrics"
"github.com/grafana/loki/operator/internal/operator"
"github.com/grafana/loki/operator/internal/validation"
@@ -60,12 +59,16 @@ func main() {
var err error
- ctrlCfg, options, err := config.LoadConfig(scheme, configFile)
+ ctrlCfg, managedAuth, options, err := config.LoadConfig(scheme, configFile)
if err != nil {
logger.Error(err, "failed to load operator configuration")
os.Exit(1)
}
+ if managedAuth != nil {
+ logger.Info("Discovered OpenShift Cluster within a managed authentication environment")
+ }
+
if ctrlCfg.Gates.LokiStackAlerts && !ctrlCfg.Gates.ServiceMonitors {
logger.Error(kverrors.New("LokiStackAlerts flag requires ServiceMonitors"), "")
os.Exit(1)
@@ -95,16 +98,12 @@ func main() {
os.Exit(1)
}
- if ctrlCfg.Gates.OpenShift.Enabled && manifestsocp.DiscoverManagedAuthEnv() != nil {
- logger.Info("discovered OpenShift Cluster within a managed authentication environment")
- ctrlCfg.Gates.OpenShift.ManagedAuthEnv = true
- }
-
if err = (&lokictrl.LokiStackReconciler{
Client: mgr.GetClient(),
Log: logger.WithName("controllers").WithName("lokistack"),
Scheme: mgr.GetScheme(),
FeatureGates: ctrlCfg.Gates,
+ AuthConfig: managedAuth,
}).SetupWithManager(mgr); err != nil {
logger.Error(err, "unable to create controller", "controller", "lokistack")
os.Exit(1)
@@ -129,17 +128,6 @@ func main() {
}
}
- if ctrlCfg.Gates.OpenShift.ManagedAuthEnabled() {
- if err = (&lokictrl.CredentialsRequestsReconciler{
- Client: mgr.GetClient(),
- Scheme: mgr.GetScheme(),
- Log: logger.WithName("controllers").WithName("lokistack-credentialsrequest"),
- }).SetupWithManager(mgr); err != nil {
- logger.Error(err, "unable to create controller", "controller", "lokistack-credentialsrequest")
- os.Exit(1)
- }
- }
-
if ctrlCfg.Gates.LokiStackWebhook {
v := &validation.LokiStackValidator{}
if err = v.SetupWebhookWithManager(mgr); err != nil {