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: