From d9e8b48b3792bb23d779a710077df633f7f70c2d Mon Sep 17 00:00:00 2001 From: Trevor Whitney Date: Fri, 24 May 2024 10:55:58 -0600 Subject: [PATCH] Squashed commit of the following: commit 0bfd0ad7844c2253a4802de6ca5a741f73f963e9 Merge: 68aa1886f efdae3df1 Author: Trevor Whitney Date: Thu May 23 17:04:32 2024 -0600 Merge branch 'main' into sample-count-and-bytes commit 68aa1886fc9cc7903e70234372c3968dc849ae6b Author: Trevor Whitney Date: Thu May 23 17:03:32 2024 -0600 feat: guard aggregation behavior behind a feature flag commit efdae3df14c47d627eb99e91466e0451db6e16f6 Author: hayden Date: Thu May 23 16:25:50 2024 -0400 feat(helm): Support for PVC Annotations for Non-Distributed Modes (#12023) Signed-off-by: hfuss Co-authored-by: J Stickler Co-authored-by: Trevor Whitney commit f0d6a92cb6dc00b9a61bf604f392db587222ece4 Author: Trevor Whitney Date: Thu May 23 14:03:32 2024 -0600 feat: reject filter queries to /patterns endpoint commit dc620e7882ccb09cacfd1e231b92d2e4fe19470b Author: Trevor Whitney Date: Wed May 8 14:08:44 2024 -0600 feat: collect and serve pre-agg bytes and count * pre-aggregate bytes and count per stream in the pattern ingester * serve bytes_over_time and count_over_time queries from the patterns endpoint commit 97212eadf15c2b5ee2cd59b7c1df71f6177cfe7e Author: Jay Clifford <45856600+Jayclifford345@users.noreply.github.com> Date: Thu May 23 12:10:48 2024 -0400 feat: Added Interactive Sandbox to Quickstart tutorial (#12701) commit 1111595179c77f9303ebdfd362f14b1ac50044cb Author: Vladyslav Diachenko <82767850+vlad-diachenko@users.noreply.github.com> Date: Thu May 23 13:18:16 2024 +0300 feat: new stream count limiter (#13006) Signed-off-by: Vladyslav Diachenko Co-authored-by: JordanRushing commit 987e551f9e21b9a612dd0b6a3e60503ce6fe13a8 Author: Quentin Bisson Date: Thu May 23 02:15:52 2024 +0200 fix: allow cluster label override in bloom dashboards (#13012) Signed-off-by: QuentinBisson commit d3c9cec22891b45ed1cb93a9eacc5dad6a117fc5 Author: Quentin Bisson Date: Thu May 23 01:59:28 2024 +0200 fix: upgrade old plugin for the loki-operational dashboard. (#13016) Signed-off-by: QuentinBisson commit 8d9fb68ae5d4f26ddc2ae184a1cb6a3b2a2c2127 Author: Quentin Bisson Date: Wed May 22 22:00:08 2024 +0200 fix: remove unneccessary disk panels for ssd read path (#13014) Signed-off-by: QuentinBisson commit 1948899999107e7f27f4b9faace64942abcdb41f Author: Quentin Bisson Date: Wed May 22 15:16:29 2024 +0200 fix: Mixins - Add missing log datasource on loki-deletion (#13011) commit efd8f5dc1b3bb3313de1ed6b26750d5bd5632b16 Author: Salva Corts Date: Wed May 22 10:43:32 2024 +0200 refactor(blooms): Add queue to bloom planner and enqueue tasks (#13005) commit d6f29fc789760318b048d005e14c91eba748b45e Author: Vitor Gomes <41302394+vitoorgomes@users.noreply.github.com> Date: Wed May 22 04:34:42 2024 +1200 docs: update otlp ingestion with correct endpoint and add endpoint to reference api docs (#12996) commit 319503643589163edce0b939de0beac074006a9f Author: Salva Corts Date: Tue May 21 13:12:24 2024 +0200 refactor(bloom planner): Compute gaps and build tasks from metas and TSDBs (#12994) commit 7a3338ead82e4c577652ab86e9a55faf200ac05a Author: Jonathan Davies Date: Tue May 21 10:41:42 2024 +0100 feat: loki/main.go: Log which config file path is used on startup (#12985) Co-authored-by: Michel Hollands <42814411+MichelHollands@users.noreply.github.com> commit bf8a27887979b2337263c55bd23aead9eed6ea0f Author: Ashwanth Date: Tue May 21 12:56:07 2024 +0530 chore: remove duplicate imports (#13001) commit 1f5291a4a3bd3c98c190d9a5dda32bbd78f18c3b Author: Ashwanth Date: Tue May 21 12:38:02 2024 +0530 fix(indexstats): do not collect stats from "IndexStats" lookups for other query types (#12978) commit 8442dca9d2341471996a73a011f206630c67e857 Author: Jay Clifford <45856600+Jayclifford345@users.noreply.github.com> Date: Mon May 20 17:52:17 2024 -0400 feat: Added getting started video (#12975) commit 75ccf2160bfe647b1cb3daffb98869e9c1c44130 Author: Christian Haudum Date: Mon May 20 17:14:40 2024 +0200 feat(blooms): Separate page buffer pools for series pages and bloom pages (#12992) Series pages are much smaller than bloom pages and therefore can make use of a separate buffer pool with different buckets. The second commit fixes a possible panic. Signed-off-by: Christian Haudum commit 94d610e5e0220da1c0bb65bdc9b46ea793dc7387 Author: Yarden Shoham Date: Mon May 20 18:05:50 2024 +0300 docs: Fix broken link in the release notes (#12990) Co-authored-by: J Stickler commit 31a13146ed5f631374b7d71d22b219286e3144db Author: choeffer Date: Mon May 20 16:39:25 2024 +0200 docs(install-monolithic): add quotation marks (#12982) Co-authored-by: Michel Hollands <42814411+MichelHollands@users.noreply.github.com> commit 8978ecf0c85dfbe18b52632112e5be20eff411cf Author: Salva Corts Date: Mon May 20 12:36:22 2024 +0200 feat: Boilerplate for new bloom build planner and worker components. (#12989) --- cmd/loki/loki-local-config.yaml | 3 + cmd/loki/main.go | 1 + docs/Makefile | 1 + docs/sources/get-started/_index.md | 2 + docs/sources/get-started/quick-start.md | 12 + docs/sources/reference/loki-http-api.md | 11 + docs/sources/release-notes/_index.md | 2 +- docs/sources/send-data/otel/_index.md | 4 +- .../install/helm/install-monolithic/_index.md | 4 +- docs/sources/setup/install/helm/reference.md | 39 ++ docs/sources/shared/configuration.md | 51 ++ pkg/bloombuild/builder/builder.go | 50 ++ pkg/bloombuild/builder/config.go | 21 + pkg/bloombuild/builder/metrics.go | 26 + pkg/bloombuild/config.go | 40 ++ pkg/bloombuild/planner/config.go | 66 +++ pkg/bloombuild/planner/metrics.go | 100 ++++ pkg/bloombuild/planner/planner.go | 484 ++++++++++++++++++ pkg/bloombuild/planner/planner_test.go | 321 ++++++++++++ pkg/bloombuild/planner/tableIterator.go | 50 ++ pkg/bloombuild/planner/task.go | 29 ++ pkg/bloombuild/planner/tsdb.go | 261 ++++++++++ pkg/bloombuild/planner/tsdb_test.go | 105 ++++ pkg/bloombuild/planner/util.go | 125 +++++ pkg/bloombuild/planner/util_test.go | 172 +++++++ pkg/ingester/instance.go | 75 +-- pkg/ingester/limiter.go | 81 ++- pkg/ingester/limiter_test.go | 47 +- pkg/ingester/owned_streams.go | 44 ++ pkg/ingester/owned_streams_test.go | 36 ++ pkg/logproto/extensions.go | 18 +- pkg/logproto/extensions_test.go | 69 +++ pkg/logproto/pattern.pb.go | 316 ++++++++++-- pkg/logproto/pattern.proto | 5 +- pkg/logql/evaluator.go | 24 +- pkg/logql/range_vector.go | 45 +- pkg/loki/loki.go | 9 +- pkg/loki/modules.go | 39 +- pkg/pattern/chunk/util.go | 14 + pkg/pattern/drain/chunk.go | 32 +- pkg/pattern/drain/chunk_test.go | 32 +- pkg/pattern/flush_test.go | 4 +- pkg/pattern/ingester.go | 80 ++- pkg/pattern/ingester_querier.go | 95 +++- pkg/pattern/ingester_querier_test.go | 173 ++++++- pkg/pattern/ingester_test.go | 266 +++++++++- pkg/pattern/instance.go | 98 +++- pkg/pattern/instance_test.go | 116 +++++ pkg/pattern/iter/batch.go | 46 +- pkg/pattern/iter/batch_test.go | 20 +- pkg/pattern/iter/iterator.go | 176 ++++++- pkg/pattern/iter/iterator_test.go | 135 +++-- pkg/pattern/iter/merge.go | 14 +- pkg/pattern/iter/merge_test.go | 188 ++++--- pkg/pattern/iter/query_client.go | 18 +- pkg/pattern/metric/chunk.go | 201 ++++++++ pkg/pattern/metric/chunk_test.go | 329 ++++++++++++ pkg/pattern/metric/config.go | 16 + pkg/pattern/metric/evaluator.go | 358 +++++++++++++ pkg/pattern/metric/evaluator_test.go | 363 +++++++++++++ pkg/pattern/ring_client.go | 86 +++- pkg/pattern/stream.go | 170 +++++- pkg/pattern/stream_test.go | 13 +- pkg/pattern/tee.go | 12 +- pkg/querier/querier.go | 4 + pkg/storage/bloom/v1/bloom.go | 14 +- pkg/storage/bloom/v1/index.go | 6 +- pkg/storage/bloom/v1/util.go | 17 +- pkg/util/limiter/combined_limits.go | 4 + pkg/util/marshal/marshal.go | 10 +- pkg/util/marshal/marshal_test.go | 82 ++- pkg/validation/limits.go | 26 + production/helm/loki/CHANGELOG.md | 4 + production/helm/loki/Chart.yaml | 2 +- production/helm/loki/README.md | 2 +- .../backend/statefulset-backend.yaml | 4 + .../loki/templates/read/statefulset-read.yaml | 4 + .../templates/single-binary/statefulset.yaml | 4 + .../templates/write/statefulset-write.yaml | 4 + production/helm/loki/values.yaml | 9 + .../dashboards/loki-bloom-compactor.json | 74 +++ .../dashboards/loki-bloom-gateway.json | 57 +++ .../dashboards/loki-deletion.json | 10 + .../dashboards/loki-operational.json | 176 ++++--- .../dashboards/loki-reads-resources.json | 147 +----- .../dashboards/loki-bloom-compactor.json | 74 +++ .../dashboards/loki-bloom-gateway.json | 57 +++ .../dashboards/loki-deletion.json | 10 + .../dashboards/loki-operational.json | 176 ++++--- .../dashboard-loki-operational.json | 178 ++++--- .../dashboards/loki-bloom-compactor.libsonnet | 65 ++- .../dashboards/loki-bloom-gateway.libsonnet | 65 ++- .../dashboards/loki-deletion.libsonnet | 1 + .../dashboards/loki-reads-resources.libsonnet | 20 +- 94 files changed, 6356 insertions(+), 793 deletions(-) create mode 100644 pkg/bloombuild/builder/builder.go create mode 100644 pkg/bloombuild/builder/config.go create mode 100644 pkg/bloombuild/builder/metrics.go create mode 100644 pkg/bloombuild/config.go create mode 100644 pkg/bloombuild/planner/config.go create mode 100644 pkg/bloombuild/planner/metrics.go create mode 100644 pkg/bloombuild/planner/planner.go create mode 100644 pkg/bloombuild/planner/planner_test.go create mode 100644 pkg/bloombuild/planner/tableIterator.go create mode 100644 pkg/bloombuild/planner/task.go create mode 100644 pkg/bloombuild/planner/tsdb.go create mode 100644 pkg/bloombuild/planner/tsdb_test.go create mode 100644 pkg/bloombuild/planner/util.go create mode 100644 pkg/bloombuild/planner/util_test.go create mode 100644 pkg/ingester/owned_streams.go create mode 100644 pkg/ingester/owned_streams_test.go create mode 100644 pkg/pattern/chunk/util.go create mode 100644 pkg/pattern/instance_test.go create mode 100644 pkg/pattern/metric/chunk.go create mode 100644 pkg/pattern/metric/chunk_test.go create mode 100644 pkg/pattern/metric/config.go create mode 100644 pkg/pattern/metric/evaluator.go create mode 100644 pkg/pattern/metric/evaluator_test.go diff --git a/cmd/loki/loki-local-config.yaml b/cmd/loki/loki-local-config.yaml index 03b579647753a..5f717a3d6a81c 100644 --- a/cmd/loki/loki-local-config.yaml +++ b/cmd/loki/loki-local-config.yaml @@ -33,6 +33,9 @@ schema_config: prefix: index_ period: 24h +pattern_ingester: + enabled: true + ruler: alertmanager_url: http://localhost:9093 diff --git a/cmd/loki/main.go b/cmd/loki/main.go index d9f4613977872..401085b3aab11 100644 --- a/cmd/loki/main.go +++ b/cmd/loki/main.go @@ -118,6 +118,7 @@ func main() { } level.Info(util_log.Logger).Log("msg", "Starting Loki", "version", version.Info()) + level.Info(util_log.Logger).Log("msg", "Loading configuration file", "filename", config.ConfigFile) err = t.Run(loki.RunOpts{StartTime: startTime}) util_log.CheckFatal("running loki", err, util_log.Logger) diff --git a/docs/Makefile b/docs/Makefile index 63fc849789c11..4bed302d71794 100644 --- a/docs/Makefile +++ b/docs/Makefile @@ -10,6 +10,7 @@ include docs.mk PODMAN := $(shell if command -v podman >/dev/null 2>&1; then echo podman; else echo docker; fi) BUILD_IN_CONTAINER ?= true +.PHONY: sources/setup/install/helm/reference.md sources/setup/install/helm/reference.md: ../production/helm/loki/reference.md.gotmpl ../production/helm/loki/values.yaml ifeq ($(BUILD_IN_CONTAINER),true) $(PODMAN) run --rm --volume "$(realpath ..):/helm-docs" -u "$$(id -u)" "docker.io/jnorwood/helm-docs:v1.11.0" \ diff --git a/docs/sources/get-started/_index.md b/docs/sources/get-started/_index.md index f82d5f9f089c2..c85f383345fbb 100644 --- a/docs/sources/get-started/_index.md +++ b/docs/sources/get-started/_index.md @@ -7,6 +7,8 @@ description: Provides an overview of the steps for implementing Grafana Loki to # Get started with Grafana Loki +{{< youtube id="1uk8LtQqsZQ" >}} + Loki is a horizontally-scalable, highly-available, multi-tenant log aggregation system inspired by Prometheus. It is designed to be very cost effective and easy to operate. It does not index the contents of the logs, but rather a set of labels for each log stream. Because all Loki implementations are unique, the installation process is diff --git a/docs/sources/get-started/quick-start.md b/docs/sources/get-started/quick-start.md index b08f07a8e7973..f459e564092e1 100644 --- a/docs/sources/get-started/quick-start.md +++ b/docs/sources/get-started/quick-start.md @@ -22,6 +22,18 @@ The Docker Compose configuration instantiates the following components, each in {{< figure max-width="75%" src="/media/docs/loki/get-started-flog-v3.png" caption="Getting started sample application" alt="Getting started sample application">}} +## Interactive Learning Environment + +{{< admonition type="note" >}} +The Interactive Learning Environment is currently in trial. Please provide feedback, report bugs, and raise issues in the [Grafana Killercoda Repository](https://github.com/grafana/killercoda). +{{< /admonition >}} + +Try out this demo within our interactive learning environment: [Loki Quickstart Sandbox](https://killercoda.com/grafana-labs/course/loki/loki-quickstart) + +- A free Killercoda account is required to verify you are not a bot. +- Tutorial instructions are located on the left-hand side of the screen. Click to move on to the next section. +- All commands run inside the interactive terminal. Grafana can also be accessed via the URL links provided within the sandbox. + ## Installing Loki and collecting sample logs Prerequisites diff --git a/docs/sources/reference/loki-http-api.md b/docs/sources/reference/loki-http-api.md index 1f85a4b48bbca..e72a03ba51b0a 100644 --- a/docs/sources/reference/loki-http-api.md +++ b/docs/sources/reference/loki-http-api.md @@ -24,6 +24,7 @@ Authorization needs to be done separately, for example, using an open-source loa These endpoints are exposed by the `distributor`, `write`, and `all` components: - [`POST /loki/api/v1/push`](#ingest-logs) +- [`POST /otlp/v1/logs`](#ingest-logs-using-otlp) A [list of clients]({{< relref "../send-data" >}}) can be found in the clients documentation. @@ -260,6 +261,16 @@ curl -H "Content-Type: application/json" \ --data-raw '{"streams": [{ "stream": { "foo": "bar2" }, "values": [ [ "1570818238000000000", "fizzbuzz" ] ] }]}' ``` +## Ingest logs using OTLP + +```bash +POST /otlp/v1/logs +``` + +`/otlp/v1/logs` lets the OpenTelemetry Collector send logs to Loki using `otlphttp` procotol. + +For information on how to configure Loki, refer to the [OTel Collector topic](https://grafana.com/docs/loki//send-data/otel/). + ## Query logs at a single point in time ```bash diff --git a/docs/sources/release-notes/_index.md b/docs/sources/release-notes/_index.md index db74b50c16d76..ebd9afda5afbd 100644 --- a/docs/sources/release-notes/_index.md +++ b/docs/sources/release-notes/_index.md @@ -8,7 +8,7 @@ weight: 100 Release notes for Loki are in the CHANGELOG for the release and listed here by version number. -- [V3.0 release notes](https://grafana.com/docs/loki//release-notes/v3.0/) +- [V3.0 release notes](https://grafana.com/docs/loki//release-notes/v3-0/) - [V2.9 release notes](https://grafana.com/docs/loki//release-notes/v2-9/) - [V2.8 release notes](https://grafana.com/docs/loki//release-notes/v2-8/) - [V2.7 release notes](https://grafana.com/docs/loki//release-notes/v2-7/) diff --git a/docs/sources/send-data/otel/_index.md b/docs/sources/send-data/otel/_index.md index 27d092a81c095..b7a67fcb14d06 100644 --- a/docs/sources/send-data/otel/_index.md +++ b/docs/sources/send-data/otel/_index.md @@ -30,7 +30,7 @@ You need to make the following changes to the [OpenTelemetry Collector config](h ```yaml exporters: otlphttp: - endpoint: http://:3100/otlp + endpoint: http://:3100/otlp/v1/logs ``` And enable it in `service.pipelines`: @@ -57,7 +57,7 @@ exporters: otlphttp: auth: authenticator: basicauth/otlp - endpoint: http://:3100/otlp + endpoint: http://:3100/otlp/v1/logs service: extensions: [basicauth/otlp] diff --git a/docs/sources/setup/install/helm/install-monolithic/_index.md b/docs/sources/setup/install/helm/install-monolithic/_index.md index 86344a9488d77..fd52c11c20034 100644 --- a/docs/sources/setup/install/helm/install-monolithic/_index.md +++ b/docs/sources/setup/install/helm/install-monolithic/_index.md @@ -47,7 +47,7 @@ If you set the `singleBinary.replicas` value to 2 or more, this chart configures type: 'filesystem' schemaConfig: configs: - - from: 2024-01-01 + - from: "2024-01-01" store: tsdb index: prefix: loki_index_ @@ -72,7 +72,7 @@ If you set the `singleBinary.replicas` value to 2 or more, this chart configures replication_factor: 3 schemaConfig: configs: - - from: 2024-01-01 + - from: "2024-01-01" store: tsdb index: prefix: loki_index_ diff --git a/docs/sources/setup/install/helm/reference.md b/docs/sources/setup/install/helm/reference.md index 53101a4832143..76b4936f20bfd 100644 --- a/docs/sources/setup/install/helm/reference.md +++ b/docs/sources/setup/install/helm/reference.md @@ -315,6 +315,7 @@ This is the generated reference for the Loki Helm Chart values. "initContainers": [], "nodeSelector": {}, "persistence": { + "annotations": {}, "dataVolumeParameters": { "emptyDir": {} }, @@ -512,6 +513,15 @@ null
 {}
 
+ + + + backend.persistence.annotations + object + Annotations for volume claim +
+{}
+
@@ -6226,6 +6236,7 @@ false "drivesPerNode": 2, "enabled": false, "persistence": { + "annotations": {}, "size": "5Gi" }, "replicas": 1, @@ -8442,6 +8453,7 @@ false "lifecycle": {}, "nodeSelector": {}, "persistence": { + "annotations": {}, "enableStatefulSetAutoDeletePVC": true, "selector": null, "size": "10Gi", @@ -8653,6 +8665,15 @@ false
 {}
 
+ + + + read.persistence.annotations + object + Annotations for volume claim +
+{}
+
@@ -9893,6 +9914,15 @@ null
 {}
 
+ + + + singleBinary.persistence.annotations + object + Annotations for volume claim +
+{}
+
@@ -10677,6 +10707,15 @@ null
 {}
 
+ + + + write.persistence.annotations + object + Annotations for volume claim +
+{}
+
diff --git a/docs/sources/shared/configuration.md b/docs/sources/shared/configuration.md index fe57f40daa581..c0e4bdeeca4d2 100644 --- a/docs/sources/shared/configuration.md +++ b/docs/sources/shared/configuration.md @@ -326,6 +326,37 @@ pattern_ingester: # merging them as bloom blocks. [bloom_compactor: ] +bloom_build: + # Flag to enable or disable the usage of the bloom-planner and bloom-builder + # components. + # CLI flag: -bloom-build.enabled + [enabled: | default = false] + + planner: + # Interval at which to re-run the bloom creation planning. + # CLI flag: -bloom-build.planner.interval + [planning_interval: | default = 8h] + + # Newest day-table offset (from today, inclusive) to build blooms for. + # Increase to lower cost by not re-writing data to object storage too + # frequently since recent data changes more often at the cost of not having + # blooms available as quickly. + # CLI flag: -bloom-build.planner.min-table-offset + [min_table_offset: | default = 1] + + # Oldest day-table offset (from today, inclusive) to compact. This can be + # used to lower cost by not trying to compact older data which doesn't + # change. This can be optimized by aligning it with the maximum + # `reject_old_samples_max_age` setting of any tenant. + # CLI flag: -bloom-build.planner.max-table-offset + [max_table_offset: | default = 2] + + # Maximum number of tasks to queue per tenant. + # CLI flag: -bloom-build.planner.max-tasks-per-tenant + [max_queued_tasks_per_tenant: | default = 30000] + + builder: + # Experimental: The bloom_gateway block configures the Loki bloom gateway # server, responsible for serving queries for filtering chunks based on filter # expressions. @@ -2947,6 +2978,11 @@ The `limits_config` block configures global and per-tenant limits in Loki. The v # CLI flag: -validation.discover-log-levels [discover_log_levels: | default = true] +# When true an ingester takes into account only the streams that it owns +# according to the ring while applying the stream limit. +# CLI flag: -ingester.use-owned-stream-count +[use_owned_stream_count: | default = false] + # Maximum number of active streams per user, per ingester. 0 to disable. # CLI flag: -ingester.max-streams-per-user [max_streams_per_user: | default = 0] @@ -3372,6 +3408,21 @@ shard_streams: # CLI flag: -bloom-compactor.max-bloom-size [bloom_compactor_max_bloom_size: | default = 128MB] +# Experimental. Whether to create blooms for the tenant. +# CLI flag: -bloom-build.enable +[bloom_creation_enabled: | default = false] + +# Experimental. Number of splits to create for the series keyspace when building +# blooms. The series keyspace is split into this many parts to parallelize bloom +# creation. +# CLI flag: -bloom-build.split-keyspace-by +[bloom_split_series_keyspace_by: | default = 256] + +# Experimental. Maximum number of builders to use when building blooms. 0 allows +# unlimited builders. +# CLI flag: -bloom-build.max-builders +[bloom_build_max_builders: | default = 0] + # Experimental. Length of the n-grams created when computing blooms from log # lines. # CLI flag: -bloom-compactor.ngram-length diff --git a/pkg/bloombuild/builder/builder.go b/pkg/bloombuild/builder/builder.go new file mode 100644 index 0000000000000..098e7d6d83f00 --- /dev/null +++ b/pkg/bloombuild/builder/builder.go @@ -0,0 +1,50 @@ +package builder + +import ( + "context" + + "github.com/go-kit/log" + "github.com/grafana/dskit/services" + "github.com/prometheus/client_golang/prometheus" + + utillog "github.com/grafana/loki/v3/pkg/util/log" +) + +type Worker struct { + services.Service + + cfg Config + metrics *Metrics + logger log.Logger +} + +func New( + cfg Config, + logger log.Logger, + r prometheus.Registerer, +) (*Worker, error) { + utillog.WarnExperimentalUse("Bloom Builder", logger) + + w := &Worker{ + cfg: cfg, + metrics: NewMetrics(r), + logger: logger, + } + + w.Service = services.NewBasicService(w.starting, w.running, w.stopping) + return w, nil +} + +func (w *Worker) starting(_ context.Context) (err error) { + w.metrics.running.Set(1) + return err +} + +func (w *Worker) stopping(_ error) error { + w.metrics.running.Set(0) + return nil +} + +func (w *Worker) running(_ context.Context) error { + return nil +} diff --git a/pkg/bloombuild/builder/config.go b/pkg/bloombuild/builder/config.go new file mode 100644 index 0000000000000..ac282ccf95ebb --- /dev/null +++ b/pkg/bloombuild/builder/config.go @@ -0,0 +1,21 @@ +package builder + +import "flag" + +// Config configures the bloom-builder component. +type Config struct { + // TODO: Add config +} + +// RegisterFlagsWithPrefix registers flags for the bloom-planner configuration. +func (cfg *Config) RegisterFlagsWithPrefix(_ string, _ *flag.FlagSet) { + // TODO: Register flags with flagsPrefix +} + +func (cfg *Config) Validate() error { + return nil +} + +type Limits interface { + // TODO: Add limits +} diff --git a/pkg/bloombuild/builder/metrics.go b/pkg/bloombuild/builder/metrics.go new file mode 100644 index 0000000000000..e8f46fa025080 --- /dev/null +++ b/pkg/bloombuild/builder/metrics.go @@ -0,0 +1,26 @@ +package builder + +import ( + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" +) + +const ( + metricsNamespace = "loki" + metricsSubsystem = "bloombuilder" +) + +type Metrics struct { + running prometheus.Gauge +} + +func NewMetrics(r prometheus.Registerer) *Metrics { + return &Metrics{ + running: promauto.With(r).NewGauge(prometheus.GaugeOpts{ + Namespace: metricsNamespace, + Subsystem: metricsSubsystem, + Name: "running", + Help: "Value will be 1 if the bloom builder is currently running on this instance", + }), + } +} diff --git a/pkg/bloombuild/config.go b/pkg/bloombuild/config.go new file mode 100644 index 0000000000000..c69c605607f5a --- /dev/null +++ b/pkg/bloombuild/config.go @@ -0,0 +1,40 @@ +package bloombuild + +import ( + "flag" + "fmt" + + "github.com/grafana/loki/v3/pkg/bloombuild/builder" + "github.com/grafana/loki/v3/pkg/bloombuild/planner" +) + +// Config configures the bloom-planner component. +type Config struct { + Enabled bool `yaml:"enabled"` + + Planner planner.Config `yaml:"planner"` + Builder builder.Config `yaml:"builder"` +} + +// RegisterFlags registers flags for the bloom building configuration. +func (cfg *Config) RegisterFlags(f *flag.FlagSet) { + f.BoolVar(&cfg.Enabled, "bloom-build.enabled", false, "Flag to enable or disable the usage of the bloom-planner and bloom-builder components.") + cfg.Planner.RegisterFlagsWithPrefix("bloom-build.planner", f) + cfg.Builder.RegisterFlagsWithPrefix("bloom-build.builder", f) +} + +func (cfg *Config) Validate() error { + if !cfg.Enabled { + return nil + } + + if err := cfg.Planner.Validate(); err != nil { + return fmt.Errorf("invalid bloom planner configuration: %w", err) + } + + if err := cfg.Builder.Validate(); err != nil { + return fmt.Errorf("invalid bloom builder configuration: %w", err) + } + + return nil +} diff --git a/pkg/bloombuild/planner/config.go b/pkg/bloombuild/planner/config.go new file mode 100644 index 0000000000000..aff25873b12f4 --- /dev/null +++ b/pkg/bloombuild/planner/config.go @@ -0,0 +1,66 @@ +package planner + +import ( + "flag" + "fmt" + "time" +) + +// Config configures the bloom-planner component. +type Config struct { + PlanningInterval time.Duration `yaml:"planning_interval"` + MinTableOffset int `yaml:"min_table_offset"` + MaxTableOffset int `yaml:"max_table_offset"` + MaxQueuedTasksPerTenant int `yaml:"max_queued_tasks_per_tenant"` +} + +// RegisterFlagsWithPrefix registers flags for the bloom-planner configuration. +func (cfg *Config) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) { + f.DurationVar(&cfg.PlanningInterval, prefix+".interval", 8*time.Hour, "Interval at which to re-run the bloom creation planning.") + f.IntVar(&cfg.MinTableOffset, prefix+".min-table-offset", 1, "Newest day-table offset (from today, inclusive) to build blooms for. Increase to lower cost by not re-writing data to object storage too frequently since recent data changes more often at the cost of not having blooms available as quickly.") + // TODO(owen-d): ideally we'd set this per tenant based on their `reject_old_samples_max_age` setting, + // but due to how we need to discover tenants, we can't do that yet. Tenant+Period discovery is done by + // iterating the table periods in object storage and looking for tenants within that period. + // In order to have this done dynamically, we'd need to account for tenant specific overrides, which are also + // dynamically reloaded. + // I'm doing it the simple way for now. + f.IntVar(&cfg.MaxTableOffset, prefix+".max-table-offset", 2, "Oldest day-table offset (from today, inclusive) to compact. This can be used to lower cost by not trying to compact older data which doesn't change. This can be optimized by aligning it with the maximum `reject_old_samples_max_age` setting of any tenant.") + f.IntVar(&cfg.MaxQueuedTasksPerTenant, prefix+".max-tasks-per-tenant", 30000, "Maximum number of tasks to queue per tenant.") +} + +func (cfg *Config) Validate() error { + if cfg.MinTableOffset > cfg.MaxTableOffset { + return fmt.Errorf("min-table-offset (%d) must be less than or equal to max-table-offset (%d)", cfg.MinTableOffset, cfg.MaxTableOffset) + } + + return nil +} + +type Limits interface { + BloomCreationEnabled(tenantID string) bool + BloomSplitSeriesKeyspaceBy(tenantID string) int + BloomBuildMaxBuilders(tenantID string) int +} + +type QueueLimits struct { + limits Limits +} + +func NewQueueLimits(limits Limits) *QueueLimits { + return &QueueLimits{limits: limits} +} + +// MaxConsumers is used to compute how many of the available builders are allowed to handle tasks for a given tenant. +// 0 is returned when neither limits are applied. 0 means all builders can be used. +func (c *QueueLimits) MaxConsumers(tenantID string, allConsumers int) int { + if c == nil || c.limits == nil { + return 0 + } + + maxBuilders := c.limits.BloomBuildMaxBuilders(tenantID) + if maxBuilders == 0 { + return 0 + } + + return min(allConsumers, maxBuilders) +} diff --git a/pkg/bloombuild/planner/metrics.go b/pkg/bloombuild/planner/metrics.go new file mode 100644 index 0000000000000..347af1926617b --- /dev/null +++ b/pkg/bloombuild/planner/metrics.go @@ -0,0 +1,100 @@ +package planner + +import ( + "time" + + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + + "github.com/grafana/loki/v3/pkg/queue" +) + +const ( + metricsNamespace = "loki" + metricsSubsystem = "bloomplanner" + + statusSuccess = "success" + statusFailure = "failure" +) + +type Metrics struct { + running prometheus.Gauge + + // Extra Queue metrics + connectedBuilders prometheus.GaugeFunc + queueDuration prometheus.Histogram + inflightRequests prometheus.Summary + + buildStarted prometheus.Counter + buildCompleted *prometheus.CounterVec + buildTime *prometheus.HistogramVec + + tenantsDiscovered prometheus.Counter +} + +func NewMetrics( + r prometheus.Registerer, + getConnectedBuilders func() float64, +) *Metrics { + return &Metrics{ + running: promauto.With(r).NewGauge(prometheus.GaugeOpts{ + Namespace: metricsNamespace, + Subsystem: metricsSubsystem, + Name: "running", + Help: "Value will be 1 if bloom planner is currently running on this instance", + }), + connectedBuilders: promauto.With(r).NewGaugeFunc(prometheus.GaugeOpts{ + Namespace: metricsNamespace, + Subsystem: metricsSubsystem, + Name: "connected_builders", + Help: "Number of builders currently connected to the planner.", + }, getConnectedBuilders), + queueDuration: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ + Namespace: metricsNamespace, + Subsystem: metricsSubsystem, + Name: "queue_duration_seconds", + Help: "Time spend by tasks in queue before getting picked up by a builder.", + Buckets: prometheus.DefBuckets, + }), + inflightRequests: promauto.With(r).NewSummary(prometheus.SummaryOpts{ + Namespace: metricsNamespace, + Subsystem: metricsSubsystem, + Name: "inflight_tasks", + Help: "Number of inflight tasks (either queued or processing) sampled at a regular interval. Quantile buckets keep track of inflight tasks over the last 60s.", + Objectives: map[float64]float64{0.5: 0.05, 0.75: 0.02, 0.8: 0.02, 0.9: 0.01, 0.95: 0.01, 0.99: 0.001}, + MaxAge: time.Minute, + AgeBuckets: 6, + }), + + buildStarted: promauto.With(r).NewCounter(prometheus.CounterOpts{ + Namespace: metricsNamespace, + Subsystem: metricsSubsystem, + Name: "build_started_total", + Help: "Total number of builds started", + }), + buildCompleted: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ + Namespace: metricsNamespace, + Subsystem: metricsSubsystem, + Name: "build_completed_total", + Help: "Total number of builds completed", + }, []string{"status"}), + buildTime: promauto.With(r).NewHistogramVec(prometheus.HistogramOpts{ + Namespace: metricsNamespace, + Subsystem: metricsSubsystem, + Name: "build_time_seconds", + Help: "Time spent during a builds cycle.", + Buckets: prometheus.DefBuckets, + }, []string{"status"}), + + tenantsDiscovered: promauto.With(r).NewCounter(prometheus.CounterOpts{ + Namespace: metricsNamespace, + Subsystem: metricsSubsystem, + Name: "tenants_discovered_total", + Help: "Number of tenants discovered during the current build iteration", + }), + } +} + +func NewQueueMetrics(r prometheus.Registerer) *queue.Metrics { + return queue.NewMetrics(r, metricsNamespace, metricsSubsystem) +} diff --git a/pkg/bloombuild/planner/planner.go b/pkg/bloombuild/planner/planner.go new file mode 100644 index 0000000000000..9a5b9f6dc238e --- /dev/null +++ b/pkg/bloombuild/planner/planner.go @@ -0,0 +1,484 @@ +package planner + +import ( + "context" + "fmt" + "sort" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/grafana/dskit/services" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/common/model" + + "github.com/grafana/loki/v3/pkg/queue" + "github.com/grafana/loki/v3/pkg/storage" + v1 "github.com/grafana/loki/v3/pkg/storage/bloom/v1" + "github.com/grafana/loki/v3/pkg/storage/config" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/bloomshipper" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb" + "github.com/grafana/loki/v3/pkg/util" + utillog "github.com/grafana/loki/v3/pkg/util/log" +) + +type Planner struct { + services.Service + // Subservices manager. + subservices *services.Manager + subservicesWatcher *services.FailureWatcher + + cfg Config + limits Limits + schemaCfg config.SchemaConfig + + tsdbStore TSDBStore + bloomStore bloomshipper.Store + + tasksQueue *queue.RequestQueue + activeUsers *util.ActiveUsersCleanupService + + metrics *Metrics + logger log.Logger +} + +func New( + cfg Config, + limits Limits, + schemaCfg config.SchemaConfig, + storeCfg storage.Config, + storageMetrics storage.ClientMetrics, + bloomStore bloomshipper.Store, + logger log.Logger, + r prometheus.Registerer, +) (*Planner, error) { + utillog.WarnExperimentalUse("Bloom Planner", logger) + + tsdbStore, err := NewTSDBStores(schemaCfg, storeCfg, storageMetrics, logger) + if err != nil { + return nil, fmt.Errorf("error creating TSDB store: %w", err) + } + + // Queue to manage tasks + queueMetrics := NewQueueMetrics(r) + tasksQueue := queue.NewRequestQueue(cfg.MaxQueuedTasksPerTenant, 0, NewQueueLimits(limits), queueMetrics) + + // Clean metrics for inactive users: do not have added tasks to the queue in the last 1 hour + activeUsers := util.NewActiveUsersCleanupService(5*time.Minute, 1*time.Hour, func(user string) { + queueMetrics.Cleanup(user) + }) + + p := &Planner{ + cfg: cfg, + limits: limits, + schemaCfg: schemaCfg, + tsdbStore: tsdbStore, + bloomStore: bloomStore, + tasksQueue: tasksQueue, + activeUsers: activeUsers, + metrics: NewMetrics(r, tasksQueue.GetConnectedConsumersMetric), + logger: logger, + } + + svcs := []services.Service{p.tasksQueue, p.activeUsers} + p.subservices, err = services.NewManager(svcs...) + if err != nil { + return nil, fmt.Errorf("error creating subservices manager: %w", err) + } + p.subservicesWatcher = services.NewFailureWatcher() + p.subservicesWatcher.WatchManager(p.subservices) + + p.Service = services.NewBasicService(p.starting, p.running, p.stopping) + return p, nil +} + +func (p *Planner) starting(_ context.Context) (err error) { + p.metrics.running.Set(1) + return err +} + +func (p *Planner) stopping(_ error) error { + p.metrics.running.Set(0) + return nil +} + +func (p *Planner) running(ctx context.Context) error { + // run once at beginning + if err := p.runOne(ctx); err != nil { + level.Error(p.logger).Log("msg", "bloom build iteration failed for the first time", "err", err) + } + + ticker := time.NewTicker(p.cfg.PlanningInterval) + defer ticker.Stop() + for { + select { + case <-ctx.Done(): + err := ctx.Err() + level.Debug(p.logger).Log("msg", "planner context done", "err", err) + return err + + case <-ticker.C: + level.Info(p.logger).Log("msg", "starting bloom build iteration") + if err := p.runOne(ctx); err != nil { + level.Error(p.logger).Log("msg", "bloom build iteration failed", "err", err) + } + } + } +} + +func (p *Planner) runOne(ctx context.Context) error { + var ( + start = time.Now() + status = statusFailure + ) + defer func() { + p.metrics.buildCompleted.WithLabelValues(status).Inc() + p.metrics.buildTime.WithLabelValues(status).Observe(time.Since(start).Seconds()) + }() + + p.metrics.buildStarted.Inc() + + tables := p.tables(time.Now()) + level.Debug(p.logger).Log("msg", "loaded tables", "tables", tables.TotalDays()) + + work, err := p.loadWork(ctx, tables) + if err != nil { + return fmt.Errorf("error loading work: %w", err) + } + + var totalTasks int + for _, w := range work { + logger := log.With(p.logger, "tenant", w.tenant, "table", w.table.Addr(), "ownership", w.ownershipRange.String()) + + gaps, err := p.findGapsForBounds(ctx, w.tenant, w.table, w.ownershipRange) + if err != nil { + level.Error(logger).Log("msg", "error finding gaps", "err", err) + continue + } + + now := time.Now() + for _, gap := range gaps { + totalTasks++ + task := Task{ + table: w.table.Addr(), + tenant: w.tenant, + OwnershipBounds: w.ownershipRange, + tsdb: gap.tsdb, + gaps: gap.gaps, + + queueTime: now, + ctx: ctx, + } + + p.activeUsers.UpdateUserTimestamp(task.tenant, now) + if err := p.tasksQueue.Enqueue(task.tenant, nil, task, nil); err != nil { + level.Error(logger).Log("msg", "error enqueuing task", "err", err) + continue + } + } + } + + level.Debug(p.logger).Log("msg", "planning completed", "tasks", totalTasks) + + status = statusSuccess + level.Info(p.logger).Log( + "msg", "bloom build iteration completed", + "duration", time.Since(start).Seconds(), + ) + return nil +} + +func (p *Planner) tables(ts time.Time) *dayRangeIterator { + // adjust the minimum by one to make it inclusive, which is more intuitive + // for a configuration variable + adjustedMin := p.cfg.MinTableOffset - 1 + minCompactionDelta := time.Duration(adjustedMin) * config.ObjectStorageIndexRequiredPeriod + maxCompactionDelta := time.Duration(p.cfg.MaxTableOffset) * config.ObjectStorageIndexRequiredPeriod + + from := ts.Add(-maxCompactionDelta).UnixNano() / int64(config.ObjectStorageIndexRequiredPeriod) * int64(config.ObjectStorageIndexRequiredPeriod) + through := ts.Add(-minCompactionDelta).UnixNano() / int64(config.ObjectStorageIndexRequiredPeriod) * int64(config.ObjectStorageIndexRequiredPeriod) + + fromDay := config.NewDayTime(model.TimeFromUnixNano(from)) + throughDay := config.NewDayTime(model.TimeFromUnixNano(through)) + level.Debug(p.logger).Log("msg", "loaded tables for compaction", "from", fromDay, "through", throughDay) + return newDayRangeIterator(fromDay, throughDay, p.schemaCfg) +} + +type tenantTableRange struct { + tenant string + table config.DayTable + ownershipRange v1.FingerprintBounds + + // TODO: Add tracking + //finished bool + //queueTime, startTime, endTime time.Time +} + +func (p *Planner) loadWork( + ctx context.Context, + tables *dayRangeIterator, +) ([]tenantTableRange, error) { + var work []tenantTableRange + + for tables.Next() && tables.Err() == nil && ctx.Err() == nil { + table := tables.At() + level.Debug(p.logger).Log("msg", "loading work for table", "table", table) + + tenants, err := p.tenants(ctx, table) + if err != nil { + return nil, fmt.Errorf("error loading tenants: %w", err) + } + level.Debug(p.logger).Log("msg", "loaded tenants", "table", table, "tenants", tenants.Len()) + + for tenants.Next() && tenants.Err() == nil && ctx.Err() == nil { + p.metrics.tenantsDiscovered.Inc() + tenant := tenants.At() + + if !p.limits.BloomCreationEnabled(tenant) { + continue + } + + splitFactor := p.limits.BloomSplitSeriesKeyspaceBy(tenant) + bounds := SplitFingerprintKeyspaceByFactor(splitFactor) + + for _, bounds := range bounds { + work = append(work, tenantTableRange{ + tenant: tenant, + table: table, + ownershipRange: bounds, + }) + } + + level.Debug(p.logger).Log("msg", "loading work for tenant", "table", table, "tenant", tenant, "splitFactor", splitFactor) + } + if err := tenants.Err(); err != nil { + level.Error(p.logger).Log("msg", "error iterating tenants", "err", err) + return nil, fmt.Errorf("error iterating tenants: %w", err) + } + + } + if err := tables.Err(); err != nil { + level.Error(p.logger).Log("msg", "error iterating tables", "err", err) + return nil, fmt.Errorf("error iterating tables: %w", err) + } + + return work, ctx.Err() +} + +func (p *Planner) tenants(ctx context.Context, table config.DayTable) (*v1.SliceIter[string], error) { + tenants, err := p.tsdbStore.UsersForPeriod(ctx, table) + if err != nil { + return nil, fmt.Errorf("error loading tenants for table (%s): %w", table, err) + } + + return v1.NewSliceIter(tenants), nil +} + +/* +Planning 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-to-date, determined by checking other meta.json files and comparing + the TSDBs they were generated from as well as their ownership ranges. +*/ +func (p *Planner) findGapsForBounds( + ctx context.Context, + tenant string, + table config.DayTable, + ownershipRange v1.FingerprintBounds, +) ([]blockPlan, error) { + logger := log.With(p.logger, "org_id", tenant, "table", table.Addr(), "ownership", ownershipRange.String()) + + // Fetch source metas to be used in both build and cleanup of out-of-date metas+blooms + metas, err := p.bloomStore.FetchMetas( + ctx, + bloomshipper.MetaSearchParams{ + TenantID: tenant, + Interval: bloomshipper.NewInterval(table.Bounds()), + Keyspace: ownershipRange, + }, + ) + if err != nil { + level.Error(logger).Log("msg", "failed to get metas", "err", err) + return nil, fmt.Errorf("failed to get metas: %w", err) + } + + level.Debug(logger).Log("msg", "found relevant metas", "metas", len(metas)) + + // Find gaps in the TSDBs for this tenant/table + gaps, err := p.findOutdatedGaps(ctx, tenant, table, ownershipRange, metas, logger) + if err != nil { + return nil, fmt.Errorf("failed to find outdated gaps: %w", err) + } + + return gaps, nil +} + +// blockPlan is a plan for all the work needed to build a meta.json +// It includes: +// - the tsdb (source of truth) which contains all the series+chunks +// we need to ensure are indexed in bloom blocks +// - a list of gaps that are out of date and need to be checked+built +// - within each gap, a list of block refs which overlap the gap are included +// so we can use them to accelerate bloom generation. They likely contain many +// of the same chunks we need to ensure are indexed, just from previous tsdb iterations. +// This is a performance optimization to avoid expensive re-reindexing +type blockPlan struct { + tsdb tsdb.SingleTenantTSDBIdentifier + gaps []GapWithBlocks +} + +func (p *Planner) findOutdatedGaps( + ctx context.Context, + tenant string, + table config.DayTable, + ownershipRange v1.FingerprintBounds, + metas []bloomshipper.Meta, + logger log.Logger, +) ([]blockPlan, error) { + // Resolve TSDBs + tsdbs, err := p.tsdbStore.ResolveTSDBs(ctx, table, tenant) + if err != nil { + level.Error(logger).Log("msg", "failed to resolve tsdbs", "err", err) + return nil, fmt.Errorf("failed to resolve tsdbs: %w", err) + } + + 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, tsdbs, metas) + if err != nil { + level.Error(logger).Log("msg", "failed to find gaps", "err", err) + return nil, fmt.Errorf("failed to find gaps: %w", err) + } + + if len(tsdbsWithGaps) == 0 { + level.Debug(logger).Log("msg", "blooms exist for all tsdbs") + return nil, nil + } + + work, err := blockPlansForGaps(tsdbsWithGaps, metas) + if err != nil { + level.Error(logger).Log("msg", "failed to create plan", "err", err) + return nil, fmt.Errorf("failed to create plan: %w", err) + } + + return work, nil +} + +// Used to signal the gaps that need to be populated for a tsdb +type tsdbGaps struct { + tsdb tsdb.SingleTenantTSDBIdentifier + gaps []v1.FingerprintBounds +} + +// gapsBetweenTSDBsAndMetas returns if the metas are up-to-date with the TSDBs. This is determined by asserting +// that for each TSDB, there are metas covering the entire ownership range which were generated from that specific TSDB. +func gapsBetweenTSDBsAndMetas( + ownershipRange v1.FingerprintBounds, + tsdbs []tsdb.SingleTenantTSDBIdentifier, + metas []bloomshipper.Meta, +) (res []tsdbGaps, err error) { + for _, db := range tsdbs { + id := db.Name() + + relevantMetas := make([]v1.FingerprintBounds, 0, len(metas)) + for _, meta := range metas { + for _, s := range meta.Sources { + if s.Name() == id { + relevantMetas = append(relevantMetas, meta.Bounds) + } + } + } + + gaps, err := FindGapsInFingerprintBounds(ownershipRange, relevantMetas) + if err != nil { + return nil, err + } + + if len(gaps) > 0 { + res = append(res, tsdbGaps{ + tsdb: db, + gaps: gaps, + }) + } + } + + return res, err +} + +// blockPlansForGaps groups tsdb gaps we wish to fill with overlapping but out of date blocks. +// This allows us to expedite bloom generation by using existing blocks to fill in the gaps +// since many will contain the same chunks. +func blockPlansForGaps(tsdbs []tsdbGaps, metas []bloomshipper.Meta) ([]blockPlan, error) { + plans := make([]blockPlan, 0, len(tsdbs)) + + for _, idx := range tsdbs { + plan := blockPlan{ + tsdb: idx.tsdb, + gaps: make([]GapWithBlocks, 0, len(idx.gaps)), + } + + for _, gap := range idx.gaps { + planGap := GapWithBlocks{ + bounds: gap, + } + + for _, meta := range metas { + + if meta.Bounds.Intersection(gap) == nil { + // this meta doesn't overlap the gap, skip + continue + } + + for _, block := range meta.Blocks { + if block.Bounds.Intersection(gap) == nil { + // this block doesn't overlap the gap, skip + continue + } + // this block overlaps the gap, add it to the plan + // for this gap + planGap.blocks = append(planGap.blocks, block) + } + } + + // ensure we sort blocks so deduping iterator works as expected + sort.Slice(planGap.blocks, func(i, j int) bool { + return planGap.blocks[i].Bounds.Less(planGap.blocks[j].Bounds) + }) + + peekingBlocks := v1.NewPeekingIter[bloomshipper.BlockRef]( + v1.NewSliceIter[bloomshipper.BlockRef]( + planGap.blocks, + ), + ) + // dedupe blocks which could be in multiple metas + itr := v1.NewDedupingIter[bloomshipper.BlockRef, bloomshipper.BlockRef]( + func(a, b bloomshipper.BlockRef) bool { + return a == b + }, + v1.Identity[bloomshipper.BlockRef], + func(a, _ bloomshipper.BlockRef) bloomshipper.BlockRef { + return a + }, + peekingBlocks, + ) + + deduped, err := v1.Collect[bloomshipper.BlockRef](itr) + if err != nil { + return nil, fmt.Errorf("failed to dedupe blocks: %w", err) + } + planGap.blocks = deduped + + plan.gaps = append(plan.gaps, planGap) + } + + plans = append(plans, plan) + } + + return plans, nil +} diff --git a/pkg/bloombuild/planner/planner_test.go b/pkg/bloombuild/planner/planner_test.go new file mode 100644 index 0000000000000..346bd145ab8dc --- /dev/null +++ b/pkg/bloombuild/planner/planner_test.go @@ -0,0 +1,321 @@ +package planner + +import ( + "testing" + "time" + + "github.com/prometheus/common/model" + "github.com/stretchr/testify/require" + + v1 "github.com/grafana/loki/v3/pkg/storage/bloom/v1" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/bloomshipper" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb" +) + +func tsdbID(n int) tsdb.SingleTenantTSDBIdentifier { + return tsdb.SingleTenantTSDBIdentifier{ + TS: time.Unix(int64(n), 0), + } +} + +func genMeta(min, max model.Fingerprint, sources []int, blocks []bloomshipper.BlockRef) bloomshipper.Meta { + m := bloomshipper.Meta{ + MetaRef: bloomshipper.MetaRef{ + Ref: bloomshipper.Ref{ + Bounds: v1.NewBounds(min, max), + }, + }, + Blocks: blocks, + } + for _, source := range sources { + m.Sources = append(m.Sources, tsdbID(source)) + } + return m +} + +func Test_gapsBetweenTSDBsAndMetas(t *testing.T) { + + for _, tc := range []struct { + desc string + err bool + exp []tsdbGaps + ownershipRange v1.FingerprintBounds + tsdbs []tsdb.SingleTenantTSDBIdentifier + metas []bloomshipper.Meta + }{ + { + desc: "non-overlapping tsdbs and metas", + err: true, + ownershipRange: v1.NewBounds(0, 10), + tsdbs: []tsdb.SingleTenantTSDBIdentifier{tsdbID(0)}, + metas: []bloomshipper.Meta{ + genMeta(11, 20, []int{0}, nil), + }, + }, + { + desc: "single tsdb", + ownershipRange: v1.NewBounds(0, 10), + tsdbs: []tsdb.SingleTenantTSDBIdentifier{tsdbID(0)}, + metas: []bloomshipper.Meta{ + genMeta(4, 8, []int{0}, nil), + }, + exp: []tsdbGaps{ + { + tsdb: tsdbID(0), + gaps: []v1.FingerprintBounds{ + v1.NewBounds(0, 3), + v1.NewBounds(9, 10), + }, + }, + }, + }, + { + desc: "multiple tsdbs with separate blocks", + ownershipRange: v1.NewBounds(0, 10), + tsdbs: []tsdb.SingleTenantTSDBIdentifier{tsdbID(0), tsdbID(1)}, + metas: []bloomshipper.Meta{ + genMeta(0, 5, []int{0}, nil), + genMeta(6, 10, []int{1}, nil), + }, + exp: []tsdbGaps{ + { + tsdb: tsdbID(0), + gaps: []v1.FingerprintBounds{ + v1.NewBounds(6, 10), + }, + }, + { + tsdb: tsdbID(1), + gaps: []v1.FingerprintBounds{ + v1.NewBounds(0, 5), + }, + }, + }, + }, + { + desc: "multiple tsdbs with the same blocks", + ownershipRange: v1.NewBounds(0, 10), + tsdbs: []tsdb.SingleTenantTSDBIdentifier{tsdbID(0), tsdbID(1)}, + metas: []bloomshipper.Meta{ + genMeta(0, 5, []int{0, 1}, nil), + genMeta(6, 8, []int{1}, nil), + }, + exp: []tsdbGaps{ + { + tsdb: tsdbID(0), + gaps: []v1.FingerprintBounds{ + v1.NewBounds(6, 10), + }, + }, + { + tsdb: tsdbID(1), + gaps: []v1.FingerprintBounds{ + v1.NewBounds(9, 10), + }, + }, + }, + }, + } { + t.Run(tc.desc, func(t *testing.T) { + gaps, err := gapsBetweenTSDBsAndMetas(tc.ownershipRange, tc.tsdbs, tc.metas) + if tc.err { + require.Error(t, err) + return + } + require.Equal(t, tc.exp, gaps) + }) + } +} + +func genBlockRef(min, max model.Fingerprint) bloomshipper.BlockRef { + bounds := v1.NewBounds(min, max) + return bloomshipper.BlockRef{ + Ref: bloomshipper.Ref{ + Bounds: bounds, + }, + } +} + +func Test_blockPlansForGaps(t *testing.T) { + for _, tc := range []struct { + desc string + ownershipRange v1.FingerprintBounds + tsdbs []tsdb.SingleTenantTSDBIdentifier + metas []bloomshipper.Meta + err bool + exp []blockPlan + }{ + { + desc: "single overlapping meta+no overlapping block", + ownershipRange: v1.NewBounds(0, 10), + tsdbs: []tsdb.SingleTenantTSDBIdentifier{tsdbID(0)}, + metas: []bloomshipper.Meta{ + genMeta(5, 20, []int{1}, []bloomshipper.BlockRef{genBlockRef(11, 20)}), + }, + exp: []blockPlan{ + { + tsdb: tsdbID(0), + gaps: []GapWithBlocks{ + { + bounds: v1.NewBounds(0, 10), + }, + }, + }, + }, + }, + { + desc: "single overlapping meta+one overlapping block", + ownershipRange: v1.NewBounds(0, 10), + tsdbs: []tsdb.SingleTenantTSDBIdentifier{tsdbID(0)}, + metas: []bloomshipper.Meta{ + genMeta(5, 20, []int{1}, []bloomshipper.BlockRef{genBlockRef(9, 20)}), + }, + exp: []blockPlan{ + { + tsdb: tsdbID(0), + gaps: []GapWithBlocks{ + { + bounds: v1.NewBounds(0, 10), + blocks: []bloomshipper.BlockRef{genBlockRef(9, 20)}, + }, + }, + }, + }, + }, + { + // the range which needs to be generated doesn't overlap with existing blocks + // from other tsdb versions since theres an up to date tsdb version block, + // but we can trim the range needing generation + desc: "trims up to date area", + ownershipRange: v1.NewBounds(0, 10), + tsdbs: []tsdb.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 + }, + exp: []blockPlan{ + { + tsdb: tsdbID(0), + gaps: []GapWithBlocks{ + { + bounds: v1.NewBounds(0, 8), + }, + }, + }, + }, + }, + { + desc: "uses old block for overlapping range", + ownershipRange: v1.NewBounds(0, 10), + tsdbs: []tsdb.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 + }, + exp: []blockPlan{ + { + tsdb: tsdbID(0), + gaps: []GapWithBlocks{ + { + bounds: v1.NewBounds(0, 8), + blocks: []bloomshipper.BlockRef{genBlockRef(5, 20)}, + }, + }, + }, + }, + }, + { + desc: "multi case", + ownershipRange: v1.NewBounds(0, 10), + tsdbs: []tsdb.SingleTenantTSDBIdentifier{tsdbID(0), tsdbID(1)}, // generate for both tsdbs + metas: []bloomshipper.Meta{ + genMeta(0, 2, []int{0}, []bloomshipper.BlockRef{ + genBlockRef(0, 1), + genBlockRef(1, 2), + }), // tsdb_0 + genMeta(6, 8, []int{0}, []bloomshipper.BlockRef{genBlockRef(6, 8)}), // tsdb_0 + + genMeta(3, 5, []int{1}, []bloomshipper.BlockRef{genBlockRef(3, 5)}), // tsdb_1 + genMeta(8, 10, []int{1}, []bloomshipper.BlockRef{genBlockRef(8, 10)}), // tsdb_1 + }, + exp: []blockPlan{ + { + tsdb: tsdbID(0), + gaps: []GapWithBlocks{ + // tsdb (id=0) can source chunks from the blocks built from tsdb (id=1) + { + bounds: v1.NewBounds(3, 5), + blocks: []bloomshipper.BlockRef{genBlockRef(3, 5)}, + }, + { + bounds: v1.NewBounds(9, 10), + blocks: []bloomshipper.BlockRef{genBlockRef(8, 10)}, + }, + }, + }, + // tsdb (id=1) can source chunks from the blocks built from tsdb (id=0) + { + tsdb: tsdbID(1), + gaps: []GapWithBlocks{ + { + bounds: v1.NewBounds(0, 2), + blocks: []bloomshipper.BlockRef{ + genBlockRef(0, 1), + genBlockRef(1, 2), + }, + }, + { + bounds: v1.NewBounds(6, 7), + blocks: []bloomshipper.BlockRef{genBlockRef(6, 8)}, + }, + }, + }, + }, + }, + { + desc: "dedupes block refs", + ownershipRange: v1.NewBounds(0, 10), + tsdbs: []tsdb.SingleTenantTSDBIdentifier{tsdbID(0)}, + metas: []bloomshipper.Meta{ + genMeta(9, 20, []int{1}, []bloomshipper.BlockRef{ + genBlockRef(1, 4), + genBlockRef(9, 20), + }), // blocks for first diff tsdb + genMeta(5, 20, []int{2}, []bloomshipper.BlockRef{ + genBlockRef(5, 10), + genBlockRef(9, 20), // same block references in prior meta (will be deduped) + }), // block for second diff tsdb + }, + exp: []blockPlan{ + { + tsdb: tsdbID(0), + gaps: []GapWithBlocks{ + { + bounds: v1.NewBounds(0, 10), + blocks: []bloomshipper.BlockRef{ + genBlockRef(1, 4), + genBlockRef(5, 10), + genBlockRef(9, 20), + }, + }, + }, + }, + }, + }, + } { + t.Run(tc.desc, func(t *testing.T) { + // we reuse the gapsBetweenTSDBsAndMetas function to generate the gaps as this function is tested + // separately and it's used to generate input in our regular code path (easier to write tests this way). + gaps, err := gapsBetweenTSDBsAndMetas(tc.ownershipRange, tc.tsdbs, tc.metas) + require.NoError(t, err) + + plans, err := blockPlansForGaps(gaps, tc.metas) + if tc.err { + require.Error(t, err) + return + } + require.Equal(t, tc.exp, plans) + + }) + } +} diff --git a/pkg/bloombuild/planner/tableIterator.go b/pkg/bloombuild/planner/tableIterator.go new file mode 100644 index 0000000000000..c17458a04806c --- /dev/null +++ b/pkg/bloombuild/planner/tableIterator.go @@ -0,0 +1,50 @@ +package planner + +import ( + "fmt" + + "github.com/grafana/loki/v3/pkg/storage/config" +) + +type dayRangeIterator struct { + min, max, cur config.DayTime + curPeriod config.PeriodConfig + schemaCfg config.SchemaConfig + err error +} + +func newDayRangeIterator(min, max config.DayTime, schemaCfg config.SchemaConfig) *dayRangeIterator { + return &dayRangeIterator{min: min, max: max, cur: min.Dec(), schemaCfg: schemaCfg} +} + +func (r *dayRangeIterator) TotalDays() int { + offset := r.cur + if r.cur.Before(r.min) { + offset = r.min + } + return int(r.max.Sub(offset.Time) / config.ObjectStorageIndexRequiredPeriod) +} + +func (r *dayRangeIterator) Next() bool { + r.cur = r.cur.Inc() + if !r.cur.Before(r.max) { + return false + } + + period, err := r.schemaCfg.SchemaForTime(r.cur.ModelTime()) + if err != nil { + r.err = fmt.Errorf("getting schema for time (%s): %w", r.cur, err) + return false + } + r.curPeriod = period + + return true +} + +func (r *dayRangeIterator) At() config.DayTable { + return config.NewDayTable(r.cur, r.curPeriod.IndexTables.Prefix) +} + +func (r *dayRangeIterator) Err() error { + return nil +} diff --git a/pkg/bloombuild/planner/task.go b/pkg/bloombuild/planner/task.go new file mode 100644 index 0000000000000..bff459fe17643 --- /dev/null +++ b/pkg/bloombuild/planner/task.go @@ -0,0 +1,29 @@ +package planner + +import ( + "context" + "time" + + v1 "github.com/grafana/loki/v3/pkg/storage/bloom/v1" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/bloomshipper" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb" +) + +// TODO: Extract this definiton to a proto file at pkg/bloombuild/protos/protos.proto + +type GapWithBlocks struct { + bounds v1.FingerprintBounds + blocks []bloomshipper.BlockRef +} + +type Task struct { + table string + tenant string + OwnershipBounds v1.FingerprintBounds + tsdb tsdb.SingleTenantTSDBIdentifier + gaps []GapWithBlocks + + // Tracking + queueTime time.Time + ctx context.Context +} diff --git a/pkg/bloombuild/planner/tsdb.go b/pkg/bloombuild/planner/tsdb.go new file mode 100644 index 0000000000000..7c15c43306db2 --- /dev/null +++ b/pkg/bloombuild/planner/tsdb.go @@ -0,0 +1,261 @@ +package planner + +import ( + "context" + "fmt" + "io" + "math" + "path" + "strings" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/pkg/errors" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/labels" + + "github.com/grafana/loki/v3/pkg/chunkenc" + baseStore "github.com/grafana/loki/v3/pkg/storage" + v1 "github.com/grafana/loki/v3/pkg/storage/bloom/v1" + "github.com/grafana/loki/v3/pkg/storage/config" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/storage" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/sharding" + "github.com/grafana/loki/v3/pkg/storage/types" +) + +const ( + gzipExtension = ".gz" +) + +type TSDBStore interface { + UsersForPeriod(ctx context.Context, table config.DayTable) ([]string, error) + ResolveTSDBs(ctx context.Context, table config.DayTable, tenant string) ([]tsdb.SingleTenantTSDBIdentifier, error) + LoadTSDB( + ctx context.Context, + table config.DayTable, + tenant string, + id tsdb.Identifier, + bounds v1.FingerprintBounds, + ) (v1.Iterator[*v1.Series], error) +} + +// BloomTSDBStore is a wrapper around the storage.Client interface which +// implements the TSDBStore interface for this pkg. +type BloomTSDBStore struct { + storage storage.Client + logger log.Logger +} + +func NewBloomTSDBStore(storage storage.Client, logger log.Logger) *BloomTSDBStore { + return &BloomTSDBStore{ + storage: storage, + logger: logger, + } +} + +func (b *BloomTSDBStore) UsersForPeriod(ctx context.Context, table config.DayTable) ([]string, error) { + _, 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.DayTable, tenant string) ([]tsdb.SingleTenantTSDBIdentifier, error) { + 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") + } + + ids := make([]tsdb.SingleTenantTSDBIdentifier, 0, len(indices)) + for _, index := range indices { + key := index.Name + if decompress := storage.IsCompressedFile(index.Name); decompress { + key = strings.TrimSuffix(key, gzipExtension) + } + + id, ok := tsdb.ParseSingleTenantTSDBPath(path.Base(key)) + if !ok { + return nil, errors.Errorf("failed to parse single tenant tsdb path: %s", key) + } + + ids = append(ids, id) + + } + return ids, nil +} + +func (b *BloomTSDBStore) LoadTSDB( + ctx context.Context, + table config.DayTable, + tenant string, + id tsdb.Identifier, + bounds v1.FingerprintBounds, +) (v1.Iterator[*v1.Series], error) { + withCompression := id.Name() + gzipExtension + + data, err := b.storage.GetUserFile(ctx, table.Addr(), tenant, withCompression) + if err != nil { + return nil, errors.Wrap(err, "failed to get file") + } + defer data.Close() + + decompressorPool := chunkenc.GetReaderPool(chunkenc.EncGZIP) + decompressor, err := decompressorPool.GetReader(data) + if err != nil { + return nil, errors.Wrap(err, "failed to get decompressor") + } + defer decompressorPool.PutReader(decompressor) + + buf, err := io.ReadAll(decompressor) + if err != nil { + return nil, errors.Wrap(err, "failed to read file") + } + + reader, err := index.NewReader(index.RealByteSlice(buf)) + if err != nil { + return nil, errors.Wrap(err, "failed to create index reader") + } + + idx := tsdb.NewTSDBIndex(reader) + defer func() { + if err := idx.Close(); err != nil { + level.Error(b.logger).Log("msg", "failed to close index", "err", err) + } + }() + + return NewTSDBSeriesIter(ctx, tenant, idx, bounds) +} + +func NewTSDBSeriesIter(ctx context.Context, user string, f sharding.ForSeries, bounds v1.FingerprintBounds) (v1.Iterator[*v1.Series], error) { + // TODO(salvacorts): Create a pool + series := make([]*v1.Series, 0, 100) + + if err := f.ForSeries( + ctx, + user, + bounds, + 0, math.MaxInt64, + func(_ labels.Labels, fp model.Fingerprint, chks []index.ChunkMeta) (stop bool) { + select { + case <-ctx.Done(): + return true + default: + res := &v1.Series{ + Fingerprint: fp, + Chunks: make(v1.ChunkRefs, 0, len(chks)), + } + for _, chk := range chks { + res.Chunks = append(res.Chunks, v1.ChunkRef{ + From: model.Time(chk.MinTime), + Through: model.Time(chk.MaxTime), + Checksum: chk.Checksum, + }) + } + + series = append(series, res) + return false + } + }, + labels.MustNewMatcher(labels.MatchEqual, "", ""), + ); err != nil { + return nil, err + } + + select { + case <-ctx.Done(): + return v1.NewEmptyIter[*v1.Series](), ctx.Err() + default: + return v1.NewCancelableIter[*v1.Series](ctx, v1.NewSliceIter[*v1.Series](series)), nil + } +} + +type TSDBStores struct { + schemaCfg config.SchemaConfig + stores []TSDBStore +} + +func NewTSDBStores( + schemaCfg config.SchemaConfig, + storeCfg baseStore.Config, + clientMetrics baseStore.ClientMetrics, + logger log.Logger, +) (*TSDBStores, error) { + res := &TSDBStores{ + schemaCfg: schemaCfg, + stores: make([]TSDBStore, len(schemaCfg.Configs)), + } + + for i, cfg := range schemaCfg.Configs { + if cfg.IndexType == types.TSDBType { + + c, err := baseStore.NewObjectClient(cfg.ObjectType, storeCfg, clientMetrics) + if err != nil { + return nil, errors.Wrap(err, "failed to create object client") + } + res.stores[i] = NewBloomTSDBStore(storage.NewIndexStorageClient(c, cfg.IndexTables.PathPrefix), logger) + } + } + + return res, nil +} + +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(period.From) { + // we have the desired period config + + if s.stores[i] != nil { + // valid: it's of tsdb type + return s.stores[i], nil + } + + // invalid + return nil, errors.Errorf( + "store for period is not of TSDB type (%s) while looking up store for (%v)", + period.IndexType, + table, + ) + } + + } + + return nil, fmt.Errorf( + "there is no store matching no matching period found for table (%v) -- too early", + table, + ) +} + +func (s *TSDBStores) UsersForPeriod(ctx context.Context, table config.DayTable) ([]string, error) { + store, err := s.storeForPeriod(table.DayTime) + if err != nil { + return nil, err + } + + return store.UsersForPeriod(ctx, table) +} + +func (s *TSDBStores) ResolveTSDBs(ctx context.Context, table config.DayTable, tenant string) ([]tsdb.SingleTenantTSDBIdentifier, error) { + store, err := s.storeForPeriod(table.DayTime) + if err != nil { + return nil, err + } + + return store.ResolveTSDBs(ctx, table, tenant) +} + +func (s *TSDBStores) LoadTSDB( + ctx context.Context, + table config.DayTable, + tenant string, + id tsdb.Identifier, + bounds v1.FingerprintBounds, +) (v1.Iterator[*v1.Series], error) { + store, err := s.storeForPeriod(table.DayTime) + if err != nil { + return nil, err + } + + return store.LoadTSDB(ctx, table, tenant, id, bounds) +} diff --git a/pkg/bloombuild/planner/tsdb_test.go b/pkg/bloombuild/planner/tsdb_test.go new file mode 100644 index 0000000000000..f47c193c2cd18 --- /dev/null +++ b/pkg/bloombuild/planner/tsdb_test.go @@ -0,0 +1,105 @@ +package planner + +import ( + "context" + "math" + "testing" + + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/labels" + "github.com/stretchr/testify/require" + + v1 "github.com/grafana/loki/v3/pkg/storage/bloom/v1" + "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb/index" +) + +type forSeriesTestImpl []*v1.Series + +func (f forSeriesTestImpl) ForSeries( + _ context.Context, + _ string, + _ index.FingerprintFilter, + _ model.Time, + _ model.Time, + fn func(labels.Labels, model.Fingerprint, []index.ChunkMeta) bool, + _ ...*labels.Matcher, +) error { + for i := range f { + unmapped := make([]index.ChunkMeta, 0, len(f[i].Chunks)) + for _, c := range f[i].Chunks { + unmapped = append(unmapped, index.ChunkMeta{ + MinTime: int64(c.From), + MaxTime: int64(c.Through), + Checksum: c.Checksum, + }) + } + + fn(nil, f[i].Fingerprint, unmapped) + } + return nil +} + +func (f forSeriesTestImpl) Close() error { + return nil +} + +func TestTSDBSeriesIter(t *testing.T) { + input := []*v1.Series{ + { + Fingerprint: 1, + Chunks: []v1.ChunkRef{ + { + From: 0, + Through: 1, + Checksum: 2, + }, + { + From: 3, + Through: 4, + Checksum: 5, + }, + }, + }, + } + srcItr := v1.NewSliceIter(input) + itr, err := NewTSDBSeriesIter(context.Background(), "", forSeriesTestImpl(input), v1.NewBounds(0, math.MaxUint64)) + require.NoError(t, err) + + v1.EqualIterators[*v1.Series]( + t, + func(a, b *v1.Series) { + require.Equal(t, a, b) + }, + itr, + srcItr, + ) +} + +func TestTSDBSeriesIter_Expiry(t *testing.T) { + t.Run("expires on creation", func(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + cancel() + itr, err := NewTSDBSeriesIter(ctx, "", forSeriesTestImpl{ + {}, // a single entry + }, v1.NewBounds(0, math.MaxUint64)) + require.Error(t, err) + require.False(t, itr.Next()) + }) + + t.Run("expires during consumption", func(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + itr, err := NewTSDBSeriesIter(ctx, "", forSeriesTestImpl{ + {}, + {}, + }, v1.NewBounds(0, math.MaxUint64)) + require.NoError(t, err) + + require.True(t, itr.Next()) + require.NoError(t, itr.Err()) + + cancel() + require.False(t, itr.Next()) + require.Error(t, itr.Err()) + }) + +} diff --git a/pkg/bloombuild/planner/util.go b/pkg/bloombuild/planner/util.go new file mode 100644 index 0000000000000..f9a97587f802f --- /dev/null +++ b/pkg/bloombuild/planner/util.go @@ -0,0 +1,125 @@ +package planner + +import ( + "fmt" + "math" + + "github.com/prometheus/common/model" + + v1 "github.com/grafana/loki/v3/pkg/storage/bloom/v1" +) + +// SplitFingerprintKeyspaceByFactor splits the keyspace covered by model.Fingerprint into contiguous non-overlapping ranges. +func SplitFingerprintKeyspaceByFactor(factor int) []v1.FingerprintBounds { + if factor <= 0 { + return nil + } + + bounds := make([]v1.FingerprintBounds, 0, factor) + + // The keyspace of a Fingerprint is from 0 to max uint64. + keyspaceSize := uint64(math.MaxUint64) + + // Calculate the size of each range. + rangeSize := keyspaceSize / uint64(factor) + + for i := 0; i < factor; i++ { + // Calculate the start and end of the range. + start := uint64(i) * rangeSize + end := start + rangeSize - 1 + + // For the last range, make sure it ends at the end of the keyspace. + if i == factor-1 { + end = keyspaceSize + } + + // Create a FingerprintBounds for the range and add it to the slice. + bounds = append(bounds, v1.FingerprintBounds{ + Min: model.Fingerprint(start), + Max: model.Fingerprint(end), + }) + } + + return bounds +} + +func FindGapsInFingerprintBounds(ownershipRange v1.FingerprintBounds, metas []v1.FingerprintBounds) (gaps []v1.FingerprintBounds, err error) { + if len(metas) == 0 { + return []v1.FingerprintBounds{ownershipRange}, nil + } + + // turn the available metas into a list of non-overlapping metas + // for easier processing + var nonOverlapping []v1.FingerprintBounds + // First, we reduce the metas into a smaller set by combining overlaps. They must be sorted. + var cur *v1.FingerprintBounds + for i := 0; i < len(metas); i++ { + j := i + 1 + + // first iteration (i == 0), set the current meta + if cur == nil { + cur = &metas[i] + } + + if j >= len(metas) { + // We've reached the end of the list. Add the last meta to the non-overlapping set. + nonOverlapping = append(nonOverlapping, *cur) + break + } + + combined := cur.Union(metas[j]) + if len(combined) == 1 { + // There was an overlap between the two tested ranges. Combine them and keep going. + cur = &combined[0] + continue + } + + // There was no overlap between the two tested ranges. Add the first to the non-overlapping set. + // and keep the second for the next iteration. + nonOverlapping = append(nonOverlapping, combined[0]) + cur = &combined[1] + } + + // Now, detect gaps between the non-overlapping metas and the ownership range. + // The left bound of the ownership range will be adjusted as we go. + leftBound := ownershipRange.Min + for _, meta := range nonOverlapping { + + clippedMeta := meta.Intersection(ownershipRange) + // should never happen as long as we are only combining metas + // that intersect with the ownership range + if clippedMeta == nil { + return nil, fmt.Errorf("meta is not within ownership range: %v", meta) + } + + searchRange := ownershipRange.Slice(leftBound, clippedMeta.Max) + // update the left bound for the next iteration + // We do the max to prevent the max bound to overflow from MaxUInt64 to 0 + leftBound = min( + max(clippedMeta.Max+1, clippedMeta.Max), + max(ownershipRange.Max+1, ownershipRange.Max), + ) + + // since we've already ensured that the meta is within the ownership range, + // we know the xor will be of length zero (when the meta is equal to the ownership range) + // or 1 (when the meta is a subset of the ownership range) + xors := searchRange.Unless(*clippedMeta) + if len(xors) == 0 { + // meta is equal to the ownership range. This means the meta + // covers this entire section of the ownership range. + continue + } + + gaps = append(gaps, xors[0]) + } + + // If the leftBound is less than the ownership range max, and it's smaller than MaxUInt64, + // There is a gap between the last meta and the end of the ownership range. + // Note: we check `leftBound < math.MaxUint64` since in the loop above we clamp the + // leftBound to MaxUint64 to prevent an overflow to 0: `max(clippedMeta.Max+1, clippedMeta.Max)` + if leftBound < math.MaxUint64 && leftBound <= ownershipRange.Max { + gaps = append(gaps, v1.NewBounds(leftBound, ownershipRange.Max)) + } + + return gaps, nil +} diff --git a/pkg/bloombuild/planner/util_test.go b/pkg/bloombuild/planner/util_test.go new file mode 100644 index 0000000000000..6755478ef7290 --- /dev/null +++ b/pkg/bloombuild/planner/util_test.go @@ -0,0 +1,172 @@ +package planner + +import ( + "math" + "testing" + + "github.com/prometheus/common/model" + "github.com/stretchr/testify/require" + + v1 "github.com/grafana/loki/v3/pkg/storage/bloom/v1" +) + +func TestSplitFingerprintKeyspaceByFactor(t *testing.T) { + for _, tt := range []struct { + name string + factor int + }{ + { + name: "Factor is 0", + factor: 0, + }, + { + name: "Factor is 1", + factor: 1, + }, + { + name: "Factor is 256", + factor: 256, + }, + } { + t.Run(tt.name, func(t *testing.T) { + got := SplitFingerprintKeyspaceByFactor(tt.factor) + + if tt.factor == 0 { + require.Empty(t, got) + return + } + + // Check overall min and max values of the ranges. + require.Equal(t, model.Fingerprint(math.MaxUint64), got[len(got)-1].Max) + require.Equal(t, model.Fingerprint(0), got[0].Min) + + // For each range, check that the max value of the previous range is one less than the min value of the current range. + for i := 1; i < len(got); i++ { + require.Equal(t, got[i-1].Max+1, got[i].Min) + } + }) + } +} + +func Test_FindGapsInFingerprintBounds(t *testing.T) { + for _, tc := range []struct { + desc string + err bool + exp []v1.FingerprintBounds + ownershipRange v1.FingerprintBounds + metas []v1.FingerprintBounds + }{ + { + desc: "error nonoverlapping metas", + err: true, + exp: nil, + ownershipRange: v1.NewBounds(0, 10), + metas: []v1.FingerprintBounds{v1.NewBounds(11, 20)}, + }, + { + desc: "one meta with entire ownership range", + err: false, + exp: nil, + ownershipRange: v1.NewBounds(0, 10), + metas: []v1.FingerprintBounds{v1.NewBounds(0, 10)}, + }, + { + desc: "two non-overlapping metas with entire ownership range", + err: false, + exp: nil, + ownershipRange: v1.NewBounds(0, 10), + metas: []v1.FingerprintBounds{ + v1.NewBounds(0, 5), + v1.NewBounds(6, 10), + }, + }, + { + desc: "two overlapping metas with entire ownership range", + err: false, + exp: nil, + ownershipRange: v1.NewBounds(0, 10), + metas: []v1.FingerprintBounds{ + v1.NewBounds(0, 6), + v1.NewBounds(4, 10), + }, + }, + { + desc: "one meta with partial ownership range", + err: false, + exp: []v1.FingerprintBounds{ + v1.NewBounds(6, 10), + }, + ownershipRange: v1.NewBounds(0, 10), + metas: []v1.FingerprintBounds{ + v1.NewBounds(0, 5), + }, + }, + { + desc: "smaller subsequent meta with partial ownership range", + err: false, + exp: []v1.FingerprintBounds{ + v1.NewBounds(8, 10), + }, + ownershipRange: v1.NewBounds(0, 10), + metas: []v1.FingerprintBounds{ + v1.NewBounds(0, 7), + v1.NewBounds(3, 4), + }, + }, + { + desc: "hole in the middle", + err: false, + exp: []v1.FingerprintBounds{ + v1.NewBounds(4, 5), + }, + ownershipRange: v1.NewBounds(0, 10), + metas: []v1.FingerprintBounds{ + v1.NewBounds(0, 3), + v1.NewBounds(6, 10), + }, + }, + { + desc: "holes on either end", + err: false, + exp: []v1.FingerprintBounds{ + v1.NewBounds(0, 2), + v1.NewBounds(8, 10), + }, + ownershipRange: v1.NewBounds(0, 10), + metas: []v1.FingerprintBounds{ + v1.NewBounds(3, 5), + v1.NewBounds(6, 7), + }, + }, + { + desc: "full ownership range with single meta", + err: false, + exp: nil, + ownershipRange: v1.NewBounds(0, math.MaxUint64), + metas: []v1.FingerprintBounds{ + v1.NewBounds(0, math.MaxUint64), + }, + }, + { + desc: "full ownership range with multiple metas", + err: false, + exp: nil, + ownershipRange: v1.NewBounds(0, math.MaxUint64), + // Three metas covering the whole 0 - MaxUint64 + metas: []v1.FingerprintBounds{ + v1.NewBounds(0, math.MaxUint64/3), + v1.NewBounds(math.MaxUint64/3+1, math.MaxUint64/2), + v1.NewBounds(math.MaxUint64/2+1, math.MaxUint64), + }, + }, + } { + t.Run(tc.desc, func(t *testing.T) { + gaps, err := FindGapsInFingerprintBounds(tc.ownershipRange, tc.metas) + if tc.err { + require.Error(t, err) + return + } + require.Equal(t, tc.exp, gaps) + }) + } +} diff --git a/pkg/ingester/instance.go b/pkg/ingester/instance.go index a4436b9d41915..7f1ec78601fff 100644 --- a/pkg/ingester/instance.go +++ b/pkg/ingester/instance.go @@ -104,7 +104,10 @@ type instance struct { tailers map[uint32]*tailer tailerMtx sync.RWMutex - limiter *Limiter + limiter *Limiter + streamCountLimiter *streamCountLimiter + ownedStreamsSvc *ownedStreamService + configs *runtime.TenantConfigs wal WAL @@ -147,11 +150,12 @@ func newInstance( if err != nil { return nil, err } - + streams := newStreamsMap() + ownedStreamsSvc := newOwnedStreamService(instanceID, limiter) c := config.SchemaConfig{Configs: periodConfigs} i := &instance{ cfg: cfg, - streams: newStreamsMap(), + streams: streams, buf: make([]byte, 0, 1024), index: invertedIndex, instanceID: instanceID, @@ -159,9 +163,11 @@ func newInstance( streamsCreatedTotal: streamsCreatedTotal.WithLabelValues(instanceID), streamsRemovedTotal: streamsRemovedTotal.WithLabelValues(instanceID), - tailers: map[uint32]*tailer{}, - limiter: limiter, - configs: configs, + tailers: map[uint32]*tailer{}, + limiter: limiter, + streamCountLimiter: newStreamCountLimiter(instanceID, streams.Len, limiter, ownedStreamsSvc), + ownedStreamsSvc: ownedStreamsSvc, + configs: configs, wal: wal, metrics: metrics, @@ -286,29 +292,11 @@ func (i *instance) createStream(ctx context.Context, pushReqStream logproto.Stre } if record != nil { - err = i.limiter.AssertMaxStreamsPerUser(i.instanceID, i.streams.Len()) + err = i.streamCountLimiter.AssertNewStreamAllowed(i.instanceID) } if err != nil { - if i.configs.LogStreamCreation(i.instanceID) { - level.Debug(util_log.Logger).Log( - "msg", "failed to create stream, exceeded limit", - "org_id", i.instanceID, - "err", err, - "stream", pushReqStream.Labels, - ) - } - - validation.DiscardedSamples.WithLabelValues(validation.StreamLimit, i.instanceID).Add(float64(len(pushReqStream.Entries))) - bytes := 0 - for _, e := range pushReqStream.Entries { - bytes += len(e.Line) - } - validation.DiscardedBytes.WithLabelValues(validation.StreamLimit, i.instanceID).Add(float64(bytes)) - if i.customStreamsTracker != nil { - i.customStreamsTracker.DiscardedBytesAdd(ctx, i.instanceID, validation.StreamLimit, labels, float64(bytes)) - } - return nil, httpgrpc.Errorf(http.StatusTooManyRequests, validation.StreamLimitErrorMsg, labels, i.instanceID) + return i.onStreamCreationError(ctx, pushReqStream, err, labels) } fp := i.getHashForLabels(labels) @@ -333,21 +321,47 @@ func (i *instance) createStream(ctx context.Context, pushReqStream logproto.Stre i.metrics.recoveredStreamsTotal.Inc() } + i.onStreamCreated(s) + + return s, nil +} + +func (i *instance) onStreamCreationError(ctx context.Context, pushReqStream logproto.Stream, err error, labels labels.Labels) (*stream, error) { + if i.configs.LogStreamCreation(i.instanceID) { + level.Debug(util_log.Logger).Log( + "msg", "failed to create stream, exceeded limit", + "org_id", i.instanceID, + "err", err, + "stream", pushReqStream.Labels, + ) + } + + validation.DiscardedSamples.WithLabelValues(validation.StreamLimit, i.instanceID).Add(float64(len(pushReqStream.Entries))) + bytes := 0 + for _, e := range pushReqStream.Entries { + bytes += len(e.Line) + } + validation.DiscardedBytes.WithLabelValues(validation.StreamLimit, i.instanceID).Add(float64(bytes)) + if i.customStreamsTracker != nil { + i.customStreamsTracker.DiscardedBytesAdd(ctx, i.instanceID, validation.StreamLimit, labels, float64(bytes)) + } + return nil, httpgrpc.Errorf(http.StatusTooManyRequests, validation.StreamLimitErrorMsg, labels, i.instanceID) +} + +func (i *instance) onStreamCreated(s *stream) { memoryStreams.WithLabelValues(i.instanceID).Inc() memoryStreamsLabelsBytes.Add(float64(len(s.labels.String()))) i.streamsCreatedTotal.Inc() i.addTailersToNewStream(s) streamsCountStats.Add(1) - + i.ownedStreamsSvc.incOwnedStreamCount() if i.configs.LogStreamCreation(i.instanceID) { level.Debug(util_log.Logger).Log( "msg", "successfully created stream", "org_id", i.instanceID, - "stream", pushReqStream.Labels, + "stream", s.labels.String(), ) } - - return s, nil } func (i *instance) createStreamByFP(ls labels.Labels, fp model.Fingerprint) (*stream, error) { @@ -407,6 +421,7 @@ func (i *instance) removeStream(s *stream) { memoryStreams.WithLabelValues(i.instanceID).Dec() memoryStreamsLabelsBytes.Sub(float64(len(s.labels.String()))) streamsCountStats.Add(-1) + i.ownedStreamsSvc.decOwnedStreamCount() } } diff --git a/pkg/ingester/limiter.go b/pkg/ingester/limiter.go index 94c77a30be7e3..daa1fe7aec8da 100644 --- a/pkg/ingester/limiter.go +++ b/pkg/ingester/limiter.go @@ -24,6 +24,7 @@ type RingCount interface { type Limits interface { UnorderedWrites(userID string) bool + UseOwnedStreamCount(userID string) bool MaxLocalStreamsPerUser(userID string) int MaxGlobalStreamsPerUser(userID string) int PerStreamRateLimit(userID string) validation.RateLimit @@ -76,46 +77,39 @@ func (l *Limiter) UnorderedWrites(userID string) bool { return l.limits.UnorderedWrites(userID) } -// AssertMaxStreamsPerUser ensures limit has not been reached compared to the current -// number of streams in input and returns an error if so. -func (l *Limiter) AssertMaxStreamsPerUser(userID string, streams int) error { - // Until the limiter actually starts, all accesses are successful. - // This is used to disable limits while recovering from the WAL. - l.mtx.RLock() - defer l.mtx.RUnlock() - if l.disabled { - return nil - } - +func (l *Limiter) GetStreamCountLimit(tenantID string) (calculatedLimit, localLimit, globalLimit, adjustedGlobalLimit int) { // Start by setting the local limit either from override or default - localLimit := l.limits.MaxLocalStreamsPerUser(userID) + localLimit = l.limits.MaxLocalStreamsPerUser(tenantID) // We can assume that streams are evenly distributed across ingesters // so we do convert the global limit into a local limit - globalLimit := l.limits.MaxGlobalStreamsPerUser(userID) - adjustedGlobalLimit := l.convertGlobalToLocalLimit(globalLimit) + globalLimit = l.limits.MaxGlobalStreamsPerUser(tenantID) + adjustedGlobalLimit = l.convertGlobalToLocalLimit(globalLimit) // Set the calculated limit to the lesser of the local limit or the new calculated global limit - calculatedLimit := l.minNonZero(localLimit, adjustedGlobalLimit) + calculatedLimit = l.minNonZero(localLimit, adjustedGlobalLimit) // If both the local and global limits are disabled, we just // use the largest int value if calculatedLimit == 0 { calculatedLimit = math.MaxInt32 } + return +} - if streams < calculatedLimit { - return nil +func (l *Limiter) minNonZero(first, second int) int { + if first == 0 || (second != 0 && first > second) { + return second } - return fmt.Errorf(errMaxStreamsPerUserLimitExceeded, userID, streams, calculatedLimit, localLimit, globalLimit, adjustedGlobalLimit) + return first } func (l *Limiter) convertGlobalToLocalLimit(globalLimit int) int { if globalLimit == 0 { return 0 } - + // todo: change to healthyInstancesInZoneCount() once // Given we don't need a super accurate count (ie. when the ingesters // topology changes) and we prefer to always be in favor of the tenant, // we can use a per-ingester limit equal to: @@ -131,12 +125,53 @@ func (l *Limiter) convertGlobalToLocalLimit(globalLimit int) int { return 0 } -func (l *Limiter) minNonZero(first, second int) int { - if first == 0 || (second != 0 && first > second) { - return second +type supplier[T any] func() T + +type streamCountLimiter struct { + tenantID string + limiter *Limiter + defaultStreamCountSupplier supplier[int] + ownedStreamSvc *ownedStreamService +} + +var noopFixedLimitSupplier = func() int { + return 0 +} + +func newStreamCountLimiter(tenantID string, defaultStreamCountSupplier supplier[int], limiter *Limiter, service *ownedStreamService) *streamCountLimiter { + return &streamCountLimiter{ + tenantID: tenantID, + limiter: limiter, + defaultStreamCountSupplier: defaultStreamCountSupplier, + ownedStreamSvc: service, } +} - return first +func (l *streamCountLimiter) AssertNewStreamAllowed(tenantID string) error { + streamCountSupplier, fixedLimitSupplier := l.getSuppliers(tenantID) + calculatedLimit, localLimit, globalLimit, adjustedGlobalLimit := l.getCurrentLimit(tenantID, fixedLimitSupplier) + actualStreamsCount := streamCountSupplier() + if actualStreamsCount < calculatedLimit { + return nil + } + + return fmt.Errorf(errMaxStreamsPerUserLimitExceeded, tenantID, actualStreamsCount, calculatedLimit, localLimit, globalLimit, adjustedGlobalLimit) +} + +func (l *streamCountLimiter) getCurrentLimit(tenantID string, fixedLimitSupplier supplier[int]) (calculatedLimit, localLimit, globalLimit, adjustedGlobalLimit int) { + calculatedLimit, localLimit, globalLimit, adjustedGlobalLimit = l.limiter.GetStreamCountLimit(tenantID) + fixedLimit := fixedLimitSupplier() + if fixedLimit > calculatedLimit { + calculatedLimit = fixedLimit + } + return +} + +func (l *streamCountLimiter) getSuppliers(tenant string) (streamCountSupplier, fixedLimitSupplier supplier[int]) { + if l.limiter.limits.UseOwnedStreamCount(tenant) { + return l.ownedStreamSvc.getOwnedStreamCount, l.ownedStreamSvc.getFixedLimit + } + return l.defaultStreamCountSupplier, noopFixedLimitSupplier } type RateLimiterStrategy interface { diff --git a/pkg/ingester/limiter_test.go b/pkg/ingester/limiter_test.go index 6186e910663e0..9d4d3b3037c6f 100644 --- a/pkg/ingester/limiter_test.go +++ b/pkg/ingester/limiter_test.go @@ -8,12 +8,13 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + "go.uber.org/atomic" "golang.org/x/time/rate" "github.com/grafana/loki/v3/pkg/validation" ) -func TestLimiter_AssertMaxStreamsPerUser(t *testing.T) { +func TestStreamCountLimiter_AssertNewStreamAllowed(t *testing.T) { tests := map[string]struct { maxLocalStreamsPerUser int maxGlobalStreamsPerUser int @@ -21,6 +22,9 @@ func TestLimiter_AssertMaxStreamsPerUser(t *testing.T) { ringIngesterCount int streams int expected error + useOwnedStreamService bool + fixedLimit int32 + ownedStreamCount int64 }{ "both local and global limit are disabled": { maxLocalStreamsPerUser: 0, @@ -94,6 +98,36 @@ func TestLimiter_AssertMaxStreamsPerUser(t *testing.T) { streams: 3000, expected: fmt.Errorf(errMaxStreamsPerUserLimitExceeded, "test", 3000, 300, 500, 1000, 300), }, + "actual limit must be used if it's greater than fixed limit": { + maxLocalStreamsPerUser: 500, + maxGlobalStreamsPerUser: 1000, + ringReplicationFactor: 3, + ringIngesterCount: 10, + useOwnedStreamService: true, + fixedLimit: 20, + ownedStreamCount: 3000, + expected: fmt.Errorf(errMaxStreamsPerUserLimitExceeded, "test", 3000, 300, 500, 1000, 300), + }, + "fixed limit must be used if it's greater than actual limit": { + maxLocalStreamsPerUser: 500, + maxGlobalStreamsPerUser: 1000, + ringReplicationFactor: 3, + ringIngesterCount: 10, + useOwnedStreamService: true, + fixedLimit: 2000, + ownedStreamCount: 2001, + expected: fmt.Errorf(errMaxStreamsPerUserLimitExceeded, "test", 2001, 2000, 500, 1000, 300), + }, + "fixed limit must not be used if both limits are disabled": { + maxLocalStreamsPerUser: 0, + maxGlobalStreamsPerUser: 0, + ringReplicationFactor: 3, + ringIngesterCount: 10, + useOwnedStreamService: true, + fixedLimit: 2000, + ownedStreamCount: 2001, + expected: nil, + }, } for testName, testData := range tests { @@ -107,11 +141,20 @@ func TestLimiter_AssertMaxStreamsPerUser(t *testing.T) { limits, err := validation.NewOverrides(validation.Limits{ MaxLocalStreamsPerUser: testData.maxLocalStreamsPerUser, MaxGlobalStreamsPerUser: testData.maxGlobalStreamsPerUser, + UseOwnedStreamCount: testData.useOwnedStreamService, }, nil) require.NoError(t, err) + ownedStreamSvc := &ownedStreamService{ + fixedLimit: atomic.NewInt32(testData.fixedLimit), + ownedStreamCount: atomic.NewInt64(testData.ownedStreamCount), + } limiter := NewLimiter(limits, NilMetrics, ring, testData.ringReplicationFactor) - actual := limiter.AssertMaxStreamsPerUser("test", testData.streams) + defaultCountSupplier := func() int { + return testData.streams + } + streamCountLimiter := newStreamCountLimiter("test", defaultCountSupplier, limiter, ownedStreamSvc) + actual := streamCountLimiter.AssertNewStreamAllowed("test") assert.Equal(t, testData.expected, actual) }) diff --git a/pkg/ingester/owned_streams.go b/pkg/ingester/owned_streams.go new file mode 100644 index 0000000000000..01cb8235f9b1a --- /dev/null +++ b/pkg/ingester/owned_streams.go @@ -0,0 +1,44 @@ +package ingester + +import "go.uber.org/atomic" + +type ownedStreamService struct { + tenantID string + limiter *Limiter + fixedLimit *atomic.Int32 + + //todo: implement job to recalculate it + ownedStreamCount *atomic.Int64 +} + +func newOwnedStreamService(tenantID string, limiter *Limiter) *ownedStreamService { + svc := &ownedStreamService{ + tenantID: tenantID, + limiter: limiter, + ownedStreamCount: atomic.NewInt64(0), + fixedLimit: atomic.NewInt32(0), + } + svc.updateFixedLimit() + return svc +} + +func (s *ownedStreamService) getOwnedStreamCount() int { + return int(s.ownedStreamCount.Load()) +} + +func (s *ownedStreamService) updateFixedLimit() { + limit, _, _, _ := s.limiter.GetStreamCountLimit(s.tenantID) + s.fixedLimit.Store(int32(limit)) +} + +func (s *ownedStreamService) getFixedLimit() int { + return int(s.fixedLimit.Load()) +} + +func (s *ownedStreamService) incOwnedStreamCount() { + s.ownedStreamCount.Inc() +} + +func (s *ownedStreamService) decOwnedStreamCount() { + s.ownedStreamCount.Dec() +} diff --git a/pkg/ingester/owned_streams_test.go b/pkg/ingester/owned_streams_test.go new file mode 100644 index 0000000000000..c7ddd9d87f29d --- /dev/null +++ b/pkg/ingester/owned_streams_test.go @@ -0,0 +1,36 @@ +package ingester + +import ( + "testing" + + "github.com/stretchr/testify/require" + + "github.com/grafana/loki/v3/pkg/validation" +) + +func Test_OwnedStreamService(t *testing.T) { + limits, err := validation.NewOverrides(validation.Limits{ + MaxGlobalStreamsPerUser: 100, + }, nil) + require.NoError(t, err) + // Mock the ring + ring := &ringCountMock{count: 30} + limiter := NewLimiter(limits, NilMetrics, ring, 3) + + service := newOwnedStreamService("test", limiter) + require.Equal(t, 0, service.getOwnedStreamCount()) + require.Equal(t, 10, service.getFixedLimit(), "fixed limit must be initialised during the instantiation") + + limits.DefaultLimits().MaxGlobalStreamsPerUser = 1000 + require.Equal(t, 10, service.getFixedLimit(), "fixed list must not be changed until update is triggered") + + service.updateFixedLimit() + require.Equal(t, 100, service.getFixedLimit()) + + service.incOwnedStreamCount() + service.incOwnedStreamCount() + require.Equal(t, 2, service.getOwnedStreamCount()) + + service.decOwnedStreamCount() + require.Equal(t, 1, service.getOwnedStreamCount()) +} diff --git a/pkg/logproto/extensions.go b/pkg/logproto/extensions.go index 3de5c0fd75801..bd726bbb0d358 100644 --- a/pkg/logproto/extensions.go +++ b/pkg/logproto/extensions.go @@ -161,7 +161,8 @@ func (r *QueryPatternsResponse) UnmarshalJSON(data []byte) error { var v struct { Status string `json:"status"` Data []struct { - Pattern string `json:"pattern"` + Pattern string `json:"pattern,omitempty"` + Labels string `json:"labels,omitempty"` Samples [][]int64 `json:"samples"` } `json:"data"` } @@ -174,7 +175,12 @@ func (r *QueryPatternsResponse) UnmarshalJSON(data []byte) error { for _, s := range d.Samples { samples = append(samples, &PatternSample{Timestamp: model.TimeFromUnix(s[0]), Value: s[1]}) } - r.Series = append(r.Series, &PatternSeries{Pattern: d.Pattern, Samples: samples}) + + if pattern := d.Pattern; pattern != "" { + r.Series = append(r.Series, NewPatternSeriesWithPattern(pattern, samples)) + } else if labels := d.Labels; labels != "" { + r.Series = append(r.Series, NewPatternSeriesWithLabels(labels, samples)) + } } return nil } @@ -188,3 +194,11 @@ func (m *ShardsResponse) Merge(other *ShardsResponse) { m.ChunkGroups = append(m.ChunkGroups, other.ChunkGroups...) m.Statistics.Merge(other.Statistics) } + +func NewPatternSeriesWithPattern(pattern string, samples []*PatternSample) *PatternSeries { + return &PatternSeries{Identifier: &PatternSeries_Pattern{pattern}, Samples: samples} +} + +func NewPatternSeriesWithLabels(labels string, samples []*PatternSample) *PatternSeries { + return &PatternSeries{Identifier: &PatternSeries_Labels{labels}, Samples: samples} +} diff --git a/pkg/logproto/extensions_test.go b/pkg/logproto/extensions_test.go index d1c96c76bbed3..de8e84e34cd35 100644 --- a/pkg/logproto/extensions_test.go +++ b/pkg/logproto/extensions_test.go @@ -3,6 +3,7 @@ package logproto import ( "testing" + "github.com/prometheus/common/model" "github.com/stretchr/testify/require" ) @@ -40,3 +41,71 @@ func TestShard_SpaceFor(t *testing.T) { }) } } + +func TestQueryPatternsResponse_UnmarshalJSON(t *testing.T) { + t.Run("unmarshals patterns", func(t *testing.T) { + mockData := []byte(`{ + "status": "success", + "data": [ + { + "pattern": "foo <*> bar", + "samples": [[1609459200, 10], [1609545600, 15]] + }, + { + "pattern": "foo <*> buzz", + "samples": [[1609459200, 20], [1609545600, 25]] + } + ] + }`) + + expectedSeries := []*PatternSeries{ + NewPatternSeriesWithPattern("foo <*> bar", []*PatternSample{ + {Timestamp: model.TimeFromUnix(1609459200), Value: 10}, + {Timestamp: model.TimeFromUnix(1609545600), Value: 15}, + }), + NewPatternSeriesWithPattern("foo <*> buzz", []*PatternSample{ + {Timestamp: model.TimeFromUnix(1609459200), Value: 20}, + {Timestamp: model.TimeFromUnix(1609545600), Value: 25}, + }), + } + + r := &QueryPatternsResponse{} + err := r.UnmarshalJSON(mockData) + + require.Nil(t, err) + require.Equal(t, expectedSeries, r.Series) + }) + + t.Run("unmarshals labels", func(t *testing.T) { + mockData := []byte(`{ + "status": "success", + "data": [ + { + "labels": "{foo=\"bar\"}", + "samples": [[1609459200, 10], [1609545600, 15]] + }, + { + "labels": "{foo=\"buzz\"}", + "samples": [[1609459200, 20], [1609545600, 25]] + } + ] + }`) + + expectedSeries := []*PatternSeries{ + NewPatternSeriesWithLabels(`{foo="bar"}`, []*PatternSample{ + {Timestamp: model.TimeFromUnix(1609459200), Value: 10}, + {Timestamp: model.TimeFromUnix(1609545600), Value: 15}, + }), + NewPatternSeriesWithLabels(`{foo="buzz"}`, []*PatternSample{ + {Timestamp: model.TimeFromUnix(1609459200), Value: 20}, + {Timestamp: model.TimeFromUnix(1609545600), Value: 25}, + }), + } + + r := &QueryPatternsResponse{} + err := r.UnmarshalJSON(mockData) + + require.Nil(t, err) + require.Equal(t, expectedSeries, r.Series) + }) +} diff --git a/pkg/logproto/pattern.pb.go b/pkg/logproto/pattern.pb.go index a666a32850127..facf0b4bfa907 100644 --- a/pkg/logproto/pattern.pb.go +++ b/pkg/logproto/pattern.pb.go @@ -146,8 +146,11 @@ func (m *QueryPatternsResponse) GetSeries() []*PatternSeries { } type PatternSeries struct { - Pattern string `protobuf:"bytes,1,opt,name=pattern,proto3" json:"pattern,omitempty"` - Samples []*PatternSample `protobuf:"bytes,2,rep,name=samples,proto3" json:"samples,omitempty"` + // Types that are valid to be assigned to Identifier: + // *PatternSeries_Pattern + // *PatternSeries_Labels + Identifier isPatternSeries_Identifier `protobuf_oneof:"identifier"` + Samples []*PatternSample `protobuf:"bytes,2,rep,name=samples,proto3" json:"samples,omitempty"` } func (m *PatternSeries) Reset() { *m = PatternSeries{} } @@ -182,9 +185,40 @@ func (m *PatternSeries) XXX_DiscardUnknown() { var xxx_messageInfo_PatternSeries proto.InternalMessageInfo -func (m *PatternSeries) GetPattern() string { +type isPatternSeries_Identifier interface { + isPatternSeries_Identifier() + Equal(interface{}) bool + MarshalTo([]byte) (int, error) + Size() int +} + +type PatternSeries_Pattern struct { + Pattern string `protobuf:"bytes,1,opt,name=pattern,proto3,oneof"` +} +type PatternSeries_Labels struct { + Labels string `protobuf:"bytes,3,opt,name=labels,proto3,oneof"` +} + +func (*PatternSeries_Pattern) isPatternSeries_Identifier() {} +func (*PatternSeries_Labels) isPatternSeries_Identifier() {} + +func (m *PatternSeries) GetIdentifier() isPatternSeries_Identifier { if m != nil { - return m.Pattern + return m.Identifier + } + return nil +} + +func (m *PatternSeries) GetPattern() string { + if x, ok := m.GetIdentifier().(*PatternSeries_Pattern); ok { + return x.Pattern + } + return "" +} + +func (m *PatternSeries) GetLabels() string { + if x, ok := m.GetIdentifier().(*PatternSeries_Labels); ok { + return x.Labels } return "" } @@ -196,6 +230,14 @@ func (m *PatternSeries) GetSamples() []*PatternSample { return nil } +// XXX_OneofWrappers is for the internal use of the proto package. +func (*PatternSeries) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*PatternSeries_Pattern)(nil), + (*PatternSeries_Labels)(nil), + } +} + type PatternSample struct { Timestamp github_com_prometheus_common_model.Time `protobuf:"varint,1,opt,name=timestamp,proto3,customtype=github.com/prometheus/common/model.Time" json:"timestamp"` Value int64 `protobuf:"varint,2,opt,name=value,proto3" json:"value,omitempty"` @@ -250,38 +292,39 @@ func init() { func init() { proto.RegisterFile("pkg/logproto/pattern.proto", fileDescriptor_aaf4192acc66a4ea) } var fileDescriptor_aaf4192acc66a4ea = []byte{ - // 483 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x52, 0xb1, 0x6e, 0xd3, 0x40, - 0x18, 0xf6, 0xd5, 0x49, 0xd3, 0x5e, 0xc5, 0x72, 0xa4, 0x60, 0x19, 0xe9, 0x1c, 0x79, 0x21, 0x93, - 0x0f, 0x52, 0x09, 0x24, 0xc6, 0x4c, 0x0c, 0x20, 0x15, 0xc3, 0x84, 0x60, 0x70, 0xda, 0xbf, 0xb6, - 0x55, 0xdb, 0xe7, 0xfa, 0xee, 0x2a, 0xb1, 0xf1, 0x08, 0x79, 0x0c, 0x1e, 0x80, 0x87, 0xe8, 0x98, - 0xb1, 0x62, 0x28, 0xc4, 0x59, 0x18, 0xfb, 0x08, 0xc8, 0x77, 0x76, 0x93, 0x56, 0x74, 0xe8, 0x92, - 0xdc, 0xff, 0x7f, 0xdf, 0xff, 0xf9, 0xbb, 0xff, 0x3b, 0xec, 0x96, 0xa7, 0x31, 0xcb, 0x78, 0x5c, - 0x56, 0x5c, 0x72, 0x56, 0x46, 0x52, 0x42, 0x55, 0x04, 0xba, 0x22, 0x3b, 0x5d, 0xdf, 0x1d, 0xc6, - 0x3c, 0xe6, 0x86, 0xd2, 0x9c, 0x0c, 0xee, 0x7a, 0x31, 0xe7, 0x71, 0x06, 0x4c, 0x57, 0x33, 0x75, - 0xc2, 0x64, 0x9a, 0x83, 0x90, 0x51, 0x5e, 0xb6, 0x84, 0x67, 0xb7, 0xc4, 0xbb, 0x43, 0x0b, 0x3e, - 0x6e, 0xc0, 0x52, 0x89, 0x44, 0xff, 0x98, 0xa6, 0xff, 0x13, 0xe1, 0xe1, 0x07, 0x05, 0xd5, 0xb7, - 0x43, 0xe3, 0x44, 0x84, 0x70, 0xa6, 0x40, 0x48, 0x32, 0xc4, 0xfd, 0xb3, 0xa6, 0xef, 0xa0, 0x11, - 0x1a, 0xef, 0x86, 0xa6, 0x20, 0x6f, 0x70, 0x5f, 0xc8, 0xa8, 0x92, 0xce, 0xd6, 0x08, 0x8d, 0xf7, - 0x26, 0x6e, 0x60, 0x1c, 0x05, 0x9d, 0xa3, 0xe0, 0x53, 0xe7, 0x68, 0xba, 0x73, 0x71, 0xe5, 0x59, - 0xf3, 0xdf, 0x1e, 0x0a, 0xcd, 0x08, 0x79, 0x85, 0x6d, 0x28, 0x8e, 0x1d, 0xfb, 0x01, 0x93, 0xcd, - 0x00, 0x21, 0xb8, 0x27, 0x24, 0x94, 0x4e, 0x6f, 0x84, 0xc6, 0x76, 0xa8, 0xcf, 0xfe, 0x5b, 0xbc, - 0x7f, 0xc7, 0xb5, 0x28, 0x79, 0x21, 0x80, 0x30, 0xbc, 0x2d, 0xa0, 0x4a, 0x41, 0x38, 0x68, 0x64, - 0x8f, 0xf7, 0x26, 0x4f, 0x83, 0x9b, 0x2d, 0xb4, 0xdc, 0x8f, 0x1a, 0x0e, 0x5b, 0x9a, 0xff, 0x05, - 0x3f, 0xba, 0x05, 0x10, 0x07, 0x0f, 0xda, 0x54, 0xda, 0xab, 0x77, 0x25, 0x79, 0x89, 0x07, 0x22, - 0xca, 0xcb, 0x0c, 0x84, 0xb3, 0x75, 0x9f, 0xb8, 0xc6, 0xc3, 0x8e, 0xe7, 0xcb, 0xb5, 0xba, 0xee, - 0x90, 0xf7, 0x78, 0xf7, 0x26, 0x34, 0xad, 0x6f, 0x4f, 0x59, 0x73, 0xdd, 0x5f, 0x57, 0xde, 0xf3, - 0x38, 0x95, 0x89, 0x9a, 0x05, 0x47, 0x3c, 0x6f, 0x12, 0xce, 0x41, 0x26, 0xa0, 0x04, 0x3b, 0xe2, - 0x79, 0xce, 0x0b, 0x96, 0xf3, 0x63, 0xc8, 0xf4, 0x92, 0xc2, 0xb5, 0x42, 0x93, 0xd2, 0x79, 0x94, - 0x29, 0xd0, 0x79, 0xd8, 0xa1, 0x29, 0x26, 0x73, 0x84, 0x07, 0xed, 0x67, 0xc9, 0x6b, 0xdc, 0x3b, - 0x54, 0x22, 0x21, 0xfb, 0x1b, 0x5e, 0x95, 0x48, 0xda, 0x98, 0xdd, 0x27, 0x77, 0xdb, 0x66, 0x8f, - 0xbe, 0x45, 0xde, 0xe1, 0xbe, 0x5e, 0x31, 0xa1, 0x6b, 0xca, 0xff, 0x5e, 0x8a, 0xeb, 0xdd, 0x8b, - 0x77, 0x5a, 0x2f, 0xd0, 0xf4, 0xeb, 0x62, 0x49, 0xad, 0xcb, 0x25, 0xb5, 0xae, 0x97, 0x14, 0x7d, - 0xaf, 0x29, 0xfa, 0x51, 0x53, 0x74, 0x51, 0x53, 0xb4, 0xa8, 0x29, 0xfa, 0x53, 0x53, 0xf4, 0xb7, - 0xa6, 0xd6, 0x75, 0x4d, 0xd1, 0x7c, 0x45, 0xad, 0xc5, 0x8a, 0x5a, 0x97, 0x2b, 0x6a, 0x7d, 0xde, - 0x5c, 0x49, 0x5c, 0x45, 0x27, 0x51, 0x11, 0xb1, 0x8c, 0x9f, 0xa6, 0xec, 0xfc, 0x80, 0x6d, 0x3e, - 0xf5, 0xd9, 0xb6, 0xfe, 0x3b, 0xf8, 0x17, 0x00, 0x00, 0xff, 0xff, 0x3b, 0x4f, 0x5c, 0x50, 0x5e, - 0x03, 0x00, 0x00, + // 511 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x53, 0xbd, 0x6e, 0xd4, 0x40, + 0x10, 0xf6, 0xc6, 0xf7, 0x93, 0x6c, 0xa0, 0x59, 0x2e, 0x60, 0x19, 0x69, 0x7d, 0x72, 0xc3, 0x55, + 0x5e, 0xb8, 0x48, 0x20, 0x51, 0x5e, 0x95, 0x02, 0xa4, 0x60, 0xa8, 0x90, 0x28, 0x7c, 0xb9, 0x39, + 0xdb, 0x8a, 0xed, 0x75, 0xbc, 0xeb, 0x48, 0x74, 0x54, 0xd4, 0xf7, 0x18, 0x3c, 0x00, 0x0f, 0x91, + 0xf2, 0xca, 0x88, 0x22, 0x70, 0xbe, 0x86, 0x32, 0x8f, 0x80, 0xbc, 0x6b, 0xe7, 0x2e, 0x11, 0x29, + 0xd2, 0xd8, 0x33, 0xf3, 0x7d, 0x33, 0xfb, 0xed, 0xcc, 0x2c, 0xb6, 0xf3, 0xd3, 0x90, 0x25, 0x3c, + 0xcc, 0x0b, 0x2e, 0x39, 0xcb, 0x03, 0x29, 0xa1, 0xc8, 0x3c, 0xe5, 0x91, 0xdd, 0x36, 0x6e, 0x0f, + 0x42, 0x1e, 0x72, 0x4d, 0xa9, 0x2d, 0x8d, 0xdb, 0x4e, 0xc8, 0x79, 0x98, 0x00, 0x53, 0xde, 0xb4, + 0x9c, 0x33, 0x19, 0xa7, 0x20, 0x64, 0x90, 0xe6, 0x0d, 0xe1, 0xf9, 0xad, 0xe2, 0xad, 0xd1, 0x80, + 0x4f, 0x6a, 0x30, 0x2f, 0x45, 0xa4, 0x3e, 0x3a, 0xe8, 0xfe, 0x44, 0x78, 0xf0, 0xa1, 0x84, 0xe2, + 0xeb, 0xb1, 0x56, 0x22, 0x7c, 0x38, 0x2b, 0x41, 0x48, 0x32, 0xc0, 0xdd, 0xb3, 0x3a, 0x6e, 0xa1, + 0x21, 0x1a, 0xed, 0xf9, 0xda, 0x21, 0x6f, 0x71, 0x57, 0xc8, 0xa0, 0x90, 0xd6, 0xce, 0x10, 0x8d, + 0xf6, 0xc7, 0xb6, 0xa7, 0x15, 0x79, 0xad, 0x22, 0xef, 0x53, 0xab, 0x68, 0xb2, 0x7b, 0x71, 0xe5, + 0x18, 0x8b, 0xdf, 0x0e, 0xf2, 0x75, 0x0a, 0x79, 0x8d, 0x4d, 0xc8, 0x66, 0x96, 0xf9, 0x80, 0xcc, + 0x3a, 0x81, 0x10, 0xdc, 0x11, 0x12, 0x72, 0xab, 0x33, 0x44, 0x23, 0xd3, 0x57, 0xb6, 0x7b, 0x84, + 0x0f, 0xee, 0xa8, 0x16, 0x39, 0xcf, 0x04, 0x10, 0x86, 0x7b, 0x02, 0x8a, 0x18, 0x84, 0x85, 0x86, + 0xe6, 0x68, 0x7f, 0xfc, 0xcc, 0xbb, 0xe9, 0x42, 0xc3, 0xfd, 0xa8, 0x60, 0xbf, 0xa1, 0xb9, 0xdf, + 0x11, 0x7e, 0x7c, 0x0b, 0x21, 0x36, 0xee, 0x37, 0x63, 0xd1, 0x77, 0x3f, 0x32, 0xfc, 0x36, 0x40, + 0x2c, 0xdc, 0x4b, 0x82, 0x29, 0x24, 0x42, 0x5d, 0xa3, 0x86, 0x1a, 0x9f, 0xbc, 0xc2, 0x7d, 0x11, + 0xa4, 0x79, 0x02, 0xc2, 0xda, 0xb9, 0xef, 0x64, 0x85, 0xfb, 0x2d, 0x6f, 0xf2, 0x08, 0xe3, 0x78, + 0x06, 0x99, 0x8c, 0xe7, 0x31, 0x14, 0xae, 0xdc, 0xe8, 0x50, 0x38, 0x79, 0x8f, 0xf7, 0x6e, 0xe6, + 0xab, 0x94, 0x98, 0x13, 0x56, 0x77, 0xe6, 0xd7, 0x95, 0xf3, 0x22, 0x8c, 0x65, 0x54, 0x4e, 0xbd, + 0x13, 0x9e, 0xd6, 0xcb, 0x90, 0x82, 0x8c, 0xa0, 0x14, 0xec, 0x84, 0xa7, 0x29, 0xcf, 0x58, 0xca, + 0x67, 0x90, 0xa8, 0x7e, 0xfa, 0x9b, 0x0a, 0xf5, 0x40, 0xcf, 0x83, 0xa4, 0x04, 0x35, 0x3a, 0xd3, + 0xd7, 0xce, 0x78, 0x81, 0x70, 0xbf, 0x39, 0x96, 0xbc, 0xc1, 0x9d, 0xe3, 0x52, 0x44, 0xe4, 0x60, + 0x4b, 0x79, 0x29, 0xa2, 0x66, 0x23, 0xec, 0xa7, 0x77, 0xc3, 0xba, 0xe5, 0xae, 0x41, 0xde, 0xe1, + 0xae, 0x9a, 0x06, 0xa1, 0x1b, 0xca, 0xff, 0x96, 0xca, 0x76, 0xee, 0xc5, 0xdb, 0x5a, 0x2f, 0xd1, + 0xe4, 0xcb, 0x72, 0x45, 0x8d, 0xcb, 0x15, 0x35, 0xae, 0x57, 0x14, 0x7d, 0xab, 0x28, 0xfa, 0x51, + 0x51, 0x74, 0x51, 0x51, 0xb4, 0xac, 0x28, 0xfa, 0x53, 0x51, 0xf4, 0xb7, 0xa2, 0xc6, 0x75, 0x45, + 0xd1, 0x62, 0x4d, 0x8d, 0xe5, 0x9a, 0x1a, 0x97, 0x6b, 0x6a, 0x7c, 0xde, 0x6e, 0x49, 0x58, 0x04, + 0xf3, 0x20, 0x0b, 0x58, 0xc2, 0x4f, 0x63, 0x76, 0x7e, 0xc8, 0xb6, 0x5f, 0xc5, 0xb4, 0xa7, 0x7e, + 0x87, 0xff, 0x02, 0x00, 0x00, 0xff, 0xff, 0x1c, 0x3a, 0x0e, 0x65, 0x89, 0x03, 0x00, 0x00, } func (this *QueryPatternsRequest) Equal(that interface{}) bool { @@ -365,7 +408,13 @@ func (this *PatternSeries) Equal(that interface{}) bool { } else if this == nil { return false } - if this.Pattern != that1.Pattern { + if that1.Identifier == nil { + if this.Identifier != nil { + return false + } + } else if this.Identifier == nil { + return false + } else if !this.Identifier.Equal(that1.Identifier) { return false } if len(this.Samples) != len(that1.Samples) { @@ -378,6 +427,54 @@ func (this *PatternSeries) Equal(that interface{}) bool { } return true } +func (this *PatternSeries_Pattern) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*PatternSeries_Pattern) + if !ok { + that2, ok := that.(PatternSeries_Pattern) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.Pattern != that1.Pattern { + return false + } + return true +} +func (this *PatternSeries_Labels) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*PatternSeries_Labels) + if !ok { + that2, ok := that.(PatternSeries_Labels) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.Labels != that1.Labels { + return false + } + return true +} func (this *PatternSample) Equal(that interface{}) bool { if that == nil { return this == nil @@ -434,15 +531,33 @@ func (this *PatternSeries) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 6) + s := make([]string, 0, 7) s = append(s, "&logproto.PatternSeries{") - s = append(s, "Pattern: "+fmt.Sprintf("%#v", this.Pattern)+",\n") + if this.Identifier != nil { + s = append(s, "Identifier: "+fmt.Sprintf("%#v", this.Identifier)+",\n") + } if this.Samples != nil { s = append(s, "Samples: "+fmt.Sprintf("%#v", this.Samples)+",\n") } s = append(s, "}") return strings.Join(s, "") } +func (this *PatternSeries_Pattern) GoString() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&logproto.PatternSeries_Pattern{` + + `Pattern:` + fmt.Sprintf("%#v", this.Pattern) + `}`}, ", ") + return s +} +func (this *PatternSeries_Labels) GoString() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&logproto.PatternSeries_Labels{` + + `Labels:` + fmt.Sprintf("%#v", this.Labels) + `}`}, ", ") + return s +} func (this *PatternSample) GoString() string { if this == nil { return "nil" @@ -715,6 +830,15 @@ func (m *PatternSeries) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.Identifier != nil { + { + size := m.Identifier.Size() + i -= size + if _, err := m.Identifier.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } + } if len(m.Samples) > 0 { for iNdEx := len(m.Samples) - 1; iNdEx >= 0; iNdEx-- { { @@ -729,16 +853,35 @@ func (m *PatternSeries) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0x12 } } - if len(m.Pattern) > 0 { - i -= len(m.Pattern) - copy(dAtA[i:], m.Pattern) - i = encodeVarintPattern(dAtA, i, uint64(len(m.Pattern))) - i-- - dAtA[i] = 0xa - } return len(dAtA) - i, nil } +func (m *PatternSeries_Pattern) MarshalTo(dAtA []byte) (int, error) { + return m.MarshalToSizedBuffer(dAtA[:m.Size()]) +} + +func (m *PatternSeries_Pattern) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i -= len(m.Pattern) + copy(dAtA[i:], m.Pattern) + i = encodeVarintPattern(dAtA, i, uint64(len(m.Pattern))) + i-- + dAtA[i] = 0xa + return len(dAtA) - i, nil +} +func (m *PatternSeries_Labels) MarshalTo(dAtA []byte) (int, error) { + return m.MarshalToSizedBuffer(dAtA[:m.Size()]) +} + +func (m *PatternSeries_Labels) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + i -= len(m.Labels) + copy(dAtA[i:], m.Labels) + i = encodeVarintPattern(dAtA, i, uint64(len(m.Labels))) + i-- + dAtA[i] = 0x1a + return len(dAtA) - i, nil +} func (m *PatternSample) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -824,9 +967,8 @@ func (m *PatternSeries) Size() (n int) { } var l int _ = l - l = len(m.Pattern) - if l > 0 { - n += 1 + l + sovPattern(uint64(l)) + if m.Identifier != nil { + n += m.Identifier.Size() } if len(m.Samples) > 0 { for _, e := range m.Samples { @@ -837,6 +979,26 @@ func (m *PatternSeries) Size() (n int) { return n } +func (m *PatternSeries_Pattern) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Pattern) + n += 1 + l + sovPattern(uint64(l)) + return n +} +func (m *PatternSeries_Labels) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Labels) + n += 1 + l + sovPattern(uint64(l)) + return n +} func (m *PatternSample) Size() (n int) { if m == nil { return 0 @@ -896,12 +1058,32 @@ func (this *PatternSeries) String() string { } repeatedStringForSamples += "}" s := strings.Join([]string{`&PatternSeries{`, - `Pattern:` + fmt.Sprintf("%v", this.Pattern) + `,`, + `Identifier:` + fmt.Sprintf("%v", this.Identifier) + `,`, `Samples:` + repeatedStringForSamples + `,`, `}`, }, "") return s } +func (this *PatternSeries_Pattern) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&PatternSeries_Pattern{`, + `Pattern:` + fmt.Sprintf("%v", this.Pattern) + `,`, + `}`, + }, "") + return s +} +func (this *PatternSeries_Labels) String() string { + if this == nil { + return "nil" + } + s := strings.Join([]string{`&PatternSeries_Labels{`, + `Labels:` + fmt.Sprintf("%v", this.Labels) + `,`, + `}`, + }, "") + return s +} func (this *PatternSample) String() string { if this == nil { return "nil" @@ -1237,7 +1419,7 @@ func (m *PatternSeries) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Pattern = string(dAtA[iNdEx:postIndex]) + m.Identifier = &PatternSeries_Pattern{string(dAtA[iNdEx:postIndex])} iNdEx = postIndex case 2: if wireType != 2 { @@ -1273,6 +1455,38 @@ func (m *PatternSeries) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Labels", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPattern + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthPattern + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthPattern + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Identifier = &PatternSeries_Labels{string(dAtA[iNdEx:postIndex])} + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipPattern(dAtA[iNdEx:]) diff --git a/pkg/logproto/pattern.proto b/pkg/logproto/pattern.proto index e92a201b3a8b1..fa03742b1af75 100644 --- a/pkg/logproto/pattern.proto +++ b/pkg/logproto/pattern.proto @@ -32,7 +32,10 @@ message QueryPatternsResponse { } message PatternSeries { - string pattern = 1; + oneof identifier { + string pattern = 1; + string labels = 3; + } repeated PatternSample samples = 2; } diff --git a/pkg/logql/evaluator.go b/pkg/logql/evaluator.go index e50d8739c30ad..b44b134c8bdde 100644 --- a/pkg/logql/evaluator.go +++ b/pkg/logql/evaluator.go @@ -410,6 +410,20 @@ type VectorAggEvaluator struct { lb *labels.Builder } +func NewVectorAggEvaluator( + nextEvaluator StepEvaluator, + expr *syntax.VectorAggregationExpr, + buf []byte, + lb *labels.Builder, +) *VectorAggEvaluator { + return &VectorAggEvaluator{ + nextEvaluator: nextEvaluator, + expr: expr, + buf: buf, + lb: lb, + } +} + func (e *VectorAggEvaluator) Next() (bool, int64, StepResult) { next, ts, r := e.nextEvaluator.Next() @@ -684,9 +698,7 @@ func newRangeAggEvaluator( return nil, err } - return &RangeVectorEvaluator{ - iter: iter, - }, nil + return NewRangeVectorEvaluator(iter), nil } } @@ -696,6 +708,12 @@ type RangeVectorEvaluator struct { err error } +func NewRangeVectorEvaluator(iter RangeVectorIterator) *RangeVectorEvaluator { + return &RangeVectorEvaluator{ + iter: iter, + } +} + func (r *RangeVectorEvaluator) Next() (bool, int64, StepResult) { next := r.iter.Next() if !next { diff --git a/pkg/logql/range_vector.go b/pkg/logql/range_vector.go index 44a8651577549..141d4865c46c7 100644 --- a/pkg/logql/range_vector.go +++ b/pkg/logql/range_vector.go @@ -75,21 +75,18 @@ func newRangeVectorIterator( if err != nil { return nil, err } - return &batchRangeVectorIterator{ - iter: it, - step: step, - end: end, - selRange: selRange, - metrics: map[string]labels.Labels{}, - window: map[string]*promql.Series{}, - agg: vectorAggregator, - current: start - step, // first loop iteration will set it to start - offset: offset, - }, nil -} - -//batch - + return NewBatchRangeVectorIterator( + it, + selRange, + step, + start, + end, + offset, + vectorAggregator, + ), nil +} + +// batch type batchRangeVectorIterator struct { iter iter.PeekingSampleIterator selRange, step, end, current, offset int64 @@ -99,6 +96,24 @@ type batchRangeVectorIterator struct { agg BatchRangeVectorAggregator } +func NewBatchRangeVectorIterator( + it iter.PeekingSampleIterator, + selRange, step, start, end, offset int64, + agg BatchRangeVectorAggregator, +) RangeVectorIterator { + return &batchRangeVectorIterator{ + iter: it, + selRange: selRange, + step: step, + end: end, + current: start - step, // first loop iteration will set it to start + offset: offset, + metrics: map[string]labels.Labels{}, + window: map[string]*promql.Series{}, + agg: agg, + } +} + func (r *batchRangeVectorIterator) Next() bool { // slides the range window to the next position r.current = r.current + r.step diff --git a/pkg/loki/loki.go b/pkg/loki/loki.go index b682c4bfaa65c..ce826f0752d4b 100644 --- a/pkg/loki/loki.go +++ b/pkg/loki/loki.go @@ -31,6 +31,7 @@ import ( "google.golang.org/grpc/health/grpc_health_v1" "github.com/grafana/loki/v3/pkg/analytics" + "github.com/grafana/loki/v3/pkg/bloombuild" "github.com/grafana/loki/v3/pkg/bloomcompactor" "github.com/grafana/loki/v3/pkg/bloomgateway" "github.com/grafana/loki/v3/pkg/compactor" @@ -90,6 +91,7 @@ type Config struct { Pattern pattern.Config `yaml:"pattern_ingester,omitempty"` IndexGateway indexgateway.Config `yaml:"index_gateway"` BloomCompactor bloomcompactor.Config `yaml:"bloom_compactor,omitempty" category:"experimental"` + BloomBuild bloombuild.Config `yaml:"bloom_build,omitempty" category:"experimental"` BloomGateway bloomgateway.Config `yaml:"bloom_gateway,omitempty" category:"experimental"` StorageConfig storage.Config `yaml:"storage_config,omitempty"` ChunkStoreConfig config.ChunkStoreConfig `yaml:"chunk_store_config,omitempty"` @@ -173,6 +175,7 @@ func (c *Config) RegisterFlags(f *flag.FlagSet) { c.Tracing.RegisterFlags(f) c.CompactorConfig.RegisterFlags(f) c.BloomCompactor.RegisterFlags(f) + c.BloomBuild.RegisterFlags(f) c.QueryScheduler.RegisterFlags(f) c.Analytics.RegisterFlags(f) c.OperationalConfig.RegisterFlags(f) @@ -328,7 +331,7 @@ type Loki struct { distributor *distributor.Distributor Ingester ingester.Interface PatternIngester *pattern.Ingester - PatternRingClient *pattern.RingClient + PatternRingClient pattern.RingClient Querier querier.Querier cacheGenerationLoader queryrangebase.CacheGenNumberLoader querierAPI *querier.QuerierAPI @@ -649,6 +652,8 @@ func (t *Loki) setupModuleManager() error { mm.RegisterModule(BloomStore, t.initBloomStore) mm.RegisterModule(BloomCompactor, t.initBloomCompactor) mm.RegisterModule(BloomCompactorRing, t.initBloomCompactorRing, modules.UserInvisibleModule) + mm.RegisterModule(BloomPlanner, t.initBloomPlanner) + mm.RegisterModule(BloomBuilder, t.initBloomBuilder) mm.RegisterModule(IndexGateway, t.initIndexGateway) mm.RegisterModule(IndexGatewayRing, t.initIndexGatewayRing, modules.UserInvisibleModule) mm.RegisterModule(IndexGatewayInterceptors, t.initIndexGatewayInterceptors, modules.UserInvisibleModule) @@ -686,6 +691,8 @@ func (t *Loki) setupModuleManager() error { IndexGateway: {Server, Store, BloomStore, IndexGatewayRing, IndexGatewayInterceptors, Analytics}, BloomGateway: {Server, BloomStore, Analytics}, BloomCompactor: {Server, BloomStore, BloomCompactorRing, Analytics, Store}, + BloomPlanner: {Server, BloomStore, Analytics, Store}, + BloomBuilder: {Server, BloomStore, Analytics, Store}, PatternIngester: {Server, MemberlistKV, Analytics}, PatternRingClient: {Server, MemberlistKV, Analytics}, IngesterQuerier: {Ring}, diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index 0280bd514d3c1..e73369aca2d72 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -38,6 +38,8 @@ import ( "github.com/grafana/loki/v3/pkg/storage/types" "github.com/grafana/loki/v3/pkg/analytics" + "github.com/grafana/loki/v3/pkg/bloombuild/builder" + "github.com/grafana/loki/v3/pkg/bloombuild/planner" "github.com/grafana/loki/v3/pkg/bloomgateway" "github.com/grafana/loki/v3/pkg/compactor" compactorclient "github.com/grafana/loki/v3/pkg/compactor/client" @@ -122,6 +124,8 @@ const ( QuerySchedulerRing string = "query-scheduler-ring" BloomCompactor string = "bloom-compactor" BloomCompactorRing string = "bloom-compactor-ring" + BloomPlanner string = "bloom-planner" + BloomBuilder string = "bloom-builder" BloomStore string = "bloom-store" All string = "all" Read string = "read" @@ -803,7 +807,7 @@ func (t *Loki) updateConfigForShipperStore() { t.Cfg.StorageConfig.TSDBShipperConfig.Mode = indexshipper.ModeWriteOnly t.Cfg.StorageConfig.TSDBShipperConfig.IngesterDBRetainPeriod = shipperQuerierIndexUpdateDelay(t.Cfg.StorageConfig.IndexCacheValidity, t.Cfg.StorageConfig.TSDBShipperConfig.ResyncInterval) - case t.Cfg.isTarget(Querier), t.Cfg.isTarget(Ruler), t.Cfg.isTarget(Read), t.Cfg.isTarget(Backend), t.isModuleActive(IndexGateway), t.Cfg.isTarget(BloomCompactor): + case t.Cfg.isTarget(Querier), t.Cfg.isTarget(Ruler), t.Cfg.isTarget(Read), t.Cfg.isTarget(Backend), t.isModuleActive(IndexGateway), t.Cfg.isTarget(BloomCompactor), t.Cfg.isTarget(BloomPlanner), t.Cfg.isTarget(BloomBuilder): // We do not want query to do any updates to index t.Cfg.StorageConfig.BoltDBShipperConfig.Mode = indexshipper.ModeReadOnly t.Cfg.StorageConfig.TSDBShipperConfig.Mode = indexshipper.ModeReadOnly @@ -1553,6 +1557,39 @@ func (t *Loki) initBloomCompactorRing() (services.Service, error) { return t.bloomCompactorRingManager, nil } +func (t *Loki) initBloomPlanner() (services.Service, error) { + if !t.Cfg.BloomBuild.Enabled { + return nil, nil + } + + logger := log.With(util_log.Logger, "component", "bloom-planner") + + return planner.New( + t.Cfg.BloomBuild.Planner, + t.Overrides, + t.Cfg.SchemaConfig, + t.Cfg.StorageConfig, + t.ClientMetrics, + t.BloomStore, + logger, + prometheus.DefaultRegisterer, + ) +} + +func (t *Loki) initBloomBuilder() (services.Service, error) { + if !t.Cfg.BloomBuild.Enabled { + return nil, nil + } + + logger := log.With(util_log.Logger, "component", "bloom-worker") + + return builder.New( + t.Cfg.BloomBuild.Builder, + logger, + prometheus.DefaultRegisterer, + ) +} + func (t *Loki) initQueryScheduler() (services.Service, error) { s, err := scheduler.NewScheduler(t.Cfg.QueryScheduler, t.Overrides, util_log.Logger, t.querySchedulerRingManager, prometheus.DefaultRegisterer, t.Cfg.MetricsNamespace) if err != nil { diff --git a/pkg/pattern/chunk/util.go b/pkg/pattern/chunk/util.go new file mode 100644 index 0000000000000..8cbde3fb0474b --- /dev/null +++ b/pkg/pattern/chunk/util.go @@ -0,0 +1,14 @@ +package chunk + +import ( + "time" + + "github.com/prometheus/common/model" +) + +const ( + TimeResolution = model.Time(int64(time.Second*10) / 1e6) + MaxChunkTime = 1 * time.Hour +) + +func TruncateTimestamp(ts, step model.Time) model.Time { return ts - ts%step } diff --git a/pkg/pattern/drain/chunk.go b/pkg/pattern/drain/chunk.go index 9b1e34e2e3a19..8be0dd64070f8 100644 --- a/pkg/pattern/drain/chunk.go +++ b/pkg/pattern/drain/chunk.go @@ -7,15 +7,12 @@ import ( "github.com/prometheus/common/model" "github.com/grafana/loki/v3/pkg/logproto" + "github.com/grafana/loki/v3/pkg/pattern/chunk" "github.com/grafana/loki/v3/pkg/pattern/iter" ) const ( - TimeResolution = model.Time(int64(time.Second*10) / 1e6) - defaultVolumeSize = 500 - - maxChunkTime = 1 * time.Hour ) type Chunks []Chunk @@ -25,7 +22,7 @@ type Chunk struct { } func newChunk(ts model.Time) Chunk { - maxSize := int(maxChunkTime.Nanoseconds()/TimeResolution.UnixNano()) + 1 + maxSize := int(chunk.MaxChunkTime.Nanoseconds()/chunk.TimeResolution.UnixNano()) + 1 v := Chunk{Samples: make([]logproto.PatternSample, 1, maxSize)} v.Samples[0] = logproto.PatternSample{ Timestamp: ts, @@ -39,11 +36,11 @@ func (c Chunk) spaceFor(ts model.Time) bool { return true } - return ts.Sub(c.Samples[0].Timestamp) < maxChunkTime + return ts.Sub(c.Samples[0].Timestamp) < chunk.MaxChunkTime } // ForRange returns samples with only the values -// in the given range [start:end) and aggregates them by step duration. +// in the given range [start:end] and aggregates them by step duration. // start and end are in milliseconds since epoch. step is a duration in milliseconds. func (c Chunk) ForRange(start, end, step model.Time) []logproto.PatternSample { if len(c.Samples) == 0 { @@ -51,7 +48,7 @@ func (c Chunk) ForRange(start, end, step model.Time) []logproto.PatternSample { } first := c.Samples[0].Timestamp last := c.Samples[len(c.Samples)-1].Timestamp - if start >= end || first >= end || last < start { + if start >= end || first > end || last < start { return nil } var lo int @@ -61,17 +58,18 @@ func (c Chunk) ForRange(start, end, step model.Time) []logproto.PatternSample { }) } hi := len(c.Samples) - if end < last { + + if end <= last { hi = sort.Search(len(c.Samples), func(i int) bool { - return c.Samples[i].Timestamp >= end + return c.Samples[i].Timestamp > end }) } - if step == TimeResolution { + if step == chunk.TimeResolution { return c.Samples[lo:hi] } // Re-scale samples into step-sized buckets - currentStep := truncateTimestamp(c.Samples[lo].Timestamp, step) + currentStep := chunk.TruncateTimestamp(c.Samples[lo].Timestamp, step) aggregatedSamples := make([]logproto.PatternSample, 0, ((c.Samples[hi-1].Timestamp-currentStep)/step)+1) aggregatedSamples = append(aggregatedSamples, logproto.PatternSample{ Timestamp: currentStep, @@ -79,7 +77,7 @@ func (c Chunk) ForRange(start, end, step model.Time) []logproto.PatternSample { }) for _, sample := range c.Samples[lo:hi] { if sample.Timestamp >= currentStep+step { - stepForSample := truncateTimestamp(sample.Timestamp, step) + stepForSample := chunk.TruncateTimestamp(sample.Timestamp, step) for i := currentStep + step; i <= stepForSample; i += step { aggregatedSamples = append(aggregatedSamples, logproto.PatternSample{ Timestamp: i, @@ -95,7 +93,7 @@ func (c Chunk) ForRange(start, end, step model.Time) []logproto.PatternSample { } func (c *Chunks) Add(ts model.Time) { - t := truncateTimestamp(ts, TimeResolution) + t := chunk.TruncateTimestamp(ts, chunk.TimeResolution) if len(*c) == 0 { *c = append(*c, newChunk(t)) @@ -123,9 +121,9 @@ func (c Chunks) Iterator(pattern string, from, through, step model.Time) iter.It if len(samples) == 0 { continue } - iters = append(iters, iter.NewSlice(pattern, samples)) + iters = append(iters, iter.NewPatternSlice(pattern, samples)) } - return iter.NewNonOverlappingIterator(pattern, iters) + return iter.NewNonOverlappingPatternIterator(pattern, iters) } func (c Chunks) samples() []*logproto.PatternSample { @@ -197,5 +195,3 @@ func (c *Chunks) size() int { } return size } - -func truncateTimestamp(ts, step model.Time) model.Time { return ts - ts%step } diff --git a/pkg/pattern/drain/chunk_test.go b/pkg/pattern/drain/chunk_test.go index 4863a6629729a..e404a9b5da779 100644 --- a/pkg/pattern/drain/chunk_test.go +++ b/pkg/pattern/drain/chunk_test.go @@ -9,28 +9,29 @@ import ( "github.com/stretchr/testify/require" "github.com/grafana/loki/v3/pkg/logproto" + "github.com/grafana/loki/v3/pkg/pattern/chunk" ) func TestAdd(t *testing.T) { cks := Chunks{} - cks.Add(TimeResolution + 1) - cks.Add(TimeResolution + 2) - cks.Add(2*TimeResolution + 1) + cks.Add(chunk.TimeResolution + 1) + cks.Add(chunk.TimeResolution + 2) + cks.Add(2*chunk.TimeResolution + 1) require.Equal(t, 1, len(cks)) require.Equal(t, 2, len(cks[0].Samples)) - cks.Add(model.TimeFromUnixNano(time.Hour.Nanoseconds()) + TimeResolution + 1) + cks.Add(model.TimeFromUnixNano(time.Hour.Nanoseconds()) + chunk.TimeResolution + 1) require.Equal(t, 2, len(cks)) require.Equal(t, 1, len(cks[1].Samples)) } func TestIterator(t *testing.T) { cks := Chunks{} - cks.Add(TimeResolution + 1) - cks.Add(TimeResolution + 2) - cks.Add(2*TimeResolution + 1) - cks.Add(model.TimeFromUnixNano(time.Hour.Nanoseconds()) + TimeResolution + 1) + cks.Add(chunk.TimeResolution + 1) + cks.Add(chunk.TimeResolution + 2) + cks.Add(2*chunk.TimeResolution + 1) + cks.Add(model.TimeFromUnixNano(time.Hour.Nanoseconds()) + chunk.TimeResolution + 1) - it := cks.Iterator("test", model.Time(0), model.Time(time.Hour.Nanoseconds()), TimeResolution) + it := cks.Iterator("test", model.Time(0), model.Time(time.Hour.Nanoseconds()), chunk.TimeResolution) require.NotNil(t, it) var samples []logproto.PatternSample @@ -137,7 +138,7 @@ func TestForRange(t *testing.T) { }, }, { - name: "Start and End Before First Element", + name: "Start before First and End Inclusive of First Element", c: &Chunk{Samples: []logproto.PatternSample{ {Timestamp: 2, Value: 2}, {Timestamp: 4, Value: 4}, @@ -145,6 +146,17 @@ func TestForRange(t *testing.T) { }}, start: 0, end: 2, + expected: []logproto.PatternSample{{Timestamp: 2, Value: 2}}, + }, + { + name: "Start and End before First Element", + c: &Chunk{Samples: []logproto.PatternSample{ + {Timestamp: 2, Value: 2}, + {Timestamp: 4, Value: 4}, + {Timestamp: 6, Value: 6}, + }}, + start: 0, + end: 1, expected: nil, }, { diff --git a/pkg/pattern/flush_test.go b/pkg/pattern/flush_test.go index 9ee4bd436992b..d2ed2384522d3 100644 --- a/pkg/pattern/flush_test.go +++ b/pkg/pattern/flush_test.go @@ -62,7 +62,7 @@ func TestSweepInstance(t *testing.T) { End: time.Unix(0, math.MaxInt64), }) require.NoError(t, err) - res, err := iter.ReadAll(it) + res, err := iter.ReadAllWithPatterns(it) require.NoError(t, err) require.Equal(t, 2, len(res.Series)) ing.sweepUsers(true, true) @@ -72,7 +72,7 @@ func TestSweepInstance(t *testing.T) { End: time.Unix(0, math.MaxInt64), }) require.NoError(t, err) - res, err = iter.ReadAll(it) + res, err = iter.ReadAllWithPatterns(it) require.NoError(t, err) require.Equal(t, 1, len(res.Series)) } diff --git a/pkg/pattern/ingester.go b/pkg/pattern/ingester.go index 1cb91a1cda299..a3d6c4ef67a0f 100644 --- a/pkg/pattern/ingester.go +++ b/pkg/pattern/ingester.go @@ -12,6 +12,7 @@ import ( "github.com/go-kit/log" "github.com/go-kit/log/level" + "github.com/grafana/dskit/httpgrpc" "github.com/grafana/dskit/ring" "github.com/grafana/dskit/services" "github.com/grafana/dskit/tenant" @@ -21,8 +22,10 @@ import ( ring_client "github.com/grafana/dskit/ring/client" "github.com/grafana/loki/v3/pkg/logproto" + "github.com/grafana/loki/v3/pkg/logql/syntax" "github.com/grafana/loki/v3/pkg/pattern/clientpool" "github.com/grafana/loki/v3/pkg/pattern/iter" + "github.com/grafana/loki/v3/pkg/pattern/metric" "github.com/grafana/loki/v3/pkg/util" util_log "github.com/grafana/loki/v3/pkg/util/log" ) @@ -36,6 +39,7 @@ type Config struct { ConcurrentFlushes int `yaml:"concurrent_flushes"` FlushCheckPeriod time.Duration `yaml:"flush_check_period"` + MetricAggregation metric.AggregationConfig `yaml:"metric_aggregation,omitempty" doc:"description=Configures the metric aggregation and storage behavior of the pattern ingester."` // For testing. factory ring_client.PoolFactory `yaml:"-"` } @@ -47,6 +51,8 @@ func (cfg *Config) RegisterFlags(fs *flag.FlagSet) { fs.BoolVar(&cfg.Enabled, "pattern-ingester.enabled", false, "Flag to enable or disable the usage of the pattern-ingester component.") fs.IntVar(&cfg.ConcurrentFlushes, "pattern-ingester.concurrent-flushes", 32, "How many flushes can happen concurrently from each stream.") fs.DurationVar(&cfg.FlushCheckPeriod, "pattern-ingester.flush-check-period", 30*time.Second, "How often should the ingester see if there are any blocks to flush. The first flush check is delayed by a random time up to 0.8x the flush check period. Additionally, there is +/- 1% jitter added to the interval.") + + cfg.MetricAggregation.RegisterFlagsWithPrefix(fs, "pattern-ingester.") } func (cfg *Config) Validate() error { @@ -238,17 +244,77 @@ func (i *Ingester) Query(req *logproto.QueryPatternsRequest, stream logproto.Pat if err != nil { return err } - iterator, err := instance.Iterator(ctx, req) - if err != nil { - return err + + expr, err := syntax.ParseExpr(req.Query) + + switch e := expr.(type) { + case syntax.SampleExpr: + var err error + iterator, err := instance.QuerySample(ctx, e, req) // this is returning a first value of 0,0 + if err != nil { + return err + } + + // TODO(twhitney): query store + // if start, end, ok := buildStoreRequest(i.cfg, req.Start, req.End, time.Now()); ok { + // storeReq := logql.SelectSampleParams{SampleQueryRequest: &logproto.SampleQueryRequest{ + // Start: start, + // End: end, + // Selector: req.Selector, + // Shards: req.Shards, + // Deletes: req.Deletes, + // Plan: req.Plan, + // }} + // storeItr, err := i.store.SelectSamples(ctx, storeReq) + // if err != nil { + // util.LogErrorWithContext(ctx, "closing iterator", it.Close) + // return err + // } + + // it = iter.NewMergeSampleIterator(ctx, []iter.SampleIterator{it, storeItr}) + // } + + defer util.LogErrorWithContext(ctx, "closing iterator", iterator.Close) + return sendMetricsSample(ctx, iterator, stream) + case syntax.LogSelectorExpr: + var err error + iterator, err := instance.Iterator(ctx, req) + if err != nil { + return err + } + defer util.LogErrorWithContext(ctx, "closing iterator", iterator.Close) + return sendPatternSample(ctx, iterator, stream) + default: + return httpgrpc.Errorf( + http.StatusBadRequest, + fmt.Sprintf("unexpected type (%T): cannot evaluate", e), + ) } - defer util.LogErrorWithContext(ctx, "closing iterator", iterator.Close) - return sendPatternSample(ctx, iterator, stream) } func sendPatternSample(ctx context.Context, it iter.Iterator, stream logproto.Pattern_QueryServer) error { for ctx.Err() == nil { - batch, err := iter.ReadBatch(it, readBatchSize) + batch, err := iter.ReadPatternsBatch(it, readBatchSize) + if err != nil { + return err + } + if err := stream.Send(batch); err != nil && err != context.Canceled { + return err + } + if len(batch.Series) == 0 { + return nil + } + } + return nil +} + +func sendMetricsSample( + ctx context.Context, + it iter.Iterator, + stream logproto.Pattern_QueryServer, +) error { + for ctx.Err() == nil { + batch, err := iter.ReadMetricsBatch(it, readBatchSize) if err != nil { return err } @@ -273,7 +339,7 @@ func (i *Ingester) GetOrCreateInstance(instanceID string) (*instance, error) { / inst, ok = i.instances[instanceID] if !ok { var err error - inst, err = newInstance(instanceID, i.logger, i.metrics) + inst, err = newInstance(instanceID, i.logger, i.metrics, i.cfg.MetricAggregation) if err != nil { return nil, err } diff --git a/pkg/pattern/ingester_querier.go b/pkg/pattern/ingester_querier.go index 13315b0a13f1a..fc62c7cfdca47 100644 --- a/pkg/pattern/ingester_querier.go +++ b/pkg/pattern/ingester_querier.go @@ -2,11 +2,10 @@ package pattern import ( "context" + "errors" "math" - "net/http" "github.com/go-kit/log" - "github.com/grafana/dskit/httpgrpc" "github.com/grafana/dskit/ring" "github.com/prometheus/client_golang/prometheus" @@ -19,18 +18,20 @@ import ( // TODO(kolesnikovae): parametrise QueryPatternsRequest const minClusterSize = 30 +var ErrParseQuery = errors.New("only label matcher, byte_over_time, and count_over_time queries without filters are supported") + type IngesterQuerier struct { cfg Config logger log.Logger - ringClient *RingClient + ringClient RingClient registerer prometheus.Registerer } func NewIngesterQuerier( cfg Config, - ringClient *RingClient, + ringClient RingClient, metricsNamespace string, registerer prometheus.Registerer, logger log.Logger, @@ -46,8 +47,68 @@ func NewIngesterQuerier( func (q *IngesterQuerier) Patterns(ctx context.Context, req *logproto.QueryPatternsRequest) (*logproto.QueryPatternsResponse, error) { _, err := syntax.ParseMatchers(req.Query, true) if err != nil { - return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error()) + // not a pattern query, so either a metric query or an error + if q.cfg.MetricAggregation.Enabled { + return q.queryMetricSamples(ctx, req) + } + + return nil, err + } + + return q.queryPatternSamples(ctx, req) +} + +func (q *IngesterQuerier) queryPatternSamples(ctx context.Context, req *logproto.QueryPatternsRequest) (*logproto.QueryPatternsResponse, error) { + iterators, err := q.query(ctx, req) + if err != nil { + return nil, err + } + + // TODO(kolesnikovae): Incorporate with pruning + resp, err := iter.ReadPatternsBatch(iter.NewMerge(iterators...), math.MaxInt32) + if err != nil { + return nil, err + } + return prunePatterns(resp, minClusterSize), nil +} + +func (q *IngesterQuerier) queryMetricSamples(ctx context.Context, req *logproto.QueryPatternsRequest) (*logproto.QueryPatternsResponse, error) { + expr, err := syntax.ParseSampleExpr(req.Query) + if err != nil { + return nil, err + } + + var selector syntax.LogSelectorExpr + switch expr.(type) { + case *syntax.VectorAggregationExpr: + selector, err = expr.(*syntax.VectorAggregationExpr).Selector() + case *syntax.RangeAggregationExpr: + selector, err = expr.(*syntax.RangeAggregationExpr).Selector() + default: + return nil, ErrParseQuery + } + + if err != nil { + return nil, err + } + + if selector == nil || selector.HasFilter() { + return nil, ErrParseQuery + } + + iterators, err := q.query(ctx, req) + if err != nil { + return nil, err + } + + resp, err := iter.ReadMetricsBatch(iter.NewMerge(iterators...), math.MaxInt32) + if err != nil { + return nil, err } + return resp, nil +} + +func (q *IngesterQuerier) query(ctx context.Context, req *logproto.QueryPatternsRequest) ([]iter.Iterator, error) { resps, err := q.forAllIngesters(ctx, func(_ context.Context, client logproto.PatternClient) (interface{}, error) { return client.Query(ctx, req) }) @@ -58,18 +119,16 @@ func (q *IngesterQuerier) Patterns(ctx context.Context, req *logproto.QueryPatte for i := range resps { iterators[i] = iter.NewQueryClientIterator(resps[i].response.(logproto.Pattern_QueryClient)) } - // TODO(kolesnikovae): Incorporate with pruning - resp, err := iter.ReadBatch(iter.NewMerge(iterators...), math.MaxInt32) - if err != nil { - return nil, err - } - return prunePatterns(resp, minClusterSize), nil + return iterators, nil } -func prunePatterns(resp *logproto.QueryPatternsResponse, minClusterSize int) *logproto.QueryPatternsResponse { +func prunePatterns( + resp *logproto.QueryPatternsResponse, + minClusterSize int, +) *logproto.QueryPatternsResponse { d := drain.New(drain.DefaultConfig(), nil) for _, p := range resp.Series { - d.TrainPattern(p.Pattern, p.Samples) + d.TrainPattern(p.GetPattern(), p.Samples) } resp.Series = resp.Series[:0] @@ -81,17 +140,15 @@ func prunePatterns(resp *logproto.QueryPatternsResponse, minClusterSize int) *lo if pattern == "" { continue } - resp.Series = append(resp.Series, &logproto.PatternSeries{ - Pattern: pattern, - Samples: cluster.Samples(), - }) + resp.Series = append(resp.Series, + logproto.NewPatternSeriesWithPattern(pattern, cluster.Samples())) } return resp } // ForAllIngesters runs f, in parallel, for all ingesters func (q *IngesterQuerier) forAllIngesters(ctx context.Context, f func(context.Context, logproto.PatternClient) (interface{}, error)) ([]ResponseFromIngesters, error) { - replicationSet, err := q.ringClient.ring.GetReplicationSetForOperation(ring.Read) + replicationSet, err := q.ringClient.Ring().GetReplicationSetForOperation(ring.Read) if err != nil { return nil, err } @@ -110,7 +167,7 @@ func (q *IngesterQuerier) forGivenIngesters(ctx context.Context, replicationSet // Nothing here } results, err := ring.DoUntilQuorum(ctx, replicationSet, cfg, func(ctx context.Context, ingester *ring.InstanceDesc) (ResponseFromIngesters, error) { - client, err := q.ringClient.pool.GetClientFor(ingester.Addr) + client, err := q.ringClient.Pool().GetClientFor(ingester.Addr) if err != nil { return ResponseFromIngesters{addr: ingester.Addr}, err } diff --git a/pkg/pattern/ingester_querier_test.go b/pkg/pattern/ingester_querier_test.go index d1016b326df73..20b3dcae83d7a 100644 --- a/pkg/pattern/ingester_querier_test.go +++ b/pkg/pattern/ingester_querier_test.go @@ -2,12 +2,19 @@ package pattern import ( "bufio" + "context" "os" "testing" + "time" + "github.com/go-kit/log" "github.com/stretchr/testify/require" + "github.com/grafana/dskit/ring" + ring_client "github.com/grafana/dskit/ring/client" + "github.com/grafana/dskit/services" "github.com/grafana/loki/v3/pkg/logproto" + "github.com/grafana/loki/v3/pkg/pattern/metric" ) func Test_prunePatterns(t *testing.T) { @@ -18,9 +25,7 @@ func Test_prunePatterns(t *testing.T) { resp := new(logproto.QueryPatternsResponse) scanner := bufio.NewScanner(file) for scanner.Scan() { - resp.Series = append(resp.Series, &logproto.PatternSeries{ - Pattern: scanner.Text(), - }) + resp.Series = append(resp.Series, logproto.NewPatternSeriesWithPattern(scanner.Text(), []*logproto.PatternSample{})) } require.NoError(t, scanner.Err()) prunePatterns(resp, 0) @@ -36,8 +41,168 @@ func Test_prunePatterns(t *testing.T) { patterns := make([]string, 0, len(resp.Series)) for _, p := range resp.Series { - patterns = append(patterns, p.Pattern) + patterns = append(patterns, p.GetPattern()) } require.Equal(t, expectedPatterns, patterns) } + +func Test_Patterns(t *testing.T) { + t.Run("it rejects metric queries with filters", func(t *testing.T) { + q := &IngesterQuerier{ + cfg: Config{ + MetricAggregation: metric.AggregationConfig{ + Enabled: true, + }, + }, + logger: log.NewNopLogger(), + ringClient: &fakeRingClient{}, + registerer: nil, + } + for _, query := range []string{ + `count_over_time({foo="bar"} |= "baz" [5m])`, + `count_over_time({foo="bar"} != "baz" [5m])`, + `count_over_time({foo="bar"} |~ "baz" [5m])`, + `count_over_time({foo="bar"} !~ "baz" [5m])`, + `count_over_time({foo="bar"} | logfmt | color="blue" [5m])`, + `sum(count_over_time({foo="bar"} |= "baz" [5m]))`, + `sum by (label)(count_over_time({foo="bar"} |= "baz" [5m]))`, + `bytes_over_time({foo="bar"} |= "baz" [5m])`, + } { + _, err := q.Patterns( + context.Background(), + &logproto.QueryPatternsRequest{ + Query: query, + }, + ) + require.Error(t, err, query) + require.ErrorIs(t, err, ErrParseQuery, query) + + } + }) + + t.Run("accepts log selector queries and count and bytes metric queries", func(t *testing.T) { + q := &IngesterQuerier{ + cfg: Config{ + MetricAggregation: metric.AggregationConfig{ + Enabled: true, + }, + }, + logger: log.NewNopLogger(), + ringClient: &fakeRingClient{}, + registerer: nil, + } + for _, query := range []string{ + `{foo="bar"}`, + `count_over_time({foo="bar"}[5m])`, + `bytes_over_time({foo="bar"}[5m])`, + `sum(count_over_time({foo="bar"}[5m]))`, + `sum(bytes_over_time({foo="bar"}[5m]))`, + `sum by (level)(count_over_time({foo="bar"}[5m]))`, + `sum by (level)(bytes_over_time({foo="bar"}[5m]))`, + } { + _, err := q.Patterns( + context.Background(), + &logproto.QueryPatternsRequest{ + Query: query, + }, + ) + require.NoError(t, err, query) + } + }) +} + +type fakeRingClient struct{} + +func (f *fakeRingClient) Pool() *ring_client.Pool { + panic("not implemented") +} + +func (f *fakeRingClient) StartAsync(ctx context.Context) error { + panic("not implemented") +} + +func (f *fakeRingClient) AwaitRunning(ctx context.Context) error { + panic("not implemented") +} + +func (f *fakeRingClient) StopAsync() { + panic("not implemented") +} + +func (f *fakeRingClient) AwaitTerminated(ctx context.Context) error { + panic("not implemented") +} + +func (f *fakeRingClient) FailureCase() error { + panic("not implemented") +} + +func (f *fakeRingClient) State() services.State { + panic("not implemented") +} + +func (f *fakeRingClient) AddListener(listener services.Listener) { + panic("not implemented") +} + +func (f *fakeRingClient) Ring() ring.ReadRing { + return &fakeRing{} +} + +type fakeRing struct{} + +func (f *fakeRing) Get( + key uint32, + op ring.Operation, + bufDescs []ring.InstanceDesc, + bufHosts []string, + bufZones []string, +) (ring.ReplicationSet, error) { + panic("not implemented") +} + +func (f *fakeRing) GetAllHealthy(op ring.Operation) (ring.ReplicationSet, error) { + panic("not implemented") +} + +func (f *fakeRing) GetReplicationSetForOperation(op ring.Operation) (ring.ReplicationSet, error) { + return ring.ReplicationSet{}, nil +} + +func (f *fakeRing) ReplicationFactor() int { + panic("not implemented") +} + +func (f *fakeRing) InstancesCount() int { + panic("not implemented") +} + +func (f *fakeRing) ShuffleShard(identifier string, size int) ring.ReadRing { + panic("not implemented") +} + +func (f *fakeRing) GetInstanceState(instanceID string) (ring.InstanceState, error) { + panic("not implemented") +} + +func (f *fakeRing) ShuffleShardWithLookback( + identifier string, + size int, + lookbackPeriod time.Duration, + now time.Time, +) ring.ReadRing { + panic("not implemented") +} + +func (f *fakeRing) HasInstance(instanceID string) bool { + panic("not implemented") +} + +func (f *fakeRing) CleanupShuffleShardCache(identifier string) { + panic("not implemented") +} + +func (f *fakeRing) GetTokenRangesForInstance(instanceID string) (ring.TokenRanges, error) { + panic("not implemented") +} diff --git a/pkg/pattern/ingester_test.go b/pkg/pattern/ingester_test.go index eff054b9ec041..9707837a4fabc 100644 --- a/pkg/pattern/ingester_test.go +++ b/pkg/pattern/ingester_test.go @@ -11,53 +11,269 @@ import ( "github.com/stretchr/testify/require" "github.com/grafana/loki/v3/pkg/logproto" + "github.com/grafana/loki/v3/pkg/logql/syntax" "github.com/grafana/loki/v3/pkg/pattern/iter" + "github.com/grafana/loki/v3/pkg/pattern/metric" "github.com/grafana/loki/pkg/push" ) func TestInstancePushQuery(t *testing.T) { lbs := labels.New(labels.Label{Name: "test", Value: "test"}) - inst, err := newInstance("foo", log.NewNopLogger(), newIngesterMetrics(nil, "test")) - require.NoError(t, err) - - err = inst.Push(context.Background(), &push.PushRequest{ - Streams: []push.Stream{ - { - Labels: lbs.String(), - Entries: []push.Entry{ + setup := func() *instance { + inst, err := newInstance("foo", log.NewNopLogger(), newIngesterMetrics(nil, "test"), metric.AggregationConfig{ + Enabled: true, + }) + require.NoError(t, err) + + return inst + } + t.Run("test pattern samples", func(t *testing.T) { + inst := setup() + err := inst.Push(context.Background(), &push.PushRequest{ + Streams: []push.Stream{ + { + Labels: lbs.String(), + Entries: []push.Entry{ + { + Timestamp: time.Unix(20, 0), + Line: "ts=1 msg=hello", + }, + }, + }, + }, + }) + + err = inst.Push(context.Background(), &push.PushRequest{ + Streams: []push.Stream{ + { + Labels: lbs.String(), + Entries: []push.Entry{ + { + Timestamp: time.Unix(30, 0), + Line: "ts=2 msg=hello", + }, + }, + }, + }, + }) + for i := 0; i <= 30; i++ { + err = inst.Push(context.Background(), &push.PushRequest{ + Streams: []push.Stream{ { - Timestamp: time.Unix(20, 0), - Line: "ts=1 msg=hello", + Labels: lbs.String(), + Entries: []push.Entry{ + { + Timestamp: time.Unix(30, 0), + Line: "foo bar foo bar", + }, + }, + }, + }, + }) + require.NoError(t, err) + } + require.NoError(t, err) + it, err := inst.Iterator(context.Background(), &logproto.QueryPatternsRequest{ + Query: "{test=\"test\"}", + Start: time.Unix(0, 0), + End: time.Unix(0, math.MaxInt64), + }) + require.NoError(t, err) + res, err := iter.ReadAllWithPatterns(it) + require.NoError(t, err) + require.Equal(t, 2, len(res.Series)) + + it, err = inst.Iterator(context.Background(), &logproto.QueryPatternsRequest{ + Query: "{test=\"test\"}", + Start: time.Unix(0, 0), + End: time.Unix(30, 0), + }) + require.NoError(t, err) + res, err = iter.ReadAllWithPatterns(it) + require.NoError(t, err) + // query should be inclusive of end time to match our + // existing metric query behavior + require.Equal(t, 2, len(res.Series)) + require.Equal(t, 2, len(res.Series[0].Samples)) + }) + + t.Run("test count_over_time samples", func(t *testing.T) { + inst := setup() + err := inst.Push(context.Background(), &push.PushRequest{ + Streams: []push.Stream{ + { + Labels: lbs.String(), + Entries: []push.Entry{ + { + Timestamp: time.Unix(0, 0), + Line: "ts=1 msg=hello", + }, }, }, }, - }, + }) + for i := 1; i <= 30; i++ { + err = inst.Push(context.Background(), &push.PushRequest{ + Streams: []push.Stream{ + { + Labels: lbs.String(), + Entries: []push.Entry{ + { + Timestamp: time.Unix(int64(20*i), 0), + Line: "foo bar foo bar", + }, + }, + }, + }, + }) + require.NoError(t, err) + } + require.NoError(t, err) + + expr, err := syntax.ParseSampleExpr(`count_over_time({test="test"}[20s])`) + require.NoError(t, err) + + it, err := inst.QuerySample(context.Background(), expr, &logproto.QueryPatternsRequest{ + Query: expr.String(), + Start: time.Unix(0, 0), + End: time.Unix(int64(20*30), 0), + Step: 10000, + }) + require.NoError(t, err) + res, err := iter.ReadAllWithLabels(it) + require.NoError(t, err) + require.Equal(t, 1, len(res.Series)) + + require.Equal(t, lbs.String(), res.Series[0].GetLabels()) + + // end - start / step -- (start is 0, step is 10s) + // plus one because end is actually inclusive for metric queries + expectedDataPoints := ((20 * 30) / 10) + 1 + require.Equal(t, expectedDataPoints, len(res.Series[0].Samples)) + require.Equal(t, int64(1), res.Series[0].Samples[0].Value) + + expr, err = syntax.ParseSampleExpr(`count_over_time({test="test"}[80s])`) + require.NoError(t, err) + + it, err = inst.QuerySample(context.Background(), expr, &logproto.QueryPatternsRequest{ + Query: expr.String(), + Start: time.Unix(0, 0), + End: time.Unix(int64(20*30), 0), + Step: 10000, + }) + require.NoError(t, err) + res, err = iter.ReadAllWithLabels(it) + require.NoError(t, err) + require.Equal(t, 1, len(res.Series)) + + require.Equal(t, lbs.String(), res.Series[0].GetLabels()) + + // end - start / step -- (start is 0, step is 10s) + // plus one because end is actually inclusive for metric queries + expectedDataPoints = ((20 * 30) / 10) + 1 + require.Equal(t, expectedDataPoints, len(res.Series[0].Samples)) + + // with a larger selection range of 80s, we expect to eventually get up to 4 per datapoint + // our pushes are spaced 20s apart, and there's 10s step, so we ecpect to see the value increase + // every 2 samples, maxing out and staying at 4 after 6 samples (since it starts a 1, not 0) + require.Equal(t, int64(1), res.Series[0].Samples[0].Value) + require.Equal(t, int64(1), res.Series[0].Samples[1].Value) + require.Equal(t, int64(2), res.Series[0].Samples[2].Value) + require.Equal(t, int64(2), res.Series[0].Samples[3].Value) + require.Equal(t, int64(3), res.Series[0].Samples[4].Value) + require.Equal(t, int64(3), res.Series[0].Samples[5].Value) + require.Equal(t, int64(4), res.Series[0].Samples[6].Value) + require.Equal(t, int64(4), res.Series[0].Samples[expectedDataPoints-1].Value) }) - for i := 0; i <= 30; i++ { - err = inst.Push(context.Background(), &push.PushRequest{ + + t.Run("test bytes_over_time samples", func(t *testing.T) { + inst := setup() + err := inst.Push(context.Background(), &push.PushRequest{ Streams: []push.Stream{ { Labels: lbs.String(), Entries: []push.Entry{ { - Timestamp: time.Unix(20, 0), - Line: "foo bar foo bar", + Timestamp: time.Unix(0, 0), + Line: "foo bar foo bars", }, }, }, }, }) + for i := 1; i <= 30; i++ { + err = inst.Push(context.Background(), &push.PushRequest{ + Streams: []push.Stream{ + { + Labels: lbs.String(), + Entries: []push.Entry{ + { + Timestamp: time.Unix(int64(20*i), 0), + Line: "foo bar foo bars", + }, + }, + }, + }, + }) + require.NoError(t, err) + } require.NoError(t, err) - } - require.NoError(t, err) - it, err := inst.Iterator(context.Background(), &logproto.QueryPatternsRequest{ - Query: "{test=\"test\"}", - Start: time.Unix(0, 0), - End: time.Unix(0, math.MaxInt64), + + expr, err := syntax.ParseSampleExpr(`bytes_over_time({test="test"}[20s])`) + require.NoError(t, err) + + it, err := inst.QuerySample(context.Background(), expr, &logproto.QueryPatternsRequest{ + Query: expr.String(), + Start: time.Unix(0, 0), + End: time.Unix(int64(20*30), 0), + Step: 10000, + }) + require.NoError(t, err) + res, err := iter.ReadAllWithLabels(it) + require.NoError(t, err) + require.Equal(t, 1, len(res.Series)) + + require.Equal(t, lbs.String(), res.Series[0].GetLabels()) + + // end - start / step -- (start is 0, step is 10s) + // plus one because end is actually inclusive for metric queries + expectedDataPoints := ((20 * 30) / 10) + 1 + require.Equal(t, expectedDataPoints, len(res.Series[0].Samples)) + require.Equal(t, int64(16), res.Series[0].Samples[0].Value) + + expr, err = syntax.ParseSampleExpr(`bytes_over_time({test="test"}[80s])`) + require.NoError(t, err) + + it, err = inst.QuerySample(context.Background(), expr, &logproto.QueryPatternsRequest{ + Query: expr.String(), + Start: time.Unix(0, 0), + End: time.Unix(int64(20*30), 0), + Step: 10000, + }) + require.NoError(t, err) + res, err = iter.ReadAllWithLabels(it) + require.NoError(t, err) + require.Equal(t, 1, len(res.Series)) + + require.Equal(t, lbs.String(), res.Series[0].GetLabels()) + + // end - start / step -- (start is 0, step is 10s) + // plus one because end is actually inclusive for metric queries + expectedDataPoints = ((20 * 30) / 10) + 1 + require.Equal(t, expectedDataPoints, len(res.Series[0].Samples)) + + // with a larger selection range of 80s, we expect to eventually get up to 64 bytes + // as each pushe is 16 bytes and are spaced 20s apart. We query with 10s step, + // so we ecpect to see the value increase by 16 bytes every 2 samples, + // maxing out and staying at 64 after 6 samples (since it starts a 1, not 0) + require.Equal(t, int64(16), res.Series[0].Samples[0].Value) + require.Equal(t, int64(16), res.Series[0].Samples[1].Value) + require.Equal(t, int64(32), res.Series[0].Samples[2].Value) + require.Equal(t, int64(32), res.Series[0].Samples[3].Value) + require.Equal(t, int64(48), res.Series[0].Samples[4].Value) + require.Equal(t, int64(48), res.Series[0].Samples[5].Value) + require.Equal(t, int64(64), res.Series[0].Samples[6].Value) + require.Equal(t, int64(64), res.Series[0].Samples[expectedDataPoints-1].Value) }) - require.NoError(t, err) - res, err := iter.ReadAll(it) - require.NoError(t, err) - require.Equal(t, 2, len(res.Series)) } diff --git a/pkg/pattern/instance.go b/pkg/pattern/instance.go index f6efa7de04435..d98b412a85e24 100644 --- a/pkg/pattern/instance.go +++ b/pkg/pattern/instance.go @@ -15,8 +15,9 @@ import ( "github.com/grafana/loki/v3/pkg/ingester/index" "github.com/grafana/loki/v3/pkg/logproto" "github.com/grafana/loki/v3/pkg/logql/syntax" - "github.com/grafana/loki/v3/pkg/pattern/drain" + "github.com/grafana/loki/v3/pkg/pattern/chunk" "github.com/grafana/loki/v3/pkg/pattern/iter" + "github.com/grafana/loki/v3/pkg/pattern/metric" "github.com/grafana/loki/v3/pkg/util" ) @@ -24,27 +25,29 @@ const indexShards = 32 // instance is a tenant instance of the pattern ingester. type instance struct { - instanceID string - buf []byte // buffer used to compute fps. - mapper *ingester.FpMapper // using of mapper no longer needs mutex because reading from streams is lock-free - streams *streamsMap - index *index.BitPrefixInvertedIndex - logger log.Logger - metrics *ingesterMetrics + instanceID string + buf []byte // buffer used to compute fps. + mapper *ingester.FpMapper // using of mapper no longer needs mutex because reading from streams is lock-free + streams *streamsMap + index *index.BitPrefixInvertedIndex + logger log.Logger + metrics *ingesterMetrics + aggregationCfg metric.AggregationConfig } -func newInstance(instanceID string, logger log.Logger, metrics *ingesterMetrics) (*instance, error) { +func newInstance(instanceID string, logger log.Logger, metrics *ingesterMetrics, aggCfg metric.AggregationConfig) (*instance, error) { index, err := index.NewBitPrefixWithShards(indexShards) if err != nil { return nil, err } i := &instance{ - buf: make([]byte, 0, 1024), - logger: logger, - instanceID: instanceID, - streams: newStreamsMap(), - index: index, - metrics: metrics, + buf: make([]byte, 0, 1024), + logger: logger, + instanceID: instanceID, + streams: newStreamsMap(), + index: index, + metrics: metrics, + aggregationCfg: aggCfg, } i.mapper = ingester.NewFPMapper(i.getLabelsFromFingerprint) return i, nil @@ -59,7 +62,7 @@ func (i *instance) Push(ctx context.Context, req *logproto.PushRequest) error { s, _, err := i.streams.LoadOrStoreNew(reqStream.Labels, func() (*stream, error) { // add stream - return i.createStream(ctx, reqStream) + return i.createStream(ctx, reqStream, i.aggregationCfg.Enabled) }, nil) if err != nil { appendErr.Add(err) @@ -82,8 +85,8 @@ func (i *instance) Iterator(ctx context.Context, req *logproto.QueryPatternsRequ } from, through := util.RoundToMilliseconds(req.Start, req.End) step := model.Time(req.Step) - if step < drain.TimeResolution { - step = drain.TimeResolution + if step < chunk.TimeResolution { + step = chunk.TimeResolution } var iters []iter.Iterator @@ -101,6 +104,61 @@ func (i *instance) Iterator(ctx context.Context, req *logproto.QueryPatternsRequ return iter.NewMerge(iters...), nil } +func (i *instance) QuerySample( + ctx context.Context, + expr syntax.SampleExpr, + req *logproto.QueryPatternsRequest, +) (iter.Iterator, error) { + if !i.aggregationCfg.Enabled { + // Should never get here, but this will prevent nil pointer panics in test + return iter.Empty, nil + } + + from, through := util.RoundToMilliseconds(req.Start, req.End) + step := model.Time(req.Step) + if step < chunk.TimeResolution { + step = chunk.TimeResolution + } + + selector, err := expr.Selector() + if err != nil { + return nil, err + } + + typ, err := metric.ExtractMetricType(expr) + if err != nil || typ == metric.Unsupported { + return nil, err + } + + var iters []iter.Iterator + err = i.forMatchingStreams( + selector.Matchers(), + func(stream *stream) error { + var iter iter.Iterator + var err error + if typ == metric.Bytes { + iter, err = stream.BytesIterator(ctx, expr, from, through, step) + } else if typ == metric.Count { + iter, err = stream.CountIterator(ctx, expr, from, through, step) + } else { + return fmt.Errorf("unsupported query operation") + } + + if err != nil { + return err + } + + iters = append(iters, iter) + return nil + }, + ) + if err != nil { + return nil, err + } + + return iter.NewMerge(iters...), nil +} + // forMatchingStreams will execute a function for each stream that matches the given matchers. func (i *instance) forMatchingStreams( matchers []*labels.Matcher, @@ -133,14 +191,14 @@ outer: return nil } -func (i *instance) createStream(_ context.Context, pushReqStream logproto.Stream) (*stream, error) { +func (i *instance) createStream(_ context.Context, pushReqStream logproto.Stream, aggregateMetrics bool) (*stream, error) { labels, err := syntax.ParseLabels(pushReqStream.Labels) if err != nil { return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error()) } fp := i.getHashForLabels(labels) sortedLabels := i.index.Add(logproto.FromLabelsToLabelAdapters(labels), fp) - s, err := newStream(fp, sortedLabels, i.metrics) + s, err := newStream(fp, sortedLabels, i.metrics, i.aggregationCfg.Enabled) if err != nil { return nil, fmt.Errorf("failed to create stream: %w", err) } diff --git a/pkg/pattern/instance_test.go b/pkg/pattern/instance_test.go new file mode 100644 index 0000000000000..1a3699148920d --- /dev/null +++ b/pkg/pattern/instance_test.go @@ -0,0 +1,116 @@ +package pattern + +import ( + "context" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/grafana/loki/pkg/push" + "github.com/grafana/loki/v3/pkg/logproto" + "github.com/grafana/loki/v3/pkg/logql/syntax" + "github.com/grafana/loki/v3/pkg/pattern/metric" + "github.com/prometheus/common/model" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestInstance_QuerySample(t *testing.T) { + ctx := context.Background() + thirtySeconds := int64(30000) + oneMin := int64(60000) + fiveMin := oneMin * 5 + now := int64(1715964275000) + then := now - fiveMin // 1715963975000 + + mockReq := &logproto.QueryPatternsRequest{ + Start: time.Unix(then/1000, 0), + End: time.Now(), + Step: oneMin, + } + + instance, err := newInstance("test", log.NewNopLogger(), nil, metric.AggregationConfig{}) + require.NoError(t, err) + + labels := model.LabelSet{ + model.LabelName("foo"): model.LabelValue("bar"), + } + + lastTsMilli := (then + oneMin + oneMin) // 1715964095000 + + // TODO(twhitney): Add a few more pushes to this or another test + instance.Push(ctx, &logproto.PushRequest{ + Streams: []push.Stream{ + { + Labels: labels.String(), + Entries: []push.Entry{ + { + Timestamp: time.Unix(then/1000, 0), + Line: "this=that color=blue", + }, + { + Timestamp: time.Unix((then+thirtySeconds)/1000, 0), + Line: "this=that color=blue", + }, + { + Timestamp: time.Unix((then+oneMin)/1000, 0), + Line: "this=that color=blue", + }, + { + Timestamp: time.Unix(lastTsMilli/1000, 0), + Line: "this=that color=blue", + }, + }, + Hash: uint64(labels.Fingerprint()), + }, + }, + }) + + t.Run("successful count over time query", func(t *testing.T) { + expr, err := syntax.ParseSampleExpr(`count_over_time({foo="bar"}[30s])`) + require.NoError(t, err) + + iter, err := instance.QuerySample(ctx, expr, mockReq) + assert.NoError(t, err) + assert.NotNil(t, iter) + + // start is request start minus range, which is 30s here + start := then - 30000 + require.True(t, start < lastTsMilli-30000) + secondPoint := start + oneMin + require.True(t, secondPoint < lastTsMilli-30000) + // this is the first point past the lastTsMilli + thirdPoint := secondPoint + oneMin + require.Equal(t, lastTsMilli-30000, thirdPoint) + + next := iter.Next() + require.True(t, next) + + sample := iter.At() + require.Equal(t, int64(4), sample.Value) + require.Equal(t, model.Time(thirdPoint), sample.Timestamp) + + next = iter.Next() + require.False(t, next) + }) + + t.Run("successful bytes over time query", func(t *testing.T) { + expr, err := syntax.ParseSampleExpr(`bytes_over_time({foo="bar"}[30s])`) + require.NoError(t, err) + + iter, err := instance.QuerySample(ctx, expr, mockReq) + assert.NoError(t, err) + assert.NotNil(t, iter) + + next := iter.Next() + require.True(t, next) + + expctedTs := (then - 30000) + oneMin + oneMin + sample := iter.At() + require.Equal(t, int64(80), sample.Value) + require.Equal(t, model.Time(expctedTs), sample.Timestamp) + + next = iter.Next() + require.False(t, next) + }) +} diff --git a/pkg/pattern/iter/batch.go b/pkg/pattern/iter/batch.go index 80ad1197c80a9..159edf1d73d28 100644 --- a/pkg/pattern/iter/batch.go +++ b/pkg/pattern/iter/batch.go @@ -6,7 +6,33 @@ import ( "github.com/grafana/loki/v3/pkg/logproto" ) -func ReadBatch(it Iterator, batchSize int) (*logproto.QueryPatternsResponse, error) { +func ReadMetricsBatch(it Iterator, batchSize int) (*logproto.QueryPatternsResponse, error) { + var ( + series = map[string][]*logproto.PatternSample{} + respSize int + ) + + for ; respSize < batchSize && it.Next(); respSize++ { + labels := it.Labels() + sample := it.At() + series[labels.String()] = append(series[labels.String()], &sample) + } + result := logproto.QueryPatternsResponse{ + Series: make([]*logproto.PatternSeries, 0, len(series)), + } + for id, samples := range series { + result.Series = append( + result.Series, + logproto.NewPatternSeriesWithLabels(id, samples), + ) + } + return &result, it.Error() +} + +func ReadPatternsBatch( + it Iterator, + batchSize int, +) (*logproto.QueryPatternsResponse, error) { var ( series = map[string][]*logproto.PatternSample{} respSize int @@ -20,15 +46,19 @@ func ReadBatch(it Iterator, batchSize int) (*logproto.QueryPatternsResponse, err result := logproto.QueryPatternsResponse{ Series: make([]*logproto.PatternSeries, 0, len(series)), } - for pattern, samples := range series { - result.Series = append(result.Series, &logproto.PatternSeries{ - Pattern: pattern, - Samples: samples, - }) + for id, samples := range series { + result.Series = append( + result.Series, + logproto.NewPatternSeriesWithPattern(id, samples), + ) } return &result, it.Error() } -func ReadAll(it Iterator) (*logproto.QueryPatternsResponse, error) { - return ReadBatch(it, math.MaxInt32) +func ReadAllWithPatterns(it Iterator) (*logproto.QueryPatternsResponse, error) { + return ReadPatternsBatch(it, math.MaxInt32) +} + +func ReadAllWithLabels(it Iterator) (*logproto.QueryPatternsResponse, error) { + return ReadMetricsBatch(it, math.MaxInt32) } diff --git a/pkg/pattern/iter/batch_test.go b/pkg/pattern/iter/batch_test.go index 7f544e23f417d..d798583d1a2ac 100644 --- a/pkg/pattern/iter/batch_test.go +++ b/pkg/pattern/iter/batch_test.go @@ -32,13 +32,13 @@ func TestReadBatch(t *testing.T) { batchSize: 2, expected: &logproto.QueryPatternsResponse{ Series: []*logproto.PatternSeries{ - { - Pattern: "foo", - Samples: []*logproto.PatternSample{ + logproto.NewPatternSeriesWithPattern( + "foo", + []*logproto.PatternSample{ {Timestamp: 10, Value: 2}, {Timestamp: 20, Value: 4}, }, - }, + ), }, }, }, @@ -49,14 +49,14 @@ func TestReadBatch(t *testing.T) { batchSize: 4, expected: &logproto.QueryPatternsResponse{ Series: []*logproto.PatternSeries{ - { - Pattern: "foo", - Samples: []*logproto.PatternSample{ + logproto.NewPatternSeriesWithPattern( + "foo", + []*logproto.PatternSample{ {Timestamp: 10, Value: 2}, {Timestamp: 20, Value: 4}, {Timestamp: 30, Value: 6}, }, - }, + ), }, }, }, @@ -64,8 +64,8 @@ func TestReadBatch(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - it := NewSlice(tt.pattern, tt.samples) - got, err := ReadBatch(it, tt.batchSize) + it := NewPatternSlice(tt.pattern, tt.samples) + got, err := ReadPatternsBatch(it, tt.batchSize) require.NoError(t, err) require.Equal(t, tt.expected, got) }) diff --git a/pkg/pattern/iter/iterator.go b/pkg/pattern/iter/iterator.go index 5a277c0f27349..2f27104f0da07 100644 --- a/pkg/pattern/iter/iterator.go +++ b/pkg/pattern/iter/iterator.go @@ -1,7 +1,9 @@ package iter import ( + "github.com/grafana/loki/v3/pkg/iter" "github.com/grafana/loki/v3/pkg/logproto" + "github.com/prometheus/prometheus/model/labels" ) var Empty Iterator = &emptyIterator{} @@ -10,49 +12,75 @@ type Iterator interface { Next() bool Pattern() string + Labels() labels.Labels At() logproto.PatternSample Error() error Close() error } -func NewSlice(pattern string, s []logproto.PatternSample) Iterator { - return &sliceIterator{ +type SampleIterator interface { + Iterator + Sample() logproto.PatternSample +} + +type PeekingIterator interface { + SampleIterator + Peek() (string, logproto.PatternSample, bool) +} + +func NewPatternSlice(pattern string, s []logproto.PatternSample) Iterator { + return &patternSliceIterator{ values: s, pattern: pattern, + labels: labels.EmptyLabels(), i: -1, } } -type sliceIterator struct { +func NewLabelsSlice(lbls labels.Labels, s []logproto.PatternSample) Iterator { + return &patternSliceIterator{ + values: s, + labels: lbls, + i: -1, + } +} + +type patternSliceIterator struct { i int pattern string + labels labels.Labels values []logproto.PatternSample } -func (s *sliceIterator) Next() bool { +func (s *patternSliceIterator) Next() bool { s.i++ return s.i < len(s.values) } -func (s *sliceIterator) Pattern() string { +func (s *patternSliceIterator) Pattern() string { return s.pattern } -func (s *sliceIterator) At() logproto.PatternSample { +func (s *patternSliceIterator) Labels() labels.Labels { + return s.labels +} + +func (s *patternSliceIterator) At() logproto.PatternSample { return s.values[s.i] } -func (s *sliceIterator) Error() error { +func (s *patternSliceIterator) Error() error { return nil } -func (s *sliceIterator) Close() error { +func (s *patternSliceIterator) Close() error { return nil } type emptyIterator struct { pattern string + labels labels.Labels } func (e *emptyIterator) Next() bool { @@ -63,6 +91,10 @@ func (e *emptyIterator) Pattern() string { return e.pattern } +func (e *emptyIterator) Labels() labels.Labels { + return e.labels +} + func (e *emptyIterator) At() logproto.PatternSample { return logproto.PatternSample{} } @@ -79,16 +111,25 @@ type nonOverlappingIterator struct { iterators []Iterator curr Iterator pattern string + labels labels.Labels } -// NewNonOverlappingIterator gives a chained iterator over a list of iterators. -func NewNonOverlappingIterator(pattern string, iterators []Iterator) Iterator { +// NewNonOverlappingPatternIterator gives a chained iterator over a list of iterators. +func NewNonOverlappingPatternIterator(pattern string, iterators []Iterator) Iterator { return &nonOverlappingIterator{ iterators: iterators, pattern: pattern, } } +// NewNonOverlappingLabelsIterator gives a chained iterator over a list of iterators. +func NewNonOverlappingLabelsIterator(labels labels.Labels, iterators []Iterator) Iterator { + return &nonOverlappingIterator{ + iterators: iterators, + labels: labels, + } +} + func (i *nonOverlappingIterator) Next() bool { for i.curr == nil || !i.curr.Next() { if len(i.iterators) == 0 { @@ -114,6 +155,10 @@ func (i *nonOverlappingIterator) Pattern() string { return i.pattern } +func (i *nonOverlappingIterator) Labels() labels.Labels { + return i.labels +} + func (i *nonOverlappingIterator) Error() error { if i.curr == nil { return nil @@ -131,3 +176,114 @@ func (i *nonOverlappingIterator) Close() error { i.iterators = nil return nil } + +type peekingIterator struct { + iter Iterator + + cache *sampleWithLabels + next *sampleWithLabels + labels labels.Labels +} + +type sampleWithLabels struct { + logproto.PatternSample + labels labels.Labels +} + +func (s *sampleWithLabels) Sample() logproto.Sample { + return logproto.Sample{ + Timestamp: s.PatternSample.Timestamp.UnixNano(), // logproto.Sample expects nano seconds + Value: float64(s.PatternSample.Value), + Hash: 0, + } +} + +func NewPeekingSampleIterator(iter Iterator) iter.PeekingSampleIterator { + // initialize the next entry so we can peek right from the start. + var cache *sampleWithLabels + next := &sampleWithLabels{} + if iter.Next() { + cache = &sampleWithLabels{ + PatternSample: iter.At(), + labels: iter.Labels(), + } + next.PatternSample = cache.PatternSample + next.labels = cache.labels + } + + return &peekingIterator{ + iter: iter, + cache: cache, + next: next, + labels: iter.Labels(), + } +} + +func (it *peekingIterator) Close() error { + return it.iter.Close() +} + +func (it *peekingIterator) Labels() string { + return it.labels.String() +} + +func (it *peekingIterator) Next() bool { + if it.cache != nil { + it.next.PatternSample = it.cache.PatternSample + it.next.labels = it.cache.labels + it.cacheNext() + return true + } + return false +} + +func (it *peekingIterator) Sample() logproto.Sample { + if it.next != nil { + return logproto.Sample{ + Timestamp: it.next.PatternSample.Timestamp.UnixNano(), // expecting nano seconds + Value: float64(it.next.PatternSample.Value), + Hash: 0, + } + } + return logproto.Sample{} +} + +func (it *peekingIterator) At() logproto.PatternSample { + if it.next != nil { + return it.next.PatternSample + } + return logproto.PatternSample{} +} + +// cacheNext caches the next element if it exists. +func (it *peekingIterator) cacheNext() { + if it.iter.Next() { + it.cache.PatternSample = it.iter.At() + it.cache.labels = it.iter.Labels() + return + } + // nothing left, remove the cached entry + it.cache = nil +} + +func (it *peekingIterator) Pattern() logproto.PatternSample { + if it.next != nil { + return it.next.PatternSample + } + return logproto.PatternSample{} +} + +func (it *peekingIterator) Peek() (string, logproto.Sample, bool) { + if it.cache != nil { + return it.cache.labels.String(), it.cache.Sample(), true + } + return "", logproto.Sample{}, false +} + +func (it *peekingIterator) Error() error { + return it.iter.Error() +} + +func (it *peekingIterator) StreamHash() uint64 { + return 0 +} diff --git a/pkg/pattern/iter/iterator_test.go b/pkg/pattern/iter/iterator_test.go index b327800575b55..dd66361d635b2 100644 --- a/pkg/pattern/iter/iterator_test.go +++ b/pkg/pattern/iter/iterator_test.go @@ -3,57 +3,111 @@ package iter import ( "testing" + "github.com/prometheus/prometheus/model/labels" "github.com/stretchr/testify/require" "github.com/grafana/loki/v3/pkg/logproto" ) func TestSliceIterator(t *testing.T) { - tests := []struct { - name string - pattern string - samples []logproto.PatternSample - want []patternSample - }{ - { - name: "1 samples", - pattern: "foo", - samples: []logproto.PatternSample{ - {Timestamp: 10, Value: 2}, + t.Run("samples with pattern", func(t *testing.T) { + tests := []struct { + name string + pattern string + samples []logproto.PatternSample + want []patternSample + }{ + { + name: "1 samples", + pattern: "foo", + samples: []logproto.PatternSample{ + {Timestamp: 10, Value: 2}, + }, + want: []patternSample{ + {"foo", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 10, Value: 2}}, + }, }, - want: []patternSample{ - {"foo", logproto.PatternSample{Timestamp: 10, Value: 2}}, + { + name: "3 samples", + pattern: "foo", + samples: []logproto.PatternSample{ + {Timestamp: 10, Value: 2}, + {Timestamp: 20, Value: 4}, + {Timestamp: 30, Value: 6}, + }, + want: []patternSample{ + {"foo", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 10, Value: 2}}, + {"foo", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 20, Value: 4}}, + {"foo", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 30, Value: 6}}, + }, }, - }, - { - name: "3 samples", - pattern: "foo", - samples: []logproto.PatternSample{ - {Timestamp: 10, Value: 2}, - {Timestamp: 20, Value: 4}, - {Timestamp: 30, Value: 6}, + { + name: "empty", + pattern: "foo", + samples: nil, + want: nil, }, - want: []patternSample{ - {"foo", logproto.PatternSample{Timestamp: 10, Value: 2}}, - {"foo", logproto.PatternSample{Timestamp: 20, Value: 4}}, - {"foo", logproto.PatternSample{Timestamp: 30, Value: 6}}, + } + + for _, tt := range tests { + tt := tt + t.Run(tt.name, func(t *testing.T) { + got := slice(NewPatternSlice(tt.pattern, tt.samples)) + require.Equal(t, tt.want, got) + }) + } + }) + + t.Run("samples with labels", func(t *testing.T) { + stream := labels.Labels{ + {Name: "test", Value: "test"}, + } + tests := []struct { + name string + labels labels.Labels + samples []logproto.PatternSample + want []patternSample + }{ + { + name: "1 samples", + labels: stream, + samples: []logproto.PatternSample{ + {Timestamp: 10, Value: 2}, + }, + want: []patternSample{ + {"", stream, logproto.PatternSample{Timestamp: 10, Value: 2}}, + }, }, - }, - { - name: "empty", - pattern: "foo", - samples: nil, - want: nil, - }, - } + { + name: "3 samples", + labels: stream, + samples: []logproto.PatternSample{ + {Timestamp: 10, Value: 2}, + {Timestamp: 20, Value: 4}, + {Timestamp: 30, Value: 6}, + }, + want: []patternSample{ + {"", stream, logproto.PatternSample{Timestamp: 10, Value: 2}}, + {"", stream, logproto.PatternSample{Timestamp: 20, Value: 4}}, + {"", stream, logproto.PatternSample{Timestamp: 30, Value: 6}}, + }, + }, + { + name: "empty", + labels: labels.EmptyLabels(), + samples: nil, + want: nil, + }, + } - for _, tt := range tests { - tt := tt - t.Run(tt.name, func(t *testing.T) { - got := slice(NewSlice(tt.pattern, tt.samples)) - require.Equal(t, tt.want, got) - }) - } + for _, tt := range tests { + tt := tt + t.Run(tt.name, func(t *testing.T) { + got := slice(NewLabelsSlice(tt.labels, tt.samples)) + require.Equal(t, tt.want, got) + }) + } + }) } func slice(it Iterator) []patternSample { @@ -63,6 +117,7 @@ func slice(it Iterator) []patternSample { samples = append(samples, patternSample{ pattern: it.Pattern(), sample: it.At(), + labels: it.Labels(), }) } if it.Error() != nil { diff --git a/pkg/pattern/iter/merge.go b/pkg/pattern/iter/merge.go index 3b0e07e33b8a8..0c7c19a633a93 100644 --- a/pkg/pattern/iter/merge.go +++ b/pkg/pattern/iter/merge.go @@ -5,6 +5,7 @@ import ( "github.com/grafana/loki/v3/pkg/logproto" "github.com/grafana/loki/v3/pkg/util/loser" + "github.com/prometheus/prometheus/model/labels" ) type mergeIterator struct { @@ -16,18 +17,22 @@ type mergeIterator struct { type patternSample struct { pattern string + labels labels.Labels sample logproto.PatternSample } var max = patternSample{ pattern: "", + labels: labels.Labels{}, sample: logproto.PatternSample{Timestamp: math.MaxInt64}, } func NewMerge(iters ...Iterator) Iterator { + // TODO: I need to call next here tree := loser.New(iters, max, func(s Iterator) patternSample { return patternSample{ pattern: s.Pattern(), + labels: s.Labels(), sample: s.At(), } }, func(e1, e2 patternSample) bool { @@ -57,10 +62,13 @@ func (m *mergeIterator) Next() bool { } m.current.pattern = m.tree.Winner().Pattern() + m.current.labels = m.tree.Winner().Labels() m.current.sample = m.tree.Winner().At() for m.tree.Next() { - if m.current.sample.Timestamp != m.tree.Winner().At().Timestamp || m.current.pattern != m.tree.Winner().Pattern() { + if m.current.sample.Timestamp != m.tree.Winner().At().Timestamp || + m.current.pattern != m.tree.Winner().Pattern() || + m.current.labels.String() != m.tree.Winner().Labels().String() { return true } m.current.sample.Value += m.tree.Winner().At().Value @@ -74,6 +82,10 @@ func (m *mergeIterator) Pattern() string { return m.current.pattern } +func (m *mergeIterator) Labels() labels.Labels { + return m.current.labels +} + func (m *mergeIterator) At() logproto.PatternSample { return m.current.sample } diff --git a/pkg/pattern/iter/merge_test.go b/pkg/pattern/iter/merge_test.go index a1d643a5a01c1..d55f417062bdc 100644 --- a/pkg/pattern/iter/merge_test.go +++ b/pkg/pattern/iter/merge_test.go @@ -3,76 +3,150 @@ package iter import ( "testing" + "github.com/prometheus/prometheus/model/labels" "github.com/stretchr/testify/require" "github.com/grafana/loki/v3/pkg/logproto" ) func TestMerge(t *testing.T) { - tests := []struct { - name string - iterators []Iterator - expected []patternSample - }{ - { - name: "Empty iterators", - iterators: []Iterator{}, - expected: nil, - }, - { - name: "Merge single iterator", - iterators: []Iterator{ - NewSlice("a", []logproto.PatternSample{ - {Timestamp: 10, Value: 2}, {Timestamp: 20, Value: 4}, {Timestamp: 30, Value: 6}, - }), + t.Run("merging patterns", func(t *testing.T) { + tests := []struct { + name string + iterators []Iterator + expected []patternSample + }{ + { + name: "Empty iterators", + iterators: []Iterator{}, + expected: nil, }, - expected: []patternSample{ - {"a", logproto.PatternSample{Timestamp: 10, Value: 2}}, - {"a", logproto.PatternSample{Timestamp: 20, Value: 4}}, - {"a", logproto.PatternSample{Timestamp: 30, Value: 6}}, + { + name: "Merge single iterator", + iterators: []Iterator{ + NewPatternSlice("a", []logproto.PatternSample{ + {Timestamp: 10, Value: 2}, {Timestamp: 20, Value: 4}, {Timestamp: 30, Value: 6}, + }), + }, + expected: []patternSample{ + {"a", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 10, Value: 2}}, + {"a", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 20, Value: 4}}, + {"a", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 30, Value: 6}}, + }, }, - }, - { - name: "Merge multiple iterators", - iterators: []Iterator{ - NewSlice("a", []logproto.PatternSample{{Timestamp: 10, Value: 2}, {Timestamp: 30, Value: 6}}), - NewSlice("b", []logproto.PatternSample{{Timestamp: 20, Value: 4}, {Timestamp: 40, Value: 8}}), + { + name: "Merge multiple iterators", + iterators: []Iterator{ + NewPatternSlice("a", []logproto.PatternSample{{Timestamp: 10, Value: 2}, {Timestamp: 30, Value: 6}}), + NewPatternSlice("b", []logproto.PatternSample{{Timestamp: 20, Value: 4}, {Timestamp: 40, Value: 8}}), + }, + expected: []patternSample{ + {"a", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 10, Value: 2}}, + {"b", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 20, Value: 4}}, + {"a", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 30, Value: 6}}, + {"b", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 40, Value: 8}}, + }, }, - expected: []patternSample{ - {"a", logproto.PatternSample{Timestamp: 10, Value: 2}}, - {"b", logproto.PatternSample{Timestamp: 20, Value: 4}}, - {"a", logproto.PatternSample{Timestamp: 30, Value: 6}}, - {"b", logproto.PatternSample{Timestamp: 40, Value: 8}}, + { + name: "Merge multiple iterators with similar samples", + iterators: []Iterator{ + NewPatternSlice("a", []logproto.PatternSample{{Timestamp: 10, Value: 2}, {Timestamp: 30, Value: 6}}), + NewPatternSlice("a", []logproto.PatternSample{{Timestamp: 10, Value: 2}, {Timestamp: 30, Value: 6}}), + NewPatternSlice("b", []logproto.PatternSample{{Timestamp: 20, Value: 4}, {Timestamp: 40, Value: 8}}), + }, + expected: []patternSample{ + {"a", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 10, Value: 4}}, + {"b", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 20, Value: 4}}, + {"a", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 30, Value: 12}}, + {"b", labels.EmptyLabels(), logproto.PatternSample{Timestamp: 40, Value: 8}}, + }, }, - }, - { - name: "Merge multiple iterators with similar samples", - iterators: []Iterator{ - NewSlice("a", []logproto.PatternSample{{Timestamp: 10, Value: 2}, {Timestamp: 30, Value: 6}}), - NewSlice("a", []logproto.PatternSample{{Timestamp: 10, Value: 2}, {Timestamp: 30, Value: 6}}), - NewSlice("b", []logproto.PatternSample{{Timestamp: 20, Value: 4}, {Timestamp: 40, Value: 8}}), + } + + for _, tt := range tests { + tt := tt + t.Run(tt.name, func(t *testing.T) { + it := NewMerge(tt.iterators...) + defer it.Close() + + var result []patternSample + for it.Next() { + result = append(result, patternSample{it.Pattern(), it.Labels(), it.At()}) + } + + require.Equal(t, tt.expected, result) + }) + } + }) + + t.Run("merging label samples", func(t *testing.T) { + stream1 := labels.Labels{labels.Label{Name: "foo", Value: "bar"}, labels.Label{Name: "ying", Value: "yang"}} + stream2 := labels.Labels{labels.Label{Name: "foo", Value: "baz"}, labels.Label{Name: "ying", Value: "yang"}} + tests := []struct { + name string + iterators []Iterator + expected []patternSample + }{ + { + name: "Empty iterators", + iterators: []Iterator{}, + expected: nil, + }, + { + name: "Merge single iterator", + iterators: []Iterator{ + NewLabelsSlice(stream1, []logproto.PatternSample{ + {Timestamp: 10, Value: 2}, {Timestamp: 20, Value: 4}, {Timestamp: 30, Value: 6}, + }), + }, + expected: []patternSample{ + {"", stream1, logproto.PatternSample{Timestamp: 10, Value: 2}}, + {"", stream1, logproto.PatternSample{Timestamp: 20, Value: 4}}, + {"", stream1, logproto.PatternSample{Timestamp: 30, Value: 6}}, + }, + }, + { + name: "Merge multiple iterators", + iterators: []Iterator{ + NewLabelsSlice(stream1, []logproto.PatternSample{{Timestamp: 10, Value: 2}, {Timestamp: 30, Value: 6}}), + NewLabelsSlice(stream2, []logproto.PatternSample{{Timestamp: 20, Value: 4}, {Timestamp: 40, Value: 8}}), + }, + expected: []patternSample{ + {"", stream1, logproto.PatternSample{Timestamp: 10, Value: 2}}, + {"", stream2, logproto.PatternSample{Timestamp: 20, Value: 4}}, + {"", stream1, logproto.PatternSample{Timestamp: 30, Value: 6}}, + {"", stream2, logproto.PatternSample{Timestamp: 40, Value: 8}}, + }, }, - expected: []patternSample{ - {"a", logproto.PatternSample{Timestamp: 10, Value: 4}}, - {"b", logproto.PatternSample{Timestamp: 20, Value: 4}}, - {"a", logproto.PatternSample{Timestamp: 30, Value: 12}}, - {"b", logproto.PatternSample{Timestamp: 40, Value: 8}}, + { + name: "Merge multiple iterators with similar samples", + iterators: []Iterator{ + NewLabelsSlice(stream1, []logproto.PatternSample{{Timestamp: 10, Value: 2}, {Timestamp: 30, Value: 6}}), + NewLabelsSlice(stream1, []logproto.PatternSample{{Timestamp: 10, Value: 2}, {Timestamp: 30, Value: 6}}), + NewLabelsSlice(stream2, []logproto.PatternSample{{Timestamp: 20, Value: 4}, {Timestamp: 40, Value: 8}}), + }, + expected: []patternSample{ + {"", stream1, logproto.PatternSample{Timestamp: 10, Value: 4}}, + {"", stream2, logproto.PatternSample{Timestamp: 20, Value: 4}}, + {"", stream1, logproto.PatternSample{Timestamp: 30, Value: 12}}, + {"", stream2, logproto.PatternSample{Timestamp: 40, Value: 8}}, + }, }, - }, - } + } - for _, tt := range tests { - tt := tt - t.Run(tt.name, func(t *testing.T) { - it := NewMerge(tt.iterators...) - defer it.Close() + for _, tt := range tests { + tt := tt + t.Run(tt.name, func(t *testing.T) { + it := NewMerge(tt.iterators...) + defer it.Close() - var result []patternSample - for it.Next() { - result = append(result, patternSample{it.Pattern(), it.At()}) - } + var result []patternSample + for it.Next() { + result = append(result, patternSample{it.Pattern(), it.Labels(), it.At()}) + } - require.Equal(t, tt.expected, result) - }) - } + require.Equal(t, tt.expected, result) + }) + } + }) } diff --git a/pkg/pattern/iter/query_client.go b/pkg/pattern/iter/query_client.go index f6c5c4fa97744..00ed407a39e9b 100644 --- a/pkg/pattern/iter/query_client.go +++ b/pkg/pattern/iter/query_client.go @@ -4,6 +4,8 @@ import ( "io" "github.com/grafana/loki/v3/pkg/logproto" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/promql/parser" ) type queryClientIterator struct { @@ -38,6 +40,10 @@ func (i *queryClientIterator) Pattern() string { return i.curr.Pattern() } +func (i *queryClientIterator) Labels() labels.Labels { + return i.curr.Labels() +} + func (i *queryClientIterator) At() logproto.PatternSample { return i.curr.At() } @@ -58,7 +64,17 @@ func NewQueryResponseIterator(resp *logproto.QueryPatternsResponse) Iterator { for j, sample := range s.Samples { samples[j] = *sample } - iters[i] = NewSlice(s.Pattern, samples) + + switch s.GetIdentifier().(type) { + case *logproto.PatternSeries_Labels: + ls, err := parser.ParseMetric(s.GetLabels()) + if err != nil { + ls = labels.Labels{} + } + iters[i] = NewLabelsSlice(ls, samples) + default: + iters[i] = NewPatternSlice(s.GetPattern(), samples) + } } return NewMerge(iters...) } diff --git a/pkg/pattern/metric/chunk.go b/pkg/pattern/metric/chunk.go new file mode 100644 index 0000000000000..c4716ba532dd1 --- /dev/null +++ b/pkg/pattern/metric/chunk.go @@ -0,0 +1,201 @@ +package metric + +import ( + "context" + "fmt" + "sort" + "time" + + "github.com/grafana/loki/v3/pkg/logproto" + "github.com/grafana/loki/v3/pkg/pattern/chunk" + "github.com/grafana/loki/v3/pkg/pattern/iter" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/labels" +) + +type MetricType int + +const ( + Bytes MetricType = iota + Count + Unsupported +) + +type Chunks struct { + chunks []Chunk + labels labels.Labels +} + +func NewChunks(labels labels.Labels) *Chunks { + return &Chunks{ + chunks: make([]Chunk, 0), + labels: labels, + } +} + +func (c *Chunks) Observe(bytes, count uint64, ts model.Time) { + if len(c.chunks) == 0 { + c.chunks = append(c.chunks, newChunk(bytes, count, ts)) + return + } + + last := &(c.chunks)[len(c.chunks)-1] + if !last.spaceFor(ts) { + c.chunks = append(c.chunks, newChunk(bytes, count, ts)) + return + } + + last.AddSample(newSample(bytes, count, ts)) +} + +func (c *Chunks) Iterator( + ctx context.Context, + typ MetricType, + from, through, step model.Time, +) (iter.Iterator, error) { + if typ == Unsupported { + return nil, fmt.Errorf("unsupported metric type") + } + + iters := make([]iter.Iterator, 0, len(c.chunks)) + for _, chunk := range c.chunks { + samples, err := chunk.ForRangeAndType(typ, from, through, step) + if err != nil { + return nil, err + } + + if len(samples) == 0 { + continue + } + + iters = append(iters, iter.NewLabelsSlice(c.labels, samples)) + } + return iter.NewNonOverlappingLabelsIterator(c.labels, iters), nil +} + +// TODO(twhitney): These values should be float64s (to match prometheus samples) or int64s (to match pattern samples) +type MetricSample struct { + Timestamp model.Time + Bytes uint64 + Count uint64 +} + +func newSample(bytes, count uint64, ts model.Time) MetricSample { + return MetricSample{ + Timestamp: ts, + Bytes: bytes, + Count: count, + } +} + +type MetricSamples []MetricSample + +type Chunk struct { + Samples MetricSamples + mint, maxt int64 +} + +func (c *Chunk) Bounds() (fromT, toT time.Time) { + return time.Unix(0, c.mint), time.Unix(0, c.maxt) +} + +func (c *Chunk) AddSample(s MetricSample) { + c.Samples = append(c.Samples, s) + ts := int64(s.Timestamp) + + if ts < c.mint { + c.mint = ts + } + + if ts > c.maxt { + c.maxt = ts + } +} + +func newChunk(bytes, count uint64, ts model.Time) Chunk { + maxSize := int(chunk.MaxChunkTime.Nanoseconds()/chunk.TimeResolution.UnixNano()) + 1 + v := Chunk{Samples: make(MetricSamples, 1, maxSize)} + v.Samples[0] = newSample(bytes, count, ts) + return v +} + +func (c *Chunk) spaceFor(ts model.Time) bool { + if len(c.Samples) == 0 { + return true + } + + return ts.Sub(c.Samples[0].Timestamp) < chunk.MaxChunkTime +} + +//TODO(twhitney): any way to remove the duplication between this and the drain chunk ForRange method? +// ForRangeAndType returns samples with only the values +// in the given range [start:end] and aggregates them by step duration. +// start and end are in milliseconds since epoch. step is a duration in milliseconds. +func (c *Chunk) ForRangeAndType( + typ MetricType, + start, end, step model.Time, +) ([]logproto.PatternSample, error) { + if typ == Unsupported { + return nil, fmt.Errorf("unsupported metric type") + } + + if len(c.Samples) == 0 { + return nil, nil + } + + first := c.Samples[0].Timestamp // why is this in the future? + last := c.Samples[len(c.Samples)-1].Timestamp + startBeforeEnd := start >= end + samplesAreAfterRange := first > end + samplesAreBeforeRange := last < start + if startBeforeEnd || samplesAreAfterRange || samplesAreBeforeRange { + return nil, nil + } + + var lo int + if start > first { + lo = sort.Search(len(c.Samples), func(i int) bool { + return c.Samples[i].Timestamp >= start + }) + } + hi := len(c.Samples) + if end < last { + hi = sort.Search(len(c.Samples), func(i int) bool { + return c.Samples[i].Timestamp > end + }) + } + + // Re-scale samples into step-sized buckets + currentStep := chunk.TruncateTimestamp(c.Samples[lo].Timestamp, step) + numOfSteps := ((c.Samples[hi-1].Timestamp - currentStep) / step) + 1 + aggregatedSamples := make([]logproto.PatternSample, 0, numOfSteps) + aggregatedSamples = append(aggregatedSamples, + logproto.PatternSample{ + Timestamp: currentStep, + Value: 0, + }) + + for _, sample := range c.Samples[lo:hi] { + if sample.Timestamp >= currentStep+step { + stepForSample := chunk.TruncateTimestamp(sample.Timestamp, step) + for i := currentStep + step; i <= stepForSample; i += step { + aggregatedSamples = append(aggregatedSamples, logproto.PatternSample{ + Timestamp: i, + Value: 0, + }) + } + currentStep = stepForSample + } + + var v int64 + if typ == Bytes { + v = int64(sample.Bytes) + } else { + v = int64(sample.Count) + } + + aggregatedSamples[len(aggregatedSamples)-1].Value += v + } + + return aggregatedSamples, nil +} diff --git a/pkg/pattern/metric/chunk_test.go b/pkg/pattern/metric/chunk_test.go new file mode 100644 index 0000000000000..32b746b0d20d0 --- /dev/null +++ b/pkg/pattern/metric/chunk_test.go @@ -0,0 +1,329 @@ +package metric + +import ( + "reflect" + "testing" + + "github.com/grafana/loki/v3/pkg/logproto" + "github.com/prometheus/common/model" + "github.com/stretchr/testify/require" +) + +func TestForRangeAndType(t *testing.T) { + testCases := []struct { + name string + c *Chunk + metricType MetricType + start model.Time + end model.Time + expected []logproto.PatternSample + }{ + { + name: "Empty count", + c: &Chunk{}, + metricType: Count, + start: 1, + end: 10, + expected: nil, + }, + { + name: "Empty bytes", + c: &Chunk{}, + metricType: Bytes, + start: 1, + end: 10, + expected: nil, + }, + { + name: "No Overlap -- count", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 2, Count: 2}, + {Timestamp: 4, Count: 4}, + {Timestamp: 6, Count: 6}, + }}, + metricType: Count, + start: 10, + end: 20, + expected: nil, + }, + { + name: "No Overlap -- bytes", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 2, Bytes: 2}, + {Timestamp: 4, Bytes: 4}, + {Timestamp: 6, Bytes: 6}, + }}, + metricType: Bytes, + start: 10, + end: 20, + expected: nil, + }, + { + name: "Complete Overlap -- count", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 2, Count: 2}, + {Timestamp: 4, Count: 4}, + {Timestamp: 6, Count: 6}, + }}, + metricType: Count, + start: 0, + end: 10, + expected: []logproto.PatternSample{ + {Timestamp: 2, Value: 2}, + {Timestamp: 4, Value: 4}, + {Timestamp: 6, Value: 6}, + }, + }, + { + name: "Complete Overlap -- bytes", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 2, Bytes: 2}, + {Timestamp: 4, Bytes: 4}, + {Timestamp: 6, Bytes: 6}, + }}, + metricType: Bytes, + start: 0, + end: 10, + expected: []logproto.PatternSample{ + {Timestamp: 2, Value: 2}, + {Timestamp: 4, Value: 4}, + {Timestamp: 6, Value: 6}, + }, + }, + { + name: "Partial Overlap -- count", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 2, Count: 2}, + {Timestamp: 4, Count: 4}, + {Timestamp: 6, Count: 6}, + }}, + metricType: Count, + start: 3, + end: 5, + expected: []logproto.PatternSample{{Timestamp: 4, Value: 4}}, + }, + { + name: "Partial Overlap -- bytes", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 2, Bytes: 2}, + {Timestamp: 4, Bytes: 4}, + {Timestamp: 6, Bytes: 6}, + }}, + metricType: Bytes, + start: 3, + end: 5, + expected: []logproto.PatternSample{{Timestamp: 4, Value: 4}}, + }, + { + name: "Single Element in Range -- count", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 2, Count: 2}, + {Timestamp: 4, Count: 4}, + {Timestamp: 6, Count: 6}, + }}, + metricType: Count, + start: 4, + end: 5, + expected: []logproto.PatternSample{{Timestamp: 4, Value: 4}}, + }, + { + name: "Single Element in Range -- bytes", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 2, Bytes: 2}, + {Timestamp: 4, Bytes: 4}, + {Timestamp: 6, Bytes: 6}, + }}, + metricType: Bytes, + start: 4, + end: 5, + expected: []logproto.PatternSample{{Timestamp: 4, Value: 4}}, + }, + { + name: "Start Before First Element -- count", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 2, Count: 2}, + {Timestamp: 4, Count: 4}, + {Timestamp: 6, Count: 6}, + }}, + metricType: Count, + start: 0, + end: 5, + expected: []logproto.PatternSample{ + {Timestamp: 2, Value: 2}, + {Timestamp: 4, Value: 4}, + }, + }, + { + name: "Start Before First Element -- bytes", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 2, Bytes: 2}, + {Timestamp: 4, Bytes: 4}, + {Timestamp: 6, Bytes: 6}, + }}, + metricType: Bytes, + start: 0, + end: 5, + expected: []logproto.PatternSample{ + {Timestamp: 2, Value: 2}, + {Timestamp: 4, Value: 4}, + }, + }, + { + name: "End After Last Element -- count", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 2, Count: 2}, + {Timestamp: 4, Count: 4}, + {Timestamp: 6, Count: 6}, + }}, + metricType: Count, + start: 5, + end: 10, + expected: []logproto.PatternSample{ + {Timestamp: 6, Value: 6}, + }, + }, + { + name: "End After Last Element -- bytes", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 2, Bytes: 2}, + {Timestamp: 4, Bytes: 4}, + {Timestamp: 6, Bytes: 6}, + }}, + metricType: Bytes, + start: 5, + end: 10, + expected: []logproto.PatternSample{ + {Timestamp: 6, Value: 6}, + }, + }, + { + name: "Start before First and End Inclusive of First Element -- count", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 2, Count: 2}, + {Timestamp: 4, Count: 4}, + {Timestamp: 6, Count: 6}, + }}, + metricType: Count, + start: 0, + end: 2, + expected: []logproto.PatternSample{{Timestamp: 2, Value: 2}}, + }, + { + name: "Start before First and End Inclusive of First Element -- bytes", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 2, Bytes: 2}, + {Timestamp: 4, Bytes: 4}, + {Timestamp: 6, Bytes: 6}, + }}, + metricType: Bytes, + start: 0, + end: 2, + expected: []logproto.PatternSample{{Timestamp: 2, Value: 2}}, + }, + { + name: "Start and End before First Element -- count", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 2, Count: 2}, + {Timestamp: 4, Count: 4}, + {Timestamp: 6, Count: 6}, + }}, + metricType: Count, + start: 0, + end: 1, + expected: nil, + }, + { + name: "Start and End before First Element -- bytes", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 2, Bytes: 2}, + {Timestamp: 4, Bytes: 4}, + {Timestamp: 6, Bytes: 6}, + }}, + metricType: Bytes, + start: 0, + end: 1, + expected: nil, + }, + { + name: "Higher resolution samples down-sampled to preceding step bucket -- count", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 1, Count: 2}, + {Timestamp: 2, Count: 4}, + {Timestamp: 3, Count: 6}, + {Timestamp: 4, Count: 8}, + {Timestamp: 5, Count: 10}, + {Timestamp: 6, Count: 12}, + }}, + metricType: Count, + start: 1, + end: 6, + expected: []logproto.PatternSample{ + {Timestamp: 0, Value: 2}, + {Timestamp: 2, Value: 10}, + {Timestamp: 4, Value: 18}, + {Timestamp: 6, Value: 12}, + }, + }, + { + name: "Higher resolution samples down-sampled to preceding step bucket -- bytes", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 1, Bytes: 2}, + {Timestamp: 2, Bytes: 4}, + {Timestamp: 3, Bytes: 6}, + {Timestamp: 4, Bytes: 8}, + {Timestamp: 5, Bytes: 10}, + {Timestamp: 6, Bytes: 12}, + }}, + metricType: Bytes, + start: 1, + end: 6, + expected: []logproto.PatternSample{ + {Timestamp: 0, Value: 2}, + {Timestamp: 2, Value: 10}, + {Timestamp: 4, Value: 18}, + {Timestamp: 6, Value: 12}, + }, + }, + { + name: "Low resolution samples insert 0 values for empty steps -- count", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 1, Count: 2}, + {Timestamp: 5, Count: 10}, + }}, + metricType: Count, + start: 1, + end: 6, + expected: []logproto.PatternSample{ + {Timestamp: 0, Value: 2}, + {Timestamp: 2, Value: 0}, + {Timestamp: 4, Value: 10}, + }, + }, + { + name: "Low resolution samples insert 0 values for empty steps -- bytes", + c: &Chunk{Samples: MetricSamples{ + {Timestamp: 1, Bytes: 2}, + {Timestamp: 5, Bytes: 10}, + }}, + metricType: Bytes, + start: 1, + end: 6, + expected: []logproto.PatternSample{ + {Timestamp: 0, Value: 2}, + {Timestamp: 2, Value: 0}, + {Timestamp: 4, Value: 10}, + }, + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + result, err := tc.c.ForRangeAndType(tc.metricType, tc.start, tc.end, model.Time(2)) + require.NoError(t, err) + if !reflect.DeepEqual(result, tc.expected) { + t.Errorf("Expected %v, got %v", tc.expected, result) + } + require.Equal(t, len(result), cap(result), "Returned slice wasn't created at the correct capacity") + }) + } +} diff --git a/pkg/pattern/metric/config.go b/pkg/pattern/metric/config.go new file mode 100644 index 0000000000000..8bdf85cdcb647 --- /dev/null +++ b/pkg/pattern/metric/config.go @@ -0,0 +1,16 @@ +package metric + +import "flag" + +type AggregationConfig struct { + Enabled bool `yaml:"enabled,omitempty" doc:"description=Whether the pattern ingester metric aggregation is enabled."` +} + +// RegisterFlags registers pattern ingester related flags. +func (cfg *AggregationConfig) RegisterFlags(fs *flag.FlagSet) { + cfg.RegisterFlagsWithPrefix(fs, "") +} + +func (cfg *AggregationConfig) RegisterFlagsWithPrefix(fs *flag.FlagSet, prefix string) { + fs.BoolVar(&cfg.Enabled, prefix+"metric-aggregation.enabled", false, "Flag to enable or disable metric aggregation.") +} diff --git a/pkg/pattern/metric/evaluator.go b/pkg/pattern/metric/evaluator.go new file mode 100644 index 0000000000000..86a4c83870b97 --- /dev/null +++ b/pkg/pattern/metric/evaluator.go @@ -0,0 +1,358 @@ +package metric + +import ( + "context" + "fmt" + "sort" + "time" + + loki_iter "github.com/grafana/loki/v3/pkg/iter" + "github.com/grafana/loki/v3/pkg/logproto" + "github.com/grafana/loki/v3/pkg/logql" + "github.com/grafana/loki/v3/pkg/logql/syntax" + "github.com/grafana/loki/v3/pkg/pattern/iter" + "github.com/grafana/loki/v3/pkg/storage/chunk/cache/resultscache" + "github.com/pkg/errors" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/promql" +) + +// TODO(twhitney): duplication with code in NewStepEvaluator +func ExtractMetricType(expr syntax.SampleExpr) (MetricType, error) { + var typ MetricType + switch e := expr.(type) { + case *syntax.VectorAggregationExpr: + if rangeExpr, ok := e.Left.(*syntax.RangeAggregationExpr); ok && e.Operation == syntax.OpTypeSum { + if rangeExpr.Operation == syntax.OpRangeTypeCount { + typ = Count + } else if rangeExpr.Operation == syntax.OpRangeTypeBytes { + typ = Bytes + } else { + return Unsupported, fmt.Errorf("unsupported aggregation operation %s", e.Operation) + } + } else { + return Unsupported, fmt.Errorf("unsupported aggregation operation %s", e.Operation) + } + case *syntax.RangeAggregationExpr: + if e.Operation == syntax.OpRangeTypeCount { + typ = Count + } else if e.Operation == syntax.OpRangeTypeBytes { + typ = Bytes + } else { + return Unsupported, fmt.Errorf("unsupported aggregation operation %s", e.Operation) + } + default: + return Unsupported, fmt.Errorf("unexpected expression type %T", e) + } + return typ, nil +} + +type SampleEvaluatorFactory interface { + // NewStepEvaluator returns a NewStepEvaluator for a given SampleExpr. + // It's explicitly passed another NewStepEvaluator + // in order to enable arbitrary computation of embedded expressions. This allows more modular & extensible + // NewStepEvaluator implementations which can be composed. + NewStepEvaluator( + ctx context.Context, + nextEvaluatorFactory SampleEvaluatorFactory, + expr syntax.SampleExpr, + typ MetricType, + from, through, step model.Time, + ) (logql.StepEvaluator, error) +} + +type SampleEvaluatorFunc func( + ctx context.Context, + nextEvaluatorFactory SampleEvaluatorFactory, + expr syntax.SampleExpr, + typ MetricType, + from, through, step model.Time, +) (logql.StepEvaluator, error) + +func (s SampleEvaluatorFunc) NewStepEvaluator( + ctx context.Context, + nextEvaluatorFactory SampleEvaluatorFactory, + expr syntax.SampleExpr, + typ MetricType, + from, through, step model.Time, +) (logql.StepEvaluator, error) { + return s(ctx, nextEvaluatorFactory, expr, typ, from, through, step) +} + +type DefaultEvaluatorFactory struct { + chunks *Chunks +} + +func NewDefaultEvaluatorFactory(chunks *Chunks) *DefaultEvaluatorFactory { + return &DefaultEvaluatorFactory{ + chunks: chunks, + } +} + +func (ev *DefaultEvaluatorFactory) NewStepEvaluator( + ctx context.Context, + evFactory SampleEvaluatorFactory, + expr syntax.SampleExpr, + typ MetricType, + from, through, step model.Time, +) (logql.StepEvaluator, error) { + switch e := expr.(type) { + case *syntax.VectorAggregationExpr: + if rangExpr, ok := e.Left.(*syntax.RangeAggregationExpr); ok && e.Operation == syntax.OpTypeSum { + // if range expression is wrapped with a vector expression + // we should send the vector expression for allowing reducing labels at the source. + evFactory = SampleEvaluatorFunc( + func(ctx context.Context, + _ SampleEvaluatorFactory, + _ syntax.SampleExpr, + typ MetricType, + from, through, step model.Time, + ) (logql.StepEvaluator, error) { + fromWithRangeAndOffset := from.Add(-rangExpr.Left.Interval).Add(-rangExpr.Left.Offset) + throughWithOffset := through.Add(-rangExpr.Left.Offset) + it, err := ev.chunks.Iterator(ctx, typ, fromWithRangeAndOffset, throughWithOffset, step) + if err != nil { + return nil, err + } + + params := NewParams( + e, + from.Add(-rangExpr.Left.Interval).Add(-rangExpr.Left.Offset), + through.Add(-rangExpr.Left.Offset), + step, + ) + return NewPatternSampleRangeAggEvaluator(iter.NewPeekingSampleIterator(it), rangExpr, params, rangExpr.Left.Offset) + }) + } + + if e.Grouping == nil { + return nil, errors.Errorf("aggregation operator '%q' without grouping", e.Operation) + } + nextEvaluator, err := evFactory.NewStepEvaluator(ctx, evFactory, e.Left, typ, from, through, step) + if err != nil { + return nil, err + } + sort.Strings(e.Grouping.Groups) + + return logql.NewVectorAggEvaluator( + nextEvaluator, + e, + make([]byte, 0, 1024), + labels.NewBuilder(labels.Labels{}), + ), nil + + case *syntax.RangeAggregationExpr: + fromWithRangeAndOffset := from.Add(-e.Left.Interval).Add(-e.Left.Offset) + throughWithOffset := through.Add(-e.Left.Offset) + it, err := ev.chunks.Iterator(ctx, typ, fromWithRangeAndOffset, throughWithOffset, step) + if err != nil { + return nil, err + } + + params := NewParams( + e, + from.Add(-e.Left.Interval).Add(-e.Left.Offset), + through.Add(-e.Left.Offset), + step, // expecting nanoseconds + ) + return NewPatternSampleRangeAggEvaluator(iter.NewPeekingSampleIterator(it), e, params, e.Left.Offset) + default: + return nil, errors.Errorf("unexpected expr type (%T)", e) + } +} + +// Need to create our own StepEvaluator since we only support bytes and count over time, +// and always sum to get those values. In order to accomplish this we need control over the +// aggregation operation.. +func NewPatternSampleRangeAggEvaluator( + it loki_iter.PeekingSampleIterator, + expr *syntax.RangeAggregationExpr, + q logql.Params, + o time.Duration, +) (logql.StepEvaluator, error) { + iter, err := newRangeVectorIterator( + it, expr, + expr.Left.Interval.Nanoseconds(), + q.Step().Nanoseconds(), + q.Start().UnixNano(), q.End().UnixNano(), o.Nanoseconds(), + ) + if err != nil { + return nil, err + } + + return logql.NewRangeVectorEvaluator(iter), nil +} + +func newRangeVectorIterator( + it loki_iter.PeekingSampleIterator, + expr *syntax.RangeAggregationExpr, + selRange, step, start, end, offset int64, +) (logql.RangeVectorIterator, error) { + // forces at least one step. + if step == 0 { + step = 1 + } + if offset != 0 { + start = start - offset + end = end - offset + } + // TODO(twhitney): do I need a streaming aggregator? + // if so the aggregator is going to make this + // a bit of a bad time, as there's currently no + // way to provide a custom one. + // + // var overlap bool + // if selRange >= step && start != end { + // overlap = true + // } + // if !overlap { + // _, err := streamingAggregator(expr) + // if err != nil { + // return nil, err + // } + // return &streamRangeVectorIterator{ + // iter: it, + // step: step, + // end: end, + // selRange: selRange, + // metrics: map[string]labels.Labels{}, + // r: expr, + // current: start - step, // first loop iteration will set it to start + // offset: offset, + // }, nil + // } + + // always sum + aggregator := logql.BatchRangeVectorAggregator(func(samples []promql.FPoint) float64 { + sum := 0.0 + for _, v := range samples { + sum += v.F + } + return sum + }) + + return logql.NewBatchRangeVectorIterator( + it, + selRange, + step, + start, + end, + offset, + aggregator, + ), nil +} + +type SeriesToSampleIterator struct { + floats []promql.FPoint + curTs int64 + cur float64 + lbls labels.Labels +} + +func NewSeriesToSampleIterator(series *promql.Series) *SeriesToSampleIterator { + return &SeriesToSampleIterator{ + floats: series.Floats, + lbls: series.Metric, + } +} + +func (s *SeriesToSampleIterator) Next() bool { + if len(s.floats) == 0 { + return false + } + + current, rest := s.floats[0], s.floats[1:] + + s.curTs = current.T + s.cur = current.F + + s.floats = rest + return true +} + +func (s *SeriesToSampleIterator) Pattern() string { + return "" +} + +func (s *SeriesToSampleIterator) Labels() labels.Labels { + return s.lbls +} + +func (s *SeriesToSampleIterator) At() logproto.PatternSample { + return logproto.PatternSample{ + Timestamp: model.Time(s.curTs), + Value: int64(s.cur), + } +} + +func (s *SeriesToSampleIterator) Error() error { + return nil +} + +func (s *SeriesToSampleIterator) Close() error { + return nil +} + +type paramCompat struct { + expr syntax.SampleExpr + from model.Time + through model.Time + step model.Time +} + +func NewParams( + expr syntax.SampleExpr, + from, through, step model.Time, +) *paramCompat { + return ¶mCompat{ + expr: expr, + from: from, + through: through, + step: step, + } +} + +func (p *paramCompat) QueryString() string { + return p.expr.String() +} + +func (p *paramCompat) Start() time.Time { + return p.from.Time() +} + +func (p *paramCompat) End() time.Time { + return p.through.Time() +} + +func (p *paramCompat) Step() time.Duration { + return time.Duration(p.step.UnixNano()) +} + +func (p *paramCompat) Interval() time.Duration { + return time.Duration(0) +} + +func (p *paramCompat) Limit() uint32 { + return 0 +} + +func (p *paramCompat) Direction() logproto.Direction { + return logproto.BACKWARD +} + +func (p *paramCompat) Shards() []string { + return []string{} +} + +func (p *paramCompat) GetExpression() syntax.Expr { + return p.expr +} + +func (p *paramCompat) GetStoreChunks() *logproto.ChunkRefGroup { + return nil +} + +func (p *paramCompat) CachingOptions() (res resultscache.CachingOptions) { + return +} diff --git a/pkg/pattern/metric/evaluator_test.go b/pkg/pattern/metric/evaluator_test.go new file mode 100644 index 0000000000000..9f23cb5546e6f --- /dev/null +++ b/pkg/pattern/metric/evaluator_test.go @@ -0,0 +1,363 @@ +package metric + +import ( + "context" + "testing" + + "github.com/grafana/loki/v3/pkg/logql" + "github.com/grafana/loki/v3/pkg/logql/syntax" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/model/labels" + "github.com/stretchr/testify/require" +) + +func Test_SampleEvaluator(t *testing.T) { + fiveMin := int64(300000) + stream := labels.Labels{ + labels.Label{ + Name: "foo", + Value: "bar", + }, + labels.Label{ + Name: "level", + Value: "debug", + }, + } + + setup := func(chunks Chunks, now int64, query string) logql.StepEvaluator { + factory := NewDefaultEvaluatorFactory(&chunks) + + expr, err := syntax.ParseSampleExpr(query) + require.NoError(t, err) + + typ, err := ExtractMetricType(expr) + require.NoError(t, err) + + evaluator, err := factory.NewStepEvaluator( + context.Background(), + factory, + expr.(syntax.SampleExpr), + typ, + model.Time(now-fiveMin), model.Time(now), model.Time(fiveMin), + ) + + require.NoError(t, err) + return evaluator + } + + chunks := func(now, then, beforeThen int64) Chunks { + nowTime := model.Time(now) + thenTime := model.Time(then) + beforeThenTime := model.Time(beforeThen) + return Chunks{ + chunks: []Chunk{ + { + Samples: []MetricSample{ + { + Timestamp: beforeThenTime, + Bytes: 1, + Count: 1, + }, + { + Timestamp: thenTime, + Bytes: 3, + Count: 2, + }, + { + Timestamp: nowTime, + Bytes: 5, + Count: 3, + }, + }, + mint: thenTime.Unix(), + maxt: nowTime.Unix(), + }, + }, + labels: stream, + } + } + + t.Run("grouping", func(t *testing.T) { + group := labels.Labels{ + labels.Label{ + Name: "level", + Value: "debug", + }, + } + t.Run("evenly aligned, non-overlapping timestamps", func(t *testing.T) { + now := int64(1715964275000) + then := now - fiveMin // 1715963975000 -- 5m before n + beforeThen := then - fiveMin // 1715963675000 -- 5m before then + chks := chunks(now, then, beforeThen) + + t.Run("count", func(t *testing.T) { + evaluator := setup(chks, now, `sum by (level)(count_over_time({foo="bar"}[5m]))`) + + resultTs := make([]int64, 3) + resultVals := make([]float64, 3) + + for i := 0; i < 3; i++ { + ok, ts, r := evaluator.Next() + require.True(t, ok) + require.True(t, then-fiveMin <= ts && ts <= now) + + // The []promqlSample in the Vector is reused, so we need to copy the value and timestamp. + resultTs[i] = r.SampleVector()[0].T + resultVals[i] = r.SampleVector()[0].F + + require.Equal(t, group, r.SampleVector()[0].Metric) + } + + ok, _, _ := evaluator.Next() + require.False(t, ok) + + require.Equal(t, beforeThen, resultTs[0]) + require.Equal(t, then, resultTs[1]) + require.Equal(t, now, resultTs[2]) + + require.Equal(t, float64(1), resultVals[0]) + require.Equal(t, float64(2), resultVals[1]) + require.Equal(t, float64(3), resultVals[2]) + }) + + t.Run("bytes", func(t *testing.T) { + evaluator := setup(chks, now, `sum by (level)(bytes_over_time({foo="bar"}[5m]))`) + + resultTs := make([]int64, 3) + resultVals := make([]float64, 3) + + for i := 0; i < 3; i++ { + ok, ts, r := evaluator.Next() + require.True(t, ok) + require.True(t, then-fiveMin <= ts && ts <= now) + + // The []promqlSample in the Vector is reused, so we need to copy the value and timestamp. + resultTs[i] = r.SampleVector()[0].T + resultVals[i] = r.SampleVector()[0].F + + require.Equal(t, group, r.SampleVector()[0].Metric) + } + + ok, _, _ := evaluator.Next() + require.False(t, ok) + + require.Equal(t, beforeThen, resultTs[0]) + require.Equal(t, then, resultTs[1]) + require.Equal(t, now, resultTs[2]) + + require.Equal(t, float64(1), resultVals[0]) + require.Equal(t, float64(3), resultVals[1]) // TODO: got 2, expected 3 + require.Equal(t, float64(5), resultVals[2]) + }) + }) + + t.Run("evenly aligned, overlapping timestamps", func(t *testing.T) { + now := int64(1715964275000) + then := now - 150000 // 1715964125000 -- 2.5m before now + beforeThen := then - 450000 // 1715963675000 -- 7.5m before then, 10m before now + chks := chunks(now, then, beforeThen) + + t.Run("count", func(t *testing.T) { + evaluator := setup(chks, now, `sum by (level)(count_over_time({foo="bar"}[5m]))`) + + resultTs := make([]int64, 3) + resultVals := make([]float64, 3) + + start := (now - fiveMin - fiveMin) // from - step + for i := 0; i < 3; i++ { + ok, ts, r := evaluator.Next() + require.True(t, ok) + require.True(t, start <= ts && ts <= now) + + // The []promqlSample in the Vector is reused, so we need to copy the value and timestamp. + resultTs[i] = r.SampleVector()[0].T + resultVals[i] = r.SampleVector()[0].F + + require.Equal(t, group, r.SampleVector()[0].Metric) + } + + ok, _, _ := evaluator.Next() + require.False(t, ok) + + require.Equal(t, now-600000, resultTs[0]) + require.Equal(t, now-fiveMin, resultTs[1]) + require.Equal(t, now, resultTs[2]) + + require.Equal(t, float64(1), resultVals[0]) + require.Equal(t, float64(0), resultVals[1]) + require.Equal(t, float64(5), resultVals[2]) + }) + + t.Run("bytes", func(t *testing.T) { + evaluator := setup(chks, now, `sum by (level)(bytes_over_time({foo="bar"}[5m]))`) + + resultTs := make([]int64, 3) + resultVals := make([]float64, 3) + + start := (now - fiveMin - fiveMin) // from - step + for i := 0; i < 3; i++ { + ok, ts, r := evaluator.Next() + require.True(t, ok) + require.True(t, start <= ts && ts <= now) + + // The []promqlSample in the Vector is reused, so we need to copy the value and timestamp. + resultTs[i] = r.SampleVector()[0].T + resultVals[i] = r.SampleVector()[0].F + + require.Equal(t, group, r.SampleVector()[0].Metric) + } + + ok, _, _ := evaluator.Next() + require.False(t, ok) + + require.Equal(t, now-600000, resultTs[0]) + require.Equal(t, now-fiveMin, resultTs[1]) + require.Equal(t, now, resultTs[2]) + + require.Equal(t, float64(1), resultVals[0]) + require.Equal(t, float64(0), resultVals[1]) + require.Equal(t, float64(8), resultVals[2]) + }) + }) + }) + + t.Run("without grouping", func(t *testing.T) { + t.Run("evenly aligned, non-overlapping timestamps", func(t *testing.T) { + now := int64(1715964275000) + then := now - fiveMin // 1715963975000 -- 5m before n + beforeThen := then - fiveMin // 1715963675000 -- 5m before then + chks := chunks(now, then, beforeThen) + + t.Run("count", func(t *testing.T) { + evaluator := setup(chks, now, `count_over_time({foo="bar"}[5m])`) + + resultTs := make([]int64, 3) + resultVals := make([]float64, 3) + + for i := 0; i < 3; i++ { + ok, ts, r := evaluator.Next() + require.True(t, ok) + require.True(t, then-fiveMin <= ts && ts <= now) + + // The []promqlSample in the Vector is reused, so we need to copy the value and timestamp. + samples := r.SampleVector() + resultTs[i] = samples[0].T + resultVals[i] = samples[0].F + + require.Equal(t, stream, samples[0].Metric) + } + + ok, _, _ := evaluator.Next() + require.False(t, ok) + + require.Equal(t, beforeThen, resultTs[0]) + require.Equal(t, then, resultTs[1]) + require.Equal(t, now, resultTs[2]) + + require.Equal(t, float64(1), resultVals[0]) + require.Equal(t, float64(2), resultVals[1]) + require.Equal(t, float64(3), resultVals[2]) + }) + + t.Run("bytes", func(t *testing.T) { + evaluator := setup(chks, now, `bytes_over_time({foo="bar"}[5m])`) + + resultTs := make([]int64, 3) + resultVals := make([]float64, 3) + + for i := 0; i < 3; i++ { + ok, ts, r := evaluator.Next() + require.True(t, ok) + require.True(t, then-fiveMin <= ts && ts <= now) + + // The []promqlSample in the Vector is reused, so we need to copy the value and timestamp. + samples := r.SampleVector() + resultTs[i] = samples[0].T + resultVals[i] = samples[0].F + + require.Equal(t, stream, samples[0].Metric) + } + + ok, _, _ := evaluator.Next() + require.False(t, ok) + + require.Equal(t, beforeThen, resultTs[0]) + require.Equal(t, then, resultTs[1]) + require.Equal(t, now, resultTs[2]) + + require.Equal(t, float64(1), resultVals[0]) + require.Equal(t, float64(3), resultVals[1]) + require.Equal(t, float64(5), resultVals[2]) + }) + }) + + t.Run("evenly aligned, overlapping timestamps", func(t *testing.T) { + now := int64(1715964275000) + then := now - 150000 // 1715964125000 -- 2.5m before now + beforeThen := then - 450000 // 1715963675000 -- 7.5m before then, 10m before now + chks := chunks(now, then, beforeThen) + + t.Run("count", func(t *testing.T) { + evaluator := setup(chks, now, `count_over_time({foo="bar"}[5m])`) + + resultTs := make([]int64, 3) + resultVals := make([]float64, 3) + + start := (now - fiveMin - fiveMin) // from - step + for i := 0; i < 3; i++ { + ok, ts, r := evaluator.Next() + require.True(t, ok) + require.True(t, start <= ts && ts <= now) + + // The []promqlSample in the Vector is reused, so we need to copy the value and timestamp. + resultTs[i] = r.SampleVector()[0].T + resultVals[i] = r.SampleVector()[0].F + + require.Equal(t, stream, r.SampleVector()[0].Metric) + } + + ok, _, _ := evaluator.Next() + require.False(t, ok) + + require.Equal(t, now-600000, resultTs[0]) + require.Equal(t, now-fiveMin, resultTs[1]) + require.Equal(t, now, resultTs[2]) + + require.Equal(t, float64(1), resultVals[0]) + require.Equal(t, float64(0), resultVals[1]) + require.Equal(t, float64(5), resultVals[2]) + }) + + t.Run("bytes", func(t *testing.T) { + evaluator := setup(chks, now, `bytes_over_time({foo="bar"}[5m])`) + + resultTs := make([]int64, 3) + resultVals := make([]float64, 3) + + start := (now - fiveMin - fiveMin) // from - step + for i := 0; i < 3; i++ { + ok, ts, r := evaluator.Next() + require.True(t, ok) + require.True(t, start <= ts && ts <= now) + + // The []promqlSample in the Vector is reused, so we need to copy the value and timestamp. + resultTs[i] = r.SampleVector()[0].T + resultVals[i] = r.SampleVector()[0].F + + require.Equal(t, stream, r.SampleVector()[0].Metric) + } + + ok, _, _ := evaluator.Next() + require.False(t, ok) + + require.Equal(t, now-600000, resultTs[0]) + require.Equal(t, now-fiveMin, resultTs[1]) + require.Equal(t, now, resultTs[2]) + + require.Equal(t, float64(1), resultVals[0]) + require.Equal(t, float64(0), resultVals[1]) + require.Equal(t, float64(8), resultVals[2]) + }) + }) + }) +} diff --git a/pkg/pattern/ring_client.go b/pkg/pattern/ring_client.go index 3ceaf481a3b9b..d1421b842422f 100644 --- a/pkg/pattern/ring_client.go +++ b/pkg/pattern/ring_client.go @@ -13,7 +13,13 @@ import ( "github.com/grafana/loki/v3/pkg/pattern/clientpool" ) -type RingClient struct { +type RingClient interface { + services.Service + Ring() ring.ReadRing + Pool() *ring_client.Pool +} + +type ringClient struct { cfg Config logger log.Logger @@ -29,10 +35,10 @@ func NewRingClient( metricsNamespace string, registerer prometheus.Registerer, logger log.Logger, -) (*RingClient, error) { +) (RingClient, error) { var err error registerer = prometheus.WrapRegistererWithPrefix(metricsNamespace+"_", registerer) - ringClient := &RingClient{ + ringClient := &ringClient{ logger: log.With(logger, "component", "pattern-ring-client"), cfg: cfg, } @@ -59,19 +65,81 @@ func NewRingClient( return ringClient, nil } -func (q *RingClient) starting(ctx context.Context) error { - return services.StartManagerAndAwaitHealthy(ctx, q.subservices) +func (r *ringClient) starting(ctx context.Context) error { + return services.StartManagerAndAwaitHealthy(ctx, r.subservices) } -func (q *RingClient) running(ctx context.Context) error { +func (r *ringClient) running(ctx context.Context) error { select { case <-ctx.Done(): return nil - case err := <-q.subservicesWatcher.Chan(): + case err := <-r.subservicesWatcher.Chan(): return fmt.Errorf("pattern tee subservices failed: %w", err) } } -func (q *RingClient) stopping(_ error) error { - return services.StopManagerAndAwaitStopped(context.Background(), q.subservices) +func (r *ringClient) stopping(_ error) error { + return services.StopManagerAndAwaitStopped(context.Background(), r.subservices) +} + +func (r *ringClient) Ring() ring.ReadRing { + return r.ring +} + +func (r *ringClient) Pool() *ring_client.Pool { + return r.pool +} + +// StartAsync starts Service asynchronously. Service must be in New State, otherwise error is returned. +// Context is used as a parent context for service own context. +func (r *ringClient) StartAsync(ctx context.Context) error { + return r.StartAsync(ctx) +} + +// AwaitRunning waits until service gets into Running state. +// If service is in New or Starting state, this method is blocking. +// If service is already in Running state, returns immediately with no error. +// If service is in a state, from which it cannot get into Running state, error is returned immediately. +func (r *ringClient) AwaitRunning(ctx context.Context) error { + return r.AwaitRunning(ctx) +} + +// StopAsync tell the service to stop. This method doesn't block and can be called multiple times. +// If Service is New, it is Terminated without having been started nor stopped. +// If Service is in Starting or Running state, this initiates shutdown and returns immediately. +// If Service has already been stopped, this method returns immediately, without taking action. +func (r *ringClient) StopAsync() { + r.StopAsync() +} + +// AwaitTerminated waits for the service to reach Terminated or Failed state. If service is already in one of these states, +// when method is called, method returns immediately. +// If service enters Terminated state, this method returns nil. +// If service enters Failed state, or context is finished before reaching Terminated or Failed, error is returned. +func (r *ringClient) AwaitTerminated(ctx context.Context) error { + return r.AwaitTerminated(ctx) +} + +// FailureCase returns error if Service is in Failed state. +// If Service is not in Failed state, this method returns nil. +func (r *ringClient) FailureCase() error { + return r.FailureCase() +} + +// State returns current state of the service. +func (r *ringClient) State() services.State { + return r.State() +} + +// AddListener adds listener to this service. Listener will be notified on subsequent state transitions +// of the service. Previous state transitions are not replayed, so it is suggested to add listeners before +// service is started. +// +// AddListener guarantees execution ordering across calls to a given listener but not across calls to +// multiple listeners. Specifically, a given listener will have its callbacks invoked in the same order +// as the service enters those states. Additionally, at most one of the listener's callbacks will execute +// at once. However, multiple listeners' callbacks may execute concurrently, and listeners may execute +// in an order different from the one in which they were registered. +func (r *ringClient) AddListener(listener services.Listener) { + r.AddListener(listener) } diff --git a/pkg/pattern/stream.go b/pkg/pattern/stream.go index f3aad280250db..0c08790dd7087 100644 --- a/pkg/pattern/stream.go +++ b/pkg/pattern/stream.go @@ -2,15 +2,21 @@ package pattern import ( "context" + "math" "sync" "time" "github.com/grafana/loki/v3/pkg/logproto" + "github.com/grafana/loki/v3/pkg/logql" + "github.com/grafana/loki/v3/pkg/logql/syntax" + "github.com/grafana/loki/v3/pkg/logqlmodel" "github.com/grafana/loki/v3/pkg/pattern/drain" "github.com/grafana/loki/v3/pkg/pattern/iter" + "github.com/grafana/loki/v3/pkg/pattern/metric" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/promql" ) type stream struct { @@ -21,6 +27,11 @@ type stream struct { patterns *drain.Drain mtx sync.Mutex + aggregateMetrics bool + metrics *metric.Chunks + + evaluator metric.SampleEvaluatorFactory + lastTs int64 } @@ -28,8 +39,9 @@ func newStream( fp model.Fingerprint, labels labels.Labels, metrics *ingesterMetrics, + aggregateMetrics bool, ) (*stream, error) { - return &stream{ + stream := &stream{ fp: fp, labels: labels, labelsString: labels.String(), @@ -38,7 +50,16 @@ func newStream( PatternsEvictedTotal: metrics.patternsDiscardedTotal, PatternsDetectedTotal: metrics.patternsDetectedTotal, }), - }, nil + aggregateMetrics: aggregateMetrics, + } + + if aggregateMetrics { + chunks := metric.NewChunks(labels) + stream.metrics = chunks + stream.evaluator = metric.NewDefaultEvaluatorFactory(chunks) + } + + return stream, nil } func (s *stream) Push( @@ -48,13 +69,22 @@ func (s *stream) Push( s.mtx.Lock() defer s.mtx.Unlock() + bytes := uint64(0) + count := uint64(len(entries)) for _, entry := range entries { if entry.Timestamp.UnixNano() < s.lastTs { continue } + + bytes += uint64(len(entry.Line)) + s.lastTs = entry.Timestamp.UnixNano() s.patterns.Train(entry.Line, entry.Timestamp.UnixNano()) } + + if s.aggregateMetrics && s.metrics != nil { + s.metrics.Observe(bytes, count, model.TimeFromUnixNano(s.lastTs)) + } return nil } @@ -75,6 +105,142 @@ func (s *stream) Iterator(_ context.Context, from, through, step model.Time) (it return iter.NewMerge(iters...), nil } +// TODO(twhitney): duplication between bytes and count iterators +func (s *stream) BytesIterator( + ctx context.Context, + expr syntax.SampleExpr, + from, through, step model.Time, +) (iter.Iterator, error) { + s.mtx.Lock() + defer s.mtx.Unlock() + + stepEvaluator, err := s.evaluator.NewStepEvaluator( + ctx, + s.evaluator, + expr, + metric.Bytes, + from, + through, + step, + ) + if err != nil { + return nil, err + } + + next, ts, r := stepEvaluator.Next() + if stepEvaluator.Error() != nil { + return nil, stepEvaluator.Error() + } + + // TODO(twhitney): actually get max series from limits + // this is only 1 series since we're already on a stream + // this this limit needs to also be enforced higher up + maxSeries := 1000 + series, err := s.JoinSampleVector( + next, + ts, + r, + stepEvaluator, + maxSeries, + from, through, step) + if err != nil { + return nil, err + } + + return metric.NewSeriesToSampleIterator(series), nil +} + +func (s *stream) JoinSampleVector( + next bool, + ts int64, + r logql.StepResult, + stepEvaluator logql.StepEvaluator, + maxSeries int, + from, through, step model.Time, +) (*promql.Series, error) { + stepCount := int(math.Ceil(float64(through.Sub(from).Nanoseconds()) / float64(step.UnixNano()))) + if stepCount <= 0 { + stepCount = 1 + } + + series := &promql.Series{ + Metric: s.labels, + Floats: make([]promql.FPoint, 0, stepCount), + } + + vec := promql.Vector{} + if next { + vec = r.SampleVector() + } + + // fail fast for the first step or instant query + if len(vec) > maxSeries { + return nil, logqlmodel.NewSeriesLimitError(maxSeries) + } + + for next { + vec = r.SampleVector() + for _, p := range vec { + series.Floats = append(series.Floats, promql.FPoint{ + T: ts, + F: p.F, + }) + } + + next, ts, r = stepEvaluator.Next() + if stepEvaluator.Error() != nil { + return nil, stepEvaluator.Error() + } + } + + return series, stepEvaluator.Error() +} + +// TODO(twhitney): duplication between bytes and count iterators +func (s *stream) CountIterator( + ctx context.Context, + expr syntax.SampleExpr, + from, through, step model.Time, +) (iter.Iterator, error) { + s.mtx.Lock() + defer s.mtx.Unlock() + + stepEvaluator, err := s.evaluator.NewStepEvaluator( + ctx, + s.evaluator, + expr, + metric.Count, + from, + through, + step, + ) + if err != nil { + return nil, err + } + + next, ts, r := stepEvaluator.Next() + if stepEvaluator.Error() != nil { + return nil, stepEvaluator.Error() + } + + // TODO(twhitney): actually get max series from limits + // this is only 1 series since we're already on a stream + // this this limit needs to also be enforced higher up + maxSeries := 1000 + series, err := s.JoinSampleVector( + next, + ts, + r, + stepEvaluator, + maxSeries, + from, through, step) + if err != nil { + return nil, err + } + + return metric.NewSeriesToSampleIterator(series), nil +} + func (s *stream) prune(olderThan time.Duration) bool { s.mtx.Lock() defer s.mtx.Unlock() diff --git a/pkg/pattern/stream_test.go b/pkg/pattern/stream_test.go index f2218816b1113..7f6f98d15b212 100644 --- a/pkg/pattern/stream_test.go +++ b/pkg/pattern/stream_test.go @@ -16,7 +16,7 @@ import ( func TestAddStream(t *testing.T) { lbs := labels.New(labels.Label{Name: "test", Value: "test"}) - stream, err := newStream(model.Fingerprint(lbs.Hash()), lbs, newIngesterMetrics(nil, "test")) + stream, err := newStream(model.Fingerprint(lbs.Hash()), lbs, newIngesterMetrics(nil, "test"), false) require.NoError(t, err) err = stream.Push(context.Background(), []push.Entry{ @@ -36,7 +36,7 @@ func TestAddStream(t *testing.T) { require.NoError(t, err) it, err := stream.Iterator(context.Background(), model.Earliest, model.Latest, model.Time(time.Second)) require.NoError(t, err) - res, err := iter.ReadAll(it) + res, err := iter.ReadAllWithPatterns(it) require.NoError(t, err) require.Equal(t, 1, len(res.Series)) require.Equal(t, int64(2), res.Series[0].Samples[0].Value) @@ -44,7 +44,12 @@ func TestAddStream(t *testing.T) { func TestPruneStream(t *testing.T) { lbs := labels.New(labels.Label{Name: "test", Value: "test"}) - stream, err := newStream(model.Fingerprint(lbs.Hash()), lbs, newIngesterMetrics(nil, "test")) + stream, err := newStream( + model.Fingerprint(lbs.Hash()), + lbs, + newIngesterMetrics(nil, "test"), + false, + ) require.NoError(t, err) err = stream.Push(context.Background(), []push.Entry{ @@ -70,7 +75,7 @@ func TestPruneStream(t *testing.T) { require.Equal(t, false, stream.prune(time.Hour)) it, err := stream.Iterator(context.Background(), model.Earliest, model.Latest, model.Time(time.Second)) require.NoError(t, err) - res, err := iter.ReadAll(it) + res, err := iter.ReadAllWithPatterns(it) require.NoError(t, err) require.Equal(t, 1, len(res.Series)) require.Equal(t, int64(1), res.Series[0].Samples[0].Value) diff --git a/pkg/pattern/tee.go b/pkg/pattern/tee.go index 70fb37e1b6929..ed90d0dd478c9 100644 --- a/pkg/pattern/tee.go +++ b/pkg/pattern/tee.go @@ -18,14 +18,14 @@ import ( type Tee struct { cfg Config logger log.Logger - ringClient *RingClient + ringClient RingClient ingesterAppends *prometheus.CounterVec } func NewTee( cfg Config, - ringClient *RingClient, + ringClient RingClient, metricsNamespace string, registerer prometheus.Registerer, logger log.Logger, @@ -38,8 +38,8 @@ func NewTee( Name: "pattern_ingester_appends_total", Help: "The total number of batch appends sent to pattern ingesters.", }, []string{"ingester", "status"}), - cfg: cfg, - ringClient: ringClient, + cfg: cfg, + ringClient: ringClient, } return t, nil @@ -58,7 +58,7 @@ func (t *Tee) Duplicate(tenant string, streams []distributor.KeyedStream) { func (t *Tee) sendStream(tenant string, stream distributor.KeyedStream) error { var descs [1]ring.InstanceDesc - replicationSet, err := t.ringClient.ring.Get(stream.HashKey, ring.WriteNoExtend, descs[:0], nil, nil) + replicationSet, err := t.ringClient.Ring().Get(stream.HashKey, ring.WriteNoExtend, descs[:0], nil, nil) if err != nil { return err } @@ -66,7 +66,7 @@ func (t *Tee) sendStream(tenant string, stream distributor.KeyedStream) error { return errors.New("no instances found") } addr := replicationSet.Instances[0].Addr - client, err := t.ringClient.pool.GetClientFor(addr) + client, err := t.ringClient.Pool().GetClientFor(addr) if err != nil { return err } diff --git a/pkg/querier/querier.go b/pkg/querier/querier.go index f2fe80566b461..b05d9411faafc 100644 --- a/pkg/querier/querier.go +++ b/pkg/querier/querier.go @@ -1047,6 +1047,10 @@ func (q *SingleTenantQuerier) Patterns(ctx context.Context, req *logproto.QueryP return nil, httpgrpc.Errorf(http.StatusNotFound, "") } res, err := q.patternQuerier.Patterns(ctx, req) + if err != nil { + return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error()) + } + return res, err } diff --git a/pkg/storage/bloom/v1/bloom.go b/pkg/storage/bloom/v1/bloom.go index 2f195e36df2b9..aa51762d4e4ec 100644 --- a/pkg/storage/bloom/v1/bloom.go +++ b/pkg/storage/bloom/v1/bloom.go @@ -24,7 +24,7 @@ type Bloom struct { func (b *Bloom) Encode(enc *encoding.Encbuf) error { // divide by 8 b/c bloom capacity is measured in bits, but we want bytes - buf := bytes.NewBuffer(BlockPool.Get(int(b.Capacity() / 8))) + buf := bytes.NewBuffer(BloomPagePool.Get(int(b.Capacity() / 8))) // TODO(owen-d): have encoder implement writer directly so we don't need // to indirect via a buffer @@ -36,7 +36,7 @@ func (b *Bloom) Encode(enc *encoding.Encbuf) error { data := buf.Bytes() enc.PutUvarint(len(data)) // length of bloom filter enc.PutBytes(data) - BlockPool.Put(data[:0]) // release to pool + BloomPagePool.Put(data[:0]) // release to pool return nil } @@ -65,8 +65,8 @@ func (b *Bloom) Decode(dec *encoding.Decbuf) error { } func LazyDecodeBloomPage(r io.Reader, pool chunkenc.ReaderPool, page BloomPageHeader) (*BloomPageDecoder, error) { - data := BlockPool.Get(page.Len)[:page.Len] - defer BlockPool.Put(data) + data := BloomPagePool.Get(page.Len)[:page.Len] + defer BloomPagePool.Put(data) _, err := io.ReadFull(r, data) if err != nil { @@ -84,7 +84,7 @@ func LazyDecodeBloomPage(r io.Reader, pool chunkenc.ReaderPool, page BloomPageHe } defer pool.PutReader(decompressor) - b := BlockPool.Get(page.DecompressedLen)[:page.DecompressedLen] + b := BloomPagePool.Get(page.DecompressedLen)[:page.DecompressedLen] if _, err = io.ReadFull(decompressor, b); err != nil { return nil, errors.Wrap(err, "decompressing bloom page") @@ -101,7 +101,7 @@ func LazyDecodeBloomPageNoCompression(r io.Reader, page BloomPageHeader) (*Bloom if page.Len != page.DecompressedLen+4 { return nil, errors.New("the Len and DecompressedLen of the page do not match") } - data := BlockPool.Get(page.Len)[:page.Len] + data := BloomPagePool.Get(page.Len)[:page.Len] _, err := io.ReadFull(r, data) if err != nil { @@ -163,7 +163,7 @@ func (d *BloomPageDecoder) Relinquish() { d.data = nil if cap(data) > 0 { - BlockPool.Put(data) + BloomPagePool.Put(data) } } diff --git a/pkg/storage/bloom/v1/index.go b/pkg/storage/bloom/v1/index.go index 674a1a883dfba..a4bcd450650c0 100644 --- a/pkg/storage/bloom/v1/index.go +++ b/pkg/storage/bloom/v1/index.go @@ -155,7 +155,7 @@ func (b *BlockIndex) NewSeriesPageDecoder(r io.ReadSeeker, header SeriesPageHead defer func() { if err != nil { metrics.pagesSkipped.WithLabelValues(pageTypeSeries, skipReasonErr).Inc() - metrics.bytesSkipped.WithLabelValues(pageTypeSeries).Add(float64(header.DecompressedLen)) + metrics.bytesSkipped.WithLabelValues(pageTypeSeries, skipReasonErr).Add(float64(header.DecompressedLen)) } else { metrics.pagesRead.WithLabelValues(pageTypeSeries).Inc() metrics.bytesRead.WithLabelValues(pageTypeSeries).Add(float64(header.DecompressedLen)) @@ -166,8 +166,8 @@ func (b *BlockIndex) NewSeriesPageDecoder(r io.ReadSeeker, header SeriesPageHead return nil, errors.Wrap(err, "seeking to series page") } - data := BlockPool.Get(header.Len)[:header.Len] - defer BlockPool.Put(data) + data := SeriesPagePool.Get(header.Len)[:header.Len] + defer SeriesPagePool.Put(data) _, err = io.ReadFull(r, data) if err != nil { return nil, errors.Wrap(err, "reading series page") diff --git a/pkg/storage/bloom/v1/util.go b/pkg/storage/bloom/v1/util.go index 06cf1f6add227..22fb47e43e799 100644 --- a/pkg/storage/bloom/v1/util.go +++ b/pkg/storage/bloom/v1/util.go @@ -32,10 +32,21 @@ var ( }, } - // 4KB -> 128MB - BlockPool = BytePool{ + // buffer pool for series pages + // 1KB 2KB 4KB 8KB 16KB 32KB 64KB 128KB + SeriesPagePool = BytePool{ pool: pool.New( - 4<<10, 128<<20, 2, + 1<<10, 128<<10, 2, + func(size int) interface{} { + return make([]byte, size) + }), + } + + // buffer pool for bloom pages + // 128KB 256KB 512KB 1MB 2MB 4MB 8MB 16MB 32MB 64MB 128MB + BloomPagePool = BytePool{ + pool: pool.New( + 128<<10, 128<<20, 2, func(size int) interface{} { return make([]byte, size) }), diff --git a/pkg/util/limiter/combined_limits.go b/pkg/util/limiter/combined_limits.go index 39684c7b43e8e..92caf2c19d681 100644 --- a/pkg/util/limiter/combined_limits.go +++ b/pkg/util/limiter/combined_limits.go @@ -1,6 +1,8 @@ package limiter import ( + bloombuilder "github.com/grafana/loki/v3/pkg/bloombuild/builder" + bloomplanner "github.com/grafana/loki/v3/pkg/bloombuild/planner" "github.com/grafana/loki/v3/pkg/bloomcompactor" "github.com/grafana/loki/v3/pkg/bloomgateway" "github.com/grafana/loki/v3/pkg/compactor" @@ -26,4 +28,6 @@ type CombinedLimits interface { indexgateway.Limits bloomgateway.Limits bloomcompactor.Limits + bloomplanner.Limits + bloombuilder.Limits } diff --git a/pkg/util/marshal/marshal.go b/pkg/util/marshal/marshal.go index 1a4d6701b1b18..609c7ede41d0a 100644 --- a/pkg/util/marshal/marshal.go +++ b/pkg/util/marshal/marshal.go @@ -202,8 +202,14 @@ func WriteQueryPatternsResponseJSON(r *logproto.QueryPatternsResponse, w io.Writ if len(r.Series) > 0 { for i, series := range r.Series { s.WriteObjectStart() - s.WriteObjectField("pattern") - s.WriteStringWithHTMLEscaped(series.Pattern) + if pattern := series.GetPattern(); pattern != "" { + s.WriteObjectField("pattern") + s.WriteStringWithHTMLEscaped(pattern) + } + if labels := series.GetLabels(); labels != "" { + s.WriteObjectField("labels") + s.WriteStringWithHTMLEscaped(labels) + } s.WriteMore() s.WriteObjectField("samples") s.WriteArrayStart() diff --git a/pkg/util/marshal/marshal_test.go b/pkg/util/marshal/marshal_test.go index c749677f77026..4688388744c9d 100644 --- a/pkg/util/marshal/marshal_test.go +++ b/pkg/util/marshal/marshal_test.go @@ -1077,13 +1077,11 @@ func Test_WriteQueryPatternsResponseJSON(t *testing.T) { { &logproto.QueryPatternsResponse{ Series: []*logproto.PatternSeries{ - { - Pattern: "foo <*> bar", - Samples: []*logproto.PatternSample{ - {Timestamp: model.TimeFromUnix(1), Value: 1}, - {Timestamp: model.TimeFromUnix(2), Value: 2}, - }, + logproto.NewPatternSeriesWithPattern("foo <*> bar", []*logproto.PatternSample{ + {Timestamp: model.TimeFromUnix(1), Value: 1}, + {Timestamp: model.TimeFromUnix(2), Value: 2}, }, + ), }, }, `{"status":"success","data":[{"pattern":"foo <*> bar","samples":[[1,1],[2,2]]}]}`, @@ -1091,20 +1089,17 @@ func Test_WriteQueryPatternsResponseJSON(t *testing.T) { { &logproto.QueryPatternsResponse{ Series: []*logproto.PatternSeries{ - { - Pattern: "foo <*> bar", - Samples: []*logproto.PatternSample{ - {Timestamp: model.TimeFromUnix(1), Value: 1}, - {Timestamp: model.TimeFromUnix(2), Value: 2}, - }, + logproto.NewPatternSeriesWithPattern("foo <*> bar", []*logproto.PatternSample{ + {Timestamp: model.TimeFromUnix(1), Value: 1}, + {Timestamp: model.TimeFromUnix(2), Value: 2}, }, - { - Pattern: "foo <*> buzz", - Samples: []*logproto.PatternSample{ + ), + logproto.NewPatternSeriesWithPattern("foo <*> buzz", + []*logproto.PatternSample{ {Timestamp: model.TimeFromUnix(3), Value: 1}, {Timestamp: model.TimeFromUnix(3), Value: 2}, }, - }, + ), }, }, `{"status":"success","data":[{"pattern":"foo <*> bar","samples":[[1,1],[2,2]]},{"pattern":"foo <*> buzz","samples":[[3,1],[3,2]]}]}`, @@ -1112,17 +1107,58 @@ func Test_WriteQueryPatternsResponseJSON(t *testing.T) { { &logproto.QueryPatternsResponse{ Series: []*logproto.PatternSeries{ - { - Pattern: "foo <*> bar", - Samples: []*logproto.PatternSample{}, + logproto.NewPatternSeriesWithPattern("foo <*> bar", + []*logproto.PatternSample{}, + ), + logproto.NewPatternSeriesWithPattern("foo <*> buzz", + []*logproto.PatternSample{}, + ), + }, + }, + `{"status":"success","data":[{"pattern":"foo <*> bar","samples":[]},{"pattern":"foo <*> buzz","samples":[]}]}`, + }, + { + &logproto.QueryPatternsResponse{ + Series: []*logproto.PatternSeries{ + logproto.NewPatternSeriesWithLabels(`{foo="bar"}`, []*logproto.PatternSample{ + {Timestamp: model.TimeFromUnix(1), Value: 1}, + {Timestamp: model.TimeFromUnix(2), Value: 2}, }, - { - Pattern: "foo <*> buzz", - Samples: []*logproto.PatternSample{}, + ), + }, + }, + `{"status":"success","data":[{"labels":"{foo=\"bar\"}","samples":[[1,1],[2,2]]}]}`, + }, + { + &logproto.QueryPatternsResponse{ + Series: []*logproto.PatternSeries{ + logproto.NewPatternSeriesWithLabels(`{foo="bar"}`, []*logproto.PatternSample{ + {Timestamp: model.TimeFromUnix(1), Value: 1}, + {Timestamp: model.TimeFromUnix(2), Value: 2}, }, + ), + logproto.NewPatternSeriesWithLabels(`{foo="buzz"}`, + []*logproto.PatternSample{ + {Timestamp: model.TimeFromUnix(3), Value: 1}, + {Timestamp: model.TimeFromUnix(3), Value: 2}, + }, + ), }, }, - `{"status":"success","data":[{"pattern":"foo <*> bar","samples":[]},{"pattern":"foo <*> buzz","samples":[]}]}`, + `{"status":"success","data":[{"labels":"{foo=\"bar\"}","samples":[[1,1],[2,2]]},{"labels":"{foo=\"buzz\"}","samples":[[3,1],[3,2]]}]}`, + }, + { + &logproto.QueryPatternsResponse{ + Series: []*logproto.PatternSeries{ + logproto.NewPatternSeriesWithLabels(`{foo="bar"}`, + []*logproto.PatternSample{}, + ), + logproto.NewPatternSeriesWithPattern(`{foo="buzz"}`, + []*logproto.PatternSample{}, + ), + }, + }, + `{"status":"success","data":[{"labels":"{foo=\"bar\"}","samples":[]},{"pattern":"{foo=\"buzz\"}","samples":[]}]}`, }, } { tc := tc diff --git a/pkg/validation/limits.go b/pkg/validation/limits.go index ca33d1f4bf425..7fde990e43656 100644 --- a/pkg/validation/limits.go +++ b/pkg/validation/limits.go @@ -85,6 +85,7 @@ type Limits struct { DiscoverLogLevels bool `yaml:"discover_log_levels" json:"discover_log_levels"` // Ingester enforced limits. + UseOwnedStreamCount bool `yaml:"use_owned_stream_count" json:"use_owned_stream_count"` MaxLocalStreamsPerUser int `yaml:"max_streams_per_user" json:"max_streams_per_user"` MaxGlobalStreamsPerUser int `yaml:"max_global_streams_per_user" json:"max_global_streams_per_user"` UnorderedWrites bool `yaml:"unordered_writes" json:"unordered_writes"` @@ -205,6 +206,10 @@ type Limits struct { BloomCompactorMaxBlockSize flagext.ByteSize `yaml:"bloom_compactor_max_block_size" json:"bloom_compactor_max_block_size" category:"experimental"` BloomCompactorMaxBloomSize flagext.ByteSize `yaml:"bloom_compactor_max_bloom_size" json:"bloom_compactor_max_bloom_size" category:"experimental"` + BloomCreationEnabled bool `yaml:"bloom_creation_enabled" json:"bloom_creation_enabled" category:"experimental"` + BloomSplitSeriesKeyspaceBy int `yaml:"bloom_split_series_keyspace_by" json:"bloom_split_series_keyspace_by" category:"experimental"` + BloomBuildMaxBuilders int `yaml:"bloom_build_max_builders" json:"bloom_build_max_builders" category:"experimental"` + BloomNGramLength int `yaml:"bloom_ngram_length" json:"bloom_ngram_length" category:"experimental"` BloomNGramSkip int `yaml:"bloom_ngram_skip" json:"bloom_ngram_skip" category:"experimental"` BloomFalsePositiveRate float64 `yaml:"bloom_false_positive_rate" json:"bloom_false_positive_rate" category:"experimental"` @@ -266,6 +271,7 @@ func (l *Limits) RegisterFlags(f *flag.FlagSet) { f.Var(&l.CreationGracePeriod, "validation.create-grace-period", "Duration which table will be created/deleted before/after it's needed; we won't accept sample from before this time.") f.IntVar(&l.MaxEntriesLimitPerQuery, "validation.max-entries-limit", 5000, "Maximum number of log entries that will be returned for a query.") + f.BoolVar(&l.UseOwnedStreamCount, "ingester.use-owned-stream-count", false, "When true an ingester takes into account only the streams that it owns according to the ring while applying the stream limit.") f.IntVar(&l.MaxLocalStreamsPerUser, "ingester.max-streams-per-user", 0, "Maximum number of active streams per user, per ingester. 0 to disable.") f.IntVar(&l.MaxGlobalStreamsPerUser, "ingester.max-global-streams-per-user", 5000, "Maximum number of active streams per user, across the cluster. 0 to disable. When the global limit is enabled, each ingester is configured with a dynamic local limit based on the replication factor and the current number of healthy ingesters, and is kept updated whenever the number of ingesters change.") @@ -380,6 +386,10 @@ func (l *Limits) RegisterFlags(f *flag.FlagSet) { ), ) + f.BoolVar(&l.BloomCreationEnabled, "bloom-build.enable", false, "Experimental. Whether to create blooms for the tenant.") + f.IntVar(&l.BloomSplitSeriesKeyspaceBy, "bloom-build.split-keyspace-by", 256, "Experimental. Number of splits to create for the series keyspace when building blooms. The series keyspace is split into this many parts to parallelize bloom creation.") + f.IntVar(&l.BloomBuildMaxBuilders, "bloom-build.max-builders", 0, "Experimental. Maximum number of builders to use when building blooms. 0 allows unlimited builders.") + _ = l.BloomCompactorMaxBloomSize.Set(defaultBloomCompactorMaxBloomSize) f.Var(&l.BloomCompactorMaxBloomSize, "bloom-compactor.max-bloom-size", fmt.Sprintf( @@ -580,6 +590,10 @@ func (o *Overrides) CreationGracePeriod(userID string) time.Duration { return time.Duration(o.getOverridesForUser(userID).CreationGracePeriod) } +func (o *Overrides) UseOwnedStreamCount(userID string) bool { + return o.getOverridesForUser(userID).UseOwnedStreamCount +} + // MaxLocalStreamsPerUser returns the maximum number of streams a user is allowed to store // in a single ingester. func (o *Overrides) MaxLocalStreamsPerUser(userID string) int { @@ -973,6 +987,18 @@ func (o *Overrides) BloomCompactorEnabled(userID string) bool { return o.getOverridesForUser(userID).BloomCompactorEnabled } +func (o *Overrides) BloomCreationEnabled(userID string) bool { + return o.getOverridesForUser(userID).BloomCreationEnabled +} + +func (o *Overrides) BloomSplitSeriesKeyspaceBy(userID string) int { + return o.getOverridesForUser(userID).BloomSplitSeriesKeyspaceBy +} + +func (o *Overrides) BloomBuildMaxBuilders(userID string) int { + return o.getOverridesForUser(userID).BloomBuildMaxBuilders +} + func (o *Overrides) BloomNGramLength(userID string) int { return o.getOverridesForUser(userID).BloomNGramLength } diff --git a/production/helm/loki/CHANGELOG.md b/production/helm/loki/CHANGELOG.md index 1606c89914f88..77b801e603631 100644 --- a/production/helm/loki/CHANGELOG.md +++ b/production/helm/loki/CHANGELOG.md @@ -13,6 +13,10 @@ Entries should include a reference to the pull request that introduced the chang [//]: # ( : do not remove this line. This locator is used by the CI pipeline to automatically create a changelog entry for each new Loki release. Add other chart versions and respective changelog entries bellow this line.) +## 6.6.0 + +- [ENHANCEMENT] Allow setting PVC annotations for all volume claim templates in simple scalable and single binary mode + ## 6.5.2 - [BUGFIX] Fixed Ingress routing for all deployment modes. diff --git a/production/helm/loki/Chart.yaml b/production/helm/loki/Chart.yaml index 989a54d146a1d..637e66d70887e 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 and Grafana Enterprise Logs supporting both simple, scalable and distributed modes. type: application appVersion: 3.0.0 -version: 6.5.2 +version: 6.6.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 55a7256c72f7f..5fa6bd548bad7 100644 --- a/production/helm/loki/README.md +++ b/production/helm/loki/README.md @@ -1,6 +1,6 @@ # loki -![Version: 6.5.2](https://img.shields.io/badge/Version-6.5.2-informational?style=flat-square) ![Type: application](https://img.shields.io/badge/Type-application-informational?style=flat-square) ![AppVersion: 3.0.0](https://img.shields.io/badge/AppVersion-3.0.0-informational?style=flat-square) +![Version: 6.6.0](https://img.shields.io/badge/Version-6.6.0-informational?style=flat-square) ![Type: application](https://img.shields.io/badge/Type-application-informational?style=flat-square) ![AppVersion: 3.0.0](https://img.shields.io/badge/AppVersion-3.0.0-informational?style=flat-square) Helm chart for Grafana Loki and Grafana Enterprise Logs supporting both simple, scalable and distributed modes. diff --git a/production/helm/loki/templates/backend/statefulset-backend.yaml b/production/helm/loki/templates/backend/statefulset-backend.yaml index f96f0a4d21217..534190d4a4533 100644 --- a/production/helm/loki/templates/backend/statefulset-backend.yaml +++ b/production/helm/loki/templates/backend/statefulset-backend.yaml @@ -266,6 +266,10 @@ spec: kind: PersistentVolumeClaim metadata: name: data + {{- with .Values.backend.persistence.annotations }} + annotations: + {{- toYaml . | nindent 10 }} + {{- end }} spec: accessModes: - ReadWriteOnce diff --git a/production/helm/loki/templates/read/statefulset-read.yaml b/production/helm/loki/templates/read/statefulset-read.yaml index 0a31de4996dfb..7696d90e65bd6 100644 --- a/production/helm/loki/templates/read/statefulset-read.yaml +++ b/production/helm/loki/templates/read/statefulset-read.yaml @@ -180,6 +180,10 @@ spec: kind: PersistentVolumeClaim metadata: name: data + {{- with .Values.read.persistence.annotations }} + annotations: + {{- toYaml . | nindent 10 }} + {{- end }} spec: accessModes: - ReadWriteOnce diff --git a/production/helm/loki/templates/single-binary/statefulset.yaml b/production/helm/loki/templates/single-binary/statefulset.yaml index 51c0062fc94ff..7bd2b9813f609 100644 --- a/production/helm/loki/templates/single-binary/statefulset.yaml +++ b/production/helm/loki/templates/single-binary/statefulset.yaml @@ -175,6 +175,10 @@ spec: kind: PersistentVolumeClaim metadata: name: storage + {{- with .Values.singleBinary.persistence.annotations }} + annotations: + {{- toYaml . | nindent 10 }} + {{- end }} spec: accessModes: - ReadWriteOnce diff --git a/production/helm/loki/templates/write/statefulset-write.yaml b/production/helm/loki/templates/write/statefulset-write.yaml index 54c936958b559..75605c27c26cb 100644 --- a/production/helm/loki/templates/write/statefulset-write.yaml +++ b/production/helm/loki/templates/write/statefulset-write.yaml @@ -193,6 +193,10 @@ spec: kind: PersistentVolumeClaim metadata: name: data + {{- with .Values.write.persistence.annotations }} + annotations: + {{- toYaml . | nindent 10 }} + {{- end }} spec: accessModes: - ReadWriteOnce diff --git a/production/helm/loki/values.yaml b/production/helm/loki/values.yaml index 3edfc24ba34fb..4c70bf16fe474 100644 --- a/production/helm/loki/values.yaml +++ b/production/helm/loki/values.yaml @@ -1294,6 +1294,8 @@ singleBinary: storageClass: null # -- Selector for persistent disk selector: null + # -- Annotations for volume claim + annotations: {} ###################################################################################################################### # # Simple Scalable Deployment (SSD) Mode @@ -1421,6 +1423,8 @@ write: storageClass: null # -- Selector for persistent disk selector: null + # -- Annotations for volume claim + annotations: {} # -- Configuration for the read pod(s) read: # -- Number of replicas for the read @@ -1528,6 +1532,8 @@ read: storageClass: null # -- Selector for persistent disk selector: null + # -- Annotations for volume claim + annotations: {} # -- Configuration for the backend pod(s) backend: # -- Number of replicas for the backend @@ -1636,6 +1642,8 @@ backend: storageClass: null # -- Selector for persistent disk selector: null + # -- Annotations for volume claim + annotations: {} ###################################################################################################################### # # Microservices Mode @@ -3091,6 +3099,7 @@ minio: purge: false persistence: size: 5Gi + annotations: {} resources: requests: cpu: 100m diff --git a/production/loki-mixin-compiled-ssd/dashboards/loki-bloom-compactor.json b/production/loki-mixin-compiled-ssd/dashboards/loki-bloom-compactor.json index c667d0c01eccb..c365fab0a7e59 100644 --- a/production/loki-mixin-compiled-ssd/dashboards/loki-bloom-compactor.json +++ b/production/loki-mixin-compiled-ssd/dashboards/loki-bloom-compactor.json @@ -32,6 +32,7 @@ }, "id": 111, "panels": [ ], + "targets": [ ], "title": "Overview", "type": "row" }, @@ -52,7 +53,9 @@ "content": "## About the Bloom Compactor\nThe compactor iterates through chunks and creates blooms out of them.\nThe size of the resulting blooms depends on the bloom filter settings, the tokenizer settings, the number of ring tokens per compactor and the total number opf compactors.\n\nCompactors are horizontally scalable and uses a ring to:\n- Shard tenants\n- Shard series fingerprints within a tenant subring.\n\nThe blooms for the series are grouped together in blocks which are flushed to object store.", "mode": "markdown" }, + "panels": [ ], "pluginVersion": "11.1.0-70005", + "targets": [ ], "transparent": true, "type": "text" }, @@ -140,6 +143,7 @@ "sort": "none" } }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -290,6 +294,7 @@ "sort": "none" } }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -371,6 +376,7 @@ "sortOrder": "Descending", "wrapLogMessage": false }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -467,6 +473,7 @@ "sort": "none" } }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -576,6 +583,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -645,6 +653,7 @@ "showThresholdMarkers": false, "sizing": "auto" }, + "panels": [ ], "pluginVersion": "11.0.0-68102", "targets": [ { @@ -665,6 +674,7 @@ "type": "gauge" } ], + "targets": [ ], "title": "Progress per pod", "type": "row" }, @@ -696,7 +706,9 @@ "content": "", "mode": "markdown" }, + "panels": [ ], "pluginVersion": "11.1.0-70005", + "targets": [ ], "title": "We use tenant sharding so each compactor will process a subset of the tenants.", "transparent": true, "type": "text" @@ -777,6 +789,7 @@ "sort": "none" } }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -893,6 +906,7 @@ "sort": "none" } }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -1000,6 +1014,7 @@ "sort": "none" } }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -1149,6 +1164,7 @@ "sort": "none" } }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -1273,6 +1289,7 @@ "sort": "none" } }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -1448,6 +1465,7 @@ "sort": "none" } }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -1576,6 +1594,7 @@ "sort": "none" } }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -1678,6 +1697,7 @@ "sort": "none" } }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -1697,6 +1717,7 @@ "type": "timeseries" } ], + "targets": [ ], "title": "Resource Usage", "type": "row" }, @@ -1791,6 +1812,7 @@ "sort": "none" } }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -1904,6 +1926,7 @@ "sort": "none" } }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -2004,6 +2027,7 @@ "sort": "none" } }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -2130,6 +2154,7 @@ "sort": "none" } }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -2150,6 +2175,7 @@ "type": "timeseries" } ], + "targets": [ ], "title": "Bloom building", "type": "row" }, @@ -2238,6 +2264,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -2332,6 +2359,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -2426,6 +2454,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -2445,6 +2474,7 @@ "type": "timeseries" } ], + "targets": [ ], "title": "Blocks building", "type": "row" }, @@ -2533,6 +2563,7 @@ "sort": "none" } }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -2628,6 +2659,7 @@ "sort": "none" } }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -2648,6 +2680,7 @@ "type": "timeseries" } ], + "targets": [ ], "title": "Metas building", "type": "row" }, @@ -2679,7 +2712,9 @@ "content": "", "mode": "markdown" }, + "panels": [ ], "pluginVersion": "11.1.0-70005", + "targets": [ ], "title": "We use tenant sharding so each compactor will process a subset of the tenants.", "transparent": true, "type": "text" @@ -2759,6 +2794,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -2878,6 +2914,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -2914,7 +2951,9 @@ "content": "", "mode": "markdown" }, + "panels": [ ], "pluginVersion": "11.1.0-70005", + "targets": [ ], "title": "Number of tenant tables processed. ", "transparent": true, "type": "text" @@ -2994,6 +3033,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -3113,6 +3153,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -3149,7 +3190,9 @@ "content": "", "mode": "markdown" }, + "panels": [ ], "pluginVersion": "11.1.0-70005", + "targets": [ ], "title": "Series per compaction (includes series copied from other blocks)", "transparent": true, "type": "text" @@ -3230,6 +3273,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -3349,6 +3393,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -3385,7 +3430,9 @@ "content": "", "mode": "markdown" }, + "panels": [ ], "pluginVersion": "11.1.0-70005", + "targets": [ ], "title": "Number of bytes from chunks added to blocks during each compaction.", "transparent": true, "type": "text" @@ -3466,6 +3513,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -3586,6 +3634,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -3605,6 +3654,7 @@ "type": "timeseries" } ], + "targets": [ ], "title": "Data processed", "type": "row" }, @@ -3640,7 +3690,9 @@ "content": "\nCompactors write blocks to the attached PVs before flushing them into the object store.\nIt also download chunks and index files.\n\nAfter compacting a given tenant, all the downloaded index files and chunks, as well as the already flushed blocks are deleted.", "mode": "markdown" }, + "panels": [ ], "pluginVersion": "11.1.0-69747", + "targets": [ ], "title": "", "transparent": true, "type": "text" @@ -3726,6 +3778,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -3850,6 +3903,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -3946,6 +4000,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -4068,6 +4123,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -4164,6 +4220,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -4286,6 +4343,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -4304,6 +4362,7 @@ "type": "timeseries" } ], + "targets": [ ], "title": "Disk Usage", "type": "row" }, @@ -4339,7 +4398,9 @@ "content": "Once all blocks and metas are built locally, the compactor flushes them to the object store.\n\nAfter each iteration, the compactor deletes the metas and blocks marked for deletion in the tombstones.", "mode": "markdown" }, + "panels": [ ], "pluginVersion": "11.1.0-69747", + "targets": [ ], "title": "", "transparent": true, "type": "text" @@ -4366,7 +4427,9 @@ "content": "---\n#### GCS\n", "mode": "markdown" }, + "panels": [ ], "pluginVersion": "11.1.0-69747", + "targets": [ ], "title": "", "transparent": true, "type": "text" @@ -4447,6 +4510,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -4541,6 +4605,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -4607,7 +4672,9 @@ "content": "---\n#### S3\n", "mode": "markdown" }, + "panels": [ ], "pluginVersion": "11.1.0-69747", + "targets": [ ], "title": "", "transparent": true, "type": "text" @@ -4688,6 +4755,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -4782,6 +4850,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -4848,7 +4917,9 @@ "content": "---\n#### Azure\nBlob Storage", "mode": "markdown" }, + "panels": [ ], "pluginVersion": "11.1.0-69747", + "targets": [ ], "title": "", "transparent": true, "type": "text" @@ -4929,6 +5000,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -5023,6 +5095,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -5068,6 +5141,7 @@ "type": "timeseries" } ], + "targets": [ ], "title": "Object Store", "type": "row" } diff --git a/production/loki-mixin-compiled-ssd/dashboards/loki-bloom-gateway.json b/production/loki-mixin-compiled-ssd/dashboards/loki-bloom-gateway.json index 27a058ae800e2..2d5e16a9d7e0f 100644 --- a/production/loki-mixin-compiled-ssd/dashboards/loki-bloom-gateway.json +++ b/production/loki-mixin-compiled-ssd/dashboards/loki-bloom-gateway.json @@ -33,6 +33,7 @@ }, "id": 73, "panels": [ ], + "targets": [ ], "title": "Overview", "type": "row" }, @@ -131,6 +132,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -223,6 +225,7 @@ "showThresholdMarkers": true, "sizing": "auto" }, + "panels": [ ], "pluginVersion": "11.1.0-70005", "targets": [ { @@ -395,6 +398,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -536,6 +540,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -577,6 +582,7 @@ "sortOrder": "Descending", "wrapLogMessage": true }, + "panels": [ ], "targets": [ { "datasource": { @@ -730,6 +736,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -847,6 +854,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -970,6 +978,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -1094,6 +1103,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -1234,6 +1244,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -1336,6 +1347,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -1437,6 +1449,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -1547,6 +1560,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -1647,6 +1661,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -1666,6 +1681,7 @@ "type": "timeseries" } ], + "targets": [ ], "title": "Resource usage", "type": "row" }, @@ -1799,6 +1815,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -1937,6 +1954,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -2032,6 +2050,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -2152,6 +2171,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -2171,6 +2191,7 @@ "type": "timeseries" } ], + "targets": [ ], "title": "QPS and Latency", "type": "row" }, @@ -2260,6 +2281,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -2380,6 +2402,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -2513,6 +2536,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -2532,6 +2556,7 @@ "type": "timeseries" } ], + "targets": [ ], "title": "Task Queue", "type": "row" }, @@ -2625,6 +2650,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -2748,6 +2774,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -2870,6 +2897,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -2901,6 +2929,7 @@ "type": "timeseries" } ], + "targets": [ ], "title": "Processing", "type": "row" }, @@ -2936,7 +2965,9 @@ "content": "", "mode": "markdown" }, + "panels": [ ], "pluginVersion": "11.1.0-70005", + "targets": [ ], "title": "We cache bloom blocks in memory to prevent the gateway from hitting the object store too often", "transparent": true, "type": "text" @@ -3021,6 +3052,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -3178,6 +3210,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -3347,6 +3380,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -3458,6 +3492,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -3581,6 +3616,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -3678,6 +3714,7 @@ "type": "timeseries" } ], + "targets": [ ], "title": "Blocks Cache", "type": "row" }, @@ -3713,7 +3750,9 @@ "content": "", "mode": "markdown" }, + "panels": [ ], "pluginVersion": "11.1.0-70005", + "targets": [ ], "title": "The gateway download bloom meta files and blocks from the object store.", "transparent": true, "type": "text" @@ -3740,7 +3779,9 @@ "content": "---\n#### GCS\n", "mode": "markdown" }, + "panels": [ ], "pluginVersion": "11.1.0-70005", + "targets": [ ], "transparent": true, "type": "text" }, @@ -3820,6 +3861,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -3913,6 +3955,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -3978,7 +4021,9 @@ "content": "---\n#### S3\n", "mode": "markdown" }, + "panels": [ ], "pluginVersion": "11.1.0-70005", + "targets": [ ], "transparent": true, "type": "text" }, @@ -4058,6 +4103,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -4151,6 +4197,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -4216,7 +4263,9 @@ "content": "---\n#### Azure\nBlob Storage\n\n", "mode": "markdown" }, + "panels": [ ], "pluginVersion": "11.1.0-70005", + "targets": [ ], "transparent": true, "type": "text" }, @@ -4296,6 +4345,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -4389,6 +4439,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -4433,6 +4484,7 @@ "type": "timeseries" } ], + "targets": [ ], "title": "Object Store", "type": "row" }, @@ -4528,6 +4580,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -4652,6 +4705,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -4770,6 +4824,7 @@ "xTickLabelRotation": 0, "xTickLabelSpacing": 0 }, + "panels": [ ], "pluginVersion": "11.0.0-67814", "targets": [ { @@ -4882,6 +4937,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -4912,6 +4968,7 @@ "type": "timeseries" } ], + "targets": [ ], "title": "Misc", "type": "row" } diff --git a/production/loki-mixin-compiled-ssd/dashboards/loki-deletion.json b/production/loki-mixin-compiled-ssd/dashboards/loki-deletion.json index 8ce9529906aa4..0718507d941fd 100644 --- a/production/loki-mixin-compiled-ssd/dashboards/loki-deletion.json +++ b/production/loki-mixin-compiled-ssd/dashboards/loki-deletion.json @@ -701,6 +701,16 @@ "tagsQuery": "", "type": "query", "useTags": false + }, + { + "hide": 0, + "label": null, + "name": "loki_datasource", + "options": [ ], + "query": "loki", + "refresh": 1, + "regex": "", + "type": "datasource" } ] }, diff --git a/production/loki-mixin-compiled-ssd/dashboards/loki-operational.json b/production/loki-mixin-compiled-ssd/dashboards/loki-operational.json index d9f668ed88a3f..911e9f7267d7c 100644 --- a/production/loki-mixin-compiled-ssd/dashboards/loki-operational.json +++ b/production/loki-mixin-compiled-ssd/dashboards/loki-operational.json @@ -1657,9 +1657,93 @@ } }, { - "columns": [ ], "datasource": "$datasource", - "fontSize": "100%", + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "custom": { + "align": "right", + "cellOptions": { + "type": "auto" + }, + "inspect": false + }, + "decimals": 2, + "displayName": "", + "mappings": [ ], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.align" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "tenant" + }, + "properties": [ + { + "id": "unit", + "value": "short" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "custom.align" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "reason" + }, + "properties": [ + { + "id": "unit", + "value": "short" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "custom.align" + } + ] + } + ] + }, "gridPos": { "h": 8, "w": 12, @@ -1667,71 +1751,20 @@ "y": 27 }, "id": 113, - "pageSize": null, - "panels": [ ], - "showHeader": true, - "sort": { - "col": 3, - "desc": true - }, - "styles": [ - { - "alias": "Time", - "align": "auto", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "", - "align": "auto", - "colorMode": null, - "colors": [ - "rgba(245, 54, 54, 0.9)", - "rgba(237, 129, 40, 0.89)", - "rgba(50, 172, 45, 0.97)" - ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "mappingType": 1, - "pattern": "tenant", - "thresholds": [ ], - "type": "string", - "unit": "short" - }, - { - "alias": "", - "align": "auto", - "colorMode": null, - "colors": [ - "rgba(245, 54, 54, 0.9)", - "rgba(237, 129, 40, 0.89)", - "rgba(50, 172, 45, 0.97)" + "options": { + "cellHeight": "sm", + "footer": { + "countRows": false, + "fields": "", + "reducer": [ + "sum" ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "mappingType": 1, - "pattern": "reason", - "thresholds": [ ], - "type": "number", - "unit": "short" + "show": false }, - { - "alias": "", - "align": "right", - "colorMode": null, - "colors": [ - "rgba(245, 54, 54, 0.9)", - "rgba(237, 129, 40, 0.89)", - "rgba(50, 172, 45, 0.97)" - ], - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "number", - "unit": "short" - } - ], + "showHeader": true + }, + "panels": [ ], + "pluginVersion": "10.4.0", "targets": [ { "expr": "topk(10, sum by (tenant, reason) (sum_over_time(increase(loki_discarded_samples_total{cluster=\"$cluster\",namespace=\"$namespace\"}[$__rate_interval])[$__range:$__rate_interval])))", @@ -1742,11 +1775,16 @@ "refId": "A" } ], - "timeFrom": null, - "timeShift": null, "title": "Discarded Lines Per Interval", - "transform": "table", - "type": "table-old" + "transformations": [ + { + "id": "merge", + "options": { + "reducers": [ ] + } + } + ], + "type": "table" } ], "targets": [ ], diff --git a/production/loki-mixin-compiled-ssd/dashboards/loki-reads-resources.json b/production/loki-mixin-compiled-ssd/dashboards/loki-reads-resources.json index d2114319a5cc6..ed5fc3dd245d4 100644 --- a/production/loki-mixin-compiled-ssd/dashboards/loki-reads-resources.json +++ b/production/loki-mixin-compiled-ssd/dashboards/loki-reads-resources.json @@ -277,147 +277,6 @@ "sort": 2 }, "type": "timeseries" - }, - { - "datasource": "$datasource", - "fieldConfig": { - "defaults": { - "custom": { - "drawStyle": "line", - "fillOpacity": 100, - "lineWidth": 0, - "pointSize": 5, - "showPoints": "never", - "spanNulls": false, - "stacking": { - "group": "A", - "mode": "normal" - } - }, - "thresholds": { - "mode": "absolute", - "steps": [ ] - }, - "unit": "Bps" - }, - "overrides": [ ] - }, - "gridPos": { }, - "id": 4, - "links": [ ], - "options": { - "legend": { - "showLegend": true - }, - "tooltip": { - "mode": "single", - "sort": "none" - } - }, - "targets": [ - { - "expr": "sum by(instance, pod, device) (rate(node_disk_written_bytes_total[$__rate_interval])) + ignoring(pod) group_right() (label_replace(count by(instance, pod, device) (container_fs_writes_bytes_total{cluster=~\"$cluster\", namespace=~\"$namespace\", container=\"loki\", pod=~\"(loki.*|enterprise-logs)-read.*\", device!~\".*sda.*\"}), \"device\", \"$1\", \"device\", \"/dev/(.*)\") * 0)\n", - "format": "time_series", - "legendFormat": "{{pod}} - {{device}}", - "legendLink": null - } - ], - "title": "Disk Writes", - "type": "timeseries" - }, - { - "datasource": "$datasource", - "fieldConfig": { - "defaults": { - "custom": { - "drawStyle": "line", - "fillOpacity": 100, - "lineWidth": 0, - "pointSize": 5, - "showPoints": "never", - "spanNulls": false, - "stacking": { - "group": "A", - "mode": "normal" - } - }, - "thresholds": { - "mode": "absolute", - "steps": [ ] - }, - "unit": "Bps" - }, - "overrides": [ ] - }, - "gridPos": { }, - "id": 5, - "links": [ ], - "options": { - "legend": { - "showLegend": true - }, - "tooltip": { - "mode": "single", - "sort": "none" - } - }, - "targets": [ - { - "expr": "sum by(instance, pod, device) (rate(node_disk_read_bytes_total[$__rate_interval])) + ignoring(pod) group_right() (label_replace(count by(instance, pod, device) (container_fs_writes_bytes_total{cluster=~\"$cluster\", namespace=~\"$namespace\", container=\"loki\", pod=~\"(loki.*|enterprise-logs)-read.*\", device!~\".*sda.*\"}), \"device\", \"$1\", \"device\", \"/dev/(.*)\") * 0)\n", - "format": "time_series", - "legendFormat": "{{pod}} - {{device}}", - "legendLink": null - } - ], - "title": "Disk Reads", - "type": "timeseries" - }, - { - "datasource": "$datasource", - "fieldConfig": { - "defaults": { - "custom": { - "drawStyle": "line", - "fillOpacity": 10, - "lineWidth": 1, - "pointSize": 5, - "showPoints": "never", - "spanNulls": false, - "stacking": { - "group": "A", - "mode": "none" - } - }, - "thresholds": { - "mode": "absolute", - "steps": [ ] - }, - "unit": "percentunit" - }, - "overrides": [ ] - }, - "gridPos": { }, - "id": 6, - "links": [ ], - "options": { - "legend": { - "showLegend": true - }, - "tooltip": { - "mode": "single", - "sort": "none" - } - }, - "targets": [ - { - "expr": "max by(persistentvolumeclaim) (kubelet_volume_stats_used_bytes{cluster=~\"$cluster\", namespace=~\"$namespace\"} / kubelet_volume_stats_capacity_bytes{cluster=~\"$cluster\", namespace=~\"$namespace\"}) and count by(persistentvolumeclaim) (kube_persistentvolumeclaim_labels{cluster=~\"$cluster\", namespace=~\"$namespace\",label_name=~\"(loki.*|enterprise-logs)-read.*\"})", - "format": "time_series", - "legendFormat": "{{persistentvolumeclaim}}", - "legendLink": null - } - ], - "title": "Disk Space Utilization", - "type": "timeseries" } ], "repeat": null, @@ -495,7 +354,7 @@ } ] }, - "id": 7, + "id": 4, "links": [ ], "options": { "legend": { @@ -596,7 +455,7 @@ } ] }, - "id": 8, + "id": 5, "links": [ ], "options": { "legend": { @@ -658,7 +517,7 @@ }, "overrides": [ ] }, - "id": 9, + "id": 6, "links": [ ], "options": { "legend": { diff --git a/production/loki-mixin-compiled/dashboards/loki-bloom-compactor.json b/production/loki-mixin-compiled/dashboards/loki-bloom-compactor.json index c667d0c01eccb..c365fab0a7e59 100644 --- a/production/loki-mixin-compiled/dashboards/loki-bloom-compactor.json +++ b/production/loki-mixin-compiled/dashboards/loki-bloom-compactor.json @@ -32,6 +32,7 @@ }, "id": 111, "panels": [ ], + "targets": [ ], "title": "Overview", "type": "row" }, @@ -52,7 +53,9 @@ "content": "## About the Bloom Compactor\nThe compactor iterates through chunks and creates blooms out of them.\nThe size of the resulting blooms depends on the bloom filter settings, the tokenizer settings, the number of ring tokens per compactor and the total number opf compactors.\n\nCompactors are horizontally scalable and uses a ring to:\n- Shard tenants\n- Shard series fingerprints within a tenant subring.\n\nThe blooms for the series are grouped together in blocks which are flushed to object store.", "mode": "markdown" }, + "panels": [ ], "pluginVersion": "11.1.0-70005", + "targets": [ ], "transparent": true, "type": "text" }, @@ -140,6 +143,7 @@ "sort": "none" } }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -290,6 +294,7 @@ "sort": "none" } }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -371,6 +376,7 @@ "sortOrder": "Descending", "wrapLogMessage": false }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -467,6 +473,7 @@ "sort": "none" } }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -576,6 +583,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -645,6 +653,7 @@ "showThresholdMarkers": false, "sizing": "auto" }, + "panels": [ ], "pluginVersion": "11.0.0-68102", "targets": [ { @@ -665,6 +674,7 @@ "type": "gauge" } ], + "targets": [ ], "title": "Progress per pod", "type": "row" }, @@ -696,7 +706,9 @@ "content": "", "mode": "markdown" }, + "panels": [ ], "pluginVersion": "11.1.0-70005", + "targets": [ ], "title": "We use tenant sharding so each compactor will process a subset of the tenants.", "transparent": true, "type": "text" @@ -777,6 +789,7 @@ "sort": "none" } }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -893,6 +906,7 @@ "sort": "none" } }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -1000,6 +1014,7 @@ "sort": "none" } }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -1149,6 +1164,7 @@ "sort": "none" } }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -1273,6 +1289,7 @@ "sort": "none" } }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -1448,6 +1465,7 @@ "sort": "none" } }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -1576,6 +1594,7 @@ "sort": "none" } }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -1678,6 +1697,7 @@ "sort": "none" } }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -1697,6 +1717,7 @@ "type": "timeseries" } ], + "targets": [ ], "title": "Resource Usage", "type": "row" }, @@ -1791,6 +1812,7 @@ "sort": "none" } }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -1904,6 +1926,7 @@ "sort": "none" } }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -2004,6 +2027,7 @@ "sort": "none" } }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -2130,6 +2154,7 @@ "sort": "none" } }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -2150,6 +2175,7 @@ "type": "timeseries" } ], + "targets": [ ], "title": "Bloom building", "type": "row" }, @@ -2238,6 +2264,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -2332,6 +2359,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -2426,6 +2454,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -2445,6 +2474,7 @@ "type": "timeseries" } ], + "targets": [ ], "title": "Blocks building", "type": "row" }, @@ -2533,6 +2563,7 @@ "sort": "none" } }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -2628,6 +2659,7 @@ "sort": "none" } }, + "panels": [ ], "pluginVersion": "11.1.0-69868", "targets": [ { @@ -2648,6 +2680,7 @@ "type": "timeseries" } ], + "targets": [ ], "title": "Metas building", "type": "row" }, @@ -2679,7 +2712,9 @@ "content": "", "mode": "markdown" }, + "panels": [ ], "pluginVersion": "11.1.0-70005", + "targets": [ ], "title": "We use tenant sharding so each compactor will process a subset of the tenants.", "transparent": true, "type": "text" @@ -2759,6 +2794,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -2878,6 +2914,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -2914,7 +2951,9 @@ "content": "", "mode": "markdown" }, + "panels": [ ], "pluginVersion": "11.1.0-70005", + "targets": [ ], "title": "Number of tenant tables processed. ", "transparent": true, "type": "text" @@ -2994,6 +3033,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -3113,6 +3153,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -3149,7 +3190,9 @@ "content": "", "mode": "markdown" }, + "panels": [ ], "pluginVersion": "11.1.0-70005", + "targets": [ ], "title": "Series per compaction (includes series copied from other blocks)", "transparent": true, "type": "text" @@ -3230,6 +3273,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -3349,6 +3393,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -3385,7 +3430,9 @@ "content": "", "mode": "markdown" }, + "panels": [ ], "pluginVersion": "11.1.0-70005", + "targets": [ ], "title": "Number of bytes from chunks added to blocks during each compaction.", "transparent": true, "type": "text" @@ -3466,6 +3513,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -3586,6 +3634,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -3605,6 +3654,7 @@ "type": "timeseries" } ], + "targets": [ ], "title": "Data processed", "type": "row" }, @@ -3640,7 +3690,9 @@ "content": "\nCompactors write blocks to the attached PVs before flushing them into the object store.\nIt also download chunks and index files.\n\nAfter compacting a given tenant, all the downloaded index files and chunks, as well as the already flushed blocks are deleted.", "mode": "markdown" }, + "panels": [ ], "pluginVersion": "11.1.0-69747", + "targets": [ ], "title": "", "transparent": true, "type": "text" @@ -3726,6 +3778,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -3850,6 +3903,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -3946,6 +4000,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -4068,6 +4123,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -4164,6 +4220,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -4286,6 +4343,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -4304,6 +4362,7 @@ "type": "timeseries" } ], + "targets": [ ], "title": "Disk Usage", "type": "row" }, @@ -4339,7 +4398,9 @@ "content": "Once all blocks and metas are built locally, the compactor flushes them to the object store.\n\nAfter each iteration, the compactor deletes the metas and blocks marked for deletion in the tombstones.", "mode": "markdown" }, + "panels": [ ], "pluginVersion": "11.1.0-69747", + "targets": [ ], "title": "", "transparent": true, "type": "text" @@ -4366,7 +4427,9 @@ "content": "---\n#### GCS\n", "mode": "markdown" }, + "panels": [ ], "pluginVersion": "11.1.0-69747", + "targets": [ ], "title": "", "transparent": true, "type": "text" @@ -4447,6 +4510,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -4541,6 +4605,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -4607,7 +4672,9 @@ "content": "---\n#### S3\n", "mode": "markdown" }, + "panels": [ ], "pluginVersion": "11.1.0-69747", + "targets": [ ], "title": "", "transparent": true, "type": "text" @@ -4688,6 +4755,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -4782,6 +4850,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -4848,7 +4917,9 @@ "content": "---\n#### Azure\nBlob Storage", "mode": "markdown" }, + "panels": [ ], "pluginVersion": "11.1.0-69747", + "targets": [ ], "title": "", "transparent": true, "type": "text" @@ -4929,6 +5000,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -5023,6 +5095,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -5068,6 +5141,7 @@ "type": "timeseries" } ], + "targets": [ ], "title": "Object Store", "type": "row" } diff --git a/production/loki-mixin-compiled/dashboards/loki-bloom-gateway.json b/production/loki-mixin-compiled/dashboards/loki-bloom-gateway.json index 27a058ae800e2..2d5e16a9d7e0f 100644 --- a/production/loki-mixin-compiled/dashboards/loki-bloom-gateway.json +++ b/production/loki-mixin-compiled/dashboards/loki-bloom-gateway.json @@ -33,6 +33,7 @@ }, "id": 73, "panels": [ ], + "targets": [ ], "title": "Overview", "type": "row" }, @@ -131,6 +132,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -223,6 +225,7 @@ "showThresholdMarkers": true, "sizing": "auto" }, + "panels": [ ], "pluginVersion": "11.1.0-70005", "targets": [ { @@ -395,6 +398,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -536,6 +540,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -577,6 +582,7 @@ "sortOrder": "Descending", "wrapLogMessage": true }, + "panels": [ ], "targets": [ { "datasource": { @@ -730,6 +736,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -847,6 +854,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -970,6 +978,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -1094,6 +1103,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -1234,6 +1244,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -1336,6 +1347,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -1437,6 +1449,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -1547,6 +1560,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -1647,6 +1661,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -1666,6 +1681,7 @@ "type": "timeseries" } ], + "targets": [ ], "title": "Resource usage", "type": "row" }, @@ -1799,6 +1815,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -1937,6 +1954,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -2032,6 +2050,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -2152,6 +2171,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -2171,6 +2191,7 @@ "type": "timeseries" } ], + "targets": [ ], "title": "QPS and Latency", "type": "row" }, @@ -2260,6 +2281,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -2380,6 +2402,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -2513,6 +2536,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -2532,6 +2556,7 @@ "type": "timeseries" } ], + "targets": [ ], "title": "Task Queue", "type": "row" }, @@ -2625,6 +2650,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -2748,6 +2774,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -2870,6 +2897,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -2901,6 +2929,7 @@ "type": "timeseries" } ], + "targets": [ ], "title": "Processing", "type": "row" }, @@ -2936,7 +2965,9 @@ "content": "", "mode": "markdown" }, + "panels": [ ], "pluginVersion": "11.1.0-70005", + "targets": [ ], "title": "We cache bloom blocks in memory to prevent the gateway from hitting the object store too often", "transparent": true, "type": "text" @@ -3021,6 +3052,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -3178,6 +3210,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -3347,6 +3380,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -3458,6 +3492,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -3581,6 +3616,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -3678,6 +3714,7 @@ "type": "timeseries" } ], + "targets": [ ], "title": "Blocks Cache", "type": "row" }, @@ -3713,7 +3750,9 @@ "content": "", "mode": "markdown" }, + "panels": [ ], "pluginVersion": "11.1.0-70005", + "targets": [ ], "title": "The gateway download bloom meta files and blocks from the object store.", "transparent": true, "type": "text" @@ -3740,7 +3779,9 @@ "content": "---\n#### GCS\n", "mode": "markdown" }, + "panels": [ ], "pluginVersion": "11.1.0-70005", + "targets": [ ], "transparent": true, "type": "text" }, @@ -3820,6 +3861,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -3913,6 +3955,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -3978,7 +4021,9 @@ "content": "---\n#### S3\n", "mode": "markdown" }, + "panels": [ ], "pluginVersion": "11.1.0-70005", + "targets": [ ], "transparent": true, "type": "text" }, @@ -4058,6 +4103,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -4151,6 +4197,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -4216,7 +4263,9 @@ "content": "---\n#### Azure\nBlob Storage\n\n", "mode": "markdown" }, + "panels": [ ], "pluginVersion": "11.1.0-70005", + "targets": [ ], "transparent": true, "type": "text" }, @@ -4296,6 +4345,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -4389,6 +4439,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -4433,6 +4484,7 @@ "type": "timeseries" } ], + "targets": [ ], "title": "Object Store", "type": "row" }, @@ -4528,6 +4580,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -4652,6 +4705,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -4770,6 +4824,7 @@ "xTickLabelRotation": 0, "xTickLabelSpacing": 0 }, + "panels": [ ], "pluginVersion": "11.0.0-67814", "targets": [ { @@ -4882,6 +4937,7 @@ "sort": "none" } }, + "panels": [ ], "targets": [ { "datasource": { @@ -4912,6 +4968,7 @@ "type": "timeseries" } ], + "targets": [ ], "title": "Misc", "type": "row" } diff --git a/production/loki-mixin-compiled/dashboards/loki-deletion.json b/production/loki-mixin-compiled/dashboards/loki-deletion.json index 2db2b7cb36586..7b048f729e2b3 100644 --- a/production/loki-mixin-compiled/dashboards/loki-deletion.json +++ b/production/loki-mixin-compiled/dashboards/loki-deletion.json @@ -701,6 +701,16 @@ "tagsQuery": "", "type": "query", "useTags": false + }, + { + "hide": 0, + "label": null, + "name": "loki_datasource", + "options": [ ], + "query": "loki", + "refresh": 1, + "regex": "", + "type": "datasource" } ] }, diff --git a/production/loki-mixin-compiled/dashboards/loki-operational.json b/production/loki-mixin-compiled/dashboards/loki-operational.json index d677775d52ef8..48684c77e5d9b 100644 --- a/production/loki-mixin-compiled/dashboards/loki-operational.json +++ b/production/loki-mixin-compiled/dashboards/loki-operational.json @@ -1754,9 +1754,93 @@ } }, { - "columns": [ ], "datasource": "$datasource", - "fontSize": "100%", + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "custom": { + "align": "right", + "cellOptions": { + "type": "auto" + }, + "inspect": false + }, + "decimals": 2, + "displayName": "", + "mappings": [ ], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.align" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "tenant" + }, + "properties": [ + { + "id": "unit", + "value": "short" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "custom.align" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "reason" + }, + "properties": [ + { + "id": "unit", + "value": "short" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "custom.align" + } + ] + } + ] + }, "gridPos": { "h": 8, "w": 12, @@ -1764,71 +1848,20 @@ "y": 27 }, "id": 113, - "pageSize": null, - "panels": [ ], - "showHeader": true, - "sort": { - "col": 3, - "desc": true - }, - "styles": [ - { - "alias": "Time", - "align": "auto", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "", - "align": "auto", - "colorMode": null, - "colors": [ - "rgba(245, 54, 54, 0.9)", - "rgba(237, 129, 40, 0.89)", - "rgba(50, 172, 45, 0.97)" - ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "mappingType": 1, - "pattern": "tenant", - "thresholds": [ ], - "type": "string", - "unit": "short" - }, - { - "alias": "", - "align": "auto", - "colorMode": null, - "colors": [ - "rgba(245, 54, 54, 0.9)", - "rgba(237, 129, 40, 0.89)", - "rgba(50, 172, 45, 0.97)" + "options": { + "cellHeight": "sm", + "footer": { + "countRows": false, + "fields": "", + "reducer": [ + "sum" ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "mappingType": 1, - "pattern": "reason", - "thresholds": [ ], - "type": "number", - "unit": "short" + "show": false }, - { - "alias": "", - "align": "right", - "colorMode": null, - "colors": [ - "rgba(245, 54, 54, 0.9)", - "rgba(237, 129, 40, 0.89)", - "rgba(50, 172, 45, 0.97)" - ], - "decimals": 2, - "pattern": "/.*/", - "thresholds": [ ], - "type": "number", - "unit": "short" - } - ], + "showHeader": true + }, + "panels": [ ], + "pluginVersion": "10.4.0", "targets": [ { "expr": "topk(10, sum by (tenant, reason) (sum_over_time(increase(loki_discarded_samples_total{cluster=\"$cluster\",namespace=\"$namespace\"}[$__rate_interval])[$__range:$__rate_interval])))", @@ -1839,11 +1872,16 @@ "refId": "A" } ], - "timeFrom": null, - "timeShift": null, "title": "Discarded Lines Per Interval", - "transform": "table", - "type": "table-old" + "transformations": [ + { + "id": "merge", + "options": { + "reducers": [ ] + } + } + ], + "type": "table" } ], "targets": [ ], diff --git a/production/loki-mixin/dashboards/dashboard-loki-operational.json b/production/loki-mixin/dashboards/dashboard-loki-operational.json index 3f215c2e90833..da51f8612ed8c 100644 --- a/production/loki-mixin/dashboards/dashboard-loki-operational.json +++ b/production/loki-mixin/dashboards/dashboard-loki-operational.json @@ -1741,9 +1741,93 @@ } }, { - "columns": [], "datasource": "$datasource", - "fontSize": "100%", + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "custom": { + "align": "right", + "cellOptions": { + "type": "auto" + }, + "inspect": false + }, + "decimals": 2, + "displayName": "", + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "short" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "Time" + }, + "properties": [ + { + "id": "displayName", + "value": "Time" + }, + { + "id": "custom.align" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "tenant" + }, + "properties": [ + { + "id": "unit", + "value": "short" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "custom.align" + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "reason" + }, + "properties": [ + { + "id": "unit", + "value": "short" + }, + { + "id": "decimals", + "value": 2 + }, + { + "id": "custom.align" + } + ] + } + ] + }, "gridPos": { "h": 8, "w": 12, @@ -1751,70 +1835,20 @@ "y": 27 }, "id": 113, - "pageSize": null, - "showHeader": true, - "sort": { - "col": 3, - "desc": true - }, - "styles": [ - { - "alias": "Time", - "align": "auto", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "hidden" - }, - { - "alias": "", - "align": "auto", - "colorMode": null, - "colors": [ - "rgba(245, 54, 54, 0.9)", - "rgba(237, 129, 40, 0.89)", - "rgba(50, 172, 45, 0.97)" - ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "mappingType": 1, - "pattern": "tenant", - "thresholds": [], - "type": "string", - "unit": "short" - }, - { - "alias": "", - "align": "auto", - "colorMode": null, - "colors": [ - "rgba(245, 54, 54, 0.9)", - "rgba(237, 129, 40, 0.89)", - "rgba(50, 172, 45, 0.97)" + "options": { + "cellHeight": "sm", + "footer": { + "countRows": false, + "fields": "", + "reducer": [ + "sum" ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "mappingType": 1, - "pattern": "reason", - "thresholds": [], - "type": "number", - "unit": "short" + "show": false }, - { - "alias": "", - "align": "right", - "colorMode": null, - "colors": [ - "rgba(245, 54, 54, 0.9)", - "rgba(237, 129, 40, 0.89)", - "rgba(50, 172, 45, 0.97)" - ], - "decimals": 2, - "pattern": "/.*/", - "thresholds": [], - "type": "number", - "unit": "short" - } - ], + "showHeader": true + }, + "panels": [], + "pluginVersion": "10.4.0", "targets": [ { "expr": "topk(10, sum by (tenant, reason) (sum_over_time(increase(loki_discarded_samples_total{cluster=\"$cluster\",namespace=\"$namespace\"}[$__rate_interval])[$__range:$__rate_interval])))", @@ -1825,11 +1859,16 @@ "refId": "A" } ], - "timeFrom": null, - "timeShift": null, "title": "Discarded Lines Per Interval", - "transform": "table", - "type": "table-old" + "transformations": [ + { + "id": "merge", + "options": { + "reducers": [] + } + } + ], + "type": "table" } ], "title": "Limits", @@ -1852,7 +1891,6 @@ "dashLength": 10, "dashes": false, "datasource": "$datasource", - "fill": 1, "fillGradient": 0, "gridPos": { @@ -2244,7 +2282,7 @@ "dashes": false, "datasource": "$datasource", "fieldConfig": { - "defaults": { + "defaults": { "unit": "binBps" } }, diff --git a/production/loki-mixin/dashboards/loki-bloom-compactor.libsonnet b/production/loki-mixin/dashboards/loki-bloom-compactor.libsonnet index cf93c7c992c53..236ebff8666d7 100644 --- a/production/loki-mixin/dashboards/loki-bloom-compactor.libsonnet +++ b/production/loki-mixin/dashboards/loki-bloom-compactor.libsonnet @@ -9,11 +9,64 @@ local raw = (import './dashboard-bloom-compactor.json'); grafanaDashboards+: { 'loki-bloom-compactor.json': - raw + - $.dashboard('Loki / Bloom Compactor', uid='bloom-compactor') - .addCluster() - .addNamespace() - .addLog() - .addTag(), + raw + { + local replaceClusterMatchers(expr) = + // Replace the recording rules cluster label with the per-cluster label + std.strReplace( + // Replace the cluster label for equality matchers with the per-cluster label + std.strReplace( + // Replace the cluster label for regex matchers with the per-cluster label + std.strReplace( + expr, + 'cluster=~"$cluster"', + $._config.per_cluster_label + '=~"$cluster"' + ), + 'cluster="$cluster"', + $._config.per_cluster_label + '="$cluster"' + ), + 'cluster_job', + $._config.per_cluster_label + '_job' + ), + + panels: [ + p { + targets: if std.objectHas(p, 'targets') then [ + e { + expr: replaceClusterMatchers(e.expr), + } + for e in p.targets + ] else [], + panels: if std.objectHas(p, 'panels') then [ + sp { + targets: if std.objectHas(sp, 'targets') then [ + spe { + expr: replaceClusterMatchers(spe.expr), + } + for spe in sp.targets + ] else [], + panels: if std.objectHas(sp, 'panels') then [ + ssp { + targets: if std.objectHas(ssp, 'targets') then [ + sspe { + expr: replaceClusterMatchers(sspe.expr), + } + for sspe in ssp.targets + ] else [], + } + for ssp in sp.panels + ] else [], + } + for sp in p.panels + ] else [], + } + for p in super.panels + ], + } + + $.dashboard('Loki / Bloom Compactor', uid='bloom-compactor') + .addCluster() + .addNamespace() + .addLog() + .addTag(), }, } diff --git a/production/loki-mixin/dashboards/loki-bloom-gateway.libsonnet b/production/loki-mixin/dashboards/loki-bloom-gateway.libsonnet index e5ca9f2ff3fd8..db9c907839117 100644 --- a/production/loki-mixin/dashboards/loki-bloom-gateway.libsonnet +++ b/production/loki-mixin/dashboards/loki-bloom-gateway.libsonnet @@ -9,11 +9,64 @@ local raw = (import './dashboard-bloom-gateway.json'); grafanaDashboards+: { 'loki-bloom-gateway.json': - raw + - $.dashboard('Loki / Bloom Gateway', uid='bloom-gateway') - .addCluster() - .addNamespace() - .addLog() - .addTag(), + raw + { + local replaceClusterMatchers(expr) = + // Replace the recording rules cluster label with the per-cluster label + std.strReplace( + // Replace the cluster label for equality matchers with the per-cluster label + std.strReplace( + // Replace the cluster label for regex matchers with the per-cluster label + std.strReplace( + expr, + 'cluster=~"$cluster"', + $._config.per_cluster_label + '=~"$cluster"' + ), + 'cluster="$cluster"', + $._config.per_cluster_label + '="$cluster"' + ), + 'cluster_job', + $._config.per_cluster_label + '_job' + ), + + panels: [ + p { + targets: if std.objectHas(p, 'targets') then [ + e { + expr: replaceClusterMatchers(e.expr), + } + for e in p.targets + ] else [], + panels: if std.objectHas(p, 'panels') then [ + sp { + targets: if std.objectHas(sp, 'targets') then [ + spe { + expr: replaceClusterMatchers(spe.expr), + } + for spe in sp.targets + ] else [], + panels: if std.objectHas(sp, 'panels') then [ + ssp { + targets: if std.objectHas(ssp, 'targets') then [ + sspe { + expr: replaceClusterMatchers(sspe.expr), + } + for sspe in ssp.targets + ] else [], + } + for ssp in sp.panels + ] else [], + } + for sp in p.panels + ] else [], + } + for p in super.panels + ], + } + + $.dashboard('Loki / Bloom Gateway', uid='bloom-gateway') + .addCluster() + .addNamespace() + .addLog() + .addTag(), }, } diff --git a/production/loki-mixin/dashboards/loki-deletion.libsonnet b/production/loki-mixin/dashboards/loki-deletion.libsonnet index 0ddb39fea6953..a1c50ecfa6911 100644 --- a/production/loki-mixin/dashboards/loki-deletion.libsonnet +++ b/production/loki-mixin/dashboards/loki-deletion.libsonnet @@ -12,6 +12,7 @@ local utils = import 'mixin-utils/utils.libsonnet'; .addCluster() .addNamespace() .addTag() + .addLog() .addRow( ($.row('Headlines') + { diff --git a/production/loki-mixin/dashboards/loki-reads-resources.libsonnet b/production/loki-mixin/dashboards/loki-reads-resources.libsonnet index 21db04ea2cf88..0ec22e131edbc 100644 --- a/production/loki-mixin/dashboards/loki-reads-resources.libsonnet +++ b/production/loki-mixin/dashboards/loki-reads-resources.libsonnet @@ -94,8 +94,24 @@ local utils = import 'mixin-utils/utils.libsonnet'; $.containerDiskSpaceUtilizationPanel('Disk Space Utilization', 'querier'), ) ) - .addRow( - grafana.row.new(if $._config.ssd.enabled then 'Read path' else 'Index Gateway') + // Add the read path for single scalable deployment only. The read path should not display disk utilization as the index gateway is present in the backend pods. + .addRowIf( + $._config.ssd.enabled, + grafana.row.new('Read path') + .addPanel( + $.CPUUsagePanel('CPU', index_gateway_pod_matcher), + ) + .addPanel( + $.memoryWorkingSetPanel('Memory (workingset)', index_gateway_pod_matcher), + ) + .addPanel( + $.goHeapInUsePanel('Memory (go heap inuse)', index_gateway_job_matcher), + ) + ) + // Otherwise we add the index gateway information + .addRowIf( + !$._config.ssd.enabled, + grafana.row.new('Index Gateway') .addPanel( $.CPUUsagePanel('CPU', index_gateway_pod_matcher), )