From de83ae0fc76fc68126fefeb55cce5f868aa7ce83 Mon Sep 17 00:00:00 2001 From: Poyzan <31743851+poyzannur@users.noreply.github.com> Date: Mon, 29 Jan 2024 11:30:10 +0000 Subject: [PATCH 01/45] Update Helm chart docs and yaml (#11800) **What this PR does / why we need it**: Following recent loki `v2.9.4` and GEL `v1.8.6` patch releases, updates Helm chart docs and yaml. Loki release drone creates is the [PR as expected](https://github.com/grafana/loki/pull/11769/files), but there seems to be a problem with the GEL drone generating the docs -which is looked at separately. Instead I manually updated them in one go. --- docs/sources/setup/install/helm/reference.md | 2 +- production/helm/loki/CHANGELOG.md | 1 + production/helm/loki/Chart.yaml | 2 +- production/helm/loki/README.md | 2 +- production/helm/loki/values.yaml | 2 +- 5 files changed, 5 insertions(+), 4 deletions(-) diff --git a/docs/sources/setup/install/helm/reference.md b/docs/sources/setup/install/helm/reference.md index 91df60bffd886..e687a560ef715 100644 --- a/docs/sources/setup/install/helm/reference.md +++ b/docs/sources/setup/install/helm/reference.md @@ -856,7 +856,7 @@ false string
-"v1.8.4"
+"v1.8.6"
 
diff --git a/production/helm/loki/CHANGELOG.md b/production/helm/loki/CHANGELOG.md index 42972ff61cef4..af81afff2a047 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.0 +- [CHANGE] Changed versions of Loki v2.9.4 and GEL v1.8.6 - [ENHANCEMENT] Bumped "grafana-agent-operator" depenency chart version to it's latest version ## 5.41.8 diff --git a/production/helm/loki/Chart.yaml b/production/helm/loki/Chart.yaml index 812e724def1ee..0f708caf4eac8 100644 --- a/production/helm/loki/Chart.yaml +++ b/production/helm/loki/Chart.yaml @@ -2,7 +2,7 @@ apiVersion: v2 name: loki description: Helm chart for Grafana Loki in simple, scalable mode type: application -appVersion: 2.9.3 +appVersion: 2.9.4 version: 5.42.0 home: https://grafana.github.io/helm-charts sources: diff --git a/production/helm/loki/README.md b/production/helm/loki/README.md index a89eaf0cd942f..8f3ab16f53672 100644 --- a/production/helm/loki/README.md +++ b/production/helm/loki/README.md @@ -1,6 +1,6 @@ # loki -![Version: 5.42.0](https://img.shields.io/badge/Version-5.42.0-informational?style=flat-square) ![Type: application](https://img.shields.io/badge/Type-application-informational?style=flat-square) ![AppVersion: 2.9.3](https://img.shields.io/badge/AppVersion-2.9.3-informational?style=flat-square) +![Version: 5.42.0](https://img.shields.io/badge/Version-5.42.0-informational?style=flat-square) ![Type: application](https://img.shields.io/badge/Type-application-informational?style=flat-square) ![AppVersion: 2.9.4](https://img.shields.io/badge/AppVersion-2.9.4-informational?style=flat-square) Helm chart for Grafana Loki in simple, scalable mode diff --git a/production/helm/loki/values.yaml b/production/helm/loki/values.yaml index c3d62491ce8e6..e2937af382a7d 100644 --- a/production/helm/loki/values.yaml +++ b/production/helm/loki/values.yaml @@ -376,7 +376,7 @@ enterprise: # Enable enterprise features, license must be provided enabled: false # Default verion of GEL to deploy - version: v1.8.4 + version: v1.8.6 # -- Optional name of the GEL cluster, otherwise will use .Release.Name # The cluster name must match what is in your GEL license cluster_name: null From c01a823f1f8d5de083dc79d67f378b8799be849c Mon Sep 17 00:00:00 2001 From: Christian Haudum Date: Mon, 29 Jan 2024 13:55:12 +0100 Subject: [PATCH 02/45] (chore) Bloom Gateway: Avoid unnecessary merging of tasks prior to passing them to the FusedQuerier (#11809) **What this PR does / why we need it**: The `requestIterator` (`v1.Iterator[v1.Request]`) can be built directly from the task and does not require any unnecessary merging of tasks, since this is done by the `v1.FusedQuerier` already. Signed-off-by: Christian Haudum --- pkg/bloomgateway/multiplexing.go | 79 +++++++++++---------------- pkg/bloomgateway/multiplexing_test.go | 24 ++++++-- pkg/bloomgateway/worker.go | 16 ++++-- 3 files changed, 59 insertions(+), 60 deletions(-) diff --git a/pkg/bloomgateway/multiplexing.go b/pkg/bloomgateway/multiplexing.go index 120e6da26f8b5..a23cb91d200c3 100644 --- a/pkg/bloomgateway/multiplexing.go +++ b/pkg/bloomgateway/multiplexing.go @@ -84,64 +84,47 @@ func (t Task) Copy(series []*logproto.GroupedChunkRefs) Task { } } -// taskMergeIterator implements v1.Iterator -type taskMergeIterator struct { - curr v1.Request - heap *v1.HeapIterator[v1.IndexedValue[*logproto.GroupedChunkRefs]] - tasks []Task - day model.Time - tokenizer *v1.NGramTokenizer - err error +func (t Task) RequestIter(tokenizer *v1.NGramTokenizer) v1.Iterator[v1.Request] { + return &requestIterator{ + series: v1.NewSliceIter(t.series), + searches: convertToSearches(t.filters, tokenizer), + channel: t.ResCh, + curr: v1.Request{}, + } } -func newTaskMergeIterator(day model.Time, tokenizer *v1.NGramTokenizer, tasks ...Task) v1.PeekingIterator[v1.Request] { - it := &taskMergeIterator{ - tasks: tasks, - curr: v1.Request{}, - day: day, - tokenizer: tokenizer, - } - it.init() - return v1.NewPeekingIter[v1.Request](it) +var _ v1.Iterator[v1.Request] = &requestIterator{} + +type requestIterator struct { + series v1.Iterator[*logproto.GroupedChunkRefs] + searches [][]byte + channel chan<- v1.Output + curr v1.Request } -func (it *taskMergeIterator) init() { - sequences := make([]v1.PeekingIterator[v1.IndexedValue[*logproto.GroupedChunkRefs]], 0, len(it.tasks)) - for i := range it.tasks { - iter := v1.NewSliceIterWithIndex(it.tasks[i].series, i) - sequences = append(sequences, iter) - } - it.heap = v1.NewHeapIterator( - func(i, j v1.IndexedValue[*logproto.GroupedChunkRefs]) bool { - return i.Value().Fingerprint < j.Value().Fingerprint - }, - sequences..., - ) - it.err = nil +// At implements v1.Iterator. +func (it *requestIterator) At() v1.Request { + + return it.curr +} + +// Err implements v1.Iterator. +func (it *requestIterator) Err() error { + return nil } -func (it *taskMergeIterator) Next() bool { - ok := it.heap.Next() +// Next implements v1.Iterator. +func (it *requestIterator) Next() bool { + ok := it.series.Next() if !ok { return false } - - group := it.heap.At() - task := it.tasks[group.Index()] - + group := it.series.At() it.curr = v1.Request{ - Fp: model.Fingerprint(group.Value().Fingerprint), - Chks: convertToChunkRefs(group.Value().Refs), - Searches: convertToSearches(task.filters, it.tokenizer), - Response: task.ResCh, + Fp: model.Fingerprint(group.Fingerprint), + Chks: convertToChunkRefs(group.Refs), + Searches: it.searches, + Response: it.channel, } return true } - -func (it *taskMergeIterator) At() v1.Request { - return it.curr -} - -func (it *taskMergeIterator) Err() error { - return it.err -} diff --git a/pkg/bloomgateway/multiplexing_test.go b/pkg/bloomgateway/multiplexing_test.go index 67277d60f232c..2215b1ed1b5d4 100644 --- a/pkg/bloomgateway/multiplexing_test.go +++ b/pkg/bloomgateway/multiplexing_test.go @@ -1,6 +1,7 @@ package bloomgateway import ( + "math" "testing" "time" @@ -8,6 +9,7 @@ import ( "github.com/stretchr/testify/require" "github.com/grafana/loki/pkg/logproto" + "github.com/grafana/loki/pkg/logql/syntax" v1 "github.com/grafana/loki/pkg/storage/bloom/v1" ) @@ -32,7 +34,6 @@ func TestTask(t *testing.T) { from, through := task.Bounds() require.Equal(t, ts.Add(-1*time.Hour), from) require.Equal(t, ts, through) - require.Equal(t, truncateDay(ts), task.day) }) } @@ -50,14 +51,18 @@ func createTasksForRequests(t *testing.T, tenant string, requests ...*logproto.F return tasks } -func TestTaskMergeIterator(t *testing.T) { +func TestTask_RequestIterator(t *testing.T) { ts := mktime("2024-01-24 12:00") - day := truncateDay(ts) tenant := "fake" tokenizer := v1.NewNGramTokenizer(4, 0) - t.Run("empty requests result in empty iterator", func(t *testing.T) { - it := newTaskMergeIterator(day, tokenizer) + t.Run("empty request yields empty iterator", func(t *testing.T) { + swb := seriesWithBounds{ + bounds: model.Interval{Start: 0, End: math.MaxInt64}, + series: []*logproto.GroupedChunkRefs{}, + } + task, _ := NewTask(tenant, swb, []syntax.LineFilter{}) + it := task.RequestIter(tokenizer) // nothing to iterate over require.False(t, it.Next()) }) @@ -97,7 +102,14 @@ func TestTaskMergeIterator(t *testing.T) { } tasks := createTasksForRequests(t, tenant, r1, r2, r3) - it := newTaskMergeIterator(day, tokenizer, tasks...) + + iters := make([]v1.PeekingIterator[v1.Request], 0, len(tasks)) + for _, task := range tasks { + iters = append(iters, v1.NewPeekingIter(task.RequestIter(tokenizer))) + } + + // merge the request iterators using the heap sort iterator + it := v1.NewHeapIterator[v1.Request](func(r1, r2 v1.Request) bool { return r1.Fp < r2.Fp }, iters...) // first item require.True(t, it.Next()) diff --git a/pkg/bloomgateway/worker.go b/pkg/bloomgateway/worker.go index 69f7859e64818..8ee8594f2e27a 100644 --- a/pkg/bloomgateway/worker.go +++ b/pkg/bloomgateway/worker.go @@ -192,7 +192,7 @@ func (w *worker) running(ctx context.Context) error { blockRefs = append(blockRefs, b.blockRef) } - err = w.processBlocksWithCallback(taskCtx, tasks[0].Tenant, day, blockRefs, tasksForBlocks) + err = w.processBlocksWithCallback(taskCtx, tasks[0].Tenant, blockRefs, tasksForBlocks) if err != nil { for _, t := range tasks { t.ErrCh <- err @@ -215,26 +215,30 @@ func (w *worker) stopping(err error) error { return nil } -func (w *worker) processBlocksWithCallback(taskCtx context.Context, tenant string, day model.Time, blockRefs []bloomshipper.BlockRef, boundedRefs []boundedTasks) error { +func (w *worker) processBlocksWithCallback(taskCtx context.Context, tenant string, blockRefs []bloomshipper.BlockRef, boundedRefs []boundedTasks) error { return w.shipper.Fetch(taskCtx, tenant, blockRefs, func(bq *v1.BlockQuerier, minFp, maxFp uint64) error { for _, b := range boundedRefs { if b.blockRef.MinFingerprint == minFp && b.blockRef.MaxFingerprint == maxFp { - return w.processBlock(bq, day, b.tasks) + return w.processBlock(bq, b.tasks) } } return nil }) } -func (w *worker) processBlock(blockQuerier *v1.BlockQuerier, day model.Time, tasks []Task) error { +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) - it := newTaskMergeIterator(day, tokenizer, tasks...) - fq := blockQuerier.Fuse([]v1.PeekingIterator[v1.Request]{it}) + 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() From e0e143a88f2e6f0c3932dd8ecc53ad5cf68b9edb Mon Sep 17 00:00:00 2001 From: Christian Haudum Date: Mon, 29 Jan 2024 16:45:35 +0100 Subject: [PATCH 03/45] Bloom Gateway: Make tasks cancelable (#11792) This PR refactors the bloom gateway workers so that tasks that have been enqueued by requests do not end up locking the results channel and therefore the worker, in case the request was cancelled (`context cancelled`) or timed out (`context deadline exceeded`). It also handles errors from the shipper in a way that they are returned to the waiting request asap so it can return and does not need to wait for all tasks to finish. This PR also fixes the worker shutdown in a way that it now gracefully stops and continues to work off the remaining tasks from the queue. --------- Signed-off-by: Christian Haudum --- pkg/bloomgateway/bloomgateway.go | 103 +++++++------- pkg/bloomgateway/bloomgateway_test.go | 107 ++++++++++++++- pkg/bloomgateway/multiplexing.go | 103 ++++++++++---- pkg/bloomgateway/multiplexing_test.go | 7 +- pkg/bloomgateway/worker.go | 191 +++++++++++++++----------- 5 files changed, 352 insertions(+), 159 deletions(-) diff --git a/pkg/bloomgateway/bloomgateway.go b/pkg/bloomgateway/bloomgateway.go index f9acd70a2ca21..f236a125d75e0 100644 --- a/pkg/bloomgateway/bloomgateway.go +++ b/pkg/bloomgateway/bloomgateway.go @@ -299,6 +299,8 @@ func (g *Gateway) FilterChunkRefs(ctx context.Context, req *logproto.FilterChunk return nil, err } + logger := log.With(g.logger, "tenant", tenantID) + // start time == end time --> empty response if req.From.Equal(req.Through) { return &logproto.FilterChunkRefResponse{ @@ -327,79 +329,60 @@ func (g *Gateway) FilterChunkRefs(ctx context.Context, req *logproto.FilterChunk return req.Refs[i].Fingerprint < req.Refs[j].Fingerprint }) - var expectedResponses int - seriesWithBloomsPerDay := partitionRequest(req) + var numSeries int + seriesByDay := partitionRequest(req) // no tasks --> empty response - if len(seriesWithBloomsPerDay) == 0 { + if len(seriesByDay) == 0 { return &logproto.FilterChunkRefResponse{ ChunkRefs: []*logproto.GroupedChunkRefs{}, }, nil } - tasks := make([]Task, 0, len(seriesWithBloomsPerDay)) - for _, seriesWithBounds := range seriesWithBloomsPerDay { - task, err := NewTask(tenantID, seriesWithBounds, req.Filters) + tasks := make([]Task, 0, len(seriesByDay)) + for _, seriesWithBounds := range seriesByDay { + task, err := NewTask(ctx, tenantID, seriesWithBounds, req.Filters) if err != nil { return nil, err } tasks = append(tasks, task) - expectedResponses += len(seriesWithBounds.series) + numSeries += len(seriesWithBounds.series) } g.activeUsers.UpdateUserTimestamp(tenantID, time.Now()) - errCh := make(chan error, 1) - resCh := make(chan v1.Output, 1) - + // Ideally we could use an unbuffered channel here, but since we return the + // request on the first error, there can be cases where the request context + // is not done yet and the consumeTask() function wants to send to the + // tasksCh, but nobody reads from it any more. + tasksCh := make(chan Task, len(tasks)) for _, task := range tasks { - level.Info(g.logger).Log("msg", "enqueue task", "task", task.ID, "day", task.day, "series", len(task.series)) + task := task + level.Info(logger).Log("msg", "enqueue task", "task", task.ID, "day", task.day, "series", len(task.series)) g.queue.Enqueue(tenantID, []string{}, task, func() { // When enqueuing, we also add the task to the pending tasks g.pendingTasks.Add(task.ID, task) }) - - // Forward responses or error to the main channels - // TODO(chaudum): Refactor to make tasks cancelable - go func(t Task) { - for { - select { - case <-ctx.Done(): - return - case err := <-t.ErrCh: - if ctx.Err() != nil { - level.Warn(g.logger).Log("msg", "received err from channel, but context is already done", "err", ctx.Err()) - return - } - errCh <- err - case res := <-t.ResCh: - level.Debug(g.logger).Log("msg", "got partial result", "task", t.ID, "tenant", tenantID, "fp_int", uint64(res.Fp), "fp_hex", res.Fp, "chunks_to_remove", res.Removals.Len()) - if ctx.Err() != nil { - level.Warn(g.logger).Log("msg", "received res from channel, but context is already done", "err", ctx.Err()) - return - } - resCh <- res - } - } - }(task) + go consumeTask(ctx, task, tasksCh, logger) } - responses := responsesPool.Get(expectedResponses) + responses := responsesPool.Get(numSeries) defer responsesPool.Put(responses) + remaining := len(tasks) outer: for { select { case <-ctx.Done(): - return nil, errors.Wrap(ctx.Err(), "waiting for results") - case err := <-errCh: - return nil, errors.Wrap(err, "waiting for results") - case res := <-resCh: - responses = append(responses, res) - // log line is helpful for debugging tests - level.Debug(g.logger).Log("msg", "got partial result", "progress", fmt.Sprintf("%d/%d", len(responses), expectedResponses)) - // wait for all parts of the full response - if len(responses) == expectedResponses { + return nil, errors.Wrap(ctx.Err(), "request failed") + case task := <-tasksCh: + level.Info(logger).Log("msg", "task done", "task", task.ID, "err", task.Err()) + if task.Err() != nil { + return nil, errors.Wrap(task.Err(), "request failed") + } + responses = append(responses, task.responses...) + remaining-- + if remaining == 0 { break outer } } @@ -415,10 +398,38 @@ outer: g.metrics.addUnfilteredCount(numChunksUnfiltered) g.metrics.addFilteredCount(len(req.Refs)) - level.Debug(g.logger).Log("msg", "return filtered chunk refs", "unfiltered", numChunksUnfiltered, "filtered", len(req.Refs)) + level.Info(logger).Log("msg", "return filtered chunk refs", "unfiltered", numChunksUnfiltered, "filtered", len(req.Refs)) return &logproto.FilterChunkRefResponse{ChunkRefs: req.Refs}, nil } +// consumeTask receives v1.Output yielded from the block querier on the task's +// result channel and stores them on the task. +// In case the context task is done, it drains the remaining items until the +// task is closed by the worker. +// Once the tasks is closed, it will send the task with the results from the +// block querier to the supplied task channel. +func consumeTask(ctx context.Context, task Task, tasksCh chan<- Task, logger log.Logger) { + logger = log.With(logger, "task", task.ID) + + for res := range task.resCh { + select { + case <-ctx.Done(): + level.Debug(logger).Log("msg", "drop partial result", "fp_int", uint64(res.Fp), "fp_hex", res.Fp, "chunks_to_remove", res.Removals.Len()) + default: + level.Debug(logger).Log("msg", "accept partial result", "fp_int", uint64(res.Fp), "fp_hex", res.Fp, "chunks_to_remove", res.Removals.Len()) + task.responses = append(task.responses, res) + } + } + + select { + case <-ctx.Done(): + // do nothing + case <-task.Done(): + // notify request handler about finished task + tasksCh <- task + } +} + func removeNotMatchingChunks(req *logproto.FilterChunkRefRequest, res v1.Output, logger log.Logger) { // binary search index of fingerprint idx := sort.Search(len(req.Refs), func(i int) bool { diff --git a/pkg/bloomgateway/bloomgateway_test.go b/pkg/bloomgateway/bloomgateway_test.go index 34dede40727c7..e9d537a4121db 100644 --- a/pkg/bloomgateway/bloomgateway_test.go +++ b/pkg/bloomgateway/bloomgateway_test.go @@ -15,6 +15,7 @@ import ( "github.com/grafana/dskit/ring" "github.com/grafana/dskit/services" "github.com/grafana/dskit/user" + "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" @@ -183,6 +184,96 @@ func TestBloomGateway_FilterChunkRefs(t *testing.T) { MaxOutstandingPerTenant: 1024, } + t.Run("shipper error is propagated", func(t *testing.T) { + reg := prometheus.NewRegistry() + gw, err := New(cfg, schemaCfg, storageCfg, limits, ss, cm, logger, reg) + require.NoError(t, err) + + 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 + + err = gw.initServices() + require.NoError(t, err) + + err = services.StartAndAwaitRunning(context.Background(), gw) + require.NoError(t, err) + t.Cleanup(func() { + err = services.StopAndAwaitTerminated(context.Background(), gw) + require.NoError(t, err) + }) + + chunkRefs := createQueryInputFromBlockData(t, tenantID, data, 10) + + // saturate workers + // then send additional request + for i := 0; i < gw.cfg.WorkerConcurrency+1; i++ { + req := &logproto.FilterChunkRefRequest{ + From: now.Add(-24 * time.Hour), + Through: now, + Refs: groupRefs(t, chunkRefs), + Filters: []syntax.LineFilter{ + {Ty: labels.MatchEqual, Match: "does not match"}, + }, + } + + ctx, cancelFn := context.WithTimeout(context.Background(), 10*time.Second) + ctx = user.InjectOrgID(ctx, tenantID) + t.Cleanup(cancelFn) + + res, err := gw.FilterChunkRefs(ctx, req) + require.ErrorContainsf(t, err, "request failed: failed to fetch block", "%+v", res) + } + }) + + 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) + require.NoError(t, err) + + 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 + + err = gw.initServices() + require.NoError(t, err) + + err = services.StartAndAwaitRunning(context.Background(), gw) + require.NoError(t, err) + t.Cleanup(func() { + err = services.StopAndAwaitTerminated(context.Background(), gw) + require.NoError(t, err) + }) + + chunkRefs := createQueryInputFromBlockData(t, tenantID, data, 100) + + // saturate workers + // then send additional request + for i := 0; i < gw.cfg.WorkerConcurrency+1; i++ { + req := &logproto.FilterChunkRefRequest{ + From: now.Add(-24 * time.Hour), + Through: now, + Refs: groupRefs(t, chunkRefs), + Filters: []syntax.LineFilter{ + {Ty: labels.MatchEqual, Match: "does not match"}, + }, + } + + ctx, cancelFn := context.WithTimeout(context.Background(), 500*time.Millisecond) + ctx = user.InjectOrgID(ctx, tenantID) + t.Cleanup(cancelFn) + + res, err := gw.FilterChunkRefs(ctx, req) + require.ErrorContainsf(t, err, context.DeadlineExceeded.Error(), "%+v", res) + } + }) + 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) @@ -428,12 +519,17 @@ func newMockBloomStore(bqs []bloomshipper.BlockQuerierWithFingerprintRange) *moc type mockBloomStore struct { bqs []bloomshipper.BlockQuerierWithFingerprintRange + // 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, tenant 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, bloomshipper.BlockRef{ @@ -452,6 +548,11 @@ func (s *mockBloomStore) Stop() {} // Fetch implements bloomshipper.Interface func (s *mockBloomStore) Fetch(_ context.Context, _ string, _ []bloomshipper.BlockRef, callback bloomshipper.ForEachBlockCallback) error { + if s.err != nil { + time.Sleep(s.delay) + return s.err + } + shuffled := make([]bloomshipper.BlockQuerierWithFingerprintRange, len(s.bqs)) _ = copy(shuffled, s.bqs) @@ -461,7 +562,11 @@ func (s *mockBloomStore) Fetch(_ context.Context, _ string, _ []bloomshipper.Blo for _, bq := range shuffled { // ignore errors in the mock - _ = callback(bq.BlockQuerier, uint64(bq.MinFp), uint64(bq.MaxFp)) + time.Sleep(s.delay) + err := callback(bq.BlockQuerier, uint64(bq.MinFp), uint64(bq.MaxFp)) + if err != nil { + return err + } } return nil } diff --git a/pkg/bloomgateway/multiplexing.go b/pkg/bloomgateway/multiplexing.go index a23cb91d200c3..97c2571948096 100644 --- a/pkg/bloomgateway/multiplexing.go +++ b/pkg/bloomgateway/multiplexing.go @@ -1,6 +1,9 @@ package bloomgateway import ( + "context" + "math/rand" + "sync" "time" "github.com/oklog/ulid" @@ -15,10 +18,29 @@ const ( Day = 24 * time.Hour ) +var ( + entropy = rand.New(rand.NewSource(time.Now().UnixNano())) +) + type tokenSettings struct { nGramLen int } +type wrappedError struct { + mu sync.Mutex + err error +} + +func (e *wrappedError) Error() string { + return e.err.Error() +} + +func (e *wrappedError) Set(err error) { + e.mu.Lock() + e.err = err + e.mu.Unlock() +} + // Task is the data structure that is enqueued to the internal queue and dequeued by query workers type Task struct { // ID is a lexcographically sortable unique identifier of the task @@ -26,10 +48,16 @@ type Task struct { // Tenant is the tenant ID Tenant string - // ErrCh is a send-only channel to write an error to - ErrCh chan error - // ResCh is a send-only channel to write partial responses to - ResCh chan v1.Output + // channel to write partial responses to + resCh chan v1.Output + // channel to notify listener that the task is done + done chan struct{} + + // the last error of the task + // needs to be a pointer so multiple copies of the task can modify its value + err *wrappedError + // the respones received from the block queriers + responses []v1.Output // series of the original request series []*logproto.GroupedChunkRefs @@ -37,6 +65,8 @@ type Task struct { filters []syntax.LineFilter // from..through date of the task's chunks bounds model.Interval + // the context from the request + ctx context.Context // TODO(chaudum): Investigate how to remove that. day model.Time @@ -45,23 +75,24 @@ type Task struct { // NewTask returns a new Task that can be enqueued to the task queue. // In addition, it returns a result and an error channel, as well // as an error if the instantiation fails. -func NewTask(tenantID string, refs seriesWithBounds, filters []syntax.LineFilter) (Task, error) { - key, err := ulid.New(ulid.Now(), nil) +func NewTask(ctx context.Context, tenantID string, refs seriesWithBounds, filters []syntax.LineFilter) (Task, error) { + key, err := ulid.New(ulid.Now(), entropy) if err != nil { return Task{}, err } - errCh := make(chan error, 1) - resCh := make(chan v1.Output, len(refs.series)) task := Task{ - ID: key, - Tenant: tenantID, - ErrCh: errCh, - ResCh: resCh, - filters: filters, - series: refs.series, - bounds: refs.bounds, - day: refs.day, + ID: key, + Tenant: tenantID, + err: new(wrappedError), + resCh: make(chan v1.Output), + filters: filters, + series: refs.series, + bounds: refs.bounds, + day: refs.day, + ctx: ctx, + done: make(chan struct{}), + responses: make([]v1.Output, 0, len(refs.series)), } return task, nil } @@ -70,17 +101,38 @@ func (t Task) Bounds() (model.Time, model.Time) { return t.bounds.Start, t.bounds.End } +func (t Task) Done() <-chan struct{} { + return t.done +} + +func (t Task) Err() error { + return t.err.err +} + +func (t Task) Close() { + close(t.resCh) + close(t.done) +} + +func (t Task) CloseWithError(err error) { + t.err.Set(err) + t.Close() +} + // Copy returns a copy of the existing task but with a new slice of grouped chunk refs func (t Task) Copy(series []*logproto.GroupedChunkRefs) Task { + // do not copy ID to distinguish it as copied task return Task{ - ID: ulid.ULID{}, // create emty ID to distinguish it as copied task - Tenant: t.Tenant, - ErrCh: t.ErrCh, - ResCh: t.ResCh, - filters: t.filters, - series: series, - bounds: t.bounds, - day: t.day, + Tenant: t.Tenant, + err: t.err, + resCh: t.resCh, + filters: t.filters, + series: series, + bounds: t.bounds, + day: t.day, + ctx: t.ctx, + done: make(chan struct{}), + responses: make([]v1.Output, 0, len(series)), } } @@ -88,7 +140,7 @@ func (t Task) RequestIter(tokenizer *v1.NGramTokenizer) v1.Iterator[v1.Request] return &requestIterator{ series: v1.NewSliceIter(t.series), searches: convertToSearches(t.filters, tokenizer), - channel: t.ResCh, + channel: t.resCh, curr: v1.Request{}, } } @@ -104,7 +156,6 @@ type requestIterator struct { // At implements v1.Iterator. func (it *requestIterator) At() v1.Request { - return it.curr } diff --git a/pkg/bloomgateway/multiplexing_test.go b/pkg/bloomgateway/multiplexing_test.go index 2215b1ed1b5d4..009c825a7e84a 100644 --- a/pkg/bloomgateway/multiplexing_test.go +++ b/pkg/bloomgateway/multiplexing_test.go @@ -1,6 +1,7 @@ package bloomgateway import ( + "context" "math" "testing" "time" @@ -29,7 +30,7 @@ func TestTask(t *testing.T) { }, } swb := partitionRequest(req)[0] - task, err := NewTask("tenant", swb, nil) + task, err := NewTask(context.Background(), "tenant", swb, nil) require.NoError(t, err) from, through := task.Bounds() require.Equal(t, ts.Add(-1*time.Hour), from) @@ -43,7 +44,7 @@ func createTasksForRequests(t *testing.T, tenant string, requests ...*logproto.F tasks := make([]Task, 0, len(requests)) for _, r := range requests { for _, swb := range partitionRequest(r) { - task, err := NewTask(tenant, swb, nil) + task, err := NewTask(context.Background(), tenant, swb, nil) require.NoError(t, err) tasks = append(tasks, task) } @@ -61,7 +62,7 @@ func TestTask_RequestIterator(t *testing.T) { bounds: model.Interval{Start: 0, End: math.MaxInt64}, series: []*logproto.GroupedChunkRefs{}, } - task, _ := NewTask(tenant, swb, []syntax.LineFilter{}) + task, _ := NewTask(context.Background(), tenant, swb, []syntax.LineFilter{}) it := task.RequestIter(tokenizer) // nothing to iterate over require.False(t, it.Next()) diff --git a/pkg/bloomgateway/worker.go b/pkg/bloomgateway/worker.go index 8ee8594f2e27a..6e08b574d683e 100644 --- a/pkg/bloomgateway/worker.go +++ b/pkg/bloomgateway/worker.go @@ -11,6 +11,7 @@ import ( "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" @@ -104,109 +105,137 @@ func (w *worker) starting(_ context.Context) error { return nil } -func (w *worker) running(ctx context.Context) error { +func (w *worker) running(_ context.Context) error { idx := queue.StartIndexWithLocalQueue - for { - select { + for st := w.State(); st == services.Running || st == services.Stopping; { + taskCtx := context.Background() + dequeueStart := time.Now() + items, newIdx, err := w.queue.DequeueMany(taskCtx, idx, w.id, w.cfg.maxItems, w.cfg.maxWaitTime) + w.metrics.dequeueWaitTime.WithLabelValues(w.id).Observe(time.Since(dequeueStart).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() + level.Error(w.logger).Log("msg", "failed to dequeue tasks", "err", err, "items", len(items)) + } + idx = newIdx - case <-ctx.Done(): - return ctx.Err() + if len(items) == 0 { + w.queue.ReleaseRequests(items) + continue + } + w.metrics.dequeuedTasks.WithLabelValues(w.id).Add(float64(len(items))) - default: - taskCtx := context.Background() - dequeueStart := time.Now() - items, newIdx, err := w.queue.DequeueMany(taskCtx, idx, w.id, w.cfg.maxItems, w.cfg.maxWaitTime) - w.metrics.dequeueWaitTime.WithLabelValues(w.id).Observe(time.Since(dequeueStart).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() - level.Error(w.logger).Log("msg", "failed to dequeue tasks", "err", err, "items", len(items)) - } - idx = newIdx + tasksPerDay := make(map[model.Time][]Task) - if len(items) == 0 { + for _, item := range items { + task, ok := item.(Task) + if !ok { + // This really should never happen, because only the bloom gateway itself can enqueue tasks. w.queue.ReleaseRequests(items) - continue + return errors.Errorf("failed to cast dequeued item to Task: %v", item) } - w.metrics.dequeuedTasks.WithLabelValues(w.id).Add(float64(len(items))) + level.Debug(w.logger).Log("msg", "dequeued task", "task", task.ID) + w.pending.Delete(task.ID) + + tasksPerDay[task.day] = append(tasksPerDay[task.day], task) + } - tasksPerDay := make(map[model.Time][]Task) + for day, tasks := range tasksPerDay { - for _, item := range items { - task, ok := item.(Task) - if !ok { - // This really should never happen, because only the bloom gateway itself can enqueue tasks. - w.queue.ReleaseRequests(items) - return errors.Errorf("failed to cast dequeued item to Task: %v", item) + // 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 } - level.Debug(w.logger).Log("msg", "dequeued task", "task", task.ID) - w.pending.Delete(task.ID) + return false + }) + // no tasks to process + // continue with tasks of next day + if len(tasks) == 0 { + continue + } - tasksPerDay[task.day] = append(tasksPerDay[task.day], task) + // interval is [Start, End) + interval := bloomshipper.Interval{ + Start: day, // inclusive + End: day.Add(Day), // non-inclusive } - for day, tasks := range tasksPerDay { - // interval is [Start, End) - interval := bloomshipper.Interval{ - Start: day, // inclusive - End: day.Add(Day), // non-inclusive - } + logger := log.With(w.logger, "day", day.Time(), "tenant", tasks[0].Tenant) + level.Debug(logger).Log("msg", "process tasks", "tasks", len(tasks)) - logger := log.With(w.logger, "day", day) - 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.ErrCh <- err - } - // continue with tasks of next day - 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 { - for _, ref := range t.series { - t.ResCh <- v1.Output{ - Fp: model.Fingerprint(ref.Fingerprint), - Removals: nil, - } - } - } - // continue with tasks of next day - continue + 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 + } - tasksForBlocks := partitionFingerprintRange(tasks, blockRefs) - blockRefs = blockRefs[:0] - for _, b := range tasksForBlocks { - blockRefs = append(blockRefs, b.blockRef) + // 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 + } - err = w.processBlocksWithCallback(taskCtx, tasks[0].Tenant, blockRefs, tasksForBlocks) - if err != nil { - for _, t := range tasks { - t.ErrCh <- err - } - // 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 } - // return dequeued items back to the pool - w.queue.ReleaseRequests(items) + 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() + } } + + // return dequeued items back to the pool + w.queue.ReleaseRequests(items) } + + return nil } func (w *worker) stopping(err error) error { @@ -252,7 +281,3 @@ func (w *worker) processBlock(blockQuerier *v1.BlockQuerier, tasks []Task) error w.metrics.bloomQueryLatency.WithLabelValues(w.id, "success").Observe(duration) return nil } - -func toModelTime(t time.Time) model.Time { - return model.TimeFromUnixNano(t.UnixNano()) -} From ecab33a93250b0ffed3b8684b72eeb9334982b7d Mon Sep 17 00:00:00 2001 From: Christian Haudum Date: Tue, 30 Jan 2024 11:51:53 +0100 Subject: [PATCH 04/45] Bloom Gateway: Extract processing logic from bloom worker into separate struct (#11812) **What this PR does / why we need it**: The processor executes a set of tasks. It cleanly separates I/O from logic and therefore is more testable than the current worker implementation. https://github.com/grafana/loki/blob/9602214abf5f0b016f1cad90e921d1e4d969856c/pkg/bloomgateway/processor.go#L33-L36 **Note**: The processor is not used yet in the worker. **Certain parts must be refactored when #11810 is merged.** --------- Signed-off-by: Christian Haudum --- pkg/bloomcompactor/bloomcompactor.go | 8 +- pkg/bloomgateway/bloomgateway_test.go | 125 ----------- pkg/bloomgateway/processor.go | 169 +++++++++++++++ pkg/bloomgateway/processor_test.go | 100 +++++++++ pkg/bloomgateway/util_test.go | 198 ++++++++++++++++++ .../stores/shipper/bloomshipper/client.go | 12 +- .../shipper/bloomshipper/client_test.go | 8 +- .../stores/shipper/bloomshipper/shipper.go | 63 +++--- .../shipper/bloomshipper/shipper_test.go | 28 ++- 9 files changed, 520 insertions(+), 191 deletions(-) create mode 100644 pkg/bloomgateway/processor.go create mode 100644 pkg/bloomgateway/processor_test.go diff --git a/pkg/bloomcompactor/bloomcompactor.go b/pkg/bloomcompactor/bloomcompactor.go index 2880b3e0ab719..5cb64a350d161 100644 --- a/pkg/bloomcompactor/bloomcompactor.go +++ b/pkg/bloomcompactor/bloomcompactor.go @@ -498,11 +498,9 @@ func (c *Compactor) runCompact(ctx context.Context, logger log.Logger, job Job, return err } metaSearchParams := bloomshipper.MetaSearchParams{ - TenantID: job.tenantID, - MinFingerprint: job.minFp, - MaxFingerprint: job.maxFp, - StartTimestamp: job.from, - EndTimestamp: job.through, + TenantID: job.tenantID, + Keyspace: bloomshipper.Keyspace{Min: job.minFp, Max: job.maxFp}, + Interval: bloomshipper.Interval{Start: job.from, End: job.through}, } var metas []bloomshipper.Meta //TODO Configure pool for these to avoid allocations diff --git a/pkg/bloomgateway/bloomgateway_test.go b/pkg/bloomgateway/bloomgateway_test.go index e9d537a4121db..c8da44a7c719b 100644 --- a/pkg/bloomgateway/bloomgateway_test.go +++ b/pkg/bloomgateway/bloomgateway_test.go @@ -3,7 +3,6 @@ package bloomgateway import ( "context" "fmt" - "math/rand" "os" "testing" "time" @@ -17,7 +16,6 @@ import ( "github.com/grafana/dskit/user" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" "github.com/stretchr/testify/require" @@ -27,29 +25,10 @@ import ( v1 "github.com/grafana/loki/pkg/storage/bloom/v1" "github.com/grafana/loki/pkg/storage/chunk/client/local" "github.com/grafana/loki/pkg/storage/config" - "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper" lokiring "github.com/grafana/loki/pkg/util/ring" "github.com/grafana/loki/pkg/validation" ) -func parseDayTime(s string) config.DayTime { - t, err := time.Parse("2006-01-02", s) - if err != nil { - panic(err) - } - return config.DayTime{ - Time: model.TimeFromUnix(t.Unix()), - } -} - -func mktime(s string) model.Time { - ts, err := time.Parse("2006-01-02 15:04", s) - if err != nil { - panic(err) - } - return model.TimeFromUnix(ts.Unix()) -} - func groupRefs(t *testing.T, chunkRefs []*logproto.ChunkRef) []*logproto.GroupedChunkRefs { t.Helper() grouped := make([]*logproto.GroupedChunkRefs, 0, len(chunkRefs)) @@ -488,107 +467,3 @@ func TestBloomGateway_RemoveNotMatchingChunks(t *testing.T) { }) } - -func createBlockQueriers(t *testing.T, numBlocks int, from, through model.Time, minFp, maxFp model.Fingerprint) ([]bloomshipper.BlockQuerierWithFingerprintRange, [][]v1.SeriesWithBloom) { - t.Helper() - step := (maxFp - minFp) / model.Fingerprint(numBlocks) - bqs := make([]bloomshipper.BlockQuerierWithFingerprintRange, 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.BlockQuerierWithFingerprintRange{ - BlockQuerier: blockQuerier, - MinFp: fromFp, - MaxFp: throughFp, - } - bqs = append(bqs, bq) - series = append(series, data) - } - return bqs, series -} - -func newMockBloomStore(bqs []bloomshipper.BlockQuerierWithFingerprintRange) *mockBloomStore { - return &mockBloomStore{bqs: bqs} -} - -type mockBloomStore struct { - bqs []bloomshipper.BlockQuerierWithFingerprintRange - // 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, tenant 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, bloomshipper.BlockRef{ - Ref: bloomshipper.Ref{ - MinFingerprint: uint64(s.bqs[i].MinFp), - MaxFingerprint: uint64(s.bqs[i].MaxFp), - TenantID: tenant, - }, - }) - } - return blocks, nil -} - -// Stop implements bloomshipper.Interface -func (s *mockBloomStore) Stop() {} - -// Fetch implements bloomshipper.Interface -func (s *mockBloomStore) Fetch(_ context.Context, _ string, _ []bloomshipper.BlockRef, callback bloomshipper.ForEachBlockCallback) error { - if s.err != nil { - time.Sleep(s.delay) - return s.err - } - - shuffled := make([]bloomshipper.BlockQuerierWithFingerprintRange, 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, uint64(bq.MinFp), uint64(bq.MaxFp)) - 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 - res := make([]*logproto.ChunkRef, 0) - for i := range data { - for j := range data[i] { - if n%nthSeries == 0 { - chk := data[i][j].Series.Chunks[0] - res = append(res, &logproto.ChunkRef{ - Fingerprint: uint64(data[i][j].Series.Fingerprint), - UserID: tenant, - From: chk.Start, - Through: chk.End, - Checksum: chk.Checksum, - }) - } - n++ - } - } - return res -} diff --git a/pkg/bloomgateway/processor.go b/pkg/bloomgateway/processor.go new file mode 100644 index 0000000000000..a2f63b3e5e155 --- /dev/null +++ b/pkg/bloomgateway/processor.go @@ -0,0 +1,169 @@ +package bloomgateway + +import ( + "context" + "math" + "sort" + + "github.com/go-kit/log" + "github.com/prometheus/common/model" + + v1 "github.com/grafana/loki/pkg/storage/bloom/v1" + "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper" +) + +type tasksForBlock struct { + blockRef bloomshipper.BlockRef + tasks []Task +} + +type metaLoader interface { + LoadMetas(context.Context, bloomshipper.MetaSearchParams) ([]bloomshipper.Meta, error) +} + +type blockLoader interface { + LoadBlocks(context.Context, []bloomshipper.BlockRef) (v1.Iterator[bloomshipper.BlockQuerierWithFingerprintRange], error) +} + +type store interface { + blockLoader + metaLoader +} + +type processor struct { + store store + logger log.Logger +} + +func (p *processor) run(ctx context.Context, tasks []Task) error { + for ts, tasks := range group(tasks, func(t Task) model.Time { return t.day }) { + interval := bloomshipper.Interval{ + Start: ts, + End: ts.Add(Day), + } + tenant := tasks[0].Tenant + err := p.processTasks(ctx, tenant, interval, []bloomshipper.Keyspace{{Min: 0, Max: math.MaxUint64}}, tasks) + if err != nil { + for _, task := range tasks { + task.CloseWithError(err) + } + return err + } + for _, task := range tasks { + task.Close() + } + } + return nil +} + +func (p *processor) processTasks(ctx context.Context, tenant string, interval bloomshipper.Interval, keyspaces []bloomshipper.Keyspace, tasks []Task) error { + minFpRange, maxFpRange := getFirstLast(keyspaces) + metaSearch := bloomshipper.MetaSearchParams{ + TenantID: tenant, + Interval: interval, + Keyspace: bloomshipper.Keyspace{Min: minFpRange.Min, Max: maxFpRange.Max}, + } + metas, err := p.store.LoadMetas(ctx, metaSearch) + if err != nil { + return err + } + blocksRefs := bloomshipper.BlocksForMetas(metas, interval, keyspaces) + return p.processBlocks(ctx, partition(tasks, blocksRefs)) +} + +func (p *processor) processBlocks(ctx context.Context, data []tasksForBlock) error { + refs := make([]bloomshipper.BlockRef, len(data)) + for _, block := range data { + refs = append(refs, block.blockRef) + } + + blockIter, err := p.store.LoadBlocks(ctx, refs) + if err != nil { + return err + } + +outer: + for blockIter.Next() { + bq := blockIter.At() + for i, block := range data { + if block.blockRef.MinFingerprint == uint64(bq.MinFp) && block.blockRef.MaxFingerprint == uint64(bq.MaxFp) { + err := p.processBlock(ctx, bq.BlockQuerier, block.tasks) + if err != nil { + return err + } + data = append(data[:i], data[i+1:]...) + continue outer + } + } + } + return nil +} + +func (p *processor) processBlock(_ context.Context, 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) + return fq.Run() +} + +// 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 group[K comparable, V any, S ~[]V](s S, f func(v V) K) map[K]S { + m := make(map[K]S) + for _, elem := range s { + m[f(elem)] = append(m[f(elem)], elem) + } + return m +} + +func partition(tasks []Task, blocks []bloomshipper.BlockRef) []tasksForBlock { + result := make([]tasksForBlock, 0, len(blocks)) + + for _, block := range blocks { + bounded := tasksForBlock{ + blockRef: block, + } + + for _, task := range tasks { + refs := task.series + min := sort.Search(len(refs), func(i int) bool { + return block.Cmp(refs[i].Fingerprint) > v1.Before + }) + + max := sort.Search(len(refs), func(i int) bool { + return block.Cmp(refs[i].Fingerprint) == v1.After + }) + + // All fingerprints fall outside of the consumer's range + if min == len(refs) || max == 0 { + continue + } + + bounded.tasks = append(bounded.tasks, task.Copy(refs[min:max])) + } + + if len(bounded.tasks) > 0 { + result = append(result, bounded) + } + + } + return result +} diff --git a/pkg/bloomgateway/processor_test.go b/pkg/bloomgateway/processor_test.go new file mode 100644 index 0000000000000..f078c31c06d32 --- /dev/null +++ b/pkg/bloomgateway/processor_test.go @@ -0,0 +1,100 @@ +package bloomgateway + +import ( + "context" + "math/rand" + "sync" + "testing" + "time" + + "github.com/prometheus/common/model" + "github.com/stretchr/testify/require" + "go.uber.org/atomic" + + "github.com/grafana/loki/pkg/logql/syntax" + v1 "github.com/grafana/loki/pkg/storage/bloom/v1" + "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper" +) + +var _ store = &dummyStore{} + +type dummyStore struct { + metas []bloomshipper.Meta + blocks []bloomshipper.BlockRef + querieres []bloomshipper.BlockQuerierWithFingerprintRange +} + +func (s *dummyStore) LoadMetas(_ context.Context, _ bloomshipper.MetaSearchParams) ([]bloomshipper.Meta, error) { + //TODO(chaudum) Filter metas based on search params + return s.metas, nil +} + +func (s *dummyStore) LoadBlocks(_ context.Context, refs []bloomshipper.BlockRef) (v1.Iterator[bloomshipper.BlockQuerierWithFingerprintRange], error) { + result := make([]bloomshipper.BlockQuerierWithFingerprintRange, len(s.querieres)) + + for _, ref := range refs { + for _, bq := range s.querieres { + if ref.MinFingerprint == uint64(bq.MinFp) && ref.MaxFingerprint == uint64(bq.MaxFp) { + result = append(result, bq) + } + } + } + + rand.Shuffle(len(result), func(i, j int) { + result[i], result[j] = result[j], result[i] + }) + + return v1.NewSliceIter(result), nil +} + +func TestProcessor(t *testing.T) { + ctx := context.Background() + 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) + p := &processor{ + store: &dummyStore{ + querieres: queriers, + metas: metas, + blocks: blocks, + }, + } + + 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.NoError(t, err) + require.Equal(t, int64(len(swb.series)), results.Load()) + }) +} diff --git a/pkg/bloomgateway/util_test.go b/pkg/bloomgateway/util_test.go index 81f0720b9bf80..61825a8c677ae 100644 --- a/pkg/bloomgateway/util_test.go +++ b/pkg/bloomgateway/util_test.go @@ -1,6 +1,9 @@ package bloomgateway import ( + "context" + "fmt" + "math/rand" "testing" "time" @@ -8,9 +11,29 @@ import ( "github.com/stretchr/testify/require" "github.com/grafana/loki/pkg/logproto" + 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" ) +func parseDayTime(s string) config.DayTime { + t, err := time.Parse("2006-01-02", s) + if err != nil { + panic(err) + } + return config.DayTime{ + Time: model.TimeFromUnix(t.Unix()), + } +} + +func mktime(s string) model.Time { + ts, err := time.Parse("2006-01-02 15:04", s) + if err != nil { + panic(err) + } + return model.TimeFromUnix(ts.Unix()) +} + func TestGetFromThrough(t *testing.T) { chunks := []*logproto.ShortRef{ {From: 0, Through: 6}, @@ -273,3 +296,178 @@ func TestPartitionRequest(t *testing.T) { } } + +func createBlockQueriers(t *testing.T, numBlocks int, from, through model.Time, minFp, maxFp model.Fingerprint) ([]bloomshipper.BlockQuerierWithFingerprintRange, [][]v1.SeriesWithBloom) { + t.Helper() + step := (maxFp - minFp) / model.Fingerprint(numBlocks) + bqs := make([]bloomshipper.BlockQuerierWithFingerprintRange, 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.BlockQuerierWithFingerprintRange{ + BlockQuerier: blockQuerier, + MinFp: fromFp, + MaxFp: throughFp, + } + 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.BlockQuerierWithFingerprintRange, [][]v1.SeriesWithBloom) { + t.Helper() + + blocks := make([]bloomshipper.BlockRef, 0, n) + metas := make([]bloomshipper.Meta, 0, n) + queriers := make([]bloomshipper.BlockQuerierWithFingerprintRange, 0, n) + series := make([][]v1.SeriesWithBloom, 0, n) + + step := (maxFp - minFp) / model.Fingerprint(n) + for i := 0; i < n; i++ { + fromFp := minFp + (step * model.Fingerprint(i)) + throughFp := fromFp + step - 1 + // last block needs to include maxFp + if i == n-1 { + throughFp = maxFp + } + ref := bloomshipper.Ref{ + TenantID: tenant, + TableName: "table_0", + MinFingerprint: uint64(fromFp), + MaxFingerprint: uint64(throughFp), + StartTimestamp: from, + EndTimestamp: through, + } + block := bloomshipper.BlockRef{ + Ref: ref, + IndexPath: "index.tsdb.gz", + BlockPath: fmt.Sprintf("block-%d", i), + } + meta := bloomshipper.Meta{ + MetaRef: bloomshipper.MetaRef{ + Ref: ref, + }, + Tombstones: []bloomshipper.BlockRef{}, + Blocks: []bloomshipper.BlockRef{block}, + } + blockQuerier, data := v1.MakeBlockQuerier(t, fromFp, throughFp, from, through) + querier := bloomshipper.BlockQuerierWithFingerprintRange{ + BlockQuerier: blockQuerier, + MinFp: fromFp, + MaxFp: throughFp, + } + queriers = append(queriers, querier) + metas = append(metas, meta) + blocks = append(blocks, block) + series = append(series, data) + } + return blocks, metas, queriers, series +} + +func newMockBloomStore(bqs []bloomshipper.BlockQuerierWithFingerprintRange) *mockBloomStore { + return &mockBloomStore{bqs: bqs} +} + +type mockBloomStore struct { + bqs []bloomshipper.BlockQuerierWithFingerprintRange + // 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, tenant 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, bloomshipper.BlockRef{ + Ref: bloomshipper.Ref{ + MinFingerprint: uint64(s.bqs[i].MinFp), + MaxFingerprint: uint64(s.bqs[i].MaxFp), + TenantID: tenant, + }, + }) + } + return blocks, nil +} + +// Stop implements bloomshipper.Interface +func (s *mockBloomStore) Stop() {} + +// Fetch implements bloomshipper.Interface +func (s *mockBloomStore) Fetch(_ context.Context, _ string, _ []bloomshipper.BlockRef, callback bloomshipper.ForEachBlockCallback) error { + if s.err != nil { + time.Sleep(s.delay) + return s.err + } + + shuffled := make([]bloomshipper.BlockQuerierWithFingerprintRange, 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, uint64(bq.MinFp), uint64(bq.MaxFp)) + 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 + res := make([]*logproto.ChunkRef, 0) + for i := range data { + for j := range data[i] { + if n%nthSeries == 0 { + chk := data[i][j].Series.Chunks[0] + res = append(res, &logproto.ChunkRef{ + Fingerprint: uint64(data[i][j].Series.Fingerprint), + UserID: tenant, + From: chk.Start, + Through: chk.End, + Checksum: chk.Checksum, + }) + } + n++ + } + } + return res +} + +func createBlockRefsFromBlockData(t *testing.T, tenant string, data []bloomshipper.BlockQuerierWithFingerprintRange) []bloomshipper.BlockRef { + t.Helper() + res := make([]bloomshipper.BlockRef, 0) + for i := range data { + res = append(res, bloomshipper.BlockRef{ + Ref: bloomshipper.Ref{ + TenantID: tenant, + TableName: "", + MinFingerprint: uint64(data[i].MinFp), + MaxFingerprint: uint64(data[i].MaxFp), + StartTimestamp: 0, + EndTimestamp: 0, + Checksum: 0, + }, + IndexPath: fmt.Sprintf("index-%d", i), + BlockPath: fmt.Sprintf("block-%d", i), + }) + } + return res +} diff --git a/pkg/storage/stores/shipper/bloomshipper/client.go b/pkg/storage/stores/shipper/bloomshipper/client.go index b189cba390b82..5c77f6fd9f636 100644 --- a/pkg/storage/stores/shipper/bloomshipper/client.go +++ b/pkg/storage/stores/shipper/bloomshipper/client.go @@ -67,9 +67,9 @@ type Meta struct { } type MetaSearchParams struct { - TenantID string - MinFingerprint, MaxFingerprint model.Fingerprint - StartTimestamp, EndTimestamp model.Time + TenantID string + Interval Interval + Keyspace Keyspace } type MetaClient interface { @@ -126,7 +126,7 @@ type BloomClient struct { } func (b *BloomClient) GetMetas(ctx context.Context, params MetaSearchParams) ([]Meta, error) { - tablesByPeriod := tablesByPeriod(b.periodicConfigs, params.StartTimestamp, params.EndTimestamp) + tablesByPeriod := tablesByPeriod(b.periodicConfigs, params.Interval.Start, params.Interval.End) var metas []Meta for periodFrom, tables := range tablesByPeriod { @@ -143,8 +143,8 @@ func (b *BloomClient) GetMetas(ctx context.Context, params MetaSearchParams) ([] if err != nil { return nil, err } - if metaRef.MaxFingerprint < uint64(params.MinFingerprint) || uint64(params.MaxFingerprint) < metaRef.MinFingerprint || - metaRef.EndTimestamp.Before(params.StartTimestamp) || metaRef.StartTimestamp.After(params.EndTimestamp) { + if metaRef.MaxFingerprint < uint64(params.Keyspace.Min) || uint64(params.Keyspace.Max) < metaRef.MinFingerprint || + metaRef.EndTimestamp.Before(params.Interval.Start) || metaRef.StartTimestamp.After(params.Interval.End) { continue } meta, err := b.downloadMeta(ctx, metaRef, periodClient) diff --git a/pkg/storage/stores/shipper/bloomshipper/client_test.go b/pkg/storage/stores/shipper/bloomshipper/client_test.go index d6043febb48c9..1cd5f562362ce 100644 --- a/pkg/storage/stores/shipper/bloomshipper/client_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/client_test.go @@ -69,11 +69,9 @@ func Test_BloomClient_GetMetas(t *testing.T) { createMetaInStorage(t, folder2, "second-period-19624", "tenantB", 0, 100, fixedDay.Add(-3*day)) actual, err := shipper.GetMetas(context.Background(), MetaSearchParams{ - TenantID: "tenantA", - MinFingerprint: 50, - MaxFingerprint: 150, - StartTimestamp: fixedDay.Add(-6 * day), - EndTimestamp: fixedDay.Add(-1*day - 1*time.Hour), + TenantID: "tenantA", + Keyspace: Keyspace{Min: 50, Max: 150}, + Interval: Interval{Start: fixedDay.Add(-6 * day), End: fixedDay.Add(-1*day - 1*time.Hour)}, }) require.NoError(t, err) require.ElementsMatch(t, expected, actual) diff --git a/pkg/storage/stores/shipper/bloomshipper/shipper.go b/pkg/storage/stores/shipper/bloomshipper/shipper.go index abcdcf370f813..5f5bd6f56d272 100644 --- a/pkg/storage/stores/shipper/bloomshipper/shipper.go +++ b/pkg/storage/stores/shipper/bloomshipper/shipper.go @@ -16,8 +16,7 @@ import ( ) type Interval struct { - Start model.Time - End model.Time + Start, End model.Time } func (i Interval) String() string { @@ -33,20 +32,14 @@ func (i Interval) Cmp(other model.Time) v1.BoundsCheck { return v1.Overlap } -type fpRange [2]uint64 - -func (r fpRange) minFp() uint64 { - return r[0] -} - -func (r fpRange) maxFp() uint64 { - return r[1] +type Keyspace struct { + Min, Max model.Fingerprint } -func (r fpRange) Cmp(other uint64) v1.BoundsCheck { - if other < r[0] { +func (r Keyspace) Cmp(other model.Fingerprint) v1.BoundsCheck { + if other < r.Min { return v1.Before - } else if other > r[1] { + } else if other > r.Max { return v1.After } return v1.Overlap @@ -94,7 +87,7 @@ func (s *Shipper) GetBlockRefs(ctx context.Context, tenantID string, interval In 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 - keyspaces := []fpRange{{0, math.MaxUint64}} + keyspaces := []Keyspace{{0, math.MaxUint64}} blockRefs, err := s.getActiveBlockRefs(ctx, tenantID, interval, keyspaces) if err != nil { return nil, fmt.Errorf("error fetching active block references : %w", err) @@ -159,34 +152,22 @@ func getFirstLast[T any](s []T) (T, T) { return s[0], s[len(s)-1] } -func (s *Shipper) getActiveBlockRefs(ctx context.Context, tenantID string, interval Interval, keyspaces []fpRange) ([]BlockRef, error) { +func (s *Shipper) getActiveBlockRefs(ctx context.Context, tenantID string, interval Interval, keyspaces []Keyspace) ([]BlockRef, error) { minFpRange, maxFpRange := getFirstLast(keyspaces) metas, err := s.client.GetMetas(ctx, MetaSearchParams{ - TenantID: tenantID, - MinFingerprint: model.Fingerprint(minFpRange.minFp()), - MaxFingerprint: model.Fingerprint(maxFpRange.maxFp()), - StartTimestamp: interval.Start, - EndTimestamp: interval.End, + TenantID: tenantID, + Keyspace: Keyspace{Min: minFpRange.Min, Max: 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)) - activeBlocks := s.findBlocks(metas, interval, keyspaces) - slices.SortStableFunc(activeBlocks, func(a, b BlockRef) int { - if a.MinFingerprint < b.MinFingerprint { - return -1 - } - if a.MinFingerprint > b.MinFingerprint { - return 1 - } - return 0 - }) - return activeBlocks, nil + return BlocksForMetas(metas, interval, keyspaces), nil } -func (s *Shipper) findBlocks(metas []Meta, interval Interval, keyspaces []fpRange) []BlockRef { +func BlocksForMetas(metas []Meta, interval Interval, keyspaces []Keyspace) []BlockRef { tombstones := make(map[string]interface{}) for _, meta := range metas { for _, tombstone := range meta.Tombstones { @@ -211,13 +192,25 @@ func (s *Shipper) findBlocks(metas []Meta, interval Interval, keyspaces []fpRang for _, ref := range blocksSet { blockRefs = append(blockRefs, ref) } + + slices.SortStableFunc(blockRefs, func(a, b BlockRef) int { + if a.MinFingerprint < b.MinFingerprint { + return -1 + } + if a.MinFingerprint > b.MinFingerprint { + return 1 + } + + return 0 + }) + return blockRefs } // isOutsideRange tests if a given BlockRef b is outside of search boundaries // defined by min/max timestamp and min/max fingerprint. // Fingerprint ranges must be sorted in ascending order. -func isOutsideRange(b BlockRef, interval Interval, keyspaces []fpRange) bool { +func isOutsideRange(b BlockRef, interval Interval, keyspaces []Keyspace) bool { // check time interval if interval.Cmp(b.EndTimestamp) == v1.Before || interval.Cmp(b.StartTimestamp) == v1.After { return true @@ -225,10 +218,10 @@ func isOutsideRange(b BlockRef, interval Interval, keyspaces []fpRange) bool { // check fingerprint ranges for _, keyspace := range keyspaces { - if keyspace.Cmp(b.MinFingerprint) == v1.Before && keyspace.Cmp(b.MaxFingerprint) == v1.After { + if keyspace.Cmp(model.Fingerprint(b.MinFingerprint)) == v1.Before && keyspace.Cmp(model.Fingerprint(b.MaxFingerprint)) == v1.After { return false } - if keyspace.Cmp(b.MinFingerprint) == v1.Overlap || keyspace.Cmp(b.MaxFingerprint) == v1.Overlap { + if keyspace.Cmp(model.Fingerprint(b.MinFingerprint)) == v1.Overlap || keyspace.Cmp(model.Fingerprint(b.MaxFingerprint)) == v1.Overlap { return false } } diff --git a/pkg/storage/stores/shipper/bloomshipper/shipper_test.go b/pkg/storage/stores/shipper/bloomshipper/shipper_test.go index 755813c3a8479..57d360de3b80d 100644 --- a/pkg/storage/stores/shipper/bloomshipper/shipper_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/shipper_test.go @@ -44,12 +44,11 @@ func Test_Shipper_findBlocks(t *testing.T) { ts := model.Now() - shipper := &Shipper{} interval := Interval{ Start: ts.Add(-2 * time.Hour), End: ts.Add(-1 * time.Hour), } - blocks := shipper.findBlocks(metas, interval, []fpRange{{100, 200}}) + blocks := BlocksForMetas(metas, interval, []Keyspace{{Min: 100, Max: 200}}) expectedBlockRefs := []BlockRef{ createMatchingBlockRef("block2"), @@ -101,9 +100,8 @@ func Test_Shipper_findBlocks(t *testing.T) { } for name, data := range tests { t.Run(name, func(t *testing.T) { - shipper := &Shipper{} ref := createBlockRef("fake-block", data.minFingerprint, data.maxFingerprint, data.startTimestamp, data.endTimestamp) - blocks := shipper.findBlocks([]Meta{{Blocks: []BlockRef{ref}}}, interval(300, 400), []fpRange{{100, 200}}) + blocks := BlocksForMetas([]Meta{{Blocks: []BlockRef{ref}}}, interval(300, 400), []Keyspace{{Min: 100, Max: 200}}) if data.filtered { require.Empty(t, blocks) return @@ -120,67 +118,67 @@ func TestIsOutsideRange(t *testing.T) { t.Run("is outside if startTs > through", func(t *testing.T) { b := createBlockRef("block", 0, math.MaxUint64, startTs, endTs) - isOutside := isOutsideRange(b, interval(0, 900), []fpRange{}) + isOutside := isOutsideRange(b, interval(0, 900), []Keyspace{}) require.True(t, isOutside) }) t.Run("is outside if startTs == through ", func(t *testing.T) { b := createBlockRef("block", 0, math.MaxUint64, startTs, endTs) - isOutside := isOutsideRange(b, interval(900, 1000), []fpRange{}) + isOutside := isOutsideRange(b, interval(900, 1000), []Keyspace{}) require.True(t, isOutside) }) t.Run("is outside if endTs < from", func(t *testing.T) { b := createBlockRef("block", 0, math.MaxUint64, startTs, endTs) - isOutside := isOutsideRange(b, interval(2100, 3000), []fpRange{}) + isOutside := isOutsideRange(b, interval(2100, 3000), []Keyspace{}) require.True(t, isOutside) }) t.Run("is outside if endFp < first fingerprint", func(t *testing.T) { b := createBlockRef("block", 0, 90, startTs, endTs) - isOutside := isOutsideRange(b, interval(startTs, endTs), []fpRange{{100, 199}}) + isOutside := isOutsideRange(b, interval(startTs, endTs), []Keyspace{{100, 199}}) require.True(t, isOutside) }) t.Run("is outside if startFp > last fingerprint", func(t *testing.T) { b := createBlockRef("block", 200, math.MaxUint64, startTs, endTs) - isOutside := isOutsideRange(b, interval(startTs, endTs), []fpRange{{0, 49}, {100, 149}}) + isOutside := isOutsideRange(b, interval(startTs, endTs), []Keyspace{{0, 49}, {100, 149}}) require.True(t, isOutside) }) t.Run("is outside if within gaps in fingerprints", func(t *testing.T) { b := createBlockRef("block", 100, 199, startTs, endTs) - isOutside := isOutsideRange(b, interval(startTs, endTs), []fpRange{{0, 99}, {200, 299}}) + isOutside := isOutsideRange(b, interval(startTs, endTs), []Keyspace{{0, 99}, {200, 299}}) require.True(t, isOutside) }) t.Run("is not outside if within fingerprints 1", func(t *testing.T) { b := createBlockRef("block", 10, 90, startTs, endTs) - isOutside := isOutsideRange(b, interval(startTs, endTs), []fpRange{{0, 99}, {200, 299}}) + isOutside := isOutsideRange(b, interval(startTs, endTs), []Keyspace{{0, 99}, {200, 299}}) require.False(t, isOutside) }) t.Run("is not outside if within fingerprints 2", func(t *testing.T) { b := createBlockRef("block", 210, 290, startTs, endTs) - isOutside := isOutsideRange(b, interval(startTs, endTs), []fpRange{{0, 99}, {200, 299}}) + isOutside := isOutsideRange(b, interval(startTs, endTs), []Keyspace{{0, 99}, {200, 299}}) require.False(t, isOutside) }) t.Run("is not outside if spans across multiple fingerprint ranges", func(t *testing.T) { b := createBlockRef("block", 50, 250, startTs, endTs) - isOutside := isOutsideRange(b, interval(startTs, endTs), []fpRange{{0, 99}, {200, 299}}) + isOutside := isOutsideRange(b, interval(startTs, endTs), []Keyspace{{0, 99}, {200, 299}}) require.False(t, isOutside) }) t.Run("is not outside if fingerprint range and time range are larger than block", func(t *testing.T) { b := createBlockRef("block", math.MaxUint64/3, math.MaxUint64/3*2, startTs, endTs) - isOutside := isOutsideRange(b, interval(0, 3000), []fpRange{{0, math.MaxUint64}}) + isOutside := isOutsideRange(b, interval(0, 3000), []Keyspace{{0, math.MaxUint64}}) require.False(t, isOutside) }) t.Run("is not outside if block fingerprint range is bigger that search keyspace", func(t *testing.T) { b := createBlockRef("block", 0x0000, 0xffff, model.Earliest, model.Latest) - isOutside := isOutsideRange(b, interval(startTs, endTs), []fpRange{{0x0100, 0xff00}}) + isOutside := isOutsideRange(b, interval(startTs, endTs), []Keyspace{{0x0100, 0xff00}}) require.False(t, isOutside) }) } From 821c0b1ae05f0702721309ecb0da80c5c53afb32 Mon Sep 17 00:00:00 2001 From: Vladyslav Diachenko <82767850+vlad-diachenko@users.noreply.github.com> Date: Tue, 30 Jan 2024 14:16:47 +0200 Subject: [PATCH 05/45] DequeueMany fix (#11797) **What this PR does / why we need it**: Previously we relied only on the index of the tenant's queue to read the requests from the same tenant's queue. However, as long as the queue is aggressively used by the consumers in parallel, there are a few edge cases when knowing the index of last used tenant's queue is not enough to guarantee that we dequeue items from exactly the same tenant's queue. **Special notes for your reviewer**: **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) --------- Signed-off-by: Vladyslav Diachenko --- pkg/bloomgateway/bloomgateway.go | 5 +- pkg/bloomgateway/worker.go | 5 +- pkg/queue/queue.go | 110 ++++++++++------ pkg/queue/queue_test.go | 215 +++++++++++++++++++++++++++++++ 4 files changed, 287 insertions(+), 48 deletions(-) diff --git a/pkg/bloomgateway/bloomgateway.go b/pkg/bloomgateway/bloomgateway.go index f236a125d75e0..3530ea0eaf993 100644 --- a/pkg/bloomgateway/bloomgateway.go +++ b/pkg/bloomgateway/bloomgateway.go @@ -200,14 +200,13 @@ func New(cfg Config, schemaCfg config.SchemaConfig, storageCfg storage.Config, o sharding: shardingStrategy, pendingTasks: makePendingTasks(pendingTasksInitialCap), workerConfig: workerConfig{ - maxWaitTime: 200 * time.Millisecond, - maxItems: 100, + maxItems: 100, }, workerMetrics: newWorkerMetrics(reg, constants.Loki, metricsSubsystem), queueMetrics: queue.NewMetrics(reg, constants.Loki, metricsSubsystem), } - g.queue = queue.NewRequestQueue(cfg.MaxOutstandingPerTenant, time.Minute, &fixedQueueLimits{100}, g.queueMetrics) + g.queue = queue.NewRequestQueue(cfg.MaxOutstandingPerTenant, time.Minute, &fixedQueueLimits{0}, g.queueMetrics) g.activeUsers = util.NewActiveUsersCleanupWithDefaultValues(g.queueMetrics.Cleanup) client, err := bloomshipper.NewBloomClient(schemaCfg.Configs, storageCfg, cm) diff --git a/pkg/bloomgateway/worker.go b/pkg/bloomgateway/worker.go index 6e08b574d683e..3b16fe4fdd7cf 100644 --- a/pkg/bloomgateway/worker.go +++ b/pkg/bloomgateway/worker.go @@ -19,8 +19,7 @@ import ( ) type workerConfig struct { - maxWaitTime time.Duration - maxItems int + maxItems int } type workerMetrics struct { @@ -111,7 +110,7 @@ func (w *worker) running(_ context.Context) error { for st := w.State(); st == services.Running || st == services.Stopping; { taskCtx := context.Background() dequeueStart := time.Now() - items, newIdx, err := w.queue.DequeueMany(taskCtx, idx, w.id, w.cfg.maxItems, w.cfg.maxWaitTime) + items, newIdx, err := w.queue.DequeueMany(taskCtx, idx, w.id, w.cfg.maxItems) w.metrics.dequeueWaitTime.WithLabelValues(w.id).Observe(time.Since(dequeueStart).Seconds()) if err != nil { // We only return an error if the queue is stopped and dequeuing did not yield any items diff --git a/pkg/queue/queue.go b/pkg/queue/queue.go index 006106aa44a61..74c6549d2ee97 100644 --- a/pkg/queue/queue.go +++ b/pkg/queue/queue.go @@ -14,11 +14,13 @@ import ( const ( // How frequently to check for disconnected queriers that should be forgotten. forgetCheckPeriod = 5 * time.Second + anyQueue = "" ) var ( ErrTooManyRequests = errors.New("too many outstanding requests") ErrStopped = errors.New("queue is stopped") + ErrQueueWasRemoved = errors.New("the queue has been removed or moved to another position") ) // QueueIndex is opaque type that allows to resume iteration over tenants between successive calls @@ -135,38 +137,43 @@ func (q *RequestQueue) ReleaseRequests(items []Request) { } // DequeueMany consumes multiple items for a single tenant from the queue. -// It returns maxItems and waits maxWait if no requests for this tenant are enqueued. +// It blocks the execution until it dequeues at least 1 request and continue reading +// until it reaches `maxItems` requests or if no requests for this tenant are enqueued. // The caller is responsible for returning the dequeued requests back to the // pool by calling ReleaseRequests(items). -func (q *RequestQueue) DequeueMany(ctx context.Context, last QueueIndex, consumerID string, maxItems int, maxWait time.Duration) ([]Request, QueueIndex, error) { - // create a context for dequeuing with a max time we want to wait to fulfill the desired maxItems - - dequeueCtx, cancel := context.WithTimeout(ctx, maxWait) - defer cancel() - - var idx QueueIndex - +func (q *RequestQueue) DequeueMany(ctx context.Context, idx QueueIndex, consumerID string, maxItems int) ([]Request, QueueIndex, error) { items := q.pool.Get(maxItems) + lastQueueName := anyQueue for { - item, newIdx, err := q.Dequeue(dequeueCtx, last, consumerID) + item, newIdx, newQueueName, isTenantQueueEmpty, err := q.dequeue(ctx, idx, lastQueueName, consumerID) if err != nil { - if err == context.DeadlineExceeded { + // the consumer must receive the items if tenants queue is removed, + // even if it has collected less than `maxItems` requests. + if errors.Is(err, ErrQueueWasRemoved) { err = nil } - return items, idx, err + return items, newIdx, err } + lastQueueName = newQueueName items = append(items, item) - idx = newIdx - if len(items) == maxItems { - return items, idx, nil + idx = newIdx.ReuseLastIndex() + if len(items) == maxItems || isTenantQueueEmpty { + return items, newIdx, nil } } } // Dequeue find next tenant queue and takes the next request off of it. Will block if there are no requests. // By passing tenant index from previous call of this method, querier guarantees that it iterates over all tenants fairly. -// If consumer finds that request from the tenant is already expired, it can get a request for the same tenant by using UserIndex.ReuseLastUser. +// Even if the consumer used UserIndex.ReuseLastUser to fetch the request from the same tenant's queue, it does not provide +// any guaranties that the previously used queue is still at this position because another consumer could already read +// the last request and the queue could be removed and another queue is already placed at this position. func (q *RequestQueue) Dequeue(ctx context.Context, last QueueIndex, consumerID string) (Request, QueueIndex, error) { + dequeue, queueIndex, _, _, err := q.dequeue(ctx, last, anyQueue, consumerID) + return dequeue, queueIndex, err +} + +func (q *RequestQueue) dequeue(ctx context.Context, last QueueIndex, wantedQueueName string, consumerID string) (Request, QueueIndex, string, bool, error) { q.mtx.Lock() defer q.mtx.Unlock() @@ -174,47 +181,66 @@ func (q *RequestQueue) Dequeue(ctx context.Context, last QueueIndex, consumerID FindQueue: // We need to wait if there are no tenants, or no pending requests for given querier. - for (q.queues.hasNoTenantQueues() || querierWait) && ctx.Err() == nil && !q.stopped { + // However, if `wantedQueueName` is not empty, the caller must not be blocked because it wants to read exactly from that queue, not others. + for (q.queues.hasNoTenantQueues() || querierWait) && ctx.Err() == nil && !q.stopped && wantedQueueName == anyQueue { querierWait = false q.cond.Wait(ctx) } + // If the current consumer wants to read from specific queue, but he does not have any queues available for him, + // return an error to notify that queue has been already removed. + if q.queues.hasNoTenantQueues() && wantedQueueName != anyQueue { + return nil, last, wantedQueueName, false, ErrQueueWasRemoved + } + if q.stopped { - return nil, last, ErrStopped + return nil, last, wantedQueueName, false, ErrStopped } if err := ctx.Err(); err != nil { - return nil, last, err + return nil, last, wantedQueueName, false, err } - for { - queue, tenant, idx := q.queues.getNextQueueForConsumer(last, consumerID) - last = idx - if queue == nil { - break + queue, tenant, idx := q.queues.getNextQueueForConsumer(last, consumerID) + last = idx + if queue == nil { + // it can be a case the consumer has other tenants queues available for him, + // it allows the consumer to pass the wait block, + // but the queue with index `last+1` has been already removed, + // for example if another consumer has read the last request from that queue, + // and as long as this consumer wants to read from specific tenant queue, + // it's necessary to return `ErrQueueWasRemoved` error. + if wantedQueueName != anyQueue { + return nil, last, wantedQueueName, false, ErrQueueWasRemoved } + // otherwise, if wantedQueueName is empty, then this consumer will go to the wait block again + // and as long as `last` index is updated, next time the consumer will request the queue + // with the new index that was returned from `getNextQueueForConsumer`. + // There are no unexpired requests, so we can get back + // and wait for more requests. + querierWait = true + goto FindQueue + } - // Pick next request from the queue. - for { - request := queue.Dequeue() - if queue.Len() == 0 { - q.queues.deleteQueue(tenant) - } - - q.queues.perUserQueueLen.Dec(tenant) - q.metrics.queueLength.WithLabelValues(tenant).Dec() + if wantedQueueName != anyQueue && wantedQueueName != queue.Name() { + // it means that the consumer received another tenants queue because it was already removed + // or another queue is already at this index + return nil, last, queue.Name(), false, ErrQueueWasRemoved + } + // Pick next request from the queue. + request := queue.Dequeue() + isTenantQueueEmpty := queue.Len() == 0 + if isTenantQueueEmpty { + q.queues.deleteQueue(tenant) + } - // Tell close() we've processed a request. - q.cond.Broadcast() + q.queues.perUserQueueLen.Dec(tenant) + q.metrics.queueLength.WithLabelValues(tenant).Dec() - return request, last, nil - } - } + // Tell close() we've processed a request. + q.cond.Broadcast() - // There are no unexpired requests, so we can get back - // and wait for more requests. - querierWait = true - goto FindQueue + return request, last, queue.Name(), isTenantQueueEmpty, nil } func (q *RequestQueue) forgetDisconnectedConsumers(_ context.Context) error { diff --git a/pkg/queue/queue_test.go b/pkg/queue/queue_test.go index 623e240733886..a21ce8af622cf 100644 --- a/pkg/queue/queue_test.go +++ b/pkg/queue/queue_test.go @@ -2,6 +2,7 @@ package queue import ( "context" + "errors" "fmt" "strconv" "sync" @@ -11,6 +12,8 @@ import ( "github.com/grafana/dskit/services" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + "go.uber.org/atomic" + "golang.org/x/sync/errgroup" "github.com/grafana/loki/pkg/util/constants" ) @@ -333,6 +336,218 @@ func TestMaxQueueSize(t *testing.T) { }) } +type mockLimits struct { + maxConsumer int +} + +func (l *mockLimits) MaxConsumers(_ string, _ int) int { + return l.maxConsumer +} + +func Test_Queue_DequeueMany(t *testing.T) { + tenantsQueueMaxSize := 100 + tests := map[string]struct { + tenantsCount int + tasksPerTenant int + consumersCount int + dequeueBatchSize int + maxConsumersPerTenant int + consumerDelay time.Duration + senderDelay time.Duration + }{ + "tenants-1_tasks-10_consumers-3_batch-2": { + tenantsCount: 1, + tasksPerTenant: 10, + consumersCount: 3, + dequeueBatchSize: 2, + }, + "tenants-10_tasks-10_consumers-2_batch-10": { + tenantsCount: 10, + tasksPerTenant: 10, + consumersCount: 2, + dequeueBatchSize: 10, + }, + "tenants-100_tasks-100_consumers-10_batch-10": { + tenantsCount: 100, + tasksPerTenant: 100, + consumersCount: 10, + dequeueBatchSize: 10, + }, + "tenants-100_tasks-100_consumers-10_batch-10_consumerDelay-10": { + tenantsCount: 100, + tasksPerTenant: 100, + consumersCount: 10, + dequeueBatchSize: 10, + consumerDelay: 10 * time.Millisecond, + }, + "tenants-100_tasks-100_consumers-10_batch-10_consumerDelay-10_senderDelay-10": { + tenantsCount: 100, + tasksPerTenant: 100, + consumersCount: 10, + dequeueBatchSize: 10, + consumerDelay: 10 * time.Millisecond, + senderDelay: 10 * time.Millisecond, + }, + "tenants-10_tasks-50_consumers-10_batch-3": { + tenantsCount: 10, + tasksPerTenant: 50, + consumersCount: 10, + dequeueBatchSize: 3, + }, + "tenants-10_tasks-50_consumers-1_batch-5": { + tenantsCount: 10, + tasksPerTenant: 50, + consumersCount: 1, + dequeueBatchSize: 5, + }, + "tenants-5_tasks-10_consumers-1_batch-2": { + tenantsCount: 5, + tasksPerTenant: 10, + consumersCount: 1, + dequeueBatchSize: 2, + }, + "tenants-1_tasks-10_consumers-1_batch-2": { + tenantsCount: 1, + tasksPerTenant: 10, + consumersCount: 1, + dequeueBatchSize: 2, + }, + "tenants-1_tasks-10_consumers-10_batch-2": { + tenantsCount: 1, + tasksPerTenant: 10, + consumersCount: 10, + dequeueBatchSize: 2, + }, + "tenants-100_tasks-10_consumers-10_batch-6_maxConsumersPerTenant-6": { + tenantsCount: 100, + tasksPerTenant: 10, + consumersCount: 10, + dequeueBatchSize: 6, + maxConsumersPerTenant: 4, + }, + "tenants-200_tasks-10_consumers-100_batch-999999_maxConsumersPerTenant-4": { + tenantsCount: 200, + tasksPerTenant: 10, + consumersCount: 100, + dequeueBatchSize: 999_999, + maxConsumersPerTenant: 4, + }, + "tenants-10_tasks-100_consumers-20_batch-5_maxConsumersPerTenant-16": { + tenantsCount: 10, + tasksPerTenant: 100, + consumersCount: 20, + dequeueBatchSize: 5, + maxConsumersPerTenant: 16, + }, + "tenants-1_tasks-100_consumers-16_batch-5_maxConsumersPerTenant-2": { + tenantsCount: 1, + tasksPerTenant: 100, + consumersCount: 16, + dequeueBatchSize: 5, + maxConsumersPerTenant: 2, + }, + } + for name, tt := range tests { + t.Run(name, func(t *testing.T) { + totalTasksCount := tt.tenantsCount * tt.tasksPerTenant + limits := &mockLimits{maxConsumer: tt.maxConsumersPerTenant} + require.LessOrEqual(t, tt.tasksPerTenant, tenantsQueueMaxSize, "test must be able to enqueue all the tasks without error") + queue := NewRequestQueue(tenantsQueueMaxSize, 0, limits, NewMetrics(nil, constants.Loki, "query_scheduler")) + for i := 0; i < tt.consumersCount; i++ { + queue.RegisterConsumerConnection(createConsumerName(i)) + } + + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + defer cancel() + wg, _ := errgroup.WithContext(ctx) + + mtx := &sync.Mutex{} + receivedTasksPerTenant := make(map[string]int, tt.tenantsCount) + receivedTasksCount := atomic.NewInt32(0) + for i := 0; i < tt.consumersCount; i++ { + consumer := createConsumerName(i) + wg.Go(func() error { + idx := StartIndexWithLocalQueue + for { + tasks, newIdx, err := queue.DequeueMany(ctx, idx, consumer, tt.dequeueBatchSize) + + if err != nil && !errors.Is(err, context.Canceled) { + return fmt.Errorf("error while dequeueing many task by %s: %w", consumer, err) + } + if err == nil && len(tasks) == 0 { + return fmt.Errorf("%s must receive at least one item if there is no error, but got 0", consumer) + } + if len(tasks) > 1 && !isAllItemsSame(tasks) { + return fmt.Errorf("expected all items to be from the same tenant, but they were not: %v", tasks) + } + + time.Sleep(tt.consumerDelay) + collectTasks(mtx, err, tasks, receivedTasksPerTenant) + receivedTasksTotal := receivedTasksCount.Add(int32(len(tasks))) + // put the slice back to the pool + queue.ReleaseRequests(tasks) + if receivedTasksTotal == int32(totalTasksCount) { + //cancel the context to release the rest of the consumers once we received all the tasks from all the queues + cancel() + return nil + } + idx = newIdx + } + }) + } + + enqueueTasksAsync(tt.tenantsCount, tt.tasksPerTenant, tt.senderDelay, wg, queue) + + require.NoError(t, wg.Wait()) + require.Len(t, receivedTasksPerTenant, tt.tenantsCount) + for tenant, actualTasksCount := range receivedTasksPerTenant { + require.Equalf(t, tt.tasksPerTenant, actualTasksCount, "%s received less tasks than expected", tenant) + } + }) + } +} + +func enqueueTasksAsync(tenantsCount int, tasksPerTenant int, senderDelay time.Duration, wg *errgroup.Group, queue *RequestQueue) { + for i := 0; i < tenantsCount; i++ { + tenant := fmt.Sprintf("tenant-%d", i) + wg.Go(func() error { + for j := 0; j < tasksPerTenant; j++ { + err := queue.Enqueue(tenant, []string{}, tenant, nil) + if err != nil { + return fmt.Errorf("error while enqueueing task for the %s : %w", tenant, err) + } + time.Sleep(senderDelay) + } + return nil + }) + } +} + +func collectTasks(mtx *sync.Mutex, err error, tasks []Request, receivedTasksPerTenant map[string]int) { + if err != nil { + return + } + mtx.Lock() + defer mtx.Unlock() + //tenant name is sent as task + tenant := tasks[0] + receivedTasksPerTenant[fmt.Sprintf("%v", tenant)] += len(tasks) +} + +func createConsumerName(i int) string { + return fmt.Sprintf("consumer-%d", i) +} + +func isAllItemsSame[T comparable](items []T) bool { + firstItem := items[0] + for i := 1; i < len(items); i++ { + if items[i] != firstItem { + return false + } + } + return true +} + func assertChanReceived(t *testing.T, c chan struct{}, timeout time.Duration, msg string) { t.Helper() From a56b56c8b0491a3b83998543673ded371cfa60f7 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 30 Jan 2024 08:45:44 -0800 Subject: [PATCH 06/45] [Bloom] Keyspace aware bloom creation, library utilities (#11810) This PR continues work refactoring the bloom compactor pkg and adds a bunch of utilities for keyspace manipulation/mgmt. A lot of the bloom block logic is contained here as a skeleton. This code is not complete, but is fine to merge as we don't use this yet and should be easier to review before it gets too large. --- pkg/bloomcompactor/v2_meta.go | 135 +++++++++++ pkg/bloomcompactor/v2controller.go | 206 ++++++++++++++++ pkg/bloomcompactor/v2controller_test.go | 223 ++++++++++++++++++ pkg/bloomcompactor/v2spec.go | 15 +- pkg/storage/bloom/v1/bounds.go | 134 +++++++++++ pkg/storage/bloom/v1/bounds_test.go | 93 ++++++++ pkg/storage/bloom/v1/util.go | 47 ---- .../shipper/indexshipper/tsdb/identifier.go | 8 + 8 files changed, 803 insertions(+), 58 deletions(-) create mode 100644 pkg/bloomcompactor/v2_meta.go create mode 100644 pkg/bloomcompactor/v2controller.go create mode 100644 pkg/bloomcompactor/v2controller_test.go create mode 100644 pkg/storage/bloom/v1/bounds.go create mode 100644 pkg/storage/bloom/v1/bounds_test.go diff --git a/pkg/bloomcompactor/v2_meta.go b/pkg/bloomcompactor/v2_meta.go new file mode 100644 index 0000000000000..1be785c0934ac --- /dev/null +++ b/pkg/bloomcompactor/v2_meta.go @@ -0,0 +1,135 @@ +package bloomcompactor + +import ( + "fmt" + "hash" + "path" + + "github.com/pkg/errors" + + v1 "github.com/grafana/loki/pkg/storage/bloom/v1" + "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb" + "github.com/grafana/loki/pkg/util/encoding" +) + +const ( + BloomPrefix = "bloom" + MetasPrefix = "metas" +) + +// TODO(owen-d): Probably want to integrate against the block shipper +// instead of defining here, but only (min,max,fp) should be required for +// the ref. Things like index-paths, etc are not needed and possibly harmful +// in the case we want to do migrations. It's easier to load a block-ref or similar +// within the context of a specific tenant+period+index path and not couple them. +type BlockRef struct { + OwnershipRange v1.FingerprintBounds + Checksum uint32 +} + +func (r BlockRef) Hash(h hash.Hash32) error { + if err := r.OwnershipRange.Hash(h); err != nil { + return err + } + + var enc encoding.Encbuf + enc.PutBE32(r.Checksum) + _, err := h.Write(enc.Get()) + return errors.Wrap(err, "writing BlockRef") +} + +type MetaRef struct { + OwnershipRange v1.FingerprintBounds + Checksum uint32 +} + +// `bloom///metas/--.json` +func (m MetaRef) Address(tenant string, period int) (string, error) { + joined := path.Join( + BloomPrefix, + fmt.Sprintf("%v", period), + tenant, + MetasPrefix, + fmt.Sprintf("%v-%v", m.OwnershipRange, m.Checksum), + ) + + return fmt.Sprintf("%s.json", joined), nil +} + +type Meta struct { + + // The fingerprint range of the block. This is the range _owned_ by the meta and + // is greater than or equal to the range of the actual data in the underlying blocks. + OwnershipRange v1.FingerprintBounds + + // Old blocks which can be deleted in the future. These should be from pervious compaction rounds. + Tombstones []BlockRef + + // The specific TSDB files used to generate the block. + Sources []tsdb.SingleTenantTSDBIdentifier + + // A list of blocks that were generated + Blocks []BlockRef +} + +// Generate MetaRef from Meta +func (m Meta) Ref() (MetaRef, error) { + checksum, err := m.Checksum() + if err != nil { + return MetaRef{}, errors.Wrap(err, "getting checksum") + } + return MetaRef{ + OwnershipRange: m.OwnershipRange, + Checksum: checksum, + }, nil +} + +func (m Meta) Checksum() (uint32, error) { + h := v1.Crc32HashPool.Get() + defer v1.Crc32HashPool.Put(h) + + _, err := h.Write([]byte(m.OwnershipRange.String())) + if err != nil { + return 0, errors.Wrap(err, "writing OwnershipRange") + } + + for _, tombstone := range m.Tombstones { + err = tombstone.Hash(h) + if err != nil { + return 0, errors.Wrap(err, "writing Tombstones") + } + } + + for _, source := range m.Sources { + err = source.Hash(h) + if err != nil { + return 0, errors.Wrap(err, "writing Sources") + } + } + + for _, block := range m.Blocks { + err = block.Hash(h) + if err != nil { + return 0, errors.Wrap(err, "writing Blocks") + } + } + + return h.Sum32(), nil + +} + +type TSDBStore interface { + ResolveTSDBs() ([]*tsdb.TSDBFile, error) +} + +type MetaStore interface { + GetMetas([]MetaRef) ([]Meta, error) + PutMeta(Meta) error + ResolveMetas(bounds v1.FingerprintBounds) ([]MetaRef, error) +} + +type BlockStore interface { + // TODO(owen-d): flesh out|integrate against bloomshipper.Client + GetBlocks([]BlockRef) ([]interface{}, error) + PutBlock(interface{}) error +} diff --git a/pkg/bloomcompactor/v2controller.go b/pkg/bloomcompactor/v2controller.go new file mode 100644 index 0000000000000..3fbcd04cd93db --- /dev/null +++ b/pkg/bloomcompactor/v2controller.go @@ -0,0 +1,206 @@ +package bloomcompactor + +import ( + "context" + "fmt" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/pkg/errors" + + v1 "github.com/grafana/loki/pkg/storage/bloom/v1" + "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb" +) + +type SimpleBloomController struct { + ownershipRange v1.FingerprintBounds // ownership range of this controller + tsdbStore TSDBStore + metaStore MetaStore + blockStore BlockStore + + // TODO(owen-d): add metrics + logger log.Logger +} + +func NewSimpleBloomController( + ownershipRange v1.FingerprintBounds, + tsdbStore TSDBStore, + metaStore MetaStore, + blockStore BlockStore, + logger log.Logger, +) *SimpleBloomController { + return &SimpleBloomController{ + ownershipRange: ownershipRange, + tsdbStore: tsdbStore, + metaStore: metaStore, + blockStore: blockStore, + logger: log.With(logger, "ownership", ownershipRange), + } +} + +func (s *SimpleBloomController) do(_ context.Context) error { + // 1. Resolve TSDBs + tsdbs, err := s.tsdbStore.ResolveTSDBs() + if err != nil { + level.Error(s.logger).Log("msg", "failed to resolve tsdbs", "err", err) + return errors.Wrap(err, "failed to resolve tsdbs") + } + + // 2. Resolve Metas + metaRefs, err := s.metaStore.ResolveMetas(s.ownershipRange) + if err != nil { + level.Error(s.logger).Log("msg", "failed to resolve metas", "err", err) + return errors.Wrap(err, "failed to resolve metas") + } + + // 3. Fetch metas + metas, err := s.metaStore.GetMetas(metaRefs) + if err != nil { + level.Error(s.logger).Log("msg", "failed to get metas", "err", err) + return errors.Wrap(err, "failed to get metas") + } + + ids := make([]tsdb.Identifier, 0, len(tsdbs)) + for _, idx := range tsdbs { + ids = append(ids, idx.Identifier) + } + + // 4. Determine which TSDBs have gaps in the ownership range and need to + // be processed. + work, err := gapsBetweenTSDBsAndMetas(s.ownershipRange, ids, metas) + if err != nil { + level.Error(s.logger).Log("msg", "failed to find gaps", "err", err) + return errors.Wrap(err, "failed to find gaps") + } + + if len(work) == 0 { + level.Debug(s.logger).Log("msg", "blooms exist for all tsdbs") + return nil + } + + // TODO(owen-d): finish + panic("not implemented") + + // 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 + // needed chunks in their blooms, for instance after a new TSDB version is generated + // but contains many of the same chunk references from the previous version. + // To do this, we'll need to take the metas we've already resolved and find blocks + // overlapping the ownership ranges we've identified as needing updates. + // With these in hand, we can download the old blocks and use them to + // accelerate bloom generation for the new blocks. +} + +type tsdbGaps struct { + tsdb tsdb.Identifier + gaps []v1.FingerprintBounds +} + +// tsdbsUpToDate returns if the metas are up to date with the tsdbs. This is determined by asserting +// that for each TSDB, there are metas covering the entire ownership range which were generated from that specific TSDB. +func gapsBetweenTSDBsAndMetas( + ownershipRange v1.FingerprintBounds, + tsdbs []tsdb.Identifier, + metas []Meta, +) (res []tsdbGaps, err error) { + for _, db := range tsdbs { + id := db.Name() + + relevantMetas := make([]v1.FingerprintBounds, 0, len(metas)) + for _, meta := range metas { + for _, s := range meta.Sources { + if s.Name() == id { + relevantMetas = append(relevantMetas, meta.OwnershipRange) + } + } + } + + gaps, err := findGaps(ownershipRange, relevantMetas) + if err != nil { + return nil, err + } + + if len(gaps) > 0 { + res = append(res, tsdbGaps{ + tsdb: db, + gaps: gaps, + }) + } + } + + return res, err +} + +func findGaps(ownershipRange v1.FingerprintBounds, metas []v1.FingerprintBounds) (gaps []v1.FingerprintBounds, err error) { + if len(metas) == 0 { + return []v1.FingerprintBounds{ownershipRange}, nil + } + + // turn the available metas into a list of non-overlapping metas + // for easier processing + var nonOverlapping []v1.FingerprintBounds + // First, we reduce the metas into a smaller set by combining overlaps. They must be sorted. + var cur *v1.FingerprintBounds + for i := 0; i < len(metas); i++ { + j := i + 1 + + // first iteration (i == 0), set the current meta + if cur == nil { + cur = &metas[i] + } + + if j >= len(metas) { + // We've reached the end of the list. Add the last meta to the non-overlapping set. + nonOverlapping = append(nonOverlapping, *cur) + break + } + + combined := cur.Union(metas[j]) + if len(combined) == 1 { + // There was an overlap between the two tested ranges. Combine them and keep going. + cur = &combined[0] + continue + } + + // There was no overlap between the two tested ranges. Add the first to the non-overlapping set. + // and keep the second for the next iteration. + nonOverlapping = append(nonOverlapping, combined[0]) + cur = &combined[1] + } + + // Now, detect gaps between the non-overlapping metas and the ownership range. + // The left bound of the ownership range will be adjusted as we go. + leftBound := ownershipRange.Min + for _, meta := range nonOverlapping { + + clippedMeta := meta.Intersection(ownershipRange) + // should never happen as long as we are only combining metas + // that intersect with the ownership range + if clippedMeta == nil { + return nil, fmt.Errorf("meta is not within ownership range: %v", meta) + } + + searchRange := ownershipRange.Slice(leftBound, clippedMeta.Max) + // update the left bound for the next iteration + leftBound = min(clippedMeta.Max+1, ownershipRange.Max+1) + + // since we've already ensured that the meta is within the ownership range, + // we know the xor will be of length zero (when the meta is equal to the ownership range) + // or 1 (when the meta is a subset of the ownership range) + xors := searchRange.Unless(*clippedMeta) + if len(xors) == 0 { + // meta is equal to the ownership range. This means the meta + // covers this entire section of the ownership range. + continue + } + + gaps = append(gaps, xors[0]) + } + + if leftBound <= ownershipRange.Max { + // There is a gap between the last meta and the end of the ownership range. + gaps = append(gaps, v1.NewBounds(leftBound, ownershipRange.Max)) + } + + return gaps, nil +} diff --git a/pkg/bloomcompactor/v2controller_test.go b/pkg/bloomcompactor/v2controller_test.go new file mode 100644 index 0000000000000..0a99f26d3ce1f --- /dev/null +++ b/pkg/bloomcompactor/v2controller_test.go @@ -0,0 +1,223 @@ +package bloomcompactor + +import ( + "testing" + "time" + + "github.com/prometheus/common/model" + "github.com/stretchr/testify/require" + + v1 "github.com/grafana/loki/pkg/storage/bloom/v1" + "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb" +) + +func Test_findGaps(t *testing.T) { + for _, tc := range []struct { + desc string + err bool + exp []v1.FingerprintBounds + ownershipRange v1.FingerprintBounds + metas []v1.FingerprintBounds + }{ + { + desc: "error nonoverlapping metas", + err: true, + exp: nil, + ownershipRange: v1.NewBounds(0, 10), + metas: []v1.FingerprintBounds{v1.NewBounds(11, 20)}, + }, + { + desc: "one meta with entire ownership range", + err: false, + exp: nil, + ownershipRange: v1.NewBounds(0, 10), + metas: []v1.FingerprintBounds{v1.NewBounds(0, 10)}, + }, + { + desc: "two non-overlapping metas with entire ownership range", + err: false, + exp: nil, + ownershipRange: v1.NewBounds(0, 10), + metas: []v1.FingerprintBounds{ + v1.NewBounds(0, 5), + v1.NewBounds(6, 10), + }, + }, + { + desc: "two overlapping metas with entire ownership range", + err: false, + exp: nil, + ownershipRange: v1.NewBounds(0, 10), + metas: []v1.FingerprintBounds{ + v1.NewBounds(0, 6), + v1.NewBounds(4, 10), + }, + }, + { + desc: "one meta with partial ownership range", + err: false, + exp: []v1.FingerprintBounds{ + v1.NewBounds(6, 10), + }, + ownershipRange: v1.NewBounds(0, 10), + metas: []v1.FingerprintBounds{ + v1.NewBounds(0, 5), + }, + }, + { + desc: "smaller subsequent meta with partial ownership range", + err: false, + exp: []v1.FingerprintBounds{ + v1.NewBounds(8, 10), + }, + ownershipRange: v1.NewBounds(0, 10), + metas: []v1.FingerprintBounds{ + v1.NewBounds(0, 7), + v1.NewBounds(3, 4), + }, + }, + { + desc: "hole in the middle", + err: false, + exp: []v1.FingerprintBounds{ + v1.NewBounds(4, 5), + }, + ownershipRange: v1.NewBounds(0, 10), + metas: []v1.FingerprintBounds{ + v1.NewBounds(0, 3), + v1.NewBounds(6, 10), + }, + }, + { + desc: "holes on either end", + err: false, + exp: []v1.FingerprintBounds{ + v1.NewBounds(0, 2), + v1.NewBounds(8, 10), + }, + ownershipRange: v1.NewBounds(0, 10), + metas: []v1.FingerprintBounds{ + v1.NewBounds(3, 5), + v1.NewBounds(6, 7), + }, + }, + } { + t.Run(tc.desc, func(t *testing.T) { + gaps, err := findGaps(tc.ownershipRange, tc.metas) + if tc.err { + require.Error(t, err) + return + } + require.Equal(t, tc.exp, gaps) + }) + } +} + +func Test_gapsBetweenTSDBsAndMetas(t *testing.T) { + id := func(n int) tsdb.SingleTenantTSDBIdentifier { + return tsdb.SingleTenantTSDBIdentifier{ + TS: time.Unix(int64(n), 0), + } + } + + meta := func(min, max model.Fingerprint, sources ...int) Meta { + m := Meta{ + OwnershipRange: v1.NewBounds(min, max), + } + for _, source := range sources { + m.Sources = append(m.Sources, id(source)) + } + return m + } + + for _, tc := range []struct { + desc string + err bool + exp []tsdbGaps + ownershipRange v1.FingerprintBounds + tsdbs []tsdb.Identifier + metas []Meta + }{ + { + desc: "non-overlapping tsdbs and metas", + err: true, + ownershipRange: v1.NewBounds(0, 10), + tsdbs: []tsdb.Identifier{id(0)}, + metas: []Meta{ + meta(11, 20, 0), + }, + }, + { + desc: "single tsdb", + ownershipRange: v1.NewBounds(0, 10), + tsdbs: []tsdb.Identifier{id(0)}, + metas: []Meta{ + meta(4, 8, 0), + }, + exp: []tsdbGaps{ + { + tsdb: id(0), + gaps: []v1.FingerprintBounds{ + v1.NewBounds(0, 3), + v1.NewBounds(9, 10), + }, + }, + }, + }, + { + desc: "multiple tsdbs with separate blocks", + ownershipRange: v1.NewBounds(0, 10), + tsdbs: []tsdb.Identifier{id(0), id(1)}, + metas: []Meta{ + meta(0, 5, 0), + meta(6, 10, 1), + }, + exp: []tsdbGaps{ + { + tsdb: id(0), + gaps: []v1.FingerprintBounds{ + v1.NewBounds(6, 10), + }, + }, + { + tsdb: id(1), + gaps: []v1.FingerprintBounds{ + v1.NewBounds(0, 5), + }, + }, + }, + }, + { + desc: "multiple tsdbs with the same blocks", + ownershipRange: v1.NewBounds(0, 10), + tsdbs: []tsdb.Identifier{id(0), id(1)}, + metas: []Meta{ + meta(0, 5, 0, 1), + meta(6, 8, 1), + }, + exp: []tsdbGaps{ + { + tsdb: id(0), + gaps: []v1.FingerprintBounds{ + v1.NewBounds(6, 10), + }, + }, + { + tsdb: id(1), + gaps: []v1.FingerprintBounds{ + v1.NewBounds(9, 10), + }, + }, + }, + }, + } { + t.Run(tc.desc, func(t *testing.T) { + gaps, err := gapsBetweenTSDBsAndMetas(tc.ownershipRange, tc.tsdbs, tc.metas) + if tc.err { + require.Error(t, err) + return + } + require.Equal(t, tc.exp, gaps) + }) + } +} diff --git a/pkg/bloomcompactor/v2spec.go b/pkg/bloomcompactor/v2spec.go index 334b79f2ae746..49e74a47188a7 100644 --- a/pkg/bloomcompactor/v2spec.go +++ b/pkg/bloomcompactor/v2spec.go @@ -115,7 +115,7 @@ func (s *SimpleBloomGenerator) populator(ctx context.Context) func(series *v1.Se return func(series *v1.Series, bloom *v1.Bloom) error { chunkItersWithFP, err := s.chunkLoader.Load(ctx, series) if err != nil { - return errors.Wrapf(err, "failed to load chunks for series: %#v", series) + return errors.Wrapf(err, "failed to load chunks for series: %+v", series) } return s.tokenizer.Populate( @@ -134,7 +134,7 @@ func (s *SimpleBloomGenerator) Generate(ctx context.Context) (skippedBlocks []*v blocksMatchingSchema := make([]v1.PeekingIterator[*v1.SeriesWithBloom], 0, len(s.blocks)) 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", fmt.Sprintf("%+v", block)) schema, err := block.Schema() if err != nil { level.Warn(logger).Log("msg", "failed to get schema for block", "err", err) @@ -142,7 +142,7 @@ func (s *SimpleBloomGenerator) Generate(ctx context.Context) (skippedBlocks []*v } 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)) + 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, block) } @@ -151,7 +151,7 @@ func (s *SimpleBloomGenerator) Generate(ctx context.Context) (skippedBlocks []*v blocksMatchingSchema = append(blocksMatchingSchema, itr) } - 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)) + 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 @@ -331,10 +331,3 @@ func (b *batchedLoader) At() v1.ChunkRefWithIter { func (b *batchedLoader) Err() error { return b.err } - -func min(a, b int) int { - if a < b { - return a - } - return b -} diff --git a/pkg/storage/bloom/v1/bounds.go b/pkg/storage/bloom/v1/bounds.go new file mode 100644 index 0000000000000..fc22866285f82 --- /dev/null +++ b/pkg/storage/bloom/v1/bounds.go @@ -0,0 +1,134 @@ +package v1 + +import ( + "hash" + + "github.com/pkg/errors" + "github.com/prometheus/common/model" + + "github.com/grafana/loki/pkg/util/encoding" +) + +type BoundsCheck uint8 + +const ( + Before BoundsCheck = iota + Overlap + After +) + +type FingerprintBounds struct { + Min, Max model.Fingerprint +} + +func NewBounds(min, max model.Fingerprint) FingerprintBounds { + return FingerprintBounds{Min: min, Max: max} +} + +func (b FingerprintBounds) Hash(h hash.Hash32) error { + var enc encoding.Encbuf + enc.PutBE64(uint64(b.Min)) + enc.PutBE64(uint64(b.Max)) + _, err := h.Write(enc.Get()) + return errors.Wrap(err, "writing OwnershipRange") +} + +func (b FingerprintBounds) String() string { + return b.Min.String() + "-" + b.Max.String() +} + +// Cmp returns the fingerprint's position relative to the bounds +func (b FingerprintBounds) Cmp(fp model.Fingerprint) BoundsCheck { + if fp < b.Min { + return Before + } else if fp > b.Max { + return After + } + return Overlap +} + +func (b FingerprintBounds) Overlaps(target FingerprintBounds) bool { + return b.Cmp(target.Min) != After && b.Cmp(target.Max) != Before +} + +// Slice returns a new fingerprint bounds clipped to the target bounds or nil if there is no overlap +func (b FingerprintBounds) Slice(min, max model.Fingerprint) *FingerprintBounds { + return b.Intersection(FingerprintBounds{Min: min, Max: max}) +} + +// Returns whether the fingerprint is fully within the target bounds +func (b FingerprintBounds) Within(target FingerprintBounds) bool { + return b.Min >= target.Min && b.Max <= target.Max +} + +// Intersection returns the intersection of the two bounds +func (b FingerprintBounds) Intersection(target FingerprintBounds) *FingerprintBounds { + if !b.Overlaps(target) { + return nil + } + + return &FingerprintBounds{ + Min: max(b.Min, target.Min), + Max: min(b.Max, target.Max), + } +} + +// Union returns the union of the two bounds +func (b FingerprintBounds) Union(target FingerprintBounds) (res []FingerprintBounds) { + if !b.Overlaps(target) { + if b.Cmp(target.Min) == Before { + return []FingerprintBounds{target, b} + } + return []FingerprintBounds{b, target} + } + + return []FingerprintBounds{ + { + Min: min(b.Min, target.Min), + Max: max(b.Max, target.Max), + }, + } +} + +// Unless returns the subspace of itself which does not intersect with the target bounds +func (b FingerprintBounds) Unless(target FingerprintBounds) (res []FingerprintBounds) { + if !b.Overlaps(target) { + return []FingerprintBounds{b} + } + + if b == target { + return nil + } + + if b.Min < target.Min { + res = append(res, FingerprintBounds{Min: b.Min, Max: min(b.Max, target.Min-1)}) + } + if target.Max < b.Max { + res = append(res, FingerprintBounds{Min: max(b.Min, target.Max+1), Max: b.Max}) + } + return res +} + +// unused, but illustrative +type BoundedIter[V any] struct { + Iterator[V] + cmp func(V) BoundsCheck +} + +func (bi *BoundedIter[V]) Next() bool { + for bi.Iterator.Next() { + switch bi.cmp(bi.Iterator.At()) { + case Before: + continue + case After: + return false + default: + return true + } + } + return false +} + +func NewBoundedIter[V any](itr Iterator[V], cmp func(V) BoundsCheck) *BoundedIter[V] { + return &BoundedIter[V]{Iterator: itr, cmp: cmp} +} diff --git a/pkg/storage/bloom/v1/bounds_test.go b/pkg/storage/bloom/v1/bounds_test.go new file mode 100644 index 0000000000000..3a80f6e6b849a --- /dev/null +++ b/pkg/storage/bloom/v1/bounds_test.go @@ -0,0 +1,93 @@ +package v1 + +import ( + "testing" + + "github.com/stretchr/testify/assert" +) + +func Test_FingerprintBounds_String(t *testing.T) { + bounds := NewBounds(1, 2) + assert.Equal(t, "0000000000000001-0000000000000002", bounds.String()) +} + +func Test_FingerprintBounds_Cmp(t *testing.T) { + bounds := NewBounds(10, 20) + assert.Equal(t, Before, bounds.Cmp(0)) + assert.Equal(t, Overlap, bounds.Cmp(10)) + assert.Equal(t, Overlap, bounds.Cmp(15)) + assert.Equal(t, Overlap, bounds.Cmp(20)) + assert.Equal(t, After, bounds.Cmp(21)) +} + +func Test_FingerprintBounds_Overlap(t *testing.T) { + bounds := NewBounds(10, 20) + assert.True(t, bounds.Overlaps(FingerprintBounds{Min: 5, Max: 15})) + assert.True(t, bounds.Overlaps(FingerprintBounds{Min: 15, Max: 25})) + assert.True(t, bounds.Overlaps(FingerprintBounds{Min: 10, Max: 20})) + assert.True(t, bounds.Overlaps(FingerprintBounds{Min: 5, Max: 25})) + assert.False(t, bounds.Overlaps(FingerprintBounds{Min: 1, Max: 9})) + assert.False(t, bounds.Overlaps(FingerprintBounds{Min: 21, Max: 30})) +} + +func Test_FingerprintBounds_Within(t *testing.T) { + target := NewBounds(10, 20) + assert.False(t, NewBounds(1, 9).Within(target)) + assert.False(t, NewBounds(21, 30).Within(target)) + assert.True(t, NewBounds(10, 20).Within(target)) + assert.True(t, NewBounds(14, 15).Within(target)) + assert.False(t, NewBounds(5, 15).Within(target)) + assert.False(t, NewBounds(15, 25).Within(target)) + assert.False(t, NewBounds(5, 25).Within(target)) +} + +func Test_FingerprintBounds_Intersection(t *testing.T) { + target := NewBounds(10, 20) + assert.Nil(t, NewBounds(1, 9).Intersection(target)) + assert.Nil(t, NewBounds(21, 30).Intersection(target)) + assert.Equal(t, &FingerprintBounds{Min: 10, Max: 20}, NewBounds(10, 20).Intersection(target)) + assert.Equal(t, &FingerprintBounds{Min: 14, Max: 15}, NewBounds(14, 15).Intersection(target)) + assert.Equal(t, &FingerprintBounds{Min: 10, Max: 15}, NewBounds(5, 15).Intersection(target)) + assert.Equal(t, &FingerprintBounds{Min: 15, Max: 20}, NewBounds(15, 25).Intersection(target)) + assert.Equal(t, &target, NewBounds(5, 25).Intersection(target)) +} + +func Test_FingerprintBounds_Union(t *testing.T) { + target := NewBounds(10, 20) + assert.Equal(t, []FingerprintBounds{ + {Min: 1, Max: 9}, + {Min: 10, Max: 20}, + }, NewBounds(1, 9).Union(target)) + assert.Equal(t, []FingerprintBounds{ + {Min: 10, Max: 20}, + {Min: 21, Max: 30}, + }, NewBounds(21, 30).Union(target)) + assert.Equal(t, []FingerprintBounds{ + {Min: 10, Max: 20}, + }, NewBounds(10, 20).Union(target)) + assert.Equal(t, []FingerprintBounds{ + {Min: 5, Max: 20}, + }, NewBounds(5, 15).Union(target)) +} + +func Test_FingerprintBounds_Xor(t *testing.T) { + target := NewBounds(10, 20) + assert.Equal(t, []FingerprintBounds{ + {Min: 1, Max: 9}, + }, NewBounds(1, 9).Unless(target)) + assert.Equal(t, []FingerprintBounds{ + {Min: 21, Max: 30}, + }, NewBounds(21, 30).Unless(target)) + assert.Nil(t, NewBounds(10, 20).Unless(target)) + assert.Equal(t, []FingerprintBounds{ + {Min: 5, Max: 9}, + }, NewBounds(5, 15).Unless(target)) + assert.Equal(t, []FingerprintBounds{ + {Min: 21, Max: 25}, + }, NewBounds(15, 25).Unless(target)) + assert.Equal(t, []FingerprintBounds{ + {Min: 5, Max: 9}, + {Min: 21, Max: 25}, + }, NewBounds(5, 25).Unless(target)) + assert.Nil(t, NewBounds(14, 15).Unless(target)) +} diff --git a/pkg/storage/bloom/v1/util.go b/pkg/storage/bloom/v1/util.go index 5a7046b5d0477..1716feef35b03 100644 --- a/pkg/storage/bloom/v1/util.go +++ b/pkg/storage/bloom/v1/util.go @@ -7,7 +7,6 @@ import ( "io" "sync" - "github.com/prometheus/common/model" "github.com/prometheus/prometheus/util/pool" ) @@ -242,49 +241,3 @@ func PointerSlice[T any](xs []T) []*T { } return out } - -type BoundsCheck uint8 - -const ( - Before BoundsCheck = iota - Overlap - After -) - -type FingerprintBounds struct { - Min, Max model.Fingerprint -} - -// Cmp returns the fingerprint's position relative to the bounds -func (b FingerprintBounds) Cmp(fp model.Fingerprint) BoundsCheck { - if fp < b.Min { - return Before - } else if fp > b.Max { - return After - } - return Overlap -} - -// unused, but illustrative -type BoundedIter[V any] struct { - Iterator[V] - cmp func(V) BoundsCheck -} - -func (bi *BoundedIter[V]) Next() bool { - for bi.Iterator.Next() { - switch bi.cmp(bi.Iterator.At()) { - case Before: - continue - case After: - return false - default: - return true - } - } - return false -} - -func NewBoundedIter[V any](itr Iterator[V], cmp func(V) BoundsCheck) *BoundedIter[V] { - return &BoundedIter[V]{Iterator: itr, cmp: cmp} -} diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/identifier.go b/pkg/storage/stores/shipper/indexshipper/tsdb/identifier.go index 72b5a8cf0aede..7bfd10ff4d4dc 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/identifier.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/identifier.go @@ -2,12 +2,14 @@ package tsdb import ( "fmt" + "hash" "path" "path/filepath" "strconv" "strings" "time" + "github.com/pkg/errors" "github.com/prometheus/common/model" ) @@ -67,6 +69,12 @@ type SingleTenantTSDBIdentifier struct { Checksum uint32 } +// implement Hash +func (i SingleTenantTSDBIdentifier) Hash(h hash.Hash32) (err error) { + _, err = h.Write([]byte(i.str())) + return errors.Wrap(err, "writing SingleTenantTSDBIdentifier") +} + // str builds filename with format + `-` + `compactor` + `-` + + `-` + `-` + func (i SingleTenantTSDBIdentifier) str() string { return fmt.Sprintf( From 9d4d101169b5311a0feffe0b20aa7b3f9e7a3ac1 Mon Sep 17 00:00:00 2001 From: Meng Ye <4025839+jk2K@users.noreply.github.com> Date: Wed, 31 Jan 2024 02:10:48 +0800 Subject: [PATCH 07/45] docs: fix runtime_config example (#11798) **What this PR does / why we need it**: **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) --------- Co-authored-by: J Stickler --- docs/sources/configure/_index.md | 3 ++- docs/sources/configure/index.template | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/sources/configure/_index.md b/docs/sources/configure/_index.md index 2a2c51544f013..283a2c9dd59a9 100644 --- a/docs/sources/configure/_index.md +++ b/docs/sources/configure/_index.md @@ -5345,7 +5345,8 @@ place in the `limits_config` section: configure a runtime configuration file: ``` - runtime_config: overrides.yaml + runtime_config: + file: overrides.yaml ``` In the `overrides.yaml` file, add `unordered_writes` for each tenant diff --git a/docs/sources/configure/index.template b/docs/sources/configure/index.template index 91bd7ab69913b..1cbcbe6d2279e 100644 --- a/docs/sources/configure/index.template +++ b/docs/sources/configure/index.template @@ -152,7 +152,8 @@ place in the `limits_config` section: configure a runtime configuration file: ``` - runtime_config: overrides.yaml + runtime_config: + file: overrides.yaml ``` In the `overrides.yaml` file, add `unordered_writes` for each tenant From 38d205b96715fbb8c5926fb112d2386c20ce3dd4 Mon Sep 17 00:00:00 2001 From: Robert Jacob Date: Tue, 30 Jan 2024 19:27:13 +0100 Subject: [PATCH 08/45] operator: Improve messages for errors in storage secret (#11824) --- operator/CHANGELOG.md | 1 + .../handlers/internal/storage/secrets.go | 88 ++++---- .../handlers/internal/storage/secrets_test.go | 194 ++++++++++-------- .../handlers/internal/storage/storage_test.go | 2 +- 4 files changed, 157 insertions(+), 128 deletions(-) diff --git a/operator/CHANGELOG.md b/operator/CHANGELOG.md index 039e37a32297f..18e28a016efee 100644 --- a/operator/CHANGELOG.md +++ b/operator/CHANGELOG.md @@ -1,5 +1,6 @@ ## Main +- [11824](https://github.com/grafana/loki/pull/11824) **xperimental**: Improve messages for errors in storage secret - [11524](https://github.com/grafana/loki/pull/11524) **JoaoBraveCoding**, **periklis**: Add OpenShift cloud credentials support for AWS STS - [11513](https://github.com/grafana/loki/pull/11513) **btaani**: Add a custom metric that collects Lokistacks requiring a schema upgrade - [11718](https://github.com/grafana/loki/pull/11718) **periklis**: Upgrade k8s.io, sigs.k8s.io and openshift deps diff --git a/operator/internal/handlers/internal/storage/secrets.go b/operator/internal/handlers/internal/storage/secrets.go index 705cabb6cf5d9..e41fa9c2c5b08 100644 --- a/operator/internal/handlers/internal/storage/secrets.go +++ b/operator/internal/handlers/internal/storage/secrets.go @@ -3,10 +3,10 @@ package storage import ( "context" "crypto/sha1" + "errors" "fmt" "sort" - "github.com/ViaQ/logerr/v2/kverrors" corev1 "k8s.io/api/core/v1" apierrors "k8s.io/apimachinery/pkg/api/errors" "sigs.k8s.io/controller-runtime/pkg/client" @@ -18,7 +18,17 @@ import ( "github.com/grafana/loki/operator/internal/status" ) -var hashSeparator = []byte(",") +var ( + hashSeparator = []byte(",") + + errSecretUnknownType = errors.New("unknown secret type") + errSecretMissingField = errors.New("missing secret field") + errSecretFieldNotAllowed = errors.New("secret field not allowed") + errSecretUnknownSSEType = errors.New("unsupported SSE type (supported: SSE-KMS, SSE-S3)") + errSecretHashError = errors.New("error calculating hash for secret") + + errS3NoAuth = errors.New("missing secret fields for static or sts authentication") +) func getSecrets(ctx context.Context, k k8s.Client, stack *lokiv1.LokiStack, fg configv1.FeatureGates) (*corev1.Secret, *corev1.Secret, error) { var ( @@ -35,7 +45,7 @@ func getSecrets(ctx context.Context, k k8s.Client, stack *lokiv1.LokiStack, fg c Requeue: false, } } - return nil, nil, kverrors.Wrap(err, "failed to lookup lokistack storage secret", "name", key) + return nil, nil, fmt.Errorf("failed to lookup lokistack storage secret: %w", err) } if fg.OpenShift.ManagedAuthEnv { @@ -57,7 +67,7 @@ func getSecrets(ctx context.Context, k k8s.Client, stack *lokiv1.LokiStack, fg c Requeue: true, } } - return nil, nil, kverrors.Wrap(err, "failed to lookup OpenShift CCO managed authentication credentials secret", "name", stack) + return nil, nil, fmt.Errorf("failed to lookup OpenShift CCO managed authentication credentials secret: %w", err) } return &storageSecret, &managedAuthSecret, nil @@ -71,7 +81,7 @@ func getSecrets(ctx context.Context, k k8s.Client, stack *lokiv1.LokiStack, fg c func extractSecrets(secretType lokiv1.ObjectStorageSecretType, objStore, managedAuth *corev1.Secret, fg configv1.FeatureGates) (storage.Options, error) { hash, err := hashSecretData(objStore) if err != nil { - return storage.Options{}, kverrors.Wrap(err, "error calculating hash for secret", "type", secretType) + return storage.Options{}, errSecretHashError } storageOpts := storage.Options{ @@ -84,7 +94,7 @@ func extractSecrets(secretType lokiv1.ObjectStorageSecretType, objStore, managed var managedAuthHash string managedAuthHash, err = hashSecretData(managedAuth) if err != nil { - return storage.Options{}, kverrors.Wrap(err, "error calculating hash for secret", "type", client.ObjectKeyFromObject(managedAuth)) + return storage.Options{}, errSecretHashError } storageOpts.OpenShift = storage.OpenShiftOptions{ @@ -107,7 +117,7 @@ func extractSecrets(secretType lokiv1.ObjectStorageSecretType, objStore, managed case lokiv1.ObjectStorageSecretAlibabaCloud: storageOpts.AlibabaCloud, err = extractAlibabaCloudConfigSecret(objStore) default: - return storage.Options{}, kverrors.New("unknown secret type", "type", secretType) + return storage.Options{}, fmt.Errorf("%w: %s", errSecretUnknownType, secretType) } if err != nil { @@ -149,19 +159,19 @@ func extractAzureConfigSecret(s *corev1.Secret) (*storage.AzureStorageConfig, er // Extract and validate mandatory fields env := s.Data[storage.KeyAzureEnvironmentName] if len(env) == 0 { - return nil, kverrors.New("missing secret field", "field", storage.KeyAzureEnvironmentName) + return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyAzureEnvironmentName) } container := s.Data[storage.KeyAzureStorageContainerName] if len(container) == 0 { - return nil, kverrors.New("missing secret field", "field", storage.KeyAzureStorageContainerName) + return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyAzureStorageContainerName) } name := s.Data[storage.KeyAzureStorageAccountName] if len(name) == 0 { - return nil, kverrors.New("missing secret field", "field", storage.KeyAzureStorageAccountName) + return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyAzureStorageAccountName) } key := s.Data[storage.KeyAzureStorageAccountKey] if len(key) == 0 { - return nil, kverrors.New("missing secret field", "field", storage.KeyAzureStorageAccountKey) + return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyAzureStorageAccountKey) } // Extract and validate optional fields @@ -178,13 +188,13 @@ func extractGCSConfigSecret(s *corev1.Secret) (*storage.GCSStorageConfig, error) // Extract and validate mandatory fields bucket := s.Data[storage.KeyGCPStorageBucketName] if len(bucket) == 0 { - return nil, kverrors.New("missing secret field", "field", storage.KeyGCPStorageBucketName) + return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyGCPStorageBucketName) } // Check if google authentication credentials is provided keyJSON := s.Data[storage.KeyGCPServiceAccountKeyFilename] if len(keyJSON) == 0 { - return nil, kverrors.New("missing google authentication credentials", "field", storage.KeyGCPServiceAccountKeyFilename) + return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyGCPServiceAccountKeyFilename) } return &storage.GCSStorageConfig{ @@ -196,7 +206,7 @@ func extractS3ConfigSecret(s *corev1.Secret, fg configv1.FeatureGates) (*storage // Extract and validate mandatory fields buckets := s.Data[storage.KeyAWSBucketNames] if len(buckets) == 0 { - return nil, kverrors.New("missing secret field", "field", storage.KeyAWSBucketNames) + return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyAWSBucketNames) } var ( @@ -233,14 +243,14 @@ func extractS3ConfigSecret(s *corev1.Secret, fg configv1.FeatureGates) (*storage cfg.Audience = storage.AWSOpenShiftAudience // Do not allow users overriding the role arn provided on Loki Operator installation if len(roleArn) != 0 { - return nil, kverrors.New("extra secret field set", "field", storage.KeyAWSRoleArn) + return nil, fmt.Errorf("%w: %s", errSecretFieldNotAllowed, storage.KeyAWSRoleArn) } if len(audience) != 0 { - return nil, kverrors.New("extra secret field set", "field", storage.KeyAWSAudience) + return nil, fmt.Errorf("%w: %s", errSecretFieldNotAllowed, storage.KeyAWSAudience) } // In the STS case region is not an optional field if len(region) == 0 { - return nil, kverrors.New("missing secret field", "field", storage.KeyAWSRegion) + return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyAWSRegion) } return cfg, nil @@ -248,13 +258,13 @@ func extractS3ConfigSecret(s *corev1.Secret, fg configv1.FeatureGates) (*storage cfg.Endpoint = string(endpoint) if len(endpoint) == 0 { - return nil, kverrors.New("missing secret field", "field", storage.KeyAWSEndpoint) + return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyAWSEndpoint) } if len(id) == 0 { - return nil, kverrors.New("missing secret field", "field", storage.KeyAWSAccessKeyID) + return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyAWSAccessKeyID) } if len(secret) == 0 { - return nil, kverrors.New("missing secret field", "field", storage.KeyAWSAccessKeySecret) + return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyAWSAccessKeySecret) } return cfg, nil @@ -264,11 +274,11 @@ func extractS3ConfigSecret(s *corev1.Secret, fg configv1.FeatureGates) (*storage // In the STS case region is not an optional field if len(region) == 0 { - return nil, kverrors.New("missing secret field", "field", storage.KeyAWSRegion) + return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyAWSRegion) } return cfg, nil default: - return nil, kverrors.New("missing secret fields for static or sts authentication") + return nil, errS3NoAuth } } @@ -283,7 +293,7 @@ func extractS3SSEConfig(d map[string][]byte) (storage.S3SSEConfig, error) { kmsEncryptionCtx = string(d[storage.KeyAWSSseKmsEncryptionContext]) kmsKeyId = string(d[storage.KeyAWSSseKmsKeyID]) if kmsKeyId == "" { - return storage.S3SSEConfig{}, kverrors.New("missing secret field", "field", storage.KeyAWSSseKmsKeyID) + return storage.S3SSEConfig{}, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyAWSSseKmsKeyID) } case storage.SSES3Type: @@ -291,13 +301,13 @@ func extractS3SSEConfig(d map[string][]byte) (storage.S3SSEConfig, error) { return storage.S3SSEConfig{}, nil default: - return storage.S3SSEConfig{}, kverrors.New("unsupported secret field value (Supported: SSE-KMS, SSE-S3)", "field", storage.KeyAWSSSEType, "value", sseType) + return storage.S3SSEConfig{}, fmt.Errorf("%w: %s", errSecretUnknownSSEType, sseType) } return storage.S3SSEConfig{ Type: sseType, - KMSKeyID: string(kmsKeyId), - KMSEncryptionContext: string(kmsEncryptionCtx), + KMSKeyID: kmsKeyId, + KMSEncryptionContext: kmsEncryptionCtx, }, nil } @@ -305,39 +315,39 @@ func extractSwiftConfigSecret(s *corev1.Secret) (*storage.SwiftStorageConfig, er // Extract and validate mandatory fields url := s.Data[storage.KeySwiftAuthURL] if len(url) == 0 { - return nil, kverrors.New("missing secret field", "field", storage.KeySwiftAuthURL) + return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeySwiftAuthURL) } username := s.Data[storage.KeySwiftUsername] if len(username) == 0 { - return nil, kverrors.New("missing secret field", "field", storage.KeySwiftUsername) + return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeySwiftUsername) } userDomainName := s.Data[storage.KeySwiftUserDomainName] if len(userDomainName) == 0 { - return nil, kverrors.New("missing secret field", "field", storage.KeySwiftUserDomainName) + return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeySwiftUserDomainName) } userDomainID := s.Data[storage.KeySwiftUserDomainID] if len(userDomainID) == 0 { - return nil, kverrors.New("missing secret field", "field", storage.KeySwiftUserDomainID) + return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeySwiftUserDomainID) } userID := s.Data[storage.KeySwiftUserID] if len(userID) == 0 { - return nil, kverrors.New("missing secret field", "field", storage.KeySwiftUserID) + return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeySwiftUserID) } password := s.Data[storage.KeySwiftPassword] if len(password) == 0 { - return nil, kverrors.New("missing secret field", "field", storage.KeySwiftPassword) + return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeySwiftPassword) } domainID := s.Data[storage.KeySwiftDomainID] if len(domainID) == 0 { - return nil, kverrors.New("missing secret field", "field", storage.KeySwiftDomainID) + return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeySwiftDomainID) } domainName := s.Data[storage.KeySwiftDomainName] if len(domainName) == 0 { - return nil, kverrors.New("missing secret field", "field", storage.KeySwiftDomainName) + return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeySwiftDomainName) } containerName := s.Data[storage.KeySwiftContainerName] if len(containerName) == 0 { - return nil, kverrors.New("missing secret field", "field", storage.KeySwiftContainerName) + return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeySwiftContainerName) } // Extract and validate optional fields @@ -367,19 +377,19 @@ func extractAlibabaCloudConfigSecret(s *corev1.Secret) (*storage.AlibabaCloudSto // Extract and validate mandatory fields endpoint := s.Data[storage.KeyAlibabaCloudEndpoint] if len(endpoint) == 0 { - return nil, kverrors.New("missing secret field", "field", storage.KeyAlibabaCloudEndpoint) + return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyAlibabaCloudEndpoint) } bucket := s.Data[storage.KeyAlibabaCloudBucket] if len(bucket) == 0 { - return nil, kverrors.New("missing secret field", "field", storage.KeyAlibabaCloudBucket) + return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyAlibabaCloudBucket) } id := s.Data[storage.KeyAlibabaCloudAccessKeyID] if len(id) == 0 { - return nil, kverrors.New("missing secret field", "field", storage.KeyAlibabaCloudAccessKeyID) + return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyAlibabaCloudAccessKeyID) } secret := s.Data[storage.KeyAlibabaCloudSecretAccessKey] if len(secret) == 0 { - return nil, kverrors.New("missing secret field", "field", storage.KeyAlibabaCloudSecretAccessKey) + return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyAlibabaCloudSecretAccessKey) } return &storage.AlibabaCloudStorageConfig{ diff --git a/operator/internal/handlers/internal/storage/secrets_test.go b/operator/internal/handlers/internal/storage/secrets_test.go index 535fd3a0aa141..70aebd18afc53 100644 --- a/operator/internal/handlers/internal/storage/secrets_test.go +++ b/operator/internal/handlers/internal/storage/secrets_test.go @@ -62,17 +62,24 @@ func TestHashSecretData(t *testing.T) { } } +func TestUnknownType(t *testing.T) { + wantError := "unknown secret type: test-unknown-type" + + _, err := extractSecrets("test-unknown-type", &corev1.Secret{}, nil, configv1.FeatureGates{}) + require.EqualError(t, err, wantError) +} + func TestAzureExtract(t *testing.T) { type test struct { - name string - secret *corev1.Secret - wantErr bool + name string + secret *corev1.Secret + wantError string } table := []test{ { - name: "missing environment", - secret: &corev1.Secret{}, - wantErr: true, + name: "missing environment", + secret: &corev1.Secret{}, + wantError: "missing secret field: environment", }, { name: "missing container", @@ -81,7 +88,7 @@ func TestAzureExtract(t *testing.T) { "environment": []byte("here"), }, }, - wantErr: true, + wantError: "missing secret field: container", }, { name: "missing account_name", @@ -91,7 +98,7 @@ func TestAzureExtract(t *testing.T) { "container": []byte("this,that"), }, }, - wantErr: true, + wantError: "missing secret field: account_name", }, { name: "missing account_key", @@ -103,7 +110,7 @@ func TestAzureExtract(t *testing.T) { "account_name": []byte("id"), }, }, - wantErr: true, + wantError: "missing secret field: account_key", }, { name: "all mandatory set", @@ -137,14 +144,13 @@ func TestAzureExtract(t *testing.T) { t.Parallel() opts, err := extractSecrets(lokiv1.ObjectStorageSecretAzure, tst.secret, nil, configv1.FeatureGates{}) - if !tst.wantErr { + if tst.wantError == "" { require.NoError(t, err) require.NotEmpty(t, opts.SecretName) require.NotEmpty(t, opts.SecretSHA1) require.Equal(t, opts.SharedStore, lokiv1.ObjectStorageSecretAzure) - } - if tst.wantErr { - require.NotNil(t, err) + } else { + require.EqualError(t, err, tst.wantError) } }) } @@ -152,15 +158,15 @@ func TestAzureExtract(t *testing.T) { func TestGCSExtract(t *testing.T) { type test struct { - name string - secret *corev1.Secret - wantErr bool + name string + secret *corev1.Secret + wantError string } table := []test{ { - name: "missing bucketname", - secret: &corev1.Secret{}, - wantErr: true, + name: "missing bucketname", + secret: &corev1.Secret{}, + wantError: "missing secret field: bucketname", }, { name: "missing key.json", @@ -169,7 +175,7 @@ func TestGCSExtract(t *testing.T) { "bucketname": []byte("here"), }, }, - wantErr: true, + wantError: "missing secret field: key.json", }, { name: "all set", @@ -188,11 +194,10 @@ func TestGCSExtract(t *testing.T) { t.Parallel() _, err := extractSecrets(lokiv1.ObjectStorageSecretGCS, tst.secret, nil, configv1.FeatureGates{}) - if !tst.wantErr { + if tst.wantError == "" { require.NoError(t, err) - } - if tst.wantErr { - require.NotNil(t, err) + } else { + require.EqualError(t, err, tst.wantError) } }) } @@ -200,24 +205,24 @@ func TestGCSExtract(t *testing.T) { func TestS3Extract(t *testing.T) { type test struct { - name string - secret *corev1.Secret - wantErr bool + name string + secret *corev1.Secret + wantError string } table := []test{ { - name: "missing endpoint", - secret: &corev1.Secret{}, - wantErr: true, + name: "missing bucketnames", + secret: &corev1.Secret{}, + wantError: "missing secret field: bucketnames", }, { - name: "missing bucketnames", + name: "missing endpoint", secret: &corev1.Secret{ Data: map[string][]byte{ - "endpoint": []byte("here"), + "bucketnames": []byte("this,that"), }, }, - wantErr: true, + wantError: "missing secret field: endpoint", }, { name: "missing access_key_id", @@ -227,7 +232,7 @@ func TestS3Extract(t *testing.T) { "bucketnames": []byte("this,that"), }, }, - wantErr: true, + wantError: "missing secret field: access_key_id", }, { name: "missing access_key_secret", @@ -238,7 +243,7 @@ func TestS3Extract(t *testing.T) { "access_key_id": []byte("id"), }, }, - wantErr: true, + wantError: "missing secret field: access_key_secret", }, { name: "unsupported SSE type", @@ -251,7 +256,7 @@ func TestS3Extract(t *testing.T) { "sse_type": []byte("unsupported"), }, }, - wantErr: true, + wantError: "unsupported SSE type (supported: SSE-KMS, SSE-S3): unsupported", }, { name: "missing SSE-KMS kms_key_id", @@ -265,7 +270,7 @@ func TestS3Extract(t *testing.T) { "sse_kms_encryption_context": []byte("kms-encryption-ctx"), }, }, - wantErr: true, + wantError: "missing secret field: sse_kms_key_id", }, { name: "all set with SSE-KMS", @@ -330,7 +335,7 @@ func TestS3Extract(t *testing.T) { "role_arn": []byte("role"), }, }, - wantErr: true, + wantError: "missing secret field: region", }, { name: "STS with region", @@ -362,14 +367,13 @@ func TestS3Extract(t *testing.T) { t.Parallel() opts, err := extractSecrets(lokiv1.ObjectStorageSecretS3, tst.secret, nil, configv1.FeatureGates{}) - if !tst.wantErr { + if tst.wantError == "" { require.NoError(t, err) require.NotEmpty(t, opts.SecretName) require.NotEmpty(t, opts.SecretSHA1) require.Equal(t, opts.SharedStore, lokiv1.ObjectStorageSecretS3) - } - if tst.wantErr { - require.NotNil(t, err) + } else { + require.EqualError(t, err, tst.wantError) } }) } @@ -386,31 +390,48 @@ func TestS3Extract_WithOpenShiftManagedAuth(t *testing.T) { name string secret *corev1.Secret managedAuthSecret *corev1.Secret - wantErr bool + wantError string } table := []test{ { - name: "missing role-arn", + name: "missing bucketnames", secret: &corev1.Secret{}, managedAuthSecret: &corev1.Secret{}, - wantErr: true, + wantError: "missing secret field: bucketnames", }, { - name: "missing region", - secret: &corev1.Secret{}, + name: "missing region", + secret: &corev1.Secret{ + Data: map[string][]byte{ + "bucketnames": []byte("this,that"), + }, + }, managedAuthSecret: &corev1.Secret{}, - wantErr: true, + wantError: "missing secret field: region", }, { - name: "override role arn not allowed", + name: "override role_arn not allowed", secret: &corev1.Secret{ ObjectMeta: metav1.ObjectMeta{Name: "test"}, Data: map[string][]byte{ - "role_arn": []byte("role-arn"), + "bucketnames": []byte("this,that"), + "role_arn": []byte("role-arn"), + }, + }, + managedAuthSecret: &corev1.Secret{}, + wantError: "secret field not allowed: role_arn", + }, + { + name: "override audience not allowed", + secret: &corev1.Secret{ + ObjectMeta: metav1.ObjectMeta{Name: "test"}, + Data: map[string][]byte{ + "bucketnames": []byte("this,that"), + "audience": []byte("test-audience"), }, }, managedAuthSecret: &corev1.Secret{}, - wantErr: true, + wantError: "secret field not allowed: audience", }, { name: "STS all set", @@ -432,7 +453,7 @@ func TestS3Extract_WithOpenShiftManagedAuth(t *testing.T) { t.Parallel() opts, err := extractSecrets(lokiv1.ObjectStorageSecretS3, tst.secret, tst.managedAuthSecret, fg) - if !tst.wantErr { + if tst.wantError == "" { require.NoError(t, err) require.NotEmpty(t, opts.SecretName) require.NotEmpty(t, opts.SecretSHA1) @@ -441,9 +462,8 @@ func TestS3Extract_WithOpenShiftManagedAuth(t *testing.T) { require.Equal(t, opts.S3.Audience, "openshift") require.Equal(t, opts.OpenShift.CloudCredentials.SecretName, tst.managedAuthSecret.Name) require.NotEmpty(t, opts.OpenShift.CloudCredentials.SHA1) - } - if tst.wantErr { - require.NotNil(t, err) + } else { + require.EqualError(t, err, tst.wantError) } }) } @@ -451,15 +471,15 @@ func TestS3Extract_WithOpenShiftManagedAuth(t *testing.T) { func TestSwiftExtract(t *testing.T) { type test struct { - name string - secret *corev1.Secret - wantErr bool + name string + secret *corev1.Secret + wantError string } table := []test{ { - name: "missing auth_url", - secret: &corev1.Secret{}, - wantErr: true, + name: "missing auth_url", + secret: &corev1.Secret{}, + wantError: "missing secret field: auth_url", }, { name: "missing username", @@ -468,7 +488,7 @@ func TestSwiftExtract(t *testing.T) { "auth_url": []byte("here"), }, }, - wantErr: true, + wantError: "missing secret field: username", }, { name: "missing user_domain_name", @@ -478,7 +498,7 @@ func TestSwiftExtract(t *testing.T) { "username": []byte("this,that"), }, }, - wantErr: true, + wantError: "missing secret field: user_domain_name", }, { name: "missing user_domain_id", @@ -489,7 +509,7 @@ func TestSwiftExtract(t *testing.T) { "user_domain_name": []byte("id"), }, }, - wantErr: true, + wantError: "missing secret field: user_domain_id", }, { name: "missing user_id", @@ -501,7 +521,7 @@ func TestSwiftExtract(t *testing.T) { "user_domain_id": []byte("secret"), }, }, - wantErr: true, + wantError: "missing secret field: user_id", }, { name: "missing password", @@ -514,7 +534,7 @@ func TestSwiftExtract(t *testing.T) { "user_id": []byte("there"), }, }, - wantErr: true, + wantError: "missing secret field: password", }, { name: "missing domain_id", @@ -528,7 +548,7 @@ func TestSwiftExtract(t *testing.T) { "password": []byte("cred"), }, }, - wantErr: true, + wantError: "missing secret field: domain_id", }, { name: "missing domain_name", @@ -543,7 +563,7 @@ func TestSwiftExtract(t *testing.T) { "domain_id": []byte("text"), }, }, - wantErr: true, + wantError: "missing secret field: domain_name", }, { name: "missing container_name", @@ -559,7 +579,7 @@ func TestSwiftExtract(t *testing.T) { "domain_name": []byte("where"), }, }, - wantErr: true, + wantError: "missing secret field: container_name", }, { name: "all set", @@ -585,14 +605,13 @@ func TestSwiftExtract(t *testing.T) { t.Parallel() opts, err := extractSecrets(lokiv1.ObjectStorageSecretSwift, tst.secret, nil, configv1.FeatureGates{}) - if !tst.wantErr { + if tst.wantError == "" { require.NoError(t, err) require.NotEmpty(t, opts.SecretName) require.NotEmpty(t, opts.SecretSHA1) require.Equal(t, opts.SharedStore, lokiv1.ObjectStorageSecretSwift) - } - if tst.wantErr { - require.NotNil(t, err) + } else { + require.EqualError(t, err, tst.wantError) } }) } @@ -600,24 +619,24 @@ func TestSwiftExtract(t *testing.T) { func TestAlibabaCloudExtract(t *testing.T) { type test struct { - name string - secret *corev1.Secret - wantErr bool + name string + secret *corev1.Secret + wantError string } table := []test{ { - name: "missing endpoint", - secret: &corev1.Secret{}, - wantErr: true, + name: "missing endpoint", + secret: &corev1.Secret{}, + wantError: "missing secret field: endpoint", }, { - name: "missing bucketnames", + name: "missing bucket", secret: &corev1.Secret{ Data: map[string][]byte{ "endpoint": []byte("here"), }, }, - wantErr: true, + wantError: "missing secret field: bucket", }, { name: "missing access_key_id", @@ -627,10 +646,10 @@ func TestAlibabaCloudExtract(t *testing.T) { "bucket": []byte("this,that"), }, }, - wantErr: true, + wantError: "missing secret field: access_key_id", }, { - name: "missing access_key_secret", + name: "missing secret_access_key", secret: &corev1.Secret{ Data: map[string][]byte{ "endpoint": []byte("here"), @@ -638,7 +657,7 @@ func TestAlibabaCloudExtract(t *testing.T) { "access_key_id": []byte("id"), }, }, - wantErr: true, + wantError: "missing secret field: secret_access_key", }, { name: "all set", @@ -659,14 +678,13 @@ func TestAlibabaCloudExtract(t *testing.T) { t.Parallel() opts, err := extractSecrets(lokiv1.ObjectStorageSecretAlibabaCloud, tst.secret, nil, configv1.FeatureGates{}) - if !tst.wantErr { + if tst.wantError == "" { require.NoError(t, err) require.NotEmpty(t, opts.SecretName) require.NotEmpty(t, opts.SecretSHA1) require.Equal(t, opts.SharedStore, lokiv1.ObjectStorageSecretAlibabaCloud) - } - if tst.wantErr { - require.NotNil(t, err) + } 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 9bc73630b2dc2..9e041bf99a23a 100644 --- a/operator/internal/handlers/internal/storage/storage_test.go +++ b/operator/internal/handlers/internal/storage/storage_test.go @@ -293,7 +293,7 @@ func TestBuildOptions_WhenInvalidSecret_SetDegraded(t *testing.T) { } degradedErr := &status.DegradedError{ - Message: "Invalid object storage secret contents: missing secret field", + Message: "Invalid object storage secret contents: missing secret field: bucketnames", Reason: lokiv1.ReasonInvalidObjectStorageSecret, Requeue: false, } From 3b7ac359bc1b5157bd75747b8058999075b3f10b Mon Sep 17 00:00:00 2001 From: Dylan Guedes Date: Tue, 30 Jan 2024 17:21:38 -0300 Subject: [PATCH 09/45] Spans: Stop emitting `GetChunks` and `IndexClient.GetChunkRefs` (#11825) **What this PR does / why we need it**: Modify our code to not emit the `GetChunks` and the `IndexClient.GetChunkRefs` spans. We don't worse our observability by getting rid of them because their data is redundant with parent spans. Example: - If an specific index-gateway or ingester struggles, the parent span `/logproto.Querier/GetChunkIDs` will report it just fine - If an specific query causes trouble the `query.Exec` span will show the query name, the `start` and `end`, etc. Also, its children will show the slower parts of the query (in case we have a weak link) **Which issue(s) this PR fixes**: N/A --- pkg/storage/stores/composite_store_entry.go | 10 ---------- .../shipper/indexshipper/tsdb/index_client.go | 19 ------------------- 2 files changed, 29 deletions(-) diff --git a/pkg/storage/stores/composite_store_entry.go b/pkg/storage/stores/composite_store_entry.go index 56f9137e502d7..7edbdab404fe6 100644 --- a/pkg/storage/stores/composite_store_entry.go +++ b/pkg/storage/stores/composite_store_entry.go @@ -46,18 +46,8 @@ func (c *storeEntry) GetChunks(ctx context.Context, userID string, from, through if ctx.Err() != nil { return nil, nil, ctx.Err() } - sp, ctx := opentracing.StartSpanFromContext(ctx, "GetChunks") - defer sp.Finish() - log := spanlogger.FromContext(ctx) - defer log.Span.Finish() shortcut, err := c.validateQueryTimeRange(ctx, userID, &from, &through) - level.Debug(log).Log( - "shortcut", shortcut, - "from", from.Time(), - "through", through.Time(), - "err", err, - ) if err != nil { return nil, nil, err } else if shortcut { diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index_client.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index_client.go index be27e9038d017..fae628a3a9078 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index_client.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index_client.go @@ -105,32 +105,13 @@ func cleanMatchers(matchers ...*labels.Matcher) ([]*labels.Matcher, *index.Shard // They share almost the same fields, so we can add the missing `KB` field to the proto and then // use that within the tsdb package. func (c *IndexClient) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, predicate chunk.Predicate) ([]logproto.ChunkRef, error) { - sp, ctx := opentracing.StartSpanFromContext(ctx, "IndexClient.GetChunkRefs") - defer sp.Finish() - - var kvps []interface{} - defer func() { - sp.LogKV(kvps...) - }() - matchers, shard, err := cleanMatchers(predicate.Matchers...) - kvps = append(kvps, - "from", from.Time(), - "through", through.Time(), - "matchers", syntax.MatchersString(matchers), - "shard", shard, - "cleanMatcherErr", err, - ) if err != nil { return nil, err } // TODO(owen-d): use a pool to reduce allocs here chks, err := c.idx.GetChunkRefs(ctx, userID, from, through, nil, shard, matchers...) - kvps = append(kvps, - "chunks", len(chks), - "indexErr", err, - ) if err != nil { return nil, err } From d4e359dea84015ea540ede840b79b99640b46b9f Mon Sep 17 00:00:00 2001 From: Paul Rogers <129207811+paul1r@users.noreply.github.com> Date: Tue, 30 Jan 2024 15:45:01 -0500 Subject: [PATCH 10/45] Change distributor latency to have a regular expression equals (#11827) Currently, the distributor latency panel uses a "=" in the code as opposed to a "=~", causing the panel to render as empty. This is to fix that issue. **What this PR does / why we need it**: **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) --- .../loki-mixin-compiled-ssd/dashboards/loki-writes.json | 6 +++--- production/loki-mixin-compiled/dashboards/loki-writes.json | 6 +++--- production/loki-mixin/dashboards/loki-writes.libsonnet | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/production/loki-mixin-compiled-ssd/dashboards/loki-writes.json b/production/loki-mixin-compiled-ssd/dashboards/loki-writes.json index 9d2544082d158..317d8a06bb67c 100644 --- a/production/loki-mixin-compiled-ssd/dashboards/loki-writes.json +++ b/production/loki-mixin-compiled-ssd/dashboards/loki-writes.json @@ -142,7 +142,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum by (le) (cluster_job_route:loki_request_duration_seconds_bucket:sum_rate{cluster=~\"$cluster\", job=~\"($namespace)/(loki|enterprise-logs)-write\", route=\"api_prom_push|loki_api_v1_push|/httpgrpc.HTTP/Handle\"})) * 1e3", + "expr": "histogram_quantile(0.99, sum by (le) (cluster_job_route:loki_request_duration_seconds_bucket:sum_rate{cluster=~\"$cluster\", job=~\"($namespace)/(loki|enterprise-logs)-write\", route=~\"api_prom_push|loki_api_v1_push|/httpgrpc.HTTP/Handle\"})) * 1e3", "format": "time_series", "intervalFactor": 2, "legendFormat": "99th Percentile", @@ -150,7 +150,7 @@ "step": 10 }, { - "expr": "histogram_quantile(0.50, sum by (le) (cluster_job_route:loki_request_duration_seconds_bucket:sum_rate{cluster=~\"$cluster\", job=~\"($namespace)/(loki|enterprise-logs)-write\", route=\"api_prom_push|loki_api_v1_push|/httpgrpc.HTTP/Handle\"})) * 1e3", + "expr": "histogram_quantile(0.50, sum by (le) (cluster_job_route:loki_request_duration_seconds_bucket:sum_rate{cluster=~\"$cluster\", job=~\"($namespace)/(loki|enterprise-logs)-write\", route=~\"api_prom_push|loki_api_v1_push|/httpgrpc.HTTP/Handle\"})) * 1e3", "format": "time_series", "intervalFactor": 2, "legendFormat": "50th Percentile", @@ -158,7 +158,7 @@ "step": 10 }, { - "expr": "1e3 * sum(cluster_job_route:loki_request_duration_seconds_sum:sum_rate{cluster=~\"$cluster\", job=~\"($namespace)/(loki|enterprise-logs)-write\", route=\"api_prom_push|loki_api_v1_push|/httpgrpc.HTTP/Handle\"}) / sum(cluster_job_route:loki_request_duration_seconds_count:sum_rate{cluster=~\"$cluster\", job=~\"($namespace)/(loki|enterprise-logs)-write\", route=\"api_prom_push|loki_api_v1_push|/httpgrpc.HTTP/Handle\"})", + "expr": "1e3 * sum(cluster_job_route:loki_request_duration_seconds_sum:sum_rate{cluster=~\"$cluster\", job=~\"($namespace)/(loki|enterprise-logs)-write\", route=~\"api_prom_push|loki_api_v1_push|/httpgrpc.HTTP/Handle\"}) / sum(cluster_job_route:loki_request_duration_seconds_count:sum_rate{cluster=~\"$cluster\", job=~\"($namespace)/(loki|enterprise-logs)-write\", route=~\"api_prom_push|loki_api_v1_push|/httpgrpc.HTTP/Handle\"})", "format": "time_series", "intervalFactor": 2, "legendFormat": "Average", diff --git a/production/loki-mixin-compiled/dashboards/loki-writes.json b/production/loki-mixin-compiled/dashboards/loki-writes.json index b7cf83f95f44b..44528b02c5531 100644 --- a/production/loki-mixin-compiled/dashboards/loki-writes.json +++ b/production/loki-mixin-compiled/dashboards/loki-writes.json @@ -142,7 +142,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum by (le) (cluster_job_route:loki_request_duration_seconds_bucket:sum_rate{cluster=~\"$cluster\", job=~\"($namespace)/distributor\", route=\"api_prom_push|loki_api_v1_push|/httpgrpc.HTTP/Handle\"})) * 1e3", + "expr": "histogram_quantile(0.99, sum by (le) (cluster_job_route:loki_request_duration_seconds_bucket:sum_rate{cluster=~\"$cluster\", job=~\"($namespace)/distributor\", route=~\"api_prom_push|loki_api_v1_push|/httpgrpc.HTTP/Handle\"})) * 1e3", "format": "time_series", "intervalFactor": 2, "legendFormat": "99th Percentile", @@ -150,7 +150,7 @@ "step": 10 }, { - "expr": "histogram_quantile(0.50, sum by (le) (cluster_job_route:loki_request_duration_seconds_bucket:sum_rate{cluster=~\"$cluster\", job=~\"($namespace)/distributor\", route=\"api_prom_push|loki_api_v1_push|/httpgrpc.HTTP/Handle\"})) * 1e3", + "expr": "histogram_quantile(0.50, sum by (le) (cluster_job_route:loki_request_duration_seconds_bucket:sum_rate{cluster=~\"$cluster\", job=~\"($namespace)/distributor\", route=~\"api_prom_push|loki_api_v1_push|/httpgrpc.HTTP/Handle\"})) * 1e3", "format": "time_series", "intervalFactor": 2, "legendFormat": "50th Percentile", @@ -158,7 +158,7 @@ "step": 10 }, { - "expr": "1e3 * sum(cluster_job_route:loki_request_duration_seconds_sum:sum_rate{cluster=~\"$cluster\", job=~\"($namespace)/distributor\", route=\"api_prom_push|loki_api_v1_push|/httpgrpc.HTTP/Handle\"}) / sum(cluster_job_route:loki_request_duration_seconds_count:sum_rate{cluster=~\"$cluster\", job=~\"($namespace)/distributor\", route=\"api_prom_push|loki_api_v1_push|/httpgrpc.HTTP/Handle\"})", + "expr": "1e3 * sum(cluster_job_route:loki_request_duration_seconds_sum:sum_rate{cluster=~\"$cluster\", job=~\"($namespace)/distributor\", route=~\"api_prom_push|loki_api_v1_push|/httpgrpc.HTTP/Handle\"}) / sum(cluster_job_route:loki_request_duration_seconds_count:sum_rate{cluster=~\"$cluster\", job=~\"($namespace)/distributor\", route=~\"api_prom_push|loki_api_v1_push|/httpgrpc.HTTP/Handle\"})", "format": "time_series", "intervalFactor": 2, "legendFormat": "Average", diff --git a/production/loki-mixin/dashboards/loki-writes.libsonnet b/production/loki-mixin/dashboards/loki-writes.libsonnet index d5c85337a29db..878a1ee7d7872 100644 --- a/production/loki-mixin/dashboards/loki-writes.libsonnet +++ b/production/loki-mixin/dashboards/loki-writes.libsonnet @@ -65,7 +65,7 @@ local utils = import 'mixin-utils/utils.libsonnet'; $.panel('Latency') + utils.latencyRecordingRulePanel( 'loki_request_duration_seconds', - dashboards['loki-writes.json'].clusterMatchers + dashboards['loki-writes.json'].matchers.distributor + [utils.selector.eq('route', 'api_prom_push|loki_api_v1_push|/httpgrpc.HTTP/Handle')], + dashboards['loki-writes.json'].clusterMatchers + dashboards['loki-writes.json'].matchers.distributor + [utils.selector.re('route', 'api_prom_push|loki_api_v1_push|/httpgrpc.HTTP/Handle')], ) ) ) From 5675baecbee2affb2bf05bb44f1c8c0516bc002c Mon Sep 17 00:00:00 2001 From: Yogev Levy <42409341+YogevLevy93@users.noreply.github.com> Date: Wed, 31 Jan 2024 11:20:37 +0200 Subject: [PATCH 11/45] Adding annotations support for statefulset-read.yaml (#11690) Adding fixing lake of annotations support for `read` StatefulSet **What this PR does / why we need it**: Currently there is no use for `read.annotations` at the `statefulset-read.yaml` file. The key exists on `values.yaml` file, with no use. **Which issue(s) this PR fixes**: Fixes #11688 **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 - [ ] `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) --------- Co-authored-by: Michel Hollands <42814411+MichelHollands@users.noreply.github.com> --- production/helm/loki/CHANGELOG.md | 4 ++++ production/helm/loki/Chart.yaml | 2 +- production/helm/loki/README.md | 2 +- .../helm/loki/templates/read/statefulset-read.yaml | 9 +++++++++ 4 files changed, 15 insertions(+), 2 deletions(-) diff --git a/production/helm/loki/CHANGELOG.md b/production/helm/loki/CHANGELOG.md index af81afff2a047..f70a5f1183003 100644 --- a/production/helm/loki/CHANGELOG.md +++ b/production/helm/loki/CHANGELOG.md @@ -13,6 +13,10 @@ Entries should include a reference to the pull request that introduced the chang [//]: # ( : do not remove this line. This locator is used by the CI pipeline to automatically create a changelog entry for each new Loki release. Add other chart versions and respective changelog entries bellow this line.) +## 5.42.1 + +- [BUGFIX] Added missing annotations to loki-read StatefulSet. + ## 5.42.0 - [CHANGE] Changed versions of Loki v2.9.4 and GEL v1.8.6 diff --git a/production/helm/loki/Chart.yaml b/production/helm/loki/Chart.yaml index 0f708caf4eac8..3217cb5988b95 100644 --- a/production/helm/loki/Chart.yaml +++ b/production/helm/loki/Chart.yaml @@ -3,7 +3,7 @@ name: loki description: Helm chart for Grafana Loki in simple, scalable mode type: application appVersion: 2.9.4 -version: 5.42.0 +version: 5.42.1 home: https://grafana.github.io/helm-charts sources: - https://github.com/grafana/loki diff --git a/production/helm/loki/README.md b/production/helm/loki/README.md index 8f3ab16f53672..ec53471e5cdcc 100644 --- a/production/helm/loki/README.md +++ b/production/helm/loki/README.md @@ -1,6 +1,6 @@ # loki -![Version: 5.42.0](https://img.shields.io/badge/Version-5.42.0-informational?style=flat-square) ![Type: application](https://img.shields.io/badge/Type-application-informational?style=flat-square) ![AppVersion: 2.9.4](https://img.shields.io/badge/AppVersion-2.9.4-informational?style=flat-square) +![Version: 5.42.1](https://img.shields.io/badge/Version-5.42.1-informational?style=flat-square) ![Type: application](https://img.shields.io/badge/Type-application-informational?style=flat-square) ![AppVersion: 2.9.4](https://img.shields.io/badge/AppVersion-2.9.4-informational?style=flat-square) Helm chart for Grafana Loki in simple, scalable mode diff --git a/production/helm/loki/templates/read/statefulset-read.yaml b/production/helm/loki/templates/read/statefulset-read.yaml index 3b0d91d926a3c..2f16179a56c43 100644 --- a/production/helm/loki/templates/read/statefulset-read.yaml +++ b/production/helm/loki/templates/read/statefulset-read.yaml @@ -9,6 +9,15 @@ metadata: labels: app.kubernetes.io/part-of: memberlist {{- include "loki.readLabels" . | nindent 4 }} + {{- if or (not (empty .Values.loki.annotations)) (not (empty .Values.read.annotations))}} + annotations: + {{- with .Values.loki.annotations }} + {{- toYaml . | nindent 4 }} + {{- end }} + {{- with .Values.read.annotations }} + {{- toYaml . | nindent 4 }} + {{- end }} + {{- end }} spec: {{- if not .Values.read.autoscaling.enabled }} replicas: {{ .Values.read.replicas }} From 6902130c9a269cba9f622b76e973accf9f9d8395 Mon Sep 17 00:00:00 2001 From: Christian Haudum Date: Wed, 31 Jan 2024 11:53:57 +0100 Subject: [PATCH 12/45] Bloom shipper: Restructure bloom store (#11828) **What this PR does / why we need it**: This PR changes the structure of how the bloom client * The `BloomStore` is the top component interface which implements the `Store` and the `Client` interfaces. The store holds a store entry for each schema period. * The `bloomStoreEntry` implements the `Store` and `Client` interfaces. It holds a bloom client for a single schema period. Additionally, the store entry also exposes a fetcher for metas (and in the future for blocks), which is responsible for getting data from cache or storage (if not available in cache). * The `BloomClient` implement the `Client` interface. The bloom client uses an object client for the schema period of the bloom client. * The `Fetcher` can fetch and cache metas. This structure is very similar to what we use for the chunk store. **Note** Before implementing `FetchBlocks()` in the `BloomStore`, I want to implement the new `FingerprintBounds` type also in the shipper and bloom gateway code to be consistent across components, as well as to make use of the new utilities exposed by them. The store implementation probably needs some specific test cases around using the correct store entry for different schema period configs. The code however is mostly taken from the chunk store implementation. --------- Signed-off-by: Christian Haudum --- pkg/bloomcompactor/bloomcompactor.go | 20 +- pkg/bloomgateway/bloomgateway.go | 9 +- pkg/bloomgateway/processor.go | 8 +- pkg/bloomgateway/processor_test.go | 18 +- .../stores/shipper/bloomshipper/client.go | 187 +++----- .../shipper/bloomshipper/client_test.go | 97 ++--- .../stores/shipper/bloomshipper/fetcher.go | 178 ++++++++ .../shipper/bloomshipper/fetcher_test.go | 176 ++++++++ .../stores/shipper/bloomshipper/shipper.go | 16 +- .../stores/shipper/bloomshipper/store.go | 404 ++++++++++++++++++ 10 files changed, 899 insertions(+), 214 deletions(-) create mode 100644 pkg/storage/stores/shipper/bloomshipper/fetcher.go create mode 100644 pkg/storage/stores/shipper/bloomshipper/fetcher_test.go create mode 100644 pkg/storage/stores/shipper/bloomshipper/store.go diff --git a/pkg/bloomcompactor/bloomcompactor.go b/pkg/bloomcompactor/bloomcompactor.go index 5cb64a350d161..52799e498b51e 100644 --- a/pkg/bloomcompactor/bloomcompactor.go +++ b/pkg/bloomcompactor/bloomcompactor.go @@ -27,6 +27,7 @@ package bloomcompactor import ( "context" "fmt" + "io" "math" "math/rand" "os" @@ -50,6 +51,7 @@ import ( "github.com/grafana/loki/pkg/bloomutils" "github.com/grafana/loki/pkg/storage" v1 "github.com/grafana/loki/pkg/storage/bloom/v1" + "github.com/grafana/loki/pkg/storage/chunk/cache" chunk_client "github.com/grafana/loki/pkg/storage/chunk/client" "github.com/grafana/loki/pkg/storage/chunk/client/local" "github.com/grafana/loki/pkg/storage/config" @@ -71,7 +73,7 @@ type Compactor struct { limits Limits // temporary workaround until store has implemented read/write shipper interface - bloomShipperClient bloomshipper.Client + bloomShipperClient bloomshipper.StoreAndClient // Client used to run operations on the bucket storing bloom blocks. storeClients map[config.DayTime]storeClient @@ -109,8 +111,10 @@ func New( reg: r, } - // Configure BloomClient for meta.json management - bloomClient, err := bloomshipper.NewBloomClient(schemaConfig.Configs, storageCfg, clientMetrics) + // TODO(chaudum): Plug in cache + var metasCache cache.Cache + var blocksCache *cache.EmbeddedCache[string, io.ReadCloser] + bloomClient, err := bloomshipper.NewBloomStore(schemaConfig.Configs, storageCfg, clientMetrics, metasCache, blocksCache, logger) if err != nil { return nil, err } @@ -506,11 +510,19 @@ func (c *Compactor) runCompact(ctx context.Context, logger log.Logger, job Job, //TODO Configure pool for these to avoid allocations var activeBloomBlocksRefs []bloomshipper.BlockRef - metas, err := c.bloomShipperClient.GetMetas(ctx, metaSearchParams) + metaRefs, fetchers, err := c.bloomShipperClient.ResolveMetas(ctx, metaSearchParams) if err != nil { return err } + for i := range fetchers { + res, err := fetchers[i].FetchMetas(ctx, metaRefs[i]) + if err != nil { + return err + } + metas = append(metas, res...) + } + // TODO This logic currently is NOT concerned with cutting blocks upon topology changes to bloom-compactors. // It may create blocks with series outside of the fp range of the compactor. Cutting blocks will be addressed in a follow-up PR. metasMatchingJob, blocksMatchingJob := matchingBlocks(metas, job) diff --git a/pkg/bloomgateway/bloomgateway.go b/pkg/bloomgateway/bloomgateway.go index 3530ea0eaf993..afe8d646ae63a 100644 --- a/pkg/bloomgateway/bloomgateway.go +++ b/pkg/bloomgateway/bloomgateway.go @@ -42,6 +42,7 @@ package bloomgateway import ( "context" "fmt" + "io" "sort" "sync" "time" @@ -59,6 +60,7 @@ import ( "github.com/grafana/loki/pkg/queue" "github.com/grafana/loki/pkg/storage" v1 "github.com/grafana/loki/pkg/storage/bloom/v1" + "github.com/grafana/loki/pkg/storage/chunk/cache" "github.com/grafana/loki/pkg/storage/config" "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper" "github.com/grafana/loki/pkg/util" @@ -209,12 +211,15 @@ func New(cfg Config, schemaCfg config.SchemaConfig, storageCfg storage.Config, o g.queue = queue.NewRequestQueue(cfg.MaxOutstandingPerTenant, time.Minute, &fixedQueueLimits{0}, g.queueMetrics) g.activeUsers = util.NewActiveUsersCleanupWithDefaultValues(g.queueMetrics.Cleanup) - client, err := bloomshipper.NewBloomClient(schemaCfg.Configs, storageCfg, cm) + // TODO(chaudum): Plug in cache + var metasCache cache.Cache + var blocksCache *cache.EmbeddedCache[string, io.ReadCloser] + store, err := bloomshipper.NewBloomStore(schemaCfg.Configs, storageCfg, cm, metasCache, blocksCache, logger) if err != nil { return nil, err } - bloomShipper, err := bloomshipper.NewShipper(client, storageCfg.BloomShipperConfig, overrides, logger, reg) + bloomShipper, err := bloomshipper.NewShipper(store, storageCfg.BloomShipperConfig, overrides, logger, reg) if err != nil { return nil, err } diff --git a/pkg/bloomgateway/processor.go b/pkg/bloomgateway/processor.go index a2f63b3e5e155..460ac3f44e038 100644 --- a/pkg/bloomgateway/processor.go +++ b/pkg/bloomgateway/processor.go @@ -17,17 +17,13 @@ type tasksForBlock struct { tasks []Task } -type metaLoader interface { - LoadMetas(context.Context, bloomshipper.MetaSearchParams) ([]bloomshipper.Meta, error) -} - type blockLoader interface { LoadBlocks(context.Context, []bloomshipper.BlockRef) (v1.Iterator[bloomshipper.BlockQuerierWithFingerprintRange], error) } type store interface { blockLoader - metaLoader + bloomshipper.Store } type processor struct { @@ -63,7 +59,7 @@ func (p *processor) processTasks(ctx context.Context, tenant string, interval bl Interval: interval, Keyspace: bloomshipper.Keyspace{Min: minFpRange.Min, Max: maxFpRange.Max}, } - metas, err := p.store.LoadMetas(ctx, metaSearch) + metas, err := p.store.FetchMetas(ctx, metaSearch) if err != nil { return err } diff --git a/pkg/bloomgateway/processor_test.go b/pkg/bloomgateway/processor_test.go index f078c31c06d32..62c6d42ae18b3 100644 --- a/pkg/bloomgateway/processor_test.go +++ b/pkg/bloomgateway/processor_test.go @@ -24,11 +24,27 @@ type dummyStore struct { querieres []bloomshipper.BlockQuerierWithFingerprintRange } -func (s *dummyStore) LoadMetas(_ context.Context, _ bloomshipper.MetaSearchParams) ([]bloomshipper.Meta, error) { +func (s *dummyStore) ResolveMetas(_ context.Context, _ bloomshipper.MetaSearchParams) ([][]bloomshipper.MetaRef, []*bloomshipper.Fetcher, error) { + //TODO(chaudum) Filter metas based on search params + refs := make([]bloomshipper.MetaRef, 0, len(s.metas)) + for _, meta := range s.metas { + refs = append(refs, meta.MetaRef) + } + return [][]bloomshipper.MetaRef{refs}, []*bloomshipper.Fetcher{nil}, nil +} + +func (s *dummyStore) FetchMetas(_ context.Context, _ bloomshipper.MetaSearchParams) ([]bloomshipper.Meta, error) { //TODO(chaudum) Filter metas based on search params return s.metas, nil } +func (s *dummyStore) Fetcher(_ model.Time) *bloomshipper.Fetcher { + return nil +} + +func (s *dummyStore) Stop() { +} + func (s *dummyStore) LoadBlocks(_ context.Context, refs []bloomshipper.BlockRef) (v1.Iterator[bloomshipper.BlockQuerierWithFingerprintRange], error) { result := make([]bloomshipper.BlockQuerierWithFingerprintRange, len(s.querieres)) diff --git a/pkg/storage/stores/shipper/bloomshipper/client.go b/pkg/storage/stores/shipper/bloomshipper/client.go index 5c77f6fd9f636..8d980782d044f 100644 --- a/pkg/storage/stores/shipper/bloomshipper/client.go +++ b/pkg/storage/stores/shipper/bloomshipper/client.go @@ -6,19 +6,17 @@ import ( "encoding/json" "fmt" "io" - "path/filepath" + "path" "strconv" "strings" - "time" + "github.com/go-kit/log" "github.com/grafana/dskit/concurrency" "github.com/prometheus/common/model" - "github.com/grafana/loki/pkg/storage" v1 "github.com/grafana/loki/pkg/storage/bloom/v1" "github.com/grafana/loki/pkg/storage/chunk/client" "github.com/grafana/loki/pkg/storage/config" - "github.com/grafana/loki/pkg/util/math" ) const ( @@ -75,7 +73,7 @@ type MetaSearchParams struct { type MetaClient interface { // Returns all metas that are within MinFingerprint-MaxFingerprint fingerprint range // and intersect time period from StartTimestamp to EndTimestamp. - GetMetas(ctx context.Context, params MetaSearchParams) ([]Meta, error) + GetMetas(ctx context.Context, metas []MetaRef) ([]Meta, error) PutMeta(ctx context.Context, meta Meta) error DeleteMeta(ctx context.Context, meta Meta) error } @@ -91,7 +89,7 @@ type Block struct { } type BlockClient interface { - GetBlock(ctx context.Context, reference BlockRef) (LazyBlock, error) + GetBlock(ctx context.Context, ref BlockRef) (LazyBlock, error) PutBlocks(ctx context.Context, blocks []Block) ([]Block, error) DeleteBlocks(ctx context.Context, blocks []BlockRef) error } @@ -102,84 +100,41 @@ type Client interface { Stop() } -// todo add logger -func NewBloomClient(periodicConfigs []config.PeriodConfig, storageConfig storage.Config, clientMetrics storage.ClientMetrics) (*BloomClient, error) { - periodicObjectClients := make(map[config.DayTime]client.ObjectClient) - for _, periodicConfig := range periodicConfigs { - objectClient, err := storage.NewObjectClient(periodicConfig.ObjectType, storageConfig, clientMetrics) - if err != nil { - return nil, fmt.Errorf("error creating object client '%s': %w", periodicConfig.ObjectType, err) - } - periodicObjectClients[periodicConfig.From] = objectClient - } - return &BloomClient{ - periodicConfigs: periodicConfigs, - storageConfig: storageConfig, - periodicObjectClients: periodicObjectClients, - }, nil -} +// Compiler check to ensure BloomClient implements the Client interface +var _ Client = &BloomClient{} type BloomClient struct { - periodicConfigs []config.PeriodConfig - storageConfig storage.Config - periodicObjectClients map[config.DayTime]client.ObjectClient + concurrency int + client client.ObjectClient + logger log.Logger } -func (b *BloomClient) GetMetas(ctx context.Context, params MetaSearchParams) ([]Meta, error) { - tablesByPeriod := tablesByPeriod(b.periodicConfigs, params.Interval.Start, params.Interval.End) - - var metas []Meta - for periodFrom, tables := range tablesByPeriod { - periodClient := b.periodicObjectClients[periodFrom] - for _, table := range tables { - prefix := filepath.Join(rootFolder, table, params.TenantID, metasFolder) - list, _, err := periodClient.List(ctx, prefix, "") - if err != nil { - return nil, fmt.Errorf("error listing metas under prefix [%s]: %w", prefix, err) - } - for _, object := range list { - metaRef, err := createMetaRef(object.Key, params.TenantID, table) - - if err != nil { - return nil, err - } - if metaRef.MaxFingerprint < uint64(params.Keyspace.Min) || uint64(params.Keyspace.Max) < metaRef.MinFingerprint || - metaRef.EndTimestamp.Before(params.Interval.Start) || metaRef.StartTimestamp.After(params.Interval.End) { - continue - } - meta, err := b.downloadMeta(ctx, metaRef, periodClient) - if err != nil { - return nil, err - } - metas = append(metas, meta) - } - } - } - return metas, nil +func NewBloomClient(client client.ObjectClient, logger log.Logger) (*BloomClient, error) { + return &BloomClient{ + concurrency: 100, // make configurable? + client: client, + logger: logger, + }, nil } func (b *BloomClient) PutMeta(ctx context.Context, meta Meta) error { - periodFrom, err := findPeriod(b.periodicConfigs, meta.StartTimestamp) - if err != nil { - return fmt.Errorf("error updloading meta file: %w", err) - } data, err := json.Marshal(meta) if err != nil { return fmt.Errorf("can not marshal the meta to json: %w", err) } - key := createMetaObjectKey(meta.MetaRef.Ref) - return b.periodicObjectClients[periodFrom].PutObject(ctx, key, bytes.NewReader(data)) + key := externalMetaKey(meta.MetaRef) + return b.client.PutObject(ctx, key, bytes.NewReader(data)) } -func createBlockObjectKey(meta Ref) string { - blockParentFolder := fmt.Sprintf("%x-%x", meta.MinFingerprint, meta.MaxFingerprint) - filename := fmt.Sprintf("%d-%d-%x", meta.StartTimestamp, meta.EndTimestamp, meta.Checksum) - return strings.Join([]string{rootFolder, meta.TableName, meta.TenantID, bloomsFolder, blockParentFolder, filename}, delimiter) +func externalBlockKey(ref BlockRef) string { + blockParentFolder := fmt.Sprintf("%x-%x", ref.MinFingerprint, ref.MaxFingerprint) + filename := fmt.Sprintf("%d-%d-%x", ref.StartTimestamp, ref.EndTimestamp, ref.Checksum) + return path.Join(rootFolder, ref.TableName, ref.TenantID, bloomsFolder, blockParentFolder, filename) } -func createMetaObjectKey(meta Ref) string { - filename := fmt.Sprintf("%x-%x-%d-%d-%x", meta.MinFingerprint, meta.MaxFingerprint, meta.StartTimestamp, meta.EndTimestamp, meta.Checksum) - return strings.Join([]string{rootFolder, meta.TableName, meta.TenantID, metasFolder, filename}, delimiter) +func externalMetaKey(ref MetaRef) string { + filename := fmt.Sprintf("%x-%x-%d-%d-%x", ref.MinFingerprint, ref.MaxFingerprint, ref.StartTimestamp, ref.EndTimestamp, ref.Checksum) + return path.Join(rootFolder, ref.TableName, ref.TenantID, metasFolder, filename) } func findPeriod(configs []config.PeriodConfig, ts model.Time) (config.DayTime, error) { @@ -193,22 +148,13 @@ func findPeriod(configs []config.PeriodConfig, ts model.Time) (config.DayTime, e } func (b *BloomClient) DeleteMeta(ctx context.Context, meta Meta) error { - periodFrom, err := findPeriod(b.periodicConfigs, meta.StartTimestamp) - if err != nil { - return err - } - key := createMetaObjectKey(meta.MetaRef.Ref) - return b.periodicObjectClients[periodFrom].DeleteObject(ctx, key) + key := externalMetaKey(meta.MetaRef) + return b.client.DeleteObject(ctx, key) } // GetBlock downloads the blocks from objectStorage and returns the downloaded block func (b *BloomClient) GetBlock(ctx context.Context, reference BlockRef) (LazyBlock, error) { - period, err := findPeriod(b.periodicConfigs, reference.StartTimestamp) - if err != nil { - return LazyBlock{}, fmt.Errorf("error while period lookup: %w", err) - } - objectClient := b.periodicObjectClients[period] - readCloser, _, err := objectClient.GetObject(ctx, createBlockObjectKey(reference.Ref)) + readCloser, _, err := b.client.GetObject(ctx, externalBlockKey(reference)) if err != nil { return LazyBlock{}, fmt.Errorf("error while fetching object from storage: %w", err) } @@ -220,26 +166,21 @@ func (b *BloomClient) GetBlock(ctx context.Context, reference BlockRef) (LazyBlo func (b *BloomClient) PutBlocks(ctx context.Context, blocks []Block) ([]Block, error) { results := make([]Block, len(blocks)) - //todo move concurrency to the config - err := concurrency.ForEachJob(ctx, len(blocks), 100, func(ctx context.Context, idx int) error { + err := concurrency.ForEachJob(ctx, len(blocks), b.concurrency, func(ctx context.Context, idx int) error { block := blocks[idx] defer func(Data io.ReadCloser) { _ = Data.Close() }(block.Data) - period, err := findPeriod(b.periodicConfigs, block.StartTimestamp) - if err != nil { - return fmt.Errorf("error uploading block file: %w", err) - } - key := createBlockObjectKey(block.Ref) - objectClient := b.periodicObjectClients[period] + var err error + key := externalBlockKey(block.BlockRef) _, err = block.Data.Seek(0, 0) if err != nil { return fmt.Errorf("error uploading block file: %w", err) } - err = objectClient.PutObject(ctx, key, block.Data) + err = b.client.PutObject(ctx, key, block.Data) if err != nil { return fmt.Errorf("error uploading block file: %w", err) } @@ -251,16 +192,10 @@ func (b *BloomClient) PutBlocks(ctx context.Context, blocks []Block) ([]Block, e } func (b *BloomClient) DeleteBlocks(ctx context.Context, references []BlockRef) error { - //todo move concurrency to the config - return concurrency.ForEachJob(ctx, len(references), 100, func(ctx context.Context, idx int) error { + return concurrency.ForEachJob(ctx, len(references), b.concurrency, func(ctx context.Context, idx int) error { ref := references[idx] - period, err := findPeriod(b.periodicConfigs, ref.StartTimestamp) - if err != nil { - return fmt.Errorf("error deleting block file: %w", err) - } - key := createBlockObjectKey(ref.Ref) - objectClient := b.periodicObjectClients[period] - err = objectClient.DeleteObject(ctx, key) + key := externalBlockKey(ref) + err := b.client.DeleteObject(ctx, key) if err != nil { return fmt.Errorf("error deleting block file: %w", err) } @@ -269,24 +204,35 @@ func (b *BloomClient) DeleteBlocks(ctx context.Context, references []BlockRef) e } func (b *BloomClient) Stop() { - for _, objectClient := range b.periodicObjectClients { - objectClient.Stop() - } + b.client.Stop() +} + +func (b *BloomClient) GetMetas(ctx context.Context, refs []MetaRef) ([]Meta, error) { + results := make([]Meta, len(refs)) + err := concurrency.ForEachJob(ctx, len(refs), b.concurrency, func(ctx context.Context, idx int) error { + meta, err := b.getMeta(ctx, refs[idx]) + if err != nil { + return err + } + results[idx] = meta + return nil + }) + return results, err } -func (b *BloomClient) downloadMeta(ctx context.Context, metaRef MetaRef, client client.ObjectClient) (Meta, error) { +func (b *BloomClient) getMeta(ctx context.Context, ref MetaRef) (Meta, error) { meta := Meta{ - MetaRef: metaRef, + MetaRef: ref, } - reader, _, err := client.GetObject(ctx, metaRef.FilePath) + reader, _, err := b.client.GetObject(ctx, ref.FilePath) if err != nil { - return Meta{}, fmt.Errorf("error downloading meta file %s : %w", metaRef.FilePath, err) + return Meta{}, fmt.Errorf("error downloading meta file %s : %w", ref.FilePath, err) } defer reader.Close() err = json.NewDecoder(reader).Decode(&meta) if err != nil { - return Meta{}, fmt.Errorf("error unmarshalling content of meta file %s: %w", metaRef.FilePath, err) + return Meta{}, fmt.Errorf("error unmarshalling content of meta file %s: %w", ref.FilePath, err) } return meta, nil } @@ -332,32 +278,11 @@ func createMetaRef(objectKey string, tenantID string, tableName string) (MetaRef }, nil } -func tablesByPeriod(periodicConfigs []config.PeriodConfig, start, end model.Time) map[config.DayTime][]string { - result := make(map[config.DayTime][]string) - for i := len(periodicConfigs) - 1; i >= 0; i-- { - periodConfig := periodicConfigs[i] - if end.Before(periodConfig.From.Time) { - continue - } - owningPeriodStartTs := math.Max64(periodConfig.From.Unix(), start.Unix()) - owningPeriodEndTs := end.Unix() - if i != len(periodicConfigs)-1 { - nextPeriodConfig := periodicConfigs[i+1] - owningPeriodEndTs = math.Min64(nextPeriodConfig.From.Add(-1*time.Second).Unix(), owningPeriodEndTs) - } - result[periodConfig.From] = tablesForRange(periodicConfigs[i], owningPeriodStartTs, owningPeriodEndTs) - if !start.Before(periodConfig.From.Time) { - break - } - } - return result -} - -func tablesForRange(periodConfig config.PeriodConfig, from, to int64) []string { +func tablesForRange(periodConfig config.PeriodConfig, from, to model.Time) []string { interval := periodConfig.IndexTables.Period step := int64(interval.Seconds()) - lower := from / step - upper := to / step + lower := from.Unix() / step + upper := to.Unix() / step tables := make([]string, 0, 1+upper-lower) prefix := periodConfig.IndexTables.Prefix for i := lower; i <= upper; i++ { diff --git a/pkg/storage/stores/shipper/bloomshipper/client_test.go b/pkg/storage/stores/shipper/bloomshipper/client_test.go index 1cd5f562362ce..28e8c3a02f0e3 100644 --- a/pkg/storage/stores/shipper/bloomshipper/client_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/client_test.go @@ -14,11 +14,13 @@ import ( "time" awsio "github.com/aws/smithy-go/io" + "github.com/go-kit/log" "github.com/google/uuid" "github.com/prometheus/common/model" "github.com/stretchr/testify/require" "github.com/grafana/loki/pkg/storage" + "github.com/grafana/loki/pkg/storage/chunk/cache" "github.com/grafana/loki/pkg/storage/config" ) @@ -46,11 +48,11 @@ func parseDayTime(s string) config.DayTime { } } -func Test_BloomClient_GetMetas(t *testing.T) { - shipper := createClient(t) +func Test_BloomClient_FetchMetas(t *testing.T) { + store := createStore(t) var expected []Meta - folder1 := shipper.storageConfig.NamedStores.Filesystem["folder-1"].Directory + folder1 := store.storageConfig.NamedStores.Filesystem["folder-1"].Directory // must not be present in results because it is outside of time range createMetaInStorage(t, folder1, "first-period-19621", "tenantA", 0, 100, fixedDay.Add(-7*day)) // must be present in the results @@ -60,7 +62,7 @@ func Test_BloomClient_GetMetas(t *testing.T) { // must be present in the results expected = append(expected, createMetaInStorage(t, folder1, "first-period-19621", "tenantA", 101, 200, fixedDay.Add(-6*day))) - folder2 := shipper.storageConfig.NamedStores.Filesystem["folder-2"].Directory + folder2 := store.storageConfig.NamedStores.Filesystem["folder-2"].Directory // must not be present in results because it's out of the time range createMetaInStorage(t, folder2, "second-period-19626", "tenantA", 0, 100, fixedDay.Add(-1*day)) // must be present in the results @@ -68,13 +70,28 @@ func Test_BloomClient_GetMetas(t *testing.T) { // must not be present in results because it belongs to another tenant createMetaInStorage(t, folder2, "second-period-19624", "tenantB", 0, 100, fixedDay.Add(-3*day)) - actual, err := shipper.GetMetas(context.Background(), MetaSearchParams{ + searchParams := MetaSearchParams{ TenantID: "tenantA", Keyspace: Keyspace{Min: 50, Max: 150}, Interval: Interval{Start: fixedDay.Add(-6 * day), End: fixedDay.Add(-1*day - 1*time.Hour)}, - }) + } + + fetched, err := store.FetchMetas(context.Background(), searchParams) require.NoError(t, err) - require.ElementsMatch(t, expected, actual) + + require.Equal(t, len(expected), len(fetched)) + require.ElementsMatch(t, expected, fetched) + + resolved, _, err := store.ResolveMetas(context.Background(), searchParams) + require.NoError(t, err) + + var resolvedRefs []MetaRef + for _, refs := range resolved { + resolvedRefs = append(resolvedRefs, refs...) + } + for i := range resolvedRefs { + require.Equal(t, fetched[i].MetaRef, resolvedRefs[i]) + } } func Test_BloomClient_PutMeta(t *testing.T) { @@ -112,7 +129,7 @@ func Test_BloomClient_PutMeta(t *testing.T) { } for name, data := range tests { t.Run(name, func(t *testing.T) { - bloomClient := createClient(t) + bloomClient := createStore(t) err := bloomClient.PutMeta(context.Background(), data.source) require.NoError(t, err) @@ -168,7 +185,7 @@ func Test_BloomClient_DeleteMeta(t *testing.T) { } for name, data := range tests { t.Run(name, func(t *testing.T) { - bloomClient := createClient(t) + bloomClient := createStore(t) directory := bloomClient.storageConfig.NamedStores.Filesystem[data.expectedStorage].Directory file := filepath.Join(directory, data.expectedFilePath) err := os.MkdirAll(file[:strings.LastIndex(file, delimiter)], 0755) @@ -186,7 +203,7 @@ func Test_BloomClient_DeleteMeta(t *testing.T) { } func Test_BloomClient_GetBlocks(t *testing.T) { - bloomClient := createClient(t) + bloomClient := createStore(t) fsNamedStores := bloomClient.storageConfig.NamedStores.Filesystem firstBlockPath := "bloom/first-period-19621/tenantA/blooms/eeee-ffff/1695272400000-1695276000000-1" firstBlockFullPath := filepath.Join(fsNamedStores["folder-1"].Directory, firstBlockPath) @@ -236,7 +253,7 @@ func Test_BloomClient_GetBlocks(t *testing.T) { } func Test_BloomClient_PutBlocks(t *testing.T) { - bloomClient := createClient(t) + bloomClient := createStore(t) blockForFirstFolderData := "data1" blockForFirstFolder := Block{ BlockRef: BlockRef{ @@ -313,7 +330,7 @@ func Test_BloomClient_PutBlocks(t *testing.T) { } func Test_BloomClient_DeleteBlocks(t *testing.T) { - bloomClient := createClient(t) + bloomClient := createStore(t) fsNamedStores := bloomClient.storageConfig.NamedStores.Filesystem block1Path := filepath.Join(fsNamedStores["folder-1"].Directory, "bloom/first-period-19621/tenantA/blooms/eeee-ffff/1695272400000-1695276000000-1") createBlockFile(t, block1Path) @@ -363,56 +380,6 @@ func createBlockFile(t *testing.T, path string) string { return fileContent } -func Test_TablesByPeriod(t *testing.T) { - configs := createPeriodConfigs() - firstPeriodFrom := configs[0].From - secondPeriodFrom := configs[1].From - tests := map[string]struct { - from, to model.Time - expectedTablesByPeriod map[config.DayTime][]string - }{ - "expected 1 table": { - from: model.TimeFromUnix(time.Date(2023, time.September, 20, 0, 0, 0, 0, time.UTC).Unix()), - to: model.TimeFromUnix(time.Date(2023, time.September, 20, 23, 59, 59, 0, time.UTC).Unix()), - expectedTablesByPeriod: map[config.DayTime][]string{ - firstPeriodFrom: {"first-period-19620"}}, - }, - "expected tables for both periods": { - from: model.TimeFromUnix(time.Date(2023, time.September, 21, 0, 0, 0, 0, time.UTC).Unix()), - to: model.TimeFromUnix(time.Date(2023, time.September, 25, 23, 59, 59, 0, time.UTC).Unix()), - expectedTablesByPeriod: map[config.DayTime][]string{ - firstPeriodFrom: {"first-period-19621", "first-period-19622", "first-period-19623"}, - secondPeriodFrom: {"second-period-19624", "second-period-19625"}, - }, - }, - "expected tables for the second period": { - from: model.TimeFromUnix(time.Date(2023, time.September, 24, 0, 0, 0, 0, time.UTC).Unix()), - to: model.TimeFromUnix(time.Date(2023, time.September, 25, 1, 0, 0, 0, time.UTC).Unix()), - expectedTablesByPeriod: map[config.DayTime][]string{ - secondPeriodFrom: {"second-period-19624", "second-period-19625"}, - }, - }, - "expected only one table from the second period": { - from: model.TimeFromUnix(time.Date(2023, time.September, 25, 0, 0, 0, 0, time.UTC).Unix()), - to: model.TimeFromUnix(time.Date(2023, time.September, 25, 1, 0, 0, 0, time.UTC).Unix()), - expectedTablesByPeriod: map[config.DayTime][]string{ - secondPeriodFrom: {"second-period-19625"}, - }, - }, - } - for name, data := range tests { - t.Run(name, func(t *testing.T) { - result := tablesByPeriod(configs, data.from, data.to) - for periodFrom, expectedTables := range data.expectedTablesByPeriod { - actualTables, exists := result[periodFrom] - require.Truef(t, exists, "tables for %s period must be provided but was not in the result: %+v", periodFrom.String(), result) - require.ElementsMatchf(t, expectedTables, actualTables, "tables mismatch for period %s", periodFrom.String()) - } - require.Len(t, result, len(data.expectedTablesByPeriod)) - }) - } -} - func Test_createMetaRef(t *testing.T) { tests := map[string]struct { objectKey string @@ -490,7 +457,7 @@ func Test_createMetaRef(t *testing.T) { } } -func createClient(t *testing.T) *BloomClient { +func createStore(t *testing.T) *BloomStore { periodicConfigs := createPeriodConfigs() namedStores := storage.NamedStores{ Filesystem: map[string]storage.NamedFSConfig{ @@ -503,9 +470,9 @@ func createClient(t *testing.T) *BloomClient { metrics := storage.NewClientMetrics() t.Cleanup(metrics.Unregister) - bloomClient, err := NewBloomClient(periodicConfigs, storageConfig, metrics) + store, err := NewBloomStore(periodicConfigs, storageConfig, metrics, cache.NewNoopCache(), nil, log.NewNopLogger()) require.NoError(t, err) - return bloomClient + return store } func createPeriodConfigs() []config.PeriodConfig { diff --git a/pkg/storage/stores/shipper/bloomshipper/fetcher.go b/pkg/storage/stores/shipper/bloomshipper/fetcher.go new file mode 100644 index 0000000000000..0b1dae5ce691f --- /dev/null +++ b/pkg/storage/stores/shipper/bloomshipper/fetcher.go @@ -0,0 +1,178 @@ +package bloomshipper + +import ( + "context" + "encoding/json" + "io" + + "github.com/go-kit/log" + "github.com/pkg/errors" + + "github.com/grafana/loki/pkg/storage/chunk/cache" +) + +// TODO(chaudum): Add metric for cache hits/misses, and bytes stored/retrieved +type metrics struct{} + +type fetcher interface { + FetchMetas(ctx context.Context, refs []MetaRef) ([]Meta, error) + // TODO(chaudum): Integrate block fetching + // FetchBlocks(ctx context.Context, refs []BlockRef) ([]Block, error) +} + +type Fetcher struct { + client Client + + metasCache cache.Cache + blocksCache *cache.EmbeddedCache[string, io.ReadCloser] + + metrics *metrics + logger log.Logger +} + +func NewFetcher(client Client, metasCache cache.Cache, blocksCache *cache.EmbeddedCache[string, io.ReadCloser], logger log.Logger) (*Fetcher, error) { + return &Fetcher{ + client: client, + metasCache: metasCache, + blocksCache: blocksCache, + logger: logger, + }, nil +} + +func (f *Fetcher) FetchMetas(ctx context.Context, refs []MetaRef) ([]Meta, error) { + if ctx.Err() != nil { + return nil, errors.Wrap(ctx.Err(), "fetch Metas") + } + + keys := make([]string, 0, len(refs)) + for _, ref := range refs { + keys = append(keys, externalMetaKey(ref)) + } + cacheHits, cacheBufs, _, err := f.metasCache.Fetch(ctx, keys) + if err != nil { + return nil, err + } + + fromCache, missing, err := f.processCacheResponse(ctx, refs, cacheHits, cacheBufs) + if err != nil { + return nil, err + } + + fromStorage, err := f.client.GetMetas(ctx, missing) + if err != nil { + return nil, err + } + + // TODO(chaudum): Make async + err = f.writeBackMetas(ctx, fromStorage) + return append(fromCache, fromStorage...), err +} + +func (f *Fetcher) processCacheResponse(_ context.Context, refs []MetaRef, keys []string, bufs [][]byte) ([]Meta, []MetaRef, error) { + + found := make(map[string][]byte, len(refs)) + for i, k := range keys { + found[k] = bufs[i] + } + + metas := make([]Meta, 0, len(found)) + missing := make([]MetaRef, 0, len(refs)-len(keys)) + + var lastErr error + for i, ref := range refs { + if raw, ok := found[externalMetaKey(ref)]; ok { + meta := Meta{ + MetaRef: ref, + } + lastErr = json.Unmarshal(raw, &meta) + metas = append(metas, meta) + } else { + missing = append(missing, refs[i]) + } + } + + return metas, missing, lastErr +} + +func (f *Fetcher) writeBackMetas(ctx context.Context, metas []Meta) error { + var err error + keys := make([]string, len(metas)) + data := make([][]byte, len(metas)) + for i := range metas { + keys[i] = externalMetaKey(metas[i].MetaRef) + data[i], err = json.Marshal(metas[i]) + } + if err != nil { + return err + } + return f.metasCache.Store(ctx, keys, data) +} + +// TODO(chaudum): Integrate block fetching + +// func (f *Fetcher) FetchBlocks(ctx context.Context, refs []BlockRef) (v1.Iterator[Block], error) { +// if ctx.Err() != nil { +// return nil, errors.Wrap(ctx.Err(), "fetch Blocks") +// } + +// keys := make([]string, 0, len(refs)) +// for _, ref := range refs { +// keys = append(keys, externalBlockKey(ref)) +// } +// found, blocksFromCache, missing, err := f.blocksCache.Fetch(ctx, keys) +// if err != nil { +// return nil, err +// } + +// if len(missing) > 0 { +// for _, key := range missing { +// for i, ref := range refs { +// if key == externalBlockKey(ref) { +// refs = append(refs[:i], refs[i+1:]...) +// i-- +// } +// } +// } + +// blocksFromStorage, err := f.client.GetBlock(ctx, refs) +// if err != nil { +// return nil, err +// } +// } + +// return nil, nil +// } + +// func (f *Fetcher) writeBackBlocks(ctx context.Context, blocks []Block) error { +// keys := make([]string, 0, len(blocks)) +// data := make([]io.ReadCloser, 0, len(blocks)) +// return f.blocksCache.Store(ctx, keys, data) +// } + +// type ChannelIter[T any] struct { +// ch <-chan T +// cur T +// } + +// func NewChannelIter[T any](ch <-chan T) *ChannelIter[T] { +// return &ChannelIter[T]{ +// ch: ch, +// } +// } + +// func (it *ChannelIter[T]) Next() bool { +// el, ok := <-it.ch +// if ok { +// it.cur = el +// return true +// } +// return false +// } + +// func (it *ChannelIter[T]) At() T { +// return it.cur +// } + +// func (it *ChannelIter[T]) Err() error { +// return nil +// } diff --git a/pkg/storage/stores/shipper/bloomshipper/fetcher_test.go b/pkg/storage/stores/shipper/bloomshipper/fetcher_test.go new file mode 100644 index 0000000000000..3e0ea64c6f253 --- /dev/null +++ b/pkg/storage/stores/shipper/bloomshipper/fetcher_test.go @@ -0,0 +1,176 @@ +package bloomshipper + +import ( + "context" + "encoding/json" + "fmt" + "path" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/prometheus/common/model" + "github.com/stretchr/testify/require" + + "github.com/grafana/loki/pkg/storage/chunk/cache" + "github.com/grafana/loki/pkg/storage/chunk/client/local" + "github.com/grafana/loki/pkg/storage/config" +) + +func makeMetas(t *testing.T, schemaCfg config.SchemaConfig, ts model.Time, keyspaces []Keyspace) []Meta { + t.Helper() + + metas := make([]Meta, len(keyspaces)) + for i, keyspace := range keyspaces { + metas[i] = Meta{ + MetaRef: MetaRef{ + Ref: Ref{ + TenantID: "fake", + TableName: fmt.Sprintf("%s%d", schemaCfg.Configs[0].IndexTables.Prefix, 0), + MinFingerprint: uint64(keyspace.Min), + MaxFingerprint: uint64(keyspace.Max), + StartTimestamp: ts, + EndTimestamp: ts, + }, + }, + Tombstones: []BlockRef{}, + Blocks: []BlockRef{}, + } + metas[i].FilePath = externalMetaKey(metas[i].MetaRef) + } + return metas +} + +func TestMetasFetcher(t *testing.T) { + dir := t.TempDir() + logger := log.NewNopLogger() + now := model.Now() + + schemaCfg := config.SchemaConfig{ + Configs: []config.PeriodConfig{ + { + From: config.DayTime{Time: 0}, + IndexType: "tsdb", + ObjectType: "filesystem", + Schema: "v13", + IndexTables: config.IndexPeriodicTableConfig{ + PathPrefix: "index/", + PeriodicTableConfig: config.PeriodicTableConfig{ + Prefix: "table_", + Period: 24 * time.Hour, + }, + }, + ChunkTables: config.PeriodicTableConfig{}, + RowShards: 16, + }, + }, + } + + tests := []struct { + name string + store []Meta // initial store state + start []Meta // initial cache state + end []Meta // final cache state + fetch []Meta // metas to fetch + }{ + { + name: "all metas found in cache", + store: []Meta{}, + start: makeMetas(t, schemaCfg, now, []Keyspace{{0x0000, 0xffff}}), + end: makeMetas(t, schemaCfg, now, []Keyspace{{0x0000, 0xffff}}), + fetch: makeMetas(t, schemaCfg, now, []Keyspace{{0x0000, 0xffff}}), + }, + { + name: "no metas found in cache", + store: makeMetas(t, schemaCfg, now, []Keyspace{{0x0000, 0xffff}}), + start: []Meta{}, + end: makeMetas(t, schemaCfg, now, []Keyspace{{0x0000, 0xffff}}), + fetch: makeMetas(t, schemaCfg, now, []Keyspace{{0x0000, 0xffff}}), + }, + { + name: "some metas found in cache", + store: makeMetas(t, schemaCfg, now, []Keyspace{{0x0000, 0xffff}, {0x10000, 0x1ffff}}), + start: makeMetas(t, schemaCfg, now, []Keyspace{{0x0000, 0xffff}}), + end: makeMetas(t, schemaCfg, now, []Keyspace{{0x0000, 0xffff}, {0x10000, 0x1ffff}}), + fetch: makeMetas(t, schemaCfg, now, []Keyspace{{0x0000, 0xffff}, {0x10000, 0x1ffff}}), + }, + } + + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + ctx := context.Background() + metasCache := cache.NewMockCache() + + oc, err := local.NewFSObjectClient(local.FSConfig{Directory: dir}) + require.NoError(t, err) + + c, err := NewBloomClient(oc, logger) + require.NoError(t, err) + + fetcher, err := NewFetcher(c, metasCache, nil, logger) + require.NoError(t, err) + + // prepare metas cache + keys := make([]string, 0, len(test.start)) + metas := make([][]byte, 0, len(test.start)) + for _, meta := range test.start { + b, err := json.Marshal(meta) + require.NoError(t, err) + metas = append(metas, b) + t.Log(string(b)) + + k := externalMetaKey(meta.MetaRef) + keys = append(keys, k) + } + require.NoError(t, metasCache.Store(ctx, keys, metas)) + + // prepare store + for _, meta := range test.store { + meta.FilePath = path.Join(dir, meta.FilePath) + err := c.PutMeta(ctx, meta) + require.NoError(t, err) + } + + actual, err := fetcher.FetchMetas(ctx, metaRefs(test.fetch)) + require.NoError(t, err) + require.ElementsMatch(t, test.fetch, actual) + + requireCachedMetas(t, test.end, metasCache.GetInternal()) + }) + } +} + +func metasFromCache(data map[string][]byte) []Meta { + metas := make([]Meta, 0, len(data)) + for k, v := range data { + meta := Meta{ + MetaRef: MetaRef{ + FilePath: k, + }, + } + _ = json.Unmarshal(v, &meta) + metas = append(metas, meta) + } + return metas +} + +func metaRefs(metas []Meta) []MetaRef { + refs := make([]MetaRef, 0, len(metas)) + for _, meta := range metas { + refs = append(refs, meta.MetaRef) + } + return refs +} + +func requireEqualMetas(t *testing.T, expected []Meta, actual []MetaRef) { + require.Equal(t, len(expected), len(actual)) + require.ElementsMatch(t, metaRefs(expected), actual) +} + +func requireCachedMetas(t *testing.T, expected []Meta, actual map[string][]byte) { + require.Equal(t, len(expected), len(actual)) + for _, meta := range expected { + _, contains := actual[meta.MetaRef.FilePath] + require.True(t, contains) + } +} diff --git a/pkg/storage/stores/shipper/bloomshipper/shipper.go b/pkg/storage/stores/shipper/bloomshipper/shipper.go index 5f5bd6f56d272..e3ab1db70c539 100644 --- a/pkg/storage/stores/shipper/bloomshipper/shipper.go +++ b/pkg/storage/stores/shipper/bloomshipper/shipper.go @@ -59,7 +59,7 @@ type Interface interface { } type Shipper struct { - client Client + store Store config config.Config logger log.Logger blockDownloader *blockDownloader @@ -69,14 +69,20 @@ type Limits interface { BloomGatewayBlocksDownloadingParallelism(tenantID string) int } -func NewShipper(client Client, config config.Config, limits Limits, logger log.Logger, reg prometheus.Registerer) (*Shipper, error) { +// TODO(chaudum): resolve and rip out +type StoreAndClient interface { + Store + Client +} + +func NewShipper(client StoreAndClient, config config.Config, limits Limits, logger log.Logger, reg prometheus.Registerer) (*Shipper, error) { logger = log.With(logger, "component", "bloom-shipper") downloader, err := newBlockDownloader(config, client, limits, logger, reg) if err != nil { return nil, fmt.Errorf("error creating block downloader: %w", err) } return &Shipper{ - client: client, + store: client, config: config, logger: logger, blockDownloader: downloader, @@ -138,7 +144,7 @@ func runCallback(callback ForEachBlockCallback, block blockWithQuerier) error { } func (s *Shipper) Stop() { - s.client.Stop() + s.store.Stop() s.blockDownloader.stop() } @@ -154,7 +160,7 @@ func getFirstLast[T any](s []T) (T, T) { func (s *Shipper) getActiveBlockRefs(ctx context.Context, tenantID string, interval Interval, keyspaces []Keyspace) ([]BlockRef, error) { minFpRange, maxFpRange := getFirstLast(keyspaces) - metas, err := s.client.GetMetas(ctx, MetaSearchParams{ + metas, err := s.store.FetchMetas(ctx, MetaSearchParams{ TenantID: tenantID, Keyspace: Keyspace{Min: minFpRange.Min, Max: maxFpRange.Max}, Interval: interval, diff --git a/pkg/storage/stores/shipper/bloomshipper/store.go b/pkg/storage/stores/shipper/bloomshipper/store.go new file mode 100644 index 0000000000000..0c2f2d5405515 --- /dev/null +++ b/pkg/storage/stores/shipper/bloomshipper/store.go @@ -0,0 +1,404 @@ +package bloomshipper + +import ( + "context" + "fmt" + "io" + "path/filepath" + "sort" + + "github.com/go-kit/log" + "github.com/pkg/errors" + "github.com/prometheus/common/model" + + "github.com/grafana/loki/pkg/storage" + "github.com/grafana/loki/pkg/storage/chunk/cache" + "github.com/grafana/loki/pkg/storage/chunk/client" + "github.com/grafana/loki/pkg/storage/config" +) + +type Store interface { + ResolveMetas(ctx context.Context, params MetaSearchParams) ([][]MetaRef, []*Fetcher, error) + FetchMetas(ctx context.Context, params MetaSearchParams) ([]Meta, error) + Fetcher(ts model.Time) *Fetcher + Stop() +} + +// Compiler check to ensure bloomStoreEntry implements the Client interface +var _ Client = &bloomStoreEntry{} + +// Compiler check to ensure bloomStoreEntry implements the Store interface +var _ Store = &bloomStoreEntry{} + +type bloomStoreEntry struct { + start model.Time + cfg config.PeriodConfig + objectClient client.ObjectClient + bloomClient Client + fetcher *Fetcher +} + +// ResolveMetas implements store. +func (b *bloomStoreEntry) ResolveMetas(ctx context.Context, params MetaSearchParams) ([][]MetaRef, []*Fetcher, error) { + var refs []MetaRef + tables := tablesForRange(b.cfg, params.Interval.Start, params.Interval.End) + for _, table := range tables { + prefix := filepath.Join(rootFolder, table, params.TenantID, metasFolder) + list, _, err := b.objectClient.List(ctx, prefix, "") + if err != nil { + return nil, nil, fmt.Errorf("error listing metas under prefix [%s]: %w", prefix, err) + } + for _, object := range list { + metaRef, err := createMetaRef(object.Key, params.TenantID, table) + + if err != nil { + return nil, nil, err + } + if metaRef.MaxFingerprint < uint64(params.Keyspace.Min) || uint64(params.Keyspace.Max) < metaRef.MinFingerprint || + metaRef.EndTimestamp.Before(params.Interval.Start) || metaRef.StartTimestamp.After(params.Interval.End) { + continue + } + refs = append(refs, metaRef) + } + } + return [][]MetaRef{refs}, []*Fetcher{b.fetcher}, nil +} + +// FetchMetas implements store. +func (b *bloomStoreEntry) FetchMetas(ctx context.Context, params MetaSearchParams) ([]Meta, error) { + metaRefs, fetchers, err := b.ResolveMetas(ctx, params) + if err != nil { + return nil, err + } + if len(metaRefs) != len(fetchers) { + return nil, errors.New("metaRefs and fetchers have unequal length") + } + + var metas []Meta + for i := range fetchers { + res, err := fetchers[i].FetchMetas(ctx, metaRefs[i]) + if err != nil { + return nil, err + } + metas = append(metas, res...) + } + return metas, nil +} + +// SearchMetas implements store. +func (b *bloomStoreEntry) Fetcher(_ model.Time) *Fetcher { + return b.fetcher +} + +// DeleteBlocks implements Client. +func (b *bloomStoreEntry) DeleteBlocks(ctx context.Context, refs []BlockRef) error { + return b.bloomClient.DeleteBlocks(ctx, refs) +} + +// DeleteMeta implements Client. +func (b *bloomStoreEntry) DeleteMeta(ctx context.Context, meta Meta) error { + return b.bloomClient.DeleteMeta(ctx, meta) +} + +// GetBlock implements Client. +func (b *bloomStoreEntry) GetBlock(ctx context.Context, ref BlockRef) (LazyBlock, error) { + return b.bloomClient.GetBlock(ctx, ref) +} + +// GetMetas implements Client. +func (b *bloomStoreEntry) GetMetas(ctx context.Context, refs []MetaRef) ([]Meta, error) { + return b.bloomClient.GetMetas(ctx, refs) +} + +// PutBlocks implements Client. +func (b *bloomStoreEntry) PutBlocks(ctx context.Context, blocks []Block) ([]Block, error) { + return b.bloomClient.PutBlocks(ctx, blocks) +} + +// PutMeta implements Client. +func (b *bloomStoreEntry) PutMeta(ctx context.Context, meta Meta) error { + return b.bloomClient.PutMeta(ctx, meta) +} + +// Stop implements Client. +func (b bloomStoreEntry) Stop() { + b.bloomClient.Stop() +} + +var _ Client = &BloomStore{} +var _ Store = &BloomStore{} + +type BloomStore struct { + stores []*bloomStoreEntry + storageConfig storage.Config +} + +func NewBloomStore( + periodicConfigs []config.PeriodConfig, + storageConfig storage.Config, + clientMetrics storage.ClientMetrics, + metasCache cache.Cache, + blocksCache *cache.EmbeddedCache[string, io.ReadCloser], + logger log.Logger, +) (*BloomStore, error) { + store := &BloomStore{ + storageConfig: storageConfig, + } + + if metasCache == nil { + metasCache = cache.NewNoopCache() + } + + // sort by From time + sort.Slice(periodicConfigs, func(i, j int) bool { + return periodicConfigs[i].From.Time.Before(periodicConfigs[i].From.Time) + }) + + for _, periodicConfig := range periodicConfigs { + objectClient, err := storage.NewObjectClient(periodicConfig.ObjectType, storageConfig, clientMetrics) + if err != nil { + return nil, errors.Wrapf(err, "creating object client for period %s", periodicConfig.From) + } + bloomClient, err := NewBloomClient(objectClient, logger) + if err != nil { + return nil, errors.Wrapf(err, "creating bloom client for period %s", periodicConfig.From) + } + fetcher, err := NewFetcher(bloomClient, metasCache, blocksCache, logger) + if err != nil { + return nil, errors.Wrapf(err, "creating fetcher for period %s", periodicConfig.From) + } + + store.stores = append(store.stores, &bloomStoreEntry{ + start: periodicConfig.From.Time, + cfg: periodicConfig, + objectClient: objectClient, + bloomClient: bloomClient, + fetcher: fetcher, + }) + } + + return store, nil +} + +// Fetcher implements Store. +func (b *BloomStore) Fetcher(ts model.Time) *Fetcher { + if store := b.getStore(ts); store != nil { + return store.Fetcher(ts) + } + return nil +} + +// ResolveMetas implements Store. +func (b *BloomStore) ResolveMetas(ctx context.Context, params MetaSearchParams) ([][]MetaRef, []*Fetcher, error) { + var refs [][]MetaRef + var fetchers []*Fetcher + err := b.forStores(ctx, params.Interval, func(innerCtx context.Context, interval Interval, store Store) error { + newParams := params + newParams.Interval = interval + metas, fetcher, err := store.ResolveMetas(innerCtx, newParams) + if err != nil { + return err + } + refs = append(refs, metas...) + fetchers = append(fetchers, fetcher...) + return nil + }) + return refs, fetchers, err +} + +// FetchMetas implements Store. +func (b *BloomStore) FetchMetas(ctx context.Context, params MetaSearchParams) ([]Meta, error) { + metaRefs, fetchers, err := b.ResolveMetas(ctx, params) + if err != nil { + return nil, err + } + if len(metaRefs) != len(fetchers) { + return nil, errors.New("metaRefs and fetchers have unequal length") + } + + var metas []Meta + for i := range fetchers { + res, err := fetchers[i].FetchMetas(ctx, metaRefs[i]) + if err != nil { + return nil, err + } + metas = append(metas, res...) + } + return metas, nil +} + +// DeleteBlocks implements Client. +func (b *BloomStore) DeleteBlocks(ctx context.Context, blocks []BlockRef) error { + for _, ref := range blocks { + err := b.storeDo( + ref.StartTimestamp, + func(s *bloomStoreEntry) error { + return s.DeleteBlocks(ctx, []BlockRef{ref}) + }, + ) + if err != nil { + return err + } + } + return nil +} + +// DeleteMeta implements Client. +func (b *BloomStore) DeleteMeta(ctx context.Context, meta Meta) error { + return b.storeDo(meta.StartTimestamp, func(s *bloomStoreEntry) error { + return s.DeleteMeta(ctx, meta) + }) +} + +// GetBlock implements Client. +func (b *BloomStore) GetBlock(ctx context.Context, ref BlockRef) (LazyBlock, error) { + var block LazyBlock + var err error + err = b.storeDo(ref.StartTimestamp, func(s *bloomStoreEntry) error { + block, err = s.GetBlock(ctx, ref) + return err + }) + return block, err +} + +// GetMetas implements Client. +func (b *BloomStore) GetMetas(ctx context.Context, metas []MetaRef) ([]Meta, error) { + var refs [][]MetaRef + var fetchers []*Fetcher + + for i := len(b.stores) - 1; i >= 0; i-- { + s := b.stores[i] + from, through := s.start, model.Latest + if i < len(b.stores)-1 { + through = b.stores[i+1].start + } + + var res []MetaRef + for _, meta := range metas { + if meta.StartTimestamp >= from && meta.StartTimestamp < through { + res = append(res, meta) + } + } + + if len(res) > 0 { + refs = append(refs, res) + fetchers = append(fetchers, s.Fetcher(s.start)) + } + } + + results := make([]Meta, 0, len(metas)) + for i := range fetchers { + res, err := fetchers[i].FetchMetas(ctx, refs[i]) + results = append(results, res...) + if err != nil { + return results, err + } + } + + return results, nil +} + +// PutBlocks implements Client. +func (b *BloomStore) PutBlocks(ctx context.Context, blocks []Block) ([]Block, error) { + results := make([]Block, 0, len(blocks)) + for _, ref := range blocks { + err := b.storeDo( + ref.StartTimestamp, + func(s *bloomStoreEntry) error { + res, err := s.PutBlocks(ctx, []Block{ref}) + results = append(results, res...) + return err + }, + ) + if err != nil { + return nil, err + } + } + return results, nil +} + +// PutMeta implements Client. +func (b *BloomStore) PutMeta(ctx context.Context, meta Meta) error { + return b.storeDo(meta.StartTimestamp, func(s *bloomStoreEntry) error { + return s.PutMeta(ctx, meta) + }) +} + +// Stop implements Client. +func (b *BloomStore) Stop() { + for _, s := range b.stores { + s.Stop() + } +} + +func (b *BloomStore) getStore(ts model.Time) *bloomStoreEntry { + // find the schema with the lowest start _after_ tm + j := sort.Search(len(b.stores), func(j int) bool { + return b.stores[j].start > ts + }) + + // reduce it by 1 because we want a schema with start <= tm + j-- + + if 0 <= j && j < len(b.stores) { + return b.stores[j] + } + + return nil +} + +func (b *BloomStore) storeDo(ts model.Time, f func(s *bloomStoreEntry) error) error { + if store := b.getStore(ts); store != nil { + return f(store) + } + return nil +} + +func (b *BloomStore) forStores(ctx context.Context, interval Interval, f func(innerCtx context.Context, interval Interval, store Store) error) error { + if len(b.stores) == 0 { + return nil + } + + from, through := interval.Start, interval.End + + // first, find the schema with the highest start _before or at_ from + i := sort.Search(len(b.stores), func(i int) bool { + return b.stores[i].start > from + }) + if i > 0 { + i-- + } else { + // This could happen if we get passed a sample from before 1970. + i = 0 + from = b.stores[0].start + } + + // next, find the schema with the lowest start _after_ through + j := sort.Search(len(b.stores), func(j int) bool { + return b.stores[j].start > through + }) + + min := func(a, b model.Time) model.Time { + if a < b { + return a + } + return b + } + + start := from + for ; i < j; i++ { + nextSchemaStarts := model.Latest + if i+1 < len(b.stores) { + nextSchemaStarts = b.stores[i+1].start + } + + end := min(through, nextSchemaStarts-1) + err := f(ctx, Interval{start, end}, b.stores[i]) + if err != nil { + return err + } + + start = nextSchemaStarts + } + return nil +} From 7a04da852ab2bc47849d4da0884a1a17d8906ce6 Mon Sep 17 00:00:00 2001 From: Sandeep Sukhani Date: Wed, 31 Jan 2024 17:01:41 +0530 Subject: [PATCH 13/45] otlp: fix otlp service.name check when no resource attributes are defined (#11837) **What this PR does / why we need it**: When no resource attributes are defined, check for `service.name` resource attributes fail with `nil pointer dereference`. This PR fixes the issue by checking the bool returned by the `Get` method on resource attributes. **Checklist** - [x] Tests updated --- pkg/loghttp/push/otlp.go | 2 +- pkg/loghttp/push/otlp_test.go | 72 +++++++++++++++++++++++++++++++++++ 2 files changed, 73 insertions(+), 1 deletion(-) diff --git a/pkg/loghttp/push/otlp.go b/pkg/loghttp/push/otlp.go index f4f937b93dc33..c25477a984e24 100644 --- a/pkg/loghttp/push/otlp.go +++ b/pkg/loghttp/push/otlp.go @@ -114,7 +114,7 @@ func otlpToLokiPushRequest(ld plog.Logs, userID string, tenantsRetention Tenants res := rls.At(i).Resource() resAttrs := res.Attributes() - if v, _ := resAttrs.Get(attrServiceName); v.AsString() == "" { + if v, ok := resAttrs.Get(attrServiceName); !ok || v.AsString() == "" { resAttrs.PutStr(attrServiceName, "unknown_service") } resourceAttributesAsStructuredMetadata := make(push.LabelsAdapter, 0, resAttrs.Len()) diff --git a/pkg/loghttp/push/otlp_test.go b/pkg/loghttp/push/otlp_test.go index d817c933a43d5..badb6cd000e4b 100644 --- a/pkg/loghttp/push/otlp_test.go +++ b/pkg/loghttp/push/otlp_test.go @@ -82,6 +82,78 @@ func TestOTLPToLokiPushRequest(t *testing.T) { mostRecentEntryTimestamp: now, }, }, + { + name: "no resource attributes defined", + otlpConfig: DefaultOTLPConfig, + generateLogs: func() plog.Logs { + ld := plog.NewLogs() + ld.ResourceLogs().AppendEmpty() + ld.ResourceLogs().At(0).ScopeLogs().AppendEmpty().LogRecords().AppendEmpty().Body().SetStr("test body") + ld.ResourceLogs().At(0).ScopeLogs().At(0).LogRecords().At(0).SetTimestamp(pcommon.Timestamp(now.UnixNano())) + return ld + }, + expectedPushRequest: logproto.PushRequest{ + Streams: []logproto.Stream{ + { + Labels: `{service_name="unknown_service"}`, + Entries: []logproto.Entry{ + { + Timestamp: now, + Line: "test body", + StructuredMetadata: push.LabelsAdapter{}, + }, + }, + }, + }, + }, + expectedStats: Stats{ + numLines: 1, + logLinesBytes: map[time.Duration]int64{ + time.Hour: 9, + }, + structuredMetadataBytes: map[time.Duration]int64{ + time.Hour: 0, + }, + streamLabelsSize: 27, + mostRecentEntryTimestamp: now, + }, + }, + { + name: "service.name not defined in resource attributes", + otlpConfig: DefaultOTLPConfig, + generateLogs: func() plog.Logs { + ld := plog.NewLogs() + ld.ResourceLogs().AppendEmpty().Resource().Attributes().PutStr("service.namespace", "foo") + ld.ResourceLogs().At(0).ScopeLogs().AppendEmpty().LogRecords().AppendEmpty().Body().SetStr("test body") + ld.ResourceLogs().At(0).ScopeLogs().At(0).LogRecords().At(0).SetTimestamp(pcommon.Timestamp(now.UnixNano())) + return ld + }, + expectedPushRequest: logproto.PushRequest{ + Streams: []logproto.Stream{ + { + Labels: `{service_name="unknown_service", service_namespace="foo"}`, + Entries: []logproto.Entry{ + { + Timestamp: now, + Line: "test body", + StructuredMetadata: push.LabelsAdapter{}, + }, + }, + }, + }, + }, + expectedStats: Stats{ + numLines: 1, + logLinesBytes: map[time.Duration]int64{ + time.Hour: 9, + }, + structuredMetadataBytes: map[time.Duration]int64{ + time.Hour: 0, + }, + streamLabelsSize: 47, + mostRecentEntryTimestamp: now, + }, + }, { name: "resource attributes and scope attributes stored as structured metadata", otlpConfig: DefaultOTLPConfig, From 45e5b427b1d5a794de7034ddcf249d09c2730ca8 Mon Sep 17 00:00:00 2001 From: Meng Ye <4025839+jk2K@users.noreply.github.com> Date: Thu, 1 Feb 2024 01:16:19 +0800 Subject: [PATCH 14/45] docs: fix row_shards doc (#11795) for v10 or greater, default is 16 refer to https://github.com/grafana/loki/blob/1002ba00dff58ed588987169c0d3a0ddac2d022b/pkg/storage/config/schema_config.go#L336 **What this PR does / why we need it**: **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) --------- Co-authored-by: J Stickler --- docs/sources/configure/_index.md | 2 +- pkg/storage/config/schema_config.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/sources/configure/_index.md b/docs/sources/configure/_index.md index 283a2c9dd59a9..25e4f70f987c3 100644 --- a/docs/sources/configure/_index.md +++ b/docs/sources/configure/_index.md @@ -4577,7 +4577,7 @@ chunks: [tags: ] # How many shards will be created. Only used if schema is v10 or greater. -[row_shards: ] +[row_shards: | default = 16] ``` ### aws_storage_config diff --git a/pkg/storage/config/schema_config.go b/pkg/storage/config/schema_config.go index d4b5902516d20..9cdda249ea520 100644 --- a/pkg/storage/config/schema_config.go +++ b/pkg/storage/config/schema_config.go @@ -164,7 +164,7 @@ type PeriodConfig struct { Schema string `yaml:"schema" doc:"description=The schema version to use, current recommended schema is v12."` IndexTables IndexPeriodicTableConfig `yaml:"index" doc:"description=Configures how the index is updated and stored."` ChunkTables PeriodicTableConfig `yaml:"chunks" doc:"description=Configured how the chunks are updated and stored."` - RowShards uint32 `yaml:"row_shards" doc:"description=How many shards will be created. Only used if schema is v10 or greater."` + RowShards uint32 `yaml:"row_shards" doc:"default=16|description=How many shards will be created. Only used if schema is v10 or greater."` // Integer representation of schema used for hot path calculation. Populated on unmarshaling. schemaInt *int `yaml:"-"` From 509f6f6b66863284e6aeb3700ff8b8bddba27043 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 31 Jan 2024 10:24:22 -0800 Subject: [PATCH 15/45] Bloom/controller wiring (#11831) Functionality & testware for building bloom block plans. --- pkg/bloomcompactor/v2_meta.go | 9 +- pkg/bloomcompactor/v2controller.go | 191 +++++++++++++++++- pkg/bloomcompactor/v2controller_test.go | 249 +++++++++++++++++++++--- pkg/bloomcompactor/v2spec.go | 2 +- pkg/bloomgateway/multiplexing.go | 2 +- pkg/storage/bloom/v1/bounds.go | 7 + pkg/storage/bloom/v1/builder.go | 6 +- pkg/storage/bloom/v1/dedupe.go | 19 +- pkg/storage/bloom/v1/dedupe_test.go | 2 +- pkg/storage/bloom/v1/util.go | 5 + 10 files changed, 448 insertions(+), 44 deletions(-) diff --git a/pkg/bloomcompactor/v2_meta.go b/pkg/bloomcompactor/v2_meta.go index 1be785c0934ac..adffb61dff5ed 100644 --- a/pkg/bloomcompactor/v2_meta.go +++ b/pkg/bloomcompactor/v2_meta.go @@ -62,7 +62,7 @@ type Meta struct { // is greater than or equal to the range of the actual data in the underlying blocks. OwnershipRange v1.FingerprintBounds - // Old blocks which can be deleted in the future. These should be from pervious compaction rounds. + // Old blocks which can be deleted in the future. These should be from previous compaction rounds. Tombstones []BlockRef // The specific TSDB files used to generate the block. @@ -119,17 +119,18 @@ func (m Meta) Checksum() (uint32, error) { } type TSDBStore interface { - ResolveTSDBs() ([]*tsdb.TSDBFile, error) + ResolveTSDBs() ([]*tsdb.SingleTenantTSDBIdentifier, error) + LoadTSDB(id tsdb.Identifier, bounds v1.FingerprintBounds) (v1.CloseableIterator[*v1.Series], error) } type MetaStore interface { + ResolveMetas(bounds v1.FingerprintBounds) ([]MetaRef, error) GetMetas([]MetaRef) ([]Meta, error) PutMeta(Meta) error - ResolveMetas(bounds v1.FingerprintBounds) ([]MetaRef, error) } type BlockStore interface { // TODO(owen-d): flesh out|integrate against bloomshipper.Client - GetBlocks([]BlockRef) ([]interface{}, error) + GetBlocks([]BlockRef) ([]*v1.Block, error) PutBlock(interface{}) error } diff --git a/pkg/bloomcompactor/v2controller.go b/pkg/bloomcompactor/v2controller.go index 3fbcd04cd93db..31f73740c1ff7 100644 --- a/pkg/bloomcompactor/v2controller.go +++ b/pkg/bloomcompactor/v2controller.go @@ -3,6 +3,7 @@ package bloomcompactor import ( "context" "fmt" + "sort" "github.com/go-kit/log" "github.com/go-kit/log/level" @@ -17,6 +18,9 @@ type SimpleBloomController struct { tsdbStore TSDBStore metaStore MetaStore blockStore BlockStore + chunkLoader ChunkLoader + rwFn func() (v1.BlockWriter, v1.BlockReader) + metrics *Metrics // TODO(owen-d): add metrics logger log.Logger @@ -27,6 +31,9 @@ func NewSimpleBloomController( tsdbStore TSDBStore, metaStore MetaStore, blockStore BlockStore, + chunkLoader ChunkLoader, + rwFn func() (v1.BlockWriter, v1.BlockReader), + metrics *Metrics, logger log.Logger, ) *SimpleBloomController { return &SimpleBloomController{ @@ -34,11 +41,14 @@ func NewSimpleBloomController( tsdbStore: tsdbStore, metaStore: metaStore, blockStore: blockStore, + chunkLoader: chunkLoader, + rwFn: rwFn, + metrics: metrics, logger: log.With(logger, "ownership", ownershipRange), } } -func (s *SimpleBloomController) do(_ context.Context) error { +func (s *SimpleBloomController) do(ctx context.Context) error { // 1. Resolve TSDBs tsdbs, err := s.tsdbStore.ResolveTSDBs() if err != nil { @@ -61,26 +71,30 @@ func (s *SimpleBloomController) do(_ context.Context) error { } ids := make([]tsdb.Identifier, 0, len(tsdbs)) - for _, idx := range tsdbs { - ids = append(ids, idx.Identifier) + for _, id := range tsdbs { + ids = append(ids, id) } // 4. Determine which TSDBs have gaps in the ownership range and need to // be processed. - work, err := gapsBetweenTSDBsAndMetas(s.ownershipRange, ids, metas) + tsdbsWithGaps, err := gapsBetweenTSDBsAndMetas(s.ownershipRange, ids, metas) if err != nil { level.Error(s.logger).Log("msg", "failed to find gaps", "err", err) return errors.Wrap(err, "failed to find gaps") } - if len(work) == 0 { + if len(tsdbsWithGaps) == 0 { level.Debug(s.logger).Log("msg", "blooms exist for all tsdbs") return nil } - // TODO(owen-d): finish - panic("not implemented") + work, err := blockPlansForGaps(tsdbsWithGaps, metas) + if err != nil { + level.Error(s.logger).Log("msg", "failed to create plan", "err", err) + return errors.Wrap(err, "failed to create plan") + } + // 5. 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 // needed chunks in their blooms, for instance after a new TSDB version is generated @@ -89,8 +103,171 @@ func (s *SimpleBloomController) do(_ context.Context) error { // overlapping the ownership ranges we've identified as needing updates. // With these in hand, we can download the old blocks and use them to // accelerate bloom generation for the new blocks. + + var ( + blockCt int + tsdbCt = len(work) + ) + + for _, plan := range work { + + 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(plan.tsdb, gap) + if err != nil { + level.Error(s.logger).Log("msg", "failed to get series and blocks", "err", err) + return errors.Wrap(err, "failed to get series and blocks") + } + + gen := NewSimpleBloomGenerator( + v1.DefaultBlockOptions, + seriesItr, + s.chunkLoader, + preExistingBlocks, + s.rwFn, + s.metrics, + log.With(s.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) + return errors.Wrap(err, "failed to generate bloom") + } + + // TODO(owen-d): dispatch this to a queue for writing, handling retries/backpressure, etc? + for newBlocks.Next() { + blockCt++ + blk := newBlocks.At() + if err := s.blockStore.PutBlock(blk); err != nil { + level.Error(s.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) + return errors.Wrap(err, "failed to generate bloom") + } + + } + } + + level.Debug(s.logger).Log("msg", "finished bloom generation", "blocks", blockCt, "tsdbs", tsdbCt) + return nil + +} + +func (s *SimpleBloomController) loadWorkForGap(id tsdb.Identifier, gap gapWithBlocks) (v1.CloseableIterator[*v1.Series], []*v1.Block, error) { + // load a series iterator for the gap + seriesItr, err := s.tsdbStore.LoadTSDB(id, gap.bounds) + if err != nil { + return nil, nil, errors.Wrap(err, "failed to load tsdb") + } + + blocks, err := s.blockStore.GetBlocks(gap.blocks) + if err != nil { + return nil, nil, errors.Wrap(err, "failed to get blocks") + } + + return seriesItr, blocks, nil +} + +type gapWithBlocks struct { + bounds v1.FingerprintBounds + blocks []BlockRef +} + +// blockPlan is a plan for all the work needed to build a meta.json +// It includes: +// - the tsdb (source of truth) which contains all the series+chunks +// we need to ensure are indexed in bloom blocks +// - a list of gaps that are out of date and need to be checked+built +// - within each gap, a list of block refs which overlap the gap are included +// so we can use them to accelerate bloom generation. They likely contain many +// of the same chunks we need to ensure are indexed, just from previous tsdb iterations. +// This is a performance optimization to avoid expensive re-reindexing +type blockPlan struct { + tsdb tsdb.Identifier + gaps []gapWithBlocks +} + +// blockPlansForGaps groups tsdb gaps we wish to fill with overlapping but out of date blocks. +// This allows us to expedite bloom generation by using existing blocks to fill in the gaps +// since many will contain the same chunks. +func blockPlansForGaps(tsdbs []tsdbGaps, metas []Meta) ([]blockPlan, error) { + plans := make([]blockPlan, 0, len(tsdbs)) + + for _, idx := range tsdbs { + plan := blockPlan{ + tsdb: idx.tsdb, + gaps: make([]gapWithBlocks, 0, len(idx.gaps)), + } + + for _, gap := range idx.gaps { + planGap := gapWithBlocks{ + bounds: gap, + } + + for _, meta := range metas { + + if meta.OwnershipRange.Intersection(gap) == nil { + // this meta doesn't overlap the gap, skip + continue + } + + for _, block := range meta.Blocks { + if block.OwnershipRange.Intersection(gap) == nil { + // this block doesn't overlap the gap, skip + continue + } + // this block overlaps the gap, add it to the plan + // for this gap + planGap.blocks = append(planGap.blocks, block) + } + } + + // ensure we sort blocks so deduping iterator works as expected + sort.Slice(planGap.blocks, func(i, j int) bool { + return planGap.blocks[i].OwnershipRange.Less(planGap.blocks[j].OwnershipRange) + }) + + peekingBlocks := v1.NewPeekingIter[BlockRef]( + v1.NewSliceIter[BlockRef]( + planGap.blocks, + ), + ) + // dedupe blocks which could be in multiple metas + itr := v1.NewDedupingIter[BlockRef, BlockRef]( + func(a, b BlockRef) bool { + return a == b + }, + v1.Identity[BlockRef], + func(a, _ BlockRef) BlockRef { + return a + }, + peekingBlocks, + ) + + deduped, err := v1.Collect[BlockRef](itr) + if err != nil { + return nil, errors.Wrap(err, "failed to dedupe blocks") + } + planGap.blocks = deduped + + plan.gaps = append(plan.gaps, planGap) + } + + plans = append(plans, plan) + } + + return plans, nil } +// Used to signal the gaps that need to be populated for a tsdb type tsdbGaps struct { tsdb tsdb.Identifier gaps []v1.FingerprintBounds diff --git a/pkg/bloomcompactor/v2controller_test.go b/pkg/bloomcompactor/v2controller_test.go index 0a99f26d3ce1f..9f3f56153af32 100644 --- a/pkg/bloomcompactor/v2controller_test.go +++ b/pkg/bloomcompactor/v2controller_test.go @@ -113,22 +113,24 @@ func Test_findGaps(t *testing.T) { } } -func Test_gapsBetweenTSDBsAndMetas(t *testing.T) { - id := func(n int) tsdb.SingleTenantTSDBIdentifier { - return tsdb.SingleTenantTSDBIdentifier{ - TS: time.Unix(int64(n), 0), - } +func tsdbID(n int) tsdb.SingleTenantTSDBIdentifier { + return tsdb.SingleTenantTSDBIdentifier{ + TS: time.Unix(int64(n), 0), } +} - meta := func(min, max model.Fingerprint, sources ...int) Meta { - m := Meta{ - OwnershipRange: v1.NewBounds(min, max), - } - for _, source := range sources { - m.Sources = append(m.Sources, id(source)) - } - return m +func genMeta(min, max model.Fingerprint, sources []int, blocks []BlockRef) Meta { + m := Meta{ + OwnershipRange: v1.NewBounds(min, max), + Blocks: blocks, + } + for _, source := range sources { + m.Sources = append(m.Sources, tsdbID(source)) } + return m +} + +func Test_gapsBetweenTSDBsAndMetas(t *testing.T) { for _, tc := range []struct { desc string @@ -142,21 +144,21 @@ func Test_gapsBetweenTSDBsAndMetas(t *testing.T) { desc: "non-overlapping tsdbs and metas", err: true, ownershipRange: v1.NewBounds(0, 10), - tsdbs: []tsdb.Identifier{id(0)}, + tsdbs: []tsdb.Identifier{tsdbID(0)}, metas: []Meta{ - meta(11, 20, 0), + genMeta(11, 20, []int{0}, nil), }, }, { desc: "single tsdb", ownershipRange: v1.NewBounds(0, 10), - tsdbs: []tsdb.Identifier{id(0)}, + tsdbs: []tsdb.Identifier{tsdbID(0)}, metas: []Meta{ - meta(4, 8, 0), + genMeta(4, 8, []int{0}, nil), }, exp: []tsdbGaps{ { - tsdb: id(0), + tsdb: tsdbID(0), gaps: []v1.FingerprintBounds{ v1.NewBounds(0, 3), v1.NewBounds(9, 10), @@ -167,20 +169,20 @@ func Test_gapsBetweenTSDBsAndMetas(t *testing.T) { { desc: "multiple tsdbs with separate blocks", ownershipRange: v1.NewBounds(0, 10), - tsdbs: []tsdb.Identifier{id(0), id(1)}, + tsdbs: []tsdb.Identifier{tsdbID(0), tsdbID(1)}, metas: []Meta{ - meta(0, 5, 0), - meta(6, 10, 1), + genMeta(0, 5, []int{0}, nil), + genMeta(6, 10, []int{1}, nil), }, exp: []tsdbGaps{ { - tsdb: id(0), + tsdb: tsdbID(0), gaps: []v1.FingerprintBounds{ v1.NewBounds(6, 10), }, }, { - tsdb: id(1), + tsdb: tsdbID(1), gaps: []v1.FingerprintBounds{ v1.NewBounds(0, 5), }, @@ -190,20 +192,20 @@ func Test_gapsBetweenTSDBsAndMetas(t *testing.T) { { desc: "multiple tsdbs with the same blocks", ownershipRange: v1.NewBounds(0, 10), - tsdbs: []tsdb.Identifier{id(0), id(1)}, + tsdbs: []tsdb.Identifier{tsdbID(0), tsdbID(1)}, metas: []Meta{ - meta(0, 5, 0, 1), - meta(6, 8, 1), + genMeta(0, 5, []int{0, 1}, nil), + genMeta(6, 8, []int{1}, nil), }, exp: []tsdbGaps{ { - tsdb: id(0), + tsdb: tsdbID(0), gaps: []v1.FingerprintBounds{ v1.NewBounds(6, 10), }, }, { - tsdb: id(1), + tsdb: tsdbID(1), gaps: []v1.FingerprintBounds{ v1.NewBounds(9, 10), }, @@ -221,3 +223,194 @@ func Test_gapsBetweenTSDBsAndMetas(t *testing.T) { }) } } + +func genBlockRef(min, max model.Fingerprint) BlockRef { + bounds := v1.NewBounds(min, max) + return BlockRef{ + OwnershipRange: bounds, + } +} + +func Test_blockPlansForGaps(t *testing.T) { + for _, tc := range []struct { + desc string + ownershipRange v1.FingerprintBounds + tsdbs []tsdb.Identifier + metas []Meta + err bool + exp []blockPlan + }{ + { + desc: "single overlapping meta+no overlapping block", + ownershipRange: v1.NewBounds(0, 10), + tsdbs: []tsdb.Identifier{tsdbID(0)}, + metas: []Meta{ + genMeta(5, 20, []int{1}, []BlockRef{genBlockRef(11, 20)}), + }, + exp: []blockPlan{ + { + tsdb: tsdbID(0), + gaps: []gapWithBlocks{ + { + bounds: v1.NewBounds(0, 10), + }, + }, + }, + }, + }, + { + desc: "single overlapping meta+one overlapping block", + ownershipRange: v1.NewBounds(0, 10), + tsdbs: []tsdb.Identifier{tsdbID(0)}, + metas: []Meta{ + genMeta(5, 20, []int{1}, []BlockRef{genBlockRef(9, 20)}), + }, + exp: []blockPlan{ + { + tsdb: tsdbID(0), + gaps: []gapWithBlocks{ + { + bounds: v1.NewBounds(0, 10), + blocks: []BlockRef{genBlockRef(9, 20)}, + }, + }, + }, + }, + }, + { + // the range which needs to be generated doesn't overlap with existing blocks + // from other tsdb versions since theres an up to date tsdb version block, + // but we can trim the range needing generation + desc: "trims up to date area", + ownershipRange: v1.NewBounds(0, 10), + tsdbs: []tsdb.Identifier{tsdbID(0)}, + metas: []Meta{ + genMeta(9, 20, []int{0}, []BlockRef{genBlockRef(9, 20)}), // block for same tsdb + genMeta(9, 20, []int{1}, []BlockRef{genBlockRef(9, 20)}), // block for different tsdb + }, + exp: []blockPlan{ + { + tsdb: tsdbID(0), + gaps: []gapWithBlocks{ + { + bounds: v1.NewBounds(0, 8), + }, + }, + }, + }, + }, + { + desc: "uses old block for overlapping range", + ownershipRange: v1.NewBounds(0, 10), + tsdbs: []tsdb.Identifier{tsdbID(0)}, + metas: []Meta{ + genMeta(9, 20, []int{0}, []BlockRef{genBlockRef(9, 20)}), // block for same tsdb + genMeta(5, 20, []int{1}, []BlockRef{genBlockRef(5, 20)}), // block for different tsdb + }, + exp: []blockPlan{ + { + tsdb: tsdbID(0), + gaps: []gapWithBlocks{ + { + bounds: v1.NewBounds(0, 8), + blocks: []BlockRef{genBlockRef(5, 20)}, + }, + }, + }, + }, + }, + { + desc: "multi case", + ownershipRange: v1.NewBounds(0, 10), + tsdbs: []tsdb.Identifier{tsdbID(0), tsdbID(1)}, // generate for both tsdbs + metas: []Meta{ + genMeta(0, 2, []int{0}, []BlockRef{ + genBlockRef(0, 1), + genBlockRef(1, 2), + }), // tsdb_0 + genMeta(6, 8, []int{0}, []BlockRef{genBlockRef(6, 8)}), // tsdb_0 + + genMeta(3, 5, []int{1}, []BlockRef{genBlockRef(3, 5)}), // tsdb_1 + genMeta(8, 10, []int{1}, []BlockRef{genBlockRef(8, 10)}), // tsdb_1 + }, + exp: []blockPlan{ + { + tsdb: tsdbID(0), + gaps: []gapWithBlocks{ + // tsdb (id=0) can source chunks from the blocks built from tsdb (id=1) + { + bounds: v1.NewBounds(3, 5), + blocks: []BlockRef{genBlockRef(3, 5)}, + }, + { + bounds: v1.NewBounds(9, 10), + blocks: []BlockRef{genBlockRef(8, 10)}, + }, + }, + }, + // tsdb (id=1) can source chunks from the blocks built from tsdb (id=0) + { + tsdb: tsdbID(1), + gaps: []gapWithBlocks{ + { + bounds: v1.NewBounds(0, 2), + blocks: []BlockRef{ + genBlockRef(0, 1), + genBlockRef(1, 2), + }, + }, + { + bounds: v1.NewBounds(6, 7), + blocks: []BlockRef{genBlockRef(6, 8)}, + }, + }, + }, + }, + }, + { + desc: "dedupes block refs", + ownershipRange: v1.NewBounds(0, 10), + tsdbs: []tsdb.Identifier{tsdbID(0)}, + metas: []Meta{ + genMeta(9, 20, []int{1}, []BlockRef{ + genBlockRef(1, 4), + genBlockRef(9, 20), + }), // blocks for first diff tsdb + genMeta(5, 20, []int{2}, []BlockRef{ + genBlockRef(5, 10), + genBlockRef(9, 20), // same block references in prior meta (will be deduped) + }), // block for second diff tsdb + }, + exp: []blockPlan{ + { + tsdb: tsdbID(0), + gaps: []gapWithBlocks{ + { + bounds: v1.NewBounds(0, 10), + blocks: []BlockRef{ + genBlockRef(1, 4), + genBlockRef(5, 10), + genBlockRef(9, 20), + }, + }, + }, + }, + }, + }, + } { + t.Run(tc.desc, func(t *testing.T) { + // we reuse the gapsBetweenTSDBsAndMetas function to generate the gaps as this function is tested + // separately and it's used to generate input in our regular code path (easier to write tests this way). + gaps, err := gapsBetweenTSDBsAndMetas(tc.ownershipRange, tc.tsdbs, tc.metas) + require.NoError(t, err) + + plans, err := blockPlansForGaps(gaps, tc.metas) + if tc.err { + require.Error(t, err) + return + } + require.Equal(t, tc.exp, plans) + + }) + } +} diff --git a/pkg/bloomcompactor/v2spec.go b/pkg/bloomcompactor/v2spec.go index 49e74a47188a7..e0d964e9e9724 100644 --- a/pkg/bloomcompactor/v2spec.go +++ b/pkg/bloomcompactor/v2spec.go @@ -103,7 +103,7 @@ func NewSimpleBloomGenerator( store: store, chunkLoader: chunkLoader, blocks: blocks, - logger: logger, + logger: log.With(logger, "component", "bloom_generator"), readWriterFn: readWriterFn, metrics: metrics, diff --git a/pkg/bloomgateway/multiplexing.go b/pkg/bloomgateway/multiplexing.go index 97c2571948096..d2722ad8f1496 100644 --- a/pkg/bloomgateway/multiplexing.go +++ b/pkg/bloomgateway/multiplexing.go @@ -56,7 +56,7 @@ type Task struct { // the last error of the task // needs to be a pointer so multiple copies of the task can modify its value err *wrappedError - // the respones received from the block queriers + // the responses received from the block queriers responses []v1.Output // series of the original request diff --git a/pkg/storage/bloom/v1/bounds.go b/pkg/storage/bloom/v1/bounds.go index fc22866285f82..6aff8ae2c1706 100644 --- a/pkg/storage/bloom/v1/bounds.go +++ b/pkg/storage/bloom/v1/bounds.go @@ -37,6 +37,13 @@ func (b FingerprintBounds) String() string { return b.Min.String() + "-" + b.Max.String() } +func (b FingerprintBounds) Less(other FingerprintBounds) bool { + if b.Min != other.Min { + return b.Min < other.Min + } + return b.Max <= other.Max +} + // Cmp returns the fingerprint's position relative to the bounds func (b FingerprintBounds) Cmp(fp model.Fingerprint) BoundsCheck { if fp < b.Min { diff --git a/pkg/storage/bloom/v1/builder.go b/pkg/storage/bloom/v1/builder.go index fc4868bd0de6a..26b9a39cfd7bf 100644 --- a/pkg/storage/bloom/v1/builder.go +++ b/pkg/storage/bloom/v1/builder.go @@ -15,6 +15,10 @@ import ( "github.com/grafana/loki/pkg/util/encoding" ) +var ( + DefaultBlockOptions = NewBlockOptions(4, 0) +) + type BlockOptions struct { // Schema determines the Schema of the block and cannot be changed Schema Schema @@ -521,7 +525,7 @@ func (mb *MergeBuilder) Build(builder *BlockBuilder) (uint32, error) { func(a, b *SeriesWithBloom) bool { return a.Series.Fingerprint == b.Series.Fingerprint }, - id[*SeriesWithBloom], + Identity[*SeriesWithBloom], func(a, b *SeriesWithBloom) *SeriesWithBloom { if len(a.Series.Chunks) > len(b.Series.Chunks) { return a diff --git a/pkg/storage/bloom/v1/dedupe.go b/pkg/storage/bloom/v1/dedupe.go index a322d8b4b2ef2..2e1a7cca42f36 100644 --- a/pkg/storage/bloom/v1/dedupe.go +++ b/pkg/storage/bloom/v1/dedupe.go @@ -12,7 +12,7 @@ type DedupeIter[A, B any] struct { } // general helper, in this case created for DedupeIter[T,T] -func id[A any](a A) A { return a } +func Identity[A any](a A) A { return a } func NewDedupingIter[A, B any]( eq func(A, B) bool, @@ -52,3 +52,20 @@ func (it *DedupeIter[A, B]) Err() error { func (it *DedupeIter[A, B]) At() B { return it.tmp } + +// Collect collects an interator into a slice. It uses +// CollectInto with a new slice +func Collect[T any](itr Iterator[T]) ([]T, error) { + return CollectInto(itr, nil) +} + +// CollectInto collects the elements of an iterator into a provided slice +// which is returned +func CollectInto[T any](itr Iterator[T], into []T) ([]T, error) { + into = into[:0] + + for itr.Next() { + into = append(into, itr.At()) + } + return into, itr.Err() +} diff --git a/pkg/storage/bloom/v1/dedupe_test.go b/pkg/storage/bloom/v1/dedupe_test.go index 443d8e3e3750e..524e3d4a13a56 100644 --- a/pkg/storage/bloom/v1/dedupe_test.go +++ b/pkg/storage/bloom/v1/dedupe_test.go @@ -28,7 +28,7 @@ func TestMergeDedupeIter(t *testing.T) { } deduper := NewDedupingIter[*SeriesWithBloom, *SeriesWithBloom]( eq, - id[*SeriesWithBloom], + Identity[*SeriesWithBloom], merge, NewPeekingIter[*SeriesWithBloom](mbq), ) diff --git a/pkg/storage/bloom/v1/util.go b/pkg/storage/bloom/v1/util.go index 1716feef35b03..31fcdc643936b 100644 --- a/pkg/storage/bloom/v1/util.go +++ b/pkg/storage/bloom/v1/util.go @@ -241,3 +241,8 @@ func PointerSlice[T any](xs []T) []*T { } return out } + +type CloseableIterator[T any] interface { + Iterator[T] + Close() error +} From 4411649a0e8aae2523e5d7131b8c6b7e78681980 Mon Sep 17 00:00:00 2001 From: Christian Haudum Date: Wed, 31 Jan 2024 19:28:33 +0100 Subject: [PATCH 16/45] (chore) Bloom shipper: Replace `Keyspace` struct with `v1.FingerprintBounds` (#11839) The latter struct has more utility functions to compare and operate on bounds. --------- Signed-off-by: Christian Haudum --- pkg/bloomcompactor/bloomcompactor.go | 2 +- pkg/bloomgateway/processor.go | 8 ++-- pkg/bloomgateway/processor_test.go | 2 +- pkg/bloomgateway/util_test.go | 20 +++++----- pkg/bloomgateway/worker.go | 4 +- pkg/storage/bloom/v1/bounds.go | 5 +++ .../stores/shipper/bloomshipper/client.go | 6 ++- .../shipper/bloomshipper/client_test.go | 3 +- .../shipper/bloomshipper/fetcher_test.go | 23 ++++++----- .../stores/shipper/bloomshipper/shipper.go | 40 ++++++------------- .../shipper/bloomshipper/shipper_test.go | 28 +++++++------ 11 files changed, 68 insertions(+), 73 deletions(-) diff --git a/pkg/bloomcompactor/bloomcompactor.go b/pkg/bloomcompactor/bloomcompactor.go index 52799e498b51e..34885ae2d3947 100644 --- a/pkg/bloomcompactor/bloomcompactor.go +++ b/pkg/bloomcompactor/bloomcompactor.go @@ -503,7 +503,7 @@ func (c *Compactor) runCompact(ctx context.Context, logger log.Logger, job Job, } metaSearchParams := bloomshipper.MetaSearchParams{ TenantID: job.tenantID, - Keyspace: bloomshipper.Keyspace{Min: job.minFp, Max: job.maxFp}, + Keyspace: v1.NewBounds(job.minFp, job.maxFp), Interval: bloomshipper.Interval{Start: job.from, End: job.through}, } var metas []bloomshipper.Meta diff --git a/pkg/bloomgateway/processor.go b/pkg/bloomgateway/processor.go index 460ac3f44e038..5685851aef512 100644 --- a/pkg/bloomgateway/processor.go +++ b/pkg/bloomgateway/processor.go @@ -38,7 +38,7 @@ func (p *processor) run(ctx context.Context, tasks []Task) error { End: ts.Add(Day), } tenant := tasks[0].Tenant - err := p.processTasks(ctx, tenant, interval, []bloomshipper.Keyspace{{Min: 0, Max: math.MaxUint64}}, tasks) + err := p.processTasks(ctx, tenant, interval, []v1.FingerprintBounds{{Min: 0, Max: math.MaxUint64}}, tasks) if err != nil { for _, task := range tasks { task.CloseWithError(err) @@ -52,12 +52,12 @@ func (p *processor) run(ctx context.Context, tasks []Task) error { return nil } -func (p *processor) processTasks(ctx context.Context, tenant string, interval bloomshipper.Interval, keyspaces []bloomshipper.Keyspace, tasks []Task) error { +func (p *processor) processTasks(ctx context.Context, tenant string, interval bloomshipper.Interval, keyspaces []v1.FingerprintBounds, tasks []Task) error { minFpRange, maxFpRange := getFirstLast(keyspaces) metaSearch := bloomshipper.MetaSearchParams{ TenantID: tenant, Interval: interval, - Keyspace: bloomshipper.Keyspace{Min: minFpRange.Min, Max: maxFpRange.Max}, + Keyspace: v1.FingerprintBounds{Min: minFpRange.Min, Max: maxFpRange.Max}, } metas, err := p.store.FetchMetas(ctx, metaSearch) if err != nil { @@ -82,7 +82,7 @@ outer: for blockIter.Next() { bq := blockIter.At() for i, block := range data { - if block.blockRef.MinFingerprint == uint64(bq.MinFp) && block.blockRef.MaxFingerprint == uint64(bq.MaxFp) { + if block.blockRef.Bounds().Equal(bq.FingerprintBounds) { err := p.processBlock(ctx, bq.BlockQuerier, block.tasks) if err != nil { return err diff --git a/pkg/bloomgateway/processor_test.go b/pkg/bloomgateway/processor_test.go index 62c6d42ae18b3..6b43e688a4cc5 100644 --- a/pkg/bloomgateway/processor_test.go +++ b/pkg/bloomgateway/processor_test.go @@ -50,7 +50,7 @@ func (s *dummyStore) LoadBlocks(_ context.Context, refs []bloomshipper.BlockRef) for _, ref := range refs { for _, bq := range s.querieres { - if ref.MinFingerprint == uint64(bq.MinFp) && ref.MaxFingerprint == uint64(bq.MaxFp) { + if ref.Bounds().Equal(bq.FingerprintBounds) { result = append(result, bq) } } diff --git a/pkg/bloomgateway/util_test.go b/pkg/bloomgateway/util_test.go index 61825a8c677ae..969f0ddacd7b6 100644 --- a/pkg/bloomgateway/util_test.go +++ b/pkg/bloomgateway/util_test.go @@ -311,9 +311,8 @@ func createBlockQueriers(t *testing.T, numBlocks int, from, through model.Time, } blockQuerier, data := v1.MakeBlockQuerier(t, fromFp, throughFp, from, through) bq := bloomshipper.BlockQuerierWithFingerprintRange{ - BlockQuerier: blockQuerier, - MinFp: fromFp, - MaxFp: throughFp, + BlockQuerier: blockQuerier, + FingerprintBounds: v1.NewBounds(fromFp, throughFp), } bqs = append(bqs, bq) series = append(series, data) @@ -359,9 +358,8 @@ func createBlocks(t *testing.T, tenant string, n int, from, through model.Time, } blockQuerier, data := v1.MakeBlockQuerier(t, fromFp, throughFp, from, through) querier := bloomshipper.BlockQuerierWithFingerprintRange{ - BlockQuerier: blockQuerier, - MinFp: fromFp, - MaxFp: throughFp, + BlockQuerier: blockQuerier, + FingerprintBounds: v1.NewBounds(fromFp, throughFp), } queriers = append(queriers, querier) metas = append(metas, meta) @@ -392,8 +390,8 @@ func (s *mockBloomStore) GetBlockRefs(_ context.Context, tenant string, _ blooms for i := range s.bqs { blocks = append(blocks, bloomshipper.BlockRef{ Ref: bloomshipper.Ref{ - MinFingerprint: uint64(s.bqs[i].MinFp), - MaxFingerprint: uint64(s.bqs[i].MaxFp), + MinFingerprint: uint64(s.bqs[i].Min), + MaxFingerprint: uint64(s.bqs[i].Max), TenantID: tenant, }, }) @@ -421,7 +419,7 @@ func (s *mockBloomStore) Fetch(_ context.Context, _ string, _ []bloomshipper.Blo for _, bq := range shuffled { // ignore errors in the mock time.Sleep(s.delay) - err := callback(bq.BlockQuerier, uint64(bq.MinFp), uint64(bq.MaxFp)) + err := callback(bq.BlockQuerier, bq.FingerprintBounds) if err != nil { return err } @@ -459,8 +457,8 @@ func createBlockRefsFromBlockData(t *testing.T, tenant string, data []bloomshipp Ref: bloomshipper.Ref{ TenantID: tenant, TableName: "", - MinFingerprint: uint64(data[i].MinFp), - MaxFingerprint: uint64(data[i].MaxFp), + MinFingerprint: uint64(data[i].Min), + MaxFingerprint: uint64(data[i].Max), StartTimestamp: 0, EndTimestamp: 0, Checksum: 0, diff --git a/pkg/bloomgateway/worker.go b/pkg/bloomgateway/worker.go index 3b16fe4fdd7cf..5c6b8a76dbb22 100644 --- a/pkg/bloomgateway/worker.go +++ b/pkg/bloomgateway/worker.go @@ -244,9 +244,9 @@ func (w *worker) stopping(err error) error { } func (w *worker) processBlocksWithCallback(taskCtx context.Context, tenant string, blockRefs []bloomshipper.BlockRef, boundedRefs []boundedTasks) error { - return w.shipper.Fetch(taskCtx, tenant, blockRefs, func(bq *v1.BlockQuerier, minFp, maxFp uint64) error { + return w.shipper.Fetch(taskCtx, tenant, blockRefs, func(bq *v1.BlockQuerier, bounds v1.FingerprintBounds) error { for _, b := range boundedRefs { - if b.blockRef.MinFingerprint == minFp && b.blockRef.MaxFingerprint == maxFp { + if b.blockRef.Bounds().Equal(bounds) { return w.processBlock(bq, b.tasks) } } diff --git a/pkg/storage/bloom/v1/bounds.go b/pkg/storage/bloom/v1/bounds.go index 6aff8ae2c1706..a41d70a89d867 100644 --- a/pkg/storage/bloom/v1/bounds.go +++ b/pkg/storage/bloom/v1/bounds.go @@ -68,6 +68,11 @@ func (b FingerprintBounds) Within(target FingerprintBounds) bool { return b.Min >= target.Min && b.Max <= target.Max } +// Returns whether the fingerprint bounds is equal to the target bounds +func (b FingerprintBounds) Equal(target FingerprintBounds) bool { + return b.Min == target.Min && b.Max == target.Max +} + // Intersection returns the intersection of the two bounds func (b FingerprintBounds) Intersection(target FingerprintBounds) *FingerprintBounds { if !b.Overlaps(target) { diff --git a/pkg/storage/stores/shipper/bloomshipper/client.go b/pkg/storage/stores/shipper/bloomshipper/client.go index 8d980782d044f..835ee13686d37 100644 --- a/pkg/storage/stores/shipper/bloomshipper/client.go +++ b/pkg/storage/stores/shipper/bloomshipper/client.go @@ -51,6 +51,10 @@ type BlockRef struct { BlockPath string } +func (b *BlockRef) Bounds() v1.FingerprintBounds { + return v1.NewBounds(model.Fingerprint(b.MinFingerprint), model.Fingerprint(b.MaxFingerprint)) +} + type MetaRef struct { Ref FilePath string @@ -67,7 +71,7 @@ type Meta struct { type MetaSearchParams struct { TenantID string Interval Interval - Keyspace Keyspace + Keyspace v1.FingerprintBounds } type MetaClient interface { diff --git a/pkg/storage/stores/shipper/bloomshipper/client_test.go b/pkg/storage/stores/shipper/bloomshipper/client_test.go index 28e8c3a02f0e3..30aac5c901e08 100644 --- a/pkg/storage/stores/shipper/bloomshipper/client_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/client_test.go @@ -20,6 +20,7 @@ import ( "github.com/stretchr/testify/require" "github.com/grafana/loki/pkg/storage" + v1 "github.com/grafana/loki/pkg/storage/bloom/v1" "github.com/grafana/loki/pkg/storage/chunk/cache" "github.com/grafana/loki/pkg/storage/config" ) @@ -72,7 +73,7 @@ func Test_BloomClient_FetchMetas(t *testing.T) { searchParams := MetaSearchParams{ TenantID: "tenantA", - Keyspace: Keyspace{Min: 50, Max: 150}, + Keyspace: v1.NewBounds(50, 150), Interval: Interval{Start: fixedDay.Add(-6 * day), End: fixedDay.Add(-1*day - 1*time.Hour)}, } diff --git a/pkg/storage/stores/shipper/bloomshipper/fetcher_test.go b/pkg/storage/stores/shipper/bloomshipper/fetcher_test.go index 3e0ea64c6f253..85117a718f629 100644 --- a/pkg/storage/stores/shipper/bloomshipper/fetcher_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/fetcher_test.go @@ -12,12 +12,13 @@ import ( "github.com/prometheus/common/model" "github.com/stretchr/testify/require" + v1 "github.com/grafana/loki/pkg/storage/bloom/v1" "github.com/grafana/loki/pkg/storage/chunk/cache" "github.com/grafana/loki/pkg/storage/chunk/client/local" "github.com/grafana/loki/pkg/storage/config" ) -func makeMetas(t *testing.T, schemaCfg config.SchemaConfig, ts model.Time, keyspaces []Keyspace) []Meta { +func makeMetas(t *testing.T, schemaCfg config.SchemaConfig, ts model.Time, keyspaces []v1.FingerprintBounds) []Meta { t.Helper() metas := make([]Meta, len(keyspaces)) @@ -76,23 +77,23 @@ func TestMetasFetcher(t *testing.T) { { name: "all metas found in cache", store: []Meta{}, - start: makeMetas(t, schemaCfg, now, []Keyspace{{0x0000, 0xffff}}), - end: makeMetas(t, schemaCfg, now, []Keyspace{{0x0000, 0xffff}}), - fetch: makeMetas(t, schemaCfg, now, []Keyspace{{0x0000, 0xffff}}), + start: makeMetas(t, schemaCfg, now, []v1.FingerprintBounds{{Min: 0x0000, Max: 0xffff}}), + end: makeMetas(t, schemaCfg, now, []v1.FingerprintBounds{{Min: 0x0000, Max: 0xffff}}), + fetch: makeMetas(t, schemaCfg, now, []v1.FingerprintBounds{{Min: 0x0000, Max: 0xffff}}), }, { name: "no metas found in cache", - store: makeMetas(t, schemaCfg, now, []Keyspace{{0x0000, 0xffff}}), + store: makeMetas(t, schemaCfg, now, []v1.FingerprintBounds{{Min: 0x0000, Max: 0xffff}}), start: []Meta{}, - end: makeMetas(t, schemaCfg, now, []Keyspace{{0x0000, 0xffff}}), - fetch: makeMetas(t, schemaCfg, now, []Keyspace{{0x0000, 0xffff}}), + end: makeMetas(t, schemaCfg, now, []v1.FingerprintBounds{{Min: 0x0000, Max: 0xffff}}), + fetch: makeMetas(t, schemaCfg, now, []v1.FingerprintBounds{{Min: 0x0000, Max: 0xffff}}), }, { name: "some metas found in cache", - store: makeMetas(t, schemaCfg, now, []Keyspace{{0x0000, 0xffff}, {0x10000, 0x1ffff}}), - start: makeMetas(t, schemaCfg, now, []Keyspace{{0x0000, 0xffff}}), - end: makeMetas(t, schemaCfg, now, []Keyspace{{0x0000, 0xffff}, {0x10000, 0x1ffff}}), - fetch: makeMetas(t, schemaCfg, now, []Keyspace{{0x0000, 0xffff}, {0x10000, 0x1ffff}}), + store: makeMetas(t, schemaCfg, now, []v1.FingerprintBounds{{Min: 0x0000, Max: 0xffff}, {Min: 0x10000, Max: 0x1ffff}}), + start: makeMetas(t, schemaCfg, now, []v1.FingerprintBounds{{Min: 0x0000, Max: 0xffff}}), + end: makeMetas(t, schemaCfg, now, []v1.FingerprintBounds{{Min: 0x0000, Max: 0xffff}, {Min: 0x10000, Max: 0x1ffff}}), + fetch: makeMetas(t, schemaCfg, now, []v1.FingerprintBounds{{Min: 0x0000, Max: 0xffff}, {Min: 0x10000, Max: 0x1ffff}}), }, } diff --git a/pkg/storage/stores/shipper/bloomshipper/shipper.go b/pkg/storage/stores/shipper/bloomshipper/shipper.go index e3ab1db70c539..54c2185fae56d 100644 --- a/pkg/storage/stores/shipper/bloomshipper/shipper.go +++ b/pkg/storage/stores/shipper/bloomshipper/shipper.go @@ -32,25 +32,12 @@ func (i Interval) Cmp(other model.Time) v1.BoundsCheck { return v1.Overlap } -type Keyspace struct { - Min, Max model.Fingerprint -} - -func (r Keyspace) Cmp(other model.Fingerprint) v1.BoundsCheck { - if other < r.Min { - return v1.Before - } else if other > r.Max { - return v1.After - } - return v1.Overlap -} - type BlockQuerierWithFingerprintRange struct { *v1.BlockQuerier - MinFp, MaxFp model.Fingerprint + v1.FingerprintBounds } -type ForEachBlockCallback func(bq *v1.BlockQuerier, minFp, maxFp uint64) error +type ForEachBlockCallback func(bq *v1.BlockQuerier, bounds v1.FingerprintBounds) error type Interface interface { GetBlockRefs(ctx context.Context, tenant string, interval Interval) ([]BlockRef, error) @@ -93,8 +80,8 @@ func (s *Shipper) GetBlockRefs(ctx context.Context, tenantID string, interval In 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 - keyspaces := []Keyspace{{0, math.MaxUint64}} - blockRefs, err := s.getActiveBlockRefs(ctx, tenantID, interval, keyspaces) + 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) } @@ -136,7 +123,7 @@ func runCallback(callback ForEachBlockCallback, block blockWithQuerier) error { _ = b.Close() }(block) - err := callback(block.closableBlockQuerier.BlockQuerier, block.MinFingerprint, block.MaxFingerprint) + err := callback(block.closableBlockQuerier.BlockQuerier, block.Bounds()) if err != nil { return fmt.Errorf("error running callback function for block %s err: %w", block.BlockPath, err) } @@ -158,11 +145,11 @@ func getFirstLast[T any](s []T) (T, T) { return s[0], s[len(s)-1] } -func (s *Shipper) getActiveBlockRefs(ctx context.Context, tenantID string, interval Interval, keyspaces []Keyspace) ([]BlockRef, error) { - minFpRange, maxFpRange := getFirstLast(keyspaces) +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: Keyspace{Min: minFpRange.Min, Max: maxFpRange.Max}, + Keyspace: v1.NewBounds(minFpRange.Min, maxFpRange.Max), Interval: interval, }) if err != nil { @@ -170,10 +157,10 @@ func (s *Shipper) getActiveBlockRefs(ctx context.Context, tenantID string, inter } level.Debug(s.logger).Log("msg", "dowloaded metas", "count", len(metas)) - return BlocksForMetas(metas, interval, keyspaces), nil + return BlocksForMetas(metas, interval, bounds), nil } -func BlocksForMetas(metas []Meta, interval Interval, keyspaces []Keyspace) []BlockRef { +func BlocksForMetas(metas []Meta, interval Interval, keyspaces []v1.FingerprintBounds) []BlockRef { tombstones := make(map[string]interface{}) for _, meta := range metas { for _, tombstone := range meta.Tombstones { @@ -216,7 +203,7 @@ func BlocksForMetas(metas []Meta, interval Interval, keyspaces []Keyspace) []Blo // isOutsideRange tests if a given BlockRef b is outside of search boundaries // defined by min/max timestamp and min/max fingerprint. // Fingerprint ranges must be sorted in ascending order. -func isOutsideRange(b BlockRef, interval Interval, keyspaces []Keyspace) bool { +func isOutsideRange(b BlockRef, interval Interval, keyspaces []v1.FingerprintBounds) bool { // check time interval if interval.Cmp(b.EndTimestamp) == v1.Before || interval.Cmp(b.StartTimestamp) == v1.After { return true @@ -224,10 +211,7 @@ func isOutsideRange(b BlockRef, interval Interval, keyspaces []Keyspace) bool { // check fingerprint ranges for _, keyspace := range keyspaces { - if keyspace.Cmp(model.Fingerprint(b.MinFingerprint)) == v1.Before && keyspace.Cmp(model.Fingerprint(b.MaxFingerprint)) == v1.After { - return false - } - if keyspace.Cmp(model.Fingerprint(b.MinFingerprint)) == v1.Overlap || keyspace.Cmp(model.Fingerprint(b.MaxFingerprint)) == v1.Overlap { + if keyspace.Within(b.Bounds()) || keyspace.Overlaps(b.Bounds()) { return false } } diff --git a/pkg/storage/stores/shipper/bloomshipper/shipper_test.go b/pkg/storage/stores/shipper/bloomshipper/shipper_test.go index 57d360de3b80d..d2311f808e26f 100644 --- a/pkg/storage/stores/shipper/bloomshipper/shipper_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/shipper_test.go @@ -8,6 +8,8 @@ import ( "github.com/prometheus/common/model" "github.com/stretchr/testify/require" + + v1 "github.com/grafana/loki/pkg/storage/bloom/v1" ) func interval(start, end model.Time) Interval { @@ -48,7 +50,7 @@ func Test_Shipper_findBlocks(t *testing.T) { Start: ts.Add(-2 * time.Hour), End: ts.Add(-1 * time.Hour), } - blocks := BlocksForMetas(metas, interval, []Keyspace{{Min: 100, Max: 200}}) + blocks := BlocksForMetas(metas, interval, []v1.FingerprintBounds{{Min: 100, Max: 200}}) expectedBlockRefs := []BlockRef{ createMatchingBlockRef("block2"), @@ -101,7 +103,7 @@ func Test_Shipper_findBlocks(t *testing.T) { for name, data := range tests { t.Run(name, func(t *testing.T) { ref := createBlockRef("fake-block", data.minFingerprint, data.maxFingerprint, data.startTimestamp, data.endTimestamp) - blocks := BlocksForMetas([]Meta{{Blocks: []BlockRef{ref}}}, interval(300, 400), []Keyspace{{Min: 100, Max: 200}}) + blocks := BlocksForMetas([]Meta{{Blocks: []BlockRef{ref}}}, interval(300, 400), []v1.FingerprintBounds{{Min: 100, Max: 200}}) if data.filtered { require.Empty(t, blocks) return @@ -118,67 +120,67 @@ func TestIsOutsideRange(t *testing.T) { t.Run("is outside if startTs > through", func(t *testing.T) { b := createBlockRef("block", 0, math.MaxUint64, startTs, endTs) - isOutside := isOutsideRange(b, interval(0, 900), []Keyspace{}) + isOutside := isOutsideRange(b, interval(0, 900), []v1.FingerprintBounds{}) require.True(t, isOutside) }) t.Run("is outside if startTs == through ", func(t *testing.T) { b := createBlockRef("block", 0, math.MaxUint64, startTs, endTs) - isOutside := isOutsideRange(b, interval(900, 1000), []Keyspace{}) + isOutside := isOutsideRange(b, interval(900, 1000), []v1.FingerprintBounds{}) require.True(t, isOutside) }) t.Run("is outside if endTs < from", func(t *testing.T) { b := createBlockRef("block", 0, math.MaxUint64, startTs, endTs) - isOutside := isOutsideRange(b, interval(2100, 3000), []Keyspace{}) + isOutside := isOutsideRange(b, interval(2100, 3000), []v1.FingerprintBounds{}) require.True(t, isOutside) }) t.Run("is outside if endFp < first fingerprint", func(t *testing.T) { b := createBlockRef("block", 0, 90, startTs, endTs) - isOutside := isOutsideRange(b, interval(startTs, endTs), []Keyspace{{100, 199}}) + isOutside := isOutsideRange(b, interval(startTs, endTs), []v1.FingerprintBounds{{Min: 100, Max: 199}}) require.True(t, isOutside) }) t.Run("is outside if startFp > last fingerprint", func(t *testing.T) { b := createBlockRef("block", 200, math.MaxUint64, startTs, endTs) - isOutside := isOutsideRange(b, interval(startTs, endTs), []Keyspace{{0, 49}, {100, 149}}) + isOutside := isOutsideRange(b, interval(startTs, endTs), []v1.FingerprintBounds{{Min: 0, Max: 49}, {Min: 100, Max: 149}}) require.True(t, isOutside) }) t.Run("is outside if within gaps in fingerprints", func(t *testing.T) { b := createBlockRef("block", 100, 199, startTs, endTs) - isOutside := isOutsideRange(b, interval(startTs, endTs), []Keyspace{{0, 99}, {200, 299}}) + isOutside := isOutsideRange(b, interval(startTs, endTs), []v1.FingerprintBounds{{Min: 0, Max: 99}, {Min: 200, Max: 299}}) require.True(t, isOutside) }) t.Run("is not outside if within fingerprints 1", func(t *testing.T) { b := createBlockRef("block", 10, 90, startTs, endTs) - isOutside := isOutsideRange(b, interval(startTs, endTs), []Keyspace{{0, 99}, {200, 299}}) + isOutside := isOutsideRange(b, interval(startTs, endTs), []v1.FingerprintBounds{{Min: 0, Max: 99}, {Min: 200, Max: 299}}) require.False(t, isOutside) }) t.Run("is not outside if within fingerprints 2", func(t *testing.T) { b := createBlockRef("block", 210, 290, startTs, endTs) - isOutside := isOutsideRange(b, interval(startTs, endTs), []Keyspace{{0, 99}, {200, 299}}) + isOutside := isOutsideRange(b, interval(startTs, endTs), []v1.FingerprintBounds{{Min: 0, Max: 99}, {Min: 200, Max: 299}}) require.False(t, isOutside) }) t.Run("is not outside if spans across multiple fingerprint ranges", func(t *testing.T) { b := createBlockRef("block", 50, 250, startTs, endTs) - isOutside := isOutsideRange(b, interval(startTs, endTs), []Keyspace{{0, 99}, {200, 299}}) + isOutside := isOutsideRange(b, interval(startTs, endTs), []v1.FingerprintBounds{{Min: 0, Max: 99}, {Min: 200, Max: 299}}) require.False(t, isOutside) }) t.Run("is not outside if fingerprint range and time range are larger than block", func(t *testing.T) { b := createBlockRef("block", math.MaxUint64/3, math.MaxUint64/3*2, startTs, endTs) - isOutside := isOutsideRange(b, interval(0, 3000), []Keyspace{{0, math.MaxUint64}}) + isOutside := isOutsideRange(b, interval(0, 3000), []v1.FingerprintBounds{{Min: 0, Max: math.MaxUint64}}) require.False(t, isOutside) }) t.Run("is not outside if block fingerprint range is bigger that search keyspace", func(t *testing.T) { b := createBlockRef("block", 0x0000, 0xffff, model.Earliest, model.Latest) - isOutside := isOutsideRange(b, interval(startTs, endTs), []Keyspace{{0x0100, 0xff00}}) + isOutside := isOutsideRange(b, interval(startTs, endTs), []v1.FingerprintBounds{{Min: 0x0100, Max: 0xff00}}) require.False(t, isOutside) }) } From 8289ca71887dcdb1715d6e0ecd42398af8719c2c Mon Sep 17 00:00:00 2001 From: Steven Dungan <114922977+stevendungan@users.noreply.github.com> Date: Wed, 31 Jan 2024 14:54:26 -0500 Subject: [PATCH 17/45] Structured metadata docs update (#11843) * Rewrote description to highlight primary use case * Added defaults for how much structured metadata can be added to log lines * Rewrote example queries **What this PR does / why we need it**: **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) --- .../get-started/labels/structured-metadata.md | 38 ++++++++++++------- 1 file changed, 24 insertions(+), 14 deletions(-) diff --git a/docs/sources/get-started/labels/structured-metadata.md b/docs/sources/get-started/labels/structured-metadata.md index db335e7712316..071339cc0bde3 100644 --- a/docs/sources/get-started/labels/structured-metadata.md +++ b/docs/sources/get-started/labels/structured-metadata.md @@ -6,24 +6,22 @@ description: Describes how to enable structure metadata for logs and how to quer # What is structured metadata {{% admonition type="warning" %}} -Structured metadata is an experimental feature and is subject to change in future releases of Grafana Loki. +Structured metadata is an experimental feature and is subject to change in future releases of Grafana Loki. This feature is not yet available for Cloud Logs users. {{% /admonition %}} {{% admonition type="warning" %}} Structured metadata was added to chunk format V4 which is used if the schema version is greater or equal to `13`. (See [Schema Config]({{< relref "../../storage#schema-config" >}}) for more details about schema versions. ) {{% /admonition %}} -One of the powerful features of Loki is parsing logs at query time to extract metadata and build labels out of it. -However, the parsing of logs at query time comes with a cost which can be significantly high for, as an example, -large JSON blobs or a poorly written query using complex regex patterns. +Selecting proper, low cardinality labels is critical to operating and querying Loki effectively. Some metadata, especially infrastructure related metadata, can be difficult to embed in log lines, and is too high cardinality to effectively store as indexed labels (and therefore reducing performance of the index). -In addition, the data extracted from logs at query time is usually high cardinality, which can’t be stored -in the index as it would increase the cardinality too much, and therefore reduce the performance of the index. - -Structured metadata is a way to attach metadata to logs without indexing them. Examples of useful metadata are -trace IDs, user IDs, and any other label that is often used in queries but has high cardinality and is expensive +Structured metadata is a way to attach metadata to logs without indexing them or including them in the log line content itself. Examples of useful metadata are +kubernetes pod names, process ID's, or any other label that is often used in queries but has high cardinality and is expensive to extract at query time. +Structured metadata can also be used to query commonly needed metadata from log lines without needing to apply a parser at query time. Large json blobs or a poorly written query using complex regex patterns, for example, come with a high performance cost. Examples of useful metadata include trace IDs or user IDs. + + ## Attaching structured metadata to log lines You have the option to attach structured metadata to log lines in the push payload along with each log line and the timestamp. @@ -34,25 +32,37 @@ See the [Promtail: Structured metadata stage]({{< relref "../../send-data/promta With Loki version 1.2.0, support for structured metadata has been added to the Logstash output plugin. For more information, see [logstash]({{< relref "../../send-data/logstash/_index.md" >}}). +{{% admonition type="warning" %}} +There are defaults for how much structured metadata can be attached per log line. +``` +# Maximum size accepted for structured metadata per log line. +# CLI flag: -limits.max-structured-metadata-size +[max_structured_metadata_size: | default = 64KB] + +# Maximum number of structured metadata entries per log line. +# CLI flag: -limits.max-structured-metadata-entries-count +[max_structured_metadata_entries_count: | default = 128] +``` +{{% /admonition %}} + ## Querying structured metadata Structured metadata is extracted automatically for each returned log line and added to the labels returned for the query. You can use labels of structured metadata to filter log line using a [label filter expression]({{< relref "../../query/log_queries#label-filter-expression" >}}). -For example, if you have a label `trace_id` attached to some of your log lines as structured metadata, you can filter log lines using: +For example, if you have a label `pod` attached to some of your log lines as structured metadata, you can filter log lines using: ```logql -{job="example"} | trace_id="0242ac120002" +{job="example"} | pod="myservice-abc1234-56789"` ``` Of course, you can filter by multiple labels of structured metadata at the same time: ```logql -{job="example"} | trace_id="0242ac120002" | user_id="superUser123" +{job="example"} | pod="myservice-abc1234-56789" | trace_id="0242ac120002" ``` -Note that since structured metadata is extracted automatically to the results labels, some metric queries might return -an error like `maximum of series (50000) reached for a single query`. You can use the [Keep]({{< relref "../../query/log_queries#keep-labels-expression" >}}) and [Drop]({{< relref "../../query/log_queries#drop-labels-expression" >}}) stages to filter out labels that you don't need. +Note that since structured metadata is extracted automatically to the results labels, some metric queries might return an error like `maximum of series (50000) reached for a single query`. You can use the [Keep]({{< relref "../../query/log_queries#keep-labels-expression" >}}) and [Drop]({{< relref "../../query/log_queries#drop-labels-expression" >}}) stages to filter out labels that you don't need. For example: ```logql From 928e3f34f8669828504a7319d5f230d03e42fbc4 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 31 Jan 2024 23:36:22 -0800 Subject: [PATCH 18/45] Blooms: Prepare compactor refactor (#11847) Prepares `bloomcompactor` pkg for integration with new logic, primarily removes/thins out existing code. --- pkg/bloomcompactor/bloomcompactor.go | 475 ++---------------- pkg/bloomcompactor/bloomcompactor_test.go | 245 --------- pkg/bloomcompactor/chunkcompactor.go | 245 --------- pkg/bloomcompactor/chunkcompactor_test.go | 229 --------- pkg/bloomcompactor/chunksbatchesiterator.go | 48 -- .../chunksbatchesiterator_test.go | 96 ---- .../{v2controller.go => controller.go} | 0 ...2controller_test.go => controller_test.go} | 0 pkg/bloomcompactor/job.go | 85 ---- pkg/bloomcompactor/mergecompactor.go | 150 ------ pkg/bloomcompactor/{v2_meta.go => meta.go} | 0 pkg/bloomcompactor/{v2spec.go => spec.go} | 0 .../{v2spec_test.go => spec_test.go} | 0 pkg/bloomcompactor/table_utils.go | 21 - pkg/bloomcompactor/utils.go | 37 -- pkg/compactor/compactor.go | 8 +- pkg/compactor/compactor_test.go | 4 +- pkg/loki/modules.go | 6 +- 18 files changed, 50 insertions(+), 1599 deletions(-) delete mode 100644 pkg/bloomcompactor/bloomcompactor_test.go delete mode 100644 pkg/bloomcompactor/chunkcompactor.go delete mode 100644 pkg/bloomcompactor/chunkcompactor_test.go delete mode 100644 pkg/bloomcompactor/chunksbatchesiterator.go delete mode 100644 pkg/bloomcompactor/chunksbatchesiterator_test.go rename pkg/bloomcompactor/{v2controller.go => controller.go} (100%) rename pkg/bloomcompactor/{v2controller_test.go => controller_test.go} (100%) delete mode 100644 pkg/bloomcompactor/job.go delete mode 100644 pkg/bloomcompactor/mergecompactor.go rename pkg/bloomcompactor/{v2_meta.go => meta.go} (100%) rename pkg/bloomcompactor/{v2spec.go => spec.go} (100%) rename pkg/bloomcompactor/{v2spec_test.go => spec_test.go} (100%) delete mode 100644 pkg/bloomcompactor/utils.go diff --git a/pkg/bloomcompactor/bloomcompactor.go b/pkg/bloomcompactor/bloomcompactor.go index 34885ae2d3947..a3862408593db 100644 --- a/pkg/bloomcompactor/bloomcompactor.go +++ b/pkg/bloomcompactor/bloomcompactor.go @@ -1,198 +1,71 @@ -/* -Bloom-compactor - -This is a standalone service that is responsible for compacting TSDB indexes into bloomfilters. -It creates and merges bloomfilters into an aggregated form, called bloom-blocks. -It maintains a list of references between bloom-blocks and TSDB indexes in files called meta.jsons. - -Bloom-compactor regularly runs to check for changes in meta.jsons and runs compaction only upon changes in TSDBs. - -bloomCompactor.Compactor - - | // Read/Write path - bloomshipper.Store** - | - bloomshipper.Shipper - | - bloomshipper.BloomClient - | - ObjectClient - | - .....................service boundary - | - object storage -*/ package bloomcompactor import ( "context" "fmt" - "io" - "math" - "math/rand" - "os" "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/prometheus/prometheus/model/labels" - - "path/filepath" - - "github.com/google/uuid" "github.com/grafana/loki/pkg/bloomutils" - "github.com/grafana/loki/pkg/storage" + "github.com/grafana/loki/pkg/compactor" v1 "github.com/grafana/loki/pkg/storage/bloom/v1" - "github.com/grafana/loki/pkg/storage/chunk/cache" - chunk_client "github.com/grafana/loki/pkg/storage/chunk/client" - "github.com/grafana/loki/pkg/storage/chunk/client/local" - "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" - shipperindex "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/index" - index_storage "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/storage" - "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb" - tsdbindex "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/index" "github.com/grafana/loki/pkg/util" ) +/* +Bloom-compactor + +This is a standalone service that is responsible for compacting TSDB indexes into bloomfilters. +It creates and merges bloomfilters into an aggregated form, called bloom-blocks. +It maintains a list of references between bloom-blocks and TSDB indexes in files called meta.jsons. + +Bloom-compactor regularly runs to check for changes in meta.jsons and runs compaction only upon changes in TSDBs. +*/ type Compactor struct { services.Service - cfg Config - logger log.Logger - schemaCfg config.SchemaConfig - limits Limits + cfg Config + logger log.Logger + limits Limits // temporary workaround until store has implemented read/write shipper interface - bloomShipperClient bloomshipper.StoreAndClient - - // Client used to run operations on the bucket storing bloom blocks. - storeClients map[config.DayTime]storeClient + store bloomshipper.StoreAndClient sharding ShardingStrategy metrics *metrics btMetrics *v1.Metrics - reg prometheus.Registerer -} - -type storeClient struct { - object chunk_client.ObjectClient - index index_storage.Client - chunk chunk_client.Client - indexShipper indexshipper.IndexShipper } func New( cfg Config, - storageCfg storage.Config, - schemaConfig config.SchemaConfig, + store bloomshipper.StoreAndClient, + sharding ShardingStrategy, limits Limits, logger log.Logger, - sharding ShardingStrategy, - clientMetrics storage.ClientMetrics, r prometheus.Registerer, ) (*Compactor, error) { c := &Compactor{ - cfg: cfg, - logger: logger, - schemaCfg: schemaConfig, - sharding: sharding, - limits: limits, - reg: r, + cfg: cfg, + store: store, + logger: logger, + sharding: sharding, + limits: limits, } - // TODO(chaudum): Plug in cache - var metasCache cache.Cache - var blocksCache *cache.EmbeddedCache[string, io.ReadCloser] - bloomClient, err := bloomshipper.NewBloomStore(schemaConfig.Configs, storageCfg, clientMetrics, metasCache, blocksCache, logger) - if err != nil { - return nil, err - } - - c.storeClients = make(map[config.DayTime]storeClient) - // initialize metrics c.btMetrics = v1.NewMetrics(prometheus.WrapRegistererWithPrefix("loki_bloom_tokenizer", r)) - - indexShipperReg := prometheus.WrapRegistererWithPrefix("loki_bloom_compactor_tsdb_shipper_", r) - - for i, periodicConfig := range schemaConfig.Configs { - if periodicConfig.IndexType != config.TSDBType { - level.Warn(c.logger).Log("msg", "skipping schema period because index type is not supported", "index_type", periodicConfig.IndexType, "period", periodicConfig.From) - continue - } - - // Configure ObjectClient and IndexShipper for series and chunk management - objectClient, err := storage.NewObjectClient(periodicConfig.ObjectType, storageCfg, clientMetrics) - if err != nil { - return nil, fmt.Errorf("error creating object client '%s': %w", periodicConfig.ObjectType, err) - } - - periodEndTime := config.DayTime{Time: math.MaxInt64} - if i < len(schemaConfig.Configs)-1 { - periodEndTime = config.DayTime{Time: schemaConfig.Configs[i+1].From.Time.Add(-time.Millisecond)} - } - - pReg := prometheus.WrapRegistererWith( - prometheus.Labels{ - "component": fmt.Sprintf( - "index-store-%s-%s", - periodicConfig.IndexType, - periodicConfig.From.String(), - ), - }, indexShipperReg) - pLogger := log.With(logger, "index-store", fmt.Sprintf("%s-%s", periodicConfig.IndexType, periodicConfig.From.String())) - - indexShipper, err := indexshipper.NewIndexShipper( - periodicConfig.IndexTables.PathPrefix, - storageCfg.TSDBShipperConfig, - objectClient, - limits, - nil, - func(p string) (shipperindex.Index, error) { - return tsdb.OpenShippableTSDB(p) - }, - periodicConfig.GetIndexTableNumberRange(periodEndTime), - pReg, - pLogger, - ) - - if err != nil { - return nil, errors.Wrap(err, "create index shipper") - } - - // The ObjectClient does not expose the key encoder it uses, - // so check the concrete type and set the FSEncoder if needed. - var keyEncoder chunk_client.KeyEncoder - switch objectClient.(type) { - case *local.FSObjectClient: - keyEncoder = chunk_client.FSEncoder - } - - c.storeClients[periodicConfig.From] = storeClient{ - object: objectClient, - index: index_storage.NewIndexStorageClient(objectClient, periodicConfig.IndexTables.PathPrefix), - chunk: chunk_client.NewClient(objectClient, keyEncoder, schemaConfig), - indexShipper: indexShipper, - } - } - - // temporary workaround until store has implemented read/write shipper interface - c.bloomShipperClient = bloomClient - c.metrics = newMetrics(r) c.metrics.compactionRunInterval.Set(cfg.CompactionInterval.Seconds()) - c.Service = services.NewBasicService(c.starting, c.running, c.stopping) return c, nil @@ -237,40 +110,23 @@ func (c *Compactor) stopping(_ error) error { func (c *Compactor) runCompaction(ctx context.Context) error { var tables []string - for _, sc := range c.storeClients { - // refresh index list cache since previous compaction would have changed the index files in the object store - sc.index.RefreshIndexTableNamesCache(ctx) - tbls, err := sc.index.ListTables(ctx) - if err != nil { - return fmt.Errorf("failed to list tables: %w", err) - } - tables = append(tables, tbls...) - } + // TODO(owen-d): resolve tables // process most recent tables first tablesIntervals := getIntervalsForTables(tables) - sortTablesByRange(tables, tablesIntervals) - - parallelism := c.cfg.MaxCompactionParallelism - if parallelism == 0 { - parallelism = len(tables) - } + compactor.SortTablesByRange(tables) - // TODO(salvacorts): We currently parallelize at the table level. We may want to parallelize at the tenant and job level as well. - // To do that, we should create a worker pool with c.cfg.MaxCompactionParallelism number of workers. - errs := multierror.New() - _ = concurrency.ForEachJob(ctx, len(tables), parallelism, func(ctx context.Context, i int) error { - tableName := tables[i] - logger := log.With(c.logger, "table", tableName) - err := c.compactTable(ctx, logger, tableName, tablesIntervals[tableName]) + // 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 { - errs.Add(err) - return nil + level.Error(logger).Log("msg", "failed to compact table", "err", err) + return errors.Wrapf(err, "failed to compact table %s", table) } - return nil - }) - - return errs.Err() + } + return nil } func (c *Compactor) compactTable(ctx context.Context, logger log.Logger, tableName string, tableInterval model.Interval) error { @@ -279,29 +135,13 @@ func (c *Compactor) compactTable(ctx context.Context, logger log.Logger, tableNa return fmt.Errorf("interrupting compaction of table: %w", err) } - schemaCfg, ok := schemaPeriodForTable(c.schemaCfg, tableName) - if !ok { - level.Error(logger).Log("msg", "skipping compaction since we can't find schema for table") - return nil - } - - sc, ok := c.storeClients[schemaCfg.From] - if !ok { - return fmt.Errorf("index store client not found for period starting at %s", schemaCfg.From.String()) - } - - _, tenants, err := sc.index.ListFiles(ctx, tableName, true) - if err != nil { - return fmt.Errorf("failed to list files for table %s: %w", tableName, err) - } + var tenants []string - c.metrics.compactionRunDiscoveredTenants.Add(float64(len(tenants))) level.Info(logger).Log("msg", "discovered tenants from bucket", "users", len(tenants)) - return c.compactUsers(ctx, logger, sc, tableName, tableInterval, tenants) + return c.compactUsers(ctx, logger, tableName, tableInterval, tenants) } -// See: https://github.com/grafana/mimir/blob/34852137c332d4050e53128481f4f6417daee91e/pkg/compactor/compactor.go#L566-L689 -func (c *Compactor) compactUsers(ctx context.Context, logger log.Logger, sc storeClient, tableName string, tableInterval model.Interval, tenants []string) error { +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)) @@ -337,7 +177,7 @@ func (c *Compactor) compactUsers(ctx context.Context, logger log.Logger, sc stor ownedTenants[tenant] = struct{}{} start := time.Now() - if err := c.compactTenantWithRetries(ctx, tenantLogger, sc, tableName, tenant); err != nil { + 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. @@ -362,7 +202,7 @@ func (c *Compactor) compactUsers(ctx context.Context, logger log.Logger, sc stor // TODO: Delete local files for unowned tenants, if there are any. } -func (c *Compactor) compactTenant(ctx context.Context, logger log.Logger, sc storeClient, tableName string, tenant string) error { +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). @@ -373,9 +213,8 @@ func (c *Compactor) compactTenant(ctx context.Context, logger log.Logger, sc sto // Tokenizer is not thread-safe so we need one per goroutine. nGramLen := c.limits.BloomNGramLength(tenant) nGramSkip := c.limits.BloomNGramSkip(tenant) - bt := v1.NewBloomTokenizer(nGramLen, nGramSkip, c.btMetrics) + _ = v1.NewBloomTokenizer(nGramLen, nGramSkip, c.btMetrics) - errs := multierror.New() rs, err := c.sharding.GetTenantSubRing(tenant).GetAllHealthy(RingOp) if err != nil { return err @@ -385,77 +224,8 @@ func (c *Compactor) compactTenant(ctx context.Context, logger log.Logger, sc sto level.Debug(logger).Log("msg", "got token range for instance", "id", tr.Instance.Id, "min", tr.MinToken, "max", tr.MaxToken) } - // TODO(owen-d): can be optimized to only query for series within the fp range of the compactor shard(s) rather than scanning all series - // and filtering out the ones that don't belong to the compactor shard(s). - _ = sc.indexShipper.ForEach(ctx, tableName, tenant, func(isMultiTenantIndex bool, idx shipperindex.Index) error { - if isMultiTenantIndex { - // Skip multi-tenant indexes - level.Debug(logger).Log("msg", "skipping multi-tenant index", "table", tableName, "index", idx.Name()) - return nil - } - - tsdbFile, ok := idx.(*tsdb.TSDBFile) - if !ok { - errs.Add(fmt.Errorf("failed to cast to TSDBFile")) - return nil - } - - tsdbIndex, ok := tsdbFile.Index.(*tsdb.TSDBIndex) - if !ok { - errs.Add(fmt.Errorf("failed to cast to TSDBIndex")) - return nil - } - - var seriesMetas []seriesMeta - - err := tsdbIndex.ForSeries( - ctx, nil, - 0, math.MaxInt64, // TODO: Replace with MaxLookBackPeriod - func(labels labels.Labels, fingerprint model.Fingerprint, chksMetas []tsdbindex.ChunkMeta) { - if !tokenRanges.Contains(uint32(fingerprint)) { - return - } - - temp := make([]tsdbindex.ChunkMeta, len(chksMetas)) - ls := labels.Copy() - _ = copy(temp, chksMetas) - //All seriesMetas given a table within fp of this compactor shard - seriesMetas = append(seriesMetas, seriesMeta{seriesFP: fingerprint, seriesLbs: ls, chunkRefs: temp}) - }, - labels.MustNewMatcher(labels.MatchEqual, "", ""), - ) - - if err != nil { - errs.Add(err) - return nil - } - - if len(seriesMetas) == 0 { - level.Debug(logger).Log("msg", "skipping index because it does not have any matching series", "table", tableName, "index", idx.Name()) - return nil - } - - job := NewJob(tenant, tableName, idx.Path(), seriesMetas) - jobLogger := log.With(logger, "job", job.String()) - c.metrics.compactionRunJobStarted.Inc() - - start := time.Now() - err = c.runCompact(ctx, jobLogger, job, bt, sc) - if err != nil { - c.metrics.compactionRunJobCompleted.WithLabelValues(statusFailure).Inc() - c.metrics.compactionRunJobTime.WithLabelValues(statusFailure).Observe(time.Since(start).Seconds()) - errs.Add(errors.Wrap(err, fmt.Sprintf("runBloomCompact failed for job %s", job.String()))) - return nil - } - - c.metrics.compactionRunJobCompleted.WithLabelValues(statusSuccess).Inc() - c.metrics.compactionRunJobTime.WithLabelValues(statusSuccess).Observe(time.Since(start).Seconds()) - level.Debug(logger).Log("msg", "compaction of job succeeded", "job", job.String(), "duration", time.Since(start)) - - return nil - }) - - return errs.Err() + // TODO(owen-d): impl + return nil } func runWithRetries( @@ -484,175 +254,14 @@ func runWithRetries( return lastErr } -func (c *Compactor) compactTenantWithRetries(ctx context.Context, logger log.Logger, sc storeClient, tableName string, tenant string) error { +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, sc, tableName, tenant) + return c.compactTenant(ctx, logger, tableName, tenant) }, ) } - -func (c *Compactor) runCompact(ctx context.Context, logger log.Logger, job Job, bt *v1.BloomTokenizer, storeClient storeClient) error { - // Ensure the context has not been canceled (ie. compactor shutdown has been triggered). - if err := ctx.Err(); err != nil { - return err - } - metaSearchParams := bloomshipper.MetaSearchParams{ - TenantID: job.tenantID, - Keyspace: v1.NewBounds(job.minFp, job.maxFp), - Interval: bloomshipper.Interval{Start: job.from, End: job.through}, - } - var metas []bloomshipper.Meta - //TODO Configure pool for these to avoid allocations - var activeBloomBlocksRefs []bloomshipper.BlockRef - - metaRefs, fetchers, err := c.bloomShipperClient.ResolveMetas(ctx, metaSearchParams) - if err != nil { - return err - } - - for i := range fetchers { - res, err := fetchers[i].FetchMetas(ctx, metaRefs[i]) - if err != nil { - return err - } - metas = append(metas, res...) - } - - // TODO This logic currently is NOT concerned with cutting blocks upon topology changes to bloom-compactors. - // It may create blocks with series outside of the fp range of the compactor. Cutting blocks will be addressed in a follow-up PR. - metasMatchingJob, blocksMatchingJob := matchingBlocks(metas, job) - - localDst := createLocalDirName(c.cfg.WorkingDirectory, job) - blockOptions := v1.NewBlockOptions(bt.GetNGramLength(), bt.GetNGramSkip()) - - defer func() { - //clean up the bloom directory - if err := os.RemoveAll(localDst); err != nil { - level.Error(logger).Log("msg", "failed to remove block directory", "dir", localDst, "err", err) - } - }() - - var resultingBlock bloomshipper.Block - defer func() { - if resultingBlock.Data != nil { - _ = resultingBlock.Data.Close() - } - }() - - level.Info(logger).Log("msg", "started compacting table", "table", job.tableName, "tenant", job.tenantID) - if len(blocksMatchingJob) == 0 && len(metasMatchingJob) > 0 { - // There is no change to any blocks, no compaction needed - level.Info(logger).Log("msg", "No changes to tsdb, no compaction needed") - return nil - } else if len(metasMatchingJob) == 0 { - // No matching existing blocks for this job, compact all series from scratch - level.Info(logger).Log("msg", "No matching existing blocks for this job, compact all series from scratch") - - builder, err := NewPersistentBlockBuilder(localDst, blockOptions) - if err != nil { - level.Error(logger).Log("msg", "failed creating block builder", "err", err) - return err - } - - // NB(owen-d): this panics/etc, but the code is being refactored and will be removed. I've replaced `bt` with `nil` - // to pass compiler checks while keeping this code around as reference - resultingBlock, err = compactNewChunks(ctx, logger, job, nil, storeClient.chunk, builder, c.limits) - if err != nil { - return level.Error(logger).Log("msg", "failed compacting new chunks", "err", err) - } - - } else if len(blocksMatchingJob) > 0 { - // When already compacted metas exists, we need to merge all blocks with amending blooms with new series - level.Info(logger).Log("msg", "already compacted metas exists, use mergeBlockBuilder") - - var populate = createPopulateFunc(ctx, job, storeClient, bt, c.limits) - - seriesIter := makeSeriesIterFromSeriesMeta(job) - - blockIters, blockPaths, err := makeBlockIterFromBlocks(ctx, logger, c.bloomShipperClient, blocksMatchingJob, c.cfg.WorkingDirectory) - defer func() { - for _, path := range blockPaths { - if err := os.RemoveAll(path); err != nil { - level.Error(logger).Log("msg", "failed removing uncompressed bloomDir", "dir", path, "err", err) - } - } - }() - - if err != nil { - level.Error(logger).Log("err", err) - return err - } - - mergeBlockBuilder, err := NewPersistentBlockBuilder(localDst, blockOptions) - if err != nil { - level.Error(logger).Log("msg", "failed creating block builder", "err", err) - return err - } - - resultingBlock, err = mergeCompactChunks(logger, populate, mergeBlockBuilder, blockIters, seriesIter, job) - if err != nil { - level.Error(logger).Log("msg", "failed merging existing blocks with new chunks", "err", err) - return err - } - - } - - archivePath := filepath.Join(c.cfg.WorkingDirectory, uuid.New().String()) - - blockToUpload, err := bloomshipper.CompressBloomBlock(resultingBlock.BlockRef, archivePath, localDst, logger) - if err != nil { - level.Error(logger).Log("msg", "failed compressing bloom blocks into tar file", "err", err) - return err - } - - defer func() { - err = os.Remove(archivePath) - if err != nil { - level.Error(logger).Log("msg", "failed removing archive file", "err", err, "file", archivePath) - } - }() - - // Do not change the signature of PutBlocks yet. - // Once block size is limited potentially, compactNewChunks will return multiple blocks, hence a list is appropriate. - storedBlocks, err := c.bloomShipperClient.PutBlocks(ctx, []bloomshipper.Block{blockToUpload}) - if err != nil { - level.Error(logger).Log("msg", "failed uploading blocks to storage", "err", err) - return err - } - - // all blocks are new and active blocks - for _, block := range storedBlocks { - activeBloomBlocksRefs = append(activeBloomBlocksRefs, block.BlockRef) - } - - // TODO delete old metas in later compactions - // After all is done, create one meta file and upload to storage - meta := bloomshipper.Meta{ - MetaRef: bloomshipper.MetaRef{ - Ref: bloomshipper.Ref{ - TenantID: job.tenantID, - TableName: job.tableName, - MinFingerprint: uint64(job.minFp), - MaxFingerprint: uint64(job.maxFp), - StartTimestamp: job.from, - EndTimestamp: job.through, - Checksum: rand.Uint32(), // Discuss if checksum is needed for Metas, why should we read all data again. - }, - }, - Tombstones: blocksMatchingJob, - Blocks: activeBloomBlocksRefs, - } - - err = c.bloomShipperClient.PutMeta(ctx, meta) - if err != nil { - level.Error(logger).Log("msg", "failed uploading meta.json to storage", "err", err) - return err - } - level.Info(logger).Log("msg", "finished compacting table", "table", job.tableName, "tenant", job.tenantID) - return nil -} diff --git a/pkg/bloomcompactor/bloomcompactor_test.go b/pkg/bloomcompactor/bloomcompactor_test.go deleted file mode 100644 index 6221610321b69..0000000000000 --- a/pkg/bloomcompactor/bloomcompactor_test.go +++ /dev/null @@ -1,245 +0,0 @@ -package bloomcompactor - -import ( - "context" - "flag" - "fmt" - "path/filepath" - "testing" - "time" - - "github.com/go-kit/log" - "github.com/grafana/dskit/flagext" - "github.com/grafana/dskit/kv" - "github.com/grafana/dskit/kv/consul" - "github.com/grafana/dskit/ring" - "github.com/grafana/dskit/services" - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/common/model" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" - - "github.com/grafana/loki/pkg/compactor" - "github.com/grafana/loki/pkg/storage" - "github.com/grafana/loki/pkg/storage/chunk/client/local" - "github.com/grafana/loki/pkg/storage/config" - "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper" - lokiring "github.com/grafana/loki/pkg/util/ring" - "github.com/grafana/loki/pkg/validation" -) - -const ( - indexTablePrefix = "table_" - workingDirName = "working-dir" -) - -func parseDayTime(s string) config.DayTime { - t, err := time.Parse("2006-01-02", s) - if err != nil { - panic(err) - } - return config.DayTime{ - Time: model.TimeFromUnix(t.Unix()), - } -} - -func TestCompactor_StartStopService(t *testing.T) { - shardingStrategy := NewNoopStrategy() - logger := log.NewNopLogger() - reg := prometheus.NewRegistry() - - cm := storage.NewClientMetrics() - t.Cleanup(cm.Unregister) - - var limits validation.Limits - limits.RegisterFlags(flag.NewFlagSet("limits", flag.PanicOnError)) - overrides, _ := validation.NewOverrides(limits, nil) - - periodConfigUnsupported := config.PeriodConfig{ - From: parseDayTime("2023-09-01"), - IndexType: config.BoltDBShipperType, - ObjectType: config.StorageTypeFileSystem, - Schema: "v13", - RowShards: 16, - IndexTables: config.IndexPeriodicTableConfig{ - PathPrefix: "index/", - PeriodicTableConfig: config.PeriodicTableConfig{ - Prefix: indexTablePrefix, - Period: config.ObjectStorageIndexRequiredPeriod, - }, - }, - } - - periodConfigSupported := config.PeriodConfig{ - From: parseDayTime("2023-10-01"), - IndexType: config.TSDBType, - ObjectType: config.StorageTypeFileSystem, - Schema: "v13", - RowShards: 16, - IndexTables: config.IndexPeriodicTableConfig{ - PathPrefix: "index/", - PeriodicTableConfig: config.PeriodicTableConfig{ - Prefix: indexTablePrefix, - Period: config.ObjectStorageIndexRequiredPeriod, - }, - }, - } - - schemaCfg := config.SchemaConfig{ - Configs: []config.PeriodConfig{ - periodConfigUnsupported, - periodConfigSupported, - }, - } - - fsDir := t.TempDir() - tsdbDir := t.TempDir() - - storageCfg := storage.Config{ - FSConfig: local.FSConfig{ - Directory: fsDir, - }, - TSDBShipperConfig: indexshipper.Config{ - ActiveIndexDirectory: filepath.Join(tsdbDir, "index"), - ResyncInterval: 1 * time.Minute, - Mode: indexshipper.ModeReadWrite, - CacheLocation: filepath.Join(tsdbDir, "cache"), - }, - } - - t.Run("ignore unsupported index types in schema config", func(t *testing.T) { - kvStore, closer := consul.NewInMemoryClient(ring.GetCodec(), logger, reg) - t.Cleanup(func() { - closer.Close() - }) - - var cfg Config - flagext.DefaultValues(&cfg) - cfg.Enabled = true - cfg.WorkingDirectory = filepath.Join(t.TempDir(), workingDirName) - cfg.Ring = lokiring.RingConfig{ - KVStore: kv.Config{ - Mock: kvStore, - }, - } - - c, err := New(cfg, storageCfg, schemaCfg, overrides, logger, shardingStrategy, cm, reg) - require.NoError(t, err) - - err = services.StartAndAwaitRunning(context.Background(), c) - require.NoError(t, err) - - require.Equal(t, 1, len(c.storeClients)) - - // supported index type TSDB is present - sc, ok := c.storeClients[periodConfigSupported.From] - require.True(t, ok) - require.NotNil(t, sc) - - // unsupported index type BoltDB is not present - _, ok = c.storeClients[periodConfigUnsupported.From] - require.False(t, ok) - - err = services.StopAndAwaitTerminated(context.Background(), c) - require.NoError(t, err) - }) -} - -func TestCompactor_RunCompaction(t *testing.T) { - logger := log.NewNopLogger() - reg := prometheus.NewRegistry() - - cm := storage.NewClientMetrics() - t.Cleanup(cm.Unregister) - - tempDir := t.TempDir() - indexDir := filepath.Join(tempDir, "index") - - schemaCfg := config.SchemaConfig{ - Configs: []config.PeriodConfig{ - { - From: config.DayTime{Time: model.Time(0)}, - IndexType: "tsdb", - ObjectType: "filesystem", - Schema: "v12", - IndexTables: config.IndexPeriodicTableConfig{ - PathPrefix: "index/", - PeriodicTableConfig: config.PeriodicTableConfig{ - Prefix: indexTablePrefix, - Period: config.ObjectStorageIndexRequiredPeriod, - }}, - }, - }, - } - - daySeconds := int64(24 * time.Hour / time.Second) - tableNumEnd := time.Now().Unix() / daySeconds - tableNumStart := tableNumEnd - 5 - for i := tableNumStart; i <= tableNumEnd; i++ { - compactor.SetupTable( - t, - filepath.Join(indexDir, fmt.Sprintf("%s%d", indexTablePrefix, i)), - compactor.IndexesConfig{ - NumUnCompactedFiles: 5, - NumCompactedFiles: 5, - }, - compactor.PerUserIndexesConfig{ - NumUsers: 5, - IndexesConfig: compactor.IndexesConfig{ - NumUnCompactedFiles: 5, - NumCompactedFiles: 5, - }, - }, - ) - } - - kvStore, cleanUp := consul.NewInMemoryClient(ring.GetCodec(), logger, nil) - t.Cleanup(func() { assert.NoError(t, cleanUp.Close()) }) - - var cfg Config - flagext.DefaultValues(&cfg) - cfg.WorkingDirectory = filepath.Join(tempDir, workingDirName) - cfg.Ring.KVStore.Mock = kvStore - cfg.Ring.ListenPort = 0 - cfg.Ring.InstanceAddr = "bloomcompactor" - cfg.Ring.InstanceID = "bloomcompactor" - - storageConfig := storage.Config{ - FSConfig: local.FSConfig{Directory: tempDir}, - TSDBShipperConfig: indexshipper.Config{ - ActiveIndexDirectory: indexDir, - ResyncInterval: 1 * time.Minute, - Mode: indexshipper.ModeReadWrite, - CacheLocation: filepath.Join(tempDir, "cache"), - }, - } - - var limits validation.Limits - limits.RegisterFlags(flag.NewFlagSet("limits", flag.PanicOnError)) - overrides, _ := validation.NewOverrides(limits, nil) - - ringManager, err := lokiring.NewRingManager("bloom-compactor", lokiring.ServerMode, cfg.Ring, 1, 1, logger, reg) - require.NoError(t, err) - - err = ringManager.StartAsync(context.Background()) - require.NoError(t, err) - require.Eventually(t, func() bool { - return ringManager.State() == services.Running - }, 1*time.Minute, 100*time.Millisecond) - defer func() { - ringManager.StopAsync() - require.Eventually(t, func() bool { - return ringManager.State() == services.Terminated - }, 1*time.Minute, 100*time.Millisecond) - }() - - shuffleSharding := NewShuffleShardingStrategy(ringManager.Ring, ringManager.RingLifecycler, overrides) - - c, err := New(cfg, storageConfig, schemaCfg, overrides, logger, shuffleSharding, cm, nil) - require.NoError(t, err) - - err = c.runCompaction(context.Background()) - require.NoError(t, err) - - // TODO: Once compaction is implemented, verify compaction here. -} diff --git a/pkg/bloomcompactor/chunkcompactor.go b/pkg/bloomcompactor/chunkcompactor.go deleted file mode 100644 index c4993ccc62a59..0000000000000 --- a/pkg/bloomcompactor/chunkcompactor.go +++ /dev/null @@ -1,245 +0,0 @@ -package bloomcompactor - -import ( - "context" - "fmt" - "io" - "os" - "path/filepath" - - "github.com/google/uuid" - - "github.com/go-kit/log" - "github.com/go-kit/log/level" - "github.com/prometheus/common/model" - - "github.com/grafana/loki/pkg/logproto" - v1 "github.com/grafana/loki/pkg/storage/bloom/v1" - "github.com/grafana/loki/pkg/storage/bloom/v1/filter" - "github.com/grafana/loki/pkg/storage/chunk" - "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper" - tsdbindex "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/index" -) - -type compactorTokenizer interface { - PopulateSeriesWithBloom(bloom *v1.SeriesWithBloom, chunkBatchesIterator v1.Iterator[[]chunk.Chunk]) error -} - -type chunkClient interface { - // TODO: Consider using lazyChunks to avoid downloading all requested chunks. - GetChunks(ctx context.Context, chunks []chunk.Chunk) ([]chunk.Chunk, error) -} - -type blockBuilder interface { - BuildFrom(itr v1.Iterator[v1.SeriesWithBloom]) (uint32, error) - Data() (io.ReadSeekCloser, error) -} - -type PersistentBlockBuilder struct { - builder *v1.BlockBuilder - localDst string -} - -func NewPersistentBlockBuilder(localDst string, blockOptions v1.BlockOptions) (*PersistentBlockBuilder, error) { - // write bloom to a local dir - b, err := v1.NewBlockBuilder(blockOptions, v1.NewDirectoryBlockWriter(localDst)) - if err != nil { - return nil, err - } - builder := PersistentBlockBuilder{ - builder: b, - localDst: localDst, - } - return &builder, nil -} - -func (p *PersistentBlockBuilder) BuildFrom(itr v1.Iterator[v1.SeriesWithBloom]) (uint32, error) { - return p.builder.BuildFrom(itr) -} - -func (p *PersistentBlockBuilder) mergeBuild(builder *v1.MergeBuilder) (uint32, error) { - return builder.Build(p.builder) -} - -func (p *PersistentBlockBuilder) Data() (io.ReadSeekCloser, error) { - blockFile, err := os.Open(filepath.Join(p.localDst, v1.BloomFileName)) - if err != nil { - return nil, err - } - return blockFile, nil -} - -func makeChunkRefs(chksMetas []tsdbindex.ChunkMeta, tenant string, fp model.Fingerprint) []chunk.Chunk { - chunkRefs := make([]chunk.Chunk, 0, len(chksMetas)) - for _, chk := range chksMetas { - chunkRefs = append(chunkRefs, chunk.Chunk{ - ChunkRef: logproto.ChunkRef{ - Fingerprint: uint64(fp), - UserID: tenant, - From: chk.From(), - Through: chk.Through(), - Checksum: chk.Checksum, - }, - }) - } - - return chunkRefs -} - -func buildBloomFromSeries(seriesMeta seriesMeta, fpRate float64, tokenizer compactorTokenizer, chunks v1.Iterator[[]chunk.Chunk]) (v1.SeriesWithBloom, error) { - // Create a bloom for this series - bloomForChks := v1.SeriesWithBloom{ - Series: &v1.Series{ - Fingerprint: seriesMeta.seriesFP, - }, - Bloom: &v1.Bloom{ - ScalableBloomFilter: *filter.NewDefaultScalableBloomFilter(fpRate), - }, - } - - // Tokenize data into n-grams - err := tokenizer.PopulateSeriesWithBloom(&bloomForChks, chunks) - if err != nil { - return v1.SeriesWithBloom{}, err - } - return bloomForChks, nil -} - -// TODO Test this when bloom block size check is implemented -func buildBlockFromBlooms( - ctx context.Context, - logger log.Logger, - builder blockBuilder, - blooms v1.Iterator[v1.SeriesWithBloom], - job Job, -) (bloomshipper.Block, error) { - // Ensure the context has not been canceled (ie. compactor shutdown has been triggered). - if err := ctx.Err(); err != nil { - return bloomshipper.Block{}, err - } - - checksum, err := builder.BuildFrom(blooms) - if err != nil { - level.Error(logger).Log("msg", "failed writing to bloom", "err", err) - return bloomshipper.Block{}, err - } - - data, err := builder.Data() - if err != nil { - level.Error(logger).Log("msg", "failed reading bloom data", "err", err) - return bloomshipper.Block{}, err - } - - block := bloomshipper.Block{ - BlockRef: bloomshipper.BlockRef{ - Ref: bloomshipper.Ref{ - TenantID: job.tenantID, - TableName: job.tableName, - MinFingerprint: uint64(job.minFp), - MaxFingerprint: uint64(job.maxFp), - StartTimestamp: job.from, - EndTimestamp: job.through, - Checksum: checksum, - }, - IndexPath: job.indexPath, - }, - Data: data, - } - - return block, nil -} - -func createLocalDirName(workingDir string, job Job) string { - dir := fmt.Sprintf("bloomBlock-%s-%s-%s-%s-%d-%d-%s", job.tableName, job.tenantID, job.minFp, job.maxFp, job.from, job.through, uuid.New().String()) - return filepath.Join(workingDir, dir) -} - -// Compacts given list of chunks, uploads them to storage and returns a list of bloomBlocks -func compactNewChunks(ctx context.Context, - logger log.Logger, - job Job, - bt compactorTokenizer, - storeClient chunkClient, - builder blockBuilder, - limits Limits, -) (bloomshipper.Block, error) { - // Ensure the context has not been canceled (ie. compactor shutdown has been triggered). - if err := ctx.Err(); err != nil { - return bloomshipper.Block{}, err - } - - bloomIter := newLazyBloomBuilder(ctx, job, storeClient, bt, logger, limits) - - // Build and upload bloomBlock to storage - block, err := buildBlockFromBlooms(ctx, logger, builder, bloomIter, job) - if err != nil { - level.Error(logger).Log("msg", "failed building bloomBlocks", "err", err) - return bloomshipper.Block{}, err - } - - return block, nil -} - -type lazyBloomBuilder struct { - ctx context.Context - metas v1.Iterator[seriesMeta] - tenant string - client chunkClient - bt compactorTokenizer - fpRate float64 - logger log.Logger - chunksBatchSize int - - cur v1.SeriesWithBloom // retured by At() - err error // returned by Err() -} - -// newLazyBloomBuilder returns an iterator that yields v1.SeriesWithBloom -// which are used by the blockBuilder to write a bloom block. -// We use an interator to avoid loading all blooms into memory first, before -// building the block. -func newLazyBloomBuilder(ctx context.Context, job Job, client chunkClient, bt compactorTokenizer, logger log.Logger, limits Limits) *lazyBloomBuilder { - return &lazyBloomBuilder{ - ctx: ctx, - metas: v1.NewSliceIter(job.seriesMetas), - client: client, - tenant: job.tenantID, - bt: bt, - fpRate: limits.BloomFalsePositiveRate(job.tenantID), - logger: logger, - chunksBatchSize: limits.BloomCompactorChunksBatchSize(job.tenantID), - } -} - -func (it *lazyBloomBuilder) Next() bool { - if !it.metas.Next() { - it.cur = v1.SeriesWithBloom{} - level.Debug(it.logger).Log("msg", "No seriesMeta") - return false - } - meta := it.metas.At() - - batchesIterator, err := newChunkBatchesIterator(it.ctx, it.client, makeChunkRefs(meta.chunkRefs, it.tenant, meta.seriesFP), it.chunksBatchSize) - if err != nil { - it.err = err - it.cur = v1.SeriesWithBloom{} - level.Debug(it.logger).Log("msg", "err creating chunks batches iterator", "err", err) - return false - } - it.cur, err = buildBloomFromSeries(meta, it.fpRate, it.bt, batchesIterator) - if err != nil { - it.err = err - it.cur = v1.SeriesWithBloom{} - level.Debug(it.logger).Log("msg", "err in buildBloomFromSeries", "err", err) - return false - } - return true -} - -func (it *lazyBloomBuilder) At() v1.SeriesWithBloom { - return it.cur -} - -func (it *lazyBloomBuilder) Err() error { - return it.err -} diff --git a/pkg/bloomcompactor/chunkcompactor_test.go b/pkg/bloomcompactor/chunkcompactor_test.go deleted file mode 100644 index 8bc94fd26537a..0000000000000 --- a/pkg/bloomcompactor/chunkcompactor_test.go +++ /dev/null @@ -1,229 +0,0 @@ -package bloomcompactor - -import ( - "context" - "io" - "testing" - "time" - - "github.com/go-kit/log" - "github.com/prometheus/common/model" - "github.com/prometheus/prometheus/model/labels" - - "github.com/stretchr/testify/require" - - "github.com/grafana/loki/pkg/chunkenc" - "github.com/grafana/loki/pkg/push" - v1 "github.com/grafana/loki/pkg/storage/bloom/v1" - "github.com/grafana/loki/pkg/storage/chunk" - "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/index" -) - -var ( - userID = "userID" - fpRate = 0.01 - - from = model.Earliest - to = model.Latest - - table = "test_table" - indexPath = "index_test_table" - - testBlockSize = 256 * 1024 - testTargetSize = 1500 * 1024 -) - -func createTestChunk(fp model.Fingerprint, lb labels.Labels) chunk.Chunk { - memChunk := chunkenc.NewMemChunk(chunkenc.ChunkFormatV4, chunkenc.EncSnappy, chunkenc.ChunkHeadFormatFor(chunkenc.ChunkFormatV4), testBlockSize, testTargetSize) - if err := memChunk.Append(&push.Entry{ - Timestamp: time.Unix(0, 1), - Line: "this is a log line", - }); err != nil { - panic(err) - } - c := chunk.NewChunk(userID, - fp, lb, chunkenc.NewFacade(memChunk, testBlockSize, testTargetSize), from, to) - - return c -} - -// Given a seriesMeta and corresponding chunks verify SeriesWithBloom can be built -func TestChunkCompactor_BuildBloomFromSeries(t *testing.T) { - label := labels.FromStrings("foo", "bar") - fp := model.Fingerprint(label.Hash()) - seriesMeta := seriesMeta{ - seriesFP: fp, - seriesLbs: label, - } - - chunks := []chunk.Chunk{createTestChunk(fp, label)} - - mbt := mockBloomTokenizer{} - bloom, err := buildBloomFromSeries(seriesMeta, fpRate, &mbt, v1.NewSliceIter([][]chunk.Chunk{chunks})) - require.NoError(t, err) - require.Equal(t, seriesMeta.seriesFP, bloom.Series.Fingerprint) - require.Equal(t, chunks, mbt.chunks) -} - -func TestChunkCompactor_CompactNewChunks(t *testing.T) { - // Setup - logger := log.NewNopLogger() - label := labels.FromStrings("foo", "bar") - fp1 := model.Fingerprint(100) - fp2 := model.Fingerprint(999) - fp3 := model.Fingerprint(200) - - chunkRef1 := index.ChunkMeta{ - Checksum: 1, - MinTime: 1, - MaxTime: 99, - } - - chunkRef2 := index.ChunkMeta{ - Checksum: 2, - MinTime: 10, - MaxTime: 999, - } - - seriesMetas := []seriesMeta{ - { - seriesFP: fp1, - seriesLbs: label, - chunkRefs: []index.ChunkMeta{chunkRef1}, - }, - { - seriesFP: fp2, - seriesLbs: label, - chunkRefs: []index.ChunkMeta{chunkRef1, chunkRef2}, - }, - { - seriesFP: fp3, - seriesLbs: label, - chunkRefs: []index.ChunkMeta{chunkRef1, chunkRef1, chunkRef2}, - }, - } - - job := NewJob(userID, table, indexPath, seriesMetas) - - mbt := mockBloomTokenizer{} - mcc := mockChunkClient{} - pbb := mockPersistentBlockBuilder{} - - // Run Compaction - compactedBlock, err := compactNewChunks(context.Background(), logger, job, &mbt, &mcc, &pbb, mockLimits{fpRate: fpRate}) - - // Validate Compaction Succeeds - require.NoError(t, err) - require.NotNil(t, compactedBlock) - - // Validate Compacted Block has expected data - require.Equal(t, job.tenantID, compactedBlock.TenantID) - require.Equal(t, job.tableName, compactedBlock.TableName) - require.Equal(t, uint64(fp1), compactedBlock.MinFingerprint) - require.Equal(t, uint64(fp2), compactedBlock.MaxFingerprint) - require.Equal(t, model.Time(chunkRef1.MinTime), compactedBlock.StartTimestamp) - require.Equal(t, model.Time(chunkRef2.MaxTime), compactedBlock.EndTimestamp) - require.Equal(t, indexPath, compactedBlock.IndexPath) -} - -func TestLazyBloomBuilder(t *testing.T) { - logger := log.NewNopLogger() - - label := labels.FromStrings("foo", "bar") - fp1 := model.Fingerprint(100) - fp2 := model.Fingerprint(999) - fp3 := model.Fingerprint(200) - - chunkRef1 := index.ChunkMeta{ - Checksum: 1, - MinTime: 1, - MaxTime: 99, - } - - chunkRef2 := index.ChunkMeta{ - Checksum: 2, - MinTime: 10, - MaxTime: 999, - } - - seriesMetas := []seriesMeta{ - { - seriesFP: fp1, - seriesLbs: label, - chunkRefs: []index.ChunkMeta{chunkRef1}, - }, - { - seriesFP: fp2, - seriesLbs: label, - chunkRefs: []index.ChunkMeta{chunkRef1, chunkRef2}, - }, - { - seriesFP: fp3, - seriesLbs: label, - chunkRefs: []index.ChunkMeta{chunkRef1, chunkRef1, chunkRef2}, - }, - } - - job := NewJob(userID, table, indexPath, seriesMetas) - - mbt := &mockBloomTokenizer{} - mcc := &mockChunkClient{} - - it := newLazyBloomBuilder(context.Background(), job, mcc, mbt, logger, mockLimits{chunksDownloadingBatchSize: 10, fpRate: fpRate}) - - // first seriesMeta has 1 chunks - require.True(t, it.Next()) - require.Equal(t, 1, mcc.requestCount) - require.Equal(t, 1, mcc.chunkCount) - require.Equal(t, fp1, it.At().Series.Fingerprint) - - // first seriesMeta has 2 chunks - require.True(t, it.Next()) - require.Equal(t, 2, mcc.requestCount) - require.Equal(t, 3, mcc.chunkCount) - require.Equal(t, fp2, it.At().Series.Fingerprint) - - // first seriesMeta has 3 chunks - require.True(t, it.Next()) - require.Equal(t, 3, mcc.requestCount) - require.Equal(t, 6, mcc.chunkCount) - require.Equal(t, fp3, it.At().Series.Fingerprint) - - // iterator is done - require.False(t, it.Next()) - require.Error(t, io.EOF, it.Err()) - require.Equal(t, v1.SeriesWithBloom{}, it.At()) -} - -type mockBloomTokenizer struct { - chunks []chunk.Chunk -} - -func (mbt *mockBloomTokenizer) PopulateSeriesWithBloom(_ *v1.SeriesWithBloom, c v1.Iterator[[]chunk.Chunk]) error { - for c.Next() { - mbt.chunks = append(mbt.chunks, c.At()...) - } - return nil -} - -type mockChunkClient struct { - requestCount int - chunkCount int -} - -func (mcc *mockChunkClient) GetChunks(_ context.Context, chks []chunk.Chunk) ([]chunk.Chunk, error) { - mcc.requestCount++ - mcc.chunkCount += len(chks) - return nil, nil -} - -type mockPersistentBlockBuilder struct { -} - -func (pbb *mockPersistentBlockBuilder) BuildFrom(_ v1.Iterator[v1.SeriesWithBloom]) (uint32, error) { - return 0, nil -} - -func (pbb *mockPersistentBlockBuilder) Data() (io.ReadSeekCloser, error) { - return nil, nil -} diff --git a/pkg/bloomcompactor/chunksbatchesiterator.go b/pkg/bloomcompactor/chunksbatchesiterator.go deleted file mode 100644 index a4494b02b7e47..0000000000000 --- a/pkg/bloomcompactor/chunksbatchesiterator.go +++ /dev/null @@ -1,48 +0,0 @@ -package bloomcompactor - -import ( - "context" - "errors" - - "github.com/grafana/loki/pkg/storage/chunk" -) - -type chunksBatchesIterator struct { - context context.Context - client chunkClient - chunksToDownload []chunk.Chunk - batchSize int - - currentBatch []chunk.Chunk - err error -} - -func newChunkBatchesIterator(context context.Context, client chunkClient, chunksToDownload []chunk.Chunk, batchSize int) (*chunksBatchesIterator, error) { - if batchSize <= 0 { - return nil, errors.New("batchSize must be greater than 0") - } - return &chunksBatchesIterator{context: context, client: client, chunksToDownload: chunksToDownload, batchSize: batchSize}, nil -} - -func (c *chunksBatchesIterator) Next() bool { - if len(c.chunksToDownload) == 0 { - return false - } - batchSize := c.batchSize - chunksToDownloadCount := len(c.chunksToDownload) - if chunksToDownloadCount < batchSize { - batchSize = chunksToDownloadCount - } - chunksToDownload := c.chunksToDownload[:batchSize] - c.chunksToDownload = c.chunksToDownload[batchSize:] - c.currentBatch, c.err = c.client.GetChunks(c.context, chunksToDownload) - return c.err == nil -} - -func (c *chunksBatchesIterator) Err() error { - return c.err -} - -func (c *chunksBatchesIterator) At() []chunk.Chunk { - return c.currentBatch -} diff --git a/pkg/bloomcompactor/chunksbatchesiterator_test.go b/pkg/bloomcompactor/chunksbatchesiterator_test.go deleted file mode 100644 index 170f2662b508b..0000000000000 --- a/pkg/bloomcompactor/chunksbatchesiterator_test.go +++ /dev/null @@ -1,96 +0,0 @@ -package bloomcompactor - -import ( - "context" - "errors" - "testing" - - "github.com/stretchr/testify/require" - - "github.com/grafana/loki/pkg/storage/chunk" - tsdbindex "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/index" -) - -func Test_chunksBatchesIterator(t *testing.T) { - tests := map[string]struct { - batchSize int - chunksToDownload []chunk.Chunk - constructorError error - - hadNextCount int - }{ - "expected error if batch size is set to 0": { - batchSize: 0, - constructorError: errors.New("batchSize must be greater than 0"), - }, - "expected no error if there are no chunks": { - hadNextCount: 0, - batchSize: 10, - }, - "expected 1 call to the client": { - chunksToDownload: createFakeChunks(10), - hadNextCount: 1, - batchSize: 20, - }, - "expected 1 call to the client(2)": { - chunksToDownload: createFakeChunks(10), - hadNextCount: 1, - batchSize: 10, - }, - "expected 2 calls to the client": { - chunksToDownload: createFakeChunks(10), - hadNextCount: 2, - batchSize: 6, - }, - "expected 10 calls to the client": { - chunksToDownload: createFakeChunks(10), - hadNextCount: 10, - batchSize: 1, - }, - } - for name, data := range tests { - t.Run(name, func(t *testing.T) { - client := &fakeClient{} - iterator, err := newChunkBatchesIterator(context.Background(), client, data.chunksToDownload, data.batchSize) - if data.constructorError != nil { - require.Equal(t, err, data.constructorError) - return - } - hadNextCount := 0 - var downloadedChunks []chunk.Chunk - for iterator.Next() { - hadNextCount++ - downloaded := iterator.At() - downloadedChunks = append(downloadedChunks, downloaded...) - require.LessOrEqual(t, len(downloaded), data.batchSize) - } - require.NoError(t, iterator.Err()) - require.Equal(t, data.chunksToDownload, downloadedChunks) - require.Equal(t, data.hadNextCount, client.callsCount) - require.Equal(t, data.hadNextCount, hadNextCount) - }) - } -} - -func createFakeChunks(count int) []chunk.Chunk { - metas := make([]tsdbindex.ChunkMeta, 0, count) - for i := 0; i < count; i++ { - metas = append(metas, tsdbindex.ChunkMeta{ - Checksum: uint32(i), - MinTime: int64(i), - MaxTime: int64(i + 100), - KB: uint32(i * 100), - Entries: uint32(i * 10), - }) - } - return makeChunkRefs(metas, "fake", 0xFFFF) -} - -type fakeClient struct { - callsCount int -} - -func (f *fakeClient) GetChunks(_ context.Context, chunks []chunk.Chunk) ([]chunk.Chunk, error) { - f.callsCount++ - return chunks, nil -} diff --git a/pkg/bloomcompactor/v2controller.go b/pkg/bloomcompactor/controller.go similarity index 100% rename from pkg/bloomcompactor/v2controller.go rename to pkg/bloomcompactor/controller.go diff --git a/pkg/bloomcompactor/v2controller_test.go b/pkg/bloomcompactor/controller_test.go similarity index 100% rename from pkg/bloomcompactor/v2controller_test.go rename to pkg/bloomcompactor/controller_test.go diff --git a/pkg/bloomcompactor/job.go b/pkg/bloomcompactor/job.go deleted file mode 100644 index bd43293c73cb6..0000000000000 --- a/pkg/bloomcompactor/job.go +++ /dev/null @@ -1,85 +0,0 @@ -package bloomcompactor - -import ( - "math" - - "github.com/prometheus/common/model" - "github.com/prometheus/prometheus/model/labels" - - "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/index" -) - -type seriesMeta struct { - seriesFP model.Fingerprint - seriesLbs labels.Labels - chunkRefs []index.ChunkMeta -} - -type Job struct { - tableName, tenantID, indexPath string - seriesMetas []seriesMeta - - // We compute them lazily. Unset value is 0. - from, through model.Time - minFp, maxFp model.Fingerprint -} - -// NewJob returns a new compaction Job. -func NewJob( - tenantID string, - tableName string, - indexPath string, - seriesMetas []seriesMeta, -) Job { - j := Job{ - tenantID: tenantID, - tableName: tableName, - indexPath: indexPath, - seriesMetas: seriesMetas, - } - j.computeBounds() - return j -} - -func (j *Job) String() string { - return j.tableName + "_" + j.tenantID + "_" -} - -func (j *Job) computeBounds() { - if len(j.seriesMetas) == 0 { - return - } - - minFrom := model.Latest - maxThrough := model.Earliest - - minFp := model.Fingerprint(math.MaxInt64) - maxFp := model.Fingerprint(0) - - for _, seriesMeta := range j.seriesMetas { - // calculate timestamp boundaries - for _, chunkRef := range seriesMeta.chunkRefs { - from, through := chunkRef.Bounds() - if minFrom > from { - minFrom = from - } - if maxThrough < through { - maxThrough = through - } - } - - // calculate fingerprint boundaries - if minFp > seriesMeta.seriesFP { - minFp = seriesMeta.seriesFP - } - if maxFp < seriesMeta.seriesFP { - maxFp = seriesMeta.seriesFP - } - } - - j.from = minFrom - j.through = maxThrough - - j.minFp = minFp - j.maxFp = maxFp -} diff --git a/pkg/bloomcompactor/mergecompactor.go b/pkg/bloomcompactor/mergecompactor.go deleted file mode 100644 index 3486e40846b8a..0000000000000 --- a/pkg/bloomcompactor/mergecompactor.go +++ /dev/null @@ -1,150 +0,0 @@ -package bloomcompactor - -import ( - "context" - - "github.com/grafana/dskit/concurrency" - - "github.com/grafana/loki/pkg/logproto" - "github.com/grafana/loki/pkg/storage/chunk" - - "github.com/go-kit/log" - "github.com/go-kit/log/level" - - v1 "github.com/grafana/loki/pkg/storage/bloom/v1" - "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper" -) - -func makeSeriesIterFromSeriesMeta(job Job) *v1.SliceIter[*v1.Series] { - // Satisfy types for series - seriesFromSeriesMeta := make([]*v1.Series, len(job.seriesMetas)) - - for i, s := range job.seriesMetas { - crefs := make([]v1.ChunkRef, len(s.chunkRefs)) - for j, chk := range s.chunkRefs { - crefs[j] = v1.ChunkRef{ - Start: chk.From(), - End: chk.Through(), - Checksum: chk.Checksum, - } - } - seriesFromSeriesMeta[i] = &v1.Series{ - Fingerprint: s.seriesFP, - Chunks: crefs, - } - } - return v1.NewSliceIter(seriesFromSeriesMeta) -} - -func makeBlockIterFromBlocks(ctx context.Context, logger log.Logger, - bloomShipperClient bloomshipper.Client, blocksToUpdate []bloomshipper.BlockRef, - workingDir string) ([]v1.PeekingIterator[*v1.SeriesWithBloom], []string, error) { - - // Download existing blocks that needs compaction - blockIters := make([]v1.PeekingIterator[*v1.SeriesWithBloom], len(blocksToUpdate)) - blockPaths := make([]string, len(blocksToUpdate)) - - err := concurrency.ForEachJob(ctx, len(blocksToUpdate), len(blocksToUpdate), func(ctx context.Context, i int) error { - b := blocksToUpdate[i] - - lazyBlock, err := bloomShipperClient.GetBlock(ctx, b) - if err != nil { - level.Error(logger).Log("msg", "failed downloading block", "err", err) - return err - } - - blockPath, err := bloomshipper.UncompressBloomBlock(&lazyBlock, workingDir, logger) - if err != nil { - level.Error(logger).Log("msg", "failed extracting block", "err", err) - return err - } - blockPaths[i] = blockPath - - reader := v1.NewDirectoryBlockReader(blockPath) - block := v1.NewBlock(reader) - blockQuerier := v1.NewBlockQuerier(block) - - blockIters[i] = v1.NewPeekingIter[*v1.SeriesWithBloom](blockQuerier) - return nil - }) - - if err != nil { - return nil, nil, err - } - return blockIters, blockPaths, nil -} - -func createPopulateFunc(_ context.Context, job Job, _ storeClient, bt *v1.BloomTokenizer, _ Limits) func(series *v1.Series, bloom *v1.Bloom) error { - return func(series *v1.Series, bloom *v1.Bloom) error { - bloomForChks := v1.SeriesWithBloom{ - Series: series, - Bloom: bloom, - } - - // Satisfy types for chunks - chunkRefs := make([]chunk.Chunk, len(series.Chunks)) - for i, chk := range series.Chunks { - chunkRefs[i] = chunk.Chunk{ - ChunkRef: logproto.ChunkRef{ - Fingerprint: uint64(series.Fingerprint), - UserID: job.tenantID, - From: chk.Start, - Through: chk.End, - Checksum: chk.Checksum, - }, - } - } - - // batchesIterator, err := newChunkBatchesIterator(ctx, storeClient.chunk, chunkRefs, limits.BloomCompactorChunksBatchSize(job.tenantID)) - // if err != nil { - // return fmt.Errorf("error creating chunks batches iterator: %w", err) - // } - // NB(owen-d): this panics/etc, but the code is being refactored and will be removed. - // I've replaced `batchesIterator` with `emptyIter` to pass compiler checks while keeping this code around as reference - err := bt.Populate(&bloomForChks, v1.NewEmptyIter[v1.ChunkRefWithIter]()) - if err != nil { - return err - } - return nil - } -} - -func mergeCompactChunks(logger log.Logger, - populate func(*v1.Series, *v1.Bloom) error, - mergeBlockBuilder *PersistentBlockBuilder, - blockIters []v1.PeekingIterator[*v1.SeriesWithBloom], seriesIter *v1.SliceIter[*v1.Series], - job Job) (bloomshipper.Block, error) { - - mergeBuilder := v1.NewMergeBuilder( - blockIters, - seriesIter, - populate) - - checksum, err := mergeBlockBuilder.mergeBuild(mergeBuilder) - if err != nil { - level.Error(logger).Log("msg", "failed merging the blooms", "err", err) - return bloomshipper.Block{}, err - } - data, err := mergeBlockBuilder.Data() - if err != nil { - level.Error(logger).Log("msg", "failed reading bloom data", "err", err) - return bloomshipper.Block{}, err - } - - mergedBlock := bloomshipper.Block{ - BlockRef: bloomshipper.BlockRef{ - Ref: bloomshipper.Ref{ - TenantID: job.tenantID, - TableName: job.tableName, - MinFingerprint: uint64(job.minFp), - MaxFingerprint: uint64(job.maxFp), - StartTimestamp: job.from, - EndTimestamp: job.through, - Checksum: checksum, - }, - IndexPath: job.indexPath, - }, - Data: data, - } - return mergedBlock, nil -} diff --git a/pkg/bloomcompactor/v2_meta.go b/pkg/bloomcompactor/meta.go similarity index 100% rename from pkg/bloomcompactor/v2_meta.go rename to pkg/bloomcompactor/meta.go diff --git a/pkg/bloomcompactor/v2spec.go b/pkg/bloomcompactor/spec.go similarity index 100% rename from pkg/bloomcompactor/v2spec.go rename to pkg/bloomcompactor/spec.go diff --git a/pkg/bloomcompactor/v2spec_test.go b/pkg/bloomcompactor/spec_test.go similarity index 100% rename from pkg/bloomcompactor/v2spec_test.go rename to pkg/bloomcompactor/spec_test.go diff --git a/pkg/bloomcompactor/table_utils.go b/pkg/bloomcompactor/table_utils.go index 91940f4cfd455..55bc2e9a328f1 100644 --- a/pkg/bloomcompactor/table_utils.go +++ b/pkg/bloomcompactor/table_utils.go @@ -1,12 +1,9 @@ package bloomcompactor import ( - "sort" - "github.com/prometheus/common/model" "github.com/grafana/loki/pkg/compactor/retention" - "github.com/grafana/loki/pkg/storage/config" ) func getIntervalsForTables(tables []string) map[string]model.Interval { @@ -17,21 +14,3 @@ func getIntervalsForTables(tables []string) map[string]model.Interval { return tablesIntervals } - -func sortTablesByRange(tables []string, intervals map[string]model.Interval) { - sort.Slice(tables, func(i, j int) bool { - // less than if start time is after produces a most recent first sort order - return intervals[tables[i]].Start.After(intervals[tables[j]].Start) - }) -} - -// TODO: comes from pkg/compactor/compactor.go -func schemaPeriodForTable(cfg config.SchemaConfig, tableName string) (config.PeriodConfig, bool) { - tableInterval := retention.ExtractIntervalFromTableName(tableName) - schemaCfg, err := cfg.SchemaForTime(tableInterval.Start) - if err != nil || schemaCfg.IndexTables.TableFor(tableInterval.Start) != tableName { - return config.PeriodConfig{}, false - } - - return schemaCfg, true -} diff --git a/pkg/bloomcompactor/utils.go b/pkg/bloomcompactor/utils.go deleted file mode 100644 index 4b9c3ff541fe2..0000000000000 --- a/pkg/bloomcompactor/utils.go +++ /dev/null @@ -1,37 +0,0 @@ -package bloomcompactor - -import "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper" - -func matchingBlocks(metas []bloomshipper.Meta, job Job) ([]bloomshipper.Meta, []bloomshipper.BlockRef) { - var metasMatchingJob []bloomshipper.Meta - var blocksMatchingJob []bloomshipper.BlockRef - oldTombstonedBlockRefs := make(map[bloomshipper.BlockRef]struct{}) - - for _, meta := range metas { - if meta.TableName != job.tableName { - continue - } - metasMatchingJob = append(metasMatchingJob, meta) - - for _, tombstonedBlockRef := range meta.Tombstones { - oldTombstonedBlockRefs[tombstonedBlockRef] = struct{}{} - } - } - - for _, meta := range metasMatchingJob { - for _, blockRef := range meta.Blocks { - if _, ok := oldTombstonedBlockRefs[blockRef]; ok { - // skip any previously tombstoned blockRefs - continue - } - - if blockRef.IndexPath == job.indexPath { - // index has not changed, no compaction needed - continue - } - blocksMatchingJob = append(blocksMatchingJob, blockRef) - } - } - - return metasMatchingJob, blocksMatchingJob -} diff --git a/pkg/compactor/compactor.go b/pkg/compactor/compactor.go index ca83238633703..8e3fa52126929 100644 --- a/pkg/compactor/compactor.go +++ b/pkg/compactor/compactor.go @@ -578,7 +578,7 @@ func (c *Compactor) stopping(_ error) error { } func (c *Compactor) CompactTable(ctx context.Context, tableName string, applyRetention bool) error { - schemaCfg, ok := schemaPeriodForTable(c.schemaConfig, tableName) + schemaCfg, ok := SchemaPeriodForTable(c.schemaConfig, tableName) if !ok { level.Error(util_log.Logger).Log("msg", "skipping compaction since we can't find schema for table", "table", tableName) return nil @@ -720,7 +720,7 @@ func (c *Compactor) RunCompaction(ctx context.Context, applyRetention bool) (err } // process most recent tables first - sortTablesByRange(tables) + SortTablesByRange(tables) // apply passed in compaction limits if c.cfg.SkipLatestNTables <= len(tables) { @@ -866,7 +866,7 @@ func (c *Compactor) ServeHTTP(w http.ResponseWriter, req *http.Request) { c.ring.ServeHTTP(w, req) } -func sortTablesByRange(tables []string) { +func SortTablesByRange(tables []string) { tableRanges := make(map[string]model.Interval) for _, table := range tables { tableRanges[table] = retention.ExtractIntervalFromTableName(table) @@ -879,7 +879,7 @@ func sortTablesByRange(tables []string) { } -func schemaPeriodForTable(cfg config.SchemaConfig, tableName string) (config.PeriodConfig, bool) { +func SchemaPeriodForTable(cfg config.SchemaConfig, tableName string) (config.PeriodConfig, bool) { tableInterval := retention.ExtractIntervalFromTableName(tableName) schemaCfg, err := cfg.SchemaForTime(tableInterval.Start) if err != nil || schemaCfg.IndexTables.TableFor(tableInterval.Start) != tableName { diff --git a/pkg/compactor/compactor_test.go b/pkg/compactor/compactor_test.go index 9f3f23424f2df..cfcc55e456d0d 100644 --- a/pkg/compactor/compactor_test.go +++ b/pkg/compactor/compactor_test.go @@ -286,7 +286,7 @@ func Test_schemaPeriodForTable(t *testing.T) { } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - actual, actualFound := schemaPeriodForTable(tt.config, tt.tableName) + actual, actualFound := SchemaPeriodForTable(tt.config, tt.tableName) require.Equal(t, tt.expectedFound, actualFound) require.Equal(t, tt.expected, actual) }) @@ -300,7 +300,7 @@ func Test_tableSort(t *testing.T) { "index_19192", } - sortTablesByRange(intervals) + SortTablesByRange(intervals) require.Equal(t, []string{"index_19195", "index_19192", "index_19191"}, intervals) } diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index 8282098c85aec..4bf5282746077 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -1424,12 +1424,10 @@ func (t *Loki) initBloomCompactor() (services.Service, error) { compactor, err := bloomcompactor.New( t.Cfg.BloomCompactor, - t.Cfg.StorageConfig, - t.Cfg.SchemaConfig, + nil, // StoreAndClient placeholder. TODO(owen-d): remove this once we have a proper store and client + shuffleSharding, t.Overrides, logger, - shuffleSharding, - t.clientMetrics, prometheus.DefaultRegisterer) if err != nil { From 1191f881905938947d49eed66306b6e7cc671f6f Mon Sep 17 00:00:00 2001 From: Christian Haudum Date: Thu, 1 Feb 2024 08:47:59 +0100 Subject: [PATCH 19/45] (chore) Bloom shipper: Extend `Interval` struct with utility functions (#11841) Signed-off-by: Christian Haudum --- pkg/bloomgateway/processor.go | 5 +- pkg/bloomgateway/worker.go | 6 +- pkg/storage/bloom/v1/bounds.go | 5 +- .../stores/shipper/bloomshipper/client.go | 28 ++++----- .../shipper/bloomshipper/client_test.go | 3 +- .../stores/shipper/bloomshipper/interval.go | 59 +++++++++++++++++++ .../shipper/bloomshipper/interval_test.go | 50 ++++++++++++++++ .../stores/shipper/bloomshipper/shipper.go | 24 +------- .../shipper/bloomshipper/shipper_test.go | 36 +++++------ .../stores/shipper/bloomshipper/store.go | 4 +- 10 files changed, 150 insertions(+), 70 deletions(-) create mode 100644 pkg/storage/stores/shipper/bloomshipper/interval.go create mode 100644 pkg/storage/stores/shipper/bloomshipper/interval_test.go diff --git a/pkg/bloomgateway/processor.go b/pkg/bloomgateway/processor.go index 5685851aef512..60753c32ac277 100644 --- a/pkg/bloomgateway/processor.go +++ b/pkg/bloomgateway/processor.go @@ -33,10 +33,7 @@ type processor struct { func (p *processor) run(ctx context.Context, tasks []Task) error { for ts, tasks := range group(tasks, func(t Task) model.Time { return t.day }) { - interval := bloomshipper.Interval{ - Start: ts, - End: ts.Add(Day), - } + interval := bloomshipper.NewInterval(ts, ts.Add(Day)) tenant := tasks[0].Tenant err := p.processTasks(ctx, tenant, interval, []v1.FingerprintBounds{{Min: 0, Max: math.MaxUint64}}, tasks) if err != nil { diff --git a/pkg/bloomgateway/worker.go b/pkg/bloomgateway/worker.go index 5c6b8a76dbb22..5a8549a6c0d3a 100644 --- a/pkg/bloomgateway/worker.go +++ b/pkg/bloomgateway/worker.go @@ -160,11 +160,7 @@ func (w *worker) running(_ context.Context) error { } // interval is [Start, End) - interval := bloomshipper.Interval{ - Start: day, // inclusive - End: day.Add(Day), // non-inclusive - } - + 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)) diff --git a/pkg/storage/bloom/v1/bounds.go b/pkg/storage/bloom/v1/bounds.go index a41d70a89d867..0e52554a393d3 100644 --- a/pkg/storage/bloom/v1/bounds.go +++ b/pkg/storage/bloom/v1/bounds.go @@ -30,7 +30,7 @@ func (b FingerprintBounds) Hash(h hash.Hash32) error { enc.PutBE64(uint64(b.Min)) enc.PutBE64(uint64(b.Max)) _, err := h.Write(enc.Get()) - return errors.Wrap(err, "writing OwnershipRange") + return errors.Wrap(err, "writing FingerprintBounds") } func (b FingerprintBounds) String() string { @@ -54,6 +54,7 @@ func (b FingerprintBounds) Cmp(fp model.Fingerprint) BoundsCheck { return Overlap } +// Overlaps returns whether the bounds (partially) overlap with the target bounds func (b FingerprintBounds) Overlaps(target FingerprintBounds) bool { return b.Cmp(target.Min) != After && b.Cmp(target.Max) != Before } @@ -63,7 +64,7 @@ func (b FingerprintBounds) Slice(min, max model.Fingerprint) *FingerprintBounds return b.Intersection(FingerprintBounds{Min: min, Max: max}) } -// Returns whether the fingerprint is fully within the target bounds +// Within returns whether the fingerprint is fully within the target bounds func (b FingerprintBounds) Within(target FingerprintBounds) bool { return b.Min >= target.Min && b.Max <= target.Max } diff --git a/pkg/storage/stores/shipper/bloomshipper/client.go b/pkg/storage/stores/shipper/bloomshipper/client.go index 835ee13686d37..053d751f59f18 100644 --- a/pkg/storage/stores/shipper/bloomshipper/client.go +++ b/pkg/storage/stores/shipper/bloomshipper/client.go @@ -45,16 +45,20 @@ func (r Ref) Cmp(fp uint64) v1.BoundsCheck { return v1.Overlap } +func (r Ref) Bounds() v1.FingerprintBounds { + return v1.NewBounds(model.Fingerprint(r.MinFingerprint), model.Fingerprint(r.MaxFingerprint)) +} + +func (r Ref) Interval() Interval { + return NewInterval(r.StartTimestamp, r.EndTimestamp) +} + type BlockRef struct { Ref IndexPath string BlockPath string } -func (b *BlockRef) Bounds() v1.FingerprintBounds { - return v1.NewBounds(model.Fingerprint(b.MinFingerprint), model.Fingerprint(b.MaxFingerprint)) -} - type MetaRef struct { Ref FilePath string @@ -282,19 +286,13 @@ func createMetaRef(objectKey string, tenantID string, tableName string) (MetaRef }, nil } -func tablesForRange(periodConfig config.PeriodConfig, from, to model.Time) []string { - interval := periodConfig.IndexTables.Period - step := int64(interval.Seconds()) - lower := from.Unix() / step - upper := to.Unix() / step +func tablesForRange(periodConfig config.PeriodConfig, interval Interval) []string { + step := int64(periodConfig.IndexTables.Period.Seconds()) + lower := interval.Start.Unix() / step + upper := interval.End.Unix() / step tables := make([]string, 0, 1+upper-lower) - prefix := periodConfig.IndexTables.Prefix for i := lower; i <= upper; i++ { - tables = append(tables, joinTableName(prefix, i)) + tables = append(tables, fmt.Sprintf("%s%d", periodConfig.IndexTables.Prefix, i)) } return tables } - -func joinTableName(prefix string, tableNumber int64) string { - return fmt.Sprintf("%s%d", prefix, tableNumber) -} diff --git a/pkg/storage/stores/shipper/bloomshipper/client_test.go b/pkg/storage/stores/shipper/bloomshipper/client_test.go index 30aac5c901e08..a6b3869229b82 100644 --- a/pkg/storage/stores/shipper/bloomshipper/client_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/client_test.go @@ -73,8 +73,9 @@ func Test_BloomClient_FetchMetas(t *testing.T) { searchParams := MetaSearchParams{ TenantID: "tenantA", + Keyspace: v1.NewBounds(50, 150), - Interval: Interval{Start: fixedDay.Add(-6 * day), End: fixedDay.Add(-1*day - 1*time.Hour)}, + Interval: NewInterval(fixedDay.Add(-6*day), fixedDay.Add(-1*day-1*time.Hour)), } fetched, err := store.FetchMetas(context.Background(), searchParams) diff --git a/pkg/storage/stores/shipper/bloomshipper/interval.go b/pkg/storage/stores/shipper/bloomshipper/interval.go new file mode 100644 index 0000000000000..2ddb3716e4995 --- /dev/null +++ b/pkg/storage/stores/shipper/bloomshipper/interval.go @@ -0,0 +1,59 @@ +package bloomshipper + +import ( + "fmt" + "hash" + + "github.com/pkg/errors" + "github.com/prometheus/common/model" + + v1 "github.com/grafana/loki/pkg/storage/bloom/v1" + "github.com/grafana/loki/pkg/util/encoding" +) + +// Interval defines a time range with start end end time +// where the start is inclusive, the end is non-inclusive. +type Interval struct { + Start, End model.Time +} + +func NewInterval(start, end model.Time) Interval { + return Interval{Start: start, End: end} +} + +func (i Interval) Hash(h hash.Hash32) error { + var enc encoding.Encbuf + enc.PutBE64(uint64(i.Start)) + enc.PutBE64(uint64(i.End)) + _, err := h.Write(enc.Get()) + return errors.Wrap(err, "writing Interval") +} + +func (i Interval) String() string { + // 13 digits are enough until Sat Nov 20 2286 17:46:39 UTC + return fmt.Sprintf("%013d-%013d", i.Start, i.End) +} + +func (i Interval) Repr() string { + return fmt.Sprintf("[%s, %s)", i.Start.Time().UTC(), i.End.Time().UTC()) +} + +// Cmp returns the position of a time relative to the interval +func (i Interval) Cmp(ts model.Time) v1.BoundsCheck { + if ts.Before(i.Start) { + return v1.Before + } else if ts.After(i.End) || ts.Equal(i.End) { + return v1.After + } + return v1.Overlap +} + +// Overlaps returns whether the interval overlaps (partially) with the target interval +func (i Interval) Overlaps(target Interval) bool { + return i.Cmp(target.Start) != v1.After && i.Cmp(target.End) != v1.Before +} + +// Within returns whether the interval is fully within the target interval +func (i Interval) Within(target Interval) bool { + return i.Start >= target.Start && i.End <= target.End +} diff --git a/pkg/storage/stores/shipper/bloomshipper/interval_test.go b/pkg/storage/stores/shipper/bloomshipper/interval_test.go new file mode 100644 index 0000000000000..2914e18030578 --- /dev/null +++ b/pkg/storage/stores/shipper/bloomshipper/interval_test.go @@ -0,0 +1,50 @@ +package bloomshipper + +import ( + "testing" + "time" + + "github.com/prometheus/common/model" + "github.com/stretchr/testify/assert" + + v1 "github.com/grafana/loki/pkg/storage/bloom/v1" +) + +func Test_Interval_String(t *testing.T) { + start := model.Time(0) + end := model.TimeFromUnix(time.Date(2024, 1, 1, 0, 0, 0, 0, time.UTC).Unix()) + interval := NewInterval(start, end) + assert.Equal(t, "0000000000000-1704067200000", interval.String()) + assert.Equal(t, "[1970-01-01 00:00:00 +0000 UTC, 2024-01-01 00:00:00 +0000 UTC)", interval.Repr()) +} + +func Test_Interval_Cmp(t *testing.T) { + interval := NewInterval(10, 20) + assert.Equal(t, v1.Before, interval.Cmp(0)) + assert.Equal(t, v1.Overlap, interval.Cmp(10)) + assert.Equal(t, v1.Overlap, interval.Cmp(15)) + assert.Equal(t, v1.After, interval.Cmp(20)) // End is not inclusive + assert.Equal(t, v1.After, interval.Cmp(21)) +} + +func Test_Interval_Overlap(t *testing.T) { + interval := NewInterval(10, 20) + assert.True(t, interval.Overlaps(Interval{Start: 5, End: 15})) + assert.True(t, interval.Overlaps(Interval{Start: 15, End: 25})) + assert.True(t, interval.Overlaps(Interval{Start: 10, End: 20})) + assert.True(t, interval.Overlaps(Interval{Start: 5, End: 25})) + assert.False(t, interval.Overlaps(Interval{Start: 1, End: 9})) + assert.False(t, interval.Overlaps(Interval{Start: 20, End: 30})) // End is not inclusive + assert.False(t, interval.Overlaps(Interval{Start: 25, End: 30})) +} + +func Test_Interval_Within(t *testing.T) { + target := NewInterval(10, 20) + assert.False(t, NewInterval(1, 9).Within(target)) + assert.False(t, NewInterval(21, 30).Within(target)) + assert.True(t, NewInterval(10, 20).Within(target)) + assert.True(t, NewInterval(14, 15).Within(target)) + assert.False(t, NewInterval(5, 15).Within(target)) + assert.False(t, NewInterval(15, 25).Within(target)) + assert.False(t, NewInterval(5, 25).Within(target)) +} diff --git a/pkg/storage/stores/shipper/bloomshipper/shipper.go b/pkg/storage/stores/shipper/bloomshipper/shipper.go index 54c2185fae56d..105e27f065b74 100644 --- a/pkg/storage/stores/shipper/bloomshipper/shipper.go +++ b/pkg/storage/stores/shipper/bloomshipper/shipper.go @@ -8,30 +8,12 @@ import ( "github.com/go-kit/log" "github.com/go-kit/log/level" "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/common/model" "golang.org/x/exp/slices" v1 "github.com/grafana/loki/pkg/storage/bloom/v1" "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper/config" ) -type Interval struct { - Start, End model.Time -} - -func (i Interval) String() string { - return fmt.Sprintf("[%s, %s)", i.Start.Time(), i.End.Time()) -} - -func (i Interval) Cmp(other model.Time) v1.BoundsCheck { - if other.Before(i.Start) { - return v1.Before - } else if other.After(i.End) || other.Equal(i.End) { - return v1.After - } - return v1.Overlap -} - type BlockQuerierWithFingerprintRange struct { *v1.BlockQuerier v1.FingerprintBounds @@ -203,14 +185,14 @@ func BlocksForMetas(metas []Meta, interval Interval, keyspaces []v1.FingerprintB // isOutsideRange tests if a given BlockRef b is outside of search boundaries // defined by min/max timestamp and min/max fingerprint. // Fingerprint ranges must be sorted in ascending order. -func isOutsideRange(b BlockRef, interval Interval, keyspaces []v1.FingerprintBounds) bool { +func isOutsideRange(b BlockRef, interval Interval, bounds []v1.FingerprintBounds) bool { // check time interval - if interval.Cmp(b.EndTimestamp) == v1.Before || interval.Cmp(b.StartTimestamp) == v1.After { + if !interval.Overlaps(b.Interval()) { return true } // check fingerprint ranges - for _, keyspace := range keyspaces { + for _, keyspace := range bounds { if keyspace.Within(b.Bounds()) || keyspace.Overlaps(b.Bounds()) { return false } diff --git a/pkg/storage/stores/shipper/bloomshipper/shipper_test.go b/pkg/storage/stores/shipper/bloomshipper/shipper_test.go index d2311f808e26f..59db4f06ae20a 100644 --- a/pkg/storage/stores/shipper/bloomshipper/shipper_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/shipper_test.go @@ -12,10 +12,6 @@ import ( v1 "github.com/grafana/loki/pkg/storage/bloom/v1" ) -func interval(start, end model.Time) Interval { - return Interval{Start: start, End: end} -} - func Test_Shipper_findBlocks(t *testing.T) { t.Run("expected block that are specified in tombstones to be filtered out", func(t *testing.T) { metas := []Meta{ @@ -46,10 +42,10 @@ func Test_Shipper_findBlocks(t *testing.T) { ts := model.Now() - interval := Interval{ - Start: ts.Add(-2 * time.Hour), - End: ts.Add(-1 * time.Hour), - } + interval := NewInterval( + ts.Add(-2*time.Hour), + ts.Add(-1*time.Hour), + ) blocks := BlocksForMetas(metas, interval, []v1.FingerprintBounds{{Min: 100, Max: 200}}) expectedBlockRefs := []BlockRef{ @@ -103,7 +99,7 @@ func Test_Shipper_findBlocks(t *testing.T) { for name, data := range tests { t.Run(name, func(t *testing.T) { ref := createBlockRef("fake-block", data.minFingerprint, data.maxFingerprint, data.startTimestamp, data.endTimestamp) - blocks := BlocksForMetas([]Meta{{Blocks: []BlockRef{ref}}}, interval(300, 400), []v1.FingerprintBounds{{Min: 100, Max: 200}}) + blocks := BlocksForMetas([]Meta{{Blocks: []BlockRef{ref}}}, NewInterval(300, 400), []v1.FingerprintBounds{{Min: 100, Max: 200}}) if data.filtered { require.Empty(t, blocks) return @@ -120,67 +116,67 @@ func TestIsOutsideRange(t *testing.T) { t.Run("is outside if startTs > through", func(t *testing.T) { b := createBlockRef("block", 0, math.MaxUint64, startTs, endTs) - isOutside := isOutsideRange(b, interval(0, 900), []v1.FingerprintBounds{}) + isOutside := isOutsideRange(b, NewInterval(0, 900), []v1.FingerprintBounds{}) require.True(t, isOutside) }) t.Run("is outside if startTs == through ", func(t *testing.T) { b := createBlockRef("block", 0, math.MaxUint64, startTs, endTs) - isOutside := isOutsideRange(b, interval(900, 1000), []v1.FingerprintBounds{}) + isOutside := isOutsideRange(b, NewInterval(900, 1000), []v1.FingerprintBounds{}) require.True(t, isOutside) }) t.Run("is outside if endTs < from", func(t *testing.T) { b := createBlockRef("block", 0, math.MaxUint64, startTs, endTs) - isOutside := isOutsideRange(b, interval(2100, 3000), []v1.FingerprintBounds{}) + isOutside := isOutsideRange(b, NewInterval(2100, 3000), []v1.FingerprintBounds{}) require.True(t, isOutside) }) t.Run("is outside if endFp < first fingerprint", func(t *testing.T) { b := createBlockRef("block", 0, 90, startTs, endTs) - isOutside := isOutsideRange(b, interval(startTs, endTs), []v1.FingerprintBounds{{Min: 100, Max: 199}}) + isOutside := isOutsideRange(b, NewInterval(startTs, endTs), []v1.FingerprintBounds{{Min: 100, Max: 199}}) require.True(t, isOutside) }) t.Run("is outside if startFp > last fingerprint", func(t *testing.T) { b := createBlockRef("block", 200, math.MaxUint64, startTs, endTs) - isOutside := isOutsideRange(b, interval(startTs, endTs), []v1.FingerprintBounds{{Min: 0, Max: 49}, {Min: 100, Max: 149}}) + isOutside := isOutsideRange(b, NewInterval(startTs, endTs), []v1.FingerprintBounds{{Min: 0, Max: 49}, {Min: 100, Max: 149}}) require.True(t, isOutside) }) t.Run("is outside if within gaps in fingerprints", func(t *testing.T) { b := createBlockRef("block", 100, 199, startTs, endTs) - isOutside := isOutsideRange(b, interval(startTs, endTs), []v1.FingerprintBounds{{Min: 0, Max: 99}, {Min: 200, Max: 299}}) + isOutside := isOutsideRange(b, NewInterval(startTs, endTs), []v1.FingerprintBounds{{Min: 0, Max: 99}, {Min: 200, Max: 299}}) require.True(t, isOutside) }) t.Run("is not outside if within fingerprints 1", func(t *testing.T) { b := createBlockRef("block", 10, 90, startTs, endTs) - isOutside := isOutsideRange(b, interval(startTs, endTs), []v1.FingerprintBounds{{Min: 0, Max: 99}, {Min: 200, Max: 299}}) + isOutside := isOutsideRange(b, NewInterval(startTs, endTs), []v1.FingerprintBounds{{Min: 0, Max: 99}, {Min: 200, Max: 299}}) require.False(t, isOutside) }) t.Run("is not outside if within fingerprints 2", func(t *testing.T) { b := createBlockRef("block", 210, 290, startTs, endTs) - isOutside := isOutsideRange(b, interval(startTs, endTs), []v1.FingerprintBounds{{Min: 0, Max: 99}, {Min: 200, Max: 299}}) + isOutside := isOutsideRange(b, NewInterval(startTs, endTs), []v1.FingerprintBounds{{Min: 0, Max: 99}, {Min: 200, Max: 299}}) require.False(t, isOutside) }) t.Run("is not outside if spans across multiple fingerprint ranges", func(t *testing.T) { b := createBlockRef("block", 50, 250, startTs, endTs) - isOutside := isOutsideRange(b, interval(startTs, endTs), []v1.FingerprintBounds{{Min: 0, Max: 99}, {Min: 200, Max: 299}}) + isOutside := isOutsideRange(b, NewInterval(startTs, endTs), []v1.FingerprintBounds{{Min: 0, Max: 99}, {Min: 200, Max: 299}}) require.False(t, isOutside) }) t.Run("is not outside if fingerprint range and time range are larger than block", func(t *testing.T) { b := createBlockRef("block", math.MaxUint64/3, math.MaxUint64/3*2, startTs, endTs) - isOutside := isOutsideRange(b, interval(0, 3000), []v1.FingerprintBounds{{Min: 0, Max: math.MaxUint64}}) + isOutside := isOutsideRange(b, NewInterval(0, 3000), []v1.FingerprintBounds{{Min: 0, Max: math.MaxUint64}}) require.False(t, isOutside) }) t.Run("is not outside if block fingerprint range is bigger that search keyspace", func(t *testing.T) { b := createBlockRef("block", 0x0000, 0xffff, model.Earliest, model.Latest) - isOutside := isOutsideRange(b, interval(startTs, endTs), []v1.FingerprintBounds{{Min: 0x0100, Max: 0xff00}}) + isOutside := isOutsideRange(b, NewInterval(startTs, endTs), []v1.FingerprintBounds{{Min: 0x0100, Max: 0xff00}}) require.False(t, isOutside) }) } diff --git a/pkg/storage/stores/shipper/bloomshipper/store.go b/pkg/storage/stores/shipper/bloomshipper/store.go index 0c2f2d5405515..b53a94526b31e 100644 --- a/pkg/storage/stores/shipper/bloomshipper/store.go +++ b/pkg/storage/stores/shipper/bloomshipper/store.go @@ -41,7 +41,7 @@ type bloomStoreEntry struct { // ResolveMetas implements store. func (b *bloomStoreEntry) ResolveMetas(ctx context.Context, params MetaSearchParams) ([][]MetaRef, []*Fetcher, error) { var refs []MetaRef - tables := tablesForRange(b.cfg, params.Interval.Start, params.Interval.End) + tables := tablesForRange(b.cfg, params.Interval) for _, table := range tables { prefix := filepath.Join(rootFolder, table, params.TenantID, metasFolder) list, _, err := b.objectClient.List(ctx, prefix, "") @@ -393,7 +393,7 @@ func (b *BloomStore) forStores(ctx context.Context, interval Interval, f func(in } end := min(through, nextSchemaStarts-1) - err := f(ctx, Interval{start, end}, b.stores[i]) + err := f(ctx, NewInterval(start, end), b.stores[i]) if err != nil { return err } From c335cd200e8403e547e867528bbb9df3c41dda1f Mon Sep 17 00:00:00 2001 From: Salva Corts Date: Thu, 1 Feb 2024 19:10:01 +0100 Subject: [PATCH 20/45] Extract shard annotation into new FingerprintFilter interface (#11834) **What this PR does / why we need it**: This PR extracts the `Match` and `Bounds` methods of TSDB's `ShardAnnotation` into a new interface `FingerprintFilter`. This will allow us to query the index for any fingerprint bounds, not just power of 2 shard factors. We now use this in the bloom compactor by calling the index `ForSeries` ([here][1]) passing a fingerprint filter with the bounds of the FP range owned by the compactor. **Special notes for your reviewer**: **Checklist** - [x] 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) [1]: https://github.com/grafana/loki/blob/de4f56e42d14eb25f22a249aca04dd0736e88d15/pkg/bloomcompactor/bloomcompactor.go#L408 --- pkg/ingester/index/bitprefix.go | 4 +-- pkg/storage/bloom/v1/bounds.go | 10 +++++++ .../shipper/indexshipper/tsdb/head_manager.go | 16 +++++----- .../shipper/indexshipper/tsdb/head_read.go | 6 ++-- .../stores/shipper/indexshipper/tsdb/index.go | 16 +++++----- .../indexshipper/tsdb/index/fingerprint.go | 4 +-- .../shipper/indexshipper/tsdb/index/index.go | 6 ++-- .../indexshipper/tsdb/index/postings.go | 4 +-- .../shipper/indexshipper/tsdb/index/shard.go | 9 ++++-- .../indexshipper/tsdb/index/shard_test.go | 2 +- .../shipper/indexshipper/tsdb/index_client.go | 9 +++--- .../tsdb/index_shipper_querier.go | 16 +++++----- .../shipper/indexshipper/tsdb/lazy_index.go | 16 +++++----- .../indexshipper/tsdb/multi_file_index.go | 16 +++++----- .../shipper/indexshipper/tsdb/multitenant.go | 16 +++++----- .../shipper/indexshipper/tsdb/querier.go | 26 ++++++++-------- .../indexshipper/tsdb/single_file_index.go | 30 +++++++++---------- 17 files changed, 111 insertions(+), 95 deletions(-) diff --git a/pkg/ingester/index/bitprefix.go b/pkg/ingester/index/bitprefix.go index 025005618d8c5..8235c2821d6ca 100644 --- a/pkg/ingester/index/bitprefix.go +++ b/pkg/ingester/index/bitprefix.go @@ -69,7 +69,7 @@ func (ii *BitPrefixInvertedIndex) getShards(shard *astmapper.ShardAnnotation) ([ } requestedShard := shard.TSDB() - minFp, maxFp := requestedShard.Bounds() + minFp, maxFp := requestedShard.GetFromThrough() // Determine how many bits we need to take from // the requested shard's min/max fingerprint values @@ -143,7 +143,7 @@ func (ii *BitPrefixInvertedIndex) Lookup(matchers []*labels.Matcher, shard *astm // Because bit prefix order is also ascending order, // the merged fingerprints from ascending shards are also in order. if filter { - minFP, maxFP := shard.TSDB().Bounds() + minFP, maxFP := shard.TSDB().GetFromThrough() minIdx := sort.Search(len(result), func(i int) bool { return result[i] >= minFP }) diff --git a/pkg/storage/bloom/v1/bounds.go b/pkg/storage/bloom/v1/bounds.go index 0e52554a393d3..961060198c393 100644 --- a/pkg/storage/bloom/v1/bounds.go +++ b/pkg/storage/bloom/v1/bounds.go @@ -59,6 +59,16 @@ func (b FingerprintBounds) Overlaps(target FingerprintBounds) bool { return b.Cmp(target.Min) != After && b.Cmp(target.Max) != Before } +// Match implements TSDBs FingerprintFilter interface +func (b FingerprintBounds) Match(fp model.Fingerprint) bool { + return b.Cmp(fp) == Overlap +} + +// GetFromThrough implements TSDBs FingerprintFilter interface +func (b FingerprintBounds) GetFromThrough() (model.Fingerprint, model.Fingerprint) { + return b.Min, b.Max +} + // Slice returns a new fingerprint bounds clipped to the target bounds or nil if there is no overlap func (b FingerprintBounds) Slice(min, max model.Fingerprint) *FingerprintBounds { return b.Intersection(FingerprintBounds{Min: min, Max: max}) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go b/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go index bae41255554db..7342fe851c577 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/head_manager.go @@ -747,22 +747,22 @@ func (t *tenantHeads) tenantIndex(userID string, from, through model.Time) (idx } -func (t *tenantHeads) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, _ []ChunkRef, shard *index.ShardAnnotation, matchers ...*labels.Matcher) ([]ChunkRef, error) { +func (t *tenantHeads) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, _ []ChunkRef, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]ChunkRef, error) { idx, ok := t.tenantIndex(userID, from, through) if !ok { return nil, nil } - return idx.GetChunkRefs(ctx, userID, from, through, nil, shard, matchers...) + return idx.GetChunkRefs(ctx, userID, from, through, nil, fpFilter, matchers...) } // Series follows the same semantics regarding the passed slice and shard as GetChunkRefs. -func (t *tenantHeads) Series(ctx context.Context, userID string, from, through model.Time, _ []Series, shard *index.ShardAnnotation, matchers ...*labels.Matcher) ([]Series, error) { +func (t *tenantHeads) Series(ctx context.Context, userID string, from, through model.Time, _ []Series, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]Series, error) { idx, ok := t.tenantIndex(userID, from, through) if !ok { return nil, nil } - return idx.Series(ctx, userID, from, through, nil, shard, matchers...) + return idx.Series(ctx, userID, from, through, nil, fpFilter, matchers...) } @@ -784,20 +784,20 @@ func (t *tenantHeads) LabelValues(ctx context.Context, userID string, from, thro } -func (t *tenantHeads) Stats(ctx context.Context, userID string, from, through model.Time, acc IndexStatsAccumulator, shard *index.ShardAnnotation, shouldIncludeChunk shouldIncludeChunk, matchers ...*labels.Matcher) error { +func (t *tenantHeads) Stats(ctx context.Context, userID string, from, through model.Time, acc IndexStatsAccumulator, fpFilter index.FingerprintFilter, shouldIncludeChunk shouldIncludeChunk, matchers ...*labels.Matcher) error { idx, ok := t.tenantIndex(userID, from, through) if !ok { return nil } - return idx.Stats(ctx, userID, from, through, acc, shard, shouldIncludeChunk, matchers...) + return idx.Stats(ctx, userID, from, through, acc, fpFilter, shouldIncludeChunk, matchers...) } -func (t *tenantHeads) Volume(ctx context.Context, userID string, from, through model.Time, acc VolumeAccumulator, shard *index.ShardAnnotation, shouldIncludeChunk shouldIncludeChunk, targetLabels []string, aggregateBy string, matchers ...*labels.Matcher) error { +func (t *tenantHeads) Volume(ctx context.Context, userID string, from, through model.Time, acc VolumeAccumulator, fpFilter index.FingerprintFilter, shouldIncludeChunk shouldIncludeChunk, targetLabels []string, aggregateBy string, matchers ...*labels.Matcher) error { idx, ok := t.tenantIndex(userID, from, through) if !ok { return nil } - return idx.Volume(ctx, userID, from, through, acc, shard, shouldIncludeChunk, targetLabels, aggregateBy, matchers...) + return idx.Volume(ctx, userID, from, through, acc, fpFilter, shouldIncludeChunk, targetLabels, aggregateBy, matchers...) } // helper only used in building TSDBs diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/head_read.go b/pkg/storage/stores/shipper/indexshipper/tsdb/head_read.go index 4ffc8ae2e9a96..203e951a435d5 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/head_read.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/head_read.go @@ -100,7 +100,7 @@ func (h *headIndexReader) LabelNames(matchers ...*labels.Matcher) ([]string, err } // Postings returns the postings list iterator for the label pairs. -func (h *headIndexReader) Postings(name string, shard *index.ShardAnnotation, values ...string) (index.Postings, error) { +func (h *headIndexReader) Postings(name string, fpFilter index.FingerprintFilter, values ...string) (index.Postings, error) { var p index.Postings switch len(values) { case 0: @@ -115,8 +115,8 @@ func (h *headIndexReader) Postings(name string, shard *index.ShardAnnotation, va p = index.Merge(res...) } - if shard != nil { - return index.NewShardedPostings(p, *shard, nil), nil + if fpFilter != nil { + return index.NewShardedPostings(p, fpFilter, nil), nil } return p, nil } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index.go index 69f4c26765883..bb294fb13f450 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index.go @@ -47,25 +47,25 @@ type Index interface { // the requested shard. If it is nil, TSDB will return all results, // regardless of shard. // Note: any shard used must be a valid factor of two, meaning `0_of_2` and `3_of_4` are fine, but `0_of_3` is not. - GetChunkRefs(ctx context.Context, userID string, from, through model.Time, res []ChunkRef, shard *index.ShardAnnotation, matchers ...*labels.Matcher) ([]ChunkRef, error) + GetChunkRefs(ctx context.Context, userID string, from, through model.Time, res []ChunkRef, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]ChunkRef, error) // Series follows the same semantics regarding the passed slice and shard as GetChunkRefs. - Series(ctx context.Context, userID string, from, through model.Time, res []Series, shard *index.ShardAnnotation, matchers ...*labels.Matcher) ([]Series, error) + Series(ctx context.Context, userID string, from, through model.Time, res []Series, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]Series, error) LabelNames(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]string, error) LabelValues(ctx context.Context, userID string, from, through model.Time, name string, matchers ...*labels.Matcher) ([]string, error) - Stats(ctx context.Context, userID string, from, through model.Time, acc IndexStatsAccumulator, shard *index.ShardAnnotation, shouldIncludeChunk shouldIncludeChunk, matchers ...*labels.Matcher) error - Volume(ctx context.Context, userID string, from, through model.Time, acc VolumeAccumulator, shard *index.ShardAnnotation, shouldIncludeChunk shouldIncludeChunk, targetLabels []string, aggregateBy string, matchers ...*labels.Matcher) error + Stats(ctx context.Context, userID string, from, through model.Time, acc IndexStatsAccumulator, fpFilter index.FingerprintFilter, shouldIncludeChunk shouldIncludeChunk, matchers ...*labels.Matcher) error + Volume(ctx context.Context, userID string, from, through model.Time, acc VolumeAccumulator, fpFilter index.FingerprintFilter, shouldIncludeChunk shouldIncludeChunk, targetLabels []string, aggregateBy string, matchers ...*labels.Matcher) error } type NoopIndex struct{} func (NoopIndex) Close() error { return nil } func (NoopIndex) Bounds() (_, through model.Time) { return } -func (NoopIndex) GetChunkRefs(_ context.Context, _ string, _, _ model.Time, _ []ChunkRef, _ *index.ShardAnnotation, _ ...*labels.Matcher) ([]ChunkRef, error) { +func (NoopIndex) GetChunkRefs(_ context.Context, _ string, _, _ model.Time, _ []ChunkRef, _ index.FingerprintFilter, _ ...*labels.Matcher) ([]ChunkRef, error) { return nil, nil } // Series follows the same semantics regarding the passed slice and shard as GetChunkRefs. -func (NoopIndex) Series(_ context.Context, _ string, _, _ model.Time, _ []Series, _ *index.ShardAnnotation, _ ...*labels.Matcher) ([]Series, error) { +func (NoopIndex) Series(_ context.Context, _ string, _, _ model.Time, _ []Series, _ index.FingerprintFilter, _ ...*labels.Matcher) ([]Series, error) { return nil, nil } func (NoopIndex) LabelNames(_ context.Context, _ string, _, _ model.Time, _ ...*labels.Matcher) ([]string, error) { @@ -75,12 +75,12 @@ func (NoopIndex) LabelValues(_ context.Context, _ string, _, _ model.Time, _ str return nil, nil } -func (NoopIndex) Stats(_ context.Context, _ string, _, _ model.Time, _ IndexStatsAccumulator, _ *index.ShardAnnotation, _ shouldIncludeChunk, _ ...*labels.Matcher) error { +func (NoopIndex) Stats(_ context.Context, _ string, _, _ model.Time, _ IndexStatsAccumulator, _ index.FingerprintFilter, _ shouldIncludeChunk, _ ...*labels.Matcher) error { return nil } func (NoopIndex) SetChunkFilterer(_ chunk.RequestChunkFilterer) {} -func (NoopIndex) Volume(_ context.Context, _ string, _, _ model.Time, _ VolumeAccumulator, _ *index.ShardAnnotation, _ shouldIncludeChunk, _ []string, _ string, _ ...*labels.Matcher) error { +func (NoopIndex) Volume(_ context.Context, _ string, _, _ model.Time, _ VolumeAccumulator, _ index.FingerprintFilter, _ shouldIncludeChunk, _ []string, _ string, _ ...*labels.Matcher) error { return nil } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index/fingerprint.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index/fingerprint.go index 646e587f706ea..f8b45682b93d0 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index/fingerprint.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index/fingerprint.go @@ -8,8 +8,8 @@ import ( // (SeriesRef, Fingerprint) tuples type FingerprintOffsets [][2]uint64 -func (xs FingerprintOffsets) Range(shard ShardAnnotation) (minOffset, maxOffset uint64) { - from, through := shard.Bounds() +func (xs FingerprintOffsets) Range(fpFilter FingerprintFilter) (minOffset, maxOffset uint64) { + from, through := fpFilter.GetFromThrough() lower := sort.Search(len(xs), func(i int) bool { return xs[i][1] >= uint64(from) }) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go index eb9681160f4ef..7aa429d367718 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index/index.go @@ -1849,7 +1849,7 @@ func (r *Reader) ChunkStats(id storage.SeriesRef, from, through int64, lbls *lab return r.dec.ChunkStats(r.version, d.Get(), id, from, through, lbls) } -func (r *Reader) Postings(name string, shard *ShardAnnotation, values ...string) (Postings, error) { +func (r *Reader) Postings(name string, fpFilter FingerprintFilter, values ...string) (Postings, error) { if r.version == FormatV1 { e, ok := r.postingsV1[name] if !ok { @@ -1947,8 +1947,8 @@ func (r *Reader) Postings(name string, shard *ShardAnnotation, values ...string) } merged := Merge(res...) - if shard != nil { - return NewShardedPostings(merged, *shard, r.fingerprintOffsets), nil + if fpFilter != nil { + return NewShardedPostings(merged, fpFilter, r.fingerprintOffsets), nil } return merged, nil diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index/postings.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index/postings.go index 028da1bd06b57..0077f845df29a 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index/postings.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index/postings.go @@ -845,8 +845,8 @@ type ShardedPostings struct { // For example (below), given a shard, we'll likely return a slight superset of offsets surrounding the shard. // ---[shard0]--- # Shard membership // -[--shard0--]- # Series returned by shardedPostings -func NewShardedPostings(p Postings, shard ShardAnnotation, offsets FingerprintOffsets) *ShardedPostings { - min, max := offsets.Range(shard) +func NewShardedPostings(p Postings, fpFilter FingerprintFilter, offsets FingerprintOffsets) *ShardedPostings { + min, max := offsets.Range(fpFilter) return &ShardedPostings{ p: p, minOffset: min, diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index/shard.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index/shard.go index 12d75a06f743b..b188ebbcb24ed 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index/shard.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index/shard.go @@ -17,6 +17,11 @@ const ( var errDisallowedIdentityShard = errors.New("shard with factor of 1 is explicitly disallowed. It's equivalent to no sharding") +type FingerprintFilter interface { + Match(model.Fingerprint) bool + GetFromThrough() (model.Fingerprint, model.Fingerprint) +} + // ShardAnnotation is a convenience struct which holds data from a parsed shard label // Of MUST be a power of 2 to ensure sharding logic works correctly. type ShardAnnotation struct { @@ -72,9 +77,9 @@ func (shard ShardAnnotation) Validate() error { return nil } -// Bounds shows the [minimum, maximum) fingerprints. If there is no maximum +// GetFromThrough shows the [minimum, maximum) fingerprints. If there is no maximum // fingerprint (for example the last shard), math.MaxUint64 is used as the maximum. -func (shard ShardAnnotation) Bounds() (model.Fingerprint, model.Fingerprint) { +func (shard ShardAnnotation) GetFromThrough() (model.Fingerprint, model.Fingerprint) { requiredBits := model.Fingerprint(shard.RequiredBits()) from := model.Fingerprint(shard.Shard) << (64 - requiredBits) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index/shard_test.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index/shard_test.go index 167e9f4baec81..f7613389c32c7 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index/shard_test.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index/shard_test.go @@ -104,7 +104,7 @@ func TestShardBounds(t *testing.T) { }, } { t.Run(tc.shard.String(), func(t *testing.T) { - from, through := tc.shard.Bounds() + from, through := tc.shard.GetFromThrough() require.Equal(t, model.Fingerprint(tc.from), from) require.Equal(t, model.Fingerprint(tc.through), through) }) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index_client.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index_client.go index fae628a3a9078..d609dc0ed27f7 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index_client.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index_client.go @@ -71,7 +71,7 @@ func NewIndexClient(idx Index, opts IndexClientOptions, l Limits) *IndexClient { // In the future, we should use dynamic sharding in TSDB to determine the shard factors // and we may no longer wish to send a shard label inside the queries, // but rather expose it as part of the stores.Index interface -func cleanMatchers(matchers ...*labels.Matcher) ([]*labels.Matcher, *index.ShardAnnotation, error) { +func cleanMatchers(matchers ...*labels.Matcher) ([]*labels.Matcher, index.FingerprintFilter, error) { // first use withoutNameLabel to make a copy with the name label removed matchers = withoutNameLabel(matchers) s, shardLabelIndex, err := astmapper.ShardFromMatchers(matchers) @@ -79,13 +79,14 @@ func cleanMatchers(matchers ...*labels.Matcher) ([]*labels.Matcher, *index.Shard return nil, nil, err } - var shard *index.ShardAnnotation + var fpFilter index.FingerprintFilter if s != nil { matchers = append(matchers[:shardLabelIndex], matchers[shardLabelIndex+1:]...) - shard = &index.ShardAnnotation{ + shard := index.ShardAnnotation{ Shard: uint32(s.Shard), Of: uint32(s.Of), } + fpFilter = shard if err := shard.Validate(); err != nil { return nil, nil, err @@ -97,7 +98,7 @@ func cleanMatchers(matchers ...*labels.Matcher) ([]*labels.Matcher, *index.Shard matchers = append(matchers, labels.MustNewMatcher(labels.MatchEqual, "", "")) } - return matchers, shard, err + return matchers, fpFilter, err } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/index_shipper_querier.go b/pkg/storage/stores/shipper/indexshipper/tsdb/index_shipper_querier.go index c07add72b6714..acace60c1e4b2 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/index_shipper_querier.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/index_shipper_querier.go @@ -84,20 +84,20 @@ func (i *indexShipperQuerier) Close() error { return nil } -func (i *indexShipperQuerier) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, res []ChunkRef, shard *tsdbindex.ShardAnnotation, matchers ...*labels.Matcher) ([]ChunkRef, error) { +func (i *indexShipperQuerier) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, res []ChunkRef, fpFilter tsdbindex.FingerprintFilter, matchers ...*labels.Matcher) ([]ChunkRef, error) { idx, err := i.indices(ctx, from, through, userID) if err != nil { return nil, err } - return idx.GetChunkRefs(ctx, userID, from, through, res, shard, matchers...) + return idx.GetChunkRefs(ctx, userID, from, through, res, fpFilter, matchers...) } -func (i *indexShipperQuerier) Series(ctx context.Context, userID string, from, through model.Time, res []Series, shard *tsdbindex.ShardAnnotation, matchers ...*labels.Matcher) ([]Series, error) { +func (i *indexShipperQuerier) Series(ctx context.Context, userID string, from, through model.Time, res []Series, fpFilter tsdbindex.FingerprintFilter, matchers ...*labels.Matcher) ([]Series, error) { idx, err := i.indices(ctx, from, through, userID) if err != nil { return nil, err } - return idx.Series(ctx, userID, from, through, res, shard, matchers...) + return idx.Series(ctx, userID, from, through, res, fpFilter, matchers...) } func (i *indexShipperQuerier) LabelNames(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]string, error) { @@ -116,22 +116,22 @@ func (i *indexShipperQuerier) LabelValues(ctx context.Context, userID string, fr return idx.LabelValues(ctx, userID, from, through, name, matchers...) } -func (i *indexShipperQuerier) Stats(ctx context.Context, userID string, from, through model.Time, acc IndexStatsAccumulator, shard *tsdbindex.ShardAnnotation, shouldIncludeChunk shouldIncludeChunk, matchers ...*labels.Matcher) error { +func (i *indexShipperQuerier) Stats(ctx context.Context, userID string, from, through model.Time, acc IndexStatsAccumulator, fpFilter tsdbindex.FingerprintFilter, shouldIncludeChunk shouldIncludeChunk, matchers ...*labels.Matcher) error { idx, err := i.indices(ctx, from, through, userID) if err != nil { return err } - return idx.Stats(ctx, userID, from, through, acc, shard, shouldIncludeChunk, matchers...) + return idx.Stats(ctx, userID, from, through, acc, fpFilter, shouldIncludeChunk, matchers...) } -func (i *indexShipperQuerier) Volume(ctx context.Context, userID string, from, through model.Time, acc VolumeAccumulator, shard *tsdbindex.ShardAnnotation, shouldIncludeChunk shouldIncludeChunk, targetLabels []string, aggregateBy string, matchers ...*labels.Matcher) error { +func (i *indexShipperQuerier) Volume(ctx context.Context, userID string, from, through model.Time, acc VolumeAccumulator, fpFilter tsdbindex.FingerprintFilter, shouldIncludeChunk shouldIncludeChunk, targetLabels []string, aggregateBy string, matchers ...*labels.Matcher) error { idx, err := i.indices(ctx, from, through, userID) if err != nil { return err } - return idx.Volume(ctx, userID, from, through, acc, shard, shouldIncludeChunk, targetLabels, aggregateBy, matchers...) + return idx.Volume(ctx, userID, from, through, acc, fpFilter, shouldIncludeChunk, targetLabels, aggregateBy, matchers...) } type resultAccumulator struct { diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/lazy_index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/lazy_index.go index b9877a6ae1ab3..327566f1a0ecc 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/lazy_index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/lazy_index.go @@ -36,19 +36,19 @@ func (f LazyIndex) Close() error { return i.Close() } -func (f LazyIndex) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, res []ChunkRef, shard *index.ShardAnnotation, matchers ...*labels.Matcher) ([]ChunkRef, error) { +func (f LazyIndex) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, res []ChunkRef, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]ChunkRef, error) { i, err := f() if err != nil { return nil, err } - return i.GetChunkRefs(ctx, userID, from, through, res, shard, matchers...) + return i.GetChunkRefs(ctx, userID, from, through, res, fpFilter, matchers...) } -func (f LazyIndex) Series(ctx context.Context, userID string, from, through model.Time, res []Series, shard *index.ShardAnnotation, matchers ...*labels.Matcher) ([]Series, error) { +func (f LazyIndex) Series(ctx context.Context, userID string, from, through model.Time, res []Series, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]Series, error) { i, err := f() if err != nil { return nil, err } - return i.Series(ctx, userID, from, through, res, shard, matchers...) + return i.Series(ctx, userID, from, through, res, fpFilter, matchers...) } func (f LazyIndex) LabelNames(ctx context.Context, userID string, from, through model.Time, matchers ...*labels.Matcher) ([]string, error) { i, err := f() @@ -65,18 +65,18 @@ func (f LazyIndex) LabelValues(ctx context.Context, userID string, from, through return i.LabelValues(ctx, userID, from, through, name, matchers...) } -func (f LazyIndex) Stats(ctx context.Context, userID string, from, through model.Time, acc IndexStatsAccumulator, shard *index.ShardAnnotation, shouldIncludeChunk shouldIncludeChunk, matchers ...*labels.Matcher) error { +func (f LazyIndex) Stats(ctx context.Context, userID string, from, through model.Time, acc IndexStatsAccumulator, fpFilter index.FingerprintFilter, shouldIncludeChunk shouldIncludeChunk, matchers ...*labels.Matcher) error { i, err := f() if err != nil { return err } - return i.Stats(ctx, userID, from, through, acc, shard, shouldIncludeChunk, matchers...) + return i.Stats(ctx, userID, from, through, acc, fpFilter, shouldIncludeChunk, matchers...) } -func (f LazyIndex) Volume(ctx context.Context, userID string, from, through model.Time, acc VolumeAccumulator, shard *index.ShardAnnotation, shouldIncludeChunk shouldIncludeChunk, targetLabels []string, aggregateBy string, matchers ...*labels.Matcher) error { +func (f LazyIndex) Volume(ctx context.Context, userID string, from, through model.Time, acc VolumeAccumulator, fpFilter index.FingerprintFilter, shouldIncludeChunk shouldIncludeChunk, targetLabels []string, aggregateBy string, matchers ...*labels.Matcher) error { i, err := f() if err != nil { return err } - return i.Volume(ctx, userID, from, through, acc, shard, shouldIncludeChunk, targetLabels, aggregateBy, matchers...) + return i.Volume(ctx, userID, from, through, acc, fpFilter, shouldIncludeChunk, targetLabels, aggregateBy, matchers...) } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/multi_file_index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/multi_file_index.go index 01935a842d539..08bf6bf4ff01e 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/multi_file_index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/multi_file_index.go @@ -131,7 +131,7 @@ func (i *MultiIndex) forMatchingIndices(ctx context.Context, from, through model } -func (i *MultiIndex) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, res []ChunkRef, shard *index.ShardAnnotation, matchers ...*labels.Matcher) ([]ChunkRef, error) { +func (i *MultiIndex) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, res []ChunkRef, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]ChunkRef, error) { acc := newResultAccumulator(func(xs []interface{}) (interface{}, error) { if res == nil { res = ChunkRefsPool.Get() @@ -165,7 +165,7 @@ func (i *MultiIndex) GetChunkRefs(ctx context.Context, userID string, from, thro from, through, func(ctx context.Context, idx Index) error { - got, err := idx.GetChunkRefs(ctx, userID, from, through, nil, shard, matchers...) + got, err := idx.GetChunkRefs(ctx, userID, from, through, nil, fpFilter, matchers...) if err != nil { return err } @@ -187,7 +187,7 @@ func (i *MultiIndex) GetChunkRefs(ctx context.Context, userID string, from, thro } -func (i *MultiIndex) Series(ctx context.Context, userID string, from, through model.Time, res []Series, shard *index.ShardAnnotation, matchers ...*labels.Matcher) ([]Series, error) { +func (i *MultiIndex) Series(ctx context.Context, userID string, from, through model.Time, res []Series, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]Series, error) { acc := newResultAccumulator(func(xs []interface{}) (interface{}, error) { if res == nil { res = SeriesPool.Get() @@ -217,7 +217,7 @@ func (i *MultiIndex) Series(ctx context.Context, userID string, from, through mo from, through, func(ctx context.Context, idx Index) error { - got, err := idx.Series(ctx, userID, from, through, nil, shard, matchers...) + got, err := idx.Series(ctx, userID, from, through, nil, fpFilter, matchers...) if err != nil { return err } @@ -354,14 +354,14 @@ func (i *MultiIndex) LabelValues(ctx context.Context, userID string, from, throu return merged.([]string), nil } -func (i *MultiIndex) Stats(ctx context.Context, userID string, from, through model.Time, acc IndexStatsAccumulator, shard *index.ShardAnnotation, shouldIncludeChunk shouldIncludeChunk, matchers ...*labels.Matcher) error { +func (i *MultiIndex) Stats(ctx context.Context, userID string, from, through model.Time, acc IndexStatsAccumulator, fpFilter index.FingerprintFilter, shouldIncludeChunk shouldIncludeChunk, matchers ...*labels.Matcher) error { return i.forMatchingIndices(ctx, from, through, func(ctx context.Context, idx Index) error { - return idx.Stats(ctx, userID, from, through, acc, shard, shouldIncludeChunk, matchers...) + return idx.Stats(ctx, userID, from, through, acc, fpFilter, shouldIncludeChunk, matchers...) }) } -func (i *MultiIndex) Volume(ctx context.Context, userID string, from, through model.Time, acc VolumeAccumulator, shard *index.ShardAnnotation, shouldIncludeChunk shouldIncludeChunk, targetLabels []string, aggregateBy string, matchers ...*labels.Matcher) error { +func (i *MultiIndex) Volume(ctx context.Context, userID string, from, through model.Time, acc VolumeAccumulator, fpFilter index.FingerprintFilter, shouldIncludeChunk shouldIncludeChunk, targetLabels []string, aggregateBy string, matchers ...*labels.Matcher) error { return i.forMatchingIndices(ctx, from, through, func(ctx context.Context, idx Index) error { - return idx.Volume(ctx, userID, from, through, acc, shard, shouldIncludeChunk, targetLabels, aggregateBy, matchers...) + return idx.Volume(ctx, userID, from, through, acc, fpFilter, shouldIncludeChunk, targetLabels, aggregateBy, matchers...) }) } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/multitenant.go b/pkg/storage/stores/shipper/indexshipper/tsdb/multitenant.go index 9dda0886dbf40..ec582b6e21489 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/multitenant.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/multitenant.go @@ -51,12 +51,12 @@ func (m *MultiTenantIndex) SetChunkFilterer(chunkFilter chunk.RequestChunkFilter func (m *MultiTenantIndex) Close() error { return m.idx.Close() } -func (m *MultiTenantIndex) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, res []ChunkRef, shard *index.ShardAnnotation, matchers ...*labels.Matcher) ([]ChunkRef, error) { - return m.idx.GetChunkRefs(ctx, userID, from, through, res, shard, withTenantLabelMatcher(userID, matchers)...) +func (m *MultiTenantIndex) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, res []ChunkRef, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]ChunkRef, error) { + return m.idx.GetChunkRefs(ctx, userID, from, through, res, fpFilter, withTenantLabelMatcher(userID, matchers)...) } -func (m *MultiTenantIndex) Series(ctx context.Context, userID string, from, through model.Time, res []Series, shard *index.ShardAnnotation, matchers ...*labels.Matcher) ([]Series, error) { - xs, err := m.idx.Series(ctx, userID, from, through, res, shard, withTenantLabelMatcher(userID, matchers)...) +func (m *MultiTenantIndex) Series(ctx context.Context, userID string, from, through model.Time, res []Series, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]Series, error) { + xs, err := m.idx.Series(ctx, userID, from, through, res, fpFilter, withTenantLabelMatcher(userID, matchers)...) if err != nil { return nil, err } @@ -89,10 +89,10 @@ func (m *MultiTenantIndex) LabelValues(ctx context.Context, userID string, from, return m.idx.LabelValues(ctx, userID, from, through, name, withTenantLabelMatcher(userID, matchers)...) } -func (m *MultiTenantIndex) Stats(ctx context.Context, userID string, from, through model.Time, acc IndexStatsAccumulator, shard *index.ShardAnnotation, shouldIncludeChunk shouldIncludeChunk, matchers ...*labels.Matcher) error { - return m.idx.Stats(ctx, userID, from, through, acc, shard, shouldIncludeChunk, withTenantLabelMatcher(userID, matchers)...) +func (m *MultiTenantIndex) Stats(ctx context.Context, userID string, from, through model.Time, acc IndexStatsAccumulator, fpFilter index.FingerprintFilter, shouldIncludeChunk shouldIncludeChunk, matchers ...*labels.Matcher) error { + return m.idx.Stats(ctx, userID, from, through, acc, fpFilter, shouldIncludeChunk, withTenantLabelMatcher(userID, matchers)...) } -func (m *MultiTenantIndex) Volume(ctx context.Context, userID string, from, through model.Time, acc VolumeAccumulator, shard *index.ShardAnnotation, shouldIncludeChunk shouldIncludeChunk, targetLabels []string, aggregateBy string, matchers ...*labels.Matcher) error { - return m.idx.Volume(ctx, userID, from, through, acc, shard, shouldIncludeChunk, targetLabels, aggregateBy, withTenantLabelMatcher(userID, matchers)...) +func (m *MultiTenantIndex) Volume(ctx context.Context, userID string, from, through model.Time, acc VolumeAccumulator, fpFilter index.FingerprintFilter, shouldIncludeChunk shouldIncludeChunk, targetLabels []string, aggregateBy string, matchers ...*labels.Matcher) error { + return m.idx.Volume(ctx, userID, from, through, acc, fpFilter, shouldIncludeChunk, targetLabels, aggregateBy, withTenantLabelMatcher(userID, matchers)...) } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/querier.go b/pkg/storage/stores/shipper/indexshipper/tsdb/querier.go index 10498bdf94e3f..b1e3306b14d1f 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/querier.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/querier.go @@ -61,7 +61,7 @@ type IndexReader interface { // The Postings here contain the offsets to the series inside the index. // Found IDs are not strictly required to point to a valid Series, e.g. // during background garbage collections. Input values must be sorted. - Postings(name string, shard *index.ShardAnnotation, values ...string) (index.Postings, error) + Postings(name string, fpFilter index.FingerprintFilter, values ...string) (index.Postings, error) // Series populates the given labels and chunk metas for the series identified // by the reference. @@ -89,7 +89,7 @@ type IndexReader interface { // PostingsForMatchers assembles a single postings iterator against the index reader // based on the given matchers. The resulting postings are not ordered by series. -func PostingsForMatchers(ix IndexReader, shard *index.ShardAnnotation, ms ...*labels.Matcher) (index.Postings, error) { +func PostingsForMatchers(ix IndexReader, fpFilter index.FingerprintFilter, ms ...*labels.Matcher) (index.Postings, error) { var its, notIts []index.Postings // See which label must be non-empty. // Optimization for case like {l=~".", l!="1"}. @@ -113,7 +113,7 @@ func PostingsForMatchers(ix IndexReader, shard *index.ShardAnnotation, ms ...*la return nil, err } - it, err := postingsForMatcher(ix, shard, inverse) + it, err := postingsForMatcher(ix, fpFilter, inverse) if err != nil { return nil, err } @@ -126,14 +126,14 @@ func PostingsForMatchers(ix IndexReader, shard *index.ShardAnnotation, ms ...*la return nil, err } - it, err := inversePostingsForMatcher(ix, shard, inverse) + it, err := inversePostingsForMatcher(ix, fpFilter, inverse) if err != nil { return nil, err } its = append(its, it) } else { // l="a" // Non-Not matcher, use normal postingsForMatcher. - it, err := postingsForMatcher(ix, shard, m) + it, err := postingsForMatcher(ix, fpFilter, m) if err != nil { return nil, err } @@ -144,7 +144,7 @@ func PostingsForMatchers(ix IndexReader, shard *index.ShardAnnotation, ms ...*la // the series which don't have the label name set too. See: // https://github.com/prometheus/prometheus/issues/3575 and // https://github.com/prometheus/prometheus/pull/3578#issuecomment-351653555 - it, err := inversePostingsForMatcher(ix, shard, m) + it, err := inversePostingsForMatcher(ix, fpFilter, m) if err != nil { return nil, err } @@ -155,7 +155,7 @@ func PostingsForMatchers(ix IndexReader, shard *index.ShardAnnotation, ms ...*la // If there's nothing to subtract from, add in everything and remove the notIts later. if len(its) == 0 && len(notIts) != 0 { k, v := index.AllPostingsKey() - allPostings, err := ix.Postings(k, shard, v) + allPostings, err := ix.Postings(k, fpFilter, v) if err != nil { return nil, err } @@ -171,12 +171,12 @@ func PostingsForMatchers(ix IndexReader, shard *index.ShardAnnotation, ms ...*la return it, nil } -func postingsForMatcher(ix IndexReader, shard *index.ShardAnnotation, m *labels.Matcher) (index.Postings, error) { +func postingsForMatcher(ix IndexReader, fpFilter index.FingerprintFilter, m *labels.Matcher) (index.Postings, error) { // This method will not return postings for missing labels. // Fast-path for equal matching. if m.Type == labels.MatchEqual { - return ix.Postings(m.Name, shard, m.Value) + return ix.Postings(m.Name, fpFilter, m.Value) } // Fast-path for set matching. @@ -184,7 +184,7 @@ func postingsForMatcher(ix IndexReader, shard *index.ShardAnnotation, m *labels. setMatches := findSetMatches(m.GetRegexString()) if len(setMatches) > 0 { sort.Strings(setMatches) - return ix.Postings(m.Name, shard, setMatches...) + return ix.Postings(m.Name, fpFilter, setMatches...) } } @@ -212,11 +212,11 @@ func postingsForMatcher(ix IndexReader, shard *index.ShardAnnotation, m *labels. if !isSorted { sort.Strings(res) } - return ix.Postings(m.Name, shard, res...) + return ix.Postings(m.Name, fpFilter, res...) } // inversePostingsForMatcher returns the postings for the series with the label name set but not matching the matcher. -func inversePostingsForMatcher(ix IndexReader, shard *index.ShardAnnotation, m *labels.Matcher) (index.Postings, error) { +func inversePostingsForMatcher(ix IndexReader, fpFilter index.FingerprintFilter, m *labels.Matcher) (index.Postings, error) { vals, err := ix.LabelValues(m.Name) if err != nil { return nil, err @@ -237,7 +237,7 @@ func inversePostingsForMatcher(ix IndexReader, shard *index.ShardAnnotation, m * if !isSorted { sort.Strings(res) } - return ix.Postings(m.Name, shard, res...) + return ix.Postings(m.Name, fpFilter, res...) } func findSetMatches(pattern string) []string { diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go index fb3a85c9f0460..0e1ae029a8677 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go @@ -157,7 +157,7 @@ func (i *TSDBIndex) SetChunkFilterer(chunkFilter chunk.RequestChunkFilterer) { // fn must NOT capture it's arguments. They're reused across series iterations and returned to // a pool after completion. -func (i *TSDBIndex) ForSeries(ctx context.Context, shard *index.ShardAnnotation, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta), matchers ...*labels.Matcher) error { +func (i *TSDBIndex) ForSeries(ctx context.Context, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta), matchers ...*labels.Matcher) error { // TODO(owen-d): use pool var ls labels.Labels @@ -169,7 +169,7 @@ func (i *TSDBIndex) ForSeries(ctx context.Context, shard *index.ShardAnnotation, filterer = i.chunkFilter.ForRequest(ctx) } - return i.forPostings(ctx, shard, from, through, matchers, func(p index.Postings) error { + return i.forPostings(ctx, fpFilter, from, through, matchers, func(p index.Postings) error { for p.Next() { hash, err := i.reader.Series(p.At(), int64(from), int64(through), &ls, &chks) if err != nil { @@ -177,7 +177,7 @@ func (i *TSDBIndex) ForSeries(ctx context.Context, shard *index.ShardAnnotation, } // skip series that belong to different shards - if shard != nil && !shard.Match(model.Fingerprint(hash)) { + if fpFilter != nil && !fpFilter.Match(model.Fingerprint(hash)) { continue } @@ -194,25 +194,25 @@ func (i *TSDBIndex) ForSeries(ctx context.Context, shard *index.ShardAnnotation, func (i *TSDBIndex) forPostings( _ context.Context, - shard *index.ShardAnnotation, + fpFilter index.FingerprintFilter, _, _ model.Time, matchers []*labels.Matcher, fn func(index.Postings) error, ) error { - p, err := PostingsForMatchers(i.reader, shard, matchers...) + p, err := PostingsForMatchers(i.reader, fpFilter, matchers...) if err != nil { return err } return fn(p) } -func (i *TSDBIndex) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, res []ChunkRef, shard *index.ShardAnnotation, matchers ...*labels.Matcher) ([]ChunkRef, error) { +func (i *TSDBIndex) GetChunkRefs(ctx context.Context, userID string, from, through model.Time, res []ChunkRef, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]ChunkRef, error) { if res == nil { res = ChunkRefsPool.Get() } res = res[:0] - if err := i.ForSeries(ctx, shard, from, through, func(ls labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta) { + if err := i.ForSeries(ctx, fpFilter, from, through, func(ls labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta) { for _, chk := range chks { res = append(res, ChunkRef{ @@ -230,13 +230,13 @@ func (i *TSDBIndex) GetChunkRefs(ctx context.Context, userID string, from, throu return res, nil } -func (i *TSDBIndex) Series(ctx context.Context, _ string, from, through model.Time, res []Series, shard *index.ShardAnnotation, matchers ...*labels.Matcher) ([]Series, error) { +func (i *TSDBIndex) Series(ctx context.Context, _ string, from, through model.Time, res []Series, fpFilter index.FingerprintFilter, matchers ...*labels.Matcher) ([]Series, error) { if res == nil { res = SeriesPool.Get() } res = res[:0] - if err := i.ForSeries(ctx, shard, from, through, func(ls labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta) { + if err := i.ForSeries(ctx, fpFilter, from, through, func(ls labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta) { if len(chks) == 0 { return } @@ -280,8 +280,8 @@ func (i *TSDBIndex) Identifier(string) SingleTenantTSDBIdentifier { } } -func (i *TSDBIndex) Stats(ctx context.Context, _ string, from, through model.Time, acc IndexStatsAccumulator, shard *index.ShardAnnotation, _ shouldIncludeChunk, matchers ...*labels.Matcher) error { - return i.forPostings(ctx, shard, from, through, matchers, func(p index.Postings) error { +func (i *TSDBIndex) Stats(ctx context.Context, _ string, from, through model.Time, acc IndexStatsAccumulator, fpFilter index.FingerprintFilter, _ shouldIncludeChunk, matchers ...*labels.Matcher) error { + return i.forPostings(ctx, fpFilter, from, through, matchers, func(p index.Postings) error { // TODO(owen-d): use pool var ls labels.Labels var filterer chunk.Filterer @@ -296,7 +296,7 @@ func (i *TSDBIndex) Stats(ctx context.Context, _ string, from, through model.Tim } // skip series that belong to different shards - if shard != nil && !shard.Match(model.Fingerprint(fp)) { + if fpFilter != nil && !fpFilter.Match(model.Fingerprint(fp)) { continue } @@ -339,7 +339,7 @@ func (i *TSDBIndex) Volume( _ string, from, through model.Time, acc VolumeAccumulator, - shard *index.ShardAnnotation, + fpFilter index.FingerprintFilter, _ shouldIncludeChunk, targetLabels []string, aggregateBy string, @@ -355,7 +355,7 @@ func (i *TSDBIndex) Volume( aggregateBySeries := seriesvolume.AggregateBySeries(aggregateBy) || aggregateBy == "" - return i.forPostings(ctx, shard, from, through, matchers, func(p index.Postings) error { + return i.forPostings(ctx, fpFilter, from, through, matchers, func(p index.Postings) error { var ls labels.Labels var filterer chunk.Filterer if i.chunkFilter != nil { @@ -369,7 +369,7 @@ func (i *TSDBIndex) Volume( } // skip series that belong to different shards - if shard != nil && !shard.Match(model.Fingerprint(fp)) { + if fpFilter != nil && !fpFilter.Match(model.Fingerprint(fp)) { continue } From 098eef7bd43503bc8967f1008b19d4b648fc9f35 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 1 Feb 2024 10:56:16 -0800 Subject: [PATCH 21/45] Blooms/bounds integration (#11848) Replaces `Ref`'s min/max fingerprint fields with `v1.FingerprintBounds` and improves a bunch of checks that use it. Builds on top of https://github.com/grafana/loki/pull/11847 --- pkg/bloomgateway/processor.go | 2 +- pkg/bloomgateway/processor_test.go | 2 +- pkg/bloomgateway/util_test.go | 14 ++-- pkg/bloomgateway/worker.go | 2 +- pkg/storage/bloom/v1/bounds.go | 28 +++++++- pkg/storage/bloom/v1/bounds_test.go | 23 +++++- .../stores/shipper/bloomshipper/client.go | 38 ++++------ .../shipper/bloomshipper/client_test.go | 70 ++++++++++--------- .../shipper/bloomshipper/fetcher_test.go | 3 +- .../stores/shipper/bloomshipper/shipper.go | 17 ++--- .../shipper/bloomshipper/shipper_test.go | 3 +- .../stores/shipper/bloomshipper/store.go | 13 +++- 12 files changed, 123 insertions(+), 92 deletions(-) diff --git a/pkg/bloomgateway/processor.go b/pkg/bloomgateway/processor.go index 60753c32ac277..117e736e4f54f 100644 --- a/pkg/bloomgateway/processor.go +++ b/pkg/bloomgateway/processor.go @@ -79,7 +79,7 @@ outer: for blockIter.Next() { bq := blockIter.At() for i, block := range data { - if block.blockRef.Bounds().Equal(bq.FingerprintBounds) { + if block.blockRef.Bounds.Equal(bq.FingerprintBounds) { err := p.processBlock(ctx, bq.BlockQuerier, block.tasks) if err != nil { return err diff --git a/pkg/bloomgateway/processor_test.go b/pkg/bloomgateway/processor_test.go index 6b43e688a4cc5..7ef8e067bac38 100644 --- a/pkg/bloomgateway/processor_test.go +++ b/pkg/bloomgateway/processor_test.go @@ -50,7 +50,7 @@ func (s *dummyStore) LoadBlocks(_ context.Context, refs []bloomshipper.BlockRef) for _, ref := range refs { for _, bq := range s.querieres { - if ref.Bounds().Equal(bq.FingerprintBounds) { + if ref.Bounds.Equal(bq.FingerprintBounds) { result = append(result, bq) } } diff --git a/pkg/bloomgateway/util_test.go b/pkg/bloomgateway/util_test.go index 969f0ddacd7b6..a705d1965780f 100644 --- a/pkg/bloomgateway/util_test.go +++ b/pkg/bloomgateway/util_test.go @@ -71,8 +71,7 @@ func TestTruncateDay(t *testing.T) { func mkBlockRef(minFp, maxFp uint64) bloomshipper.BlockRef { return bloomshipper.BlockRef{ Ref: bloomshipper.Ref{ - MinFingerprint: minFp, - MaxFingerprint: maxFp, + Bounds: v1.NewBounds(model.Fingerprint(minFp), model.Fingerprint(maxFp)), }, } } @@ -339,8 +338,7 @@ func createBlocks(t *testing.T, tenant string, n int, from, through model.Time, ref := bloomshipper.Ref{ TenantID: tenant, TableName: "table_0", - MinFingerprint: uint64(fromFp), - MaxFingerprint: uint64(throughFp), + Bounds: v1.NewBounds(fromFp, throughFp), StartTimestamp: from, EndTimestamp: through, } @@ -390,9 +388,8 @@ func (s *mockBloomStore) GetBlockRefs(_ context.Context, tenant string, _ blooms for i := range s.bqs { blocks = append(blocks, bloomshipper.BlockRef{ Ref: bloomshipper.Ref{ - MinFingerprint: uint64(s.bqs[i].Min), - MaxFingerprint: uint64(s.bqs[i].Max), - TenantID: tenant, + Bounds: v1.NewBounds(s.bqs[i].Min, s.bqs[i].Max), + TenantID: tenant, }, }) } @@ -457,8 +454,7 @@ func createBlockRefsFromBlockData(t *testing.T, tenant string, data []bloomshipp Ref: bloomshipper.Ref{ TenantID: tenant, TableName: "", - MinFingerprint: uint64(data[i].Min), - MaxFingerprint: uint64(data[i].Max), + Bounds: v1.NewBounds(data[i].Min, data[i].Max), StartTimestamp: 0, EndTimestamp: 0, Checksum: 0, diff --git a/pkg/bloomgateway/worker.go b/pkg/bloomgateway/worker.go index 5a8549a6c0d3a..34a01e50c4354 100644 --- a/pkg/bloomgateway/worker.go +++ b/pkg/bloomgateway/worker.go @@ -242,7 +242,7 @@ func (w *worker) stopping(err error) error { func (w *worker) processBlocksWithCallback(taskCtx context.Context, tenant string, blockRefs []bloomshipper.BlockRef, boundedRefs []boundedTasks) error { return w.shipper.Fetch(taskCtx, tenant, blockRefs, func(bq *v1.BlockQuerier, bounds v1.FingerprintBounds) error { for _, b := range boundedRefs { - if b.blockRef.Bounds().Equal(bounds) { + if b.blockRef.Bounds.Equal(bounds) { return w.processBlock(bq, b.tasks) } } diff --git a/pkg/storage/bloom/v1/bounds.go b/pkg/storage/bloom/v1/bounds.go index 961060198c393..507f6e45aefc2 100644 --- a/pkg/storage/bloom/v1/bounds.go +++ b/pkg/storage/bloom/v1/bounds.go @@ -1,7 +1,9 @@ package v1 import ( + "fmt" "hash" + "strings" "github.com/pkg/errors" "github.com/prometheus/common/model" @@ -17,6 +19,26 @@ const ( After ) +// ParseBoundsFromAddr parses a fingerprint bounds from a string +func ParseBoundsFromAddr(s string) (FingerprintBounds, error) { + parts := strings.Split(s, "-") + return ParseBoundsFromParts(parts[0], parts[1]) +} + +// ParseBoundsFromParts parses a fingerprint bounds already separated strings +func ParseBoundsFromParts(a, b string) (FingerprintBounds, error) { + minFingerprint, err := model.ParseFingerprint(a) + if err != nil { + return FingerprintBounds{}, fmt.Errorf("error parsing minFingerprint %s : %w", a, err) + } + maxFingerprint, err := model.ParseFingerprint(b) + if err != nil { + return FingerprintBounds{}, fmt.Errorf("error parsing maxFingerprint %s : %w", b, err) + } + + return NewBounds(minFingerprint, maxFingerprint), nil +} + type FingerprintBounds struct { Min, Max model.Fingerprint } @@ -33,8 +55,12 @@ func (b FingerprintBounds) Hash(h hash.Hash32) error { return errors.Wrap(err, "writing FingerprintBounds") } +// Addr returns the string representation of the fingerprint bounds for use in +// content addressable storage. +// TODO(owen-d): incorporate this into the schema so we can change it, +// similar to `{,Parse}ExternalKey` func (b FingerprintBounds) String() string { - return b.Min.String() + "-" + b.Max.String() + return fmt.Sprintf("%016x-%016x", uint64(b.Min), uint64(b.Max)) } func (b FingerprintBounds) Less(other FingerprintBounds) bool { diff --git a/pkg/storage/bloom/v1/bounds_test.go b/pkg/storage/bloom/v1/bounds_test.go index 3a80f6e6b849a..1dabe1b70dca0 100644 --- a/pkg/storage/bloom/v1/bounds_test.go +++ b/pkg/storage/bloom/v1/bounds_test.go @@ -3,12 +3,31 @@ package v1 import ( "testing" + "github.com/prometheus/common/model" "github.com/stretchr/testify/assert" ) +func Test_ParseFingerprint(t *testing.T) { + fp, err := model.ParseFingerprint("7d0") + assert.NoError(t, err) + assert.Equal(t, model.Fingerprint(2000), fp) +} + func Test_FingerprintBounds_String(t *testing.T) { - bounds := NewBounds(1, 2) - assert.Equal(t, "0000000000000001-0000000000000002", bounds.String()) + bounds := NewBounds(10, 2000) + assert.Equal(t, "000000000000000a-00000000000007d0", bounds.String()) +} + +func Test_ParseBoundsFromAddr(t *testing.T) { + bounds, err := ParseBoundsFromAddr("a-7d0") + assert.NoError(t, err) + assert.Equal(t, NewBounds(10, 2000), bounds) +} + +func Test_ParseBoundsFromParts(t *testing.T) { + bounds, err := ParseBoundsFromParts("a", "7d0") + assert.NoError(t, err) + assert.Equal(t, NewBounds(10, 2000), bounds) } func Test_FingerprintBounds_Cmp(t *testing.T) { diff --git a/pkg/storage/stores/shipper/bloomshipper/client.go b/pkg/storage/stores/shipper/bloomshipper/client.go index 053d751f59f18..6944d51da6412 100644 --- a/pkg/storage/stores/shipper/bloomshipper/client.go +++ b/pkg/storage/stores/shipper/bloomshipper/client.go @@ -28,25 +28,16 @@ const ( ) type Ref struct { - TenantID string - TableName string - MinFingerprint, MaxFingerprint uint64 - StartTimestamp, EndTimestamp model.Time - Checksum uint32 + TenantID string + TableName string + Bounds v1.FingerprintBounds + StartTimestamp, EndTimestamp model.Time + Checksum uint32 } // Cmp returns the fingerprint's position relative to the bounds func (r Ref) Cmp(fp uint64) v1.BoundsCheck { - if fp < r.MinFingerprint { - return v1.Before - } else if fp > r.MaxFingerprint { - return v1.After - } - return v1.Overlap -} - -func (r Ref) Bounds() v1.FingerprintBounds { - return v1.NewBounds(model.Fingerprint(r.MinFingerprint), model.Fingerprint(r.MaxFingerprint)) + return r.Bounds.Cmp(model.Fingerprint(fp)) } func (r Ref) Interval() Interval { @@ -135,13 +126,13 @@ func (b *BloomClient) PutMeta(ctx context.Context, meta Meta) error { } func externalBlockKey(ref BlockRef) string { - blockParentFolder := fmt.Sprintf("%x-%x", ref.MinFingerprint, ref.MaxFingerprint) + blockParentFolder := ref.Bounds.String() filename := fmt.Sprintf("%d-%d-%x", ref.StartTimestamp, ref.EndTimestamp, ref.Checksum) return path.Join(rootFolder, ref.TableName, ref.TenantID, bloomsFolder, blockParentFolder, filename) } func externalMetaKey(ref MetaRef) string { - filename := fmt.Sprintf("%x-%x-%d-%d-%x", ref.MinFingerprint, ref.MaxFingerprint, ref.StartTimestamp, ref.EndTimestamp, ref.Checksum) + filename := fmt.Sprintf("%s-%d-%d-%x", ref.Bounds.String(), ref.StartTimestamp, ref.EndTimestamp, ref.Checksum) return path.Join(rootFolder, ref.TableName, ref.TenantID, metasFolder, filename) } @@ -251,15 +242,11 @@ func createMetaRef(objectKey string, tenantID string, tableName string) (MetaRef if len(parts) != 5 { return MetaRef{}, fmt.Errorf("%s filename parts count must be 5 but was %d: [%s]", objectKey, len(parts), strings.Join(parts, ", ")) } - - minFingerprint, err := strconv.ParseUint(parts[0], 16, 64) + bounds, err := v1.ParseBoundsFromParts(parts[0], parts[1]) if err != nil { - return MetaRef{}, fmt.Errorf("error parsing minFingerprint %s : %w", parts[0], err) - } - maxFingerprint, err := strconv.ParseUint(parts[1], 16, 64) - if err != nil { - return MetaRef{}, fmt.Errorf("error parsing maxFingerprint %s : %w", parts[1], err) + return MetaRef{}, fmt.Errorf("error parsing bounds %s : %w", parts[0], err) } + startTimestamp, err := strconv.ParseInt(parts[2], 10, 64) if err != nil { return MetaRef{}, fmt.Errorf("error parsing startTimestamp %s : %w", parts[2], err) @@ -276,8 +263,7 @@ func createMetaRef(objectKey string, tenantID string, tableName string) (MetaRef Ref: Ref{ TenantID: tenantID, TableName: tableName, - MinFingerprint: minFingerprint, - MaxFingerprint: maxFingerprint, + Bounds: bounds, StartTimestamp: model.Time(startTimestamp), EndTimestamp: model.Time(endTimestamp), Checksum: uint32(checksum), diff --git a/pkg/storage/stores/shipper/bloomshipper/client_test.go b/pkg/storage/stores/shipper/bloomshipper/client_test.go index a6b3869229b82..3d6eac07521de 100644 --- a/pkg/storage/stores/shipper/bloomshipper/client_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/client_test.go @@ -113,7 +113,7 @@ func Test_BloomClient_PutMeta(t *testing.T) { "ignored-file-path-during-uploading", ), expectedStorage: "folder-1", - expectedFilePath: "bloom/first-period-19621/tenantA/metas/ff-fff-1695272400000-1695276000000-aaa", + expectedFilePath: fmt.Sprintf("bloom/first-period-19621/tenantA/metas/%s-1695272400000-1695276000000-aaa", v1.NewBounds(0xff, 0xfff)), }, "expected meta to be uploaded to the second folder": { source: createMetaEntity("tenantA", @@ -126,7 +126,7 @@ func Test_BloomClient_PutMeta(t *testing.T) { "ignored-file-path-during-uploading", ), expectedStorage: "folder-2", - expectedFilePath: "bloom/second-period-19625/tenantA/metas/c8-12c-1695600000000-1695603600000-bbb", + expectedFilePath: fmt.Sprintf("bloom/second-period-19625/tenantA/metas/%s-1695600000000-1695603600000-bbb", v1.NewBounds(200, 300)), }, } for name, data := range tests { @@ -169,7 +169,7 @@ func Test_BloomClient_DeleteMeta(t *testing.T) { "ignored-file-path-during-uploading", ), expectedStorage: "folder-1", - expectedFilePath: "bloom/first-period-19621/tenantA/metas/ff-fff-1695272400000-1695276000000-aaa", + expectedFilePath: fmt.Sprintf("bloom/first-period-19621/tenantA/metas/%s-1695272400000-1695276000000-aaa", v1.NewBounds(0xff, 0xfff)), }, "expected meta to be delete from the second folder": { source: createMetaEntity("tenantA", @@ -182,7 +182,7 @@ func Test_BloomClient_DeleteMeta(t *testing.T) { "ignored-file-path-during-uploading", ), expectedStorage: "folder-2", - expectedFilePath: "bloom/second-period-19625/tenantA/metas/c8-12c-1695600000000-1695603600000-bbb", + expectedFilePath: fmt.Sprintf("bloom/second-period-19625/tenantA/metas/%s-1695600000000-1695603600000-bbb", v1.NewBounds(200, 300)), }, } for name, data := range tests { @@ -207,10 +207,10 @@ func Test_BloomClient_DeleteMeta(t *testing.T) { func Test_BloomClient_GetBlocks(t *testing.T) { bloomClient := createStore(t) fsNamedStores := bloomClient.storageConfig.NamedStores.Filesystem - firstBlockPath := "bloom/first-period-19621/tenantA/blooms/eeee-ffff/1695272400000-1695276000000-1" + firstBlockPath := fmt.Sprintf("bloom/first-period-19621/tenantA/blooms/%s/1695272400000-1695276000000-1", v1.NewBounds(0xeeee, 0xffff)) firstBlockFullPath := filepath.Join(fsNamedStores["folder-1"].Directory, firstBlockPath) firstBlockData := createBlockFile(t, firstBlockFullPath) - secondBlockPath := "bloom/second-period-19624/tenantA/blooms/aaaa-bbbb/1695531600000-1695535200000-2" + secondBlockPath := fmt.Sprintf("bloom/second-period-19624/tenantA/blooms/%s/1695531600000-1695535200000-2", v1.NewBounds(0xaaaa, 0xbbbb)) secondBlockFullPath := filepath.Join(fsNamedStores["folder-2"].Directory, secondBlockPath) secondBlockData := createBlockFile(t, secondBlockFullPath) require.FileExists(t, firstBlockFullPath) @@ -220,8 +220,7 @@ func Test_BloomClient_GetBlocks(t *testing.T) { Ref: Ref{ TenantID: "tenantA", TableName: "first-period-19621", - MinFingerprint: 0xeeee, - MaxFingerprint: 0xffff, + Bounds: v1.NewBounds(0xeeee, 0xffff), StartTimestamp: Date(2023, time.September, 21, 5, 0, 0), EndTimestamp: Date(2023, time.September, 21, 6, 0, 0), Checksum: 1, @@ -232,8 +231,7 @@ func Test_BloomClient_GetBlocks(t *testing.T) { Ref: Ref{ TenantID: "tenantA", TableName: "second-period-19624", - MinFingerprint: 0xaaaa, - MaxFingerprint: 0xbbbb, + Bounds: v1.NewBounds(0xaaaa, 0xbbbb), StartTimestamp: Date(2023, time.September, 24, 5, 0, 0), EndTimestamp: Date(2023, time.September, 24, 6, 0, 0), Checksum: 2, @@ -262,8 +260,7 @@ func Test_BloomClient_PutBlocks(t *testing.T) { Ref: Ref{ TenantID: "tenantA", TableName: "first-period-19621", - MinFingerprint: 0xeeee, - MaxFingerprint: 0xffff, + Bounds: v1.NewBounds(0xeeee, 0xffff), StartTimestamp: Date(2023, time.September, 21, 5, 0, 0), EndTimestamp: Date(2023, time.September, 21, 6, 0, 0), Checksum: 1, @@ -279,8 +276,7 @@ func Test_BloomClient_PutBlocks(t *testing.T) { Ref: Ref{ TenantID: "tenantA", TableName: "second-period-19624", - MinFingerprint: 0xaaaa, - MaxFingerprint: 0xbbbb, + Bounds: v1.NewBounds(0xaaaa, 0xbbbb), StartTimestamp: Date(2023, time.September, 24, 5, 0, 0), EndTimestamp: Date(2023, time.September, 24, 6, 0, 0), Checksum: 2, @@ -295,11 +291,17 @@ func Test_BloomClient_PutBlocks(t *testing.T) { require.Len(t, results, 2) firstResultBlock := results[0] path := firstResultBlock.BlockPath - require.Equal(t, "bloom/first-period-19621/tenantA/blooms/eeee-ffff/1695272400000-1695276000000-1", path) + require.Equal(t, + fmt.Sprintf( + "bloom/first-period-19621/tenantA/blooms/%s/1695272400000-1695276000000-1", + v1.NewBounds(0xeeee, 0xffff), + ), + path, + ) require.Equal(t, blockForFirstFolder.TenantID, firstResultBlock.TenantID) require.Equal(t, blockForFirstFolder.TableName, firstResultBlock.TableName) - require.Equal(t, blockForFirstFolder.MinFingerprint, firstResultBlock.MinFingerprint) - require.Equal(t, blockForFirstFolder.MaxFingerprint, firstResultBlock.MaxFingerprint) + require.Equal(t, blockForFirstFolder.Bounds.Min, firstResultBlock.Bounds.Min) + require.Equal(t, blockForFirstFolder.Bounds.Max, firstResultBlock.Bounds.Max) require.Equal(t, blockForFirstFolder.StartTimestamp, firstResultBlock.StartTimestamp) require.Equal(t, blockForFirstFolder.EndTimestamp, firstResultBlock.EndTimestamp) require.Equal(t, blockForFirstFolder.Checksum, firstResultBlock.Checksum) @@ -313,11 +315,17 @@ func Test_BloomClient_PutBlocks(t *testing.T) { secondResultBlock := results[1] path = secondResultBlock.BlockPath - require.Equal(t, "bloom/second-period-19624/tenantA/blooms/aaaa-bbbb/1695531600000-1695535200000-2", path) + require.Equal(t, + fmt.Sprintf( + "bloom/second-period-19624/tenantA/blooms/%s/1695531600000-1695535200000-2", + v1.NewBounds(0xaaaa, 0xbbbb), + ), + path, + ) require.Equal(t, blockForSecondFolder.TenantID, secondResultBlock.TenantID) require.Equal(t, blockForSecondFolder.TableName, secondResultBlock.TableName) - require.Equal(t, blockForSecondFolder.MinFingerprint, secondResultBlock.MinFingerprint) - require.Equal(t, blockForSecondFolder.MaxFingerprint, secondResultBlock.MaxFingerprint) + require.Equal(t, blockForSecondFolder.Bounds.Min, secondResultBlock.Bounds.Min) + require.Equal(t, blockForSecondFolder.Bounds.Max, secondResultBlock.Bounds.Max) require.Equal(t, blockForSecondFolder.StartTimestamp, secondResultBlock.StartTimestamp) require.Equal(t, blockForSecondFolder.EndTimestamp, secondResultBlock.EndTimestamp) require.Equal(t, blockForSecondFolder.Checksum, secondResultBlock.Checksum) @@ -334,9 +342,9 @@ func Test_BloomClient_PutBlocks(t *testing.T) { func Test_BloomClient_DeleteBlocks(t *testing.T) { bloomClient := createStore(t) fsNamedStores := bloomClient.storageConfig.NamedStores.Filesystem - block1Path := filepath.Join(fsNamedStores["folder-1"].Directory, "bloom/first-period-19621/tenantA/blooms/eeee-ffff/1695272400000-1695276000000-1") + block1Path := filepath.Join(fsNamedStores["folder-1"].Directory, "bloom/first-period-19621/tenantA/blooms/000000000000eeee-000000000000ffff/1695272400000-1695276000000-1") createBlockFile(t, block1Path) - block2Path := filepath.Join(fsNamedStores["folder-2"].Directory, "bloom/second-period-19624/tenantA/blooms/aaaa-bbbb/1695531600000-1695535200000-2") + block2Path := filepath.Join(fsNamedStores["folder-2"].Directory, "bloom/second-period-19624/tenantA/blooms/000000000000aaaa-000000000000bbbb/1695531600000-1695535200000-2") createBlockFile(t, block2Path) require.FileExists(t, block1Path) require.FileExists(t, block2Path) @@ -346,8 +354,7 @@ func Test_BloomClient_DeleteBlocks(t *testing.T) { Ref: Ref{ TenantID: "tenantA", TableName: "second-period-19624", - MinFingerprint: 0xaaaa, - MaxFingerprint: 0xbbbb, + Bounds: v1.NewBounds(0xaaaa, 0xbbbb), StartTimestamp: Date(2023, time.September, 24, 5, 0, 0), EndTimestamp: Date(2023, time.September, 24, 6, 0, 0), Checksum: 2, @@ -358,8 +365,7 @@ func Test_BloomClient_DeleteBlocks(t *testing.T) { Ref: Ref{ TenantID: "tenantA", TableName: "first-period-19621", - MinFingerprint: 0xeeee, - MaxFingerprint: 0xffff, + Bounds: v1.NewBounds(0xeeee, 0xffff), StartTimestamp: Date(2023, time.September, 21, 5, 0, 0), EndTimestamp: Date(2023, time.September, 21, 6, 0, 0), Checksum: 1, @@ -399,8 +405,7 @@ func Test_createMetaRef(t *testing.T) { Ref: Ref{ TenantID: "tenant1", TableName: "table1", - MinFingerprint: 0xaaa, - MaxFingerprint: 0xbbb, + Bounds: v1.NewBounds(0xaaa, 0xbbb), StartTimestamp: 1234567890, EndTimestamp: 9876543210, Checksum: 0xabcdef, @@ -535,8 +540,7 @@ func createMetaEntity( Ref: Ref{ TenantID: tenant, TableName: tableName, - MinFingerprint: minFingerprint, - MaxFingerprint: maxFingerprint, + Bounds: v1.NewBounds(model.Fingerprint(minFingerprint), model.Fingerprint(maxFingerprint)), StartTimestamp: startTimestamp, EndTimestamp: endTimestamp, Checksum: metaChecksum, @@ -548,8 +552,7 @@ func createMetaEntity( Ref: Ref{ TenantID: tenant, Checksum: metaChecksum + 1, - MinFingerprint: minFingerprint, - MaxFingerprint: maxFingerprint, + Bounds: v1.NewBounds(model.Fingerprint(minFingerprint), model.Fingerprint(maxFingerprint)), StartTimestamp: startTimestamp, EndTimestamp: endTimestamp, }, @@ -562,8 +565,7 @@ func createMetaEntity( Ref: Ref{ TenantID: tenant, Checksum: metaChecksum + 2, - MinFingerprint: minFingerprint, - MaxFingerprint: maxFingerprint, + Bounds: v1.NewBounds(model.Fingerprint(minFingerprint), model.Fingerprint(maxFingerprint)), StartTimestamp: startTimestamp, EndTimestamp: endTimestamp, }, diff --git a/pkg/storage/stores/shipper/bloomshipper/fetcher_test.go b/pkg/storage/stores/shipper/bloomshipper/fetcher_test.go index 85117a718f629..3bb3e1348f1bd 100644 --- a/pkg/storage/stores/shipper/bloomshipper/fetcher_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/fetcher_test.go @@ -28,8 +28,7 @@ func makeMetas(t *testing.T, schemaCfg config.SchemaConfig, ts model.Time, keysp Ref: Ref{ TenantID: "fake", TableName: fmt.Sprintf("%s%d", schemaCfg.Configs[0].IndexTables.Prefix, 0), - MinFingerprint: uint64(keyspace.Min), - MaxFingerprint: uint64(keyspace.Max), + Bounds: keyspace, StartTimestamp: ts, EndTimestamp: ts, }, diff --git a/pkg/storage/stores/shipper/bloomshipper/shipper.go b/pkg/storage/stores/shipper/bloomshipper/shipper.go index 105e27f065b74..fad9b2616b6bc 100644 --- a/pkg/storage/stores/shipper/bloomshipper/shipper.go +++ b/pkg/storage/stores/shipper/bloomshipper/shipper.go @@ -4,11 +4,11 @@ import ( "context" "fmt" "math" + "sort" "github.com/go-kit/log" "github.com/go-kit/log/level" "github.com/prometheus/client_golang/prometheus" - "golang.org/x/exp/slices" v1 "github.com/grafana/loki/pkg/storage/bloom/v1" "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper/config" @@ -105,7 +105,7 @@ func runCallback(callback ForEachBlockCallback, block blockWithQuerier) error { _ = b.Close() }(block) - err := callback(block.closableBlockQuerier.BlockQuerier, block.Bounds()) + err := callback(block.closableBlockQuerier.BlockQuerier, block.Bounds) if err != nil { return fmt.Errorf("error running callback function for block %s err: %w", block.BlockPath, err) } @@ -168,15 +168,8 @@ func BlocksForMetas(metas []Meta, interval Interval, keyspaces []v1.FingerprintB blockRefs = append(blockRefs, ref) } - slices.SortStableFunc(blockRefs, func(a, b BlockRef) int { - if a.MinFingerprint < b.MinFingerprint { - return -1 - } - if a.MinFingerprint > b.MinFingerprint { - return 1 - } - - return 0 + sort.Slice(blockRefs, func(i, j int) bool { + return blockRefs[i].Bounds.Less(blockRefs[j].Bounds) }) return blockRefs @@ -193,7 +186,7 @@ func isOutsideRange(b BlockRef, interval Interval, bounds []v1.FingerprintBounds // check fingerprint ranges for _, keyspace := range bounds { - if keyspace.Within(b.Bounds()) || keyspace.Overlaps(b.Bounds()) { + if keyspace.Overlaps(b.Bounds) { return false } } diff --git a/pkg/storage/stores/shipper/bloomshipper/shipper_test.go b/pkg/storage/stores/shipper/bloomshipper/shipper_test.go index 59db4f06ae20a..f3ef55a4f3901 100644 --- a/pkg/storage/stores/shipper/bloomshipper/shipper_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/shipper_test.go @@ -195,8 +195,7 @@ func createBlockRef( Ref: Ref{ TenantID: "fake", TableName: fmt.Sprintf("%d", day), - MinFingerprint: minFingerprint, - MaxFingerprint: maxFingerprint, + Bounds: v1.NewBounds(model.Fingerprint(minFingerprint), model.Fingerprint(maxFingerprint)), StartTimestamp: startTimestamp, EndTimestamp: endTimestamp, Checksum: 0, diff --git a/pkg/storage/stores/shipper/bloomshipper/store.go b/pkg/storage/stores/shipper/bloomshipper/store.go index b53a94526b31e..d4da6573ba5df 100644 --- a/pkg/storage/stores/shipper/bloomshipper/store.go +++ b/pkg/storage/stores/shipper/bloomshipper/store.go @@ -12,6 +12,7 @@ import ( "github.com/prometheus/common/model" "github.com/grafana/loki/pkg/storage" + v1 "github.com/grafana/loki/pkg/storage/bloom/v1" "github.com/grafana/loki/pkg/storage/chunk/cache" "github.com/grafana/loki/pkg/storage/chunk/client" "github.com/grafana/loki/pkg/storage/config" @@ -54,10 +55,20 @@ func (b *bloomStoreEntry) ResolveMetas(ctx context.Context, params MetaSearchPar if err != nil { return nil, nil, err } - if metaRef.MaxFingerprint < uint64(params.Keyspace.Min) || uint64(params.Keyspace.Max) < metaRef.MinFingerprint || + + // LIST calls return keys in lexicographic order. + // Since fingerprints are the first part of the path, + // we can stop iterating once we find an item greater + // than the keyspace we're looking for + if params.Keyspace.Cmp(metaRef.Bounds.Min) == v1.After { + break + } + + if !params.Keyspace.Overlaps(metaRef.Bounds) || metaRef.EndTimestamp.Before(params.Interval.Start) || metaRef.StartTimestamp.After(params.Interval.End) { continue } + refs = append(refs, metaRef) } } From 009c53a75e9a196ec5cd25c15106ae8ba6eda812 Mon Sep 17 00:00:00 2001 From: Robert Jacob Date: Thu, 1 Feb 2024 21:12:41 +0100 Subject: [PATCH 22/45] operator: Add support for running with Azure Workload Identity (#11802) --- operator/CHANGELOG.md | 1 + .../handlers/internal/storage/secrets.go | 56 ++++++-- .../handlers/internal/storage/secrets_test.go | 62 ++++++++- .../internal/config/loki-config.yaml | 4 + .../internal/manifests/storage/configure.go | 16 ++- .../manifests/storage/configure_test.go | 124 ++++++++++++++++++ .../internal/manifests/storage/options.go | 7 +- operator/internal/manifests/storage/var.go | 23 +++- 8 files changed, 272 insertions(+), 21 deletions(-) diff --git a/operator/CHANGELOG.md b/operator/CHANGELOG.md index 18e28a016efee..ccded5758a3ab 100644 --- a/operator/CHANGELOG.md +++ b/operator/CHANGELOG.md @@ -1,5 +1,6 @@ ## Main +- [11802](https://github.com/grafana/loki/pull/11802) **xperimental**: Add support for running with Azure Workload Identity - [11824](https://github.com/grafana/loki/pull/11824) **xperimental**: Improve messages for errors in storage secret - [11524](https://github.com/grafana/loki/pull/11524) **JoaoBraveCoding**, **periklis**: Add OpenShift cloud credentials support for AWS STS - [11513](https://github.com/grafana/loki/pull/11513) **btaani**: Add a custom metric that collects Lokistacks requiring a schema upgrade diff --git a/operator/internal/handlers/internal/storage/secrets.go b/operator/internal/handlers/internal/storage/secrets.go index e41fa9c2c5b08..76ba037eb89bd 100644 --- a/operator/internal/handlers/internal/storage/secrets.go +++ b/operator/internal/handlers/internal/storage/secrets.go @@ -28,6 +28,9 @@ var ( errSecretHashError = errors.New("error calculating hash for secret") errS3NoAuth = errors.New("missing secret fields for static or sts authentication") + + 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") ) func getSecrets(ctx context.Context, k k8s.Client, stack *lokiv1.LokiStack, fg configv1.FeatureGates) (*corev1.Secret, *corev1.Secret, error) { @@ -165,25 +168,58 @@ func extractAzureConfigSecret(s *corev1.Secret) (*storage.AzureStorageConfig, er if len(container) == 0 { return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyAzureStorageContainerName) } - name := s.Data[storage.KeyAzureStorageAccountName] - if len(name) == 0 { - return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyAzureStorageAccountName) - } - key := s.Data[storage.KeyAzureStorageAccountKey] - if len(key) == 0 { - return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyAzureStorageAccountKey) + workloadIdentity, err := validateAzureCredentials(s) + if err != nil { + return nil, err } // Extract and validate optional fields endpointSuffix := s.Data[storage.KeyAzureStorageEndpointSuffix] return &storage.AzureStorageConfig{ - Env: string(env), - Container: string(container), - EndpointSuffix: string(endpointSuffix), + Env: string(env), + Container: string(container), + EndpointSuffix: string(endpointSuffix), + WorkloadIdentity: workloadIdentity, }, nil } +func validateAzureCredentials(s *corev1.Secret) (workloadIdentity bool, err error) { + accountName := s.Data[storage.KeyAzureStorageAccountName] + accountKey := s.Data[storage.KeyAzureStorageAccountKey] + clientID := s.Data[storage.KeyAzureStorageClientID] + tenantID := s.Data[storage.KeyAzureStorageTenantID] + subscriptionID := s.Data[storage.KeyAzureStorageSubscriptionID] + + if len(accountName) == 0 { + return false, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyAzureStorageAccountName) + } + + if len(accountKey) == 0 && len(clientID) == 0 { + return false, errAzureNoCredentials + } + + if len(accountKey) > 0 && len(clientID) > 0 { + return false, errAzureMixedCredentials + } + + if len(accountKey) > 0 { + // have both account_name and account_key -> no workload identity federation + return false, nil + } + + // assume workload-identity from here on + if len(tenantID) == 0 { + return false, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyAzureStorageTenantID) + } + + if len(subscriptionID) == 0 { + return false, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyAzureStorageSubscriptionID) + } + + return true, nil +} + func extractGCSConfigSecret(s *corev1.Secret) (*storage.GCSStorageConfig, error) { // Extract and validate mandatory fields bucket := s.Data[storage.KeyGCPStorageBucketName] diff --git a/operator/internal/handlers/internal/storage/secrets_test.go b/operator/internal/handlers/internal/storage/secrets_test.go index 70aebd18afc53..9d32a594e1874 100644 --- a/operator/internal/handlers/internal/storage/secrets_test.go +++ b/operator/internal/handlers/internal/storage/secrets_test.go @@ -101,7 +101,7 @@ func TestAzureExtract(t *testing.T) { wantError: "missing secret field: account_name", }, { - name: "missing account_key", + name: "no account_key or client_id", secret: &corev1.Secret{ ObjectMeta: metav1.ObjectMeta{Name: "test"}, Data: map[string][]byte{ @@ -110,10 +110,51 @@ func TestAzureExtract(t *testing.T) { "account_name": []byte("id"), }, }, - wantError: "missing secret field: account_key", + wantError: errAzureNoCredentials.Error(), }, { - name: "all mandatory set", + name: "both account_key and client_id set", + secret: &corev1.Secret{ + ObjectMeta: metav1.ObjectMeta{Name: "test"}, + Data: map[string][]byte{ + "environment": []byte("here"), + "container": []byte("this,that"), + "account_name": []byte("test-account-name"), + "account_key": []byte("test-account-key"), + "client_id": []byte("test-client-id"), + }, + }, + wantError: errAzureMixedCredentials.Error(), + }, + { + name: "missing tenant_id", + secret: &corev1.Secret{ + ObjectMeta: metav1.ObjectMeta{Name: "test"}, + Data: map[string][]byte{ + "environment": []byte("here"), + "container": []byte("this,that"), + "account_name": []byte("test-account-name"), + "client_id": []byte("test-client-id"), + }, + }, + wantError: "missing secret field: tenant_id", + }, + { + name: "missing subscription_id", + secret: &corev1.Secret{ + ObjectMeta: metav1.ObjectMeta{Name: "test"}, + Data: map[string][]byte{ + "environment": []byte("here"), + "container": []byte("this,that"), + "account_name": []byte("test-account-name"), + "client_id": []byte("test-client-id"), + "tenant_id": []byte("test-tenant-id"), + }, + }, + wantError: "missing secret field: subscription_id", + }, + { + name: "mandatory for normal authentication set", secret: &corev1.Secret{ ObjectMeta: metav1.ObjectMeta{Name: "test"}, Data: map[string][]byte{ @@ -124,6 +165,21 @@ func TestAzureExtract(t *testing.T) { }, }, }, + { + name: "mandatory for workload-identity set", + secret: &corev1.Secret{ + ObjectMeta: metav1.ObjectMeta{Name: "test"}, + Data: map[string][]byte{ + "environment": []byte("here"), + "container": []byte("this,that"), + "account_name": []byte("test-account-name"), + "client_id": []byte("test-client-id"), + "tenant_id": []byte("test-tenant-id"), + "subscription_id": []byte("test-subscription-id"), + "region": []byte("test-region"), + }, + }, + }, { name: "all set including optional", secret: &corev1.Secret{ diff --git a/operator/internal/manifests/internal/config/loki-config.yaml b/operator/internal/manifests/internal/config/loki-config.yaml index 61c0de401dc10..f908253a0c228 100644 --- a/operator/internal/manifests/internal/config/loki-config.yaml +++ b/operator/internal/manifests/internal/config/loki-config.yaml @@ -13,7 +13,11 @@ common: environment: {{ .Env }} container_name: {{ .Container }} account_name: ${AZURE_STORAGE_ACCOUNT_NAME} + {{- if .WorkloadIdentity }} + use_federated_token: true + {{- else }} account_key: ${AZURE_STORAGE_ACCOUNT_KEY} + {{- end }} {{- with .EndpointSuffix }} endpoint_suffix: {{ . }} {{- end }} diff --git a/operator/internal/manifests/storage/configure.go b/operator/internal/manifests/storage/configure.go index 6f7b22c4bd8ce..b4ff697b1fe4a 100644 --- a/operator/internal/manifests/storage/configure.go +++ b/operator/internal/manifests/storage/configure.go @@ -56,7 +56,6 @@ func ConfigureStatefulSet(d *appsv1.StatefulSet, opts Options) error { // With this, the deployment will expose credentials specific environment variables. func configureDeployment(d *appsv1.Deployment, opts Options) error { p := ensureObjectStoreCredentials(&d.Spec.Template.Spec, opts) - if err := mergo.Merge(&d.Spec.Template.Spec, p, mergo.WithOverride); err != nil { return kverrors.Wrap(err, "failed to merge gcs object storage spec ") } @@ -83,7 +82,6 @@ func configureDeploymentCA(d *appsv1.Deployment, tls *TLSConfig) error { // With this, the statefulset will expose credentials specific environment variable. func configureStatefulSet(s *appsv1.StatefulSet, opts Options) error { p := ensureObjectStoreCredentials(&s.Spec.Template.Spec, opts) - if err := mergo.Merge(&s.Spec.Template.Spec, p, mergo.WithOverride); err != nil { return kverrors.Wrap(err, "failed to merge gcs object storage spec ") } @@ -195,6 +193,14 @@ func managedAuthCredentials(opts Options) []corev1.EnvVar { envVarFromValue(EnvAWSWebIdentityTokenFile, path.Join(opts.S3.WebIdentityTokenFile, "token")), } } + case lokiv1.ObjectStorageSecretAzure: + return []corev1.EnvVar{ + envVarFromSecret(EnvAzureStorageAccountName, opts.SecretName, KeyAzureStorageAccountName), + envVarFromSecret(EnvAzureClientID, opts.SecretName, KeyAzureStorageClientID), + envVarFromSecret(EnvAzureTenantID, opts.SecretName, KeyAzureStorageTenantID), + envVarFromSecret(EnvAzureSubscriptionID, opts.SecretName, KeyAzureStorageSubscriptionID), + envVarFromValue(EnvAzureFederatedTokenFile, path.Join(azureTokenVolumeDirectory, "token")), + } default: return []corev1.EnvVar{} } @@ -273,6 +279,8 @@ func managedAuthEnabled(opts Options) bool { switch opts.SharedStore { case lokiv1.ObjectStorageSecretS3: return opts.S3 != nil && opts.S3.STS + case lokiv1.ObjectStorageSecretAzure: + return opts.Azure != nil && opts.Azure.WorkloadIdentity default: return false } @@ -293,6 +301,8 @@ func saTokenVolumeMount(opts Options) corev1.VolumeMount { switch opts.SharedStore { case lokiv1.ObjectStorageSecretS3: tokenPath = opts.S3.WebIdentityTokenFile + case lokiv1.ObjectStorageSecretAzure: + tokenPath = azureTokenVolumeDirectory } return corev1.VolumeMount{ Name: saTokenVolumeName, @@ -312,6 +322,8 @@ func saTokenVolume(opts Options) corev1.Volume { if opts.OpenShift.Enabled { audience = AWSOpenShiftAudience } + case lokiv1.ObjectStorageSecretAzure: + audience = azureDefaultAudience } return corev1.Volume{ Name: saTokenVolumeName, diff --git a/operator/internal/manifests/storage/configure_test.go b/operator/internal/manifests/storage/configure_test.go index 3b3029733554d..0b64a8eb8328e 100644 --- a/operator/internal/manifests/storage/configure_test.go +++ b/operator/internal/manifests/storage/configure_test.go @@ -168,6 +168,130 @@ func TestConfigureDeploymentForStorageType(t *testing.T) { }, }, }, + { + desc: "object storage Azure with WIF", + opts: Options{ + SecretName: "test", + SharedStore: lokiv1.ObjectStorageSecretAzure, + Azure: &AzureStorageConfig{ + 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/azure/serviceaccount", + }, + }, + Env: []corev1.EnvVar{ + { + Name: EnvAzureStorageAccountName, + ValueFrom: &corev1.EnvVarSource{ + SecretKeyRef: &corev1.SecretKeySelector{ + LocalObjectReference: corev1.LocalObjectReference{ + Name: "test", + }, + Key: KeyAzureStorageAccountName, + }, + }, + }, + { + Name: EnvAzureClientID, + ValueFrom: &corev1.EnvVarSource{ + SecretKeyRef: &corev1.SecretKeySelector{ + LocalObjectReference: corev1.LocalObjectReference{ + Name: "test", + }, + Key: KeyAzureStorageClientID, + }, + }, + }, + { + Name: EnvAzureTenantID, + ValueFrom: &corev1.EnvVarSource{ + SecretKeyRef: &corev1.SecretKeySelector{ + LocalObjectReference: corev1.LocalObjectReference{ + Name: "test", + }, + Key: KeyAzureStorageTenantID, + }, + }, + }, + { + Name: EnvAzureSubscriptionID, + ValueFrom: &corev1.EnvVarSource{ + SecretKeyRef: &corev1.SecretKeySelector{ + LocalObjectReference: corev1.LocalObjectReference{ + Name: "test", + }, + Key: KeyAzureStorageSubscriptionID, + }, + }, + }, + { + Name: EnvAzureFederatedTokenFile, + Value: "/var/run/secrets/azure/serviceaccount/token", + }, + }, + }, + }, + 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: azureDefaultAudience, + ExpirationSeconds: ptr.To[int64](3600), + Path: corev1.ServiceAccountTokenKey, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, { desc: "object storage GCS", opts: Options{ diff --git a/operator/internal/manifests/storage/options.go b/operator/internal/manifests/storage/options.go index 80efb24f62c8b..e1348297ab59c 100644 --- a/operator/internal/manifests/storage/options.go +++ b/operator/internal/manifests/storage/options.go @@ -25,9 +25,10 @@ type Options struct { // AzureStorageConfig for Azure storage config type AzureStorageConfig struct { - Env string - Container string - EndpointSuffix string + Env string + Container string + EndpointSuffix string + WorkloadIdentity bool } // GCSStorageConfig for GCS storage config diff --git a/operator/internal/manifests/storage/var.go b/operator/internal/manifests/storage/var.go index d77de3262d314..048af9e8a88e8 100644 --- a/operator/internal/manifests/storage/var.go +++ b/operator/internal/manifests/storage/var.go @@ -13,7 +13,7 @@ const ( EnvAWSSseKmsEncryptionContext = "AWS_SSE_KMS_ENCRYPTION_CONTEXT" // EnvAWSRoleArn is the environment variable to specify the AWS role ARN secret for the federated identity workflow. EnvAWSRoleArn = "AWS_ROLE_ARN" - // EnvAWSWebIdentityToken is the environment variable to specify the path to the web identity token file used in the federated identity workflow. + // EnvAWSWebIdentityTokenFile is the environment variable to specify the path to the web identity token file used in the federated identity workflow. EnvAWSWebIdentityTokenFile = "AWS_WEB_IDENTITY_TOKEN_FILE" // EnvAWSCredentialsFile is the environment variable to specify the path to the shared credentials file EnvAWSCredentialsFile = "AWS_SHARED_CREDENTIALS_FILE" @@ -23,6 +23,14 @@ const ( EnvAzureStorageAccountName = "AZURE_STORAGE_ACCOUNT_NAME" // EnvAzureStorageAccountKey is the environment variable to specify the Azure storage account key to access the container. EnvAzureStorageAccountKey = "AZURE_STORAGE_ACCOUNT_KEY" + // EnvAzureClientID is the environment variable used to pass the Managed Identity client-ID to the container. + EnvAzureClientID = "AZURE_CLIENT_ID" + // EnvAzureTenantID is the environment variable used to pass the Managed Identity tenant-ID to the container. + EnvAzureTenantID = "AZURE_TENANT_ID" + // EnvAzureSubscriptionID is the environment variable used to pass the Managed Identity subscription-ID to the container. + EnvAzureSubscriptionID = "AZURE_SUBSCRIPTION_ID" + // EnvAzureFederatedTokenFile is the environment variable used to store the path to the Managed Identity token. + EnvAzureFederatedTokenFile = "AZURE_FEDERATED_TOKEN_FILE" // EnvGoogleApplicationCredentials is the environment variable to specify path to key.json EnvGoogleApplicationCredentials = "GOOGLE_APPLICATION_CREDENTIALS" // EnvSwiftPassword is the environment variable to specify the OpenStack Swift password. @@ -66,6 +74,12 @@ const ( KeyAzureStorageAccountKey = "account_key" // KeyAzureStorageAccountName is the secret data key for the Azure storage account name. KeyAzureStorageAccountName = "account_name" + // KeyAzureStorageClientID contains the UUID of the Managed Identity accessing the storage. + KeyAzureStorageClientID = "client_id" + // KeyAzureStorageTenantID contains the UUID of the Tenant hosting the Managed Identity. + KeyAzureStorageTenantID = "tenant_id" + // KeyAzureStorageSubscriptionID contains the UUID of the subscription hosting the Managed Identity. + KeyAzureStorageSubscriptionID = "subscription_id" // KeyAzureStorageContainerName is the secret data key for the Azure storage container name. KeyAzureStorageContainerName = "container" // KeyAzureStorageEndpointSuffix is the secret data key for the Azure storage endpoint URL suffix. @@ -100,11 +114,11 @@ const ( KeySwiftRegion = "region" // KeySwiftUserDomainID is the secret data key for the OpenStack Swift user domain id. KeySwiftUserDomainID = "user_domain_id" - // KeySwiftUserDomainID is the secret data key for the OpenStack Swift user domain name. + // KeySwiftUserDomainName is the secret data key for the OpenStack Swift user domain name. KeySwiftUserDomainName = "user_domain_name" // KeySwiftUserID is the secret data key for the OpenStack Swift user id. KeySwiftUserID = "user_id" - // KeySwiftPassword is the secret data key for the OpenStack Swift password. + // KeySwiftUsername is the secret data key for the OpenStack Swift password. KeySwiftUsername = "username" saTokenVolumeK8sDirectory = "/var/run/secrets/kubernetes.io/serviceaccount" @@ -120,5 +134,8 @@ const ( awsDefaultAudience = "sts.amazonaws.com" AWSOpenShiftAudience = "openshift" + azureDefaultAudience = "api://AzureADTokenExchange" + azureTokenVolumeDirectory = "/var/run/secrets/azure/serviceaccount" + AnnotationCredentialsRequestsSecretRef = "loki.grafana.com/credentials-request-secret-ref" ) From d413c28fa33feb04a92f74ea8bddb311b6269f48 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 2 Feb 2024 00:52:52 -0800 Subject: [PATCH 23/45] Blooms: Align compactor to shipper (#11855) Does some alignment work between the `bloomcompactor` and the `bloomshipper` pkgs. Notably: * Uses `bloomshipper.BlockRef` everywhere (removes old bloomshipper struct * Integrates `v1.FingerprintBounds` in `Ref` struct * `Location` interface to distinguish local paths vs paths in object storage for certain types (`{Meta,Bloom}Ref`s) * Introduces `KeyResolver` interface to generate locations from these structs * Integrates `KeyResolver` into our bloom store. In the future, this will allow us to change key structures across schema boundaries when we want to change|improve them. * Removes `BlockPath` from `BlockRef` in favor of the new resolving functionality. This is also beneficial because it lets us _calculate_ locations from the pure Ref objects, rather than tie some arbitrary state to them which can change (or not be populated by accident). --- pkg/bloomcompactor/controller.go | 21 +-- pkg/bloomcompactor/controller_test.go | 49 +++--- pkg/bloomcompactor/meta.go | 30 +--- pkg/bloomgateway/util_test.go | 7 +- .../shipper/bloomshipper/block_downloader.go | 46 ++--- .../bloomshipper/block_downloader_test.go | 63 ++++--- .../stores/shipper/bloomshipper/client.go | 47 +++-- .../shipper/bloomshipper/client_test.go | 163 +++++------------- .../shipper/bloomshipper/compress_utils.go | 30 +--- .../stores/shipper/bloomshipper/resolver.go | 121 +++++++++++++ .../stores/shipper/bloomshipper/shipper.go | 37 ++-- .../shipper/bloomshipper/shipper_test.go | 53 +++--- .../stores/shipper/bloomshipper/store.go | 43 ++++- 13 files changed, 386 insertions(+), 324 deletions(-) create mode 100644 pkg/storage/stores/shipper/bloomshipper/resolver.go diff --git a/pkg/bloomcompactor/controller.go b/pkg/bloomcompactor/controller.go index 31f73740c1ff7..c9b3ecae35c2c 100644 --- a/pkg/bloomcompactor/controller.go +++ b/pkg/bloomcompactor/controller.go @@ -10,6 +10,7 @@ import ( "github.com/pkg/errors" v1 "github.com/grafana/loki/pkg/storage/bloom/v1" + "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper" "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb" ) @@ -178,7 +179,7 @@ func (s *SimpleBloomController) loadWorkForGap(id tsdb.Identifier, gap gapWithBl type gapWithBlocks struct { bounds v1.FingerprintBounds - blocks []BlockRef + blocks []bloomshipper.BlockRef } // blockPlan is a plan for all the work needed to build a meta.json @@ -220,7 +221,7 @@ func blockPlansForGaps(tsdbs []tsdbGaps, metas []Meta) ([]blockPlan, error) { } for _, block := range meta.Blocks { - if block.OwnershipRange.Intersection(gap) == nil { + if block.Bounds.Intersection(gap) == nil { // this block doesn't overlap the gap, skip continue } @@ -232,27 +233,27 @@ func blockPlansForGaps(tsdbs []tsdbGaps, metas []Meta) ([]blockPlan, error) { // ensure we sort blocks so deduping iterator works as expected sort.Slice(planGap.blocks, func(i, j int) bool { - return planGap.blocks[i].OwnershipRange.Less(planGap.blocks[j].OwnershipRange) + return planGap.blocks[i].Bounds.Less(planGap.blocks[j].Bounds) }) - peekingBlocks := v1.NewPeekingIter[BlockRef]( - v1.NewSliceIter[BlockRef]( + peekingBlocks := v1.NewPeekingIter[bloomshipper.BlockRef]( + v1.NewSliceIter[bloomshipper.BlockRef]( planGap.blocks, ), ) // dedupe blocks which could be in multiple metas - itr := v1.NewDedupingIter[BlockRef, BlockRef]( - func(a, b BlockRef) bool { + itr := v1.NewDedupingIter[bloomshipper.BlockRef, bloomshipper.BlockRef]( + func(a, b bloomshipper.BlockRef) bool { return a == b }, - v1.Identity[BlockRef], - func(a, _ BlockRef) BlockRef { + v1.Identity[bloomshipper.BlockRef], + func(a, _ bloomshipper.BlockRef) bloomshipper.BlockRef { return a }, peekingBlocks, ) - deduped, err := v1.Collect[BlockRef](itr) + deduped, err := v1.Collect[bloomshipper.BlockRef](itr) if err != nil { return nil, errors.Wrap(err, "failed to dedupe blocks") } diff --git a/pkg/bloomcompactor/controller_test.go b/pkg/bloomcompactor/controller_test.go index 9f3f56153af32..1f89a0e318efd 100644 --- a/pkg/bloomcompactor/controller_test.go +++ b/pkg/bloomcompactor/controller_test.go @@ -8,6 +8,7 @@ import ( "github.com/stretchr/testify/require" v1 "github.com/grafana/loki/pkg/storage/bloom/v1" + "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper" "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb" ) @@ -119,7 +120,7 @@ func tsdbID(n int) tsdb.SingleTenantTSDBIdentifier { } } -func genMeta(min, max model.Fingerprint, sources []int, blocks []BlockRef) Meta { +func genMeta(min, max model.Fingerprint, sources []int, blocks []bloomshipper.BlockRef) Meta { m := Meta{ OwnershipRange: v1.NewBounds(min, max), Blocks: blocks, @@ -224,10 +225,12 @@ func Test_gapsBetweenTSDBsAndMetas(t *testing.T) { } } -func genBlockRef(min, max model.Fingerprint) BlockRef { +func genBlockRef(min, max model.Fingerprint) bloomshipper.BlockRef { bounds := v1.NewBounds(min, max) - return BlockRef{ - OwnershipRange: bounds, + return bloomshipper.BlockRef{ + Ref: bloomshipper.Ref{ + Bounds: bounds, + }, } } @@ -245,7 +248,7 @@ func Test_blockPlansForGaps(t *testing.T) { ownershipRange: v1.NewBounds(0, 10), tsdbs: []tsdb.Identifier{tsdbID(0)}, metas: []Meta{ - genMeta(5, 20, []int{1}, []BlockRef{genBlockRef(11, 20)}), + genMeta(5, 20, []int{1}, []bloomshipper.BlockRef{genBlockRef(11, 20)}), }, exp: []blockPlan{ { @@ -263,7 +266,7 @@ func Test_blockPlansForGaps(t *testing.T) { ownershipRange: v1.NewBounds(0, 10), tsdbs: []tsdb.Identifier{tsdbID(0)}, metas: []Meta{ - genMeta(5, 20, []int{1}, []BlockRef{genBlockRef(9, 20)}), + genMeta(5, 20, []int{1}, []bloomshipper.BlockRef{genBlockRef(9, 20)}), }, exp: []blockPlan{ { @@ -271,7 +274,7 @@ func Test_blockPlansForGaps(t *testing.T) { gaps: []gapWithBlocks{ { bounds: v1.NewBounds(0, 10), - blocks: []BlockRef{genBlockRef(9, 20)}, + blocks: []bloomshipper.BlockRef{genBlockRef(9, 20)}, }, }, }, @@ -285,8 +288,8 @@ func Test_blockPlansForGaps(t *testing.T) { ownershipRange: v1.NewBounds(0, 10), tsdbs: []tsdb.Identifier{tsdbID(0)}, metas: []Meta{ - genMeta(9, 20, []int{0}, []BlockRef{genBlockRef(9, 20)}), // block for same tsdb - genMeta(9, 20, []int{1}, []BlockRef{genBlockRef(9, 20)}), // block for different tsdb + genMeta(9, 20, []int{0}, []bloomshipper.BlockRef{genBlockRef(9, 20)}), // block for same tsdb + genMeta(9, 20, []int{1}, []bloomshipper.BlockRef{genBlockRef(9, 20)}), // block for different tsdb }, exp: []blockPlan{ { @@ -304,8 +307,8 @@ func Test_blockPlansForGaps(t *testing.T) { ownershipRange: v1.NewBounds(0, 10), tsdbs: []tsdb.Identifier{tsdbID(0)}, metas: []Meta{ - genMeta(9, 20, []int{0}, []BlockRef{genBlockRef(9, 20)}), // block for same tsdb - genMeta(5, 20, []int{1}, []BlockRef{genBlockRef(5, 20)}), // block for different tsdb + genMeta(9, 20, []int{0}, []bloomshipper.BlockRef{genBlockRef(9, 20)}), // block for same tsdb + genMeta(5, 20, []int{1}, []bloomshipper.BlockRef{genBlockRef(5, 20)}), // block for different tsdb }, exp: []blockPlan{ { @@ -313,7 +316,7 @@ func Test_blockPlansForGaps(t *testing.T) { gaps: []gapWithBlocks{ { bounds: v1.NewBounds(0, 8), - blocks: []BlockRef{genBlockRef(5, 20)}, + blocks: []bloomshipper.BlockRef{genBlockRef(5, 20)}, }, }, }, @@ -324,14 +327,14 @@ func Test_blockPlansForGaps(t *testing.T) { ownershipRange: v1.NewBounds(0, 10), tsdbs: []tsdb.Identifier{tsdbID(0), tsdbID(1)}, // generate for both tsdbs metas: []Meta{ - genMeta(0, 2, []int{0}, []BlockRef{ + genMeta(0, 2, []int{0}, []bloomshipper.BlockRef{ genBlockRef(0, 1), genBlockRef(1, 2), }), // tsdb_0 - genMeta(6, 8, []int{0}, []BlockRef{genBlockRef(6, 8)}), // tsdb_0 + genMeta(6, 8, []int{0}, []bloomshipper.BlockRef{genBlockRef(6, 8)}), // tsdb_0 - genMeta(3, 5, []int{1}, []BlockRef{genBlockRef(3, 5)}), // tsdb_1 - genMeta(8, 10, []int{1}, []BlockRef{genBlockRef(8, 10)}), // tsdb_1 + genMeta(3, 5, []int{1}, []bloomshipper.BlockRef{genBlockRef(3, 5)}), // tsdb_1 + genMeta(8, 10, []int{1}, []bloomshipper.BlockRef{genBlockRef(8, 10)}), // tsdb_1 }, exp: []blockPlan{ { @@ -340,11 +343,11 @@ func Test_blockPlansForGaps(t *testing.T) { // tsdb (id=0) can source chunks from the blocks built from tsdb (id=1) { bounds: v1.NewBounds(3, 5), - blocks: []BlockRef{genBlockRef(3, 5)}, + blocks: []bloomshipper.BlockRef{genBlockRef(3, 5)}, }, { bounds: v1.NewBounds(9, 10), - blocks: []BlockRef{genBlockRef(8, 10)}, + blocks: []bloomshipper.BlockRef{genBlockRef(8, 10)}, }, }, }, @@ -354,14 +357,14 @@ func Test_blockPlansForGaps(t *testing.T) { gaps: []gapWithBlocks{ { bounds: v1.NewBounds(0, 2), - blocks: []BlockRef{ + blocks: []bloomshipper.BlockRef{ genBlockRef(0, 1), genBlockRef(1, 2), }, }, { bounds: v1.NewBounds(6, 7), - blocks: []BlockRef{genBlockRef(6, 8)}, + blocks: []bloomshipper.BlockRef{genBlockRef(6, 8)}, }, }, }, @@ -372,11 +375,11 @@ func Test_blockPlansForGaps(t *testing.T) { ownershipRange: v1.NewBounds(0, 10), tsdbs: []tsdb.Identifier{tsdbID(0)}, metas: []Meta{ - genMeta(9, 20, []int{1}, []BlockRef{ + genMeta(9, 20, []int{1}, []bloomshipper.BlockRef{ genBlockRef(1, 4), genBlockRef(9, 20), }), // blocks for first diff tsdb - genMeta(5, 20, []int{2}, []BlockRef{ + genMeta(5, 20, []int{2}, []bloomshipper.BlockRef{ genBlockRef(5, 10), genBlockRef(9, 20), // same block references in prior meta (will be deduped) }), // block for second diff tsdb @@ -387,7 +390,7 @@ func Test_blockPlansForGaps(t *testing.T) { gaps: []gapWithBlocks{ { bounds: v1.NewBounds(0, 10), - blocks: []BlockRef{ + blocks: []bloomshipper.BlockRef{ genBlockRef(1, 4), genBlockRef(5, 10), genBlockRef(9, 20), diff --git a/pkg/bloomcompactor/meta.go b/pkg/bloomcompactor/meta.go index adffb61dff5ed..c0a333c5c907e 100644 --- a/pkg/bloomcompactor/meta.go +++ b/pkg/bloomcompactor/meta.go @@ -2,14 +2,13 @@ package bloomcompactor import ( "fmt" - "hash" "path" "github.com/pkg/errors" v1 "github.com/grafana/loki/pkg/storage/bloom/v1" + "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper" "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb" - "github.com/grafana/loki/pkg/util/encoding" ) const ( @@ -17,27 +16,6 @@ const ( MetasPrefix = "metas" ) -// TODO(owen-d): Probably want to integrate against the block shipper -// instead of defining here, but only (min,max,fp) should be required for -// the ref. Things like index-paths, etc are not needed and possibly harmful -// in the case we want to do migrations. It's easier to load a block-ref or similar -// within the context of a specific tenant+period+index path and not couple them. -type BlockRef struct { - OwnershipRange v1.FingerprintBounds - Checksum uint32 -} - -func (r BlockRef) Hash(h hash.Hash32) error { - if err := r.OwnershipRange.Hash(h); err != nil { - return err - } - - var enc encoding.Encbuf - enc.PutBE32(r.Checksum) - _, err := h.Write(enc.Get()) - return errors.Wrap(err, "writing BlockRef") -} - type MetaRef struct { OwnershipRange v1.FingerprintBounds Checksum uint32 @@ -63,13 +41,13 @@ type Meta struct { OwnershipRange v1.FingerprintBounds // Old blocks which can be deleted in the future. These should be from previous compaction rounds. - Tombstones []BlockRef + Tombstones []bloomshipper.BlockRef // The specific TSDB files used to generate the block. Sources []tsdb.SingleTenantTSDBIdentifier // A list of blocks that were generated - Blocks []BlockRef + Blocks []bloomshipper.BlockRef } // Generate MetaRef from Meta @@ -131,6 +109,6 @@ type MetaStore interface { type BlockStore interface { // TODO(owen-d): flesh out|integrate against bloomshipper.Client - GetBlocks([]BlockRef) ([]*v1.Block, error) + GetBlocks([]bloomshipper.BlockRef) ([]*v1.Block, error) PutBlock(interface{}) error } diff --git a/pkg/bloomgateway/util_test.go b/pkg/bloomgateway/util_test.go index a705d1965780f..d60ab5f13a190 100644 --- a/pkg/bloomgateway/util_test.go +++ b/pkg/bloomgateway/util_test.go @@ -2,7 +2,6 @@ package bloomgateway import ( "context" - "fmt" "math/rand" "testing" "time" @@ -343,9 +342,7 @@ func createBlocks(t *testing.T, tenant string, n int, from, through model.Time, EndTimestamp: through, } block := bloomshipper.BlockRef{ - Ref: ref, - IndexPath: "index.tsdb.gz", - BlockPath: fmt.Sprintf("block-%d", i), + Ref: ref, } meta := bloomshipper.Meta{ MetaRef: bloomshipper.MetaRef{ @@ -459,8 +456,6 @@ func createBlockRefsFromBlockData(t *testing.T, tenant string, data []bloomshipp EndTimestamp: 0, Checksum: 0, }, - IndexPath: fmt.Sprintf("index-%d", i), - BlockPath: fmt.Sprintf("block-%d", i), }) } return res diff --git a/pkg/storage/stores/shipper/bloomshipper/block_downloader.go b/pkg/storage/stores/shipper/bloomshipper/block_downloader.go index 81355f78e84ec..8d28ca03dcc63 100644 --- a/pkg/storage/stores/shipper/bloomshipper/block_downloader.go +++ b/pkg/storage/stores/shipper/bloomshipper/block_downloader.go @@ -4,10 +4,10 @@ import ( "context" "errors" "fmt" + "io" "os" "path" "path/filepath" - "strconv" "sync" "time" @@ -175,12 +175,12 @@ type cacheDownloadingStrategy struct { } func (s *cacheDownloadingStrategy) downloadBlock(task *BlockDownloadingTask, logger log.Logger) (blockWithQuerier, error) { - blockPath := task.block.BlockPath - s.keyMutex.LockKey(blockPath) + key := s.blockClient.Block(task.block).Addr() + s.keyMutex.LockKey(key) defer func() { - _ = s.keyMutex.UnlockKey(blockPath) + _ = s.keyMutex.UnlockKey(key) }() - blockFromCache, exists := s.blocksCache.Get(task.ctx, task.block.BlockPath) + blockFromCache, exists := s.blocksCache.Get(task.ctx, key) if exists { return blockWithQuerier{ BlockRef: task.block, @@ -193,10 +193,10 @@ func (s *cacheDownloadingStrategy) downloadBlock(task *BlockDownloadingTask, log return blockWithQuerier{}, err } blockFromCache = newCachedBlock(directory, s.config.BlocksCache.RemoveDirectoryGracefulPeriod, logger) - err = s.blocksCache.Store(task.ctx, []string{task.block.BlockPath}, []*cachedBlock{blockFromCache}) + err = s.blocksCache.Store(task.ctx, []string{key}, []*cachedBlock{blockFromCache}) if err != nil { - level.Error(logger).Log("msg", "error storing the block in the cache", "block", blockPath, "err", err) - return blockWithQuerier{}, fmt.Errorf("error storing the block %s in the cache : %w", blockPath, err) + level.Error(logger).Log("msg", "error storing the block in the cache", "block", key, "err", err) + return blockWithQuerier{}, fmt.Errorf("error storing the block %s in the cache : %w", key, err) } return blockWithQuerier{ BlockRef: task.block, @@ -229,20 +229,20 @@ func (s *storageDownloadingStrategy) close() { } func downloadBlockToDirectory(logger log.Logger, task *BlockDownloadingTask, workingDirectory string, blockClient BlockClient) (string, error) { - blockPath := task.block.BlockPath + blockPath := filepath.Join(workingDirectory, blockClient.Block(task.block).LocalPath()) level.Debug(logger).Log("msg", "start downloading the block", "block", blockPath) block, err := blockClient.GetBlock(task.ctx, task.block) if err != nil { level.Error(logger).Log("msg", "error downloading the block", "block", blockPath, "err", err) return "", fmt.Errorf("error downloading the block %s : %w", blockPath, err) } - directory, err := extractBlock(&block, time.Now(), workingDirectory, logger) + err = extractBlock(block.Data, blockPath, logger) if err != nil { level.Error(logger).Log("msg", "error extracting the block", "block", blockPath, "err", err) return "", fmt.Errorf("error extracting the block %s : %w", blockPath, err) } - level.Debug(logger).Log("msg", "block has been downloaded and extracted", "block", task.block.BlockPath, "directory", directory) - return directory, nil + level.Debug(logger).Log("msg", "block has been downloaded and extracted", "block", blockPath) + return blockPath, nil } func (d *blockDownloader) downloadBlocks(ctx context.Context, tenantID string, references []BlockRef) (chan blockWithQuerier, chan error) { @@ -256,10 +256,10 @@ func (d *blockDownloader) downloadBlocks(ctx context.Context, tenantID string, r for _, reference := range references { task := NewBlockDownloadingTask(ctx, reference, blocksCh, errCh) - level.Debug(d.logger).Log("msg", "enqueuing task to download block", "block", reference.BlockPath) + level.Debug(d.logger).Log("msg", "enqueuing task to download block", "block", reference) err := d.queue.Enqueue(tenantID, nil, task, nil) if err != nil { - errCh <- fmt.Errorf("error enquing downloading task for block %s : %w", reference.BlockPath, err) + errCh <- fmt.Errorf("error enquing downloading task for block %s : %w", reference, err) return blocksCh, errCh } } @@ -272,15 +272,15 @@ type blockWithQuerier struct { } // extract the files into directory and returns absolute path to this directory. -func extractBlock(block *LazyBlock, ts time.Time, workingDirectory string, logger log.Logger) (string, error) { - workingDirectoryPath := filepath.Join(workingDirectory, block.BlockPath, strconv.FormatInt(ts.UnixNano(), 10)) - err := os.MkdirAll(workingDirectoryPath, os.ModePerm) +func extractBlock(data io.ReadCloser, blockDir string, logger log.Logger) error { + + err := os.MkdirAll(blockDir, os.ModePerm) if err != nil { - return "", fmt.Errorf("can not create directory to extract the block: %w", err) + return fmt.Errorf("can not create directory to extract the block: %w", err) } - archivePath, err := writeDataToTempFile(workingDirectoryPath, block) + archivePath, err := writeDataToTempFile(blockDir, data) if err != nil { - return "", fmt.Errorf("error writing data to temp file: %w", err) + return fmt.Errorf("error writing data to temp file: %w", err) } defer func() { err = os.Remove(archivePath) @@ -288,11 +288,11 @@ func extractBlock(block *LazyBlock, ts time.Time, workingDirectory string, logge level.Error(logger).Log("msg", "error removing temp archive file", "err", err) } }() - err = extractArchive(archivePath, workingDirectoryPath) + err = extractArchive(archivePath, blockDir) if err != nil { - return "", fmt.Errorf("error extracting archive: %w", err) + return fmt.Errorf("error extracting archive: %w", err) } - return workingDirectoryPath, nil + return nil } func (d *blockDownloader) stop() { diff --git a/pkg/storage/stores/shipper/bloomshipper/block_downloader_test.go b/pkg/storage/stores/shipper/bloomshipper/block_downloader_test.go index ffe715c857ec6..8fc1b3e976e0f 100644 --- a/pkg/storage/stores/shipper/bloomshipper/block_downloader_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/block_downloader_test.go @@ -7,7 +7,6 @@ import ( "io" "os" "path/filepath" - "strconv" "sync" "testing" "time" @@ -15,6 +14,7 @@ import ( "github.com/go-kit/log" "github.com/google/uuid" "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/common/model" "github.com/stretchr/testify/require" "go.uber.org/atomic" @@ -42,12 +42,12 @@ func Test_blockDownloader_downloadBlocks(t *testing.T) { }, blockClient, overrides, log.NewNopLogger(), prometheus.DefaultRegisterer) require.NoError(t, err) blocksCh, errorsCh := downloader.downloadBlocks(context.Background(), "fake", blockReferences) - downloadedBlocks := make(map[string]any, len(blockReferences)) + downloadedBlocks := make(map[BlockRef]any, len(blockReferences)) done := make(chan bool) go func() { for i := 0; i < 20; i++ { block := <-blocksCh - downloadedBlocks[block.BlockPath] = nil + downloadedBlocks[block.BlockRef] = nil } done <- true }() @@ -111,12 +111,12 @@ func Test_blockDownloader_downloadBlock(t *testing.T) { require.NoError(t, err) blocksCh, errorsCh := downloader.downloadBlocks(context.Background(), "fake", blockReferences) - downloadedBlocks := make(map[string]any, len(blockReferences)) + downloadedBlocks := make(map[BlockRef]any, len(blockReferences)) done := make(chan bool) go func() { for i := 0; i < 20; i++ { block := <-blocksCh - downloadedBlocks[block.BlockPath] = nil + downloadedBlocks[block.BlockRef] = nil } done <- true }() @@ -132,12 +132,12 @@ func Test_blockDownloader_downloadBlock(t *testing.T) { require.Equal(t, int32(20), blockClient.getBlockCalls.Load()) blocksCh, errorsCh = downloader.downloadBlocks(context.Background(), "fake", blockReferences) - downloadedBlocks = make(map[string]any, len(blockReferences)) + downloadedBlocks = make(map[BlockRef]any, len(blockReferences)) done = make(chan bool) go func() { for i := 0; i < 20; i++ { block := <-blocksCh - downloadedBlocks[block.BlockPath] = nil + downloadedBlocks[block.BlockRef] = nil } done <- true }() @@ -313,7 +313,7 @@ func Test_closableBlockQuerier(t *testing.T) { // creates fake blocks and returns map[block-path]Block and mockBlockClient func createFakeBlocks(t *testing.T, count int) ([]BlockRef, *mockBlockClient) { - mockData := make(map[string]blockSupplier, count) + mockData := make(map[BlockRef]blockSupplier, count) refs := make([]BlockRef, 0, count) for i := 0; i < count; i++ { archivePath, _, _ := createBlockArchive(t) @@ -321,9 +321,16 @@ func createFakeBlocks(t *testing.T, count int) ([]BlockRef, *mockBlockClient) { //ensure file can be opened require.NoError(t, err) blockRef := BlockRef{ - BlockPath: fmt.Sprintf("block-path-%d", i), + Ref: Ref{ + TenantID: "", + TableName: "", + Bounds: v1.NewBounds(model.Fingerprint(i), model.Fingerprint(i+1)), + StartTimestamp: 0, + EndTimestamp: 0, + Checksum: 0, + }, } - mockData[blockRef.BlockPath] = func() LazyBlock { + mockData[blockRef] = func() LazyBlock { file, _ := os.OpenFile(archivePath, os.O_RDONLY, 0700) return LazyBlock{ BlockRef: blockRef, @@ -339,19 +346,20 @@ type blockSupplier func() LazyBlock type mockBlockClient struct { responseDelay time.Duration - mockData map[string]blockSupplier + mockData map[BlockRef]blockSupplier getBlockCalls atomic.Int32 + defaultKeyResolver } func (m *mockBlockClient) GetBlock(_ context.Context, reference BlockRef) (LazyBlock, error) { m.getBlockCalls.Inc() time.Sleep(m.responseDelay) - supplier, exists := m.mockData[reference.BlockPath] + supplier, exists := m.mockData[reference] if exists { return supplier(), nil } - return LazyBlock{}, fmt.Errorf("block %s is not found in mockData", reference.BlockPath) + return LazyBlock{}, fmt.Errorf("block %s is not found in mockData", reference) } func (m *mockBlockClient) PutBlocks(_ context.Context, _ []Block) ([]Block, error) { @@ -368,26 +376,31 @@ func Test_blockDownloader_extractBlock(t *testing.T) { require.NoError(t, err) workingDir := t.TempDir() - ts := time.Now().UTC() block := LazyBlock{ - BlockRef: BlockRef{BlockPath: "first-period-19621/tenantA/metas/ff-fff-1695272400-1695276000-aaa"}, - Data: blockFile, + BlockRef: BlockRef{ + Ref: Ref{ + TenantID: "", + TableName: "", + Bounds: v1.NewBounds(0, 1), + StartTimestamp: 0, + EndTimestamp: 0, + Checksum: 0, + }, + }, + Data: blockFile, } - actualPath, err := extractBlock(&block, ts, workingDir, nil) - + err = extractBlock(block.Data, workingDir, nil) require.NoError(t, err) - expectedPath := filepath.Join(workingDir, block.BlockPath, strconv.FormatInt(ts.UnixNano(), 10)) - require.Equal(t, expectedPath, actualPath, - "expected archive to be extracted to working directory under the same path as blockPath and with timestamp suffix") - require.FileExists(t, filepath.Join(expectedPath, v1.BloomFileName)) - require.FileExists(t, filepath.Join(expectedPath, v1.SeriesFileName)) - actualBloomFileContent, err := os.ReadFile(filepath.Join(expectedPath, v1.BloomFileName)) + require.FileExists(t, filepath.Join(workingDir, v1.BloomFileName)) + require.FileExists(t, filepath.Join(workingDir, v1.SeriesFileName)) + + actualBloomFileContent, err := os.ReadFile(filepath.Join(workingDir, v1.BloomFileName)) require.NoError(t, err) require.Equal(t, bloomFileContent, string(actualBloomFileContent)) - actualSeriesFileContent, err := os.ReadFile(filepath.Join(expectedPath, v1.SeriesFileName)) + actualSeriesFileContent, err := os.ReadFile(filepath.Join(workingDir, v1.SeriesFileName)) require.NoError(t, err) require.Equal(t, seriesFileContent, string(actualSeriesFileContent)) } diff --git a/pkg/storage/stores/shipper/bloomshipper/client.go b/pkg/storage/stores/shipper/bloomshipper/client.go index 6944d51da6412..cfdd057db047d 100644 --- a/pkg/storage/stores/shipper/bloomshipper/client.go +++ b/pkg/storage/stores/shipper/bloomshipper/client.go @@ -5,6 +5,7 @@ import ( "context" "encoding/json" "fmt" + "hash" "io" "path" "strconv" @@ -12,11 +13,13 @@ import ( "github.com/go-kit/log" "github.com/grafana/dskit/concurrency" + "github.com/pkg/errors" "github.com/prometheus/common/model" v1 "github.com/grafana/loki/pkg/storage/bloom/v1" "github.com/grafana/loki/pkg/storage/chunk/client" "github.com/grafana/loki/pkg/storage/config" + "github.com/grafana/loki/pkg/util/encoding" ) const ( @@ -35,6 +38,26 @@ type Ref struct { Checksum uint32 } +// Hash hashes the ref +// NB(owen-d): we don't include the tenant in the hash +// as it's not included in the data and leaving it out gives +// flexibility for migrating data between tenants +func (r Ref) Hash(h hash.Hash32) error { + if err := r.Bounds.Hash(h); err != nil { + return err + } + + var enc encoding.Encbuf + + enc.PutString(r.TableName) + enc.PutBE64(uint64(r.StartTimestamp)) + enc.PutBE64(uint64(r.EndTimestamp)) + enc.PutBE32(r.Checksum) + + _, err := h.Write(enc.Get()) + return errors.Wrap(err, "writing BlockRef") +} + // Cmp returns the fingerprint's position relative to the bounds func (r Ref) Cmp(fp uint64) v1.BoundsCheck { return r.Bounds.Cmp(model.Fingerprint(fp)) @@ -46,8 +69,10 @@ func (r Ref) Interval() Interval { type BlockRef struct { Ref - IndexPath string - BlockPath string +} + +func (r BlockRef) String() string { + return defaultKeyResolver{}.Block(r).Addr() } type MetaRef struct { @@ -88,6 +113,7 @@ type Block struct { } type BlockClient interface { + KeyResolver GetBlock(ctx context.Context, ref BlockRef) (LazyBlock, error) PutBlocks(ctx context.Context, blocks []Block) ([]Block, error) DeleteBlocks(ctx context.Context, blocks []BlockRef) error @@ -103,6 +129,7 @@ type Client interface { var _ Client = &BloomClient{} type BloomClient struct { + KeyResolver concurrency int client client.ObjectClient logger log.Logger @@ -110,7 +137,8 @@ type BloomClient struct { func NewBloomClient(client client.ObjectClient, logger log.Logger) (*BloomClient, error) { return &BloomClient{ - concurrency: 100, // make configurable? + KeyResolver: defaultKeyResolver{}, // TODO(owen-d): hook into schema, similar to `{,Parse}ExternalKey` + concurrency: 100, // make configurable? client: client, logger: logger, }, nil @@ -125,12 +153,6 @@ func (b *BloomClient) PutMeta(ctx context.Context, meta Meta) error { return b.client.PutObject(ctx, key, bytes.NewReader(data)) } -func externalBlockKey(ref BlockRef) string { - blockParentFolder := ref.Bounds.String() - filename := fmt.Sprintf("%d-%d-%x", ref.StartTimestamp, ref.EndTimestamp, ref.Checksum) - return path.Join(rootFolder, ref.TableName, ref.TenantID, bloomsFolder, blockParentFolder, filename) -} - func externalMetaKey(ref MetaRef) string { filename := fmt.Sprintf("%s-%d-%d-%x", ref.Bounds.String(), ref.StartTimestamp, ref.EndTimestamp, ref.Checksum) return path.Join(rootFolder, ref.TableName, ref.TenantID, metasFolder, filename) @@ -153,7 +175,7 @@ func (b *BloomClient) DeleteMeta(ctx context.Context, meta Meta) error { // GetBlock downloads the blocks from objectStorage and returns the downloaded block func (b *BloomClient) GetBlock(ctx context.Context, reference BlockRef) (LazyBlock, error) { - readCloser, _, err := b.client.GetObject(ctx, externalBlockKey(reference)) + readCloser, _, err := b.client.GetObject(ctx, b.Block(reference).Addr()) if err != nil { return LazyBlock{}, fmt.Errorf("error while fetching object from storage: %w", err) } @@ -173,7 +195,7 @@ func (b *BloomClient) PutBlocks(ctx context.Context, blocks []Block) ([]Block, e var err error - key := externalBlockKey(block.BlockRef) + key := b.Block(block.BlockRef).Addr() _, err = block.Data.Seek(0, 0) if err != nil { return fmt.Errorf("error uploading block file: %w", err) @@ -183,7 +205,6 @@ func (b *BloomClient) PutBlocks(ctx context.Context, blocks []Block) ([]Block, e if err != nil { return fmt.Errorf("error uploading block file: %w", err) } - block.BlockPath = key results[idx] = block return nil }) @@ -193,7 +214,7 @@ func (b *BloomClient) PutBlocks(ctx context.Context, blocks []Block) ([]Block, e func (b *BloomClient) DeleteBlocks(ctx context.Context, references []BlockRef) error { return concurrency.ForEachJob(ctx, len(references), b.concurrency, func(ctx context.Context, idx int) error { ref := references[idx] - key := externalBlockKey(ref) + key := b.Block(ref).Addr() err := b.client.DeleteObject(ctx, key) if err != nil { return fmt.Errorf("error deleting block file: %w", err) diff --git a/pkg/storage/stores/shipper/bloomshipper/client_test.go b/pkg/storage/stores/shipper/bloomshipper/client_test.go index 3d6eac07521de..80770a5735679 100644 --- a/pkg/storage/stores/shipper/bloomshipper/client_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/client_test.go @@ -205,17 +205,6 @@ func Test_BloomClient_DeleteMeta(t *testing.T) { } func Test_BloomClient_GetBlocks(t *testing.T) { - bloomClient := createStore(t) - fsNamedStores := bloomClient.storageConfig.NamedStores.Filesystem - firstBlockPath := fmt.Sprintf("bloom/first-period-19621/tenantA/blooms/%s/1695272400000-1695276000000-1", v1.NewBounds(0xeeee, 0xffff)) - firstBlockFullPath := filepath.Join(fsNamedStores["folder-1"].Directory, firstBlockPath) - firstBlockData := createBlockFile(t, firstBlockFullPath) - secondBlockPath := fmt.Sprintf("bloom/second-period-19624/tenantA/blooms/%s/1695531600000-1695535200000-2", v1.NewBounds(0xaaaa, 0xbbbb)) - secondBlockFullPath := filepath.Join(fsNamedStores["folder-2"].Directory, secondBlockPath) - secondBlockData := createBlockFile(t, secondBlockFullPath) - require.FileExists(t, firstBlockFullPath) - require.FileExists(t, secondBlockFullPath) - firstBlockRef := BlockRef{ Ref: Ref{ TenantID: "tenantA", @@ -225,7 +214,6 @@ func Test_BloomClient_GetBlocks(t *testing.T) { EndTimestamp: Date(2023, time.September, 21, 6, 0, 0), Checksum: 1, }, - BlockPath: firstBlockPath, } secondBlockRef := BlockRef{ Ref: Ref{ @@ -236,9 +224,23 @@ func Test_BloomClient_GetBlocks(t *testing.T) { EndTimestamp: Date(2023, time.September, 24, 6, 0, 0), Checksum: 2, }, - BlockPath: secondBlockPath, } + bloomClient := createStore(t) + fsNamedStores := bloomClient.storageConfig.NamedStores.Filesystem + firstBlockFullPath := NewPrefixedResolver( + fsNamedStores["folder-1"].Directory, + defaultKeyResolver{}, + ).Block(firstBlockRef).LocalPath() + firstBlockData := createBlockFile(t, firstBlockFullPath) + secondBlockFullPath := NewPrefixedResolver( + fsNamedStores["folder-2"].Directory, + defaultKeyResolver{}, + ).Block(secondBlockRef).LocalPath() + secondBlockData := createBlockFile(t, secondBlockFullPath) + require.FileExists(t, firstBlockFullPath) + require.FileExists(t, secondBlockFullPath) + downloadedFirstBlock, err := bloomClient.GetBlock(context.Background(), firstBlockRef) require.NoError(t, err) firstBlockActualData, err := io.ReadAll(downloadedFirstBlock.Data) @@ -254,8 +256,7 @@ func Test_BloomClient_GetBlocks(t *testing.T) { func Test_BloomClient_PutBlocks(t *testing.T) { bloomClient := createStore(t) - blockForFirstFolderData := "data1" - blockForFirstFolder := Block{ + block := Block{ BlockRef: BlockRef{ Ref: Ref{ TenantID: "tenantA", @@ -265,118 +266,44 @@ func Test_BloomClient_PutBlocks(t *testing.T) { EndTimestamp: Date(2023, time.September, 21, 6, 0, 0), Checksum: 1, }, - IndexPath: uuid.New().String(), - }, - Data: awsio.ReadSeekNopCloser{ReadSeeker: bytes.NewReader([]byte(blockForFirstFolderData))}, - } - - blockForSecondFolderData := "data2" - blockForSecondFolder := Block{ - BlockRef: BlockRef{ - Ref: Ref{ - TenantID: "tenantA", - TableName: "second-period-19624", - Bounds: v1.NewBounds(0xaaaa, 0xbbbb), - StartTimestamp: Date(2023, time.September, 24, 5, 0, 0), - EndTimestamp: Date(2023, time.September, 24, 6, 0, 0), - Checksum: 2, - }, - IndexPath: uuid.New().String(), }, - Data: awsio.ReadSeekNopCloser{ReadSeeker: bytes.NewReader([]byte(blockForSecondFolderData))}, + Data: awsio.ReadSeekNopCloser{ReadSeeker: bytes.NewReader([]byte("data"))}, } - - results, err := bloomClient.PutBlocks(context.Background(), []Block{blockForFirstFolder, blockForSecondFolder}) + _, err := bloomClient.PutBlocks(context.Background(), []Block{block}) require.NoError(t, err) - require.Len(t, results, 2) - firstResultBlock := results[0] - path := firstResultBlock.BlockPath - require.Equal(t, - fmt.Sprintf( - "bloom/first-period-19621/tenantA/blooms/%s/1695272400000-1695276000000-1", - v1.NewBounds(0xeeee, 0xffff), - ), - path, - ) - require.Equal(t, blockForFirstFolder.TenantID, firstResultBlock.TenantID) - require.Equal(t, blockForFirstFolder.TableName, firstResultBlock.TableName) - require.Equal(t, blockForFirstFolder.Bounds.Min, firstResultBlock.Bounds.Min) - require.Equal(t, blockForFirstFolder.Bounds.Max, firstResultBlock.Bounds.Max) - require.Equal(t, blockForFirstFolder.StartTimestamp, firstResultBlock.StartTimestamp) - require.Equal(t, blockForFirstFolder.EndTimestamp, firstResultBlock.EndTimestamp) - require.Equal(t, blockForFirstFolder.Checksum, firstResultBlock.Checksum) - require.Equal(t, blockForFirstFolder.IndexPath, firstResultBlock.IndexPath) - folder1 := bloomClient.storageConfig.NamedStores.Filesystem["folder-1"].Directory - savedFilePath := filepath.Join(folder1, path) - require.FileExists(t, savedFilePath) - savedData, err := os.ReadFile(savedFilePath) + got, err := bloomClient.GetBlock(context.Background(), block.BlockRef) require.NoError(t, err) - require.Equal(t, blockForFirstFolderData, string(savedData)) - - secondResultBlock := results[1] - path = secondResultBlock.BlockPath - require.Equal(t, - fmt.Sprintf( - "bloom/second-period-19624/tenantA/blooms/%s/1695531600000-1695535200000-2", - v1.NewBounds(0xaaaa, 0xbbbb), - ), - path, - ) - require.Equal(t, blockForSecondFolder.TenantID, secondResultBlock.TenantID) - require.Equal(t, blockForSecondFolder.TableName, secondResultBlock.TableName) - require.Equal(t, blockForSecondFolder.Bounds.Min, secondResultBlock.Bounds.Min) - require.Equal(t, blockForSecondFolder.Bounds.Max, secondResultBlock.Bounds.Max) - require.Equal(t, blockForSecondFolder.StartTimestamp, secondResultBlock.StartTimestamp) - require.Equal(t, blockForSecondFolder.EndTimestamp, secondResultBlock.EndTimestamp) - require.Equal(t, blockForSecondFolder.Checksum, secondResultBlock.Checksum) - require.Equal(t, blockForSecondFolder.IndexPath, secondResultBlock.IndexPath) - folder2 := bloomClient.storageConfig.NamedStores.Filesystem["folder-2"].Directory - - savedFilePath = filepath.Join(folder2, path) - require.FileExists(t, savedFilePath) - savedData, err = os.ReadFile(savedFilePath) + require.Equal(t, block.BlockRef, got.BlockRef) + data, err := io.ReadAll(got.Data) require.NoError(t, err) - require.Equal(t, blockForSecondFolderData, string(savedData)) + require.Equal(t, "data", string(data)) } func Test_BloomClient_DeleteBlocks(t *testing.T) { - bloomClient := createStore(t) - fsNamedStores := bloomClient.storageConfig.NamedStores.Filesystem - block1Path := filepath.Join(fsNamedStores["folder-1"].Directory, "bloom/first-period-19621/tenantA/blooms/000000000000eeee-000000000000ffff/1695272400000-1695276000000-1") - createBlockFile(t, block1Path) - block2Path := filepath.Join(fsNamedStores["folder-2"].Directory, "bloom/second-period-19624/tenantA/blooms/000000000000aaaa-000000000000bbbb/1695531600000-1695535200000-2") - createBlockFile(t, block2Path) - require.FileExists(t, block1Path) - require.FileExists(t, block2Path) - - blocksToDelete := []BlockRef{ - { - Ref: Ref{ - TenantID: "tenantA", - TableName: "second-period-19624", - Bounds: v1.NewBounds(0xaaaa, 0xbbbb), - StartTimestamp: Date(2023, time.September, 24, 5, 0, 0), - EndTimestamp: Date(2023, time.September, 24, 6, 0, 0), - Checksum: 2, - }, - IndexPath: uuid.New().String(), - }, - { - Ref: Ref{ - TenantID: "tenantA", - TableName: "first-period-19621", - Bounds: v1.NewBounds(0xeeee, 0xffff), - StartTimestamp: Date(2023, time.September, 21, 5, 0, 0), - EndTimestamp: Date(2023, time.September, 21, 6, 0, 0), - Checksum: 1, - }, - IndexPath: uuid.New().String(), + block := BlockRef{ + Ref: Ref{ + TenantID: "tenantA", + TableName: "first-period-19621", + Bounds: v1.NewBounds(0xeeee, 0xffff), + StartTimestamp: Date(2023, time.September, 21, 5, 0, 0), + EndTimestamp: Date(2023, time.September, 21, 6, 0, 0), + Checksum: 1, }, } - err := bloomClient.DeleteBlocks(context.Background(), blocksToDelete) + + bloomClient := createStore(t) + fsNamedStores := bloomClient.storageConfig.NamedStores.Filesystem + blockFullPath := NewPrefixedResolver( + fsNamedStores["folder-1"].Directory, + defaultKeyResolver{}, + ).Block(block).LocalPath() + _ = createBlockFile(t, blockFullPath) + require.FileExists(t, blockFullPath) + + err := bloomClient.DeleteBlocks(context.Background(), []BlockRef{block}) require.NoError(t, err) - require.NoFileExists(t, block1Path) - require.NoFileExists(t, block2Path) + require.NoFileExists(t, blockFullPath) + } func createBlockFile(t *testing.T, path string) string { @@ -556,8 +483,6 @@ func createMetaEntity( StartTimestamp: startTimestamp, EndTimestamp: endTimestamp, }, - IndexPath: uuid.New().String(), - BlockPath: uuid.New().String(), }, }, Blocks: []BlockRef{ @@ -569,8 +494,6 @@ func createMetaEntity( StartTimestamp: startTimestamp, EndTimestamp: endTimestamp, }, - IndexPath: uuid.New().String(), - BlockPath: uuid.New().String(), }, }, } diff --git a/pkg/storage/stores/shipper/bloomshipper/compress_utils.go b/pkg/storage/stores/shipper/bloomshipper/compress_utils.go index 96af5e987c3d4..5f11e81a5b3b6 100644 --- a/pkg/storage/stores/shipper/bloomshipper/compress_utils.go +++ b/pkg/storage/stores/shipper/bloomshipper/compress_utils.go @@ -32,32 +32,8 @@ func CompressBloomBlock(ref BlockRef, archivePath, localDst string, logger log.L return blockToUpload, nil } -func UncompressBloomBlock(block *LazyBlock, workingDirectory string, logger log.Logger) (string, error) { - workingDirectoryPath := filepath.Join(workingDirectory, block.BlockPath) - err := os.MkdirAll(workingDirectoryPath, os.ModePerm) - if err != nil { - return "", fmt.Errorf("can not create directory to extract the block: %w", err) - } - archivePath, err := writeDataToTempFile(workingDirectoryPath, block) - if err != nil { - return "", fmt.Errorf("error writing data to temp file: %w", err) - } - level.Info(logger).Log("msg", "extracting archive", "archive", archivePath, "workingDirectory", workingDirectoryPath, "blockPath", block.BlockPath) - defer func() { - err = os.Remove(archivePath) - if err != nil { - level.Error(logger).Log("msg", "removing archive file", "err", err, "file", archivePath) - } - }() - err = extractArchive(archivePath, workingDirectoryPath) - if err != nil { - return "", fmt.Errorf("error extracting archive: %w", err) - } - return workingDirectoryPath, nil -} - -func writeDataToTempFile(workingDirectoryPath string, block *LazyBlock) (string, error) { - defer block.Data.Close() +func writeDataToTempFile(workingDirectoryPath string, data io.ReadCloser) (string, error) { + defer data.Close() archivePath := filepath.Join(workingDirectoryPath, uuid.New().String()) archiveFile, err := os.Create(archivePath) @@ -65,7 +41,7 @@ func writeDataToTempFile(workingDirectoryPath string, block *LazyBlock) (string, return "", fmt.Errorf("error creating empty file to store the archiver: %w", err) } defer archiveFile.Close() - _, err = io.Copy(archiveFile, block.Data) + _, err = io.Copy(archiveFile, data) if err != nil { return "", fmt.Errorf("error writing data to archive file: %w", err) } diff --git a/pkg/storage/stores/shipper/bloomshipper/resolver.go b/pkg/storage/stores/shipper/bloomshipper/resolver.go new file mode 100644 index 0000000000000..6278af9c6d04d --- /dev/null +++ b/pkg/storage/stores/shipper/bloomshipper/resolver.go @@ -0,0 +1,121 @@ +package bloomshipper + +import ( + "fmt" + "path" + "path/filepath" +) + +const ( + BloomPrefix = "bloom" + MetasPrefix = "metas" + BlocksPrefix = "blocks" +) + +// KeyResolver is an interface for resolving keys to locations. +// This is used to determine where items are stored in object storage _and_ on disk. +// Using an interface allows us to abstract away platform specifics +// (e.g. OS path-specifics, object storage difference, etc) +// TODO(owen-d): implement resolvers that schema-aware, allowing us to change +// the locations of data across schema boundaries (for instance to upgrade|improve). +type KeyResolver interface { + Meta(MetaRef) Location + Block(BlockRef) Location +} + +type defaultKeyResolver struct{} + +func (defaultKeyResolver) Meta(ref MetaRef) Location { + return simpleLocation{ + BloomPrefix, + fmt.Sprintf("%v", ref.TableName), + ref.TenantID, + MetasPrefix, + fmt.Sprintf("%v-%v", ref.Bounds, ref.Checksum), + } +} + +func (defaultKeyResolver) Block(ref BlockRef) Location { + return simpleLocation{ + BloomPrefix, + fmt.Sprintf("%v", ref.TableName), + ref.TenantID, + BlocksPrefix, + ref.Bounds.String(), + fmt.Sprintf("%d-%d-%x", ref.StartTimestamp, ref.EndTimestamp, ref.Checksum), + } +} + +type PrefixedResolver struct { + prefix string + KeyResolver +} + +func NewPrefixedResolver(prefix string, resolver KeyResolver) KeyResolver { + return PrefixedResolver{ + prefix: prefix, + KeyResolver: resolver, + } +} + +func (p PrefixedResolver) Meta(ref MetaRef) Location { + return locations{ + key(p.prefix), + p.KeyResolver.Meta(ref), + } +} + +func (p PrefixedResolver) Block(ref BlockRef) Location { + return locations{ + key(p.prefix), + p.KeyResolver.Block(ref), + } +} + +type Location interface { + Addr() string // object storage location + LocalPath() string // local path version +} + +// simplest Location implementor, just a string +type key string + +func (k key) Addr() string { + return string(k) +} + +func (k key) LocalPath() string { + return string(k) +} + +// simpleLocation is a simple implementation of Location combining multiple strings +type simpleLocation []string + +func (xs simpleLocation) LocalPath() string { + return filepath.Join(xs...) +} + +func (xs simpleLocation) Addr() string { + return path.Join(xs...) +} + +// helper type for combining multiple locations into one +type locations []Location + +func (ls locations) Addr() string { + xs := make([]string, 0, len(ls)) + for _, l := range ls { + xs = append(xs, l.Addr()) + } + + return path.Join(xs...) +} + +func (ls locations) LocalPath() string { + xs := make([]string, 0, len(ls)) + for _, l := range ls { + xs = append(xs, l.LocalPath()) + } + + return filepath.Join(xs...) +} diff --git a/pkg/storage/stores/shipper/bloomshipper/shipper.go b/pkg/storage/stores/shipper/bloomshipper/shipper.go index fad9b2616b6bc..9ccffcd643024 100644 --- a/pkg/storage/stores/shipper/bloomshipper/shipper.go +++ b/pkg/storage/stores/shipper/bloomshipper/shipper.go @@ -107,7 +107,7 @@ func runCallback(callback ForEachBlockCallback, block blockWithQuerier) error { err := callback(block.closableBlockQuerier.BlockQuerier, block.Bounds) if err != nil { - return fmt.Errorf("error running callback function for block %s err: %w", block.BlockPath, err) + return fmt.Errorf("error running callback function for block %s err: %w", block.BlockRef, err) } return nil } @@ -142,37 +142,36 @@ func (s *Shipper) getActiveBlockRefs(ctx context.Context, tenantID string, inter 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 { - tombstones := make(map[string]interface{}) + blocks := make(map[BlockRef]bool) // block -> isTombstoned + for _, meta := range metas { for _, tombstone := range meta.Tombstones { - tombstones[tombstone.BlockPath] = nil + blocks[tombstone] = true } - } - blocksSet := make(map[string]BlockRef) - for _, meta := range metas { for _, block := range meta.Blocks { - if _, contains := tombstones[block.BlockPath]; contains { + tombstoned, ok := blocks[block] + if ok && tombstoned { // skip tombstoned blocks continue } - if isOutsideRange(block, interval, keyspaces) { - // skip block that are outside of interval or keyspaces - continue - } - blocksSet[block.BlockPath] = block + blocks[block] = false } } - blockRefs := make([]BlockRef, 0, len(blocksSet)) - for _, ref := range blocksSet { - blockRefs = append(blockRefs, ref) - } - sort.Slice(blockRefs, func(i, j int) bool { - return blockRefs[i].Bounds.Less(blockRefs[j].Bounds) + refs := make([]BlockRef, 0, len(blocks)) + for ref, tombstoned := range blocks { + if !tombstoned && !isOutsideRange(ref, interval, keyspaces) { + refs = append(refs, ref) + } + } + sort.Slice(refs, func(i, j int) bool { + return refs[i].Bounds.Less(refs[j].Bounds) }) - return blockRefs + return refs } // isOutsideRange tests if a given BlockRef b is outside of search boundaries diff --git a/pkg/storage/stores/shipper/bloomshipper/shipper_test.go b/pkg/storage/stores/shipper/bloomshipper/shipper_test.go index f3ef55a4f3901..8ad776d4164aa 100644 --- a/pkg/storage/stores/shipper/bloomshipper/shipper_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/shipper_test.go @@ -18,24 +18,24 @@ func Test_Shipper_findBlocks(t *testing.T) { { Blocks: []BlockRef{ //this blockRef is marked as deleted in the next meta - createMatchingBlockRef("block1"), - createMatchingBlockRef("block2"), + createMatchingBlockRef(1), + createMatchingBlockRef(2), }, }, { Blocks: []BlockRef{ //this blockRef is marked as deleted in the next meta - createMatchingBlockRef("block3"), - createMatchingBlockRef("block4"), + createMatchingBlockRef(3), + createMatchingBlockRef(4), }, }, { Tombstones: []BlockRef{ - createMatchingBlockRef("block1"), - createMatchingBlockRef("block3"), + createMatchingBlockRef(1), + createMatchingBlockRef(3), }, Blocks: []BlockRef{ - createMatchingBlockRef("block5"), + createMatchingBlockRef(5), }, }, } @@ -49,9 +49,9 @@ func Test_Shipper_findBlocks(t *testing.T) { blocks := BlocksForMetas(metas, interval, []v1.FingerprintBounds{{Min: 100, Max: 200}}) expectedBlockRefs := []BlockRef{ - createMatchingBlockRef("block2"), - createMatchingBlockRef("block4"), - createMatchingBlockRef("block5"), + createMatchingBlockRef(2), + createMatchingBlockRef(4), + createMatchingBlockRef(5), } require.ElementsMatch(t, expectedBlockRefs, blocks) }) @@ -98,7 +98,7 @@ func Test_Shipper_findBlocks(t *testing.T) { } for name, data := range tests { t.Run(name, func(t *testing.T) { - ref := createBlockRef("fake-block", data.minFingerprint, data.maxFingerprint, data.startTimestamp, data.endTimestamp) + ref := createBlockRef(data.minFingerprint, data.maxFingerprint, data.startTimestamp, data.endTimestamp) blocks := BlocksForMetas([]Meta{{Blocks: []BlockRef{ref}}}, NewInterval(300, 400), []v1.FingerprintBounds{{Min: 100, Max: 200}}) if data.filtered { require.Empty(t, blocks) @@ -115,78 +115,79 @@ func TestIsOutsideRange(t *testing.T) { endTs := model.Time(2000) t.Run("is outside if startTs > through", func(t *testing.T) { - b := createBlockRef("block", 0, math.MaxUint64, startTs, endTs) + b := createBlockRef(0, math.MaxUint64, startTs, endTs) isOutside := isOutsideRange(b, NewInterval(0, 900), []v1.FingerprintBounds{}) require.True(t, isOutside) }) t.Run("is outside if startTs == through ", func(t *testing.T) { - b := createBlockRef("block", 0, math.MaxUint64, startTs, endTs) + b := createBlockRef(0, math.MaxUint64, startTs, endTs) isOutside := isOutsideRange(b, NewInterval(900, 1000), []v1.FingerprintBounds{}) require.True(t, isOutside) }) t.Run("is outside if endTs < from", func(t *testing.T) { - b := createBlockRef("block", 0, math.MaxUint64, startTs, endTs) + b := createBlockRef(0, math.MaxUint64, startTs, endTs) isOutside := isOutsideRange(b, NewInterval(2100, 3000), []v1.FingerprintBounds{}) require.True(t, isOutside) }) t.Run("is outside if endFp < first fingerprint", func(t *testing.T) { - b := createBlockRef("block", 0, 90, startTs, endTs) + b := createBlockRef(0, 90, startTs, endTs) isOutside := isOutsideRange(b, NewInterval(startTs, endTs), []v1.FingerprintBounds{{Min: 100, Max: 199}}) require.True(t, isOutside) }) t.Run("is outside if startFp > last fingerprint", func(t *testing.T) { - b := createBlockRef("block", 200, math.MaxUint64, startTs, endTs) + b := createBlockRef(200, math.MaxUint64, startTs, endTs) isOutside := isOutsideRange(b, NewInterval(startTs, endTs), []v1.FingerprintBounds{{Min: 0, Max: 49}, {Min: 100, Max: 149}}) require.True(t, isOutside) }) t.Run("is outside if within gaps in fingerprints", func(t *testing.T) { - b := createBlockRef("block", 100, 199, startTs, endTs) + b := createBlockRef(100, 199, startTs, endTs) isOutside := isOutsideRange(b, NewInterval(startTs, endTs), []v1.FingerprintBounds{{Min: 0, Max: 99}, {Min: 200, Max: 299}}) require.True(t, isOutside) }) t.Run("is not outside if within fingerprints 1", func(t *testing.T) { - b := createBlockRef("block", 10, 90, startTs, endTs) + b := createBlockRef(10, 90, startTs, endTs) isOutside := isOutsideRange(b, NewInterval(startTs, endTs), []v1.FingerprintBounds{{Min: 0, Max: 99}, {Min: 200, Max: 299}}) require.False(t, isOutside) }) t.Run("is not outside if within fingerprints 2", func(t *testing.T) { - b := createBlockRef("block", 210, 290, startTs, endTs) + b := createBlockRef(210, 290, startTs, endTs) isOutside := isOutsideRange(b, NewInterval(startTs, endTs), []v1.FingerprintBounds{{Min: 0, Max: 99}, {Min: 200, Max: 299}}) require.False(t, isOutside) }) t.Run("is not outside if spans across multiple fingerprint ranges", func(t *testing.T) { - b := createBlockRef("block", 50, 250, startTs, endTs) + b := createBlockRef(50, 250, startTs, endTs) isOutside := isOutsideRange(b, NewInterval(startTs, endTs), []v1.FingerprintBounds{{Min: 0, Max: 99}, {Min: 200, Max: 299}}) require.False(t, isOutside) }) t.Run("is not outside if fingerprint range and time range are larger than block", func(t *testing.T) { - b := createBlockRef("block", math.MaxUint64/3, math.MaxUint64/3*2, startTs, endTs) + b := createBlockRef(math.MaxUint64/3, math.MaxUint64/3*2, startTs, endTs) isOutside := isOutsideRange(b, NewInterval(0, 3000), []v1.FingerprintBounds{{Min: 0, Max: math.MaxUint64}}) require.False(t, isOutside) }) t.Run("is not outside if block fingerprint range is bigger that search keyspace", func(t *testing.T) { - b := createBlockRef("block", 0x0000, 0xffff, model.Earliest, model.Latest) + b := createBlockRef(0x0000, 0xffff, model.Earliest, model.Latest) isOutside := isOutsideRange(b, NewInterval(startTs, endTs), []v1.FingerprintBounds{{Min: 0x0100, Max: 0xff00}}) require.False(t, isOutside) }) } -func createMatchingBlockRef(blockPath string) BlockRef { - return createBlockRef(blockPath, 0, math.MaxUint64, model.Time(0), model.Time(math.MaxInt64)) +func createMatchingBlockRef(checksum uint32) BlockRef { + block := createBlockRef(0, math.MaxUint64, model.Time(0), model.Time(math.MaxInt64)) + block.Checksum = checksum + return block } func createBlockRef( - blockPath string, minFingerprint, maxFingerprint uint64, startTimestamp, endTimestamp model.Time, ) BlockRef { @@ -200,7 +201,5 @@ func createBlockRef( EndTimestamp: endTimestamp, Checksum: 0, }, - // block path is unique, and it's used to distinguish the blocks so the rest of the fields might be skipped in this test - BlockPath: blockPath, } } diff --git a/pkg/storage/stores/shipper/bloomshipper/store.go b/pkg/storage/stores/shipper/bloomshipper/store.go index d4da6573ba5df..8bcb65d304ca7 100644 --- a/pkg/storage/stores/shipper/bloomshipper/store.go +++ b/pkg/storage/stores/shipper/bloomshipper/store.go @@ -32,11 +32,12 @@ var _ Client = &bloomStoreEntry{} var _ Store = &bloomStoreEntry{} type bloomStoreEntry struct { - start model.Time - cfg config.PeriodConfig - objectClient client.ObjectClient - bloomClient Client - fetcher *Fetcher + start model.Time + cfg config.PeriodConfig + objectClient client.ObjectClient + bloomClient Client + fetcher *Fetcher + defaultKeyResolver // TODO(owen-d): impl schema aware resolvers } // ResolveMetas implements store. @@ -191,6 +192,38 @@ func NewBloomStore( return store, nil } +// Impements KeyResolver +func (b *BloomStore) Meta(ref MetaRef) (loc Location) { + _ = b.storeDo(ref.StartTimestamp, func(s *bloomStoreEntry) error { + loc = s.Meta(ref) + return nil + }) + + // NB(owen-d): should not happen unless a ref is requested outside the store's accepted range. + // This should be prevented during query validation + if loc == nil { + loc = defaultKeyResolver{}.Meta(ref) + } + + return +} + +// Impements KeyResolver +func (b *BloomStore) Block(ref BlockRef) (loc Location) { + _ = b.storeDo(ref.StartTimestamp, func(s *bloomStoreEntry) error { + loc = s.Block(ref) + return nil + }) + + // NB(owen-d): should not happen unless a ref is requested outside the store's accepted range. + // This should be prevented during query validation + if loc == nil { + loc = defaultKeyResolver{}.Block(ref) + } + + return +} + // Fetcher implements Store. func (b *BloomStore) Fetcher(ts model.Time) *Fetcher { if store := b.getStore(ts); store != nil { From 5132f6d2a76a26d6e3b8b7127094d16eb26e9442 Mon Sep 17 00:00:00 2001 From: Nikki Ashton Date: Fri, 2 Feb 2024 11:07:29 +0100 Subject: [PATCH 24/45] Remove trailing tab character in helm templates (#11805) **What this PR does / why we need it**: Remove trailing tab character in helm templates **Which issue(s) this PR fixes**: Fixes #11385 **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 - [ ] `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) --- production/helm/loki/CHANGELOG.md | 4 ++++ production/helm/loki/Chart.yaml | 2 +- production/helm/loki/README.md | 2 +- .../helm/loki/templates/backend/statefulset-backend.yaml | 2 +- production/helm/loki/templates/read/statefulset-read.yaml | 2 +- production/helm/loki/templates/single-binary/statefulset.yaml | 2 +- 6 files changed, 9 insertions(+), 5 deletions(-) diff --git a/production/helm/loki/CHANGELOG.md b/production/helm/loki/CHANGELOG.md index f70a5f1183003..9a5b2a3cb4f27 100644 --- a/production/helm/loki/CHANGELOG.md +++ b/production/helm/loki/CHANGELOG.md @@ -13,6 +13,10 @@ Entries should include a reference to the pull request that introduced the chang [//]: # ( : do not remove this line. This locator is used by the CI pipeline to automatically create a changelog entry for each new Loki release. Add other chart versions and respective changelog entries bellow this line.) +## 5.42.2 + +- [BUGFIX] Remove trailing tab character in statefulset templates + ## 5.42.1 - [BUGFIX] Added missing annotations to loki-read StatefulSet. diff --git a/production/helm/loki/Chart.yaml b/production/helm/loki/Chart.yaml index 3217cb5988b95..183333ba652f4 100644 --- a/production/helm/loki/Chart.yaml +++ b/production/helm/loki/Chart.yaml @@ -3,7 +3,7 @@ name: loki description: Helm chart for Grafana Loki in simple, scalable mode type: application appVersion: 2.9.4 -version: 5.42.1 +version: 5.42.2 home: https://grafana.github.io/helm-charts sources: - https://github.com/grafana/loki diff --git a/production/helm/loki/README.md b/production/helm/loki/README.md index ec53471e5cdcc..5b6cbe5a2fa1b 100644 --- a/production/helm/loki/README.md +++ b/production/helm/loki/README.md @@ -1,6 +1,6 @@ # loki -![Version: 5.42.1](https://img.shields.io/badge/Version-5.42.1-informational?style=flat-square) ![Type: application](https://img.shields.io/badge/Type-application-informational?style=flat-square) ![AppVersion: 2.9.4](https://img.shields.io/badge/AppVersion-2.9.4-informational?style=flat-square) +![Version: 5.42.2](https://img.shields.io/badge/Version-5.42.2-informational?style=flat-square) ![Type: application](https://img.shields.io/badge/Type-application-informational?style=flat-square) ![AppVersion: 2.9.4](https://img.shields.io/badge/AppVersion-2.9.4-informational?style=flat-square) Helm chart for Grafana Loki in simple, scalable mode diff --git a/production/helm/loki/templates/backend/statefulset-backend.yaml b/production/helm/loki/templates/backend/statefulset-backend.yaml index 0bdef7e8e7774..1b1be055f6266 100644 --- a/production/helm/loki/templates/backend/statefulset-backend.yaml +++ b/production/helm/loki/templates/backend/statefulset-backend.yaml @@ -263,7 +263,7 @@ spec: {{- end }} {{- if .Values.backend.persistence.volumeClaimsEnabled }} volumeClaimTemplates: - - apiVersion: v1 + - apiVersion: v1 kind: PersistentVolumeClaim metadata: name: data diff --git a/production/helm/loki/templates/read/statefulset-read.yaml b/production/helm/loki/templates/read/statefulset-read.yaml index 2f16179a56c43..eaea4aaecbc29 100644 --- a/production/helm/loki/templates/read/statefulset-read.yaml +++ b/production/helm/loki/templates/read/statefulset-read.yaml @@ -175,7 +175,7 @@ spec: {{- toYaml . | nindent 8 }} {{- end }} volumeClaimTemplates: - - apiVersion: v1 + - apiVersion: v1 kind: PersistentVolumeClaim metadata: name: data diff --git a/production/helm/loki/templates/single-binary/statefulset.yaml b/production/helm/loki/templates/single-binary/statefulset.yaml index b74cc60500d6a..96c8974b72c26 100644 --- a/production/helm/loki/templates/single-binary/statefulset.yaml +++ b/production/helm/loki/templates/single-binary/statefulset.yaml @@ -176,7 +176,7 @@ spec: {{- end }} {{- if .Values.singleBinary.persistence.enabled }} volumeClaimTemplates: - - apiVersion: v1 + - apiVersion: v1 kind: PersistentVolumeClaim metadata: name: storage From 0f34d9155e7c207add442a961f356f6111dc8ca7 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Fri, 2 Feb 2024 13:18:00 -0800 Subject: [PATCH 25/45] Blooms/block metadata (#11859) A few updates to the bloom library: * Uses `FingerprintBounds` in series headers * Encodes `BlockOptions` in the series file so we can later read the target page & block sizes the block was generated with in addition to the schema. * Introduces `BlockMetadata` struct and loads it correctly from blocks. This struct will be used to convert to the `BlockRef`s from the `bloomshipper` pkg and used in the bloom compactor + bloom gateway * Integrates checksums better into block building and XORs the headers metadata from each file (blooms, series) together to generate a final checksum for the block (a combination of both files). --- pkg/storage/bloom/v1/block.go | 46 ++++++++--- pkg/storage/bloom/v1/bloom.go | 19 ++--- pkg/storage/bloom/v1/builder.go | 109 +++++++++++++------------- pkg/storage/bloom/v1/builder_test.go | 26 +++++- pkg/storage/bloom/v1/index.go | 56 ++++++------- pkg/storage/bloom/v1/index_querier.go | 2 +- 6 files changed, 157 insertions(+), 101 deletions(-) diff --git a/pkg/storage/bloom/v1/block.go b/pkg/storage/bloom/v1/block.go index b1b08be008405..6d13349855f40 100644 --- a/pkg/storage/bloom/v1/block.go +++ b/pkg/storage/bloom/v1/block.go @@ -7,21 +7,23 @@ import ( "github.com/prometheus/common/model" ) +type BlockMetadata struct { + Options BlockOptions + Series SeriesHeader + Checksum uint32 +} + type Block struct { // covers series pages index BlockIndex // covers bloom pages blooms BloomBlock - // TODO(owen-d): implement - // synthetic header for the entire block - // built from all the pages in the index - header SeriesHeader + metadata BlockMetadata reader BlockReader // should this be decoupled from the struct (accepted as method arg instead)? initialized bool - dataRange SeriesHeader } func NewBlock(reader BlockReader) *Block { @@ -38,30 +40,49 @@ func (b *Block) LoadHeaders() error { return errors.Wrap(err, "getting index reader") } - if err := b.index.DecodeHeaders(idx); err != nil { + indexChecksum, err := b.index.DecodeHeaders(idx) + if err != nil { return errors.Wrap(err, "decoding index") } + b.metadata.Options = b.index.opts + // TODO(owen-d): better pattern xs := make([]SeriesHeader, 0, len(b.index.pageHeaders)) for _, h := range b.index.pageHeaders { xs = append(xs, h.SeriesHeader) } - b.dataRange = aggregateHeaders(xs) + b.metadata.Series = aggregateHeaders(xs) blooms, err := b.reader.Blooms() if err != nil { return errors.Wrap(err, "getting blooms reader") } - if err := b.blooms.DecodeHeaders(blooms); err != nil { + bloomChecksum, err := b.blooms.DecodeHeaders(blooms) + if err != nil { return errors.Wrap(err, "decoding blooms") } b.initialized = true + + if !b.metadata.Options.Schema.Compatible(b.blooms.schema) { + return fmt.Errorf( + "schema mismatch: index (%v) vs blooms (%v)", + b.metadata.Options.Schema, b.blooms.schema, + ) + } + + b.metadata.Checksum = combineChecksums(indexChecksum, bloomChecksum) } return nil } +// XOR checksums as a simple checksum combiner with the benefit that +// each part can be recomputed by XORing the result against the other +func combineChecksums(index, blooms uint32) uint32 { + return index ^ blooms +} + // convenience method func (b *Block) Querier() *BlockQuerier { return NewBlockQuerier(b) @@ -75,11 +96,18 @@ func (b *Block) Blooms() *LazyBloomIter { return NewLazyBloomIter(b) } +func (b *Block) Metadata() (BlockMetadata, error) { + if err := b.LoadHeaders(); err != nil { + return BlockMetadata{}, err + } + return b.metadata, nil +} + func (b *Block) Schema() (Schema, error) { if err := b.LoadHeaders(); err != nil { return Schema{}, err } - return b.index.schema, nil + return b.metadata.Options.Schema, nil } type BlockQuerier struct { diff --git a/pkg/storage/bloom/v1/bloom.go b/pkg/storage/bloom/v1/bloom.go index be95f96862eac..20c310ef695c0 100644 --- a/pkg/storage/bloom/v1/bloom.go +++ b/pkg/storage/bloom/v1/bloom.go @@ -171,9 +171,9 @@ func NewBloomBlock(encoding chunkenc.Encoding) BloomBlock { } } -func (b *BloomBlock) DecodeHeaders(r io.ReadSeeker) error { +func (b *BloomBlock) DecodeHeaders(r io.ReadSeeker) (uint32, error) { if err := b.schema.DecodeFrom(r); err != nil { - return errors.Wrap(err, "decoding schema") + return 0, errors.Wrap(err, "decoding schema") } var ( @@ -182,35 +182,36 @@ func (b *BloomBlock) DecodeHeaders(r io.ReadSeeker) error { ) // last 12 bytes are (headers offset: 8 byte u64, checksum: 4 byte u32) if _, err := r.Seek(-12, io.SeekEnd); err != nil { - return errors.Wrap(err, "seeking to bloom headers metadata") + return 0, errors.Wrap(err, "seeking to bloom headers metadata") } dec.B, err = io.ReadAll(r) if err != nil { - return errors.Wrap(err, "reading bloom headers metadata") + return 0, errors.Wrap(err, "reading bloom headers metadata") } headerOffset := dec.Be64() + checksum := dec.Be32() if _, err := r.Seek(int64(headerOffset), io.SeekStart); err != nil { - return errors.Wrap(err, "seeking to bloom headers") + return 0, errors.Wrap(err, "seeking to bloom headers") } dec.B, err = io.ReadAll(r) if err != nil { - return errors.Wrap(err, "reading bloom page headers") + return 0, errors.Wrap(err, "reading bloom page headers") } if err := dec.CheckCrc(castagnoliTable); err != nil { - return errors.Wrap(err, "checksumming page headers") + return 0, errors.Wrap(err, "checksumming page headers") } b.pageHeaders = make([]BloomPageHeader, dec.Uvarint()) for i := 0; i < len(b.pageHeaders); i++ { header := &b.pageHeaders[i] if err := header.Decode(&dec); err != nil { - return errors.Wrapf(err, "decoding %dth series header", i) + return 0, errors.Wrapf(err, "decoding %dth series header", i) } } - return nil + return checksum, nil } func (b *BloomBlock) BloomPageDecoder(r io.ReadSeeker, pageIdx int) (*BloomPageDecoder, error) { diff --git a/pkg/storage/bloom/v1/builder.go b/pkg/storage/bloom/v1/builder.go index 26b9a39cfd7bf..ac7a83baad374 100644 --- a/pkg/storage/bloom/v1/builder.go +++ b/pkg/storage/bloom/v1/builder.go @@ -5,7 +5,6 @@ import ( "fmt" "hash" "io" - "sort" "github.com/pkg/errors" "github.com/prometheus/common/model" @@ -21,15 +20,46 @@ var ( type BlockOptions struct { // Schema determines the Schema of the block and cannot be changed + // without recreating the block from underlying data Schema Schema // The following options can be changed on the fly. // For instance, adding another page to a block with - // a different target page size is supported. + // a different target page size is supported, although + // the block will store the original sizes it was created with // target size in bytes (decompressed) // of each page type - SeriesPageSize, BloomPageSize, BlockSize int + SeriesPageSize, BloomPageSize, BlockSize uint64 +} + +func (b BlockOptions) Len() int { + return 3*8 + b.Schema.Len() +} + +func (b *BlockOptions) DecodeFrom(r io.ReadSeeker) error { + buf := make([]byte, b.Len()) + _, err := io.ReadFull(r, buf) + if err != nil { + return errors.Wrap(err, "reading block options") + } + + dec := encoding.DecWith(buf) + + if err := b.Schema.Decode(&dec); err != nil { + return errors.Wrap(err, "decoding schema") + } + b.SeriesPageSize = dec.Be64() + b.BloomPageSize = dec.Be64() + b.BlockSize = dec.Be64() + return nil +} + +func (b BlockOptions) Encode(enc *encoding.Encbuf) { + b.Schema.Encode(enc) + enc.PutBE64(b.SeriesPageSize) + enc.PutBE64(b.BloomPageSize) + enc.PutBE64(b.BlockSize) } type BlockBuilder struct { @@ -90,14 +120,19 @@ func (b *BlockBuilder) BuildFrom(itr Iterator[SeriesWithBloom]) (uint32, error) return 0, errors.Wrap(err, "iterating series with blooms") } - checksum, err := b.blooms.Close() + return b.Close() +} + +func (b *BlockBuilder) Close() (uint32, error) { + bloomChecksum, err := b.blooms.Close() if err != nil { return 0, errors.Wrap(err, "closing bloom file") } - if err := b.index.Close(); err != nil { + indexCheckSum, err := b.index.Close() + if err != nil { return 0, errors.Wrap(err, "closing series file") } - return checksum, nil + return combineChecksums(indexCheckSum, bloomChecksum), nil } func (b *BlockBuilder) AddSeries(series SeriesWithBloom) error { @@ -131,7 +166,7 @@ func NewBloomBlockBuilder(opts BlockOptions, writer io.WriteCloser) *BloomBlockB return &BloomBlockBuilder{ opts: opts, writer: writer, - page: NewPageWriter(opts.BloomPageSize), + page: NewPageWriter(int(opts.BloomPageSize)), scratch: &encoding.Encbuf{}, } } @@ -307,16 +342,16 @@ func NewIndexBuilder(opts BlockOptions, writer io.WriteCloser) *IndexBuilder { return &IndexBuilder{ opts: opts, writer: writer, - page: NewPageWriter(opts.SeriesPageSize), + page: NewPageWriter(int(opts.SeriesPageSize)), scratch: &encoding.Encbuf{}, } } -func (b *IndexBuilder) WriteSchema() error { +func (b *IndexBuilder) WriteOpts() error { b.scratch.Reset() - b.opts.Schema.Encode(b.scratch) + b.opts.Encode(b.scratch) if _, err := b.writer.Write(b.scratch.Get()); err != nil { - return errors.Wrap(err, "writing schema") + return errors.Wrap(err, "writing opts+schema") } b.writtenSchema = true b.offset += b.scratch.Len() @@ -325,8 +360,8 @@ func (b *IndexBuilder) WriteSchema() error { func (b *IndexBuilder) Append(series SeriesWithOffset) error { if !b.writtenSchema { - if err := b.WriteSchema(); err != nil { - return errors.Wrap(err, "writing schema") + if err := b.WriteOpts(); err != nil { + return errors.Wrap(err, "appending series") } } @@ -408,8 +443,7 @@ func (b *IndexBuilder) flushPage() error { DecompressedLen: decompressedLen, SeriesHeader: SeriesHeader{ NumSeries: b.page.Count(), - FromFp: b.fromFp, - ThroughFp: b.previousFp, + Bounds: NewBounds(b.fromFp, b.previousFp), FromTs: b.fromTs, ThroughTs: b.throughTs, }, @@ -428,10 +462,10 @@ func (b *IndexBuilder) flushPage() error { return nil } -func (b *IndexBuilder) Close() error { +func (b *IndexBuilder) Close() (uint32, error) { if b.page.Count() > 0 { if err := b.flushPage(); err != nil { - return errors.Wrap(err, "flushing final series page") + return 0, errors.Wrap(err, "flushing final series page") } } @@ -451,39 +485,9 @@ func (b *IndexBuilder) Close() error { b.scratch.PutHash(crc32Hash) _, err := b.writer.Write(b.scratch.Get()) if err != nil { - return errors.Wrap(err, "writing series page headers") + return 0, errors.Wrap(err, "writing series page headers") } - return errors.Wrap(b.writer.Close(), "closing series writer") -} - -// SortBlocksIntoOverlappingGroups sorts a list of blocks into a sorted list of lists, -// where each list contains blocks that overlap with each other. -// TODO(owen-d): implement as an iterator so we don't have to load all blocks at once -// NB: unused now, but likely useful when we want to optimize compaction. I wrote this expecting to need it now -// but it feels unsavory to remove it -func SortBlocksIntoOverlappingGroups(xs []*Block) (groups [][]*Block) { - sort.Slice(xs, func(i, j int) bool { - a, b := xs[i].index, xs[j].index - return a.pageHeaders[0].FromFp <= b.pageHeaders[0].FromFp - }) - - var curGroup []*Block - for _, x := range xs { - switch { - case len(curGroup) == 0: - curGroup = append(curGroup, x) - case curGroup[len(curGroup)-1].dataRange.OverlapFingerprintRange(x.dataRange): - curGroup = append(curGroup, x) - default: - groups = append(groups, curGroup) - curGroup = []*Block{x} - } - } - - if len(curGroup) > 0 { - groups = append(groups, curGroup) - } - return groups + return crc32Hash.Sum32(), errors.Wrap(b.writer.Close(), "closing series writer") } // Simplistic implementation of a merge builder that builds a single block @@ -586,12 +590,9 @@ func (mb *MergeBuilder) Build(builder *BlockBuilder) (uint32, error) { } } - checksum, err := builder.blooms.Close() + checksum, err := builder.Close() if err != nil { - return 0, errors.Wrap(err, "closing bloom file") - } - if err := builder.index.Close(); err != nil { - return 0, errors.Wrap(err, "closing series file") + return 0, errors.Wrap(err, "closing block") } return checksum, nil } diff --git a/pkg/storage/bloom/v1/builder_test.go b/pkg/storage/bloom/v1/builder_test.go index cb28f0cb53357..6bf2c26e7b585 100644 --- a/pkg/storage/bloom/v1/builder_test.go +++ b/pkg/storage/bloom/v1/builder_test.go @@ -9,8 +9,32 @@ import ( "github.com/stretchr/testify/require" "github.com/grafana/loki/pkg/chunkenc" + "github.com/grafana/loki/pkg/util/encoding" ) +func TestBlockOptionsRoundTrip(t *testing.T) { + opts := BlockOptions{ + Schema: Schema{ + version: V1, + encoding: chunkenc.EncSnappy, + nGramLength: 10, + nGramSkip: 2, + }, + SeriesPageSize: 100, + BloomPageSize: 10 << 10, + BlockSize: 10 << 20, + } + + var enc encoding.Encbuf + opts.Encode(&enc) + + var got BlockOptions + err := got.DecodeFrom(bytes.NewReader(enc.Get())) + require.Nil(t, err) + + require.Equal(t, opts, got) +} + func TestBlockBuilderRoundTrip(t *testing.T) { numSeries := 100 numKeysPerSeries := 10000 @@ -334,7 +358,7 @@ func TestMergeBuilder_Roundtrip(t *testing.T) { checksum, err := mb.Build(builder) require.Nil(t, err) - require.Equal(t, uint32(0x2ec4fd6a), checksum) + require.Equal(t, uint32(0xe306ec6e), 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/index.go b/pkg/storage/bloom/v1/index.go index 10c1e41fd1139..e3a14dc5453ea 100644 --- a/pkg/storage/bloom/v1/index.go +++ b/pkg/storage/bloom/v1/index.go @@ -2,6 +2,7 @@ package v1 import ( "bytes" + "fmt" "io" "github.com/pkg/errors" @@ -17,6 +18,10 @@ type Schema struct { nGramLength, nGramSkip uint64 } +func (s Schema) String() string { + return fmt.Sprintf("v%d,encoding=%s,ngram=%d,skip=%d", s.version, s.encoding, s.nGramLength, s.nGramSkip) +} + func (s Schema) Compatible(other Schema) bool { return s == other } @@ -89,19 +94,14 @@ func (s *Schema) Decode(dec *encoding.Decbuf) error { // Block index is a set of series pages along with // the headers for each page type BlockIndex struct { - schema Schema - pageHeaders []SeriesPageHeaderWithOffset // headers for each series page -} + opts BlockOptions -func NewBlockIndex(encoding chunkenc.Encoding) BlockIndex { - return BlockIndex{ - schema: Schema{version: DefaultSchemaVersion, encoding: encoding}, - } + pageHeaders []SeriesPageHeaderWithOffset // headers for each series page } -func (b *BlockIndex) DecodeHeaders(r io.ReadSeeker) error { - if err := b.schema.DecodeFrom(r); err != nil { - return errors.Wrap(err, "decoding schema") +func (b *BlockIndex) DecodeHeaders(r io.ReadSeeker) (uint32, error) { + if err := b.opts.DecodeFrom(r); err != nil { + return 0, errors.Wrap(err, "decoding block options") } var ( @@ -111,24 +111,25 @@ func (b *BlockIndex) DecodeHeaders(r io.ReadSeeker) error { // last 12 bytes are (headers offset: 8 byte u64, checksum: 4 byte u32) if _, err := r.Seek(-12, io.SeekEnd); err != nil { - return errors.Wrap(err, "seeking to bloom headers metadata") + return 0, errors.Wrap(err, "seeking to bloom headers metadata") } dec.B, err = io.ReadAll(r) if err != nil { - return errors.Wrap(err, "reading bloom headers metadata") + return 0, errors.Wrap(err, "reading bloom headers metadata") } headerOffset := dec.Be64() + checksum := dec.Be32() if _, err := r.Seek(int64(headerOffset), io.SeekStart); err != nil { - return errors.Wrap(err, "seeking to index headers") + return 0, errors.Wrap(err, "seeking to index headers") } dec.B, err = io.ReadAll(r) if err != nil { - return errors.Wrap(err, "reading index page headers") + return 0, errors.Wrap(err, "reading index page headers") } if err := dec.CheckCrc(castagnoliTable); err != nil { - return errors.Wrap(err, "checksumming page headers") + return 0, errors.Wrap(err, "checksumming page headers") } b.pageHeaders = make( @@ -139,12 +140,12 @@ func (b *BlockIndex) DecodeHeaders(r io.ReadSeeker) error { for i := 0; i < len(b.pageHeaders); i++ { var s SeriesPageHeaderWithOffset if err := s.Decode(&dec); err != nil { - return errors.Wrapf(err, "decoding %dth series header", i) + return 0, errors.Wrapf(err, "decoding %dth series header", i) } b.pageHeaders[i] = s } - return nil + return checksum, nil } // decompress page and return an iterator over the bytes @@ -167,7 +168,7 @@ func (b *BlockIndex) NewSeriesPageDecoder(r io.ReadSeeker, header SeriesPageHead return nil, errors.Wrap(err, "checksumming series page") } - decompressor, err := b.schema.DecompressorPool().GetReader(bytes.NewReader(dec.Get())) + decompressor, err := b.opts.Schema.DecompressorPool().GetReader(bytes.NewReader(dec.Get())) if err != nil { return nil, errors.Wrap(err, "getting decompressor") } @@ -213,12 +214,12 @@ func (h *SeriesPageHeaderWithOffset) Decode(dec *encoding.Decbuf) error { type SeriesHeader struct { NumSeries int - FromFp, ThroughFp model.Fingerprint + Bounds FingerprintBounds FromTs, ThroughTs model.Time } func (h SeriesHeader) OverlapFingerprintRange(other SeriesHeader) bool { - return h.ThroughFp >= other.FromFp && h.FromFp <= other.ThroughFp + return h.Bounds.Overlaps(other.Bounds) } // build one aggregated header for the entire block @@ -227,9 +228,10 @@ func aggregateHeaders(xs []SeriesHeader) SeriesHeader { return SeriesHeader{} } + fromFp, _ := xs[0].Bounds.GetFromThrough() + _, throughFP := xs[len(xs)-1].Bounds.GetFromThrough() res := SeriesHeader{ - FromFp: xs[0].FromFp, - ThroughFp: xs[len(xs)-1].ThroughFp, + Bounds: NewBounds(fromFp, throughFP), } for _, x := range xs { @@ -245,16 +247,16 @@ func aggregateHeaders(xs []SeriesHeader) SeriesHeader { func (h *SeriesHeader) Encode(enc *encoding.Encbuf) { enc.PutUvarint(h.NumSeries) - enc.PutUvarint64(uint64(h.FromFp)) - enc.PutUvarint64(uint64(h.ThroughFp)) + enc.PutUvarint64(uint64(h.Bounds.Min)) + enc.PutUvarint64(uint64(h.Bounds.Max)) enc.PutVarint64(int64(h.FromTs)) enc.PutVarint64(int64(h.ThroughTs)) } func (h *SeriesHeader) Decode(dec *encoding.Decbuf) error { h.NumSeries = dec.Uvarint() - h.FromFp = model.Fingerprint(dec.Uvarint64()) - h.ThroughFp = model.Fingerprint(dec.Uvarint64()) + h.Bounds.Min = model.Fingerprint(dec.Uvarint64()) + h.Bounds.Max = model.Fingerprint(dec.Uvarint64()) h.FromTs = model.Time(dec.Varint64()) h.ThroughTs = model.Time(dec.Varint64()) return dec.Err() @@ -305,7 +307,7 @@ func (d *SeriesPageDecoder) Next() bool { } func (d *SeriesPageDecoder) Seek(fp model.Fingerprint) { - if fp > d.header.ThroughFp { + if fp > d.header.Bounds.Max { // shortcut: we know the fingerprint is too large so nothing in this page // will match the seek call, which returns the first found fingerprint >= fp. // so masquerade the index as if we've already iterated through diff --git a/pkg/storage/bloom/v1/index_querier.go b/pkg/storage/bloom/v1/index_querier.go index 005f480e68e9c..142b6423185b6 100644 --- a/pkg/storage/bloom/v1/index_querier.go +++ b/pkg/storage/bloom/v1/index_querier.go @@ -49,7 +49,7 @@ func (it *LazySeriesIter) Seek(fp model.Fingerprint) error { // first potentially relevant page desiredPage := sort.Search(len(it.b.index.pageHeaders), func(i int) bool { header := it.b.index.pageHeaders[i] - return header.ThroughFp >= fp + return header.Bounds.Max >= fp }) switch { From c350641a64b041228616c84c69e6fe834b5bd581 Mon Sep 17 00:00:00 2001 From: Sandeep Sukhani Date: Mon, 5 Feb 2024 14:34:17 +0530 Subject: [PATCH 26/45] otlp: add docs for per tenant otlp config (#11849) **What this PR does / why we need it**: In PR #11143 we added support for per tenant otlp config. This PR adds the relevant documentation to explain how the config looks and works. **Checklist** - [x] Documentation added --------- Co-authored-by: J Stickler --- docs/sources/configure/_index.md | 34 +++++++- docs/sources/send-data/otel/_index.md | 101 ++++++++++++++++++++++- pkg/loghttp/push/otlp_config.go | 16 ++-- pkg/loghttp/push/otlp_config_test.go | 6 +- tools/doc-generator/parse/parser.go | 2 + tools/doc-generator/parse/root_blocks.go | 9 +- 6 files changed, 151 insertions(+), 17 deletions(-) diff --git a/docs/sources/configure/_index.md b/docs/sources/configure/_index.md index 25e4f70f987c3..b675f85157423 100644 --- a/docs/sources/configure/_index.md +++ b/docs/sources/configure/_index.md @@ -3143,14 +3143,22 @@ shard_streams: # OTLP log ingestion configurations otlp_config: + # Configuration for resource attributes to store them as index labels or + # Structured Metadata or drop them altogether resource_attributes: - [ignore_defaults: ] + # Configure whether to ignore the default list of resource attributes to be + # stored as index labels and only use the given resource attributes config + [ignore_defaults: | default = false] - [attributes: ] + [attributes_config: ] - [scope_attributes: ] + # Configuration for scope attributes to store them as Structured Metadata or + # drop them altogether + [scope_attributes: ] - [log_attributes: ] + # Configuration for log attributes to store them as Structured Metadata or + # drop them altogether + [log_attributes: ] ``` ### frontend_worker @@ -5292,6 +5300,24 @@ Named store from this example can be used by setting object_store to store-1 in [cos: ] ``` +### attributes_config + +Define actions for matching OpenTelemetry (OTEL) attributes. + +```yaml +# Configures action to take on matching attributes. It allows one of +# [structured_metadata, drop] for all attribute types. It additionally allows +# index_label action for resource attributes +[action: | default = ""] + +# List of attributes to configure how to store them or drop them altogether +[attributes: ] + +# Regex to choose attributes to configure how to store them or drop them +# altogether +[regex: ] +``` + ## Runtime Configuration file Loki has a concept of "runtime config" file, which is simply a file that is reloaded while Loki is running. It is used by some Loki components to allow operator to change some aspects of Loki configuration without restarting it. File is specified by using `-runtime-config.file=` flag and reload period (which defaults to 10 seconds) can be changed by `-runtime-config.reload-period=` flag. Previously this mechanism was only used by limits overrides, and flags were called `-limits.per-user-override-config=` and `-limits.per-user-override-period=10s` respectively. These are still used, if `-runtime-config.file=` is not specified. diff --git a/docs/sources/send-data/otel/_index.md b/docs/sources/send-data/otel/_index.md index 915d17f75ab0c..84d1226316ecf 100644 --- a/docs/sources/send-data/otel/_index.md +++ b/docs/sources/send-data/otel/_index.md @@ -69,7 +69,7 @@ service: ## Format considerations -Since the OpenTelemetry protocol differs from the Loki storage model, here is how data in the OpenTelemetry format will be mapped to the Loki data model during ingestion: +Since the OpenTelemetry protocol differs from the Loki storage model, here is how data in the OpenTelemetry format will be mapped by default to the Loki data model during ingestion, which can be changed as explained later: - Index labels: Resource attributes map well to index labels in Loki, since both usually identify the source of the logs. Because Loki has a limit of 30 index labels, we have selected the following resource attributes to be stored as index labels, while the remaining attributes are stored as [Structured Metadata]({{< relref "../../get-started/labels/structured-metadata" >}}) with each log entry: - cloud.availability_zone @@ -116,3 +116,102 @@ Things to note before ingesting OpenTelemetry logs to Loki: - Stringification of non-string Attribute values While converting Attribute values in OTLP to Index label values or Structured Metadata, any non-string values are converted to string using [AsString method from the OTEL collector lib](https://github.com/open-telemetry/opentelemetry-collector/blob/ab3d6c5b64701e690aaa340b0a63f443ff22c1f0/pdata/pcommon/value.go#L353). + +### Changing the default mapping of OTLP to Loki Format + +Loki supports [per tenant]({{< relref "../../configure#limits_config" >}}) OTLP config which lets you change the default mapping of OTLP to Loki format for each tenant. +It currently only supports changing the storage of Attributes. Here is how the config looks like: + +```yaml +# OTLP log ingestion configurations +otlp_config: + # Configuration for Resource Attributes to store them as index labels or + # Structured Metadata or drop them altogether + resource_attributes: + # Configure whether to ignore the default list of Resource Attributes to be + # stored as Index Labels and only use the given Resource Attributes config + [ignore_defaults: ] + + [attributes_config: ] + + # Configuration for Scope Attributes to store them as Structured Metadata or + # drop them altogether + [scope_attributes: ] + + # Configuration for Log Attributes to store them as Structured Metadata or + # drop them altogether + [log_attributes: ] + +attributes_config: + # Configures action to take on matching Attributes. It allows one of + # [structured_metadata, drop] for all Attribute types. It additionally allows + # index_label action for Resource Attributes + [action: | default = ""] + + # List of attributes to configure how to store them or drop them altogether + [attributes: ] + + # Regex to choose attributes to configure how to store them or drop them + # altogether + [regex: ] +``` + +Here are some example configs to change the default mapping of OTLP to Loki format: + +#### Example 1: + +```yaml +otlp_config: + resource_attributes: + attributes_config: + - action: index_label + attributes: + - service.group +``` + +With the example config, here is how various kinds of Attributes would be stored: +* Store all 17 Resource Attributes mentioned earlier and `service.group` Resource Attribute as index labels. +* Store remaining Resource Attributes as Structured Metadata. +* Store all the Scope and Log Attributes as Structured Metadata. + +#### Example 2: + +```yaml +otlp_config: + resource_attributes: + ignore_defaults: true + attributes_config: + - action: index_label + regex: service.group +``` + +With the example config, here is how various kinds of Attributes would be stored: +* **Only** store `service.group` Resource Attribute as index labels. +* Store remaining Resource Attributes as Structured Metadata. +* Store all the Scope and Log Attributes as Structured Metadata. + +#### Example 2: + +```yaml +otlp_config: + resource_attributes: + attributes_config: + - action: index_label + regex: service.group + scope_attributes: + - action: drop + attributes: + - method.name + log_attributes: + - action: structured_metadata + attributes: + - user.id + - action: drop + regex: .* +``` + +With the example config, here is how various kinds of Attributes would be stored: +* Store all 17 Resource Attributes mentioned earlier and `service.group` Resource Attribute as index labels. +* Store remaining Resource Attributes as Structured Metadata. +* Drop Scope Attribute named `method.name` and store all other Scope Attributes as Structured Metadata. +* Store Log Attribute named `user.id` as Structured Metadata and drop all other Log Attributes. \ No newline at end of file diff --git a/pkg/loghttp/push/otlp_config.go b/pkg/loghttp/push/otlp_config.go index 64120d4a6252e..44c0e932f9c12 100644 --- a/pkg/loghttp/push/otlp_config.go +++ b/pkg/loghttp/push/otlp_config.go @@ -56,9 +56,9 @@ var DefaultOTLPConfig = OTLPConfig{ } type OTLPConfig struct { - ResourceAttributes ResourceAttributesConfig `yaml:"resource_attributes,omitempty"` - ScopeAttributes []AttributesConfig `yaml:"scope_attributes,omitempty"` - LogAttributes []AttributesConfig `yaml:"log_attributes,omitempty"` + ResourceAttributes ResourceAttributesConfig `yaml:"resource_attributes,omitempty" doc:"description=Configuration for resource attributes to store them as index labels or Structured Metadata or drop them altogether"` + ScopeAttributes []AttributesConfig `yaml:"scope_attributes,omitempty" doc:"description=Configuration for scope attributes to store them as Structured Metadata or drop them altogether"` + LogAttributes []AttributesConfig `yaml:"log_attributes,omitempty" doc:"description=Configuration for log attributes to store them as Structured Metadata or drop them altogether"` } func (c *OTLPConfig) UnmarshalYAML(unmarshal func(interface{}) error) error { @@ -115,9 +115,9 @@ func (c *OTLPConfig) Validate() error { } type AttributesConfig struct { - Action Action `yaml:"action,omitempty"` - Attributes []string `yaml:"attributes,omitempty"` - Regex relabel.Regexp `yaml:"regex,omitempty"` + Action Action `yaml:"action,omitempty" doc:"description=Configures action to take on matching attributes. It allows one of [structured_metadata, drop] for all attribute types. It additionally allows index_label action for resource attributes"` + Attributes []string `yaml:"attributes,omitempty" doc:"description=List of attributes to configure how to store them or drop them altogether"` + Regex relabel.Regexp `yaml:"regex,omitempty" doc:"description=Regex to choose attributes to configure how to store them or drop them altogether"` } func (c *AttributesConfig) UnmarshalYAML(unmarshal func(interface{}) error) error { @@ -146,8 +146,8 @@ func (c *AttributesConfig) UnmarshalYAML(unmarshal func(interface{}) error) erro } type ResourceAttributesConfig struct { - IgnoreDefaults bool `yaml:"ignore_defaults,omitempty"` - AttributesConfig []AttributesConfig `yaml:"attributes,omitempty"` + IgnoreDefaults bool `yaml:"ignore_defaults,omitempty" doc:"default=false|description=Configure whether to ignore the default list of resource attributes to be stored as index labels and only use the given resource attributes config"` + AttributesConfig []AttributesConfig `yaml:"attributes_config,omitempty"` } func (c *ResourceAttributesConfig) UnmarshalYAML(unmarshal func(interface{}) error) error { diff --git a/pkg/loghttp/push/otlp_config_test.go b/pkg/loghttp/push/otlp_config_test.go index a1cfc15ff52c8..5fa6251628507 100644 --- a/pkg/loghttp/push/otlp_config_test.go +++ b/pkg/loghttp/push/otlp_config_test.go @@ -19,7 +19,7 @@ func TestUnmarshalOTLPConfig(t *testing.T) { name: "only resource_attributes set", yamlConfig: []byte(` resource_attributes: - attributes: + attributes_config: - action: index_label regex: foo`), expectedCfg: OTLPConfig{ @@ -39,7 +39,7 @@ resource_attributes: yamlConfig: []byte(` resource_attributes: ignore_defaults: true - attributes: + attributes_config: - action: index_label regex: foo`), expectedCfg: OTLPConfig{ @@ -82,7 +82,7 @@ scope_attributes: name: "all 3 set", yamlConfig: []byte(` resource_attributes: - attributes: + attributes_config: - action: index_label regex: foo scope_attributes: diff --git a/tools/doc-generator/parse/parser.go b/tools/doc-generator/parse/parser.go index 1003c1e08753d..d5896a0666bf5 100644 --- a/tools/doc-generator/parse/parser.go +++ b/tools/doc-generator/parse/parser.go @@ -462,6 +462,8 @@ func getCustomFieldType(t reflect.Type) (string, bool) { return "remote_write_config...", true case reflect.TypeOf(validation.OverwriteMarshalingStringMap{}).String(): return "headers", true + case reflect.TypeOf(relabel.Regexp{}).String(): + return fieldString, true default: return "", false } diff --git a/tools/doc-generator/parse/root_blocks.go b/tools/doc-generator/parse/root_blocks.go index 66a7a72321bb3..25a4e785ded63 100644 --- a/tools/doc-generator/parse/root_blocks.go +++ b/tools/doc-generator/parse/root_blocks.go @@ -20,6 +20,7 @@ import ( "github.com/grafana/loki/pkg/distributor" "github.com/grafana/loki/pkg/ingester" ingester_client "github.com/grafana/loki/pkg/ingester/client" + "github.com/grafana/loki/pkg/loghttp/push" "github.com/grafana/loki/pkg/loki/common" frontend "github.com/grafana/loki/pkg/lokifrontend" "github.com/grafana/loki/pkg/querier" @@ -272,6 +273,12 @@ storage_config: store-1: endpoint: s3://foo-bucket region: us-west1 -Named store from this example can be used by setting object_store to store-1 in period_config.`}, +Named store from this example can be used by setting object_store to store-1 in period_config.`, + }, + { + Name: "attributes_config", + StructType: []reflect.Type{reflect.TypeOf(push.AttributesConfig{})}, + Desc: "Define actions for matching OpenTelemetry (OTEL) attributes.", + }, } ) From 69919a303bc99f943f4fbbdda9e6fb255b17490a Mon Sep 17 00:00:00 2001 From: Ashwanth Date: Mon, 5 Feb 2024 16:02:26 +0530 Subject: [PATCH 27/45] ruler: add filtering support for prometheus rules api (#11817) **What this PR does / why we need it**: Adds `rule_name`, `rule_group`, `file` and `type` query parameters for filtering the response of `/prometheus/api/v1/rules` endpoint. Replicates mimir's functionality: https://github.com/grafana/mimir/pull/5291 - all of them are optional. - `type` paremeter accepts either `alert` or `record` - `rule_name`, `rule_group`, `file` can accept multiple values and they filter the response accordingly. There is a minor change in behavior: `/prometheus/api/v1/rules` endpoint will no longer return empty rule groups which is inline with both [prometheus](https://github.com/prometheus/prometheus/pull/12270/files#diff-315f251cdd7e93fcec1e7e9505744da1d1828f30d2b61d1f4ce963fa26bf1909R1403) and [mimir](https://github.com/grafana/mimir/pull/5291/files#diff-e5424c21c0e827bd1c9d3f669ed605897696bdc27993bc8bfd7113eba787b49dR1120). This is not a breaking change since rule groups with [no rules fail validation](https://github.com/grafana/loki/blob/27fbd62505f4412e3cb9180b1a5a66518bba9752/pkg/ruler/base/manager.go#L295) and cannot be created. **Which issue(s) this PR fixes**: Fixes https://github.com/grafana/loki/issues/9295 **Special notes for your reviewer**: **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [x] Documentation added - [x] 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` - [ ] 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) --- CHANGELOG.md | 1 + docs/sources/reference/api.md | 6 +- pkg/ruler/base/api.go | 68 ++-- pkg/ruler/base/api_test.go | 567 ++++++++++++++++++++++++++++++---- pkg/ruler/base/ruler.go | 82 ++++- pkg/ruler/base/ruler.pb.go | 367 +++++++++++++++++++--- pkg/ruler/base/ruler.proto | 12 +- pkg/ruler/base/ruler_test.go | 269 ++++++++++------ 8 files changed, 1126 insertions(+), 246 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 37599ae8d347c..5e5adea5d2a02 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -53,6 +53,7 @@ * [11679](https://github.com/grafana/loki/pull/11679) **dannykopping** Cache: extending #11535 to align custom ingester query split with cache keys for correct caching of results. * [11143](https://github.com/grafana/loki/pull/11143) **sandeepsukhani** otel: Add support for per tenant configuration for mapping otlp data to loki format * [11499](https://github.com/grafana/loki/pull/11284) **jmichalek132** Config: Adds `frontend.log-query-request-headers` to enable logging of request headers in query logs. +* [11817](https://github.com/grafana/loki/pull/11817) **ashwanthgoli** Ruler: Add support for filtering results of `/prometheus/api/v1/rules` endpoint by rule_name, rule_group, file and type. ##### Fixes * [11074](https://github.com/grafana/loki/pull/11074) **hainenber** Fix panic in lambda-promtail due to mishandling of empty DROP_LABELS env var. diff --git a/docs/sources/reference/api.md b/docs/sources/reference/api.md index 2e48e178534d2..cf384859c6a71 100644 --- a/docs/sources/reference/api.md +++ b/docs/sources/reference/api.md @@ -1178,11 +1178,15 @@ Deletes all the rule groups in a namespace (including the namespace itself). Thi ### List rules ``` -GET /prometheus/api/v1/rules +GET /prometheus/api/v1/rules?type={alert|record}&file={}&rule_group={}&rule_name={} ``` Prometheus-compatible rules endpoint to list alerting and recording rules that are currently loaded. +The `type` parameter is optional. If set, only the specified type of rule is returned. + +The `file`, `rule_group` and `rule_name` parameters are optional, and can accept multiple values. If set, the response content is filtered accordingly. + For more information, refer to the [Prometheus rules](https://prometheus.io/docs/prometheus/latest/querying/api/#rules) documentation. ### List alerts diff --git a/pkg/ruler/base/api.go b/pkg/ruler/base/api.go index 53fb3e457460c..04a303993228b 100644 --- a/pkg/ruler/base/api.go +++ b/pkg/ruler/base/api.go @@ -2,6 +2,7 @@ package base import ( "encoding/json" + "fmt" "io" "net/http" "net/url" @@ -101,10 +102,10 @@ type recordingRule struct { EvaluationTime float64 `json:"evaluationTime"` } -func respondError(logger log.Logger, w http.ResponseWriter, msg string) { +func respondError(logger log.Logger, w http.ResponseWriter, status int, errorType v1.ErrorType, msg string) { b, err := json.Marshal(&response{ Status: "error", - ErrorType: v1.ErrServer, + ErrorType: errorType, Error: msg, Data: nil, }) @@ -115,12 +116,20 @@ func respondError(logger log.Logger, w http.ResponseWriter, msg string) { return } - w.WriteHeader(http.StatusInternalServerError) + w.WriteHeader(status) if n, err := w.Write(b); err != nil { level.Error(logger).Log("msg", "error writing response", "bytesWritten", n, "err", err) } } +func respondInvalidRequest(logger log.Logger, w http.ResponseWriter, msg string) { + respondError(logger, w, http.StatusBadRequest, v1.ErrBadData, msg) +} + +func respondServerError(logger log.Logger, w http.ResponseWriter, msg string) { + respondError(logger, w, http.StatusInternalServerError, v1.ErrServer, msg) +} + // API is used to handle HTTP requests for the ruler service type API struct { ruler *Ruler @@ -143,15 +152,34 @@ func (a *API) PrometheusRules(w http.ResponseWriter, req *http.Request) { userID, err := tenant.TenantID(req.Context()) if err != nil || userID == "" { level.Error(logger).Log("msg", "error extracting org id from context", "err", err) - respondError(logger, w, "no valid org id found") + respondServerError(logger, w, "no valid org id found") return } - w.Header().Set("Content-Type", "application/json") - rgs, err := a.ruler.GetRules(req.Context()) + var rulesReq = RulesRequest{ + Filter: AnyRule, + RuleName: req.URL.Query()["rule_name"], + RuleGroup: req.URL.Query()["rule_group"], + File: req.URL.Query()["file"], + } + + ruleTypeFilter := strings.ToLower(req.URL.Query().Get("type")) + if ruleTypeFilter != "" { + switch ruleTypeFilter { + case "alert": + rulesReq.Filter = AlertingRule + case "record": + rulesReq.Filter = RecordingRule + default: + respondInvalidRequest(logger, w, fmt.Sprintf("not supported value %q", ruleTypeFilter)) + return + } + } + + rgs, err := a.ruler.GetRules(req.Context(), &rulesReq) if err != nil { - respondError(logger, w, err.Error()) + respondServerError(logger, w, err.Error()) return } @@ -221,7 +249,7 @@ func (a *API) PrometheusRules(w http.ResponseWriter, req *http.Request) { }) if err != nil { level.Error(logger).Log("msg", "error marshaling json response", "err", err) - respondError(logger, w, "unable to marshal the requested data") + respondServerError(logger, w, "unable to marshal the requested data") return } w.Header().Set("Content-Type", "application/json") @@ -236,15 +264,15 @@ func (a *API) PrometheusAlerts(w http.ResponseWriter, req *http.Request) { userID, err := tenant.TenantID(req.Context()) if err != nil || userID == "" { level.Error(logger).Log("msg", "error extracting org id from context", "err", err) - respondError(logger, w, "no valid org id found") + respondServerError(logger, w, "no valid org id found") return } w.Header().Set("Content-Type", "application/json") - rgs, err := a.ruler.GetRules(req.Context()) + rgs, err := a.ruler.GetRules(req.Context(), &RulesRequest{Filter: AlertingRule}) if err != nil { - respondError(logger, w, err.Error()) + respondServerError(logger, w, err.Error()) return } @@ -272,7 +300,7 @@ func (a *API) PrometheusAlerts(w http.ResponseWriter, req *http.Request) { }) if err != nil { level.Error(logger).Log("msg", "error marshaling json response", "err", err) - respondError(logger, w, "unable to marshal the requested data") + respondServerError(logger, w, "unable to marshal the requested data") return } w.Header().Set("Content-Type", "application/json") @@ -314,7 +342,7 @@ func respondAccepted(w http.ResponseWriter, logger log.Logger) { }) if err != nil { level.Error(logger).Log("msg", "error marshaling json response", "err", err) - respondError(logger, w, "unable to marshal the requested data") + respondServerError(logger, w, "unable to marshal the requested data") return } w.Header().Set("Content-Type", "application/json") @@ -466,7 +494,7 @@ func (a *API) ListRules(w http.ResponseWriter, req *http.Request) { pr, err := parseRequest(req, false, false) if err != nil { - respondError(logger, w, err.Error()) + respondServerError(logger, w, err.Error()) return } @@ -504,7 +532,7 @@ func (a *API) GetRuleGroup(w http.ResponseWriter, req *http.Request) { logger := util_log.WithContext(req.Context(), a.logger) pr, err := parseRequest(req, true, true) if err != nil { - respondError(logger, w, err.Error()) + respondServerError(logger, w, err.Error()) return } @@ -526,7 +554,7 @@ func (a *API) CreateRuleGroup(w http.ResponseWriter, req *http.Request) { logger := util_log.WithContext(req.Context(), a.logger) pr, err := parseRequest(req, true, false) if err != nil { - respondError(logger, w, err.Error()) + respondServerError(logger, w, err.Error()) return } @@ -600,7 +628,7 @@ func (a *API) DeleteNamespace(w http.ResponseWriter, req *http.Request) { pr, err := parseRequest(req, true, false) if err != nil { - respondError(logger, w, err.Error()) + respondServerError(logger, w, err.Error()) return } @@ -610,7 +638,7 @@ func (a *API) DeleteNamespace(w http.ResponseWriter, req *http.Request) { http.Error(w, err.Error(), http.StatusNotFound) return } - respondError(logger, w, err.Error()) + respondServerError(logger, w, err.Error()) return } @@ -622,7 +650,7 @@ func (a *API) DeleteRuleGroup(w http.ResponseWriter, req *http.Request) { pr, err := parseRequest(req, true, true) if err != nil { - respondError(logger, w, err.Error()) + respondServerError(logger, w, err.Error()) return } @@ -632,7 +660,7 @@ func (a *API) DeleteRuleGroup(w http.ResponseWriter, req *http.Request) { http.Error(w, err.Error(), http.StatusNotFound) return } - respondError(logger, w, err.Error()) + respondServerError(logger, w, err.Error()) return } diff --git a/pkg/ruler/base/api_test.go b/pkg/ruler/base/api_test.go index 9f0e7b46cbd79..c14f5de8d4614 100644 --- a/pkg/ruler/base/api_test.go +++ b/pkg/ruler/base/api_test.go @@ -8,6 +8,7 @@ import ( "io" "net/http" "net/http/httptest" + "net/url" "strings" "testing" "time" @@ -16,54 +17,102 @@ import ( "github.com/gorilla/mux" "github.com/grafana/dskit/services" "github.com/grafana/dskit/user" + v1 "github.com/prometheus/client_golang/api/prometheus/v1" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/rulefmt" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "gopkg.in/yaml.v3" "github.com/grafana/loki/pkg/ruler/rulespb" ) -func TestRuler_rules(t *testing.T) { - cfg := defaultRulerConfig(t, newMockRuleStore(mockRules)) - - r := newTestRuler(t, cfg) - defer services.StopAndAwaitTerminated(context.Background(), r) //nolint:errcheck +func TestRuler_PrometheusRules(t *testing.T) { + const ( + userID = "user1" + interval = time.Minute + ) - a := NewAPI(r, r.store, log.NewNopLogger()) + groupName := func(group int) string { + return fmt.Sprintf("group%d+", group) + } - req := requestFor(t, "GET", "https://localhost:8080/api/prom/api/v1/rules", nil, "user1") - w := httptest.NewRecorder() - a.PrometheusRules(w, req) + namespaceName := func(ns int) string { + return fmt.Sprintf("namespace%d+", ns) + } - resp := w.Result() - body, _ := io.ReadAll(resp.Body) + makeFilterTestRules := func() rulespb.RuleGroupList { + result := rulespb.RuleGroupList{} + for ns := 1; ns <= 3; ns++ { + for group := 1; group <= 3; group++ { + g := &rulespb.RuleGroupDesc{ + Name: groupName(group), + Namespace: namespaceName(ns), + User: userID, + Rules: []*rulespb.RuleDesc{ + createRecordingRule("NonUniqueNamedRule", `count_over_time({foo="bar"}[5m])`), + createAlertingRule(fmt.Sprintf("UniqueNamedRuleN%dG%d", ns, group), `count_over_time({foo="bar"}[5m]) < 1`), + }, + Interval: interval, + } + result = append(result, g) + } + } + return result + } - // Check status code and status response - responseJSON := response{} - err := json.Unmarshal(body, &responseJSON) - require.NoError(t, err) - require.Equal(t, http.StatusOK, resp.StatusCode) - require.Equal(t, responseJSON.Status, "success") + filterTestExpectedRule := func(name string) *recordingRule { + return &recordingRule{ + Name: name, + Query: `count_over_time({foo="bar"}[5m])`, + Health: "unknown", + Type: "recording", + } + } + filterTestExpectedAlert := func(name string) *alertingRule { + return &alertingRule{ + Name: name, + Query: `count_over_time({foo="bar"}[5m]) < 1`, + State: "inactive", + Health: "unknown", + Type: "alerting", + Alerts: []*Alert{}, + } + } - // Testing the running rules for user1 in the mock store - expectedResponse, _ := json.Marshal(response{ - Status: "success", - Data: &RuleDiscovery{ - RuleGroups: []*RuleGroup{ + testCases := map[string]struct { + configuredRules rulespb.RuleGroupList + expectedConfigured int + expectedStatusCode int + expectedErrorType v1.ErrorType + expectedRules []*RuleGroup + queryParams string + }{ + "should load and evaluate the configured rules": { + configuredRules: rulespb.RuleGroupList{ + &rulespb.RuleGroupDesc{ + Name: "group1", + Namespace: "namespace1", + User: userID, + Rules: []*rulespb.RuleDesc{createRecordingRule("COUNT_RULE", `count_over_time({foo="bar"}[5m])`), createAlertingRule("COUNT_ALERT", `count_over_time({foo="bar"}[5m]) < 1`)}, + Interval: interval, + }, + }, + expectedConfigured: 1, + expectedRules: []*RuleGroup{ { Name: "group1", File: "namespace1", Rules: []rule{ &recordingRule{ - Name: "UP_RULE", - Query: "up", + Name: "COUNT_RULE", + Query: `count_over_time({foo="bar"}[5m])`, Health: "unknown", Type: "recording", }, &alertingRule{ - Name: "UP_ALERT", - Query: "up < 1", + Name: "COUNT_ALERT", + Query: `count_over_time({foo="bar"}[5m]) < 1`, State: "inactive", Health: "unknown", Type: "alerting", @@ -71,55 +120,34 @@ func TestRuler_rules(t *testing.T) { }, }, Interval: 60, - Limit: 10, }, }, }, - }) - - require.Equal(t, string(expectedResponse), string(body)) -} - -func TestRuler_rules_special_characters(t *testing.T) { - cfg := defaultRulerConfig(t, newMockRuleStore(mockSpecialCharRules)) - - r := newTestRuler(t, cfg) - defer services.StopAndAwaitTerminated(context.Background(), r) //nolint:errcheck - - a := NewAPI(r, r.store, log.NewNopLogger()) - - req := requestFor(t, http.MethodGet, "https://localhost:8080/api/prom/api/v1/rules", nil, "user1") - w := httptest.NewRecorder() - a.PrometheusRules(w, req) - - resp := w.Result() - body, _ := io.ReadAll(resp.Body) - - // Check status code and status response - responseJSON := response{} - err := json.Unmarshal(body, &responseJSON) - require.NoError(t, err) - require.Equal(t, http.StatusOK, resp.StatusCode) - require.Equal(t, responseJSON.Status, "success") - - // Testing the running rules for user1 in the mock store - expectedResponse, _ := json.Marshal(response{ - Status: "success", - Data: &RuleDiscovery{ - RuleGroups: []*RuleGroup{ + "should load and evaluate rule groups and namespaces with special characters": { + configuredRules: rulespb.RuleGroupList{ + &rulespb.RuleGroupDesc{ + Name: ")(_+?/|group1+/?", + Namespace: ")(_+?/|namespace1+/?", + User: userID, + Rules: []*rulespb.RuleDesc{createRecordingRule("COUNT_RULE", `count_over_time({foo="bar"}[5m])`), createAlertingRule("COUNT_ALERT", `count_over_time({foo="bar"}[5m]) < 1`)}, + Interval: interval, + }, + }, + expectedConfigured: 1, + expectedRules: []*RuleGroup{ { Name: ")(_+?/|group1+/?", File: ")(_+?/|namespace1+/?", Rules: []rule{ &recordingRule{ - Name: "UP_RULE", - Query: "up", + Name: "COUNT_RULE", + Query: `count_over_time({foo="bar"}[5m])`, Health: "unknown", Type: "recording", }, &alertingRule{ - Name: "UP_ALERT", - Query: "up < 1", + Name: "COUNT_ALERT", + Query: `count_over_time({foo="bar"}[5m]) < 1`, State: "inactive", Health: "unknown", Type: "alerting", @@ -127,16 +155,407 @@ func TestRuler_rules_special_characters(t *testing.T) { }, }, Interval: 60, - Limit: 10, }, }, }, - }) + "API returns only alerts": { + configuredRules: rulespb.RuleGroupList{ + &rulespb.RuleGroupDesc{ + Name: "group1", + Namespace: "namespace1", + User: userID, + Rules: []*rulespb.RuleDesc{createRecordingRule("COUNT_RULE", `count_over_time({foo="bar"}[5m])`), createAlertingRule("COUNT_ALERT", `count_over_time({foo="bar"}[5m]) < 1`)}, + Interval: interval, + }, + }, + expectedConfigured: 1, + queryParams: "?type=alert", + expectedRules: []*RuleGroup{ + { + Name: "group1", + File: "namespace1", + Rules: []rule{ + &alertingRule{ + Name: "COUNT_ALERT", + Query: `count_over_time({foo="bar"}[5m]) < 1`, + State: "inactive", + Health: "unknown", + Type: "alerting", + Alerts: []*Alert{}, + }, + }, + Interval: 60, + }, + }, + }, + "API returns only rules": { + configuredRules: rulespb.RuleGroupList{ + &rulespb.RuleGroupDesc{ + Name: "group1", + Namespace: "namespace1", + User: userID, + Rules: []*rulespb.RuleDesc{createRecordingRule("COUNT_RULE", `count_over_time({foo="bar"}[5m])`), createAlertingRule("COUNT_ALERT", `count_over_time({foo="bar"}[5m]) < 1`)}, + Interval: interval, + }, + }, + expectedConfigured: 1, + queryParams: "?type=record", + expectedRules: []*RuleGroup{ + { + Name: "group1", + File: "namespace1", + Rules: []rule{ + &recordingRule{ + Name: "COUNT_RULE", + Query: `count_over_time({foo="bar"}[5m])`, + Health: "unknown", + Type: "recording", + }, + }, + Interval: 60, + }, + }, + }, + "Invalid type param": { + configuredRules: rulespb.RuleGroupList{}, + expectedConfigured: 0, + queryParams: "?type=foo", + expectedStatusCode: http.StatusBadRequest, + expectedErrorType: v1.ErrBadData, + expectedRules: []*RuleGroup{}, + }, + "when filtering by an unknown namespace then the API returns nothing": { + configuredRules: makeFilterTestRules(), + expectedConfigured: len(makeFilterTestRules()), + queryParams: "?file=unknown", + expectedRules: []*RuleGroup{}, + }, + "when filtering by a single known namespace then the API returns only rules from that namespace": { + configuredRules: makeFilterTestRules(), + expectedConfigured: len(makeFilterTestRules()), + queryParams: "?" + url.Values{"file": []string{namespaceName(1)}}.Encode(), + expectedRules: []*RuleGroup{ + { + Name: groupName(1), + File: namespaceName(1), + Rules: []rule{ + filterTestExpectedRule("NonUniqueNamedRule"), + filterTestExpectedAlert("UniqueNamedRuleN1G1"), + }, + Interval: 60, + }, + { + Name: groupName(2), + File: namespaceName(1), + Rules: []rule{ + filterTestExpectedRule("NonUniqueNamedRule"), + filterTestExpectedAlert("UniqueNamedRuleN1G2"), + }, + Interval: 60, + }, + { + Name: groupName(3), + File: namespaceName(1), + Rules: []rule{ + filterTestExpectedRule("NonUniqueNamedRule"), + filterTestExpectedAlert("UniqueNamedRuleN1G3"), + }, + Interval: 60, + }, + }, + }, + "when filtering by a multiple known namespaces then the API returns rules from both namespaces": { + configuredRules: makeFilterTestRules(), + expectedConfigured: len(makeFilterTestRules()), + queryParams: "?" + url.Values{"file": []string{namespaceName(1), namespaceName(2)}}.Encode(), + expectedRules: []*RuleGroup{ + { + Name: groupName(1), + File: namespaceName(1), + Rules: []rule{ + filterTestExpectedRule("NonUniqueNamedRule"), + filterTestExpectedAlert("UniqueNamedRuleN1G1"), + }, + Interval: 60, + }, + { + Name: groupName(2), + File: namespaceName(1), + Rules: []rule{ + filterTestExpectedRule("NonUniqueNamedRule"), + filterTestExpectedAlert("UniqueNamedRuleN1G2"), + }, + Interval: 60, + }, + { + Name: groupName(3), + File: namespaceName(1), + Rules: []rule{ + filterTestExpectedRule("NonUniqueNamedRule"), + filterTestExpectedAlert("UniqueNamedRuleN1G3"), + }, + Interval: 60, + }, + { + Name: groupName(1), + File: namespaceName(2), + Rules: []rule{ + filterTestExpectedRule("NonUniqueNamedRule"), + filterTestExpectedAlert("UniqueNamedRuleN2G1"), + }, + Interval: 60, + }, + { + Name: groupName(2), + File: namespaceName(2), + Rules: []rule{ + filterTestExpectedRule("NonUniqueNamedRule"), + filterTestExpectedAlert("UniqueNamedRuleN2G2"), + }, + Interval: 60, + }, + { + Name: groupName(3), + File: namespaceName(2), + Rules: []rule{ + filterTestExpectedRule("NonUniqueNamedRule"), + filterTestExpectedAlert("UniqueNamedRuleN2G3"), + }, + Interval: 60, + }, + }, + }, + "when filtering by an unknown group then the API returns nothing": { + configuredRules: makeFilterTestRules(), + expectedConfigured: len(makeFilterTestRules()), + queryParams: "?rule_group=unknown", + expectedRules: []*RuleGroup{}, + }, + "when filtering by a known group then the API returns only rules from that group": { + configuredRules: makeFilterTestRules(), + expectedConfigured: len(makeFilterTestRules()), + queryParams: "?" + url.Values{"rule_group": []string{groupName(2)}}.Encode(), + expectedRules: []*RuleGroup{ + { + Name: groupName(2), + File: namespaceName(1), + Rules: []rule{ + filterTestExpectedRule("NonUniqueNamedRule"), + filterTestExpectedAlert("UniqueNamedRuleN1G2"), + }, + Interval: 60, + }, + { + Name: groupName(2), + File: namespaceName(2), + Rules: []rule{ + filterTestExpectedRule("NonUniqueNamedRule"), + filterTestExpectedAlert("UniqueNamedRuleN2G2"), + }, + Interval: 60, + }, + { + Name: groupName(2), + File: namespaceName(3), + Rules: []rule{ + filterTestExpectedRule("NonUniqueNamedRule"), + filterTestExpectedAlert("UniqueNamedRuleN3G2"), + }, + Interval: 60, + }, + }, + }, + "when filtering by multiple known groups then the API returns rules from both groups": { + configuredRules: makeFilterTestRules(), + expectedConfigured: len(makeFilterTestRules()), + queryParams: "?" + url.Values{"rule_group": []string{groupName(2), groupName(3)}}.Encode(), + expectedRules: []*RuleGroup{ + { + Name: groupName(2), + File: namespaceName(1), + Rules: []rule{ + filterTestExpectedRule("NonUniqueNamedRule"), + filterTestExpectedAlert("UniqueNamedRuleN1G2"), + }, + Interval: 60, + }, + { + Name: groupName(3), + File: namespaceName(1), + Rules: []rule{ + filterTestExpectedRule("NonUniqueNamedRule"), + filterTestExpectedAlert("UniqueNamedRuleN1G3"), + }, + Interval: 60, + }, + { + Name: groupName(2), + File: namespaceName(2), + Rules: []rule{ + filterTestExpectedRule("NonUniqueNamedRule"), + filterTestExpectedAlert("UniqueNamedRuleN2G2"), + }, + Interval: 60, + }, + { + Name: groupName(3), + File: namespaceName(2), + Rules: []rule{ + filterTestExpectedRule("NonUniqueNamedRule"), + filterTestExpectedAlert("UniqueNamedRuleN2G3"), + }, + Interval: 60, + }, + { + Name: groupName(2), + File: namespaceName(3), + Rules: []rule{ + filterTestExpectedRule("NonUniqueNamedRule"), + filterTestExpectedAlert("UniqueNamedRuleN3G2"), + }, + Interval: 60, + }, + { + Name: groupName(3), + File: namespaceName(3), + Rules: []rule{ + filterTestExpectedRule("NonUniqueNamedRule"), + filterTestExpectedAlert("UniqueNamedRuleN3G3"), + }, + Interval: 60, + }, + }, + }, - require.Equal(t, string(expectedResponse), string(body)) + "when filtering by an unknown rule name then the API returns all empty groups": { + configuredRules: makeFilterTestRules(), + expectedConfigured: len(makeFilterTestRules()), + queryParams: "?rule_name=unknown", + expectedRules: []*RuleGroup{}, + }, + "when filtering by a known rule name then the API returns only rules with that name": { + configuredRules: makeFilterTestRules(), + expectedConfigured: len(makeFilterTestRules()), + queryParams: "?" + url.Values{"rule_name": []string{"UniqueNamedRuleN1G2"}}.Encode(), + expectedRules: []*RuleGroup{ + { + Name: groupName(2), + File: namespaceName(1), + Rules: []rule{ + filterTestExpectedAlert("UniqueNamedRuleN1G2"), + }, + Interval: 60, + }, + }, + }, + "when filtering by multiple known rule names then the API returns both rules": { + configuredRules: makeFilterTestRules(), + expectedConfigured: len(makeFilterTestRules()), + queryParams: "?" + url.Values{"rule_name": []string{"UniqueNamedRuleN1G2", "UniqueNamedRuleN2G3"}}.Encode(), + expectedRules: []*RuleGroup{ + { + Name: groupName(2), + File: namespaceName(1), + Rules: []rule{ + filterTestExpectedAlert("UniqueNamedRuleN1G2"), + }, + Interval: 60, + }, + { + Name: groupName(3), + File: namespaceName(2), + Rules: []rule{ + filterTestExpectedAlert("UniqueNamedRuleN2G3"), + }, + Interval: 60, + }, + }, + }, + "when filtering by a known namespace and group then the API returns only rules from that namespace and group": { + configuredRules: makeFilterTestRules(), + expectedConfigured: len(makeFilterTestRules()), + queryParams: "?" + url.Values{ + "file": []string{namespaceName(3)}, + "rule_group": []string{groupName(2)}, + }.Encode(), + expectedRules: []*RuleGroup{ + { + Name: groupName(2), + File: namespaceName(3), + Rules: []rule{ + &recordingRule{ + Name: "NonUniqueNamedRule", + Query: `count_over_time({foo="bar"}[5m])`, + Health: "unknown", + Type: "recording", + }, + &alertingRule{ + Name: "UniqueNamedRuleN3G2", + Query: `count_over_time({foo="bar"}[5m]) < 1`, + State: "inactive", + Health: "unknown", + Type: "alerting", + Alerts: []*Alert{}, + }, + }, + Interval: 60, + }, + }, + }, + } + + for name, tc := range testCases { + t.Run(name, func(t *testing.T) { + storageRules := map[string]rulespb.RuleGroupList{ + userID: tc.configuredRules, + } + cfg := defaultRulerConfig(t, newMockRuleStore(storageRules)) + + r := newTestRuler(t, cfg) + defer services.StopAndAwaitTerminated(context.Background(), r) //nolint:errcheck + + a := NewAPI(r, r.store, log.NewNopLogger()) + + req := requestFor(t, "GET", "https://localhost:8080/api/prom/api/v1/rules"+tc.queryParams, nil, "user1") + w := httptest.NewRecorder() + a.PrometheusRules(w, req) + + resp := w.Result() + if tc.expectedStatusCode != 0 { + require.Equal(t, tc.expectedStatusCode, resp.StatusCode) + } else { + require.Equal(t, http.StatusOK, resp.StatusCode) + } + + body, _ := io.ReadAll(resp.Body) + + // Check status code and status response + responseJSON := response{} + err := json.Unmarshal(body, &responseJSON) + require.NoError(t, err) + + if tc.expectedErrorType != "" { + assert.Equal(t, "error", responseJSON.Status) + assert.Equal(t, tc.expectedErrorType, responseJSON.ErrorType) + return + } + require.Equal(t, responseJSON.Status, "success") + + // Testing the running rules + expectedResponse, err := json.Marshal(response{ + Status: "success", + Data: &RuleDiscovery{ + RuleGroups: tc.expectedRules, + }, + }) + + require.NoError(t, err) + require.Equal(t, string(expectedResponse), string(body)) + }) + } } -func TestRuler_alerts(t *testing.T) { +func TestRuler_PrometheusAlerts(t *testing.T) { cfg := defaultRulerConfig(t, newMockRuleStore(mockRules)) r := newTestRuler(t, cfg) @@ -593,3 +1012,17 @@ func requestFor(t *testing.T, method string, url string, body io.Reader, userID return req.WithContext(ctx) } + +func createRecordingRule(record, expr string) *rulespb.RuleDesc { + return &rulespb.RuleDesc{ + Record: record, + Expr: expr, + } +} + +func createAlertingRule(alert, expr string) *rulespb.RuleDesc { + return &rulespb.RuleDesc{ + Alert: alert, + Expr: expr, + } +} diff --git a/pkg/ruler/base/ruler.go b/pkg/ruler/base/ruler.go index eba29f0baed2e..a0dc0df08f2d7 100644 --- a/pkg/ruler/base/ruler.go +++ b/pkg/ruler/base/ruler.go @@ -799,26 +799,68 @@ func RemoveRuleTokenFromGroupName(name string) string { // GetRules retrieves the running rules from this ruler and all running rulers in the ring if // sharding is enabled -func (r *Ruler) GetRules(ctx context.Context) ([]*GroupStateDesc, error) { +func (r *Ruler) GetRules(ctx context.Context, req *RulesRequest) ([]*GroupStateDesc, error) { userID, err := tenant.TenantID(ctx) if err != nil { return nil, fmt.Errorf("no user id found in context") } if r.cfg.EnableSharding { - return r.getShardedRules(ctx, userID) + return r.getShardedRules(ctx, userID, req) } - return r.getLocalRules(userID) + return r.getLocalRules(userID, req) } -func (r *Ruler) getLocalRules(userID string) ([]*GroupStateDesc, error) { +type StringFilterSet map[string]struct{} + +func makeStringFilterSet(values []string) StringFilterSet { + set := make(map[string]struct{}, len(values)) + for _, v := range values { + set[v] = struct{}{} + } + return set +} + +// IsFiltered returns whether to filter the value or not. +// If the set is empty, then nothing is filtered. +func (fs StringFilterSet) IsFiltered(val string) bool { + if len(fs) == 0 { + return false + } + _, ok := fs[val] + return !ok +} + +func (r *Ruler) getLocalRules(userID string, req *RulesRequest) ([]*GroupStateDesc, error) { + var getRecordingRules, getAlertingRules bool + + switch req.Filter { + case AlertingRule: + getAlertingRules = true + case RecordingRule: + getRecordingRules = true + case AnyRule: + getAlertingRules = true + getRecordingRules = true + default: + return nil, fmt.Errorf("unexpected rule filter %s", req.Filter) + } + + fileSet := makeStringFilterSet(req.File) + groupSet := makeStringFilterSet(req.RuleGroup) + ruleSet := makeStringFilterSet(req.RuleName) + groups := r.manager.GetRules(userID) groupDescs := make([]*GroupStateDesc, 0, len(groups)) prefix := filepath.Join(r.cfg.RulePath, userID) + "/" for _, group := range groups { + if groupSet.IsFiltered(group.Name()) { + continue + } + interval := group.Interval() // The mapped filename is url path escaped encoded to make handling `/` characters easier @@ -827,6 +869,10 @@ func (r *Ruler) getLocalRules(userID string) ([]*GroupStateDesc, error) { return nil, errors.Wrap(err, "unable to decode rule filename") } + if fileSet.IsFiltered(decodedNamespace) { + continue + } + groupDesc := &GroupStateDesc{ Group: &rulespb.RuleGroupDesc{ Name: group.Name(), @@ -840,6 +886,10 @@ func (r *Ruler) getLocalRules(userID string) ([]*GroupStateDesc, error) { EvaluationDuration: group.GetEvaluationTime(), } for _, r := range group.Rules() { + if ruleSet.IsFiltered(r.Name()) { + continue + } + lastError := "" if r.LastError() != nil { lastError = r.LastError().Error() @@ -848,6 +898,10 @@ func (r *Ruler) getLocalRules(userID string) ([]*GroupStateDesc, error) { var ruleDesc *RuleStateDesc switch rule := r.(type) { case *promRules.AlertingRule: + if !getAlertingRules { + continue + } + rule.ActiveAlerts() alerts := []*AlertStateDesc{} for _, a := range rule.ActiveAlerts() { @@ -879,6 +933,10 @@ func (r *Ruler) getLocalRules(userID string) ([]*GroupStateDesc, error) { EvaluationDuration: rule.GetEvaluationDuration(), } case *promRules.RecordingRule: + if !getRecordingRules { + continue + } + ruleDesc = &RuleStateDesc{ Rule: &rulespb.RuleDesc{ Record: rule.Name(), @@ -895,12 +953,16 @@ func (r *Ruler) getLocalRules(userID string) ([]*GroupStateDesc, error) { } groupDesc.ActiveRules = append(groupDesc.ActiveRules, ruleDesc) } - groupDescs = append(groupDescs, groupDesc) + + // Prometheus does not return a rule group if it has no rules after filtering. + if len(groupDesc.ActiveRules) > 0 { + groupDescs = append(groupDescs, groupDesc) + } } return groupDescs, nil } -func (r *Ruler) getShardedRules(ctx context.Context, userID string) ([]*GroupStateDesc, error) { +func (r *Ruler) getShardedRules(ctx context.Context, userID string, rulesReq *RulesRequest) ([]*GroupStateDesc, error) { ring := ring.ReadRing(r.ring) if shardSize := r.limits.RulerTenantShardSize(userID); shardSize > 0 && r.cfg.ShardingStrategy == util.ShardingStrategyShuffle { @@ -933,7 +995,7 @@ func (r *Ruler) getShardedRules(ctx context.Context, userID string) ([]*GroupSta return errors.Wrapf(err, "unable to get client for ruler %s", addr) } - newGrps, err := rulerClient.Rules(ctx, &RulesRequest{}) + newGrps, err := rulerClient.Rules(ctx, rulesReq) if err != nil || newGrps == nil { return fmt.Errorf("unable to retrieve rules from ruler %s: %w", addr, err) } @@ -975,13 +1037,13 @@ func (r *Ruler) getShardedRules(ctx context.Context, userID string) ([]*GroupSta } // Rules implements the rules service -func (r *Ruler) Rules(ctx context.Context, _ *RulesRequest) (*RulesResponse, error) { +func (r *Ruler) Rules(ctx context.Context, req *RulesRequest) (*RulesResponse, error) { userID, err := tenant.TenantID(ctx) if err != nil { return nil, fmt.Errorf("no user id found in context") } - groupDescs, err := r.getLocalRules(userID) + groupDescs, err := r.getLocalRules(userID, req) if err != nil { return nil, err } @@ -1033,7 +1095,7 @@ func (r *Ruler) DeleteTenantConfiguration(w http.ResponseWriter, req *http.Reque err = r.store.DeleteNamespace(req.Context(), userID, "") // Empty namespace = delete all rule groups. if err != nil && !errors.Is(err, rulestore.ErrGroupNamespaceNotFound) { - respondError(logger, w, err.Error()) + respondServerError(logger, w, err.Error()) return } diff --git a/pkg/ruler/base/ruler.pb.go b/pkg/ruler/base/ruler.pb.go index b53b55afc96ee..63904a26e3927 100644 --- a/pkg/ruler/base/ruler.pb.go +++ b/pkg/ruler/base/ruler.pb.go @@ -22,6 +22,7 @@ import ( math "math" math_bits "math/bits" reflect "reflect" + strconv "strconv" strings "strings" time "time" ) @@ -38,7 +39,35 @@ var _ = time.Kitchen // proto package needs to be updated. const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package +type RulesRequest_RuleType int32 + +const ( + AnyRule RulesRequest_RuleType = 0 + AlertingRule RulesRequest_RuleType = 1 + RecordingRule RulesRequest_RuleType = 2 +) + +var RulesRequest_RuleType_name = map[int32]string{ + 0: "AnyRule", + 1: "AlertingRule", + 2: "RecordingRule", +} + +var RulesRequest_RuleType_value = map[string]int32{ + "AnyRule": 0, + "AlertingRule": 1, + "RecordingRule": 2, +} + +func (RulesRequest_RuleType) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_ca810a0fd7057a73, []int{0, 0} +} + type RulesRequest struct { + Filter RulesRequest_RuleType `protobuf:"varint,1,opt,name=filter,proto3,enum=base.RulesRequest_RuleType" json:"filter,omitempty"` + RuleName []string `protobuf:"bytes,2,rep,name=rule_name,json=ruleName,proto3" json:"rule_name,omitempty"` + RuleGroup []string `protobuf:"bytes,3,rep,name=rule_group,json=ruleGroup,proto3" json:"rule_group,omitempty"` + File []string `protobuf:"bytes,4,rep,name=file,proto3" json:"file,omitempty"` } func (m *RulesRequest) Reset() { *m = RulesRequest{} } @@ -73,6 +102,34 @@ func (m *RulesRequest) XXX_DiscardUnknown() { var xxx_messageInfo_RulesRequest proto.InternalMessageInfo +func (m *RulesRequest) GetFilter() RulesRequest_RuleType { + if m != nil { + return m.Filter + } + return AnyRule +} + +func (m *RulesRequest) GetRuleName() []string { + if m != nil { + return m.RuleName + } + return nil +} + +func (m *RulesRequest) GetRuleGroup() []string { + if m != nil { + return m.RuleGroup + } + return nil +} + +func (m *RulesRequest) GetFile() []string { + if m != nil { + return m.File + } + return nil +} + type RulesResponse struct { Groups []*GroupStateDesc `protobuf:"bytes,1,rep,name=groups,proto3" json:"groups,omitempty"` } @@ -370,6 +427,7 @@ func (m *AlertStateDesc) GetValidUntil() time.Time { } func init() { + proto.RegisterEnum("base.RulesRequest_RuleType", RulesRequest_RuleType_name, RulesRequest_RuleType_value) proto.RegisterType((*RulesRequest)(nil), "base.RulesRequest") proto.RegisterType((*RulesResponse)(nil), "base.RulesResponse") proto.RegisterType((*GroupStateDesc)(nil), "base.GroupStateDesc") @@ -380,52 +438,66 @@ func init() { func init() { proto.RegisterFile("pkg/ruler/base/ruler.proto", fileDescriptor_ca810a0fd7057a73) } var fileDescriptor_ca810a0fd7057a73 = []byte{ - // 682 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x54, 0x41, 0x6f, 0xd3, 0x30, - 0x14, 0x8e, 0xbb, 0xb6, 0x6b, 0xdd, 0x31, 0x24, 0x6f, 0x42, 0x59, 0x01, 0x77, 0x2a, 0x97, 0x09, - 0xa1, 0x44, 0x0c, 0x84, 0x84, 0x10, 0x42, 0x9d, 0x36, 0xb8, 0xec, 0x80, 0x32, 0xe0, 0x3a, 0xb9, - 0xad, 0x97, 0x45, 0xf3, 0xe2, 0x60, 0x3b, 0x95, 0xb8, 0xf1, 0x13, 0x76, 0xe4, 0xca, 0x8d, 0x9f, - 0xb2, 0xe3, 0x8e, 0x13, 0x48, 0x83, 0x65, 0x17, 0x8e, 0xfb, 0x01, 0x1c, 0x90, 0xed, 0x64, 0xc9, - 0x60, 0x1c, 0x2a, 0xb4, 0x4b, 0x6b, 0xbf, 0xf7, 0xbe, 0xef, 0x7b, 0xef, 0xb3, 0x1d, 0xd8, 0x4d, - 0xf6, 0x42, 0x5f, 0xa4, 0x8c, 0x0a, 0x7f, 0x48, 0x24, 0xb5, 0x4b, 0x2f, 0x11, 0x5c, 0x71, 0x54, - 0xd7, 0x91, 0xee, 0x62, 0xc8, 0x43, 0x6e, 0x02, 0xbe, 0x5e, 0xd9, 0x5c, 0x17, 0x87, 0x9c, 0x87, - 0x8c, 0xfa, 0x66, 0x37, 0x4c, 0x77, 0xfc, 0x71, 0x2a, 0x88, 0x8a, 0x78, 0x9c, 0xe7, 0x7b, 0x7f, - 0xe6, 0x55, 0xb4, 0x4f, 0xa5, 0x22, 0xfb, 0x49, 0x5e, 0x70, 0x5b, 0x0b, 0x33, 0x1e, 0x5a, 0xe6, - 0x62, 0x91, 0x27, 0xef, 0x96, 0x5d, 0xe9, 0x5f, 0x99, 0x0c, 0xed, 0xbf, 0x4d, 0xf7, 0xe7, 0xe1, - 0x5c, 0xa0, 0xb7, 0x01, 0x7d, 0x9f, 0x52, 0xa9, 0xfa, 0xcf, 0xe1, 0x8d, 0x7c, 0x2f, 0x13, 0x1e, - 0x4b, 0x8a, 0x1e, 0xc0, 0x66, 0x28, 0x78, 0x9a, 0x48, 0x17, 0x2c, 0xcf, 0xac, 0x74, 0x56, 0x17, - 0x3d, 0x3d, 0x8a, 0xf7, 0x4a, 0xc7, 0xb6, 0x14, 0x51, 0x74, 0x9d, 0xca, 0x51, 0x90, 0xd7, 0xf4, - 0x3f, 0xd7, 0xe0, 0xfc, 0xe5, 0x14, 0xba, 0x0f, 0x1b, 0x26, 0xe9, 0x82, 0x65, 0x60, 0xf0, 0x56, - 0x5e, 0xab, 0x98, 0x4a, 0x83, 0xb7, 0x25, 0xe8, 0x09, 0x9c, 0x23, 0x23, 0x15, 0x4d, 0xe8, 0xb6, - 0x29, 0x72, 0x6b, 0x46, 0x72, 0xc1, 0x4a, 0x6a, 0x44, 0xa9, 0xd8, 0xb1, 0x85, 0xa6, 0x59, 0xf4, - 0x0e, 0x2e, 0xd0, 0x09, 0x61, 0xa9, 0xb1, 0xed, 0x4d, 0x61, 0x8f, 0x3b, 0x63, 0x14, 0xbb, 0x9e, - 0x35, 0xd0, 0x2b, 0x0c, 0xf4, 0x2e, 0x2a, 0xd6, 0x5a, 0x87, 0x27, 0x3d, 0xe7, 0xe0, 0x7b, 0x0f, - 0x04, 0x57, 0x11, 0xa0, 0x2d, 0x88, 0xca, 0xf0, 0x7a, 0x7e, 0x2c, 0x6e, 0xdd, 0xd0, 0x2e, 0xfd, - 0x45, 0x5b, 0x14, 0x58, 0xd6, 0x4f, 0x9a, 0xf5, 0x0a, 0x78, 0xff, 0x5b, 0xcd, 0x7a, 0x5c, 0x5a, - 0x74, 0x0f, 0xd6, 0xf5, 0xbc, 0xb9, 0x43, 0x37, 0x2b, 0x0e, 0x99, 0x51, 0x4d, 0x12, 0x2d, 0xc2, - 0x86, 0xd4, 0x08, 0xb7, 0xb6, 0x0c, 0x56, 0xda, 0x81, 0xdd, 0xa0, 0x5b, 0xb0, 0xb9, 0x4b, 0x09, - 0x53, 0xbb, 0x66, 0xd8, 0x76, 0x90, 0xef, 0xd0, 0x1d, 0xd8, 0x66, 0x44, 0xaa, 0x0d, 0x21, 0xb8, - 0x30, 0x0d, 0xb7, 0x83, 0x32, 0xa0, 0x0f, 0x95, 0x30, 0x2a, 0x94, 0x74, 0x1b, 0xd5, 0x43, 0x1d, - 0xe8, 0x58, 0xe5, 0x50, 0x6d, 0xcd, 0xbf, 0xdc, 0x6d, 0x5e, 0x8f, 0xbb, 0xb3, 0xff, 0xe7, 0xee, - 0xaf, 0x3a, 0x9c, 0xbf, 0x3c, 0x47, 0xe9, 0x1c, 0xa8, 0x3a, 0xc7, 0x60, 0x93, 0x91, 0x21, 0x65, - 0xc5, 0x2d, 0x5b, 0xf2, 0x2e, 0x5e, 0xce, 0x26, 0x0d, 0xc9, 0xe8, 0xc3, 0xa6, 0xce, 0xbe, 0x26, - 0x91, 0x58, 0x7b, 0xaa, 0x15, 0xbf, 0x9e, 0xf4, 0x1e, 0x86, 0x91, 0xda, 0x4d, 0x87, 0xde, 0x88, - 0xef, 0xfb, 0xa1, 0x20, 0x3b, 0x24, 0x26, 0x3e, 0xe3, 0x7b, 0x91, 0x5f, 0x7d, 0x80, 0x9e, 0xc1, - 0x0d, 0xc6, 0x24, 0x51, 0x54, 0x04, 0xb9, 0x06, 0x9a, 0xc0, 0x0e, 0x89, 0x63, 0xae, 0x4c, 0x93, - 0xd2, 0x9d, 0xb9, 0x46, 0xc9, 0xaa, 0x90, 0x9e, 0x5d, 0x7b, 0x44, 0xcd, 0x1d, 0x00, 0x81, 0xdd, - 0xa0, 0x01, 0x6c, 0xe7, 0xef, 0x8c, 0x28, 0xb7, 0x31, 0xc5, 0x39, 0xb6, 0x2c, 0x6c, 0xa0, 0xd0, - 0x0b, 0xd8, 0xda, 0x89, 0x04, 0x1d, 0x6b, 0x86, 0x69, 0x6e, 0xc2, 0xac, 0x41, 0x0d, 0x14, 0xda, - 0x80, 0x1d, 0x41, 0x25, 0x67, 0x13, 0xcb, 0x31, 0x3b, 0x05, 0x07, 0x2c, 0x80, 0x03, 0x85, 0x5e, - 0xc2, 0x39, 0x7d, 0xaf, 0xb7, 0x25, 0x8d, 0x95, 0xe6, 0x69, 0x4d, 0xc3, 0xa3, 0x91, 0x5b, 0x34, - 0x56, 0xb6, 0x9d, 0x09, 0x61, 0xd1, 0x78, 0x3b, 0x8d, 0x55, 0xc4, 0xdc, 0xf6, 0x34, 0x34, 0x06, - 0xf8, 0x56, 0xe3, 0x56, 0x9f, 0xc1, 0x86, 0x7e, 0xb7, 0x02, 0xad, 0xda, 0x85, 0x44, 0xa8, 0xfc, - 0x7a, 0x15, 0x5f, 0xd9, 0xee, 0xc2, 0xa5, 0x98, 0xfd, 0xd2, 0xf6, 0x9d, 0xb5, 0xc7, 0x47, 0xa7, - 0xd8, 0x39, 0x3e, 0xc5, 0xce, 0xf9, 0x29, 0x06, 0x1f, 0x33, 0x0c, 0xbe, 0x64, 0x18, 0x1c, 0x66, - 0x18, 0x1c, 0x65, 0x18, 0xfc, 0xc8, 0x30, 0xf8, 0x99, 0x61, 0xe7, 0x3c, 0xc3, 0xe0, 0xe0, 0x0c, - 0x3b, 0x47, 0x67, 0xd8, 0x39, 0x3e, 0xc3, 0xce, 0xb0, 0x69, 0x9a, 0x7b, 0xf4, 0x3b, 0x00, 0x00, - 0xff, 0xff, 0x35, 0xb6, 0x62, 0xce, 0x80, 0x06, 0x00, 0x00, + // 791 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x55, 0x4f, 0x4f, 0xdb, 0x48, + 0x14, 0xf7, 0xe4, 0xbf, 0x27, 0x81, 0x65, 0x07, 0xb4, 0x32, 0x61, 0x71, 0xa2, 0xec, 0x25, 0x5a, + 0xad, 0x6c, 0x6d, 0x58, 0xad, 0xb4, 0xda, 0x5d, 0x55, 0x41, 0xd0, 0x5e, 0x50, 0x55, 0x19, 0xda, + 0x6b, 0x34, 0x49, 0x26, 0xc6, 0x62, 0xe2, 0x71, 0xc7, 0xe3, 0x48, 0xdc, 0xfa, 0x11, 0x38, 0xf6, + 0xda, 0x5b, 0x3f, 0x0a, 0x47, 0x7a, 0x43, 0xad, 0x44, 0x4b, 0xb8, 0xf4, 0xc8, 0x07, 0xe8, 0xa1, + 0x9a, 0x19, 0x9b, 0x84, 0x42, 0x0f, 0x51, 0xc5, 0x05, 0xe6, 0xfd, 0xf9, 0xfd, 0xde, 0x7b, 0xbf, + 0x79, 0xe3, 0xc0, 0x7a, 0x74, 0xe4, 0xbb, 0x3c, 0xa1, 0x84, 0xbb, 0x7d, 0x1c, 0x13, 0x7d, 0x74, + 0x22, 0xce, 0x04, 0x43, 0x05, 0xe9, 0xa9, 0xaf, 0xf9, 0xcc, 0x67, 0xca, 0xe1, 0xca, 0x93, 0x8e, + 0xd5, 0x6d, 0x9f, 0x31, 0x9f, 0x12, 0x57, 0x59, 0xfd, 0x64, 0xe4, 0x0e, 0x13, 0x8e, 0x45, 0xc0, + 0xc2, 0x34, 0xde, 0xf8, 0x36, 0x2e, 0x82, 0x31, 0x89, 0x05, 0x1e, 0x47, 0x69, 0xc2, 0x86, 0x2c, + 0x4c, 0x99, 0xaf, 0x99, 0xb3, 0x43, 0x1a, 0xdc, 0x9c, 0x75, 0x25, 0xff, 0xc6, 0x51, 0x5f, 0xff, + 0xd7, 0xe1, 0xd6, 0x3b, 0x00, 0x6b, 0x9e, 0xb4, 0x3d, 0xf2, 0x32, 0x21, 0xb1, 0x40, 0x5b, 0xb0, + 0x34, 0x0a, 0xa8, 0x20, 0xdc, 0x02, 0x4d, 0xd0, 0x5e, 0xee, 0x6c, 0x38, 0xb2, 0x75, 0x67, 0x3e, + 0x47, 0x19, 0x07, 0xc7, 0x11, 0xf1, 0xd2, 0x54, 0xb4, 0x01, 0x4d, 0x49, 0xda, 0x0b, 0xf1, 0x98, + 0x58, 0xb9, 0x66, 0xbe, 0x6d, 0x7a, 0x15, 0xe9, 0x78, 0x8a, 0xc7, 0x04, 0x6d, 0x42, 0xa8, 0x82, + 0x3e, 0x67, 0x49, 0x64, 0xe5, 0x55, 0x54, 0xa5, 0x3f, 0x91, 0x0e, 0x84, 0x60, 0x61, 0x14, 0x50, + 0x62, 0x15, 0x54, 0x40, 0x9d, 0x5b, 0xff, 0xc1, 0x4a, 0x56, 0x03, 0x55, 0x61, 0xb9, 0x1b, 0x1e, + 0x4b, 0x73, 0xc5, 0x40, 0x2b, 0xb0, 0xd6, 0xa5, 0x84, 0x8b, 0x20, 0xf4, 0x95, 0x07, 0xa0, 0x9f, + 0xe1, 0x92, 0x47, 0x06, 0x8c, 0x0f, 0x33, 0x57, 0xae, 0xf5, 0x3f, 0x5c, 0x4a, 0xdb, 0x8d, 0x23, + 0x16, 0xc6, 0x04, 0xfd, 0x01, 0x4b, 0xaa, 0x78, 0x6c, 0x81, 0x66, 0xbe, 0x5d, 0xed, 0xac, 0xe9, + 0x99, 0x54, 0xfd, 0x7d, 0x81, 0x05, 0xd9, 0x21, 0xf1, 0xc0, 0x4b, 0x73, 0x5a, 0x6f, 0x72, 0x70, + 0xf9, 0x76, 0x08, 0xfd, 0x0e, 0x8b, 0xba, 0x7b, 0xa9, 0x89, 0xc4, 0x6b, 0x09, 0xbd, 0x6c, 0x08, + 0x85, 0xd7, 0x29, 0xe8, 0x6f, 0x58, 0xc3, 0x03, 0x11, 0x4c, 0x48, 0x4f, 0x25, 0x29, 0x39, 0xaa, + 0x9d, 0xd5, 0x99, 0x8c, 0xb3, 0x8a, 0x55, 0x9d, 0xa8, 0x9a, 0x45, 0x2f, 0xe0, 0x2a, 0x99, 0x60, + 0x9a, 0xa8, 0xab, 0x3f, 0xc8, 0xae, 0xd8, 0xca, 0xab, 0x8a, 0x75, 0x47, 0x2f, 0x81, 0x93, 0x2d, + 0x81, 0x73, 0x93, 0xb1, 0x5d, 0x39, 0xbd, 0x68, 0x18, 0x27, 0x1f, 0x1b, 0xc0, 0xbb, 0x8f, 0x00, + 0xed, 0x43, 0x34, 0x73, 0xef, 0xa4, 0xab, 0x65, 0x15, 0x14, 0xed, 0xfa, 0x1d, 0xda, 0x2c, 0x41, + 0xb3, 0xbe, 0x96, 0xac, 0xf7, 0xc0, 0x5b, 0x1f, 0x72, 0x5a, 0xe3, 0x99, 0x44, 0xbf, 0xc1, 0x82, + 0x9c, 0x37, 0x55, 0xe8, 0xa7, 0x39, 0x85, 0xd4, 0xa8, 0x2a, 0x88, 0xd6, 0x60, 0x31, 0x96, 0x08, + 0x2b, 0xd7, 0x04, 0x6d, 0xd3, 0xd3, 0x06, 0xfa, 0x05, 0x96, 0x0e, 0x09, 0xa6, 0xe2, 0x50, 0x0d, + 0x6b, 0x7a, 0xa9, 0x85, 0x7e, 0x85, 0x26, 0xc5, 0xb1, 0xd8, 0xe5, 0x9c, 0x71, 0xd5, 0xb0, 0xe9, + 0xcd, 0x1c, 0xf2, 0x52, 0xb1, 0x5c, 0x85, 0xd8, 0x2a, 0xce, 0x5f, 0xaa, 0x5a, 0x8f, 0xb9, 0x4b, + 0xd5, 0x39, 0xdf, 0x53, 0xb7, 0xf4, 0x30, 0xea, 0x96, 0x7f, 0x4c, 0xdd, 0x2f, 0x05, 0xb8, 0x7c, + 0x7b, 0x8e, 0x99, 0x72, 0x60, 0x5e, 0x39, 0x0a, 0x4b, 0x14, 0xf7, 0x09, 0xcd, 0xb6, 0x6c, 0xdd, + 0xb9, 0x79, 0xfd, 0x7b, 0xc4, 0xc7, 0x83, 0xe3, 0x3d, 0x19, 0x7d, 0x86, 0x03, 0xbe, 0xfd, 0x8f, + 0xac, 0xf8, 0xfe, 0xa2, 0xf1, 0xa7, 0x1f, 0x88, 0xc3, 0xa4, 0xef, 0x0c, 0xd8, 0xd8, 0xf5, 0x39, + 0x1e, 0xe1, 0x10, 0xbb, 0x94, 0x1d, 0x05, 0xee, 0xfc, 0x47, 0xc4, 0x51, 0xb8, 0xee, 0x10, 0x47, + 0x82, 0x70, 0x2f, 0xad, 0x81, 0x26, 0xb0, 0x8a, 0xc3, 0x90, 0x09, 0xd5, 0x64, 0xac, 0x5e, 0xf2, + 0x43, 0x95, 0x9c, 0x2f, 0x24, 0x67, 0x97, 0x1a, 0x11, 0xb5, 0x03, 0xc0, 0xd3, 0x06, 0xea, 0x42, + 0x33, 0x7d, 0x67, 0x58, 0x58, 0xc5, 0x05, 0xee, 0xb1, 0xa2, 0x61, 0x5d, 0x81, 0x1e, 0xc1, 0xca, + 0x28, 0xe0, 0x64, 0x28, 0x19, 0x16, 0xd9, 0x84, 0xb2, 0x42, 0x75, 0x05, 0xda, 0x85, 0x55, 0x4e, + 0x62, 0x46, 0x27, 0x9a, 0xa3, 0xbc, 0x00, 0x07, 0xcc, 0x80, 0x5d, 0x81, 0x1e, 0xc3, 0x9a, 0xdc, + 0xeb, 0x5e, 0x4c, 0x42, 0x21, 0x79, 0x2a, 0x8b, 0xf0, 0x48, 0xe4, 0x3e, 0x09, 0x85, 0x6e, 0x67, + 0x82, 0x69, 0x30, 0xec, 0x25, 0xa1, 0x08, 0xa8, 0x65, 0x2e, 0x42, 0xa3, 0x80, 0xcf, 0x25, 0xae, + 0xf3, 0x2f, 0x2c, 0xca, 0x77, 0xcb, 0x51, 0x47, 0x1f, 0x62, 0x84, 0xee, 0xfe, 0x08, 0xd4, 0x57, + 0x6f, 0xf9, 0xf4, 0x97, 0xb6, 0x65, 0x6c, 0xff, 0x75, 0x76, 0x69, 0x1b, 0xe7, 0x97, 0xb6, 0x71, + 0x7d, 0x69, 0x83, 0x57, 0x53, 0x1b, 0xbc, 0x9d, 0xda, 0xe0, 0x74, 0x6a, 0x83, 0xb3, 0xa9, 0x0d, + 0x3e, 0x4d, 0x6d, 0xf0, 0x79, 0x6a, 0x1b, 0xd7, 0x53, 0x1b, 0x9c, 0x5c, 0xd9, 0xc6, 0xd9, 0x95, + 0x6d, 0x9c, 0x5f, 0xd9, 0x46, 0xbf, 0xa4, 0x9a, 0xdb, 0xfa, 0x1a, 0x00, 0x00, 0xff, 0xff, 0xf0, + 0x4e, 0x48, 0x39, 0x44, 0x07, 0x00, 0x00, +} + +func (x RulesRequest_RuleType) String() string { + s, ok := RulesRequest_RuleType_name[int32(x)] + if ok { + return s + } + return strconv.Itoa(int(x)) } - func (this *RulesRequest) Equal(that interface{}) bool { if that == nil { return this == nil @@ -445,6 +517,33 @@ func (this *RulesRequest) Equal(that interface{}) bool { } else if this == nil { return false } + if this.Filter != that1.Filter { + return false + } + if len(this.RuleName) != len(that1.RuleName) { + return false + } + for i := range this.RuleName { + if this.RuleName[i] != that1.RuleName[i] { + return false + } + } + if len(this.RuleGroup) != len(that1.RuleGroup) { + return false + } + for i := range this.RuleGroup { + if this.RuleGroup[i] != that1.RuleGroup[i] { + return false + } + } + if len(this.File) != len(that1.File) { + return false + } + for i := range this.File { + if this.File[i] != that1.File[i] { + return false + } + } return true } func (this *RulesResponse) Equal(that interface{}) bool { @@ -623,8 +722,12 @@ func (this *RulesRequest) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 4) + s := make([]string, 0, 8) s = append(s, "&base.RulesRequest{") + s = append(s, "Filter: "+fmt.Sprintf("%#v", this.Filter)+",\n") + s = append(s, "RuleName: "+fmt.Sprintf("%#v", this.RuleName)+",\n") + s = append(s, "RuleGroup: "+fmt.Sprintf("%#v", this.RuleGroup)+",\n") + s = append(s, "File: "+fmt.Sprintf("%#v", this.File)+",\n") s = append(s, "}") return strings.Join(s, "") } @@ -804,6 +907,38 @@ func (m *RulesRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if len(m.File) > 0 { + for iNdEx := len(m.File) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.File[iNdEx]) + copy(dAtA[i:], m.File[iNdEx]) + i = encodeVarintRuler(dAtA, i, uint64(len(m.File[iNdEx]))) + i-- + dAtA[i] = 0x22 + } + } + if len(m.RuleGroup) > 0 { + for iNdEx := len(m.RuleGroup) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.RuleGroup[iNdEx]) + copy(dAtA[i:], m.RuleGroup[iNdEx]) + i = encodeVarintRuler(dAtA, i, uint64(len(m.RuleGroup[iNdEx]))) + i-- + dAtA[i] = 0x1a + } + } + if len(m.RuleName) > 0 { + for iNdEx := len(m.RuleName) - 1; iNdEx >= 0; iNdEx-- { + i -= len(m.RuleName[iNdEx]) + copy(dAtA[i:], m.RuleName[iNdEx]) + i = encodeVarintRuler(dAtA, i, uint64(len(m.RuleName[iNdEx]))) + i-- + dAtA[i] = 0x12 + } + } + if m.Filter != 0 { + i = encodeVarintRuler(dAtA, i, uint64(m.Filter)) + i-- + dAtA[i] = 0x8 + } return len(dAtA) - i, nil } @@ -1116,6 +1251,27 @@ func (m *RulesRequest) Size() (n int) { } var l int _ = l + if m.Filter != 0 { + n += 1 + sovRuler(uint64(m.Filter)) + } + if len(m.RuleName) > 0 { + for _, s := range m.RuleName { + l = len(s) + n += 1 + l + sovRuler(uint64(l)) + } + } + if len(m.RuleGroup) > 0 { + for _, s := range m.RuleGroup { + l = len(s) + n += 1 + l + sovRuler(uint64(l)) + } + } + if len(m.File) > 0 { + for _, s := range m.File { + l = len(s) + n += 1 + l + sovRuler(uint64(l)) + } + } return n } @@ -1241,6 +1397,10 @@ func (this *RulesRequest) String() string { return "nil" } s := strings.Join([]string{`&RulesRequest{`, + `Filter:` + fmt.Sprintf("%v", this.Filter) + `,`, + `RuleName:` + fmt.Sprintf("%v", this.RuleName) + `,`, + `RuleGroup:` + fmt.Sprintf("%v", this.RuleGroup) + `,`, + `File:` + fmt.Sprintf("%v", this.File) + `,`, `}`, }, "") return s @@ -1354,6 +1514,121 @@ func (m *RulesRequest) Unmarshal(dAtA []byte) error { return fmt.Errorf("proto: RulesRequest: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Filter", wireType) + } + m.Filter = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuler + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Filter |= RulesRequest_RuleType(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RuleName", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuler + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRuler + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRuler + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.RuleName = append(m.RuleName, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RuleGroup", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuler + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRuler + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRuler + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.RuleGroup = append(m.RuleGroup, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field File", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRuler + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRuler + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRuler + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.File = append(m.File, string(dAtA[iNdEx:postIndex])) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipRuler(dAtA[iNdEx:]) diff --git a/pkg/ruler/base/ruler.proto b/pkg/ruler/base/ruler.proto index 02a3c072533fd..0b143728efe61 100644 --- a/pkg/ruler/base/ruler.proto +++ b/pkg/ruler/base/ruler.proto @@ -18,7 +18,17 @@ service Ruler { rpc Rules(RulesRequest) returns (RulesResponse) {} } -message RulesRequest {} +message RulesRequest { + enum RuleType { + AnyRule = 0; + AlertingRule = 1; + RecordingRule = 2; + } + RuleType filter = 1; + repeated string rule_name = 2; + repeated string rule_group = 3; + repeated string file = 4; +} message RulesResponse { repeated GroupStateDesc groups = 1; diff --git a/pkg/ruler/base/ruler_test.go b/pkg/ruler/base/ruler_test.go index 99839ed652536..d17691e1bb6a0 100644 --- a/pkg/ruler/base/ruler_test.go +++ b/pkg/ruler/base/ruler_test.go @@ -401,32 +401,80 @@ func TestGetRules(t *testing.T) { shuffleShardSize int } - expectedRules := expectedRulesMap{ + allRules := expectedRulesMap{ "ruler1": map[string]rulespb.RuleGroupList{ "user1": { - &rulespb.RuleGroupDesc{User: "user1", Namespace: "namespace", Name: "first", Interval: 10 * time.Second, Limit: 10}, - &rulespb.RuleGroupDesc{User: "user1", Namespace: "namespace", Name: "second", Interval: 10 * time.Second, Limit: 10}, + &rulespb.RuleGroupDesc{User: "user1", Namespace: "namespace", Name: "first", Rules: []*rulespb.RuleDesc{createRecordingRule("COUNT_RULE", `count_over_time({foo="bar"}[5m])`)}, Interval: 10 * time.Second, Limit: 10}, + &rulespb.RuleGroupDesc{User: "user1", Namespace: "namespace", Name: "second", Rules: []*rulespb.RuleDesc{createRecordingRule("COUNT_RULE", `count_over_time({foo="bar"}[5m])`)}, Interval: 10 * time.Second, Limit: 10}, + &rulespb.RuleGroupDesc{User: "user1", Namespace: "namespace", Name: "third", Rules: []*rulespb.RuleDesc{createAlertingRule("COUNT_ALERT", `count_over_time({foo="bar"}[5m]) > 0`)}, Interval: 10 * time.Second, Limit: 10}, }, "user2": { - &rulespb.RuleGroupDesc{User: "user2", Namespace: "namespace", Name: "third", Interval: 10 * time.Second, Limit: 10}, + &rulespb.RuleGroupDesc{User: "user2", Namespace: "namespace", Name: "first", Rules: []*rulespb.RuleDesc{createRecordingRule("COUNT_RULE", `count_over_time({foo="bar"}[5m])`)}, Interval: 10 * time.Second, Limit: 10}, }, }, "ruler2": map[string]rulespb.RuleGroupList{ "user1": { - &rulespb.RuleGroupDesc{User: "user1", Namespace: "namespace", Name: "third", Interval: 10 * time.Second, Limit: 10}, + &rulespb.RuleGroupDesc{User: "user1", Namespace: "namespace", Name: "fourth", Rules: []*rulespb.RuleDesc{createRecordingRule("COUNT_RULE", `count_over_time({foo="bar"}[5m])`)}, Interval: 10 * time.Second, Limit: 10}, }, "user2": { - &rulespb.RuleGroupDesc{User: "user2", Namespace: "namespace", Name: "first", Interval: 10 * time.Second, Limit: 10}, - &rulespb.RuleGroupDesc{User: "user2", Namespace: "namespace", Name: "second", Interval: 10 * time.Second, Limit: 10}, + &rulespb.RuleGroupDesc{User: "user2", Namespace: "namespace", Name: "second", Rules: []*rulespb.RuleDesc{createRecordingRule("COUNT_RULE", `count_over_time({foo="bar"}[5m])`)}, Interval: 10 * time.Second, Limit: 10}, + &rulespb.RuleGroupDesc{User: "user2", Namespace: "namespace", Name: "third", Rules: []*rulespb.RuleDesc{createRecordingRule("COUNT_RULE", `count_over_time({foo="bar"}[5m])`)}, Interval: 10 * time.Second, Limit: 10}, + }, + "user3": { + &rulespb.RuleGroupDesc{User: "user3", Namespace: "namespace", Name: "first", Rules: []*rulespb.RuleDesc{createAlertingRule("COUNT_ALERT", `count_over_time({foo="bar"}[5m]) > 0`)}, Interval: 10 * time.Second, Limit: 10}, }, }, "ruler3": map[string]rulespb.RuleGroupList{ "user3": { - &rulespb.RuleGroupDesc{User: "user3", Namespace: "namespace", Name: "third", Interval: 10 * time.Second, Limit: 10}, + &rulespb.RuleGroupDesc{User: "user3", Namespace: "namespace", Name: "second", Rules: []*rulespb.RuleDesc{createRecordingRule("COUNT_RULE", `count_over_time({foo="bar"}[5m])`)}, Interval: 10 * time.Second, Limit: 10}, }, "user2": { - &rulespb.RuleGroupDesc{User: "user2", Namespace: "namespace", Name: "forth", Interval: 10 * time.Second, Limit: 10}, - &rulespb.RuleGroupDesc{User: "user2", Namespace: "namespace", Name: "fifty", Interval: 10 * time.Second, Limit: 10}, + &rulespb.RuleGroupDesc{User: "user2", Namespace: "namespace", Name: "forth", Rules: []*rulespb.RuleDesc{createRecordingRule("COUNT_RULE", `count_over_time({foo="bar"}[5m])`)}, Interval: 10 * time.Second, Limit: 10}, + &rulespb.RuleGroupDesc{User: "user2", Namespace: "namespace", Name: "fifth", Rules: []*rulespb.RuleDesc{createRecordingRule("COUNT_RULE", `count_over_time({foo="bar"}[5m])`)}, Interval: 10 * time.Second, Limit: 10}, + }, + }, + } + + expectedAlertRules := expectedRulesMap{ + "ruler1": map[string]rulespb.RuleGroupList{ + "user1": { + &rulespb.RuleGroupDesc{User: "user1", Namespace: "namespace", Name: "third", Rules: []*rulespb.RuleDesc{createAlertingRule("COUNT_ALERT", `count_over_time({foo="bar"}[5m]) > 0`)}, Interval: 10 * time.Second, Limit: 10}, + }, + }, + "ruler2": map[string]rulespb.RuleGroupList{ + "user3": { + &rulespb.RuleGroupDesc{User: "user3", Namespace: "namespace", Name: "first", Rules: []*rulespb.RuleDesc{createAlertingRule("COUNT_ALERT", `count_over_time({foo="bar"}[5m]) > 0`)}, Interval: 10 * time.Second, Limit: 10}, + }, + }, + "ruler3": map[string]rulespb.RuleGroupList{}, + } + + expectedRecordingRules := expectedRulesMap{ + "ruler1": map[string]rulespb.RuleGroupList{ + "user1": { + &rulespb.RuleGroupDesc{User: "user1", Namespace: "namespace", Name: "first", Rules: []*rulespb.RuleDesc{createRecordingRule("COUNT_RULE", `count_over_time({foo="bar"}[5m])`)}, Interval: 10 * time.Second, Limit: 10}, + &rulespb.RuleGroupDesc{User: "user1", Namespace: "namespace", Name: "second", Rules: []*rulespb.RuleDesc{createRecordingRule("COUNT_RULE", `count_over_time({foo="bar"}[5m])`)}, Interval: 10 * time.Second, Limit: 10}, + }, + "user2": { + &rulespb.RuleGroupDesc{User: "user2", Namespace: "namespace", Name: "first", Rules: []*rulespb.RuleDesc{createRecordingRule("COUNT_RULE", `count_over_time({foo="bar"}[5m])`)}, Interval: 10 * time.Second, Limit: 10}, + }, + }, + "ruler2": map[string]rulespb.RuleGroupList{ + "user1": { + &rulespb.RuleGroupDesc{User: "user1", Namespace: "namespace", Name: "fourth", Rules: []*rulespb.RuleDesc{createRecordingRule("COUNT_RULE", `count_over_time({foo="bar"}[5m])`)}, Interval: 10 * time.Second, Limit: 10}, + }, + "user2": { + &rulespb.RuleGroupDesc{User: "user2", Namespace: "namespace", Name: "second", Rules: []*rulespb.RuleDesc{createRecordingRule("COUNT_RULE", `count_over_time({foo="bar"}[5m])`)}, Interval: 10 * time.Second, Limit: 10}, + &rulespb.RuleGroupDesc{User: "user2", Namespace: "namespace", Name: "third", Rules: []*rulespb.RuleDesc{createRecordingRule("COUNT_RULE", `count_over_time({foo="bar"}[5m])`)}, Interval: 10 * time.Second, Limit: 10}, + }, + }, + "ruler3": map[string]rulespb.RuleGroupList{ + "user3": { + &rulespb.RuleGroupDesc{User: "user3", Namespace: "namespace", Name: "second", Rules: []*rulespb.RuleDesc{createRecordingRule("COUNT_RULE", `count_over_time({foo="bar"}[5m])`)}, Interval: 10 * time.Second, Limit: 10}, + }, + "user2": { + &rulespb.RuleGroupDesc{User: "user2", Namespace: "namespace", Name: "forth", Rules: []*rulespb.RuleDesc{createRecordingRule("COUNT_RULE", `count_over_time({foo="bar"}[5m])`)}, Interval: 10 * time.Second, Limit: 10}, + &rulespb.RuleGroupDesc{User: "user2", Namespace: "namespace", Name: "fifth", Rules: []*rulespb.RuleDesc{createRecordingRule("COUNT_RULE", `count_over_time({foo="bar"}[5m])`)}, Interval: 10 * time.Second, Limit: 10}, }, }, } @@ -446,115 +494,134 @@ func TestGetRules(t *testing.T) { } for name, tc := range testCases { - t.Run(name, func(t *testing.T) { - kvStore, cleanUp := consul.NewInMemoryClient(ring.GetCodec(), log.NewNopLogger(), nil) - t.Cleanup(func() { assert.NoError(t, cleanUp.Close()) }) - allRulesByUser := map[string]rulespb.RuleGroupList{} - allRulesByRuler := map[string]rulespb.RuleGroupList{} - allTokensByRuler := map[string][]uint32{} - rulerAddrMap := map[string]*Ruler{} - - createRuler := func(id string) *Ruler { - cfg := defaultRulerConfig(t, newMockRuleStore(allRulesByUser)) - - cfg.ShardingStrategy = tc.shardingStrategy - cfg.EnableSharding = tc.sharding - cfg.ShardingAlgo = tc.shardingAlgo - - cfg.Ring = RingConfig{ - InstanceID: id, - InstanceAddr: id, - KVStore: kv.Config{ - Mock: kvStore, - }, + for _, ruleType := range []RulesRequest_RuleType{AnyRule, AlertingRule, RecordingRule} { + t.Run(name+" "+ruleType.String(), func(t *testing.T) { + kvStore, cleanUp := consul.NewInMemoryClient(ring.GetCodec(), log.NewNopLogger(), nil) + t.Cleanup(func() { assert.NoError(t, cleanUp.Close()) }) + allRulesByUser := map[string]rulespb.RuleGroupList{} + filteredRulesByUser := map[string]rulespb.RuleGroupList{} + allRulesByRuler := map[string]rulespb.RuleGroupList{} + allTokensByRuler := map[string][]uint32{} + rulerAddrMap := map[string]*Ruler{} + + createRuler := func(id string) *Ruler { + cfg := defaultRulerConfig(t, newMockRuleStore(allRulesByUser)) + + cfg.ShardingStrategy = tc.shardingStrategy + cfg.EnableSharding = tc.sharding + cfg.ShardingAlgo = tc.shardingAlgo + + cfg.Ring = RingConfig{ + InstanceID: id, + InstanceAddr: id, + KVStore: kv.Config{ + Mock: kvStore, + }, + } + m := loki_storage.NewClientMetrics() + defer m.Unregister() + r := buildRuler(t, cfg, nil, m, rulerAddrMap) + r.limits = ruleLimits{tenantShard: tc.shuffleShardSize} + rulerAddrMap[id] = r + if r.ring != nil { + require.NoError(t, services.StartAndAwaitRunning(context.Background(), r.ring)) + t.Cleanup(r.ring.StopAsync) + } + return r } - m := loki_storage.NewClientMetrics() - defer m.Unregister() - r := buildRuler(t, cfg, nil, m, rulerAddrMap) - r.limits = ruleLimits{tenantShard: tc.shuffleShardSize} - rulerAddrMap[id] = r - if r.ring != nil { - require.NoError(t, services.StartAndAwaitRunning(context.Background(), r.ring)) - t.Cleanup(r.ring.StopAsync) + + for rID, r := range allRules { + createRuler(rID) + for user, rules := range r { + allRulesByUser[user] = append(allRulesByUser[user], rules...) + allRulesByRuler[rID] = append(allRulesByRuler[rID], rules...) + allTokensByRuler[rID] = generateTokenForGroups(rules, 1) + } } - return r - } - for rID, r := range expectedRules { - createRuler(rID) - for user, rules := range r { - allRulesByUser[user] = append(allRulesByUser[user], rules...) - allRulesByRuler[rID] = append(allRulesByRuler[rID], rules...) - allTokensByRuler[rID] = generateTokenForGroups(rules, 1) + var filteredRules expectedRulesMap + switch ruleType { + case AlertingRule: + filteredRules = expectedAlertRules + case RecordingRule: + filteredRules = expectedRecordingRules + default: + filteredRules = allRules } - } - if tc.sharding { - err := kvStore.CAS(context.Background(), ringKey, func(in interface{}) (out interface{}, retry bool, err error) { - d, _ := in.(*ring.Desc) - if d == nil { - d = ring.NewDesc() + for _, r := range filteredRules { + for user, rules := range r { + filteredRulesByUser[user] = append(filteredRulesByUser[user], rules...) } - for rID, tokens := range allTokensByRuler { - d.AddIngester(rID, rulerAddrMap[rID].lifecycler.GetInstanceAddr(), "", tokens, ring.ACTIVE, time.Now()) + } + + if tc.sharding { + err := kvStore.CAS(context.Background(), ringKey, func(in interface{}) (out interface{}, retry bool, err error) { + d, _ := in.(*ring.Desc) + if d == nil { + d = ring.NewDesc() + } + for rID, tokens := range allTokensByRuler { + d.AddIngester(rID, rulerAddrMap[rID].lifecycler.GetInstanceAddr(), "", tokens, ring.ACTIVE, time.Now()) + } + return d, true, nil + }) + require.NoError(t, err) + // Wait a bit to make sure ruler's ring is updated. + time.Sleep(100 * time.Millisecond) + } + + forEachRuler := func(f func(rID string, r *Ruler)) { + for rID, r := range rulerAddrMap { + f(rID, r) } - return d, true, nil + } + + // Sync Rules + forEachRuler(func(_ string, r *Ruler) { + r.syncRules(context.Background(), rulerSyncReasonInitial) }) - require.NoError(t, err) - // Wait a bit to make sure ruler's ring is updated. - time.Sleep(100 * time.Millisecond) - } - forEachRuler := func(f func(rID string, r *Ruler)) { - for rID, r := range rulerAddrMap { - f(rID, r) + for u := range filteredRulesByUser { + ctx := user.InjectOrgID(context.Background(), u) + forEachRuler(func(_ string, r *Ruler) { + rules, err := r.GetRules(ctx, &RulesRequest{Filter: ruleType}) + require.NoError(t, err) + require.Equal(t, len(filteredRulesByUser[u]), len(rules)) + if tc.sharding { + mockPoolClient := r.clientsPool.(*mockRulerClientsPool) + + if tc.shardingStrategy == util.ShardingStrategyShuffle { + require.Equal(t, int32(tc.shuffleShardSize), mockPoolClient.numberOfCalls.Load()) + } else { + require.Equal(t, int32(len(rulerAddrMap)), mockPoolClient.numberOfCalls.Load()) + } + mockPoolClient.numberOfCalls.Store(0) + } + }) } - } - // Sync Rules - forEachRuler(func(_ string, r *Ruler) { - r.syncRules(context.Background(), rulerSyncReasonInitial) - }) + totalLoadedRules := 0 + totalConfiguredRules := 0 - for u := range allRulesByUser { - ctx := user.InjectOrgID(context.Background(), u) - forEachRuler(func(_ string, r *Ruler) { - rules, err := r.GetRules(ctx) + forEachRuler(func(rID string, r *Ruler) { + localRules, err := r.listRules(context.Background()) require.NoError(t, err) - require.Equal(t, len(allRulesByUser[u]), len(rules)) - if tc.sharding { - mockPoolClient := r.clientsPool.(*mockRulerClientsPool) - - if tc.shardingStrategy == util.ShardingStrategyShuffle { - require.Equal(t, int32(tc.shuffleShardSize), mockPoolClient.numberOfCalls.Load()) - } else { - require.Equal(t, int32(len(rulerAddrMap)), mockPoolClient.numberOfCalls.Load()) - } - mockPoolClient.numberOfCalls.Store(0) + for _, rules := range localRules { + totalLoadedRules += len(rules) } + totalConfiguredRules += len(allRulesByRuler[rID]) }) - } - - totalLoadedRules := 0 - totalConfiguredRules := 0 - forEachRuler(func(rID string, r *Ruler) { - localRules, err := r.listRules(context.Background()) - require.NoError(t, err) - for _, rules := range localRules { - totalLoadedRules += len(rules) + if tc.sharding { + require.Equal(t, totalConfiguredRules, totalLoadedRules) + } else { + // Not sharding means that all rules will be loaded on all rulers + numberOfRulers := len(rulerAddrMap) + require.Equal(t, totalConfiguredRules*numberOfRulers, totalLoadedRules) } - totalConfiguredRules += len(allRulesByRuler[rID]) }) - - if tc.sharding { - require.Equal(t, totalConfiguredRules, totalLoadedRules) - } else { - // Not sharding means that all rules will be loaded on all rulers - numberOfRulers := len(rulerAddrMap) - require.Equal(t, totalConfiguredRules*numberOfRulers, totalLoadedRules) - } - }) + } } } From 486d14572d31107dc88e8c36652cfc5677d6b705 Mon Sep 17 00:00:00 2001 From: Karsten Jeschkies Date: Mon, 5 Feb 2024 11:56:57 +0100 Subject: [PATCH 28/45] Annotate code with lint issues (#11823) **What this PR does / why we need it**: This should make it easier to find and fix lint issues. **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) --- .github/workflows/checks.yml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/.github/workflows/checks.yml b/.github/workflows/checks.yml index cb9bc3a867ee7..353c09e601a74 100644 --- a/.github/workflows/checks.yml +++ b/.github/workflows/checks.yml @@ -10,6 +10,11 @@ jobs: steps: - uses: actions/checkout@v4 - run: git config --global --add safe.directory "$GITHUB_WORKSPACE" + - name: golangci-lint + uses: golangci/golangci-lint-action@08e2f20817b15149a52b5b3ebe7de50aff2ba8c5 + with: + version: v1.55.1 + only-new-issues: true - run: make lint - run: make check-doc - run: make check-mod From b1a0272357349bc346be156bb0f3338da8a11b24 Mon Sep 17 00:00:00 2001 From: Karsten Jeschkies Date: Mon, 5 Feb 2024 12:33:48 +0100 Subject: [PATCH 29/45] Run GitHub action checks on pull requests against main. (#11864) **What this PR does / why we need it**: The GitHub action `checks` should run on each pull request. **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) --- .github/workflows/checks.yml | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/.github/workflows/checks.yml b/.github/workflows/checks.yml index 353c09e601a74..0919f92a6e701 100644 --- a/.github/workflows/checks.yml +++ b/.github/workflows/checks.yml @@ -1,5 +1,8 @@ name: Checks -on: [push] +on: + pull_request: + branches: + - main jobs: checks: runs-on: ubuntu-latest From 0358b6d968c343f33e60df829a06c0b2fb27172c Mon Sep 17 00:00:00 2001 From: Salva Corts Date: Mon, 5 Feb 2024 13:46:48 +0100 Subject: [PATCH 30/45] Correctly check chunks in bloom block querier. (#11862) **What this PR does / why we need it**: This PR fixes the block querier to correctly test blooms for chunks. Before this PR, for each chunk we were checking the same set of search terms against the bloom. Therefore, we were not filtering chunks, but only series. This PR uses the chunk ref as a prefix for the search term. **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/storage/bloom/v1/block.go | 17 +++++++++++++---- pkg/storage/bloom/v1/bloom_tokenizer.go | 16 ++++++++++++---- pkg/storage/bloom/v1/bloom_tokenizer_test.go | 2 +- pkg/storage/bloom/v1/fuse.go | 16 +++++++++++++--- pkg/storage/bloom/v1/fuse_test.go | 3 ++- pkg/storage/bloom/v1/test_util.go | 11 ++++++++++- pkg/storage/bloom/v1/tokenizer_test.go | 4 ++-- 7 files changed, 53 insertions(+), 16 deletions(-) diff --git a/pkg/storage/bloom/v1/block.go b/pkg/storage/bloom/v1/block.go index 6d13349855f40..c5b115df19e57 100644 --- a/pkg/storage/bloom/v1/block.go +++ b/pkg/storage/bloom/v1/block.go @@ -174,6 +174,11 @@ func (bq *BlockQuerier) Err() error { // passed as the `chks` argument. Chunks will be removed from the result set if they are indexed in the bloom // and fail to pass all the searches. func (bq *BlockQuerier) CheckChunksForSeries(fp model.Fingerprint, chks ChunkRefs, searches [][]byte) (ChunkRefs, error) { + schema, err := bq.Schema() + if err != nil { + return chks, fmt.Errorf("getting schema: %w", err) + } + if err := bq.Seek(fp); err != nil { return chks, errors.Wrapf(err, "seeking to series for fp: %v", fp) } @@ -205,18 +210,22 @@ func (bq *BlockQuerier) CheckChunksForSeries(fp model.Fingerprint, chks ChunkRef } } - // TODO(owen-d): pool, memoize chunk search prefix creation + // TODO(salvacorts): pool tokenBuf + var tokenBuf []byte + var prefixLen int // Check chunks individually now mustCheck, inBlooms := chks.Compare(series.Chunks, true) outer: for _, chk := range inBlooms { + // Get buf to concatenate the chunk and search token + tokenBuf, prefixLen = prefixedToken(schema.NGramLen(), chk, tokenBuf) for _, search := range searches { - // TODO(owen-d): meld chunk + search into a single byte slice from the block schema - var combined = search + tokenBuf = append(tokenBuf[:prefixLen], search...) - if !bloom.Test(combined) { + if !bloom.Test(tokenBuf) { + // chunk didn't pass the search, continue to the next chunk continue outer } } diff --git a/pkg/storage/bloom/v1/bloom_tokenizer.go b/pkg/storage/bloom/v1/bloom_tokenizer.go index c9ff6f23cc0f1..7dd0d8ae44974 100644 --- a/pkg/storage/bloom/v1/bloom_tokenizer.go +++ b/pkg/storage/bloom/v1/bloom_tokenizer.go @@ -61,14 +61,19 @@ func clearCache(cache map[string]interface{}) { // of specific ngram length, along with the length of the prefix. // It ensures enough capacity for the prefix and the token so additional tokens can be created // without allocations by appending them to the prefix length -func prefixedToken(ngram int, chk ChunkRef) ([]byte, int) { - var enc encoding.Encbuf +// If the buffer is nil or too small, a new one is created. The buffer is returned for reuse. +func prefixedToken(ngram int, chk ChunkRef, buf []byte) ([]byte, int) { + enc := encoding.EncWith(buf) + enc.Reset() enc.PutBE64(uint64(chk.Start)) enc.PutBE64(uint64(chk.End)) enc.PutBE32(chk.Checksum) prefixLn := enc.Len() // record the length of the prefix - enc.PutBytes(make([]byte, ngram*MaxRuneLen)) // ensure enough capacity for the ngram + // If the buffer is too small, ensure enough capacity for the ngram + if cap(enc.Get()) < prefixLn+ngram*MaxRuneLen { + enc.PutBytes(make([]byte, ngram*MaxRuneLen)) + } // return the underlying byte slice and the length of the prefix return enc.Get(), prefixLn @@ -86,10 +91,13 @@ func (bt *BloomTokenizer) Populate(swb *SeriesWithBloom, chks Iterator[ChunkRefW clearCache(bt.cache) + var tokenBuf []byte + var prefixLn int + for chks.Err() == nil && chks.Next() { chk := chks.At() itr := chk.Itr - tokenBuf, prefixLn := prefixedToken(bt.lineTokenizer.N, chk.Ref) + tokenBuf, prefixLn = prefixedToken(bt.lineTokenizer.N, chk.Ref, tokenBuf) defer itr.Close() diff --git a/pkg/storage/bloom/v1/bloom_tokenizer_test.go b/pkg/storage/bloom/v1/bloom_tokenizer_test.go index c2b0481e2db58..4ba4ec3eb4f1e 100644 --- a/pkg/storage/bloom/v1/bloom_tokenizer_test.go +++ b/pkg/storage/bloom/v1/bloom_tokenizer_test.go @@ -56,7 +56,7 @@ func TestPrefixedKeyCreation(t *testing.T) { }, } { t.Run(tc.desc, func(t *testing.T) { - token, prefixLn := prefixedToken(tc.ngram, ref) + token, prefixLn := prefixedToken(tc.ngram, ref, nil) require.Equal(t, 20, prefixLn) require.Equal(t, tc.expLen, len(token)) // first 8 bytes should be zeros from `from` diff --git a/pkg/storage/bloom/v1/fuse.go b/pkg/storage/bloom/v1/fuse.go index c397a7a55fd57..77937ecfc08e3 100644 --- a/pkg/storage/bloom/v1/fuse.go +++ b/pkg/storage/bloom/v1/fuse.go @@ -56,6 +56,11 @@ func NewFusedQuerier(bq *BlockQuerier, inputs []PeekingIterator[Request]) *Fused } func (fq *FusedQuerier) Run() error { + schema, err := fq.bq.Schema() + if err != nil { + return errors.Wrap(err, "getting schema") + } + for fq.inputs.Next() { // find all queries for the next relevant fingerprint nextBatch := fq.inputs.At() @@ -119,13 +124,18 @@ func (fq *FusedQuerier) Run() error { // TODO(owen-d): pool var removals ChunkRefs + // TODO(salvacorts): pool tokenBuf + var tokenBuf []byte + var prefixLen int + chunkLoop: for _, chk := range inBlooms { + // Get buf to concatenate the chunk and search token + tokenBuf, prefixLen = prefixedToken(schema.NGramLen(), chk, tokenBuf) for _, search := range input.Searches { - // TODO(owen-d): meld chunk + search into a single byte slice from the block schema - var combined = search + tokenBuf = append(tokenBuf[:prefixLen], search...) - if !bloom.ScalableBloomFilter.Test(combined) { + if !bloom.Test(tokenBuf) { removals = append(removals, chk) continue chunkLoop } diff --git a/pkg/storage/bloom/v1/fuse_test.go b/pkg/storage/bloom/v1/fuse_test.go index 1b51320e1566b..4102ea5de8da9 100644 --- a/pkg/storage/bloom/v1/fuse_test.go +++ b/pkg/storage/bloom/v1/fuse_test.go @@ -20,7 +20,7 @@ func TestFusedQuerier(t *testing.T) { reader := NewByteReader(indexBuf, bloomsBuf) numSeries := 100 numKeysPerSeries := 10000 - data, _ := MkBasicSeriesWithBlooms(numSeries, numKeysPerSeries, 0, 0xffff, 0, 10000) + data, keys := MkBasicSeriesWithBlooms(numSeries, numKeysPerSeries, 0, 0xffff, 0, 10000) builder, err := NewBlockBuilder( BlockOptions{ @@ -53,6 +53,7 @@ func TestFusedQuerier(t *testing.T) { Fp: data[idx].Series.Fingerprint, Chks: data[idx].Series.Chunks, Response: ch, + Searches: keys[idx], }) } inputs = append(inputs, reqs) diff --git a/pkg/storage/bloom/v1/test_util.go b/pkg/storage/bloom/v1/test_util.go index fb9c8a0e4a389..ada8cd7319e90 100644 --- a/pkg/storage/bloom/v1/test_util.go +++ b/pkg/storage/bloom/v1/test_util.go @@ -47,13 +47,14 @@ func MakeBlockQuerier(t testing.TB, fromFp, throughFp model.Fingerprint, fromTs, } func MkBasicSeriesWithBlooms(nSeries, keysPerSeries 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) step := (throughFp - fromFp) / model.Fingerprint(nSeries) timeDelta := time.Duration(throughTs.Sub(fromTs).Nanoseconds() / int64(nSeries)) - tokenizer := NewNGramTokenizer(4, 0) + tokenizer := NewNGramTokenizer(nGramLen, 0) for i := 0; i < nSeries; i++ { var series Series series.Fingerprint = fromFp + model.Fingerprint(i)*step @@ -74,8 +75,16 @@ func MkBasicSeriesWithBlooms(nSeries, keysPerSeries int, fromFp, throughFp model 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) + tokenBuf = append(tokenBuf[:prefixLen], key...) + bloom.Add(tokenBuf) + } } } diff --git a/pkg/storage/bloom/v1/tokenizer_test.go b/pkg/storage/bloom/v1/tokenizer_test.go index b70d9610fab47..eec9645fcd25e 100644 --- a/pkg/storage/bloom/v1/tokenizer_test.go +++ b/pkg/storage/bloom/v1/tokenizer_test.go @@ -171,7 +171,7 @@ func BenchmarkTokens(b *testing.B) { { desc: "v2", f: func() func() { - buf, prefixLn := prefixedToken(v2Three.N, ChunkRef{}) + buf, prefixLn := prefixedToken(v2Three.N, ChunkRef{}, nil) return func() { itr := NewPrefixedTokenIter(buf, prefixLn, v2Three.Tokens(lorem)) for itr.Next() { @@ -188,7 +188,7 @@ func BenchmarkTokens(b *testing.B) { { desc: "v2", f: func() func() { - buf, prefixLn := prefixedToken(v2Three.N, ChunkRef{}) + buf, prefixLn := prefixedToken(v2Three.N, ChunkRef{}, nil) return func() { itr := NewPrefixedTokenIter(buf, prefixLn, v2ThreeSkip1.Tokens(lorem)) for itr.Next() { From 9b7e48aca910001c1250e8e5046c8a481ad23344 Mon Sep 17 00:00:00 2001 From: Dylan Guedes Date: Mon, 5 Feb 2024 13:15:06 -0300 Subject: [PATCH 31/45] Ingesters: Revert "Add metrics.go log line when reading from an ingester (#11571)" (#11818) **What this PR does / why we need it**: This reverts commit f04d0db43457353dddc71239808583d1c9cbfdbd Although it is useful, the amount of logging it generates is too massive. In the future we might revisit it but with a different approach. --- pkg/ingester/ingester.go | 37 ++---------------- pkg/ingester/instance.go | 24 +++++------- pkg/ingester/instance_test.go | 20 +++++----- pkg/logql/metrics.go | 74 ++++------------------------------- 4 files changed, 31 insertions(+), 124 deletions(-) diff --git a/pkg/ingester/ingester.go b/pkg/ingester/ingester.go index a6d252a733ec0..adff06187c648 100644 --- a/pkg/ingester/ingester.go +++ b/pkg/ingester/ingester.go @@ -871,13 +871,10 @@ func (i *Ingester) GetOrCreateInstance(instanceID string) (*instance, error) { / } // Query the ingests for log streams matching a set of matchers. -func (i *Ingester) Query(req *logproto.QueryRequest, queryServer logproto.Querier_QueryServer) (err error) { +func (i *Ingester) Query(req *logproto.QueryRequest, queryServer logproto.Querier_QueryServer) error { // initialize stats collection for ingester queries. _, ctx := stats.NewContext(queryServer.Context()) - start := time.Now().UTC() - var lines int32 - if req.Plan == nil { parsed, err := syntax.ParseLogSelector(req.Selector, true) if err != nil { @@ -888,17 +885,6 @@ func (i *Ingester) Query(req *logproto.QueryRequest, queryServer logproto.Querie } } - defer func() { - status := "successful" - if err != nil { - status = "failed" - } - statsCtx := stats.FromContext(ctx) - execTime := time.Since(start) - logql.RecordIngesterStreamsQueryMetrics(ctx, i.logger, req.Start, req.End, req.Selector, status, req.Limit, lines, req.Shards, - statsCtx.Result(execTime, time.Duration(0), 0)) - }() - instanceID, err := tenant.TenantID(ctx) if err != nil { return err @@ -940,17 +926,14 @@ func (i *Ingester) Query(req *logproto.QueryRequest, queryServer logproto.Querie batchLimit = -1 } - lines, err = sendBatches(ctx, it, queryServer, batchLimit) - return err + return sendBatches(ctx, it, queryServer, batchLimit) } // QuerySample the ingesters for series from logs matching a set of matchers. -func (i *Ingester) QuerySample(req *logproto.SampleQueryRequest, queryServer logproto.Querier_QuerySampleServer) (err error) { +func (i *Ingester) QuerySample(req *logproto.SampleQueryRequest, queryServer logproto.Querier_QuerySampleServer) error { // initialize stats collection for ingester queries. _, ctx := stats.NewContext(queryServer.Context()) sp := opentracing.SpanFromContext(ctx) - start := time.Now().UTC() - var lines int32 // If the plan is empty we want all series to be returned. if req.Plan == nil { @@ -963,17 +946,6 @@ func (i *Ingester) QuerySample(req *logproto.SampleQueryRequest, queryServer log } } - defer func() { - status := "successful" - if err != nil { - status = "failed" - } - statsCtx := stats.FromContext(ctx) - execTime := time.Since(start) - logql.RecordIngesterSeriesQueryMetrics(ctx, i.logger, req.Start, req.End, req.Selector, status, lines, req.Shards, - statsCtx.Result(execTime, time.Duration(0), 0)) - }() - instanceID, err := tenant.TenantID(ctx) if err != nil { return err @@ -1012,8 +984,7 @@ func (i *Ingester) QuerySample(req *logproto.SampleQueryRequest, queryServer log defer util.LogErrorWithContext(ctx, "closing iterator", it.Close) - lines, err = sendSampleBatches(ctx, it, queryServer) - return err + return sendSampleBatches(ctx, it, queryServer) } // asyncStoreMaxLookBack returns a max look back period only if active index type is one of async index stores like `boltdb-shipper` and `tsdb`. diff --git a/pkg/ingester/instance.go b/pkg/ingester/instance.go index 4521daaf20123..f29628d85eeb8 100644 --- a/pkg/ingester/instance.go +++ b/pkg/ingester/instance.go @@ -949,9 +949,8 @@ type QuerierQueryServer interface { Send(res *logproto.QueryResponse) error } -func sendBatches(ctx context.Context, i iter.EntryIterator, queryServer QuerierQueryServer, limit int32) (int32, error) { +func sendBatches(ctx context.Context, i iter.EntryIterator, queryServer QuerierQueryServer, limit int32) error { stats := stats.FromContext(ctx) - var lines int32 // send until the limit is reached. for limit != 0 && !isDone(ctx) { @@ -961,7 +960,7 @@ func sendBatches(ctx context.Context, i iter.EntryIterator, queryServer QuerierQ } batch, batchSize, err := iter.ReadBatch(i, fetchSize) if err != nil { - return lines, err + return err } if limit > 0 { @@ -970,49 +969,46 @@ func sendBatches(ctx context.Context, i iter.EntryIterator, queryServer QuerierQ stats.AddIngesterBatch(int64(batchSize)) batch.Stats = stats.Ingester() - lines += int32(batchSize) if isDone(ctx) { break } if err := queryServer.Send(batch); err != nil && err != context.Canceled { - return lines, err + return err } // We check this after sending an empty batch to make sure stats are sent if len(batch.Streams) == 0 { - return lines, err + return nil } stats.Reset() } - return lines, nil + return nil } -func sendSampleBatches(ctx context.Context, it iter.SampleIterator, queryServer logproto.Querier_QuerySampleServer) (int32, error) { - var lines int32 +func sendSampleBatches(ctx context.Context, it iter.SampleIterator, queryServer logproto.Querier_QuerySampleServer) error { sp := opentracing.SpanFromContext(ctx) stats := stats.FromContext(ctx) for !isDone(ctx) { batch, size, err := iter.ReadSampleBatch(it, queryBatchSampleSize) if err != nil { - return lines, err + return err } stats.AddIngesterBatch(int64(size)) batch.Stats = stats.Ingester() - lines += int32(size) if isDone(ctx) { break } if err := queryServer.Send(batch); err != nil && err != context.Canceled { - return lines, err + return err } // We check this after sending an empty batch to make sure stats are sent if len(batch.Series) == 0 { - return lines, nil + return nil } stats.Reset() @@ -1021,7 +1017,7 @@ func sendSampleBatches(ctx context.Context, it iter.SampleIterator, queryServer } } - return lines, nil + return nil } func shouldConsiderStream(stream *stream, reqFrom, reqThrough time.Time) bool { diff --git a/pkg/ingester/instance_test.go b/pkg/ingester/instance_test.go index ea36cee5ddc98..48c3a8b0bccd4 100644 --- a/pkg/ingester/instance_test.go +++ b/pkg/ingester/instance_test.go @@ -614,16 +614,16 @@ func Test_Iterator(t *testing.T) { // assert the order is preserved. var res *logproto.QueryResponse - lines, err := sendBatches(context.TODO(), it, - fakeQueryServer( - func(qr *logproto.QueryResponse) error { - res = qr - return nil - }, - ), - int32(2)) - require.NoError(t, err) - require.Equal(t, int32(2), lines) + require.NoError(t, + sendBatches(context.TODO(), it, + fakeQueryServer( + func(qr *logproto.QueryResponse) error { + res = qr + return nil + }, + ), + int32(2)), + ) require.Equal(t, 2, len(res.Streams)) // each entry translated into a unique stream require.Equal(t, 1, len(res.Streams[0].Entries)) diff --git a/pkg/logql/metrics.go b/pkg/logql/metrics.go index 67cfee24a0567..63051e362eae6 100644 --- a/pkg/logql/metrics.go +++ b/pkg/logql/metrics.go @@ -26,15 +26,13 @@ import ( ) const ( - QueryTypeMetric = "metric" - QueryTypeFilter = "filter" - QueryTypeLimited = "limited" - QueryTypeLabels = "labels" - QueryTypeSeries = "series" - QueryTypeIngesterStreams = "ingester_streams" - QueryTypeIngesterSeries = "ingester_series" - QueryTypeStats = "stats" - QueryTypeVolume = "volume" + QueryTypeMetric = "metric" + QueryTypeFilter = "filter" + QueryTypeLimited = "limited" + QueryTypeLabels = "labels" + QueryTypeSeries = "series" + QueryTypeStats = "stats" + QueryTypeVolume = "volume" latencyTypeSlow = "slow" latencyTypeFast = "fast" @@ -251,64 +249,6 @@ func PrintMatches(matches []string) string { return strings.Join(matches, ":") } -func RecordIngesterStreamsQueryMetrics(ctx context.Context, log log.Logger, start, end time.Time, query string, status string, limit uint32, returnedLines int32, shards []string, stats logql_stats.Result) { - recordIngesterQueryMetrics(ctx, QueryTypeIngesterStreams, log, start, end, query, status, &limit, returnedLines, shards, stats) -} - -func RecordIngesterSeriesQueryMetrics(ctx context.Context, log log.Logger, start, end time.Time, query string, status string, returnedLines int32, shards []string, stats logql_stats.Result) { - recordIngesterQueryMetrics(ctx, QueryTypeIngesterSeries, log, start, end, query, status, nil, returnedLines, shards, stats) -} - -func recordIngesterQueryMetrics(ctx context.Context, queryType string, log log.Logger, start, end time.Time, query string, status string, limit *uint32, returnedLines int32, shards []string, stats logql_stats.Result) { - var ( - logger = fixLogger(ctx, log) - latencyType = latencyTypeFast - ) - - // Tag throughput metric by latency type based on a threshold. - // Latency below the threshold is fast, above is slow. - if stats.Summary.ExecTime > slowQueryThresholdSecond { - latencyType = latencyTypeSlow - } - - logValues := make([]interface{}, 0, 23) - logValues = append(logValues, - "latency", latencyType, - "query_type", queryType, - "start", start.Format(time.RFC3339Nano), - "end", end.Format(time.RFC3339Nano), - "start_delta", time.Since(start), - "end_delta", time.Since(end), - "length", end.Sub(start), - "duration", time.Duration(int64(stats.Summary.ExecTime*float64(time.Second))), - "status", status, - "query", query, - "query_hash", util.HashedQuery(query), - "returned_lines", returnedLines, - "throughput", strings.Replace(humanize.Bytes(uint64(stats.Summary.BytesProcessedPerSecond)), " ", "", 1), - "total_bytes", strings.Replace(humanize.Bytes(uint64(stats.Summary.TotalBytesProcessed)), " ", "", 1), - "total_bytes_structured_metadata", strings.Replace(humanize.Bytes(uint64(stats.Summary.TotalStructuredMetadataBytesProcessed)), " ", "", 1), - "lines_per_second", stats.Summary.LinesProcessedPerSecond, - "total_lines", stats.Summary.TotalLinesProcessed, - "post_filter_lines", stats.Summary.TotalPostFilterLines, - "total_entries", stats.Summary.TotalEntriesReturned, - "chunk_refs_fetch_time", stats.ChunkRefsFetchTime()) - - if limit != nil { - logValues = append(logValues, - "limit", *limit) - } - shard := extractShard(shards) - if shard != nil { - logValues = append(logValues, - "shard_num", shard.Shard, - "shard_count", shard.Of, - ) - } - - level.Info(logger).Log(logValues...) -} - func RecordSeriesQueryMetrics(ctx context.Context, log log.Logger, start, end time.Time, match []string, status string, shards []string, stats logql_stats.Result) { var ( logger = fixLogger(ctx, log) From 24c760a75e600d07548ac5aadb56712f9865af38 Mon Sep 17 00:00:00 2001 From: Periklis Tsirakidis Date: Mon, 5 Feb 2024 17:52:26 +0100 Subject: [PATCH 32/45] operator: Allow custom audience for managed-auth on STS (#11854) --- operator/CHANGELOG.md | 1 + .../handlers/internal/storage/secrets.go | 5 +---- .../handlers/internal/storage/secrets_test.go | 13 ------------- .../manifests/openshift/credentialsrequest.go | 3 ++- .../openshift/credentialsrequest_test.go | 5 +++-- .../internal/manifests/storage/configure.go | 18 ++---------------- .../manifests/storage/configure_test.go | 18 ++++++++---------- operator/internal/manifests/storage/options.go | 13 ++++++------- operator/internal/manifests/storage/var.go | 5 ++--- 9 files changed, 25 insertions(+), 56 deletions(-) diff --git a/operator/CHANGELOG.md b/operator/CHANGELOG.md index ccded5758a3ab..6e2b1e741b02f 100644 --- a/operator/CHANGELOG.md +++ b/operator/CHANGELOG.md @@ -1,5 +1,6 @@ ## Main +- [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 - [11824](https://github.com/grafana/loki/pull/11824) **xperimental**: Improve messages for errors in storage secret - [11524](https://github.com/grafana/loki/pull/11524) **JoaoBraveCoding**, **periklis**: Add OpenShift cloud credentials support for AWS STS diff --git a/operator/internal/handlers/internal/storage/secrets.go b/operator/internal/handlers/internal/storage/secrets.go index 76ba037eb89bd..d2d39e5ac8570 100644 --- a/operator/internal/handlers/internal/storage/secrets.go +++ b/operator/internal/handlers/internal/storage/secrets.go @@ -276,14 +276,11 @@ func extractS3ConfigSecret(s *corev1.Secret, fg configv1.FeatureGates) (*storage switch { case fg.OpenShift.ManagedAuthEnabled(): cfg.STS = true - cfg.Audience = storage.AWSOpenShiftAudience + cfg.Audience = string(audience) // Do not allow users overriding the role arn provided on Loki Operator installation if len(roleArn) != 0 { return nil, fmt.Errorf("%w: %s", errSecretFieldNotAllowed, storage.KeyAWSRoleArn) } - if len(audience) != 0 { - return nil, fmt.Errorf("%w: %s", errSecretFieldNotAllowed, storage.KeyAWSAudience) - } // In the STS case region is not an optional field if len(region) == 0 { return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyAWSRegion) diff --git a/operator/internal/handlers/internal/storage/secrets_test.go b/operator/internal/handlers/internal/storage/secrets_test.go index 9d32a594e1874..51dc6e15f670e 100644 --- a/operator/internal/handlers/internal/storage/secrets_test.go +++ b/operator/internal/handlers/internal/storage/secrets_test.go @@ -477,18 +477,6 @@ func TestS3Extract_WithOpenShiftManagedAuth(t *testing.T) { managedAuthSecret: &corev1.Secret{}, wantError: "secret field not allowed: role_arn", }, - { - name: "override audience not allowed", - secret: &corev1.Secret{ - ObjectMeta: metav1.ObjectMeta{Name: "test"}, - Data: map[string][]byte{ - "bucketnames": []byte("this,that"), - "audience": []byte("test-audience"), - }, - }, - managedAuthSecret: &corev1.Secret{}, - wantError: "secret field not allowed: audience", - }, { name: "STS all set", secret: &corev1.Secret{ @@ -515,7 +503,6 @@ func TestS3Extract_WithOpenShiftManagedAuth(t *testing.T) { require.NotEmpty(t, opts.SecretSHA1) require.Equal(t, opts.SharedStore, lokiv1.ObjectStorageSecretS3) require.True(t, opts.S3.STS) - require.Equal(t, opts.S3.Audience, "openshift") require.Equal(t, opts.OpenShift.CloudCredentials.SecretName, tst.managedAuthSecret.Name) require.NotEmpty(t, opts.OpenShift.CloudCredentials.SHA1) } else { diff --git a/operator/internal/manifests/openshift/credentialsrequest.go b/operator/internal/manifests/openshift/credentialsrequest.go index 8fc2c5d3f5129..d2da20a194534 100644 --- a/operator/internal/manifests/openshift/credentialsrequest.go +++ b/operator/internal/manifests/openshift/credentialsrequest.go @@ -43,8 +43,9 @@ func BuildCredentialsRequest(opts Options) (*cloudcredentialv1.CredentialsReques ProviderSpec: providerSpec, ServiceAccountNames: []string{ stack.Name, + rulerServiceAccountName(opts), }, - CloudTokenPath: path.Join(storage.SATokenVolumeOcpDirectory, "token"), + CloudTokenPath: path.Join(storage.AWSTokenVolumeDirectory, "token"), }, }, nil } diff --git a/operator/internal/manifests/openshift/credentialsrequest_test.go b/operator/internal/manifests/openshift/credentialsrequest_test.go index 0672cadfc210f..21b193c8c7d7e 100644 --- a/operator/internal/manifests/openshift/credentialsrequest_test.go +++ b/operator/internal/manifests/openshift/credentialsrequest_test.go @@ -45,7 +45,7 @@ func TestBuildCredentialsRequest_HasSecretRef_MatchingLokiStackNamespace(t *test require.Equal(t, opts.BuildOpts.LokiStackNamespace, credReq.Spec.SecretRef.Namespace) } -func TestBuildCredentialsRequest_HasServiceAccountNames_ContainsLokiStackName(t *testing.T) { +func TestBuildCredentialsRequest_HasServiceAccountNames_ContainsAllLokiStackServiceAccounts(t *testing.T) { opts := Options{ BuildOpts: BuildOptions{ LokiStackName: "a-stack", @@ -61,6 +61,7 @@ func TestBuildCredentialsRequest_HasServiceAccountNames_ContainsLokiStackName(t credReq, err := BuildCredentialsRequest(opts) require.NoError(t, err) require.Contains(t, credReq.Spec.ServiceAccountNames, opts.BuildOpts.LokiStackName) + require.Contains(t, credReq.Spec.ServiceAccountNames, rulerServiceAccountName(opts)) } func TestBuildCredentialsRequest_CloudTokenPath_MatchinOpenShiftSADirectory(t *testing.T) { @@ -78,7 +79,7 @@ func TestBuildCredentialsRequest_CloudTokenPath_MatchinOpenShiftSADirectory(t *t credReq, err := BuildCredentialsRequest(opts) require.NoError(t, err) - require.True(t, strings.HasPrefix(credReq.Spec.CloudTokenPath, storage.SATokenVolumeOcpDirectory)) + require.True(t, strings.HasPrefix(credReq.Spec.CloudTokenPath, storage.AWSTokenVolumeDirectory)) } func TestBuildCredentialsRequest_FollowsNamingConventions(t *testing.T) { diff --git a/operator/internal/manifests/storage/configure.go b/operator/internal/manifests/storage/configure.go index b4ff697b1fe4a..da5f6970da171 100644 --- a/operator/internal/manifests/storage/configure.go +++ b/operator/internal/manifests/storage/configure.go @@ -125,7 +125,6 @@ func ensureObjectStoreCredentials(p *corev1.PodSpec, opts Options) corev1.PodSpe }) if managedAuthEnabled(opts) { - setSATokenPath(&opts) container.Env = append(container.Env, managedAuthCredentials(opts)...) volumes = append(volumes, saTokenVolume(opts)) container.VolumeMounts = append(container.VolumeMounts, saTokenVolumeMount(opts)) @@ -190,7 +189,7 @@ func managedAuthCredentials(opts Options) []corev1.EnvVar { } else { return []corev1.EnvVar{ envVarFromSecret(EnvAWSRoleArn, opts.SecretName, KeyAWSRoleArn), - envVarFromValue(EnvAWSWebIdentityTokenFile, path.Join(opts.S3.WebIdentityTokenFile, "token")), + envVarFromValue(EnvAWSWebIdentityTokenFile, path.Join(AWSTokenVolumeDirectory, "token")), } } case lokiv1.ObjectStorageSecretAzure: @@ -286,21 +285,11 @@ func managedAuthEnabled(opts Options) bool { } } -func setSATokenPath(opts *Options) { - switch opts.SharedStore { - case lokiv1.ObjectStorageSecretS3: - opts.S3.WebIdentityTokenFile = saTokenVolumeK8sDirectory - if opts.OpenShift.Enabled { - opts.S3.WebIdentityTokenFile = SATokenVolumeOcpDirectory - } - } -} - func saTokenVolumeMount(opts Options) corev1.VolumeMount { var tokenPath string switch opts.SharedStore { case lokiv1.ObjectStorageSecretS3: - tokenPath = opts.S3.WebIdentityTokenFile + tokenPath = AWSTokenVolumeDirectory case lokiv1.ObjectStorageSecretAzure: tokenPath = azureTokenVolumeDirectory } @@ -319,9 +308,6 @@ func saTokenVolume(opts Options) corev1.Volume { if opts.S3.Audience != "" { audience = opts.S3.Audience } - if opts.OpenShift.Enabled { - audience = AWSOpenShiftAudience - } case lokiv1.ObjectStorageSecretAzure: audience = azureDefaultAudience } diff --git a/operator/internal/manifests/storage/configure_test.go b/operator/internal/manifests/storage/configure_test.go index 0b64a8eb8328e..0c0505fbe63e5 100644 --- a/operator/internal/manifests/storage/configure_test.go +++ b/operator/internal/manifests/storage/configure_test.go @@ -461,7 +461,7 @@ func TestConfigureDeploymentForStorageType(t *testing.T) { { Name: saTokenVolumeName, ReadOnly: false, - MountPath: "/var/run/secrets/kubernetes.io/serviceaccount", + MountPath: "/var/run/secrets/aws/serviceaccount", }, }, Env: []corev1.EnvVar{ @@ -478,7 +478,7 @@ func TestConfigureDeploymentForStorageType(t *testing.T) { }, { Name: "AWS_WEB_IDENTITY_TOKEN_FILE", - Value: "/var/run/secrets/kubernetes.io/serviceaccount/token", + Value: "/var/run/secrets/aws/serviceaccount/token", }, }, }, @@ -520,8 +520,7 @@ func TestConfigureDeploymentForStorageType(t *testing.T) { SecretName: "test", SharedStore: lokiv1.ObjectStorageSecretS3, S3: &S3StorageConfig{ - STS: true, - Audience: "test", + STS: true, }, OpenShift: OpenShiftOptions{ Enabled: true, @@ -560,7 +559,7 @@ func TestConfigureDeploymentForStorageType(t *testing.T) { { Name: saTokenVolumeName, ReadOnly: false, - MountPath: "/var/run/secrets/openshift/serviceaccount", + MountPath: "/var/run/secrets/aws/serviceaccount", }, { Name: "cloud-credentials", @@ -596,7 +595,7 @@ func TestConfigureDeploymentForStorageType(t *testing.T) { Sources: []corev1.VolumeProjection{ { ServiceAccountToken: &corev1.ServiceAccountTokenProjection{ - Audience: "openshift", + Audience: awsDefaultAudience, ExpirationSeconds: ptr.To[int64](3600), Path: corev1.ServiceAccountTokenKey, }, @@ -1090,8 +1089,7 @@ func TestConfigureStatefulSetForStorageType(t *testing.T) { SecretName: "test", SharedStore: lokiv1.ObjectStorageSecretS3, S3: &S3StorageConfig{ - STS: true, - Audience: "test", + STS: true, }, OpenShift: OpenShiftOptions{ Enabled: true, @@ -1130,7 +1128,7 @@ func TestConfigureStatefulSetForStorageType(t *testing.T) { { Name: saTokenVolumeName, ReadOnly: false, - MountPath: "/var/run/secrets/openshift/serviceaccount", + MountPath: "/var/run/secrets/aws/serviceaccount", }, { Name: "cloud-credentials", @@ -1166,7 +1164,7 @@ func TestConfigureStatefulSetForStorageType(t *testing.T) { Sources: []corev1.VolumeProjection{ { ServiceAccountToken: &corev1.ServiceAccountTokenProjection{ - Audience: "openshift", + Audience: awsDefaultAudience, ExpirationSeconds: ptr.To[int64](3600), Path: corev1.ServiceAccountTokenKey, }, diff --git a/operator/internal/manifests/storage/options.go b/operator/internal/manifests/storage/options.go index e1348297ab59c..86aa494318519 100644 --- a/operator/internal/manifests/storage/options.go +++ b/operator/internal/manifests/storage/options.go @@ -38,13 +38,12 @@ type GCSStorageConfig struct { // S3StorageConfig for S3 storage config type S3StorageConfig struct { - Endpoint string - Region string - Buckets string - WebIdentityTokenFile string - Audience string - STS bool - SSE S3SSEConfig + Endpoint string + Region string + Buckets string + Audience string + STS bool + SSE S3SSEConfig } type S3SSEType string diff --git a/operator/internal/manifests/storage/var.go b/operator/internal/manifests/storage/var.go index 048af9e8a88e8..6184cff0463da 100644 --- a/operator/internal/manifests/storage/var.go +++ b/operator/internal/manifests/storage/var.go @@ -122,7 +122,6 @@ const ( KeySwiftUsername = "username" saTokenVolumeK8sDirectory = "/var/run/secrets/kubernetes.io/serviceaccount" - SATokenVolumeOcpDirectory = "/var/run/secrets/openshift/serviceaccount" saTokenVolumeName = "bound-sa-token" saTokenExpiration int64 = 3600 @@ -131,8 +130,8 @@ const ( storageTLSVolume = "storage-tls" caDirectory = "/etc/storage/ca" - awsDefaultAudience = "sts.amazonaws.com" - AWSOpenShiftAudience = "openshift" + awsDefaultAudience = "sts.amazonaws.com" + AWSTokenVolumeDirectory = "/var/run/secrets/aws/serviceaccount" azureDefaultAudience = "api://AzureADTokenExchange" azureTokenVolumeDirectory = "/var/run/secrets/azure/serviceaccount" From f97fb093f32927e1479a1cca215e48161afb0407 Mon Sep 17 00:00:00 2001 From: Christian Haudum Date: Mon, 5 Feb 2024 21:48:35 +0100 Subject: [PATCH 33/45] (chore) Bloom store: Rewrite block downloading/caching in bloom store fetcher (#11857) **What this PR does / why we need it**: 1. The `BloomStore` interface gets a new function `FetchBlocks()`, which accepts `[]BlockRef` and returns `[]BlockDirectory`. The fetcher implements a new function `FetchBlocks()` which returns a list of block directories. A block directory represents a local file path that contains the extracted files of a bloom block. It also holds a counter of active readers that access the directory. This is used for safely deleting the directory in case it needs to be removed from disk (eg. max disk/cache size reached). The fetcher resolves the block directory from three places: 1. Cache: The in-memory cache that holds actively accessed directories and keeps track of used disk size. 2. Filesystem: In case the cache was emptied (e.g. when restarting the process) but the block directory is present on disk, it can be re-assambled into a bloom directory that can be put to cache. 3. Storage: If the directory is not present locally, the block archive is downloaded and extracted and the block directory is put to cache. 2. The `{Meta,Block}Client` interfaces are unified: Both clients define the same set of operations: Get{Meta,Block} Get{Meta,Block}s Put{Meta,Block} Delete{Meta,Block}s 3. The `blockDownloader` is replaced by a simple queue powered by a channel. The queue is implemented using generics, so it can be re-used for different request/response types. 4. Code related to "block caching" moved into a separate file `cache.go` and unused code is removed. --------- Signed-off-by: Christian Haudum Co-authored-by: Owen Diehl --- pkg/bloomgateway/bloomgateway.go | 3 +- pkg/bloomgateway/processor_test.go | 4 + .../shipper/bloomshipper/block_downloader.go | 407 ---------------- .../bloomshipper/block_downloader_test.go | 439 ------------------ .../stores/shipper/bloomshipper/cache.go | 120 +++++ .../stores/shipper/bloomshipper/cache_test.go | 80 ++++ .../stores/shipper/bloomshipper/client.go | 184 +++----- .../shipper/bloomshipper/client_test.go | 255 +++++----- .../shipper/bloomshipper/compress_utils.go | 22 + .../bloomshipper/compress_utils_test.go | 69 +++ .../stores/shipper/bloomshipper/fetcher.go | 351 ++++++++++---- .../shipper/bloomshipper/fetcher_test.go | 130 +++++- .../stores/shipper/bloomshipper/resolver.go | 35 ++ .../stores/shipper/bloomshipper/shipper.go | 70 +-- .../stores/shipper/bloomshipper/store.go | 196 ++++++-- 15 files changed, 1091 insertions(+), 1274 deletions(-) delete mode 100644 pkg/storage/stores/shipper/bloomshipper/block_downloader.go delete mode 100644 pkg/storage/stores/shipper/bloomshipper/block_downloader_test.go create mode 100644 pkg/storage/stores/shipper/bloomshipper/cache.go create mode 100644 pkg/storage/stores/shipper/bloomshipper/cache_test.go create mode 100644 pkg/storage/stores/shipper/bloomshipper/compress_utils_test.go diff --git a/pkg/bloomgateway/bloomgateway.go b/pkg/bloomgateway/bloomgateway.go index afe8d646ae63a..1e7a54f1d1e33 100644 --- a/pkg/bloomgateway/bloomgateway.go +++ b/pkg/bloomgateway/bloomgateway.go @@ -42,7 +42,6 @@ package bloomgateway import ( "context" "fmt" - "io" "sort" "sync" "time" @@ -213,7 +212,7 @@ func New(cfg Config, schemaCfg config.SchemaConfig, storageCfg storage.Config, o // TODO(chaudum): Plug in cache var metasCache cache.Cache - var blocksCache *cache.EmbeddedCache[string, io.ReadCloser] + var blocksCache *cache.EmbeddedCache[string, bloomshipper.BlockDirectory] store, err := bloomshipper.NewBloomStore(schemaCfg.Configs, storageCfg, cm, metasCache, blocksCache, logger) if err != nil { return nil, err diff --git a/pkg/bloomgateway/processor_test.go b/pkg/bloomgateway/processor_test.go index 7ef8e067bac38..33f7513fbf592 100644 --- a/pkg/bloomgateway/processor_test.go +++ b/pkg/bloomgateway/processor_test.go @@ -38,6 +38,10 @@ func (s *dummyStore) FetchMetas(_ context.Context, _ bloomshipper.MetaSearchPara return s.metas, nil } +func (s *dummyStore) FetchBlocks(_ context.Context, _ []bloomshipper.BlockRef) ([]bloomshipper.BlockDirectory, error) { + panic("don't call me") +} + func (s *dummyStore) Fetcher(_ model.Time) *bloomshipper.Fetcher { return nil } diff --git a/pkg/storage/stores/shipper/bloomshipper/block_downloader.go b/pkg/storage/stores/shipper/bloomshipper/block_downloader.go deleted file mode 100644 index 8d28ca03dcc63..0000000000000 --- a/pkg/storage/stores/shipper/bloomshipper/block_downloader.go +++ /dev/null @@ -1,407 +0,0 @@ -package bloomshipper - -import ( - "context" - "errors" - "fmt" - "io" - "os" - "path" - "path/filepath" - "sync" - "time" - - "github.com/go-kit/log" - "github.com/go-kit/log/level" - "github.com/grafana/dskit/services" - "github.com/prometheus/client_golang/prometheus" - "go.uber.org/atomic" - "k8s.io/utils/keymutex" - - "github.com/grafana/loki/pkg/logqlmodel/stats" - "github.com/grafana/loki/pkg/queue" - 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" - "github.com/grafana/loki/pkg/util" - "github.com/grafana/loki/pkg/util/constants" -) - -type blockDownloader struct { - logger log.Logger - - queueMetrics *queue.Metrics - queue *queue.RequestQueue - - limits Limits - activeUsersService *util.ActiveUsersCleanupService - - ctx context.Context - manager *services.Manager - wg sync.WaitGroup - - strategy downloadingStrategy -} - -type queueLimits struct { - limits Limits -} - -func (l *queueLimits) MaxConsumers(tenantID string, _ int) int { - return l.limits.BloomGatewayBlocksDownloadingParallelism(tenantID) -} - -func newBlockDownloader(config config.Config, blockClient BlockClient, limits Limits, logger log.Logger, reg prometheus.Registerer) (*blockDownloader, error) { - queueMetrics := queue.NewMetrics(reg, constants.Loki, "bloom_blocks_downloader") - //add cleanup service - downloadingQueue := queue.NewRequestQueue(config.BlocksDownloadingQueue.MaxTasksEnqueuedPerTenant, time.Minute, &queueLimits{limits: limits}, queueMetrics) - activeUsersService := util.NewActiveUsersCleanupWithDefaultValues(queueMetrics.Cleanup) - - ctx := context.Background() - manager, err := services.NewManager(downloadingQueue, activeUsersService) - if err != nil { - return nil, fmt.Errorf("error creating service manager: %w", err) - } - err = services.StartManagerAndAwaitHealthy(ctx, manager) - if err != nil { - return nil, fmt.Errorf("error starting service manager: %w", err) - } - - strategy := createDownloadingStrategy(config, blockClient, reg, logger) - b := &blockDownloader{ - ctx: ctx, - logger: logger, - queueMetrics: queueMetrics, - queue: downloadingQueue, - strategy: strategy, - activeUsersService: activeUsersService, - limits: limits, - manager: manager, - wg: sync.WaitGroup{}, - } - - for i := 0; i < config.BlocksDownloadingQueue.WorkersCount; i++ { - b.wg.Add(1) - go b.serveDownloadingTasks(fmt.Sprintf("worker-%d", i)) - } - return b, nil -} - -type BlockDownloadingTask struct { - ctx context.Context - block BlockRef - // ErrCh is a send-only channel to write an error to - ErrCh chan<- error - // ResultsCh is a send-only channel to return the block querier for the downloaded block - ResultsCh chan<- blockWithQuerier -} - -func NewBlockDownloadingTask(ctx context.Context, block BlockRef, resCh chan<- blockWithQuerier, errCh chan<- error) *BlockDownloadingTask { - return &BlockDownloadingTask{ - ctx: ctx, - block: block, - ErrCh: errCh, - ResultsCh: resCh, - } -} - -func (d *blockDownloader) serveDownloadingTasks(workerID string) { - // defer first, so it gets executed as last of the deferred functions - defer d.wg.Done() - - logger := log.With(d.logger, "worker", workerID) - level.Debug(logger).Log("msg", "starting worker") - - d.queue.RegisterConsumerConnection(workerID) - defer d.queue.UnregisterConsumerConnection(workerID) - - idx := queue.StartIndexWithLocalQueue - - for { - item, newIdx, err := d.queue.Dequeue(d.ctx, idx, workerID) - if err != nil { - if !errors.Is(err, queue.ErrStopped) && !errors.Is(err, context.Canceled) { - level.Error(logger).Log("msg", "failed to dequeue task", "err", err) - continue - } - level.Info(logger).Log("msg", "stopping worker") - return - } - task, ok := item.(*BlockDownloadingTask) - if !ok { - level.Error(logger).Log("msg", "failed to cast to BlockDownloadingTask", "item", fmt.Sprintf("%+v", item), "type", fmt.Sprintf("%T", item)) - continue - } - - idx = newIdx - result, err := d.strategy.downloadBlock(task, logger) - if err != nil { - task.ErrCh <- err - continue - } - task.ResultsCh <- result - continue - } -} - -func createDownloadingStrategy(cfg config.Config, blockClient BlockClient, reg prometheus.Registerer, logger log.Logger) downloadingStrategy { - if cfg.BlocksCache.EmbeddedCacheConfig.Enabled { - blocksCache := NewBlocksCache(cfg, reg, logger) - return &cacheDownloadingStrategy{ - config: cfg, - workingDirectory: cfg.WorkingDirectory, - blockClient: blockClient, - blocksCache: blocksCache, - keyMutex: keymutex.NewHashed(cfg.BlocksDownloadingQueue.WorkersCount), - } - } - return &storageDownloadingStrategy{ - workingDirectory: cfg.WorkingDirectory, - blockClient: blockClient, - } -} - -type downloadingStrategy interface { - downloadBlock(task *BlockDownloadingTask, logger log.Logger) (blockWithQuerier, error) - close() -} - -type cacheDownloadingStrategy struct { - config config.Config - workingDirectory string - blockClient BlockClient - blocksCache *cache.EmbeddedCache[string, *cachedBlock] - keyMutex keymutex.KeyMutex -} - -func (s *cacheDownloadingStrategy) downloadBlock(task *BlockDownloadingTask, logger log.Logger) (blockWithQuerier, error) { - key := s.blockClient.Block(task.block).Addr() - s.keyMutex.LockKey(key) - defer func() { - _ = s.keyMutex.UnlockKey(key) - }() - blockFromCache, exists := s.blocksCache.Get(task.ctx, key) - if exists { - return blockWithQuerier{ - BlockRef: task.block, - closableBlockQuerier: newBlockQuerierFromCache(blockFromCache), - }, nil - } - - directory, err := downloadBlockToDirectory(logger, task, s.workingDirectory, s.blockClient) - if err != nil { - return blockWithQuerier{}, err - } - blockFromCache = newCachedBlock(directory, s.config.BlocksCache.RemoveDirectoryGracefulPeriod, logger) - err = s.blocksCache.Store(task.ctx, []string{key}, []*cachedBlock{blockFromCache}) - if err != nil { - level.Error(logger).Log("msg", "error storing the block in the cache", "block", key, "err", err) - return blockWithQuerier{}, fmt.Errorf("error storing the block %s in the cache : %w", key, err) - } - return blockWithQuerier{ - BlockRef: task.block, - closableBlockQuerier: newBlockQuerierFromCache(blockFromCache), - }, nil -} - -func (s *cacheDownloadingStrategy) close() { - s.blocksCache.Stop() -} - -type storageDownloadingStrategy struct { - workingDirectory string - blockClient BlockClient -} - -func (s *storageDownloadingStrategy) downloadBlock(task *BlockDownloadingTask, logger log.Logger) (blockWithQuerier, error) { - directory, err := downloadBlockToDirectory(logger, task, s.workingDirectory, s.blockClient) - if err != nil { - return blockWithQuerier{}, err - } - return blockWithQuerier{ - BlockRef: task.block, - closableBlockQuerier: newBlockQuerierFromFS(directory), - }, nil -} - -func (s *storageDownloadingStrategy) close() { - // noop implementation -} - -func downloadBlockToDirectory(logger log.Logger, task *BlockDownloadingTask, workingDirectory string, blockClient BlockClient) (string, error) { - blockPath := filepath.Join(workingDirectory, blockClient.Block(task.block).LocalPath()) - level.Debug(logger).Log("msg", "start downloading the block", "block", blockPath) - block, err := blockClient.GetBlock(task.ctx, task.block) - if err != nil { - level.Error(logger).Log("msg", "error downloading the block", "block", blockPath, "err", err) - return "", fmt.Errorf("error downloading the block %s : %w", blockPath, err) - } - err = extractBlock(block.Data, blockPath, logger) - if err != nil { - level.Error(logger).Log("msg", "error extracting the block", "block", blockPath, "err", err) - return "", fmt.Errorf("error extracting the block %s : %w", blockPath, err) - } - level.Debug(logger).Log("msg", "block has been downloaded and extracted", "block", blockPath) - return blockPath, nil -} - -func (d *blockDownloader) downloadBlocks(ctx context.Context, tenantID string, references []BlockRef) (chan blockWithQuerier, chan error) { - d.activeUsersService.UpdateUserTimestamp(tenantID, time.Now()) - // we need to have errCh with size that can keep max count of errors to prevent the case when - // the queue worker reported the error to this channel before the current goroutine - // and this goroutine will go to the deadlock because it won't be able to report an error - // because nothing reads this channel at this moment. - errCh := make(chan error, len(references)) - blocksCh := make(chan blockWithQuerier, len(references)) - - for _, reference := range references { - task := NewBlockDownloadingTask(ctx, reference, blocksCh, errCh) - level.Debug(d.logger).Log("msg", "enqueuing task to download block", "block", reference) - err := d.queue.Enqueue(tenantID, nil, task, nil) - if err != nil { - errCh <- fmt.Errorf("error enquing downloading task for block %s : %w", reference, err) - return blocksCh, errCh - } - } - return blocksCh, errCh -} - -type blockWithQuerier struct { - BlockRef - *closableBlockQuerier -} - -// extract the files into directory and returns absolute path to this directory. -func extractBlock(data io.ReadCloser, blockDir string, logger log.Logger) error { - - err := os.MkdirAll(blockDir, os.ModePerm) - if err != nil { - return fmt.Errorf("can not create directory to extract the block: %w", err) - } - archivePath, err := writeDataToTempFile(blockDir, data) - if err != nil { - return fmt.Errorf("error writing data to temp file: %w", err) - } - defer func() { - err = os.Remove(archivePath) - if err != nil { - level.Error(logger).Log("msg", "error removing temp archive file", "err", err) - } - }() - err = extractArchive(archivePath, blockDir) - if err != nil { - return fmt.Errorf("error extracting archive: %w", err) - } - return nil -} - -func (d *blockDownloader) stop() { - _ = services.StopManagerAndAwaitStopped(d.ctx, d.manager) - d.wg.Wait() - d.strategy.close() -} - -type closableBlockQuerier struct { - *v1.BlockQuerier - Close func() error -} - -func newBlockQuerierFromCache(cached *cachedBlock) *closableBlockQuerier { - cached.activeQueriers.Inc() - return &closableBlockQuerier{ - BlockQuerier: createBlockQuerier(cached.blockDirectory), - Close: func() error { - cached.activeQueriers.Dec() - return nil - }, - } -} - -func newBlockQuerierFromFS(blockDirectory string) *closableBlockQuerier { - return &closableBlockQuerier{ - BlockQuerier: createBlockQuerier(blockDirectory), - Close: func() error { - return deleteFolder(blockDirectory) - }, - } -} - -func createBlockQuerier(directory string) *v1.BlockQuerier { - reader := v1.NewDirectoryBlockReader(directory) - block := v1.NewBlock(reader) - return v1.NewBlockQuerier(block) -} - -func NewBlocksCache(config config.Config, reg prometheus.Registerer, logger log.Logger) *cache.EmbeddedCache[string, *cachedBlock] { - return cache.NewTypedEmbeddedCache[string, *cachedBlock]( - "bloom-blocks-cache", - config.BlocksCache.EmbeddedCacheConfig, - reg, - logger, - stats.BloomBlocksCache, - calculateBlockDirectorySize, - func(key string, value *cachedBlock) { - value.removeDirectoryAsync() - }) -} - -func calculateBlockDirectorySize(entry *cache.Entry[string, *cachedBlock]) uint64 { - value := entry.Value - bloomFileStats, _ := os.Lstat(path.Join(value.blockDirectory, v1.BloomFileName)) - seriesFileStats, _ := os.Lstat(path.Join(value.blockDirectory, v1.SeriesFileName)) - return uint64(bloomFileStats.Size() + seriesFileStats.Size()) -} - -func newCachedBlock(blockDirectory string, removeDirectoryTimeout time.Duration, logger log.Logger) *cachedBlock { - return &cachedBlock{ - blockDirectory: blockDirectory, - removeDirectoryTimeout: removeDirectoryTimeout, - logger: logger, - activeQueriersCheckInterval: defaultActiveQueriersCheckInterval, - } -} - -type cachedBlock struct { - blockDirectory string - removeDirectoryTimeout time.Duration - activeQueriers atomic.Int32 - logger log.Logger - activeQueriersCheckInterval time.Duration -} - -const defaultActiveQueriersCheckInterval = 100 * time.Millisecond - -func (b *cachedBlock) removeDirectoryAsync() { - go func() { - timeout := time.After(b.removeDirectoryTimeout) - ticker := time.NewTicker(b.activeQueriersCheckInterval) - defer ticker.Stop() - for { - select { - case <-ticker.C: - if b.activeQueriers.Load() == 0 { - err := deleteFolder(b.blockDirectory) - if err == nil { - return - } - level.Error(b.logger).Log("msg", "error deleting block directory", "err", err) - } - case <-timeout: - level.Warn(b.logger).Log("msg", "force deleting block folder after timeout", "timeout", b.removeDirectoryTimeout) - err := deleteFolder(b.blockDirectory) - if err == nil { - return - } - level.Error(b.logger).Log("msg", "error force deleting block directory", "err", err) - } - } - }() -} - -func deleteFolder(folderPath string) error { - err := os.RemoveAll(folderPath) - if err != nil { - return fmt.Errorf("error deleting bloom block directory: %w", err) - } - return nil -} diff --git a/pkg/storage/stores/shipper/bloomshipper/block_downloader_test.go b/pkg/storage/stores/shipper/bloomshipper/block_downloader_test.go deleted file mode 100644 index 8fc1b3e976e0f..0000000000000 --- a/pkg/storage/stores/shipper/bloomshipper/block_downloader_test.go +++ /dev/null @@ -1,439 +0,0 @@ -package bloomshipper - -import ( - "bytes" - "context" - "fmt" - "io" - "os" - "path/filepath" - "sync" - "testing" - "time" - - "github.com/go-kit/log" - "github.com/google/uuid" - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/common/model" - "github.com/stretchr/testify/require" - "go.uber.org/atomic" - - 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" - "github.com/grafana/loki/pkg/util" - "github.com/grafana/loki/pkg/validation" -) - -func Test_blockDownloader_downloadBlocks(t *testing.T) { - overrides, err := validation.NewOverrides(validation.Limits{BloomGatewayBlocksDownloadingParallelism: 20}, nil) - require.NoError(t, err) - workingDirectory := t.TempDir() - - blockReferences, blockClient := createFakeBlocks(t, 20) - blockClient.responseDelay = 100 * time.Millisecond - workersCount := 10 - downloader, err := newBlockDownloader(config.Config{ - WorkingDirectory: workingDirectory, - BlocksDownloadingQueue: config.DownloadingQueueConfig{ - WorkersCount: workersCount, - MaxTasksEnqueuedPerTenant: 20, - }, - }, blockClient, overrides, log.NewNopLogger(), prometheus.DefaultRegisterer) - require.NoError(t, err) - blocksCh, errorsCh := downloader.downloadBlocks(context.Background(), "fake", blockReferences) - downloadedBlocks := make(map[BlockRef]any, len(blockReferences)) - done := make(chan bool) - go func() { - for i := 0; i < 20; i++ { - block := <-blocksCh - downloadedBlocks[block.BlockRef] = nil - } - done <- true - }() - - select { - //20 blocks, 10 workers, fixed delay 100ms per block: the total downloading time must be ~200ms. - case <-time.After(2 * time.Second): - t.Fatalf("test must complete before the timeout") - case err := <-errorsCh: - require.NoError(t, err) - case <-done: - } - require.Len(t, downloadedBlocks, 20, "all 20 block must be downloaded") - - // We want all workers to be connected to the queue - require.Equal(t, workersCount, int(downloader.queue.GetConnectedConsumersMetric())) - - downloader.stop() - - // We want all workers to be disconnected from the queue - require.Equal(t, 0, int(downloader.queue.GetConnectedConsumersMetric())) -} - -func Test_blockDownloader_downloadBlock(t *testing.T) { - tests := map[string]struct { - cacheEnabled bool - expectedTotalGetBlocksCalls int32 - }{ - "cache disabled": { - cacheEnabled: false, - expectedTotalGetBlocksCalls: 40, - }, - "cache enabled": { - cacheEnabled: true, - expectedTotalGetBlocksCalls: 20, - }, - } - for name, testData := range tests { - t.Run(name, func(t *testing.T) { - overrides, err := validation.NewOverrides(validation.Limits{BloomGatewayBlocksDownloadingParallelism: 20}, nil) - require.NoError(t, err) - workingDirectory := t.TempDir() - - blockReferences, blockClient := createFakeBlocks(t, 20) - workersCount := 10 - downloader, err := newBlockDownloader(config.Config{ - WorkingDirectory: workingDirectory, - BlocksDownloadingQueue: config.DownloadingQueueConfig{ - WorkersCount: workersCount, - MaxTasksEnqueuedPerTenant: 20, - }, - BlocksCache: config.BlocksCacheConfig{ - EmbeddedCacheConfig: cache.EmbeddedCacheConfig{ - Enabled: testData.cacheEnabled, - MaxSizeItems: 20, - }, - RemoveDirectoryGracefulPeriod: 1 * time.Second, - }, - }, blockClient, overrides, log.NewNopLogger(), prometheus.NewRegistry()) - t.Cleanup(downloader.stop) - require.NoError(t, err) - - blocksCh, errorsCh := downloader.downloadBlocks(context.Background(), "fake", blockReferences) - downloadedBlocks := make(map[BlockRef]any, len(blockReferences)) - done := make(chan bool) - go func() { - for i := 0; i < 20; i++ { - block := <-blocksCh - downloadedBlocks[block.BlockRef] = nil - } - done <- true - }() - - select { - case <-time.After(2 * time.Second): - t.Fatalf("test must complete before the timeout") - case err := <-errorsCh: - require.NoError(t, err) - case <-done: - } - require.Len(t, downloadedBlocks, 20, "all 20 block must be downloaded") - require.Equal(t, int32(20), blockClient.getBlockCalls.Load()) - - blocksCh, errorsCh = downloader.downloadBlocks(context.Background(), "fake", blockReferences) - downloadedBlocks = make(map[BlockRef]any, len(blockReferences)) - done = make(chan bool) - go func() { - for i := 0; i < 20; i++ { - block := <-blocksCh - downloadedBlocks[block.BlockRef] = nil - } - done <- true - }() - - select { - case <-time.After(2 * time.Second): - t.Fatalf("test must complete before the timeout") - case err := <-errorsCh: - require.NoError(t, err) - case <-done: - } - require.Len(t, downloadedBlocks, 20, "all 20 block must be downloaded") - require.Equal(t, testData.expectedTotalGetBlocksCalls, blockClient.getBlockCalls.Load()) - }) - } -} - -func Test_blockDownloader_downloadBlock_deduplication(t *testing.T) { - tests := map[string]struct { - cacheEnabled bool - expectedTotalGetBlocksCalls int32 - }{ - "requests to blockClient must be deduplicated by blockPath if cache is enabled": { - cacheEnabled: true, - expectedTotalGetBlocksCalls: 1, - }, - "requests to blockClient must NOT be deduplicated by blockPath if cache is disabled": { - cacheEnabled: false, - expectedTotalGetBlocksCalls: 10, - }, - } - for name, testData := range tests { - t.Run(name, func(t *testing.T) { - - overrides, err := validation.NewOverrides(validation.Limits{BloomGatewayBlocksDownloadingParallelism: 20}, nil) - require.NoError(t, err) - workingDirectory := t.TempDir() - - blockReferences, blockClient := createFakeBlocks(t, 1) - workersCount := 10 - downloader, err := newBlockDownloader(config.Config{ - WorkingDirectory: workingDirectory, - BlocksDownloadingQueue: config.DownloadingQueueConfig{ - WorkersCount: workersCount, - MaxTasksEnqueuedPerTenant: 20, - }, - BlocksCache: config.BlocksCacheConfig{ - EmbeddedCacheConfig: cache.EmbeddedCacheConfig{ - Enabled: testData.cacheEnabled, - MaxSizeItems: 20, - }, - RemoveDirectoryGracefulPeriod: 1 * time.Second, - }, - }, blockClient, overrides, log.NewNopLogger(), prometheus.NewRegistry()) - t.Cleanup(downloader.stop) - require.NoError(t, err) - - var blocksDownloadedCount atomic.Uint32 - mutex := sync.Mutex{} - multiError := util.MultiError{} - waitGroup := sync.WaitGroup{} - for i := 0; i < 10; i++ { - waitGroup.Add(1) - go func() { - defer waitGroup.Done() - blocksCh, errCh := downloader.downloadBlocks(context.Background(), "fake", blockReferences) - var err error - select { - case <-blocksCh: - blocksDownloadedCount.Inc() - case downloaderErr := <-errCh: - err = downloaderErr - case <-time.After(1 * time.Second): - err = fmt.Errorf("timeout in the test waiting for a single block to be downloaded") - } - if err == nil { - return - } - mutex.Lock() - defer mutex.Unlock() - multiError.Add(err) - }() - } - waitGroup.Wait() - - require.NoError(t, multiError.Err()) - require.Equal(t, uint32(10), blocksDownloadedCount.Load()) - require.Equal(t, testData.expectedTotalGetBlocksCalls, blockClient.getBlockCalls.Load()) - }) - } -} - -func Test_cachedBlock(t *testing.T) { - tests := map[string]struct { - releaseQuerier bool - expectDirectoryToBeDeletedWithin time.Duration - }{ - "expected block directory to be removed once all queriers are released": { - releaseQuerier: true, - // four times grater than activeQueriersCheckInterval - expectDirectoryToBeDeletedWithin: 200 * time.Millisecond, - }, - "expected block directory to be force removed after timeout": { - releaseQuerier: false, - // four times grater than removeDirectoryTimeout - expectDirectoryToBeDeletedWithin: 2 * time.Second, - }, - } - for name, testData := range tests { - t.Run(name, func(t *testing.T) { - extractedBlockDirectory := t.TempDir() - blockFilePath, _, _ := createBlockArchive(t) - err := extractArchive(blockFilePath, extractedBlockDirectory) - require.NoError(t, err) - require.DirExists(t, extractedBlockDirectory) - - cached := &cachedBlock{ - blockDirectory: extractedBlockDirectory, - removeDirectoryTimeout: 500 * time.Millisecond, - activeQueriersCheckInterval: 50 * time.Millisecond, - logger: log.NewLogfmtLogger(os.Stderr), - } - cached.activeQueriers.Inc() - cached.removeDirectoryAsync() - //ensure directory exists - require.Never(t, func() bool { - return directoryDoesNotExist(extractedBlockDirectory) - }, 200*time.Millisecond, 50*time.Millisecond) - - if testData.releaseQuerier { - cached.activeQueriers.Dec() - } - //ensure directory does not exist - require.Eventually(t, func() bool { - return directoryDoesNotExist(extractedBlockDirectory) - }, testData.expectDirectoryToBeDeletedWithin, 50*time.Millisecond) - }) - } -} - -func Test_closableBlockQuerier(t *testing.T) { - t.Run("cached", func(t *testing.T) { - blockFilePath, _, _ := createBlockArchive(t) - extractedBlockDirectory := t.TempDir() - err := extractArchive(blockFilePath, extractedBlockDirectory) - require.NoError(t, err) - - cached := &cachedBlock{blockDirectory: extractedBlockDirectory, removeDirectoryTimeout: 100 * time.Millisecond} - require.Equal(t, int32(0), cached.activeQueriers.Load()) - querier := newBlockQuerierFromCache(cached) - require.Equal(t, int32(1), cached.activeQueriers.Load()) - require.NoError(t, querier.Close()) - require.Equal(t, int32(0), cached.activeQueriers.Load()) - }) - - t.Run("file system", func(t *testing.T) { - blockFilePath, _, _ := createBlockArchive(t) - extractedBlockDirectory := t.TempDir() - err := extractArchive(blockFilePath, extractedBlockDirectory) - require.NoError(t, err) - - querier := newBlockQuerierFromFS(extractedBlockDirectory) - require.DirExists(t, extractedBlockDirectory) - - require.NoError(t, querier.Close()) - - //ensure directory does not exist - require.Eventually(t, func() bool { - return directoryDoesNotExist(extractedBlockDirectory) - }, 1*time.Second, 100*time.Millisecond) - }) -} - -// creates fake blocks and returns map[block-path]Block and mockBlockClient -func createFakeBlocks(t *testing.T, count int) ([]BlockRef, *mockBlockClient) { - mockData := make(map[BlockRef]blockSupplier, count) - refs := make([]BlockRef, 0, count) - for i := 0; i < count; i++ { - archivePath, _, _ := createBlockArchive(t) - _, err := os.OpenFile(archivePath, os.O_RDONLY, 0700) - //ensure file can be opened - require.NoError(t, err) - blockRef := BlockRef{ - Ref: Ref{ - TenantID: "", - TableName: "", - Bounds: v1.NewBounds(model.Fingerprint(i), model.Fingerprint(i+1)), - StartTimestamp: 0, - EndTimestamp: 0, - Checksum: 0, - }, - } - mockData[blockRef] = func() LazyBlock { - file, _ := os.OpenFile(archivePath, os.O_RDONLY, 0700) - return LazyBlock{ - BlockRef: blockRef, - Data: file, - } - } - refs = append(refs, blockRef) - } - return refs, &mockBlockClient{mockData: mockData} -} - -type blockSupplier func() LazyBlock - -type mockBlockClient struct { - responseDelay time.Duration - mockData map[BlockRef]blockSupplier - getBlockCalls atomic.Int32 - defaultKeyResolver -} - -func (m *mockBlockClient) GetBlock(_ context.Context, reference BlockRef) (LazyBlock, error) { - m.getBlockCalls.Inc() - time.Sleep(m.responseDelay) - supplier, exists := m.mockData[reference] - if exists { - return supplier(), nil - } - - return LazyBlock{}, fmt.Errorf("block %s is not found in mockData", reference) -} - -func (m *mockBlockClient) PutBlocks(_ context.Context, _ []Block) ([]Block, error) { - panic("implement me") -} - -func (m *mockBlockClient) DeleteBlocks(_ context.Context, _ []BlockRef) error { - panic("implement me") -} - -func Test_blockDownloader_extractBlock(t *testing.T) { - blockFilePath, bloomFileContent, seriesFileContent := createBlockArchive(t) - blockFile, err := os.OpenFile(blockFilePath, os.O_RDONLY, 0700) - require.NoError(t, err) - - workingDir := t.TempDir() - block := LazyBlock{ - BlockRef: BlockRef{ - Ref: Ref{ - TenantID: "", - TableName: "", - Bounds: v1.NewBounds(0, 1), - StartTimestamp: 0, - EndTimestamp: 0, - Checksum: 0, - }, - }, - Data: blockFile, - } - - err = extractBlock(block.Data, workingDir, nil) - require.NoError(t, err) - - require.FileExists(t, filepath.Join(workingDir, v1.BloomFileName)) - require.FileExists(t, filepath.Join(workingDir, v1.SeriesFileName)) - - actualBloomFileContent, err := os.ReadFile(filepath.Join(workingDir, v1.BloomFileName)) - require.NoError(t, err) - require.Equal(t, bloomFileContent, string(actualBloomFileContent)) - - actualSeriesFileContent, err := os.ReadFile(filepath.Join(workingDir, v1.SeriesFileName)) - require.NoError(t, err) - require.Equal(t, seriesFileContent, string(actualSeriesFileContent)) -} - -func directoryDoesNotExist(path string) bool { - _, err := os.Lstat(path) - return err != nil -} - -const testArchiveFileName = "test-block-archive" - -func createBlockArchive(t *testing.T) (string, string, string) { - dir := t.TempDir() - mockBlockDir := filepath.Join(dir, "mock-block-dir") - err := os.MkdirAll(mockBlockDir, 0777) - require.NoError(t, err) - bloomFile, err := os.Create(filepath.Join(mockBlockDir, v1.BloomFileName)) - require.NoError(t, err) - bloomFileContent := uuid.NewString() - _, err = io.Copy(bloomFile, bytes.NewReader([]byte(bloomFileContent))) - require.NoError(t, err) - - seriesFile, err := os.Create(filepath.Join(mockBlockDir, v1.SeriesFileName)) - require.NoError(t, err) - seriesFileContent := uuid.NewString() - _, err = io.Copy(seriesFile, bytes.NewReader([]byte(seriesFileContent))) - require.NoError(t, err) - - blockFilePath := filepath.Join(dir, testArchiveFileName) - file, err := os.OpenFile(blockFilePath, os.O_CREATE|os.O_RDWR, 0700) - require.NoError(t, err) - err = v1.TarGz(file, v1.NewDirectoryBlockReader(mockBlockDir)) - require.NoError(t, err) - - return blockFilePath, bloomFileContent, seriesFileContent -} diff --git a/pkg/storage/stores/shipper/bloomshipper/cache.go b/pkg/storage/stores/shipper/bloomshipper/cache.go new file mode 100644 index 0000000000000..7fe678a1c38fb --- /dev/null +++ b/pkg/storage/stores/shipper/bloomshipper/cache.go @@ -0,0 +1,120 @@ +package bloomshipper + +import ( + "fmt" + "os" + "path" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/prometheus/client_golang/prometheus" + "go.uber.org/atomic" + + "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 ClosableBlockQuerier struct { + *v1.BlockQuerier + Close func() error +} + +func NewBlocksCache(config config.Config, reg prometheus.Registerer, logger log.Logger) *cache.EmbeddedCache[string, BlockDirectory] { + return cache.NewTypedEmbeddedCache[string, BlockDirectory]( + "bloom-blocks-cache", + config.BlocksCache.EmbeddedCacheConfig, + reg, + logger, + stats.BloomBlocksCache, + calculateBlockDirectorySize, + func(_ string, value BlockDirectory) { + value.removeDirectoryAsync() + }) +} + +func calculateBlockDirectorySize(entry *cache.Entry[string, BlockDirectory]) uint64 { + value := entry.Value + bloomFileStats, _ := os.Lstat(path.Join(value.Path, v1.BloomFileName)) + seriesFileStats, _ := os.Lstat(path.Join(value.Path, v1.SeriesFileName)) + return uint64(bloomFileStats.Size() + seriesFileStats.Size()) +} + +func NewBlockDirectory(ref BlockRef, path string, logger log.Logger) BlockDirectory { + return BlockDirectory{ + BlockRef: ref, + Path: path, + activeQueriers: atomic.NewInt32(0), + removeDirectoryTimeout: time.Minute, + logger: logger, + activeQueriersCheckInterval: defaultActiveQueriersCheckInterval, + } +} + +// A BlockDirectory is a local file path that contains a bloom block. +// It maintains a counter for currently active readers. +type BlockDirectory struct { + BlockRef + Path string + removeDirectoryTimeout time.Duration + activeQueriers *atomic.Int32 + logger log.Logger + activeQueriersCheckInterval time.Duration +} + +func (b BlockDirectory) Block() *v1.Block { + return v1.NewBlock(v1.NewDirectoryBlockReader(b.Path)) +} + +// BlockQuerier returns a new block querier from the directory. +// It increments the counter of active queriers for this directory. +// The counter is decreased when the returned querier is closed. +func (b BlockDirectory) BlockQuerier() *ClosableBlockQuerier { + b.activeQueriers.Inc() + return &ClosableBlockQuerier{ + BlockQuerier: v1.NewBlockQuerier(b.Block()), + Close: func() error { + _ = b.activeQueriers.Dec() + return nil + }, + } +} + +const defaultActiveQueriersCheckInterval = 100 * time.Millisecond + +func (b *BlockDirectory) removeDirectoryAsync() { + go func() { + timeout := time.After(b.removeDirectoryTimeout) + ticker := time.NewTicker(b.activeQueriersCheckInterval) + defer ticker.Stop() + for { + select { + case <-ticker.C: + if b.activeQueriers.Load() == 0 { + err := deleteFolder(b.Path) + if err == nil { + return + } + level.Error(b.logger).Log("msg", "error deleting block directory", "err", err) + } + case <-timeout: + level.Warn(b.logger).Log("msg", "force deleting block folder after timeout", "timeout", b.removeDirectoryTimeout) + err := deleteFolder(b.Path) + if err == nil { + return + } + level.Error(b.logger).Log("msg", "error force deleting block directory", "err", err) + } + } + }() +} + +func deleteFolder(folderPath string) error { + err := os.RemoveAll(folderPath) + if err != nil { + return fmt.Errorf("error deleting bloom block directory: %w", err) + } + return nil +} diff --git a/pkg/storage/stores/shipper/bloomshipper/cache_test.go b/pkg/storage/stores/shipper/bloomshipper/cache_test.go new file mode 100644 index 0000000000000..de916377a3f7b --- /dev/null +++ b/pkg/storage/stores/shipper/bloomshipper/cache_test.go @@ -0,0 +1,80 @@ +package bloomshipper + +import ( + "testing" + "time" + + "github.com/go-kit/log" + "github.com/stretchr/testify/require" + "go.uber.org/atomic" +) + +func TestBlockDirectory_Cleanup(t *testing.T) { + checkInterval := 50 * time.Millisecond + timeout := 200 * time.Millisecond + + tests := map[string]struct { + releaseQuerier bool + expectDirectoryToBeDeletedWithin time.Duration + }{ + "expect directory to be removed once all queriers are released": { + releaseQuerier: true, + expectDirectoryToBeDeletedWithin: 2 * checkInterval, + }, + "expect directory to be force removed after timeout": { + releaseQuerier: false, + expectDirectoryToBeDeletedWithin: 2 * timeout, + }, + } + for name, tc := range tests { + tc := tc + t.Run(name, func(t *testing.T) { + extractedBlockDirectory := t.TempDir() + blockFilePath, _, _, _ := createBlockArchive(t) + err := extractArchive(blockFilePath, extractedBlockDirectory) + require.NoError(t, err) + require.DirExists(t, extractedBlockDirectory) + + cached := BlockDirectory{ + Path: extractedBlockDirectory, + removeDirectoryTimeout: timeout, + activeQueriersCheckInterval: checkInterval, + logger: log.NewNopLogger(), + activeQueriers: atomic.NewInt32(0), + } + // acquire directory + cached.activeQueriers.Inc() + // start cleanup goroutine + cached.removeDirectoryAsync() + + if tc.releaseQuerier { + // release directory + cached.activeQueriers.Dec() + } + + // ensure directory does not exist any more + require.Eventually(t, func() bool { + return directoryDoesNotExist(extractedBlockDirectory) + }, tc.expectDirectoryToBeDeletedWithin, 10*time.Millisecond) + }) + } +} + +func Test_ClosableBlockQuerier(t *testing.T) { + blockFilePath, _, _, _ := createBlockArchive(t) + extractedBlockDirectory := t.TempDir() + err := extractArchive(blockFilePath, extractedBlockDirectory) + require.NoError(t, err) + + cached := BlockDirectory{ + Path: extractedBlockDirectory, + removeDirectoryTimeout: 100 * time.Millisecond, + activeQueriers: atomic.NewInt32(0), + } + + querier := cached.BlockQuerier() + require.Equal(t, int32(1), cached.activeQueriers.Load()) + require.NoError(t, querier.Close()) + require.Equal(t, int32(0), cached.activeQueriers.Load()) + +} diff --git a/pkg/storage/stores/shipper/bloomshipper/client.go b/pkg/storage/stores/shipper/bloomshipper/client.go index cfdd057db047d..396e8808c1759 100644 --- a/pkg/storage/stores/shipper/bloomshipper/client.go +++ b/pkg/storage/stores/shipper/bloomshipper/client.go @@ -7,9 +7,6 @@ import ( "fmt" "hash" "io" - "path" - "strconv" - "strings" "github.com/go-kit/log" "github.com/grafana/dskit/concurrency" @@ -77,7 +74,10 @@ func (r BlockRef) String() string { type MetaRef struct { Ref - FilePath string +} + +func (r MetaRef) String() string { + return defaultKeyResolver{}.Meta(r).Addr() } // todo rename it @@ -95,16 +95,11 @@ type MetaSearchParams struct { } type MetaClient interface { - // Returns all metas that are within MinFingerprint-MaxFingerprint fingerprint range - // and intersect time period from StartTimestamp to EndTimestamp. - GetMetas(ctx context.Context, metas []MetaRef) ([]Meta, error) + KeyResolver + GetMeta(ctx context.Context, ref MetaRef) (Meta, error) + GetMetas(ctx context.Context, refs []MetaRef) ([]Meta, error) PutMeta(ctx context.Context, meta Meta) error - DeleteMeta(ctx context.Context, meta Meta) error -} - -type LazyBlock struct { - BlockRef - Data io.ReadCloser + DeleteMetas(ctx context.Context, refs []MetaRef) error } type Block struct { @@ -114,9 +109,10 @@ type Block struct { type BlockClient interface { KeyResolver - GetBlock(ctx context.Context, ref BlockRef) (LazyBlock, error) - PutBlocks(ctx context.Context, blocks []Block) ([]Block, error) - DeleteBlocks(ctx context.Context, blocks []BlockRef) error + GetBlock(ctx context.Context, ref BlockRef) (BlockDirectory, error) + GetBlocks(ctx context.Context, refs []BlockRef) ([]BlockDirectory, error) + PutBlock(ctx context.Context, block Block) error + DeleteBlocks(ctx context.Context, refs []BlockRef) error } type Client interface { @@ -133,12 +129,14 @@ type BloomClient struct { concurrency int client client.ObjectClient logger log.Logger + fsResolver KeyResolver } -func NewBloomClient(client client.ObjectClient, logger log.Logger) (*BloomClient, error) { +func NewBloomClient(cfg bloomStoreConfig, client client.ObjectClient, logger log.Logger) (*BloomClient, error) { return &BloomClient{ KeyResolver: defaultKeyResolver{}, // TODO(owen-d): hook into schema, similar to `{,Parse}ExternalKey` - concurrency: 100, // make configurable? + fsResolver: NewPrefixedResolver(cfg.workingDir, defaultKeyResolver{}), + concurrency: cfg.numWorkers, client: client, logger: logger, }, nil @@ -149,68 +147,72 @@ func (b *BloomClient) PutMeta(ctx context.Context, meta Meta) error { if err != nil { return fmt.Errorf("can not marshal the meta to json: %w", err) } - key := externalMetaKey(meta.MetaRef) + key := b.Meta(meta.MetaRef).Addr() return b.client.PutObject(ctx, key, bytes.NewReader(data)) } -func externalMetaKey(ref MetaRef) string { - filename := fmt.Sprintf("%s-%d-%d-%x", ref.Bounds.String(), ref.StartTimestamp, ref.EndTimestamp, ref.Checksum) - return path.Join(rootFolder, ref.TableName, ref.TenantID, metasFolder, filename) -} - -func findPeriod(configs []config.PeriodConfig, ts model.Time) (config.DayTime, error) { - for i := len(configs) - 1; i >= 0; i-- { - periodConfig := configs[i] - if periodConfig.From.Before(ts) || periodConfig.From.Equal(ts) { - return periodConfig.From, nil - } - } - return config.DayTime{}, fmt.Errorf("can not find period for timestamp %d", ts) -} +func (b *BloomClient) DeleteMetas(ctx context.Context, refs []MetaRef) error { + err := concurrency.ForEachJob(ctx, len(refs), b.concurrency, func(ctx context.Context, idx int) error { + key := b.Meta(refs[idx]).Addr() + return b.client.DeleteObject(ctx, key) + }) -func (b *BloomClient) DeleteMeta(ctx context.Context, meta Meta) error { - key := externalMetaKey(meta.MetaRef) - return b.client.DeleteObject(ctx, key) + return err } // GetBlock downloads the blocks from objectStorage and returns the downloaded block -func (b *BloomClient) GetBlock(ctx context.Context, reference BlockRef) (LazyBlock, error) { - readCloser, _, err := b.client.GetObject(ctx, b.Block(reference).Addr()) +func (b *BloomClient) GetBlock(ctx context.Context, ref BlockRef) (BlockDirectory, error) { + key := b.Block(ref).Addr() + readCloser, _, err := b.client.GetObject(ctx, key) if err != nil { - return LazyBlock{}, fmt.Errorf("error while fetching object from storage: %w", err) + return BlockDirectory{}, fmt.Errorf("failed to get block from storage: %w", err) } - return LazyBlock{ - BlockRef: reference, - Data: readCloser, - }, nil -} - -func (b *BloomClient) PutBlocks(ctx context.Context, blocks []Block) ([]Block, error) { - results := make([]Block, len(blocks)) - err := concurrency.ForEachJob(ctx, len(blocks), b.concurrency, func(ctx context.Context, idx int) error { - block := blocks[idx] - defer func(Data io.ReadCloser) { - _ = Data.Close() - }(block.Data) - var err error + path := b.fsResolver.Block(ref).LocalPath() + err = extractBlock(readCloser, path, b.logger) + if err != nil { + return BlockDirectory{}, fmt.Errorf("failed to extract block into directory : %w", err) + } - key := b.Block(block.BlockRef).Addr() - _, err = block.Data.Seek(0, 0) - if err != nil { - return fmt.Errorf("error uploading block file: %w", err) - } + return NewBlockDirectory(ref, path, b.logger), nil +} - err = b.client.PutObject(ctx, key, block.Data) +func (b *BloomClient) GetBlocks(ctx context.Context, refs []BlockRef) ([]BlockDirectory, error) { + // TODO(chaudum): Integrate download queue + // The current implementation does brute-force download of all blocks with maximum concurrency. + // However, we want that a single block is downloaded only exactly once, even if it is requested + // multiple times concurrently. + results := make([]BlockDirectory, len(refs)) + err := concurrency.ForEachJob(ctx, len(refs), b.concurrency, func(ctx context.Context, idx int) error { + block, err := b.GetBlock(ctx, refs[idx]) if err != nil { - return fmt.Errorf("error uploading block file: %w", err) + return err } results[idx] = block return nil }) + return results, err } +func (b *BloomClient) PutBlock(ctx context.Context, block Block) error { + defer func(Data io.ReadCloser) { + _ = Data.Close() + }(block.Data) + + key := b.Block(block.BlockRef).Addr() + _, err := block.Data.Seek(0, 0) + if err != nil { + return fmt.Errorf("error uploading block file %s : %w", key, err) + } + + err = b.client.PutObject(ctx, key, block.Data) + if err != nil { + return fmt.Errorf("error uploading block file: %w", err) + } + return nil +} + func (b *BloomClient) DeleteBlocks(ctx context.Context, references []BlockRef) error { return concurrency.ForEachJob(ctx, len(references), b.concurrency, func(ctx context.Context, idx int) error { ref := references[idx] @@ -230,7 +232,7 @@ func (b *BloomClient) Stop() { func (b *BloomClient) GetMetas(ctx context.Context, refs []MetaRef) ([]Meta, error) { results := make([]Meta, len(refs)) err := concurrency.ForEachJob(ctx, len(refs), b.concurrency, func(ctx context.Context, idx int) error { - meta, err := b.getMeta(ctx, refs[idx]) + meta, err := b.GetMeta(ctx, refs[idx]) if err != nil { return err } @@ -240,66 +242,30 @@ func (b *BloomClient) GetMetas(ctx context.Context, refs []MetaRef) ([]Meta, err return results, err } -func (b *BloomClient) getMeta(ctx context.Context, ref MetaRef) (Meta, error) { +func (b *BloomClient) GetMeta(ctx context.Context, ref MetaRef) (Meta, error) { meta := Meta{ MetaRef: ref, } - reader, _, err := b.client.GetObject(ctx, ref.FilePath) + key := b.KeyResolver.Meta(ref).Addr() + reader, _, err := b.client.GetObject(ctx, key) if err != nil { - return Meta{}, fmt.Errorf("error downloading meta file %s : %w", ref.FilePath, err) + return Meta{}, fmt.Errorf("error downloading meta file %s : %w", key, err) } defer reader.Close() err = json.NewDecoder(reader).Decode(&meta) if err != nil { - return Meta{}, fmt.Errorf("error unmarshalling content of meta file %s: %w", ref.FilePath, err) + return Meta{}, fmt.Errorf("error unmarshalling content of meta file %s: %w", key, err) } return meta, nil } -func createMetaRef(objectKey string, tenantID string, tableName string) (MetaRef, error) { - fileName := objectKey[strings.LastIndex(objectKey, delimiter)+1:] - parts := strings.Split(fileName, fileNamePartDelimiter) - if len(parts) != 5 { - return MetaRef{}, fmt.Errorf("%s filename parts count must be 5 but was %d: [%s]", objectKey, len(parts), strings.Join(parts, ", ")) - } - bounds, err := v1.ParseBoundsFromParts(parts[0], parts[1]) - if err != nil { - return MetaRef{}, fmt.Errorf("error parsing bounds %s : %w", parts[0], err) - } - - startTimestamp, err := strconv.ParseInt(parts[2], 10, 64) - if err != nil { - return MetaRef{}, fmt.Errorf("error parsing startTimestamp %s : %w", parts[2], err) - } - endTimestamp, err := strconv.ParseInt(parts[3], 10, 64) - if err != nil { - return MetaRef{}, fmt.Errorf("error parsing endTimestamp %s : %w", parts[3], err) - } - checksum, err := strconv.ParseUint(parts[4], 16, 64) - if err != nil { - return MetaRef{}, fmt.Errorf("error parsing checksum %s : %w", parts[4], err) - } - return MetaRef{ - Ref: Ref{ - TenantID: tenantID, - TableName: tableName, - Bounds: bounds, - StartTimestamp: model.Time(startTimestamp), - EndTimestamp: model.Time(endTimestamp), - Checksum: uint32(checksum), - }, - FilePath: objectKey, - }, nil -} - -func tablesForRange(periodConfig config.PeriodConfig, interval Interval) []string { - step := int64(periodConfig.IndexTables.Period.Seconds()) - lower := interval.Start.Unix() / step - upper := interval.End.Unix() / step - tables := make([]string, 0, 1+upper-lower) - for i := lower; i <= upper; i++ { - tables = append(tables, fmt.Sprintf("%s%d", periodConfig.IndexTables.Prefix, i)) +func findPeriod(configs []config.PeriodConfig, ts model.Time) (config.DayTime, error) { + for i := len(configs) - 1; i >= 0; i-- { + periodConfig := configs[i] + if !periodConfig.From.After(ts) { + return periodConfig.From, nil + } } - return tables + return config.DayTime{}, fmt.Errorf("can not find period for timestamp %d", ts) } diff --git a/pkg/storage/stores/shipper/bloomshipper/client_test.go b/pkg/storage/stores/shipper/bloomshipper/client_test.go index 80770a5735679..03a15f11bd3d3 100644 --- a/pkg/storage/stores/shipper/bloomshipper/client_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/client_test.go @@ -1,12 +1,11 @@ package bloomshipper import ( + "archive/tar" "bytes" "context" "encoding/json" - "fmt" "io" - "math/rand" "os" "path/filepath" "strings" @@ -19,10 +18,12 @@ import ( "github.com/prometheus/common/model" "github.com/stretchr/testify/require" + "github.com/grafana/loki/pkg/chunkenc" "github.com/grafana/loki/pkg/storage" v1 "github.com/grafana/loki/pkg/storage/bloom/v1" "github.com/grafana/loki/pkg/storage/chunk/cache" "github.com/grafana/loki/pkg/storage/config" + bloomshipperconfig "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper/config" ) const ( @@ -53,27 +54,26 @@ func Test_BloomClient_FetchMetas(t *testing.T) { store := createStore(t) var expected []Meta - folder1 := store.storageConfig.NamedStores.Filesystem["folder-1"].Directory + // metas that belong to 1st schema stored in folder-1 // must not be present in results because it is outside of time range - createMetaInStorage(t, folder1, "first-period-19621", "tenantA", 0, 100, fixedDay.Add(-7*day)) + createMetaInStorage(t, store, "19621", "tenantA", 0, 100, fixedDay.Add(-7*day)) // must be present in the results - expected = append(expected, createMetaInStorage(t, folder1, "first-period-19621", "tenantA", 0, 100, fixedDay.Add(-6*day))) + expected = append(expected, createMetaInStorage(t, store, "19621", "tenantA", 0, 100, fixedDay.Add(-6*day))) // must not be present in results because it belongs to another tenant - createMetaInStorage(t, folder1, "first-period-19621", "tenantB", 0, 100, fixedDay.Add(-6*day)) + createMetaInStorage(t, store, "19621", "tenantB", 0, 100, fixedDay.Add(-6*day)) // must be present in the results - expected = append(expected, createMetaInStorage(t, folder1, "first-period-19621", "tenantA", 101, 200, fixedDay.Add(-6*day))) + expected = append(expected, createMetaInStorage(t, store, "19621", "tenantA", 101, 200, fixedDay.Add(-6*day))) - folder2 := store.storageConfig.NamedStores.Filesystem["folder-2"].Directory + // metas that belong to 2nd schema stored in folder-2 // must not be present in results because it's out of the time range - createMetaInStorage(t, folder2, "second-period-19626", "tenantA", 0, 100, fixedDay.Add(-1*day)) + createMetaInStorage(t, store, "19626", "tenantA", 0, 100, fixedDay.Add(-1*day)) // must be present in the results - expected = append(expected, createMetaInStorage(t, folder2, "second-period-19625", "tenantA", 0, 100, fixedDay.Add(-2*day))) + expected = append(expected, createMetaInStorage(t, store, "19625", "tenantA", 0, 100, fixedDay.Add(-2*day))) // must not be present in results because it belongs to another tenant - createMetaInStorage(t, folder2, "second-period-19624", "tenantB", 0, 100, fixedDay.Add(-3*day)) + createMetaInStorage(t, store, "19624", "tenantB", 0, 100, fixedDay.Add(-3*day)) searchParams := MetaSearchParams{ TenantID: "tenantA", - Keyspace: v1.NewBounds(50, 150), Interval: NewInterval(fixedDay.Add(-6*day), fixedDay.Add(-1*day-1*time.Hour)), } @@ -82,7 +82,11 @@ func Test_BloomClient_FetchMetas(t *testing.T) { require.NoError(t, err) require.Equal(t, len(expected), len(fetched)) - require.ElementsMatch(t, expected, fetched) + for i := range expected { + require.Equal(t, expected[i].String(), fetched[i].String()) + require.ElementsMatch(t, expected[i].Blocks, fetched[i].Blocks) + require.ElementsMatch(t, expected[i].Tombstones, fetched[i].Tombstones) + } resolved, _, err := store.ResolveMetas(context.Background(), searchParams) require.NoError(t, err) @@ -104,29 +108,25 @@ func Test_BloomClient_PutMeta(t *testing.T) { }{ "expected meta to be uploaded to the first folder": { source: createMetaEntity("tenantA", - "first-period-19621", + "table_19621", 0xff, 0xfff, Date(2023, time.September, 21, 5, 0, 0), Date(2023, time.September, 21, 6, 0, 0), - 0xaaa, - "ignored-file-path-during-uploading", ), expectedStorage: "folder-1", - expectedFilePath: fmt.Sprintf("bloom/first-period-19621/tenantA/metas/%s-1695272400000-1695276000000-aaa", v1.NewBounds(0xff, 0xfff)), + expectedFilePath: "bloom/table_19621/tenantA/metas/00000000000000ff-0000000000000fff-0", }, "expected meta to be uploaded to the second folder": { source: createMetaEntity("tenantA", - "second-period-19625", + "table_19625", 200, 300, Date(2023, time.September, 25, 0, 0, 0), Date(2023, time.September, 25, 1, 0, 0), - 0xbbb, - "ignored-file-path-during-uploading", ), expectedStorage: "folder-2", - expectedFilePath: fmt.Sprintf("bloom/second-period-19625/tenantA/metas/%s-1695600000000-1695603600000-bbb", v1.NewBounds(200, 300)), + expectedFilePath: "bloom/table_19625/tenantA/metas/00000000000000c8-000000000000012c-0", }, } for name, data := range tests { @@ -138,7 +138,7 @@ func Test_BloomClient_PutMeta(t *testing.T) { directory := bloomClient.storageConfig.NamedStores.Filesystem[data.expectedStorage].Directory filePath := filepath.Join(directory, data.expectedFilePath) - require.FileExists(t, filePath) + require.FileExistsf(t, filePath, data.source.String()) content, err := os.ReadFile(filePath) require.NoError(t, err) result := Meta{} @@ -160,45 +160,43 @@ func Test_BloomClient_DeleteMeta(t *testing.T) { }{ "expected meta to be deleted from the first folder": { source: createMetaEntity("tenantA", - "first-period-19621", + "table_19621", 0xff, 0xfff, Date(2023, time.September, 21, 5, 0, 0), Date(2023, time.September, 21, 6, 0, 0), - 0xaaa, - "ignored-file-path-during-uploading", ), expectedStorage: "folder-1", - expectedFilePath: fmt.Sprintf("bloom/first-period-19621/tenantA/metas/%s-1695272400000-1695276000000-aaa", v1.NewBounds(0xff, 0xfff)), + expectedFilePath: "bloom/table_19621/tenantA/metas/00000000000000ff-0000000000000fff-0", }, "expected meta to be delete from the second folder": { source: createMetaEntity("tenantA", - "second-period-19625", + "table_19625", 200, 300, Date(2023, time.September, 25, 0, 0, 0), Date(2023, time.September, 25, 1, 0, 0), - 0xbbb, - "ignored-file-path-during-uploading", ), expectedStorage: "folder-2", - expectedFilePath: fmt.Sprintf("bloom/second-period-19625/tenantA/metas/%s-1695600000000-1695603600000-bbb", v1.NewBounds(200, 300)), + expectedFilePath: "bloom/table_19625/tenantA/metas/00000000000000c8-000000000000012c-0", }, } for name, data := range tests { t.Run(name, func(t *testing.T) { bloomClient := createStore(t) + directory := bloomClient.storageConfig.NamedStores.Filesystem[data.expectedStorage].Directory file := filepath.Join(directory, data.expectedFilePath) - err := os.MkdirAll(file[:strings.LastIndex(file, delimiter)], 0755) - require.NoError(t, err) - err = os.WriteFile(file, []byte("dummy content"), 0700) + + // requires that Test_BloomClient_PutMeta does not fail + err := bloomClient.PutMeta(context.Background(), data.source) require.NoError(t, err) + require.FileExists(t, file, data.source.String()) - err = bloomClient.DeleteMeta(context.Background(), data.source) + err = bloomClient.DeleteMetas(context.Background(), []MetaRef{data.source.MetaRef}) require.NoError(t, err) - require.NoFileExists(t, file) + require.NoFileExists(t, file, data.source.String()) }) } @@ -208,7 +206,7 @@ func Test_BloomClient_GetBlocks(t *testing.T) { firstBlockRef := BlockRef{ Ref: Ref{ TenantID: "tenantA", - TableName: "first-period-19621", + TableName: "schema_a_table_19621", Bounds: v1.NewBounds(0xeeee, 0xffff), StartTimestamp: Date(2023, time.September, 21, 5, 0, 0), EndTimestamp: Date(2023, time.September, 21, 6, 0, 0), @@ -218,7 +216,7 @@ func Test_BloomClient_GetBlocks(t *testing.T) { secondBlockRef := BlockRef{ Ref: Ref{ TenantID: "tenantA", - TableName: "second-period-19624", + TableName: "schema_b_table_19624", Bounds: v1.NewBounds(0xaaaa, 0xbbbb), StartTimestamp: Date(2023, time.September, 24, 5, 0, 0), EndTimestamp: Date(2023, time.September, 24, 6, 0, 0), @@ -227,31 +225,34 @@ func Test_BloomClient_GetBlocks(t *testing.T) { } bloomClient := createStore(t) + fsNamedStores := bloomClient.storageConfig.NamedStores.Filesystem + firstBlockFullPath := NewPrefixedResolver( fsNamedStores["folder-1"].Directory, defaultKeyResolver{}, ).Block(firstBlockRef).LocalPath() - firstBlockData := createBlockFile(t, firstBlockFullPath) + _ = createBlockFile(t, firstBlockFullPath) + require.FileExists(t, firstBlockFullPath) + secondBlockFullPath := NewPrefixedResolver( fsNamedStores["folder-2"].Directory, defaultKeyResolver{}, ).Block(secondBlockRef).LocalPath() - secondBlockData := createBlockFile(t, secondBlockFullPath) - require.FileExists(t, firstBlockFullPath) + _ = createBlockFile(t, secondBlockFullPath) require.FileExists(t, secondBlockFullPath) - downloadedFirstBlock, err := bloomClient.GetBlock(context.Background(), firstBlockRef) - require.NoError(t, err) - firstBlockActualData, err := io.ReadAll(downloadedFirstBlock.Data) + _, err := bloomClient.GetBlock(context.Background(), firstBlockRef) require.NoError(t, err) - require.Equal(t, firstBlockData, string(firstBlockActualData)) + // firstBlockActualData, err := io.ReadAll(downloadedFirstBlock.Data) + // require.NoError(t, err) + // require.Equal(t, firstBlockData, string(firstBlockActualData)) - downloadedSecondBlock, err := bloomClient.GetBlock(context.Background(), secondBlockRef) - require.NoError(t, err) - secondBlockActualData, err := io.ReadAll(downloadedSecondBlock.Data) + _, err = bloomClient.GetBlock(context.Background(), secondBlockRef) require.NoError(t, err) - require.Equal(t, secondBlockData, string(secondBlockActualData)) + // secondBlockActualData, err := io.ReadAll(downloadedSecondBlock.Data) + // require.NoError(t, err) + // require.Equal(t, secondBlockData, string(secondBlockActualData)) } func Test_BloomClient_PutBlocks(t *testing.T) { @@ -260,7 +261,7 @@ func Test_BloomClient_PutBlocks(t *testing.T) { BlockRef: BlockRef{ Ref: Ref{ TenantID: "tenantA", - TableName: "first-period-19621", + TableName: "table_19621", Bounds: v1.NewBounds(0xeeee, 0xffff), StartTimestamp: Date(2023, time.September, 21, 5, 0, 0), EndTimestamp: Date(2023, time.September, 21, 6, 0, 0), @@ -269,21 +270,25 @@ func Test_BloomClient_PutBlocks(t *testing.T) { }, Data: awsio.ReadSeekNopCloser{ReadSeeker: bytes.NewReader([]byte("data"))}, } - _, err := bloomClient.PutBlocks(context.Background(), []Block{block}) - require.NoError(t, err) - got, err := bloomClient.GetBlock(context.Background(), block.BlockRef) - require.NoError(t, err) - require.Equal(t, block.BlockRef, got.BlockRef) - data, err := io.ReadAll(got.Data) + err := bloomClient.PutBlock(context.Background(), block) require.NoError(t, err) - require.Equal(t, "data", string(data)) + + _ = bloomClient.storeDo(block.StartTimestamp, func(s *bloomStoreEntry) error { + c := s.bloomClient.(*BloomClient) + rc, _, err := c.client.GetObject(context.Background(), block.BlockRef.String()) + require.NoError(t, err) + data, err := io.ReadAll(rc) + require.NoError(t, err) + require.Equal(t, "data", string(data)) + return nil + }) } func Test_BloomClient_DeleteBlocks(t *testing.T) { block := BlockRef{ Ref: Ref{ TenantID: "tenantA", - TableName: "first-period-19621", + TableName: "table_19621", Bounds: v1.NewBounds(0xeeee, 0xffff), StartTimestamp: Date(2023, time.September, 21, 5, 0, 0), EndTimestamp: Date(2023, time.September, 21, 6, 0, 0), @@ -306,81 +311,90 @@ func Test_BloomClient_DeleteBlocks(t *testing.T) { } -func createBlockFile(t *testing.T, path string) string { - err := os.MkdirAll(path[:strings.LastIndex(path, "/")], 0755) +func createBlockFile(t *testing.T, dst string) string { + err := os.MkdirAll(dst[:strings.LastIndex(dst, "/")], 0755) require.NoError(t, err) fileContent := uuid.NewString() - err = os.WriteFile(path, []byte(fileContent), 0700) + + src := filepath.Join(t.TempDir(), fileContent) + err = os.WriteFile(src, []byte(fileContent), 0700) require.NoError(t, err) + + fp, err := os.OpenFile(dst, os.O_CREATE|os.O_RDWR, 0700) + require.NoError(t, err) + defer fp.Close() + + TarGz(t, fp, src) + return fileContent } -func Test_createMetaRef(t *testing.T) { - tests := map[string]struct { - objectKey string - tenantID string - tableName string +func TarGz(t *testing.T, dst io.Writer, file string) { + src, err := os.Open(file) + require.NoError(t, err) + defer src.Close() + + gzipper := chunkenc.GetWriterPool(chunkenc.EncGZIP).GetWriter(dst) + defer gzipper.Close() + + tarballer := tar.NewWriter(gzipper) + defer tarballer.Close() + + for _, f := range []*os.File{src} { + info, err := f.Stat() + require.NoError(t, err) + + header, err := tar.FileInfoHeader(info, f.Name()) + require.NoError(t, err) + + err = tarballer.WriteHeader(header) + require.NoError(t, err) + + _, err = io.Copy(tarballer, f) + require.NoError(t, err) + } +} +func Test_ParseMetaKey(t *testing.T) { + tests := map[string]struct { + objectKey string expectedRef MetaRef expectedErr string }{ "ValidObjectKey": { - objectKey: "bloom/ignored-during-parsing-table-name/ignored-during-parsing-tenant-ID/metas/aaa-bbb-1234567890-9876543210-abcdef", - tenantID: "tenant1", - tableName: "table1", + objectKey: "bloom/table/tenant/metas/aaa-bbb-abcdef", expectedRef: MetaRef{ Ref: Ref{ - TenantID: "tenant1", - TableName: "table1", + TenantID: "tenant", + TableName: "table", Bounds: v1.NewBounds(0xaaa, 0xbbb), - StartTimestamp: 1234567890, - EndTimestamp: 9876543210, + StartTimestamp: 0, // ignored + EndTimestamp: 0, // ignored Checksum: 0xabcdef, }, - FilePath: "bloom/ignored-during-parsing-table-name/ignored-during-parsing-tenant-ID/metas/aaa-bbb-1234567890-9876543210-abcdef", }, }, "InvalidObjectKeyDelimiterCount": { objectKey: "invalid/key/with/too/many/objectKeyWithoutDelimiters", - tenantID: "tenant1", - tableName: "table1", expectedRef: MetaRef{}, - expectedErr: "filename parts count must be 5 but was 1: [objectKeyWithoutDelimiters]", + expectedErr: "failed to split filename parts", }, "InvalidMinFingerprint": { - objectKey: "invalid/folder/key/metas/zzz-bbb-123-9876543210-abcdef", - tenantID: "tenant1", - tableName: "table1", - expectedErr: "error parsing minFingerprint zzz", + objectKey: "invalid/folder/key/metas/zzz-bbb-abcdef", + expectedErr: "failed to parse bounds", }, "InvalidMaxFingerprint": { - objectKey: "invalid/folder/key/metas/123-zzz-1234567890-9876543210-abcdef", - tenantID: "tenant1", - tableName: "table1", - expectedErr: "error parsing maxFingerprint zzz", - }, - "InvalidStartTimestamp": { - objectKey: "invalid/folder/key/metas/aaa-bbb-abc-9876543210-abcdef", - tenantID: "tenant1", - tableName: "table1", - expectedErr: "error parsing startTimestamp abc", - }, - "InvalidEndTimestamp": { - objectKey: "invalid/folder/key/metas/aaa-bbb-1234567890-xyz-abcdef", - tenantID: "tenant1", - tableName: "table1", - expectedErr: "error parsing endTimestamp xyz", + objectKey: "invalid/folder/key/metas/123-zzz-abcdef", + expectedErr: "failed to parse bounds", }, "InvalidChecksum": { - objectKey: "invalid/folder/key/metas/aaa-bbb-1234567890-9876543210-ghijklm", - tenantID: "tenant1", - tableName: "table1", - expectedErr: "error parsing checksum ghijklm", + objectKey: "invalid/folder/key/metas/aaa-bbb-ghijklm", + expectedErr: "failed to parse checksum", }, } for name, data := range tests { t.Run(name, func(t *testing.T) { - actualRef, err := createMetaRef(data.objectKey, data.tenantID, data.tableName) + actualRef, err := defaultKeyResolver{}.ParseMetaKey(key(data.objectKey)) if data.expectedErr != "" { require.ErrorContains(t, err, data.expectedErr) return @@ -400,7 +414,15 @@ func createStore(t *testing.T) *BloomStore { }} //required to populate StoreType map in named config require.NoError(t, namedStores.Validate()) - storageConfig := storage.Config{NamedStores: namedStores} + storageConfig := storage.Config{ + NamedStores: namedStores, + BloomShipperConfig: bloomshipperconfig.Config{ + WorkingDirectory: t.TempDir(), + BlocksDownloadingQueue: bloomshipperconfig.DownloadingQueueConfig{ + WorkersCount: 1, + }, + }, + } metrics := storage.NewClientMetrics() t.Cleanup(metrics.Unregister) @@ -418,7 +440,8 @@ func createPeriodConfigs() []config.PeriodConfig { IndexTables: config.IndexPeriodicTableConfig{ PeriodicTableConfig: config.PeriodicTableConfig{ Period: day, - Prefix: "first-period-", + // TODO(chaudum): Integrate {,Parse}MetaKey into schema config + // Prefix: "schema_a_table_", }}, }, { @@ -428,28 +451,21 @@ func createPeriodConfigs() []config.PeriodConfig { IndexTables: config.IndexPeriodicTableConfig{ PeriodicTableConfig: config.PeriodicTableConfig{ Period: day, - Prefix: "second-period-", + // TODO(chaudum): Integrate {,Parse}MetaKey into schema config + // Prefix: "schema_b_table_", }}, }, } return periodicConfigs } -func createMetaInStorage(t *testing.T, folder string, tableName string, tenant string, minFingerprint uint64, maxFingerprint uint64, start model.Time) Meta { +func createMetaInStorage(t *testing.T, s Client, tableName string, tenant string, minFingerprint uint64, maxFingerprint uint64, start model.Time) Meta { end := start.Add(12 * time.Hour) - metaChecksum := rand.Uint32() - // make sure this is equal to the createMetaObjectKey() - metaFileName := fmt.Sprintf("%x-%x-%d-%d-%x", minFingerprint, maxFingerprint, start, end, metaChecksum) - metaFilePath := filepath.Join(rootFolder, tableName, tenant, metasFolder, metaFileName) - err := os.MkdirAll(filepath.Join(folder, metaFilePath[:strings.LastIndex(metaFilePath, delimiter)]), 0700) - require.NoError(t, err) - meta := createMetaEntity(tenant, tableName, minFingerprint, maxFingerprint, start, end, metaChecksum, metaFilePath) - - metaFileContent, err := json.Marshal(meta) - require.NoError(t, err) - err = os.WriteFile(filepath.Join(folder, metaFilePath), metaFileContent, 0644) + meta := createMetaEntity(tenant, tableName, minFingerprint, maxFingerprint, start, end) + err := s.PutMeta(context.Background(), meta) require.NoError(t, err) + t.Log("create meta in store", meta.String()) return meta } @@ -460,8 +476,7 @@ func createMetaEntity( maxFingerprint uint64, startTimestamp model.Time, endTimestamp model.Time, - metaChecksum uint32, - metaFilePath string) Meta { +) Meta { return Meta{ MetaRef: MetaRef{ Ref: Ref{ @@ -470,15 +485,12 @@ func createMetaEntity( Bounds: v1.NewBounds(model.Fingerprint(minFingerprint), model.Fingerprint(maxFingerprint)), StartTimestamp: startTimestamp, EndTimestamp: endTimestamp, - Checksum: metaChecksum, }, - FilePath: metaFilePath, }, Tombstones: []BlockRef{ { Ref: Ref{ TenantID: tenant, - Checksum: metaChecksum + 1, Bounds: v1.NewBounds(model.Fingerprint(minFingerprint), model.Fingerprint(maxFingerprint)), StartTimestamp: startTimestamp, EndTimestamp: endTimestamp, @@ -489,7 +501,6 @@ func createMetaEntity( { Ref: Ref{ TenantID: tenant, - Checksum: metaChecksum + 2, Bounds: v1.NewBounds(model.Fingerprint(minFingerprint), model.Fingerprint(maxFingerprint)), StartTimestamp: startTimestamp, EndTimestamp: endTimestamp, diff --git a/pkg/storage/stores/shipper/bloomshipper/compress_utils.go b/pkg/storage/stores/shipper/bloomshipper/compress_utils.go index 5f11e81a5b3b6..332c900fe29d5 100644 --- a/pkg/storage/stores/shipper/bloomshipper/compress_utils.go +++ b/pkg/storage/stores/shipper/bloomshipper/compress_utils.go @@ -55,3 +55,25 @@ func extractArchive(archivePath string, workingDirectoryPath string) error { } return v1.UnTarGz(workingDirectoryPath, file) } + +func extractBlock(data io.ReadCloser, blockDir string, logger log.Logger) error { + err := os.MkdirAll(blockDir, os.ModePerm) + if err != nil { + return fmt.Errorf("can not create directory to extract the block: %w", err) + } + archivePath, err := writeDataToTempFile(blockDir, data) + if err != nil { + return fmt.Errorf("error writing data to temp file: %w", err) + } + defer func() { + err = os.Remove(archivePath) + if err != nil { + level.Error(logger).Log("msg", "error removing temp archive file", "err", err) + } + }() + err = extractArchive(archivePath, blockDir) + if err != nil { + return fmt.Errorf("error extracting archive: %w", err) + } + return nil +} diff --git a/pkg/storage/stores/shipper/bloomshipper/compress_utils_test.go b/pkg/storage/stores/shipper/bloomshipper/compress_utils_test.go new file mode 100644 index 0000000000000..4fddf8e9c3485 --- /dev/null +++ b/pkg/storage/stores/shipper/bloomshipper/compress_utils_test.go @@ -0,0 +1,69 @@ +package bloomshipper + +import ( + "bytes" + "io" + "os" + "path/filepath" + "testing" + + "github.com/google/uuid" + "github.com/stretchr/testify/require" + + v1 "github.com/grafana/loki/pkg/storage/bloom/v1" +) + +func Test_blockDownloader_extractBlock(t *testing.T) { + blockFilePath, _, bloomFileContent, seriesFileContent := createBlockArchive(t) + blockFile, err := os.OpenFile(blockFilePath, os.O_RDONLY, 0700) + require.NoError(t, err) + + workingDir := t.TempDir() + + err = extractBlock(blockFile, workingDir, nil) + require.NoError(t, err) + + require.FileExists(t, filepath.Join(workingDir, v1.BloomFileName)) + require.FileExists(t, filepath.Join(workingDir, v1.SeriesFileName)) + + actualBloomFileContent, err := os.ReadFile(filepath.Join(workingDir, v1.BloomFileName)) + require.NoError(t, err) + require.Equal(t, bloomFileContent, string(actualBloomFileContent)) + + actualSeriesFileContent, err := os.ReadFile(filepath.Join(workingDir, v1.SeriesFileName)) + require.NoError(t, err) + require.Equal(t, seriesFileContent, string(actualSeriesFileContent)) +} + +func directoryDoesNotExist(path string) bool { + _, err := os.Lstat(path) + return err != nil +} + +const testArchiveFileName = "test-block-archive" + +func createBlockArchive(t *testing.T) (string, string, string, string) { + dir := t.TempDir() + mockBlockDir := filepath.Join(dir, "mock-block-dir") + err := os.MkdirAll(mockBlockDir, 0777) + require.NoError(t, err) + bloomFile, err := os.Create(filepath.Join(mockBlockDir, v1.BloomFileName)) + require.NoError(t, err) + bloomFileContent := uuid.NewString() + _, err = io.Copy(bloomFile, bytes.NewReader([]byte(bloomFileContent))) + require.NoError(t, err) + + seriesFile, err := os.Create(filepath.Join(mockBlockDir, v1.SeriesFileName)) + require.NoError(t, err) + seriesFileContent := uuid.NewString() + _, err = io.Copy(seriesFile, bytes.NewReader([]byte(seriesFileContent))) + require.NoError(t, err) + + blockFilePath := filepath.Join(dir, testArchiveFileName) + file, err := os.OpenFile(blockFilePath, os.O_CREATE|os.O_RDWR, 0700) + require.NoError(t, err) + err = v1.TarGz(file, v1.NewDirectoryBlockReader(mockBlockDir)) + require.NoError(t, err) + + return blockFilePath, mockBlockDir, bloomFileContent, seriesFileContent +} diff --git a/pkg/storage/stores/shipper/bloomshipper/fetcher.go b/pkg/storage/stores/shipper/bloomshipper/fetcher.go index 0b1dae5ce691f..cfb0f392d6931 100644 --- a/pkg/storage/stores/shipper/bloomshipper/fetcher.go +++ b/pkg/storage/stores/shipper/bloomshipper/fetcher.go @@ -3,11 +3,17 @@ package bloomshipper import ( "context" "encoding/json" - "io" + "fmt" + "os" + "path/filepath" + "sync" "github.com/go-kit/log" + "github.com/go-kit/log/level" "github.com/pkg/errors" + "k8s.io/utils/keymutex" + v1 "github.com/grafana/loki/pkg/storage/bloom/v1" "github.com/grafana/loki/pkg/storage/chunk/cache" ) @@ -16,27 +22,40 @@ type metrics struct{} type fetcher interface { FetchMetas(ctx context.Context, refs []MetaRef) ([]Meta, error) - // TODO(chaudum): Integrate block fetching - // FetchBlocks(ctx context.Context, refs []BlockRef) ([]Block, error) + FetchBlocks(ctx context.Context, refs []BlockRef) ([]BlockDirectory, error) + Close() } +// Compiler check to ensure Fetcher implements the fetcher interface +var _ fetcher = &Fetcher{} + type Fetcher struct { client Client - metasCache cache.Cache - blocksCache *cache.EmbeddedCache[string, io.ReadCloser] + metasCache cache.Cache + blocksCache *cache.EmbeddedCache[string, BlockDirectory] + localFSResolver KeyResolver + + q *downloadQueue[BlockRef, BlockDirectory] metrics *metrics logger log.Logger } -func NewFetcher(client Client, metasCache cache.Cache, blocksCache *cache.EmbeddedCache[string, io.ReadCloser], logger log.Logger) (*Fetcher, error) { - return &Fetcher{ - client: client, - metasCache: metasCache, - blocksCache: blocksCache, - logger: logger, - }, nil +func NewFetcher(cfg bloomStoreConfig, client Client, metasCache cache.Cache, blocksCache *cache.EmbeddedCache[string, BlockDirectory], logger log.Logger) (*Fetcher, error) { + fetcher := &Fetcher{ + client: client, + metasCache: metasCache, + blocksCache: blocksCache, + localFSResolver: NewPrefixedResolver(cfg.workingDir, defaultKeyResolver{}), + logger: logger, + } + fetcher.q = newDownloadQueue[BlockRef, BlockDirectory](1000, cfg.numWorkers, fetcher.processTask, logger) + return fetcher, nil +} + +func (f *Fetcher) Close() { + f.q.close() } func (f *Fetcher) FetchMetas(ctx context.Context, refs []MetaRef) ([]Meta, error) { @@ -46,14 +65,14 @@ func (f *Fetcher) FetchMetas(ctx context.Context, refs []MetaRef) ([]Meta, error keys := make([]string, 0, len(refs)) for _, ref := range refs { - keys = append(keys, externalMetaKey(ref)) + keys = append(keys, f.client.Meta(ref).Addr()) } cacheHits, cacheBufs, _, err := f.metasCache.Fetch(ctx, keys) if err != nil { return nil, err } - fromCache, missing, err := f.processCacheResponse(ctx, refs, cacheHits, cacheBufs) + fromCache, missing, err := f.processMetasCacheResponse(ctx, refs, cacheHits, cacheBufs) if err != nil { return nil, err } @@ -63,13 +82,11 @@ func (f *Fetcher) FetchMetas(ctx context.Context, refs []MetaRef) ([]Meta, error return nil, err } - // TODO(chaudum): Make async err = f.writeBackMetas(ctx, fromStorage) return append(fromCache, fromStorage...), err } -func (f *Fetcher) processCacheResponse(_ context.Context, refs []MetaRef, keys []string, bufs [][]byte) ([]Meta, []MetaRef, error) { - +func (f *Fetcher) processMetasCacheResponse(_ context.Context, refs []MetaRef, keys []string, bufs [][]byte) ([]Meta, []MetaRef, error) { found := make(map[string][]byte, len(refs)) for i, k := range keys { found[k] = bufs[i] @@ -80,7 +97,7 @@ func (f *Fetcher) processCacheResponse(_ context.Context, refs []MetaRef, keys [ var lastErr error for i, ref := range refs { - if raw, ok := found[externalMetaKey(ref)]; ok { + if raw, ok := found[f.client.Meta(ref).Addr()]; ok { meta := Meta{ MetaRef: ref, } @@ -99,7 +116,7 @@ func (f *Fetcher) writeBackMetas(ctx context.Context, metas []Meta) error { keys := make([]string, len(metas)) data := make([][]byte, len(metas)) for i := range metas { - keys[i] = externalMetaKey(metas[i].MetaRef) + keys[i] = f.client.Meta(metas[i].MetaRef).Addr() data[i], err = json.Marshal(metas[i]) } if err != nil { @@ -108,71 +125,231 @@ func (f *Fetcher) writeBackMetas(ctx context.Context, metas []Meta) error { return f.metasCache.Store(ctx, keys, data) } -// TODO(chaudum): Integrate block fetching - -// func (f *Fetcher) FetchBlocks(ctx context.Context, refs []BlockRef) (v1.Iterator[Block], error) { -// if ctx.Err() != nil { -// return nil, errors.Wrap(ctx.Err(), "fetch Blocks") -// } - -// keys := make([]string, 0, len(refs)) -// for _, ref := range refs { -// keys = append(keys, externalBlockKey(ref)) -// } -// found, blocksFromCache, missing, err := f.blocksCache.Fetch(ctx, keys) -// if err != nil { -// return nil, err -// } - -// if len(missing) > 0 { -// for _, key := range missing { -// for i, ref := range refs { -// if key == externalBlockKey(ref) { -// refs = append(refs[:i], refs[i+1:]...) -// i-- -// } -// } -// } - -// blocksFromStorage, err := f.client.GetBlock(ctx, refs) -// if err != nil { -// return nil, err -// } -// } - -// return nil, nil -// } - -// func (f *Fetcher) writeBackBlocks(ctx context.Context, blocks []Block) error { -// keys := make([]string, 0, len(blocks)) -// data := make([]io.ReadCloser, 0, len(blocks)) -// return f.blocksCache.Store(ctx, keys, data) -// } - -// type ChannelIter[T any] struct { -// ch <-chan T -// cur T -// } - -// func NewChannelIter[T any](ch <-chan T) *ChannelIter[T] { -// return &ChannelIter[T]{ -// ch: ch, -// } -// } - -// func (it *ChannelIter[T]) Next() bool { -// el, ok := <-it.ch -// if ok { -// it.cur = el -// return true -// } -// return false -// } - -// func (it *ChannelIter[T]) At() T { -// return it.cur -// } - -// func (it *ChannelIter[T]) Err() error { -// return nil -// } +func (f *Fetcher) FetchBlocksWithQueue(ctx context.Context, refs []BlockRef) ([]BlockDirectory, error) { + responses := make(chan BlockDirectory, len(refs)) + errors := make(chan error, len(refs)) + for _, ref := range refs { + f.q.enqueue(downloadTask[BlockRef, BlockDirectory]{ + ctx: ctx, + item: ref, + key: f.client.Block(ref).Addr(), + results: responses, + errors: errors, + }) + } + + results := make([]BlockDirectory, len(refs)) + +outer: + for i := 0; i < len(refs); i++ { + select { + case err := <-errors: + return results, err + case res := <-responses: + for j, ref := range refs { + if res.BlockRef == ref { + results[j] = res + continue outer + } + } + return results, fmt.Errorf("no matching request found for response %s", res) + } + } + + return results, nil +} + +func (f *Fetcher) processTask(ctx context.Context, task downloadTask[BlockRef, BlockDirectory]) { + if ctx.Err() != nil { + task.errors <- ctx.Err() + return + } + + refs := []BlockRef{task.item} + results, err := f.FetchBlocks(ctx, refs) + if err != nil { + task.errors <- err + return + } + + for _, res := range results { + task.results <- res + } +} + +// FetchBlocks returns a list of block directories +// It resolves them from three locations: +// 1. from cache +// 2. from file system +// 3. from remote storage +func (f *Fetcher) FetchBlocks(ctx context.Context, refs []BlockRef) ([]BlockDirectory, error) { + if ctx.Err() != nil { + return nil, errors.Wrap(ctx.Err(), "fetch Blocks") + } + + keys := make([]string, 0, len(refs)) + for _, ref := range refs { + keys = append(keys, f.client.Block(ref).Addr()) + } + cacheHits, cacheBufs, _, err := f.blocksCache.Fetch(ctx, keys) + if err != nil { + return nil, err + } + + results := make([]BlockDirectory, 0, len(refs)) + + fromCache, missing, err := f.processBlocksCacheResponse(ctx, refs, cacheHits, cacheBufs) + if err != nil { + return nil, err + } + results = append(results, fromCache...) + + fromLocalFS, missing, err := f.loadBlocksFromFS(ctx, missing) + if err != nil { + return nil, err + } + results = append(results, fromLocalFS...) + + fromStorage, err := f.client.GetBlocks(ctx, missing) + if err != nil { + return nil, err + } + results = append(results, fromStorage...) + + err = f.writeBackBlocks(ctx, fromStorage) + return results, err +} + +func (f *Fetcher) processBlocksCacheResponse(_ context.Context, refs []BlockRef, keys []string, entries []BlockDirectory) ([]BlockDirectory, []BlockRef, error) { + found := make(map[string]BlockDirectory, len(refs)) + for i, k := range keys { + found[k] = entries[i] + } + + blockDirs := make([]BlockDirectory, 0, len(found)) + missing := make([]BlockRef, 0, len(refs)-len(keys)) + + var lastErr error + for i, ref := range refs { + if raw, ok := found[f.client.Block(ref).Addr()]; ok { + blockDirs = append(blockDirs, raw) + } else { + missing = append(missing, refs[i]) + } + } + + return blockDirs, missing, lastErr +} + +func (f *Fetcher) loadBlocksFromFS(_ context.Context, refs []BlockRef) ([]BlockDirectory, []BlockRef, error) { + blockDirs := make([]BlockDirectory, 0, len(refs)) + missing := make([]BlockRef, 0, len(refs)) + + for _, ref := range refs { + path := f.localFSResolver.Block(ref).LocalPath() + if ok, clean := f.isBlockDir(path); ok { + blockDirs = append(blockDirs, NewBlockDirectory(ref, path, f.logger)) + } else { + _ = clean(path) + missing = append(missing, ref) + } + } + + return blockDirs, missing, nil +} + +var noopClean = func(string) error { return nil } + +func (f *Fetcher) isBlockDir(path string) (bool, func(string) error) { + info, err := os.Stat(path) + if err != nil && os.IsNotExist(err) { + level.Warn(f.logger).Log("msg", "path does not exist", "path", path) + return false, noopClean + } + if !info.IsDir() { + return false, os.Remove + } + for _, file := range []string{ + filepath.Join(path, v1.BloomFileName), + filepath.Join(path, v1.SeriesFileName), + } { + if _, err := os.Stat(file); err != nil && os.IsNotExist(err) { + level.Warn(f.logger).Log("msg", "path does not contain required file", "path", path, "file", file) + return false, os.RemoveAll + } + } + return true, nil +} + +func (f *Fetcher) writeBackBlocks(ctx context.Context, blocks []BlockDirectory) error { + keys := make([]string, len(blocks)) + for i := range blocks { + keys[i] = f.client.Block(blocks[i].BlockRef).Addr() + } + return f.blocksCache.Store(ctx, keys, blocks) +} + +type processFunc[T any, R any] func(context.Context, downloadTask[T, R]) + +type downloadTask[T any, R any] struct { + ctx context.Context + item T + key string + results chan<- R + errors chan<- error +} + +type downloadQueue[T any, R any] struct { + queue chan downloadTask[T, R] + mu keymutex.KeyMutex + wg sync.WaitGroup + done chan struct{} + process processFunc[T, R] + logger log.Logger +} + +func newDownloadQueue[T any, R any](size, workers int, process processFunc[T, R], logger log.Logger) *downloadQueue[T, R] { + q := &downloadQueue[T, R]{ + queue: make(chan downloadTask[T, R], size), + mu: keymutex.NewHashed(workers), + done: make(chan struct{}), + process: process, + logger: logger, + } + for i := 0; i < workers; i++ { + q.wg.Add(1) + go q.runWorker() + } + return q +} + +func (q *downloadQueue[T, R]) enqueue(t downloadTask[T, R]) { + q.queue <- t +} + +func (q *downloadQueue[T, R]) runWorker() { + defer q.wg.Done() + for { + select { + case <-q.done: + return + case task := <-q.queue: + q.do(task.ctx, task) + } + } +} + +func (q *downloadQueue[T, R]) do(ctx context.Context, task downloadTask[T, R]) { + q.mu.LockKey(task.key) + defer func() { + err := q.mu.UnlockKey(task.key) + level.Error(q.logger).Log("msg", "failed to unlock key in block lock", "err", err) + }() + + q.process(ctx, task) +} + +func (q *downloadQueue[T, R]) close() { + close(q.done) + q.wg.Wait() +} diff --git a/pkg/storage/stores/shipper/bloomshipper/fetcher_test.go b/pkg/storage/stores/shipper/bloomshipper/fetcher_test.go index 3bb3e1348f1bd..fd1e6157a1cae 100644 --- a/pkg/storage/stores/shipper/bloomshipper/fetcher_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/fetcher_test.go @@ -4,7 +4,8 @@ import ( "context" "encoding/json" "fmt" - "path" + "os" + "path/filepath" "testing" "time" @@ -36,7 +37,6 @@ func makeMetas(t *testing.T, schemaCfg config.SchemaConfig, ts model.Time, keysp Tombstones: []BlockRef{}, Blocks: []BlockRef{}, } - metas[i].FilePath = externalMetaKey(metas[i].MetaRef) } return metas } @@ -100,14 +100,15 @@ func TestMetasFetcher(t *testing.T) { t.Run(test.name, func(t *testing.T) { ctx := context.Background() metasCache := cache.NewMockCache() + cfg := bloomStoreConfig{workingDir: t.TempDir(), numWorkers: 1} oc, err := local.NewFSObjectClient(local.FSConfig{Directory: dir}) require.NoError(t, err) - c, err := NewBloomClient(oc, logger) + c, err := NewBloomClient(cfg, oc, logger) require.NoError(t, err) - fetcher, err := NewFetcher(c, metasCache, nil, logger) + fetcher, err := NewFetcher(cfg, c, metasCache, nil, logger) require.NoError(t, err) // prepare metas cache @@ -117,16 +118,14 @@ func TestMetasFetcher(t *testing.T) { b, err := json.Marshal(meta) require.NoError(t, err) metas = append(metas, b) - t.Log(string(b)) - k := externalMetaKey(meta.MetaRef) + k := meta.String() keys = append(keys, k) } require.NoError(t, metasCache.Store(ctx, keys, metas)) // prepare store for _, meta := range test.store { - meta.FilePath = path.Join(dir, meta.FilePath) err := c.PutMeta(ctx, meta) require.NoError(t, err) } @@ -140,13 +139,120 @@ func TestMetasFetcher(t *testing.T) { } } +func TestFetcher_LoadBlocksFromFS(t *testing.T) { + base := t.TempDir() + cfg := bloomStoreConfig{workingDir: base, numWorkers: 1} + resolver := NewPrefixedResolver(base, defaultKeyResolver{}) + + refs := []BlockRef{ + // no directory for block + {Ref: Ref{TenantID: "tenant", TableName: "12345", Bounds: v1.NewBounds(0x0000, 0x0fff)}}, + // invalid directory for block + {Ref: Ref{TenantID: "tenant", TableName: "12345", Bounds: v1.NewBounds(0x1000, 0x1fff)}}, + // valid directory for block + {Ref: Ref{TenantID: "tenant", TableName: "12345", Bounds: v1.NewBounds(0x2000, 0x2fff)}}, + } + dirs := []string{ + resolver.Block(refs[0]).LocalPath(), + resolver.Block(refs[1]).LocalPath(), + resolver.Block(refs[2]).LocalPath(), + } + + createBlockDir(t, dirs[1]) + _ = os.Remove(filepath.Join(dirs[1], "bloom")) // remove file to make it invalid + + createBlockDir(t, dirs[2]) + + oc, err := local.NewFSObjectClient(local.FSConfig{Directory: base}) + require.NoError(t, err) + c, err := NewBloomClient(cfg, oc, log.NewNopLogger()) + require.NoError(t, err) + + fetcher, err := NewFetcher(cfg, c, nil, nil, log.NewNopLogger()) + require.NoError(t, err) + + found, missing, err := fetcher.loadBlocksFromFS(context.Background(), refs) + require.NoError(t, err) + + require.Len(t, found, 1) + require.Len(t, missing, 2) + + require.Equal(t, refs[2], found[0].BlockRef) + require.ElementsMatch(t, refs[0:2], missing) +} + +func createBlockDir(t *testing.T, path string) { + _ = os.MkdirAll(path, 0755) + + fp, err := os.Create(filepath.Join(path, v1.BloomFileName)) + require.NoError(t, err) + _ = fp.Close() + + fp, err = os.Create(filepath.Join(path, v1.SeriesFileName)) + require.NoError(t, err) + _ = fp.Close() +} + +func TestFetcher_IsBlockDir(t *testing.T) { + fetcher, _ := NewFetcher(bloomStoreConfig{}, nil, nil, nil, log.NewNopLogger()) + + t.Run("path does not exist", func(t *testing.T) { + base := t.TempDir() + exists, _ := fetcher.isBlockDir(filepath.Join(base, "doesnotexist")) + require.False(t, exists) + }) + + t.Run("path is not a directory", func(t *testing.T) { + base := t.TempDir() + fp, err := os.Create(filepath.Join(base, "block")) + require.NoError(t, err) + _ = fp.Close() + exists, _ := fetcher.isBlockDir(filepath.Join(base, "block")) + require.False(t, exists) + }) + + t.Run("bloom file does not exist", func(t *testing.T) { + base := t.TempDir() + dir := filepath.Join(base, "block") + _ = os.MkdirAll(dir, 0755) + fp, err := os.Create(filepath.Join(dir, v1.SeriesFileName)) + require.NoError(t, err) + _ = fp.Close() + exists, _ := fetcher.isBlockDir(dir) + require.False(t, exists) + }) + + t.Run("series file does not exist", func(t *testing.T) { + base := t.TempDir() + dir := filepath.Join(base, "block") + _ = os.MkdirAll(dir, 0755) + fp, err := os.Create(filepath.Join(dir, v1.BloomFileName)) + require.NoError(t, err) + _ = fp.Close() + exists, _ := fetcher.isBlockDir(dir) + require.False(t, exists) + }) + + t.Run("valid directory", func(t *testing.T) { + base := t.TempDir() + dir := filepath.Join(base, "block") + _ = os.MkdirAll(dir, 0755) + fp, err := os.Create(filepath.Join(dir, v1.BloomFileName)) + require.NoError(t, err) + _ = fp.Close() + fp, err = os.Create(filepath.Join(dir, v1.SeriesFileName)) + require.NoError(t, err) + _ = fp.Close() + exists, _ := fetcher.isBlockDir(dir) + require.True(t, exists) + }) +} + func metasFromCache(data map[string][]byte) []Meta { metas := make([]Meta, 0, len(data)) - for k, v := range data { + for _, v := range data { meta := Meta{ - MetaRef: MetaRef{ - FilePath: k, - }, + MetaRef: MetaRef{}, } _ = json.Unmarshal(v, &meta) metas = append(metas, meta) @@ -170,7 +276,7 @@ func requireEqualMetas(t *testing.T, expected []Meta, actual []MetaRef) { func requireCachedMetas(t *testing.T, expected []Meta, actual map[string][]byte) { require.Equal(t, len(expected), len(actual)) for _, meta := range expected { - _, contains := actual[meta.MetaRef.FilePath] + _, contains := actual[meta.String()] require.True(t, contains) } } diff --git a/pkg/storage/stores/shipper/bloomshipper/resolver.go b/pkg/storage/stores/shipper/bloomshipper/resolver.go index 6278af9c6d04d..40a59cee42dbc 100644 --- a/pkg/storage/stores/shipper/bloomshipper/resolver.go +++ b/pkg/storage/stores/shipper/bloomshipper/resolver.go @@ -4,6 +4,10 @@ import ( "fmt" "path" "path/filepath" + "strconv" + "strings" + + v1 "github.com/grafana/loki/pkg/storage/bloom/v1" ) const ( @@ -20,6 +24,7 @@ const ( // the locations of data across schema boundaries (for instance to upgrade|improve). type KeyResolver interface { Meta(MetaRef) Location + ParseMetaKey(Location) (MetaRef, error) Block(BlockRef) Location } @@ -35,6 +40,36 @@ func (defaultKeyResolver) Meta(ref MetaRef) Location { } } +func (defaultKeyResolver) ParseMetaKey(loc Location) (MetaRef, error) { + dir, fn := path.Split(loc.Addr()) + fnParts := strings.Split(fn, "-") + if len(fnParts) != 3 { + return MetaRef{}, fmt.Errorf("failed to split filename parts of meta key %s : len must be 3, but was %d", loc, len(fnParts)) + } + bounds, err := v1.ParseBoundsFromParts(fnParts[0], fnParts[1]) + if err != nil { + return MetaRef{}, fmt.Errorf("failed to parse bounds of meta key %s : %w", loc, err) + } + checksum, err := strconv.ParseUint(fnParts[2], 16, 64) + if err != nil { + return MetaRef{}, fmt.Errorf("failed to parse checksum of meta key %s : %w", loc, err) + } + + dirParts := strings.Split(path.Clean(dir), "/") + if len(dirParts) < 4 { + return MetaRef{}, fmt.Errorf("directory parts count must be 4 or greater, but was %d : [%s]", len(dirParts), loc) + } + + return MetaRef{ + Ref: Ref{ + TenantID: dirParts[len(dirParts)-2], + TableName: dirParts[len(dirParts)-3], + Bounds: bounds, + Checksum: uint32(checksum), + }, + }, nil +} + func (defaultKeyResolver) Block(ref BlockRef) Location { return simpleLocation{ BloomPrefix, diff --git a/pkg/storage/stores/shipper/bloomshipper/shipper.go b/pkg/storage/stores/shipper/bloomshipper/shipper.go index 9ccffcd643024..0e0604962ed02 100644 --- a/pkg/storage/stores/shipper/bloomshipper/shipper.go +++ b/pkg/storage/stores/shipper/bloomshipper/shipper.go @@ -28,10 +28,9 @@ type Interface interface { } type Shipper struct { - store Store - config config.Config - logger log.Logger - blockDownloader *blockDownloader + store Store + config config.Config + logger log.Logger } type Limits interface { @@ -44,17 +43,12 @@ type StoreAndClient interface { Client } -func NewShipper(client StoreAndClient, config config.Config, limits Limits, logger log.Logger, reg prometheus.Registerer) (*Shipper, error) { +func NewShipper(client StoreAndClient, config config.Config, _ Limits, logger log.Logger, _ prometheus.Registerer) (*Shipper, error) { logger = log.With(logger, "component", "bloom-shipper") - downloader, err := newBlockDownloader(config, client, limits, logger, reg) - if err != nil { - return nil, fmt.Errorf("error creating block downloader: %w", err) - } return &Shipper{ - store: client, - config: config, - logger: logger, - blockDownloader: downloader, + store: client, + config: config, + logger: logger, }, nil } @@ -70,51 +64,31 @@ func (s *Shipper) GetBlockRefs(ctx context.Context, tenantID string, interval In return blockRefs, nil } -func (s *Shipper) Fetch(ctx context.Context, tenantID string, blocks []BlockRef, callback ForEachBlockCallback) error { - cancelContext, cancelFunc := context.WithCancel(ctx) - defer cancelFunc() - blocksChannel, errorsChannel := s.blockDownloader.downloadBlocks(cancelContext, tenantID, blocks) - - // track how many blocks are still remaning to be downloaded - remaining := len(blocks) - - for { - select { - case <-ctx.Done(): - return fmt.Errorf("failed to fetch blocks: %w", ctx.Err()) - case result, sentBeforeClosed := <-blocksChannel: - if !sentBeforeClosed { - return nil - } - err := runCallback(callback, result) - if err != nil { - return err - } - remaining-- - if remaining == 0 { - return nil - } - case err := <-errorsChannel: - return fmt.Errorf("error downloading blocks : %w", err) +func (s *Shipper) Fetch(ctx context.Context, _ string, blocks []BlockRef, callback ForEachBlockCallback) error { + blockDirs, err := s.store.FetchBlocks(ctx, blocks) + if err != nil { + return err + } + + for _, dir := range blockDirs { + err := runCallback(callback, dir.BlockQuerier(), dir.BlockRef.Bounds) + if err != nil { + return err } } + return nil } -func runCallback(callback ForEachBlockCallback, block blockWithQuerier) error { - defer func(b blockWithQuerier) { +func runCallback(callback ForEachBlockCallback, bq *ClosableBlockQuerier, bounds v1.FingerprintBounds) error { + defer func(b *ClosableBlockQuerier) { _ = b.Close() - }(block) + }(bq) - err := callback(block.closableBlockQuerier.BlockQuerier, block.Bounds) - if err != nil { - return fmt.Errorf("error running callback function for block %s err: %w", block.BlockRef, err) - } - return nil + return callback(bq.BlockQuerier, bounds) } func (s *Shipper) Stop() { s.store.Stop() - s.blockDownloader.stop() } // getFirstLast returns the first and last item of a fingerprint slice diff --git a/pkg/storage/stores/shipper/bloomshipper/store.go b/pkg/storage/stores/shipper/bloomshipper/store.go index 8bcb65d304ca7..4de9b453eca29 100644 --- a/pkg/storage/stores/shipper/bloomshipper/store.go +++ b/pkg/storage/stores/shipper/bloomshipper/store.go @@ -3,7 +3,6 @@ package bloomshipper import ( "context" "fmt" - "io" "path/filepath" "sort" @@ -21,10 +20,16 @@ import ( type Store interface { ResolveMetas(ctx context.Context, params MetaSearchParams) ([][]MetaRef, []*Fetcher, error) FetchMetas(ctx context.Context, params MetaSearchParams) ([]Meta, error) + FetchBlocks(ctx context.Context, refs []BlockRef) ([]BlockDirectory, error) Fetcher(ts model.Time) *Fetcher Stop() } +type bloomStoreConfig struct { + workingDir string + numWorkers int +} + // Compiler check to ensure bloomStoreEntry implements the Client interface var _ Client = &bloomStoreEntry{} @@ -51,7 +56,7 @@ func (b *bloomStoreEntry) ResolveMetas(ctx context.Context, params MetaSearchPar return nil, nil, fmt.Errorf("error listing metas under prefix [%s]: %w", prefix, err) } for _, object := range list { - metaRef, err := createMetaRef(object.Key, params.TenantID, table) + metaRef, err := b.ParseMetaKey(key(object.Key)) if err != nil { return nil, nil, err @@ -65,8 +70,8 @@ func (b *bloomStoreEntry) ResolveMetas(ctx context.Context, params MetaSearchPar break } - if !params.Keyspace.Overlaps(metaRef.Bounds) || - metaRef.EndTimestamp.Before(params.Interval.Start) || metaRef.StartTimestamp.After(params.Interval.End) { + // Only check keyspace for now, because we don't have start/end timestamps in the refs + if !params.Keyspace.Overlaps(metaRef.Bounds) { continue } @@ -97,7 +102,12 @@ func (b *bloomStoreEntry) FetchMetas(ctx context.Context, params MetaSearchParam return metas, nil } -// SearchMetas implements store. +// FetchBlocks implements Store. +func (b *bloomStoreEntry) FetchBlocks(ctx context.Context, refs []BlockRef) ([]BlockDirectory, error) { + return b.fetcher.FetchBlocksWithQueue(ctx, refs) +} + +// Fetcher implements Store. func (b *bloomStoreEntry) Fetcher(_ model.Time) *Fetcher { return b.fetcher } @@ -108,23 +118,33 @@ func (b *bloomStoreEntry) DeleteBlocks(ctx context.Context, refs []BlockRef) err } // DeleteMeta implements Client. -func (b *bloomStoreEntry) DeleteMeta(ctx context.Context, meta Meta) error { - return b.bloomClient.DeleteMeta(ctx, meta) +func (b *bloomStoreEntry) DeleteMetas(ctx context.Context, refs []MetaRef) error { + return b.bloomClient.DeleteMetas(ctx, refs) } // GetBlock implements Client. -func (b *bloomStoreEntry) GetBlock(ctx context.Context, ref BlockRef) (LazyBlock, error) { +func (b *bloomStoreEntry) GetBlock(ctx context.Context, ref BlockRef) (BlockDirectory, error) { return b.bloomClient.GetBlock(ctx, ref) } +// GetBlocks implements Client. +func (b *bloomStoreEntry) GetBlocks(ctx context.Context, refs []BlockRef) ([]BlockDirectory, error) { + return b.fetcher.FetchBlocks(ctx, refs) +} + +// GetMeta implements Client. +func (b *bloomStoreEntry) GetMeta(ctx context.Context, ref MetaRef) (Meta, error) { + return b.bloomClient.GetMeta(ctx, ref) +} + // GetMetas implements Client. func (b *bloomStoreEntry) GetMetas(ctx context.Context, refs []MetaRef) ([]Meta, error) { - return b.bloomClient.GetMetas(ctx, refs) + return b.fetcher.FetchMetas(ctx, refs) } // PutBlocks implements Client. -func (b *bloomStoreEntry) PutBlocks(ctx context.Context, blocks []Block) ([]Block, error) { - return b.bloomClient.PutBlocks(ctx, blocks) +func (b *bloomStoreEntry) PutBlock(ctx context.Context, block Block) error { + return b.bloomClient.PutBlock(ctx, block) } // PutMeta implements Client. @@ -135,14 +155,19 @@ func (b *bloomStoreEntry) PutMeta(ctx context.Context, meta Meta) error { // Stop implements Client. func (b bloomStoreEntry) Stop() { b.bloomClient.Stop() + b.fetcher.Close() } +// Compiler check to ensure BloomStore implements the Client interface var _ Client = &BloomStore{} + +// Compiler check to ensure BloomStore implements the Store interface var _ Store = &BloomStore{} type BloomStore struct { - stores []*bloomStoreEntry - storageConfig storage.Config + stores []*bloomStoreEntry + storageConfig storage.Config + defaultKeyResolver // TODO(owen-d): impl schema aware resolvers } func NewBloomStore( @@ -150,7 +175,7 @@ func NewBloomStore( storageConfig storage.Config, clientMetrics storage.ClientMetrics, metasCache cache.Cache, - blocksCache *cache.EmbeddedCache[string, io.ReadCloser], + blocksCache *cache.EmbeddedCache[string, BlockDirectory], logger log.Logger, ) (*BloomStore, error) { store := &BloomStore{ @@ -166,16 +191,24 @@ func NewBloomStore( return periodicConfigs[i].From.Time.Before(periodicConfigs[i].From.Time) }) + // TODO(chaudum): Remove wrapper + cfg := bloomStoreConfig{ + workingDir: storageConfig.BloomShipperConfig.WorkingDirectory, + numWorkers: storageConfig.BloomShipperConfig.BlocksDownloadingQueue.WorkersCount, + } + for _, periodicConfig := range periodicConfigs { objectClient, err := storage.NewObjectClient(periodicConfig.ObjectType, storageConfig, clientMetrics) if err != nil { return nil, errors.Wrapf(err, "creating object client for period %s", periodicConfig.From) } - bloomClient, err := NewBloomClient(objectClient, logger) + + bloomClient, err := NewBloomClient(cfg, objectClient, logger) if err != nil { return nil, errors.Wrapf(err, "creating bloom client for period %s", periodicConfig.From) } - fetcher, err := NewFetcher(bloomClient, metasCache, blocksCache, logger) + + fetcher, err := NewFetcher(cfg, bloomClient, metasCache, blocksCache, logger) if err != nil { return nil, errors.Wrapf(err, "creating fetcher for period %s", periodicConfig.From) } @@ -271,9 +304,14 @@ func (b *BloomStore) FetchMetas(ctx context.Context, params MetaSearchParams) ([ return metas, nil } +// FetchBlocks implements Store. +func (b *BloomStore) FetchBlocks(ctx context.Context, refs []BlockRef) ([]BlockDirectory, error) { + return b.GetBlocks(ctx, refs) +} + // DeleteBlocks implements Client. -func (b *BloomStore) DeleteBlocks(ctx context.Context, blocks []BlockRef) error { - for _, ref := range blocks { +func (b *BloomStore) DeleteBlocks(ctx context.Context, refs []BlockRef) error { + for _, ref := range refs { err := b.storeDo( ref.StartTimestamp, func(s *bloomStoreEntry) error { @@ -287,22 +325,85 @@ func (b *BloomStore) DeleteBlocks(ctx context.Context, blocks []BlockRef) error return nil } -// DeleteMeta implements Client. -func (b *BloomStore) DeleteMeta(ctx context.Context, meta Meta) error { - return b.storeDo(meta.StartTimestamp, func(s *bloomStoreEntry) error { - return s.DeleteMeta(ctx, meta) - }) +// DeleteMetas implements Client. +func (b *BloomStore) DeleteMetas(ctx context.Context, refs []MetaRef) error { + for _, ref := range refs { + err := b.storeDo( + ref.StartTimestamp, + func(s *bloomStoreEntry) error { + return s.DeleteMetas(ctx, []MetaRef{ref}) + }, + ) + if err != nil { + return err + } + } + return nil } // GetBlock implements Client. -func (b *BloomStore) GetBlock(ctx context.Context, ref BlockRef) (LazyBlock, error) { - var block LazyBlock - var err error - err = b.storeDo(ref.StartTimestamp, func(s *bloomStoreEntry) error { - block, err = s.GetBlock(ctx, ref) - return err +func (b *BloomStore) GetBlock(ctx context.Context, ref BlockRef) (BlockDirectory, error) { + res := make([]BlockDirectory, 1) + err := b.storeDo(ref.StartTimestamp, func(s *bloomStoreEntry) error { + block, err := s.GetBlock(ctx, ref) + if err != nil { + return err + } + res[0] = block + return nil }) - return block, err + return res[0], err +} + +// GetBlocks implements Client. +func (b *BloomStore) GetBlocks(ctx context.Context, blocks []BlockRef) ([]BlockDirectory, error) { + var refs [][]BlockRef + var fetchers []*Fetcher + + for i := len(b.stores) - 1; i >= 0; i-- { + s := b.stores[i] + from, through := s.start, model.Latest + if i < len(b.stores)-1 { + through = b.stores[i+1].start + } + + var res []BlockRef + for _, meta := range blocks { + if meta.StartTimestamp >= from && meta.StartTimestamp < through { + res = append(res, meta) + } + } + + if len(res) > 0 { + refs = append(refs, res) + fetchers = append(fetchers, s.Fetcher(s.start)) + } + } + + results := make([]BlockDirectory, 0, len(blocks)) + for i := range fetchers { + res, err := fetchers[i].FetchBlocksWithQueue(ctx, refs[i]) + results = append(results, res...) + if err != nil { + return results, err + } + } + + return results, nil +} + +// GetMeta implements Client. +func (b *BloomStore) GetMeta(ctx context.Context, ref MetaRef) (Meta, error) { + res := make([]Meta, 1) + err := b.storeDo(ref.StartTimestamp, func(s *bloomStoreEntry) error { + meta, err := s.GetMeta(ctx, ref) + if err != nil { + return err + } + res[0] = meta + return nil + }) + return res[0], err } // GetMetas implements Client. @@ -342,23 +443,11 @@ func (b *BloomStore) GetMetas(ctx context.Context, metas []MetaRef) ([]Meta, err return results, nil } -// PutBlocks implements Client. -func (b *BloomStore) PutBlocks(ctx context.Context, blocks []Block) ([]Block, error) { - results := make([]Block, 0, len(blocks)) - for _, ref := range blocks { - err := b.storeDo( - ref.StartTimestamp, - func(s *bloomStoreEntry) error { - res, err := s.PutBlocks(ctx, []Block{ref}) - results = append(results, res...) - return err - }, - ) - if err != nil { - return nil, err - } - } - return results, nil +// PutBlock implements Client. +func (b *BloomStore) PutBlock(ctx context.Context, block Block) error { + return b.storeDo(block.StartTimestamp, func(s *bloomStoreEntry) error { + return s.PutBlock(ctx, block) + }) } // PutMeta implements Client. @@ -395,7 +484,7 @@ func (b *BloomStore) storeDo(ts model.Time, f func(s *bloomStoreEntry) error) er if store := b.getStore(ts); store != nil { return f(store) } - return nil + return fmt.Errorf("no store found for timestamp %s", ts.Time()) } func (b *BloomStore) forStores(ctx context.Context, interval Interval, f func(innerCtx context.Context, interval Interval, store Store) error) error { @@ -446,3 +535,14 @@ func (b *BloomStore) forStores(ctx context.Context, interval Interval, f func(in } return nil } + +func tablesForRange(periodConfig config.PeriodConfig, interval Interval) []string { + step := int64(periodConfig.IndexTables.Period.Seconds()) + lower := interval.Start.Unix() / step + upper := interval.End.Unix() / step + tables := make([]string, 0, 1+upper-lower) + for i := lower; i <= upper; i++ { + tables = append(tables, fmt.Sprintf("%s%d", periodConfig.IndexTables.Prefix, i)) + } + return tables +} From 2e3fa3b861889c754ebf879c9b251147269d9d20 Mon Sep 17 00:00:00 2001 From: Callum Styan Date: Mon, 5 Feb 2024 13:16:44 -0800 Subject: [PATCH 34/45] quantile sharding: switch to ddsketch sparse constructor and remove quantile vector pooling in Join function (#11844) This should reduce our memory usage for our quantile sketches by about half, at least in our current situation we'll OOM after ~160-170 minutes instead of ~90 :cry: --------- Signed-off-by: Callum Styan Co-authored-by: Karsten Jeschkies --- pkg/logql/quantile_over_time_sketch.go | 20 ++---- pkg/logql/quantile_over_time_sketch_test.go | 78 ++++----------------- pkg/logql/sketch/quantile.go | 2 +- 3 files changed, 21 insertions(+), 79 deletions(-) diff --git a/pkg/logql/quantile_over_time_sketch.go b/pkg/logql/quantile_over_time_sketch.go index f9f05f99c9979..507c72b208ab8 100644 --- a/pkg/logql/quantile_over_time_sketch.go +++ b/pkg/logql/quantile_over_time_sketch.go @@ -14,7 +14,6 @@ import ( "github.com/grafana/loki/pkg/logproto" "github.com/grafana/loki/pkg/logql/sketch" "github.com/grafana/loki/pkg/logqlmodel" - "github.com/grafana/loki/pkg/queue" ) const ( @@ -116,7 +115,6 @@ func (m ProbabilisticQuantileMatrix) Release() { for _, vec := range m { vec.Release() } - quantileVectorPool.Put(m) } func (m ProbabilisticQuantileMatrix) ToProto() *logproto.QuantileSketchMatrix { @@ -238,24 +236,20 @@ func probabilisticQuantileSampleFromProto(proto *logproto.QuantileSketchSample) type quantileSketchBatchRangeVectorIterator struct { *batchRangeVectorIterator - at []ProbabilisticQuantileSample } func (r *quantileSketchBatchRangeVectorIterator) At() (int64, StepResult) { - if r.at == nil { - r.at = make([]ProbabilisticQuantileSample, 0, len(r.window)) - } - r.at = r.at[:0] + at := make([]ProbabilisticQuantileSample, 0, len(r.window)) // convert ts from nano to milli seconds as the iterator work with nanoseconds ts := r.current/1e+6 + r.offset/1e+6 for _, series := range r.window { - r.at = append(r.at, ProbabilisticQuantileSample{ + at = append(at, ProbabilisticQuantileSample{ F: r.agg(series.Floats), T: ts, Metric: series.Metric, }) } - return ts, ProbabilisticQuantileVector(r.at) + return ts, ProbabilisticQuantileVector(at) } func (r *quantileSketchBatchRangeVectorIterator) agg(samples []promql.FPoint) sketch.QuantileSketch { @@ -268,9 +262,6 @@ func (r *quantileSketchBatchRangeVectorIterator) agg(samples []promql.FPoint) sk return s } -// quantileVectorPool slice of ProbabilisticQuantileVector [64, 128, 256, ..., 65536] -var quantileVectorPool = queue.NewSlicePool[ProbabilisticQuantileVector](1<<6, 1<<16, 2) - // JoinQuantileSketchVector joins the results from stepEvaluator into a ProbabilisticQuantileMatrix. func JoinQuantileSketchVector(next bool, r StepResult, stepEvaluator StepEvaluator, params Params) (promql_parser.Value, error) { vec := r.QuantileSketchVec() @@ -283,8 +274,7 @@ func JoinQuantileSketchVector(next bool, r StepResult, stepEvaluator StepEvaluat stepCount = 1 } - // The result is released to the pool when the matrix is serialized. - result := quantileVectorPool.Get(stepCount) + result := make(ProbabilisticQuantileMatrix, 0, stepCount) for next { result = append(result, vec) @@ -295,7 +285,7 @@ func JoinQuantileSketchVector(next bool, r StepResult, stepEvaluator StepEvaluat } } - return ProbabilisticQuantileMatrix(result), stepEvaluator.Error() + return result, stepEvaluator.Error() } // QuantileSketchMatrixStepEvaluator steps through a matrix of quantile sketch diff --git a/pkg/logql/quantile_over_time_sketch_test.go b/pkg/logql/quantile_over_time_sketch_test.go index 4dcd079eeacc4..dc1ff31f509a4 100644 --- a/pkg/logql/quantile_over_time_sketch_test.go +++ b/pkg/logql/quantile_over_time_sketch_test.go @@ -113,81 +113,33 @@ func (e errorStepEvaluator) Error() error { func (e errorStepEvaluator) Explain(Node) {} func BenchmarkJoinQuantileSketchVector(b *testing.B) { - results := make([]ProbabilisticQuantileVector, 100) - for i := range results { - results[i] = make(ProbabilisticQuantileVector, 10) - for j := range results[i] { - results[i][j] = ProbabilisticQuantileSample{ - T: int64(i), - F: newRandomSketch(), - Metric: []labels.Label{{Name: "foo", Value: fmt.Sprintf("bar-%d", j)}}, - } - } - } - ev := &sliceStepEvaluator{ - slice: results, - cur: 1, - } + selRange := (5 * time.Second).Nanoseconds() + step := (30 * time.Second) + offset := int64(0) + start := time.Unix(10, 0) + end := time.Unix(100, 0) // (end - start) / step == len(results) params := LiteralParams{ - start: time.Unix(0, 0), - end: time.Unix(int64(len(results)), 0), - step: time.Second, + start: start, + end: end, + step: step, } b.ReportAllocs() b.ResetTimer() for n := 0; n < b.N; n++ { - // Reset step evaluator - ev.cur = 1 - r, err := JoinQuantileSketchVector(true, results[0], ev, params) + iter := newQuantileSketchIterator(newfakePeekingSampleIterator(samples), selRange, step.Nanoseconds(), start.UnixNano(), end.UnixNano(), offset) + ev := &QuantileSketchStepEvaluator{ + iter: iter, + } + _, _, r := ev.Next() + m, err := JoinQuantileSketchVector(true, r.QuantileSketchVec(), ev, params) require.NoError(b, err) - r.(ProbabilisticQuantileMatrix).Release() - } -} - -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 -} - -type sliceStepEvaluator struct { - err error - slice []ProbabilisticQuantileVector - cur int -} - -// Close implements StepEvaluator. -func (*sliceStepEvaluator) Close() error { - return nil -} - -// Error implements StepEvaluator. -func (ev *sliceStepEvaluator) Error() error { - return ev.err -} - -// Explain implements StepEvaluator. -func (*sliceStepEvaluator) Explain(Node) {} - -// Next implements StepEvaluator. -func (ev *sliceStepEvaluator) Next() (ok bool, ts int64, r StepResult) { - if ev.cur >= len(ev.slice) { - return false, 0, nil + m.(ProbabilisticQuantileMatrix).Release() } - - r = ev.slice[ev.cur] - ts = ev.slice[ev.cur][0].T - ev.cur++ - ok = ev.cur < len(ev.slice) - return } func BenchmarkQuantileBatchRangeVectorIteratorAt(b *testing.B) { diff --git a/pkg/logql/sketch/quantile.go b/pkg/logql/sketch/quantile.go index 8042ea53741e3..1fa20c38e5bcc 100644 --- a/pkg/logql/sketch/quantile.go +++ b/pkg/logql/sketch/quantile.go @@ -47,7 +47,7 @@ const relativeAccuracy = 0.01 var ddsketchPool = sync.Pool{ New: func() any { m, _ := mapping.NewCubicallyInterpolatedMapping(relativeAccuracy) - return ddsketch.NewDDSketchFromStoreProvider(m, store.DefaultProvider) + return ddsketch.NewDDSketchFromStoreProvider(m, store.SparseStoreConstructor) }, } From 73edf7a9430aa3a8dc91370fc34de9be92bbe88a Mon Sep 17 00:00:00 2001 From: Christian Haudum Date: Mon, 5 Feb 2024 22:31:51 +0100 Subject: [PATCH 35/45] (chore) Bloomshipper: Separate store and client (#11865) **What this PR does / why we need it**: This PR removes the `StoreAndClient` interface that was accepted by the `BloomShipper`. Since the `BloomStore` had to not only implement the `Store` interface, but also the `Client` interface, it caused re-implementation of the same methods in different ways. Now the shipper solely relies on the `Store` interface. See individual commit messages for more context. Tests have been rewritten from scratch and placed in their own respective test files for store and client. --------- Signed-off-by: Christian Haudum --- go.mod | 2 +- pkg/bloomcompactor/bloomcompactor.go | 4 +- .../testutils/inmemory_storage_client.go | 51 +- .../shipper/bloomshipper/client_test.go | 662 +++++++----------- .../stores/shipper/bloomshipper/fetcher.go | 4 +- .../stores/shipper/bloomshipper/shipper.go | 8 +- .../stores/shipper/bloomshipper/store.go | 192 +---- .../stores/shipper/bloomshipper/store_test.go | 268 +++++++ 8 files changed, 586 insertions(+), 605 deletions(-) create mode 100644 pkg/storage/stores/shipper/bloomshipper/store_test.go diff --git a/go.mod b/go.mod index f8db7e46affae..87ea0fd075852 100644 --- a/go.mod +++ b/go.mod @@ -119,7 +119,6 @@ require ( github.com/DmitriyVTitov/size v1.5.0 github.com/IBM/go-sdk-core/v5 v5.13.1 github.com/IBM/ibm-cos-sdk-go v1.10.0 - github.com/aws/smithy-go v1.11.1 github.com/axiomhq/hyperloglog v0.0.0-20230201085229-3ddf4bad03dc github.com/d4l3k/messagediff v1.2.1 github.com/efficientgo/core v1.0.0-rc.2 @@ -183,6 +182,7 @@ require ( github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.1 // indirect github.com/aws/aws-sdk-go-v2/service/sso v1.11.1 // indirect github.com/aws/aws-sdk-go-v2/service/sts v1.16.1 // indirect + github.com/aws/smithy-go v1.11.1 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/census-instrumentation/opencensus-proto v0.4.1 // indirect github.com/cncf/udpa/go v0.0.0-20220112060539-c52dc94e7fbe // indirect diff --git a/pkg/bloomcompactor/bloomcompactor.go b/pkg/bloomcompactor/bloomcompactor.go index a3862408593db..cf3b3fafcb6d1 100644 --- a/pkg/bloomcompactor/bloomcompactor.go +++ b/pkg/bloomcompactor/bloomcompactor.go @@ -38,7 +38,7 @@ type Compactor struct { limits Limits // temporary workaround until store has implemented read/write shipper interface - store bloomshipper.StoreAndClient + store bloomshipper.Store sharding ShardingStrategy @@ -48,7 +48,7 @@ type Compactor struct { func New( cfg Config, - store bloomshipper.StoreAndClient, + store bloomshipper.Store, sharding ShardingStrategy, limits Limits, logger log.Logger, diff --git a/pkg/storage/chunk/client/testutils/inmemory_storage_client.go b/pkg/storage/chunk/client/testutils/inmemory_storage_client.go index 48a7a6c650ee5..15e2ddb2564bd 100644 --- a/pkg/storage/chunk/client/testutils/inmemory_storage_client.go +++ b/pkg/storage/chunk/client/testutils/inmemory_storage_client.go @@ -33,9 +33,10 @@ const ( // MockStorage is a fake in-memory StorageClient. type MockStorage struct { + *InMemoryObjectClient + mtx sync.RWMutex tables map[string]*mockTable - objects map[string][]byte schemaCfg config.SchemaConfig numIndexWrites int @@ -43,6 +44,25 @@ type MockStorage struct { mode MockStorageMode } +// compiler check +var _ client.ObjectClient = &InMemoryObjectClient{} + +type InMemoryObjectClient struct { + objects map[string][]byte + mtx sync.RWMutex + mode MockStorageMode +} + +func NewInMemoryObjectClient() *InMemoryObjectClient { + return &InMemoryObjectClient{ + objects: make(map[string][]byte), + } +} + +func (m *InMemoryObjectClient) Internals() map[string][]byte { + return m.objects +} + type mockTable struct { items map[string][]mockItem write, read int64 @@ -64,6 +84,7 @@ func ResetMockStorage() { func NewMockStorage() *MockStorage { if singleton == nil { singleton = &MockStorage{ + InMemoryObjectClient: NewInMemoryObjectClient(), schemaCfg: config.SchemaConfig{ Configs: []config.PeriodConfig{ { @@ -73,8 +94,7 @@ func NewMockStorage() *MockStorage { }, }, }, - tables: map[string]*mockTable{}, - objects: map[string][]byte{}, + tables: map[string]*mockTable{}, } } return singleton @@ -109,6 +129,7 @@ func (*MockStorage) Stop() { func (m *MockStorage) SetMode(mode MockStorageMode) { m.mode = mode + m.InMemoryObjectClient.mode = mode } // ListTables implements StorageClient. @@ -370,7 +391,8 @@ func (m *MockStorage) query(ctx context.Context, query index.Query, callback fun return nil } -func (m *MockStorage) ObjectExists(_ context.Context, objectKey string) (bool, error) { +// ObjectExists implments client.ObjectClient +func (m *InMemoryObjectClient) ObjectExists(_ context.Context, objectKey string) (bool, error) { m.mtx.RLock() defer m.mtx.RUnlock() @@ -386,7 +408,8 @@ func (m *MockStorage) ObjectExists(_ context.Context, objectKey string) (bool, e return true, nil } -func (m *MockStorage) GetObject(_ context.Context, objectKey string) (io.ReadCloser, int64, error) { +// GetObject implements client.ObjectClient. +func (m *InMemoryObjectClient) GetObject(_ context.Context, objectKey string) (io.ReadCloser, int64, error) { m.mtx.RLock() defer m.mtx.RUnlock() @@ -402,7 +425,8 @@ func (m *MockStorage) GetObject(_ context.Context, objectKey string) (io.ReadClo return io.NopCloser(bytes.NewReader(buf)), int64(len(buf)), nil } -func (m *MockStorage) PutObject(_ context.Context, objectKey string, object io.ReadSeeker) error { +// PutObject implements client.ObjectClient. +func (m *InMemoryObjectClient) PutObject(_ context.Context, objectKey string, object io.ReadSeeker) error { buf, err := io.ReadAll(object) if err != nil { return err @@ -419,7 +443,8 @@ func (m *MockStorage) PutObject(_ context.Context, objectKey string, object io.R return nil } -func (m *MockStorage) IsObjectNotFoundErr(err error) bool { +// IsObjectNotFoundErr implements client.ObjectClient. +func (m *InMemoryObjectClient) IsObjectNotFoundErr(err error) bool { return errors.Is(err, errStorageObjectNotFound) } @@ -427,9 +452,11 @@ func (m *MockStorage) IsChunkNotFoundErr(err error) bool { return m.IsObjectNotFoundErr(err) } -func (m *MockStorage) IsRetryableErr(error) bool { return false } +// IsRetryableErr implements client.ObjectClient. +func (m *InMemoryObjectClient) IsRetryableErr(error) bool { return false } -func (m *MockStorage) DeleteObject(_ context.Context, objectKey string) error { +// DeleteObject implements client.ObjectClient. +func (m *InMemoryObjectClient) DeleteObject(_ context.Context, objectKey string) error { m.mtx.Lock() defer m.mtx.Unlock() @@ -446,7 +473,7 @@ func (m *MockStorage) DeleteObject(_ context.Context, objectKey string) error { } // List implements chunk.ObjectClient. -func (m *MockStorage) List(_ context.Context, prefix, delimiter string) ([]client.StorageObject, []client.StorageCommonPrefix, error) { +func (m *InMemoryObjectClient) List(_ context.Context, prefix, delimiter string) ([]client.StorageObject, []client.StorageCommonPrefix, error) { m.mtx.RLock() defer m.mtx.RUnlock() @@ -494,6 +521,10 @@ func (m *MockStorage) List(_ context.Context, prefix, delimiter string) ([]clien return storageObjects, commonPrefixes, nil } +// Stop implements client.ObjectClient +func (*InMemoryObjectClient) Stop() { +} + type mockWriteBatch struct { inserts []struct { tableName, hashValue string diff --git a/pkg/storage/stores/shipper/bloomshipper/client_test.go b/pkg/storage/stores/shipper/bloomshipper/client_test.go index 03a15f11bd3d3..02d80429d7f11 100644 --- a/pkg/storage/stores/shipper/bloomshipper/client_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/client_test.go @@ -1,43 +1,29 @@ package bloomshipper import ( - "archive/tar" "bytes" "context" "encoding/json" - "io" + "fmt" "os" - "path/filepath" - "strings" "testing" "time" - awsio "github.com/aws/smithy-go/io" "github.com/go-kit/log" - "github.com/google/uuid" "github.com/prometheus/common/model" "github.com/stretchr/testify/require" - "github.com/grafana/loki/pkg/chunkenc" - "github.com/grafana/loki/pkg/storage" v1 "github.com/grafana/loki/pkg/storage/bloom/v1" - "github.com/grafana/loki/pkg/storage/chunk/cache" + "github.com/grafana/loki/pkg/storage/chunk/client/testutils" "github.com/grafana/loki/pkg/storage/config" - bloomshipperconfig "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper/config" ) -const ( - day = 24 * time.Hour -) - -var ( - // table 19627 - fixedDay = Date(2023, time.September, 27, 0, 0, 0) -) - -func Date(year int, month time.Month, day, hour, min, sec int) model.Time { - date := time.Date(year, month, day, hour, min, sec, 0, time.UTC) - return model.TimeFromUnixNano(date.UnixNano()) +func parseTime(s string) model.Time { + t, err := time.Parse("2006-01-02 15:04", s) + if err != nil { + panic(err) + } + return model.TimeFromUnix(t.Unix()) } func parseDayTime(s string) config.DayTime { @@ -50,462 +36,310 @@ func parseDayTime(s string) config.DayTime { } } -func Test_BloomClient_FetchMetas(t *testing.T) { - store := createStore(t) - - var expected []Meta - // metas that belong to 1st schema stored in folder-1 - // must not be present in results because it is outside of time range - createMetaInStorage(t, store, "19621", "tenantA", 0, 100, fixedDay.Add(-7*day)) - // must be present in the results - expected = append(expected, createMetaInStorage(t, store, "19621", "tenantA", 0, 100, fixedDay.Add(-6*day))) - // must not be present in results because it belongs to another tenant - createMetaInStorage(t, store, "19621", "tenantB", 0, 100, fixedDay.Add(-6*day)) - // must be present in the results - expected = append(expected, createMetaInStorage(t, store, "19621", "tenantA", 101, 200, fixedDay.Add(-6*day))) - - // metas that belong to 2nd schema stored in folder-2 - // must not be present in results because it's out of the time range - createMetaInStorage(t, store, "19626", "tenantA", 0, 100, fixedDay.Add(-1*day)) - // must be present in the results - expected = append(expected, createMetaInStorage(t, store, "19625", "tenantA", 0, 100, fixedDay.Add(-2*day))) - // must not be present in results because it belongs to another tenant - createMetaInStorage(t, store, "19624", "tenantB", 0, 100, fixedDay.Add(-3*day)) - - searchParams := MetaSearchParams{ - TenantID: "tenantA", - Keyspace: v1.NewBounds(50, 150), - Interval: NewInterval(fixedDay.Add(-6*day), fixedDay.Add(-1*day-1*time.Hour)), +func newMockBloomClient(t *testing.T) (*BloomClient, string) { + oc := testutils.NewInMemoryObjectClient() + dir := t.TempDir() + logger := log.NewLogfmtLogger(os.Stderr) + cfg := bloomStoreConfig{ + workingDir: dir, + numWorkers: 3, } - - fetched, err := store.FetchMetas(context.Background(), searchParams) + client, err := NewBloomClient(cfg, oc, logger) require.NoError(t, err) + return client, dir +} - require.Equal(t, len(expected), len(fetched)) - for i := range expected { - require.Equal(t, expected[i].String(), fetched[i].String()) - require.ElementsMatch(t, expected[i].Blocks, fetched[i].Blocks) - require.ElementsMatch(t, expected[i].Tombstones, fetched[i].Tombstones) +func putMeta(c *BloomClient, tenant string, start model.Time, minFp, maxFp model.Fingerprint) (Meta, error) { + step := int64((24 * time.Hour).Seconds()) + day := start.Unix() / step + meta := Meta{ + MetaRef: MetaRef{ + Ref: Ref{ + TenantID: tenant, + Bounds: v1.NewBounds(minFp, maxFp), + TableName: fmt.Sprintf("table_%d", day), + // Unused + // StartTimestamp: start, + // EndTimestamp: start.Add(12 * time.Hour), + }, + }, + Blocks: []BlockRef{}, + Tombstones: []BlockRef{}, } + raw, _ := json.Marshal(meta) + return meta, c.client.PutObject(context.Background(), c.Meta(meta.MetaRef).Addr(), bytes.NewReader(raw)) +} - resolved, _, err := store.ResolveMetas(context.Background(), searchParams) +func TestBloomClient_GetMeta(t *testing.T) { + c, _ := newMockBloomClient(t) + ctx := context.Background() + + m, err := putMeta(c, "tenant", parseTime("2024-02-05 00:00"), 0x0000, 0xffff) require.NoError(t, err) - var resolvedRefs []MetaRef - for _, refs := range resolved { - resolvedRefs = append(resolvedRefs, refs...) - } - for i := range resolvedRefs { - require.Equal(t, fetched[i].MetaRef, resolvedRefs[i]) - } + t.Run("exists", func(t *testing.T) { + meta, err := c.GetMeta(ctx, m.MetaRef) + require.NoError(t, err) + require.Equal(t, meta, m) + }) + + t.Run("does not exist", func(t *testing.T) { + meta, err := c.GetMeta(ctx, MetaRef{}) + require.Error(t, err) + require.True(t, c.client.IsObjectNotFoundErr(err)) + require.Equal(t, meta, Meta{}) + }) } -func Test_BloomClient_PutMeta(t *testing.T) { - tests := map[string]struct { - source Meta - expectedFilePath string - expectedStorage string - }{ - "expected meta to be uploaded to the first folder": { - source: createMetaEntity("tenantA", - "table_19621", - 0xff, - 0xfff, - Date(2023, time.September, 21, 5, 0, 0), - Date(2023, time.September, 21, 6, 0, 0), - ), - expectedStorage: "folder-1", - expectedFilePath: "bloom/table_19621/tenantA/metas/00000000000000ff-0000000000000fff-0", - }, - "expected meta to be uploaded to the second folder": { - source: createMetaEntity("tenantA", - "table_19625", - 200, - 300, - Date(2023, time.September, 25, 0, 0, 0), - Date(2023, time.September, 25, 1, 0, 0), - ), - expectedStorage: "folder-2", - expectedFilePath: "bloom/table_19625/tenantA/metas/00000000000000c8-000000000000012c-0", - }, - } - for name, data := range tests { - t.Run(name, func(t *testing.T) { - bloomClient := createStore(t) - - err := bloomClient.PutMeta(context.Background(), data.source) - require.NoError(t, err) - - directory := bloomClient.storageConfig.NamedStores.Filesystem[data.expectedStorage].Directory - filePath := filepath.Join(directory, data.expectedFilePath) - require.FileExistsf(t, filePath, data.source.String()) - content, err := os.ReadFile(filePath) - require.NoError(t, err) - result := Meta{} - err = json.Unmarshal(content, &result) - require.NoError(t, err) - - require.Equal(t, data.source.Blocks, result.Blocks) - require.Equal(t, data.source.Tombstones, result.Tombstones) - }) - } +func TestBloomClient_GetMetas(t *testing.T) { + c, _ := newMockBloomClient(t) + ctx := context.Background() + + m1, err := putMeta(c, "tenant", parseTime("2024-02-05 00:00"), 0x0000, 0x0fff) + require.NoError(t, err) + m2, err := putMeta(c, "tenant", parseTime("2024-02-05 00:00"), 0x1000, 0xffff) + require.NoError(t, err) + + t.Run("exists", func(t *testing.T) { + metas, err := c.GetMetas(ctx, []MetaRef{m1.MetaRef, m2.MetaRef}) + require.NoError(t, err) + require.Equal(t, metas, []Meta{m1, m2}) + }) + t.Run("does not exist", func(t *testing.T) { + metas, err := c.GetMetas(ctx, []MetaRef{{}}) + require.Error(t, err) + require.True(t, c.client.IsObjectNotFoundErr(err)) + require.Equal(t, metas, []Meta{{}}) + }) } -func Test_BloomClient_DeleteMeta(t *testing.T) { - tests := map[string]struct { - source Meta - expectedFilePath string - expectedStorage string - }{ - "expected meta to be deleted from the first folder": { - source: createMetaEntity("tenantA", - "table_19621", - 0xff, - 0xfff, - Date(2023, time.September, 21, 5, 0, 0), - Date(2023, time.September, 21, 6, 0, 0), - ), - expectedStorage: "folder-1", - expectedFilePath: "bloom/table_19621/tenantA/metas/00000000000000ff-0000000000000fff-0", - }, - "expected meta to be delete from the second folder": { - source: createMetaEntity("tenantA", - "table_19625", - 200, - 300, - Date(2023, time.September, 25, 0, 0, 0), - Date(2023, time.September, 25, 1, 0, 0), - ), - expectedStorage: "folder-2", - expectedFilePath: "bloom/table_19625/tenantA/metas/00000000000000c8-000000000000012c-0", +func TestBloomClient_PutMeta(t *testing.T) { + c, _ := newMockBloomClient(t) + ctx := context.Background() + + meta := Meta{ + MetaRef: MetaRef{ + Ref: Ref{ + TenantID: "tenant", + Bounds: v1.NewBounds(0x0000, 0xffff), + TableName: "table_1234", + // Unused + // StartTimestamp: start, + // EndTimestamp: start.Add(12 * time.Hour), + }, }, + Blocks: []BlockRef{}, + Tombstones: []BlockRef{}, } - for name, data := range tests { - t.Run(name, func(t *testing.T) { - bloomClient := createStore(t) - directory := bloomClient.storageConfig.NamedStores.Filesystem[data.expectedStorage].Directory - file := filepath.Join(directory, data.expectedFilePath) - - // requires that Test_BloomClient_PutMeta does not fail - err := bloomClient.PutMeta(context.Background(), data.source) - require.NoError(t, err) - require.FileExists(t, file, data.source.String()) + err := c.PutMeta(ctx, meta) + require.NoError(t, err) - err = bloomClient.DeleteMetas(context.Background(), []MetaRef{data.source.MetaRef}) - require.NoError(t, err) + oc := c.client.(*testutils.InMemoryObjectClient) + stored := oc.Internals() + _, found := stored[c.Meta(meta.MetaRef).Addr()] + require.True(t, found) - require.NoFileExists(t, file, data.source.String()) - }) - } + fromStorage, err := c.GetMeta(ctx, meta.MetaRef) + require.NoError(t, err) + require.Equal(t, meta, fromStorage) } -func Test_BloomClient_GetBlocks(t *testing.T) { - firstBlockRef := BlockRef{ - Ref: Ref{ - TenantID: "tenantA", - TableName: "schema_a_table_19621", - Bounds: v1.NewBounds(0xeeee, 0xffff), - StartTimestamp: Date(2023, time.September, 21, 5, 0, 0), - EndTimestamp: Date(2023, time.September, 21, 6, 0, 0), - Checksum: 1, - }, - } - secondBlockRef := BlockRef{ - Ref: Ref{ - TenantID: "tenantA", - TableName: "schema_b_table_19624", - Bounds: v1.NewBounds(0xaaaa, 0xbbbb), - StartTimestamp: Date(2023, time.September, 24, 5, 0, 0), - EndTimestamp: Date(2023, time.September, 24, 6, 0, 0), - Checksum: 2, - }, - } +func TestBloomClient_DeleteMetas(t *testing.T) { + c, _ := newMockBloomClient(t) + ctx := context.Background() - bloomClient := createStore(t) + m1, err := putMeta(c, "tenant", parseTime("2024-02-05 00:00"), 0x0000, 0xffff) + require.NoError(t, err) + m2, err := putMeta(c, "tenant", parseTime("2024-02-06 00:00"), 0x0000, 0xffff) + require.NoError(t, err) + m3, err := putMeta(c, "tenant", parseTime("2024-02-07 00:00"), 0x0000, 0xffff) + require.NoError(t, err) + + oc := c.client.(*testutils.InMemoryObjectClient) + stored := oc.Internals() + _, found := stored[c.Meta(m1.MetaRef).Addr()] + require.True(t, found) + _, found = stored[c.Meta(m2.MetaRef).Addr()] + require.True(t, found) + _, found = stored[c.Meta(m3.MetaRef).Addr()] + require.True(t, found) + + t.Run("all deleted", func(t *testing.T) { + err = c.DeleteMetas(ctx, []MetaRef{m1.MetaRef, m2.MetaRef}) + require.NoError(t, err) + + _, found = stored[c.Meta(m1.MetaRef).Addr()] + require.False(t, found) + _, found = stored[c.Meta(m2.MetaRef).Addr()] + require.False(t, found) + }) + + t.Run("some not found", func(t *testing.T) { + err = c.DeleteMetas(ctx, []MetaRef{m3.MetaRef, m1.MetaRef}) + require.Error(t, err) + require.True(t, c.client.IsObjectNotFoundErr(err)) - fsNamedStores := bloomClient.storageConfig.NamedStores.Filesystem + _, found = stored[c.Meta(m3.MetaRef).Addr()] + require.False(t, found) + }) +} - firstBlockFullPath := NewPrefixedResolver( - fsNamedStores["folder-1"].Directory, - defaultKeyResolver{}, - ).Block(firstBlockRef).LocalPath() - _ = createBlockFile(t, firstBlockFullPath) - require.FileExists(t, firstBlockFullPath) +func putBlock(t *testing.T, c *BloomClient, tenant string, start model.Time, minFp, maxFp model.Fingerprint) (Block, error) { + step := int64((24 * time.Hour).Seconds()) + day := start.Unix() / step - secondBlockFullPath := NewPrefixedResolver( - fsNamedStores["folder-2"].Directory, - defaultKeyResolver{}, - ).Block(secondBlockRef).LocalPath() - _ = createBlockFile(t, secondBlockFullPath) - require.FileExists(t, secondBlockFullPath) + tmpDir := t.TempDir() + fp, _ := os.CreateTemp(t.TempDir(), "*.tar.gz") - _, err := bloomClient.GetBlock(context.Background(), firstBlockRef) + blockWriter := v1.NewDirectoryBlockWriter(tmpDir) + err := blockWriter.Init() require.NoError(t, err) - // firstBlockActualData, err := io.ReadAll(downloadedFirstBlock.Data) - // require.NoError(t, err) - // require.Equal(t, firstBlockData, string(firstBlockActualData)) - _, err = bloomClient.GetBlock(context.Background(), secondBlockRef) + err = v1.TarGz(fp, v1.NewDirectoryBlockReader(tmpDir)) require.NoError(t, err) - // secondBlockActualData, err := io.ReadAll(downloadedSecondBlock.Data) - // require.NoError(t, err) - // require.Equal(t, secondBlockData, string(secondBlockActualData)) -} -func Test_BloomClient_PutBlocks(t *testing.T) { - bloomClient := createStore(t) + _, _ = fp.Seek(0, 0) + block := Block{ BlockRef: BlockRef{ Ref: Ref{ - TenantID: "tenantA", - TableName: "table_19621", - Bounds: v1.NewBounds(0xeeee, 0xffff), - StartTimestamp: Date(2023, time.September, 21, 5, 0, 0), - EndTimestamp: Date(2023, time.September, 21, 6, 0, 0), - Checksum: 1, + TenantID: tenant, + Bounds: v1.NewBounds(minFp, maxFp), + TableName: fmt.Sprintf("table_%d", day), + StartTimestamp: start, + EndTimestamp: start.Add(12 * time.Hour), }, }, - Data: awsio.ReadSeekNopCloser{ReadSeeker: bytes.NewReader([]byte("data"))}, + Data: fp, } - err := bloomClient.PutBlock(context.Background(), block) - require.NoError(t, err) - - _ = bloomClient.storeDo(block.StartTimestamp, func(s *bloomStoreEntry) error { - c := s.bloomClient.(*BloomClient) - rc, _, err := c.client.GetObject(context.Background(), block.BlockRef.String()) - require.NoError(t, err) - data, err := io.ReadAll(rc) - require.NoError(t, err) - require.Equal(t, "data", string(data)) - return nil - }) + return block, c.client.PutObject(context.Background(), c.Block(block.BlockRef).Addr(), block.Data) } -func Test_BloomClient_DeleteBlocks(t *testing.T) { - block := BlockRef{ - Ref: Ref{ - TenantID: "tenantA", - TableName: "table_19621", - Bounds: v1.NewBounds(0xeeee, 0xffff), - StartTimestamp: Date(2023, time.September, 21, 5, 0, 0), - EndTimestamp: Date(2023, time.September, 21, 6, 0, 0), - Checksum: 1, - }, - } - - bloomClient := createStore(t) - fsNamedStores := bloomClient.storageConfig.NamedStores.Filesystem - blockFullPath := NewPrefixedResolver( - fsNamedStores["folder-1"].Directory, - defaultKeyResolver{}, - ).Block(block).LocalPath() - _ = createBlockFile(t, blockFullPath) - require.FileExists(t, blockFullPath) +func TestBloomClient_GetBlock(t *testing.T) { + c, _ := newMockBloomClient(t) + ctx := context.Background() - err := bloomClient.DeleteBlocks(context.Background(), []BlockRef{block}) + b, err := putBlock(t, c, "tenant", parseTime("2024-02-05 00:00"), 0x0000, 0xffff) require.NoError(t, err) - require.NoFileExists(t, blockFullPath) + t.Run("exists", func(t *testing.T) { + blockDir, err := c.GetBlock(ctx, b.BlockRef) + require.NoError(t, err) + require.Equal(t, b.BlockRef, blockDir.BlockRef) + }) + + t.Run("does not exist", func(t *testing.T) { + blockDir, err := c.GetBlock(ctx, BlockRef{}) + require.Error(t, err) + require.True(t, c.client.IsObjectNotFoundErr(err)) + require.Equal(t, blockDir, BlockDirectory{}) + }) } -func createBlockFile(t *testing.T, dst string) string { - err := os.MkdirAll(dst[:strings.LastIndex(dst, "/")], 0755) - require.NoError(t, err) - fileContent := uuid.NewString() +func TestBloomClient_GetBlocks(t *testing.T) { + c, _ := newMockBloomClient(t) + ctx := context.Background() - src := filepath.Join(t.TempDir(), fileContent) - err = os.WriteFile(src, []byte(fileContent), 0700) + b1, err := putBlock(t, c, "tenant", parseTime("2024-02-05 00:00"), 0x0000, 0x0fff) require.NoError(t, err) - - fp, err := os.OpenFile(dst, os.O_CREATE|os.O_RDWR, 0700) + b2, err := putBlock(t, c, "tenant", parseTime("2024-02-05 00:00"), 0x1000, 0xffff) require.NoError(t, err) - defer fp.Close() - TarGz(t, fp, src) + t.Run("exists", func(t *testing.T) { + blockDirs, err := c.GetBlocks(ctx, []BlockRef{b1.BlockRef, b2.BlockRef}) + require.NoError(t, err) + require.Equal(t, []BlockRef{b1.BlockRef, b2.BlockRef}, []BlockRef{blockDirs[0].BlockRef, blockDirs[1].BlockRef}) + }) - return fileContent + t.Run("does not exist", func(t *testing.T) { + _, err := c.GetBlocks(ctx, []BlockRef{{}}) + require.Error(t, err) + require.True(t, c.client.IsObjectNotFoundErr(err)) + }) } -func TarGz(t *testing.T, dst io.Writer, file string) { - src, err := os.Open(file) - require.NoError(t, err) - defer src.Close() - - gzipper := chunkenc.GetWriterPool(chunkenc.EncGZIP).GetWriter(dst) - defer gzipper.Close() +func TestBloomClient_PutBlock(t *testing.T) { + c, _ := newMockBloomClient(t) + ctx := context.Background() - tarballer := tar.NewWriter(gzipper) - defer tarballer.Close() + start := parseTime("2024-02-05 12:00") - for _, f := range []*os.File{src} { - info, err := f.Stat() - require.NoError(t, err) + tmpDir := t.TempDir() + fp, _ := os.CreateTemp(t.TempDir(), "*.tar.gz") - header, err := tar.FileInfoHeader(info, f.Name()) - require.NoError(t, err) - - err = tarballer.WriteHeader(header) - require.NoError(t, err) + blockWriter := v1.NewDirectoryBlockWriter(tmpDir) + err := blockWriter.Init() + require.NoError(t, err) - _, err = io.Copy(tarballer, f) - require.NoError(t, err) - } -} + err = v1.TarGz(fp, v1.NewDirectoryBlockReader(tmpDir)) + require.NoError(t, err) -func Test_ParseMetaKey(t *testing.T) { - tests := map[string]struct { - objectKey string - expectedRef MetaRef - expectedErr string - }{ - "ValidObjectKey": { - objectKey: "bloom/table/tenant/metas/aaa-bbb-abcdef", - expectedRef: MetaRef{ - Ref: Ref{ - TenantID: "tenant", - TableName: "table", - Bounds: v1.NewBounds(0xaaa, 0xbbb), - StartTimestamp: 0, // ignored - EndTimestamp: 0, // ignored - Checksum: 0xabcdef, - }, + block := Block{ + BlockRef: BlockRef{ + Ref: Ref{ + TenantID: "tenant", + Bounds: v1.NewBounds(0x0000, 0xffff), + TableName: "table_1234", + StartTimestamp: start, + EndTimestamp: start.Add(12 * time.Hour), }, }, - "InvalidObjectKeyDelimiterCount": { - objectKey: "invalid/key/with/too/many/objectKeyWithoutDelimiters", - expectedRef: MetaRef{}, - expectedErr: "failed to split filename parts", - }, - "InvalidMinFingerprint": { - objectKey: "invalid/folder/key/metas/zzz-bbb-abcdef", - expectedErr: "failed to parse bounds", - }, - "InvalidMaxFingerprint": { - objectKey: "invalid/folder/key/metas/123-zzz-abcdef", - expectedErr: "failed to parse bounds", - }, - "InvalidChecksum": { - objectKey: "invalid/folder/key/metas/aaa-bbb-ghijklm", - expectedErr: "failed to parse checksum", - }, + Data: fp, } - for name, data := range tests { - t.Run(name, func(t *testing.T) { - actualRef, err := defaultKeyResolver{}.ParseMetaKey(key(data.objectKey)) - if data.expectedErr != "" { - require.ErrorContains(t, err, data.expectedErr) - return - } - require.NoError(t, err) - require.Equal(t, data.expectedRef, actualRef) - }) - } -} -func createStore(t *testing.T) *BloomStore { - periodicConfigs := createPeriodConfigs() - namedStores := storage.NamedStores{ - Filesystem: map[string]storage.NamedFSConfig{ - "folder-1": {Directory: t.TempDir()}, - "folder-2": {Directory: t.TempDir()}, - }} - //required to populate StoreType map in named config - require.NoError(t, namedStores.Validate()) - storageConfig := storage.Config{ - NamedStores: namedStores, - BloomShipperConfig: bloomshipperconfig.Config{ - WorkingDirectory: t.TempDir(), - BlocksDownloadingQueue: bloomshipperconfig.DownloadingQueueConfig{ - WorkersCount: 1, - }, - }, - } + err = c.PutBlock(ctx, block) + require.NoError(t, err) - metrics := storage.NewClientMetrics() - t.Cleanup(metrics.Unregister) - store, err := NewBloomStore(periodicConfigs, storageConfig, metrics, cache.NewNoopCache(), nil, log.NewNopLogger()) + oc := c.client.(*testutils.InMemoryObjectClient) + stored := oc.Internals() + _, found := stored[c.Block(block.BlockRef).Addr()] + require.True(t, found) + + blockDir, err := c.GetBlock(ctx, block.BlockRef) require.NoError(t, err) - return store -} -func createPeriodConfigs() []config.PeriodConfig { - periodicConfigs := []config.PeriodConfig{ - { - ObjectType: "folder-1", - // from 2023-09-20: table range [19620:19623] - From: parseDayTime("2023-09-20"), - IndexTables: config.IndexPeriodicTableConfig{ - PeriodicTableConfig: config.PeriodicTableConfig{ - Period: day, - // TODO(chaudum): Integrate {,Parse}MetaKey into schema config - // Prefix: "schema_a_table_", - }}, - }, - { - ObjectType: "folder-2", - // from 2023-09-24: table range [19624:19627] - From: parseDayTime("2023-09-24"), - IndexTables: config.IndexPeriodicTableConfig{ - PeriodicTableConfig: config.PeriodicTableConfig{ - Period: day, - // TODO(chaudum): Integrate {,Parse}MetaKey into schema config - // Prefix: "schema_b_table_", - }}, - }, - } - return periodicConfigs + require.Equal(t, block.BlockRef, blockDir.BlockRef) } -func createMetaInStorage(t *testing.T, s Client, tableName string, tenant string, minFingerprint uint64, maxFingerprint uint64, start model.Time) Meta { - end := start.Add(12 * time.Hour) +func TestBloomClient_DeleteBlocks(t *testing.T) { + c, _ := newMockBloomClient(t) + ctx := context.Background() - meta := createMetaEntity(tenant, tableName, minFingerprint, maxFingerprint, start, end) - err := s.PutMeta(context.Background(), meta) + b1, err := putBlock(t, c, "tenant", parseTime("2024-02-05 00:00"), 0x0000, 0xffff) + require.NoError(t, err) + b2, err := putBlock(t, c, "tenant", parseTime("2024-02-06 00:00"), 0x0000, 0xffff) + require.NoError(t, err) + b3, err := putBlock(t, c, "tenant", parseTime("2024-02-07 00:00"), 0x0000, 0xffff) require.NoError(t, err) - t.Log("create meta in store", meta.String()) - return meta -} -func createMetaEntity( - tenant string, - tableName string, - minFingerprint uint64, - maxFingerprint uint64, - startTimestamp model.Time, - endTimestamp model.Time, -) Meta { - return Meta{ - MetaRef: MetaRef{ - Ref: Ref{ - TenantID: tenant, - TableName: tableName, - Bounds: v1.NewBounds(model.Fingerprint(minFingerprint), model.Fingerprint(maxFingerprint)), - StartTimestamp: startTimestamp, - EndTimestamp: endTimestamp, - }, - }, - Tombstones: []BlockRef{ - { - Ref: Ref{ - TenantID: tenant, - Bounds: v1.NewBounds(model.Fingerprint(minFingerprint), model.Fingerprint(maxFingerprint)), - StartTimestamp: startTimestamp, - EndTimestamp: endTimestamp, - }, - }, - }, - Blocks: []BlockRef{ - { - Ref: Ref{ - TenantID: tenant, - Bounds: v1.NewBounds(model.Fingerprint(minFingerprint), model.Fingerprint(maxFingerprint)), - StartTimestamp: startTimestamp, - EndTimestamp: endTimestamp, - }, - }, - }, - } + oc := c.client.(*testutils.InMemoryObjectClient) + stored := oc.Internals() + _, found := stored[c.Block(b1.BlockRef).Addr()] + require.True(t, found) + _, found = stored[c.Block(b2.BlockRef).Addr()] + require.True(t, found) + _, found = stored[c.Block(b3.BlockRef).Addr()] + require.True(t, found) + + t.Run("all deleted", func(t *testing.T) { + err = c.DeleteBlocks(ctx, []BlockRef{b1.BlockRef, b2.BlockRef}) + require.NoError(t, err) + + _, found = stored[c.Block(b1.BlockRef).Addr()] + require.False(t, found) + _, found = stored[c.Block(b2.BlockRef).Addr()] + require.False(t, found) + }) + + t.Run("some not found", func(t *testing.T) { + err = c.DeleteBlocks(ctx, []BlockRef{b3.BlockRef, b1.BlockRef}) + require.Error(t, err) + require.True(t, c.client.IsObjectNotFoundErr(err)) + + _, found = stored[c.Block(b3.BlockRef).Addr()] + require.False(t, found) + }) } diff --git a/pkg/storage/stores/shipper/bloomshipper/fetcher.go b/pkg/storage/stores/shipper/bloomshipper/fetcher.go index cfb0f392d6931..6efb654809da7 100644 --- a/pkg/storage/stores/shipper/bloomshipper/fetcher.go +++ b/pkg/storage/stores/shipper/bloomshipper/fetcher.go @@ -343,7 +343,9 @@ func (q *downloadQueue[T, R]) do(ctx context.Context, task downloadTask[T, R]) { q.mu.LockKey(task.key) defer func() { err := q.mu.UnlockKey(task.key) - level.Error(q.logger).Log("msg", "failed to unlock key in block lock", "err", err) + if err != nil { + level.Error(q.logger).Log("msg", "failed to unlock key in block lock", "key", task.key, "err", err) + } }() q.process(ctx, task) diff --git a/pkg/storage/stores/shipper/bloomshipper/shipper.go b/pkg/storage/stores/shipper/bloomshipper/shipper.go index 0e0604962ed02..614f9c6898194 100644 --- a/pkg/storage/stores/shipper/bloomshipper/shipper.go +++ b/pkg/storage/stores/shipper/bloomshipper/shipper.go @@ -37,13 +37,7 @@ type Limits interface { BloomGatewayBlocksDownloadingParallelism(tenantID string) int } -// TODO(chaudum): resolve and rip out -type StoreAndClient interface { - Store - Client -} - -func NewShipper(client StoreAndClient, config config.Config, _ Limits, logger log.Logger, _ prometheus.Registerer) (*Shipper, error) { +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, diff --git a/pkg/storage/stores/shipper/bloomshipper/store.go b/pkg/storage/stores/shipper/bloomshipper/store.go index 4de9b453eca29..4e3dada7b44e9 100644 --- a/pkg/storage/stores/shipper/bloomshipper/store.go +++ b/pkg/storage/stores/shipper/bloomshipper/store.go @@ -30,9 +30,6 @@ type bloomStoreConfig struct { numWorkers int } -// Compiler check to ensure bloomStoreEntry implements the Client interface -var _ Client = &bloomStoreEntry{} - // Compiler check to ensure bloomStoreEntry implements the Store interface var _ Store = &bloomStoreEntry{} @@ -78,6 +75,12 @@ func (b *bloomStoreEntry) ResolveMetas(ctx context.Context, params MetaSearchPar refs = append(refs, metaRef) } } + + // return empty metaRefs/fetchers if there are no refs + if len(refs) == 0 { + return [][]MetaRef{}, []*Fetcher{}, nil + } + return [][]MetaRef{refs}, []*Fetcher{b.fetcher}, nil } @@ -112,55 +115,12 @@ func (b *bloomStoreEntry) Fetcher(_ model.Time) *Fetcher { return b.fetcher } -// DeleteBlocks implements Client. -func (b *bloomStoreEntry) DeleteBlocks(ctx context.Context, refs []BlockRef) error { - return b.bloomClient.DeleteBlocks(ctx, refs) -} - -// DeleteMeta implements Client. -func (b *bloomStoreEntry) DeleteMetas(ctx context.Context, refs []MetaRef) error { - return b.bloomClient.DeleteMetas(ctx, refs) -} - -// GetBlock implements Client. -func (b *bloomStoreEntry) GetBlock(ctx context.Context, ref BlockRef) (BlockDirectory, error) { - return b.bloomClient.GetBlock(ctx, ref) -} - -// GetBlocks implements Client. -func (b *bloomStoreEntry) GetBlocks(ctx context.Context, refs []BlockRef) ([]BlockDirectory, error) { - return b.fetcher.FetchBlocks(ctx, refs) -} - -// GetMeta implements Client. -func (b *bloomStoreEntry) GetMeta(ctx context.Context, ref MetaRef) (Meta, error) { - return b.bloomClient.GetMeta(ctx, ref) -} - -// GetMetas implements Client. -func (b *bloomStoreEntry) GetMetas(ctx context.Context, refs []MetaRef) ([]Meta, error) { - return b.fetcher.FetchMetas(ctx, refs) -} - -// PutBlocks implements Client. -func (b *bloomStoreEntry) PutBlock(ctx context.Context, block Block) error { - return b.bloomClient.PutBlock(ctx, block) -} - -// PutMeta implements Client. -func (b *bloomStoreEntry) PutMeta(ctx context.Context, meta Meta) error { - return b.bloomClient.PutMeta(ctx, meta) -} - -// Stop implements Client. +// Stop implements Store. func (b bloomStoreEntry) Stop() { b.bloomClient.Stop() b.fetcher.Close() } -// Compiler check to ensure BloomStore implements the Client interface -var _ Client = &BloomStore{} - // Compiler check to ensure BloomStore implements the Store interface var _ Store = &BloomStore{} @@ -267,8 +227,9 @@ func (b *BloomStore) Fetcher(ts model.Time) *Fetcher { // ResolveMetas implements Store. func (b *BloomStore) ResolveMetas(ctx context.Context, params MetaSearchParams) ([][]MetaRef, []*Fetcher, error) { - var refs [][]MetaRef - var fetchers []*Fetcher + refs := make([][]MetaRef, 0, len(b.stores)) + fetchers := make([]*Fetcher, 0, len(b.stores)) + err := b.forStores(ctx, params.Interval, func(innerCtx context.Context, interval Interval, store Store) error { newParams := params newParams.Interval = interval @@ -276,10 +237,14 @@ func (b *BloomStore) ResolveMetas(ctx context.Context, params MetaSearchParams) if err != nil { return err } - refs = append(refs, metas...) - fetchers = append(fetchers, fetcher...) + if len(metas) > 0 { + // only append if there are any results + refs = append(refs, metas...) + fetchers = append(fetchers, fetcher...) + } return nil }) + return refs, fetchers, err } @@ -293,70 +258,22 @@ func (b *BloomStore) FetchMetas(ctx context.Context, params MetaSearchParams) ([ return nil, errors.New("metaRefs and fetchers have unequal length") } - var metas []Meta + metas := []Meta{} for i := range fetchers { res, err := fetchers[i].FetchMetas(ctx, metaRefs[i]) if err != nil { return nil, err } - metas = append(metas, res...) + if len(res) > 0 { + metas = append(metas, res...) + } } return metas, nil } // FetchBlocks implements Store. -func (b *BloomStore) FetchBlocks(ctx context.Context, refs []BlockRef) ([]BlockDirectory, error) { - return b.GetBlocks(ctx, refs) -} - -// DeleteBlocks implements Client. -func (b *BloomStore) DeleteBlocks(ctx context.Context, refs []BlockRef) error { - for _, ref := range refs { - err := b.storeDo( - ref.StartTimestamp, - func(s *bloomStoreEntry) error { - return s.DeleteBlocks(ctx, []BlockRef{ref}) - }, - ) - if err != nil { - return err - } - } - return nil -} - -// DeleteMetas implements Client. -func (b *BloomStore) DeleteMetas(ctx context.Context, refs []MetaRef) error { - for _, ref := range refs { - err := b.storeDo( - ref.StartTimestamp, - func(s *bloomStoreEntry) error { - return s.DeleteMetas(ctx, []MetaRef{ref}) - }, - ) - if err != nil { - return err - } - } - return nil -} - -// GetBlock implements Client. -func (b *BloomStore) GetBlock(ctx context.Context, ref BlockRef) (BlockDirectory, error) { - res := make([]BlockDirectory, 1) - err := b.storeDo(ref.StartTimestamp, func(s *bloomStoreEntry) error { - block, err := s.GetBlock(ctx, ref) - if err != nil { - return err - } - res[0] = block - return nil - }) - return res[0], err -} +func (b *BloomStore) FetchBlocks(ctx context.Context, blocks []BlockRef) ([]BlockDirectory, error) { -// GetBlocks implements Client. -func (b *BloomStore) GetBlocks(ctx context.Context, blocks []BlockRef) ([]BlockDirectory, error) { var refs [][]BlockRef var fetchers []*Fetcher @@ -392,72 +309,7 @@ func (b *BloomStore) GetBlocks(ctx context.Context, blocks []BlockRef) ([]BlockD return results, nil } -// GetMeta implements Client. -func (b *BloomStore) GetMeta(ctx context.Context, ref MetaRef) (Meta, error) { - res := make([]Meta, 1) - err := b.storeDo(ref.StartTimestamp, func(s *bloomStoreEntry) error { - meta, err := s.GetMeta(ctx, ref) - if err != nil { - return err - } - res[0] = meta - return nil - }) - return res[0], err -} - -// GetMetas implements Client. -func (b *BloomStore) GetMetas(ctx context.Context, metas []MetaRef) ([]Meta, error) { - var refs [][]MetaRef - var fetchers []*Fetcher - - for i := len(b.stores) - 1; i >= 0; i-- { - s := b.stores[i] - from, through := s.start, model.Latest - if i < len(b.stores)-1 { - through = b.stores[i+1].start - } - - var res []MetaRef - for _, meta := range metas { - if meta.StartTimestamp >= from && meta.StartTimestamp < through { - res = append(res, meta) - } - } - - if len(res) > 0 { - refs = append(refs, res) - fetchers = append(fetchers, s.Fetcher(s.start)) - } - } - - results := make([]Meta, 0, len(metas)) - for i := range fetchers { - res, err := fetchers[i].FetchMetas(ctx, refs[i]) - results = append(results, res...) - if err != nil { - return results, err - } - } - - return results, nil -} - -// PutBlock implements Client. -func (b *BloomStore) PutBlock(ctx context.Context, block Block) error { - return b.storeDo(block.StartTimestamp, func(s *bloomStoreEntry) error { - return s.PutBlock(ctx, block) - }) -} - -// PutMeta implements Client. -func (b *BloomStore) PutMeta(ctx context.Context, meta Meta) error { - return b.storeDo(meta.StartTimestamp, func(s *bloomStoreEntry) error { - return s.PutMeta(ctx, meta) - }) -} - -// Stop implements Client. +// Stop implements Store. func (b *BloomStore) Stop() { for _, s := range b.stores { s.Stop() diff --git a/pkg/storage/stores/shipper/bloomshipper/store_test.go b/pkg/storage/stores/shipper/bloomshipper/store_test.go new file mode 100644 index 0000000000000..87c2ed067c45f --- /dev/null +++ b/pkg/storage/stores/shipper/bloomshipper/store_test.go @@ -0,0 +1,268 @@ +package bloomshipper + +import ( + "bytes" + "context" + "encoding/json" + "os" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/common/model" + "github.com/stretchr/testify/require" + + "github.com/grafana/loki/pkg/storage" + v1 "github.com/grafana/loki/pkg/storage/bloom/v1" + "github.com/grafana/loki/pkg/storage/chunk/cache" + storageconfig "github.com/grafana/loki/pkg/storage/config" + "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper/config" +) + +func newMockBloomStore(t *testing.T) (*BloomStore, string) { + workDir := t.TempDir() + + periodicConfigs := []storageconfig.PeriodConfig{ + { + ObjectType: storageconfig.StorageTypeInMemory, + From: parseDayTime("2024-01-01"), + IndexTables: storageconfig.IndexPeriodicTableConfig{ + PeriodicTableConfig: storageconfig.PeriodicTableConfig{ + Period: 24 * time.Hour, + // TODO(chaudum): Integrate {,Parse}MetaKey into schema config + // Prefix: "schema_a_table_", + }}, + }, + { + ObjectType: storageconfig.StorageTypeInMemory, + From: parseDayTime("2024-02-01"), + IndexTables: storageconfig.IndexPeriodicTableConfig{ + PeriodicTableConfig: storageconfig.PeriodicTableConfig{ + Period: 24 * time.Hour, + // TODO(chaudum): Integrate {,Parse}MetaKey into schema config + // Prefix: "schema_b_table_", + }}, + }, + } + + storageConfig := storage.Config{ + BloomShipperConfig: config.Config{ + WorkingDirectory: workDir, + BlocksDownloadingQueue: config.DownloadingQueueConfig{ + WorkersCount: 1, + }, + BlocksCache: config.BlocksCacheConfig{ + EmbeddedCacheConfig: cache.EmbeddedCacheConfig{ + MaxSizeItems: 1000, + TTL: 1 * time.Hour, + }, + }, + }, + } + + metrics := storage.NewClientMetrics() + t.Cleanup(metrics.Unregister) + logger := log.NewLogfmtLogger(os.Stderr) + + metasCache := cache.NewMockCache() + blocksCache := NewBlocksCache(storageConfig.BloomShipperConfig, prometheus.NewPedanticRegistry(), logger) + store, err := NewBloomStore(periodicConfigs, storageConfig, metrics, metasCache, blocksCache, logger) + require.NoError(t, err) + t.Cleanup(store.Stop) + + return store, workDir +} + +func createMetaInStorage(store *BloomStore, tenant string, start model.Time, minFp, maxFp model.Fingerprint) (Meta, error) { + meta := Meta{ + MetaRef: MetaRef{ + Ref: Ref{ + TenantID: tenant, + Bounds: v1.NewBounds(minFp, maxFp), + // Unused + // StartTimestamp: start, + // EndTimestamp: start.Add(12 * time.Hour), + }, + }, + Blocks: []BlockRef{}, + Tombstones: []BlockRef{}, + } + err := store.storeDo(start, func(s *bloomStoreEntry) error { + raw, _ := json.Marshal(meta) + meta.MetaRef.Ref.TableName = tablesForRange(s.cfg, NewInterval(start, start.Add(12*time.Hour)))[0] + return s.objectClient.PutObject(context.Background(), s.Meta(meta.MetaRef).Addr(), bytes.NewReader(raw)) + }) + return meta, err +} + +func createBlockInStorage(t *testing.T, store *BloomStore, tenant string, start model.Time, minFp, maxFp model.Fingerprint) (Block, error) { + tmpDir := t.TempDir() + fp, _ := os.CreateTemp(t.TempDir(), "*.tar.gz") + + blockWriter := v1.NewDirectoryBlockWriter(tmpDir) + err := blockWriter.Init() + require.NoError(t, err) + + err = v1.TarGz(fp, v1.NewDirectoryBlockReader(tmpDir)) + require.NoError(t, err) + + _, _ = fp.Seek(0, 0) + + block := Block{ + BlockRef: BlockRef{ + Ref: Ref{ + TenantID: tenant, + Bounds: v1.NewBounds(minFp, maxFp), + StartTimestamp: start, + EndTimestamp: start.Add(12 * time.Hour), + }, + }, + Data: fp, + } + err = store.storeDo(start, func(s *bloomStoreEntry) error { + block.BlockRef.Ref.TableName = tablesForRange(s.cfg, NewInterval(start, start.Add(12*time.Hour)))[0] + return s.objectClient.PutObject(context.Background(), s.Block(block.BlockRef).Addr(), block.Data) + }) + return block, err +} + +func TestBloomStore_ResolveMetas(t *testing.T) { + store, _ := newMockBloomStore(t) + + // schema 1 + // outside of interval, outside of bounds + _, _ = createMetaInStorage(store, "tenant", parseTime("2024-01-19 00:00"), 0x00010000, 0x0001ffff) + // outside of interval, inside of bounds + _, _ = createMetaInStorage(store, "tenant", parseTime("2024-01-19 00:00"), 0x00000000, 0x0000ffff) + // inside of interval, outside of bounds + _, _ = createMetaInStorage(store, "tenant", parseTime("2024-01-20 00:00"), 0x00010000, 0x0001ffff) + // inside of interval, inside of bounds + m1, _ := createMetaInStorage(store, "tenant", parseTime("2024-01-20 00:00"), 0x00000000, 0x0000ffff) + + // schema 2 + // inside of interval, inside of bounds + m2, _ := createMetaInStorage(store, "tenant", parseTime("2024-02-05 00:00"), 0x00000000, 0x0000ffff) + // inside of interval, outside of bounds + _, _ = createMetaInStorage(store, "tenant", parseTime("2024-02-05 00:00"), 0x00010000, 0x0001ffff) + // outside of interval, inside of bounds + _, _ = createMetaInStorage(store, "tenant", parseTime("2024-02-11 00:00"), 0x00000000, 0x0000ffff) + // outside of interval, outside of bounds + _, _ = createMetaInStorage(store, "tenant", parseTime("2024-02-11 00:00"), 0x00010000, 0x0001ffff) + + t.Run("tenant matches", func(t *testing.T) { + ctx := context.Background() + params := MetaSearchParams{ + "tenant", + NewInterval(parseTime("2024-01-20 00:00"), parseTime("2024-02-10 00:00")), + v1.NewBounds(0x00000000, 0x0000ffff), + } + + refs, fetchers, err := store.ResolveMetas(ctx, params) + require.NoError(t, err) + require.Len(t, refs, 2) + require.Len(t, fetchers, 2) + + require.Equal(t, [][]MetaRef{{m1.MetaRef}, {m2.MetaRef}}, refs) + }) + + t.Run("tenant does not match", func(t *testing.T) { + ctx := context.Background() + params := MetaSearchParams{ + "other", + NewInterval(parseTime("2024-01-20 00:00"), parseTime("2024-02-10 00:00")), + v1.NewBounds(0x00000000, 0x0000ffff), + } + + refs, fetchers, err := store.ResolveMetas(ctx, params) + require.NoError(t, err) + require.Len(t, refs, 0) + require.Len(t, fetchers, 0) + require.Equal(t, [][]MetaRef{}, refs) + }) +} + +func TestBloomStore_FetchMetas(t *testing.T) { + store, _ := newMockBloomStore(t) + + // schema 1 + // outside of interval, outside of bounds + _, _ = createMetaInStorage(store, "tenant", parseTime("2024-01-19 00:00"), 0x00010000, 0x0001ffff) + // outside of interval, inside of bounds + _, _ = createMetaInStorage(store, "tenant", parseTime("2024-01-19 00:00"), 0x00000000, 0x0000ffff) + // inside of interval, outside of bounds + _, _ = createMetaInStorage(store, "tenant", parseTime("2024-01-20 00:00"), 0x00010000, 0x0001ffff) + // inside of interval, inside of bounds + m1, _ := createMetaInStorage(store, "tenant", parseTime("2024-01-20 00:00"), 0x00000000, 0x0000ffff) + + // schema 2 + // inside of interval, inside of bounds + m2, _ := createMetaInStorage(store, "tenant", parseTime("2024-02-05 00:00"), 0x00000000, 0x0000ffff) + // inside of interval, outside of bounds + _, _ = createMetaInStorage(store, "tenant", parseTime("2024-02-05 00:00"), 0x00010000, 0x0001ffff) + // outside of interval, inside of bounds + _, _ = createMetaInStorage(store, "tenant", parseTime("2024-02-11 00:00"), 0x00000000, 0x0000ffff) + // outside of interval, outside of bounds + _, _ = createMetaInStorage(store, "tenant", parseTime("2024-02-11 00:00"), 0x00010000, 0x0001ffff) + + t.Run("tenant matches", func(t *testing.T) { + ctx := context.Background() + params := MetaSearchParams{ + "tenant", + NewInterval(parseTime("2024-01-20 00:00"), parseTime("2024-02-10 00:00")), + v1.NewBounds(0x00000000, 0x0000ffff), + } + + metas, err := store.FetchMetas(ctx, params) + require.NoError(t, err) + require.Len(t, metas, 2) + + require.Equal(t, []Meta{m1, m2}, metas) + }) + + t.Run("tenant does not match", func(t *testing.T) { + ctx := context.Background() + params := MetaSearchParams{ + "other", + NewInterval(parseTime("2024-01-20 00:00"), parseTime("2024-02-10 00:00")), + v1.NewBounds(0x00000000, 0x0000ffff), + } + + metas, err := store.FetchMetas(ctx, params) + require.NoError(t, err) + require.Len(t, metas, 0) + require.Equal(t, []Meta{}, metas) + }) +} + +func TestBloomStore_FetchBlocks(t *testing.T) { + store, _ := newMockBloomStore(t) + + // schema 1 + b1, _ := createBlockInStorage(t, store, "tenant", parseTime("2024-01-20 00:00"), 0x00000000, 0x0000ffff) + b2, _ := createBlockInStorage(t, store, "tenant", parseTime("2024-01-20 00:00"), 0x00010000, 0x0001ffff) + // schema 2 + b3, _ := createBlockInStorage(t, store, "tenant", parseTime("2024-02-05 00:00"), 0x00000000, 0x0000ffff) + b4, _ := createBlockInStorage(t, store, "tenant", parseTime("2024-02-05 00:00"), 0x00000000, 0x0001ffff) + + ctx := context.Background() + + // first call fetches two blocks from cache + blockDirs, err := store.FetchBlocks(ctx, []BlockRef{b1.BlockRef, b3.BlockRef}) + require.NoError(t, err) + require.Len(t, blockDirs, 2) + + require.ElementsMatch(t, []BlockRef{b1.BlockRef, b3.BlockRef}, []BlockRef{blockDirs[0].BlockRef, blockDirs[1].BlockRef}) + + // second call fetches two blocks from cache and two from storage + blockDirs, err = store.FetchBlocks(ctx, []BlockRef{b1.BlockRef, b2.BlockRef, b3.BlockRef, b4.BlockRef}) + require.NoError(t, err) + require.Len(t, blockDirs, 4) + + // Note the order: b1 and b2 come from cache, so they are in the beginning of the response + // Do we need to sort the response based on the request order of block refs? + require.ElementsMatch(t, + []BlockRef{b1.BlockRef, b3.BlockRef, b2.BlockRef, b4.BlockRef}, + []BlockRef{blockDirs[0].BlockRef, blockDirs[1].BlockRef, blockDirs[2].BlockRef, blockDirs[3].BlockRef}, + ) +} From 3a92834de05a0aba39ea98a8654233a780b3b900 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Mon, 5 Feb 2024 23:49:44 -0800 Subject: [PATCH 36/45] [Blooms] Stateless archiving (#11872) Moves the archival code to an interface based approach, removing the explicit fs dependency. We can now implement with in-memory solutions. Also refactors to use a tar-file iterator and marks tests as parallelizable. --- pkg/storage/bloom/v1/archive.go | 40 +++++++------ pkg/storage/bloom/v1/archive_test.go | 1 + pkg/storage/bloom/v1/block_writer.go | 1 + pkg/storage/bloom/v1/bloom_tokenizer_test.go | 3 + pkg/storage/bloom/v1/bounds_test.go | 10 ++++ pkg/storage/bloom/v1/builder_test.go | 4 ++ pkg/storage/bloom/v1/dedupe_test.go | 1 + pkg/storage/bloom/v1/fuse_test.go | 1 + pkg/storage/bloom/v1/index_test.go | 3 + pkg/storage/bloom/v1/iter_test.go | 1 + pkg/storage/bloom/v1/merge_test.go | 4 ++ pkg/storage/bloom/v1/reader.go | 61 ++++++++++++++++++++ pkg/storage/bloom/v1/tokenizer_test.go | 2 + pkg/storage/bloom/v1/util_test.go | 1 + 14 files changed, 115 insertions(+), 18 deletions(-) diff --git a/pkg/storage/bloom/v1/archive.go b/pkg/storage/bloom/v1/archive.go index 4c0b124a05cf4..07ed9cd76d7f3 100644 --- a/pkg/storage/bloom/v1/archive.go +++ b/pkg/storage/bloom/v1/archive.go @@ -11,9 +11,16 @@ import ( "github.com/grafana/loki/pkg/chunkenc" ) -func TarGz(dst io.Writer, src *DirectoryBlockReader) error { - if err := src.Init(); err != nil { - return errors.Wrap(err, "error initializing directory block reader") +type TarEntry struct { + Name string + Size int64 + Body io.ReadSeeker +} + +func TarGz(dst io.Writer, reader BlockReader) error { + itr, err := reader.TarEntries() + if err != nil { + return errors.Wrap(err, "error getting tar entries") } gzipper := chunkenc.GetWriterPool(chunkenc.EncGZIP).GetWriter(dst) @@ -22,27 +29,24 @@ func TarGz(dst io.Writer, src *DirectoryBlockReader) error { tarballer := tar.NewWriter(gzipper) defer tarballer.Close() - for _, f := range []*os.File{src.index, src.blooms} { - info, err := f.Stat() - if err != nil { - return errors.Wrapf(err, "error stat'ing file %s", f.Name()) - } - - header, err := tar.FileInfoHeader(info, f.Name()) - if err != nil { - return errors.Wrapf(err, "error creating tar header for file %s", f.Name()) + for itr.Next() { + entry := itr.At() + hdr := &tar.Header{ + Name: entry.Name, + Mode: 0600, + Size: entry.Size, } - if err := tarballer.WriteHeader(header); err != nil { - return errors.Wrapf(err, "error writing tar header for file %s", f.Name()) + if err := tarballer.WriteHeader(hdr); err != nil { + return errors.Wrapf(err, "error writing tar header for file %s", entry.Name) } - if _, err := io.Copy(tarballer, f); err != nil { - return errors.Wrapf(err, "error writing file %s to tarball", f.Name()) + if _, err := io.Copy(tarballer, entry.Body); err != nil { + return errors.Wrapf(err, "error writing file %s to tarball", entry.Name) } - } - return nil + + return itr.Err() } func UnTarGz(dst string, r io.Reader) error { diff --git a/pkg/storage/bloom/v1/archive_test.go b/pkg/storage/bloom/v1/archive_test.go index e0c9407210f20..7dac04e4c8f1a 100644 --- a/pkg/storage/bloom/v1/archive_test.go +++ b/pkg/storage/bloom/v1/archive_test.go @@ -11,6 +11,7 @@ import ( ) func TestArchive(t *testing.T) { + t.Parallel() // for writing files to two dirs for comparison and ensuring they're equal dir1 := t.TempDir() dir2 := t.TempDir() diff --git a/pkg/storage/bloom/v1/block_writer.go b/pkg/storage/bloom/v1/block_writer.go index 99ab65ef9cd40..b7954264ae8ba 100644 --- a/pkg/storage/bloom/v1/block_writer.go +++ b/pkg/storage/bloom/v1/block_writer.go @@ -12,6 +12,7 @@ import ( ) const ( + FileMode = 0644 BloomFileName = "bloom" SeriesFileName = "series" ) diff --git a/pkg/storage/bloom/v1/bloom_tokenizer_test.go b/pkg/storage/bloom/v1/bloom_tokenizer_test.go index 4ba4ec3eb4f1e..682c12f9e37df 100644 --- a/pkg/storage/bloom/v1/bloom_tokenizer_test.go +++ b/pkg/storage/bloom/v1/bloom_tokenizer_test.go @@ -33,6 +33,7 @@ var ( ) func TestPrefixedKeyCreation(t *testing.T) { + t.Parallel() var ones uint64 = 0xffffffffffffffff ref := ChunkRef{ @@ -76,6 +77,7 @@ func TestPrefixedKeyCreation(t *testing.T) { } func TestSetLineTokenizer(t *testing.T) { + t.Parallel() bt := NewBloomTokenizer(DefaultNGramLength, DefaultNGramSkip, metrics) // Validate defaults @@ -89,6 +91,7 @@ func TestSetLineTokenizer(t *testing.T) { } func TestTokenizerPopulate(t *testing.T) { + t.Parallel() var testLine = "this is a log line" bt := NewBloomTokenizer(DefaultNGramLength, DefaultNGramSkip, metrics) diff --git a/pkg/storage/bloom/v1/bounds_test.go b/pkg/storage/bloom/v1/bounds_test.go index 1dabe1b70dca0..629eac61a2af7 100644 --- a/pkg/storage/bloom/v1/bounds_test.go +++ b/pkg/storage/bloom/v1/bounds_test.go @@ -8,29 +8,34 @@ import ( ) func Test_ParseFingerprint(t *testing.T) { + t.Parallel() fp, err := model.ParseFingerprint("7d0") assert.NoError(t, err) assert.Equal(t, model.Fingerprint(2000), fp) } func Test_FingerprintBounds_String(t *testing.T) { + t.Parallel() bounds := NewBounds(10, 2000) assert.Equal(t, "000000000000000a-00000000000007d0", bounds.String()) } func Test_ParseBoundsFromAddr(t *testing.T) { + t.Parallel() bounds, err := ParseBoundsFromAddr("a-7d0") assert.NoError(t, err) assert.Equal(t, NewBounds(10, 2000), bounds) } func Test_ParseBoundsFromParts(t *testing.T) { + t.Parallel() bounds, err := ParseBoundsFromParts("a", "7d0") assert.NoError(t, err) assert.Equal(t, NewBounds(10, 2000), bounds) } func Test_FingerprintBounds_Cmp(t *testing.T) { + t.Parallel() bounds := NewBounds(10, 20) assert.Equal(t, Before, bounds.Cmp(0)) assert.Equal(t, Overlap, bounds.Cmp(10)) @@ -40,6 +45,7 @@ func Test_FingerprintBounds_Cmp(t *testing.T) { } func Test_FingerprintBounds_Overlap(t *testing.T) { + t.Parallel() bounds := NewBounds(10, 20) assert.True(t, bounds.Overlaps(FingerprintBounds{Min: 5, Max: 15})) assert.True(t, bounds.Overlaps(FingerprintBounds{Min: 15, Max: 25})) @@ -50,6 +56,7 @@ func Test_FingerprintBounds_Overlap(t *testing.T) { } func Test_FingerprintBounds_Within(t *testing.T) { + t.Parallel() target := NewBounds(10, 20) assert.False(t, NewBounds(1, 9).Within(target)) assert.False(t, NewBounds(21, 30).Within(target)) @@ -61,6 +68,7 @@ func Test_FingerprintBounds_Within(t *testing.T) { } func Test_FingerprintBounds_Intersection(t *testing.T) { + t.Parallel() target := NewBounds(10, 20) assert.Nil(t, NewBounds(1, 9).Intersection(target)) assert.Nil(t, NewBounds(21, 30).Intersection(target)) @@ -72,6 +80,7 @@ func Test_FingerprintBounds_Intersection(t *testing.T) { } func Test_FingerprintBounds_Union(t *testing.T) { + t.Parallel() target := NewBounds(10, 20) assert.Equal(t, []FingerprintBounds{ {Min: 1, Max: 9}, @@ -90,6 +99,7 @@ func Test_FingerprintBounds_Union(t *testing.T) { } func Test_FingerprintBounds_Xor(t *testing.T) { + t.Parallel() target := NewBounds(10, 20) assert.Equal(t, []FingerprintBounds{ {Min: 1, Max: 9}, diff --git a/pkg/storage/bloom/v1/builder_test.go b/pkg/storage/bloom/v1/builder_test.go index 6bf2c26e7b585..1b988871143ae 100644 --- a/pkg/storage/bloom/v1/builder_test.go +++ b/pkg/storage/bloom/v1/builder_test.go @@ -13,6 +13,7 @@ import ( ) func TestBlockOptionsRoundTrip(t *testing.T) { + t.Parallel() opts := BlockOptions{ Schema: Schema{ version: V1, @@ -123,6 +124,7 @@ func TestBlockBuilderRoundTrip(t *testing.T) { } func TestMergeBuilder(t *testing.T) { + t.Parallel() nBlocks := 10 numSeries := 100 @@ -209,6 +211,7 @@ func TestMergeBuilder(t *testing.T) { } func TestBlockReset(t *testing.T) { + t.Parallel() numSeries := 100 numKeysPerSeries := 10000 data, _ := MkBasicSeriesWithBlooms(numSeries, numKeysPerSeries, 1, 0xffff, 0, 10000) @@ -260,6 +263,7 @@ func TestBlockReset(t *testing.T) { // disjoint data. It then merges the two sets of blocks and ensures that the merged blocks contain // one copy of the first set (duplicate data) and one copy of the second set (disjoint data). func TestMergeBuilder_Roundtrip(t *testing.T) { + t.Parallel() numSeries := 100 numKeysPerSeries := 100 minTs, maxTs := model.Time(0), model.Time(10000) diff --git a/pkg/storage/bloom/v1/dedupe_test.go b/pkg/storage/bloom/v1/dedupe_test.go index 524e3d4a13a56..7e12f25247036 100644 --- a/pkg/storage/bloom/v1/dedupe_test.go +++ b/pkg/storage/bloom/v1/dedupe_test.go @@ -7,6 +7,7 @@ import ( ) func TestMergeDedupeIter(t *testing.T) { + t.Parallel() var ( numSeries = 100 numKeysPerSeries = 10000 diff --git a/pkg/storage/bloom/v1/fuse_test.go b/pkg/storage/bloom/v1/fuse_test.go index 4102ea5de8da9..be0b6d30ed7df 100644 --- a/pkg/storage/bloom/v1/fuse_test.go +++ b/pkg/storage/bloom/v1/fuse_test.go @@ -13,6 +13,7 @@ import ( ) func TestFusedQuerier(t *testing.T) { + t.Parallel() // references for linking in memory reader+writer indexBuf := bytes.NewBuffer(nil) bloomsBuf := bytes.NewBuffer(nil) diff --git a/pkg/storage/bloom/v1/index_test.go b/pkg/storage/bloom/v1/index_test.go index be8d807447dac..b24a142588d2d 100644 --- a/pkg/storage/bloom/v1/index_test.go +++ b/pkg/storage/bloom/v1/index_test.go @@ -10,6 +10,7 @@ import ( ) func TestBloomOffsetEncoding(t *testing.T) { + t.Parallel() src := BloomOffset{Page: 1, ByteOffset: 2} enc := &encoding.Encbuf{} src.Encode(enc, BloomOffset{}) @@ -22,6 +23,7 @@ func TestBloomOffsetEncoding(t *testing.T) { } func TestSeriesEncoding(t *testing.T) { + t.Parallel() src := SeriesWithOffset{ Series: Series{ Fingerprint: model.Fingerprint(1), @@ -54,6 +56,7 @@ func TestSeriesEncoding(t *testing.T) { } func TestChunkRefCompare(t *testing.T) { + t.Parallel() for _, tc := range []struct { desc string left, right, exclusive, inclusive ChunkRefs diff --git a/pkg/storage/bloom/v1/iter_test.go b/pkg/storage/bloom/v1/iter_test.go index 3ec8ead536e75..91604dd27868c 100644 --- a/pkg/storage/bloom/v1/iter_test.go +++ b/pkg/storage/bloom/v1/iter_test.go @@ -7,6 +7,7 @@ import ( ) func TestSliceIterWithIndex(t *testing.T) { + t.Parallel() t.Run("SliceIterWithIndex implements PeekingIterator interface", func(t *testing.T) { xs := []string{"a", "b", "c"} it := NewSliceIterWithIndex(xs, 123) diff --git a/pkg/storage/bloom/v1/merge_test.go b/pkg/storage/bloom/v1/merge_test.go index 1b91366c2cb6d..545ff2dc168d3 100644 --- a/pkg/storage/bloom/v1/merge_test.go +++ b/pkg/storage/bloom/v1/merge_test.go @@ -7,6 +7,7 @@ import ( ) func TestMergeBlockQuerier_NonOverlapping(t *testing.T) { + t.Parallel() var ( numSeries = 100 numKeysPerSeries = 10000 @@ -34,6 +35,7 @@ func TestMergeBlockQuerier_NonOverlapping(t *testing.T) { } func TestMergeBlockQuerier_Duplicate(t *testing.T) { + t.Parallel() var ( numSeries = 100 numKeysPerSeries = 10000 @@ -64,6 +66,8 @@ func TestMergeBlockQuerier_Duplicate(t *testing.T) { } func TestMergeBlockQuerier_Overlapping(t *testing.T) { + t.Parallel() + var ( numSeries = 100 numKeysPerSeries = 10000 diff --git a/pkg/storage/bloom/v1/reader.go b/pkg/storage/bloom/v1/reader.go index d5c70a2b64d83..d73ce38c1ca8d 100644 --- a/pkg/storage/bloom/v1/reader.go +++ b/pkg/storage/bloom/v1/reader.go @@ -12,6 +12,7 @@ import ( type BlockReader interface { Index() (io.ReadSeeker, error) Blooms() (io.ReadSeeker, error) + TarEntries() (Iterator[TarEntry], error) } // In memory reader @@ -31,6 +32,33 @@ func (r *ByteReader) Blooms() (io.ReadSeeker, error) { return bytes.NewReader(r.blooms.Bytes()), nil } +func (r *ByteReader) TarEntries() (Iterator[TarEntry], error) { + indexLn := r.index.Len() + index, err := r.Index() + if err != nil { + return nil, err + } + bloomLn := r.blooms.Len() + blooms, err := r.Blooms() + if err != nil { + return nil, err + } + entries := []TarEntry{ + { + Name: SeriesFileName, + Size: int64(indexLn), + Body: index, + }, + { + Name: BloomFileName, + Size: int64(bloomLn), + Body: blooms, + }, + } + + return NewSliceIter[TarEntry](entries), err +} + // File reader type DirectoryBlockReader struct { dir string @@ -81,3 +109,36 @@ func (r *DirectoryBlockReader) Blooms() (io.ReadSeeker, error) { } return r.blooms, nil } + +func (r *DirectoryBlockReader) TarEntries() (Iterator[TarEntry], error) { + if !r.initialized { + if err := r.Init(); err != nil { + return nil, err + } + } + + idxInfo, err := r.index.Stat() + if err != nil { + return nil, errors.Wrap(err, "error stat'ing series file") + } + + bloomInfo, err := r.blooms.Stat() + if err != nil { + return nil, errors.Wrap(err, "error stat'ing bloom file") + } + + entries := []TarEntry{ + { + Name: SeriesFileName, + Size: idxInfo.Size(), + Body: r.index, + }, + { + Name: BloomFileName, + Size: bloomInfo.Size(), + Body: r.blooms, + }, + } + + return NewSliceIter[TarEntry](entries), nil +} diff --git a/pkg/storage/bloom/v1/tokenizer_test.go b/pkg/storage/bloom/v1/tokenizer_test.go index eec9645fcd25e..e50d056f666ca 100644 --- a/pkg/storage/bloom/v1/tokenizer_test.go +++ b/pkg/storage/bloom/v1/tokenizer_test.go @@ -9,6 +9,7 @@ import ( const BigFile = "../../../logql/sketch/testdata/war_peace.txt" func TestNGramIterator(t *testing.T) { + t.Parallel() var ( three = NewNGramTokenizer(3, 0) threeSkip1 = NewNGramTokenizer(3, 1) @@ -72,6 +73,7 @@ func TestNGramIterator(t *testing.T) { } func TestPrefixedIterator(t *testing.T) { + t.Parallel() var ( three = NewNGramTokenizer(3, 0) ) diff --git a/pkg/storage/bloom/v1/util_test.go b/pkg/storage/bloom/v1/util_test.go index e3f9a18792555..ad89a226ec7f7 100644 --- a/pkg/storage/bloom/v1/util_test.go +++ b/pkg/storage/bloom/v1/util_test.go @@ -7,6 +7,7 @@ import ( ) func TestPeekingIterator(t *testing.T) { + t.Parallel() data := []int{1, 2, 3, 4, 5} itr := NewPeekingIter[int](NewSliceIter[int](data)) From 162360926326eea2e57b3b15e42b2924d121e550 Mon Sep 17 00:00:00 2001 From: Christian Haudum Date: Tue, 6 Feb 2024 12:30:16 +0100 Subject: [PATCH 37/45] Use correct header name in docs for query fairness in scheduler (#11875) --- docs/sources/operations/query-fairness/_index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sources/operations/query-fairness/_index.md b/docs/sources/operations/query-fairness/_index.md index 39f9ede21fbab..44b3c15f8f9ad 100644 --- a/docs/sources/operations/query-fairness/_index.md +++ b/docs/sources/operations/query-fairness/_index.md @@ -115,7 +115,7 @@ you would usually want to avoid this scenario and control yourself where the hea When using Grafana as the Loki user interface, you can, for example, create multiple data sources with the same tenant, but with a different additional HTTP header -`X-Loki-Scope-Actor` and restrict which Grafana user can use which data source. +`X-Loki-Actor-Path` and restrict which Grafana user can use which data source. Alternatively, if you have a proxy for authentication in front of Loki, you can pass the (hashed) user from the authentication as downstream header to Loki. From 6b46e373fb6a3ea58fd22293da5b78d203f09e08 Mon Sep 17 00:00:00 2001 From: Robert Jacob Date: Tue, 6 Feb 2024 19:07:01 +0100 Subject: [PATCH 38/45] operator: Integrate support for OpenShift-managed credentials in Azure (#11868) --- operator/CHANGELOG.md | 1 + .../apis/config/v1/projectconfig_types.go | 4 +- .../loki-operator.clusterserviceversion.yaml | 2 +- .../loki-operator.clusterserviceversion.yaml | 2 +- .../loki-operator.clusterserviceversion.yaml | 4 +- .../loki-operator.clusterserviceversion.yaml | 2 +- .../loki/credentialsrequests_controller.go | 13 +- .../credentialsrequests_controller_test.go | 15 +- .../handlers/credentialsrequest_create.go | 26 +- .../credentialsrequest_create_test.go | 70 +- .../handlers/internal/storage/secrets.go | 40 +- .../handlers/internal/storage/secrets_test.go | 101 +- .../manifests/openshift/credentialsrequest.go | 41 + .../internal/manifests/openshift/options.go | 10 +- .../internal/manifests/storage/configure.go | 13 + .../manifests/storage/configure_test.go | 1038 ++++++++++++++--- .../internal/manifests/storage/options.go | 1 + operator/internal/manifests/storage/var.go | 8 + 18 files changed, 1168 insertions(+), 223 deletions(-) diff --git a/operator/CHANGELOG.md b/operator/CHANGELOG.md index 6e2b1e741b02f..d978c0c8f423d 100644 --- a/operator/CHANGELOG.md +++ b/operator/CHANGELOG.md @@ -1,5 +1,6 @@ ## Main +- [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 - [11824](https://github.com/grafana/loki/pull/11824) **xperimental**: Improve messages for errors in storage secret diff --git a/operator/apis/config/v1/projectconfig_types.go b/operator/apis/config/v1/projectconfig_types.go index ba7cc703c5bb8..06ff8cb090598 100644 --- a/operator/apis/config/v1/projectconfig_types.go +++ b/operator/apis/config/v1/projectconfig_types.go @@ -56,7 +56,9 @@ type OpenShiftFeatureGates struct { ManagedAuthEnv bool } -func (o OpenShiftFeatureGates) ManagedAuthEnabled() 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 } diff --git a/operator/bundle/community-openshift/manifests/loki-operator.clusterserviceversion.yaml b/operator/bundle/community-openshift/manifests/loki-operator.clusterserviceversion.yaml index 4b20f814804a2..6854bf38ff661 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-25T11:08:43Z" + createdAt: "2024-01-31T16:48:07Z" description: The Community Loki Operator provides Kubernetes native deployment and management of Loki and related logging components. features.operators.openshift.io/disconnected: "true" diff --git a/operator/bundle/community/manifests/loki-operator.clusterserviceversion.yaml b/operator/bundle/community/manifests/loki-operator.clusterserviceversion.yaml index 81575be404e82..f8c37162b5a44 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-25T11:08:41Z" + createdAt: "2024-01-31T16:48:04Z" description: The Community Loki Operator provides Kubernetes native deployment and management of Loki and related logging components. operators.operatorframework.io/builder: operator-sdk-unknown diff --git a/operator/bundle/openshift/manifests/loki-operator.clusterserviceversion.yaml b/operator/bundle/openshift/manifests/loki-operator.clusterserviceversion.yaml index b79f4ea7a2f49..234ddb423a3aa 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-25T11:08:45Z" + createdAt: "2024-01-31T16:48:10Z" description: | The Loki Operator for OCP provides a means for configuring and managing a Loki stack for cluster logging. ## Prerequisites and Requirements @@ -165,7 +165,7 @@ metadata: features.operators.openshift.io/proxy-aware: "true" features.operators.openshift.io/tls-profiles: "true" features.operators.openshift.io/token-auth-aws: "true" - features.operators.openshift.io/token-auth-azure: "false" + features.operators.openshift.io/token-auth-azure: "true" features.operators.openshift.io/token-auth-gcp: "false" olm.skipRange: '>=5.7.0-0 <5.9.0' operatorframework.io/cluster-monitoring: "true" diff --git a/operator/config/manifests/openshift/bases/loki-operator.clusterserviceversion.yaml b/operator/config/manifests/openshift/bases/loki-operator.clusterserviceversion.yaml index 0e724292edbb6..48a221736e2dc 100644 --- a/operator/config/manifests/openshift/bases/loki-operator.clusterserviceversion.yaml +++ b/operator/config/manifests/openshift/bases/loki-operator.clusterserviceversion.yaml @@ -21,7 +21,7 @@ metadata: features.operators.openshift.io/proxy-aware: "true" features.operators.openshift.io/tls-profiles: "true" features.operators.openshift.io/token-auth-aws: "true" - features.operators.openshift.io/token-auth-azure: "false" + features.operators.openshift.io/token-auth-azure: "true" features.operators.openshift.io/token-auth-gcp: "false" olm.skipRange: '>=5.7.0-0 <5.9.0' operatorframework.io/cluster-monitoring: "true" diff --git a/operator/controllers/loki/credentialsrequests_controller.go b/operator/controllers/loki/credentialsrequests_controller.go index 61d0b58423e90..efd0226c6a340 100644 --- a/operator/controllers/loki/credentialsrequests_controller.go +++ b/operator/controllers/loki/credentialsrequests_controller.go @@ -4,6 +4,7 @@ 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" @@ -46,7 +47,17 @@ func (r *CredentialsRequestsReconciler) Reconcile(ctx context.Context, req ctrl. return ctrl.Result{}, nil } - secretRef, err := handlers.CreateCredentialsRequest(ctx, r.Client, req.NamespacedName) + 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 } diff --git a/operator/controllers/loki/credentialsrequests_controller_test.go b/operator/controllers/loki/credentialsrequests_controller_test.go index e6738c1d1796e..3c91ee2275e97 100644 --- a/operator/controllers/loki/credentialsrequests_controller_test.go +++ b/operator/controllers/loki/credentialsrequests_controller_test.go @@ -6,6 +6,7 @@ import ( 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" @@ -81,16 +82,24 @@ func TestCredentialsRequestController_CreateCredentialsRequest_WhenLokiStackNotA 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 { - if key.Name == r.Name && key.Namespace == r.Namespace { - k.SetClientObject(out, &s) + 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 apierrors.NewNotFound(schema.GroupResource{}, "lokistack not found") + return nil } k.CreateStub = func(_ context.Context, o client.Object, _ ...client.CreateOption) error { diff --git a/operator/internal/handlers/credentialsrequest_create.go b/operator/internal/handlers/credentialsrequest_create.go index 477528326b9a5..6074e10b2d5af 100644 --- a/operator/internal/handlers/credentialsrequest_create.go +++ b/operator/internal/handlers/credentialsrequest_create.go @@ -2,23 +2,47 @@ package handlers import ( "context" + "errors" "github.com/ViaQ/logerr/v2/kverrors" + corev1 "k8s.io/api/core/v1" apierrors "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" + "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") ) // 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) (string, error) { +func CreateCredentialsRequest(ctx context.Context, k k8s.Client, stack client.ObjectKey, secret *corev1.Secret) (string, error) { managedAuthEnv := openshift.DiscoverManagedAuthEnv() if managedAuthEnv == nil { return "", nil } + if managedAuthEnv.Azure != nil && managedAuthEnv.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 + } + + region := secret.Data[storage.KeyAzureRegion] + if len(region) == 0 { + return "", errAzureNoRegion + } + + managedAuthEnv.Azure.Region = string(region) + } + opts := openshift.Options{ BuildOpts: openshift.BuildOptions{ LokiStackName: stack.Name, diff --git a/operator/internal/handlers/credentialsrequest_create_test.go b/operator/internal/handlers/credentialsrequest_create_test.go index f6bf9c0f1b526..df903eaec662f 100644 --- a/operator/internal/handlers/credentialsrequest_create_test.go +++ b/operator/internal/handlers/credentialsrequest_create_test.go @@ -4,7 +4,9 @@ import ( "context" "testing" + cloudcredentialv1 "github.com/openshift/cloud-credential-operator/pkg/apis/cloudcredential/v1" "github.com/stretchr/testify/require" + corev1 "k8s.io/api/core/v1" "k8s.io/apimachinery/pkg/api/errors" "k8s.io/apimachinery/pkg/runtime/schema" "sigs.k8s.io/controller-runtime/pkg/client" @@ -16,7 +18,7 @@ func TestCreateCredentialsRequest_DoNothing_WhenManagedAuthEnvMissing(t *testing k := &k8sfakes.FakeClient{} key := client.ObjectKey{Name: "my-stack", Namespace: "ns"} - secretRef, err := CreateCredentialsRequest(context.Background(), k, key) + secretRef, err := CreateCredentialsRequest(context.Background(), k, key, nil) require.NoError(t, err) require.Empty(t, secretRef) } @@ -27,12 +29,74 @@ func TestCreateCredentialsRequest_CreateNewResource(t *testing.T) { t.Setenv("ROLEARN", "a-role-arn") - secretRef, err := CreateCredentialsRequest(context.Background(), k, key) + secretRef, err := CreateCredentialsRequest(context.Background(), k, key, nil) require.NoError(t, err) require.NotEmpty(t, secretRef) require.Equal(t, 1, k.CreateCallCount()) } +func TestCreateCredentialsRequest_CreateNewResourceAzure(t *testing.T) { + wantRegion := "test-region" + + k := &k8sfakes.FakeClient{} + key := client.ObjectKey{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") + + secretRef, err := CreateCredentialsRequest(context.Background(), k, key, secret) + 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) + + providerSpec := &cloudcredentialv1.AzureProviderSpec{} + require.NoError(t, cloudcredentialv1.Codec.DecodeProviderSpec(credReq.Spec.ProviderSpec, providerSpec)) + + require.Equal(t, wantRegion, providerSpec.AzureRegion) +} + +func TestCreateCredentialsRequest_CreateNewResourceAzure_Errors(t *testing.T) { + k := &k8sfakes.FakeClient{} + key := 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(), + }, + } + + 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) + require.EqualError(t, err, tc.wantError) + }) + } +} + func TestCreateCredentialsRequest_DoNothing_WhenCredentialsRequestExist(t *testing.T) { k := &k8sfakes.FakeClient{} key := client.ObjectKey{Name: "my-stack", Namespace: "ns"} @@ -43,7 +107,7 @@ func TestCreateCredentialsRequest_DoNothing_WhenCredentialsRequestExist(t *testi return errors.NewAlreadyExists(schema.GroupResource{}, "credentialsrequest exists") } - secretRef, err := CreateCredentialsRequest(context.Background(), k, key) + secretRef, err := CreateCredentialsRequest(context.Background(), k, key, nil) require.NoError(t, err) require.NotEmpty(t, secretRef) require.Equal(t, 1, k.CreateCallCount()) diff --git a/operator/internal/handlers/internal/storage/secrets.go b/operator/internal/handlers/internal/storage/secrets.go index d2d39e5ac8570..6b8275d2d28ae 100644 --- a/operator/internal/handlers/internal/storage/secrets.go +++ b/operator/internal/handlers/internal/storage/secrets.go @@ -29,8 +29,9 @@ var ( errS3NoAuth = errors.New("missing secret fields for static or sts authentication") - 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") + 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") ) func getSecrets(ctx context.Context, k k8s.Client, stack *lokiv1.LokiStack, fg configv1.FeatureGates) (*corev1.Secret, *corev1.Secret, error) { @@ -110,7 +111,7 @@ func extractSecrets(secretType lokiv1.ObjectStorageSecretType, objStore, managed switch secretType { case lokiv1.ObjectStorageSecretAzure: - storageOpts.Azure, err = extractAzureConfigSecret(objStore) + storageOpts.Azure, err = extractAzureConfigSecret(objStore, fg) case lokiv1.ObjectStorageSecretGCS: storageOpts.GCS, err = extractGCSConfigSecret(objStore) case lokiv1.ObjectStorageSecretS3: @@ -158,41 +159,62 @@ func hashSecretData(s *corev1.Secret) (string, error) { return fmt.Sprintf("%x", h.Sum(nil)), nil } -func extractAzureConfigSecret(s *corev1.Secret) (*storage.AzureStorageConfig, error) { +func extractAzureConfigSecret(s *corev1.Secret, fg configv1.FeatureGates) (*storage.AzureStorageConfig, error) { // Extract and validate mandatory fields env := s.Data[storage.KeyAzureEnvironmentName] if len(env) == 0 { return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyAzureEnvironmentName) } + + accountName := s.Data[storage.KeyAzureStorageAccountName] + if len(accountName) == 0 { + return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyAzureStorageAccountName) + } + container := s.Data[storage.KeyAzureStorageContainerName] if len(container) == 0 { return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyAzureStorageContainerName) } - workloadIdentity, err := validateAzureCredentials(s) + + workloadIdentity, err := validateAzureCredentials(s, fg) if err != nil { return nil, err } // Extract and validate optional fields endpointSuffix := s.Data[storage.KeyAzureStorageEndpointSuffix] + audience := s.Data[storage.KeyAzureAudience] + + if !workloadIdentity && len(audience) > 0 { + return nil, fmt.Errorf("%w: %s", errSecretFieldNotAllowed, storage.KeyAzureAudience) + } return &storage.AzureStorageConfig{ Env: string(env), Container: string(container), EndpointSuffix: string(endpointSuffix), + Audience: string(audience), WorkloadIdentity: workloadIdentity, }, nil } -func validateAzureCredentials(s *corev1.Secret) (workloadIdentity bool, err error) { - accountName := s.Data[storage.KeyAzureStorageAccountName] +func validateAzureCredentials(s *corev1.Secret, fg configv1.FeatureGates) (workloadIdentity bool, err error) { accountKey := s.Data[storage.KeyAzureStorageAccountKey] clientID := s.Data[storage.KeyAzureStorageClientID] tenantID := s.Data[storage.KeyAzureStorageTenantID] subscriptionID := s.Data[storage.KeyAzureStorageSubscriptionID] - if len(accountName) == 0 { - return false, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyAzureStorageAccountName) + if fg.OpenShift.ManagedAuthEnabled() { + region := s.Data[storage.KeyAzureRegion] + if len(region) == 0 { + return false, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyAzureRegion) + } + + if len(accountKey) > 0 || len(clientID) > 0 || len(tenantID) > 0 || len(subscriptionID) > 0 { + return false, errAzureManagedIdentityNoOverride + } + + return true, nil } if len(accountKey) == 0 && len(clientID) == 0 { diff --git a/operator/internal/handlers/internal/storage/secrets_test.go b/operator/internal/handlers/internal/storage/secrets_test.go index 51dc6e15f670e..94b6ae2e3aaa1 100644 --- a/operator/internal/handlers/internal/storage/secrets_test.go +++ b/operator/internal/handlers/internal/storage/secrets_test.go @@ -71,9 +71,11 @@ func TestUnknownType(t *testing.T) { func TestAzureExtract(t *testing.T) { type test struct { - name string - secret *corev1.Secret - wantError string + name string + secret *corev1.Secret + managedSecret *corev1.Secret + featureGates configv1.FeatureGates + wantError string } table := []test{ { @@ -82,23 +84,23 @@ func TestAzureExtract(t *testing.T) { wantError: "missing secret field: environment", }, { - name: "missing container", + name: "missing account_name", secret: &corev1.Secret{ Data: map[string][]byte{ "environment": []byte("here"), }, }, - wantError: "missing secret field: container", + wantError: "missing secret field: account_name", }, { - name: "missing account_name", + name: "missing container", secret: &corev1.Secret{ Data: map[string][]byte{ - "environment": []byte("here"), - "container": []byte("this,that"), + "environment": []byte("here"), + "account_name": []byte("id"), }, }, - wantError: "missing secret field: account_name", + wantError: "missing secret field: container", }, { name: "no account_key or client_id", @@ -153,6 +155,64 @@ func TestAzureExtract(t *testing.T) { }, wantError: "missing secret field: subscription_id", }, + { + name: "managed auth - no region", + secret: &corev1.Secret{ + ObjectMeta: metav1.ObjectMeta{Name: "test"}, + Data: map[string][]byte{ + "environment": []byte("here"), + "account_name": []byte("test-account-name"), + "container": []byte("this,that"), + }, + }, + managedSecret: &corev1.Secret{ + Data: map[string][]byte{}, + }, + featureGates: configv1.FeatureGates{ + OpenShift: configv1.OpenShiftFeatureGates{ + Enabled: true, + ManagedAuthEnv: true, + }, + }, + wantError: "missing secret field: region", + }, + { + name: "managed auth - no auth override", + secret: &corev1.Secret{ + ObjectMeta: metav1.ObjectMeta{Name: "test"}, + Data: map[string][]byte{ + "environment": []byte("here"), + "account_name": []byte("test-account-name"), + "container": []byte("this,that"), + "region": []byte("test-region"), + "account_key": []byte("test-account-key"), + }, + }, + managedSecret: &corev1.Secret{ + Data: map[string][]byte{}, + }, + featureGates: configv1.FeatureGates{ + OpenShift: configv1.OpenShiftFeatureGates{ + Enabled: true, + ManagedAuthEnv: true, + }, + }, + wantError: errAzureManagedIdentityNoOverride.Error(), + }, + { + name: "audience used with static authentication", + secret: &corev1.Secret{ + ObjectMeta: metav1.ObjectMeta{Name: "test"}, + Data: map[string][]byte{ + "environment": []byte("here"), + "container": []byte("this,that"), + "account_name": []byte("id"), + "account_key": []byte("secret"), + "audience": []byte("test-audience"), + }, + }, + wantError: "secret field not allowed: audience", + }, { name: "mandatory for normal authentication set", secret: &corev1.Secret{ @@ -180,6 +240,27 @@ func TestAzureExtract(t *testing.T) { }, }, }, + { + name: "mandatory for managed workload-identity set", + secret: &corev1.Secret{ + ObjectMeta: metav1.ObjectMeta{Name: "test"}, + Data: map[string][]byte{ + "environment": []byte("here"), + "account_name": []byte("test-account-name"), + "container": []byte("this,that"), + "region": []byte("test-region"), + }, + }, + managedSecret: &corev1.Secret{ + Data: map[string][]byte{}, + }, + featureGates: configv1.FeatureGates{ + OpenShift: configv1.OpenShiftFeatureGates{ + Enabled: true, + ManagedAuthEnv: true, + }, + }, + }, { name: "all set including optional", secret: &corev1.Secret{ @@ -199,7 +280,7 @@ func TestAzureExtract(t *testing.T) { t.Run(tst.name, func(t *testing.T) { t.Parallel() - opts, err := extractSecrets(lokiv1.ObjectStorageSecretAzure, tst.secret, nil, configv1.FeatureGates{}) + opts, err := extractSecrets(lokiv1.ObjectStorageSecretAzure, tst.secret, tst.managedSecret, tst.featureGates) if tst.wantError == "" { require.NoError(t, err) require.NotEmpty(t, opts.SecretName) diff --git a/operator/internal/manifests/openshift/credentialsrequest.go b/operator/internal/manifests/openshift/credentialsrequest.go index d2da20a194534..2962b61d0d1ef 100644 --- a/operator/internal/manifests/openshift/credentialsrequest.go +++ b/operator/internal/manifests/openshift/credentialsrequest.go @@ -74,6 +74,34 @@ 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 + + spec = &cloudcredentialv1.AzureProviderSpec{ + Permissions: []string{ + "Microsoft.Storage/storageAccounts/blobServices/read", + "Microsoft.Storage/storageAccounts/blobServices/containers/read", + "Microsoft.Storage/storageAccounts/blobServices/containers/write", + "Microsoft.Storage/storageAccounts/blobServices/generateUserDelegationKey/action", + "Microsoft.Storage/storageAccounts/read", + "Microsoft.Storage/storageAccounts/write", + "Microsoft.Storage/storageAccounts/delete", + "Microsoft.Storage/storageAccounts/listKeys/action", + "Microsoft.Resources/tags/write", + }, + DataPermissions: []string{ + "Microsoft.Storage/storageAccounts/blobServices/containers/blobs/delete", + "Microsoft.Storage/storageAccounts/blobServices/containers/blobs/write", + "Microsoft.Storage/storageAccounts/blobServices/containers/blobs/read", + "Microsoft.Storage/storageAccounts/blobServices/containers/blobs/add/action", + "Microsoft.Storage/storageAccounts/blobServices/containers/blobs/move/action", + }, + AzureClientID: azure.ClientID, + AzureRegion: azure.Region, + AzureSubscriptionID: azure.SubscriptionID, + AzureTenantID: azure.TenantID, + } + secretName = fmt.Sprintf("%s-azure-creds", stackName) } encodedSpec, err := cloudcredentialv1.Codec.EncodeProviderSpec(spec.DeepCopyObject()) @@ -84,6 +112,11 @@ 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{ @@ -91,6 +124,14 @@ func DiscoverManagedAuthEnv() *ManagedAuthEnv { RoleARN: roleARN, }, } + case clientID != "" && tenantID != "" && subscriptionID != "": + return &ManagedAuthEnv{ + Azure: &AzureWIFEnvironment{ + ClientID: clientID, + SubscriptionID: subscriptionID, + TenantID: tenantID, + }, + } } return nil diff --git a/operator/internal/manifests/openshift/options.go b/operator/internal/manifests/openshift/options.go index e5d33a3355269..9bc2e4faae36e 100644 --- a/operator/internal/manifests/openshift/options.go +++ b/operator/internal/manifests/openshift/options.go @@ -59,8 +59,16 @@ type AWSSTSEnv struct { RoleARN string } +type AzureWIFEnvironment struct { + ClientID string + SubscriptionID string + TenantID string + Region string +} + type ManagedAuthEnv struct { - AWS *AWSSTSEnv + AWS *AWSSTSEnv + Azure *AzureWIFEnvironment } // NewOptions returns an openshift options struct. diff --git a/operator/internal/manifests/storage/configure.go b/operator/internal/manifests/storage/configure.go index da5f6970da171..f3fd86ebbaa1c 100644 --- a/operator/internal/manifests/storage/configure.go +++ b/operator/internal/manifests/storage/configure.go @@ -193,6 +193,16 @@ func managedAuthCredentials(opts Options) []corev1.EnvVar { } } case lokiv1.ObjectStorageSecretAzure: + if opts.OpenShift.ManagedAuthEnabled() { + return []corev1.EnvVar{ + envVarFromSecret(EnvAzureStorageAccountName, opts.SecretName, KeyAzureStorageAccountName), + 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")), + } + } + return []corev1.EnvVar{ envVarFromSecret(EnvAzureStorageAccountName, opts.SecretName, KeyAzureStorageAccountName), envVarFromSecret(EnvAzureClientID, opts.SecretName, KeyAzureStorageClientID), @@ -310,6 +320,9 @@ func saTokenVolume(opts Options) corev1.Volume { } case lokiv1.ObjectStorageSecretAzure: audience = azureDefaultAudience + if opts.Azure.Audience != "" { + audience = opts.Azure.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 0c0505fbe63e5..03e22682f4028 100644 --- a/operator/internal/manifests/storage/configure_test.go +++ b/operator/internal/manifests/storage/configure_test.go @@ -293,10 +293,14 @@ func TestConfigureDeploymentForStorageType(t *testing.T) { }, }, { - desc: "object storage GCS", + desc: "object storage Azure with WIF and custom audience", opts: Options{ SecretName: "test", - SharedStore: lokiv1.ObjectStorageSecretGCS, + SharedStore: lokiv1.ObjectStorageSecretAzure, + Azure: &AzureStorageConfig{ + WorkloadIdentity: true, + Audience: "custom-audience", + }, }, dpl: &appsv1.Deployment{ Spec: appsv1.DeploymentSpec{ @@ -324,161 +328,60 @@ func TestConfigureDeploymentForStorageType(t *testing.T) { ReadOnly: false, MountPath: "/etc/storage/secrets", }, - }, - Env: []corev1.EnvVar{ - { - Name: EnvGoogleApplicationCredentials, - Value: "/etc/storage/secrets/key.json", - }, - }, - }, - }, - Volumes: []corev1.Volume{ - { - Name: "test", - VolumeSource: corev1.VolumeSource{ - Secret: &corev1.SecretVolumeSource{ - SecretName: "test", - }, - }, - }, - }, - }, - }, - }, - }, - }, - { - desc: "object storage S3", - opts: Options{ - SecretName: "test", - SharedStore: lokiv1.ObjectStorageSecretS3, - }, - 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", + Name: saTokenVolumeName, ReadOnly: false, - MountPath: "/etc/storage/secrets", + MountPath: "/var/run/secrets/azure/serviceaccount", }, }, Env: []corev1.EnvVar{ { - Name: EnvAWSAccessKeyID, + Name: EnvAzureStorageAccountName, ValueFrom: &corev1.EnvVarSource{ SecretKeyRef: &corev1.SecretKeySelector{ LocalObjectReference: corev1.LocalObjectReference{ Name: "test", }, - Key: KeyAWSAccessKeyID, + Key: KeyAzureStorageAccountName, }, }, }, { - Name: EnvAWSAccessKeySecret, + Name: EnvAzureClientID, ValueFrom: &corev1.EnvVarSource{ SecretKeyRef: &corev1.SecretKeySelector{ LocalObjectReference: corev1.LocalObjectReference{ Name: "test", }, - Key: KeyAWSAccessKeySecret, + Key: KeyAzureStorageClientID, }, }, }, - }, - }, - }, - Volumes: []corev1.Volume{ - { - Name: "test", - VolumeSource: corev1.VolumeSource{ - Secret: &corev1.SecretVolumeSource{ - SecretName: "test", - }, - }, - }, - }, - }, - }, - }, - }, - }, - { - desc: "object storage S3 in STS Mode", - opts: Options{ - SecretName: "test", - SharedStore: lokiv1.ObjectStorageSecretS3, - S3: &S3StorageConfig{ - STS: true, - Audience: "test", - }, - }, - 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/aws/serviceaccount", + Name: EnvAzureTenantID, + ValueFrom: &corev1.EnvVarSource{ + SecretKeyRef: &corev1.SecretKeySelector{ + LocalObjectReference: corev1.LocalObjectReference{ + Name: "test", + }, + Key: KeyAzureStorageTenantID, + }, + }, }, - }, - Env: []corev1.EnvVar{ { - Name: EnvAWSRoleArn, + Name: EnvAzureSubscriptionID, ValueFrom: &corev1.EnvVarSource{ SecretKeyRef: &corev1.SecretKeySelector{ LocalObjectReference: corev1.LocalObjectReference{ Name: "test", }, - Key: KeyAWSRoleArn, + Key: KeyAzureStorageSubscriptionID, }, }, }, { - Name: "AWS_WEB_IDENTITY_TOKEN_FILE", - Value: "/var/run/secrets/aws/serviceaccount/token", + Name: EnvAzureFederatedTokenFile, + Value: "/var/run/secrets/azure/serviceaccount/token", }, }, }, @@ -499,7 +402,7 @@ func TestConfigureDeploymentForStorageType(t *testing.T) { Sources: []corev1.VolumeProjection{ { ServiceAccountToken: &corev1.ServiceAccountTokenProjection{ - Audience: "test", + Audience: "custom-audience", ExpirationSeconds: ptr.To[int64](3600), Path: corev1.ServiceAccountTokenKey, }, @@ -515,12 +418,12 @@ func TestConfigureDeploymentForStorageType(t *testing.T) { }, }, { - desc: "object storage S3 in STS Mode in OpenShift", + desc: "object storage Azure with WIF and OpenShift Managed Credentials", opts: Options{ SecretName: "test", - SharedStore: lokiv1.ObjectStorageSecretS3, - S3: &S3StorageConfig{ - STS: true, + SharedStore: lokiv1.ObjectStorageSecretAzure, + Azure: &AzureStorageConfig{ + WorkloadIdentity: true, }, OpenShift: OpenShiftOptions{ Enabled: true, @@ -559,22 +462,61 @@ func TestConfigureDeploymentForStorageType(t *testing.T) { { Name: saTokenVolumeName, ReadOnly: false, - MountPath: "/var/run/secrets/aws/serviceaccount", + MountPath: "/var/run/secrets/azure/serviceaccount", }, { Name: "cloud-credentials", - ReadOnly: false, - MountPath: "/etc/storage/managed-auth", + MountPath: managedAuthSecretDirectory, }, }, Env: []corev1.EnvVar{ { - Name: "AWS_SHARED_CREDENTIALS_FILE", - Value: "/etc/storage/managed-auth/credentials", + Name: EnvAzureStorageAccountName, + ValueFrom: &corev1.EnvVarSource{ + SecretKeyRef: &corev1.SecretKeySelector{ + LocalObjectReference: corev1.LocalObjectReference{ + Name: "test", + }, + Key: KeyAzureStorageAccountName, + }, + }, }, { - Name: "AWS_SDK_LOAD_CONFIG", - Value: "true", + Name: EnvAzureClientID, + ValueFrom: &corev1.EnvVarSource{ + SecretKeyRef: &corev1.SecretKeySelector{ + LocalObjectReference: corev1.LocalObjectReference{ + Name: "cloud-credentials", + }, + Key: azureManagedCredentialKeyClientID, + }, + }, + }, + { + Name: EnvAzureTenantID, + ValueFrom: &corev1.EnvVarSource{ + SecretKeyRef: &corev1.SecretKeySelector{ + LocalObjectReference: corev1.LocalObjectReference{ + Name: "cloud-credentials", + }, + Key: azureManagedCredentialKeyTenantID, + }, + }, + }, + { + Name: EnvAzureSubscriptionID, + ValueFrom: &corev1.EnvVarSource{ + SecretKeyRef: &corev1.SecretKeySelector{ + LocalObjectReference: corev1.LocalObjectReference{ + Name: "cloud-credentials", + }, + Key: azureManagedCredentialKeySubscriptionID, + }, + }, + }, + { + Name: EnvAzureFederatedTokenFile, + Value: "/var/run/secrets/azure/serviceaccount/token", }, }, }, @@ -595,7 +537,7 @@ func TestConfigureDeploymentForStorageType(t *testing.T) { Sources: []corev1.VolumeProjection{ { ServiceAccountToken: &corev1.ServiceAccountTokenProjection{ - Audience: awsDefaultAudience, + Audience: azureDefaultAudience, ExpirationSeconds: ptr.To[int64](3600), Path: corev1.ServiceAccountTokenKey, }, @@ -619,16 +561,10 @@ func TestConfigureDeploymentForStorageType(t *testing.T) { }, }, { - desc: "object storage S3 with SSE KMS encryption context", + desc: "object storage GCS", opts: Options{ SecretName: "test", - SharedStore: lokiv1.ObjectStorageSecretS3, - S3: &S3StorageConfig{ - SSE: S3SSEConfig{ - Type: SSEKMSType, - KMSEncryptionContext: "test", - }, - }, + SharedStore: lokiv1.ObjectStorageSecretGCS, }, dpl: &appsv1.Deployment{ Spec: appsv1.DeploymentSpec{ @@ -659,37 +595,8 @@ func TestConfigureDeploymentForStorageType(t *testing.T) { }, Env: []corev1.EnvVar{ { - Name: EnvAWSAccessKeyID, - ValueFrom: &corev1.EnvVarSource{ - SecretKeyRef: &corev1.SecretKeySelector{ - LocalObjectReference: corev1.LocalObjectReference{ - Name: "test", - }, - Key: KeyAWSAccessKeyID, - }, - }, - }, - { - Name: EnvAWSAccessKeySecret, - ValueFrom: &corev1.EnvVarSource{ - SecretKeyRef: &corev1.SecretKeySelector{ - LocalObjectReference: corev1.LocalObjectReference{ - Name: "test", - }, - Key: KeyAWSAccessKeySecret, - }, - }, - }, - { - Name: EnvAWSSseKmsEncryptionContext, - ValueFrom: &corev1.EnvVarSource{ - SecretKeyRef: &corev1.SecretKeySelector{ - LocalObjectReference: corev1.LocalObjectReference{ - Name: "test", - }, - Key: KeyAWSSseKmsEncryptionContext, - }, - }, + Name: EnvGoogleApplicationCredentials, + Value: "/etc/storage/secrets/key.json", }, }, }, @@ -710,10 +617,10 @@ func TestConfigureDeploymentForStorageType(t *testing.T) { }, }, { - desc: "object storage Swift", + desc: "object storage S3", opts: Options{ SecretName: "test", - SharedStore: lokiv1.ObjectStorageSecretSwift, + SharedStore: lokiv1.ObjectStorageSecretS3, }, dpl: &appsv1.Deployment{ Spec: appsv1.DeploymentSpec{ @@ -744,24 +651,385 @@ func TestConfigureDeploymentForStorageType(t *testing.T) { }, Env: []corev1.EnvVar{ { - Name: EnvSwiftUsername, + Name: EnvAWSAccessKeyID, ValueFrom: &corev1.EnvVarSource{ SecretKeyRef: &corev1.SecretKeySelector{ LocalObjectReference: corev1.LocalObjectReference{ Name: "test", }, - Key: KeySwiftUsername, + Key: KeyAWSAccessKeyID, }, }, }, { - Name: EnvSwiftPassword, + Name: EnvAWSAccessKeySecret, ValueFrom: &corev1.EnvVarSource{ SecretKeyRef: &corev1.SecretKeySelector{ LocalObjectReference: corev1.LocalObjectReference{ Name: "test", }, - Key: KeySwiftPassword, + Key: KeyAWSAccessKeySecret, + }, + }, + }, + }, + }, + }, + Volumes: []corev1.Volume{ + { + Name: "test", + VolumeSource: corev1.VolumeSource{ + Secret: &corev1.SecretVolumeSource{ + SecretName: "test", + }, + }, + }, + }, + }, + }, + }, + }, + }, + { + desc: "object storage S3 in STS Mode", + opts: Options{ + SecretName: "test", + SharedStore: lokiv1.ObjectStorageSecretS3, + S3: &S3StorageConfig{ + STS: true, + Audience: "test", + }, + }, + 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/aws/serviceaccount", + }, + }, + Env: []corev1.EnvVar{ + { + Name: EnvAWSRoleArn, + ValueFrom: &corev1.EnvVarSource{ + SecretKeyRef: &corev1.SecretKeySelector{ + LocalObjectReference: corev1.LocalObjectReference{ + Name: "test", + }, + Key: KeyAWSRoleArn, + }, + }, + }, + { + Name: "AWS_WEB_IDENTITY_TOKEN_FILE", + Value: "/var/run/secrets/aws/serviceaccount/token", + }, + }, + }, + }, + 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 in STS Mode in OpenShift", + opts: Options{ + SecretName: "test", + SharedStore: lokiv1.ObjectStorageSecretS3, + S3: &S3StorageConfig{ + STS: true, + }, + OpenShift: OpenShiftOptions{ + Enabled: true, + CloudCredentials: CloudCredentials{ + SecretName: "cloud-credentials", + SHA1: "deadbeef", + }, + }, + }, + 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/aws/serviceaccount", + }, + { + Name: "cloud-credentials", + ReadOnly: false, + MountPath: "/etc/storage/managed-auth", + }, + }, + Env: []corev1.EnvVar{ + { + Name: "AWS_SHARED_CREDENTIALS_FILE", + Value: "/etc/storage/managed-auth/credentials", + }, + { + Name: "AWS_SDK_LOAD_CONFIG", + Value: "true", + }, + }, + }, + }, + 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: awsDefaultAudience, + ExpirationSeconds: ptr.To[int64](3600), + Path: corev1.ServiceAccountTokenKey, + }, + }, + }, + }, + }, + }, + { + Name: "cloud-credentials", + VolumeSource: corev1.VolumeSource{ + Secret: &corev1.SecretVolumeSource{ + SecretName: "cloud-credentials", + }, + }, + }, + }, + }, + }, + }, + }, + }, + { + desc: "object storage S3 with SSE KMS encryption context", + opts: Options{ + SecretName: "test", + SharedStore: lokiv1.ObjectStorageSecretS3, + S3: &S3StorageConfig{ + SSE: S3SSEConfig{ + Type: SSEKMSType, + KMSEncryptionContext: "test", + }, + }, + }, + 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", + }, + }, + Env: []corev1.EnvVar{ + { + Name: EnvAWSAccessKeyID, + ValueFrom: &corev1.EnvVarSource{ + SecretKeyRef: &corev1.SecretKeySelector{ + LocalObjectReference: corev1.LocalObjectReference{ + Name: "test", + }, + Key: KeyAWSAccessKeyID, + }, + }, + }, + { + Name: EnvAWSAccessKeySecret, + ValueFrom: &corev1.EnvVarSource{ + SecretKeyRef: &corev1.SecretKeySelector{ + LocalObjectReference: corev1.LocalObjectReference{ + Name: "test", + }, + Key: KeyAWSAccessKeySecret, + }, + }, + }, + { + Name: EnvAWSSseKmsEncryptionContext, + ValueFrom: &corev1.EnvVarSource{ + SecretKeyRef: &corev1.SecretKeySelector{ + LocalObjectReference: corev1.LocalObjectReference{ + Name: "test", + }, + Key: KeyAWSSseKmsEncryptionContext, + }, + }, + }, + }, + }, + }, + Volumes: []corev1.Volume{ + { + Name: "test", + VolumeSource: corev1.VolumeSource{ + Secret: &corev1.SecretVolumeSource{ + SecretName: "test", + }, + }, + }, + }, + }, + }, + }, + }, + }, + { + desc: "object storage Swift", + opts: Options{ + SecretName: "test", + SharedStore: lokiv1.ObjectStorageSecretSwift, + }, + 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", + }, + }, + Env: []corev1.EnvVar{ + { + Name: EnvSwiftUsername, + ValueFrom: &corev1.EnvVarSource{ + SecretKeyRef: &corev1.SecretKeySelector{ + LocalObjectReference: corev1.LocalObjectReference{ + Name: "test", + }, + Key: KeySwiftUsername, + }, + }, + }, + { + Name: EnvSwiftPassword, + ValueFrom: &corev1.EnvVarSource{ + SecretKeyRef: &corev1.SecretKeySelector{ + LocalObjectReference: corev1.LocalObjectReference{ + Name: "test", + }, + Key: KeySwiftPassword, }, }, }, @@ -806,10 +1074,161 @@ func TestConfigureStatefulSetForStorageType(t *testing.T) { tc := []tt{ { - desc: "object storage AlibabaCloud", + desc: "object storage AlibabaCloud", + opts: Options{ + SecretName: "test", + SharedStore: lokiv1.ObjectStorageSecretAlibabaCloud, + }, + 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", + }, + }, + Env: []corev1.EnvVar{ + { + Name: EnvAlibabaCloudAccessKeyID, + ValueFrom: &corev1.EnvVarSource{ + SecretKeyRef: &corev1.SecretKeySelector{ + LocalObjectReference: corev1.LocalObjectReference{ + Name: "test", + }, + Key: KeyAlibabaCloudAccessKeyID, + }, + }, + }, + { + Name: EnvAlibabaCloudAccessKeySecret, + ValueFrom: &corev1.EnvVarSource{ + SecretKeyRef: &corev1.SecretKeySelector{ + LocalObjectReference: corev1.LocalObjectReference{ + Name: "test", + }, + Key: KeyAlibabaCloudSecretAccessKey, + }, + }, + }, + }, + }, + }, + Volumes: []corev1.Volume{ + { + Name: "test", + VolumeSource: corev1.VolumeSource{ + Secret: &corev1.SecretVolumeSource{ + SecretName: "test", + }, + }, + }, + }, + }, + }, + }, + }, + }, + { + desc: "object storage Azure", + opts: Options{ + SecretName: "test", + SharedStore: lokiv1.ObjectStorageSecretAzure, + }, + 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", + }, + }, + Env: []corev1.EnvVar{ + { + Name: EnvAzureStorageAccountName, + ValueFrom: &corev1.EnvVarSource{ + SecretKeyRef: &corev1.SecretKeySelector{ + LocalObjectReference: corev1.LocalObjectReference{ + Name: "test", + }, + Key: KeyAzureStorageAccountName, + }, + }, + }, + { + Name: EnvAzureStorageAccountKey, + ValueFrom: &corev1.EnvVarSource{ + SecretKeyRef: &corev1.SecretKeySelector{ + LocalObjectReference: corev1.LocalObjectReference{ + Name: "test", + }, + Key: KeyAzureStorageAccountKey, + }, + }, + }, + }, + }, + }, + Volumes: []corev1.Volume{ + { + Name: "test", + VolumeSource: corev1.VolumeSource{ + Secret: &corev1.SecretVolumeSource{ + SecretName: "test", + }, + }, + }, + }, + }, + }, + }, + }, + }, + { + desc: "object storage Azure with WIF", opts: Options{ SecretName: "test", - SharedStore: lokiv1.ObjectStorageSecretAlibabaCloud, + SharedStore: lokiv1.ObjectStorageSecretAzure, + Azure: &AzureStorageConfig{ + WorkloadIdentity: true, + }, }, sts: &appsv1.StatefulSet{ Spec: appsv1.StatefulSetSpec{ @@ -837,30 +1256,61 @@ func TestConfigureStatefulSetForStorageType(t *testing.T) { ReadOnly: false, MountPath: "/etc/storage/secrets", }, + { + Name: saTokenVolumeName, + ReadOnly: false, + MountPath: "/var/run/secrets/azure/serviceaccount", + }, }, Env: []corev1.EnvVar{ { - Name: EnvAlibabaCloudAccessKeyID, + Name: EnvAzureStorageAccountName, ValueFrom: &corev1.EnvVarSource{ SecretKeyRef: &corev1.SecretKeySelector{ LocalObjectReference: corev1.LocalObjectReference{ Name: "test", }, - Key: KeyAlibabaCloudAccessKeyID, + Key: KeyAzureStorageAccountName, }, }, }, { - Name: EnvAlibabaCloudAccessKeySecret, + Name: EnvAzureClientID, ValueFrom: &corev1.EnvVarSource{ SecretKeyRef: &corev1.SecretKeySelector{ LocalObjectReference: corev1.LocalObjectReference{ Name: "test", }, - Key: KeyAlibabaCloudSecretAccessKey, + Key: KeyAzureStorageClientID, + }, + }, + }, + { + Name: EnvAzureTenantID, + ValueFrom: &corev1.EnvVarSource{ + SecretKeyRef: &corev1.SecretKeySelector{ + LocalObjectReference: corev1.LocalObjectReference{ + Name: "test", + }, + Key: KeyAzureStorageTenantID, + }, + }, + }, + { + Name: EnvAzureSubscriptionID, + ValueFrom: &corev1.EnvVarSource{ + SecretKeyRef: &corev1.SecretKeySelector{ + LocalObjectReference: corev1.LocalObjectReference{ + Name: "test", + }, + Key: KeyAzureStorageSubscriptionID, }, }, }, + { + Name: EnvAzureFederatedTokenFile, + Value: "/var/run/secrets/azure/serviceaccount/token", + }, }, }, }, @@ -873,6 +1323,22 @@ func TestConfigureStatefulSetForStorageType(t *testing.T) { }, }, }, + { + Name: saTokenVolumeName, + VolumeSource: corev1.VolumeSource{ + Projected: &corev1.ProjectedVolumeSource{ + Sources: []corev1.VolumeProjection{ + { + ServiceAccountToken: &corev1.ServiceAccountTokenProjection{ + Audience: azureDefaultAudience, + ExpirationSeconds: ptr.To[int64](3600), + Path: corev1.ServiceAccountTokenKey, + }, + }, + }, + }, + }, + }, }, }, }, @@ -880,10 +1346,14 @@ func TestConfigureStatefulSetForStorageType(t *testing.T) { }, }, { - desc: "object storage Azure", + desc: "object storage Azure with WIF and custom audience", opts: Options{ SecretName: "test", SharedStore: lokiv1.ObjectStorageSecretAzure, + Azure: &AzureStorageConfig{ + WorkloadIdentity: true, + Audience: "custom-audience", + }, }, sts: &appsv1.StatefulSet{ Spec: appsv1.StatefulSetSpec{ @@ -911,6 +1381,11 @@ func TestConfigureStatefulSetForStorageType(t *testing.T) { ReadOnly: false, MountPath: "/etc/storage/secrets", }, + { + Name: saTokenVolumeName, + ReadOnly: false, + MountPath: "/var/run/secrets/azure/serviceaccount", + }, }, Env: []corev1.EnvVar{ { @@ -925,16 +1400,177 @@ func TestConfigureStatefulSetForStorageType(t *testing.T) { }, }, { - Name: EnvAzureStorageAccountKey, + Name: EnvAzureClientID, ValueFrom: &corev1.EnvVarSource{ SecretKeyRef: &corev1.SecretKeySelector{ LocalObjectReference: corev1.LocalObjectReference{ Name: "test", }, - Key: KeyAzureStorageAccountKey, + Key: KeyAzureStorageClientID, + }, + }, + }, + { + Name: EnvAzureTenantID, + ValueFrom: &corev1.EnvVarSource{ + SecretKeyRef: &corev1.SecretKeySelector{ + LocalObjectReference: corev1.LocalObjectReference{ + Name: "test", + }, + Key: KeyAzureStorageTenantID, + }, + }, + }, + { + Name: EnvAzureSubscriptionID, + ValueFrom: &corev1.EnvVarSource{ + SecretKeyRef: &corev1.SecretKeySelector{ + LocalObjectReference: corev1.LocalObjectReference{ + Name: "test", + }, + Key: KeyAzureStorageSubscriptionID, + }, + }, + }, + { + Name: EnvAzureFederatedTokenFile, + Value: "/var/run/secrets/azure/serviceaccount/token", + }, + }, + }, + }, + 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: "custom-audience", + ExpirationSeconds: ptr.To[int64](3600), + Path: corev1.ServiceAccountTokenKey, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + }, + { + desc: "object storage Azure with WIF and OpenShift Managed Credentials", + opts: Options{ + SecretName: "test", + SharedStore: lokiv1.ObjectStorageSecretAzure, + Azure: &AzureStorageConfig{ + WorkloadIdentity: true, + }, + OpenShift: OpenShiftOptions{ + Enabled: true, + CloudCredentials: CloudCredentials{ + SecretName: "cloud-credentials", + SHA1: "deadbeef", + }, + }, + }, + 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/azure/serviceaccount", + }, + { + Name: "cloud-credentials", + MountPath: managedAuthSecretDirectory, + }, + }, + Env: []corev1.EnvVar{ + { + Name: EnvAzureStorageAccountName, + ValueFrom: &corev1.EnvVarSource{ + SecretKeyRef: &corev1.SecretKeySelector{ + LocalObjectReference: corev1.LocalObjectReference{ + Name: "test", + }, + Key: KeyAzureStorageAccountName, + }, + }, + }, + { + Name: EnvAzureClientID, + ValueFrom: &corev1.EnvVarSource{ + SecretKeyRef: &corev1.SecretKeySelector{ + LocalObjectReference: corev1.LocalObjectReference{ + Name: "cloud-credentials", + }, + Key: azureManagedCredentialKeyClientID, + }, + }, + }, + { + Name: EnvAzureTenantID, + ValueFrom: &corev1.EnvVarSource{ + SecretKeyRef: &corev1.SecretKeySelector{ + LocalObjectReference: corev1.LocalObjectReference{ + Name: "cloud-credentials", + }, + Key: azureManagedCredentialKeyTenantID, + }, + }, + }, + { + Name: EnvAzureSubscriptionID, + ValueFrom: &corev1.EnvVarSource{ + SecretKeyRef: &corev1.SecretKeySelector{ + LocalObjectReference: corev1.LocalObjectReference{ + Name: "cloud-credentials", + }, + Key: azureManagedCredentialKeySubscriptionID, }, }, }, + { + Name: EnvAzureFederatedTokenFile, + Value: "/var/run/secrets/azure/serviceaccount/token", + }, }, }, }, @@ -947,6 +1583,30 @@ func TestConfigureStatefulSetForStorageType(t *testing.T) { }, }, }, + { + Name: saTokenVolumeName, + VolumeSource: corev1.VolumeSource{ + Projected: &corev1.ProjectedVolumeSource{ + Sources: []corev1.VolumeProjection{ + { + ServiceAccountToken: &corev1.ServiceAccountTokenProjection{ + Audience: azureDefaultAudience, + ExpirationSeconds: ptr.To[int64](3600), + Path: corev1.ServiceAccountTokenKey, + }, + }, + }, + }, + }, + }, + { + Name: "cloud-credentials", + 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 86aa494318519..4c62ce7513755 100644 --- a/operator/internal/manifests/storage/options.go +++ b/operator/internal/manifests/storage/options.go @@ -28,6 +28,7 @@ type AzureStorageConfig struct { Env string Container string EndpointSuffix string + Audience string WorkloadIdentity bool } diff --git a/operator/internal/manifests/storage/var.go b/operator/internal/manifests/storage/var.go index 6184cff0463da..418fb27152bd3 100644 --- a/operator/internal/manifests/storage/var.go +++ b/operator/internal/manifests/storage/var.go @@ -86,6 +86,10 @@ const ( KeyAzureStorageEndpointSuffix = "endpoint_suffix" // KeyAzureEnvironmentName is the secret data key for the Azure cloud environment name. KeyAzureEnvironmentName = "environment" + // KeyAzureRegion is the secret data key for storing the Azure cloud region. + KeyAzureRegion = "region" + // KeyAzureAudience is the secret data key for customizing the audience used for the ServiceAccount token. + KeyAzureAudience = "audience" // KeyGCPStorageBucketName is the secret data key for the GCS bucket name. KeyGCPStorageBucketName = "bucketname" @@ -136,5 +140,9 @@ const ( azureDefaultAudience = "api://AzureADTokenExchange" azureTokenVolumeDirectory = "/var/run/secrets/azure/serviceaccount" + azureManagedCredentialKeyClientID = "azure_client_id" + azureManagedCredentialKeyTenantID = "azure_tenant_id" + azureManagedCredentialKeySubscriptionID = "azure_subscription_id" + AnnotationCredentialsRequestsSecretRef = "loki.grafana.com/credentials-request-secret-ref" ) From af66ecec6a38e88dfb0de09508ecdeddd25a23fe Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Tue, 6 Feb 2024 10:27:03 -0800 Subject: [PATCH 39/45] [Blooms] bloomshipper meta integration (#11874) Builds on top of https://github.com/grafana/loki/pull/11872, standardizing on the `bloomshipper` types `Meta` and `MetaRef`. --- pkg/bloomcompactor/controller.go | 84 ++++++++++------ pkg/bloomcompactor/controller_test.go | 36 ++++--- pkg/bloomcompactor/meta.go | 98 ------------------- pkg/bloomcompactor/spec.go | 43 ++++++-- pkg/bloomcompactor/spec_test.go | 10 +- pkg/storage/bloom/v1/block.go | 4 + pkg/storage/bloom/v1/util.go | 18 ++++ .../stores/shipper/bloomshipper/cache.go | 11 ++- .../stores/shipper/bloomshipper/client.go | 59 ++++++++++- 9 files changed, 208 insertions(+), 155 deletions(-) diff --git a/pkg/bloomcompactor/controller.go b/pkg/bloomcompactor/controller.go index c9b3ecae35c2c..f6487d5f61d11 100644 --- a/pkg/bloomcompactor/controller.go +++ b/pkg/bloomcompactor/controller.go @@ -14,11 +14,19 @@ 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 - metaStore MetaStore - blockStore BlockStore + bloomStore bloomshipper.Store + uploader uploader chunkLoader ChunkLoader rwFn func() (v1.BlockWriter, v1.BlockReader) metrics *Metrics @@ -28,20 +36,23 @@ type SimpleBloomController struct { } func NewSimpleBloomController( + tenant, table string, ownershipRange v1.FingerprintBounds, tsdbStore TSDBStore, - metaStore MetaStore, - blockStore BlockStore, + 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, - metaStore: metaStore, - blockStore: blockStore, + bloomStore: blockStore, + uploader: uploader, chunkLoader: chunkLoader, rwFn: rwFn, metrics: metrics, @@ -57,18 +68,8 @@ func (s *SimpleBloomController) do(ctx context.Context) error { return errors.Wrap(err, "failed to resolve tsdbs") } - // 2. Resolve Metas - metaRefs, err := s.metaStore.ResolveMetas(s.ownershipRange) - if err != nil { - level.Error(s.logger).Log("msg", "failed to resolve metas", "err", err) - return errors.Wrap(err, "failed to resolve metas") - } - - // 3. Fetch metas - metas, err := s.metaStore.GetMetas(metaRefs) - if err != nil { - level.Error(s.logger).Log("msg", "failed to get metas", "err", err) - return errors.Wrap(err, "failed to get metas") + if len(tsdbs) == 0 { + return nil } ids := make([]tsdb.Identifier, 0, len(tsdbs)) @@ -76,7 +77,21 @@ func (s *SimpleBloomController) do(ctx context.Context) error { ids = append(ids, id) } - // 4. Determine which TSDBs have gaps in the ownership range and need to + // 2. Fetch metas + metas, err := s.bloomStore.FetchMetas( + ctx, + bloomshipper.MetaSearchParams{ + TenantID: s.tenant, + Interval: bloomshipper.Interval{}, // TODO(owen-d): gen interval + Keyspace: s.ownershipRange, + }, + ) + if err != nil { + level.Error(s.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) if err != nil { @@ -95,7 +110,7 @@ func (s *SimpleBloomController) do(ctx context.Context) error { return errors.Wrap(err, "failed to create plan") } - // 5. Generate Blooms + // 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 // needed chunks in their blooms, for instance after a new TSDB version is generated @@ -115,7 +130,7 @@ 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(plan.tsdb, gap) + seriesItr, preExistingBlocks, err := s.loadWorkForGap(ctx, plan.tsdb, gap) if err != nil { level.Error(s.logger).Log("msg", "failed to get series and blocks", "err", err) return errors.Wrap(err, "failed to get series and blocks") @@ -142,7 +157,11 @@ func (s *SimpleBloomController) do(ctx context.Context) error { for newBlocks.Next() { blockCt++ blk := newBlocks.At() - if err := s.blockStore.PutBlock(blk); err != nil { + + if err := s.uploader.PutBlock( + ctx, + bloomshipper.BlockFrom(s.tenant, s.table, blk), + ); err != nil { level.Error(s.logger).Log("msg", "failed to write block", "err", err) return errors.Wrap(err, "failed to write block") } @@ -157,24 +176,31 @@ 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) return nil } -func (s *SimpleBloomController) loadWorkForGap(id tsdb.Identifier, gap gapWithBlocks) (v1.CloseableIterator[*v1.Series], []*v1.Block, error) { +func (s *SimpleBloomController) loadWorkForGap(ctx context.Context, id tsdb.Identifier, gap gapWithBlocks) (v1.CloseableIterator[*v1.Series], []*bloomshipper.ClosableBlockQuerier, error) { // load a series iterator for the gap seriesItr, err := s.tsdbStore.LoadTSDB(id, gap.bounds) if err != nil { return nil, nil, errors.Wrap(err, "failed to load tsdb") } - blocks, err := s.blockStore.GetBlocks(gap.blocks) + blocks, err := s.bloomStore.FetchBlocks(ctx, gap.blocks) if err != nil { return nil, nil, errors.Wrap(err, "failed to get blocks") } + results := make([]*bloomshipper.ClosableBlockQuerier, 0, len(blocks)) + for _, block := range blocks { + results = append(results, block.BlockQuerier()) + } - return seriesItr, blocks, nil + return seriesItr, results, nil } type gapWithBlocks struct { @@ -199,7 +225,7 @@ type blockPlan struct { // blockPlansForGaps groups tsdb gaps we wish to fill with overlapping but out of date blocks. // This allows us to expedite bloom generation by using existing blocks to fill in the gaps // since many will contain the same chunks. -func blockPlansForGaps(tsdbs []tsdbGaps, metas []Meta) ([]blockPlan, error) { +func blockPlansForGaps(tsdbs []tsdbGaps, metas []bloomshipper.Meta) ([]blockPlan, error) { plans := make([]blockPlan, 0, len(tsdbs)) for _, idx := range tsdbs { @@ -215,7 +241,7 @@ func blockPlansForGaps(tsdbs []tsdbGaps, metas []Meta) ([]blockPlan, error) { for _, meta := range metas { - if meta.OwnershipRange.Intersection(gap) == nil { + if meta.Bounds.Intersection(gap) == nil { // this meta doesn't overlap the gap, skip continue } @@ -279,7 +305,7 @@ type tsdbGaps struct { func gapsBetweenTSDBsAndMetas( ownershipRange v1.FingerprintBounds, tsdbs []tsdb.Identifier, - metas []Meta, + metas []bloomshipper.Meta, ) (res []tsdbGaps, err error) { for _, db := range tsdbs { id := db.Name() @@ -288,7 +314,7 @@ func gapsBetweenTSDBsAndMetas( for _, meta := range metas { for _, s := range meta.Sources { if s.Name() == id { - relevantMetas = append(relevantMetas, meta.OwnershipRange) + relevantMetas = append(relevantMetas, meta.Bounds) } } } diff --git a/pkg/bloomcompactor/controller_test.go b/pkg/bloomcompactor/controller_test.go index 1f89a0e318efd..0660a5b601eea 100644 --- a/pkg/bloomcompactor/controller_test.go +++ b/pkg/bloomcompactor/controller_test.go @@ -120,10 +120,14 @@ func tsdbID(n int) tsdb.SingleTenantTSDBIdentifier { } } -func genMeta(min, max model.Fingerprint, sources []int, blocks []bloomshipper.BlockRef) Meta { - m := Meta{ - OwnershipRange: v1.NewBounds(min, max), - Blocks: blocks, +func genMeta(min, max model.Fingerprint, sources []int, blocks []bloomshipper.BlockRef) bloomshipper.Meta { + m := bloomshipper.Meta{ + MetaRef: bloomshipper.MetaRef{ + Ref: bloomshipper.Ref{ + Bounds: v1.NewBounds(min, max), + }, + }, + Blocks: blocks, } for _, source := range sources { m.Sources = append(m.Sources, tsdbID(source)) @@ -139,14 +143,14 @@ func Test_gapsBetweenTSDBsAndMetas(t *testing.T) { exp []tsdbGaps ownershipRange v1.FingerprintBounds tsdbs []tsdb.Identifier - metas []Meta + metas []bloomshipper.Meta }{ { desc: "non-overlapping tsdbs and metas", err: true, ownershipRange: v1.NewBounds(0, 10), tsdbs: []tsdb.Identifier{tsdbID(0)}, - metas: []Meta{ + metas: []bloomshipper.Meta{ genMeta(11, 20, []int{0}, nil), }, }, @@ -154,7 +158,7 @@ func Test_gapsBetweenTSDBsAndMetas(t *testing.T) { desc: "single tsdb", ownershipRange: v1.NewBounds(0, 10), tsdbs: []tsdb.Identifier{tsdbID(0)}, - metas: []Meta{ + metas: []bloomshipper.Meta{ genMeta(4, 8, []int{0}, nil), }, exp: []tsdbGaps{ @@ -171,7 +175,7 @@ func Test_gapsBetweenTSDBsAndMetas(t *testing.T) { desc: "multiple tsdbs with separate blocks", ownershipRange: v1.NewBounds(0, 10), tsdbs: []tsdb.Identifier{tsdbID(0), tsdbID(1)}, - metas: []Meta{ + metas: []bloomshipper.Meta{ genMeta(0, 5, []int{0}, nil), genMeta(6, 10, []int{1}, nil), }, @@ -194,7 +198,7 @@ func Test_gapsBetweenTSDBsAndMetas(t *testing.T) { desc: "multiple tsdbs with the same blocks", ownershipRange: v1.NewBounds(0, 10), tsdbs: []tsdb.Identifier{tsdbID(0), tsdbID(1)}, - metas: []Meta{ + metas: []bloomshipper.Meta{ genMeta(0, 5, []int{0, 1}, nil), genMeta(6, 8, []int{1}, nil), }, @@ -239,7 +243,7 @@ func Test_blockPlansForGaps(t *testing.T) { desc string ownershipRange v1.FingerprintBounds tsdbs []tsdb.Identifier - metas []Meta + metas []bloomshipper.Meta err bool exp []blockPlan }{ @@ -247,7 +251,7 @@ func Test_blockPlansForGaps(t *testing.T) { desc: "single overlapping meta+no overlapping block", ownershipRange: v1.NewBounds(0, 10), tsdbs: []tsdb.Identifier{tsdbID(0)}, - metas: []Meta{ + metas: []bloomshipper.Meta{ genMeta(5, 20, []int{1}, []bloomshipper.BlockRef{genBlockRef(11, 20)}), }, exp: []blockPlan{ @@ -265,7 +269,7 @@ func Test_blockPlansForGaps(t *testing.T) { desc: "single overlapping meta+one overlapping block", ownershipRange: v1.NewBounds(0, 10), tsdbs: []tsdb.Identifier{tsdbID(0)}, - metas: []Meta{ + metas: []bloomshipper.Meta{ genMeta(5, 20, []int{1}, []bloomshipper.BlockRef{genBlockRef(9, 20)}), }, exp: []blockPlan{ @@ -287,7 +291,7 @@ func Test_blockPlansForGaps(t *testing.T) { desc: "trims up to date area", ownershipRange: v1.NewBounds(0, 10), tsdbs: []tsdb.Identifier{tsdbID(0)}, - metas: []Meta{ + metas: []bloomshipper.Meta{ genMeta(9, 20, []int{0}, []bloomshipper.BlockRef{genBlockRef(9, 20)}), // block for same tsdb genMeta(9, 20, []int{1}, []bloomshipper.BlockRef{genBlockRef(9, 20)}), // block for different tsdb }, @@ -306,7 +310,7 @@ func Test_blockPlansForGaps(t *testing.T) { desc: "uses old block for overlapping range", ownershipRange: v1.NewBounds(0, 10), tsdbs: []tsdb.Identifier{tsdbID(0)}, - metas: []Meta{ + metas: []bloomshipper.Meta{ genMeta(9, 20, []int{0}, []bloomshipper.BlockRef{genBlockRef(9, 20)}), // block for same tsdb genMeta(5, 20, []int{1}, []bloomshipper.BlockRef{genBlockRef(5, 20)}), // block for different tsdb }, @@ -326,7 +330,7 @@ func Test_blockPlansForGaps(t *testing.T) { desc: "multi case", ownershipRange: v1.NewBounds(0, 10), tsdbs: []tsdb.Identifier{tsdbID(0), tsdbID(1)}, // generate for both tsdbs - metas: []Meta{ + metas: []bloomshipper.Meta{ genMeta(0, 2, []int{0}, []bloomshipper.BlockRef{ genBlockRef(0, 1), genBlockRef(1, 2), @@ -374,7 +378,7 @@ func Test_blockPlansForGaps(t *testing.T) { desc: "dedupes block refs", ownershipRange: v1.NewBounds(0, 10), tsdbs: []tsdb.Identifier{tsdbID(0)}, - metas: []Meta{ + metas: []bloomshipper.Meta{ genMeta(9, 20, []int{1}, []bloomshipper.BlockRef{ genBlockRef(1, 4), genBlockRef(9, 20), diff --git a/pkg/bloomcompactor/meta.go b/pkg/bloomcompactor/meta.go index c0a333c5c907e..2f2c2cd9de16e 100644 --- a/pkg/bloomcompactor/meta.go +++ b/pkg/bloomcompactor/meta.go @@ -1,13 +1,7 @@ package bloomcompactor import ( - "fmt" - "path" - - "github.com/pkg/errors" - v1 "github.com/grafana/loki/pkg/storage/bloom/v1" - "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper" "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb" ) @@ -16,99 +10,7 @@ const ( MetasPrefix = "metas" ) -type MetaRef struct { - OwnershipRange v1.FingerprintBounds - Checksum uint32 -} - -// `bloom///metas/--.json` -func (m MetaRef) Address(tenant string, period int) (string, error) { - joined := path.Join( - BloomPrefix, - fmt.Sprintf("%v", period), - tenant, - MetasPrefix, - fmt.Sprintf("%v-%v", m.OwnershipRange, m.Checksum), - ) - - return fmt.Sprintf("%s.json", joined), nil -} - -type Meta struct { - - // The fingerprint range of the block. This is the range _owned_ by the meta and - // is greater than or equal to the range of the actual data in the underlying blocks. - OwnershipRange v1.FingerprintBounds - - // Old blocks which can be deleted in the future. These should be from previous compaction rounds. - Tombstones []bloomshipper.BlockRef - - // The specific TSDB files used to generate the block. - Sources []tsdb.SingleTenantTSDBIdentifier - - // A list of blocks that were generated - Blocks []bloomshipper.BlockRef -} - -// Generate MetaRef from Meta -func (m Meta) Ref() (MetaRef, error) { - checksum, err := m.Checksum() - if err != nil { - return MetaRef{}, errors.Wrap(err, "getting checksum") - } - return MetaRef{ - OwnershipRange: m.OwnershipRange, - Checksum: checksum, - }, nil -} - -func (m Meta) Checksum() (uint32, error) { - h := v1.Crc32HashPool.Get() - defer v1.Crc32HashPool.Put(h) - - _, err := h.Write([]byte(m.OwnershipRange.String())) - if err != nil { - return 0, errors.Wrap(err, "writing OwnershipRange") - } - - for _, tombstone := range m.Tombstones { - err = tombstone.Hash(h) - if err != nil { - return 0, errors.Wrap(err, "writing Tombstones") - } - } - - for _, source := range m.Sources { - err = source.Hash(h) - if err != nil { - return 0, errors.Wrap(err, "writing Sources") - } - } - - for _, block := range m.Blocks { - err = block.Hash(h) - if err != nil { - return 0, errors.Wrap(err, "writing Blocks") - } - } - - return h.Sum32(), nil - -} - type TSDBStore interface { ResolveTSDBs() ([]*tsdb.SingleTenantTSDBIdentifier, error) LoadTSDB(id tsdb.Identifier, bounds v1.FingerprintBounds) (v1.CloseableIterator[*v1.Series], error) } - -type MetaStore interface { - ResolveMetas(bounds v1.FingerprintBounds) ([]MetaRef, error) - GetMetas([]MetaRef) ([]Meta, error) - PutMeta(Meta) error -} - -type BlockStore interface { - // TODO(owen-d): flesh out|integrate against bloomshipper.Client - GetBlocks([]bloomshipper.BlockRef) ([]*v1.Block, error) - PutBlock(interface{}) error -} diff --git a/pkg/bloomcompactor/spec.go b/pkg/bloomcompactor/spec.go index e0d964e9e9724..341e3977502c3 100644 --- a/pkg/bloomcompactor/spec.go +++ b/pkg/bloomcompactor/spec.go @@ -13,11 +13,14 @@ import ( "github.com/prometheus/client_golang/prometheus/promauto" "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" v1 "github.com/grafana/loki/pkg/storage/bloom/v1" "github.com/grafana/loki/pkg/storage/chunk" + "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper" "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb" ) @@ -71,7 +74,7 @@ type SimpleBloomGenerator struct { 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 []*v1.Block + blocks []*bloomshipper.ClosableBlockQuerier // options to build blocks with opts v1.BlockOptions @@ -92,7 +95,7 @@ func NewSimpleBloomGenerator( opts v1.BlockOptions, store v1.Iterator[*v1.Series], chunkLoader ChunkLoader, - blocks []*v1.Block, + blocks []*bloomshipper.ClosableBlockQuerier, readWriterFn func() (v1.BlockWriter, v1.BlockReader), metrics *Metrics, logger log.Logger, @@ -129,38 +132,62 @@ func (s *SimpleBloomGenerator) populator(ctx context.Context) func(series *v1.Se } -func (s *SimpleBloomGenerator) Generate(ctx context.Context) (skippedBlocks []*v1.Block, results v1.Iterator[*v1.Block], err error) { +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.ClosableBlockQuerier, 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) + } + }() + 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)) - schema, err := block.Schema() + md, err := block.Metadata() + schema := md.Options.Schema if err != nil { level.Warn(logger).Log("msg", "failed to get schema for block", "err", err) - skippedBlocks = append(skippedBlocks, block) + 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, block) + 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](v1.NewBlockQuerier(block)) + itr := v1.NewPeekingIter[*v1.SeriesWithBloom](block) blocksMatchingSchema = append(blocksMatchingSchema, itr) + // append needed block to close list (when finished) + toClose = append(toClose, 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() + 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") } + _, err = mergeBuilder.Build(blockBuilder) if err != nil { return skippedBlocks, nil, errors.Wrap(err, "failed to build bloom block") diff --git a/pkg/bloomcompactor/spec_test.go b/pkg/bloomcompactor/spec_test.go index 08c722d06e5d4..efc0d70f2020a 100644 --- a/pkg/bloomcompactor/spec_test.go +++ b/pkg/bloomcompactor/spec_test.go @@ -10,6 +10,7 @@ import ( "github.com/stretchr/testify/require" v1 "github.com/grafana/loki/pkg/storage/bloom/v1" + "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper" ) func blocksFromSchema(t *testing.T, n int, options v1.BlockOptions) (res []*v1.Block, data []v1.SeriesWithBloom) { @@ -63,11 +64,18 @@ func (dummyChunkLoader) Load(_ context.Context, series *v1.Series) (*ChunkItersB } func dummyBloomGen(opts v1.BlockOptions, store v1.Iterator[*v1.Series], blocks []*v1.Block) *SimpleBloomGenerator { + bqs := make([]*bloomshipper.ClosableBlockQuerier, 0, len(blocks)) + for _, b := range blocks { + bqs = append(bqs, &bloomshipper.ClosableBlockQuerier{ + BlockQuerier: v1.NewBlockQuerier(b), + }) + } + return NewSimpleBloomGenerator( opts, store, dummyChunkLoader{}, - blocks, + bqs, func() (v1.BlockWriter, v1.BlockReader) { indexBuf := bytes.NewBuffer(nil) bloomsBuf := bytes.NewBuffer(nil) diff --git a/pkg/storage/bloom/v1/block.go b/pkg/storage/bloom/v1/block.go index c5b115df19e57..0b10b9d5b058e 100644 --- a/pkg/storage/bloom/v1/block.go +++ b/pkg/storage/bloom/v1/block.go @@ -127,6 +127,10 @@ func NewBlockQuerier(b *Block) *BlockQuerier { } } +func (bq *BlockQuerier) Metadata() (BlockMetadata, error) { + return bq.block.Metadata() +} + func (bq *BlockQuerier) Schema() (Schema, error) { return bq.block.Schema() } diff --git a/pkg/storage/bloom/v1/util.go b/pkg/storage/bloom/v1/util.go index 31fcdc643936b..27d347bfb4d0f 100644 --- a/pkg/storage/bloom/v1/util.go +++ b/pkg/storage/bloom/v1/util.go @@ -246,3 +246,21 @@ type CloseableIterator[T any] interface { Iterator[T] Close() error } + +type PeekingCloseableIterator[T any] interface { + PeekingIterator[T] + CloseableIterator[T] +} + +type PeekCloseIter[T any] struct { + *PeekIter[T] + close func() error +} + +func NewPeekCloseIter[T any](itr CloseableIterator[T]) *PeekCloseIter[T] { + return &PeekCloseIter[T]{PeekIter: NewPeekingIter(itr), close: itr.Close} +} + +func (it *PeekCloseIter[T]) Close() error { + return it.close() +} diff --git a/pkg/storage/stores/shipper/bloomshipper/cache.go b/pkg/storage/stores/shipper/bloomshipper/cache.go index 7fe678a1c38fb..2f9e98f89d9c0 100644 --- a/pkg/storage/stores/shipper/bloomshipper/cache.go +++ b/pkg/storage/stores/shipper/bloomshipper/cache.go @@ -19,7 +19,14 @@ import ( type ClosableBlockQuerier struct { *v1.BlockQuerier - Close func() error + close func() error +} + +func (c *ClosableBlockQuerier) Close() error { + if c.close != nil { + return c.close() + } + return nil } func NewBlocksCache(config config.Config, reg prometheus.Registerer, logger log.Logger) *cache.EmbeddedCache[string, BlockDirectory] { @@ -75,7 +82,7 @@ func (b BlockDirectory) BlockQuerier() *ClosableBlockQuerier { b.activeQueriers.Inc() return &ClosableBlockQuerier{ BlockQuerier: v1.NewBlockQuerier(b.Block()), - Close: func() error { + close: func() error { _ = b.activeQueriers.Dec() return nil }, diff --git a/pkg/storage/stores/shipper/bloomshipper/client.go b/pkg/storage/stores/shipper/bloomshipper/client.go index 396e8808c1759..2e31106548d1a 100644 --- a/pkg/storage/stores/shipper/bloomshipper/client.go +++ b/pkg/storage/stores/shipper/bloomshipper/client.go @@ -16,6 +16,7 @@ import ( v1 "github.com/grafana/loki/pkg/storage/bloom/v1" "github.com/grafana/loki/pkg/storage/chunk/client" "github.com/grafana/loki/pkg/storage/config" + "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb" "github.com/grafana/loki/pkg/util/encoding" ) @@ -84,8 +85,49 @@ func (r MetaRef) String() string { type Meta struct { MetaRef `json:"-"` + // The specific TSDB files used to generate the block. + Sources []tsdb.SingleTenantTSDBIdentifier + + // Old blocks which can be deleted in the future. These should be from previous compaction rounds. Tombstones []BlockRef - Blocks []BlockRef + + // A list of blocks that were generated + Blocks []BlockRef +} + +// TODO(owen-d): use this to update internal ref's checksum. +func (m Meta) Checksum() (uint32, error) { + h := v1.Crc32HashPool.Get() + defer v1.Crc32HashPool.Put(h) + + err := m.Bounds.Hash(h) + if err != nil { + return 0, errors.Wrap(err, "writing OwnershipRange") + } + + for _, tombstone := range m.Tombstones { + err = tombstone.Hash(h) + if err != nil { + return 0, errors.Wrap(err, "writing Tombstones") + } + } + + for _, source := range m.Sources { + err = source.Hash(h) + if err != nil { + return 0, errors.Wrap(err, "writing Sources") + } + } + + for _, block := range m.Blocks { + err = block.Hash(h) + if err != nil { + return 0, errors.Wrap(err, "writing Blocks") + } + } + + return h.Sum32(), nil + } type MetaSearchParams struct { @@ -107,6 +149,21 @@ type Block struct { Data io.ReadSeekCloser } +func BlockFrom(tenant, table string, blk *v1.Block) Block { + md, _ := blk.Metadata() + ref := Ref{ + TenantID: tenant, + TableName: table, + Bounds: md.Series.Bounds, + StartTimestamp: md.Series.FromTs, + EndTimestamp: md.Series.ThroughTs, + Checksum: md.Checksum, + } + return Block{ + BlockRef: BlockRef{Ref: ref}, + } +} + type BlockClient interface { KeyResolver GetBlock(ctx context.Context, ref BlockRef) (BlockDirectory, error) From 69d152b618bd1e3f09712fa24f3070232f55d01a Mon Sep 17 00:00:00 2001 From: Christian Haudum Date: Tue, 6 Feb 2024 19:27:20 +0100 Subject: [PATCH 40/45] (chore) BloomStore: Clean up `FetchBlocks()` (#11876) **What this PR does / why we need it**: * Removes `FetchBlocksWithQueue` and replace it with `FetchBlocks` * Operate on single `BlockRef` when loading a block directory from cache/fs/storage. * Ensure order of responses from the FetchBlocks request --------- Signed-off-by: Christian Haudum --- pkg/bloomgateway/util_test.go | 4 +- pkg/bloomgateway/worker.go | 2 +- .../stores/shipper/bloomshipper/fetcher.go | 128 ++++++++---------- .../stores/shipper/bloomshipper/shipper.go | 15 +- .../stores/shipper/bloomshipper/store.go | 16 ++- .../stores/shipper/bloomshipper/store_test.go | 6 +- 6 files changed, 85 insertions(+), 86 deletions(-) diff --git a/pkg/bloomgateway/util_test.go b/pkg/bloomgateway/util_test.go index d60ab5f13a190..b77d2fe68f47a 100644 --- a/pkg/bloomgateway/util_test.go +++ b/pkg/bloomgateway/util_test.go @@ -396,8 +396,8 @@ func (s *mockBloomStore) GetBlockRefs(_ context.Context, tenant string, _ blooms // Stop implements bloomshipper.Interface func (s *mockBloomStore) Stop() {} -// Fetch implements bloomshipper.Interface -func (s *mockBloomStore) Fetch(_ context.Context, _ string, _ []bloomshipper.BlockRef, callback bloomshipper.ForEachBlockCallback) error { +// 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 diff --git a/pkg/bloomgateway/worker.go b/pkg/bloomgateway/worker.go index 34a01e50c4354..0f7db8a9ca586 100644 --- a/pkg/bloomgateway/worker.go +++ b/pkg/bloomgateway/worker.go @@ -240,7 +240,7 @@ func (w *worker) stopping(err error) error { } func (w *worker) processBlocksWithCallback(taskCtx context.Context, tenant string, blockRefs []bloomshipper.BlockRef, boundedRefs []boundedTasks) error { - return w.shipper.Fetch(taskCtx, tenant, blockRefs, func(bq *v1.BlockQuerier, bounds v1.FingerprintBounds) 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) diff --git a/pkg/storage/stores/shipper/bloomshipper/fetcher.go b/pkg/storage/stores/shipper/bloomshipper/fetcher.go index 6efb654809da7..6b477c1a0aa3b 100644 --- a/pkg/storage/stores/shipper/bloomshipper/fetcher.go +++ b/pkg/storage/stores/shipper/bloomshipper/fetcher.go @@ -3,7 +3,6 @@ package bloomshipper import ( "context" "encoding/json" - "fmt" "os" "path/filepath" "sync" @@ -125,120 +124,96 @@ func (f *Fetcher) writeBackMetas(ctx context.Context, metas []Meta) error { return f.metasCache.Store(ctx, keys, data) } -func (f *Fetcher) FetchBlocksWithQueue(ctx context.Context, refs []BlockRef) ([]BlockDirectory, error) { - responses := make(chan BlockDirectory, len(refs)) - errors := make(chan error, len(refs)) - for _, ref := range refs { - f.q.enqueue(downloadTask[BlockRef, BlockDirectory]{ +func (f *Fetcher) FetchBlocks(ctx context.Context, refs []BlockRef) ([]BlockDirectory, error) { + n := len(refs) + + responses := make(chan downloadResponse[BlockDirectory], n) + errors := make(chan error, n) + for i := 0; i < n; i++ { + f.q.enqueue(downloadRequest[BlockRef, BlockDirectory]{ ctx: ctx, - item: ref, - key: f.client.Block(ref).Addr(), + item: refs[i], + key: f.client.Block(refs[i]).Addr(), + idx: i, results: responses, errors: errors, }) } results := make([]BlockDirectory, len(refs)) - -outer: - for i := 0; i < len(refs); i++ { + for i := 0; i < n; i++ { select { case err := <-errors: return results, err case res := <-responses: - for j, ref := range refs { - if res.BlockRef == ref { - results[j] = res - continue outer - } - } - return results, fmt.Errorf("no matching request found for response %s", res) + results[res.idx] = res.item } } return results, nil } -func (f *Fetcher) processTask(ctx context.Context, task downloadTask[BlockRef, BlockDirectory]) { +func (f *Fetcher) processTask(ctx context.Context, task downloadRequest[BlockRef, BlockDirectory]) { if ctx.Err() != nil { task.errors <- ctx.Err() return } - refs := []BlockRef{task.item} - results, err := f.FetchBlocks(ctx, refs) + result, err := f.fetchBlock(ctx, task.item) if err != nil { task.errors <- err return } - for _, res := range results { - task.results <- res + task.results <- downloadResponse[BlockDirectory]{ + item: result, + key: task.key, + idx: task.idx, } } -// FetchBlocks returns a list of block directories -// It resolves them from three locations: +// fetchBlock resolves a block from three locations: // 1. from cache // 2. from file system // 3. from remote storage -func (f *Fetcher) FetchBlocks(ctx context.Context, refs []BlockRef) ([]BlockDirectory, error) { - if ctx.Err() != nil { - return nil, errors.Wrap(ctx.Err(), "fetch Blocks") - } +func (f *Fetcher) fetchBlock(ctx context.Context, ref BlockRef) (BlockDirectory, error) { + var zero BlockDirectory - keys := make([]string, 0, len(refs)) - for _, ref := range refs { - keys = append(keys, f.client.Block(ref).Addr()) - } - cacheHits, cacheBufs, _, err := f.blocksCache.Fetch(ctx, keys) - if err != nil { - return nil, err + if ctx.Err() != nil { + return zero, errors.Wrap(ctx.Err(), "fetch block") } - results := make([]BlockDirectory, 0, len(refs)) + keys := []string{f.client.Block(ref).Addr()} - fromCache, missing, err := f.processBlocksCacheResponse(ctx, refs, cacheHits, cacheBufs) + _, fromCache, _, err := f.blocksCache.Fetch(ctx, keys) if err != nil { - return nil, err + return zero, err } - results = append(results, fromCache...) - fromLocalFS, missing, err := f.loadBlocksFromFS(ctx, missing) - if err != nil { - return nil, err + // item found in cache + if len(fromCache) == 1 { + return fromCache[0], nil } - results = append(results, fromLocalFS...) - fromStorage, err := f.client.GetBlocks(ctx, missing) + fromLocalFS, _, err := f.loadBlocksFromFS(ctx, []BlockRef{ref}) if err != nil { - return nil, err + return zero, err } - results = append(results, fromStorage...) - - err = f.writeBackBlocks(ctx, fromStorage) - return results, err -} -func (f *Fetcher) processBlocksCacheResponse(_ context.Context, refs []BlockRef, keys []string, entries []BlockDirectory) ([]BlockDirectory, []BlockRef, error) { - found := make(map[string]BlockDirectory, len(refs)) - for i, k := range keys { - found[k] = entries[i] + // item found on local file system + if len(fromLocalFS) == 1 { + err = f.writeBackBlocks(ctx, fromLocalFS) + return fromLocalFS[0], err } - blockDirs := make([]BlockDirectory, 0, len(found)) - missing := make([]BlockRef, 0, len(refs)-len(keys)) - - var lastErr error - for i, ref := range refs { - if raw, ok := found[f.client.Block(ref).Addr()]; ok { - blockDirs = append(blockDirs, raw) - } else { - missing = append(missing, refs[i]) - } + fromStorage, err := f.client.GetBlock(ctx, ref) + if err != nil { + return zero, err } - return blockDirs, missing, lastErr + // item found in storage + err = f.writeBackBlocks(ctx, []BlockDirectory{fromStorage}) + return fromStorage, err } func (f *Fetcher) loadBlocksFromFS(_ context.Context, refs []BlockRef) ([]BlockDirectory, []BlockRef, error) { @@ -289,18 +264,25 @@ func (f *Fetcher) writeBackBlocks(ctx context.Context, blocks []BlockDirectory) return f.blocksCache.Store(ctx, keys, blocks) } -type processFunc[T any, R any] func(context.Context, downloadTask[T, R]) +type processFunc[T any, R any] func(context.Context, downloadRequest[T, R]) -type downloadTask[T any, R any] struct { +type downloadRequest[T any, R any] struct { ctx context.Context item T key string - results chan<- R + idx int + results chan<- downloadResponse[R] errors chan<- error } +type downloadResponse[R any] struct { + item R + key string + idx int +} + type downloadQueue[T any, R any] struct { - queue chan downloadTask[T, R] + queue chan downloadRequest[T, R] mu keymutex.KeyMutex wg sync.WaitGroup done chan struct{} @@ -310,7 +292,7 @@ type downloadQueue[T any, R any] struct { func newDownloadQueue[T any, R any](size, workers int, process processFunc[T, R], logger log.Logger) *downloadQueue[T, R] { q := &downloadQueue[T, R]{ - queue: make(chan downloadTask[T, R], size), + queue: make(chan downloadRequest[T, R], size), mu: keymutex.NewHashed(workers), done: make(chan struct{}), process: process, @@ -323,7 +305,7 @@ func newDownloadQueue[T any, R any](size, workers int, process processFunc[T, R] return q } -func (q *downloadQueue[T, R]) enqueue(t downloadTask[T, R]) { +func (q *downloadQueue[T, R]) enqueue(t downloadRequest[T, R]) { q.queue <- t } @@ -339,7 +321,7 @@ func (q *downloadQueue[T, R]) runWorker() { } } -func (q *downloadQueue[T, R]) do(ctx context.Context, task downloadTask[T, R]) { +func (q *downloadQueue[T, R]) do(ctx context.Context, task downloadRequest[T, R]) { q.mu.LockKey(task.key) defer func() { err := q.mu.UnlockKey(task.key) diff --git a/pkg/storage/stores/shipper/bloomshipper/shipper.go b/pkg/storage/stores/shipper/bloomshipper/shipper.go index 614f9c6898194..cff17296de9c0 100644 --- a/pkg/storage/stores/shipper/bloomshipper/shipper.go +++ b/pkg/storage/stores/shipper/bloomshipper/shipper.go @@ -23,7 +23,7 @@ type ForEachBlockCallback func(bq *v1.BlockQuerier, bounds v1.FingerprintBounds) type Interface interface { GetBlockRefs(ctx context.Context, tenant string, interval Interval) ([]BlockRef, error) - Fetch(ctx context.Context, tenant string, blocks []BlockRef, callback ForEachBlockCallback) error + ForEach(ctx context.Context, tenant string, blocks []BlockRef, callback ForEachBlockCallback) error Stop() } @@ -58,14 +58,21 @@ func (s *Shipper) GetBlockRefs(ctx context.Context, tenantID string, interval In return blockRefs, nil } -func (s *Shipper) Fetch(ctx context.Context, _ string, blocks []BlockRef, callback ForEachBlockCallback) error { +func (s *Shipper) ForEach(ctx context.Context, _ string, blocks []BlockRef, callback ForEachBlockCallback) error { blockDirs, err := s.store.FetchBlocks(ctx, blocks) if err != nil { return err } - for _, dir := range blockDirs { - err := runCallback(callback, dir.BlockQuerier(), dir.BlockRef.Bounds) + if len(blockDirs) != len(blocks) { + return fmt.Errorf("number of responses (%d) does not match number of requests (%d)", len(blockDirs), len(blocks)) + } + + for i := range blocks { + if blockDirs[i].BlockRef != blocks[i] { + return fmt.Errorf("invalid order of responses: expected: %v, got: %v", blocks[i], blockDirs[i].BlockRef) + } + err := runCallback(callback, blockDirs[i].BlockQuerier(), blockDirs[i].BlockRef.Bounds) if err != nil { return err } diff --git a/pkg/storage/stores/shipper/bloomshipper/store.go b/pkg/storage/stores/shipper/bloomshipper/store.go index 4e3dada7b44e9..c6acc9466da67 100644 --- a/pkg/storage/stores/shipper/bloomshipper/store.go +++ b/pkg/storage/stores/shipper/bloomshipper/store.go @@ -9,6 +9,7 @@ import ( "github.com/go-kit/log" "github.com/pkg/errors" "github.com/prometheus/common/model" + "golang.org/x/exp/slices" "github.com/grafana/loki/pkg/storage" v1 "github.com/grafana/loki/pkg/storage/bloom/v1" @@ -107,7 +108,7 @@ func (b *bloomStoreEntry) FetchMetas(ctx context.Context, params MetaSearchParam // FetchBlocks implements Store. func (b *bloomStoreEntry) FetchBlocks(ctx context.Context, refs []BlockRef) ([]BlockDirectory, error) { - return b.fetcher.FetchBlocksWithQueue(ctx, refs) + return b.fetcher.FetchBlocks(ctx, refs) } // Fetcher implements Store. @@ -299,13 +300,24 @@ func (b *BloomStore) FetchBlocks(ctx context.Context, blocks []BlockRef) ([]Bloc results := make([]BlockDirectory, 0, len(blocks)) for i := range fetchers { - res, err := fetchers[i].FetchBlocksWithQueue(ctx, refs[i]) + res, err := fetchers[i].FetchBlocks(ctx, refs[i]) results = append(results, res...) if err != nil { return results, err } } + // sort responses (results []BlockDirectory) based on requests (blocks []BlockRef) + slices.SortFunc(results, func(a, b BlockDirectory) int { + ia, ib := slices.Index(blocks, a.BlockRef), slices.Index(blocks, b.BlockRef) + if ia < ib { + return -1 + } else if ia > ib { + return +1 + } + return 0 + }) + return results, nil } diff --git a/pkg/storage/stores/shipper/bloomshipper/store_test.go b/pkg/storage/stores/shipper/bloomshipper/store_test.go index 87c2ed067c45f..aa0ca46e0660b 100644 --- a/pkg/storage/stores/shipper/bloomshipper/store_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/store_test.go @@ -259,10 +259,8 @@ func TestBloomStore_FetchBlocks(t *testing.T) { require.NoError(t, err) require.Len(t, blockDirs, 4) - // Note the order: b1 and b2 come from cache, so they are in the beginning of the response - // Do we need to sort the response based on the request order of block refs? - require.ElementsMatch(t, - []BlockRef{b1.BlockRef, b3.BlockRef, b2.BlockRef, b4.BlockRef}, + require.Equal(t, + []BlockRef{b1.BlockRef, b2.BlockRef, b3.BlockRef, b4.BlockRef}, []BlockRef{blockDirs[0].BlockRef, blockDirs[1].BlockRef, blockDirs[2].BlockRef, blockDirs[3].BlockRef}, ) } From 7bbbf232d0dbe9a7099441862c795828210bb0c4 Mon Sep 17 00:00:00 2001 From: Christian Haudum Date: Tue, 6 Feb 2024 19:29:30 +0100 Subject: [PATCH 41/45] Extend BloomStore with `Client(model.Time)` function (#11881) **What this PR does / why we need it**: For certain operations, we want direct access to the `BloomClient`. Since every schema period has its own client, the `Client()` function accepts a `model.Timestamp` for which the client should be returned. The function may return an error, if no client for the given time could be found. --------- Signed-off-by: Christian Haudum --- pkg/bloomgateway/processor_test.go | 8 +++-- .../stores/shipper/bloomshipper/store.go | 31 +++++++++++++++---- 2 files changed, 31 insertions(+), 8 deletions(-) diff --git a/pkg/bloomgateway/processor_test.go b/pkg/bloomgateway/processor_test.go index 33f7513fbf592..7cb37d97a0278 100644 --- a/pkg/bloomgateway/processor_test.go +++ b/pkg/bloomgateway/processor_test.go @@ -42,8 +42,12 @@ func (s *dummyStore) FetchBlocks(_ context.Context, _ []bloomshipper.BlockRef) ( panic("don't call me") } -func (s *dummyStore) Fetcher(_ model.Time) *bloomshipper.Fetcher { - return nil +func (s *dummyStore) Fetcher(_ model.Time) (*bloomshipper.Fetcher, error) { + return nil, nil +} + +func (s *dummyStore) Client(_ model.Time) (bloomshipper.Client, error) { + return nil, nil } func (s *dummyStore) Stop() { diff --git a/pkg/storage/stores/shipper/bloomshipper/store.go b/pkg/storage/stores/shipper/bloomshipper/store.go index c6acc9466da67..5f1b7bf8d1494 100644 --- a/pkg/storage/stores/shipper/bloomshipper/store.go +++ b/pkg/storage/stores/shipper/bloomshipper/store.go @@ -18,11 +18,16 @@ import ( "github.com/grafana/loki/pkg/storage/config" ) +var ( + errNoStore = errors.New("no store found for time") +) + type Store interface { ResolveMetas(ctx context.Context, params MetaSearchParams) ([][]MetaRef, []*Fetcher, error) FetchMetas(ctx context.Context, params MetaSearchParams) ([]Meta, error) FetchBlocks(ctx context.Context, refs []BlockRef) ([]BlockDirectory, error) - Fetcher(ts model.Time) *Fetcher + Fetcher(ts model.Time) (*Fetcher, error) + Client(ts model.Time) (Client, error) Stop() } @@ -112,8 +117,13 @@ func (b *bloomStoreEntry) FetchBlocks(ctx context.Context, refs []BlockRef) ([]B } // Fetcher implements Store. -func (b *bloomStoreEntry) Fetcher(_ model.Time) *Fetcher { - return b.fetcher +func (b *bloomStoreEntry) Fetcher(_ model.Time) (*Fetcher, error) { + return b.fetcher, nil +} + +// Client implements Store. +func (b *bloomStoreEntry) Client(_ model.Time) (Client, error) { + return b.bloomClient, nil } // Stop implements Store. @@ -219,11 +229,19 @@ func (b *BloomStore) Block(ref BlockRef) (loc Location) { } // Fetcher implements Store. -func (b *BloomStore) Fetcher(ts model.Time) *Fetcher { +func (b *BloomStore) Fetcher(ts model.Time) (*Fetcher, error) { if store := b.getStore(ts); store != nil { return store.Fetcher(ts) } - return nil + return nil, errNoStore +} + +// Client implements Store. +func (b *BloomStore) Client(ts model.Time) (Client, error) { + if store := b.getStore(ts); store != nil { + return store.Client(ts) + } + return nil, errNoStore } // ResolveMetas implements Store. @@ -294,7 +312,7 @@ func (b *BloomStore) FetchBlocks(ctx context.Context, blocks []BlockRef) ([]Bloc if len(res) > 0 { refs = append(refs, res) - fetchers = append(fetchers, s.Fetcher(s.start)) + fetchers = append(fetchers, s.fetcher) } } @@ -341,6 +359,7 @@ func (b *BloomStore) getStore(ts model.Time) *bloomStoreEntry { return b.stores[j] } + // should in theory never happen return nil } From c4ac168a8c7885847fee505a6b63708f8c30aca4 Mon Sep 17 00:00:00 2001 From: Christian Haudum Date: Tue, 6 Feb 2024 21:58:19 +0100 Subject: [PATCH 42/45] BloomStore: Change signature of `FetchBlocks()` to return `[]*CloseableBlockQuerier` (#11884) **What this PR does / why we need it**: In order to encapsulate the reference counting on the `BlockDirectory`, the `BloomStore` now returns a slice of `*CloseableBlockQuerier` instead of the directory itself. The caller is responsible for closing the returned querier in order to release the reader resource and decrement the ref counter. The PR also renames `ClosableBlockQuerier` to `CloseableBlockQuerier`. --------- Signed-off-by: Christian Haudum --- pkg/bloomcompactor/controller.go | 8 +--- pkg/bloomcompactor/spec.go | 6 +-- pkg/bloomcompactor/spec_test.go | 4 +- pkg/bloomgateway/processor.go | 20 ++++---- pkg/bloomgateway/processor_test.go | 17 +++---- pkg/bloomgateway/util_test.go | 47 ++++++++++--------- .../stores/shipper/bloomshipper/cache.go | 32 ++++++++----- .../stores/shipper/bloomshipper/cache_test.go | 20 ++++---- .../stores/shipper/bloomshipper/fetcher.go | 8 ++-- .../stores/shipper/bloomshipper/shipper.go | 31 ++++-------- .../stores/shipper/bloomshipper/store.go | 12 ++--- .../stores/shipper/bloomshipper/store_test.go | 12 ++--- 12 files changed, 100 insertions(+), 117 deletions(-) diff --git a/pkg/bloomcompactor/controller.go b/pkg/bloomcompactor/controller.go index f6487d5f61d11..2002d8ce2a8bc 100644 --- a/pkg/bloomcompactor/controller.go +++ b/pkg/bloomcompactor/controller.go @@ -184,7 +184,7 @@ func (s *SimpleBloomController) do(ctx context.Context) error { } -func (s *SimpleBloomController) loadWorkForGap(ctx context.Context, id tsdb.Identifier, gap gapWithBlocks) (v1.CloseableIterator[*v1.Series], []*bloomshipper.ClosableBlockQuerier, error) { +func (s *SimpleBloomController) loadWorkForGap(ctx context.Context, 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) if err != nil { @@ -195,12 +195,8 @@ func (s *SimpleBloomController) loadWorkForGap(ctx context.Context, id tsdb.Iden if err != nil { return nil, nil, errors.Wrap(err, "failed to get blocks") } - results := make([]*bloomshipper.ClosableBlockQuerier, 0, len(blocks)) - for _, block := range blocks { - results = append(results, block.BlockQuerier()) - } - return seriesItr, results, nil + return seriesItr, blocks, nil } type gapWithBlocks struct { diff --git a/pkg/bloomcompactor/spec.go b/pkg/bloomcompactor/spec.go index 341e3977502c3..bf9a0a02387b4 100644 --- a/pkg/bloomcompactor/spec.go +++ b/pkg/bloomcompactor/spec.go @@ -74,7 +74,7 @@ type SimpleBloomGenerator struct { 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.ClosableBlockQuerier + blocks []*bloomshipper.CloseableBlockQuerier // options to build blocks with opts v1.BlockOptions @@ -95,7 +95,7 @@ func NewSimpleBloomGenerator( opts v1.BlockOptions, store v1.Iterator[*v1.Series], chunkLoader ChunkLoader, - blocks []*bloomshipper.ClosableBlockQuerier, + blocks []*bloomshipper.CloseableBlockQuerier, readWriterFn func() (v1.BlockWriter, v1.BlockReader), metrics *Metrics, logger log.Logger, @@ -136,7 +136,7 @@ func (s *SimpleBloomGenerator) Generate(ctx context.Context) (skippedBlocks []v1 var closeErrors multierror.MultiError blocksMatchingSchema := make([]v1.PeekingIterator[*v1.SeriesWithBloom], 0, len(s.blocks)) - toClose := make([]*bloomshipper.ClosableBlockQuerier, 0, len(s.blocks)) + toClose := make([]*bloomshipper.CloseableBlockQuerier, 0, len(s.blocks)) // Close all remaining blocks on exit defer func() { for _, block := range toClose { diff --git a/pkg/bloomcompactor/spec_test.go b/pkg/bloomcompactor/spec_test.go index efc0d70f2020a..c43a4b715a1e7 100644 --- a/pkg/bloomcompactor/spec_test.go +++ b/pkg/bloomcompactor/spec_test.go @@ -64,9 +64,9 @@ func (dummyChunkLoader) Load(_ context.Context, series *v1.Series) (*ChunkItersB } func dummyBloomGen(opts v1.BlockOptions, store v1.Iterator[*v1.Series], blocks []*v1.Block) *SimpleBloomGenerator { - bqs := make([]*bloomshipper.ClosableBlockQuerier, 0, len(blocks)) + bqs := make([]*bloomshipper.CloseableBlockQuerier, 0, len(blocks)) for _, b := range blocks { - bqs = append(bqs, &bloomshipper.ClosableBlockQuerier{ + bqs = append(bqs, &bloomshipper.CloseableBlockQuerier{ BlockQuerier: v1.NewBlockQuerier(b), }) } diff --git a/pkg/bloomgateway/processor.go b/pkg/bloomgateway/processor.go index 117e736e4f54f..26895bc43eda5 100644 --- a/pkg/bloomgateway/processor.go +++ b/pkg/bloomgateway/processor.go @@ -17,17 +17,8 @@ type tasksForBlock struct { tasks []Task } -type blockLoader interface { - LoadBlocks(context.Context, []bloomshipper.BlockRef) (v1.Iterator[bloomshipper.BlockQuerierWithFingerprintRange], error) -} - -type store interface { - blockLoader - bloomshipper.Store -} - type processor struct { - store store + store bloomshipper.Store logger log.Logger } @@ -70,17 +61,20 @@ func (p *processor) processBlocks(ctx context.Context, data []tasksForBlock) err refs = append(refs, block.blockRef) } - blockIter, err := p.store.LoadBlocks(ctx, refs) + bqs, err := p.store.FetchBlocks(ctx, refs) if err != nil { return err } + blockIter := v1.NewSliceIter(bqs) + outer: for blockIter.Next() { bq := blockIter.At() for i, block := range data { - if block.blockRef.Bounds.Equal(bq.FingerprintBounds) { + if block.blockRef.Bounds.Equal(bq.Bounds) { err := p.processBlock(ctx, bq.BlockQuerier, block.tasks) + bq.Close() if err != nil { return err } @@ -88,6 +82,8 @@ outer: continue outer } } + // should not happen, but close anyway + bq.Close() } return nil } diff --git a/pkg/bloomgateway/processor_test.go b/pkg/bloomgateway/processor_test.go index 7cb37d97a0278..d39ba61a89613 100644 --- a/pkg/bloomgateway/processor_test.go +++ b/pkg/bloomgateway/processor_test.go @@ -12,16 +12,15 @@ import ( "go.uber.org/atomic" "github.com/grafana/loki/pkg/logql/syntax" - v1 "github.com/grafana/loki/pkg/storage/bloom/v1" "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper" ) -var _ store = &dummyStore{} +var _ bloomshipper.Store = &dummyStore{} type dummyStore struct { metas []bloomshipper.Meta blocks []bloomshipper.BlockRef - querieres []bloomshipper.BlockQuerierWithFingerprintRange + querieres []*bloomshipper.CloseableBlockQuerier } func (s *dummyStore) ResolveMetas(_ context.Context, _ bloomshipper.MetaSearchParams) ([][]bloomshipper.MetaRef, []*bloomshipper.Fetcher, error) { @@ -38,10 +37,6 @@ func (s *dummyStore) FetchMetas(_ context.Context, _ bloomshipper.MetaSearchPara return s.metas, nil } -func (s *dummyStore) FetchBlocks(_ context.Context, _ []bloomshipper.BlockRef) ([]bloomshipper.BlockDirectory, error) { - panic("don't call me") -} - func (s *dummyStore) Fetcher(_ model.Time) (*bloomshipper.Fetcher, error) { return nil, nil } @@ -53,12 +48,12 @@ func (s *dummyStore) Client(_ model.Time) (bloomshipper.Client, error) { func (s *dummyStore) Stop() { } -func (s *dummyStore) LoadBlocks(_ context.Context, refs []bloomshipper.BlockRef) (v1.Iterator[bloomshipper.BlockQuerierWithFingerprintRange], error) { - result := make([]bloomshipper.BlockQuerierWithFingerprintRange, len(s.querieres)) +func (s *dummyStore) FetchBlocks(_ context.Context, refs []bloomshipper.BlockRef) ([]*bloomshipper.CloseableBlockQuerier, error) { + result := make([]*bloomshipper.CloseableBlockQuerier, 0, len(s.querieres)) for _, ref := range refs { for _, bq := range s.querieres { - if ref.Bounds.Equal(bq.FingerprintBounds) { + if ref.Bounds.Equal(bq.Bounds) { result = append(result, bq) } } @@ -68,7 +63,7 @@ func (s *dummyStore) LoadBlocks(_ context.Context, refs []bloomshipper.BlockRef) result[i], result[j] = result[j], result[i] }) - return v1.NewSliceIter(result), nil + return result, nil } func TestProcessor(t *testing.T) { diff --git a/pkg/bloomgateway/util_test.go b/pkg/bloomgateway/util_test.go index b77d2fe68f47a..f19564b43ef59 100644 --- a/pkg/bloomgateway/util_test.go +++ b/pkg/bloomgateway/util_test.go @@ -295,10 +295,10 @@ func TestPartitionRequest(t *testing.T) { } -func createBlockQueriers(t *testing.T, numBlocks int, from, through model.Time, minFp, maxFp model.Fingerprint) ([]bloomshipper.BlockQuerierWithFingerprintRange, [][]v1.SeriesWithBloom) { +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.BlockQuerierWithFingerprintRange, 0, 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)) @@ -308,9 +308,15 @@ func createBlockQueriers(t *testing.T, numBlocks int, from, through model.Time, throughFp = maxFp } blockQuerier, data := v1.MakeBlockQuerier(t, fromFp, throughFp, from, through) - bq := bloomshipper.BlockQuerierWithFingerprintRange{ - BlockQuerier: blockQuerier, - FingerprintBounds: v1.NewBounds(fromFp, throughFp), + 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) @@ -318,12 +324,12 @@ func createBlockQueriers(t *testing.T, numBlocks int, from, through model.Time, 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.BlockQuerierWithFingerprintRange, [][]v1.SeriesWithBloom) { +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) metas := make([]bloomshipper.Meta, 0, n) - queriers := make([]bloomshipper.BlockQuerierWithFingerprintRange, 0, n) + queriers := make([]*bloomshipper.CloseableBlockQuerier, 0, n) series := make([][]v1.SeriesWithBloom, 0, n) step := (maxFp - minFp) / model.Fingerprint(n) @@ -352,9 +358,9 @@ func createBlocks(t *testing.T, tenant string, n int, from, through model.Time, Blocks: []bloomshipper.BlockRef{block}, } blockQuerier, data := v1.MakeBlockQuerier(t, fromFp, throughFp, from, through) - querier := bloomshipper.BlockQuerierWithFingerprintRange{ - BlockQuerier: blockQuerier, - FingerprintBounds: v1.NewBounds(fromFp, throughFp), + querier := &bloomshipper.CloseableBlockQuerier{ + BlockQuerier: blockQuerier, + BlockRef: block, } queriers = append(queriers, querier) metas = append(metas, meta) @@ -364,12 +370,12 @@ func createBlocks(t *testing.T, tenant string, n int, from, through model.Time, return blocks, metas, queriers, series } -func newMockBloomStore(bqs []bloomshipper.BlockQuerierWithFingerprintRange) *mockBloomStore { +func newMockBloomStore(bqs []*bloomshipper.CloseableBlockQuerier) *mockBloomStore { return &mockBloomStore{bqs: bqs} } type mockBloomStore struct { - bqs []bloomshipper.BlockQuerierWithFingerprintRange + bqs []*bloomshipper.CloseableBlockQuerier // mock how long it takes to serve block queriers delay time.Duration // mock response error when serving block queriers in ForEach @@ -379,16 +385,11 @@ type mockBloomStore struct { var _ bloomshipper.Interface = &mockBloomStore{} // GetBlockRefs implements bloomshipper.Interface -func (s *mockBloomStore) GetBlockRefs(_ context.Context, tenant string, _ bloomshipper.Interval) ([]bloomshipper.BlockRef, error) { +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, bloomshipper.BlockRef{ - Ref: bloomshipper.Ref{ - Bounds: v1.NewBounds(s.bqs[i].Min, s.bqs[i].Max), - TenantID: tenant, - }, - }) + blocks = append(blocks, s.bqs[i].BlockRef) } return blocks, nil } @@ -403,7 +404,7 @@ func (s *mockBloomStore) ForEach(_ context.Context, _ string, _ []bloomshipper.B return s.err } - shuffled := make([]bloomshipper.BlockQuerierWithFingerprintRange, len(s.bqs)) + shuffled := make([]*bloomshipper.CloseableBlockQuerier, len(s.bqs)) _ = copy(shuffled, s.bqs) rand.Shuffle(len(shuffled), func(i, j int) { @@ -413,7 +414,7 @@ func (s *mockBloomStore) ForEach(_ context.Context, _ string, _ []bloomshipper.B for _, bq := range shuffled { // ignore errors in the mock time.Sleep(s.delay) - err := callback(bq.BlockQuerier, bq.FingerprintBounds) + err := callback(bq.BlockQuerier, bq.Bounds) if err != nil { return err } @@ -443,7 +444,7 @@ func createQueryInputFromBlockData(t *testing.T, tenant string, data [][]v1.Seri return res } -func createBlockRefsFromBlockData(t *testing.T, tenant string, data []bloomshipper.BlockQuerierWithFingerprintRange) []bloomshipper.BlockRef { +func createBlockRefsFromBlockData(t *testing.T, tenant string, data []*bloomshipper.CloseableBlockQuerier) []bloomshipper.BlockRef { t.Helper() res := make([]bloomshipper.BlockRef, 0) for i := range data { @@ -451,7 +452,7 @@ func createBlockRefsFromBlockData(t *testing.T, tenant string, data []bloomshipp Ref: bloomshipper.Ref{ TenantID: tenant, TableName: "", - Bounds: v1.NewBounds(data[i].Min, data[i].Max), + Bounds: v1.NewBounds(data[i].Bounds.Min, data[i].Bounds.Max), StartTimestamp: 0, EndTimestamp: 0, Checksum: 0, diff --git a/pkg/storage/stores/shipper/bloomshipper/cache.go b/pkg/storage/stores/shipper/bloomshipper/cache.go index 2f9e98f89d9c0..52899a03fea0a 100644 --- a/pkg/storage/stores/shipper/bloomshipper/cache.go +++ b/pkg/storage/stores/shipper/bloomshipper/cache.go @@ -17,12 +17,13 @@ import ( "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper/config" ) -type ClosableBlockQuerier struct { +type CloseableBlockQuerier struct { + BlockRef *v1.BlockQuerier close func() error } -func (c *ClosableBlockQuerier) Close() error { +func (c *CloseableBlockQuerier) Close() error { if c.close != nil { return c.close() } @@ -53,7 +54,7 @@ func NewBlockDirectory(ref BlockRef, path string, logger log.Logger) BlockDirect return BlockDirectory{ BlockRef: ref, Path: path, - activeQueriers: atomic.NewInt32(0), + refCount: atomic.NewInt32(0), removeDirectoryTimeout: time.Minute, logger: logger, activeQueriersCheckInterval: defaultActiveQueriersCheckInterval, @@ -66,7 +67,7 @@ type BlockDirectory struct { BlockRef Path string removeDirectoryTimeout time.Duration - activeQueriers *atomic.Int32 + refCount *atomic.Int32 logger log.Logger activeQueriersCheckInterval time.Duration } @@ -75,17 +76,24 @@ func (b BlockDirectory) Block() *v1.Block { return v1.NewBlock(v1.NewDirectoryBlockReader(b.Path)) } +func (b BlockDirectory) Acquire() { + _ = b.refCount.Inc() +} + +func (b BlockDirectory) Release() error { + _ = b.refCount.Dec() + return nil +} + // BlockQuerier returns a new block querier from the directory. // It increments the counter of active queriers for this directory. // The counter is decreased when the returned querier is closed. -func (b BlockDirectory) BlockQuerier() *ClosableBlockQuerier { - b.activeQueriers.Inc() - return &ClosableBlockQuerier{ +func (b BlockDirectory) BlockQuerier() *CloseableBlockQuerier { + b.Acquire() + return &CloseableBlockQuerier{ BlockQuerier: v1.NewBlockQuerier(b.Block()), - close: func() error { - _ = b.activeQueriers.Dec() - return nil - }, + BlockRef: b.BlockRef, + close: b.Release, } } @@ -99,7 +107,7 @@ func (b *BlockDirectory) removeDirectoryAsync() { for { select { case <-ticker.C: - if b.activeQueriers.Load() == 0 { + if b.refCount.Load() == 0 { err := deleteFolder(b.Path) if err == nil { return diff --git a/pkg/storage/stores/shipper/bloomshipper/cache_test.go b/pkg/storage/stores/shipper/bloomshipper/cache_test.go index de916377a3f7b..c85f0382bafdd 100644 --- a/pkg/storage/stores/shipper/bloomshipper/cache_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/cache_test.go @@ -35,21 +35,21 @@ func TestBlockDirectory_Cleanup(t *testing.T) { require.NoError(t, err) require.DirExists(t, extractedBlockDirectory) - cached := BlockDirectory{ + blockDir := BlockDirectory{ Path: extractedBlockDirectory, removeDirectoryTimeout: timeout, activeQueriersCheckInterval: checkInterval, logger: log.NewNopLogger(), - activeQueriers: atomic.NewInt32(0), + refCount: atomic.NewInt32(0), } // acquire directory - cached.activeQueriers.Inc() + blockDir.refCount.Inc() // start cleanup goroutine - cached.removeDirectoryAsync() + blockDir.removeDirectoryAsync() if tc.releaseQuerier { // release directory - cached.activeQueriers.Dec() + blockDir.refCount.Dec() } // ensure directory does not exist any more @@ -66,15 +66,15 @@ func Test_ClosableBlockQuerier(t *testing.T) { err := extractArchive(blockFilePath, extractedBlockDirectory) require.NoError(t, err) - cached := BlockDirectory{ + blockDir := BlockDirectory{ Path: extractedBlockDirectory, removeDirectoryTimeout: 100 * time.Millisecond, - activeQueriers: atomic.NewInt32(0), + refCount: atomic.NewInt32(0), } - querier := cached.BlockQuerier() - require.Equal(t, int32(1), cached.activeQueriers.Load()) + querier := blockDir.BlockQuerier() + require.Equal(t, int32(1), blockDir.refCount.Load()) require.NoError(t, querier.Close()) - require.Equal(t, int32(0), cached.activeQueriers.Load()) + require.Equal(t, int32(0), blockDir.refCount.Load()) } diff --git a/pkg/storage/stores/shipper/bloomshipper/fetcher.go b/pkg/storage/stores/shipper/bloomshipper/fetcher.go index 6b477c1a0aa3b..bb9a70644f5e6 100644 --- a/pkg/storage/stores/shipper/bloomshipper/fetcher.go +++ b/pkg/storage/stores/shipper/bloomshipper/fetcher.go @@ -21,7 +21,7 @@ type metrics struct{} type fetcher interface { FetchMetas(ctx context.Context, refs []MetaRef) ([]Meta, error) - FetchBlocks(ctx context.Context, refs []BlockRef) ([]BlockDirectory, error) + FetchBlocks(ctx context.Context, refs []BlockRef) ([]*CloseableBlockQuerier, error) Close() } @@ -124,7 +124,7 @@ func (f *Fetcher) writeBackMetas(ctx context.Context, metas []Meta) error { return f.metasCache.Store(ctx, keys, data) } -func (f *Fetcher) FetchBlocks(ctx context.Context, refs []BlockRef) ([]BlockDirectory, error) { +func (f *Fetcher) FetchBlocks(ctx context.Context, refs []BlockRef) ([]*CloseableBlockQuerier, error) { n := len(refs) responses := make(chan downloadResponse[BlockDirectory], n) @@ -140,13 +140,13 @@ func (f *Fetcher) FetchBlocks(ctx context.Context, refs []BlockRef) ([]BlockDire }) } - results := make([]BlockDirectory, len(refs)) + results := make([]*CloseableBlockQuerier, n) for i := 0; i < n; i++ { select { case err := <-errors: return results, err case res := <-responses: - results[res.idx] = res.item + results[res.idx] = res.item.BlockQuerier() } } diff --git a/pkg/storage/stores/shipper/bloomshipper/shipper.go b/pkg/storage/stores/shipper/bloomshipper/shipper.go index cff17296de9c0..6d6322c91382b 100644 --- a/pkg/storage/stores/shipper/bloomshipper/shipper.go +++ b/pkg/storage/stores/shipper/bloomshipper/shipper.go @@ -14,11 +14,6 @@ import ( "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper/config" ) -type BlockQuerierWithFingerprintRange struct { - *v1.BlockQuerier - v1.FingerprintBounds -} - type ForEachBlockCallback func(bq *v1.BlockQuerier, bounds v1.FingerprintBounds) error type Interface interface { @@ -58,21 +53,21 @@ func (s *Shipper) GetBlockRefs(ctx context.Context, tenantID string, interval In return blockRefs, nil } -func (s *Shipper) ForEach(ctx context.Context, _ string, blocks []BlockRef, callback ForEachBlockCallback) error { - blockDirs, err := s.store.FetchBlocks(ctx, blocks) +func (s *Shipper) ForEach(ctx context.Context, _ string, refs []BlockRef, callback ForEachBlockCallback) error { + bqs, err := s.store.FetchBlocks(ctx, refs) + if err != nil { return err } - if len(blockDirs) != len(blocks) { - return fmt.Errorf("number of responses (%d) does not match number of requests (%d)", len(blockDirs), len(blocks)) + if len(bqs) != len(refs) { + return fmt.Errorf("number of response (%d) does not match number of requests (%d)", len(bqs), len(refs)) } - for i := range blocks { - if blockDirs[i].BlockRef != blocks[i] { - return fmt.Errorf("invalid order of responses: expected: %v, got: %v", blocks[i], blockDirs[i].BlockRef) - } - err := runCallback(callback, blockDirs[i].BlockQuerier(), blockDirs[i].BlockRef.Bounds) + for i := range bqs { + err := callback(bqs[i].BlockQuerier, bqs[i].Bounds) + // close querier to decrement ref count + bqs[i].Close() if err != nil { return err } @@ -80,14 +75,6 @@ func (s *Shipper) ForEach(ctx context.Context, _ string, blocks []BlockRef, call return nil } -func runCallback(callback ForEachBlockCallback, bq *ClosableBlockQuerier, bounds v1.FingerprintBounds) error { - defer func(b *ClosableBlockQuerier) { - _ = b.Close() - }(bq) - - return callback(bq.BlockQuerier, bounds) -} - func (s *Shipper) Stop() { s.store.Stop() } diff --git a/pkg/storage/stores/shipper/bloomshipper/store.go b/pkg/storage/stores/shipper/bloomshipper/store.go index 5f1b7bf8d1494..c95d04122117f 100644 --- a/pkg/storage/stores/shipper/bloomshipper/store.go +++ b/pkg/storage/stores/shipper/bloomshipper/store.go @@ -25,7 +25,7 @@ var ( type Store interface { ResolveMetas(ctx context.Context, params MetaSearchParams) ([][]MetaRef, []*Fetcher, error) FetchMetas(ctx context.Context, params MetaSearchParams) ([]Meta, error) - FetchBlocks(ctx context.Context, refs []BlockRef) ([]BlockDirectory, error) + FetchBlocks(ctx context.Context, refs []BlockRef) ([]*CloseableBlockQuerier, error) Fetcher(ts model.Time) (*Fetcher, error) Client(ts model.Time) (Client, error) Stop() @@ -112,7 +112,7 @@ func (b *bloomStoreEntry) FetchMetas(ctx context.Context, params MetaSearchParam } // FetchBlocks implements Store. -func (b *bloomStoreEntry) FetchBlocks(ctx context.Context, refs []BlockRef) ([]BlockDirectory, error) { +func (b *bloomStoreEntry) FetchBlocks(ctx context.Context, refs []BlockRef) ([]*CloseableBlockQuerier, error) { return b.fetcher.FetchBlocks(ctx, refs) } @@ -291,7 +291,7 @@ func (b *BloomStore) FetchMetas(ctx context.Context, params MetaSearchParams) ([ } // FetchBlocks implements Store. -func (b *BloomStore) FetchBlocks(ctx context.Context, blocks []BlockRef) ([]BlockDirectory, error) { +func (b *BloomStore) FetchBlocks(ctx context.Context, blocks []BlockRef) ([]*CloseableBlockQuerier, error) { var refs [][]BlockRef var fetchers []*Fetcher @@ -316,7 +316,7 @@ func (b *BloomStore) FetchBlocks(ctx context.Context, blocks []BlockRef) ([]Bloc } } - results := make([]BlockDirectory, 0, len(blocks)) + results := make([]*CloseableBlockQuerier, 0, len(blocks)) for i := range fetchers { res, err := fetchers[i].FetchBlocks(ctx, refs[i]) results = append(results, res...) @@ -325,8 +325,8 @@ func (b *BloomStore) FetchBlocks(ctx context.Context, blocks []BlockRef) ([]Bloc } } - // sort responses (results []BlockDirectory) based on requests (blocks []BlockRef) - slices.SortFunc(results, func(a, b BlockDirectory) int { + // sort responses (results []*CloseableBlockQuerier) based on requests (blocks []BlockRef) + slices.SortFunc(results, func(a, b *CloseableBlockQuerier) int { ia, ib := slices.Index(blocks, a.BlockRef), slices.Index(blocks, b.BlockRef) if ia < ib { return -1 diff --git a/pkg/storage/stores/shipper/bloomshipper/store_test.go b/pkg/storage/stores/shipper/bloomshipper/store_test.go index aa0ca46e0660b..42b35111688b2 100644 --- a/pkg/storage/stores/shipper/bloomshipper/store_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/store_test.go @@ -248,19 +248,19 @@ func TestBloomStore_FetchBlocks(t *testing.T) { ctx := context.Background() // first call fetches two blocks from cache - blockDirs, err := store.FetchBlocks(ctx, []BlockRef{b1.BlockRef, b3.BlockRef}) + bqs, err := store.FetchBlocks(ctx, []BlockRef{b1.BlockRef, b3.BlockRef}) require.NoError(t, err) - require.Len(t, blockDirs, 2) + require.Len(t, bqs, 2) - require.ElementsMatch(t, []BlockRef{b1.BlockRef, b3.BlockRef}, []BlockRef{blockDirs[0].BlockRef, blockDirs[1].BlockRef}) + require.Equal(t, []BlockRef{b1.BlockRef, b3.BlockRef}, []BlockRef{bqs[0].BlockRef, bqs[1].BlockRef}) // second call fetches two blocks from cache and two from storage - blockDirs, err = store.FetchBlocks(ctx, []BlockRef{b1.BlockRef, b2.BlockRef, b3.BlockRef, b4.BlockRef}) + bqs, err = store.FetchBlocks(ctx, []BlockRef{b1.BlockRef, b2.BlockRef, b3.BlockRef, b4.BlockRef}) require.NoError(t, err) - require.Len(t, blockDirs, 4) + require.Len(t, bqs, 4) require.Equal(t, []BlockRef{b1.BlockRef, b2.BlockRef, b3.BlockRef, b4.BlockRef}, - []BlockRef{blockDirs[0].BlockRef, blockDirs[1].BlockRef, blockDirs[2].BlockRef, blockDirs[3].BlockRef}, + []BlockRef{bqs[0].BlockRef, bqs[1].BlockRef, bqs[2].BlockRef, bqs[3].BlockRef}, ) } From 46c6118eb2a2d56406042431dfbed288df6e924b Mon Sep 17 00:00:00 2001 From: J Stickler Date: Tue, 6 Feb 2024 19:30:05 -0500 Subject: [PATCH 43/45] Update docs to support OTEL GA (#11858) **What this PR does / why we need it**: Updating the docs in advance of OTEL release to remove "experimental" notes. Also restructured release notes so that most recent content is first. --- .../get-started/labels/structured-metadata.md | 4 ---- docs/sources/release-notes/v2-9.md | 12 +++++++----- docs/sources/send-data/otel/_index.md | 4 ---- 3 files changed, 7 insertions(+), 13 deletions(-) diff --git a/docs/sources/get-started/labels/structured-metadata.md b/docs/sources/get-started/labels/structured-metadata.md index 071339cc0bde3..e199402e0b000 100644 --- a/docs/sources/get-started/labels/structured-metadata.md +++ b/docs/sources/get-started/labels/structured-metadata.md @@ -5,10 +5,6 @@ description: Describes how to enable structure metadata for logs and how to quer --- # What is structured metadata -{{% admonition type="warning" %}} -Structured metadata is an experimental feature and is subject to change in future releases of Grafana Loki. This feature is not yet available for Cloud Logs users. -{{% /admonition %}} - {{% admonition type="warning" %}} Structured metadata was added to chunk format V4 which is used if the schema version is greater or equal to `13`. (See [Schema Config]({{< relref "../../storage#schema-config" >}}) for more details about schema versions. ) {{% /admonition %}} diff --git a/docs/sources/release-notes/v2-9.md b/docs/sources/release-notes/v2-9.md index 8355dd02abf08..68d3da85bc4dd 100644 --- a/docs/sources/release-notes/v2-9.md +++ b/docs/sources/release-notes/v2-9.md @@ -9,6 +9,8 @@ Grafana Labs is excited to announce the release of Loki 2.9.0 Here's a summary o ## Features and enhancements +- **Structured metadata**: The [Structured Metadata](https://grafana.com/docs/loki/latest/get-started/labels/structured-metadata/) feature, which was introduced as experimental in release 2.9.0, is generally available as of release 2.9.4. + - **Query Language Improvements**: Several improvements to the query language that speed up line parsing and regex matching. [PR #8646](https://github.com/grafana/loki/pull/8646), [PR #8659](https://github.com/grafana/loki/pull/8659), [PR #8724](https://github.com/grafana/loki/pull/8724), [PR #8734](https://github.com/grafana/loki/pull/8734), [PR #8739](https://github.com/grafana/loki/pull/8739), [PR #8763](https://github.com/grafana/loki/pull/8763), [PR #8890](https://github.com/grafana/loki/pull/8890), [PR #8914](https://github.com/grafana/loki/pull/8914) - **Remote rule evaluation**: Rule evaluation can now be handled by queriers to improve speed. [PR #8744](https://github.com/grafana/loki/pull/8744) [PR #8848](https://github.com/grafana/loki/pull/8848) @@ -33,13 +35,13 @@ Grafana Labs is excited to announce the release of Loki 2.9.0 Here's a summary o ## Bug fixes -### 2.9.1 (2023-09-14) - -* Update Docker base images to mitigate security vulnerability CVE-2022-48174 -* Fix bugs in indexshipper (`tsdb`, `boltdb-shipper`) that could result in not showing all ingested logs in query results. - ### 2.9.2 (2023-10-16) * Upgrade go to v1.21.3, golang.org/x/net to v0.17.0 and grpc-go to v1.56.3 to patch CVE-2023-39325 / CVE-2023-44487 For a full list of all changes and fixes, look at the [CHANGELOG](https://github.com/grafana/loki/blob/release-2.9.x/CHANGELOG.md). + +### 2.9.1 (2023-09-14) + +* Update Docker base images to mitigate security vulnerability CVE-2022-48174 +* Fix bugs in indexshipper (`tsdb`, `boltdb-shipper`) that could result in not showing all ingested logs in query results. diff --git a/docs/sources/send-data/otel/_index.md b/docs/sources/send-data/otel/_index.md index 84d1226316ecf..12f9cdd0e4af5 100644 --- a/docs/sources/send-data/otel/_index.md +++ b/docs/sources/send-data/otel/_index.md @@ -9,10 +9,6 @@ weight: 250 # Ingesting logs to Loki using OpenTelemetry Collector -{{% admonition type="warning" %}} -OpenTelemetry logs ingestion is an experimental feature and is subject to change in future releases of Grafana Loki. -{{% /admonition %}} - Loki natively supports ingesting OpenTelemetry logs over HTTP. For ingesting logs to Loki using the OpenTelemetry Collector, you must use the [`otlphttp` exporter](https://github.com/open-telemetry/opentelemetry-collector/tree/main/exporter/otlphttpexporter). From ba0538c2a509e55ae4ece3482591c9a8e8ded3a9 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 7 Feb 2024 08:21:58 -0800 Subject: [PATCH 44/45] [Blooms] tsdb iterator implementation for bloom construction (#11886) Builds a lazy iterator over a tsdb file for bloom construction. This is just the library utility, not the integration. --- pkg/bloomcompactor/tsdb.go | 111 ++++++++++++++++++ pkg/bloomcompactor/tsdb_test.go | 86 ++++++++++++++ .../indexshipper/tsdb/single_file_index.go | 2 + 3 files changed, 199 insertions(+) create mode 100644 pkg/bloomcompactor/tsdb.go create mode 100644 pkg/bloomcompactor/tsdb_test.go diff --git a/pkg/bloomcompactor/tsdb.go b/pkg/bloomcompactor/tsdb.go new file mode 100644 index 0000000000000..bb4383cc84f60 --- /dev/null +++ b/pkg/bloomcompactor/tsdb.go @@ -0,0 +1,111 @@ +package bloomcompactor + +import ( + "context" + "math" + + "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/tsdb/index" +) + +// TSDBStore is an interface for interacting with the TSDB, +// modeled off a relevant subset of the `tsdb.TSDBIndex` struct +type forSeries interface { + ForSeries( + ctx context.Context, + fpFilter index.FingerprintFilter, + from model.Time, + through model.Time, + fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta), + matchers ...*labels.Matcher, + ) error + Close() error +} + +type TSDBSeriesIter struct { + f forSeries + bounds v1.FingerprintBounds + ctx context.Context + + ch chan *v1.Series + initialized bool + next *v1.Series + err error +} + +func NewTSDBSeriesIter(ctx context.Context, f forSeries, bounds v1.FingerprintBounds) *TSDBSeriesIter { + return &TSDBSeriesIter{ + f: f, + bounds: bounds, + ctx: ctx, + ch: make(chan *v1.Series), + } +} + +func (t *TSDBSeriesIter) Next() bool { + if !t.initialized { + t.initialized = true + t.background() + } + + select { + case <-t.ctx.Done(): + return false + case next, ok := <-t.ch: + t.next = next + return ok + } +} + +func (t *TSDBSeriesIter) At() *v1.Series { + return t.next +} + +func (t *TSDBSeriesIter) Err() error { + if t.err != nil { + return t.err + } + + return t.ctx.Err() +} + +func (t *TSDBSeriesIter) Close() error { + return t.f.Close() +} + +// background iterates over the tsdb file, populating the next +// value via a channel to handle backpressure +func (t *TSDBSeriesIter) background() { + go func() { + t.err = t.f.ForSeries( + t.ctx, + t.bounds, + 0, math.MaxInt64, + func(_ labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta) { + + res := &v1.Series{ + Fingerprint: fp, + Chunks: make(v1.ChunkRefs, 0, len(chks)), + } + for _, chk := range chks { + res.Chunks = append(res.Chunks, v1.ChunkRef{ + Start: model.Time(chk.MinTime), + End: model.Time(chk.MaxTime), + Checksum: chk.Checksum, + }) + } + + select { + case <-t.ctx.Done(): + return + case t.ch <- res: + } + }, + labels.MustNewMatcher(labels.MatchEqual, "", ""), + ) + close(t.ch) + }() +} diff --git a/pkg/bloomcompactor/tsdb_test.go b/pkg/bloomcompactor/tsdb_test.go new file mode 100644 index 0000000000000..08f301758bf53 --- /dev/null +++ b/pkg/bloomcompactor/tsdb_test.go @@ -0,0 +1,86 @@ +package bloomcompactor + +import ( + "context" + "math" + "testing" + + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/labels" + "github.com/stretchr/testify/require" + + v1 "github.com/grafana/loki/pkg/storage/bloom/v1" + "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb/index" +) + +type forSeriesTestImpl []*v1.Series + +func (f forSeriesTestImpl) ForSeries( + _ context.Context, + _ index.FingerprintFilter, + _ model.Time, + _ model.Time, + fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta), + _ ...*labels.Matcher, +) error { + for i := range f { + unmapped := make([]index.ChunkMeta, 0, len(f[i].Chunks)) + for _, c := range f[i].Chunks { + unmapped = append(unmapped, index.ChunkMeta{ + MinTime: int64(c.Start), + MaxTime: int64(c.End), + Checksum: c.Checksum, + }) + } + + fn(nil, f[i].Fingerprint, unmapped) + } + return nil +} + +func (f forSeriesTestImpl) Close() error { + return nil +} + +func TestTSDBSeriesIter(t *testing.T) { + input := []*v1.Series{ + { + Fingerprint: 1, + Chunks: []v1.ChunkRef{ + { + Start: 0, + End: 1, + Checksum: 2, + }, + { + Start: 3, + End: 4, + Checksum: 5, + }, + }, + }, + } + srcItr := v1.NewSliceIter(input) + itr := NewTSDBSeriesIter(context.Background(), forSeriesTestImpl(input), v1.NewBounds(0, math.MaxUint64)) + + v1.EqualIterators[*v1.Series]( + t, + func(a, b *v1.Series) { + require.Equal(t, a, b) + }, + itr, + srcItr, + ) +} + +func TestTSDBSeriesIter_Expiry(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + cancel() + itr := NewTSDBSeriesIter(ctx, forSeriesTestImpl{ + {}, // a single entry + }, v1.NewBounds(0, math.MaxUint64)) + + require.False(t, itr.Next()) + require.Error(t, itr.Err()) + +} diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go b/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go index 0e1ae029a8677..63809f6b1356e 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/single_file_index.go @@ -157,6 +157,8 @@ func (i *TSDBIndex) SetChunkFilterer(chunkFilter chunk.RequestChunkFilterer) { // fn must NOT capture it's arguments. They're reused across series iterations and returned to // a pool after completion. +// TODO(owen-d): have callback return a bool whether to continue or not in order to short-circuit +// when applicable func (i *TSDBIndex) ForSeries(ctx context.Context, fpFilter index.FingerprintFilter, from model.Time, through model.Time, fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta), matchers ...*labels.Matcher) error { // TODO(owen-d): use pool From b26bd6d77be4e38c8273a9d88429acb463c83596 Mon Sep 17 00:00:00 2001 From: Derek Cadzow Date: Wed, 7 Feb 2024 12:33:14 -0500 Subject: [PATCH 45/45] changed first person to comply with style guide (#11890) **What this PR does / why we need it**: changed first person to comply with style guide **Which issue(s) this PR fixes**: Fixes # No issue **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/setup/install/docker.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sources/setup/install/docker.md b/docs/sources/setup/install/docker.md index 51df7f9288c85..8e9007c0fdf43 100644 --- a/docs/sources/setup/install/docker.md +++ b/docs/sources/setup/install/docker.md @@ -9,7 +9,7 @@ weight: 400 # Install Loki with Docker or Docker Compose You can install Loki and Promtail with Docker or Docker Compose if you are evaluating, testing, or developing Loki. -For production, we recommend installing with Tanka or Helm. +For production, Grafana recommends installing with Tanka or Helm. The configuration acquired with these installation instructions run Loki as a single binary.