From b2e4cc39a88409c17215ac484d7c98adcfe53750 Mon Sep 17 00:00:00 2001 From: Paul Rogers <129207811+paul1r@users.noreply.github.com> Date: Tue, 13 Feb 2024 12:45:15 -0500 Subject: [PATCH 01/19] Add mutex for t.stopped to prevent data races (#11933) **What this PR does / why we need it**: This addresses the data race present on the `t.stopped` variable in `tail.go`. ``` ================== WARNING: DATA RACE Write at 0x00c00098b198 by goroutine 568: github.com/grafana/loki/pkg/querier.(*Tailer).close() /Users/progers/dev/src/github.com/grafana/loki/pkg/querier/tail.go:272 +0x104 github.com/grafana/loki/pkg/querier.TestTailer.func7.2() /Users/progers/dev/src/github.com/grafana/loki/pkg/querier/tail_test.go:169 +0x34 runtime.deferreturn() /opt/homebrew/Cellar/go/1.21.6/libexec/src/runtime/panic.go:477 +0x34 testing.tRunner() /opt/homebrew/Cellar/go/1.21.6/libexec/src/testing/testing.go:1595 +0x1b0 testing.(*T).Run.func1() /opt/homebrew/Cellar/go/1.21.6/libexec/src/testing/testing.go:1648 +0x40 Previous read at 0x00c00098b198 by goroutine 569: github.com/grafana/loki/pkg/querier.(*Tailer).loop() /Users/progers/dev/src/github.com/grafana/loki/pkg/querier/tail.go:88 +0x13c github.com/grafana/loki/pkg/querier.newTailer.func1() /Users/progers/dev/src/github.com/grafana/loki/pkg/querier/tail.go:342 +0x34 Goroutine 568 (running) created at: testing.(*T).Run() /opt/homebrew/Cellar/go/1.21.6/libexec/src/testing/testing.go:1648 +0x5e8 github.com/grafana/loki/pkg/querier.TestTailer() /Users/progers/dev/src/github.com/grafana/loki/pkg/querier/tail_test.go:158 +0x10dc testing.tRunner() /opt/homebrew/Cellar/go/1.21.6/libexec/src/testing/testing.go:1595 +0x1b0 testing.(*T).Run.func1() /opt/homebrew/Cellar/go/1.21.6/libexec/src/testing/testing.go:1648 +0x40 Goroutine 569 (running) created at: github.com/grafana/loki/pkg/querier.newTailer() /Users/progers/dev/src/github.com/grafana/loki/pkg/querier/tail.go:342 +0x300 github.com/grafana/loki/pkg/querier.TestTailer.func7() /Users/progers/dev/src/github.com/grafana/loki/pkg/querier/tail_test.go:168 +0x138 testing.tRunner() /opt/homebrew/Cellar/go/1.21.6/libexec/src/testing/testing.go:1595 +0x1b0 testing.(*T).Run.func1() /opt/homebrew/Cellar/go/1.21.6/libexec/src/testing/testing.go:1648 +0x40 ================== ``` **Which issue(s) this PR fixes**: Relates to: https://github.com/grafana/loki/issues/8586 **Special notes for your reviewer**: **Checklist** - [ ] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] Documentation added - [ ] Tests updated - [ ] `CHANGELOG.md` updated - [ ] If the change is worth mentioning in the release notes, add `add-to-release-notes` label - [ ] Changes that require user attention or interaction to upgrade are documented in `docs/sources/setup/upgrade/_index.md` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213) - [ ] If the change is deprecating or removing a configuration option, update the `deprecated-config.yaml` and `deleted-config.yaml` files respectively in the `tools/deprecated-config-checker` directory. [Example PR](https://github.com/grafana/loki/pull/10840/commits/0d4416a4b03739583349934b96f272fb4f685d15) --- pkg/querier/http.go | 2 +- pkg/querier/tail.go | 15 ++++++++++----- pkg/querier/tail_test.go | 2 +- 3 files changed, 12 insertions(+), 7 deletions(-) diff --git a/pkg/querier/http.go b/pkg/querier/http.go index dc29c2f61e04f..a508bf9f7286b 100644 --- a/pkg/querier/http.go +++ b/pkg/querier/http.go @@ -199,7 +199,7 @@ func (q *QuerierAPI) TailHandler(w http.ResponseWriter, r *http.Request) { } level.Error(logger).Log("msg", "Error from client", "err", err) break - } else if tailer.stopped { + } else if tailer.stopped.Load() { return } diff --git a/pkg/querier/tail.go b/pkg/querier/tail.go index 1b3cfd5fcb8c0..35cb4bc18e7a7 100644 --- a/pkg/querier/tail.go +++ b/pkg/querier/tail.go @@ -6,6 +6,8 @@ import ( "sync" "time" + "go.uber.org/atomic" + "github.com/go-kit/log" "github.com/go-kit/log/level" "github.com/pkg/errors" @@ -51,7 +53,7 @@ type Tailer struct { querierTailClients map[string]logproto.Querier_TailClient // addr -> grpc clients for tailing logs from ingesters querierTailClientsMtx sync.RWMutex - stopped bool + stopped atomic.Bool delayFor time.Duration responseChan chan *loghttp.TailResponse closeErrChan chan error @@ -85,7 +87,8 @@ func (t *Tailer) loop() { droppedEntries := make([]loghttp.DroppedEntry, 0) - for !t.stopped { + stopped := t.stopped.Load() + for !stopped { select { case <-checkConnectionTicker.C: // Try to reconnect dropped ingesters and connect to new ingesters @@ -214,7 +217,8 @@ func (t *Tailer) readTailClient(addr string, querierTailClient logproto.Querier_ logger := util_log.WithContext(querierTailClient.Context(), t.logger) for { - if t.stopped { + stopped := t.stopped.Load() + if stopped { if err := querierTailClient.CloseSend(); err != nil { level.Error(logger).Log("msg", "Error closing grpc tail client", "err", err) } @@ -223,7 +227,7 @@ func (t *Tailer) readTailClient(addr string, querierTailClient logproto.Querier_ resp, err = querierTailClient.Recv() if err != nil { // We don't want to log error when its due to stopping the tail request - if !t.stopped { + if !stopped { level.Error(logger).Log("msg", "Error receiving response from grpc tail client", "err", err) } break @@ -269,7 +273,8 @@ func (t *Tailer) close() error { t.metrics.tailsActive.Dec() t.metrics.tailedStreamsActive.Sub(t.activeStreamCount()) - t.stopped = true + t.stopped.Store(true) + return t.openStreamIterator.Close() } diff --git a/pkg/querier/tail_test.go b/pkg/querier/tail_test.go index d0b17ea126e2a..07d3743af03c5 100644 --- a/pkg/querier/tail_test.go +++ b/pkg/querier/tail_test.go @@ -389,7 +389,7 @@ func readFromTailer(tailer *Tailer, maxEntries int) ([]*loghttp.TailResponse, er timeoutTicker := time.NewTicker(timeout) defer timeoutTicker.Stop() - for !tailer.stopped && entriesCount < maxEntries { + for !tailer.stopped.Load() && entriesCount < maxEntries { select { case <-timeoutTicker.C: return nil, errors.New("timeout expired while reading responses from Tailer") From 25785e03d5f529443ecef331e176999308cc65b0 Mon Sep 17 00:00:00 2001 From: Derek Cadzow Date: Tue, 13 Feb 2024 15:20:48 -0500 Subject: [PATCH 02/19] Deleting old and irrelevant information (#11929) --- docs/variables.mk | 3 --- 1 file changed, 3 deletions(-) diff --git a/docs/variables.mk b/docs/variables.mk index afa0a9e867366..1ec7dbab57677 100644 --- a/docs/variables.mk +++ b/docs/variables.mk @@ -1,8 +1,5 @@ # List of projects to provide to the make-docs script. PROJECTS := loki -# Use alternative image until make-docs 3.0.0 is rolled out. -export DOCS_IMAGE := grafana/docs-base:dbd975af06 - # Set the DOC_VALIDATOR_IMAGE to match the one defined in CI. export DOC_VALIDATOR_IMAGE := $(shell sed -En 's, *image: "(grafana/doc-validator.*)",\1,p' "$(shell git rev-parse --show-toplevel)/.github/workflows/doc-validator.yml") From 1c43991ddcbb801a6a6d7a535062c7c615b0423a Mon Sep 17 00:00:00 2001 From: Christian Haudum Date: Wed, 14 Feb 2024 09:49:11 +0100 Subject: [PATCH 03/19] Remove `bloomcompactor.DayTable` in favour of `config.DayTime` (#11917) Both structs shared the same semantics. This PR moves additional functionality from the `DayTable` to the `DayTime` struct. To get the table name of a day (ordinal number of day since unix epoch), call `DayTime.Table()`. Signed-off-by: Christian Haudum --- pkg/bloomcompactor/bloomcompactor.go | 15 +++---- pkg/bloomcompactor/config.go | 38 ---------------- pkg/bloomcompactor/controller.go | 13 +++--- pkg/bloomcompactor/tsdb.go | 34 +++++++------- pkg/bloomgateway/bloomgateway.go | 2 +- pkg/bloomgateway/multiplexing.go | 7 +-- pkg/bloomgateway/processor.go | 10 ++--- pkg/bloomgateway/processor_test.go | 5 ++- pkg/bloomgateway/util.go | 5 ++- pkg/bloomgateway/util_test.go | 12 ++--- pkg/querier/queryrange/limits.go | 4 +- pkg/storage/config/schema_config.go | 45 ++++++++++++++++++- .../stores/shipper/bloomshipper/client.go | 2 +- 13 files changed, 98 insertions(+), 94 deletions(-) diff --git a/pkg/bloomcompactor/bloomcompactor.go b/pkg/bloomcompactor/bloomcompactor.go index ed1f50ae72582..566b836609d10 100644 --- a/pkg/bloomcompactor/bloomcompactor.go +++ b/pkg/bloomcompactor/bloomcompactor.go @@ -169,11 +169,11 @@ func runWithRetries( type tenantTable struct { tenant string - table DayTable + table config.DayTime ownershipRange v1.FingerprintBounds } -func (c *Compactor) tenants(ctx context.Context, table DayTable) (v1.Iterator[string], error) { +func (c *Compactor) tenants(ctx context.Context, table config.DayTime) (v1.Iterator[string], error) { tenants, err := c.tsdbStore.UsersForPeriod(ctx, table) if err != nil { return nil, errors.Wrap(err, "getting tenants") @@ -214,10 +214,9 @@ func (c *Compactor) tables(ts time.Time) *dayRangeIterator { from := ts.Add(-maxCompactionPeriod).UnixNano() / int64(config.ObjectStorageIndexRequiredPeriod) * int64(config.ObjectStorageIndexRequiredPeriod) through := ts.Add(-minCompactionPeriod).UnixNano() / int64(config.ObjectStorageIndexRequiredPeriod) * int64(config.ObjectStorageIndexRequiredPeriod) - fromDay := DayTable(model.TimeFromUnixNano(from)) - throughDay := DayTable(model.TimeFromUnixNano(through)) + fromDay := config.NewDayTime(model.TimeFromUnixNano(from)) + throughDay := config.NewDayTime(model.TimeFromUnixNano(through)) return newDayRangeIterator(fromDay, throughDay) - } func (c *Compactor) loadWork(ctx context.Context, ch chan<- tenantTable) error { @@ -295,10 +294,10 @@ func (c *Compactor) compactTenantTable(ctx context.Context, tt tenantTable) erro } type dayRangeIterator struct { - min, max, cur DayTable + min, max, cur config.DayTime } -func newDayRangeIterator(min, max DayTable) *dayRangeIterator { +func newDayRangeIterator(min, max config.DayTime) *dayRangeIterator { return &dayRangeIterator{min: min, max: max, cur: min.Dec()} } @@ -307,7 +306,7 @@ func (r *dayRangeIterator) Next() bool { return r.cur.Before(r.max) } -func (r *dayRangeIterator) At() DayTable { +func (r *dayRangeIterator) At() config.DayTime { return r.cur } diff --git a/pkg/bloomcompactor/config.go b/pkg/bloomcompactor/config.go index dd821d81c906b..15f9aa86c040f 100644 --- a/pkg/bloomcompactor/config.go +++ b/pkg/bloomcompactor/config.go @@ -5,10 +5,6 @@ import ( "fmt" "time" - "github.com/prometheus/common/model" - - "github.com/grafana/loki/pkg/storage/config" - "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper" "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/downloads" "github.com/grafana/loki/pkg/util/ring" ) @@ -70,37 +66,3 @@ type Limits interface { BloomFalsePositiveRate(tenantID string) float64 BloomCompactorMaxBlockSize(tenantID string) int } - -// TODO(owen-d): Remove this type in favor of config.DayTime -type DayTable model.Time - -func (d DayTable) String() string { - return fmt.Sprintf("%d", d.ModelTime().Time().UnixNano()/int64(config.ObjectStorageIndexRequiredPeriod)) -} - -func (d DayTable) Inc() DayTable { - return DayTable(d.ModelTime().Add(config.ObjectStorageIndexRequiredPeriod)) -} - -func (d DayTable) Dec() DayTable { - return DayTable(d.ModelTime().Add(-config.ObjectStorageIndexRequiredPeriod)) -} - -func (d DayTable) Before(other DayTable) bool { - return d.ModelTime().Before(model.Time(other)) -} - -func (d DayTable) After(other DayTable) bool { - return d.ModelTime().After(model.Time(other)) -} - -func (d DayTable) ModelTime() model.Time { - return model.Time(d) -} - -func (d DayTable) Bounds() bloomshipper.Interval { - return bloomshipper.Interval{ - Start: model.Time(d), - End: model.Time(d.Inc()), - } -} diff --git a/pkg/bloomcompactor/controller.go b/pkg/bloomcompactor/controller.go index 47d9627d92e1a..8470fd9ad7082 100644 --- a/pkg/bloomcompactor/controller.go +++ b/pkg/bloomcompactor/controller.go @@ -13,6 +13,7 @@ import ( "github.com/pkg/errors" 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" "github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/tsdb" ) @@ -55,7 +56,7 @@ func (s *SimpleBloomController) rwFn() (v1.BlockWriter, v1.BlockReader) { func (s *SimpleBloomController) buildBlocks( ctx context.Context, - table DayTable, + table config.DayTime, tenant string, ownershipRange v1.FingerprintBounds, ) error { @@ -78,15 +79,11 @@ func (s *SimpleBloomController) buildBlocks( } // 2. Fetch metas - bounds := table.Bounds() metas, err := s.bloomStore.FetchMetas( ctx, bloomshipper.MetaSearchParams{ TenantID: tenant, - Interval: bloomshipper.Interval{ - Start: bounds.Start, - End: bounds.End, - }, + Interval: bloomshipper.NewInterval(table.Bounds()), Keyspace: ownershipRange, }, ) @@ -176,7 +173,7 @@ func (s *SimpleBloomController) buildBlocks( blockCt++ blk := newBlocks.At() - built, err := bloomshipper.BlockFrom(tenant, table.String(), blk) + built, err := bloomshipper.BlockFrom(tenant, table.Table(), blk) if err != nil { level.Error(logger).Log("msg", "failed to build block", "err", err) return errors.Wrap(err, "failed to build block") @@ -214,7 +211,7 @@ func (s *SimpleBloomController) buildBlocks( func (s *SimpleBloomController) loadWorkForGap( ctx context.Context, - table DayTable, + table config.DayTime, tenant string, id tsdb.Identifier, gap gapWithBlocks, diff --git a/pkg/bloomcompactor/tsdb.go b/pkg/bloomcompactor/tsdb.go index e6fd92961c46c..ad7b2eafac4cd 100644 --- a/pkg/bloomcompactor/tsdb.go +++ b/pkg/bloomcompactor/tsdb.go @@ -26,11 +26,11 @@ const ( ) type TSDBStore interface { - UsersForPeriod(ctx context.Context, table DayTable) ([]string, error) - ResolveTSDBs(ctx context.Context, table DayTable, tenant string) ([]tsdb.SingleTenantTSDBIdentifier, error) + UsersForPeriod(ctx context.Context, table config.DayTime) ([]string, error) + ResolveTSDBs(ctx context.Context, table config.DayTime, tenant string) ([]tsdb.SingleTenantTSDBIdentifier, error) LoadTSDB( ctx context.Context, - table DayTable, + table config.DayTime, tenant string, id tsdb.Identifier, bounds v1.FingerprintBounds, @@ -49,13 +49,13 @@ func NewBloomTSDBStore(storage storage.Client) *BloomTSDBStore { } } -func (b *BloomTSDBStore) UsersForPeriod(ctx context.Context, table DayTable) ([]string, error) { - _, users, err := b.storage.ListFiles(ctx, table.String(), true) // bypass cache for ease of testing +func (b *BloomTSDBStore) UsersForPeriod(ctx context.Context, table config.DayTime) ([]string, error) { + _, users, err := b.storage.ListFiles(ctx, table.Table(), true) // bypass cache for ease of testing return users, err } -func (b *BloomTSDBStore) ResolveTSDBs(ctx context.Context, table DayTable, tenant string) ([]tsdb.SingleTenantTSDBIdentifier, error) { - indices, err := b.storage.ListUserFiles(ctx, table.String(), tenant, true) // bypass cache for ease of testing +func (b *BloomTSDBStore) ResolveTSDBs(ctx context.Context, table config.DayTime, tenant string) ([]tsdb.SingleTenantTSDBIdentifier, error) { + indices, err := b.storage.ListUserFiles(ctx, table.Table(), tenant, true) // bypass cache for ease of testing if err != nil { return nil, errors.Wrap(err, "failed to list user files") } @@ -80,14 +80,14 @@ func (b *BloomTSDBStore) ResolveTSDBs(ctx context.Context, table DayTable, tenan func (b *BloomTSDBStore) LoadTSDB( ctx context.Context, - table DayTable, + table config.DayTime, tenant string, id tsdb.Identifier, bounds v1.FingerprintBounds, ) (v1.CloseableIterator[*v1.Series], error) { withCompression := id.Name() + gzipExtension - data, err := b.storage.GetUserFile(ctx, table.String(), tenant, withCompression) + data, err := b.storage.GetUserFile(ctx, table.Table(), tenant, withCompression) if err != nil { return nil, errors.Wrap(err, "failed to get file") } @@ -244,11 +244,11 @@ func NewTSDBStores( return res, nil } -func (s *TSDBStores) storeForPeriod(table DayTable) (TSDBStore, error) { +func (s *TSDBStores) storeForPeriod(table config.DayTime) (TSDBStore, error) { for i := len(s.schemaCfg.Configs) - 1; i >= 0; i-- { period := s.schemaCfg.Configs[i] - if !table.Before(DayTable(period.From.Time)) { + if !table.Before(period.From) { // we have the desired period config if s.stores[i] != nil { @@ -260,19 +260,19 @@ func (s *TSDBStores) storeForPeriod(table DayTable) (TSDBStore, error) { return nil, errors.Errorf( "store for period is not of TSDB type (%s) while looking up store for (%v)", period.IndexType, - table.ModelTime().Time(), + table, ) } } return nil, fmt.Errorf( - "There is no store matching no matching period found for table (%v) -- too early", - table.ModelTime().Time(), + "there is no store matching no matching period found for table (%v) -- too early", + table, ) } -func (s *TSDBStores) UsersForPeriod(ctx context.Context, table DayTable) ([]string, error) { +func (s *TSDBStores) UsersForPeriod(ctx context.Context, table config.DayTime) ([]string, error) { store, err := s.storeForPeriod(table) if err != nil { return nil, err @@ -281,7 +281,7 @@ func (s *TSDBStores) UsersForPeriod(ctx context.Context, table DayTable) ([]stri return store.UsersForPeriod(ctx, table) } -func (s *TSDBStores) ResolveTSDBs(ctx context.Context, table DayTable, tenant string) ([]tsdb.SingleTenantTSDBIdentifier, error) { +func (s *TSDBStores) ResolveTSDBs(ctx context.Context, table config.DayTime, tenant string) ([]tsdb.SingleTenantTSDBIdentifier, error) { store, err := s.storeForPeriod(table) if err != nil { return nil, err @@ -292,7 +292,7 @@ func (s *TSDBStores) ResolveTSDBs(ctx context.Context, table DayTable, tenant st func (s *TSDBStores) LoadTSDB( ctx context.Context, - table DayTable, + table config.DayTime, tenant string, id tsdb.Identifier, bounds v1.FingerprintBounds, diff --git a/pkg/bloomgateway/bloomgateway.go b/pkg/bloomgateway/bloomgateway.go index abecbf6773fd3..58f709f0be2f8 100644 --- a/pkg/bloomgateway/bloomgateway.go +++ b/pkg/bloomgateway/bloomgateway.go @@ -369,7 +369,7 @@ func (g *Gateway) FilterChunkRefs(ctx context.Context, req *logproto.FilterChunk tasksCh := make(chan Task, len(tasks)) for _, task := range tasks { task := task - level.Info(logger).Log("msg", "enqueue task", "task", task.ID, "day", task.day, "series", len(task.series)) + level.Info(logger).Log("msg", "enqueue task", "task", task.ID, "table", task.table, "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) diff --git a/pkg/bloomgateway/multiplexing.go b/pkg/bloomgateway/multiplexing.go index d2722ad8f1496..c952c9f6b87fd 100644 --- a/pkg/bloomgateway/multiplexing.go +++ b/pkg/bloomgateway/multiplexing.go @@ -12,6 +12,7 @@ import ( "github.com/grafana/loki/pkg/logproto" "github.com/grafana/loki/pkg/logql/syntax" v1 "github.com/grafana/loki/pkg/storage/bloom/v1" + "github.com/grafana/loki/pkg/storage/config" ) const ( @@ -69,7 +70,7 @@ type Task struct { ctx context.Context // TODO(chaudum): Investigate how to remove that. - day model.Time + table config.DayTime } // NewTask returns a new Task that can be enqueued to the task queue. @@ -89,7 +90,7 @@ func NewTask(ctx context.Context, tenantID string, refs seriesWithBounds, filter filters: filters, series: refs.series, bounds: refs.bounds, - day: refs.day, + table: refs.table, ctx: ctx, done: make(chan struct{}), responses: make([]v1.Output, 0, len(refs.series)), @@ -129,7 +130,7 @@ func (t Task) Copy(series []*logproto.GroupedChunkRefs) Task { filters: t.filters, series: series, bounds: t.bounds, - day: t.day, + table: t.table, ctx: t.ctx, done: make(chan struct{}), responses: make([]v1.Output, 0, len(series)), diff --git a/pkg/bloomgateway/processor.go b/pkg/bloomgateway/processor.go index 4fe9c38483cbe..5eab7a858c74b 100644 --- a/pkg/bloomgateway/processor.go +++ b/pkg/bloomgateway/processor.go @@ -7,9 +7,9 @@ import ( "time" "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/config" "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper" ) @@ -35,10 +35,9 @@ type processor struct { } func (p *processor) run(ctx context.Context, tasks []Task) error { - for ts, tasks := range group(tasks, func(t Task) model.Time { return t.day }) { - interval := bloomshipper.NewInterval(ts, ts.Add(Day)) + for ts, tasks := range group(tasks, func(t Task) config.DayTime { return t.table }) { tenant := tasks[0].Tenant - err := p.processTasks(ctx, tenant, interval, []v1.FingerprintBounds{{Min: 0, Max: math.MaxUint64}}, tasks) + err := p.processTasks(ctx, tenant, ts, []v1.FingerprintBounds{{Min: 0, Max: math.MaxUint64}}, tasks) if err != nil { for _, task := range tasks { task.CloseWithError(err) @@ -52,8 +51,9 @@ func (p *processor) run(ctx context.Context, tasks []Task) error { return nil } -func (p *processor) processTasks(ctx context.Context, tenant string, interval bloomshipper.Interval, keyspaces []v1.FingerprintBounds, tasks []Task) error { +func (p *processor) processTasks(ctx context.Context, tenant string, day config.DayTime, keyspaces []v1.FingerprintBounds, tasks []Task) error { minFpRange, maxFpRange := getFirstLast(keyspaces) + interval := bloomshipper.NewInterval(day.Bounds()) metaSearch := bloomshipper.MetaSearchParams{ TenantID: tenant, Interval: interval, diff --git a/pkg/bloomgateway/processor_test.go b/pkg/bloomgateway/processor_test.go index c4c8f8457b3a1..27d0068753d5b 100644 --- a/pkg/bloomgateway/processor_test.go +++ b/pkg/bloomgateway/processor_test.go @@ -15,6 +15,7 @@ import ( "go.uber.org/atomic" "github.com/grafana/loki/pkg/logql/syntax" + "github.com/grafana/loki/pkg/storage/config" "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper" "github.com/grafana/loki/pkg/util/constants" ) @@ -109,7 +110,7 @@ func TestProcessor(t *testing.T) { Start: now.Add(-1 * time.Hour), End: now, }, - day: truncateDay(now), + table: config.NewDayTime(truncateDay(now)), } filters := []syntax.LineFilter{ {Ty: 0, Match: "no match"}, @@ -153,7 +154,7 @@ func TestProcessor(t *testing.T) { Start: now.Add(-1 * time.Hour), End: now, }, - day: truncateDay(now), + table: config.NewDayTime(truncateDay(now)), } filters := []syntax.LineFilter{ {Ty: 0, Match: "no match"}, diff --git a/pkg/bloomgateway/util.go b/pkg/bloomgateway/util.go index cf72aec3b5b4b..3793076f7c385 100644 --- a/pkg/bloomgateway/util.go +++ b/pkg/bloomgateway/util.go @@ -11,6 +11,7 @@ import ( "github.com/grafana/loki/pkg/logproto" "github.com/grafana/loki/pkg/logql/syntax" 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" ) @@ -121,7 +122,7 @@ func partitionFingerprintRange(tasks []Task, blocks []bloomshipper.BlockRef) (re type seriesWithBounds struct { bounds model.Interval - day model.Time + table config.DayTime series []*logproto.GroupedChunkRefs } @@ -173,7 +174,7 @@ func partitionRequest(req *logproto.FilterChunkRefRequest) []seriesWithBounds { Start: minTs, End: maxTs, }, - day: day, + table: config.NewDayTime(day), series: res, }) } diff --git a/pkg/bloomgateway/util_test.go b/pkg/bloomgateway/util_test.go index 8fc37f20bac8e..5f4d254e8f045 100644 --- a/pkg/bloomgateway/util_test.go +++ b/pkg/bloomgateway/util_test.go @@ -176,7 +176,7 @@ func TestPartitionRequest(t *testing.T) { exp: []seriesWithBounds{ { bounds: model.Interval{Start: ts.Add(-60 * time.Minute), End: ts.Add(-45 * time.Minute)}, - day: mktime("2024-01-24 00:00"), + table: config.NewDayTime(mktime("2024-01-24 00:00")), series: []*logproto.GroupedChunkRefs{ { Fingerprint: 0x00, @@ -217,7 +217,7 @@ func TestPartitionRequest(t *testing.T) { exp: []seriesWithBounds{ { bounds: model.Interval{Start: ts.Add(-23 * time.Hour), End: ts.Add(-22 * time.Hour)}, - day: mktime("2024-01-23 00:00"), + table: config.NewDayTime(mktime("2024-01-23 00:00")), series: []*logproto.GroupedChunkRefs{ { Fingerprint: 0x00, @@ -229,7 +229,7 @@ func TestPartitionRequest(t *testing.T) { }, { bounds: model.Interval{Start: ts.Add(-2 * time.Hour), End: ts.Add(-1 * time.Hour)}, - day: mktime("2024-01-24 00:00"), + table: config.NewDayTime(mktime("2024-01-24 00:00")), series: []*logproto.GroupedChunkRefs{ { Fingerprint: 0x01, @@ -258,7 +258,7 @@ func TestPartitionRequest(t *testing.T) { exp: []seriesWithBounds{ { bounds: model.Interval{Start: ts.Add(-13 * time.Hour), End: ts.Add(-11 * time.Hour)}, - day: mktime("2024-01-23 00:00"), + table: config.NewDayTime(mktime("2024-01-23 00:00")), series: []*logproto.GroupedChunkRefs{ { Fingerprint: 0x00, @@ -270,7 +270,7 @@ func TestPartitionRequest(t *testing.T) { }, { bounds: model.Interval{Start: ts.Add(-13 * time.Hour), End: ts.Add(-11 * time.Hour)}, - day: mktime("2024-01-24 00:00"), + table: config.NewDayTime(mktime("2024-01-24 00:00")), series: []*logproto.GroupedChunkRefs{ { Fingerprint: 0x00, @@ -311,7 +311,7 @@ func createBlocks(t *testing.T, tenant string, n int, from, through model.Time, } ref := bloomshipper.Ref{ TenantID: tenant, - TableName: "table_0", + TableName: config.NewDayTime(truncateDay(from)).Table(), Bounds: v1.NewBounds(fromFp, throughFp), StartTimestamp: from, EndTimestamp: through, diff --git a/pkg/querier/queryrange/limits.go b/pkg/querier/queryrange/limits.go index 79cc9ad16a36d..2d14531909695 100644 --- a/pkg/querier/queryrange/limits.go +++ b/pkg/querier/queryrange/limits.go @@ -575,7 +575,7 @@ func WeightedParallelism( // config because query is in future // or // there is overlap with current config - finalOrFuture := i == len(configs)-1 || configs[i].From.After(end) + finalOrFuture := i == len(configs)-1 || configs[i].From.Time.After(end) if finalOrFuture { return true } @@ -605,7 +605,7 @@ func WeightedParallelism( var tsdbDur, otherDur time.Duration - for ; i < len(configs) && configs[i].From.Before(end); i++ { + for ; i < len(configs) && configs[i].From.Time.Before(end); i++ { _, from := minMaxModelTime(start, configs[i].From.Time) through := end if i+1 < len(configs) { diff --git a/pkg/storage/config/schema_config.go b/pkg/storage/config/schema_config.go index 9cdda249ea520..30b9de98b14ba 100644 --- a/pkg/storage/config/schema_config.go +++ b/pkg/storage/config/schema_config.go @@ -200,6 +200,14 @@ func (cfg *PeriodConfig) GetIndexTableNumberRange(schemaEndDate DayTime) TableRa } } +func (cfg *PeriodConfig) GetFullTableName(t model.Time) string { + return NewDayTime(t).TableWithPrefix(cfg) +} + +func NewDayTime(d model.Time) DayTime { + return DayTime{d} +} + // DayTime is a model.Time what holds day-aligned values, and marshals to/from // YAML in YYYY-MM-DD format. type DayTime struct { @@ -225,10 +233,45 @@ func (d *DayTime) UnmarshalYAML(unmarshal func(interface{}) error) error { return nil } -func (d *DayTime) String() string { +func (d DayTime) String() string { return d.Time.Time().UTC().Format("2006-01-02") } +func (d DayTime) Table() string { + return fmt.Sprintf("%d", + d.ModelTime().Time().UnixNano()/int64(ObjectStorageIndexRequiredPeriod)) +} + +func (d DayTime) TableWithPrefix(cfg *PeriodConfig) string { + return fmt.Sprintf("%s%d", + cfg.IndexTables.Prefix, + d.ModelTime().Time().UnixNano()/int64(ObjectStorageIndexRequiredPeriod)) +} + +func (d DayTime) Inc() DayTime { + return DayTime{d.Add(ObjectStorageIndexRequiredPeriod)} +} + +func (d DayTime) Dec() DayTime { + return DayTime{d.Add(-ObjectStorageIndexRequiredPeriod)} +} + +func (d DayTime) Before(other DayTime) bool { + return d.Time.Before(other.Time) +} + +func (d DayTime) After(other DayTime) bool { + return d.Time.After(other.Time) +} + +func (d DayTime) ModelTime() model.Time { + return d.Time +} + +func (d DayTime) Bounds() (model.Time, model.Time) { + return d.Time, d.Inc().Time +} + // SchemaConfig contains the config for our chunk index schemas type SchemaConfig struct { Configs []PeriodConfig `yaml:"configs"` diff --git a/pkg/storage/stores/shipper/bloomshipper/client.go b/pkg/storage/stores/shipper/bloomshipper/client.go index 80eba70d18cdb..1dbfac579c5aa 100644 --- a/pkg/storage/stores/shipper/bloomshipper/client.go +++ b/pkg/storage/stores/shipper/bloomshipper/client.go @@ -345,7 +345,7 @@ func (b *BloomClient) GetMeta(ctx context.Context, ref MetaRef) (Meta, error) { func findPeriod(configs []config.PeriodConfig, ts model.Time) (config.DayTime, error) { for i := len(configs) - 1; i >= 0; i-- { periodConfig := configs[i] - if !periodConfig.From.After(ts) { + if !periodConfig.From.Time.After(ts) { return periodConfig.From, nil } } From 1b4d23f9b754458a311ff82fd6b1d52134bcc5e7 Mon Sep 17 00:00:00 2001 From: Justin Burnham Date: Wed, 14 Feb 2024 00:59:50 -0800 Subject: [PATCH 04/19] Ruler: Disable x-scope-orgid header append in remote write (#11819) Co-authored-by: Michel Hollands <42814411+MichelHollands@users.noreply.github.com> Co-authored-by: Danny Kopping --- CHANGELOG.md | 1 + docs/sources/configure/_index.md | 4 ++++ pkg/ruler/config.go | 2 ++ pkg/ruler/registry.go | 6 +++-- pkg/ruler/registry_test.go | 39 ++++++++++++++++++++++++++++++++ 5 files changed, 50 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 60b9e3dc5e2c8..68841de451dba 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -6,6 +6,7 @@ ##### Enhancements +* [11819](https://github.com/grafana/loki/pull/11819) **jburnham**: Ruler: Add the ability to disable the `X-Scope-OrgId` tenant identification header in remote write requests. * [11633](https://github.com/grafana/loki/pull/11633) **cyriltovena**: Add profiling integrations to tracing instrumentation. * [11571](https://github.com/grafana/loki/pull/11571) **MichelHollands**: Add a metrics.go log line for requests from querier to ingester * [11477](https://github.com/grafana/loki/pull/11477) **MichelHollands**: support GET for /ingester/shutdown diff --git a/docs/sources/configure/_index.md b/docs/sources/configure/_index.md index d5dd9b43bd146..382890b5bcab7 100644 --- a/docs/sources/configure/_index.md +++ b/docs/sources/configure/_index.md @@ -1274,6 +1274,10 @@ remote_write: # CLI flag: -ruler.remote-write.config-refresh-period [config_refresh_period: | default = 10s] + # Add X-Scope-OrgID header in remote write requests. + # CLI flag: -ruler.remote-write.add-org-id-header + [add_org_id_header: | default = true] + # Configuration for rule evaluation. evaluation: # The evaluation mode for the ruler. Can be either 'local' or 'remote'. If set diff --git a/pkg/ruler/config.go b/pkg/ruler/config.go index 22a19851a4305..7d948baa0c30d 100644 --- a/pkg/ruler/config.go +++ b/pkg/ruler/config.go @@ -56,6 +56,7 @@ type RemoteWriteConfig struct { Clients map[string]config.RemoteWriteConfig `yaml:"clients,omitempty" doc:"description=Configure remote write clients. A map with remote client id as key."` Enabled bool `yaml:"enabled"` ConfigRefreshPeriod time.Duration `yaml:"config_refresh_period"` + AddOrgIDHeader bool `yaml:"add_org_id_header" doc:"description=Add X-Scope-OrgID header in remote write requests."` } func (c *RemoteWriteConfig) Validate() error { @@ -108,6 +109,7 @@ func (c *RemoteWriteConfig) Clone() (*RemoteWriteConfig, error) { // RegisterFlags adds the flags required to config this to the given FlagSet. func (c *RemoteWriteConfig) RegisterFlags(f *flag.FlagSet) { + f.BoolVar(&c.AddOrgIDHeader, "ruler.remote-write.add-org-id-header", true, "Add X-Scope-OrgID header in remote write requests.") f.BoolVar(&c.Enabled, "ruler.remote-write.enabled", false, "Enable remote-write functionality.") f.DurationVar(&c.ConfigRefreshPeriod, "ruler.remote-write.config-refresh-period", 10*time.Second, "Minimum period to wait between refreshing remote-write reconfigurations. This should be greater than or equivalent to -limits.per-user-override-period.") diff --git a/pkg/ruler/registry.go b/pkg/ruler/registry.go index adb4f7cf8667b..90a68d60c90b1 100644 --- a/pkg/ruler/registry.go +++ b/pkg/ruler/registry.go @@ -212,8 +212,10 @@ func (r *walRegistry) getTenantConfig(tenant string) (instance.Config, error) { } } - // always inject the X-Scope-OrgId header for multi-tenant metrics backends - clt.Headers[user.OrgIDHeaderName] = tenant + if rwCfg.AddOrgIDHeader { + // inject the X-Scope-OrgId header for multi-tenant metrics backends + clt.Headers[user.OrgIDHeaderName] = tenant + } rwCfg.Clients[id] = clt diff --git a/pkg/ruler/registry_test.go b/pkg/ruler/registry_test.go index 9e200e43ad3a2..46ab9a7084576 100644 --- a/pkg/ruler/registry_test.go +++ b/pkg/ruler/registry_test.go @@ -47,6 +47,7 @@ const remote2 = "remote-2" var remoteURL, _ = url.Parse("http://remote-write") var backCompatCfg = Config{ RemoteWrite: RemoteWriteConfig{ + AddOrgIDHeader: true, Client: &config.RemoteWriteConfig{ URL: &promConfig.URL{URL: remoteURL}, QueueConfig: config.QueueConfig{ @@ -105,6 +106,7 @@ var backCompatCfg = Config{ var remoteURL2, _ = url.Parse("http://remote-write2") var cfg = Config{ RemoteWrite: RemoteWriteConfig{ + AddOrgIDHeader: true, Clients: map[string]config.RemoteWriteConfig{ remote1: { URL: &promConfig.URL{URL: remoteURL}, @@ -751,6 +753,43 @@ func TestTenantRemoteWriteHeadersNoOverride(t *testing.T) { assert.ElementsMatch(t, actual, expected, "Headers do not match") } +func TestTenantRemoteWriteHeadersNoOrgIDHeader(t *testing.T) { + backCompatCfg.RemoteWrite.AddOrgIDHeader = false + reg := setupRegistry(t, backCompatCfg, newFakeLimitsBackwardCompat()) + + tenantCfg, err := reg.getTenantConfig(enabledRWTenant) + require.NoError(t, err) + + assert.Len(t, tenantCfg.RemoteWrite[0].Headers, 1) + // ensure that X-Scope-OrgId header is missing + assert.Equal(t, tenantCfg.RemoteWrite[0].Headers[user.OrgIDHeaderName], "") + // the original header must be present + assert.Equal(t, tenantCfg.RemoteWrite[0].Headers["Base"], "value") + + cfg.RemoteWrite.AddOrgIDHeader = false + reg = setupRegistry(t, cfg, newFakeLimits()) + + tenantCfg, err = reg.getTenantConfig(enabledRWTenant) + require.NoError(t, err) + + // Ensure that overrides take plus and that X-Scope-OrgID header is still missing + expected := []map[string]string{ + { + "Base": "value", + }, + { + "Base": "value2", + }, + } + + actual := []map[string]string{} + for _, rw := range tenantCfg.RemoteWrite { + actual = append(actual, rw.Headers) + } + + assert.ElementsMatch(t, actual, expected, "Headers do not match") +} + func TestRelabelConfigOverrides(t *testing.T) { reg := setupRegistry(t, backCompatCfg, newFakeLimitsBackwardCompat()) From 9e7725b31b19792dad692afd9ad7e9804c04bfc1 Mon Sep 17 00:00:00 2001 From: Ashwanth Date: Wed, 14 Feb 2024 16:46:38 +0530 Subject: [PATCH 05/19] feat(metadata): introduce a separate split interval for recent query window (#11897) --- CHANGELOG.md | 1 + docs/sources/configure/_index.md | 24 ++ pkg/bloomgateway/cache.go | 1 + pkg/querier/queryrange/index_stats_cache.go | 1 + pkg/querier/queryrange/labels_cache.go | 33 +- pkg/querier/queryrange/labels_cache_test.go | 334 +++++++--------- pkg/querier/queryrange/limits/definitions.go | 2 + .../queryrangebase/results_cache.go | 2 + .../queryrangebase/results_cache_test.go | 3 + pkg/querier/queryrange/roundtrip.go | 3 +- pkg/querier/queryrange/roundtrip_test.go | 52 ++- pkg/querier/queryrange/series_cache.go | 9 +- pkg/querier/queryrange/series_cache_test.go | 366 +++++++---------- .../queryrange/split_by_interval_test.go | 370 +++++++++++++++++- pkg/querier/queryrange/splitters.go | 69 +++- pkg/querier/queryrange/volume_cache.go | 1 + pkg/storage/chunk/cache/resultscache/cache.go | 24 +- .../chunk/cache/resultscache/cache_test.go | 120 +++++- pkg/validation/limits.go | 37 +- 19 files changed, 955 insertions(+), 497 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 68841de451dba..7f091ed06f883 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -55,6 +55,7 @@ * [11143](https://github.com/grafana/loki/pull/11143) **sandeepsukhani** otel: Add support for per tenant configuration for mapping otlp data to loki format * [11499](https://github.com/grafana/loki/pull/11284) **jmichalek132** Config: Adds `frontend.log-query-request-headers` to enable logging of request headers in query logs. * [11817](https://github.com/grafana/loki/pull/11817) **ashwanthgoli** Ruler: Add support for filtering results of `/prometheus/api/v1/rules` endpoint by rule_name, rule_group, file and type. +* [11897](https://github.com/grafana/loki/pull/11897) **ashwanthgoli** Metadata: Introduces a separate split interval of `split_recent_metadata_queries_by_interval` for `recent_metadata_query_window` to help with caching recent metadata query results. ##### Fixes * [11074](https://github.com/grafana/loki/pull/11074) **hainenber** Fix panic in lambda-promtail due to mishandling of empty DROP_LABELS env var. diff --git a/docs/sources/configure/_index.md b/docs/sources/configure/_index.md index 382890b5bcab7..c30f8da01fa23 100644 --- a/docs/sources/configure/_index.md +++ b/docs/sources/configure/_index.md @@ -2911,6 +2911,30 @@ The `limits_config` block configures global and per-tenant limits in Loki. # CLI flag: -querier.split-metadata-queries-by-interval [split_metadata_queries_by_interval: | default = 1d] +# Experimental. Split interval to use for the portion of metadata request that +# falls within `recent_metadata_query_window`. Rest of the request which is +# outside the window still uses `split_metadata_queries_by_interval`. If set to +# 0, the entire request defaults to using a split interval of +# `split_metadata_queries_by_interval.`. +# CLI flag: -experimental.querier.split-recent-metadata-queries-by-interval +[split_recent_metadata_queries_by_interval: | default = 1h] + +# Experimental. Metadata query window inside which +# `split_recent_metadata_queries_by_interval` gets applied, portion of the +# metadata request that falls in this window is split using +# `split_recent_metadata_queries_by_interval`. The value 0 disables using a +# different split interval for recent metadata queries. +# +# This is added to improve cacheability of recent metadata queries. Query split +# interval also determines the interval used in cache key. The default split +# interval of 24h is useful for caching long queries, each cache key holding 1 +# day's results. But metadata queries are often shorter than 24h, to cache them +# effectively we need a smaller split interval. `recent_metadata_query_window` +# along with `split_recent_metadata_queries_by_interval` help configure a +# shorter split interval for recent metadata queries. +# CLI flag: -experimental.querier.recent-metadata-query-window +[recent_metadata_query_window: | default = 0s] + # Interval to use for time-based splitting when a request is within the # `query_ingesters_within` window; defaults to `split-queries-by-interval` by # setting to 0. diff --git a/pkg/bloomgateway/cache.go b/pkg/bloomgateway/cache.go index fe40b87e95488..6c573cb47d6de 100644 --- a/pkg/bloomgateway/cache.go +++ b/pkg/bloomgateway/cache.go @@ -182,6 +182,7 @@ func NewBloomGatewayClientCacheMiddleware( }, cacheGen, retentionEnabled, + false, ) return &ClientCache{ diff --git a/pkg/querier/queryrange/index_stats_cache.go b/pkg/querier/queryrange/index_stats_cache.go index d52f2e22323ff..a91721bf36873 100644 --- a/pkg/querier/queryrange/index_stats_cache.go +++ b/pkg/querier/queryrange/index_stats_cache.go @@ -123,6 +123,7 @@ func NewIndexStatsCacheMiddleware( }, parallelismForReq, retentionEnabled, + false, metrics, ) } diff --git a/pkg/querier/queryrange/labels_cache.go b/pkg/querier/queryrange/labels_cache.go index 66c811490403d..3a940e34fa034 100644 --- a/pkg/querier/queryrange/labels_cache.go +++ b/pkg/querier/queryrange/labels_cache.go @@ -11,21 +11,42 @@ import ( "github.com/grafana/loki/pkg/querier/queryrange/queryrangebase" "github.com/grafana/loki/pkg/storage/chunk/cache" "github.com/grafana/loki/pkg/storage/chunk/cache/resultscache" - "github.com/grafana/loki/pkg/util" + "github.com/grafana/loki/pkg/util/validation" ) type cacheKeyLabels struct { Limits transformer UserIDTransformer - iqo util.IngesterQueryOptions +} + +// metadataSplitIntervalForTimeRange returns split interval for series and label requests. +// If `recent_metadata_query_window` is configured and the query start interval is within this window, +// it returns `split_recent_metadata_queries_by_interval`. +// For other cases, the default split interval of `split_metadata_queries_by_interval` will be used. +func metadataSplitIntervalForTimeRange(limits Limits, tenantIDs []string, ref, start time.Time) time.Duration { + split := validation.MaxDurationOrZeroPerTenant(tenantIDs, limits.MetadataQuerySplitDuration) + + recentMetadataQueryWindow := validation.MaxDurationOrZeroPerTenant(tenantIDs, limits.RecentMetadataQueryWindow) + recentMetadataQuerySplitInterval := validation.MaxDurationOrZeroPerTenant(tenantIDs, limits.RecentMetadataQuerySplitDuration) + + // if either of the options are not configured, use the default metadata split interval + if recentMetadataQueryWindow == 0 || recentMetadataQuerySplitInterval == 0 { + return split + } + + // if the query start is not before window start, it would be split using recentMetadataQuerySplitInterval + if windowStart := ref.Add(-recentMetadataQueryWindow); !start.Before(windowStart) { + split = recentMetadataQuerySplitInterval + } + + return split } // GenerateCacheKey generates a cache key based on the userID, split duration and the interval of the request. // It also includes the label name and the provided query for label values request. func (i cacheKeyLabels) GenerateCacheKey(ctx context.Context, userID string, r resultscache.Request) string { lr := r.(*LabelRequest) - - split := SplitIntervalForTimeRange(i.iqo, i.Limits, i.MetadataQuerySplitDuration, []string{userID}, time.Now().UTC(), r.GetEnd().UTC()) + split := metadataSplitIntervalForTimeRange(i.Limits, []string{userID}, time.Now().UTC(), r.GetStart().UTC()) var currentInterval int64 if denominator := int64(split / time.Millisecond); denominator > 0 { @@ -80,7 +101,6 @@ func NewLabelsCacheMiddleware( merger queryrangebase.Merger, c cache.Cache, cacheGenNumberLoader queryrangebase.CacheGenNumberLoader, - iqo util.IngesterQueryOptions, shouldCache queryrangebase.ShouldCacheFn, parallelismForReq queryrangebase.ParallelismForReqFn, retentionEnabled bool, @@ -90,7 +110,7 @@ func NewLabelsCacheMiddleware( return queryrangebase.NewResultsCacheMiddleware( logger, c, - cacheKeyLabels{limits, transformer, iqo}, + cacheKeyLabels{limits, transformer}, limits, merger, labelsExtractor{}, @@ -100,6 +120,7 @@ func NewLabelsCacheMiddleware( }, parallelismForReq, retentionEnabled, + true, metrics, ) } diff --git a/pkg/querier/queryrange/labels_cache_test.go b/pkg/querier/queryrange/labels_cache_test.go index 4c645b8d19ce9..90b85cb1faf82 100644 --- a/pkg/querier/queryrange/labels_cache_test.go +++ b/pkg/querier/queryrange/labels_cache_test.go @@ -70,7 +70,6 @@ func TestLabelsCache(t *testing.T) { cache.NewMockCache(), nil, nil, - nil, func(_ context.Context, _ []string, _ queryrangebase.Request) int { return 1 }, @@ -82,173 +81,124 @@ func TestLabelsCache(t *testing.T) { return cacheMiddleware } - cacheMiddleware := setupCacheMW() - for _, values := range []bool{false, true} { - prefix := "labels" - if values { - prefix = "label values" - } - t.Run(prefix+": cache the response for the same request", func(t *testing.T) { - start := testTime.Truncate(time.Millisecond) - end := start.Add(time.Hour) - - labelsReq := LabelRequest{ - LabelRequest: logproto.LabelRequest{ - Start: &start, - End: &end, - }, - } - - if values { - labelsReq.Values = true - labelsReq.Name = "foo" - labelsReq.Query = `{cluster="eu-west1"}` - } - - labelsResp := &LokiLabelNamesResponse{ - Status: "success", - Version: uint32(loghttp.VersionV1), - Data: []string{"bar", "buzz"}, - Statistics: stats.Result{ - Summary: stats.Summary{ - Splits: 1, - }, + composeLabelsResp := func(lbls []string, splits int64) *LokiLabelNamesResponse { + return &LokiLabelNamesResponse{ + Status: "success", + Version: uint32(loghttp.VersionV1), + Data: lbls, + Statistics: stats.Result{ + Summary: stats.Summary{ + Splits: splits, }, - } - - called := 0 - handler := cacheMiddleware.Wrap(queryrangebase.HandlerFunc(func(_ context.Context, r queryrangebase.Request) (queryrangebase.Response, error) { - called++ - - // should request the entire length with no partitioning as nothing is cached yet. - require.Equal(t, labelsReq.GetStart(), r.GetStart()) - require.Equal(t, labelsReq.GetEnd(), r.GetEnd()) - - got := r.(*LabelRequest) - require.Equal(t, labelsReq.GetName(), got.GetName()) - require.Equal(t, labelsReq.GetValues(), got.GetValues()) - require.Equal(t, labelsReq.GetQuery(), got.GetQuery()) - - return labelsResp, nil - })) + }, + } - ctx := user.InjectOrgID(context.Background(), "fake") - got, err := handler.Do(ctx, &labelsReq) - require.NoError(t, err) - require.Equal(t, 1, called) // called actual handler, as not cached. - require.Equal(t, labelsResp, got) + } - // Doing same request again shouldn't change anything. - called = 0 - got, err = handler.Do(ctx, &labelsReq) - require.NoError(t, err) - require.Equal(t, 0, called) - require.Equal(t, labelsResp, got) - }) + start := testTime.Truncate(time.Millisecond) + end := start.Add(time.Hour) + labelsReq := &LabelRequest{ + LabelRequest: logproto.LabelRequest{ + Start: &start, + End: &end, + }, } + labelsResp := composeLabelsResp([]string{"bar", "buzz"}, 1) + + var downstreamHandlerFunc func(context.Context, queryrangebase.Request) (queryrangebase.Response, error) + downstreamHandler := &mockDownstreamHandler{fn: func(ctx context.Context, req queryrangebase.Request) (queryrangebase.Response, error) { + return downstreamHandlerFunc(ctx, req) + }} - // reset cacheMiddleware - cacheMiddleware = setupCacheMW() for _, values := range []bool{false, true} { + labelsReq := labelsReq prefix := "labels" + if values { - prefix = "label values" + prefix = "label values: " + labelsReq.Values = true + labelsReq.Name = "foo" + labelsReq.Query = `{cluster="eu-west1"}` } - t.Run(prefix+": a new request with overlapping time range should reuse part of the previous request for the overlap", func(t *testing.T) { - cacheMiddleware := setupCacheMW() - - start := testTime.Truncate(time.Millisecond) - end := start.Add(time.Hour) - labelsReq1 := LabelRequest{ - LabelRequest: logproto.LabelRequest{ - Start: &start, - End: &end, - }, - } - - if values { - labelsReq1.Values = true - labelsReq1.Name = "foo" - labelsReq1.Query = `{cluster="eu-west1"}` - } - - labelsResp1 := &LokiLabelNamesResponse{ - Status: "success", - Version: uint32(loghttp.VersionV1), - Data: []string{"bar", "buzz"}, - Statistics: stats.Result{ - Summary: stats.Summary{ - Splits: 1, - }, - }, - } + for _, tc := range []struct { + name string + req queryrangebase.Request + expectedQueryStart, expectedQueryEnd time.Time + downstreamResponse *LokiLabelNamesResponse + downstreamCalls int + expectedReponse *LokiLabelNamesResponse + }{ + { + name: "return cached response for the same request", + downstreamCalls: 0, + expectedReponse: labelsResp, + req: labelsReq, + }, + { + name: "a new request with overlapping time range should reuse results of the previous request", + req: labelsReq.WithStartEnd(labelsReq.GetStart(), labelsReq.GetEnd().Add(15*time.Minute)), + expectedQueryStart: labelsReq.GetEnd(), + expectedQueryEnd: labelsReq.GetEnd().Add(15 * time.Minute), + downstreamCalls: 1, + downstreamResponse: composeLabelsResp([]string{"fizz"}, 1), + expectedReponse: composeLabelsResp([]string{"bar", "buzz", "fizz"}, 2), + }, + { + // To avoid returning incorrect results, we only use extents that are entirely within the requested query range. + name: "cached response not entirely within the requested range", + req: labelsReq.WithStartEnd(labelsReq.GetStart().Add(15*time.Minute), labelsReq.GetEnd().Add(-15*time.Minute)), + expectedQueryStart: labelsReq.GetStart().Add(15 * time.Minute), + expectedQueryEnd: labelsReq.GetEnd().Add(-15 * time.Minute), + downstreamCalls: 1, + downstreamResponse: composeLabelsResp([]string{"buzz", "fizz"}, 1), + expectedReponse: composeLabelsResp([]string{"buzz", "fizz"}, 1), + }, + } { + t.Run(prefix+tc.name, func(t *testing.T) { + cacheMiddleware := setupCacheMW() + downstreamHandler.ResetCount() + downstreamHandlerFunc = func(_ context.Context, r queryrangebase.Request) (queryrangebase.Response, error) { + // should request the entire length with no partitioning as nothing is cached yet. + require.Equal(t, labelsReq.GetStart(), r.GetStart()) + require.Equal(t, labelsReq.GetEnd(), r.GetEnd()) + + got := r.(*LabelRequest) + require.Equal(t, labelsReq.GetName(), got.GetName()) + require.Equal(t, labelsReq.GetValues(), got.GetValues()) + require.Equal(t, labelsReq.GetQuery(), got.GetQuery()) + + return labelsResp, nil + } - called := 0 - handler := cacheMiddleware.Wrap(queryrangebase.HandlerFunc(func(_ context.Context, r queryrangebase.Request) (queryrangebase.Response, error) { - called++ + handler := cacheMiddleware.Wrap(downstreamHandler) - // should request the entire length with no partitioning as nothing is cached yet. - require.Equal(t, labelsReq1.GetStart(), r.GetStart()) - require.Equal(t, labelsReq1.GetEnd(), r.GetEnd()) + ctx := user.InjectOrgID(context.Background(), "fake") + got, err := handler.Do(ctx, labelsReq) + require.NoError(t, err) + require.Equal(t, 1, downstreamHandler.Called()) // call downstream handler, as not cached. + require.Equal(t, labelsResp, got) - got := r.(*LabelRequest) - require.Equal(t, labelsReq1.GetName(), got.GetName()) - require.Equal(t, labelsReq1.GetValues(), got.GetValues()) - require.Equal(t, labelsReq1.GetQuery(), got.GetQuery()) + downstreamHandler.ResetCount() + downstreamHandlerFunc = func(_ context.Context, r queryrangebase.Request) (queryrangebase.Response, error) { + require.Equal(t, tc.expectedQueryStart, r.GetStart()) + require.Equal(t, tc.expectedQueryEnd, r.GetEnd()) - return labelsResp1, nil - })) + got := r.(*LabelRequest) + require.Equal(t, labelsReq.GetName(), got.GetName()) + require.Equal(t, labelsReq.GetValues(), got.GetValues()) + require.Equal(t, labelsReq.GetQuery(), got.GetQuery()) - ctx := user.InjectOrgID(context.Background(), "fake") - got, err := handler.Do(ctx, &labelsReq1) - require.NoError(t, err) - require.Equal(t, 1, called) - require.Equal(t, labelsResp1, got) + return tc.downstreamResponse, nil + } - labelsReq2 := labelsReq1.WithStartEnd(labelsReq1.GetStart().Add(15*time.Minute), labelsReq1.GetEnd().Add(15*time.Minute)) + got, err = handler.Do(ctx, tc.req) + require.NoError(t, err) + require.Equal(t, tc.downstreamCalls, downstreamHandler.Called()) + require.Equal(t, tc.expectedReponse, got) - called = 0 - handler = cacheMiddleware.Wrap(queryrangebase.HandlerFunc(func(_ context.Context, r queryrangebase.Request) (queryrangebase.Response, error) { - called++ - - // make downstream request only for the non-overlapping portion of the query. - require.Equal(t, labelsReq1.GetEnd(), r.GetStart()) - require.Equal(t, labelsReq2.GetEnd(), r.GetEnd()) - - got := r.(*LabelRequest) - require.Equal(t, labelsReq1.GetName(), got.GetName()) - require.Equal(t, labelsReq1.GetValues(), got.GetValues()) - require.Equal(t, labelsReq1.GetQuery(), got.GetQuery()) - - return &LokiLabelNamesResponse{ - Status: "success", - Version: uint32(loghttp.VersionV1), - Data: []string{"fizz"}, - Statistics: stats.Result{ - Summary: stats.Summary{ - Splits: 1, - }, - }, - }, nil - })) - - got, err = handler.Do(ctx, labelsReq2) - require.NoError(t, err) - require.Equal(t, 1, called) - // two splits as we merge the results from the extent and downstream request - labelsResp1.Statistics.Summary.Splits = 2 - require.Equal(t, &LokiLabelNamesResponse{ - Status: "success", - Version: uint32(loghttp.VersionV1), - Data: []string{"bar", "buzz", "fizz"}, - Statistics: stats.Result{ - Summary: stats.Summary{ - Splits: 2, - }, - }, - }, got) - }) + }) + } } } @@ -310,7 +260,6 @@ func TestLabelCache_freshness(t *testing.T) { cache.NewMockCache(), nil, nil, - nil, func(_ context.Context, _ []string, _ queryrangebase.Request) int { return 1 }, @@ -363,67 +312,46 @@ func TestLabelCache_freshness(t *testing.T) { func TestLabelQueryCacheKey(t *testing.T) { const ( - defaultTenant = "a" - alternateTenant = "b" - defaultSplit = time.Hour - ingesterSplit = 90 * time.Minute - ingesterQueryWindow = defaultSplit * 3 + defaultSplit = time.Hour + recentMetadataSplitDuration = 30 * time.Minute + recentMetadataQueryWindow = time.Hour ) l := fakeLimits{ - metadataSplitDuration: map[string]time.Duration{defaultTenant: defaultSplit, alternateTenant: defaultSplit}, - ingesterSplitDuration: map[string]time.Duration{defaultTenant: ingesterSplit}, + metadataSplitDuration: map[string]time.Duration{tenantID: defaultSplit}, + recentMetadataSplitDuration: map[string]time.Duration{tenantID: recentMetadataSplitDuration}, + recentMetadataQueryWindow: map[string]time.Duration{tenantID: recentMetadataQueryWindow}, } cases := []struct { - name, tenantID string - start, end time.Time - expectedSplit time.Duration - iqo util.IngesterQueryOptions - values bool + name string + start, end time.Time + expectedSplit time.Duration + values bool + limits Limits }{ { - name: "outside ingester query window", - tenantID: defaultTenant, - start: time.Now().Add(-6 * time.Hour), - end: time.Now().Add(-5 * time.Hour), + name: "outside recent metadata query window", + start: time.Now().Add(-3 * time.Hour), + end: time.Now().Add(-2 * time.Hour), expectedSplit: defaultSplit, - iqo: ingesterQueryOpts{ - queryIngestersWithin: ingesterQueryWindow, - queryStoreOnly: false, - }, - }, - { - name: "within ingester query window", - tenantID: defaultTenant, - start: time.Now().Add(-6 * time.Hour), - end: time.Now().Add(-ingesterQueryWindow / 2), - expectedSplit: ingesterSplit, - iqo: ingesterQueryOpts{ - queryIngestersWithin: ingesterQueryWindow, - queryStoreOnly: false, - }, + limits: l, }, { - name: "within ingester query window, but query store only", - tenantID: defaultTenant, - start: time.Now().Add(-6 * time.Hour), - end: time.Now().Add(-ingesterQueryWindow / 2), - expectedSplit: defaultSplit, - iqo: ingesterQueryOpts{ - queryIngestersWithin: ingesterQueryWindow, - queryStoreOnly: true, - }, + name: "within recent metadata query window", + start: time.Now().Add(-30 * time.Minute), + end: time.Now(), + expectedSplit: recentMetadataSplitDuration, + limits: l, }, { - name: "within ingester query window, but no ingester split duration configured", - tenantID: alternateTenant, - start: time.Now().Add(-6 * time.Hour), - end: time.Now().Add(-ingesterQueryWindow / 2), + name: "within recent metadata query window, but recent split duration is not configured", + start: time.Now().Add(-30 * time.Minute), + end: time.Now(), expectedSplit: defaultSplit, - iqo: ingesterQueryOpts{ - queryIngestersWithin: ingesterQueryWindow, - queryStoreOnly: false, + limits: fakeLimits{ + metadataSplitDuration: map[string]time.Duration{tenantID: defaultSplit}, + recentMetadataQueryWindow: map[string]time.Duration{tenantID: recentMetadataQueryWindow}, }, }, } @@ -431,7 +359,7 @@ func TestLabelQueryCacheKey(t *testing.T) { for _, values := range []bool{true, false} { for _, tc := range cases { t.Run(fmt.Sprintf("%s (values: %v)", tc.name, values), func(t *testing.T) { - keyGen := cacheKeyLabels{l, nil, tc.iqo} + keyGen := cacheKeyLabels{tc.limits, nil} r := &LabelRequest{ LabelRequest: logproto.LabelRequest{ @@ -453,12 +381,12 @@ func TestLabelQueryCacheKey(t *testing.T) { // and therefore we can't know the current interval apriori without duplicating the logic var pattern *regexp.Regexp if values { - pattern = regexp.MustCompile(fmt.Sprintf(`labelvalues:%s:%s:%s:(\d+):%d`, tc.tenantID, labelName, regexp.QuoteMeta(query), tc.expectedSplit)) + pattern = regexp.MustCompile(fmt.Sprintf(`labelvalues:%s:%s:%s:(\d+):%d`, tenantID, labelName, regexp.QuoteMeta(query), tc.expectedSplit)) } else { - pattern = regexp.MustCompile(fmt.Sprintf(`labels:%s:(\d+):%d`, tc.tenantID, tc.expectedSplit)) + pattern = regexp.MustCompile(fmt.Sprintf(`labels:%s:(\d+):%d`, tenantID, tc.expectedSplit)) } - require.Regexp(t, pattern, keyGen.GenerateCacheKey(context.Background(), tc.tenantID, r)) + require.Regexp(t, pattern, keyGen.GenerateCacheKey(context.Background(), tenantID, r)) }) } } diff --git a/pkg/querier/queryrange/limits/definitions.go b/pkg/querier/queryrange/limits/definitions.go index e12255883bf4a..3e78b34420760 100644 --- a/pkg/querier/queryrange/limits/definitions.go +++ b/pkg/querier/queryrange/limits/definitions.go @@ -15,6 +15,8 @@ type Limits interface { logql.Limits QuerySplitDuration(string) time.Duration MetadataQuerySplitDuration(string) time.Duration + RecentMetadataQuerySplitDuration(string) time.Duration + RecentMetadataQueryWindow(string) time.Duration IngesterQuerySplitDuration(string) time.Duration MaxQuerySeries(context.Context, string) int MaxEntriesLimitPerQuery(context.Context, string) int diff --git a/pkg/querier/queryrange/queryrangebase/results_cache.go b/pkg/querier/queryrange/queryrangebase/results_cache.go index 097dc264d32a0..3511fe0b7dd30 100644 --- a/pkg/querier/queryrange/queryrangebase/results_cache.go +++ b/pkg/querier/queryrange/queryrangebase/results_cache.go @@ -127,6 +127,7 @@ func NewResultsCacheMiddleware( shouldCache ShouldCacheFn, parallelismForReq ParallelismForReqFn, retentionEnabled bool, + onlyUseEntireExtent bool, metrics *ResultsCacheMetrics, ) (Middleware, error) { if cacheGenNumberLoader != nil { @@ -172,6 +173,7 @@ func NewResultsCacheMiddleware( parallelismForReqWrapper, cacheGenNumberLoader, retentionEnabled, + onlyUseEntireExtent, ) return out diff --git a/pkg/querier/queryrange/queryrangebase/results_cache_test.go b/pkg/querier/queryrange/queryrangebase/results_cache_test.go index ff5e5be09a48f..6706e6a2d9fa7 100644 --- a/pkg/querier/queryrange/queryrangebase/results_cache_test.go +++ b/pkg/querier/queryrange/queryrangebase/results_cache_test.go @@ -422,6 +422,7 @@ func TestResultsCache(t *testing.T) { return mockLimits{}.MaxQueryParallelism(context.Background(), "fake") }, false, + false, nil, ) require.NoError(t, err) @@ -468,6 +469,7 @@ func TestResultsCacheRecent(t *testing.T) { return mockLimits{}.MaxQueryParallelism(context.Background(), "fake") }, false, + false, nil, ) require.NoError(t, err) @@ -578,6 +580,7 @@ func TestResultsCacheShouldCacheFunc(t *testing.T) { return mockLimits{}.MaxQueryParallelism(context.Background(), "fake") }, false, + false, nil, ) require.NoError(t, err) diff --git a/pkg/querier/queryrange/roundtrip.go b/pkg/querier/queryrange/roundtrip.go index 8223704eea021..10246f4d8277e 100644 --- a/pkg/querier/queryrange/roundtrip.go +++ b/pkg/querier/queryrange/roundtrip.go @@ -514,7 +514,6 @@ func NewSeriesTripperware( merger, c, cacheGenNumLoader, - iqo, func(_ context.Context, r base.Request) bool { return !r.GetCachingOptions().Disabled }, @@ -600,7 +599,6 @@ func NewLabelsTripperware( merger, c, cacheGenNumLoader, - iqo, func(_ context.Context, r base.Request) bool { return !r.GetCachingOptions().Disabled }, @@ -679,6 +677,7 @@ func NewMetricTripperware(cfg Config, engineOpts logql.EngineOpts, log log.Logge ) }, retentionEnabled, + false, metrics.ResultsCacheMetrics, ) if err != nil { diff --git a/pkg/querier/queryrange/roundtrip_test.go b/pkg/querier/queryrange/roundtrip_test.go index c7c7cff4595a1..7d74b0dd615c8 100644 --- a/pkg/querier/queryrange/roundtrip_test.go +++ b/pkg/querier/queryrange/roundtrip_test.go @@ -1237,24 +1237,26 @@ func TestMetricsTripperware_SplitShardStats(t *testing.T) { } type fakeLimits struct { - maxQueryLength time.Duration - maxQueryParallelism int - tsdbMaxQueryParallelism int - maxQueryLookback time.Duration - maxEntriesLimitPerQuery int - maxSeries int - splitDuration map[string]time.Duration - metadataSplitDuration map[string]time.Duration - ingesterSplitDuration map[string]time.Duration - minShardingLookback time.Duration - queryTimeout time.Duration - requiredLabels []string - requiredNumberLabels int - maxQueryBytesRead int - maxQuerierBytesRead int - maxStatsCacheFreshness time.Duration - maxMetadataCacheFreshness time.Duration - volumeEnabled bool + maxQueryLength time.Duration + maxQueryParallelism int + tsdbMaxQueryParallelism int + maxQueryLookback time.Duration + maxEntriesLimitPerQuery int + maxSeries int + splitDuration map[string]time.Duration + metadataSplitDuration map[string]time.Duration + recentMetadataSplitDuration map[string]time.Duration + recentMetadataQueryWindow map[string]time.Duration + ingesterSplitDuration map[string]time.Duration + minShardingLookback time.Duration + queryTimeout time.Duration + requiredLabels []string + requiredNumberLabels int + maxQueryBytesRead int + maxQuerierBytesRead int + maxStatsCacheFreshness time.Duration + maxMetadataCacheFreshness time.Duration + volumeEnabled bool } func (f fakeLimits) QuerySplitDuration(key string) time.Duration { @@ -1271,6 +1273,20 @@ func (f fakeLimits) MetadataQuerySplitDuration(key string) time.Duration { return f.metadataSplitDuration[key] } +func (f fakeLimits) RecentMetadataQuerySplitDuration(key string) time.Duration { + if f.recentMetadataSplitDuration == nil { + return 0 + } + return f.recentMetadataSplitDuration[key] +} + +func (f fakeLimits) RecentMetadataQueryWindow(key string) time.Duration { + if f.recentMetadataQueryWindow == nil { + return 0 + } + return f.recentMetadataQueryWindow[key] +} + func (f fakeLimits) IngesterQuerySplitDuration(key string) time.Duration { if f.ingesterSplitDuration == nil { return 0 diff --git a/pkg/querier/queryrange/series_cache.go b/pkg/querier/queryrange/series_cache.go index bbbf96e2dd70e..5120d61fb0b4f 100644 --- a/pkg/querier/queryrange/series_cache.go +++ b/pkg/querier/queryrange/series_cache.go @@ -17,21 +17,18 @@ import ( "github.com/grafana/loki/pkg/querier/queryrange/queryrangebase" "github.com/grafana/loki/pkg/storage/chunk/cache" "github.com/grafana/loki/pkg/storage/chunk/cache/resultscache" - "github.com/grafana/loki/pkg/util" "github.com/grafana/loki/pkg/util/validation" ) type cacheKeySeries struct { Limits transformer UserIDTransformer - iqo util.IngesterQueryOptions } // GenerateCacheKey generates a cache key based on the userID, matchers, split duration and the interval of the request. func (i cacheKeySeries) GenerateCacheKey(ctx context.Context, userID string, r resultscache.Request) string { sr := r.(*LokiSeriesRequest) - - split := SplitIntervalForTimeRange(i.iqo, i.Limits, i.MetadataQuerySplitDuration, []string{userID}, time.Now().UTC(), r.GetEnd().UTC()) + split := metadataSplitIntervalForTimeRange(i.Limits, []string{userID}, time.Now().UTC(), r.GetStart().UTC()) var currentInterval int64 if denominator := int64(split / time.Millisecond); denominator > 0 { @@ -87,7 +84,6 @@ func NewSeriesCacheMiddleware( merger queryrangebase.Merger, c cache.Cache, cacheGenNumberLoader queryrangebase.CacheGenNumberLoader, - iqo util.IngesterQueryOptions, shouldCache queryrangebase.ShouldCacheFn, parallelismForReq queryrangebase.ParallelismForReqFn, retentionEnabled bool, @@ -97,7 +93,7 @@ func NewSeriesCacheMiddleware( return queryrangebase.NewResultsCacheMiddleware( logger, c, - cacheKeySeries{limits, transformer, iqo}, + cacheKeySeries{limits, transformer}, limits, merger, seriesExtractor{}, @@ -107,6 +103,7 @@ func NewSeriesCacheMiddleware( }, parallelismForReq, retentionEnabled, + true, metrics, ) } diff --git a/pkg/querier/queryrange/series_cache_test.go b/pkg/querier/queryrange/series_cache_test.go index d73efa9deea87..6ba869a69411a 100644 --- a/pkg/querier/queryrange/series_cache_test.go +++ b/pkg/querier/queryrange/series_cache_test.go @@ -78,7 +78,6 @@ func TestSeriesCache(t *testing.T) { cache.NewMockCache(), nil, nil, - nil, func(_ context.Context, _ []string, _ queryrangebase.Request) int { return 1 }, @@ -91,195 +90,135 @@ func TestSeriesCache(t *testing.T) { return cacheMiddleware } - t.Run("caches the response for the same request", func(t *testing.T) { - cacheMiddleware := setupCacheMW() - from, through := util.RoundToMilliseconds(testTime, testTime.Add(1*time.Hour)) - - seriesReq := &LokiSeriesRequest{ - StartTs: from.Time(), - EndTs: through.Time(), - Match: []string{`{namespace=~".*"}`}, - Path: seriesAPIPath, + composeSeriesResp := func(series [][]logproto.SeriesIdentifier_LabelsEntry, splits int64) *LokiSeriesResponse { + var data []logproto.SeriesIdentifier + for _, v := range series { + data = append(data, logproto.SeriesIdentifier{Labels: v}) } - seriesResp := &LokiSeriesResponse{ + return &LokiSeriesResponse{ Status: "success", Version: uint32(loghttp.VersionV1), - Data: []logproto.SeriesIdentifier{ - { - Labels: []logproto.SeriesIdentifier_LabelsEntry{{Key: "cluster", Value: "eu-west"}, {Key: "namespace", Value: "prod"}}, - }, - }, + Data: data, Statistics: stats.Result{ Summary: stats.Summary{ - Splits: 1, + Splits: splits, }, }, } + } - called := 0 - handler := cacheMiddleware.Wrap(queryrangebase.HandlerFunc(func(_ context.Context, r queryrangebase.Request) (queryrangebase.Response, error) { - called++ - - // should request the entire length with no partitioning as nothing is cached yet. - require.Equal(t, seriesReq.GetStart(), r.GetStart()) - require.Equal(t, seriesReq.GetEnd(), r.GetEnd()) - - return seriesResp, nil - })) - - ctx := user.InjectOrgID(context.Background(), "fake") - got, err := handler.Do(ctx, seriesReq) - require.NoError(t, err) - require.Equal(t, 1, called) // called actual handler, as not cached. - require.Equal(t, seriesResp, got) - - // Doing same request again shouldn't change anything. - called = 0 - got, err = handler.Do(ctx, seriesReq) - require.NoError(t, err) - require.Equal(t, 0, called) - require.Equal(t, seriesResp, got) - }) - - t.Run("a new request with overlapping time range should reuse part of the previous request for the overlap", func(t *testing.T) { - cacheMiddleware := setupCacheMW() - - from, through := util.RoundToMilliseconds(testTime, testTime.Add(1*time.Hour)) - req1 := &LokiSeriesRequest{ - StartTs: from.Time(), - EndTs: through.Time(), - Match: []string{`{namespace=~".*"}`}, - Path: seriesAPIPath, - } - resp1 := &LokiSeriesResponse{ - Status: "success", - Version: uint32(loghttp.VersionV1), - Data: []logproto.SeriesIdentifier{ - { - Labels: []logproto.SeriesIdentifier_LabelsEntry{{Key: "cluster", Value: "us-central"}, {Key: "namespace", Value: "dev"}}, - }, - { - Labels: []logproto.SeriesIdentifier_LabelsEntry{{Key: "cluster", Value: "eu-west"}, {Key: "namespace", Value: "prod"}}, - }, - }, - Statistics: stats.Result{ - Summary: stats.Summary{ - Splits: 1, - }, - }, - } - - called := 0 - handler := cacheMiddleware.Wrap(queryrangebase.HandlerFunc(func(_ context.Context, r queryrangebase.Request) (queryrangebase.Response, error) { - called++ - - // should request the entire length with no partitioning as nothing is cached yet. - require.Equal(t, req1.GetStart(), r.GetStart()) - require.Equal(t, req1.GetEnd(), r.GetEnd()) + var downstreamHandlerFunc func(context.Context, queryrangebase.Request) (queryrangebase.Response, error) + downstreamHandler := &mockDownstreamHandler{fn: func(ctx context.Context, req queryrangebase.Request) (queryrangebase.Response, error) { + return downstreamHandlerFunc(ctx, req) + }} - return resp1, nil - })) + from, through := util.RoundToMilliseconds(testTime, testTime.Add(1*time.Hour)) + seriesReq := &LokiSeriesRequest{ + StartTs: from.Time(), + EndTs: through.Time(), + Match: []string{`{namespace=~".*"}`}, + Path: seriesAPIPath, + } + seriesResp := composeSeriesResp([][]logproto.SeriesIdentifier_LabelsEntry{ + {{Key: "cluster", Value: "us-central"}, {Key: "namespace", Value: "dev"}}, + {{Key: "cluster", Value: "eu-west"}, {Key: "namespace", Value: "prod"}}, + }, 1) + + for _, tc := range []struct { + name string + req queryrangebase.Request + expectedQueryStart, expectedQueryEnd time.Time + downstreamResponse *LokiSeriesResponse + downstreamCalls int + expectedReponse *LokiSeriesResponse + }{ + { + name: "return cached response for the same request", + downstreamCalls: 0, + expectedReponse: seriesResp, + req: seriesReq, + }, + { + name: "a new request with overlapping time range should reuse results of the previous request", + req: seriesReq.WithStartEnd(seriesReq.GetStart(), seriesReq.GetEnd().Add(15*time.Minute)), + expectedQueryStart: seriesReq.GetEnd(), + expectedQueryEnd: seriesReq.GetEnd().Add(15 * time.Minute), + downstreamCalls: 1, + downstreamResponse: composeSeriesResp([][]logproto.SeriesIdentifier_LabelsEntry{ + {{Key: "cluster", Value: "us-central"}, {Key: "namespace", Value: "prod"}}, + }, 1), + expectedReponse: composeSeriesResp([][]logproto.SeriesIdentifier_LabelsEntry{ + {{Key: "cluster", Value: "us-central"}, {Key: "namespace", Value: "dev"}}, + {{Key: "cluster", Value: "eu-west"}, {Key: "namespace", Value: "prod"}}, + {{Key: "cluster", Value: "us-central"}, {Key: "namespace", Value: "prod"}}, + }, 2), + }, + { + // To avoid returning incorrect results, we only use extents that are entirely within the requested query range. + name: "cached response not entirely within the requested range", + req: seriesReq.WithStartEnd(seriesReq.GetStart().Add(15*time.Minute), seriesReq.GetEnd().Add(-15*time.Minute)), + expectedQueryStart: seriesReq.GetStart().Add(15 * time.Minute), + expectedQueryEnd: seriesReq.GetEnd().Add(-15 * time.Minute), + downstreamCalls: 1, + downstreamResponse: composeSeriesResp([][]logproto.SeriesIdentifier_LabelsEntry{ + {{Key: "cluster", Value: "us-central"}, {Key: "namespace", Value: "prod"}}, + }, 1), + expectedReponse: composeSeriesResp([][]logproto.SeriesIdentifier_LabelsEntry{ + {{Key: "cluster", Value: "us-central"}, {Key: "namespace", Value: "prod"}}, + }, 1), + }, + } { + t.Run(tc.name, func(t *testing.T) { + cacheMiddleware := setupCacheMW() + downstreamHandler.ResetCount() + downstreamHandlerFunc = func(_ context.Context, r queryrangebase.Request) (queryrangebase.Response, error) { + require.Equal(t, seriesReq.GetStart(), r.GetStart()) + require.Equal(t, seriesReq.GetEnd(), r.GetEnd()) - ctx := user.InjectOrgID(context.Background(), "fake") - got, err := handler.Do(ctx, req1) - require.NoError(t, err) - require.Equal(t, 1, called) - require.Equal(t, resp1, got) + return seriesResp, nil + } - req2 := req1.WithStartEnd(req1.GetStart().Add(15*time.Minute), req1.GetEnd().Add(15*time.Minute)) + handler := cacheMiddleware.Wrap(downstreamHandler) - called = 0 - handler = cacheMiddleware.Wrap(queryrangebase.HandlerFunc(func(_ context.Context, r queryrangebase.Request) (queryrangebase.Response, error) { - called++ + ctx := user.InjectOrgID(context.Background(), "fake") + got, err := handler.Do(ctx, seriesReq) + require.NoError(t, err) + require.Equal(t, 1, downstreamHandler.Called()) // calls downstream handler, as not cached. + require.Equal(t, seriesResp, got) - // make downstream request only for the non-overlapping portion of the query. - require.Equal(t, req1.GetEnd(), r.GetStart()) - require.Equal(t, req1.GetEnd().Add(15*time.Minute), r.GetEnd()) + downstreamHandler.ResetCount() + downstreamHandlerFunc = func(_ context.Context, r queryrangebase.Request) (queryrangebase.Response, error) { + require.Equal(t, tc.expectedQueryStart, r.GetStart()) + require.Equal(t, tc.expectedQueryEnd, r.GetEnd()) - return &LokiSeriesResponse{ - Status: "success", - Version: uint32(loghttp.VersionV1), - Data: []logproto.SeriesIdentifier{ - { - Labels: []logproto.SeriesIdentifier_LabelsEntry{{Key: "cluster", Value: "us-central"}, {Key: "namespace", Value: "prod"}}, - }, - }, - Statistics: stats.Result{ - Summary: stats.Summary{ - Splits: 1, - }, - }, - }, nil - })) + return tc.downstreamResponse, nil + } - got, err = handler.Do(ctx, req2) - require.NoError(t, err) - require.Equal(t, 1, called) - // two splits as we merge the results from the extent and downstream request - resp1.Statistics.Summary.Splits = 2 - require.Equal(t, &LokiSeriesResponse{ - Status: "success", - Version: uint32(loghttp.VersionV1), - Data: []logproto.SeriesIdentifier{ - { - Labels: []logproto.SeriesIdentifier_LabelsEntry{{Key: "cluster", Value: "us-central"}, {Key: "namespace", Value: "dev"}}, - }, - { - Labels: []logproto.SeriesIdentifier_LabelsEntry{{Key: "cluster", Value: "eu-west"}, {Key: "namespace", Value: "prod"}}, - }, - { - Labels: []logproto.SeriesIdentifier_LabelsEntry{{Key: "cluster", Value: "us-central"}, {Key: "namespace", Value: "prod"}}, - }, - }, - Statistics: stats.Result{ - Summary: stats.Summary{ - Splits: 2, - }, - }, - }, got) - }) + got, err = handler.Do(ctx, tc.req) + require.NoError(t, err) + require.Equal(t, tc.downstreamCalls, downstreamHandler.Called()) + require.Equal(t, tc.expectedReponse, got) + }) + } t.Run("caches are only valid for the same request parameters", func(t *testing.T) { cacheMiddleware := setupCacheMW() - - from, through := util.RoundToMilliseconds(testTime, testTime.Add(1*time.Hour)) - seriesReq := &LokiSeriesRequest{ - StartTs: from.Time(), - EndTs: through.Time(), - Match: []string{`{namespace=~".*"}`}, - Path: seriesAPIPath, - } - seriesResp := &LokiSeriesResponse{ - Status: "success", - Version: uint32(loghttp.VersionV1), - Data: []logproto.SeriesIdentifier{ - { - Labels: []logproto.SeriesIdentifier_LabelsEntry{{Key: "cluster", Value: "eu-west"}, {Key: "namespace", Value: "prod"}}, - }, - }, - Statistics: stats.Result{ - Summary: stats.Summary{ - Splits: 1, - }, - }, - } - - called := 0 - handler := cacheMiddleware.Wrap(queryrangebase.HandlerFunc(func(_ context.Context, r queryrangebase.Request) (queryrangebase.Response, error) { - called++ - - // should request the entire length as none of the subsequent queries hit the cache. + downstreamHandler.ResetCount() + downstreamHandlerFunc = func(_ context.Context, r queryrangebase.Request) (queryrangebase.Response, error) { require.Equal(t, seriesReq.GetStart(), r.GetStart()) require.Equal(t, seriesReq.GetEnd(), r.GetEnd()) + return seriesResp, nil - })) + } + + handler := cacheMiddleware.Wrap(downstreamHandler) // initial call to fill cache ctx := user.InjectOrgID(context.Background(), "fake") _, err := handler.Do(ctx, seriesReq) require.NoError(t, err) - require.Equal(t, 1, called) + require.Equal(t, 1, downstreamHandler.Called()) type testCase struct { fn func(*LokiSeriesRequest) @@ -297,7 +236,7 @@ func TestSeriesCache(t *testing.T) { } for name, tc := range testCases { - called = 0 + downstreamHandler.ResetCount() seriesReq := seriesReq if tc.fn != nil { @@ -310,7 +249,7 @@ func TestSeriesCache(t *testing.T) { _, err = handler.Do(ctx, seriesReq) require.NoError(t, err) - require.Equal(t, 1, called, name) + require.Equal(t, 1, downstreamHandler.Called(), name) } }) } @@ -371,7 +310,6 @@ func TestSeriesCache_freshness(t *testing.T) { cache.NewMockCache(), nil, nil, - nil, func(_ context.Context, _ []string, _ queryrangebase.Request) int { return 1 }, @@ -428,76 +366,54 @@ func TestSeriesCache_freshness(t *testing.T) { func TestSeriesQueryCacheKey(t *testing.T) { const ( - defaultTenant = "a" - alternateTenant = "b" - defaultSplit = time.Hour - ingesterSplit = 90 * time.Minute - ingesterQueryWindow = defaultSplit * 3 + defaultSplit = time.Hour + recentMetadataSplitDuration = 30 * time.Minute + recentMetadataQueryWindow = time.Hour ) l := fakeLimits{ - metadataSplitDuration: map[string]time.Duration{defaultTenant: defaultSplit, alternateTenant: defaultSplit}, - ingesterSplitDuration: map[string]time.Duration{defaultTenant: ingesterSplit}, + metadataSplitDuration: map[string]time.Duration{tenantID: defaultSplit}, + recentMetadataSplitDuration: map[string]time.Duration{tenantID: recentMetadataSplitDuration}, + recentMetadataQueryWindow: map[string]time.Duration{tenantID: recentMetadataQueryWindow}, } cases := []struct { - name, tenantID string - start, end time.Time - expectedSplit time.Duration - iqo util.IngesterQueryOptions - values bool + name string + start, end time.Time + expectedSplit time.Duration + values bool + limits Limits }{ { - name: "outside ingester query window", - tenantID: defaultTenant, - start: time.Now().Add(-6 * time.Hour), - end: time.Now().Add(-5 * time.Hour), + name: "outside recent metadata query window", + start: time.Now().Add(-3 * time.Hour), + end: time.Now().Add(-2 * time.Hour), expectedSplit: defaultSplit, - iqo: ingesterQueryOpts{ - queryIngestersWithin: ingesterQueryWindow, - queryStoreOnly: false, - }, + limits: l, }, { - name: "within ingester query window", - tenantID: defaultTenant, - start: time.Now().Add(-6 * time.Hour), - end: time.Now().Add(-ingesterQueryWindow / 2), - expectedSplit: ingesterSplit, - iqo: ingesterQueryOpts{ - queryIngestersWithin: ingesterQueryWindow, - queryStoreOnly: false, - }, + name: "within recent metadata query window", + start: time.Now().Add(-30 * time.Minute), + end: time.Now(), + expectedSplit: recentMetadataSplitDuration, + limits: l, }, { - name: "within ingester query window, but query store only", - tenantID: defaultTenant, - start: time.Now().Add(-6 * time.Hour), - end: time.Now().Add(-ingesterQueryWindow / 2), + name: "within recent metadata query window, but recent split duration is not configured", + start: time.Now().Add(-30 * time.Minute), + end: time.Now(), expectedSplit: defaultSplit, - iqo: ingesterQueryOpts{ - queryIngestersWithin: ingesterQueryWindow, - queryStoreOnly: true, - }, - }, - { - name: "within ingester query window, but no ingester split duration configured", - tenantID: alternateTenant, - start: time.Now().Add(-6 * time.Hour), - end: time.Now().Add(-ingesterQueryWindow / 2), - expectedSplit: defaultSplit, - iqo: ingesterQueryOpts{ - queryIngestersWithin: ingesterQueryWindow, - queryStoreOnly: false, + limits: fakeLimits{ + metadataSplitDuration: map[string]time.Duration{tenantID: defaultSplit}, + recentMetadataQueryWindow: map[string]time.Duration{tenantID: recentMetadataQueryWindow}, }, }, } - for _, tc := range cases { t.Run(tc.name, func(t *testing.T) { matchers := []string{`{namespace="prod"}`, `{service="foo"}`} - keyGen := cacheKeySeries{l, nil, tc.iqo} + keyGen := cacheKeySeries{tc.limits, nil} r := &LokiSeriesRequest{ StartTs: tc.start, @@ -508,9 +424,27 @@ func TestSeriesQueryCacheKey(t *testing.T) { // we use regex here because cache key always refers to the current time to get the ingester query window, // and therefore we can't know the current interval apriori without duplicating the logic - pattern := regexp.MustCompile(fmt.Sprintf(`series:%s:%s:(\d+):%d`, tc.tenantID, regexp.QuoteMeta(keyGen.joinMatchers(matchers)), tc.expectedSplit)) + pattern := regexp.MustCompile(fmt.Sprintf(`series:%s:%s:(\d+):%d`, tenantID, regexp.QuoteMeta(keyGen.joinMatchers(matchers)), tc.expectedSplit)) - require.Regexp(t, pattern, keyGen.GenerateCacheKey(context.Background(), tc.tenantID, r)) + require.Regexp(t, pattern, keyGen.GenerateCacheKey(context.Background(), tenantID, r)) }) } } + +type mockDownstreamHandler struct { + called int + fn func(context.Context, queryrangebase.Request) (queryrangebase.Response, error) +} + +func (m *mockDownstreamHandler) Called() int { + return m.called +} + +func (m *mockDownstreamHandler) ResetCount() { + m.called = 0 +} + +func (m *mockDownstreamHandler) Do(ctx context.Context, req queryrangebase.Request) (queryrangebase.Response, error) { + m.called++ + return m.fn(ctx, req) +} diff --git a/pkg/querier/queryrange/split_by_interval_test.go b/pkg/querier/queryrange/split_by_interval_test.go index acf8c495becce..c559bad17be1d 100644 --- a/pkg/querier/queryrange/split_by_interval_test.go +++ b/pkg/querier/queryrange/split_by_interval_test.go @@ -159,10 +159,12 @@ func Test_splitQuery(t *testing.T) { t.Run(requestType, func(t *testing.T) { for name, intervals := range map[string]struct { - input interval - expected []interval - splitInterval time.Duration - splitter splitter + input interval + expected []interval + expectedWithoutIngesterSplits []interval + splitInterval time.Duration + splitter splitter + recentMetadataQueryWindowEnabled bool }{ "no change": { input: interval{ @@ -255,6 +257,16 @@ func Test_splitQuery(t *testing.T) { end: refTime, }, }, + expectedWithoutIngesterSplits: []interval{ + { + start: refTime.Add(-time.Hour).Truncate(time.Second), + end: time.Date(2023, 1, 15, 8, 0, 0, 0, time.UTC).Add(-expectedSplitGap), + }, + { + start: time.Date(2023, 1, 15, 8, 0, 0, 0, time.UTC), + end: refTime, + }, + }, splitInterval: time.Hour, splitter: newDefaultSplitter( fakeLimits{ingesterSplitDuration: map[string]time.Duration{tenantID: 90 * time.Minute}}, @@ -295,6 +307,32 @@ func Test_splitQuery(t *testing.T) { end: refTime, }, }, + expectedWithoutIngesterSplits: []interval{ + { + start: refTime.Add(-4 * time.Hour).Add(-30 * time.Minute).Truncate(time.Second), + end: time.Date(2023, 1, 15, 4, 0, 0, 0, time.UTC).Add(-expectedSplitGap), + }, + { + start: time.Date(2023, 1, 15, 4, 0, 0, 0, time.UTC), + end: time.Date(2023, 1, 15, 5, 0, 0, 0, time.UTC).Add(-expectedSplitGap), + }, + { + start: time.Date(2023, 1, 15, 5, 0, 0, 0, time.UTC), + end: time.Date(2023, 1, 15, 6, 0, 0, 0, time.UTC).Add(-expectedSplitGap), + }, + { + start: time.Date(2023, 1, 15, 6, 0, 0, 0, time.UTC), + end: time.Date(2023, 1, 15, 7, 0, 0, 0, time.UTC).Add(-expectedSplitGap), + }, + { + start: time.Date(2023, 1, 15, 7, 0, 0, 0, time.UTC), + end: time.Date(2023, 1, 15, 8, 0, 0, 0, time.UTC).Add(-expectedSplitGap), + }, + { + start: time.Date(2023, 1, 15, 8, 0, 0, 0, time.UTC), + end: refTime, + }, + }, splitInterval: time.Hour, splitter: newDefaultSplitter( fakeLimits{ingesterSplitDuration: map[string]time.Duration{tenantID: 90 * time.Minute}}, @@ -394,11 +432,63 @@ func Test_splitQuery(t *testing.T) { ingesterQueryOpts{queryIngestersWithin: 3 * time.Hour, queryStoreOnly: true}, ), }, + "metadata recent query window should not affect other query types": { + input: interval{ + start: refTime.Add(-4 * time.Hour).Truncate(time.Second), + end: refTime, + }, + expected: []interval{ + { + start: refTime.Add(-4 * time.Hour).Truncate(time.Second), + end: time.Date(2023, 1, 15, 5, 0, 0, 0, time.UTC).Add(-expectedSplitGap), + }, + { + start: time.Date(2023, 1, 15, 5, 0, 0, 0, time.UTC), + end: time.Date(2023, 1, 15, 6, 0, 0, 0, time.UTC).Add(-expectedSplitGap), + }, + { + start: time.Date(2023, 1, 15, 6, 0, 0, 0, time.UTC), + end: time.Date(2023, 1, 15, 7, 0, 0, 0, time.UTC).Add(-expectedSplitGap), + }, + { + start: time.Date(2023, 1, 15, 7, 0, 0, 0, time.UTC), + end: time.Date(2023, 1, 15, 8, 0, 0, 0, time.UTC).Add(-expectedSplitGap), + }, + { + start: time.Date(2023, 1, 15, 8, 0, 0, 0, time.UTC), + end: refTime, + }, + }, + splitInterval: time.Hour, + splitter: newDefaultSplitter( + fakeLimits{ + recentMetadataSplitDuration: map[string]time.Duration{tenantID: 30 * time.Minute}, + recentMetadataQueryWindow: map[string]time.Duration{tenantID: 3 * time.Hour}, + }, nil, + ), + recentMetadataQueryWindowEnabled: true, + }, } { t.Run(name, func(t *testing.T) { req := tc.requestBuilderFunc(intervals.input.start, intervals.input.end) var want []queryrangebase.Request - for _, exp := range intervals.expected { + + // ingester splits do not apply for metadata queries + var expected []interval + switch req.(type) { + case *LabelRequest, *LokiSeriesRequest: + expected = intervals.expectedWithoutIngesterSplits + + if intervals.recentMetadataQueryWindowEnabled { + t.Skip("this flow is tested in Test_splitRecentMetadataQuery") + } + } + + if expected == nil { + expected = intervals.expected + } + + for _, exp := range expected { want = append(want, tc.requestBuilderFunc(exp.start, exp.end)) } @@ -412,22 +502,245 @@ func Test_splitQuery(t *testing.T) { splits, err := intervals.splitter.split(refTime, []string{tenantID}, req, intervals.splitInterval) require.NoError(t, err) - if !assert.Equal(t, want, splits) { - t.Logf("expected and actual do not match\n") - defer t.Fail() + assertSplits(t, want, splits) + }) + } + }) + } +} - if len(want) != len(splits) { - t.Logf("expected %d splits, got %d\n", len(want), len(splits)) - return - } +func Test_splitRecentMetadataQuery(t *testing.T) { + type interval struct { + start, end time.Time + } - for j := 0; j < len(want); j++ { - exp := want[j] - act := splits[j] - equal := assert.Equal(t, exp, act) - t.Logf("\t#%d [matches: %v]: expected %q/%q got %q/%q\n", j, equal, exp.GetStart(), exp.GetEnd(), act.GetStart(), act.GetEnd()) - } + expectedSplitGap := util.SplitGap + + for requestType, tc := range map[string]struct { + requestBuilderFunc func(start, end time.Time) queryrangebase.Request + }{ + "series request": { + requestBuilderFunc: func(start, end time.Time) queryrangebase.Request { + return &LokiSeriesRequest{ + Match: []string{"match1"}, + StartTs: start, + EndTs: end, + Path: "/series", + Shards: []string{"shard1"}, + } + }, + }, + "label names request": { + requestBuilderFunc: func(start, end time.Time) queryrangebase.Request { + return NewLabelRequest(start, end, `{foo="bar"}`, "", "/labels") + }, + }, + "label values request": { + requestBuilderFunc: func(start, end time.Time) queryrangebase.Request { + return NewLabelRequest(start, end, `{foo="bar"}`, "test", "/label/test/values") + }, + }, + } { + t.Run(requestType, func(t *testing.T) { + for name, intervals := range map[string]struct { + input interval + expected []interval + splitInterval time.Duration + splitter splitter + }{ + "wholly within recent metadata query window": { + input: interval{ + start: refTime.Add(-time.Hour), + end: refTime, + }, + expected: []interval{ + { + start: refTime.Add(-time.Hour), + end: time.Date(2023, 1, 15, 7, 30, 0, 0, time.UTC).Add(-expectedSplitGap), + }, + { + start: time.Date(2023, 1, 15, 7, 30, 0, 0, time.UTC), + end: time.Date(2023, 1, 15, 8, 0, 0, 0, time.UTC).Add(-expectedSplitGap), + }, + { + start: time.Date(2023, 1, 15, 8, 0, 0, 0, time.UTC), + end: refTime, + }, + }, + splitInterval: time.Hour, + splitter: newDefaultSplitter( + fakeLimits{ + recentMetadataSplitDuration: map[string]time.Duration{tenantID: 30 * time.Minute}, + recentMetadataQueryWindow: map[string]time.Duration{tenantID: 2 * time.Hour}, + }, nil, + ), + }, + "start aligns with recent metadata query window": { + input: interval{ + start: refTime.Add(-1 * time.Hour), + end: refTime, + }, + expected: []interval{ + { + start: refTime.Add(-time.Hour), + end: time.Date(2023, 1, 15, 7, 30, 0, 0, time.UTC).Add(-expectedSplitGap), + }, + { + start: time.Date(2023, 1, 15, 7, 30, 0, 0, time.UTC), + end: time.Date(2023, 1, 15, 8, 0, 0, 0, time.UTC).Add(-expectedSplitGap), + }, + { + start: time.Date(2023, 1, 15, 8, 0, 0, 0, time.UTC), + end: refTime, + }, + }, + splitInterval: time.Hour, + splitter: newDefaultSplitter( + fakeLimits{ + recentMetadataSplitDuration: map[string]time.Duration{tenantID: 30 * time.Minute}, + recentMetadataQueryWindow: map[string]time.Duration{tenantID: 1 * time.Hour}, + }, nil, + ), + }, + "partially within recent metadata query window": { + input: interval{ + start: refTime.Add(-3 * time.Hour), + end: refTime, + }, + expected: []interval{ + { + start: refTime.Add(-3 * time.Hour), + end: time.Date(2023, 1, 15, 6, 0, 0, 0, time.UTC).Add(-expectedSplitGap), + }, + { + start: time.Date(2023, 1, 15, 6, 0, 0, 0, time.UTC), + end: time.Date(2023, 1, 15, 7, 0, 0, 0, time.UTC).Add(-expectedSplitGap), + }, + { + start: time.Date(2023, 1, 15, 7, 0, 0, 0, time.UTC), + end: refTime.Add(-time.Hour).Add(-expectedSplitGap), + }, + // apply split_recent_metadata_queries_by_interval for recent metadata queries + { + start: refTime.Add(-time.Hour), + end: time.Date(2023, 1, 15, 7, 30, 0, 0, time.UTC).Add(-expectedSplitGap), + }, + { + start: time.Date(2023, 1, 15, 7, 30, 0, 0, time.UTC), + end: time.Date(2023, 1, 15, 8, 0, 0, 0, time.UTC).Add(-expectedSplitGap), + }, + { + start: time.Date(2023, 1, 15, 8, 0, 0, 0, time.UTC), + end: refTime, + }, + }, + splitInterval: time.Hour, + splitter: newDefaultSplitter( + fakeLimits{ + recentMetadataSplitDuration: map[string]time.Duration{tenantID: 30 * time.Minute}, + recentMetadataQueryWindow: map[string]time.Duration{tenantID: 1 * time.Hour}, + }, nil, + ), + }, + "outside recent metadata query window": { + input: interval{ + start: refTime.Add(-4 * time.Hour), + end: refTime.Add(-2 * time.Hour), + }, + expected: []interval{ + { + start: refTime.Add(-4 * time.Hour), + end: time.Date(2023, 1, 15, 5, 0, 0, 0, time.UTC).Add(-expectedSplitGap), + }, + { + start: time.Date(2023, 1, 15, 5, 0, 0, 0, time.UTC), + end: time.Date(2023, 1, 15, 6, 0, 0, 0, time.UTC).Add(-expectedSplitGap), + }, + { + start: time.Date(2023, 1, 15, 6, 0, 0, 0, time.UTC), + end: refTime.Add(-2 * time.Hour), + }, + }, + splitInterval: time.Hour, + splitter: newDefaultSplitter( + fakeLimits{ + recentMetadataSplitDuration: map[string]time.Duration{tenantID: 30 * time.Minute}, + recentMetadataQueryWindow: map[string]time.Duration{tenantID: 1 * time.Hour}, + }, nil, + ), + }, + "end aligns with recent metadata query window": { + input: interval{ + start: refTime.Add(-3 * time.Hour), + end: refTime.Add(-1 * time.Hour), + }, + expected: []interval{ + { + start: refTime.Add(-3 * time.Hour), + end: time.Date(2023, 1, 15, 6, 0, 0, 0, time.UTC).Add(-expectedSplitGap), + }, + { + start: time.Date(2023, 1, 15, 6, 0, 0, 0, time.UTC), + end: time.Date(2023, 1, 15, 7, 0, 0, 0, time.UTC).Add(-expectedSplitGap), + }, + { + start: time.Date(2023, 1, 15, 7, 0, 0, 0, time.UTC), + end: refTime.Add(-1 * time.Hour), + }, + }, + splitInterval: time.Hour, + splitter: newDefaultSplitter( + fakeLimits{ + recentMetadataSplitDuration: map[string]time.Duration{tenantID: 30 * time.Minute}, + recentMetadataQueryWindow: map[string]time.Duration{tenantID: 1 * time.Hour}, + }, nil, + ), + }, + "recent metadata window not configured": { + input: interval{ + start: refTime.Add(-3 * time.Hour), + end: refTime, + }, + expected: []interval{ + { + start: refTime.Add(-3 * time.Hour), + end: time.Date(2023, 1, 15, 6, 0, 0, 0, time.UTC).Add(-expectedSplitGap), + }, + { + start: time.Date(2023, 1, 15, 6, 0, 0, 0, time.UTC), + end: time.Date(2023, 1, 15, 7, 0, 0, 0, time.UTC).Add(-expectedSplitGap), + }, + { + start: time.Date(2023, 1, 15, 7, 0, 0, 0, time.UTC), + end: time.Date(2023, 1, 15, 8, 0, 0, 0, time.UTC).Add(-expectedSplitGap), + }, + { + start: time.Date(2023, 1, 15, 8, 0, 0, 0, time.UTC), + end: refTime, + }, + }, + splitInterval: time.Hour, + }, + } { + t.Run(name, func(t *testing.T) { + req := tc.requestBuilderFunc(intervals.input.start, intervals.input.end) + var want []queryrangebase.Request + + for _, exp := range intervals.expected { + want = append(want, tc.requestBuilderFunc(exp.start, exp.end)) + } + + if intervals.splitInterval == 0 { + intervals.splitInterval = time.Hour } + + if intervals.splitter == nil { + intervals.splitter = newDefaultSplitter(fakeLimits{}, nil) + } + + splits, err := intervals.splitter.split(refTime, []string{tenantID}, req, intervals.splitInterval) + require.NoError(t, err) + assertSplits(t, want, splits) }) } }) @@ -1610,3 +1923,24 @@ func Test_DoesntDeadlock(t *testing.T) { // Allow for 1% increase in goroutines require.LessOrEqual(t, endingGoroutines, startingGoroutines*101/100) } + +func assertSplits(t *testing.T, want, splits []queryrangebase.Request) { + t.Helper() + + if !assert.Equal(t, want, splits) { + t.Logf("expected and actual do not match\n") + defer t.Fail() + + if len(want) != len(splits) { + t.Logf("expected %d splits, got %d\n", len(want), len(splits)) + return + } + + for j := 0; j < len(want); j++ { + exp := want[j] + act := splits[j] + equal := assert.Equal(t, exp, act) + t.Logf("\t#%d [matches: %v]: expected %q/%q got %q/%q\n", j, equal, exp.GetStart(), exp.GetEnd(), act.GetStart(), act.GetEnd()) + } + } +} diff --git a/pkg/querier/queryrange/splitters.go b/pkg/querier/queryrange/splitters.go index 0aaecf35cb96d..eddcc10edf491 100644 --- a/pkg/querier/queryrange/splitters.go +++ b/pkg/querier/queryrange/splitters.go @@ -91,31 +91,56 @@ func (s *defaultSplitter) split(execTime time.Time, tenantIDs []string, req quer } var ( - ingesterSplits []queryrangebase.Request - origStart = req.GetStart().UTC() - origEnd = req.GetEnd().UTC() + splitsBeforeRebound []queryrangebase.Request + origStart = req.GetStart().UTC() + origEnd = req.GetEnd().UTC() + start, end = origStart, origEnd + + reboundOrigQuery bool + splitIntervalBeforeRebound time.Duration ) - start, end, needsIngesterSplits := ingesterQueryBounds(execTime, s.iqo, req) + switch req.(type) { + // not applying `split_ingester_queries_by_interval` for metadata queries since it solves a different problem of reducing the subqueries sent to the ingesters. + // we instead prefer `split_recent_metadata_queries_by_interval` for metadata queries which favours shorter subqueries to improve cache effectiveness. + // even though the number of subqueries increase, caching should deamplify it overtime. + case *LokiSeriesRequest, *LabelRequest: + var ( + recentMetadataQueryWindow = validation.MaxDurationOrZeroPerTenant(tenantIDs, s.limits.RecentMetadataQueryWindow) + recentMetadataQuerySplitInterval = validation.MaxDurationOrZeroPerTenant(tenantIDs, s.limits.RecentMetadataQuerySplitDuration) + ) + + // if either of them are not configured, we fallback to the default split interval for the entire query length. + if recentMetadataQueryWindow == 0 || recentMetadataQuerySplitInterval == 0 { + break + } - if ingesterQueryInterval := validation.MaxDurationOrZeroPerTenant(tenantIDs, s.limits.IngesterQuerySplitDuration); ingesterQueryInterval != 0 && needsIngesterSplits { - // perform splitting using special interval (`split_ingester_queries_by_interval`) - util.ForInterval(ingesterQueryInterval, start, end, endTimeInclusive, factory) + start, end, reboundOrigQuery = recentMetadataQueryBounds(execTime, recentMetadataQueryWindow, req) + splitIntervalBeforeRebound = recentMetadataQuerySplitInterval + default: + if ingesterQueryInterval := validation.MaxDurationOrZeroPerTenant(tenantIDs, s.limits.IngesterQuerySplitDuration); ingesterQueryInterval != 0 { + start, end, reboundOrigQuery = ingesterQueryBounds(execTime, s.iqo, req) + splitIntervalBeforeRebound = ingesterQueryInterval + } + } - // rebound after ingester queries have been split out + if reboundOrigQuery { + util.ForInterval(splitIntervalBeforeRebound, start, end, endTimeInclusive, factory) + + // rebound after query portion within ingester query window or recent metadata query window has been split out end = start - start = req.GetStart().UTC() + start = origStart if endTimeInclusive { end = end.Add(-util.SplitGap) } - // query only overlaps ingester query window, nothing more to do + // query only overlaps ingester query window or recent metadata query window, nothing more to do if start.After(end) || start.Equal(end) { return reqs, nil } // copy the splits, reset the results - ingesterSplits = reqs + splitsBeforeRebound = reqs reqs = nil } else { start = origStart @@ -123,10 +148,10 @@ func (s *defaultSplitter) split(execTime time.Time, tenantIDs []string, req quer } // perform splitting over the rest of the time range - util.ForInterval(interval, origStart, end, endTimeInclusive, factory) + util.ForInterval(interval, start, end, endTimeInclusive, factory) - // move the ingester splits to the end to maintain correct order - reqs = append(reqs, ingesterSplits...) + // move the ingester or recent metadata splits to the end to maintain correct order + reqs = append(reqs, splitsBeforeRebound...) return reqs, nil } @@ -270,6 +295,22 @@ func (s *metricQuerySplitter) buildMetricSplits(step int64, interval time.Durati } } +func recentMetadataQueryBounds(execTime time.Time, recentMetadataQueryWindow time.Duration, req queryrangebase.Request) (time.Time, time.Time, bool) { + start, end := req.GetStart().UTC(), req.GetEnd().UTC() + windowStart := execTime.UTC().Add(-recentMetadataQueryWindow) + + // rebound only if the query end is strictly inside the window + if !windowStart.Before(end) { + return start, end, false + } + + if windowStart.Before(start) { + windowStart = start + } + + return windowStart, end, true +} + // ingesterQueryBounds determines if we need to split time ranges overlapping the ingester query window (`query_ingesters_within`) // and retrieve the bounds for those specific splits func ingesterQueryBounds(execTime time.Time, iqo util.IngesterQueryOptions, req queryrangebase.Request) (time.Time, time.Time, bool) { diff --git a/pkg/querier/queryrange/volume_cache.go b/pkg/querier/queryrange/volume_cache.go index 147d61912db90..5ae2af4111150 100644 --- a/pkg/querier/queryrange/volume_cache.go +++ b/pkg/querier/queryrange/volume_cache.go @@ -131,6 +131,7 @@ func NewVolumeCacheMiddleware( }, parallelismForReq, retentionEnabled, + false, metrics, ) } diff --git a/pkg/storage/chunk/cache/resultscache/cache.go b/pkg/storage/chunk/cache/resultscache/cache.go index d05d71837404c..3ea3e727b502d 100644 --- a/pkg/storage/chunk/cache/resultscache/cache.go +++ b/pkg/storage/chunk/cache/resultscache/cache.go @@ -58,6 +58,7 @@ type ResultsCache struct { merger ResponseMerger shouldCacheReq ShouldCacheReqFn shouldCacheRes ShouldCacheResFn + onlyUseEntireExtent bool parallelismForReq func(ctx context.Context, tenantIDs []string, r Request) int } @@ -79,7 +80,7 @@ func NewResultsCache( shouldCacheRes ShouldCacheResFn, parallelismForReq func(ctx context.Context, tenantIDs []string, r Request) int, cacheGenNumberLoader CacheGenNumberLoader, - retentionEnabled bool, + retentionEnabled, onlyUseEntireExtent bool, ) *ResultsCache { return &ResultsCache{ logger: logger, @@ -95,6 +96,7 @@ func NewResultsCache( shouldCacheReq: shouldCacheReq, shouldCacheRes: shouldCacheRes, parallelismForReq: parallelismForReq, + onlyUseEntireExtent: onlyUseEntireExtent, } } @@ -334,6 +336,25 @@ func (s ResultsCache) partition(req Request, extents []Extent) ([]Request, []Res continue } + if s.onlyUseEntireExtent && (start > extent.GetStart() || end < extent.GetEnd()) { + // It is not possible to extract the overlapping portion of an extent for all request types. + // Metadata results for one cannot be extracted as the data portion is just a list of strings with no associated timestamp. + // To avoid returning incorrect results, we only use extents that are entirely within the requested query range. + // + // Start End + // ┌────────────────────────┐ + // │ Req │ + // └────────────────────────┘ + // + // ◄──────────────► only this extent can be used. Remaining portion of the query will be added to requests. + // + // + // ◄──────X───────► cannot be partially extracted. will be discarded if onlyUseEntireExtent is set. + // ◄───────X──────► + // ◄───────────────X──────────────────► + continue + } + // If this extent is tiny and request is not tiny, discard it: more efficient to do a few larger queries. // Hopefully tiny request can make tiny extent into not-so-tiny extent. @@ -353,6 +374,7 @@ func (s ResultsCache) partition(req Request, extents []Extent) ([]Request, []Res if err != nil { return nil, nil, err } + // extract the overlap from the cached extent. cachedResponses = append(cachedResponses, s.extractor.Extract(start, end, res, extent.GetStart(), extent.GetEnd())) start = extent.End diff --git a/pkg/storage/chunk/cache/resultscache/cache_test.go b/pkg/storage/chunk/cache/resultscache/cache_test.go index bacedd2dda6ba..cff371097a681 100644 --- a/pkg/storage/chunk/cache/resultscache/cache_test.go +++ b/pkg/storage/chunk/cache/resultscache/cache_test.go @@ -61,7 +61,6 @@ func TestPartition(t *testing.T) { mkAPIResponse(0, 100, 10), }, }, - { name: "Test with a complete miss.", input: &MockRequest{ @@ -182,6 +181,123 @@ func TestPartition(t *testing.T) { } } +func TestPartition_onlyUseEntireExtent(t *testing.T) { + for _, tc := range []struct { + name string + input Request + prevCachedResponse []Extent + expectedRequests []Request + expectedCachedResponse []Response + }{ + { + name: "overlapping extent - right", + input: &MockRequest{ + Start: time.UnixMilli(0), + End: time.UnixMilli(100), + }, + prevCachedResponse: []Extent{ + mkExtent(60, 120), + }, + expectedRequests: []Request{ + &MockRequest{ + Start: time.UnixMilli(0), + End: time.UnixMilli(100), + }, + }, + }, + { + name: "overlapping extent - left", + input: &MockRequest{ + Start: time.UnixMilli(20), + End: time.UnixMilli(100), + }, + prevCachedResponse: []Extent{ + mkExtent(0, 50), + }, + expectedRequests: []Request{ + &MockRequest{ + Start: time.UnixMilli(20), + End: time.UnixMilli(100), + }, + }, + }, + { + name: "overlapping extent larger than the request", + input: &MockRequest{ + Start: time.UnixMilli(20), + End: time.UnixMilli(100), + }, + prevCachedResponse: []Extent{ + mkExtent(0, 120), + }, + expectedRequests: []Request{ + &MockRequest{ + Start: time.UnixMilli(20), + End: time.UnixMilli(100), + }, + }, + }, + { + name: "overlapping extent within the requested query range", + input: &MockRequest{ + Start: time.UnixMilli(0), + End: time.UnixMilli(120), + }, + prevCachedResponse: []Extent{ + mkExtent(0, 100), + }, + expectedRequests: []Request{ + &MockRequest{ + Start: time.UnixMilli(100), + End: time.UnixMilli(120), + }, + }, + expectedCachedResponse: []Response{ + mkAPIResponse(0, 100, 10), + }, + }, + { + name: "multiple overlapping extents", + input: &MockRequest{ + Start: time.UnixMilli(50), + End: time.UnixMilli(200), + }, + prevCachedResponse: []Extent{ + mkExtent(0, 80), + mkExtent(100, 150), + mkExtent(150, 180), + mkExtent(200, 250), + }, + expectedRequests: []Request{ + &MockRequest{ + Start: time.UnixMilli(50), + End: time.UnixMilli(100), + }, + &MockRequest{ + Start: time.UnixMilli(180), + End: time.UnixMilli(200), + }, + }, + expectedCachedResponse: []Response{ + mkAPIResponse(100, 150, 10), + mkAPIResponse(150, 180, 10), + }, + }, + } { + t.Run(tc.name, func(t *testing.T) { + s := ResultsCache{ + extractor: MockExtractor{}, + minCacheExtent: 10, + onlyUseEntireExtent: true, + } + reqs, resps, err := s.partition(tc.input, tc.prevCachedResponse) + require.Nil(t, err) + require.Equal(t, tc.expectedRequests, reqs) + require.Equal(t, tc.expectedCachedResponse, resps) + }) + } +} + func TestHandleHit(t *testing.T) { for _, tc := range []struct { name string @@ -491,6 +607,7 @@ func TestResultsCacheMaxFreshness(t *testing.T) { }, nil, false, + false, ) require.NoError(t, err) @@ -534,6 +651,7 @@ func Test_resultsCache_MissingData(t *testing.T) { }, nil, false, + false, ) require.NoError(t, err) ctx := context.Background() diff --git a/pkg/validation/limits.go b/pkg/validation/limits.go index 262631643c723..13885c0fcb52d 100644 --- a/pkg/validation/limits.go +++ b/pkg/validation/limits.go @@ -107,14 +107,16 @@ type Limits struct { QueryTimeout model.Duration `yaml:"query_timeout" json:"query_timeout"` // Query frontend enforced limits. The default is actually parameterized by the queryrange config. - QuerySplitDuration model.Duration `yaml:"split_queries_by_interval" json:"split_queries_by_interval"` - MetadataQuerySplitDuration model.Duration `yaml:"split_metadata_queries_by_interval" json:"split_metadata_queries_by_interval"` - IngesterQuerySplitDuration model.Duration `yaml:"split_ingester_queries_by_interval" json:"split_ingester_queries_by_interval"` - MinShardingLookback model.Duration `yaml:"min_sharding_lookback" json:"min_sharding_lookback"` - MaxQueryBytesRead flagext.ByteSize `yaml:"max_query_bytes_read" json:"max_query_bytes_read"` - MaxQuerierBytesRead flagext.ByteSize `yaml:"max_querier_bytes_read" json:"max_querier_bytes_read"` - VolumeEnabled bool `yaml:"volume_enabled" json:"volume_enabled" doc:"description=Enable log-volume endpoints."` - VolumeMaxSeries int `yaml:"volume_max_series" json:"volume_max_series" doc:"description=The maximum number of aggregated series in a log-volume response"` + QuerySplitDuration model.Duration `yaml:"split_queries_by_interval" json:"split_queries_by_interval"` + MetadataQuerySplitDuration model.Duration `yaml:"split_metadata_queries_by_interval" json:"split_metadata_queries_by_interval"` + RecentMetadataQuerySplitDuration model.Duration `yaml:"split_recent_metadata_queries_by_interval" json:"split_recent_metadata_queries_by_interval"` + RecentMetadataQueryWindow model.Duration `yaml:"recent_metadata_query_window" json:"recent_metadata_query_window"` + IngesterQuerySplitDuration model.Duration `yaml:"split_ingester_queries_by_interval" json:"split_ingester_queries_by_interval"` + MinShardingLookback model.Duration `yaml:"min_sharding_lookback" json:"min_sharding_lookback"` + MaxQueryBytesRead flagext.ByteSize `yaml:"max_query_bytes_read" json:"max_query_bytes_read"` + MaxQuerierBytesRead flagext.ByteSize `yaml:"max_querier_bytes_read" json:"max_querier_bytes_read"` + VolumeEnabled bool `yaml:"volume_enabled" json:"volume_enabled" doc:"description=Enable log-volume endpoints."` + VolumeMaxSeries int `yaml:"volume_max_series" json:"volume_max_series" doc:"description=The maximum number of aggregated series in a log-volume response"` // Ruler defaults and limits. RulerMaxRulesPerRuleGroup int `yaml:"ruler_max_rules_per_rule_group" json:"ruler_max_rules_per_rule_group"` @@ -306,13 +308,14 @@ func (l *Limits) RegisterFlags(f *flag.FlagSet) { _ = l.QuerySplitDuration.Set("1h") f.Var(&l.QuerySplitDuration, "querier.split-queries-by-interval", "Split queries by a time interval and execute in parallel. The value 0 disables splitting by time. This also determines how cache keys are chosen when result caching is enabled.") - // with metadata caching, it is not possible to extract a subset of labels/series from a cached extent because unlike samples they are not associated with a timestamp. - // as a result, we could return inaccurate results. example: returning results from an entire 1h extent for a 5m query - // Setting max_metadata_cache_freshness to 24h should help us avoid caching recent data and preseve the correctness. - // For the portion of the request beyond the freshness window, granularity of the cached metadata results is determined by split_metadata_queries_by_interval. _ = l.MetadataQuerySplitDuration.Set("24h") f.Var(&l.MetadataQuerySplitDuration, "querier.split-metadata-queries-by-interval", "Split metadata queries by a time interval and execute in parallel. The value 0 disables splitting metadata queries by time. This also determines how cache keys are chosen when label/series result caching is enabled.") + _ = l.RecentMetadataQuerySplitDuration.Set("1h") + f.Var(&l.RecentMetadataQuerySplitDuration, "experimental.querier.split-recent-metadata-queries-by-interval", "Experimental. Split interval to use for the portion of metadata request that falls within `recent_metadata_query_window`. Rest of the request which is outside the window still uses `split_metadata_queries_by_interval`. If set to 0, the entire request defaults to using a split interval of `split_metadata_queries_by_interval.`.") + + f.Var(&l.RecentMetadataQueryWindow, "experimental.querier.recent-metadata-query-window", "Experimental. Metadata query window inside which `split_recent_metadata_queries_by_interval` gets applied, portion of the metadata request that falls in this window is split using `split_recent_metadata_queries_by_interval`. The value 0 disables using a different split interval for recent metadata queries.\n\nThis is added to improve cacheability of recent metadata queries. Query split interval also determines the interval used in cache key. The default split interval of 24h is useful for caching long queries, each cache key holding 1 day's results. But metadata queries are often shorter than 24h, to cache them effectively we need a smaller split interval. `recent_metadata_query_window` along with `split_recent_metadata_queries_by_interval` help configure a shorter split interval for recent metadata queries.") + _ = l.IngesterQuerySplitDuration.Set("0s") f.Var(&l.IngesterQuerySplitDuration, "querier.split-ingester-queries-by-interval", "Interval to use for time-based splitting when a request is within the `query_ingesters_within` window; defaults to `split-queries-by-interval` by setting to 0.") @@ -598,6 +601,16 @@ func (o *Overrides) MetadataQuerySplitDuration(userID string) time.Duration { return time.Duration(o.getOverridesForUser(userID).MetadataQuerySplitDuration) } +// RecentMetadataQuerySplitDuration returns the tenant specific splitby interval for recent metadata queries. +func (o *Overrides) RecentMetadataQuerySplitDuration(userID string) time.Duration { + return time.Duration(o.getOverridesForUser(userID).RecentMetadataQuerySplitDuration) +} + +// RecentMetadataQueryWindow returns the tenant specific time window used to determine recent metadata queries. +func (o *Overrides) RecentMetadataQueryWindow(userID string) time.Duration { + return time.Duration(o.getOverridesForUser(userID).RecentMetadataQueryWindow) +} + // IngesterQuerySplitDuration returns the tenant specific splitby interval applied in the query frontend when querying // during the `query_ingesters_within` window. func (o *Overrides) IngesterQuerySplitDuration(userID string) time.Duration { From 256f3971052848963fc4d4c9e24a346afbe1b32c Mon Sep 17 00:00:00 2001 From: Zirko <64951262+QuantumEnigmaa@users.noreply.github.com> Date: Wed, 14 Feb 2024 14:24:04 +0100 Subject: [PATCH 06/19] Helm: bump chart version (#11932) Signed-off-by: QuantumEnigmaa Co-authored-by: Michel Hollands <42814411+MichelHollands@users.noreply.github.com> --- production/helm/loki/Chart.yaml | 2 +- production/helm/loki/README.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/production/helm/loki/Chart.yaml b/production/helm/loki/Chart.yaml index 183333ba652f4..d3b9186bfa77c 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.2 +version: 5.42.3 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 5b6cbe5a2fa1b..a9766865305dd 100644 --- a/production/helm/loki/README.md +++ b/production/helm/loki/README.md @@ -1,6 +1,6 @@ # loki -![Version: 5.42.2](https://img.shields.io/badge/Version-5.42.2-informational?style=flat-square) ![Type: application](https://img.shields.io/badge/Type-application-informational?style=flat-square) ![AppVersion: 2.9.4](https://img.shields.io/badge/AppVersion-2.9.4-informational?style=flat-square) +![Version: 5.42.3](https://img.shields.io/badge/Version-5.42.3-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 From 687978826f065de2d9501b55a41104ad89b3e321 Mon Sep 17 00:00:00 2001 From: Roman Danko Date: Wed, 14 Feb 2024 15:41:37 +0100 Subject: [PATCH 07/19] Helm: Allow to define resources for GrafanaAgent pods (#11851) Co-authored-by: Michel Hollands <42814411+MichelHollands@users.noreply.github.com> --- CHANGELOG.md | 1 + docs/sources/setup/install/helm/reference.md | 9 +++++++++ production/helm/loki/CHANGELOG.md | 5 +++++ production/helm/loki/Chart.yaml | 2 +- production/helm/loki/README.md | 2 +- .../helm/loki/templates/monitoring/grafana-agent.yaml | 4 ++++ production/helm/loki/values.yaml | 7 +++++++ 7 files changed, 28 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 7f091ed06f883..ca45604012430 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -6,6 +6,7 @@ ##### Enhancements +* [11851](https://github.com/grafana/loki/pull/11851) **elcomtik**: Helm: Allow the definition of resources for GrafanaAgent pods. * [11819](https://github.com/grafana/loki/pull/11819) **jburnham**: Ruler: Add the ability to disable the `X-Scope-OrgId` tenant identification header in remote write requests. * [11633](https://github.com/grafana/loki/pull/11633) **cyriltovena**: Add profiling integrations to tracing instrumentation. * [11571](https://github.com/grafana/loki/pull/11571) **MichelHollands**: Add a metrics.go log line for requests from querier to ingester diff --git a/docs/sources/setup/install/helm/reference.md b/docs/sources/setup/install/helm/reference.md index e687a560ef715..e7dbfdbdd3f65 100644 --- a/docs/sources/setup/install/helm/reference.md +++ b/docs/sources/setup/install/helm/reference.md @@ -2806,6 +2806,15 @@ true
 null
 
+ + + + monitoring.selfMonitoring.grafanaAgent.resources + object + Resource requests and limits for the grafanaAgent pods +
+{}
+
diff --git a/production/helm/loki/CHANGELOG.md b/production/helm/loki/CHANGELOG.md index 068d37a495530..47d8f6333e4e9 100644 --- a/production/helm/loki/CHANGELOG.md +++ b/production/helm/loki/CHANGELOG.md @@ -13,6 +13,11 @@ 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.43.0 + +- [ENHANCEMENT] Allow the definition of resources for GrafanaAgent pods + ## 5.42.3 - [BUGFIX] Added condition for `egress-discovery` networkPolicies and ciliumNetworkPolicies. diff --git a/production/helm/loki/Chart.yaml b/production/helm/loki/Chart.yaml index d3b9186bfa77c..ffa62c88d5cd3 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.3 +version: 5.43.0 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 a9766865305dd..5db87e6d801e0 100644 --- a/production/helm/loki/README.md +++ b/production/helm/loki/README.md @@ -1,6 +1,6 @@ # loki -![Version: 5.42.3](https://img.shields.io/badge/Version-5.42.3-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.43.0](https://img.shields.io/badge/Version-5.43.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/templates/monitoring/grafana-agent.yaml b/production/helm/loki/templates/monitoring/grafana-agent.yaml index 010d9604aab70..a047e5f862517 100644 --- a/production/helm/loki/templates/monitoring/grafana-agent.yaml +++ b/production/helm/loki/templates/monitoring/grafana-agent.yaml @@ -30,6 +30,10 @@ spec: {{- include "loki.selectorLabels" $ | nindent 8 }} {{- end }} {{- end }} + {{- with .resources }} + resources: + {{- toYaml . | nindent 4 }} + {{- end }} {{- with .tolerations }} tolerations: {{- toYaml . | nindent 4 }} diff --git a/production/helm/loki/values.yaml b/production/helm/loki/values.yaml index e2937af382a7d..6f5a779bc8111 100644 --- a/production/helm/loki/values.yaml +++ b/production/helm/loki/values.yaml @@ -671,6 +671,13 @@ monitoring: enableConfigReadAPI: false # -- The name of the PriorityClass for GrafanaAgent pods priorityClassName: null + # -- Resource requests and limits for the grafanaAgent pods + resources: {} + # limits: + # memory: 200Mi + # requests: + # cpu: 50m + # memory: 100Mi # -- Tolerations for GrafanaAgent pods tolerations: [] # PodLogs configuration From 3c06b360ecbfb7b8ca56d01d59fb69661510d9b9 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 14 Feb 2024 11:47:11 -0800 Subject: [PATCH 08/19] meta & removal support for bloom compaction (#11941) Signed-off-by: Owen Diehl --- pkg/bloomcompactor/bloomcompactor.go | 2 +- pkg/bloomcompactor/controller.go | 380 ++++++++++++++---- pkg/bloomcompactor/controller_test.go | 165 +++++++- pkg/bloomgateway/util_test.go | 4 +- pkg/storage/bloom/v1/bounds.go | 8 +- pkg/storage/bloom/v1/bounds_test.go | 2 +- pkg/storage/bloom/v1/fuse_test.go | 1 - .../stores/shipper/bloomshipper/client.go | 59 ++- .../shipper/bloomshipper/client_test.go | 8 +- .../shipper/bloomshipper/fetcher_test.go | 4 +- .../stores/shipper/bloomshipper/shipper.go | 2 +- .../shipper/bloomshipper/shipper_test.go | 2 +- .../stores/shipper/bloomshipper/store_test.go | 4 +- 13 files changed, 526 insertions(+), 115 deletions(-) diff --git a/pkg/bloomcompactor/bloomcompactor.go b/pkg/bloomcompactor/bloomcompactor.go index 566b836609d10..fa3b205aa31b7 100644 --- a/pkg/bloomcompactor/bloomcompactor.go +++ b/pkg/bloomcompactor/bloomcompactor.go @@ -290,7 +290,7 @@ func (c *Compactor) runWorkers(ctx context.Context, ch <-chan tenantTable) error func (c *Compactor) compactTenantTable(ctx context.Context, tt tenantTable) error { level.Info(c.logger).Log("msg", "compacting", "org_id", tt.tenant, "table", tt.table, "ownership", tt.ownershipRange) - return c.controller.buildBlocks(ctx, tt.table, tt.tenant, tt.ownershipRange) + return c.controller.compactTenant(ctx, tt.table, tt.tenant, tt.ownershipRange) } type dayRangeIterator struct { diff --git a/pkg/bloomcompactor/controller.go b/pkg/bloomcompactor/controller.go index 8470fd9ad7082..c29aec86ac5c2 100644 --- a/pkg/bloomcompactor/controller.go +++ b/pkg/bloomcompactor/controller.go @@ -54,7 +54,20 @@ func (s *SimpleBloomController) rwFn() (v1.BlockWriter, v1.BlockReader) { return v1.NewMemoryBlockWriter(indexBuf, bloomsBuf), v1.NewByteReader(indexBuf, bloomsBuf) } -func (s *SimpleBloomController) buildBlocks( +/* +Compaction works as follows, split across many functions for clarity: + 1. Fetch all meta.jsons for the given tenant and table which overlap the ownership range of this compactor. + 2. Load current TSDBs for this tenant/table. + 3. For each live TSDB (there should be only 1, but this works with multiple), find any gaps + (fingerprint ranges) which are not up date, determined by checking other meta.jsons and comparing + the tsdbs they were generated from + their ownership ranges. + 4. Build new bloom blocks for each gap, using the series and chunks from the TSDBs and any existing + blocks which overlap the gaps to accelerate bloom generation. + 5. Write the new blocks and metas to the store. + 6. Determine if any meta.jsons overlap the ownership range but are outdated, and remove them and + their associated blocks if so. +*/ +func (s *SimpleBloomController) compactTenant( ctx context.Context, table config.DayTime, tenant string, @@ -62,23 +75,13 @@ func (s *SimpleBloomController) buildBlocks( ) error { logger := log.With(s.logger, "ownership", ownershipRange, "org_id", tenant, "table", table) - // 1. Resolve TSDBs - tsdbs, err := s.tsdbStore.ResolveTSDBs(ctx, table, tenant) + client, err := s.bloomStore.Client(table.ModelTime()) if err != nil { - level.Error(logger).Log("msg", "failed to resolve tsdbs", "err", err) - return errors.Wrap(err, "failed to resolve tsdbs") - } - - if len(tsdbs) == 0 { - return nil + level.Error(logger).Log("msg", "failed to get client", "err", err, "table", table.String()) + return errors.Wrap(err, "failed to get client") } - ids := make([]tsdb.Identifier, 0, len(tsdbs)) - for _, id := range tsdbs { - ids = append(ids, id) - } - - // 2. Fetch metas + // Fetch source metas to be used in both compaction and cleanup of out-of-date metas+blooms metas, err := s.bloomStore.FetchMetas( ctx, bloomshipper.MetaSearchParams{ @@ -92,31 +95,163 @@ func (s *SimpleBloomController) buildBlocks( return errors.Wrap(err, "failed to get metas") } - // 3. Determine which TSDBs have gaps in the ownership range and need to + // build compaction plans + work, err := s.findOutdatedGaps(ctx, tenant, table, ownershipRange, metas, logger) + if err != nil { + return errors.Wrap(err, "failed to find outdated gaps") + } + + // build new blocks + built, err := s.buildGaps(ctx, tenant, table, client, work, logger) + if err != nil { + return errors.Wrap(err, "failed to build gaps") + } + + // in order to delete outdates metas which only partially fall within the ownership range, + // we need to fetcha all metas in the entire bound range of the first set of metas we've resolved + /* + For instance, we have the following ownership range and we resolve `meta1` in our first Fetch call + because it overlaps the ownership range, we'll need to fetch newer metas that may overlap it in order + to check if it safely can be deleted. This falls partially outside our specific ownership range, but + we can safely run multiple deletes by treating their removal as idempotent. + |-------------ownership range-----------------| + |-------meta1-------| + + we fetch this before possibly deleting meta1 |------| + */ + superset := ownershipRange + for _, meta := range metas { + union := superset.Union(meta.Bounds) + if len(union) > 1 { + level.Error(logger).Log("msg", "meta bounds union is not a single range", "union", union) + return errors.New("meta bounds union is not a single range") + } + superset = union[0] + } + + metas, err = s.bloomStore.FetchMetas( + ctx, + bloomshipper.MetaSearchParams{ + TenantID: tenant, + Interval: bloomshipper.NewInterval(table.Bounds()), + Keyspace: superset, + }, + ) + if err != nil { + level.Error(logger).Log("msg", "failed to get meta superset range", "err", err, "superset", superset) + return errors.Wrap(err, "failed to get meta supseret range") + } + + // combine built and pre-existing metas + // in preparation for removing outdated metas + metas = append(metas, built...) + + outdated := outdatedMetas(metas) + for _, meta := range outdated { + for _, block := range meta.Blocks { + if err := client.DeleteBlocks(ctx, []bloomshipper.BlockRef{block}); err != nil { + if client.IsObjectNotFoundErr(err) { + level.Debug(logger).Log("msg", "block not found while attempting delete, continuing", "block", block) + continue + } + + level.Error(logger).Log("msg", "failed to delete blocks", "err", err) + return errors.Wrap(err, "failed to delete blocks") + } + } + + if err := client.DeleteMetas(ctx, []bloomshipper.MetaRef{meta.MetaRef}); err != nil { + if client.IsObjectNotFoundErr(err) { + level.Debug(logger).Log("msg", "meta not found while attempting delete, continuing", "meta", meta.MetaRef) + } else { + level.Error(logger).Log("msg", "failed to delete metas", "err", err) + return errors.Wrap(err, "failed to delete metas") + } + } + } + + level.Debug(logger).Log("msg", "finished compaction") + return nil + +} + +func (s *SimpleBloomController) findOutdatedGaps( + ctx context.Context, + tenant string, + table config.DayTime, + ownershipRange v1.FingerprintBounds, + metas []bloomshipper.Meta, + logger log.Logger, +) ([]blockPlan, error) { + // Resolve TSDBs + tsdbs, err := s.tsdbStore.ResolveTSDBs(ctx, table, tenant) + if err != nil { + level.Error(logger).Log("msg", "failed to resolve tsdbs", "err", err) + return nil, errors.Wrap(err, "failed to resolve tsdbs") + } + + if len(tsdbs) == 0 { + return nil, nil + } + + // Determine which TSDBs have gaps in the ownership range and need to // be processed. - tsdbsWithGaps, err := gapsBetweenTSDBsAndMetas(ownershipRange, ids, metas) + tsdbsWithGaps, err := gapsBetweenTSDBsAndMetas(ownershipRange, tsdbs, metas) if err != nil { level.Error(logger).Log("msg", "failed to find gaps", "err", err) - return errors.Wrap(err, "failed to find gaps") + return nil, errors.Wrap(err, "failed to find gaps") } if len(tsdbsWithGaps) == 0 { level.Debug(logger).Log("msg", "blooms exist for all tsdbs") - return nil + return nil, nil } work, err := blockPlansForGaps(tsdbsWithGaps, metas) if err != nil { level.Error(logger).Log("msg", "failed to create plan", "err", err) - return errors.Wrap(err, "failed to create plan") + return nil, errors.Wrap(err, "failed to create plan") + } + + return work, nil +} + +func (s *SimpleBloomController) loadWorkForGap( + ctx context.Context, + table config.DayTime, + tenant string, + id tsdb.Identifier, + gap gapWithBlocks, +) (v1.CloseableIterator[*v1.Series], v1.CloseableIterator[*bloomshipper.CloseableBlockQuerier], error) { + // load a series iterator for the gap + seriesItr, err := s.tsdbStore.LoadTSDB(ctx, table, tenant, id, gap.bounds) + if err != nil { + return nil, nil, errors.Wrap(err, "failed to load tsdb") + } + + // load a blocks iterator for the gap + fetcher, err := s.bloomStore.Fetcher(table.ModelTime()) + if err != nil { + return nil, nil, errors.Wrap(err, "failed to get fetcher") + } + + blocksIter, err := newBatchedBlockLoader(ctx, fetcher, gap.blocks) + if err != nil { + return nil, nil, errors.Wrap(err, "failed to load blocks") } - nGramSize := uint64(s.limits.BloomNGramLength(tenant)) - nGramSkip := uint64(s.limits.BloomNGramSkip(tenant)) - maxBlockSize := uint64(s.limits.BloomCompactorMaxBlockSize(tenant)) - blockOpts := v1.NewBlockOptions(nGramSize, nGramSkip, maxBlockSize) + return seriesItr, blocksIter, nil +} - // 4. Generate Blooms +func (s *SimpleBloomController) buildGaps( + ctx context.Context, + tenant string, + table config.DayTime, + client bloomshipper.Client, + work []blockPlan, + logger log.Logger, +) ([]bloomshipper.Meta, error) { + // Generate Blooms // Now that we have the gaps, we will generate a bloom block for each gap. // We can accelerate this by using existing blocks which may already contain // needed chunks in their blooms, for instance after a new TSDB version is generated @@ -127,19 +262,37 @@ func (s *SimpleBloomController) buildBlocks( // accelerate bloom generation for the new blocks. var ( - blockCt int - tsdbCt = len(work) + blockCt int + tsdbCt = len(work) + nGramSize = uint64(s.limits.BloomNGramLength(tenant)) + nGramSkip = uint64(s.limits.BloomNGramSkip(tenant)) + maxBlockSize = uint64(s.limits.BloomCompactorMaxBlockSize(tenant)) + blockOpts = v1.NewBlockOptions(nGramSize, nGramSkip, maxBlockSize) + created []bloomshipper.Meta ) for _, plan := range work { - for _, gap := range plan.gaps { + for i := range plan.gaps { + gap := plan.gaps[i] + + meta := bloomshipper.Meta{ + MetaRef: bloomshipper.MetaRef{ + Ref: bloomshipper.Ref{ + TenantID: tenant, + TableName: table.String(), + Bounds: gap.bounds, + }, + }, + Sources: []tsdb.SingleTenantTSDBIdentifier{plan.tsdb}, + } + // Fetch blocks that aren't up to date but are in the desired fingerprint range // to try and accelerate bloom creation seriesItr, blocksIter, err := s.loadWorkForGap(ctx, table, tenant, plan.tsdb, gap) if err != nil { level.Error(logger).Log("msg", "failed to get series and blocks", "err", err) - return errors.Wrap(err, "failed to get series and blocks") + return nil, errors.Wrap(err, "failed to get series and blocks") } gen := NewSimpleBloomGenerator( @@ -159,24 +312,17 @@ func (s *SimpleBloomController) buildBlocks( // TODO(owen-d): metrics level.Error(logger).Log("msg", "failed to generate bloom", "err", err) s.closeLoadedBlocks(loaded, blocksIter) - return errors.Wrap(err, "failed to generate bloom") - } - - client, err := s.bloomStore.Client(table.ModelTime()) - if err != nil { - level.Error(logger).Log("msg", "failed to get client", "err", err) - s.closeLoadedBlocks(loaded, blocksIter) - return errors.Wrap(err, "failed to get client") + return nil, errors.Wrap(err, "failed to generate bloom") } for newBlocks.Next() && newBlocks.Err() == nil { blockCt++ blk := newBlocks.At() - built, err := bloomshipper.BlockFrom(tenant, table.Table(), blk) + built, err := bloomshipper.BlockFrom(tenant, table.String(), blk) if err != nil { level.Error(logger).Log("msg", "failed to build block", "err", err) - return errors.Wrap(err, "failed to build block") + return nil, errors.Wrap(err, "failed to build block") } if err := client.PutBlock( @@ -185,55 +331,153 @@ func (s *SimpleBloomController) buildBlocks( ); err != nil { level.Error(logger).Log("msg", "failed to write block", "err", err) s.closeLoadedBlocks(loaded, blocksIter) - return errors.Wrap(err, "failed to write block") + return nil, errors.Wrap(err, "failed to write block") } + + meta.Blocks = append(meta.Blocks, built.BlockRef) } if err := newBlocks.Err(); err != nil { // TODO(owen-d): metrics level.Error(logger).Log("msg", "failed to generate bloom", "err", err) s.closeLoadedBlocks(loaded, blocksIter) - return errors.Wrap(err, "failed to generate bloom") + return nil, errors.Wrap(err, "failed to generate bloom") } // Close pre-existing blocks s.closeLoadedBlocks(loaded, blocksIter) + + // Write the new meta + ref, err := bloomshipper.MetaRefFrom(tenant, table.String(), gap.bounds, meta.Sources, meta.Blocks) + if err != nil { + level.Error(logger).Log("msg", "failed to checksum meta", "err", err) + return nil, errors.Wrap(err, "failed to checksum meta") + } + meta.MetaRef = ref + + if err := client.PutMeta(ctx, meta); err != nil { + level.Error(logger).Log("msg", "failed to write meta", "err", err) + return nil, errors.Wrap(err, "failed to write meta") + } + created = append(created, meta) } } - // TODO(owen-d): build meta from blocks - // TODO(owen-d): reap tombstones, old metas - level.Debug(logger).Log("msg", "finished bloom generation", "blocks", blockCt, "tsdbs", tsdbCt) - return nil - + return created, nil } -func (s *SimpleBloomController) loadWorkForGap( - ctx context.Context, - table config.DayTime, - tenant string, - id tsdb.Identifier, - gap gapWithBlocks, -) (v1.CloseableIterator[*v1.Series], v1.CloseableIterator[*bloomshipper.CloseableBlockQuerier], error) { - // load a series iterator for the gap - seriesItr, err := s.tsdbStore.LoadTSDB(ctx, table, tenant, id, gap.bounds) - if err != nil { - return nil, nil, errors.Wrap(err, "failed to load tsdb") +// outdatedMetas returns metas that are outdated and need to be removed, +// determined by if their entire ownership range is covered by other metas with newer +// TSDBs +func outdatedMetas(metas []bloomshipper.Meta) (outdated []bloomshipper.Meta) { + // first, ensure data is sorted so we can take advantage of that + sort.Slice(metas, func(i, j int) bool { + return metas[i].Bounds.Less(metas[j].Bounds) + }) + + // NB(owen-d): time complexity shouldn't be a problem + // given the number of metas should be low (famous last words, i know). + for i := range metas { + a := metas[i] + + var overlaps []v1.FingerprintBounds + + for j := range metas { + if j == i { + continue + } + + b := metas[j] + intersection := a.Bounds.Intersection(b.Bounds) + if intersection == nil { + if a.Bounds.Cmp(b.Bounds.Min) == v1.After { + // All subsequent metas will be newer, so we can break + break + } + // otherwise, just check the next meta + continue + } + + // we can only remove older data, not data which may be newer + if !tsdbsStrictlyNewer(b.Sources, a.Sources) { + continue + } + + // because we've sorted the metas, we only have to test overlaps against the last + // overlap we found (if any) + if len(overlaps) == 0 { + overlaps = append(overlaps, *intersection) + continue + } + + // best effort at merging overlaps first pass + last := overlaps[len(overlaps)-1] + overlaps = append(overlaps[:len(overlaps)-1], last.Union(*intersection)...) + + } + + if coversFullRange(a.Bounds, overlaps) { + outdated = append(outdated, a) + } } + return +} - // load a blocks iterator for the gap - fetcher, err := s.bloomStore.Fetcher(table.ModelTime()) - if err != nil { - return nil, nil, errors.Wrap(err, "failed to get fetcher") +func coversFullRange(bounds v1.FingerprintBounds, overlaps []v1.FingerprintBounds) bool { + // if there are no overlaps, the range is not covered + if len(overlaps) == 0 { + return false } - blocksIter, err := newBatchedBlockLoader(ctx, fetcher, gap.blocks) - if err != nil { - return nil, nil, errors.Wrap(err, "failed to load blocks") + // keep track of bounds which need to be filled in order + // for the overlaps to cover the full range + missing := []v1.FingerprintBounds{bounds} + ignores := make(map[int]bool) + for _, overlap := range overlaps { + var i int + for { + if i >= len(missing) { + break + } + + if ignores[i] { + i++ + continue + } + + remaining := missing[i].Unless(overlap) + switch len(remaining) { + case 0: + // this range is covered, ignore it + ignores[i] = true + case 1: + // this range is partially covered, updated it + missing[i] = remaining[0] + case 2: + // this range has been partially covered in the middle, + // split it into two ranges and append + ignores[i] = true + missing = append(missing, remaining...) + } + i++ + } + } - return seriesItr, blocksIter, nil + return len(ignores) == len(missing) +} + +// tsdbStrictlyNewer returns if all of the tsdbs in a are newer than all of the tsdbs in b +func tsdbsStrictlyNewer(as, bs []tsdb.SingleTenantTSDBIdentifier) bool { + for _, a := range as { + for _, b := range bs { + if a.TS.Before(b.TS) { + return false + } + } + } + return true } func (s *SimpleBloomController) closeLoadedBlocks(toClose []io.Closer, it v1.CloseableIterator[*bloomshipper.CloseableBlockQuerier]) { @@ -275,7 +519,7 @@ type gapWithBlocks struct { // of the same chunks we need to ensure are indexed, just from previous tsdb iterations. // This is a performance optimization to avoid expensive re-reindexing type blockPlan struct { - tsdb tsdb.Identifier + tsdb tsdb.SingleTenantTSDBIdentifier gaps []gapWithBlocks } @@ -353,7 +597,7 @@ func blockPlansForGaps(tsdbs []tsdbGaps, metas []bloomshipper.Meta) ([]blockPlan // Used to signal the gaps that need to be populated for a tsdb type tsdbGaps struct { - tsdb tsdb.Identifier + tsdb tsdb.SingleTenantTSDBIdentifier gaps []v1.FingerprintBounds } @@ -361,7 +605,7 @@ type tsdbGaps struct { // 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, + tsdbs []tsdb.SingleTenantTSDBIdentifier, metas []bloomshipper.Meta, ) (res []tsdbGaps, err error) { for _, db := range tsdbs { diff --git a/pkg/bloomcompactor/controller_test.go b/pkg/bloomcompactor/controller_test.go index 0660a5b601eea..72653c292b18b 100644 --- a/pkg/bloomcompactor/controller_test.go +++ b/pkg/bloomcompactor/controller_test.go @@ -142,14 +142,14 @@ func Test_gapsBetweenTSDBsAndMetas(t *testing.T) { err bool exp []tsdbGaps ownershipRange v1.FingerprintBounds - tsdbs []tsdb.Identifier + tsdbs []tsdb.SingleTenantTSDBIdentifier metas []bloomshipper.Meta }{ { desc: "non-overlapping tsdbs and metas", err: true, ownershipRange: v1.NewBounds(0, 10), - tsdbs: []tsdb.Identifier{tsdbID(0)}, + tsdbs: []tsdb.SingleTenantTSDBIdentifier{tsdbID(0)}, metas: []bloomshipper.Meta{ genMeta(11, 20, []int{0}, nil), }, @@ -157,7 +157,7 @@ func Test_gapsBetweenTSDBsAndMetas(t *testing.T) { { desc: "single tsdb", ownershipRange: v1.NewBounds(0, 10), - tsdbs: []tsdb.Identifier{tsdbID(0)}, + tsdbs: []tsdb.SingleTenantTSDBIdentifier{tsdbID(0)}, metas: []bloomshipper.Meta{ genMeta(4, 8, []int{0}, nil), }, @@ -174,7 +174,7 @@ func Test_gapsBetweenTSDBsAndMetas(t *testing.T) { { desc: "multiple tsdbs with separate blocks", ownershipRange: v1.NewBounds(0, 10), - tsdbs: []tsdb.Identifier{tsdbID(0), tsdbID(1)}, + tsdbs: []tsdb.SingleTenantTSDBIdentifier{tsdbID(0), tsdbID(1)}, metas: []bloomshipper.Meta{ genMeta(0, 5, []int{0}, nil), genMeta(6, 10, []int{1}, nil), @@ -197,7 +197,7 @@ func Test_gapsBetweenTSDBsAndMetas(t *testing.T) { { desc: "multiple tsdbs with the same blocks", ownershipRange: v1.NewBounds(0, 10), - tsdbs: []tsdb.Identifier{tsdbID(0), tsdbID(1)}, + tsdbs: []tsdb.SingleTenantTSDBIdentifier{tsdbID(0), tsdbID(1)}, metas: []bloomshipper.Meta{ genMeta(0, 5, []int{0, 1}, nil), genMeta(6, 8, []int{1}, nil), @@ -242,7 +242,7 @@ func Test_blockPlansForGaps(t *testing.T) { for _, tc := range []struct { desc string ownershipRange v1.FingerprintBounds - tsdbs []tsdb.Identifier + tsdbs []tsdb.SingleTenantTSDBIdentifier metas []bloomshipper.Meta err bool exp []blockPlan @@ -250,7 +250,7 @@ func Test_blockPlansForGaps(t *testing.T) { { desc: "single overlapping meta+no overlapping block", ownershipRange: v1.NewBounds(0, 10), - tsdbs: []tsdb.Identifier{tsdbID(0)}, + tsdbs: []tsdb.SingleTenantTSDBIdentifier{tsdbID(0)}, metas: []bloomshipper.Meta{ genMeta(5, 20, []int{1}, []bloomshipper.BlockRef{genBlockRef(11, 20)}), }, @@ -268,7 +268,7 @@ func Test_blockPlansForGaps(t *testing.T) { { desc: "single overlapping meta+one overlapping block", ownershipRange: v1.NewBounds(0, 10), - tsdbs: []tsdb.Identifier{tsdbID(0)}, + tsdbs: []tsdb.SingleTenantTSDBIdentifier{tsdbID(0)}, metas: []bloomshipper.Meta{ genMeta(5, 20, []int{1}, []bloomshipper.BlockRef{genBlockRef(9, 20)}), }, @@ -290,7 +290,7 @@ func Test_blockPlansForGaps(t *testing.T) { // but we can trim the range needing generation desc: "trims up to date area", ownershipRange: v1.NewBounds(0, 10), - tsdbs: []tsdb.Identifier{tsdbID(0)}, + tsdbs: []tsdb.SingleTenantTSDBIdentifier{tsdbID(0)}, metas: []bloomshipper.Meta{ genMeta(9, 20, []int{0}, []bloomshipper.BlockRef{genBlockRef(9, 20)}), // block for same tsdb genMeta(9, 20, []int{1}, []bloomshipper.BlockRef{genBlockRef(9, 20)}), // block for different tsdb @@ -309,7 +309,7 @@ func Test_blockPlansForGaps(t *testing.T) { { desc: "uses old block for overlapping range", ownershipRange: v1.NewBounds(0, 10), - tsdbs: []tsdb.Identifier{tsdbID(0)}, + tsdbs: []tsdb.SingleTenantTSDBIdentifier{tsdbID(0)}, metas: []bloomshipper.Meta{ genMeta(9, 20, []int{0}, []bloomshipper.BlockRef{genBlockRef(9, 20)}), // block for same tsdb genMeta(5, 20, []int{1}, []bloomshipper.BlockRef{genBlockRef(5, 20)}), // block for different tsdb @@ -329,7 +329,7 @@ func Test_blockPlansForGaps(t *testing.T) { { desc: "multi case", ownershipRange: v1.NewBounds(0, 10), - tsdbs: []tsdb.Identifier{tsdbID(0), tsdbID(1)}, // generate for both tsdbs + tsdbs: []tsdb.SingleTenantTSDBIdentifier{tsdbID(0), tsdbID(1)}, // generate for both tsdbs metas: []bloomshipper.Meta{ genMeta(0, 2, []int{0}, []bloomshipper.BlockRef{ genBlockRef(0, 1), @@ -377,7 +377,7 @@ func Test_blockPlansForGaps(t *testing.T) { { desc: "dedupes block refs", ownershipRange: v1.NewBounds(0, 10), - tsdbs: []tsdb.Identifier{tsdbID(0)}, + tsdbs: []tsdb.SingleTenantTSDBIdentifier{tsdbID(0)}, metas: []bloomshipper.Meta{ genMeta(9, 20, []int{1}, []bloomshipper.BlockRef{ genBlockRef(1, 4), @@ -421,3 +421,144 @@ func Test_blockPlansForGaps(t *testing.T) { }) } } + +func Test_coversFullRange(t *testing.T) { + for _, tc := range []struct { + desc string + src v1.FingerprintBounds + overlaps []v1.FingerprintBounds + exp bool + }{ + { + desc: "empty", + src: v1.NewBounds(0, 10), + overlaps: []v1.FingerprintBounds{}, + exp: false, + }, + { + desc: "single_full_range", + src: v1.NewBounds(0, 10), + overlaps: []v1.FingerprintBounds{ + v1.NewBounds(0, 10), + }, + exp: true, + }, + { + desc: "single_partial_range", + src: v1.NewBounds(0, 10), + overlaps: []v1.FingerprintBounds{ + v1.NewBounds(0, 5), + }, + exp: false, + }, + { + desc: "multiple_full_ranges", + src: v1.NewBounds(0, 10), + overlaps: []v1.FingerprintBounds{ + v1.NewBounds(0, 5), + v1.NewBounds(6, 10), + }, + exp: true, + }, + { + desc: "multiple_partial_ranges", + src: v1.NewBounds(0, 10), + overlaps: []v1.FingerprintBounds{ + v1.NewBounds(0, 5), + v1.NewBounds(7, 8), + }, + exp: false, + }, + { + desc: "wraps_partial_range", + src: v1.NewBounds(10, 20), + overlaps: []v1.FingerprintBounds{ + v1.NewBounds(0, 12), + v1.NewBounds(13, 15), + v1.NewBounds(19, 21), + }, + exp: false, + }, + { + desc: "wraps_full_range", + src: v1.NewBounds(10, 20), + overlaps: []v1.FingerprintBounds{ + v1.NewBounds(0, 12), + v1.NewBounds(13, 15), + v1.NewBounds(16, 25), + }, + exp: true, + }, + } { + t.Run(tc.desc, func(t *testing.T) { + require.Equal(t, tc.exp, coversFullRange(tc.src, tc.overlaps)) + }) + } +} + +func Test_OutdatedMetas(t *testing.T) { + gen := func(bounds v1.FingerprintBounds, tsdbTimes ...model.Time) (meta bloomshipper.Meta) { + for _, tsdbTime := range tsdbTimes { + meta.Sources = append(meta.Sources, tsdb.SingleTenantTSDBIdentifier{TS: tsdbTime.Time()}) + } + meta.Bounds = bounds + return meta + } + + for _, tc := range []struct { + desc string + metas []bloomshipper.Meta + exp []bloomshipper.Meta + }{ + { + desc: "no metas", + metas: nil, + exp: nil, + }, + { + desc: "single meta", + metas: []bloomshipper.Meta{ + gen(v1.NewBounds(0, 10), 0), + }, + exp: nil, + }, + { + desc: "single outdated meta", + metas: []bloomshipper.Meta{ + gen(v1.NewBounds(0, 10), 0), + gen(v1.NewBounds(0, 10), 1), + }, + exp: []bloomshipper.Meta{ + gen(v1.NewBounds(0, 10), 0), + }, + }, + { + desc: "single outdated via partitions", + metas: []bloomshipper.Meta{ + gen(v1.NewBounds(0, 5), 0), + gen(v1.NewBounds(6, 10), 0), + gen(v1.NewBounds(0, 10), 1), + }, + exp: []bloomshipper.Meta{ + gen(v1.NewBounds(0, 5), 0), + gen(v1.NewBounds(6, 10), 0), + }, + }, + { + desc: "multi tsdbs", + metas: []bloomshipper.Meta{ + gen(v1.NewBounds(0, 5), 0, 1), + gen(v1.NewBounds(6, 10), 0, 1), + gen(v1.NewBounds(0, 10), 2, 3), + }, + exp: []bloomshipper.Meta{ + gen(v1.NewBounds(0, 5), 0, 1), + gen(v1.NewBounds(6, 10), 0, 1), + }, + }, + } { + t.Run(tc.desc, func(t *testing.T) { + require.Equal(t, tc.exp, outdatedMetas(tc.metas)) + }) + } +} diff --git a/pkg/bloomgateway/util_test.go b/pkg/bloomgateway/util_test.go index 5f4d254e8f045..6850cf31811c0 100644 --- a/pkg/bloomgateway/util_test.go +++ b/pkg/bloomgateway/util_test.go @@ -323,8 +323,8 @@ func createBlocks(t *testing.T, tenant string, n int, from, through model.Time, MetaRef: bloomshipper.MetaRef{ Ref: ref, }, - Tombstones: []bloomshipper.BlockRef{}, - Blocks: []bloomshipper.BlockRef{blockRef}, + BlockTombstones: []bloomshipper.BlockRef{}, + Blocks: []bloomshipper.BlockRef{blockRef}, } block, data, _ := v1.MakeBlock(t, n, fromFp, throughFp, from, through) // Printing fingerprints and the log lines of its chunks comes handy for debugging... diff --git a/pkg/storage/bloom/v1/bounds.go b/pkg/storage/bloom/v1/bounds.go index 507f6e45aefc2..8f3edd914209d 100644 --- a/pkg/storage/bloom/v1/bounds.go +++ b/pkg/storage/bloom/v1/bounds.go @@ -125,10 +125,10 @@ func (b FingerprintBounds) Intersection(target FingerprintBounds) *FingerprintBo // 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} + if b.Less(target) { + return []FingerprintBounds{b, target} } - return []FingerprintBounds{b, target} + return []FingerprintBounds{target, b} } return []FingerprintBounds{ @@ -145,7 +145,7 @@ func (b FingerprintBounds) Unless(target FingerprintBounds) (res []FingerprintBo return []FingerprintBounds{b} } - if b == target { + if b.Within(target) { return nil } diff --git a/pkg/storage/bloom/v1/bounds_test.go b/pkg/storage/bloom/v1/bounds_test.go index 629eac61a2af7..e8362a2b283ff 100644 --- a/pkg/storage/bloom/v1/bounds_test.go +++ b/pkg/storage/bloom/v1/bounds_test.go @@ -98,7 +98,7 @@ func Test_FingerprintBounds_Union(t *testing.T) { }, NewBounds(5, 15).Union(target)) } -func Test_FingerprintBounds_Xor(t *testing.T) { +func Test_FingerprintBounds_Unless(t *testing.T) { t.Parallel() target := NewBounds(10, 20) assert.Equal(t, []FingerprintBounds{ diff --git a/pkg/storage/bloom/v1/fuse_test.go b/pkg/storage/bloom/v1/fuse_test.go index 2ab0a51bced2c..1e35895794c22 100644 --- a/pkg/storage/bloom/v1/fuse_test.go +++ b/pkg/storage/bloom/v1/fuse_test.go @@ -13,7 +13,6 @@ import ( ) func TestFusedQuerier(t *testing.T) { - t.Parallel() // references for linking in memory reader+writer indexBuf := bytes.NewBuffer(nil) bloomsBuf := bytes.NewBuffer(nil) diff --git a/pkg/storage/stores/shipper/bloomshipper/client.go b/pkg/storage/stores/shipper/bloomshipper/client.go index 1dbfac579c5aa..882b0eab41c24 100644 --- a/pkg/storage/stores/shipper/bloomshipper/client.go +++ b/pkg/storage/stores/shipper/bloomshipper/client.go @@ -88,45 +88,66 @@ type Meta struct { // The specific TSDB files used to generate the block. Sources []tsdb.SingleTenantTSDBIdentifier + // TODO(owen-d): remove, unused // Old blocks which can be deleted in the future. These should be from previous compaction rounds. - Tombstones []BlockRef + BlockTombstones []BlockRef // A list of blocks that were generated Blocks []BlockRef } -// TODO(owen-d): use this to update internal ref's checksum. -func (m Meta) Checksum() (uint32, error) { +func MetaRefFrom( + tenant, + table string, + bounds v1.FingerprintBounds, + sources []tsdb.SingleTenantTSDBIdentifier, + blocks []BlockRef, +) (MetaRef, error) { + h := v1.Crc32HashPool.Get() defer v1.Crc32HashPool.Put(h) - err := m.Bounds.Hash(h) + err := bounds.Hash(h) if err != nil { - return 0, errors.Wrap(err, "writing OwnershipRange") + return MetaRef{}, errors.Wrap(err, "writing OwnershipRange") } - for _, tombstone := range m.Tombstones { - err = tombstone.Hash(h) + for _, source := range sources { + err = source.Hash(h) if err != nil { - return 0, errors.Wrap(err, "writing Tombstones") + return MetaRef{}, errors.Wrap(err, "writing Sources") } } - for _, source := range m.Sources { - err = source.Hash(h) - if err != nil { - return 0, errors.Wrap(err, "writing Sources") + var ( + start, end model.Time + ) + + for i, block := range blocks { + if i == 0 || block.StartTimestamp.Before(start) { + start = block.StartTimestamp + } + + if block.EndTimestamp.After(end) { + end = block.EndTimestamp } - } - for _, block := range m.Blocks { err = block.Hash(h) if err != nil { - return 0, errors.Wrap(err, "writing Blocks") + return MetaRef{}, errors.Wrap(err, "writing Blocks") } } - return h.Sum32(), nil + return MetaRef{ + Ref: Ref{ + TenantID: tenant, + TableName: table, + Bounds: bounds, + StartTimestamp: start, + EndTimestamp: end, + Checksum: h.Sum32(), + }, + }, nil } @@ -200,6 +221,7 @@ type BlockClient interface { type Client interface { MetaClient BlockClient + IsObjectNotFoundErr(err error) bool Stop() } @@ -224,6 +246,10 @@ func NewBloomClient(cfg bloomStoreConfig, client client.ObjectClient, logger log }, nil } +func (b *BloomClient) IsObjectNotFoundErr(err error) bool { + return b.client.IsObjectNotFoundErr(err) +} + func (b *BloomClient) PutMeta(ctx context.Context, meta Meta) error { data, err := json.Marshal(meta) if err != nil { @@ -300,6 +326,7 @@ func (b *BloomClient) DeleteBlocks(ctx context.Context, references []BlockRef) e ref := references[idx] key := b.Block(ref).Addr() err := b.client.DeleteObject(ctx, key) + if err != nil { return fmt.Errorf("error deleting block file: %w", err) } diff --git a/pkg/storage/stores/shipper/bloomshipper/client_test.go b/pkg/storage/stores/shipper/bloomshipper/client_test.go index 02d80429d7f11..897ed519946a7 100644 --- a/pkg/storage/stores/shipper/bloomshipper/client_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/client_test.go @@ -63,8 +63,8 @@ func putMeta(c *BloomClient, tenant string, start model.Time, minFp, maxFp model // EndTimestamp: start.Add(12 * time.Hour), }, }, - Blocks: []BlockRef{}, - Tombstones: []BlockRef{}, + Blocks: []BlockRef{}, + BlockTombstones: []BlockRef{}, } raw, _ := json.Marshal(meta) return meta, c.client.PutObject(context.Background(), c.Meta(meta.MetaRef).Addr(), bytes.NewReader(raw)) @@ -129,8 +129,8 @@ func TestBloomClient_PutMeta(t *testing.T) { // EndTimestamp: start.Add(12 * time.Hour), }, }, - Blocks: []BlockRef{}, - Tombstones: []BlockRef{}, + Blocks: []BlockRef{}, + BlockTombstones: []BlockRef{}, } err := c.PutMeta(ctx, meta) diff --git a/pkg/storage/stores/shipper/bloomshipper/fetcher_test.go b/pkg/storage/stores/shipper/bloomshipper/fetcher_test.go index fd1e6157a1cae..40a695e0b8e6c 100644 --- a/pkg/storage/stores/shipper/bloomshipper/fetcher_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/fetcher_test.go @@ -34,8 +34,8 @@ func makeMetas(t *testing.T, schemaCfg config.SchemaConfig, ts model.Time, keysp EndTimestamp: ts, }, }, - Tombstones: []BlockRef{}, - Blocks: []BlockRef{}, + BlockTombstones: []BlockRef{}, + Blocks: []BlockRef{}, } } return metas diff --git a/pkg/storage/stores/shipper/bloomshipper/shipper.go b/pkg/storage/stores/shipper/bloomshipper/shipper.go index dee8f2464bcbb..fd755b0a204a7 100644 --- a/pkg/storage/stores/shipper/bloomshipper/shipper.go +++ b/pkg/storage/stores/shipper/bloomshipper/shipper.go @@ -60,7 +60,7 @@ func BlocksForMetas(metas []Meta, interval Interval, keyspaces []v1.FingerprintB blocks := make(map[BlockRef]bool) // block -> isTombstoned for _, meta := range metas { - for _, tombstone := range meta.Tombstones { + for _, tombstone := range meta.BlockTombstones { blocks[tombstone] = true } for _, block := range meta.Blocks { diff --git a/pkg/storage/stores/shipper/bloomshipper/shipper_test.go b/pkg/storage/stores/shipper/bloomshipper/shipper_test.go index a3a9442ddfb03..c9e47f91fea28 100644 --- a/pkg/storage/stores/shipper/bloomshipper/shipper_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/shipper_test.go @@ -31,7 +31,7 @@ func TestBloomShipper_findBlocks(t *testing.T) { }, }, { - Tombstones: []BlockRef{ + BlockTombstones: []BlockRef{ createMatchingBlockRef(1), createMatchingBlockRef(3), }, diff --git a/pkg/storage/stores/shipper/bloomshipper/store_test.go b/pkg/storage/stores/shipper/bloomshipper/store_test.go index 56179cde97c36..ca86cb94fa963 100644 --- a/pkg/storage/stores/shipper/bloomshipper/store_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/store_test.go @@ -83,8 +83,8 @@ func createMetaInStorage(store *BloomStore, tenant string, start model.Time, min // EndTimestamp: start.Add(12 * time.Hour), }, }, - Blocks: []BlockRef{}, - Tombstones: []BlockRef{}, + Blocks: []BlockRef{}, + BlockTombstones: []BlockRef{}, } err := store.storeDo(start, func(s *bloomStoreEntry) error { raw, _ := json.Marshal(meta) From bd12e163390f8f85fc04a1deba6140f46d726c1b Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 14 Feb 2024 12:10:24 -0800 Subject: [PATCH 09/19] [Blooms] Use correct table address function (#11955) --- pkg/bloomcompactor/controller.go | 8 ++++---- pkg/bloomcompactor/table_utils.go | 16 ---------------- pkg/bloomcompactor/tsdb.go | 6 +++--- pkg/bloomgateway/util_test.go | 2 +- pkg/storage/config/schema_config.go | 4 +++- 5 files changed, 11 insertions(+), 25 deletions(-) delete mode 100644 pkg/bloomcompactor/table_utils.go diff --git a/pkg/bloomcompactor/controller.go b/pkg/bloomcompactor/controller.go index c29aec86ac5c2..cc801dc27e550 100644 --- a/pkg/bloomcompactor/controller.go +++ b/pkg/bloomcompactor/controller.go @@ -77,7 +77,7 @@ func (s *SimpleBloomController) compactTenant( client, err := s.bloomStore.Client(table.ModelTime()) if err != nil { - level.Error(logger).Log("msg", "failed to get client", "err", err, "table", table.String()) + level.Error(logger).Log("msg", "failed to get client", "err", err, "table", table.Addr()) return errors.Wrap(err, "failed to get client") } @@ -280,7 +280,7 @@ func (s *SimpleBloomController) buildGaps( MetaRef: bloomshipper.MetaRef{ Ref: bloomshipper.Ref{ TenantID: tenant, - TableName: table.String(), + TableName: table.Addr(), Bounds: gap.bounds, }, }, @@ -319,7 +319,7 @@ func (s *SimpleBloomController) buildGaps( blockCt++ blk := newBlocks.At() - built, err := bloomshipper.BlockFrom(tenant, table.String(), blk) + built, err := bloomshipper.BlockFrom(tenant, table.Addr(), blk) if err != nil { level.Error(logger).Log("msg", "failed to build block", "err", err) return nil, errors.Wrap(err, "failed to build block") @@ -348,7 +348,7 @@ func (s *SimpleBloomController) buildGaps( s.closeLoadedBlocks(loaded, blocksIter) // Write the new meta - ref, err := bloomshipper.MetaRefFrom(tenant, table.String(), gap.bounds, meta.Sources, meta.Blocks) + ref, err := bloomshipper.MetaRefFrom(tenant, table.Addr(), gap.bounds, meta.Sources, meta.Blocks) if err != nil { level.Error(logger).Log("msg", "failed to checksum meta", "err", err) return nil, errors.Wrap(err, "failed to checksum meta") diff --git a/pkg/bloomcompactor/table_utils.go b/pkg/bloomcompactor/table_utils.go deleted file mode 100644 index 55bc2e9a328f1..0000000000000 --- a/pkg/bloomcompactor/table_utils.go +++ /dev/null @@ -1,16 +0,0 @@ -package bloomcompactor - -import ( - "github.com/prometheus/common/model" - - "github.com/grafana/loki/pkg/compactor/retention" -) - -func getIntervalsForTables(tables []string) map[string]model.Interval { - tablesIntervals := make(map[string]model.Interval, len(tables)) - for _, table := range tables { - tablesIntervals[table] = retention.ExtractIntervalFromTableName(table) - } - - return tablesIntervals -} diff --git a/pkg/bloomcompactor/tsdb.go b/pkg/bloomcompactor/tsdb.go index ad7b2eafac4cd..d19e185a9275b 100644 --- a/pkg/bloomcompactor/tsdb.go +++ b/pkg/bloomcompactor/tsdb.go @@ -50,12 +50,12 @@ func NewBloomTSDBStore(storage storage.Client) *BloomTSDBStore { } func (b *BloomTSDBStore) UsersForPeriod(ctx context.Context, table config.DayTime) ([]string, error) { - _, users, err := b.storage.ListFiles(ctx, table.Table(), true) // bypass cache for ease of testing + _, users, err := b.storage.ListFiles(ctx, table.Addr(), true) // bypass cache for ease of testing return users, err } func (b *BloomTSDBStore) ResolveTSDBs(ctx context.Context, table config.DayTime, tenant string) ([]tsdb.SingleTenantTSDBIdentifier, error) { - indices, err := b.storage.ListUserFiles(ctx, table.Table(), tenant, true) // bypass cache for ease of testing + indices, err := b.storage.ListUserFiles(ctx, table.Addr(), tenant, true) // bypass cache for ease of testing if err != nil { return nil, errors.Wrap(err, "failed to list user files") } @@ -87,7 +87,7 @@ func (b *BloomTSDBStore) LoadTSDB( ) (v1.CloseableIterator[*v1.Series], error) { withCompression := id.Name() + gzipExtension - data, err := b.storage.GetUserFile(ctx, table.Table(), tenant, withCompression) + data, err := b.storage.GetUserFile(ctx, table.Addr(), tenant, withCompression) if err != nil { return nil, errors.Wrap(err, "failed to get file") } diff --git a/pkg/bloomgateway/util_test.go b/pkg/bloomgateway/util_test.go index 6850cf31811c0..9b5ce6e897bb9 100644 --- a/pkg/bloomgateway/util_test.go +++ b/pkg/bloomgateway/util_test.go @@ -311,7 +311,7 @@ func createBlocks(t *testing.T, tenant string, n int, from, through model.Time, } ref := bloomshipper.Ref{ TenantID: tenant, - TableName: config.NewDayTime(truncateDay(from)).Table(), + TableName: config.NewDayTime(truncateDay(from)).Addr(), Bounds: v1.NewBounds(fromFp, throughFp), StartTimestamp: from, EndTimestamp: through, diff --git a/pkg/storage/config/schema_config.go b/pkg/storage/config/schema_config.go index 30b9de98b14ba..b7c92c62c3d94 100644 --- a/pkg/storage/config/schema_config.go +++ b/pkg/storage/config/schema_config.go @@ -237,7 +237,9 @@ func (d DayTime) String() string { return d.Time.Time().UTC().Format("2006-01-02") } -func (d DayTime) Table() string { +// Addr returns the unix day offset as a string, which is used +// as the address for the index table in storage. +func (d DayTime) Addr() string { return fmt.Sprintf("%d", d.ModelTime().Time().UnixNano()/int64(ObjectStorageIndexRequiredPeriod)) } From 6434df73459f91676ca2463313a354fb9cb6d3d9 Mon Sep 17 00:00:00 2001 From: Salva Corts Date: Thu, 15 Feb 2024 00:58:26 +0100 Subject: [PATCH 10/19] Bloom compactor shuffle-sharding (#11947) --- pkg/bloomcompactor/bloomcompactor.go | 38 ++++- pkg/bloomcompactor/bloomcompactor_test.go | 197 ++++++++++++++++++++++ pkg/bloomcompactor/sharding.go | 58 ------- pkg/bloomcompactor/sharding_test.go | 149 ---------------- pkg/bloomutils/ring.go | 23 ++- pkg/bloomutils/ring_test.go | 19 +-- pkg/loki/modules.go | 3 +- pkg/util/ring/sharding.go | 73 ++------ 8 files changed, 259 insertions(+), 301 deletions(-) create mode 100644 pkg/bloomcompactor/bloomcompactor_test.go delete mode 100644 pkg/bloomcompactor/sharding.go delete mode 100644 pkg/bloomcompactor/sharding_test.go diff --git a/pkg/bloomcompactor/bloomcompactor.go b/pkg/bloomcompactor/bloomcompactor.go index fa3b205aa31b7..36dcf36ed1fb7 100644 --- a/pkg/bloomcompactor/bloomcompactor.go +++ b/pkg/bloomcompactor/bloomcompactor.go @@ -2,7 +2,6 @@ package bloomcompactor import ( "context" - "math" "sync" "time" @@ -11,16 +10,23 @@ import ( "github.com/grafana/dskit/backoff" "github.com/grafana/dskit/concurrency" "github.com/grafana/dskit/multierror" + "github.com/grafana/dskit/ring" "github.com/grafana/dskit/services" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/common/model" + "github.com/grafana/loki/pkg/bloomutils" "github.com/grafana/loki/pkg/storage" v1 "github.com/grafana/loki/pkg/storage/bloom/v1" "github.com/grafana/loki/pkg/storage/config" "github.com/grafana/loki/pkg/storage/stores" "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper" + util_ring "github.com/grafana/loki/pkg/util/ring" +) + +var ( + RingOp = ring.NewOp([]ring.InstanceState{ring.JOINING, ring.ACTIVE}, nil) ) /* @@ -47,7 +53,7 @@ type Compactor struct { // temporary workaround until bloomStore has implemented read/write shipper interface bloomStore bloomshipper.Store - sharding ShardingStrategy + sharding util_ring.TenantSharding metrics *Metrics btMetrics *v1.Metrics @@ -59,7 +65,7 @@ func New( storeCfg storage.Config, clientMetrics storage.ClientMetrics, fetcherProvider stores.ChunkFetcherProvider, - sharding ShardingStrategy, + sharding util_ring.TenantSharding, limits Limits, logger log.Logger, r prometheus.Registerer, @@ -182,9 +188,24 @@ func (c *Compactor) tenants(ctx context.Context, table config.DayTime) (v1.Itera return v1.NewSliceIter(tenants), nil } -// TODO(owen-d): implement w/ subrings -func (c *Compactor) ownsTenant(_ string) (ownershipRange v1.FingerprintBounds, owns bool) { - return v1.NewBounds(0, math.MaxUint64), true +// ownsTenant returns the ownership range for the tenant, if the compactor owns the tenant, and an error. +func (c *Compactor) ownsTenant(tenant string) (v1.FingerprintBounds, bool, error) { + tenantRing, owned := c.sharding.OwnsTenant(tenant) + if !owned { + return v1.FingerprintBounds{}, false, nil + } + + rs, err := tenantRing.GetAllHealthy(RingOp) + if err != nil { + return v1.FingerprintBounds{}, false, errors.Wrap(err, "getting ring healthy instances") + + } + + ownershipBounds, err := bloomutils.GetInstanceWithTokenRange(c.cfg.Ring.InstanceID, rs.Instances) + if err != nil { + return v1.FingerprintBounds{}, false, errors.Wrap(err, "getting instance token range") + } + return ownershipBounds, true, nil } // runs a single round of compaction for all relevant tenants and tables @@ -232,7 +253,10 @@ func (c *Compactor) loadWork(ctx context.Context, ch chan<- tenantTable) error { for tenants.Next() && tenants.Err() == nil && ctx.Err() == nil { tenant := tenants.At() - ownershipRange, owns := c.ownsTenant(tenant) + ownershipRange, owns, err := c.ownsTenant(tenant) + if err != nil { + return errors.Wrap(err, "checking tenant ownership") + } if !owns { continue } diff --git a/pkg/bloomcompactor/bloomcompactor_test.go b/pkg/bloomcompactor/bloomcompactor_test.go new file mode 100644 index 0000000000000..475ba8ec0585d --- /dev/null +++ b/pkg/bloomcompactor/bloomcompactor_test.go @@ -0,0 +1,197 @@ +package bloomcompactor + +import ( + "context" + "flag" + "fmt" + "math" + "testing" + "time" + + "github.com/grafana/dskit/services" + "github.com/prometheus/client_golang/prometheus" + "github.com/stretchr/testify/require" + + v1 "github.com/grafana/loki/pkg/storage/bloom/v1" + util_log "github.com/grafana/loki/pkg/util/log" + lokiring "github.com/grafana/loki/pkg/util/ring" + util_ring "github.com/grafana/loki/pkg/util/ring" + "github.com/grafana/loki/pkg/validation" +) + +func TestCompactor_ownsTenant(t *testing.T) { + for _, tc := range []struct { + name string + limits Limits + compactors int + + expectedCompactorsOwningTenant int + }{ + { + name: "no sharding with one instance", + limits: mockLimits{ + shardSize: 0, + }, + compactors: 1, + expectedCompactorsOwningTenant: 1, + }, + { + name: "no sharding with multiple instances", + limits: mockLimits{ + shardSize: 0, + }, + compactors: 10, + expectedCompactorsOwningTenant: 10, + }, + { + name: "sharding with one instance", + limits: mockLimits{ + shardSize: 5, + }, + compactors: 1, + expectedCompactorsOwningTenant: 1, + }, + { + name: "sharding with multiple instances", + limits: mockLimits{ + shardSize: 5, + }, + compactors: 10, + expectedCompactorsOwningTenant: 5, + }, + } { + t.Run(tc.name, func(t *testing.T) { + var ringManagers []*lokiring.RingManager + var compactors []*Compactor + for i := 0; i < tc.compactors; i++ { + var ringCfg lokiring.RingConfig + ringCfg.RegisterFlagsWithPrefix("", "", flag.NewFlagSet("ring", flag.PanicOnError)) + ringCfg.KVStore.Store = "inmemory" + ringCfg.InstanceID = fmt.Sprintf("bloom-compactor-%d", i) + ringCfg.InstanceAddr = fmt.Sprintf("localhost-%d", i) + + ringManager, err := lokiring.NewRingManager("bloom-compactor", lokiring.ServerMode, ringCfg, 1, 1, util_log.Logger, prometheus.NewRegistry()) + require.NoError(t, err) + require.NoError(t, ringManager.StartAsync(context.Background())) + + shuffleSharding := util_ring.NewTenantShuffleSharding(ringManager.Ring, ringManager.RingLifecycler, tc.limits.BloomCompactorShardSize) + + compactor := &Compactor{ + cfg: Config{ + Ring: ringCfg, + }, + sharding: shuffleSharding, + limits: tc.limits, + } + + ringManagers = append(ringManagers, ringManager) + compactors = append(compactors, compactor) + } + defer func() { + // Stop all rings and wait for them to stop. + for _, ringManager := range ringManagers { + ringManager.StopAsync() + require.Eventually(t, func() bool { + return ringManager.State() == services.Terminated + }, 1*time.Minute, 100*time.Millisecond) + } + }() + + // Wait for all rings to see each other. + for _, ringManager := range ringManagers { + require.Eventually(t, func() bool { + running := ringManager.State() == services.Running + discovered := ringManager.Ring.InstancesCount() == tc.compactors + return running && discovered + }, 1*time.Minute, 100*time.Millisecond) + } + + var compactorOwnsTenant int + var compactorOwnershipRange []v1.FingerprintBounds + for _, compactor := range compactors { + ownershipRange, ownsTenant, err := compactor.ownsTenant("tenant") + require.NoError(t, err) + if ownsTenant { + compactorOwnsTenant++ + compactorOwnershipRange = append(compactorOwnershipRange, ownershipRange) + } + } + require.Equal(t, tc.expectedCompactorsOwningTenant, compactorOwnsTenant) + + coveredKeySpace := v1.NewBounds(math.MaxUint64, 0) + for i, boundsA := range compactorOwnershipRange { + for j, boundsB := range compactorOwnershipRange { + if i == j { + continue + } + // Assert that the fingerprint key-space is not overlapping + require.False(t, boundsA.Overlaps(boundsB)) + } + + if boundsA.Min < coveredKeySpace.Min { + coveredKeySpace.Min = boundsA.Min + } + if boundsA.Max > coveredKeySpace.Max { + coveredKeySpace.Max = boundsA.Max + } + + // Assert that the fingerprint key-space is evenly distributed across the compactors + // We do some adjustments if the key-space is not evenly distributable, so we use a delta of 10 + // to account for that and check that the key-space is reasonably evenly distributed. + fpPerTenant := math.MaxUint64 / uint64(tc.expectedCompactorsOwningTenant) + boundsLen := uint64(boundsA.Max - boundsA.Min) + require.InDelta(t, fpPerTenant, boundsLen, 10) + } + // Assert that the fingerprint key-space is complete + require.True(t, coveredKeySpace.Equal(v1.NewBounds(0, math.MaxUint64))) + }) + } +} + +type mockLimits struct { + shardSize int +} + +func (m mockLimits) AllByUserID() map[string]*validation.Limits { + panic("implement me") +} + +func (m mockLimits) DefaultLimits() *validation.Limits { + panic("implement me") +} + +func (m mockLimits) VolumeMaxSeries(_ string) int { + panic("implement me") +} + +func (m mockLimits) BloomCompactorShardSize(_ string) int { + return m.shardSize +} + +func (m mockLimits) BloomCompactorChunksBatchSize(_ string) int { + panic("implement me") +} + +func (m mockLimits) BloomCompactorMaxTableAge(_ string) time.Duration { + panic("implement me") +} + +func (m mockLimits) BloomCompactorEnabled(_ string) bool { + panic("implement me") +} + +func (m mockLimits) BloomNGramLength(_ string) int { + panic("implement me") +} + +func (m mockLimits) BloomNGramSkip(_ string) int { + panic("implement me") +} + +func (m mockLimits) BloomFalsePositiveRate(_ string) float64 { + panic("implement me") +} + +func (m mockLimits) BloomCompactorMaxBlockSize(_ string) int { + panic("implement me") +} diff --git a/pkg/bloomcompactor/sharding.go b/pkg/bloomcompactor/sharding.go deleted file mode 100644 index 9b3009bd50652..0000000000000 --- a/pkg/bloomcompactor/sharding.go +++ /dev/null @@ -1,58 +0,0 @@ -package bloomcompactor - -import ( - "github.com/grafana/dskit/ring" - - util_ring "github.com/grafana/loki/pkg/util/ring" -) - -var ( - // TODO: Should we include LEAVING instances in the replication set? - RingOp = ring.NewOp([]ring.InstanceState{ring.JOINING, ring.ACTIVE}, nil) -) - -// ShardingStrategy describes whether compactor "owns" given user or job. -type ShardingStrategy interface { - util_ring.TenantSharding - OwnsFingerprint(tenantID string, fp uint64) (bool, error) -} - -type ShuffleShardingStrategy struct { - util_ring.TenantSharding - ringLifeCycler *ring.BasicLifecycler -} - -func NewShuffleShardingStrategy(r *ring.Ring, ringLifecycler *ring.BasicLifecycler, limits Limits) *ShuffleShardingStrategy { - s := ShuffleShardingStrategy{ - TenantSharding: util_ring.NewTenantShuffleSharding(r, ringLifecycler, limits.BloomCompactorShardSize), - ringLifeCycler: ringLifecycler, - } - - return &s -} - -// OwnsFingerprint makes sure only a single compactor processes the fingerprint. -func (s *ShuffleShardingStrategy) OwnsFingerprint(tenantID string, fp uint64) (bool, error) { - if !s.OwnsTenant(tenantID) { - return false, nil - } - - tenantRing := s.GetTenantSubRing(tenantID) - fpSharding := util_ring.NewFingerprintShuffleSharding(tenantRing, s.ringLifeCycler, RingOp) - return fpSharding.OwnsFingerprint(fp) -} - -// NoopStrategy is an implementation of the ShardingStrategy that does not -// filter anything. -type NoopStrategy struct { - util_ring.NoopStrategy -} - -// OwnsFingerprint implements TenantShuffleSharding. -func (s *NoopStrategy) OwnsFingerprint(_ string, _ uint64) (bool, error) { - return true, nil -} - -func NewNoopStrategy() *NoopStrategy { - return &NoopStrategy{NoopStrategy: util_ring.NoopStrategy{}} -} diff --git a/pkg/bloomcompactor/sharding_test.go b/pkg/bloomcompactor/sharding_test.go deleted file mode 100644 index 4e79752279fb9..0000000000000 --- a/pkg/bloomcompactor/sharding_test.go +++ /dev/null @@ -1,149 +0,0 @@ -package bloomcompactor - -import ( - "context" - "flag" - "fmt" - "testing" - "time" - - "github.com/grafana/dskit/services" - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/common/model" - "github.com/prometheus/prometheus/model/labels" - "github.com/stretchr/testify/require" - - util_log "github.com/grafana/loki/pkg/util/log" - lokiring "github.com/grafana/loki/pkg/util/ring" - "github.com/grafana/loki/pkg/validation" -) - -func TestShuffleSharding(t *testing.T) { - const shardSize = 2 - const rings = 4 - const tenants = 2000 - const jobsPerTenant = 200 - - var limits validation.Limits - limits.RegisterFlags(flag.NewFlagSet("limits", flag.PanicOnError)) - overrides, err := validation.NewOverrides(limits, nil) - require.NoError(t, err) - - var ringManagers []*lokiring.RingManager - var shards []*ShuffleShardingStrategy - for i := 0; i < rings; i++ { - var ringCfg lokiring.RingConfig - ringCfg.RegisterFlagsWithPrefix("", "", flag.NewFlagSet("ring", flag.PanicOnError)) - ringCfg.KVStore.Store = "inmemory" - ringCfg.InstanceID = fmt.Sprintf("bloom-compactor-%d", i) - ringCfg.InstanceAddr = fmt.Sprintf("localhost-%d", i) - - ringManager, err := lokiring.NewRingManager("bloom-compactor", lokiring.ServerMode, ringCfg, 1, 1, util_log.Logger, prometheus.NewRegistry()) - require.NoError(t, err) - require.NoError(t, ringManager.StartAsync(context.Background())) - - sharding := NewShuffleShardingStrategy(ringManager.Ring, ringManager.RingLifecycler, mockLimits{ - Overrides: overrides, - bloomCompactorShardSize: shardSize, - }) - - ringManagers = append(ringManagers, ringManager) - shards = append(shards, sharding) - } - - // Wait for all rings to see each other. - for i := 0; i < rings; i++ { - require.Eventually(t, func() bool { - running := ringManagers[i].State() == services.Running - discovered := ringManagers[i].Ring.InstancesCount() == rings - return running && discovered - }, 1*time.Minute, 100*time.Millisecond) - } - - // This is kind of an un-deterministic test, because sharding is random - // and the seed is initialized by the ring lib. - // Here we'll generate a bunch of tenants and test that if the sharding doesn't own the tenant, - // that's because the tenant is owned by other ring instances. - shard := shards[0] - otherShards := shards[1:] - var ownedTenants, ownedJobs int - for i := 0; i < tenants; i++ { - tenant := fmt.Sprintf("tenant-%d", i) - ownsTenant := shard.OwnsTenant(tenant) - - var tenantOwnedByOther int - for _, other := range otherShards { - otherOwns := other.OwnsTenant(tenant) - if otherOwns { - tenantOwnedByOther++ - } - } - - // If this shard owns the tenant, shardSize-1 other members should also own the tenant. - // Otherwise, shardSize other members should own the tenant. - if ownsTenant { - require.Equal(t, shardSize-1, tenantOwnedByOther) - ownedTenants++ - } else { - require.Equal(t, shardSize, tenantOwnedByOther) - } - - for j := 0; j < jobsPerTenant; j++ { - lbls := labels.FromStrings("namespace", fmt.Sprintf("namespace-%d", j)) - fp := model.Fingerprint(lbls.Hash()) - ownsFingerprint, err := shard.OwnsFingerprint(tenant, uint64(fp)) - require.NoError(t, err) - - var jobOwnedByOther int - for _, other := range otherShards { - otherOwns, err := other.OwnsFingerprint(tenant, uint64(fp)) - require.NoError(t, err) - if otherOwns { - jobOwnedByOther++ - } - } - - // If this shard owns the job, no one else should own the job. - // And if this shard doesn't own the job, only one of the other shards should own the job. - if ownsFingerprint { - require.Equal(t, 0, jobOwnedByOther) - ownedJobs++ - } else { - require.Equal(t, 1, jobOwnedByOther) - } - } - } - - t.Logf("owned tenants: %d (out of %d)", ownedTenants, tenants) - t.Logf("owned jobs: %d (out of %d)", ownedJobs, tenants*jobsPerTenant) - - // Stop all rings and wait for them to stop. - for i := 0; i < rings; i++ { - ringManagers[i].StopAsync() - require.Eventually(t, func() bool { - return ringManagers[i].State() == services.Terminated - }, 1*time.Minute, 100*time.Millisecond) - } -} - -type mockLimits struct { - *validation.Overrides - bloomCompactorShardSize int - chunksDownloadingBatchSize int - fpRate float64 -} - -func (m mockLimits) BloomFalsePositiveRate(_ string) float64 { - return m.fpRate -} - -func (m mockLimits) BloomCompactorShardSize(_ string) int { - return m.bloomCompactorShardSize -} - -func (m mockLimits) BloomCompactorChunksBatchSize(_ string) int { - if m.chunksDownloadingBatchSize != 0 { - return m.chunksDownloadingBatchSize - } - return 1 -} diff --git a/pkg/bloomutils/ring.go b/pkg/bloomutils/ring.go index 08e62a13acb71..d05b91d644df2 100644 --- a/pkg/bloomutils/ring.go +++ b/pkg/bloomutils/ring.go @@ -8,6 +8,7 @@ import ( "sort" "github.com/grafana/dskit/ring" + "github.com/prometheus/common/model" "golang.org/x/exp/slices" v1 "github.com/grafana/loki/pkg/storage/bloom/v1" @@ -38,11 +39,11 @@ func (i InstancesWithTokenRange) Contains(token uint32) bool { return false } -// GetInstanceTokenRange calculates the token range for a specific instance +// GetInstanceWithTokenRange calculates the token range for a specific instance // with given id based on the first token in the ring. // This assumes that each instance in the ring is configured with only a single // token. -func GetInstanceWithTokenRange(id string, instances []ring.InstanceDesc) InstancesWithTokenRange { +func GetInstanceWithTokenRange(id string, instances []ring.InstanceDesc) (v1.FingerprintBounds, error) { // Sorting the tokens of the instances would not be necessary if there is // only a single token per instances, however, since we only assume one @@ -64,23 +65,21 @@ func GetInstanceWithTokenRange(id string, instances []ring.InstanceDesc) Instanc // instance with Id == id not found if idx == -1 { - return InstancesWithTokenRange{} + return v1.FingerprintBounds{}, ring.ErrInstanceNotFound } - i := uint32(idx) - n := uint32(len(instances)) - step := math.MaxUint32 / n + i := uint64(idx) + n := uint64(len(instances)) + step := math.MaxUint64 / n - minToken := step * i - maxToken := step*i + step - 1 + minToken := model.Fingerprint(step * i) + maxToken := model.Fingerprint(step*i + step - 1) if i == n-1 { // extend the last token tange to MaxUint32 - maxToken = math.MaxUint32 + maxToken = math.MaxUint64 } - return InstancesWithTokenRange{ - {MinToken: minToken, MaxToken: maxToken, Instance: instances[i]}, - } + return v1.NewBounds(minToken, maxToken), nil } // GetInstancesWithTokenRanges calculates the token ranges for a specific diff --git a/pkg/bloomutils/ring_test.go b/pkg/bloomutils/ring_test.go index 30da072021edf..1346559372c30 100644 --- a/pkg/bloomutils/ring_test.go +++ b/pkg/bloomutils/ring_test.go @@ -6,6 +6,8 @@ import ( "github.com/grafana/dskit/ring" "github.com/stretchr/testify/require" + + v1 "github.com/grafana/loki/pkg/storage/bloom/v1" ) func TestBloomGatewayClient_SortInstancesByToken(t *testing.T) { @@ -67,7 +69,7 @@ func TestBloomGatewayClient_GetInstanceWithTokenRange(t *testing.T) { for name, tc := range map[string]struct { id string input []ring.InstanceDesc - expected InstancesWithTokenRange + expected v1.FingerprintBounds }{ "first instance includes 0 token": { id: "3", @@ -76,9 +78,7 @@ func TestBloomGatewayClient_GetInstanceWithTokenRange(t *testing.T) { {Id: "2", Tokens: []uint32{5}}, {Id: "3", Tokens: []uint32{1}}, }, - expected: InstancesWithTokenRange{ - {Instance: ring.InstanceDesc{Id: "3", Tokens: []uint32{1}}, MinToken: 0, MaxToken: math.MaxUint32/3 - 1}, - }, + expected: v1.NewBounds(0, math.MaxUint64/3-1), }, "middle instance": { id: "1", @@ -87,9 +87,7 @@ func TestBloomGatewayClient_GetInstanceWithTokenRange(t *testing.T) { {Id: "2", Tokens: []uint32{5}}, {Id: "3", Tokens: []uint32{1}}, }, - expected: InstancesWithTokenRange{ - {Instance: ring.InstanceDesc{Id: "1", Tokens: []uint32{3}}, MinToken: math.MaxUint32 / 3, MaxToken: math.MaxUint32/3*2 - 1}, - }, + expected: v1.NewBounds(math.MaxUint64/3, math.MaxUint64/3*2-1), }, "last instance includes MaxUint32 token": { id: "2", @@ -98,14 +96,13 @@ func TestBloomGatewayClient_GetInstanceWithTokenRange(t *testing.T) { {Id: "2", Tokens: []uint32{5}}, {Id: "3", Tokens: []uint32{1}}, }, - expected: InstancesWithTokenRange{ - {Instance: ring.InstanceDesc{Id: "2", Tokens: []uint32{5}}, MinToken: math.MaxUint32 / 3 * 2, MaxToken: math.MaxUint32}, - }, + expected: v1.NewBounds(math.MaxUint64/3*2, math.MaxUint64), }, } { tc := tc t.Run(name, func(t *testing.T) { - result := GetInstanceWithTokenRange(tc.id, tc.input) + result, err := GetInstanceWithTokenRange(tc.id, tc.input) + require.NoError(t, err) require.Equal(t, tc.expected, result) }) } diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index 111d313956881..15ee955355a62 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -75,6 +75,7 @@ import ( util_log "github.com/grafana/loki/pkg/util/log" "github.com/grafana/loki/pkg/util/querylimits" lokiring "github.com/grafana/loki/pkg/util/ring" + util_ring "github.com/grafana/loki/pkg/util/ring" serverutil "github.com/grafana/loki/pkg/util/server" "github.com/grafana/loki/pkg/validation" ) @@ -1418,7 +1419,7 @@ func (t *Loki) initBloomCompactor() (services.Service, error) { logger := log.With(util_log.Logger, "component", "bloom-compactor") - shuffleSharding := bloomcompactor.NewShuffleShardingStrategy(t.bloomCompactorRingManager.Ring, t.bloomCompactorRingManager.RingLifecycler, t.Overrides) + shuffleSharding := util_ring.NewTenantShuffleSharding(t.bloomCompactorRingManager.Ring, t.bloomCompactorRingManager.RingLifecycler, t.Overrides.BloomCompactorShardSize) compactor, err := bloomcompactor.New( t.Cfg.BloomCompactor, diff --git a/pkg/util/ring/sharding.go b/pkg/util/ring/sharding.go index 45a53cf40cfe7..460e22d7f4c45 100644 --- a/pkg/util/ring/sharding.go +++ b/pkg/util/ring/sharding.go @@ -2,12 +2,10 @@ package ring import ( "github.com/grafana/dskit/ring" - "github.com/prometheus/common/model" ) type TenantSharding interface { - GetTenantSubRing(tenantID string) ring.ReadRing - OwnsTenant(tenantID string) bool + OwnsTenant(tenantID string) (tenantRing ring.ReadRing, owned bool) } type TenantShuffleSharding struct { @@ -28,60 +26,19 @@ func NewTenantShuffleSharding( } } -func (s *TenantShuffleSharding) GetTenantSubRing(tenantID string) ring.ReadRing { - shardSize := s.shardSizeForTenant(tenantID) - +func (s *TenantShuffleSharding) OwnsTenant(tenantID string) (ring.ReadRing, bool) { // A shard size of 0 means shuffle sharding is disabled for this specific user, + shardSize := s.shardSizeForTenant(tenantID) if shardSize <= 0 { - return s.r + return s.r, true } - return s.r.ShuffleShard(tenantID, shardSize) -} - -func (s *TenantShuffleSharding) OwnsTenant(tenantID string) bool { - subRing := s.GetTenantSubRing(tenantID) - return subRing.HasInstance(s.ringLifeCycler.GetInstanceID()) -} - -type FingerprintSharding interface { - OwnsFingerprint(fp model.Fingerprint) (bool, error) -} - -// FingerprintShuffleSharding is not thread-safe. -type FingerprintShuffleSharding struct { - r ring.ReadRing - ringLifeCycler *ring.BasicLifecycler - ringOp ring.Operation - - // Buffers for ring.Get() calls. - bufDescs []ring.InstanceDesc - bufHosts, bufZones []string -} - -func NewFingerprintShuffleSharding( - r ring.ReadRing, - ringLifeCycler *ring.BasicLifecycler, - ringOp ring.Operation, -) *FingerprintShuffleSharding { - s := FingerprintShuffleSharding{ - r: r, - ringLifeCycler: ringLifeCycler, - ringOp: ringOp, + subRing := s.r.ShuffleShard(tenantID, shardSize) + if subRing.HasInstance(s.ringLifeCycler.GetInstanceID()) { + return subRing, true } - s.bufDescs, s.bufHosts, s.bufZones = ring.MakeBuffersForGet() - - return &s -} - -func (s *FingerprintShuffleSharding) OwnsFingerprint(fp uint64) (bool, error) { - rs, err := s.r.Get(uint32(fp), s.ringOp, s.bufDescs, s.bufHosts, s.bufZones) - if err != nil { - return false, err - } - - return rs.Includes(s.ringLifeCycler.GetInstanceAddr()), nil + return nil, false } // NoopStrategy is an implementation of the ShardingStrategy that does not @@ -89,16 +46,6 @@ func (s *FingerprintShuffleSharding) OwnsFingerprint(fp uint64) (bool, error) { type NoopStrategy struct{} // OwnsTenant implements TenantShuffleSharding. -func (s *NoopStrategy) OwnsTenant(_ string) bool { - return false -} - -// GetTenantSubRing implements TenantShuffleSharding. -func (s *NoopStrategy) GetTenantSubRing(_ string) ring.ReadRing { - return nil -} - -// OwnsFingerprint implements FingerprintSharding. -func (s *NoopStrategy) OwnsFingerprint(_ uint64) (bool, error) { - return false, nil +func (s *NoopStrategy) OwnsTenant(_ string) (ring.ReadRing, bool) { + return nil, false } From d0fae5cd690a6a1523004a4952e1957b2d8c263d Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Wed, 14 Feb 2024 16:21:44 -0800 Subject: [PATCH 11/19] [blooms] Remove unnecessary token sorting (#11958) --- pkg/bloomutils/ring.go | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) diff --git a/pkg/bloomutils/ring.go b/pkg/bloomutils/ring.go index d05b91d644df2..20bb446ba15d3 100644 --- a/pkg/bloomutils/ring.go +++ b/pkg/bloomutils/ring.go @@ -45,16 +45,8 @@ func (i InstancesWithTokenRange) Contains(token uint32) bool { // token. func GetInstanceWithTokenRange(id string, instances []ring.InstanceDesc) (v1.FingerprintBounds, error) { - // Sorting the tokens of the instances would not be necessary if there is - // only a single token per instances, however, since we only assume one - // token, but don't enforce one token, we keep the sorting. - for _, inst := range instances { - sort.Slice(inst.Tokens, func(i, j int) bool { - return inst.Tokens[i] < inst.Tokens[j] - }) - } - - // Sort instances + // Sort instances -- they may not be sorted + // because they're usually accessed by looking up the tokens (which are sorted) sort.Slice(instances, func(i, j int) bool { return instances[i].Tokens[0] < instances[j].Tokens[0] }) From 2177037ee3d47bfacc425cae200e60d8b32dbd04 Mon Sep 17 00:00:00 2001 From: Christian Haudum Date: Thu, 15 Feb 2024 12:34:16 +0100 Subject: [PATCH 12/19] (chore): Simplify implementation of instance sort iterator (#11959) Signed-off-by: Christian Haudum --- pkg/bloomutils/iter.go | 37 -------------------------- pkg/bloomutils/ring.go | 53 +++++++++++++++++++++---------------- pkg/bloomutils/ring_test.go | 5 ++++ 3 files changed, 35 insertions(+), 60 deletions(-) delete mode 100644 pkg/bloomutils/iter.go diff --git a/pkg/bloomutils/iter.go b/pkg/bloomutils/iter.go deleted file mode 100644 index fdbe4a5e62587..0000000000000 --- a/pkg/bloomutils/iter.go +++ /dev/null @@ -1,37 +0,0 @@ -package bloomutils - -import ( - "io" - - v1 "github.com/grafana/loki/pkg/storage/bloom/v1" -) - -// sortMergeIterator implements v1.Iterator -type sortMergeIterator[T any, C comparable, R any] struct { - curr *R - heap *v1.HeapIterator[v1.IndexedValue[C]] - items []T - transform func(T, C, *R) *R - err error -} - -func (it *sortMergeIterator[T, C, R]) Next() bool { - ok := it.heap.Next() - if !ok { - it.err = io.EOF - return false - } - - group := it.heap.At() - it.curr = it.transform(it.items[group.Index()], group.Value(), it.curr) - - return true -} - -func (it *sortMergeIterator[T, C, R]) At() R { - return *it.curr -} - -func (it *sortMergeIterator[T, C, R]) Err() error { - return it.err -} diff --git a/pkg/bloomutils/ring.go b/pkg/bloomutils/ring.go index 20bb446ba15d3..6da275f607c22 100644 --- a/pkg/bloomutils/ring.go +++ b/pkg/bloomutils/ring.go @@ -107,31 +107,38 @@ func GetInstancesWithTokenRanges(id string, instances []ring.InstanceDesc) Insta // NewInstanceSortMergeIterator creates an iterator that yields instanceWithToken elements // where the token of the elements are sorted in ascending order. func NewInstanceSortMergeIterator(instances []ring.InstanceDesc) v1.Iterator[InstanceWithTokenRange] { - it := &sortMergeIterator[ring.InstanceDesc, uint32, InstanceWithTokenRange]{ - items: instances, - transform: func(item ring.InstanceDesc, val uint32, prev *InstanceWithTokenRange) *InstanceWithTokenRange { - var prevToken uint32 - if prev != nil { - prevToken = prev.MaxToken + 1 - } - return &InstanceWithTokenRange{Instance: item, MinToken: prevToken, MaxToken: val} - }, - } - sequences := make([]v1.PeekingIterator[v1.IndexedValue[uint32]], 0, len(instances)) - for i := range instances { - sort.Slice(instances[i].Tokens, func(a, b int) bool { - return instances[i].Tokens[a] < instances[i].Tokens[b] - }) - iter := v1.NewIterWithIndex[uint32](v1.NewSliceIter(instances[i].Tokens), i) - sequences = append(sequences, v1.NewPeekingIter[v1.IndexedValue[uint32]](iter)) + + tokenIters := make([]v1.PeekingIterator[v1.IndexedValue[uint32]], 0, len(instances)) + for i, inst := range instances { + sort.Slice(inst.Tokens, func(a, b int) bool { return inst.Tokens[a] < inst.Tokens[b] }) + itr := v1.NewIterWithIndex(v1.NewSliceIter[uint32](inst.Tokens), i) + tokenIters = append(tokenIters, v1.NewPeekingIter[v1.IndexedValue[uint32]](itr)) } - it.heap = v1.NewHeapIterator( - func(i, j v1.IndexedValue[uint32]) bool { - return i.Value() < j.Value() + + heapIter := v1.NewHeapIterator[v1.IndexedValue[uint32]]( + func(iv1, iv2 v1.IndexedValue[uint32]) bool { + return iv1.Value() < iv2.Value() }, - sequences..., + tokenIters..., ) - it.err = nil - return it + prevToken := -1 + return v1.NewDedupingIter[v1.IndexedValue[uint32], InstanceWithTokenRange]( + func(iv v1.IndexedValue[uint32], iwtr InstanceWithTokenRange) bool { + return false + }, + func(iv v1.IndexedValue[uint32]) InstanceWithTokenRange { + minToken, maxToken := uint32(prevToken+1), iv.Value() + prevToken = int(maxToken) + return InstanceWithTokenRange{ + Instance: instances[iv.Index()], + MinToken: minToken, + MaxToken: maxToken, + } + }, + func(iv v1.IndexedValue[uint32], iwtr InstanceWithTokenRange) InstanceWithTokenRange { + panic("must not be called, because Eq() is always false") + }, + v1.NewPeekingIter(heapIter), + ) } diff --git a/pkg/bloomutils/ring_test.go b/pkg/bloomutils/ring_test.go index 1346559372c30..6cac31949eef3 100644 --- a/pkg/bloomutils/ring_test.go +++ b/pkg/bloomutils/ring_test.go @@ -11,6 +11,11 @@ import ( ) func TestBloomGatewayClient_SortInstancesByToken(t *testing.T) { + // | 1 2 3 4 5 6 7 8 9 | + // ---------+----------------------------+ + // ID 1 | * * | + // ID 2 | * * | + // ID 3 | * | input := []ring.InstanceDesc{ {Id: "1", Tokens: []uint32{5, 9}}, {Id: "2", Tokens: []uint32{3, 7}}, From 5967ceeb431f515f36bdf4e333cf1cdb23f52193 Mon Sep 17 00:00:00 2001 From: Christian Haudum Date: Thu, 15 Feb 2024 13:16:17 +0100 Subject: [PATCH 13/19] Replace min/max token with `TokenRange` in bloom ring utilities (#11960) This PR replaces min/max token fields with a `TokenRange` field that uses the `Range[uint32]` type. The `Range[uint32]` uses similar semantics as the `FingerprintBounds` we use for fingerprint ranges. Signed-off-by: Christian Haudum --- pkg/bloomcompactor/bloomcompactor.go | 4 +- pkg/bloomgateway/client.go | 31 +++----- pkg/bloomgateway/client_test.go | 39 +++++----- pkg/bloomutils/ring.go | 106 ++++++++++++++------------- pkg/bloomutils/ring_test.go | 68 +++++------------ 5 files changed, 108 insertions(+), 140 deletions(-) diff --git a/pkg/bloomcompactor/bloomcompactor.go b/pkg/bloomcompactor/bloomcompactor.go index 36dcf36ed1fb7..5a579f95fdb77 100644 --- a/pkg/bloomcompactor/bloomcompactor.go +++ b/pkg/bloomcompactor/bloomcompactor.go @@ -201,11 +201,11 @@ func (c *Compactor) ownsTenant(tenant string) (v1.FingerprintBounds, bool, error } - ownershipBounds, err := bloomutils.GetInstanceWithTokenRange(c.cfg.Ring.InstanceID, rs.Instances) + keyRange, err := bloomutils.KeyRangeForInstance(c.cfg.Ring.InstanceID, rs.Instances, bloomutils.Uint64Range) if err != nil { return v1.FingerprintBounds{}, false, errors.Wrap(err, "getting instance token range") } - return ownershipBounds, true, nil + return v1.NewBounds(model.Fingerprint(keyRange.Min), model.Fingerprint(keyRange.Max)), true, nil } // runs a single round of compaction for all relevant tenants and tables diff --git a/pkg/bloomgateway/client.go b/pkg/bloomgateway/client.go index 9a75e4e87c26b..28400749404ce 100644 --- a/pkg/bloomgateway/client.go +++ b/pkg/bloomgateway/client.go @@ -304,25 +304,23 @@ func serverAddressesWithTokenRanges(subRing ring.ReadRing, instances []ring.Inst for it.Next() { // We can use on of the tokens from the token range // to obtain all addresses for that token. - rs, err := subRing.Get(it.At().MaxToken, BlocksOwnerRead, bufDescs, bufHosts, bufZones) + rs, err := subRing.Get(it.At().TokenRange.Max, BlocksOwnerRead, bufDescs, bufHosts, bufZones) if err != nil { return nil, errors.Wrap(err, "bloom gateway get ring") } servers = append(servers, addrsWithTokenRange{ - id: it.At().Instance.Id, - addrs: rs.GetAddresses(), - minToken: it.At().MinToken, - maxToken: it.At().MaxToken, + id: it.At().Instance.Id, + addrs: rs.GetAddresses(), + tokenRange: it.At().TokenRange, }) } - if len(servers) > 0 && servers[len(servers)-1].maxToken < math.MaxUint32 { + if len(servers) > 0 && servers[len(servers)-1].tokenRange.Max < math.MaxUint32 { // append the instance for the token range between the greates token and MaxUint32 servers = append(servers, addrsWithTokenRange{ - id: servers[0].id, - addrs: servers[0].addrs, - minToken: servers[len(servers)-1].maxToken + 1, - maxToken: math.MaxUint32, + id: servers[0].id, + addrs: servers[0].addrs, + tokenRange: bloomutils.NewTokenRange(servers[len(servers)-1].tokenRange.Max+1, math.MaxUint32), }) } return servers, nil @@ -334,18 +332,13 @@ type instanceWithToken struct { } type addrsWithTokenRange struct { - id string - addrs []string - minToken, maxToken uint32 + id string + addrs []string + tokenRange bloomutils.Range[uint32] } func (s addrsWithTokenRange) cmp(token uint32) v1.BoundsCheck { - if token < s.minToken { - return v1.Before - } else if token > s.maxToken { - return v1.After - } - return v1.Overlap + return s.tokenRange.Cmp(token) } type instanceWithFingerprints struct { diff --git a/pkg/bloomgateway/client_test.go b/pkg/bloomgateway/client_test.go index b1716de8150ea..440347d1b2487 100644 --- a/pkg/bloomgateway/client_test.go +++ b/pkg/bloomgateway/client_test.go @@ -19,6 +19,9 @@ import ( "github.com/grafana/loki/pkg/validation" ) +// short constructor +var newTr = bloomutils.NewTokenRange + func TestBloomGatewayClient(t *testing.T) { logger := log.NewNopLogger() reg := prometheus.NewRegistry() @@ -53,10 +56,10 @@ func TestBloomGatewayClient_PartitionFingerprintsByAddresses(t *testing.T) { {Fingerprint: 401}, // out of bounds, will be dismissed } servers := []addrsWithTokenRange{ - {id: "instance-1", addrs: []string{"10.0.0.1"}, minToken: 0, maxToken: 100}, - {id: "instance-2", addrs: []string{"10.0.0.2"}, minToken: 101, maxToken: 200}, - {id: "instance-3", addrs: []string{"10.0.0.3"}, minToken: 201, maxToken: 300}, - {id: "instance-2", addrs: []string{"10.0.0.2"}, minToken: 301, maxToken: 400}, + {id: "instance-1", addrs: []string{"10.0.0.1"}, tokenRange: newTr(0, 100)}, + {id: "instance-2", addrs: []string{"10.0.0.2"}, tokenRange: newTr(101, 200)}, + {id: "instance-3", addrs: []string{"10.0.0.3"}, tokenRange: newTr(201, 300)}, + {id: "instance-2", addrs: []string{"10.0.0.2"}, tokenRange: newTr(301, 400)}, } // partition fingerprints @@ -135,9 +138,9 @@ func TestBloomGatewayClient_PartitionFingerprintsByAddresses(t *testing.T) { {Fingerprint: 350}, } servers := []addrsWithTokenRange{ - {id: "instance-1", addrs: []string{"10.0.0.1"}, minToken: 0, maxToken: 200}, - {id: "instance-2", addrs: []string{"10.0.0.2"}, minToken: 100, maxToken: 300}, - {id: "instance-3", addrs: []string{"10.0.0.3"}, minToken: 200, maxToken: 400}, + {id: "instance-1", addrs: []string{"10.0.0.1"}, tokenRange: newTr(0, 200)}, + {id: "instance-2", addrs: []string{"10.0.0.2"}, tokenRange: newTr(100, 300)}, + {id: "instance-3", addrs: []string{"10.0.0.3"}, tokenRange: newTr(200, 400)}, } // partition fingerprints @@ -174,10 +177,10 @@ func TestBloomGatewayClient_ServerAddressesWithTokenRanges(t *testing.T) { {Id: "instance-3", Addr: "10.0.0.3", Tokens: []uint32{math.MaxUint32 / 6 * 5}}, }, expected: []addrsWithTokenRange{ - {id: "instance-1", addrs: []string{"10.0.0.1"}, minToken: 0, maxToken: math.MaxUint32 / 6 * 1}, - {id: "instance-2", addrs: []string{"10.0.0.2"}, minToken: math.MaxUint32/6*1 + 1, maxToken: math.MaxUint32 / 6 * 3}, - {id: "instance-3", addrs: []string{"10.0.0.3"}, minToken: math.MaxUint32/6*3 + 1, maxToken: math.MaxUint32 / 6 * 5}, - {id: "instance-1", addrs: []string{"10.0.0.1"}, minToken: math.MaxUint32/6*5 + 1, maxToken: math.MaxUint32}, + {id: "instance-1", addrs: []string{"10.0.0.1"}, tokenRange: newTr(0, math.MaxUint32/6*1)}, + {id: "instance-2", addrs: []string{"10.0.0.2"}, tokenRange: newTr(math.MaxUint32/6*1+1, math.MaxUint32/6*3)}, + {id: "instance-3", addrs: []string{"10.0.0.3"}, tokenRange: newTr(math.MaxUint32/6*3+1, math.MaxUint32/6*5)}, + {id: "instance-1", addrs: []string{"10.0.0.1"}, tokenRange: newTr(math.MaxUint32/6*5+1, math.MaxUint32)}, }, }, "MinUint32 and MaxUint32 are tokens in the ring": { @@ -186,10 +189,10 @@ func TestBloomGatewayClient_ServerAddressesWithTokenRanges(t *testing.T) { {Id: "instance-2", Addr: "10.0.0.2", Tokens: []uint32{math.MaxUint32 / 3 * 1, math.MaxUint32}}, }, expected: []addrsWithTokenRange{ - {id: "instance-1", addrs: []string{"10.0.0.1"}, minToken: 0, maxToken: 0}, - {id: "instance-2", addrs: []string{"10.0.0.2"}, minToken: 1, maxToken: math.MaxUint32 / 3}, - {id: "instance-1", addrs: []string{"10.0.0.1"}, minToken: math.MaxUint32/3*1 + 1, maxToken: math.MaxUint32 / 3 * 2}, - {id: "instance-2", addrs: []string{"10.0.0.2"}, minToken: math.MaxUint32/3*2 + 1, maxToken: math.MaxUint32}, + {id: "instance-1", addrs: []string{"10.0.0.1"}, tokenRange: newTr(0, 0)}, + {id: "instance-2", addrs: []string{"10.0.0.2"}, tokenRange: newTr(1, math.MaxUint32/3)}, + {id: "instance-1", addrs: []string{"10.0.0.1"}, tokenRange: newTr(math.MaxUint32/3*1+1, math.MaxUint32/3*2)}, + {id: "instance-2", addrs: []string{"10.0.0.2"}, tokenRange: newTr(math.MaxUint32/3*2+1, math.MaxUint32)}, }, }, } @@ -215,7 +218,7 @@ func TestBloomGatewayClient_GroupFingerprintsByServer(t *testing.T) { it := bloomutils.NewInstanceSortMergeIterator(instances) for it.Next() { - t.Log(it.At().MaxToken, it.At().Instance.Addr) + t.Log(it.At().TokenRange.Max, it.At().Instance.Addr) } testCases := []struct { @@ -357,10 +360,10 @@ type mockRing struct { // Get implements ring.ReadRing. func (r *mockRing) Get(key uint32, _ ring.Operation, _ []ring.InstanceDesc, _ []string, _ []string) (ring.ReplicationSet, error) { idx, _ := sort.Find(len(r.ranges), func(i int) int { - if r.ranges[i].MaxToken < key { + if r.ranges[i].TokenRange.Max < key { return 1 } - if r.ranges[i].MaxToken > key { + if r.ranges[i].TokenRange.Max > key { return -1 } return 0 diff --git a/pkg/bloomutils/ring.go b/pkg/bloomutils/ring.go index 6da275f607c22..102d3ed5e9a5e 100644 --- a/pkg/bloomutils/ring.go +++ b/pkg/bloomutils/ring.go @@ -1,33 +1,62 @@ // This file contains a bunch of utility functions for bloom components. -// TODO: Find a better location for this package package bloomutils import ( + "errors" + "fmt" "math" "sort" "github.com/grafana/dskit/ring" - "github.com/prometheus/common/model" + "golang.org/x/exp/constraints" "golang.org/x/exp/slices" v1 "github.com/grafana/loki/pkg/storage/bloom/v1" ) -type InstanceWithTokenRange struct { - Instance ring.InstanceDesc - MinToken, MaxToken uint32 +var ( + Uint32Range = Range[uint32]{Min: 0, Max: math.MaxUint32} + Uint64Range = Range[uint64]{Min: 0, Max: math.MaxUint64} +) + +type Range[T constraints.Integer] struct { + Min, Max T } -func (i InstanceWithTokenRange) Cmp(token uint32) v1.BoundsCheck { - if token < i.MinToken { +func (r Range[T]) String() string { + return fmt.Sprintf("%016x-%016x", r.Min, r.Max) +} + +func (r Range[T]) Less(other Range[T]) bool { + if r.Min != other.Min { + return r.Min < other.Min + } + return r.Max <= other.Max +} + +func (r Range[T]) Cmp(t T) v1.BoundsCheck { + if t < r.Min { return v1.Before - } else if token > i.MaxToken { + } else if t > r.Max { return v1.After } return v1.Overlap } +func NewTokenRange(min, max uint32) Range[uint32] { + return Range[uint32]{min, max} +} + +type InstanceWithTokenRange struct { + Instance ring.InstanceDesc + TokenRange Range[uint32] +} + +func (i InstanceWithTokenRange) Cmp(token uint32) v1.BoundsCheck { + return i.TokenRange.Cmp(token) +} + type InstancesWithTokenRange []InstanceWithTokenRange func (i InstancesWithTokenRange) Contains(token uint32) bool { @@ -39,11 +68,11 @@ func (i InstancesWithTokenRange) Contains(token uint32) bool { return false } -// GetInstanceWithTokenRange calculates the token range for a specific instance +// KeyRangeForInstance calculates the token range for a specific instance // with given id based on the first token in the ring. // This assumes that each instance in the ring is configured with only a single // token. -func GetInstanceWithTokenRange(id string, instances []ring.InstanceDesc) (v1.FingerprintBounds, error) { +func KeyRangeForInstance[T constraints.Integer](id string, instances []ring.InstanceDesc, keyspace Range[T]) (Range[T], error) { // Sort instances -- they may not be sorted // because they're usually accessed by looking up the tokens (which are sorted) @@ -57,57 +86,31 @@ func GetInstanceWithTokenRange(id string, instances []ring.InstanceDesc) (v1.Fin // instance with Id == id not found if idx == -1 { - return v1.FingerprintBounds{}, ring.ErrInstanceNotFound + return Range[T]{}, ring.ErrInstanceNotFound } - i := uint64(idx) - n := uint64(len(instances)) - step := math.MaxUint64 / n + diff := keyspace.Max - keyspace.Min + i := T(idx) + n := T(len(instances)) - minToken := model.Fingerprint(step * i) - maxToken := model.Fingerprint(step*i + step - 1) + if diff < n { + return Range[T]{}, errors.New("keyspace is smaller than amount of instances") + } + + step := diff / n + min := step * i + max := step*i + step - 1 if i == n-1 { // extend the last token tange to MaxUint32 - maxToken = math.MaxUint64 + max = (keyspace.Max - keyspace.Min) } - return v1.NewBounds(minToken, maxToken), nil -} - -// GetInstancesWithTokenRanges calculates the token ranges for a specific -// instance with given id based on all tokens in the ring. -// If the instances in the ring are configured with a single token, such as the -// bloom compactor, use GetInstanceWithTokenRange() instead. -func GetInstancesWithTokenRanges(id string, instances []ring.InstanceDesc) InstancesWithTokenRange { - servers := make([]InstanceWithTokenRange, 0, len(instances)) - it := NewInstanceSortMergeIterator(instances) - var firstInst ring.InstanceDesc - var lastToken uint32 - for it.Next() { - if firstInst.Id == "" { - firstInst = it.At().Instance - } - if it.At().Instance.Id == id { - servers = append(servers, it.At()) - } - lastToken = it.At().MaxToken - } - // append token range from lastToken+1 to MaxUint32 - // only if the instance with the first token is the current one - if len(servers) > 0 && firstInst.Id == id { - servers = append(servers, InstanceWithTokenRange{ - MinToken: lastToken + 1, - MaxToken: math.MaxUint32, - Instance: servers[0].Instance, - }) - } - return servers + return Range[T]{min, max}, nil } // NewInstanceSortMergeIterator creates an iterator that yields instanceWithToken elements // where the token of the elements are sorted in ascending order. func NewInstanceSortMergeIterator(instances []ring.InstanceDesc) v1.Iterator[InstanceWithTokenRange] { - tokenIters := make([]v1.PeekingIterator[v1.IndexedValue[uint32]], 0, len(instances)) for i, inst := range instances { sort.Slice(inst.Tokens, func(a, b int) bool { return inst.Tokens[a] < inst.Tokens[b] }) @@ -131,9 +134,8 @@ func NewInstanceSortMergeIterator(instances []ring.InstanceDesc) v1.Iterator[Ins minToken, maxToken := uint32(prevToken+1), iv.Value() prevToken = int(maxToken) return InstanceWithTokenRange{ - Instance: instances[iv.Index()], - MinToken: minToken, - MaxToken: maxToken, + Instance: instances[iv.Index()], + TokenRange: NewTokenRange(minToken, maxToken), } }, func(iv v1.IndexedValue[uint32], iwtr InstanceWithTokenRange) InstanceWithTokenRange { diff --git a/pkg/bloomutils/ring_test.go b/pkg/bloomutils/ring_test.go index 6cac31949eef3..c9ff6cf5e1d60 100644 --- a/pkg/bloomutils/ring_test.go +++ b/pkg/bloomutils/ring_test.go @@ -6,27 +6,25 @@ import ( "github.com/grafana/dskit/ring" "github.com/stretchr/testify/require" - - v1 "github.com/grafana/loki/pkg/storage/bloom/v1" ) -func TestBloomGatewayClient_SortInstancesByToken(t *testing.T) { - // | 1 2 3 4 5 6 7 8 9 | - // ---------+----------------------------+ - // ID 1 | * * | - // ID 2 | * * | - // ID 3 | * | +func TestBloomGatewayClient_InstanceSortMergeIterator(t *testing.T) { + // | 0 1 2 3 4 5 6 7 8 9 | + // ---------+---------------------+ + // ID 1 | ***o ***o | + // ID 2 | ***o ***o | + // ID 3 | **o | input := []ring.InstanceDesc{ {Id: "1", Tokens: []uint32{5, 9}}, {Id: "2", Tokens: []uint32{3, 7}}, {Id: "3", Tokens: []uint32{1}}, } expected := []InstanceWithTokenRange{ - {Instance: input[2], MinToken: 0, MaxToken: 1}, - {Instance: input[1], MinToken: 2, MaxToken: 3}, - {Instance: input[0], MinToken: 4, MaxToken: 5}, - {Instance: input[1], MinToken: 6, MaxToken: 7}, - {Instance: input[0], MinToken: 8, MaxToken: 9}, + {Instance: input[2], TokenRange: NewTokenRange(0, 1)}, + {Instance: input[1], TokenRange: NewTokenRange(2, 3)}, + {Instance: input[0], TokenRange: NewTokenRange(4, 5)}, + {Instance: input[1], TokenRange: NewTokenRange(6, 7)}, + {Instance: input[0], TokenRange: NewTokenRange(8, 9)}, } var i int @@ -38,43 +36,15 @@ func TestBloomGatewayClient_SortInstancesByToken(t *testing.T) { } } -func TestBloomGatewayClient_GetInstancesWithTokenRanges(t *testing.T) { - t.Run("instance does not own first token in the ring", func(t *testing.T) { - input := []ring.InstanceDesc{ - {Id: "1", Tokens: []uint32{5, 9}}, - {Id: "2", Tokens: []uint32{3, 7}}, - {Id: "3", Tokens: []uint32{1}}, - } - expected := InstancesWithTokenRange{ - {Instance: input[1], MinToken: 2, MaxToken: 3}, - {Instance: input[1], MinToken: 6, MaxToken: 7}, - } - - result := GetInstancesWithTokenRanges("2", input) - require.Equal(t, expected, result) - }) - - t.Run("instance owns first token in the ring", func(t *testing.T) { - input := []ring.InstanceDesc{ - {Id: "1", Tokens: []uint32{5, 9}}, - {Id: "2", Tokens: []uint32{3, 7}}, - {Id: "3", Tokens: []uint32{1}}, - } - expected := InstancesWithTokenRange{ - {Instance: input[2], MinToken: 0, MaxToken: 1}, - {Instance: input[2], MinToken: 10, MaxToken: math.MaxUint32}, - } - - result := GetInstancesWithTokenRanges("3", input) - require.Equal(t, expected, result) - }) +func uint64Range(min, max uint64) Range[uint64] { + return Range[uint64]{min, max} } -func TestBloomGatewayClient_GetInstanceWithTokenRange(t *testing.T) { +func TestBloomGatewayClient_KeyRangeForInstance(t *testing.T) { for name, tc := range map[string]struct { id string input []ring.InstanceDesc - expected v1.FingerprintBounds + expected Range[uint64] }{ "first instance includes 0 token": { id: "3", @@ -83,7 +53,7 @@ func TestBloomGatewayClient_GetInstanceWithTokenRange(t *testing.T) { {Id: "2", Tokens: []uint32{5}}, {Id: "3", Tokens: []uint32{1}}, }, - expected: v1.NewBounds(0, math.MaxUint64/3-1), + expected: uint64Range(0, math.MaxUint64/3-1), }, "middle instance": { id: "1", @@ -92,7 +62,7 @@ func TestBloomGatewayClient_GetInstanceWithTokenRange(t *testing.T) { {Id: "2", Tokens: []uint32{5}}, {Id: "3", Tokens: []uint32{1}}, }, - expected: v1.NewBounds(math.MaxUint64/3, math.MaxUint64/3*2-1), + expected: uint64Range(math.MaxUint64/3, math.MaxUint64/3*2-1), }, "last instance includes MaxUint32 token": { id: "2", @@ -101,12 +71,12 @@ func TestBloomGatewayClient_GetInstanceWithTokenRange(t *testing.T) { {Id: "2", Tokens: []uint32{5}}, {Id: "3", Tokens: []uint32{1}}, }, - expected: v1.NewBounds(math.MaxUint64/3*2, math.MaxUint64), + expected: uint64Range(math.MaxUint64/3*2, math.MaxUint64), }, } { tc := tc t.Run(name, func(t *testing.T) { - result, err := GetInstanceWithTokenRange(tc.id, tc.input) + result, err := KeyRangeForInstance(tc.id, tc.input, Uint64Range) require.NoError(t, err) require.Equal(t, tc.expected, result) }) From c328a4f42e8d4296a92df51345a5c02494c2e6fc Mon Sep 17 00:00:00 2001 From: J Stickler Date: Thu, 15 Feb 2024 09:41:06 -0500 Subject: [PATCH 14/19] Fixing broken links (#11956) --- docs/sources/release-notes/cadence.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/sources/release-notes/cadence.md b/docs/sources/release-notes/cadence.md index f13781cf1c5f3..ef6fbcaf072fd 100644 --- a/docs/sources/release-notes/cadence.md +++ b/docs/sources/release-notes/cadence.md @@ -8,7 +8,7 @@ weight: 1 ## Stable Releases -Loki releases (this includes [Promtail](/clients/promtail), [Loki Canary](/operations/loki-canary/), etc) use the following +Loki releases (this includes [Promtail](https://grafana.com/docs/loki//send-data/promtail/), [Loki Canary](https://grafana.com/docs/loki//operations/loki-canary/), etc.) use the following naming scheme: `MAJOR`.`MINOR`.`PATCH`. - `MAJOR` (roughly once a year): these releases include large new features and possible backwards-compatibility breaks. @@ -18,14 +18,14 @@ naming scheme: `MAJOR`.`MINOR`.`PATCH`. {{% admonition type="note" %}} While our naming scheme resembles [Semantic Versioning](https://semver.org/), at this time we do not strictly follow its guidelines to the letter. Our goal is to provide regular releases that are as stable as possible, and we take backwards-compatibility -seriously. As with any software, always read the [release notes](/release-notes) and the [upgrade guide](/upgrading) whenever +seriously. As with any software, always read the [release notes](https://grafana.com/docs/loki//release-notes/) and the [upgrade guide](https://grafana.com/docs/loki//setup/upgrade/) whenever choosing a new version of Loki to install. {{% /admonition %}} New releases are based of a [weekly release](#weekly-releases) which we have vetted for stability over a number of weeks. We strongly recommend keeping up-to-date with patch releases as they are released. We post updates of new releases in the `#loki` channel -of our [Slack community](/community/getting-in-touch). +of our [Slack community](https://grafana.com/docs/loki//community/getting-in-touch/). You can find all of our releases [on GitHub](https://github.com/grafana/loki/releases) and on [Docker Hub](https://hub.docker.com/r/grafana/loki). From a955ba93362cd3349d68511c01a26deec8a70f80 Mon Sep 17 00:00:00 2001 From: Zirko <64951262+QuantumEnigmaa@users.noreply.github.com> Date: Thu, 15 Feb 2024 16:22:18 +0100 Subject: [PATCH 15/19] Helm: fix ciliumNetworkPolicy template in the chart (#11963) Signed-off-by: QuantumEnigmaa --- production/helm/loki/CHANGELOG.md | 4 +++ production/helm/loki/Chart.yaml | 2 +- production/helm/loki/README.md | 2 +- .../loki/templates/ciliumnetworkpolicy.yaml | 29 +++++++++++-------- 4 files changed, 23 insertions(+), 14 deletions(-) diff --git a/production/helm/loki/CHANGELOG.md b/production/helm/loki/CHANGELOG.md index 47d8f6333e4e9..d232a7d6c7eae 100644 --- a/production/helm/loki/CHANGELOG.md +++ b/production/helm/loki/CHANGELOG.md @@ -14,6 +14,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.43.1 + +- [BUGFIX] Fix `toPorts` fields in the `ciliumnetworkpolicy` template + ## 5.43.0 - [ENHANCEMENT] Allow the definition of resources for GrafanaAgent pods diff --git a/production/helm/loki/Chart.yaml b/production/helm/loki/Chart.yaml index ffa62c88d5cd3..49d7ca836b8a4 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.43.0 +version: 5.43.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 5db87e6d801e0..a4ef51dd9d5e6 100644 --- a/production/helm/loki/README.md +++ b/production/helm/loki/README.md @@ -1,6 +1,6 @@ # loki -![Version: 5.43.0](https://img.shields.io/badge/Version-5.43.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.43.1](https://img.shields.io/badge/Version-5.43.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/ciliumnetworkpolicy.yaml b/production/helm/loki/templates/ciliumnetworkpolicy.yaml index ddcef3b61a8ec..1eb1644dd850e 100644 --- a/production/helm/loki/templates/ciliumnetworkpolicy.yaml +++ b/production/helm/loki/templates/ciliumnetworkpolicy.yaml @@ -60,8 +60,9 @@ spec: {{- include "loki.selectorLabels" . | nindent 6 }} ingress: - toPorts: - - port: http - protocol: TCP + - ports: + - port: http + protocol: TCP {{- if .Values.networkPolicy.ingress.namespaceSelector }} fromEndpoints: - matchLabels: @@ -85,8 +86,9 @@ spec: {{- include "loki.selectorLabels" . | nindent 6 }} ingress: - toPorts: - - port: http-metrics - protocol: TCP + - ports: + - port: http-metrics + protocol: TCP {{- if .Values.networkPolicy.metrics.cidrs }} {{- range $cidr := .Values.networkPolicy.metrics.cidrs }} toCIDR: @@ -116,8 +118,9 @@ spec: {{- include "loki.backendSelectorLabels" . | nindent 6 }} egress: - toPorts: - - port: {{ .Values.networkPolicy.alertmanager.port }} - protocol: TCP + - ports: + - port: "{{ .Values.networkPolicy.alertmanager.port }}" + protocol: TCP {{- if .Values.networkPolicy.alertmanager.namespaceSelector }} toEndpoints: - matchLabels: @@ -142,10 +145,11 @@ spec: {{- include "loki.selectorLabels" . | nindent 6 }} egress: - toPorts: - {{- range $port := .Values.networkPolicy.externalStorage.ports }} - - port: {{ $port }} - protocol: TCP - {{- end }} + - ports: + {{- range $port := .Values.networkPolicy.externalStorage.ports }} + - port: "{{ $port }}" + protocol: TCP + {{- end }}à {{- if .Values.networkPolicy.externalStorage.cidrs }} {{- range $cidr := .Values.networkPolicy.externalStorage.cidrs }} toCIDR: @@ -171,8 +175,9 @@ spec: {{- include "loki.selectorLabels" . | nindent 6 }} egress: - toPorts: - - port: {{ .Values.networkPolicy.discovery.port }} - protocol: TCP + - ports: + - port: "{{ .Values.networkPolicy.discovery.port }}" + protocol: TCP {{- if .Values.networkPolicy.discovery.namespaceSelector }} toEndpoints: - matchLabels: From cad4b8e749ccc19872473520f0456a43edc23be6 Mon Sep 17 00:00:00 2001 From: JordanRushing Date: Thu, 15 Feb 2024 11:36:37 -0600 Subject: [PATCH 16/19] Reduce Distributor auto-forget unhealthy cycles from 10->2 (#11935) Signed-off-by: JordanRushing --- pkg/distributor/distributor.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/distributor/distributor.go b/pkg/distributor/distributor.go index a5229b0ca1498..f47148fa42b0d 100644 --- a/pkg/distributor/distributor.go +++ b/pkg/distributor/distributor.go @@ -52,7 +52,7 @@ import ( const ( ringKey = "distributor" - ringAutoForgetUnhealthyPeriods = 10 + ringAutoForgetUnhealthyPeriods = 2 ) var ( From 4e1b210017a19519631db2c451858782de28835a Mon Sep 17 00:00:00 2001 From: Robert Jacob Date: Thu, 15 Feb 2024 18:58:53 +0100 Subject: [PATCH 17/19] operator: Provide Azure region for managed credentials using environment variable (#11964) --- operator/CHANGELOG.md | 1 + operator/internal/config/managed_auth.go | 2 + ...equest_create.go => credentialsrequest.go} | 31 ---------- ...ate_test.go => credentialsrequest_test.go} | 61 ++----------------- .../handlers/internal/storage/secrets.go | 7 --- .../handlers/internal/storage/secrets_test.go | 21 ------- .../manifests/openshift/credentialsrequest.go | 11 ++++ .../internal/manifests/storage/options.go | 1 - operator/internal/manifests/storage/var.go | 2 - 9 files changed, 19 insertions(+), 118 deletions(-) rename operator/internal/handlers/{credentialsrequest_create.go => credentialsrequest.go} (67%) rename operator/internal/handlers/{credentialsrequest_create_test.go => credentialsrequest_test.go} (71%) diff --git a/operator/CHANGELOG.md b/operator/CHANGELOG.md index 59afb29708782..d504e4ee31b52 100644 --- a/operator/CHANGELOG.md +++ b/operator/CHANGELOG.md @@ -1,5 +1,6 @@ ## Main +- [11964](https://github.com/grafana/loki/pull/11964) **xperimental**: Provide Azure region for managed credentials using environment variable - [11920](https://github.com/grafana/loki/pull/11920) **xperimental**: Refactor handling of credentials in managed-auth mode - [11869](https://github.com/grafana/loki/pull/11869) **periklis**: Add support for running with Google Workload Identity - [11868](https://github.com/grafana/loki/pull/11868) **xperimental**: Integrate support for OpenShift-managed credentials in Azure diff --git a/operator/internal/config/managed_auth.go b/operator/internal/config/managed_auth.go index 73598e7032f8f..76f9d72f3c262 100644 --- a/operator/internal/config/managed_auth.go +++ b/operator/internal/config/managed_auth.go @@ -26,6 +26,7 @@ func discoverManagedAuthConfig() *ManagedAuthConfig { clientID := os.Getenv("CLIENTID") tenantID := os.Getenv("TENANTID") subscriptionID := os.Getenv("SUBSCRIPTIONID") + region := os.Getenv("REGION") switch { case roleARN != "": @@ -40,6 +41,7 @@ func discoverManagedAuthConfig() *ManagedAuthConfig { ClientID: clientID, SubscriptionID: subscriptionID, TenantID: tenantID, + Region: region, }, } } diff --git a/operator/internal/handlers/credentialsrequest_create.go b/operator/internal/handlers/credentialsrequest.go similarity index 67% rename from operator/internal/handlers/credentialsrequest_create.go rename to operator/internal/handlers/credentialsrequest.go index 50e06375ffd8b..0d562332dc9d5 100644 --- a/operator/internal/handlers/credentialsrequest_create.go +++ b/operator/internal/handlers/credentialsrequest.go @@ -2,12 +2,10 @@ package handlers import ( "context" - "errors" "fmt" "github.com/ViaQ/logerr/v2/kverrors" "github.com/go-logr/logr" - corev1 "k8s.io/api/core/v1" apierrors "k8s.io/apimachinery/pkg/api/errors" "k8s.io/apimachinery/pkg/runtime" ctrl "sigs.k8s.io/controller-runtime" @@ -19,11 +17,8 @@ import ( "github.com/grafana/loki/operator/internal/external/k8s" "github.com/grafana/loki/operator/internal/manifests" "github.com/grafana/loki/operator/internal/manifests/openshift" - "github.com/grafana/loki/operator/internal/manifests/storage" ) -var errAzureNoRegion = errors.New("can not create CredentialsRequest: missing secret field: region") - // CreateCredentialsRequest creates a new CredentialsRequest resource for a Lokistack // to request a cloud credentials Secret resource from the OpenShift cloud-credentials-operator. func CreateCredentialsRequest(ctx context.Context, log logr.Logger, scheme *runtime.Scheme, managedAuth *config.ManagedAuthConfig, k k8s.Client, req ctrl.Request) error { @@ -39,32 +34,6 @@ func CreateCredentialsRequest(ctx context.Context, log logr.Logger, scheme *runt return kverrors.Wrap(err, "failed to lookup LokiStack", "name", req.String()) } - if managedAuth.Azure != nil && managedAuth.Azure.Region == "" { - // Managed environment for Azure does not provide Region, but we need this for the CredentialsRequest. - // This looks like an oversight when creating the UI in OpenShift, but for now we need to pull this data - // from somewhere else -> the Azure Storage Secret - storageSecretName := client.ObjectKey{ - Namespace: stack.Namespace, - Name: stack.Spec.Storage.Secret.Name, - } - storageSecret := &corev1.Secret{} - if err := k.Get(ctx, storageSecretName, storageSecret); err != nil { - if apierrors.IsNotFound(err) { - // Skip this error here as it will be picked up by the LokiStack handler instead - ll.Error(err, "could not find secret for LokiStack", "name", req.String()) - return nil - } - return err - } - - region := storageSecret.Data[storage.KeyAzureRegion] - if len(region) == 0 { - return errAzureNoRegion - } - - managedAuth.Azure.Region = string(region) - } - opts := openshift.Options{ BuildOpts: openshift.BuildOptions{ LokiStackName: stack.Name, diff --git a/operator/internal/handlers/credentialsrequest_create_test.go b/operator/internal/handlers/credentialsrequest_test.go similarity index 71% rename from operator/internal/handlers/credentialsrequest_create_test.go rename to operator/internal/handlers/credentialsrequest_test.go index 626302a113274..dd6dfb50d77dc 100644 --- a/operator/internal/handlers/credentialsrequest_create_test.go +++ b/operator/internal/handlers/credentialsrequest_test.go @@ -6,7 +6,6 @@ import ( cloudcredentialv1 "github.com/openshift/cloud-credential-operator/pkg/apis/cloudcredential/v1" "github.com/stretchr/testify/require" - corev1 "k8s.io/api/core/v1" "k8s.io/apimachinery/pkg/api/errors" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/apimachinery/pkg/runtime/schema" @@ -19,7 +18,7 @@ import ( "github.com/grafana/loki/operator/internal/external/k8s/k8sfakes" ) -func credentialsRequestFakeClient(cr *cloudcredentialv1.CredentialsRequest, lokistack *lokiv1.LokiStack, secret *corev1.Secret) *k8sfakes.FakeClient { +func credentialsRequestFakeClient(cr *cloudcredentialv1.CredentialsRequest, lokistack *lokiv1.LokiStack) *k8sfakes.FakeClient { k := &k8sfakes.FakeClient{} k.GetStub = func(_ context.Context, name types.NamespacedName, object client.Object, _ ...client.GetOption) error { switch object.(type) { @@ -33,11 +32,6 @@ func credentialsRequestFakeClient(cr *cloudcredentialv1.CredentialsRequest, loki return errors.NewNotFound(schema.GroupResource{}, name.Name) } k.SetClientObject(object, lokistack) - case *corev1.Secret: - if secret == nil { - return errors.NewNotFound(schema.GroupResource{}, name.Name) - } - k.SetClientObject(object, secret) } return nil } @@ -58,7 +52,7 @@ func TestCreateCredentialsRequest_CreateNewResource(t *testing.T) { }, } - k := credentialsRequestFakeClient(nil, lokistack, nil) + k := credentialsRequestFakeClient(nil, lokistack) req := ctrl.Request{ NamespacedName: client.ObjectKey{Name: "my-stack", Namespace: "ns"}, } @@ -89,13 +83,8 @@ func TestCreateCredentialsRequest_CreateNewResourceAzure(t *testing.T) { Namespace: "ns", }, } - secret := &corev1.Secret{ - Data: map[string][]byte{ - "region": []byte(wantRegion), - }, - } - k := credentialsRequestFakeClient(nil, lokistack, secret) + k := credentialsRequestFakeClient(nil, lokistack) req := ctrl.Request{ NamespacedName: client.ObjectKey{Name: "my-stack", Namespace: "ns"}, } @@ -105,6 +94,7 @@ func TestCreateCredentialsRequest_CreateNewResourceAzure(t *testing.T) { ClientID: "test-client-id", SubscriptionID: "test-tenant-id", TenantID: "test-subscription-id", + Region: "test-region", }, } @@ -122,47 +112,6 @@ func TestCreateCredentialsRequest_CreateNewResourceAzure(t *testing.T) { require.Equal(t, wantRegion, providerSpec.AzureRegion) } -func TestCreateCredentialsRequest_CreateNewResourceAzure_Errors(t *testing.T) { - lokistack := &lokiv1.LokiStack{ - ObjectMeta: metav1.ObjectMeta{ - Name: "my-stack", - Namespace: "ns", - }, - } - req := ctrl.Request{ - NamespacedName: client.ObjectKey{Name: "my-stack", Namespace: "ns"}, - } - - tt := []struct { - secret *corev1.Secret - wantError string - }{ - { - secret: &corev1.Secret{}, - wantError: errAzureNoRegion.Error(), - }, - } - - for _, tc := range tt { - tc := tc - t.Run(tc.wantError, func(t *testing.T) { - t.Parallel() - - managedAuth := &config.ManagedAuthConfig{ - Azure: &config.AzureEnvironment{ - ClientID: "test-client-id", - SubscriptionID: "test-tenant-id", - TenantID: "test-subscription-id", - }, - } - k := credentialsRequestFakeClient(nil, lokistack, tc.secret) - - err := CreateCredentialsRequest(context.Background(), logger, scheme, managedAuth, k, req) - require.EqualError(t, err, tc.wantError) - }) - } -} - func TestCreateCredentialsRequest_DoNothing_WhenCredentialsRequestExist(t *testing.T) { req := ctrl.Request{ NamespacedName: client.ObjectKey{Name: "my-stack", Namespace: "ns"}, @@ -187,7 +136,7 @@ func TestCreateCredentialsRequest_DoNothing_WhenCredentialsRequestExist(t *testi }, } - k := credentialsRequestFakeClient(cr, lokistack, nil) + k := credentialsRequestFakeClient(cr, lokistack) err := CreateCredentialsRequest(context.Background(), logger, scheme, managedAuth, k, req) require.NoError(t, err) diff --git a/operator/internal/handlers/internal/storage/secrets.go b/operator/internal/handlers/internal/storage/secrets.go index 99bafb911ec26..2492eea4d4191 100644 --- a/operator/internal/handlers/internal/storage/secrets.go +++ b/operator/internal/handlers/internal/storage/secrets.go @@ -182,18 +182,11 @@ func extractAzureConfigSecret(s *corev1.Secret, fg configv1.FeatureGates) (*stor // Extract and validate optional fields endpointSuffix := s.Data[storage.KeyAzureStorageEndpointSuffix] audience := s.Data[storage.KeyAzureAudience] - region := s.Data[storage.KeyAzureRegion] if !workloadIdentity && len(audience) > 0 { return nil, fmt.Errorf("%w: %s", errSecretFieldNotAllowed, storage.KeyAzureAudience) } - if fg.OpenShift.ManagedAuthEnv { - if len(region) == 0 { - return nil, fmt.Errorf("%w: %s", errSecretMissingField, storage.KeyAzureRegion) - } - } - return &storage.AzureStorageConfig{ Env: string(env), Container: string(container), diff --git a/operator/internal/handlers/internal/storage/secrets_test.go b/operator/internal/handlers/internal/storage/secrets_test.go index 1363cd4a660a6..ca3623b718c1b 100644 --- a/operator/internal/handlers/internal/storage/secrets_test.go +++ b/operator/internal/handlers/internal/storage/secrets_test.go @@ -156,27 +156,6 @@ func TestAzureExtract(t *testing.T) { }, wantError: "missing secret field: subscription_id", }, - { - name: "managed auth - no region", - secret: &corev1.Secret{ - ObjectMeta: metav1.ObjectMeta{Name: "test"}, - Data: map[string][]byte{ - "environment": []byte("here"), - "account_name": []byte("test-account-name"), - "container": []byte("this,that"), - }, - }, - managedSecret: &corev1.Secret{ - Data: map[string][]byte{}, - }, - featureGates: configv1.FeatureGates{ - OpenShift: configv1.OpenShiftFeatureGates{ - Enabled: true, - ManagedAuthEnv: true, - }, - }, - wantError: "missing secret field: region", - }, { name: "managed auth - no auth override", secret: &corev1.Secret{ diff --git a/operator/internal/manifests/openshift/credentialsrequest.go b/operator/internal/manifests/openshift/credentialsrequest.go index 0e97dd97c2b19..0c0a19adc98d3 100644 --- a/operator/internal/manifests/openshift/credentialsrequest.go +++ b/operator/internal/manifests/openshift/credentialsrequest.go @@ -12,6 +12,8 @@ import ( "github.com/grafana/loki/operator/internal/manifests/storage" ) +const azureFallbackRegion = "centralus" + func BuildCredentialsRequest(opts Options) (*cloudcredentialv1.CredentialsRequest, error) { stack := client.ObjectKey{Name: opts.BuildOpts.LokiStackName, Namespace: opts.BuildOpts.LokiStackNamespace} @@ -62,6 +64,15 @@ func encodeProviderSpec(env *config.ManagedAuthConfig) (*runtime.RawExtension, e } case env.Azure != nil: azure := env.Azure + if azure.Region == "" { + // The OpenShift Console currently does not provide a UI to configure the Azure Region + // for an operator using managed credentials. Because the CredentialsRequest is currently + // not used to create a Managed Identity, the region is actually never used. + // We default to the US region if nothing is set, so that the CredentialsRequest can be + // created. This should have no effect on the generated credential secret. + // The region can be configured by setting an environment variable on the operator Subscription. + azure.Region = azureFallbackRegion + } spec = &cloudcredentialv1.AzureProviderSpec{ Permissions: []string{ diff --git a/operator/internal/manifests/storage/options.go b/operator/internal/manifests/storage/options.go index 6693d2261e978..56e2b8e870df1 100644 --- a/operator/internal/manifests/storage/options.go +++ b/operator/internal/manifests/storage/options.go @@ -63,7 +63,6 @@ type AzureStorageConfig struct { Container string EndpointSuffix string Audience string - Region string WorkloadIdentity bool } diff --git a/operator/internal/manifests/storage/var.go b/operator/internal/manifests/storage/var.go index cbd944a821c34..1f236406bdd09 100644 --- a/operator/internal/manifests/storage/var.go +++ b/operator/internal/manifests/storage/var.go @@ -88,8 +88,6 @@ const ( KeyAzureStorageEndpointSuffix = "endpoint_suffix" // KeyAzureEnvironmentName is the secret data key for the Azure cloud environment name. KeyAzureEnvironmentName = "environment" - // KeyAzureRegion is the secret data key for storing the Azure cloud region. - KeyAzureRegion = "region" // KeyAzureAudience is the secret data key for customizing the audience used for the ServiceAccount token. KeyAzureAudience = "audience" From 443720f47b3e79d608cb4cc42c6f12e108352b29 Mon Sep 17 00:00:00 2001 From: Owen Diehl Date: Thu, 15 Feb 2024 11:03:07 -0800 Subject: [PATCH 18/19] tsdb parsing handles uint (#11969) --- .../shipper/indexshipper/tsdb/identifier.go | 2 +- .../indexshipper/tsdb/identifier_test.go | 27 +++++++++++++------ 2 files changed, 20 insertions(+), 9 deletions(-) diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/identifier.go b/pkg/storage/stores/shipper/indexshipper/tsdb/identifier.go index 451688d040e36..943127f3e6b67 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/identifier.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/identifier.go @@ -128,7 +128,7 @@ func ParseSingleTenantTSDBPath(p string) (id SingleTenantTSDBIdentifier, ok bool return } - checksum, err := strconv.ParseInt(elems[4], 16, 32) + checksum, err := strconv.ParseUint(elems[4], 16, 32) if err != nil { return } diff --git a/pkg/storage/stores/shipper/indexshipper/tsdb/identifier_test.go b/pkg/storage/stores/shipper/indexshipper/tsdb/identifier_test.go index 7fcd56f89b0eb..b21e8352b7a84 100644 --- a/pkg/storage/stores/shipper/indexshipper/tsdb/identifier_test.go +++ b/pkg/storage/stores/shipper/indexshipper/tsdb/identifier_test.go @@ -1,6 +1,8 @@ package tsdb import ( + "fmt" + "math" "testing" "time" @@ -9,11 +11,10 @@ import ( func TestParseSingleTenantTSDBPath(t *testing.T) { for _, tc := range []struct { - desc string - input string - id SingleTenantTSDBIdentifier - parent string - ok bool + desc string + input string + id SingleTenantTSDBIdentifier + ok bool }{ { desc: "simple_works", @@ -24,8 +25,18 @@ func TestParseSingleTenantTSDBPath(t *testing.T) { Through: 10, Checksum: 255, }, - parent: "parent", - ok: true, + ok: true, + }, + { + desc: "uint32_max_checksum_works", + input: fmt.Sprintf("1-compactor-1-10-%x.tsdb", math.MaxUint32), + id: SingleTenantTSDBIdentifier{ + TS: time.Unix(1, 0), + From: 1, + Through: 10, + Checksum: math.MaxUint32, + }, + ok: true, }, { desc: "wrong uploader name", @@ -45,8 +56,8 @@ func TestParseSingleTenantTSDBPath(t *testing.T) { } { t.Run(tc.desc, func(t *testing.T) { id, ok := ParseSingleTenantTSDBPath(tc.input) - require.Equal(t, tc.id, id) require.Equal(t, tc.ok, ok) + require.Equal(t, tc.id, id) }) } } From 543aaab0553f3367688415a25b908453041644e4 Mon Sep 17 00:00:00 2001 From: Salva Corts Date: Thu, 15 Feb 2024 20:21:41 +0100 Subject: [PATCH 19/19] (Blooms) Add metrics to compactor (#11966) --- pkg/bloomcompactor/bloomcompactor.go | 21 ++++- pkg/bloomcompactor/controller.go | 15 ++-- pkg/bloomcompactor/metrics.go | 113 +++++++++++++-------------- pkg/storage/bloom/v1/util.go | 26 ++++++ pkg/storage/bloom/v1/util_test.go | 26 ++++++ 5 files changed, 135 insertions(+), 66 deletions(-) diff --git a/pkg/bloomcompactor/bloomcompactor.go b/pkg/bloomcompactor/bloomcompactor.go index 5a579f95fdb77..e8dc880f9d9db 100644 --- a/pkg/bloomcompactor/bloomcompactor.go +++ b/pkg/bloomcompactor/bloomcompactor.go @@ -109,9 +109,7 @@ func New( c.logger, ) - c.metrics.compactionRunInterval.Set(cfg.CompactionInterval.Seconds()) c.Service = services.NewBasicService(c.starting, c.running, c.stopping) - return c, nil } @@ -138,11 +136,17 @@ func (c *Compactor) running(ctx context.Context) error { case <-ctx.Done(): return ctx.Err() - case <-ticker.C: + case start := <-ticker.C: + c.metrics.compactionsStarted.Inc() if err := c.runOne(ctx); err != nil { - level.Error(c.logger).Log("msg", "compaction iteration failed", "err", err) + level.Error(c.logger).Log("msg", "compaction iteration failed", "err", err, "duration", time.Since(start)) + c.metrics.compactionCompleted.WithLabelValues(statusFailure).Inc() + c.metrics.compactionTime.WithLabelValues(statusFailure).Observe(time.Since(start).Seconds()) return err } + level.Info(c.logger).Log("msg", "compaction iteration completed", "duration", time.Since(start)) + c.metrics.compactionCompleted.WithLabelValues(statusSuccess).Inc() + c.metrics.compactionTime.WithLabelValues(statusSuccess).Observe(time.Since(start).Seconds()) } } } @@ -252,14 +256,17 @@ func (c *Compactor) loadWork(ctx context.Context, ch chan<- tenantTable) error { } for tenants.Next() && tenants.Err() == nil && ctx.Err() == nil { + c.metrics.tenantsDiscovered.Inc() tenant := tenants.At() ownershipRange, owns, err := c.ownsTenant(tenant) if err != nil { return errors.Wrap(err, "checking tenant ownership") } if !owns { + c.metrics.tenantsSkipped.Inc() continue } + c.metrics.tenantsOwned.Inc() select { case ch <- tenantTable{tenant: tenant, table: table, ownershipRange: ownershipRange}: @@ -296,7 +303,11 @@ func (c *Compactor) runWorkers(ctx context.Context, ch <-chan tenantTable) error return nil } + start := time.Now() + c.metrics.tenantsStarted.Inc() if err := c.compactTenantTable(ctx, tt); err != nil { + c.metrics.tenantsCompleted.WithLabelValues(statusFailure).Inc() + c.metrics.tenantsCompletedTime.WithLabelValues(statusFailure).Observe(time.Since(start).Seconds()) return errors.Wrapf( err, "compacting tenant table (%s) for tenant (%s) with ownership (%s)", @@ -305,6 +316,8 @@ func (c *Compactor) runWorkers(ctx context.Context, ch <-chan tenantTable) error tt.ownershipRange, ) } + c.metrics.tenantsCompleted.WithLabelValues(statusSuccess).Inc() + c.metrics.tenantsCompletedTime.WithLabelValues(statusSuccess).Observe(time.Since(start).Seconds()) } } diff --git a/pkg/bloomcompactor/controller.go b/pkg/bloomcompactor/controller.go index cc801dc27e550..089ab800c7e31 100644 --- a/pkg/bloomcompactor/controller.go +++ b/pkg/bloomcompactor/controller.go @@ -25,7 +25,6 @@ type SimpleBloomController struct { metrics *Metrics limits Limits - // TODO(owen-d): add metrics logger log.Logger } @@ -269,6 +268,7 @@ func (s *SimpleBloomController) buildGaps( maxBlockSize = uint64(s.limits.BloomCompactorMaxBlockSize(tenant)) blockOpts = v1.NewBlockOptions(nGramSize, nGramSkip, maxBlockSize) created []bloomshipper.Meta + totalSeries uint64 ) for _, plan := range work { @@ -295,10 +295,15 @@ func (s *SimpleBloomController) buildGaps( return nil, errors.Wrap(err, "failed to get series and blocks") } + // Blocks are built consuming the series iterator. For observability, we wrap the series iterator + // with a counter iterator to count the number of times Next() is called on it. + // This is used to observe the number of series that are being processed. + seriesItrWithCounter := v1.NewCounterIter[*v1.Series](seriesItr) + gen := NewSimpleBloomGenerator( tenant, blockOpts, - seriesItr, + seriesItrWithCounter, s.chunkLoader, blocksIter, s.rwFn, @@ -307,9 +312,7 @@ func (s *SimpleBloomController) buildGaps( ) _, loaded, newBlocks, err := gen.Generate(ctx) - if err != nil { - // TODO(owen-d): metrics level.Error(logger).Log("msg", "failed to generate bloom", "err", err) s.closeLoadedBlocks(loaded, blocksIter) return nil, errors.Wrap(err, "failed to generate bloom") @@ -338,7 +341,6 @@ func (s *SimpleBloomController) buildGaps( } if err := newBlocks.Err(); err != nil { - // TODO(owen-d): metrics level.Error(logger).Log("msg", "failed to generate bloom", "err", err) s.closeLoadedBlocks(loaded, blocksIter) return nil, errors.Wrap(err, "failed to generate bloom") @@ -360,9 +362,12 @@ func (s *SimpleBloomController) buildGaps( return nil, errors.Wrap(err, "failed to write meta") } created = append(created, meta) + + totalSeries += uint64(seriesItrWithCounter.Count()) } } + s.metrics.tenantsSeries.Observe(float64(totalSeries)) level.Debug(logger).Log("msg", "finished bloom generation", "blocks", blockCt, "tsdbs", tsdbCt) return created, nil } diff --git a/pkg/bloomcompactor/metrics.go b/pkg/bloomcompactor/metrics.go index b02ac32aca727..350e3ed7e480e 100644 --- a/pkg/bloomcompactor/metrics.go +++ b/pkg/bloomcompactor/metrics.go @@ -16,105 +16,104 @@ const ( ) type Metrics struct { - bloomMetrics *v1.Metrics - chunkSize prometheus.Histogram // uncompressed size of all chunks summed per series + bloomMetrics *v1.Metrics + compactorRunning prometheus.Gauge + chunkSize prometheus.Histogram // uncompressed size of all chunks summed per series - compactionRunsStarted prometheus.Counter - compactionRunsCompleted *prometheus.CounterVec - compactionRunTime *prometheus.HistogramVec - compactionRunDiscoveredTenants prometheus.Counter - compactionRunSkippedTenants prometheus.Counter - compactionRunTenantsCompleted *prometheus.CounterVec - compactionRunTenantsTime *prometheus.HistogramVec - compactionRunJobStarted prometheus.Counter - compactionRunJobCompleted *prometheus.CounterVec - compactionRunJobTime *prometheus.HistogramVec - compactionRunInterval prometheus.Gauge - compactorRunning prometheus.Gauge + compactionsStarted prometheus.Counter + compactionCompleted *prometheus.CounterVec + compactionTime *prometheus.HistogramVec + + tenantsDiscovered prometheus.Counter + tenantsOwned prometheus.Counter + tenantsSkipped prometheus.Counter + tenantsStarted prometheus.Counter + tenantsCompleted *prometheus.CounterVec + tenantsCompletedTime *prometheus.HistogramVec + tenantsSeries prometheus.Histogram } func NewMetrics(r prometheus.Registerer, bloomMetrics *v1.Metrics) *Metrics { m := Metrics{ bloomMetrics: bloomMetrics, + compactorRunning: promauto.With(r).NewGauge(prometheus.GaugeOpts{ + Namespace: metricsNamespace, + Subsystem: metricsSubsystem, + Name: "running", + Help: "Value will be 1 if compactor is currently running on this instance", + }), chunkSize: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ - Name: "bloom_chunk_series_size", - Help: "Uncompressed size of chunks in a series", - Buckets: prometheus.ExponentialBucketsRange(1024, 1073741824, 10), + Namespace: metricsNamespace, + Subsystem: metricsSubsystem, + Name: "chunk_series_size", + Help: "Uncompressed size of chunks in a series", + Buckets: prometheus.ExponentialBucketsRange(1024, 1073741824, 10), }), - compactionRunsStarted: promauto.With(r).NewCounter(prometheus.CounterOpts{ + + compactionsStarted: promauto.With(r).NewCounter(prometheus.CounterOpts{ Namespace: metricsNamespace, Subsystem: metricsSubsystem, - Name: "runs_started_total", + Name: "compactions_started", Help: "Total number of compactions started", }), - compactionRunsCompleted: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ + compactionCompleted: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ Namespace: metricsNamespace, Subsystem: metricsSubsystem, - Name: "runs_completed_total", - Help: "Total number of compactions completed successfully", + Name: "compactions_completed", + Help: "Total number of compactions completed", }, []string{"status"}), - compactionRunTime: promauto.With(r).NewHistogramVec(prometheus.HistogramOpts{ + compactionTime: promauto.With(r).NewHistogramVec(prometheus.HistogramOpts{ Namespace: metricsNamespace, Subsystem: metricsSubsystem, - Name: "runs_time_seconds", + Name: "compactions_time_seconds", Help: "Time spent during a compaction cycle.", Buckets: prometheus.DefBuckets, }, []string{"status"}), - compactionRunDiscoveredTenants: promauto.With(r).NewCounter(prometheus.CounterOpts{ + + tenantsDiscovered: promauto.With(r).NewCounter(prometheus.CounterOpts{ Namespace: metricsNamespace, Subsystem: metricsSubsystem, Name: "tenants_discovered", Help: "Number of tenants discovered during the current compaction run", }), - compactionRunSkippedTenants: promauto.With(r).NewCounter(prometheus.CounterOpts{ + tenantsOwned: promauto.With(r).NewCounter(prometheus.CounterOpts{ Namespace: metricsNamespace, Subsystem: metricsSubsystem, - Name: "tenants_skipped", - Help: "Number of tenants skipped during the current compaction run", + Name: "tenants_owned", + Help: "Number of tenants owned by this instance", }), - compactionRunTenantsCompleted: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ - Namespace: metricsNamespace, - Subsystem: metricsSubsystem, - Name: "tenants_completed", - Help: "Number of tenants successfully processed during the current compaction run", - }, []string{"status"}), - compactionRunTenantsTime: promauto.With(r).NewHistogramVec(prometheus.HistogramOpts{ + tenantsSkipped: promauto.With(r).NewCounter(prometheus.CounterOpts{ Namespace: metricsNamespace, Subsystem: metricsSubsystem, - Name: "tenants_time_seconds", - Help: "Time spent processing tenants.", - Buckets: prometheus.DefBuckets, - }, []string{"status"}), - compactionRunJobStarted: promauto.With(r).NewCounter(prometheus.CounterOpts{ + Name: "tenants_skipped", + Help: "Number of tenants skipped since they are not owned by this instance", + }), + tenantsStarted: promauto.With(r).NewCounter(prometheus.CounterOpts{ Namespace: metricsNamespace, Subsystem: metricsSubsystem, - Name: "job_started", - Help: "Number of jobs started processing during the current compaction run", + Name: "tenants_started", + Help: "Number of tenants started to process during the current compaction run", }), - compactionRunJobCompleted: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ + tenantsCompleted: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ Namespace: metricsNamespace, Subsystem: metricsSubsystem, - Name: "job_completed", - Help: "Number of jobs successfully processed during the current compaction run", + Name: "tenants_completed", + Help: "Number of tenants successfully processed during the current compaction run", }, []string{"status"}), - compactionRunJobTime: promauto.With(r).NewHistogramVec(prometheus.HistogramOpts{ + tenantsCompletedTime: promauto.With(r).NewHistogramVec(prometheus.HistogramOpts{ Namespace: metricsNamespace, Subsystem: metricsSubsystem, - Name: "job_time_seconds", - Help: "Time spent processing jobs.", + Name: "tenants_time_seconds", + Help: "Time spent processing tenants.", Buckets: prometheus.DefBuckets, }, []string{"status"}), - compactionRunInterval: promauto.With(r).NewGauge(prometheus.GaugeOpts{ - Namespace: metricsNamespace, - Subsystem: metricsSubsystem, - Name: "compaction_interval_seconds", - Help: "The configured interval on which compaction is run in seconds", - }), - compactorRunning: promauto.With(r).NewGauge(prometheus.GaugeOpts{ + tenantsSeries: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ Namespace: metricsNamespace, Subsystem: metricsSubsystem, - Name: "running", - Help: "Value will be 1 if compactor is currently running on this instance", + Name: "tenants_series", + Help: "Number of series processed per tenant in the owned fingerprint-range.", + // Up to 10M series per tenant, way more than what we expect given our max_global_streams_per_user limits + Buckets: prometheus.ExponentialBucketsRange(1, 10000000, 10), }), } diff --git a/pkg/storage/bloom/v1/util.go b/pkg/storage/bloom/v1/util.go index 3b9e0631b715d..67c0087a0d4b1 100644 --- a/pkg/storage/bloom/v1/util.go +++ b/pkg/storage/bloom/v1/util.go @@ -276,3 +276,29 @@ func NewPeekCloseIter[T any](itr CloseableIterator[T]) *PeekCloseIter[T] { func (it *PeekCloseIter[T]) Close() error { return it.close() } + +type CounterIterator[T any] interface { + Iterator[T] + Count() int +} + +type CounterIter[T any] struct { + Iterator[T] // the underlying iterator + count int +} + +func NewCounterIter[T any](itr Iterator[T]) *CounterIter[T] { + return &CounterIter[T]{Iterator: itr} +} + +func (it *CounterIter[T]) Next() bool { + if it.Iterator.Next() { + it.count++ + return true + } + return false +} + +func (it *CounterIter[T]) Count() int { + return it.count +} diff --git a/pkg/storage/bloom/v1/util_test.go b/pkg/storage/bloom/v1/util_test.go index ad89a226ec7f7..afafa4d05a870 100644 --- a/pkg/storage/bloom/v1/util_test.go +++ b/pkg/storage/bloom/v1/util_test.go @@ -26,3 +26,29 @@ func TestPeekingIterator(t *testing.T) { require.False(t, itr.Next()) } + +func TestCounterIter(t *testing.T) { + t.Parallel() + + data := []int{1, 2, 3, 4, 5} + itr := NewCounterIter[int](NewSliceIter[int](data)) + peekItr := NewPeekingIter[int](itr) + + // Consume the outer iter and use peek + for { + if _, ok := peekItr.Peek(); !ok { + break + } + if !peekItr.Next() { + break + } + } + // Both iterators should be exhausted + require.False(t, itr.Next()) + require.Nil(t, itr.Err()) + require.False(t, peekItr.Next()) + require.Nil(t, peekItr.Err()) + + // Assert that the count is correct and peeking hasn't jeopardized the count + require.Equal(t, len(data), itr.Count()) +}