diff --git a/docs/sources/operations/query-acceleration-blooms.md b/docs/sources/operations/query-acceleration-blooms.md index 8f61287a94699..1d18a0350c4b2 100644 --- a/docs/sources/operations/query-acceleration-blooms.md +++ b/docs/sources/operations/query-acceleration-blooms.md @@ -45,7 +45,7 @@ and querying the bloom filters that only pays off at large scale deployments. To start building and using blooms you need to: - Deploy the [Bloom Planner and Builder](#bloom-planner-and-builder) components and enable the component in the [Bloom Build config][bloom-build-cfg]. - Deploy the [Bloom Gateway](#bloom-gateway) component (as a [microservice][microservices] or via the [SSD][ssd] Backend target) and enable the component in the [Bloom Gateway config][bloom-gateway-cfg]. -- Enable blooms filtering and compaction for each tenant individually, or for all of them by default. +- Enable blooms building and filtering for each tenant individually, or for all of them by default. ```yaml # Configuration block for the bloom creation. @@ -119,7 +119,7 @@ Builders process tasks sequentially by pulling them from the queue. The amount o all pending tasks before the next planning iteration depends on the value of `-bloom-build.planner.bloom_split_series_keyspace_by`, the amount of tenants, and the log volume of the streams. -The maximum block size is configured per tenant via `-bloom-compactor.max-block-size`. +The maximum block size is configured per tenant via `-bloom-build.max-block-size`. The actual block size might exceed this limit given that we append streams blooms to the block until the block is larger than the configured maximum size. Blocks are created in memory and as soon as they are written to the object store they are freed. Chunks and TSDB files are downloaded from the object store to the file system. diff --git a/docs/sources/setup/upgrade/_index.md b/docs/sources/setup/upgrade/_index.md index de3e38a4548b3..1b68d61828973 100644 --- a/docs/sources/setup/upgrade/_index.md +++ b/docs/sources/setup/upgrade/_index.md @@ -46,6 +46,36 @@ parameter contains a log selector query instead of returning inconsistent result Loki changes the default value of `-ruler.alertmanager-use-v2` from `false` to `true`. Alertmanager APIv1 was deprecated in Alertmanager 0.16.0 and is removed as of 0.27.0. +### Experimental Bloom Filters + +{{% admonition type="note" %}} +Experimental features are subject to rapid change and/or removal, which can introduce breaking changes even between minor version. +They also don't follow the deprecation lifecycle of regular features. +{{% /admonition %}} + +The bloom compactor component, which builds bloom filter blocks for query acceleration, has been removed in favor of two new components: bloom planner and bloom builder. +Please consult the [Query Acceleration with Blooms](https://grafana.com/docs/loki//operations/query-acceleration-blooms/) docs for more information. + +CLI arguments (and their YAML counterparts) of per-tenant settings that have been removed as part of this change: + +* `-bloom-compactor.enable-compaction` +* `-bloom-compactor.shard-size` +* `-bloom-compactor.shard-size` +* `-bloom-compactor.shard-size` + +CLI arguments of per-tenant settings that have been moved to a different prefix as part of this change: + +* `-bloom-compactor.max-page-size` changed to `-bloom-builder.max-page-size` +* `-bloom-compactor.max-block-size` changed to `-bloom-builder.max-block-size` +* `-bloom-compactor.ngram-length` changed to `-bloom-builder.ngram-length` +* `-bloom-compactor.ngram-skip` changed to `-bloom-builder.ngram-skip` +* `-bloom-compactor.false-positive-rate` changed to `-bloom-builder.false-positive-rate` +* `-bloom-compactor.block-encoding` changed to `-bloom-builder.block-encoding` + +Their YAML counterparts in the `limits_config` block are kept identical. + +All other CLI arguments (and their YAML counterparts) prefixed with `-bloom-compactor.` have been removed. + ## 3.0.0 {{% admonition type="note" %}} diff --git a/docs/sources/shared/configuration.md b/docs/sources/shared/configuration.md index 12aa2eecf5ad4..f9ab42c293211 100644 --- a/docs/sources/shared/configuration.md +++ b/docs/sources/shared/configuration.md @@ -817,68 +817,9 @@ pattern_ingester: # object store. [index_gateway: ] -# Experimental: The bloom_compactor block configures the Loki bloom compactor -# server, responsible for compacting stream indexes into bloom filters and -# 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] - - retention: - # Enable bloom retention. - # CLI flag: -bloom-build.planner.retention.enabled - [enabled: | default = false] - - builder: - # The grpc_client block configures the gRPC client used to communicate - # between a client and server component in Loki. - # The CLI flags prefix for this block configuration is: - # bloom-gateway-client.grpc - [grpc_config: ] - - # Hostname (and port) of the bloom planner - # CLI flag: -bloom-build.builder.planner-address - [planner_address: | default = ""] - - backoff_config: - # Minimum delay when backing off. - # CLI flag: -bloom-build.builder.backoff.backoff-min-period - [min_period: | default = 100ms] - - # Maximum delay when backing off. - # CLI flag: -bloom-build.builder.backoff.backoff-max-period - [max_period: | default = 10s] - - # Number of times to backoff and retry before failing. - # CLI flag: -bloom-build.builder.backoff.backoff-retries - [max_retries: | default = 10] +# Experimental: The bloom_build block configures the Loki bloom planner and +# builder servers, responsible for building bloom filters. +[bloom_build: ] # Experimental: The bloom_gateway block configures the Loki bloom gateway # server, responsible for serving queries for filtering chunks based on filter @@ -1466,152 +1407,67 @@ The `azure_storage_config` block configures the connection to Azure object stora [max_retry_delay: | default = 500ms] ``` -### bloom_compactor +### bloom_build -Experimental: The `bloom_compactor` block configures the Loki bloom compactor server, responsible for compacting stream indexes into bloom filters and merging them as bloom blocks. +Experimental: The `bloom_build` block configures the Loki bloom planner and builder servers, responsible for building bloom filters. ```yaml -# Defines the ring to be used by the bloom-compactor servers. In case this isn't -# configured, this block supports inheriting configuration from the common ring -# section. -ring: - kvstore: - # Backend storage to use for the ring. Supported values are: consul, etcd, - # inmemory, memberlist, multi. - # CLI flag: -bloom-compactor.ring.store - [store: | default = "consul"] - - # The prefix for the keys in the store. Should end with a /. - # CLI flag: -bloom-compactor.ring.prefix - [prefix: | default = "collectors/"] - - # Configuration for a Consul client. Only applies if the selected kvstore is - # consul. - # The CLI flags prefix for this block configuration is: bloom-compactor.ring - [consul: ] - - # Configuration for an ETCD v3 client. Only applies if the selected kvstore - # is etcd. - # The CLI flags prefix for this block configuration is: bloom-compactor.ring - [etcd: ] - - multi: - # Primary backend storage used by multi-client. - # CLI flag: -bloom-compactor.ring.multi.primary - [primary: | default = ""] - - # Secondary backend storage used by multi-client. - # CLI flag: -bloom-compactor.ring.multi.secondary - [secondary: | default = ""] - - # Mirror writes to secondary store. - # CLI flag: -bloom-compactor.ring.multi.mirror-enabled - [mirror_enabled: | default = false] - - # Timeout for storing value to secondary store. - # CLI flag: -bloom-compactor.ring.multi.mirror-timeout - [mirror_timeout: | default = 2s] - - # Period at which to heartbeat to the ring. 0 = disabled. - # CLI flag: -bloom-compactor.ring.heartbeat-period - [heartbeat_period: | default = 15s] - - # The heartbeat timeout after which compactors are considered unhealthy within - # the ring. 0 = never (timeout disabled). - # CLI flag: -bloom-compactor.ring.heartbeat-timeout - [heartbeat_timeout: | default = 1m] - - # File path where tokens are stored. If empty, tokens are not stored at - # shutdown and restored at startup. - # CLI flag: -bloom-compactor.ring.tokens-file-path - [tokens_file_path: | default = ""] - - # True to enable zone-awareness and replicate blocks across different - # availability zones. - # CLI flag: -bloom-compactor.ring.zone-awareness-enabled - [zone_awareness_enabled: | default = false] - - # Number of tokens to use in the ring per compactor. Higher number of tokens - # will result in more and smaller files (metas and blocks.) - # CLI flag: -bloom-compactor.ring.num-tokens - [num_tokens: | default = 10] - - # Instance ID to register in the ring. - # CLI flag: -bloom-compactor.ring.instance-id - [instance_id: | default = ""] - - # Name of network interface to read address from. - # CLI flag: -bloom-compactor.ring.instance-interface-names - [instance_interface_names: | default = []] - - # Port to advertise in the ring (defaults to server.grpc-listen-port). - # CLI flag: -bloom-compactor.ring.instance-port - [instance_port: | default = 0] - - # IP address to advertise in the ring. - # CLI flag: -bloom-compactor.ring.instance-addr - [instance_addr: | default = ""] - - # The availability zone where this instance is running. Required if - # zone-awareness is enabled. - # CLI flag: -bloom-compactor.ring.instance-availability-zone - [instance_availability_zone: | default = ""] - - # Enable using a IPv6 instance address. - # CLI flag: -bloom-compactor.ring.instance-enable-ipv6 - [instance_enable_ipv6: | default = false] - -# Flag to enable or disable the usage of the bloom-compactor component. -# CLI flag: -bloom-compactor.enabled +# Flag to enable or disable the usage of the bloom-planner and bloom-builder +# components. +# CLI flag: -bloom-build.enabled [enabled: | default = false] -# Interval at which to re-run the compaction operation. -# CLI flag: -bloom-compactor.compaction-interval -[compaction_interval: | default = 10m] - -# Newest day-table offset (from today, inclusive) to compact. 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-compactor.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-compactor.max-table-offset -[max_table_offset: | default = 2] - -# Number of workers to run in parallel for compaction. -# CLI flag: -bloom-compactor.worker-parallelism -[worker_parallelism: | default = 1] - -# Minimum backoff time between retries. -# CLI flag: -bloom-compactor.compaction-retries-min-backoff -[compaction_retries_min_backoff: | default = 10s] +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] + + retention: + # Enable bloom retention. + # CLI flag: -bloom-build.planner.retention.enabled + [enabled: | default = false] -# Maximum backoff time between retries. -# CLI flag: -bloom-compactor.compaction-retries-max-backoff -[compaction_retries_max_backoff: | default = 1m] +builder: + # The grpc_client block configures the gRPC client used to communicate between + # a client and server component in Loki. + # The CLI flags prefix for this block configuration is: + # bloom-gateway-client.grpc + [grpc_config: ] -# Number of retries to perform when compaction fails. -# CLI flag: -bloom-compactor.compaction-retries -[compaction_retries: | default = 3] + # Hostname (and port) of the bloom planner + # CLI flag: -bloom-build.builder.planner-address + [planner_address: | default = ""] -# Maximum number of tables to compact in parallel. While increasing this value, -# please make sure compactor has enough disk space allocated to be able to store -# and compact as many tables. -# CLI flag: -bloom-compactor.max-compaction-parallelism -[max_compaction_parallelism: | default = 1] + backoff_config: + # Minimum delay when backing off. + # CLI flag: -bloom-build.builder.backoff.backoff-min-period + [min_period: | default = 100ms] -retention: - # Enable bloom retention. - # CLI flag: -bloom-compactor.retention.enabled - [enabled: | default = false] + # Maximum delay when backing off. + # CLI flag: -bloom-build.builder.backoff.backoff-max-period + [max_period: | default = 10s] - # Max lookback days for retention. - # CLI flag: -bloom-compactor.retention.max-lookback-days - [max_lookback_days: | default = 365] + # Number of times to backoff and retry before failing. + # CLI flag: -bloom-build.builder.backoff.backoff-retries + [max_retries: | default = 10] ``` ### bloom_gateway @@ -2382,7 +2238,6 @@ compactor_ring: Configuration for a Consul client. Only applies if the selected kvstore is `consul`. The supported CLI flags `` used to reference this configuration block are: -- `bloom-compactor.ring` - `common.storage.ring` - `compactor.ring` - `distributor.ring` @@ -2603,7 +2458,6 @@ otlp_config: Configuration for an ETCD v3 client. Only applies if the selected kvstore is `etcd`. The supported CLI flags `` used to reference this configuration block are: -- `bloom-compactor.ring` - `common.storage.ring` - `compactor.ring` - `distributor.ring` @@ -3959,27 +3813,22 @@ shard_streams: # CLI flag: -bloom-gateway.cache-key-interval [bloom_gateway_cache_key_interval: | default = 15m] -# Experimental. The shard size defines how many bloom compactors should be used -# by a tenant when computing blooms. If it's set to 0, shuffle sharding is -# disabled. -# CLI flag: -bloom-compactor.shard-size -[bloom_compactor_shard_size: | default = 0] - -# Experimental. Whether to compact chunks into bloom filters. -# CLI flag: -bloom-compactor.enable-compaction -[bloom_compactor_enable_compaction: | default = false] +# 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. The maximum bloom block size. A value of 0 sets an unlimited -# size. Default is 200MB. The actual block size might exceed this limit since -# blooms will be added to blocks until the block exceeds the maximum block size. -# CLI flag: -bloom-compactor.max-block-size -[bloom_compactor_max_block_size: | default = 200MB] +# Experimental. Maximum number of retries for a failed task. If a task fails +# more than this number of times, it is considered failed and will not be +# retried. A value of 0 disables this limit. +# CLI flag: -bloom-build.task-max-retries +[bloom_build_task_max_retries: | default = 3] -# Experimental. The maximum bloom size per log stream. A log stream whose -# generated bloom filter exceeds this size will be discarded. A value of 0 sets -# an unlimited size. Default is 128MB. -# CLI flag: -bloom-compactor.max-bloom-size -[bloom_compactor_max_bloom_size: | default = 128MB] +# Experimental. Timeout for a builder to finish a task. If a builder does not +# respond within this time, it is considered failed and the task will be +# requeued. 0 disables the timeout. +# CLI flag: -bloom-build.builder-response-timeout +[bloom_build_builder_response_timeout: | default = 0s] # Experimental. Whether to create blooms for the tenant. # CLI flag: -bloom-build.enable @@ -3991,41 +3840,36 @@ shard_streams: # 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. Timeout for a builder to finish a task. If a builder does not -# respond within this time, it is considered failed and the task will be -# requeued. 0 disables the timeout. -# CLI flag: -bloom-build.builder-response-timeout -[bloom_build_builder_response_timeout: | default = 0s] - -# Experimental. Maximum number of retries for a failed task. If a task fails -# more than this number of times, it is considered failed and will not be -# retried. A value of 0 disables this limit. -# CLI flag: -bloom-build.task-max-retries -[bloom_build_task_max_retries: | default = 3] - # Experimental. Length of the n-grams created when computing blooms from log # lines. -# CLI flag: -bloom-compactor.ngram-length +# CLI flag: -bloom-build.ngram-length [bloom_ngram_length: | default = 4] # Experimental. Skip factor for the n-grams created when computing blooms from # log lines. -# CLI flag: -bloom-compactor.ngram-skip +# CLI flag: -bloom-build.ngram-skip [bloom_ngram_skip: | default = 1] # Experimental. Scalable Bloom Filter desired false-positive rate. -# CLI flag: -bloom-compactor.false-positive-rate +# CLI flag: -bloom-build.false-positive-rate [bloom_false_positive_rate: | default = 0.01] # Experimental. Compression algorithm for bloom block pages. -# CLI flag: -bloom-compactor.block-encoding +# CLI flag: -bloom-build.block-encoding [bloom_block_encoding: | default = "none"] +# Experimental. The maximum bloom block size. A value of 0 sets an unlimited +# size. Default is 200MB. The actual block size might exceed this limit since +# blooms will be added to blocks until the block exceeds the maximum block size. +# CLI flag: -bloom-build.max-block-size +[bloom_max_block_size: | default = 200MB] + +# Experimental. The maximum bloom size per log stream. A log stream whose +# generated bloom filter exceeds this size will be discarded. A value of 0 sets +# an unlimited size. Default is 128MB. +# CLI flag: -bloom-build.max-bloom-size +[bloom_max_bloom_size: | default = 128MB] + # Allow user to send structured metadata in push payload. # CLI flag: -validation.allow-structured-metadata [allow_structured_metadata: | default = true] diff --git a/integration/bloom_building_test.go b/integration/bloom_building_test.go index 46e8570c47717..2c4662eef4cb3 100644 --- a/integration/bloom_building_test.go +++ b/integration/bloom_building_test.go @@ -92,6 +92,7 @@ func TestBloomBuilding(t *testing.T) { "-target=bloom-planner", "-bloom-build.enabled=true", "-bloom-build.enable=true", + "-bloom-build.builder.planner-address=localhost:9095", // hack to succeed config validation "-bloom-build.planner.interval=15s", "-bloom-build.planner.min-table-offset=0", // Disable table offset so we process today's data. "-bloom.cache-list-ops=0", // Disable cache list operations to avoid caching issues. diff --git a/integration/cluster/cluster.go b/integration/cluster/cluster.go index 2ee68e15cc178..57bc182d0c8cb 100644 --- a/integration/cluster/cluster.go +++ b/integration/cluster/cluster.go @@ -89,7 +89,7 @@ storage_config: bloom_gateway: enabled: false -bloom_compactor: +bloom_build: enabled: false compactor: diff --git a/integration/loki_micro_services_test.go b/integration/loki_micro_services_test.go index 48f9123d96eb1..cfa46c1e9d85b 100644 --- a/integration/loki_micro_services_test.go +++ b/integration/loki_micro_services_test.go @@ -5,14 +5,11 @@ package integration import ( "context" "encoding/json" - "fmt" - "math/rand" "strings" "sync" "testing" "time" - "github.com/go-kit/log/level" dto "github.com/prometheus/client_model/go" "github.com/prometheus/common/expfmt" "github.com/prometheus/prometheus/model/labels" @@ -1078,218 +1075,6 @@ func TestCategorizedLabels(t *testing.T) { } } -func TestBloomFiltersEndToEnd(t *testing.T) { - t.Skip("skipping until blooms have settled") - commonFlags := []string{ - "-bloom-compactor.compaction-interval=10s", - "-bloom-compactor.enable-compaction=true", - "-bloom-compactor.enabled=true", - "-bloom-gateway.enable-filtering=true", - "-bloom-gateway.enabled=true", - "-compactor.compaction-interval=1s", - "-frontend.default-validity=0s", - "-ingester.flush-on-shutdown=true", - "-ingester.wal-enabled=false", - "-query-scheduler.use-scheduler-ring=false", - "-store.index-cache-read.embedded-cache.enabled=true", - "-querier.split-queries-by-interval=24h", - } - - tenantID := randStringRunes() - - clu := cluster.New( - level.DebugValue(), - cluster.SchemaWithTSDB, - func(c *cluster.Cluster) { c.SetSchemaVer("v13") }, - ) - - defer func() { - assert.NoError(t, clu.Cleanup()) - }() - - var ( - tDistributor = clu.AddComponent( - "distributor", - append( - commonFlags, - "-target=distributor", - )..., - ) - tIndexGateway = clu.AddComponent( - "index-gateway", - append( - commonFlags, - "-target=index-gateway", - )..., - ) - tBloomGateway = clu.AddComponent( - "bloom-gateway", - append( - commonFlags, - "-target=bloom-gateway", - )..., - ) - ) - require.NoError(t, clu.Run()) - - var ( - tIngester = clu.AddComponent( - "ingester", - append( - commonFlags, - "-target=ingester", - "-tsdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(), - )..., - ) - tQueryScheduler = clu.AddComponent( - "query-scheduler", - append( - commonFlags, - "-target=query-scheduler", - "-tsdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(), - )..., - ) - tCompactor = clu.AddComponent( - "compactor", - append( - commonFlags, - "-target=compactor", - "-tsdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(), - )..., - ) - tBloomCompactor = clu.AddComponent( - "bloom-compactor", - append( - commonFlags, - "-target=bloom-compactor", - "-tsdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(), - )..., - ) - ) - require.NoError(t, clu.Run()) - - // finally, run the query-frontend and querier. - var ( - tQueryFrontend = clu.AddComponent( - "query-frontend", - append( - commonFlags, - "-target=query-frontend", - "-frontend.scheduler-address="+tQueryScheduler.GRPCURL(), - "-common.compactor-address="+tCompactor.HTTPURL(), - "-tsdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(), - )..., - ) - _ = clu.AddComponent( - "querier", - append( - commonFlags, - "-target=querier", - "-querier.scheduler-address="+tQueryScheduler.GRPCURL(), - "-common.compactor-address="+tCompactor.HTTPURL(), - "-tsdb.shipper.index-gateway-client.server-address="+tIndexGateway.GRPCURL(), - )..., - ) - ) - require.NoError(t, clu.Run()) - - now := time.Now() - - cliDistributor := client.New(tenantID, "", tDistributor.HTTPURL()) - cliDistributor.Now = now - - cliIngester := client.New(tenantID, "", tIngester.HTTPURL()) - cliIngester.Now = now - - cliQueryFrontend := client.New(tenantID, "", tQueryFrontend.HTTPURL()) - cliQueryFrontend.Now = now - - cliIndexGateway := client.New(tenantID, "", tIndexGateway.HTTPURL()) - cliIndexGateway.Now = now - - cliBloomGateway := client.New(tenantID, "", tBloomGateway.HTTPURL()) - cliBloomGateway.Now = now - - cliBloomCompactor := client.New(tenantID, "", tBloomCompactor.HTTPURL()) - cliBloomCompactor.Now = now - - lineTpl := `caller=loki_micro_services_test.go msg="push log line" id="%s"` - // ingest logs from 10 different pods - // from now-60m to now-55m - // each line contains a random, unique string - // that string is used to verify filtering using bloom gateway - uniqueStrings := make([]string, 5*60) - for i := 0; i < len(uniqueStrings); i++ { - id := randStringRunes() - id = fmt.Sprintf("%s-%d", id, i) - uniqueStrings[i] = id - pod := fmt.Sprintf("pod-%d", i%10) - line := fmt.Sprintf(lineTpl, id) - err := cliDistributor.PushLogLine( - line, - now.Add(-1*time.Hour).Add(time.Duration(i)*time.Second), - nil, - map[string]string{"pod": pod}, - ) - require.NoError(t, err) - } - - // restart ingester to flush chunks and that there are zero chunks in memory - require.NoError(t, cliIngester.Flush()) - require.NoError(t, tIngester.Restart()) - - // wait for compactor to compact index and for bloom compactor to build bloom filters - require.Eventually(t, func() bool { - // verify metrics that observe usage of block for filtering - metrics, err := cliBloomCompactor.Metrics() - require.NoError(t, err) - successfulRunCount, labels, err := extractMetric(`loki_bloomcompactor_runs_completed_total`, metrics) - if err != nil { - return false - } - t.Log("bloom compactor runs", successfulRunCount, labels) - if labels["status"] != "success" { - return false - } - - return successfulRunCount == 1 - }, 30*time.Second, time.Second) - - // use bloom gateway to perform needle in the haystack queries - randIdx := rand.Intn(len(uniqueStrings)) - q := fmt.Sprintf(`{job="varlog"} |= "%s"`, uniqueStrings[randIdx]) - start := now.Add(-90 * time.Minute) - end := now.Add(-30 * time.Minute) - resp, err := cliQueryFrontend.RunRangeQueryWithStartEnd(context.Background(), q, start, end) - require.NoError(t, err) - - // verify response - require.Len(t, resp.Data.Stream, 1) - expectedLine := fmt.Sprintf(lineTpl, uniqueStrings[randIdx]) - require.Equal(t, expectedLine, resp.Data.Stream[0].Values[0][1]) - - // verify metrics that observe usage of block for filtering - bloomGwMetrics, err := cliBloomGateway.Metrics() - require.NoError(t, err) - - unfilteredCount := getMetricValue(t, "loki_bloom_gateway_chunkrefs_pre_filtering", bloomGwMetrics) - require.Equal(t, float64(10), unfilteredCount) - - filteredCount := getMetricValue(t, "loki_bloom_gateway_chunkrefs_post_filtering", bloomGwMetrics) - require.Equal(t, float64(1), filteredCount) - - mf, err := extractMetricFamily("loki_bloom_gateway_bloom_query_latency", bloomGwMetrics) - require.NoError(t, err) - - count := getValueFromMetricFamilyWithFunc(mf, &dto.LabelPair{ - Name: proto.String("status"), - Value: proto.String("success"), - }, func(m *dto.Metric) uint64 { - return m.Histogram.GetSampleCount() - }) - require.Equal(t, uint64(1), count) -} - func getValueFromMF(mf *dto.MetricFamily, lbs []*dto.LabelPair) float64 { return getValueFromMetricFamilyWithFunc(mf, lbs[0], func(m *dto.Metric) float64 { return m.Counter.GetValue() }) } diff --git a/pkg/bloombuild/builder/builder.go b/pkg/bloombuild/builder/builder.go index 045f96bc7f591..672ce24f0f403 100644 --- a/pkg/bloombuild/builder/builder.go +++ b/pkg/bloombuild/builder/builder.go @@ -310,8 +310,8 @@ func (b *Builder) processTask( blockCt int nGramSize = uint64(b.limits.BloomNGramLength(tenant)) nGramSkip = uint64(b.limits.BloomNGramSkip(tenant)) - maxBlockSize = uint64(b.limits.BloomCompactorMaxBlockSize(tenant)) - maxBloomSize = uint64(b.limits.BloomCompactorMaxBloomSize(tenant)) + maxBlockSize = uint64(b.limits.BloomMaxBlockSize(tenant)) + maxBloomSize = uint64(b.limits.BloomMaxBloomSize(tenant)) blockOpts = v1.NewBlockOptions(blockEnc, nGramSize, nGramSkip, maxBlockSize, maxBloomSize) created []bloomshipper.Meta totalSeries int diff --git a/pkg/bloombuild/builder/builder_test.go b/pkg/bloombuild/builder/builder_test.go index b04a34fb6eeb2..38ed1b1930fd9 100644 --- a/pkg/bloombuild/builder/builder_test.go +++ b/pkg/bloombuild/builder/builder_test.go @@ -234,11 +234,11 @@ func (f fakeLimits) BloomNGramSkip(_ string) int { panic("implement me") } -func (f fakeLimits) BloomCompactorMaxBlockSize(_ string) int { +func (f fakeLimits) BloomMaxBlockSize(_ string) int { panic("implement me") } -func (f fakeLimits) BloomCompactorMaxBloomSize(_ string) int { +func (f fakeLimits) BloomMaxBloomSize(_ string) int { panic("implement me") } diff --git a/pkg/bloombuild/builder/config.go b/pkg/bloombuild/builder/config.go index deeeb951465ab..ddacfd884e10c 100644 --- a/pkg/bloombuild/builder/config.go +++ b/pkg/bloombuild/builder/config.go @@ -40,6 +40,6 @@ type Limits interface { BloomBlockEncoding(tenantID string) string BloomNGramLength(tenantID string) int BloomNGramSkip(tenantID string) int - BloomCompactorMaxBlockSize(tenantID string) int - BloomCompactorMaxBloomSize(tenantID string) int + BloomMaxBlockSize(tenantID string) int + BloomMaxBloomSize(tenantID string) int } diff --git a/pkg/bloomcompactor/batch.go b/pkg/bloomcompactor/batch.go deleted file mode 100644 index c4e1043b44831..0000000000000 --- a/pkg/bloomcompactor/batch.go +++ /dev/null @@ -1,357 +0,0 @@ -package bloomcompactor - -import ( - "context" - "io" - "math" - "time" - - "github.com/grafana/dskit/multierror" - "golang.org/x/exp/slices" - - "github.com/grafana/loki/v3/pkg/chunkenc" - iter "github.com/grafana/loki/v3/pkg/iter/v2" - "github.com/grafana/loki/v3/pkg/logproto" - logql_log "github.com/grafana/loki/v3/pkg/logql/log" - v1 "github.com/grafana/loki/v3/pkg/storage/bloom/v1" - "github.com/grafana/loki/v3/pkg/storage/chunk" - "github.com/grafana/loki/v3/pkg/storage/stores/shipper/bloomshipper" -) - -type Fetcher[A, B any] interface { - Fetch(ctx context.Context, inputs []A) ([]B, error) -} - -type FetchFunc[A, B any] func(ctx context.Context, inputs []A) ([]B, error) - -func (f FetchFunc[A, B]) Fetch(ctx context.Context, inputs []A) ([]B, error) { - return f(ctx, inputs) -} - -// batchedLoader implements `v1.Iterator[C]` in batches -type batchedLoader[A, B, C any] struct { - metrics *Metrics - batchSize int - ctx context.Context - fetchers []Fetcher[A, B] - work [][]A - - mapper func(B) (C, error) - cur C - batch []B - err error -} - -const batchedLoaderDefaultBatchSize = 50 - -func newBatchedLoader[A, B, C any]( - ctx context.Context, - fetchers []Fetcher[A, B], - inputs [][]A, - mapper func(B) (C, error), - batchSize int, -) *batchedLoader[A, B, C] { - return &batchedLoader[A, B, C]{ - batchSize: max(batchSize, 1), - ctx: ctx, - fetchers: fetchers, - work: inputs, - mapper: mapper, - } -} - -func (b *batchedLoader[A, B, C]) Next() bool { - - // iterate work until we have non-zero length batch - for len(b.batch) == 0 { - - // empty batch + no work remaining = we're done - if len(b.work) == 0 { - return false - } - - // setup next batch - next := b.work[0] - batchSize := min(b.batchSize, len(next)) - toFetch := next[:batchSize] - fetcher := b.fetchers[0] - - // update work - b.work[0] = b.work[0][batchSize:] - if len(b.work[0]) == 0 { - // if we've exhausted work from this set of inputs, - // set pointer to next set of inputs - // and their respective fetcher - b.work = b.work[1:] - b.fetchers = b.fetchers[1:] - } - - // there was no work in this batch; continue (should not happen) - if len(toFetch) == 0 { - continue - } - - b.batch, b.err = fetcher.Fetch(b.ctx, toFetch) - // error fetching, short-circuit iteration - if b.err != nil { - return false - } - } - - return b.prepNext() -} - -func (b *batchedLoader[_, B, C]) prepNext() bool { - b.cur, b.err = b.mapper(b.batch[0]) - b.batch = b.batch[1:] - return b.err == nil -} - -func (b *batchedLoader[_, _, C]) At() C { - return b.cur -} - -func (b *batchedLoader[_, _, _]) Err() error { - return b.err -} - -// to ensure memory is bounded while loading chunks -// TODO(owen-d): testware -func newBatchedChunkLoader( - ctx context.Context, - fetchers []Fetcher[chunk.Chunk, chunk.Chunk], - inputs [][]chunk.Chunk, - metrics *Metrics, - batchSize int, -) *batchedLoader[chunk.Chunk, chunk.Chunk, v1.ChunkRefWithIter] { - - mapper := func(c chunk.Chunk) (v1.ChunkRefWithIter, error) { - chk := c.Data.(*chunkenc.Facade).LokiChunk() - metrics.chunkSize.Observe(float64(chk.UncompressedSize())) - itr, err := chk.Iterator( - ctx, - time.Unix(0, 0), - time.Unix(0, math.MaxInt64), - logproto.FORWARD, - logql_log.NewNoopPipeline().ForStream(nil), - ) - - if err != nil { - return v1.ChunkRefWithIter{}, err - } - - return v1.ChunkRefWithIter{ - Ref: v1.ChunkRef{ - From: c.From, - Through: c.Through, - Checksum: c.Checksum, - }, - Itr: itr, - }, nil - } - return newBatchedLoader(ctx, fetchers, inputs, mapper, batchSize) -} - -func newBatchedBlockLoader( - ctx context.Context, - fetcher Fetcher[bloomshipper.BlockRef, *bloomshipper.CloseableBlockQuerier], - blocks []bloomshipper.BlockRef, - batchSize int, -) *batchedLoader[bloomshipper.BlockRef, *bloomshipper.CloseableBlockQuerier, *bloomshipper.CloseableBlockQuerier] { - - fetchers := []Fetcher[bloomshipper.BlockRef, *bloomshipper.CloseableBlockQuerier]{fetcher} - inputs := [][]bloomshipper.BlockRef{blocks} - mapper := func(a *bloomshipper.CloseableBlockQuerier) (*bloomshipper.CloseableBlockQuerier, error) { - return a, nil - } - - return newBatchedLoader(ctx, fetchers, inputs, mapper, batchSize) -} - -// compiler checks -var _ iter.Iterator[*v1.SeriesWithBlooms] = &blockLoadingIter{} -var _ iter.CloseIterator[*v1.SeriesWithBlooms] = &blockLoadingIter{} -var _ iter.ResetIterator[*v1.SeriesWithBlooms] = &blockLoadingIter{} - -// TODO(chaudum): testware -func newBlockLoadingIter(ctx context.Context, blocks []bloomshipper.BlockRef, fetcher FetchFunc[bloomshipper.BlockRef, *bloomshipper.CloseableBlockQuerier], batchSize int) *blockLoadingIter { - - return &blockLoadingIter{ - ctx: ctx, - fetcher: fetcher, - inputs: blocks, - batchSize: batchSize, - loaded: make(map[io.Closer]struct{}), - } -} - -type blockLoadingIter struct { - // constructor arguments - ctx context.Context - fetcher Fetcher[bloomshipper.BlockRef, *bloomshipper.CloseableBlockQuerier] - inputs []bloomshipper.BlockRef - overlapping iter.Iterator[[]bloomshipper.BlockRef] - batchSize int - // optional arguments - filter func(*bloomshipper.CloseableBlockQuerier) bool - // internals - initialized bool - err error - iter iter.Iterator[*v1.SeriesWithBlooms] - loader *batchedLoader[bloomshipper.BlockRef, *bloomshipper.CloseableBlockQuerier, *bloomshipper.CloseableBlockQuerier] - loaded map[io.Closer]struct{} -} - -// At implements v1.Iterator. -func (i *blockLoadingIter) At() *v1.SeriesWithBlooms { - if !i.initialized { - panic("iterator not initialized") - } - return i.iter.At() -} - -// Err implements v1.Iterator. -func (i *blockLoadingIter) Err() error { - if !i.initialized { - panic("iterator not initialized") - } - if i.err != nil { - return i.err - } - return i.iter.Err() -} - -func (i *blockLoadingIter) init() { - if i.initialized { - return - } - - // group overlapping blocks - i.overlapping = overlappingBlocksIter(i.inputs) - - // set initial iter - i.iter = iter.NewEmptyIter[*v1.SeriesWithBlooms]() - - // set "match all" filter function if not present - if i.filter == nil { - i.filter = func(cbq *bloomshipper.CloseableBlockQuerier) bool { return true } - } - - // done - i.initialized = true -} - -// load next populates the underlying iter via relevant batches -// and returns the result of iter.Next() -func (i *blockLoadingIter) loadNext() bool { - for i.overlapping.Next() { - blockRefs := i.overlapping.At() - - loader := newBatchedBlockLoader(i.ctx, i.fetcher, blockRefs, i.batchSize) - filtered := iter.NewFilterIter[*bloomshipper.CloseableBlockQuerier](loader, i.filter) - - iters := make([]iter.PeekIterator[*v1.SeriesWithBlooms], 0, len(blockRefs)) - for filtered.Next() { - bq := filtered.At() - i.loaded[bq] = struct{}{} - itr, err := bq.SeriesIter() - if err != nil { - i.err = err - i.iter = iter.NewEmptyIter[*v1.SeriesWithBlooms]() - return false - } - iters = append(iters, itr) - } - - if err := filtered.Err(); err != nil { - i.err = err - i.iter = iter.NewEmptyIter[*v1.SeriesWithBlooms]() - return false - } - - // edge case: we've filtered out all blocks in the batch; check next batch - if len(iters) == 0 { - continue - } - - // Turn the list of blocks into a single iterator that returns the next series - mergedBlocks := v1.NewHeapIterForSeriesWithBloom(iters...) - // two overlapping blocks can conceivably have the same series, so we need to dedupe, - // preferring the one with the most chunks already indexed since we'll have - // to add fewer chunks to the bloom - i.iter = iter.NewDedupingIter[*v1.SeriesWithBlooms, *v1.SeriesWithBlooms]( - func(a, b *v1.SeriesWithBlooms) bool { - return a.Series.Fingerprint == b.Series.Fingerprint - }, - iter.Identity[*v1.SeriesWithBlooms], - func(a, b *v1.SeriesWithBlooms) *v1.SeriesWithBlooms { - if len(a.Series.Chunks) > len(b.Series.Chunks) { - return a - } - return b - }, - iter.NewPeekIter(mergedBlocks), - ) - return i.iter.Next() - } - - i.iter = iter.NewEmptyIter[*v1.SeriesWithBlooms]() - i.err = i.overlapping.Err() - return false -} - -// Next implements v1.Iterator. -func (i *blockLoadingIter) Next() bool { - i.init() - return i.iter.Next() || i.loadNext() -} - -// Close implements v1.CloseableIterator. -func (i *blockLoadingIter) Close() error { - var err multierror.MultiError - for k := range i.loaded { - err.Add(k.Close()) - } - return err.Err() -} - -// Reset implements v1.ResettableIterator. -// TODO(chaudum) Cache already fetched blocks to to avoid the overhead of -// creating the reader. -func (i *blockLoadingIter) Reset() error { - if !i.initialized { - return nil - } - // close loaded queriers - err := i.Close() - i.initialized = false - clear(i.loaded) - return err -} - -func (i *blockLoadingIter) Filter(filter func(*bloomshipper.CloseableBlockQuerier) bool) { - if i.initialized { - panic("iterator already initialized") - } - i.filter = filter -} - -func overlappingBlocksIter(inputs []bloomshipper.BlockRef) iter.Iterator[[]bloomshipper.BlockRef] { - // can we assume sorted blocks? - peekIter := iter.NewPeekIter(iter.NewSliceIter(inputs)) - - return iter.NewDedupingIter[bloomshipper.BlockRef, []bloomshipper.BlockRef]( - func(a bloomshipper.BlockRef, b []bloomshipper.BlockRef) bool { - minFp := b[0].Bounds.Min - maxFp := slices.MaxFunc(b, func(a, b bloomshipper.BlockRef) int { return int(a.Bounds.Max - b.Bounds.Max) }).Bounds.Max - return a.Bounds.Overlaps(v1.NewBounds(minFp, maxFp)) - }, - func(a bloomshipper.BlockRef) []bloomshipper.BlockRef { - return []bloomshipper.BlockRef{a} - }, - func(a bloomshipper.BlockRef, b []bloomshipper.BlockRef) []bloomshipper.BlockRef { - return append(b, a) - }, - peekIter, - ) -} diff --git a/pkg/bloomcompactor/batch_test.go b/pkg/bloomcompactor/batch_test.go deleted file mode 100644 index 09d595459b509..0000000000000 --- a/pkg/bloomcompactor/batch_test.go +++ /dev/null @@ -1,210 +0,0 @@ -package bloomcompactor - -import ( - "context" - "errors" - "testing" - - "github.com/stretchr/testify/require" - - v2 "github.com/grafana/loki/v3/pkg/iter/v2" - "github.com/grafana/loki/v3/pkg/storage/stores/shipper/bloomshipper" -) - -func TestBatchedLoader(t *testing.T) { - t.Parallel() - - errMapper := func(i int) (int, error) { - return 0, errors.New("bzzt") - } - successMapper := func(i int) (int, error) { - return i, nil - } - - expired, cancel := context.WithCancel(context.Background()) - cancel() - - for _, tc := range []struct { - desc string - ctx context.Context - batchSize int - mapper func(int) (int, error) - err bool - inputs [][]int - exp []int - }{ - { - desc: "OneBatch", - ctx: context.Background(), - batchSize: 2, - mapper: successMapper, - err: false, - inputs: [][]int{{0, 1}}, - exp: []int{0, 1}, - }, - { - desc: "ZeroBatchSizeStillWorks", - ctx: context.Background(), - batchSize: 0, - mapper: successMapper, - err: false, - inputs: [][]int{{0, 1}}, - exp: []int{0, 1}, - }, - { - desc: "OneBatchLessThanFull", - ctx: context.Background(), - batchSize: 2, - mapper: successMapper, - err: false, - inputs: [][]int{{0}}, - exp: []int{0}, - }, - { - desc: "TwoBatches", - ctx: context.Background(), - batchSize: 2, - mapper: successMapper, - err: false, - inputs: [][]int{{0, 1, 2, 3}}, - exp: []int{0, 1, 2, 3}, - }, - { - desc: "MultipleBatchesMultipleLoaders", - ctx: context.Background(), - batchSize: 2, - mapper: successMapper, - err: false, - inputs: [][]int{{0, 1}, {2}, {3, 4, 5}}, - exp: []int{0, 1, 2, 3, 4, 5}, - }, - { - desc: "HandlesEmptyInputs", - ctx: context.Background(), - batchSize: 2, - mapper: successMapper, - err: false, - inputs: [][]int{{0, 1, 2, 3}, nil, {4}}, - exp: []int{0, 1, 2, 3, 4}, - }, - { - desc: "Timeout", - ctx: expired, - batchSize: 2, - mapper: successMapper, - err: true, - inputs: [][]int{{0}}, - }, - { - desc: "MappingFailure", - ctx: context.Background(), - batchSize: 2, - mapper: errMapper, - err: true, - inputs: [][]int{{0}}, - }, - } { - tc := tc - t.Run(tc.desc, func(t *testing.T) { - fetchers := make([]Fetcher[int, int], 0, len(tc.inputs)) - for range tc.inputs { - fetchers = append( - fetchers, - FetchFunc[int, int](func(ctx context.Context, xs []int) ([]int, error) { - if ctx.Err() != nil { - return nil, ctx.Err() - } - return xs, nil - }), - ) - } - - loader := newBatchedLoader[int, int, int]( - tc.ctx, - fetchers, - tc.inputs, - tc.mapper, - tc.batchSize, - ) - - got, err := v2.Collect[int](loader) - if tc.err { - require.Error(t, err) - return - } - require.NoError(t, err) - require.Equal(t, tc.exp, got) - - }) - } -} - -func TestOverlappingBlocksIter(t *testing.T) { - t.Parallel() - for _, tc := range []struct { - desc string - inp []bloomshipper.BlockRef - exp int // expected groups - }{ - { - desc: "Empty", - inp: []bloomshipper.BlockRef{}, - exp: 0, - }, - { - desc: "NonOverlapping", - inp: []bloomshipper.BlockRef{ - genBlockRef(0x0000, 0x00ff), - genBlockRef(0x0100, 0x01ff), - genBlockRef(0x0200, 0x02ff), - }, - exp: 3, - }, - { - desc: "AllOverlapping", - inp: []bloomshipper.BlockRef{ - genBlockRef(0x0000, 0x02ff), // |-----------| - genBlockRef(0x0100, 0x01ff), // |---| - genBlockRef(0x0200, 0x02ff), // |---| - }, - exp: 1, - }, - { - desc: "PartialOverlapping", - inp: []bloomshipper.BlockRef{ - genBlockRef(0x0000, 0x01ff), // group 1 |-------| - genBlockRef(0x0100, 0x02ff), // group 1 |-------| - genBlockRef(0x0200, 0x03ff), // group 1 |-------| - genBlockRef(0x0200, 0x02ff), // group 1 |---| - }, - exp: 1, - }, - { - desc: "PartialOverlapping", - inp: []bloomshipper.BlockRef{ - genBlockRef(0x0000, 0x01ff), // group 1 |-------| - genBlockRef(0x0100, 0x02ff), // group 1 |-------| - genBlockRef(0x0100, 0x01ff), // group 1 |---| - genBlockRef(0x0300, 0x03ff), // group 2 |---| - genBlockRef(0x0310, 0x03ff), // group 2 |-| - }, - exp: 2, - }, - } { - tc := tc - t.Run(tc.desc, func(t *testing.T) { - it := overlappingBlocksIter(tc.inp) - var overlapping [][]bloomshipper.BlockRef - var i int - for it.Next() && it.Err() == nil { - require.NotNil(t, it.At()) - overlapping = append(overlapping, it.At()) - for _, r := range it.At() { - t.Log(i, r) - } - i++ - } - require.Equal(t, tc.exp, len(overlapping)) - }) - } -} diff --git a/pkg/bloomcompactor/bloomcompactor.go b/pkg/bloomcompactor/bloomcompactor.go deleted file mode 100644 index 6f07389a0bb4a..0000000000000 --- a/pkg/bloomcompactor/bloomcompactor.go +++ /dev/null @@ -1,510 +0,0 @@ -package bloomcompactor - -import ( - "context" - "sync" - "time" - - "github.com/go-kit/log" - "github.com/go-kit/log/level" - "github.com/grafana/dskit/backoff" - "github.com/grafana/dskit/concurrency" - "github.com/grafana/dskit/multierror" - "github.com/grafana/dskit/ring" - "github.com/grafana/dskit/services" - "github.com/pkg/errors" - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/common/model" - - "github.com/grafana/loki/v3/pkg/bloomutils" - iter "github.com/grafana/loki/v3/pkg/iter/v2" - "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" - "github.com/grafana/loki/v3/pkg/storage/stores/shipper/bloomshipper" - utillog "github.com/grafana/loki/v3/pkg/util/log" - util_ring "github.com/grafana/loki/v3/pkg/util/ring" -) - -var ( - RingOp = ring.NewOp([]ring.InstanceState{ring.JOINING, ring.ACTIVE}, nil) -) - -/* -Bloom-compactor - -This is a standalone service that is responsible for compacting TSDB indexes into bloomfilters. -It creates and merges bloomfilters into an aggregated form, called bloom-blocks. -It maintains a list of references between bloom-blocks and TSDB indexes in files called meta.jsons. - -Bloom-compactor regularly runs to check for changes in meta.jsons and runs compaction only upon changes in TSDBs. -*/ -type Compactor struct { - services.Service - - cfg Config - schemaCfg config.SchemaConfig - logger log.Logger - limits Limits - - tsdbStore TSDBStore - // TODO(owen-d): ShardingStrategy - controller *SimpleBloomController - retentionManager *RetentionManager - - // temporary workaround until bloomStore has implemented read/write shipper interface - bloomStore bloomshipper.StoreBase - - sharding util_ring.TenantSharding - - metrics *Metrics -} - -func New( - cfg Config, - schemaCfg config.SchemaConfig, - storeCfg storage.Config, - clientMetrics storage.ClientMetrics, - fetcherProvider stores.ChunkFetcherProvider, - ring ring.ReadRing, - ringLifeCycler *ring.BasicLifecycler, - limits Limits, - store bloomshipper.Store, - logger log.Logger, - r prometheus.Registerer, -) (*Compactor, error) { - utillog.WarnExperimentalUse("Bloom Compactor", logger) - c := &Compactor{ - cfg: cfg, - schemaCfg: schemaCfg, - logger: logger, - sharding: util_ring.NewTenantShuffleSharding(ring, ringLifeCycler, limits.BloomCompactorShardSize), - limits: limits, - bloomStore: store, - metrics: NewMetrics(r, store.BloomMetrics()), - } - - tsdbStore, err := NewTSDBStores(schemaCfg, storeCfg, clientMetrics, logger) - if err != nil { - return nil, errors.Wrap(err, "failed to create TSDB store") - } - c.tsdbStore = tsdbStore - - chunkLoader := NewStoreChunkLoader( - fetcherProvider, - c.metrics, - ) - - c.controller = NewSimpleBloomController( - c.tsdbStore, - c.bloomStore, - chunkLoader, - c.limits, - c.metrics, - c.logger, - ) - - c.retentionManager = NewRetentionManager( - c.cfg.RetentionConfig, - c.limits, - c.bloomStore, - newFirstTokenRetentionSharding(ring, ringLifeCycler), - c.metrics, - c.logger, - ) - - c.Service = services.NewBasicService(c.starting, c.running, c.stopping) - return c, nil -} - -func (c *Compactor) starting(_ context.Context) (err error) { - c.metrics.compactorRunning.Set(1) - return err -} - -func (c *Compactor) stopping(_ error) error { - c.metrics.compactorRunning.Set(0) - return nil -} - -func (c *Compactor) running(ctx context.Context) error { - // run once at beginning - if err := c.runOne(ctx); err != nil { - return err - } - - ticker := time.NewTicker(c.cfg.CompactionInterval) - defer ticker.Stop() - for { - select { - case <-ctx.Done(): - err := ctx.Err() - level.Debug(c.logger).Log("msg", "compactor context done", "err", err) - return err - - case <-ticker.C: - if err := c.runOne(ctx); err != nil { - return err - } - } - } -} - -func runWithRetries( - ctx context.Context, - minBackoff, maxBackoff time.Duration, - maxRetries int, - f func(ctx context.Context) error, -) error { - var lastErr error - - retries := backoff.New(ctx, backoff.Config{ - MinBackoff: minBackoff, - MaxBackoff: maxBackoff, - MaxRetries: maxRetries, - }) - - for retries.Ongoing() { - lastErr = f(ctx) - if lastErr == nil { - return nil - } - - retries.Wait() - } - - return lastErr -} - -type tenantTableRange struct { - tenant string - table config.DayTable - ownershipRange v1.FingerprintBounds - - finished bool - queueTime, startTime, endTime time.Time -} - -func (c *Compactor) tenants(ctx context.Context, table config.DayTable) (*iter.SliceIter[string], error) { - tenants, err := c.tsdbStore.UsersForPeriod(ctx, table) - if err != nil { - return nil, errors.Wrap(err, "getting tenants") - } - - return iter.NewSliceIter(tenants), nil -} - -// ownsTenant returns the ownership range for the tenant, if the compactor owns the tenant, and an error. -func (c *Compactor) ownsTenant(tenant string) ([]v1.FingerprintBounds, bool, error) { - if !c.limits.BloomCompactorEnabled(tenant) { - return nil, false, nil - } - tenantRing, owned := c.sharding.OwnsTenant(tenant) - if !owned { - return nil, false, nil - } - - // TODO(owen-d): use .GetTokenRangesForInstance() - // when it's supported for non zone-aware rings - // instead of doing all this manually - - rs, err := tenantRing.GetAllHealthy(RingOp) - if err != nil { - return nil, false, errors.Wrap(err, "getting ring healthy instances") - } - - ranges, err := bloomutils.TokenRangesForInstance(c.cfg.Ring.InstanceID, rs.Instances) - if err != nil { - return nil, false, errors.Wrap(err, "getting token ranges for instance") - } - - keyspaces := bloomutils.KeyspacesFromTokenRanges(ranges) - return keyspaces, true, nil -} - -// runs a single round of compaction for all relevant tenants and tables -func (c *Compactor) runOne(ctx context.Context) error { - c.metrics.compactionsStarted.Inc() - start := time.Now() - level.Info(c.logger).Log("msg", "running bloom compaction", "workers", c.cfg.WorkerParallelism) - var workersErr, retentionErr error - var wg sync.WaitGroup - input := make(chan *tenantTableRange) - - // Launch retention (will return instantly if retention is disabled or not owned by this compactor) - wg.Add(1) - go func() { - retentionErr = c.retentionManager.Apply(ctx) - wg.Done() - }() - - tables := c.tables(time.Now()) - level.Debug(c.logger).Log("msg", "loaded tables", "tables", tables.TotalDays()) - - tracker, err := newCompactionTracker(tables.TotalDays()) - if err != nil { - return errors.Wrap(err, "creating compaction tracker") - } - - wg.Add(1) - go func() { - workersErr = c.runWorkers(ctx, input, tracker) - wg.Done() - }() - - err = c.loadWork(ctx, tables, input, tracker) - - wg.Wait() - duration := time.Since(start) - err = multierror.New(retentionErr, workersErr, err, ctx.Err()).Err() - - if err != nil { - level.Error(c.logger).Log("msg", "compaction iteration failed", "err", err, "duration", duration) - c.metrics.compactionCompleted.WithLabelValues(statusFailure).Inc() - c.metrics.compactionTime.WithLabelValues(statusFailure).Observe(time.Since(start).Seconds()) - return err - } - - c.metrics.compactionCompleted.WithLabelValues(statusSuccess).Inc() - c.metrics.compactionTime.WithLabelValues(statusSuccess).Observe(time.Since(start).Seconds()) - level.Info(c.logger).Log("msg", "compaction iteration completed", "duration", duration) - return nil -} - -func (c *Compactor) tables(ts time.Time) *dayRangeIterator { - // adjust the minimum by one to make it inclusive, which is more intuitive - // for a configuration variable - adjustedMin := min(c.cfg.MinTableOffset - 1) - minCompactionDelta := time.Duration(adjustedMin) * config.ObjectStorageIndexRequiredPeriod - maxCompactionDelta := time.Duration(c.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(c.logger).Log("msg", "loaded tables for compaction", "from", fromDay, "through", throughDay) - return newDayRangeIterator(fromDay, throughDay, c.schemaCfg) -} - -func (c *Compactor) loadWork( - ctx context.Context, - tables *dayRangeIterator, - ch chan<- *tenantTableRange, - tracker *compactionTracker, -) error { - - for tables.Next() && tables.Err() == nil && ctx.Err() == nil { - table := tables.At() - - level.Debug(c.logger).Log("msg", "loading work for table", "table", table) - - tenants, err := c.tenants(ctx, table) - if err != nil { - return errors.Wrap(err, "getting tenants") - } - nTenants := tenants.Remaining() - - type ownedTenant struct { - tenant string - ownershipRanges []v1.FingerprintBounds - } - - // build owned tenants separately and load them all prior to compaction in order to - // accurately report progress - var ownedTenants []ownedTenant - - for tenants.Next() && tenants.Err() == nil && ctx.Err() == nil { - c.metrics.tenantsDiscovered.Inc() - tenant := tenants.At() - ownershipRanges, owns, err := c.ownsTenant(tenant) - if err != nil { - return errors.Wrap(err, "checking tenant ownership") - } - if !owns { - level.Debug(c.logger).Log("msg", "skipping tenant", "tenant", tenant, "table", table) - c.metrics.tenantsSkipped.Inc() - continue - } - c.metrics.tenantsOwned.Inc() - ownedTenants = append(ownedTenants, ownedTenant{tenant, ownershipRanges}) - } - if err := tenants.Err(); err != nil { - level.Error(c.logger).Log("msg", "error iterating tenants", "err", err) - return errors.Wrap(err, "iterating tenants") - } - - level.Debug(c.logger).Log("msg", "loaded tenants", "table", table, "tenants", nTenants, "owned_tenants", len(ownedTenants)) - tracker.registerTable(table.DayTime, len(ownedTenants)) - - for _, t := range ownedTenants { - // loop over ranges, registering them in the tracker; - // we add them to the tracker before queueing them - // so progress reporting is aware of all tenant/table - // pairs prior to execution. Otherwise, progress could - // decrease over time as more work is discovered. - var inputs []*tenantTableRange - for _, ownershipRange := range t.ownershipRanges { - tt := tenantTableRange{ - tenant: t.tenant, - table: table, - ownershipRange: ownershipRange, - } - tracker.update(tt.tenant, tt.table.DayTime, tt.ownershipRange, tt.ownershipRange.Min) - inputs = append(inputs, &tt) - } - - // iterate the inputs, queueing them - for _, tt := range inputs { - level.Debug(c.logger).Log("msg", "enqueueing work for tenant", "tenant", tt.tenant, "table", table, "ownership", tt.ownershipRange.String()) - tt.queueTime = time.Now() // accurrately report queue time - select { - case ch <- tt: - case <-ctx.Done(): - return ctx.Err() - } - } - } - - if err := tenants.Err(); err != nil { - level.Error(c.logger).Log("msg", "error iterating tenants", "err", err) - return errors.Wrap(err, "iterating tenants") - } - - } - - if err := tables.Err(); err != nil { - level.Error(c.logger).Log("msg", "error iterating tables", "err", err) - return errors.Wrap(err, "iterating tables") - } - - close(ch) - return ctx.Err() -} - -func (c *Compactor) runWorkers( - ctx context.Context, - input <-chan *tenantTableRange, - tracker *compactionTracker, -) error { - - // TODO(owen-d): refactor for cleanliness - reporterCtx, cancel := context.WithCancel(ctx) - var wg sync.WaitGroup - wg.Add(1) - go func() { - ticker := time.NewTicker(30 * time.Second) - for { - select { - case <-ticker.C: - c.metrics.progress.Set(tracker.progress()) - case <-reporterCtx.Done(): - c.metrics.progress.Set(tracker.progress()) - wg.Done() - ticker.Stop() - return - } - } - }() - - err := concurrency.ForEachJob(ctx, c.cfg.WorkerParallelism, c.cfg.WorkerParallelism, func(ctx context.Context, idx int) error { - - for { - select { - case <-ctx.Done(): - return ctx.Err() - - case tt, ok := <-input: - if !ok { - return nil - } - c.metrics.tenantsStarted.Inc() - err := c.compactTenantTable(ctx, tt, tracker) - duration := tt.endTime.Sub(tt.startTime) - c.metrics.timePerTenant.WithLabelValues(tt.tenant).Add(duration.Seconds()) - progress := tracker.progress() - - if err != nil { - c.metrics.tenantTableRanges.WithLabelValues(statusFailure).Inc() - return errors.Wrapf( - err, - "compacting tenant table (%s) for tenant (%s) with ownership (%s)", - tt.table, - tt.tenant, - tt.ownershipRange, - ) - } - level.Debug(c.logger).Log( - "msg", "finished compacting tenant table", - "tenant", tt.tenant, - "table", tt.table, - "ownership", tt.ownershipRange.String(), - "duration", duration, - "current_progress", progress, - ) - c.metrics.tenantTableRanges.WithLabelValues(statusSuccess).Inc() - } - } - - }) - cancel() - wg.Wait() - - return err - -} - -func (c *Compactor) compactTenantTable(ctx context.Context, tt *tenantTableRange, tracker *compactionTracker) error { - level.Info(c.logger).Log("msg", "compacting", "org_id", tt.tenant, "table", tt.table, "ownership", tt.ownershipRange.String()) - tt.startTime = time.Now() - err := c.controller.compactTenant(ctx, tt.table, tt.tenant, tt.ownershipRange, tracker) - tt.finished = true - tt.endTime = time.Now() - tracker.update(tt.tenant, tt.table.DayTime, tt.ownershipRange, tt.ownershipRange.Max) - level.Info(c.logger).Log("msg", "finished compacting", "org_id", tt.tenant, "table", tt.table, "ownership", tt.ownershipRange.String(), "err", err) - return err -} - -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 = errors.Wrapf(err, "getting schema for time (%s)", r.cur) - 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/bloomcompactor/bloomcompactor_test.go b/pkg/bloomcompactor/bloomcompactor_test.go deleted file mode 100644 index 2a82782b722c6..0000000000000 --- a/pkg/bloomcompactor/bloomcompactor_test.go +++ /dev/null @@ -1,284 +0,0 @@ -package bloomcompactor - -import ( - "context" - "flag" - "fmt" - "math" - "testing" - "time" - - "github.com/grafana/dskit/ring" - "github.com/grafana/dskit/services" - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/common/model" - "github.com/stretchr/testify/require" - - "github.com/grafana/loki/v3/pkg/bloomutils" - "github.com/grafana/loki/v3/pkg/chunkenc" - v1 "github.com/grafana/loki/v3/pkg/storage/bloom/v1" - "github.com/grafana/loki/v3/pkg/storage/config" - util_log "github.com/grafana/loki/v3/pkg/util/log" - lokiring "github.com/grafana/loki/v3/pkg/util/ring" - util_ring "github.com/grafana/loki/v3/pkg/util/ring" - "github.com/grafana/loki/v3/pkg/validation" -) - -func TestCompactor_ownsTenant(t *testing.T) { - for _, tc := range []struct { - name string - limits Limits - compactors int - - expectedCompactorsOwningTenant int - }{ - { - name: "no sharding with one instance", - limits: mockLimits{ - shardSize: 0, - }, - compactors: 1, - expectedCompactorsOwningTenant: 1, - }, - { - name: "no sharding with multiple instances", - limits: mockLimits{ - shardSize: 0, - }, - compactors: 10, - expectedCompactorsOwningTenant: 10, - }, - { - name: "sharding with one instance", - limits: mockLimits{ - shardSize: 5, - }, - compactors: 1, - expectedCompactorsOwningTenant: 1, - }, - { - name: "sharding with multiple instances", - limits: mockLimits{ - shardSize: 5, - }, - compactors: 10, - expectedCompactorsOwningTenant: 5, - }, - } { - t.Run(tc.name, func(t *testing.T) { - var ringManagers []*lokiring.RingManager - var compactors []*Compactor - for i := 0; i < tc.compactors; i++ { - var cfg Config - cfg.RegisterFlags(flag.NewFlagSet("ring", flag.PanicOnError)) - cfg.Ring.KVStore.Store = "inmemory" - cfg.Ring.InstanceID = fmt.Sprintf("bloom-compactor-%d", i) - cfg.Ring.InstanceAddr = fmt.Sprintf("localhost-%d", i) - - ringManager, err := lokiring.NewRingManager("bloom-compactor", lokiring.ServerMode, cfg.Ring, 1, cfg.Ring.NumTokens, util_log.Logger, prometheus.NewRegistry()) - require.NoError(t, err) - require.NoError(t, ringManager.StartAsync(context.Background())) - - shuffleSharding := util_ring.NewTenantShuffleSharding(ringManager.Ring, ringManager.RingLifecycler, tc.limits.BloomCompactorShardSize) - - compactor := &Compactor{ - cfg: cfg, - sharding: shuffleSharding, - limits: tc.limits, - } - - ringManagers = append(ringManagers, ringManager) - compactors = append(compactors, compactor) - } - defer func() { - // Stop all rings and wait for them to stop. - for _, ringManager := range ringManagers { - ringManager.StopAsync() - require.Eventually(t, func() bool { - return ringManager.State() == services.Terminated - }, 1*time.Minute, 100*time.Millisecond) - } - }() - - // Wait for all rings to see each other. - for _, ringManager := range ringManagers { - require.Eventually(t, func() bool { - running := ringManager.State() == services.Running - discovered := ringManager.Ring.InstancesCount() == tc.compactors - return running && discovered - }, 1*time.Minute, 100*time.Millisecond) - } - - var compactorOwnsTenant int - var compactorOwnershipRange []v1.FingerprintBounds - for _, compactor := range compactors { - ownershipRange, ownsTenant, err := compactor.ownsTenant("tenant") - require.NoError(t, err) - if ownsTenant { - compactorOwnsTenant++ - compactorOwnershipRange = append(compactorOwnershipRange, ownershipRange...) - } - } - require.Equal(t, tc.expectedCompactorsOwningTenant, compactorOwnsTenant) - - coveredKeySpace := v1.NewBounds(math.MaxUint64, 0) - for i, boundsA := range compactorOwnershipRange { - for j, boundsB := range compactorOwnershipRange { - if i == j { - continue - } - // Assert that the fingerprint key-space is not overlapping - require.False(t, boundsA.Overlaps(boundsB)) - } - - if boundsA.Min < coveredKeySpace.Min { - coveredKeySpace.Min = boundsA.Min - } - if boundsA.Max > coveredKeySpace.Max { - coveredKeySpace.Max = boundsA.Max - } - - } - // Assert that the fingerprint key-space is complete - require.True(t, coveredKeySpace.Equal(v1.NewBounds(0, math.MaxUint64))) - }) - } -} - -type mockLimits struct { - shardSize int -} - -func (m mockLimits) RetentionPeriod(_ string) time.Duration { - panic("implement me") -} - -func (m mockLimits) StreamRetention(_ string) []validation.StreamRetention { - panic("implement me") -} - -func (m mockLimits) AllByUserID() map[string]*validation.Limits { - panic("implement me") -} - -func (m mockLimits) DefaultLimits() *validation.Limits { - panic("implement me") -} - -func (m mockLimits) VolumeMaxSeries(_ string) int { - panic("implement me") -} - -func (m mockLimits) BloomCompactorShardSize(_ string) int { - return m.shardSize -} - -func (m mockLimits) BloomCompactorEnabled(_ string) bool { - return true -} - -func (m mockLimits) BloomNGramLength(_ string) int { - panic("implement me") -} - -func (m mockLimits) BloomNGramSkip(_ string) int { - panic("implement me") -} - -func (m mockLimits) BloomFalsePositiveRate(_ string) float64 { - panic("implement me") -} - -func (m mockLimits) BloomBlockEncoding(_ string) string { - return chunkenc.EncNone.String() -} - -func (m mockLimits) BloomCompactorMaxBlockSize(_ string) int { - panic("implement me") -} - -func (m mockLimits) BloomCompactorMaxBloomSize(_ string) int { - panic("implement me") -} - -func TestTokenRangesForInstance(t *testing.T) { - desc := func(id int, tokens ...uint32) ring.InstanceDesc { - return ring.InstanceDesc{Id: fmt.Sprintf("%d", id), Tokens: tokens} - } - - tests := map[string]struct { - input []ring.InstanceDesc - exp map[string]ring.TokenRanges - err bool - }{ - "no nodes": { - input: []ring.InstanceDesc{}, - exp: map[string]ring.TokenRanges{ - "0": {0, math.MaxUint32}, // have to put one in here to trigger test - }, - err: true, - }, - "one node": { - input: []ring.InstanceDesc{ - desc(0, 0, 100), - }, - exp: map[string]ring.TokenRanges{ - "0": {0, math.MaxUint32}, - }, - }, - "two nodes": { - input: []ring.InstanceDesc{ - desc(0, 25, 75), - desc(1, 10, 50, 100), - }, - exp: map[string]ring.TokenRanges{ - "0": {10, 24, 50, 74}, - "1": {0, 9, 25, 49, 75, math.MaxUint32}, - }, - }, - "consecutive tokens": { - input: []ring.InstanceDesc{ - desc(0, 99), - desc(1, 100), - }, - exp: map[string]ring.TokenRanges{ - "0": {0, 98, 100, math.MaxUint32}, - "1": {99, 99}, - }, - }, - "extremes": { - input: []ring.InstanceDesc{ - desc(0, 0), - desc(1, math.MaxUint32), - }, - exp: map[string]ring.TokenRanges{ - "0": {math.MaxUint32, math.MaxUint32}, - "1": {0, math.MaxUint32 - 1}, - }, - }, - } - - for desc, test := range tests { - t.Run(desc, func(t *testing.T) { - for id := range test.exp { - ranges, err := bloomutils.TokenRangesForInstance(id, test.input) - if test.err { - require.Error(t, err) - continue - } - require.NoError(t, err) - require.Equal(t, test.exp[id], ranges) - } - }) - } -} - -func parseDayTime(s string) config.DayTime { - t, err := time.Parse("2006-01-02", s) - if err != nil { - panic(err) - } - return config.DayTime{ - Time: model.TimeFromUnix(t.Unix()), - } -} diff --git a/pkg/bloomcompactor/config.go b/pkg/bloomcompactor/config.go deleted file mode 100644 index 82daac0eac39f..0000000000000 --- a/pkg/bloomcompactor/config.go +++ /dev/null @@ -1,98 +0,0 @@ -package bloomcompactor - -import ( - "flag" - "fmt" - "time" - - "github.com/pkg/errors" - - "github.com/grafana/loki/v3/pkg/util/ring" -) - -const ( - ringReplicationFactor = 1 -) - -// Config configures the bloom-compactor component. -type Config struct { - // Ring configures the ring store used to save and retrieve the different Bloom-Compactor instances. - // In case it isn't explicitly set, it follows the same behavior of the other rings (ex: using the common configuration - // section and the ingester configuration by default). - Ring ring.RingConfig `yaml:"ring,omitempty" doc:"description=Defines the ring to be used by the bloom-compactor servers. In case this isn't configured, this block supports inheriting configuration from the common ring section."` - // Enabled configures whether bloom-compactors should be used to compact index values into bloomfilters - Enabled bool `yaml:"enabled"` - CompactionInterval time.Duration `yaml:"compaction_interval"` - MinTableOffset int `yaml:"min_table_offset"` - MaxTableOffset int `yaml:"max_table_offset"` - WorkerParallelism int `yaml:"worker_parallelism"` - RetryMinBackoff time.Duration `yaml:"compaction_retries_min_backoff"` - RetryMaxBackoff time.Duration `yaml:"compaction_retries_max_backoff"` - CompactionRetries int `yaml:"compaction_retries"` - - MaxCompactionParallelism int `yaml:"max_compaction_parallelism"` - - RetentionConfig RetentionConfig `yaml:"retention"` -} - -// RegisterFlags registers flags for the Bloom-Compactor configuration. -func (cfg *Config) RegisterFlags(f *flag.FlagSet) { - f.BoolVar(&cfg.Enabled, "bloom-compactor.enabled", false, "Flag to enable or disable the usage of the bloom-compactor component.") - f.DurationVar(&cfg.CompactionInterval, "bloom-compactor.compaction-interval", 10*time.Minute, "Interval at which to re-run the compaction operation.") - f.IntVar(&cfg.WorkerParallelism, "bloom-compactor.worker-parallelism", 1, "Number of workers to run in parallel for compaction.") - // TODO(owen-d): This is a confusing name. Rename it to `min_table_offset` - f.IntVar(&cfg.MinTableOffset, "bloom-compactor.min-table-offset", 1, "Newest day-table offset (from today, inclusive) to compact. 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, "bloom-compactor.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.DurationVar(&cfg.RetryMinBackoff, "bloom-compactor.compaction-retries-min-backoff", 10*time.Second, "Minimum backoff time between retries.") - f.DurationVar(&cfg.RetryMaxBackoff, "bloom-compactor.compaction-retries-max-backoff", time.Minute, "Maximum backoff time between retries.") - f.IntVar(&cfg.CompactionRetries, "bloom-compactor.compaction-retries", 3, "Number of retries to perform when compaction fails.") - f.IntVar(&cfg.MaxCompactionParallelism, "bloom-compactor.max-compaction-parallelism", 1, "Maximum number of tables to compact in parallel. While increasing this value, please make sure compactor has enough disk space allocated to be able to store and compact as many tables.") - cfg.RetentionConfig.RegisterFlags(f) - - // Ring - skipFlags := []string{ - "bloom-compactor.ring.num-tokens", - "bloom-compactor.ring.replication-factor", - } - cfg.Ring.RegisterFlagsWithPrefix("bloom-compactor.", "collectors/", f, skipFlags...) - // Overrides - f.IntVar(&cfg.Ring.NumTokens, "bloom-compactor.ring.num-tokens", 10, "Number of tokens to use in the ring per compactor. Higher number of tokens will result in more and smaller files (metas and blocks.)") - // Ignored - f.IntVar(&cfg.Ring.ReplicationFactor, "bloom-compactor.ring.replication-factor", ringReplicationFactor, fmt.Sprintf("IGNORED: Replication factor is fixed to %d", ringReplicationFactor)) -} - -func (cfg *Config) Validate() error { - if !cfg.Enabled { - return nil - } - - if err := cfg.RetentionConfig.Validate(); err != nil { - return err - } - - 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) - } - if cfg.Ring.ReplicationFactor != ringReplicationFactor { - return errors.New("Replication factor must not be changed as it will not take effect") - } - return nil -} - -type Limits interface { - RetentionLimits - BloomCompactorShardSize(tenantID string) int - BloomCompactorEnabled(tenantID string) bool - BloomNGramLength(tenantID string) int - BloomNGramSkip(tenantID string) int - BloomFalsePositiveRate(tenantID string) float64 - BloomCompactorMaxBlockSize(tenantID string) int - BloomCompactorMaxBloomSize(tenantID string) int - BloomBlockEncoding(tenantID string) string -} diff --git a/pkg/bloomcompactor/controller.go b/pkg/bloomcompactor/controller.go deleted file mode 100644 index d53ba80b0123b..0000000000000 --- a/pkg/bloomcompactor/controller.go +++ /dev/null @@ -1,779 +0,0 @@ -package bloomcompactor - -import ( - "context" - "fmt" - "math" - "os" - "sort" - "sync" - - "github.com/go-kit/log" - "github.com/go-kit/log/level" - "github.com/pkg/errors" - "github.com/prometheus/common/model" - - "github.com/grafana/loki/v3/pkg/chunkenc" - iter "github.com/grafana/loki/v3/pkg/iter/v2" - 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" -) - -type SimpleBloomController struct { - tsdbStore TSDBStore - bloomStore bloomshipper.StoreBase - chunkLoader ChunkLoader - metrics *Metrics - limits Limits - - logger log.Logger -} - -func NewSimpleBloomController( - tsdbStore TSDBStore, - blockStore bloomshipper.StoreBase, - chunkLoader ChunkLoader, - limits Limits, - metrics *Metrics, - logger log.Logger, -) *SimpleBloomController { - return &SimpleBloomController{ - tsdbStore: tsdbStore, - bloomStore: blockStore, - chunkLoader: chunkLoader, - metrics: metrics, - limits: limits, - logger: logger, - } -} - -func (s *SimpleBloomController) writerReaderFunc() (v1.BlockWriter, v1.BlockReader) { - dir, err := os.MkdirTemp("", "bloom-block-") - if err != nil { - panic(err) - } - return v1.NewDirectoryBlockWriter(dir), v1.NewDirectoryBlockReader(dir) -} - -/* -Compaction works as follows, split across many functions for clarity: - 1. Fetch all meta.jsons for the given tenant and table which overlap the ownership range of this compactor. - 2. Load current TSDBs for this tenant/table. - 3. For each live TSDB (there should be only 1, but this works with multiple), find any gaps - (fingerprint ranges) which are not up date, determined by checking other meta.jsons and comparing - the tsdbs they were generated from + their ownership ranges. - 4. Build new bloom blocks for each gap, using the series and chunks from the TSDBs and any existing - blocks which overlap the gaps to accelerate bloom generation. - 5. Write the new blocks and metas to the store. - 6. Determine if any meta.jsons overlap the ownership range but are outdated, and remove them and - their associated blocks if so. -*/ -func (s *SimpleBloomController) compactTenant( - ctx context.Context, - table config.DayTable, - tenant string, - ownershipRange v1.FingerprintBounds, - tracker *compactionTracker, -) error { - logger := log.With(s.logger, "org_id", tenant, "table", table.Addr(), "ownership", ownershipRange.String()) - - client, err := s.bloomStore.Client(table.ModelTime()) - if err != nil { - level.Error(logger).Log("msg", "failed to get client", "err", err) - return errors.Wrap(err, "failed to get client") - } - - // Fetch source metas to be used in both compaction and cleanup of out-of-date metas+blooms - metas, err := s.bloomStore.FetchMetas( - ctx, - bloomshipper.MetaSearchParams{ - TenantID: tenant, - Interval: bloomshipper.NewInterval(table.Bounds()), - Keyspace: ownershipRange, - }, - ) - if err != nil { - level.Error(logger).Log("msg", "failed to get metas", "err", err) - return errors.Wrap(err, "failed to get metas") - } - - level.Debug(logger).Log("msg", "found relevant metas", "metas", len(metas)) - - // fetch all metas overlapping our ownership range so we can safely - // check which metas can be deleted even if they only partially overlap out ownership range - superset, err := s.fetchSuperSet(ctx, tenant, table, ownershipRange, metas, logger) - if err != nil { - return errors.Wrap(err, "failed to fetch superset") - } - - // build compaction plans - work, err := s.findOutdatedGaps(ctx, tenant, table, ownershipRange, metas, logger) - if err != nil { - return errors.Wrap(err, "failed to find outdated gaps") - } - - // build new blocks - built, err := s.buildGaps(ctx, tenant, table, ownershipRange, client, work, tracker, logger) - if err != nil { - return errors.Wrap(err, "failed to build gaps") - } - - // combine built and superset metas - // in preparation for removing outdated ones - combined := append(superset, built...) - - outdated, err := outdatedMetas(combined) - if err != nil { - return errors.Wrap(err, "failed to find outdated metas") - } - level.Debug(logger).Log("msg", "found outdated metas", "outdated", len(outdated)) - - var ( - deletedMetas int - deletedBlocks int - ) - defer func() { - s.metrics.metasDeleted.Add(float64(deletedMetas)) - s.metrics.blocksDeleted.Add(float64(deletedBlocks)) - }() - - for _, meta := range outdated { - for _, block := range meta.Blocks { - err := client.DeleteBlocks(ctx, []bloomshipper.BlockRef{block}) - if err != nil { - if client.IsObjectNotFoundErr(err) { - level.Debug(logger).Log("msg", "block not found while attempting delete, continuing", "block", block.String()) - } else { - level.Error(logger).Log("msg", "failed to delete block", "err", err, "block", block.String()) - return errors.Wrap(err, "failed to delete block") - } - } - deletedBlocks++ - level.Debug(logger).Log("msg", "removed outdated block", "block", block.String()) - } - - err = client.DeleteMetas(ctx, []bloomshipper.MetaRef{meta.MetaRef}) - if err != nil { - if client.IsObjectNotFoundErr(err) { - level.Debug(logger).Log("msg", "meta not found while attempting delete, continuing", "meta", meta.MetaRef.String()) - } else { - level.Error(logger).Log("msg", "failed to delete meta", "err", err, "meta", meta.MetaRef.String()) - return errors.Wrap(err, "failed to delete meta") - } - } - deletedMetas++ - level.Debug(logger).Log("msg", "removed outdated meta", "meta", meta.MetaRef.String()) - } - - level.Debug(logger).Log("msg", "finished compaction") - return nil -} - -// fetchSuperSet fetches all metas which overlap the ownership range of the first set of metas we've resolved -func (s *SimpleBloomController) fetchSuperSet( - ctx context.Context, - tenant string, - table config.DayTable, - ownershipRange v1.FingerprintBounds, - metas []bloomshipper.Meta, - logger log.Logger, -) ([]bloomshipper.Meta, error) { - // in order to delete outdates metas which only partially fall within the ownership range, - // we need to fetcha all metas in the entire bound range of the first set of metas we've resolved - /* - For instance, we have the following ownership range and we resolve `meta1` in our first Fetch call - because it overlaps the ownership range, we'll need to fetch newer metas that may overlap it in order - to check if it safely can be deleted. This falls partially outside our specific ownership range, but - we can safely run multiple deletes by treating their removal as idempotent. - |-------------ownership range-----------------| - |-------meta1-------| - - we fetch this before possibly deleting meta1 |------| - */ - superset := ownershipRange - for _, meta := range metas { - union := superset.Union(meta.Bounds) - if len(union) > 1 { - level.Error(logger).Log("msg", "meta bounds union is not a single range", "union", union) - return nil, errors.New("meta bounds union is not a single range") - } - superset = union[0] - } - - within := superset.Within(ownershipRange) - level.Debug(logger).Log( - "msg", "looking for superset metas", - "superset", superset.String(), - "superset_within", within, - ) - - if within { - // we don't need to fetch any more metas - // NB(owen-d): here we copy metas into the output. This is slightly inefficient, but - // helps prevent mutability bugs by returning the same slice as the input. - results := make([]bloomshipper.Meta, len(metas)) - copy(results, metas) - return results, nil - } - - supersetMetas, err := s.bloomStore.FetchMetas( - ctx, - bloomshipper.MetaSearchParams{ - TenantID: tenant, - Interval: bloomshipper.NewInterval(table.Bounds()), - Keyspace: superset, - }, - ) - - if err != nil { - level.Error(logger).Log("msg", "failed to get meta superset range", "err", err, "superset", superset) - return nil, errors.Wrap(err, "failed to get meta supseret range") - } - - level.Debug(logger).Log( - "msg", "found superset metas", - "metas", len(metas), - "fresh_metas", len(supersetMetas), - "delta", len(supersetMetas)-len(metas), - ) - - return supersetMetas, nil -} - -func (s *SimpleBloomController) findOutdatedGaps( - ctx context.Context, - tenant string, - table config.DayTable, - ownershipRange v1.FingerprintBounds, - metas []bloomshipper.Meta, - logger log.Logger, -) ([]blockPlan, error) { - // Resolve TSDBs - tsdbs, err := s.tsdbStore.ResolveTSDBs(ctx, table, tenant) - if err != nil { - level.Error(logger).Log("msg", "failed to resolve tsdbs", "err", err) - return nil, errors.Wrap(err, "failed to resolve tsdbs") - } - - if len(tsdbs) == 0 { - return nil, nil - } - - // Determine which TSDBs have gaps in the ownership range and need to - // be processed. - tsdbsWithGaps, err := gapsBetweenTSDBsAndMetas(ownershipRange, tsdbs, metas) - if err != nil { - level.Error(logger).Log("msg", "failed to find gaps", "err", err) - return nil, errors.Wrap(err, "failed to find gaps") - } - - if len(tsdbsWithGaps) == 0 { - level.Debug(logger).Log("msg", "blooms exist for all tsdbs") - return nil, nil - } - - work, err := blockPlansForGaps(tsdbsWithGaps, metas) - if err != nil { - level.Error(logger).Log("msg", "failed to create plan", "err", err) - return nil, errors.Wrap(err, "failed to create plan") - } - - return work, nil -} - -func (s *SimpleBloomController) loadWorkForGap( - ctx context.Context, - table config.DayTable, - tenant string, - id tsdb.Identifier, - gap gapWithBlocks, -) (iter.Iterator[*v1.Series], iter.CloseResetIterator[*v1.SeriesWithBlooms], error) { - // load a series iterator for the gap - seriesItr, err := s.tsdbStore.LoadTSDB(ctx, table, tenant, id, gap.bounds) - if err != nil { - return nil, nil, errors.Wrap(err, "failed to load tsdb") - } - - // load a blocks iterator for the gap - fetcher, err := s.bloomStore.Fetcher(table.ModelTime()) - if err != nil { - return nil, nil, errors.Wrap(err, "failed to get fetcher") - } - - // NB(owen-d): we filter out nil blocks here to avoid panics in the bloom generator since the fetcher - // input->output length and indexing in its contract - // NB(chaudum): Do we want to fetch in strict mode and fail instead? - f := FetchFunc[bloomshipper.BlockRef, *bloomshipper.CloseableBlockQuerier](func(ctx context.Context, refs []bloomshipper.BlockRef) ([]*bloomshipper.CloseableBlockQuerier, error) { - blks, err := fetcher.FetchBlocks(ctx, refs, bloomshipper.WithFetchAsync(false), bloomshipper.WithIgnoreNotFound(true)) - if err != nil { - return nil, err - } - exists := make([]*bloomshipper.CloseableBlockQuerier, 0, len(blks)) - for _, blk := range blks { - if blk != nil { - exists = append(exists, blk) - } - } - return exists, nil - }) - blocksIter := newBlockLoadingIter(ctx, gap.blocks, f, 10) - - return seriesItr, blocksIter, nil -} - -func (s *SimpleBloomController) buildGaps( - ctx context.Context, - tenant string, - table config.DayTable, - ownershipRange v1.FingerprintBounds, - client bloomshipper.Client, - work []blockPlan, - tracker *compactionTracker, - logger log.Logger, -) ([]bloomshipper.Meta, error) { - // Generate Blooms - // Now that we have the gaps, we will generate a bloom block for each gap. - // We can accelerate this by using existing blocks which may already contain - // needed chunks in their blooms, for instance after a new TSDB version is generated - // but contains many of the same chunk references from the previous version. - // To do this, we'll need to take the metas we've already resolved and find blocks - // overlapping the ownership ranges we've identified as needing updates. - // With these in hand, we can download the old blocks and use them to - // accelerate bloom generation for the new blocks. - - blockEnc, err := chunkenc.ParseEncoding(s.limits.BloomBlockEncoding(tenant)) - if err != nil { - return nil, errors.Wrap(err, "failed to parse block encoding") - } - - var ( - blockCt int - tsdbCt = len(work) - nGramSize = uint64(s.limits.BloomNGramLength(tenant)) - nGramSkip = uint64(s.limits.BloomNGramSkip(tenant)) - maxBlockSize = uint64(s.limits.BloomCompactorMaxBlockSize(tenant)) - maxBloomSize = uint64(s.limits.BloomCompactorMaxBloomSize(tenant)) - blockOpts = v1.NewBlockOptions(blockEnc, nGramSize, nGramSkip, maxBlockSize, maxBloomSize) - created []bloomshipper.Meta - totalSeries int - bytesAdded int - ) - - for i, plan := range work { - - reporter := biasedReporter(func(fp model.Fingerprint) { - tracker.update(tenant, table.DayTime, ownershipRange, fp) - }, ownershipRange, i, len(work)) - - for i := range plan.gaps { - gap := plan.gaps[i] - logger := log.With(logger, "gap", gap.bounds.String(), "tsdb", plan.tsdb.Name()) - - meta := bloomshipper.Meta{ - MetaRef: bloomshipper.MetaRef{ - Ref: bloomshipper.Ref{ - TenantID: tenant, - TableName: table.Addr(), - Bounds: gap.bounds, - }, - }, - Sources: []tsdb.SingleTenantTSDBIdentifier{plan.tsdb}, - } - - // Fetch blocks that aren't up to date but are in the desired fingerprint range - // to try and accelerate bloom creation - level.Debug(logger).Log("msg", "loading series and blocks for gap", "blocks", len(gap.blocks)) - seriesItr, blocksIter, err := s.loadWorkForGap(ctx, table, tenant, plan.tsdb, gap) - if err != nil { - level.Error(logger).Log("msg", "failed to get series and blocks", "err", err) - return nil, errors.Wrap(err, "failed to get series and blocks") - } - - // TODO(owen-d): more elegant error handling than sync.OnceFunc - closeBlocksIter := sync.OnceFunc(func() { - if err := blocksIter.Close(); err != nil { - level.Error(logger).Log("msg", "failed to close blocks iterator", "err", err) - } - }) - defer closeBlocksIter() - - // Blocks are built consuming the series iterator. For observability, we wrap the series iterator - // with a counter iterator to count the number of times Next() is called on it. - // This is used to observe the number of series that are being processed. - seriesItrWithCounter := iter.NewCounterIter[*v1.Series](seriesItr) - - gen := NewSimpleBloomGenerator( - tenant, - blockOpts, - seriesItrWithCounter, - s.chunkLoader, - blocksIter, - s.writerReaderFunc, - reporter, - s.metrics, - logger, - ) - - level.Debug(logger).Log("msg", "generating blocks", "overlapping_blocks", len(gap.blocks)) - - newBlocks := gen.Generate(ctx) - if err != nil { - level.Error(logger).Log("msg", "failed to generate bloom", "err", err) - return nil, errors.Wrap(err, "failed to generate bloom") - } - - for newBlocks.Next() && newBlocks.Err() == nil { - blockCt++ - blk := newBlocks.At() - - built, err := bloomshipper.BlockFrom(tenant, table.Addr(), blk) - if err != nil { - level.Error(logger).Log("msg", "failed to build block", "err", err) - if err = blk.Reader().Cleanup(); err != nil { - level.Error(logger).Log("msg", "failed to cleanup block directory", "err", err) - } - return nil, errors.Wrap(err, "failed to build block") - } - - if err := client.PutBlock( - ctx, - built, - ); err != nil { - level.Error(logger).Log("msg", "failed to write block", "err", err) - if err = blk.Reader().Cleanup(); err != nil { - level.Error(logger).Log("msg", "failed to cleanup block directory", "err", err) - } - return nil, errors.Wrap(err, "failed to write block") - } - s.metrics.blocksCreated.Inc() - - if err := blk.Reader().Cleanup(); err != nil { - level.Error(logger).Log("msg", "failed to cleanup block directory", "err", err) - } - - totalGapKeyspace := (gap.bounds.Max - gap.bounds.Min) - progress := (built.Bounds.Max - gap.bounds.Min) - pct := float64(progress) / float64(totalGapKeyspace) * 100 - level.Debug(logger).Log( - "msg", "uploaded block", - "block", built.BlockRef.String(), - "progress_pct", fmt.Sprintf("%.2f", pct), - ) - - meta.Blocks = append(meta.Blocks, built.BlockRef) - } - - if err := newBlocks.Err(); err != nil { - level.Error(logger).Log("msg", "failed to generate bloom", "err", err) - return nil, errors.Wrap(err, "failed to generate bloom") - } - - closeBlocksIter() - bytesAdded += newBlocks.Bytes() - totalSeries += seriesItrWithCounter.Count() - s.metrics.blocksReused.Add(float64(len(gap.blocks))) - - // Write the new meta - // TODO(owen-d): put total size in log, total time in metrics+log - ref, err := bloomshipper.MetaRefFrom(tenant, table.Addr(), gap.bounds, meta.Sources, meta.Blocks) - if err != nil { - level.Error(logger).Log("msg", "failed to checksum meta", "err", err) - return nil, errors.Wrap(err, "failed to checksum meta") - } - meta.MetaRef = ref - - if err := client.PutMeta(ctx, meta); err != nil { - level.Error(logger).Log("msg", "failed to write meta", "err", err) - return nil, errors.Wrap(err, "failed to write meta") - } - - s.metrics.metasCreated.Inc() - level.Debug(logger).Log("msg", "uploaded meta", "meta", meta.MetaRef.String()) - created = append(created, meta) - } - } - - s.metrics.seriesPerCompaction.Observe(float64(totalSeries)) - s.metrics.bytesPerCompaction.Observe(float64(bytesAdded)) - level.Debug(logger).Log("msg", "finished bloom generation", "blocks", blockCt, "tsdbs", tsdbCt, "series", totalSeries, "bytes_added", bytesAdded) - return created, nil -} - -// Simple way to ensure increasing progress reporting -// and likely unused in practice because we don't expect to see more than 1 tsdb to compact. -// We assume each TSDB accounts for the same amount of work and only move progress forward -// depending on the current TSDB's index. For example, if we have 2 TSDBs and a fingerprint -// range from 0-100 (valid for both TSDBs), we'll limit reported progress for each TSDB to 50%. -func biasedReporter( - f func(model.Fingerprint), - ownershipRange v1.FingerprintBounds, - i, - total int, -) func(model.Fingerprint) { - return func(fp model.Fingerprint) { - clipped := min(max(fp, ownershipRange.Min), ownershipRange.Max) - delta := (clipped - ownershipRange.Min) / model.Fingerprint(total) - step := model.Fingerprint(ownershipRange.Range() / uint64(total)) - res := ownershipRange.Min + (step * model.Fingerprint(i)) + delta - f(res) - } -} - -func coversFullRange(bounds v1.FingerprintBounds, overlaps []v1.FingerprintBounds) bool { - // if there are no overlaps, the range is not covered - if len(overlaps) == 0 { - return false - } - - // keep track of bounds which need to be filled in order - // for the overlaps to cover the full range - missing := []v1.FingerprintBounds{bounds} - ignores := make(map[int]bool) - for _, overlap := range overlaps { - var i int - for { - if i >= len(missing) { - break - } - - if ignores[i] { - i++ - continue - } - - remaining := missing[i].Unless(overlap) - switch len(remaining) { - case 0: - // this range is covered, ignore it - ignores[i] = true - case 1: - // this range is partially covered, updated it - missing[i] = remaining[0] - case 2: - // this range has been partially covered in the middle, - // split it into two ranges and append - ignores[i] = true - missing = append(missing, remaining...) - } - i++ - } - - } - - return len(ignores) == len(missing) -} - -type gapWithBlocks struct { - bounds v1.FingerprintBounds - blocks []bloomshipper.BlockRef -} - -// blockPlan is a plan for all the work needed to build a meta.json -// It includes: -// - the tsdb (source of truth) which contains all the series+chunks -// we need to ensure are indexed in bloom blocks -// - a list of gaps that are out of date and need to be checked+built -// - within each gap, a list of block refs which overlap the gap are included -// so we can use them to accelerate bloom generation. They likely contain many -// of the same chunks we need to ensure are indexed, just from previous tsdb iterations. -// This is a performance optimization to avoid expensive re-reindexing -type blockPlan struct { - tsdb tsdb.SingleTenantTSDBIdentifier - gaps []gapWithBlocks -} - -// blockPlansForGaps groups tsdb gaps we wish to fill with overlapping but out of date blocks. -// This allows us to expedite bloom generation by using existing blocks to fill in the gaps -// since many will contain the same chunks. -func blockPlansForGaps(tsdbs []tsdbGaps, metas []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 := iter.NewPeekIter[bloomshipper.BlockRef]( - iter.NewSliceIter[bloomshipper.BlockRef]( - planGap.blocks, - ), - ) - // dedupe blocks which could be in multiple metas - itr := iter.NewDedupingIter[bloomshipper.BlockRef, bloomshipper.BlockRef]( - func(a, b bloomshipper.BlockRef) bool { - return a == b - }, - iter.Identity[bloomshipper.BlockRef], - func(a, _ bloomshipper.BlockRef) bloomshipper.BlockRef { - return a - }, - peekingBlocks, - ) - - deduped, err := iter.Collect[bloomshipper.BlockRef](itr) - if err != nil { - return nil, errors.Wrap(err, "failed to dedupe blocks") - } - planGap.blocks = deduped - - plan.gaps = append(plan.gaps, planGap) - } - - plans = append(plans, plan) - } - - return plans, nil -} - -// Used to signal the gaps that need to be populated for a tsdb -type tsdbGaps struct { - tsdb tsdb.SingleTenantTSDBIdentifier - gaps []v1.FingerprintBounds -} - -// tsdbsUpToDate returns if the metas are up to date with the tsdbs. This is determined by asserting -// that for each TSDB, there are metas covering the entire ownership range which were generated from that specific TSDB. -func gapsBetweenTSDBsAndMetas( - ownershipRange v1.FingerprintBounds, - tsdbs []tsdb.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 := findGaps(ownershipRange, relevantMetas) - if err != nil { - return nil, err - } - - if len(gaps) > 0 { - res = append(res, tsdbGaps{ - tsdb: db, - gaps: gaps, - }) - } - } - - return res, err -} - -func findGaps(ownershipRange v1.FingerprintBounds, metas []v1.FingerprintBounds) (gaps []v1.FingerprintBounds, err error) { - if len(metas) == 0 { - return []v1.FingerprintBounds{ownershipRange}, nil - } - - // turn the available metas into a list of non-overlapping metas - // for easier processing - var nonOverlapping []v1.FingerprintBounds - // First, we reduce the metas into a smaller set by combining overlaps. They must be sorted. - var cur *v1.FingerprintBounds - for i := 0; i < len(metas); i++ { - j := i + 1 - - // first iteration (i == 0), set the current meta - if cur == nil { - cur = &metas[i] - } - - if j >= len(metas) { - // We've reached the end of the list. Add the last meta to the non-overlapping set. - nonOverlapping = append(nonOverlapping, *cur) - break - } - - combined := cur.Union(metas[j]) - if len(combined) == 1 { - // There was an overlap between the two tested ranges. Combine them and keep going. - cur = &combined[0] - continue - } - - // There was no overlap between the two tested ranges. Add the first to the non-overlapping set. - // and keep the second for the next iteration. - nonOverlapping = append(nonOverlapping, combined[0]) - cur = &combined[1] - } - - // Now, detect gaps between the non-overlapping metas and the ownership range. - // The left bound of the ownership range will be adjusted as we go. - leftBound := ownershipRange.Min - for _, meta := range nonOverlapping { - - clippedMeta := meta.Intersection(ownershipRange) - // should never happen as long as we are only combining metas - // that intersect with the ownership range - if clippedMeta == nil { - return nil, fmt.Errorf("meta is not within ownership range: %v", meta) - } - - searchRange := ownershipRange.Slice(leftBound, clippedMeta.Max) - // update the left bound for the next iteration - // 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/bloomcompactor/controller_test.go b/pkg/bloomcompactor/controller_test.go deleted file mode 100644 index 5c6a506473476..0000000000000 --- a/pkg/bloomcompactor/controller_test.go +++ /dev/null @@ -1,584 +0,0 @@ -package bloomcompactor - -import ( - "fmt" - "math" - "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 Test_findGaps(t *testing.T) { - for _, tc := range []struct { - desc string - err bool - exp []v1.FingerprintBounds - ownershipRange v1.FingerprintBounds - metas []v1.FingerprintBounds - }{ - { - desc: "error nonoverlapping metas", - err: true, - exp: nil, - ownershipRange: v1.NewBounds(0, 10), - metas: []v1.FingerprintBounds{v1.NewBounds(11, 20)}, - }, - { - desc: "one meta with entire ownership range", - err: false, - exp: nil, - ownershipRange: v1.NewBounds(0, 10), - metas: []v1.FingerprintBounds{v1.NewBounds(0, 10)}, - }, - { - desc: "two non-overlapping metas with entire ownership range", - err: false, - exp: nil, - ownershipRange: v1.NewBounds(0, 10), - metas: []v1.FingerprintBounds{ - v1.NewBounds(0, 5), - v1.NewBounds(6, 10), - }, - }, - { - desc: "two overlapping metas with entire ownership range", - err: false, - exp: nil, - ownershipRange: v1.NewBounds(0, 10), - metas: []v1.FingerprintBounds{ - v1.NewBounds(0, 6), - v1.NewBounds(4, 10), - }, - }, - { - desc: "one meta with partial ownership range", - err: false, - exp: []v1.FingerprintBounds{ - v1.NewBounds(6, 10), - }, - ownershipRange: v1.NewBounds(0, 10), - metas: []v1.FingerprintBounds{ - v1.NewBounds(0, 5), - }, - }, - { - desc: "smaller subsequent meta with partial ownership range", - err: false, - exp: []v1.FingerprintBounds{ - v1.NewBounds(8, 10), - }, - ownershipRange: v1.NewBounds(0, 10), - metas: []v1.FingerprintBounds{ - v1.NewBounds(0, 7), - v1.NewBounds(3, 4), - }, - }, - { - desc: "hole in the middle", - err: false, - exp: []v1.FingerprintBounds{ - v1.NewBounds(4, 5), - }, - ownershipRange: v1.NewBounds(0, 10), - metas: []v1.FingerprintBounds{ - v1.NewBounds(0, 3), - v1.NewBounds(6, 10), - }, - }, - { - desc: "holes on either end", - err: false, - exp: []v1.FingerprintBounds{ - v1.NewBounds(0, 2), - v1.NewBounds(8, 10), - }, - ownershipRange: v1.NewBounds(0, 10), - metas: []v1.FingerprintBounds{ - v1.NewBounds(3, 5), - v1.NewBounds(6, 7), - }, - }, - { - 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 := findGaps(tc.ownershipRange, tc.metas) - if tc.err { - require.Error(t, err) - return - } - require.Equal(t, tc.exp, gaps) - }) - } -} - -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) - - }) - } -} - -func Test_coversFullRange(t *testing.T) { - for _, tc := range []struct { - desc string - src v1.FingerprintBounds - overlaps []v1.FingerprintBounds - exp bool - }{ - { - desc: "empty", - src: v1.NewBounds(0, 10), - overlaps: []v1.FingerprintBounds{}, - exp: false, - }, - { - desc: "single_full_range", - src: v1.NewBounds(0, 10), - overlaps: []v1.FingerprintBounds{ - v1.NewBounds(0, 10), - }, - exp: true, - }, - { - desc: "single_partial_range", - src: v1.NewBounds(0, 10), - overlaps: []v1.FingerprintBounds{ - v1.NewBounds(0, 5), - }, - exp: false, - }, - { - desc: "multiple_full_ranges", - src: v1.NewBounds(0, 10), - overlaps: []v1.FingerprintBounds{ - v1.NewBounds(0, 5), - v1.NewBounds(6, 10), - }, - exp: true, - }, - { - desc: "multiple_partial_ranges", - src: v1.NewBounds(0, 10), - overlaps: []v1.FingerprintBounds{ - v1.NewBounds(0, 5), - v1.NewBounds(7, 8), - }, - exp: false, - }, - { - desc: "wraps_partial_range", - src: v1.NewBounds(10, 20), - overlaps: []v1.FingerprintBounds{ - v1.NewBounds(0, 12), - v1.NewBounds(13, 15), - v1.NewBounds(19, 21), - }, - exp: false, - }, - { - desc: "wraps_full_range", - src: v1.NewBounds(10, 20), - overlaps: []v1.FingerprintBounds{ - v1.NewBounds(0, 12), - v1.NewBounds(13, 15), - v1.NewBounds(16, 25), - }, - exp: true, - }, - } { - t.Run(tc.desc, func(t *testing.T) { - require.Equal(t, tc.exp, coversFullRange(tc.src, tc.overlaps)) - }) - } -} - -func TestBiasedReporter(t *testing.T) { - for i, tc := range []struct { - bounds v1.FingerprintBounds - originalFPs [][]model.Fingerprint - expectedFPs [][]model.Fingerprint - }{ - { - bounds: v1.NewBounds(0, 10), - originalFPs: [][]model.Fingerprint{ - {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}, - {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}, - }, - expectedFPs: [][]model.Fingerprint{ - {0, 0, 1, 1, 2, 2, 3, 3, 4, 4}, - {5, 5, 6, 6, 7, 7, 8, 8, 9, 9}, - }, - }, - { - bounds: v1.NewBounds(0, 9), // small resolution loss when dividing by 2 - originalFPs: [][]model.Fingerprint{ - {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}, - {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}, - }, - expectedFPs: [][]model.Fingerprint{ - {0, 0, 1, 1, 2, 2, 3, 3, 4, 4}, - {4, 4, 5, 5, 6, 6, 7, 7, 8, 8}, - }, - }, - { - bounds: v1.NewBounds(0, 10), - originalFPs: [][]model.Fingerprint{ - {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}, - {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}, - {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}, - }, - expectedFPs: [][]model.Fingerprint{ - {0, 0, 0, 1, 1, 1, 2, 2, 2, 3}, - {3, 3, 3, 4, 4, 4, 5, 5, 5, 6}, - {6, 6, 6, 7, 7, 7, 8, 8, 8, 9}, - }, - }, - } { - t.Run(fmt.Sprint(i), func(t *testing.T) { - for i, inputs := range tc.originalFPs { - - validator := func(exp []model.Fingerprint) func(model.Fingerprint) { - j := 0 - return func(fp model.Fingerprint) { - require.Equal(t, int(exp[j]), int(fp)) - j++ - } - }(tc.expectedFPs[i]) - - biased := biasedReporter(validator, tc.bounds, i, len(tc.originalFPs)) - - for _, fp := range inputs { - biased(fp) - } - - } - }) - } -} diff --git a/pkg/bloomcompactor/metrics.go b/pkg/bloomcompactor/metrics.go deleted file mode 100644 index d569a4dbfd82d..0000000000000 --- a/pkg/bloomcompactor/metrics.go +++ /dev/null @@ -1,229 +0,0 @@ -package bloomcompactor - -import ( - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/client_golang/prometheus/promauto" - - v1 "github.com/grafana/loki/v3/pkg/storage/bloom/v1" -) - -const ( - metricsNamespace = "loki" - metricsSubsystem = "bloomcompactor" - - statusSuccess = "success" - statusFailure = "failure" - - tenantLabel = "tenant" -) - -type Metrics struct { - bloomMetrics *v1.Metrics - compactorRunning prometheus.Gauge - chunkSize prometheus.Histogram // uncompressed size of all chunks summed per series - - compactionsStarted prometheus.Counter - compactionCompleted *prometheus.CounterVec - compactionTime *prometheus.HistogramVec - - tenantsDiscovered prometheus.Counter - tenantsOwned prometheus.Counter - tenantsSkipped prometheus.Counter - tenantsStarted prometheus.Counter - tenantTableRanges *prometheus.CounterVec - seriesPerCompaction prometheus.Histogram - bytesPerCompaction prometheus.Histogram - - blocksReused prometheus.Counter - - blocksCreated prometheus.Counter - blocksDeleted prometheus.Counter - metasCreated prometheus.Counter - metasDeleted prometheus.Counter - - progress prometheus.Gauge - timePerTenant *prometheus.CounterVec - - // Retention metrics - retentionRunning prometheus.Gauge - retentionTime *prometheus.HistogramVec - retentionDaysPerIteration *prometheus.HistogramVec - retentionTenantsPerIteration *prometheus.HistogramVec - retentionTenantsExceedingLookback prometheus.Gauge -} - -func NewMetrics(r prometheus.Registerer, bloomMetrics *v1.Metrics) *Metrics { - m := Metrics{ - bloomMetrics: bloomMetrics, - compactorRunning: promauto.With(r).NewGauge(prometheus.GaugeOpts{ - Namespace: metricsNamespace, - Subsystem: metricsSubsystem, - Name: "running", - Help: "Value will be 1 if compactor is currently running on this instance", - }), - chunkSize: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ - Namespace: metricsNamespace, - Subsystem: metricsSubsystem, - Name: "chunk_series_size", - Help: "Uncompressed size of chunks in a series", - // 256B -> 100GB, 10 buckets - Buckets: prometheus.ExponentialBucketsRange(256, 100<<30, 10), - }), - - compactionsStarted: promauto.With(r).NewCounter(prometheus.CounterOpts{ - Namespace: metricsNamespace, - Subsystem: metricsSubsystem, - Name: "compactions_started_total", - Help: "Total number of compactions started", - }), - compactionCompleted: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ - Namespace: metricsNamespace, - Subsystem: metricsSubsystem, - Name: "compactions_completed_total", - Help: "Total number of compactions completed", - }, []string{"status"}), - compactionTime: promauto.With(r).NewHistogramVec(prometheus.HistogramOpts{ - Namespace: metricsNamespace, - Subsystem: metricsSubsystem, - Name: "compactions_time_seconds", - Help: "Time spent during a compaction 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 compaction run", - }), - tenantsOwned: promauto.With(r).NewCounter(prometheus.CounterOpts{ - Namespace: metricsNamespace, - Subsystem: metricsSubsystem, - Name: "tenants_owned", - Help: "Number of tenants owned by this instance", - }), - tenantsSkipped: promauto.With(r).NewCounter(prometheus.CounterOpts{ - Namespace: metricsNamespace, - Subsystem: metricsSubsystem, - Name: "tenants_skipped_total", - Help: "Number of tenants skipped since they are not owned by this instance", - }), - tenantsStarted: promauto.With(r).NewCounter(prometheus.CounterOpts{ - Namespace: metricsNamespace, - Subsystem: metricsSubsystem, - Name: "tenants_started_total", - Help: "Number of tenants started to process during the current compaction run", - }), - tenantTableRanges: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ - Namespace: metricsNamespace, - Subsystem: metricsSubsystem, - Name: "tenant_table_ranges_completed_total", - Help: "Number of tenants table ranges (table, tenant, keyspace) processed during the current compaction run", - }, []string{"status"}), - seriesPerCompaction: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ - Namespace: metricsNamespace, - Subsystem: metricsSubsystem, - Name: "series_per_compaction", - Help: "Number of series during compaction (tenant, table, fingerprint-range). Includes series which copied from other blocks and don't need to be indexed", - // Up to 10M series per tenant, way more than what we expect given our max_global_streams_per_user limits - Buckets: prometheus.ExponentialBucketsRange(1, 10e6, 10), - }), - bytesPerCompaction: promauto.With(r).NewHistogram(prometheus.HistogramOpts{ - Namespace: metricsNamespace, - Subsystem: metricsSubsystem, - Name: "bytes_per_compaction", - Help: "Number of source bytes from chunks added during a compaction cycle (the tenant, table, keyspace tuple).", - // 1KB -> 100GB, 10 buckets - Buckets: prometheus.ExponentialBucketsRange(1<<10, 100<<30, 10), - }), - blocksReused: promauto.With(r).NewCounter(prometheus.CounterOpts{ - Namespace: metricsNamespace, - Subsystem: metricsSubsystem, - Name: "blocks_reused_total", - Help: "Number of overlapping bloom blocks reused when creating new blocks", - }), - blocksCreated: promauto.With(r).NewCounter(prometheus.CounterOpts{ - Namespace: metricsNamespace, - Subsystem: metricsSubsystem, - Name: "blocks_created_total", - Help: "Number of blocks created", - }), - blocksDeleted: promauto.With(r).NewCounter(prometheus.CounterOpts{ - Namespace: metricsNamespace, - Subsystem: metricsSubsystem, - Name: "blocks_deleted_total", - Help: "Number of blocks deleted", - }), - metasCreated: promauto.With(r).NewCounter(prometheus.CounterOpts{ - Namespace: metricsNamespace, - Subsystem: metricsSubsystem, - Name: "metas_created_total", - Help: "Number of metas created", - }), - metasDeleted: promauto.With(r).NewCounter(prometheus.CounterOpts{ - Namespace: metricsNamespace, - Subsystem: metricsSubsystem, - Name: "metas_deleted_total", - Help: "Number of metas deleted", - }), - - progress: promauto.With(r).NewGauge(prometheus.GaugeOpts{ - Namespace: metricsNamespace, - Subsystem: metricsSubsystem, - Name: "progress", - Help: "Progress of the compaction process as a percentage. 1 means compaction is complete.", - }), - - // TODO(owen-d): cleanup tenant metrics over time as ring changes - // TODO(owen-d): histogram for distributions? - timePerTenant: promauto.With(r).NewCounterVec(prometheus.CounterOpts{ - Namespace: metricsNamespace, - Subsystem: metricsSubsystem, - Name: "tenant_compaction_seconds_total", - Help: "Time spent processing a tenant.", - }, []string{tenantLabel}), - - // Retention - retentionRunning: promauto.With(r).NewGauge(prometheus.GaugeOpts{ - Namespace: metricsNamespace, - Subsystem: metricsSubsystem, - Name: "retention_running", - Help: "1 if retention is running in this compactor.", - }), - - retentionTime: promauto.With(r).NewHistogramVec(prometheus.HistogramOpts{ - Namespace: metricsNamespace, - Subsystem: metricsSubsystem, - Name: "retention_time_seconds", - Help: "Time this retention process took to complete.", - Buckets: prometheus.DefBuckets, - }, []string{"status"}), - - retentionDaysPerIteration: promauto.With(r).NewHistogramVec(prometheus.HistogramOpts{ - Namespace: metricsNamespace, - Subsystem: metricsSubsystem, - Name: "retention_days_processed", - Help: "Number of days iterated over during the retention process.", - // 1day -> 5 years, 10 buckets - Buckets: prometheus.ExponentialBucketsRange(1, 365*5, 10), - }, []string{"status"}), - - retentionTenantsPerIteration: promauto.With(r).NewHistogramVec(prometheus.HistogramOpts{ - Namespace: metricsNamespace, - Subsystem: metricsSubsystem, - Name: "retention_tenants_processed", - Help: "Number of tenants on which retention was applied during the retention process.", - // 1 tenant -> 10k tenants, 10 buckets - Buckets: prometheus.ExponentialBucketsRange(1, 10000, 10), - }, []string{"status"}), - - retentionTenantsExceedingLookback: promauto.With(r).NewGauge(prometheus.GaugeOpts{ - Namespace: metricsNamespace, - Subsystem: metricsSubsystem, - Name: "retention_tenants_exceeding_lookback", - Help: "Number of tenants with a retention exceeding the configured retention lookback.", - }), - } - - return &m -} diff --git a/pkg/bloomcompactor/retention.go b/pkg/bloomcompactor/retention.go deleted file mode 100644 index caaf80ffb9c3f..0000000000000 --- a/pkg/bloomcompactor/retention.go +++ /dev/null @@ -1,320 +0,0 @@ -package bloomcompactor - -import ( - "context" - "flag" - "math" - "slices" - "time" - - "github.com/go-kit/log" - "github.com/go-kit/log/level" - "github.com/grafana/dskit/ring" - "github.com/pkg/errors" - "github.com/prometheus/common/model" - - "github.com/grafana/loki/v3/pkg/storage/chunk/client" - storageconfig "github.com/grafana/loki/v3/pkg/storage/config" - "github.com/grafana/loki/v3/pkg/storage/stores/shipper/bloomshipper" - "github.com/grafana/loki/v3/pkg/validation" -) - -type retentionSharding interface { - OwnsRetention() (bool, error) -} - -type firstTokenRetentionSharding struct { - ring ring.ReadRing - ringLifeCycler *ring.BasicLifecycler -} - -func newFirstTokenRetentionSharding(ring ring.ReadRing, ringLifeCycler *ring.BasicLifecycler) *firstTokenRetentionSharding { - return &firstTokenRetentionSharding{ - ring: ring, - ringLifeCycler: ringLifeCycler, - } -} - -// OwnsRetention returns true if the compactor should apply retention. -// This is determined by checking if the compactor owns the smaller token in the ring. -// Note that during a ring topology change, more than one compactor may attempt to apply retention. -// This is fine since retention consists on deleting old data which should be idempotent. -func (s *firstTokenRetentionSharding) OwnsRetention() (bool, error) { - rs, err := s.ring.GetAllHealthy(RingOp) - if err != nil { - return false, errors.Wrap(err, "getting ring healthy instances") - } - if len(rs.Instances) == 0 { - return false, errors.New("no healthy instances in ring") - } - - // Lookup the instance with smaller token - instance := slices.MinFunc(rs.Instances, func(a, b ring.InstanceDesc) int { - smallerA := slices.Min(a.GetTokens()) - smallerB := slices.Min(b.GetTokens()) - if smallerA < smallerB { - return -1 - } - if smallerA > smallerB { - return 1 - } - return 0 - }) - - return instance.GetId() == s.ringLifeCycler.GetInstanceID(), nil -} - -type RetentionConfig struct { - Enabled bool `yaml:"enabled"` - MaxLookbackDays int `yaml:"max_lookback_days"` -} - -func (cfg *RetentionConfig) RegisterFlags(f *flag.FlagSet) { - f.BoolVar(&cfg.Enabled, "bloom-compactor.retention.enabled", false, "Enable bloom retention.") - f.IntVar(&cfg.MaxLookbackDays, "bloom-compactor.retention.max-lookback-days", 365, "Max lookback days for retention.") -} - -func (cfg *RetentionConfig) Validate() error { - if !cfg.Enabled { - return nil - } - - if cfg.MaxLookbackDays < 1 { - return errors.New("max lookback days must be a positive number") - } - return nil -} - -type RetentionLimits interface { - RetentionPeriod(userID string) time.Duration - StreamRetention(userID string) []validation.StreamRetention - AllByUserID() map[string]*validation.Limits - DefaultLimits() *validation.Limits -} - -type RetentionManager struct { - cfg RetentionConfig - limits RetentionLimits - bloomStore bloomshipper.StoreBase - sharding retentionSharding - metrics *Metrics - logger log.Logger - lastDayRun storageconfig.DayTime - - // For testing - now func() model.Time -} - -func NewRetentionManager( - cfg RetentionConfig, - limits RetentionLimits, - bloomStore bloomshipper.StoreBase, - sharding retentionSharding, - metrics *Metrics, - logger log.Logger, -) *RetentionManager { - return &RetentionManager{ - cfg: cfg, - limits: limits, - bloomStore: bloomStore, - sharding: sharding, - metrics: metrics, - logger: log.With(logger, "subcomponent", "retention-manager"), - now: model.Now, - lastDayRun: storageconfig.NewDayTime(0), - } -} - -func (r *RetentionManager) Apply(ctx context.Context) error { - if !r.cfg.Enabled { - level.Debug(r.logger).Log("msg", "retention is disabled") - return nil - } - - start := r.now() - today := storageconfig.NewDayTime(start) - if !today.After(r.lastDayRun) { - // We've already run retention for today - return nil - } - - ownsRetention, err := r.sharding.OwnsRetention() - if err != nil { - return errors.Wrap(err, "checking if compactor owns retention") - } - if !ownsRetention { - level.Debug(r.logger).Log("msg", "this compactor doesn't own retention") - return nil - } - - level.Info(r.logger).Log("msg", "Applying retention", "today", today.String(), "lastDayRun", r.lastDayRun.String()) - r.metrics.retentionRunning.Set(1) - defer r.metrics.retentionRunning.Set(0) - - tenantsRetention := retentionByTenant(r.limits) - r.reportTenantsExceedingLookback(tenantsRetention) - - defaultLimits := r.limits.DefaultLimits() - defaultRetention := findLongestRetention(time.Duration(defaultLimits.RetentionPeriod), defaultLimits.StreamRetention) - - smallestRetention := smallestEnabledRetention(defaultRetention, tenantsRetention) - if smallestRetention == 0 { - level.Debug(r.logger).Log("msg", "no retention period set for any tenant, skipping retention") - return nil - } - - // Start day is today minus the smallest retention period. - // Note that the last retention day is exclusive. E.g. 30 days retention means we keep 30 days of data, - // thus we start deleting data from the 31st day onwards. - startDay := storageconfig.NewDayTime(today.Add(-smallestRetention)).Dec() - // End day is today minus the max lookback days - endDay := storageconfig.NewDayTime(today.Add(-time.Duration(r.cfg.MaxLookbackDays) * 24 * time.Hour)) - - var daysProcessed int - tenantsRetentionApplied := make(map[string]struct{}, 100) - for day := startDay; day.After(endDay); day = day.Dec() { - dayLogger := log.With(r.logger, "day", day.String()) - bloomClient, err := r.bloomStore.Client(day.ModelTime()) - if err != nil { - level.Error(dayLogger).Log("msg", "failed to get bloom store client", "err", err) - break - } - objectClient := bloomClient.ObjectClient() - - tenants, err := r.bloomStore.TenantFilesForInterval( - ctx, bloomshipper.NewInterval(day.Bounds()), - func(tenant string, _ client.StorageObject) bool { - // Filter out tenants whose retention hasn't expired yet - globalRetention := r.limits.RetentionPeriod(tenant) - streamRetention := r.limits.StreamRetention(tenant) - tenantRetention := findLongestRetention(globalRetention, streamRetention) - expirationDay := storageconfig.NewDayTime(today.Add(-tenantRetention)) - return day.Before(expirationDay) - }, - ) - if err != nil { - r.metrics.retentionTime.WithLabelValues(statusFailure).Observe(time.Since(start.Time()).Seconds()) - r.metrics.retentionDaysPerIteration.WithLabelValues(statusFailure).Observe(float64(daysProcessed)) - r.metrics.retentionTenantsPerIteration.WithLabelValues(statusFailure).Observe(float64(len(tenantsRetentionApplied))) - return errors.Wrap(err, "getting users for period") - } - - if len(tenants) == 0 { - // No tenants for this day means we can break here since previous - // retention iterations have already deleted all tenants - break - } - - for tenant, objects := range tenants { - if len(objects) == 0 { - continue - } - - tenantLogger := log.With(dayLogger, "tenant", tenant) - level.Info(tenantLogger).Log("msg", "applying retention to tenant", "keys", len(objects)) - - // Note: we cannot delete the tenant directory directly because it is not an - // actual key in the object store. Instead, we need to delete all keys one by one. - for _, object := range objects { - if err := objectClient.DeleteObject(ctx, object.Key); err != nil { - r.metrics.retentionTime.WithLabelValues(statusFailure).Observe(time.Since(start.Time()).Seconds()) - r.metrics.retentionDaysPerIteration.WithLabelValues(statusFailure).Observe(float64(daysProcessed)) - r.metrics.retentionTenantsPerIteration.WithLabelValues(statusFailure).Observe(float64(len(tenantsRetentionApplied))) - return errors.Wrapf(err, "deleting key %s", object.Key) - } - } - - tenantsRetentionApplied[tenant] = struct{}{} - } - - daysProcessed++ - } - - r.lastDayRun = today - r.metrics.retentionTime.WithLabelValues(statusSuccess).Observe(time.Since(start.Time()).Seconds()) - r.metrics.retentionDaysPerIteration.WithLabelValues(statusSuccess).Observe(float64(daysProcessed)) - r.metrics.retentionTenantsPerIteration.WithLabelValues(statusSuccess).Observe(float64(len(tenantsRetentionApplied))) - level.Info(r.logger).Log("msg", "finished applying retention", "daysProcessed", daysProcessed, "tenants", len(tenantsRetentionApplied)) - - return nil -} - -func (r *RetentionManager) reportTenantsExceedingLookback(retentionByTenant map[string]time.Duration) { - if len(retentionByTenant) == 0 { - r.metrics.retentionTenantsExceedingLookback.Set(0) - return - } - - var tenantsExceedingLookback int - for tenant, retention := range retentionByTenant { - if retention > time.Duration(r.cfg.MaxLookbackDays)*24*time.Hour { - level.Warn(r.logger).Log("msg", "tenant retention exceeds max lookback days", "tenant", tenant, "retention", retention.String()) - } - tenantsExceedingLookback++ - } - - r.metrics.retentionTenantsExceedingLookback.Set(float64(tenantsExceedingLookback)) -} - -func findLongestRetention(globalRetention time.Duration, streamRetention []validation.StreamRetention) time.Duration { - if len(streamRetention) == 0 { - return globalRetention - } - - maxStreamRetention := slices.MaxFunc(streamRetention, func(a, b validation.StreamRetention) int { - return int(a.Period - b.Period) - }) - - if time.Duration(maxStreamRetention.Period) > globalRetention { - return time.Duration(maxStreamRetention.Period) - } - return globalRetention -} - -func retentionByTenant(limits RetentionLimits) map[string]time.Duration { - all := limits.AllByUserID() - if len(all) == 0 { - return nil - } - - retentions := make(map[string]time.Duration, len(all)) - for tenant, lim := range all { - retention := findLongestRetention(time.Duration(lim.RetentionPeriod), lim.StreamRetention) - if retention == 0 { - continue - } - retentions[tenant] = retention - } - - return retentions -} - -// smallestEnabledRetention returns the smallest retention period across all tenants and the default. -func smallestEnabledRetention(defaultRetention time.Duration, perTenantRetention map[string]time.Duration) time.Duration { - if len(perTenantRetention) == 0 { - return defaultRetention - } - - smallest := time.Duration(math.MaxInt64) - if defaultRetention != 0 { - smallest = defaultRetention - } - - for _, retention := range perTenantRetention { - // Skip unlimited retention - if retention == 0 { - continue - } - - if retention < smallest { - smallest = retention - } - } - - if smallest == time.Duration(math.MaxInt64) { - // No tenant nor defaults configures a retention - return 0 - } - - return smallest -} diff --git a/pkg/bloomcompactor/retention_test.go b/pkg/bloomcompactor/retention_test.go deleted file mode 100644 index e610ab5b02e02..0000000000000 --- a/pkg/bloomcompactor/retention_test.go +++ /dev/null @@ -1,882 +0,0 @@ -package bloomcompactor - -import ( - "context" - "flag" - "fmt" - "math" - "os" - "testing" - "time" - - "github.com/go-kit/log" - "github.com/grafana/dskit/services" - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/common/model" - "github.com/stretchr/testify/require" - - "github.com/grafana/loki/v3/pkg/storage" - v1 "github.com/grafana/loki/v3/pkg/storage/bloom/v1" - "github.com/grafana/loki/v3/pkg/storage/chunk/cache" - "github.com/grafana/loki/v3/pkg/storage/chunk/client/local" - storageconfig "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/bloomshipper/config" - "github.com/grafana/loki/v3/pkg/storage/types" - util_log "github.com/grafana/loki/v3/pkg/util/log" - "github.com/grafana/loki/v3/pkg/util/mempool" - lokiring "github.com/grafana/loki/v3/pkg/util/ring" - "github.com/grafana/loki/v3/pkg/validation" -) - -var testTime = parseDayTime("2024-12-31").ModelTime() - -func TestRetention(t *testing.T) { - for _, tc := range []struct { - name string - ownsRetention bool - cfg RetentionConfig - lim mockRetentionLimits - prePopulate func(t *testing.T, schemaCfg storageconfig.SchemaConfig, bloomStore *bloomshipper.BloomStore) - expectErr bool - check func(t *testing.T, bloomStore *bloomshipper.BloomStore) - }{ - { - name: "retention disabled", - ownsRetention: true, - cfg: RetentionConfig{ - Enabled: false, - MaxLookbackDays: 2 * 365, - }, - lim: mockRetentionLimits{ - retention: map[string]time.Duration{ - "1": 30 * 24 * time.Hour, - "2": 200 * 24 * time.Hour, - "3": 500 * 24 * time.Hour, - }, - }, - prePopulate: func(t *testing.T, schemaCfg storageconfig.SchemaConfig, bloomStore *bloomshipper.BloomStore) { - putMetasForLastNDays(t, schemaCfg, bloomStore, "1", testTime, 200) - putMetasForLastNDays(t, schemaCfg, bloomStore, "2", testTime, 50) - putMetasForLastNDays(t, schemaCfg, bloomStore, "3", testTime, 500) - }, - check: func(t *testing.T, bloomStore *bloomshipper.BloomStore) { - metas := getGroupedMetasForLastNDays(t, bloomStore, "1", testTime, 500) - require.Equal(t, 1, len(metas)) - require.Equal(t, 200, len(metas[0])) - metas = getGroupedMetasForLastNDays(t, bloomStore, "2", testTime, 500) - require.Equal(t, 1, len(metas)) - require.Equal(t, 50, len(metas[0])) - metas = getGroupedMetasForLastNDays(t, bloomStore, "3", testTime, 500) - require.Equal(t, 1, len(metas)) - require.Equal(t, 500, len(metas[0])) - }, - }, - { - name: "compactor does not own retention", - ownsRetention: false, - cfg: RetentionConfig{ - Enabled: true, - MaxLookbackDays: 2 * 365, - }, - lim: mockRetentionLimits{ - retention: map[string]time.Duration{ - "1": 30 * 24 * time.Hour, - "2": 200 * 24 * time.Hour, - "3": 500 * 24 * time.Hour, - }, - }, - prePopulate: func(t *testing.T, schemaCfg storageconfig.SchemaConfig, bloomStore *bloomshipper.BloomStore) { - putMetasForLastNDays(t, schemaCfg, bloomStore, "1", testTime, 200) - putMetasForLastNDays(t, schemaCfg, bloomStore, "2", testTime, 50) - putMetasForLastNDays(t, schemaCfg, bloomStore, "3", testTime, 500) - }, - check: func(t *testing.T, bloomStore *bloomshipper.BloomStore) { - metas := getGroupedMetasForLastNDays(t, bloomStore, "1", testTime, 500) - require.Equal(t, 1, len(metas)) - require.Equal(t, 200, len(metas[0])) - metas = getGroupedMetasForLastNDays(t, bloomStore, "2", testTime, 500) - require.Equal(t, 1, len(metas)) - require.Equal(t, 50, len(metas[0])) - metas = getGroupedMetasForLastNDays(t, bloomStore, "3", testTime, 500) - require.Equal(t, 1, len(metas)) - require.Equal(t, 500, len(metas[0])) - }, - }, - { - name: "unlimited retention", - ownsRetention: true, - cfg: RetentionConfig{ - Enabled: true, - MaxLookbackDays: 2 * 365, - }, - lim: mockRetentionLimits{ - retention: map[string]time.Duration{ - "1": 0, - }, - }, - prePopulate: func(t *testing.T, schemaCfg storageconfig.SchemaConfig, bloomStore *bloomshipper.BloomStore) { - putMetasForLastNDays(t, schemaCfg, bloomStore, "1", testTime, 200) - }, - check: func(t *testing.T, bloomStore *bloomshipper.BloomStore) { - metas := getGroupedMetasForLastNDays(t, bloomStore, "1", testTime, 500) - require.Equal(t, 1, len(metas)) - require.Equal(t, 200, len(metas[0])) - }, - }, - { - name: "default retention", - ownsRetention: true, - cfg: RetentionConfig{ - Enabled: true, - MaxLookbackDays: 2 * 365, - }, - lim: mockRetentionLimits{ - defaultRetention: 30 * 24 * time.Hour, - }, - prePopulate: func(t *testing.T, schemaCfg storageconfig.SchemaConfig, bloomStore *bloomshipper.BloomStore) { - putMetasForLastNDays(t, schemaCfg, bloomStore, "1", testTime, 200) - }, - check: func(t *testing.T, bloomStore *bloomshipper.BloomStore) { - metas := getGroupedMetasForLastNDays(t, bloomStore, "1", testTime, 500) - require.Equal(t, 1, len(metas)) - require.Equal(t, 31, len(metas[0])) - }, - }, - { - name: "retention lookback smaller than max retention", - ownsRetention: true, - cfg: RetentionConfig{ - Enabled: true, - MaxLookbackDays: 100, - }, - lim: mockRetentionLimits{ - retention: map[string]time.Duration{ - "1": 30 * 24 * time.Hour, - "2": 20 * 24 * time.Hour, - "3": 200 * 24 * time.Hour, - "4": 400 * 24 * time.Hour, - }, - streamRetention: map[string][]validation.StreamRetention{ - "1": { - { - Period: model.Duration(30 * 24 * time.Hour), - }, - { - Period: model.Duration(40 * 24 * time.Hour), - }, - }, - "2": { - { - Period: model.Duration(10 * 24 * time.Hour), - }, - }, - }, - }, - prePopulate: func(t *testing.T, schemaCfg storageconfig.SchemaConfig, bloomStore *bloomshipper.BloomStore) { - putMetasForLastNDays(t, schemaCfg, bloomStore, "1", testTime, 200) - putMetasForLastNDays(t, schemaCfg, bloomStore, "2", testTime, 50) - putMetasForLastNDays(t, schemaCfg, bloomStore, "3", testTime, 500) - putMetasForLastNDays(t, schemaCfg, bloomStore, "4", testTime, 500) - }, - check: func(t *testing.T, bloomStore *bloomshipper.BloomStore) { - // Tenant 1 has 40 days of retention, and we wrote 200 days of metas - // We should get two groups: 0th-40th and 101th-200th - metas := getGroupedMetasForLastNDays(t, bloomStore, "1", testTime, 500) - require.Equal(t, 2, len(metas)) - require.Equal(t, 41, len(metas[0])) // 0-40th day - require.Equal(t, 100, len(metas[1])) // 100th-200th day - - // Tenant 2 has 20 days of retention, and we wrote 50 days of metas - // We should get one group: 0th-20th - metas = getGroupedMetasForLastNDays(t, bloomStore, "2", testTime, 500) - require.Equal(t, 1, len(metas)) - require.Equal(t, 21, len(metas[0])) // 0th-20th - - // Tenant 3 has 200 days of retention, and we wrote 500 days of metas - // Since the manager looks up to 100 days, we shouldn't have deleted any metas - metas = getGroupedMetasForLastNDays(t, bloomStore, "3", testTime, 500) - require.Equal(t, 1, len(metas)) - require.Equal(t, 500, len(metas[0])) // 0th-500th - - // Tenant 4 has 400 days of retention, and we wrote 500 days of metas - // Since the manager looks up to 100 days, we shouldn't have deleted any metas - metas = getGroupedMetasForLastNDays(t, bloomStore, "4", testTime, 500) - require.Equal(t, 1, len(metas)) - require.Equal(t, 500, len(metas[0])) // 0th-500th - }, - }, - { - name: "retention lookback bigger than max retention", - ownsRetention: true, - cfg: RetentionConfig{ - Enabled: true, - MaxLookbackDays: 2 * 365, - }, - lim: mockRetentionLimits{ - retention: map[string]time.Duration{ - "1": 30 * 24 * time.Hour, - "2": 20 * 24 * time.Hour, - "3": 200 * 24 * time.Hour, - "4": 400 * 24 * time.Hour, - }, - streamRetention: map[string][]validation.StreamRetention{ - "1": { - { - Period: model.Duration(30 * 24 * time.Hour), - }, - { - Period: model.Duration(40 * 24 * time.Hour), - }, - }, - "2": { - { - Period: model.Duration(10 * 24 * time.Hour), - }, - }, - }, - }, - prePopulate: func(t *testing.T, schemaCfg storageconfig.SchemaConfig, bloomStore *bloomshipper.BloomStore) { - putMetasForLastNDays(t, schemaCfg, bloomStore, "1", testTime, 200) - putMetasForLastNDays(t, schemaCfg, bloomStore, "2", testTime, 50) - putMetasForLastNDays(t, schemaCfg, bloomStore, "3", testTime, 500) - putMetasForLastNDays(t, schemaCfg, bloomStore, "4", testTime, 500) - }, - check: func(t *testing.T, bloomStore *bloomshipper.BloomStore) { - // Tenant 1 has 40 days of retention, and we wrote 200 days of metas - // We should get one groups: 0th-40th - metas := getGroupedMetasForLastNDays(t, bloomStore, "1", testTime, 500) - require.Equal(t, 1, len(metas)) - require.Equal(t, 41, len(metas[0])) // 0-40th day - - // Tenant 2 has 20 days of retention, and we wrote 50 days of metas - // We should get one group: 0th-20th - metas = getGroupedMetasForLastNDays(t, bloomStore, "2", testTime, 500) - require.Equal(t, 1, len(metas)) - require.Equal(t, 21, len(metas[0])) // 0th-20th - - // Tenant 3 has 200 days of retention, and we wrote 500 days of metas - // We should get one group: 0th-200th - metas = getGroupedMetasForLastNDays(t, bloomStore, "3", testTime, 500) - require.Equal(t, 1, len(metas)) - require.Equal(t, 201, len(metas[0])) // 0th-200th - - // Tenant 4 has 400 days of retention, and we wrote 500 days of metas - // Since the manager looks up to 100 days, we shouldn't have deleted any metas - metas = getGroupedMetasForLastNDays(t, bloomStore, "4", testTime, 500) - require.Equal(t, 1, len(metas)) - require.Equal(t, 401, len(metas[0])) // 0th-400th - }, - }, - { - name: "hit no tenants in table", - ownsRetention: true, - cfg: RetentionConfig{ - Enabled: true, - MaxLookbackDays: 2 * 365, - }, - lim: mockRetentionLimits{ - retention: map[string]time.Duration{ - "1": 30 * 24 * time.Hour, - }, - }, - prePopulate: func(t *testing.T, schemaCfg storageconfig.SchemaConfig, bloomStore *bloomshipper.BloomStore) { - // Place metas with a gap of 50 days. [0th-100th], [151th-200th] - putMetasForLastNDays(t, schemaCfg, bloomStore, "1", testTime, 100) - putMetasForLastNDays(t, schemaCfg, bloomStore, "1", testTime.Add(-150*24*time.Hour), 50) - }, - check: func(t *testing.T, bloomStore *bloomshipper.BloomStore) { - // We should get two groups: 0th-30th and 151th-200th - metas := getGroupedMetasForLastNDays(t, bloomStore, "1", testTime, 500) - require.Equal(t, 2, len(metas)) - require.Equal(t, 31, len(metas[0])) // 0th-30th day - require.Equal(t, 50, len(metas[1])) // 151th-200th day - }, - }, - } { - t.Run(tc.name, func(t *testing.T) { - bloomStore, schema, _, err := NewMockBloomStore(t) - require.NoError(t, err) - - rm := NewRetentionManager( - tc.cfg, - tc.lim, - bloomStore, - mockSharding{ - ownsRetention: tc.ownsRetention, - }, - NewMetrics(nil, v1.NewMetrics(nil)), - util_log.Logger, - ) - rm.now = func() model.Time { - return testTime - } - - tc.prePopulate(t, schema, bloomStore) - - err = rm.Apply(context.Background()) - if tc.expectErr { - require.Error(t, err) - return - } - require.NoError(t, err) - - tc.check(t, bloomStore) - }) - } -} - -func TestRetentionRunsOncePerDay(t *testing.T) { - bloomStore, schema, _, err := NewMockBloomStore(t) - require.NoError(t, err) - - rm := NewRetentionManager( - RetentionConfig{ - Enabled: true, - MaxLookbackDays: 365, - }, - mockRetentionLimits{ - retention: map[string]time.Duration{ - "1": 30 * 24 * time.Hour, - }, - }, - bloomStore, - mockSharding{ - ownsRetention: true, - }, - NewMetrics(nil, v1.NewMetrics(nil)), - util_log.Logger, - ) - rm.now = func() model.Time { - return testTime - } - - // Write metas for the last 100 days and run retention - putMetasForLastNDays(t, schema, bloomStore, "1", testTime, 100) - err = rm.Apply(context.Background()) - require.NoError(t, err) - - // We should get only the first 30 days of metas - metas := getGroupedMetasForLastNDays(t, bloomStore, "1", testTime, 100) - require.Equal(t, 1, len(metas)) - require.Equal(t, 31, len(metas[0])) // 0th-30th day - - // We now change the now() time to be a bit later in the day - rm.now = func() model.Time { - return testTime.Add(1 * time.Hour) - } - - // Write metas again and run retention. Since we already ran retention at now()'s day, - // Apply should be a noop, and therefore we should be able to get all the 100 days of metas - putMetasForLastNDays(t, schema, bloomStore, "1", testTime, 100) - err = rm.Apply(context.Background()) - require.NoError(t, err) - - metas = getGroupedMetasForLastNDays(t, bloomStore, "1", testTime, 100) - require.Equal(t, 1, len(metas)) - require.Equal(t, 100, len(metas[0])) - - // We now change the now() time to be the next day, retention should run again - rm.now = func() model.Time { - return testTime.Add(24 * time.Hour) - } - err = rm.Apply(context.Background()) - require.NoError(t, err) - - // We should only see the first 30 days of metas - metas = getGroupedMetasForLastNDays(t, bloomStore, "1", testTime, 100) - require.Equal(t, 1, len(metas)) - require.Equal(t, 30, len(metas[0])) // 0th-30th day -} - -func TestOwnsRetention(t *testing.T) { - for _, tc := range []struct { - name string - numCompactors int - }{ - { - name: "single compactor", - numCompactors: 1, - }, - { - name: "multiple compactors", - numCompactors: 100, - }, - } { - t.Run(tc.name, func(t *testing.T) { - var ringManagers []*lokiring.RingManager - for i := 0; i < tc.numCompactors; i++ { - var cfg Config - cfg.RegisterFlags(flag.NewFlagSet("ring", flag.PanicOnError)) - cfg.Ring.KVStore.Store = "inmemory" - cfg.Ring.InstanceID = fmt.Sprintf("bloom-compactor-%d", i) - cfg.Ring.InstanceAddr = fmt.Sprintf("localhost-%d", i) - - ringManager, err := lokiring.NewRingManager("bloom-compactor", lokiring.ServerMode, cfg.Ring, 1, cfg.Ring.NumTokens, util_log.Logger, prometheus.NewRegistry()) - require.NoError(t, err) - require.NoError(t, ringManager.StartAsync(context.Background())) - - ringManagers = append(ringManagers, ringManager) - } - t.Cleanup(func() { - // Stop all rings and wait for them to stop. - for _, ringManager := range ringManagers { - ringManager.StopAsync() - require.Eventually(t, func() bool { - return ringManager.State() == services.Terminated - }, 1*time.Minute, 100*time.Millisecond) - } - }) - - // Wait for all rings to see each other. - for _, ringManager := range ringManagers { - require.Eventually(t, func() bool { - running := ringManager.State() == services.Running - discovered := ringManager.Ring.InstancesCount() == tc.numCompactors - return running && discovered - }, 1*time.Minute, 100*time.Millisecond) - } - - var shardings []retentionSharding - for _, ringManager := range ringManagers { - shardings = append(shardings, newFirstTokenRetentionSharding(ringManager.Ring, ringManager.RingLifecycler)) - } - - var ownsRetention int - for _, sharding := range shardings { - owns, err := sharding.OwnsRetention() - require.NoError(t, err) - if owns { - ownsRetention++ - } - } - - require.Equal(t, 1, ownsRetention) - }) - } -} - -func TestFindLongestRetention(t *testing.T) { - for _, tc := range []struct { - name string - globalRetention time.Duration - streamRetention []validation.StreamRetention - expectedRetention time.Duration - }{ - { - name: "no retention", - expectedRetention: 0, - }, - { - name: "global retention", - globalRetention: 30 * 24 * time.Hour, - expectedRetention: 30 * 24 * time.Hour, - }, - { - name: "stream retention", - streamRetention: []validation.StreamRetention{ - { - Period: model.Duration(30 * 24 * time.Hour), - }, - }, - expectedRetention: 30 * 24 * time.Hour, - }, - { - name: "two stream retention", - streamRetention: []validation.StreamRetention{ - { - Period: model.Duration(30 * 24 * time.Hour), - }, - { - Period: model.Duration(40 * 24 * time.Hour), - }, - }, - expectedRetention: 40 * 24 * time.Hour, - }, - { - name: "stream retention bigger than global", - globalRetention: 20 * 24 * time.Hour, - streamRetention: []validation.StreamRetention{ - { - Period: model.Duration(30 * 24 * time.Hour), - }, - { - Period: model.Duration(40 * 24 * time.Hour), - }, - }, - expectedRetention: 40 * 24 * time.Hour, - }, - { - name: "global retention bigger than stream", - globalRetention: 40 * 24 * time.Hour, - streamRetention: []validation.StreamRetention{ - { - Period: model.Duration(20 * 24 * time.Hour), - }, - { - Period: model.Duration(30 * 24 * time.Hour), - }, - }, - expectedRetention: 40 * 24 * time.Hour, - }, - } { - t.Run(tc.name, func(t *testing.T) { - retention := findLongestRetention(tc.globalRetention, tc.streamRetention) - require.Equal(t, tc.expectedRetention, retention) - }) - } -} - -func TestSmallestRetention(t *testing.T) { - for _, tc := range []struct { - name string - limits RetentionLimits - expectedRetention time.Duration - expectedHasRetention bool - }{ - { - name: "no retention", - limits: mockRetentionLimits{}, - expectedRetention: 0, - }, - { - name: "default global retention", - limits: mockRetentionLimits{ - defaultRetention: 30 * 24 * time.Hour, - }, - expectedRetention: 30 * 24 * time.Hour, - }, - { - name: "default stream retention", - limits: mockRetentionLimits{ - defaultStreamRetention: []validation.StreamRetention{ - { - Period: model.Duration(30 * 24 * time.Hour), - }, - }, - }, - expectedRetention: 30 * 24 * time.Hour, - }, - { - name: "tenant configured unlimited", - limits: mockRetentionLimits{ - retention: map[string]time.Duration{ - "1": 0, - }, - defaultRetention: 30 * 24 * time.Hour, - }, - expectedRetention: 30 * 24 * time.Hour, - }, - { - name: "no default one tenant", - limits: mockRetentionLimits{ - retention: map[string]time.Duration{ - "1": 30 * 24 * time.Hour, - }, - streamRetention: map[string][]validation.StreamRetention{ - "1": { - { - Period: model.Duration(40 * 24 * time.Hour), - }, - }, - }, - }, - expectedRetention: 40 * 24 * time.Hour, - }, - { - name: "no default two tenants", - limits: mockRetentionLimits{ - retention: map[string]time.Duration{ - "1": 30 * 24 * time.Hour, - "2": 20 * 24 * time.Hour, - }, - streamRetention: map[string][]validation.StreamRetention{ - "1": { - { - Period: model.Duration(40 * 24 * time.Hour), - }, - }, - "2": { - { - Period: model.Duration(10 * 24 * time.Hour), - }, - }, - }, - }, - expectedRetention: 20 * 24 * time.Hour, - }, - { - name: "default bigger than tenant", - limits: mockRetentionLimits{ - retention: map[string]time.Duration{ - "1": 10 * 24 * time.Hour, - }, - streamRetention: map[string][]validation.StreamRetention{ - "1": { - { - Period: model.Duration(20 * 24 * time.Hour), - }, - }, - }, - defaultRetention: 40 * 24 * time.Hour, - defaultStreamRetention: []validation.StreamRetention{ - { - Period: model.Duration(30 * 24 * time.Hour), - }, - }, - }, - expectedRetention: 20 * 24 * time.Hour, - }, - { - name: "tenant bigger than default", - limits: mockRetentionLimits{ - retention: map[string]time.Duration{ - "1": 30 * 24 * time.Hour, - }, - streamRetention: map[string][]validation.StreamRetention{ - "1": { - { - Period: model.Duration(40 * 24 * time.Hour), - }, - }, - }, - defaultRetention: 10 * 24 * time.Hour, - defaultStreamRetention: []validation.StreamRetention{ - { - Period: model.Duration(20 * 24 * time.Hour), - }, - }, - }, - expectedRetention: 20 * 24 * time.Hour, - }, - } { - t.Run(tc.name, func(t *testing.T) { - defaultLim := tc.limits.DefaultLimits() - defaultRetention := findLongestRetention(time.Duration(defaultLim.RetentionPeriod), defaultLim.StreamRetention) - tenantsRetention := retentionByTenant(tc.limits) - - retention := smallestEnabledRetention(defaultRetention, tenantsRetention) - require.Equal(t, tc.expectedRetention, retention) - }) - } -} - -func TestRetentionConfigValidate(t *testing.T) { - for _, tc := range []struct { - name string - cfg RetentionConfig - expectErr bool - }{ - { - name: "enabled and valid", - cfg: RetentionConfig{ - Enabled: true, - MaxLookbackDays: 2 * 365, - }, - expectErr: false, - }, - { - name: "invalid max lookback days", - cfg: RetentionConfig{ - Enabled: true, - MaxLookbackDays: 0, - }, - expectErr: true, - }, - { - name: "disabled and invalid", - cfg: RetentionConfig{ - Enabled: false, - MaxLookbackDays: 0, - }, - expectErr: false, - }, - } { - t.Run(tc.name, func(t *testing.T) { - err := tc.cfg.Validate() - if tc.expectErr { - require.Error(t, err) - return - } - require.NoError(t, err) - }) - } -} - -func putMetasForLastNDays(t *testing.T, schemaCfg storageconfig.SchemaConfig, bloomStore *bloomshipper.BloomStore, tenant string, start model.Time, days int) { - const metasPerDay = 2 - - startDay := storageconfig.NewDayTime(start) - endDay := storageconfig.NewDayTime(startDay.Add(-time.Duration(days) * 24 * time.Hour)) - for day := startDay; day.After(endDay); day = day.Dec() { - period, err := schemaCfg.SchemaForTime(day.ModelTime()) - require.NoError(t, err) - - dayTable := storageconfig.NewDayTable(day, period.IndexTables.Prefix) - bloomClient, err := bloomStore.Client(dayTable.ModelTime()) - require.NoErrorf(t, err, "failed to get bloom client for day %d: %s", day, err) - - for i := 0; i < metasPerDay; i++ { - err = bloomClient.PutMeta(context.Background(), bloomshipper.Meta{ - MetaRef: bloomshipper.MetaRef{ - Ref: bloomshipper.Ref{ - TenantID: tenant, - TableName: dayTable.String(), - Bounds: v1.NewBounds(model.Fingerprint(i*100), model.Fingerprint(i*100+100)), - }, - }, - Blocks: []bloomshipper.BlockRef{}, - }) - require.NoError(t, err) - } - } -} - -// getMetasForLastNDays returns groups of continuous metas for the last N days. -func getGroupedMetasForLastNDays(t *testing.T, bloomStore *bloomshipper.BloomStore, tenant string, start model.Time, days int) [][][]bloomshipper.Meta { - metasGrouped := make([][][]bloomshipper.Meta, 0) - currentGroup := make([][]bloomshipper.Meta, 0) - - startDay := storageconfig.NewDayTime(start) - endDay := storageconfig.NewDayTime(startDay.Add(-time.Duration(days) * 24 * time.Hour)) - - for day := startDay; day.After(endDay); day = day.Dec() { - metas, err := bloomStore.FetchMetas(context.Background(), bloomshipper.MetaSearchParams{ - TenantID: tenant, - Interval: bloomshipper.NewInterval(day.Bounds()), - Keyspace: v1.NewBounds(0, math.MaxUint64), - }) - require.NoError(t, err) - if len(metas) == 0 { - // We have reached the end of the metas group: cut a new group - if len(currentGroup) > 0 { - metasGrouped = append(metasGrouped, currentGroup) - currentGroup = make([][]bloomshipper.Meta, 0) - } - continue - } - currentGroup = append(currentGroup, metas) - } - - // Append the last group if it's not empty - if len(currentGroup) > 0 { - metasGrouped = append(metasGrouped, currentGroup) - } - - return metasGrouped -} - -func NewMockBloomStore(t *testing.T) (*bloomshipper.BloomStore, storageconfig.SchemaConfig, string, error) { - workDir := t.TempDir() - return NewMockBloomStoreWithWorkDir(t, workDir) -} - -func NewMockBloomStoreWithWorkDir(t *testing.T, workDir string) (*bloomshipper.BloomStore, storageconfig.SchemaConfig, string, error) { - schemaCfg := storageconfig.SchemaConfig{ - Configs: []storageconfig.PeriodConfig{ - { - ObjectType: types.StorageTypeFileSystem, - From: storageconfig.DayTime{ - Time: testTime.Add(-2 * 365 * 24 * time.Hour), // -2 year - }, - IndexTables: storageconfig.IndexPeriodicTableConfig{ - PeriodicTableConfig: storageconfig.PeriodicTableConfig{ - Period: 24 * time.Hour, - Prefix: "schema_a_table_", - }}, - }, - { - ObjectType: types.StorageTypeFileSystem, - From: storageconfig.DayTime{ - Time: testTime.Add(-365 * 24 * time.Hour), // -1 year - }, - IndexTables: storageconfig.IndexPeriodicTableConfig{ - PeriodicTableConfig: storageconfig.PeriodicTableConfig{ - Period: 24 * time.Hour, - Prefix: "schema_b_table_", - }}, - }, - }, - } - - storageConfig := storage.Config{ - FSConfig: local.FSConfig{ - Directory: workDir, - }, - BloomShipperConfig: config.Config{ - WorkingDirectory: []string{workDir}, - DownloadParallelism: 1, - BlocksCache: config.BlocksCacheConfig{ - SoftLimit: 1 << 20, - HardLimit: 2 << 20, - TTL: time.Hour, - PurgeInterval: time.Hour, - }, - }, - } - - reg := prometheus.NewPedanticRegistry() - metrics := storage.NewClientMetrics() - t.Cleanup(metrics.Unregister) - logger := log.NewLogfmtLogger(os.Stderr) - - metasCache := cache.NewMockCache() - blocksCache := bloomshipper.NewFsBlocksCache(storageConfig.BloomShipperConfig.BlocksCache, prometheus.NewPedanticRegistry(), logger) - - store, err := bloomshipper.NewBloomStore(schemaCfg.Configs, storageConfig, metrics, metasCache, blocksCache, &mempool.SimpleHeapAllocator{}, reg, logger) - if err == nil { - t.Cleanup(store.Stop) - } - - return store, schemaCfg, workDir, err -} - -type mockRetentionLimits struct { - retention map[string]time.Duration - streamRetention map[string][]validation.StreamRetention - defaultRetention time.Duration - defaultStreamRetention []validation.StreamRetention -} - -func (m mockRetentionLimits) RetentionPeriod(tenant string) time.Duration { - return m.retention[tenant] -} - -func (m mockRetentionLimits) StreamRetention(tenant string) []validation.StreamRetention { - return m.streamRetention[tenant] -} - -func (m mockRetentionLimits) AllByUserID() map[string]*validation.Limits { - tenants := make(map[string]*validation.Limits, len(m.retention)) - - for tenant, retention := range m.retention { - if _, ok := tenants[tenant]; !ok { - tenants[tenant] = &validation.Limits{} - } - tenants[tenant].RetentionPeriod = model.Duration(retention) - } - - for tenant, streamRetention := range m.streamRetention { - if _, ok := tenants[tenant]; !ok { - tenants[tenant] = &validation.Limits{} - } - tenants[tenant].StreamRetention = streamRetention - } - - return tenants -} - -func (m mockRetentionLimits) DefaultLimits() *validation.Limits { - return &validation.Limits{ - RetentionPeriod: model.Duration(m.defaultRetention), - StreamRetention: m.defaultStreamRetention, - } -} - -type mockSharding struct { - ownsRetention bool -} - -func (m mockSharding) OwnsRetention() (bool, error) { - return m.ownsRetention, nil -} diff --git a/pkg/bloomcompactor/spec.go b/pkg/bloomcompactor/spec.go deleted file mode 100644 index 696f192970b68..0000000000000 --- a/pkg/bloomcompactor/spec.go +++ /dev/null @@ -1,312 +0,0 @@ -package bloomcompactor - -import ( - "context" - "fmt" - "io" - - "github.com/go-kit/log" - "github.com/go-kit/log/level" - "github.com/pkg/errors" - "github.com/prometheus/common/model" - - iter "github.com/grafana/loki/v3/pkg/iter/v2" - "github.com/grafana/loki/v3/pkg/logproto" - v1 "github.com/grafana/loki/v3/pkg/storage/bloom/v1" - "github.com/grafana/loki/v3/pkg/storage/chunk" - "github.com/grafana/loki/v3/pkg/storage/chunk/fetcher" - "github.com/grafana/loki/v3/pkg/storage/stores" - "github.com/grafana/loki/v3/pkg/storage/stores/shipper/bloomshipper" - "github.com/grafana/loki/v3/pkg/storage/stores/shipper/indexshipper/tsdb" -) - -// inclusive range -type Keyspace struct { - min, max model.Fingerprint -} - -func (k Keyspace) Cmp(other Keyspace) v1.BoundsCheck { - if other.max < k.min { - return v1.Before - } else if other.min > k.max { - return v1.After - } - return v1.Overlap -} - -// Store is likely bound within. This allows specifying impls like ShardedStore -// to only request the shard-range needed from the existing store. -type BloomGenerator interface { - Generate(ctx context.Context) (skippedBlocks []v1.BlockMetadata, toClose []io.Closer, results iter.Iterator[*v1.Block], err error) -} - -// Simple implementation of a BloomGenerator. -type SimpleBloomGenerator struct { - userID string - store iter.Iterator[*v1.Series] - chunkLoader ChunkLoader - blocksIter iter.ResetIterator[*v1.SeriesWithBlooms] - - // options to build blocks with - opts v1.BlockOptions - - metrics *Metrics - logger log.Logger - - writerReaderFunc func() (v1.BlockWriter, v1.BlockReader) - reporter func(model.Fingerprint) - - tokenizer *v1.BloomTokenizer -} - -// SimpleBloomGenerator is a foundational implementation of BloomGenerator. -// It mainly wires up a few different components to generate bloom filters for a set of blocks -// and handles schema compatibility: -// Blocks which are incompatible with the schema are skipped and will have their chunks reindexed -func NewSimpleBloomGenerator( - userID string, - opts v1.BlockOptions, - store iter.Iterator[*v1.Series], - chunkLoader ChunkLoader, - blocksIter iter.ResetIterator[*v1.SeriesWithBlooms], - writerReaderFunc func() (v1.BlockWriter, v1.BlockReader), - reporter func(model.Fingerprint), - metrics *Metrics, - logger log.Logger, -) *SimpleBloomGenerator { - return &SimpleBloomGenerator{ - userID: userID, - opts: opts, - store: store, - chunkLoader: chunkLoader, - blocksIter: blocksIter, - logger: log.With( - logger, - "component", "bloom_generator", - "org_id", userID, - ), - writerReaderFunc: writerReaderFunc, - metrics: metrics, - reporter: reporter, - - tokenizer: v1.NewBloomTokenizer( - opts.Schema.NGramLen(), - opts.Schema.NGramSkip(), - int(opts.UnencodedBlockOptions.MaxBloomSizeBytes), - metrics.bloomMetrics, - log.With( - logger, - "component", "bloom_tokenizer", - "org_id", userID, - ), - ), - } -} - -func (s *SimpleBloomGenerator) populator(ctx context.Context) v1.BloomPopulatorFunc { - return func( - series *v1.Series, - srcBlooms iter.SizedIterator[*v1.Bloom], - toAdd v1.ChunkRefs, - ch chan *v1.BloomCreation, - ) { - level.Debug(s.logger).Log( - "msg", "populating bloom filter", - "stage", "before", - "fp", series.Fingerprint, - "chunks", len(series.Chunks), - ) - chunkItersWithFP := s.chunkLoader.Load(ctx, s.userID, &v1.Series{ - Fingerprint: series.Fingerprint, - Chunks: toAdd, - }) - - s.tokenizer.Populate(srcBlooms, chunkItersWithFP.itr, ch) - - if s.reporter != nil { - s.reporter(series.Fingerprint) - } - } -} - -func (s *SimpleBloomGenerator) Generate(ctx context.Context) *LazyBlockBuilderIterator { - level.Debug(s.logger).Log("msg", "generating bloom filters for blocks", "schema", fmt.Sprintf("%+v", s.opts.Schema)) - - series := iter.NewPeekIter(s.store) - - // TODO: Use interface - impl, ok := s.blocksIter.(*blockLoadingIter) - if ok { - impl.Filter( - func(bq *bloomshipper.CloseableBlockQuerier) bool { - - logger := log.With(s.logger, "block", bq.BlockRef) - md, err := bq.Metadata() - schema := md.Options.Schema - if err != nil { - level.Warn(logger).Log("msg", "failed to get schema for block", "err", err) - bq.Close() // close unused querier - return false - } - - if !s.opts.Schema.Compatible(schema) { - level.Warn(logger).Log("msg", "block schema incompatible with options", "generator_schema", fmt.Sprintf("%+v", s.opts.Schema), "block_schema", fmt.Sprintf("%+v", schema)) - bq.Close() // close unused querier - return false - } - - level.Debug(logger).Log("msg", "adding compatible block to bloom generation inputs") - return true - }, - ) - } - - return NewLazyBlockBuilderIterator(ctx, s.opts, s.metrics, s.populator(ctx), s.writerReaderFunc, series, s.blocksIter) -} - -// LazyBlockBuilderIterator is a lazy iterator over blocks that builds -// each block by adding series to them until they are full. -type LazyBlockBuilderIterator struct { - ctx context.Context - opts v1.BlockOptions - metrics *Metrics - populate v1.BloomPopulatorFunc - writerReaderFunc func() (v1.BlockWriter, v1.BlockReader) - series iter.PeekIterator[*v1.Series] - blocks iter.ResetIterator[*v1.SeriesWithBlooms] - - bytesAdded int - curr *v1.Block - err error -} - -func NewLazyBlockBuilderIterator( - ctx context.Context, - opts v1.BlockOptions, - metrics *Metrics, - populate v1.BloomPopulatorFunc, - writerReaderFunc func() (v1.BlockWriter, v1.BlockReader), - series iter.PeekIterator[*v1.Series], - blocks iter.ResetIterator[*v1.SeriesWithBlooms], -) *LazyBlockBuilderIterator { - return &LazyBlockBuilderIterator{ - ctx: ctx, - opts: opts, - metrics: metrics, - populate: populate, - writerReaderFunc: writerReaderFunc, - series: series, - blocks: blocks, - } -} - -func (b *LazyBlockBuilderIterator) Bytes() (bytes int) { - return b.bytesAdded -} - -func (b *LazyBlockBuilderIterator) Next() bool { - // No more series to process - if _, hasNext := b.series.Peek(); !hasNext { - return false - } - - if err := b.ctx.Err(); err != nil { - b.err = errors.Wrap(err, "context canceled") - return false - } - - if err := b.blocks.Reset(); err != nil { - b.err = errors.Wrap(err, "reset blocks iterator") - return false - } - - mergeBuilder := v1.NewMergeBuilder(b.blocks, b.series, b.populate, b.metrics.bloomMetrics) - writer, reader := b.writerReaderFunc() - blockBuilder, err := v1.NewBlockBuilder(b.opts, writer) - if err != nil { - b.err = errors.Wrap(err, "failed to create bloom block builder") - return false - } - _, sourceBytes, err := mergeBuilder.Build(blockBuilder) - b.bytesAdded += sourceBytes - - if err != nil { - b.err = errors.Wrap(err, "failed to build bloom block") - return false - } - - b.curr = v1.NewBlock(reader, b.metrics.bloomMetrics) - return true -} - -func (b *LazyBlockBuilderIterator) At() *v1.Block { - return b.curr -} - -func (b *LazyBlockBuilderIterator) Err() error { - return b.err -} - -// IndexLoader loads an index. This helps us do things like -// load TSDBs for a specific period excluding multitenant (pre-compacted) indices -type indexLoader interface { - Index() (tsdb.Index, error) -} - -// ChunkItersByFingerprint models the chunks belonging to a fingerprint -type ChunkItersByFingerprint struct { - fp model.Fingerprint - itr iter.Iterator[v1.ChunkRefWithIter] -} - -// ChunkLoader loads chunks from a store -type ChunkLoader interface { - Load(ctx context.Context, userID string, series *v1.Series) *ChunkItersByFingerprint -} - -// StoreChunkLoader loads chunks from a store -type StoreChunkLoader struct { - fetcherProvider stores.ChunkFetcherProvider - metrics *Metrics -} - -func NewStoreChunkLoader(fetcherProvider stores.ChunkFetcherProvider, metrics *Metrics) *StoreChunkLoader { - return &StoreChunkLoader{ - fetcherProvider: fetcherProvider, - metrics: metrics, - } -} - -func (s *StoreChunkLoader) Load(ctx context.Context, userID string, series *v1.Series) *ChunkItersByFingerprint { - // NB(owen-d): This is probably unnecessary as we should only have one fetcher - // because we'll only be working on a single index period at a time, but this should protect - // us in the case of refactoring/changing this and likely isn't a perf bottleneck. - chksByFetcher := make(map[*fetcher.Fetcher][]chunk.Chunk) - for _, chk := range series.Chunks { - fetcher := s.fetcherProvider.GetChunkFetcher(chk.From) - chksByFetcher[fetcher] = append(chksByFetcher[fetcher], chunk.Chunk{ - ChunkRef: logproto.ChunkRef{ - Fingerprint: uint64(series.Fingerprint), - UserID: userID, - From: chk.From, - Through: chk.Through, - Checksum: chk.Checksum, - }, - }) - } - - var ( - fetchers = make([]Fetcher[chunk.Chunk, chunk.Chunk], 0, len(chksByFetcher)) - inputs = make([][]chunk.Chunk, 0, len(chksByFetcher)) - ) - for fetcher, chks := range chksByFetcher { - fn := FetchFunc[chunk.Chunk, chunk.Chunk](fetcher.FetchChunks) - fetchers = append(fetchers, fn) - inputs = append(inputs, chks) - } - - return &ChunkItersByFingerprint{ - fp: series.Fingerprint, - itr: newBatchedChunkLoader(ctx, fetchers, inputs, s.metrics, batchedLoaderDefaultBatchSize), - } -} diff --git a/pkg/bloomcompactor/spec_test.go b/pkg/bloomcompactor/spec_test.go deleted file mode 100644 index 8ee914b5c8982..0000000000000 --- a/pkg/bloomcompactor/spec_test.go +++ /dev/null @@ -1,170 +0,0 @@ -package bloomcompactor - -import ( - "bytes" - "context" - "fmt" - "testing" - - "github.com/go-kit/log" - "github.com/prometheus/common/model" - "github.com/stretchr/testify/require" - - "github.com/grafana/loki/v3/pkg/chunkenc" - v2 "github.com/grafana/loki/v3/pkg/iter/v2" - 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/util/mempool" -) - -func blocksFromSchema(t *testing.T, n int, options v1.BlockOptions) (res []*v1.Block, data []v1.SeriesWithBlooms, refs []bloomshipper.BlockRef) { - return blocksFromSchemaWithRange(t, n, options, 0, 0xffff) -} - -// splits 100 series across `n` non-overlapping blocks. -// uses options to build blocks with. -func blocksFromSchemaWithRange(t *testing.T, n int, options v1.BlockOptions, fromFP, throughFp model.Fingerprint) (res []*v1.Block, data []v1.SeriesWithBlooms, refs []bloomshipper.BlockRef) { - if 100%n != 0 { - panic("100 series must be evenly divisible by n") - } - - numSeries := 100 - data, _ = v1.MkBasicSeriesWithBlooms(numSeries, fromFP, throughFp, 0, 10000) - - seriesPerBlock := numSeries / n - - for i := 0; i < n; i++ { - // references for linking in memory reader+writer - indexBuf := bytes.NewBuffer(nil) - bloomsBuf := bytes.NewBuffer(nil) - writer := v1.NewMemoryBlockWriter(indexBuf, bloomsBuf) - reader := v1.NewByteReader(indexBuf, bloomsBuf) - - builder, err := v1.NewBlockBuilder( - options, - writer, - ) - require.Nil(t, err) - - minIdx, maxIdx := i*seriesPerBlock, (i+1)*seriesPerBlock - - itr := v2.NewSliceIter[v1.SeriesWithBlooms](data[minIdx:maxIdx]) - _, err = builder.BuildFrom(itr) - require.Nil(t, err) - - res = append(res, v1.NewBlock(reader, v1.NewMetrics(nil))) - ref := genBlockRef(data[minIdx].Series.Fingerprint, data[maxIdx-1].Series.Fingerprint) - t.Log("create block", ref) - refs = append(refs, ref) - } - - return res, data, refs -} - -// doesn't actually load any chunks -type dummyChunkLoader struct{} - -func (dummyChunkLoader) Load(_ context.Context, _ string, series *v1.Series) *ChunkItersByFingerprint { - return &ChunkItersByFingerprint{ - fp: series.Fingerprint, - itr: v2.NewEmptyIter[v1.ChunkRefWithIter](), - } -} - -func dummyBloomGen(t *testing.T, opts v1.BlockOptions, store v2.Iterator[*v1.Series], blocks []*v1.Block, refs []bloomshipper.BlockRef) *SimpleBloomGenerator { - bqs := make([]*bloomshipper.CloseableBlockQuerier, 0, len(blocks)) - for i, b := range blocks { - bqs = append(bqs, &bloomshipper.CloseableBlockQuerier{ - BlockRef: refs[i], - BlockQuerier: v1.NewBlockQuerier(b, &mempool.SimpleHeapAllocator{}, v1.DefaultMaxPageSize), - }) - } - - fetcher := func(_ context.Context, refs []bloomshipper.BlockRef) ([]*bloomshipper.CloseableBlockQuerier, error) { - res := make([]*bloomshipper.CloseableBlockQuerier, 0, len(refs)) - for _, ref := range refs { - for _, bq := range bqs { - if ref.Bounds.Equal(bq.Bounds) { - res = append(res, bq) - } - } - } - t.Log("req", refs) - t.Log("res", res) - return res, nil - } - - blocksIter := newBlockLoadingIter(context.Background(), refs, FetchFunc[bloomshipper.BlockRef, *bloomshipper.CloseableBlockQuerier](fetcher), 1) - - return NewSimpleBloomGenerator( - "fake", - opts, - store, - dummyChunkLoader{}, - blocksIter, - func() (v1.BlockWriter, v1.BlockReader) { - indexBuf := bytes.NewBuffer(nil) - bloomsBuf := bytes.NewBuffer(nil) - return v1.NewMemoryBlockWriter(indexBuf, bloomsBuf), v1.NewByteReader(indexBuf, bloomsBuf) - }, - nil, - NewMetrics(nil, v1.NewMetrics(nil)), - log.NewNopLogger(), - ) -} - -func TestSimpleBloomGenerator(t *testing.T) { - const maxBlockSize = 100 << 20 // 100MB - for _, enc := range []chunkenc.Encoding{chunkenc.EncNone, chunkenc.EncGZIP, chunkenc.EncSnappy} { - for _, tc := range []struct { - desc string - fromSchema, toSchema v1.BlockOptions - overlapping bool - }{ - { - desc: "SkipsIncompatibleSchemas", - fromSchema: v1.NewBlockOptions(enc, 3, 0, maxBlockSize, 0), - toSchema: v1.NewBlockOptions(enc, 4, 0, maxBlockSize, 0), - }, - { - desc: "CombinesBlocks", - fromSchema: v1.NewBlockOptions(enc, 4, 0, maxBlockSize, 0), - toSchema: v1.NewBlockOptions(enc, 4, 0, maxBlockSize, 0), - }, - } { - t.Run(fmt.Sprintf("%s/%s", tc.desc, enc), func(t *testing.T) { - sourceBlocks, data, refs := blocksFromSchemaWithRange(t, 2, tc.fromSchema, 0x00000, 0x6ffff) - storeItr := v2.NewMapIter[v1.SeriesWithBlooms, *v1.Series]( - v2.NewSliceIter[v1.SeriesWithBlooms](data), - func(swb v1.SeriesWithBlooms) *v1.Series { - return swb.Series - }, - ) - - gen := dummyBloomGen(t, tc.toSchema, storeItr, sourceBlocks, refs) - results := gen.Generate(context.Background()) - - var outputBlocks []*v1.Block - for results.Next() { - outputBlocks = append(outputBlocks, results.At()) - } - // require.Equal(t, tc.outputBlocks, len(outputBlocks)) - - // Check all the input series are present in the output blocks. - expectedRefs := v1.PointerSlice(data) - outputRefs := make([]*v1.SeriesWithBlooms, 0, len(data)) - for _, block := range outputBlocks { - bq := v1.NewBlockQuerier(block, &mempool.SimpleHeapAllocator{}, v1.DefaultMaxPageSize).Iter() - for bq.Next() { - outputRefs = append(outputRefs, bq.At()) - } - } - require.Equal(t, len(expectedRefs), len(outputRefs)) - for i := range expectedRefs { - require.Equal(t, expectedRefs[i].Series, outputRefs[i].Series) - } - }) - } - } - -} diff --git a/pkg/bloomcompactor/tracker.go b/pkg/bloomcompactor/tracker.go deleted file mode 100644 index 1c9bde0a4ae71..0000000000000 --- a/pkg/bloomcompactor/tracker.go +++ /dev/null @@ -1,123 +0,0 @@ -package bloomcompactor - -import ( - "fmt" - "math" - "sync" - - "github.com/pkg/errors" - "github.com/prometheus/common/model" - - v1 "github.com/grafana/loki/v3/pkg/storage/bloom/v1" - "github.com/grafana/loki/v3/pkg/storage/config" -) - -type tableRangeProgress struct { - tenant string - table config.DayTime - bounds v1.FingerprintBounds - - lastFP model.Fingerprint -} - -type compactionTracker struct { - sync.Mutex - - nTables int - // tables -> n_tenants - metadata map[config.DayTime]int - - // table -> tenant -> workload_id -> keyspace - tables map[config.DayTime]map[string]map[string]*tableRangeProgress -} - -func newCompactionTracker(nTables int) (*compactionTracker, error) { - if nTables <= 0 { - return nil, errors.New("nTables must be positive") - } - - return &compactionTracker{ - nTables: nTables, - tables: make(map[config.DayTime]map[string]map[string]*tableRangeProgress), - metadata: make(map[config.DayTime]int), - }, nil -} - -func (c *compactionTracker) registerTable(tbl config.DayTime, nTenants int) { - c.Lock() - defer c.Unlock() - c.metadata[tbl] = nTenants - c.tables[tbl] = make(map[string]map[string]*tableRangeProgress) -} - -func (c *compactionTracker) update( - tenant string, - table config.DayTime, - bounds v1.FingerprintBounds, - mostRecentFP model.Fingerprint, -) { - c.Lock() - defer c.Unlock() - key := fmt.Sprintf("%s_%s_%s", tenant, table.String(), bounds.String()) - tbl, ok := c.tables[table] - if !ok { - panic(fmt.Sprintf("table not registered: %s", table.String())) - } - workloads, ok := tbl[tenant] - if !ok { - workloads = make(map[string]*tableRangeProgress) - tbl[tenant] = workloads - } - workloads[key] = &tableRangeProgress{ - tenant: tenant, - table: table, - bounds: bounds, - // ensure lastFP is at least the minimum fp for each range; - // this handles the case when the first fingeprint hasn't been processed yet. - // as a precaution we also clip the lastFP to the bounds. - lastFP: min(max(mostRecentFP, bounds.Min), bounds.Max), - } -} - -// Returns progress in (0-1) range, bounded to 3 decimals. -// compaction progress is measured by the following: -// 1. The number of days of data that has been compacted -// as a percentage of the total number of days of data that needs to be compacted. -// 2. Within each day, the number of tenants that have been compacted -// as a percentage of the total number of tenants that need to be compacted. -// 3. Within each tenant, the percent of the keyspaces that have been compacted. -// NB(owen-d): this treats all tenants equally, when this may not be the case wrt -// the work they have to do. This is a simplification and can be x-referenced with -// the tenant_compaction_seconds_total metric to see how much time is being spent on -// each tenant while the compaction tracker shows total compaction progress across -// all tables and tenants. -func (c *compactionTracker) progress() (progress float64) { - c.Lock() - defer c.Unlock() - - perTablePct := 1. / float64(c.nTables) - - // for all registered tables, determine the number of registered tenants - for tbl, nTenants := range c.metadata { - perTenantPct := perTablePct / float64(nTenants) - - // iterate tenants in each table - for _, tenant := range c.tables[tbl] { - var ( - totalKeyspace uint64 - finishedKeyspace uint64 - ) - - // iterate table ranges for each tenant+table pair - for _, batch := range tenant { - totalKeyspace += batch.bounds.Range() - finishedKeyspace += uint64(batch.lastFP - batch.bounds.Min) - } - - tenantProgress := float64(finishedKeyspace) / float64(totalKeyspace) - progress += perTenantPct * tenantProgress - } - } - - return math.Round(progress*1000) / 1000 -} diff --git a/pkg/bloomcompactor/tracker_test.go b/pkg/bloomcompactor/tracker_test.go deleted file mode 100644 index e23eb55d6dc64..0000000000000 --- a/pkg/bloomcompactor/tracker_test.go +++ /dev/null @@ -1,106 +0,0 @@ -package bloomcompactor - -import ( - "testing" - - "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/config" -) - -func mkTblRange(tenant string, tbl config.DayTime, from, through model.Fingerprint) *tenantTableRange { - return &tenantTableRange{ - tenant: tenant, - table: config.NewDayTable(tbl, ""), - ownershipRange: v1.NewBounds(from, through), - } -} - -func updateTracker(tr *compactionTracker, tt *tenantTableRange, lastFP model.Fingerprint) { - tr.update(tt.tenant, tt.table.DayTime, tt.ownershipRange, lastFP) -} - -func TestCompactionTrackerClipsRange(t *testing.T) { - // test invalid table number - tracker, err := newCompactionTracker(1) - require.NoError(t, err) - - day1 := parseDayTime("2024-01-01") - tracker.registerTable(day1, 1) - - work := mkTblRange("a", day1, 0, 10) - updateTracker(tracker, work, 0) - require.Equal(t, 0., tracker.progress()) - updateTracker(tracker, work, work.ownershipRange.Min) - require.Equal(t, 0., tracker.progress()) - updateTracker(tracker, work, 5) - require.Equal(t, 0.5, tracker.progress()) - updateTracker(tracker, work, work.ownershipRange.Max*2) - require.Equal(t, 1., tracker.progress()) - updateTracker(tracker, work, work.ownershipRange.Max) - require.Equal(t, 1., tracker.progress()) -} - -func TestCompactionTrackerFull(t *testing.T) { - // test invalid table number - _, err := newCompactionTracker(0) - require.Error(t, err) - - tracker, err := newCompactionTracker(2) - require.NoError(t, err) - - day1 := parseDayTime("2024-01-01") - day2 := parseDayTime("2024-01-02") - - tracker.registerTable(day1, 2) - tracker.registerTable(day2, 3) - require.Equal(t, 0., tracker.progress()) - - aDayOneOffsetZero := mkTblRange("a", day1, 0, 10) - aDayOneOffsetOne := mkTblRange("a", day1, 40, 50) - bDayOneOffsetZero := mkTblRange("b", day1, 10, 20) - - // register the workloads for day0_tenantA - updateTracker(tracker, aDayOneOffsetZero, 0) - updateTracker(tracker, aDayOneOffsetOne, 0) - - require.Equal(t, 0., tracker.progress()) - updateTracker(tracker, aDayOneOffsetZero, aDayOneOffsetZero.ownershipRange.Max) // simulate finish - require.Equal(t, 0.125, tracker.progress()) - updateTracker(tracker, aDayOneOffsetOne, aDayOneOffsetOne.ownershipRange.Max) // simulate finish - require.Equal(t, 0.25, tracker.progress()) - - // register the workloads for day0_tenantB - updateTracker(tracker, bDayOneOffsetZero, 0) - - require.Equal(t, 0.25, tracker.progress()) - // simulate half finish (partial workload progress) - updateTracker( - tracker, - bDayOneOffsetZero, - bDayOneOffsetZero.ownershipRange.Min+model.Fingerprint(bDayOneOffsetZero.ownershipRange.Range())/2, - ) - require.Equal(t, 0.375, tracker.progress()) - // simulate finish - updateTracker(tracker, bDayOneOffsetZero, bDayOneOffsetZero.ownershipRange.Max) - require.Equal(t, 0.5, tracker.progress()) - - aDayTwoOffsetZero := mkTblRange("a", day2, 0, 10) - bDayTwoOffsetZero := mkTblRange("b", day2, 10, 20) - cDayTwoOffsetZero := mkTblRange("c", day2, 20, 30) - updateTracker(tracker, aDayTwoOffsetZero, 0) - updateTracker(tracker, bDayTwoOffsetZero, 0) - updateTracker(tracker, cDayTwoOffsetZero, 0) - require.Equal(t, 0.5, tracker.progress()) - - // simulate finish for the a & b - updateTracker(tracker, aDayTwoOffsetZero, aDayTwoOffsetZero.ownershipRange.Max) - updateTracker(tracker, bDayTwoOffsetZero, bDayTwoOffsetZero.ownershipRange.Max) - require.Equal(t, 0.833, tracker.progress()) - - // simulate finish for the c - updateTracker(tracker, cDayTwoOffsetZero, cDayTwoOffsetZero.ownershipRange.Max) - require.Equal(t, 1., tracker.progress()) -} diff --git a/pkg/bloomcompactor/tsdb.go b/pkg/bloomcompactor/tsdb.go deleted file mode 100644 index c522cc6dbcef2..0000000000000 --- a/pkg/bloomcompactor/tsdb.go +++ /dev/null @@ -1,262 +0,0 @@ -package bloomcompactor - -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" - iter "github.com/grafana/loki/v3/pkg/iter/v2" - 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, - ) (iter.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, -) (iter.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) (iter.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 iter.NewEmptyIter[*v1.Series](), ctx.Err() - default: - return iter.NewCancelableIter[*v1.Series](ctx, iter.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, -) (iter.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/bloomcompactor/tsdb_test.go b/pkg/bloomcompactor/tsdb_test.go deleted file mode 100644 index b81880d83b46a..0000000000000 --- a/pkg/bloomcompactor/tsdb_test.go +++ /dev/null @@ -1,106 +0,0 @@ -package bloomcompactor - -import ( - "context" - "math" - "testing" - - "github.com/prometheus/common/model" - "github.com/prometheus/prometheus/model/labels" - "github.com/stretchr/testify/require" - - v2 "github.com/grafana/loki/v3/pkg/iter/v2" - 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 := v2.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/bloomcompactor/versioned_range.go b/pkg/bloomcompactor/versioned_range.go deleted file mode 100644 index 8af56a0754cc3..0000000000000 --- a/pkg/bloomcompactor/versioned_range.go +++ /dev/null @@ -1,260 +0,0 @@ -package bloomcompactor - -import ( - "sort" - - "github.com/prometheus/common/model" - - v1 "github.com/grafana/loki/v3/pkg/storage/bloom/v1" - "github.com/grafana/loki/v3/pkg/storage/stores/shipper/bloomshipper" -) - -type tsdbToken struct { - through model.Fingerprint // inclusive - version int // TSDB version -} - -// a ring of token ranges used to identify old metas. -// each token represents that a TSDB version has covered the entire range -// up to that point from the previous token. -type tsdbTokenRange []tsdbToken - -func (t tsdbTokenRange) Len() int { - return len(t) -} - -func (t tsdbTokenRange) Less(i, j int) bool { - return t[i].through < t[j].through -} - -func (t tsdbTokenRange) Swap(i, j int) { - t[i], t[j] = t[j], t[i] -} - -// Add ensures a versioned set of bounds is added to the range. If the bounds are already -// covered by a more up to date version, it returns false. -func (t tsdbTokenRange) Add(version int, bounds v1.FingerprintBounds) (res tsdbTokenRange, added bool) { - // allows attempting to join neighboring token ranges with identical versions - // that aren't known until the end of the function - var shouldReassemble bool - var reassembleFrom int - defer func() { - if shouldReassemble { - res = res.reassemble(reassembleFrom) - } - }() - - // special case: first token - if len(t) == 0 { - tok := tsdbToken{through: bounds.Max, version: version} - // special case: first token is included in bounds, no need to fill negative space - if bounds.Min == 0 { - return append(t, tok), true - } - // Use a negative version to indicate that the range is not covered by any version. - return append(t, tsdbToken{through: bounds.Min - 1, version: -1}, tok), true - } - - // For non-nil token ranges, we continually update the range with newer versions. - for { - // find first token that covers the start of the range - i := sort.Search(len(t), func(i int) bool { - return t[i].through >= bounds.Min - }) - - if i == len(t) { - tok := tsdbToken{through: bounds.Max, version: version} - - // edge case: there is no gap between the previous token range - // and the new one; - // skip adding a negative token - if t[len(t)-1].through == bounds.Min-1 { - return append(t, tok), true - } - - // the range is not covered by any version and we are at the end of the range. - // Add a negative token and the new token. - negative := tsdbToken{through: bounds.Min - 1, version: -1} - return append(t, negative, tok), true - } - - // Otherwise, we've found a token that covers the start of the range. - newer := t[i].version < version - preExisting := t.boundsForToken(i) - if !newer { - if bounds.Within(preExisting) { - // The range is already covered by a more up to date version, no need - // to add anything, but honor if an earlier token was added - return t, added - } - - // The range is partially covered by a more up to date version; - // update the range we need to check and continue - bounds = v1.NewBounds(preExisting.Max+1, bounds.Max) - continue - } - - // If we need to update the range, there are 5 cases: - // 1. `equal`: the incoming range equals an existing range () - // ------ # addition - // ------ # src - // 2. `subset`: the incoming range is a subset of an existing range - // ------ # addition - // -------- # src - // 3. `overflow_both_sides`: the incoming range is a superset of an existing range. This is not possible - // because the first token in the ring implicitly covers the left bound (zero) of all possible fps. - // Therefore, we can skip this case. - // ------ # addition - // ---- # src - // 4. `right_overflow`: the incoming range overflows the right side of an existing range - // ------ # addition - // ------ # src - // 5. `left_overflow`: the incoming range overflows the left side of an existing range. This can be skipped - // for the same reason as `superset`. - // ------ # addition - // ------ # src - - // 1) (`equal`): we're replacing the same bounds - if bounds.Equal(preExisting) { - t[i].version = version - return t, true - } - - // 2) (`subset`): the incoming range is a subset of an existing range - if bounds.Within(preExisting) { - // 2a) the incoming range touches the existing range's minimum bound - if bounds.Min == preExisting.Min { - tok := tsdbToken{through: bounds.Max, version: version} - t = append(t, tsdbToken{}) - copy(t[i+1:], t[i:]) - t[i] = tok - return t, true - } - // 2b) the incoming range touches the existing range's maximum bound - if bounds.Max == preExisting.Max { - t[i].through = bounds.Min - 1 - tok := tsdbToken{through: bounds.Max, version: version} - t = append(t, tsdbToken{}) - copy(t[i+2:], t[i+1:]) - t[i+1] = tok - return t, true - } - - // 2c) the incoming range is does not touch either edge; - // add two tokens (the new one and a new left-bound for the old range) - tok := tsdbToken{through: bounds.Max, version: version} - t = append(t, tsdbToken{}, tsdbToken{}) - copy(t[i+2:], t[i:]) - t[i+1] = tok - t[i].through = bounds.Min - 1 - return t, true - } - - // 4) (`right_overflow`): the incoming range overflows the right side of an existing range - - // 4a) shortcut: the incoming range is a right-overlapping superset of the existing range. - // replace the existing token's version, update reassembly targets for merging neighboring ranges - // w/ the same version, and continue - if preExisting.Min == bounds.Min { - t[i].version = version - bounds.Min = preExisting.Max + 1 - added = true - if !shouldReassemble { - reassembleFrom = i - shouldReassemble = true - } - continue - } - - // 4b) the incoming range overlaps the right side of the existing range but - // does not touch the left side; - // add a new token for the right side of the existing range then update the reassembly targets - // and continue - overlap := tsdbToken{through: t[i].through, version: version} - t[i].through = bounds.Min - 1 - t = append(t, tsdbToken{}) - copy(t[i+2:], t[i+1:]) - t[i+1] = overlap - added = true - bounds.Min = overlap.through + 1 - if !shouldReassemble { - reassembleFrom = i + 1 - shouldReassemble = true - } - continue - } -} - -func (t tsdbTokenRange) boundsForToken(i int) v1.FingerprintBounds { - if i == 0 { - return v1.FingerprintBounds{Min: 0, Max: t[i].through} - } - return v1.FingerprintBounds{Min: t[i-1].through + 1, Max: t[i].through} -} - -// reassemble merges neighboring tokens with the same version -func (t tsdbTokenRange) reassemble(from int) tsdbTokenRange { - reassembleTo := from - for i := from; i < len(t)-1; i++ { - if t[i].version != t[i+1].version { - break - } - reassembleTo = i + 1 - } - - if reassembleTo == from { - return t - } - t[from].through = t[reassembleTo].through - copy(t[from+1:], t[reassembleTo+1:]) - return t[:len(t)-(reassembleTo-from)] -} - -func outdatedMetas(metas []bloomshipper.Meta) (outdated []bloomshipper.Meta, err error) { - // Sort metas descending by most recent source when checking - // for outdated metas (older metas are discarded if they don't change the range). - sort.Slice(metas, func(i, j int) bool { - a, aExists := metas[i].MostRecentSource() - b, bExists := metas[j].MostRecentSource() - - if !aExists && !bExists { - // stable sort two sourceless metas by their bounds (easier testing) - return metas[i].Bounds.Less(metas[j].Bounds) - } - - if !aExists { - // If a meta has no sources, it's out of date by definition. - // By convention we sort it to the beginning of the list and will mark it for removal later - return true - } - - if !bExists { - // if a exists but b does not, mark b as lesser, sorting b to the - // front - return false - } - return !a.TS.Before(b.TS) - }) - - var ( - tokenRange tsdbTokenRange - added bool - ) - - for _, meta := range metas { - mostRecent, exists := meta.MostRecentSource() - if !exists { - // if the meta exists but does not reference a TSDB, it's out of date - // TODO(owen-d): this shouldn't happen, figure out why - outdated = append(outdated, meta) - } - version := int(model.TimeFromUnixNano(mostRecent.TS.UnixNano())) - tokenRange, added = tokenRange.Add(version, meta.Bounds) - if !added { - outdated = append(outdated, meta) - } - } - - return outdated, nil - -} diff --git a/pkg/bloomcompactor/versioned_range_test.go b/pkg/bloomcompactor/versioned_range_test.go deleted file mode 100644 index 67db348036ffa..0000000000000 --- a/pkg/bloomcompactor/versioned_range_test.go +++ /dev/null @@ -1,352 +0,0 @@ -package bloomcompactor - -import ( - "testing" - - "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 Test_TsdbTokenRange(t *testing.T) { - type addition struct { - version int - bounds v1.FingerprintBounds - } - type exp struct { - added bool - err bool - } - mk := func(version int, min, max model.Fingerprint) addition { - return addition{version, v1.FingerprintBounds{Min: min, Max: max}} - } - tok := func(version int, through model.Fingerprint) tsdbToken { - return tsdbToken{version: version, through: through} - } - - for _, tc := range []struct { - desc string - additions []addition - exp []bool - result tsdbTokenRange - }{ - { - desc: "ascending versions", - additions: []addition{ - mk(1, 0, 10), - mk(2, 11, 20), - mk(3, 15, 25), - }, - exp: []bool{true, true, true}, - result: tsdbTokenRange{ - tok(1, 10), - tok(2, 14), - tok(3, 25), - }, - }, - { - desc: "descending versions", - additions: []addition{ - mk(3, 15, 25), - mk(2, 11, 20), - mk(1, 0, 10), - }, - exp: []bool{true, true, true}, - result: tsdbTokenRange{ - tok(1, 10), - tok(2, 14), - tok(3, 25), - }, - }, - { - desc: "simple", - additions: []addition{ - mk(3, 0, 10), - mk(2, 11, 20), - mk(1, 15, 25), - }, - exp: []bool{true, true, true}, - result: tsdbTokenRange{ - tok(3, 10), - tok(2, 20), - tok(1, 25), - }, - }, - { - desc: "simple replacement", - additions: []addition{ - mk(3, 10, 20), - mk(2, 0, 9), - }, - exp: []bool{true, true}, - result: tsdbTokenRange{ - tok(2, 9), - tok(3, 20), - }, - }, - { - desc: "complex", - additions: []addition{ - mk(5, 30, 50), - mk(4, 20, 45), - mk(3, 25, 70), - mk(2, 10, 20), - mk(1, 1, 5), - }, - exp: []bool{true, true, true, true, true, true}, - result: tsdbTokenRange{ - tok(-1, 0), - tok(1, 5), - tok(-1, 9), - tok(2, 19), - tok(4, 29), - tok(5, 50), - tok(3, 70), - }, - }, - { - desc: "neighboring upper range", - additions: []addition{ - mk(5, 30, 50), - mk(4, 51, 60), - }, - exp: []bool{true, true}, - result: tsdbTokenRange{ - tok(-1, 29), - tok(5, 50), - tok(4, 60), - }, - }, - { - desc: "non-neighboring upper range", - additions: []addition{ - mk(5, 30, 50), - mk(4, 55, 60), - }, - exp: []bool{true, true}, - result: tsdbTokenRange{ - tok(-1, 29), - tok(5, 50), - tok(-1, 54), - tok(4, 60), - }, - }, - { - desc: "earlier version within", - additions: []addition{ - mk(5, 30, 50), - mk(4, 40, 45), - }, - exp: []bool{true, false}, - result: tsdbTokenRange{ - tok(-1, 29), - tok(5, 50), - }, - }, - { - desc: "earlier version right overlapping", - additions: []addition{ - mk(5, 10, 20), - mk(4, 15, 25), - }, - exp: []bool{true, true}, - result: tsdbTokenRange{ - tok(-1, 9), - tok(5, 20), - tok(4, 25), - }, - }, - { - desc: "older version overlaps two", - additions: []addition{ - mk(3, 10, 20), - mk(2, 21, 30), - mk(1, 15, 25), - }, - exp: []bool{true, true, false}, - result: tsdbTokenRange{ - tok(-1, 9), - tok(3, 20), - tok(2, 30), - }, - }, - { - desc: "older version overlaps two w middle", - additions: []addition{ - mk(3, 10, 20), - mk(2, 22, 30), - mk(1, 15, 25), - }, - exp: []bool{true, true, true}, - result: tsdbTokenRange{ - tok(-1, 9), - tok(3, 20), - tok(1, 21), - tok(2, 30), - }, - }, - { - desc: "newer right overflow", - additions: []addition{ - mk(1, 30, 50), - mk(2, 40, 60), - }, - exp: []bool{true, true}, - result: tsdbTokenRange{ - tok(-1, 29), - tok(1, 39), - tok(2, 60), - }, - }, - { - desc: "newer right overflow superset", - additions: []addition{ - mk(1, 30, 50), - mk(2, 30, 60), - }, - exp: []bool{true, true}, - result: tsdbTokenRange{ - tok(-1, 29), - tok(2, 60), - }, - }, - { - desc: "newer right overflow partial", - additions: []addition{ - mk(1, 30, 50), - mk(2, 40, 60), - }, - exp: []bool{true, true}, - result: tsdbTokenRange{ - tok(-1, 29), - tok(1, 39), - tok(2, 60), - }, - }, - } { - t.Run(tc.desc, func(t *testing.T) { - var ( - tr tsdbTokenRange - added bool - ) - for i, a := range tc.additions { - tr, added = tr.Add(a.version, a.bounds) - exp := tc.exp[i] - require.Equal(t, exp, added, "on iteration %d", i) - } - require.Equal(t, tc.result, tr) - }) - } -} - -func Test_OutdatedMetas(t *testing.T) { - gen := func(bounds v1.FingerprintBounds, tsdbTimes ...model.Time) (meta bloomshipper.Meta) { - for _, tsdbTime := range tsdbTimes { - meta.Sources = append(meta.Sources, tsdb.SingleTenantTSDBIdentifier{TS: tsdbTime.Time()}) - } - meta.Bounds = bounds - return meta - } - - for _, tc := range []struct { - desc string - metas []bloomshipper.Meta - exp []bloomshipper.Meta - }{ - { - desc: "no metas", - metas: nil, - exp: nil, - }, - { - desc: "single meta", - metas: []bloomshipper.Meta{ - gen(v1.NewBounds(0, 10), 0), - }, - exp: nil, - }, - { - desc: "single outdated meta", - metas: []bloomshipper.Meta{ - gen(v1.NewBounds(0, 10), 0), - gen(v1.NewBounds(0, 10), 1), - }, - exp: []bloomshipper.Meta{ - gen(v1.NewBounds(0, 10), 0), - }, - }, - { - desc: "single outdated via partitions", - metas: []bloomshipper.Meta{ - gen(v1.NewBounds(0, 5), 0), - gen(v1.NewBounds(6, 10), 0), - gen(v1.NewBounds(0, 10), 1), - }, - exp: []bloomshipper.Meta{ - gen(v1.NewBounds(6, 10), 0), - gen(v1.NewBounds(0, 5), 0), - }, - }, - { - desc: "same tsdb versions", - metas: []bloomshipper.Meta{ - gen(v1.NewBounds(0, 5), 0), - gen(v1.NewBounds(6, 10), 0), - gen(v1.NewBounds(0, 10), 1), - }, - exp: []bloomshipper.Meta{ - gen(v1.NewBounds(6, 10), 0), - gen(v1.NewBounds(0, 5), 0), - }, - }, - { - desc: "multi version ordering", - metas: []bloomshipper.Meta{ - gen(v1.NewBounds(0, 5), 0), - gen(v1.NewBounds(0, 10), 1), // only part of the range is outdated, must keep - gen(v1.NewBounds(8, 10), 2), - }, - exp: []bloomshipper.Meta{ - gen(v1.NewBounds(0, 5), 0), - }, - }, - { - desc: "metas without sources are removed", - metas: []bloomshipper.Meta{ - gen(v1.NewBounds(0, 5), 0), - gen(v1.NewBounds(6, 10), 0), - gen(v1.NewBounds(0, 10), 1), - gen(v1.NewBounds(11, 15)), // Meta without sources - }, - exp: []bloomshipper.Meta{ - gen(v1.NewBounds(11, 15)), // Meta without sources - gen(v1.NewBounds(6, 10), 0), - gen(v1.NewBounds(0, 5), 0), - }, - }, - { - desc: "metas without sources are interleaved", - metas: []bloomshipper.Meta{ - gen(v1.NewBounds(0, 5), 0), - gen(v1.NewBounds(6, 10)), // Meta without sources - gen(v1.NewBounds(0, 10), 1), - gen(v1.NewBounds(11, 15)), // Meta without sources - gen(v1.NewBounds(16, 20), 2), - }, - exp: []bloomshipper.Meta{ - gen(v1.NewBounds(6, 10)), // Meta without sources - gen(v1.NewBounds(11, 15)), // Meta without sources - gen(v1.NewBounds(0, 5), 0), - }, - }, - } { - t.Run(tc.desc, func(t *testing.T) { - outdated, err := outdatedMetas(tc.metas) - require.NoError(t, err) - require.Equal(t, tc.exp, outdated) - }) - } -} diff --git a/pkg/bloomutils/ring.go b/pkg/bloomutils/ring.go deleted file mode 100644 index 9743298e89b4d..0000000000000 --- a/pkg/bloomutils/ring.go +++ /dev/null @@ -1,178 +0,0 @@ -// This file contains a bunch of utility functions for bloom components. - -package bloomutils - -import ( - "errors" - "fmt" - "math" - "sort" - - "github.com/grafana/dskit/ring" - "github.com/prometheus/common/model" - "golang.org/x/exp/constraints" - "golang.org/x/exp/slices" - - v1 "github.com/grafana/loki/v3/pkg/storage/bloom/v1" -) - -var ( - Uint32Range = Range[uint32]{Min: 0, Max: math.MaxUint32} - Uint64Range = Range[uint64]{Min: 0, Max: math.MaxUint64} -) - -type Range[T constraints.Unsigned] struct { - Min, Max T -} - -func (r Range[T]) String() string { - return fmt.Sprintf("%016x-%016x", r.Min, r.Max) -} - -func (r Range[T]) Less(other Range[T]) bool { - if r.Min != other.Min { - return r.Min < other.Min - } - return r.Max <= other.Max -} - -func (r Range[T]) Cmp(t T) v1.BoundsCheck { - if t < r.Min { - return v1.Before - } else if t > r.Max { - return v1.After - } - return v1.Overlap -} - -func NewRange[T constraints.Unsigned](min, max T) Range[T] { - return Range[T]{Min: min, Max: max} -} - -func NewTokenRange(min, max uint32) Range[uint32] { - return Range[uint32]{Min: min, Max: max} -} - -type InstanceWithTokenRange struct { - Instance ring.InstanceDesc - TokenRange Range[uint32] -} - -func (i InstanceWithTokenRange) Cmp(token uint32) v1.BoundsCheck { - return i.TokenRange.Cmp(token) -} - -type InstancesWithTokenRange []InstanceWithTokenRange - -func (i InstancesWithTokenRange) Contains(token uint32) bool { - for _, instance := range i { - if instance.Cmp(token) == v1.Overlap { - return true - } - } - return false -} - -// TODO(owen-d): use https://github.com/grafana/loki/pull/11975 after merge -func KeyspacesFromTokenRanges(tokenRanges ring.TokenRanges) []v1.FingerprintBounds { - keyspaces := make([]v1.FingerprintBounds, 0, len(tokenRanges)/2) - for i := 0; i < len(tokenRanges)-1; i += 2 { - keyspaces = append(keyspaces, v1.FingerprintBounds{ - Min: model.Fingerprint(tokenRanges[i]) << 32, - Max: model.Fingerprint(tokenRanges[i+1])<<32 | model.Fingerprint(math.MaxUint32), - }) - } - return keyspaces -} - -func TokenRangesForInstance(id string, instances []ring.InstanceDesc) (ranges ring.TokenRanges, err error) { - var ownedTokens map[uint32]struct{} - - // lifted from grafana/dskit/ring/model.go <*Desc>.GetTokens() - toks := make([][]uint32, 0, len(instances)) - for _, instance := range instances { - if instance.Id == id { - ranges = make(ring.TokenRanges, 0, 2*(len(instance.Tokens)+1)) - ownedTokens = make(map[uint32]struct{}, len(instance.Tokens)) - for _, tok := range instance.Tokens { - ownedTokens[tok] = struct{}{} - } - } - - // Tokens may not be sorted for an older version which, so we enforce sorting here. - tokens := instance.Tokens - if !sort.IsSorted(ring.Tokens(tokens)) { - sort.Sort(ring.Tokens(tokens)) - } - - toks = append(toks, tokens) - } - - if cap(ranges) == 0 { - return nil, fmt.Errorf("instance %s not found", id) - } - - allTokens := ring.MergeTokens(toks) - if len(allTokens) == 0 { - return nil, errors.New("no tokens in the ring") - } - - // mostly lifted from grafana/dskit/ring/token_range.go <*Ring>.GetTokenRangesForInstance() - - // non-zero value means we're now looking for start of the range. Zero value means we're looking for next end of range (ie. token owned by this instance). - rangeEnd := uint32(0) - - // if this instance claimed the first token, it owns the wrap-around range, which we'll break into two separate ranges - firstToken := allTokens[0] - _, ownsFirstToken := ownedTokens[firstToken] - - if ownsFirstToken { - // we'll start by looking for the beginning of the range that ends with math.MaxUint32 - rangeEnd = math.MaxUint32 - } - - // walk the ring backwards, alternating looking for ends and starts of ranges - for i := len(allTokens) - 1; i > 0; i-- { - token := allTokens[i] - _, owned := ownedTokens[token] - - if rangeEnd == 0 { - // we're looking for the end of the next range - if owned { - rangeEnd = token - 1 - } - } else { - // we have a range end, and are looking for the start of the range - if !owned { - ranges = append(ranges, rangeEnd, token) - rangeEnd = 0 - } - } - } - - // finally look at the first token again - // - if we have a range end, check if we claimed token 0 - // - if we don't, we have our start - // - if we do, the start is 0 - // - if we don't have a range end, check if we claimed token 0 - // - if we don't, do nothing - // - if we do, add the range of [0, token-1] - // - BUT, if the token itself is 0, do nothing, because we don't own the tokens themselves (we should be covered by the already added range that ends with MaxUint32) - - if rangeEnd == 0 { - if ownsFirstToken && firstToken != 0 { - ranges = append(ranges, firstToken-1, 0) - } - } else { - if ownsFirstToken { - ranges = append(ranges, rangeEnd, 0) - } else { - ranges = append(ranges, rangeEnd, firstToken) - } - } - - // Ensure returned ranges are sorted. - slices.Sort(ranges) - - return ranges, nil -} diff --git a/pkg/bloomutils/ring_test.go b/pkg/bloomutils/ring_test.go deleted file mode 100644 index 8a373696c7c92..0000000000000 --- a/pkg/bloomutils/ring_test.go +++ /dev/null @@ -1,48 +0,0 @@ -package bloomutils - -import ( - "fmt" - "math" - "testing" - - "github.com/grafana/dskit/ring" - "github.com/stretchr/testify/require" - - v1 "github.com/grafana/loki/v3/pkg/storage/bloom/v1" -) - -func uint64Range(min, max uint64) Range[uint64] { - return Range[uint64]{min, max} -} - -func TestKeyspacesFromTokenRanges(t *testing.T) { - for i, tc := range []struct { - tokenRanges ring.TokenRanges - exp []v1.FingerprintBounds - }{ - { - tokenRanges: ring.TokenRanges{ - 0, math.MaxUint32 / 2, - math.MaxUint32/2 + 1, math.MaxUint32, - }, - exp: []v1.FingerprintBounds{ - v1.NewBounds(0, math.MaxUint64/2), - v1.NewBounds(math.MaxUint64/2+1, math.MaxUint64), - }, - }, - { - tokenRanges: ring.TokenRanges{ - 0, math.MaxUint8, - math.MaxUint16, math.MaxUint16 << 1, - }, - exp: []v1.FingerprintBounds{ - v1.NewBounds(0, 0xff00000000|math.MaxUint32), - v1.NewBounds(math.MaxUint16<<32, math.MaxUint16<<33|math.MaxUint32), - }, - }, - } { - t.Run(fmt.Sprint(i), func(t *testing.T) { - require.Equal(t, tc.exp, KeyspacesFromTokenRanges(tc.tokenRanges)) - }) - } -} diff --git a/pkg/loki/config_wrapper.go b/pkg/loki/config_wrapper.go index 48deb5151bb57..91b4c329a5c62 100644 --- a/pkg/loki/config_wrapper.go +++ b/pkg/loki/config_wrapper.go @@ -336,20 +336,6 @@ func applyConfigToRings(r, defaults *ConfigWrapper, rc lokiring.RingConfig, merg r.IndexGateway.Ring.ZoneAwarenessEnabled = rc.ZoneAwarenessEnabled r.IndexGateway.Ring.KVStore = rc.KVStore } - - // BloomCompactor - if mergeWithExisting || reflect.DeepEqual(r.BloomCompactor.Ring, defaults.BloomCompactor.Ring) { - r.BloomCompactor.Ring.HeartbeatTimeout = rc.HeartbeatTimeout - r.BloomCompactor.Ring.HeartbeatPeriod = rc.HeartbeatPeriod - r.BloomCompactor.Ring.InstancePort = rc.InstancePort - r.BloomCompactor.Ring.InstanceAddr = rc.InstanceAddr - r.BloomCompactor.Ring.InstanceID = rc.InstanceID - r.BloomCompactor.Ring.InstanceInterfaceNames = rc.InstanceInterfaceNames - r.BloomCompactor.Ring.InstanceZone = rc.InstanceZone - r.BloomCompactor.Ring.ZoneAwarenessEnabled = rc.ZoneAwarenessEnabled - r.BloomCompactor.Ring.KVStore = rc.KVStore - r.BloomCompactor.Ring.NumTokens = rc.NumTokens - } } func applyTokensFilePath(cfg *ConfigWrapper) error { @@ -381,13 +367,6 @@ func applyTokensFilePath(cfg *ConfigWrapper) error { } cfg.IndexGateway.Ring.TokensFilePath = f - // Bloom-Compactor - f, err = tokensFile(cfg, "bloom-compactor.tokens") - if err != nil { - return err - } - cfg.BloomCompactor.Ring.TokensFilePath = f - // Pattern f, err = tokensFile(cfg, "pattern.tokens") if err != nil { @@ -480,10 +459,6 @@ func appendLoopbackInterface(cfg, defaults *ConfigWrapper) { if reflect.DeepEqual(cfg.IndexGateway.Ring.InstanceInterfaceNames, defaults.IndexGateway.Ring.InstanceInterfaceNames) { cfg.IndexGateway.Ring.InstanceInterfaceNames = append(cfg.IndexGateway.Ring.InstanceInterfaceNames, loopbackIface) } - - if reflect.DeepEqual(cfg.BloomCompactor.Ring.InstanceInterfaceNames, defaults.BloomCompactor.Ring.InstanceInterfaceNames) { - cfg.BloomCompactor.Ring.InstanceInterfaceNames = append(cfg.BloomCompactor.Ring.InstanceInterfaceNames, loopbackIface) - } } // applyMemberlistConfig will change the default ingester, distributor, ruler, and query scheduler ring configurations to use memberlist. @@ -498,7 +473,6 @@ func applyMemberlistConfig(r *ConfigWrapper) { r.QueryScheduler.SchedulerRing.KVStore.Store = memberlistStr r.CompactorConfig.CompactorRing.KVStore.Store = memberlistStr r.IndexGateway.Ring.KVStore.Store = memberlistStr - r.BloomCompactor.Ring.KVStore.Store = memberlistStr } var ErrTooManyStorageConfigs = errors.New("too many storage configs provided in the common config, please only define one storage backend") diff --git a/pkg/loki/loki.go b/pkg/loki/loki.go index 01074ddf80416..724676e5987a1 100644 --- a/pkg/loki/loki.go +++ b/pkg/loki/loki.go @@ -32,7 +32,6 @@ import ( "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" compactorclient "github.com/grafana/loki/v3/pkg/compactor/client" @@ -98,7 +97,6 @@ type Config struct { IngesterRF1 ingester_rf1.Config `yaml:"ingester_rf1,omitempty"` 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"` @@ -188,7 +186,6 @@ func (c *Config) RegisterFlags(f *flag.FlagSet) { c.MemberlistKV.RegisterFlags(f) c.Tracing.RegisterFlags(f) c.CompactorConfig.RegisterFlags(f) - c.BloomCompactor.RegisterFlags(f) c.BloomBuild.RegisterFlags(f) c.QueryScheduler.RegisterFlags(f) c.Analytics.RegisterFlags(f) @@ -290,8 +287,8 @@ func (c *Config) Validate() error { if err := c.QueryRange.Validate(); err != nil { errs = append(errs, errors.Wrap(err, "CONFIG ERROR: invalid query_range config")) } - if err := c.BloomCompactor.Validate(); err != nil { - errs = append(errs, errors.Wrap(err, "CONFIG ERROR: invalid bloom_compactor config")) + if err := c.BloomBuild.Validate(); err != nil { + errs = append(errs, errors.Wrap(err, "CONFIG ERROR: invalid bloom_build config")) } if err := c.BloomGateway.Validate(); err != nil { errs = append(errs, errors.Wrap(err, "CONFIG ERROR: invalid bloom_gateway config")) @@ -375,8 +372,6 @@ type Loki struct { querySchedulerRingManager *lokiring.RingManager usageReport *analytics.Reporter indexGatewayRingManager *lokiring.RingManager - bloomCompactorRingManager *lokiring.RingManager - bloomGatewayRingManager *lokiring.RingManager MetastoreClient *metastoreclient.Client ClientMetrics storage.ClientMetrics @@ -692,8 +687,6 @@ func (t *Loki) setupModuleManager() error { mm.RegisterModule(TableManager, t.initTableManager) mm.RegisterModule(Compactor, t.initCompactor) mm.RegisterModule(BloomStore, t.initBloomStore, modules.UserInvisibleModule) - 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) @@ -736,7 +729,6 @@ func (t *Loki) setupModuleManager() error { Compactor: {Server, Overrides, MemberlistKV, Analytics}, 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}, BloomStore: {IndexGatewayRing}, @@ -748,12 +740,11 @@ func (t *Loki) setupModuleManager() error { IngesterQuerier: {Ring}, QuerySchedulerRing: {Overrides, MemberlistKV}, IndexGatewayRing: {Overrides, MemberlistKV}, - BloomCompactorRing: {Overrides, MemberlistKV}, MemberlistKV: {Server}, Read: {QueryFrontend, Querier}, Write: {Ingester, IngesterRF1, Distributor, PatternIngester}, - Backend: {QueryScheduler, Ruler, Compactor, IndexGateway, BloomGateway, BloomCompactor}, + Backend: {QueryScheduler, Ruler, Compactor, IndexGateway, BloomGateway}, All: {QueryScheduler, QueryFrontend, Querier, Ingester, IngesterRF1, PatternIngester, Distributor, Ruler, Compactor, Metastore}, } diff --git a/pkg/loki/modules.go b/pkg/loki/modules.go index 60e2683b599ff..8099e812d1f55 100644 --- a/pkg/loki/modules.go +++ b/pkg/loki/modules.go @@ -37,7 +37,6 @@ import ( "github.com/grafana/loki/v3/pkg/bloombuild/builder" "github.com/grafana/loki/v3/pkg/bloombuild/planner" bloomprotos "github.com/grafana/loki/v3/pkg/bloombuild/protos" - "github.com/grafana/loki/v3/pkg/bloomcompactor" "github.com/grafana/loki/v3/pkg/bloomgateway" "github.com/grafana/loki/v3/pkg/compactor" compactorclient "github.com/grafana/loki/v3/pkg/compactor/client" @@ -133,8 +132,6 @@ const ( IndexGatewayInterceptors string = "index-gateway-interceptors" QueryScheduler string = "query-scheduler" 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" @@ -149,10 +146,9 @@ const ( ) const ( - schedulerRingKey = "scheduler" - indexGatewayRingKey = "index-gateway" - bloomGatewayRingKey = "bloom-gateway" - bloomCompactorRingKey = "bloom-compactor" + schedulerRingKey = "scheduler" + indexGatewayRingKey = "index-gateway" + bloomGatewayRingKey = "bloom-gateway" ) func (t *Loki) initServer() (services.Service, error) { @@ -294,7 +290,6 @@ func (t *Loki) initRuntimeConfig() (services.Service, error) { // By doing the initialization here instead of per-module init function, we avoid the problem // of projects based on Loki forgetting the wiring if they override module's init method (they also don't have access to private symbols). t.Cfg.CompactorConfig.CompactorRing.KVStore.Multi.ConfigProvider = multiClientRuntimeConfigChannel(t.runtimeConfig) - t.Cfg.BloomCompactor.Ring.KVStore.Multi.ConfigProvider = multiClientRuntimeConfigChannel(t.runtimeConfig) t.Cfg.Distributor.DistributorRing.KVStore.Multi.ConfigProvider = multiClientRuntimeConfigChannel(t.runtimeConfig) t.Cfg.IndexGateway.Ring.KVStore.Multi.ConfigProvider = multiClientRuntimeConfigChannel(t.runtimeConfig) t.Cfg.Ingester.LifecyclerConfig.RingConfig.KVStore.Multi.ConfigProvider = multiClientRuntimeConfigChannel(t.runtimeConfig) @@ -953,7 +948,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(IngesterRF1), 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): + case t.Cfg.isTarget(IngesterRF1), t.Cfg.isTarget(Querier), t.Cfg.isTarget(Ruler), t.Cfg.isTarget(Read), t.Cfg.isTarget(Backend), t.isModuleActive(IndexGateway), 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 @@ -1457,7 +1452,6 @@ func (t *Loki) initMemberlistKV() (services.Service, error) { t.Cfg.Ingester.LifecyclerConfig.RingConfig.KVStore.MemberlistKV = t.MemberlistKV.GetMemberlistKV t.Cfg.QueryScheduler.SchedulerRing.KVStore.MemberlistKV = t.MemberlistKV.GetMemberlistKV t.Cfg.Ruler.Ring.KVStore.MemberlistKV = t.MemberlistKV.GetMemberlistKV - t.Cfg.BloomCompactor.Ring.KVStore.MemberlistKV = t.MemberlistKV.GetMemberlistKV t.Cfg.Pattern.LifecyclerConfig.RingConfig.KVStore.MemberlistKV = t.MemberlistKV.GetMemberlistKV t.Cfg.IngesterRF1.LifecyclerConfig.RingConfig.KVStore.MemberlistKV = t.MemberlistKV.GetMemberlistKV t.Server.HTTP.Handle("/memberlist", t.MemberlistKV) @@ -1597,7 +1591,7 @@ func (t *Loki) initIndexGateway() (services.Service, error) { } resolver := bloomgateway.NewBlockResolver(t.BloomStore, logger) querierCfg := bloomgateway.QuerierConfig{ - MinTableOffset: t.Cfg.BloomCompactor.MinTableOffset, + MinTableOffset: t.Cfg.BloomBuild.Planner.MinTableOffset, } bloomQuerier = bloomgateway.NewQuerier(bloomGatewayClient, querierCfg, t.Overrides, resolver, prometheus.DefaultRegisterer, logger) } @@ -1658,52 +1652,6 @@ func (t *Loki) initIndexGatewayInterceptors() (services.Service, error) { return nil, nil } -func (t *Loki) initBloomCompactor() (services.Service, error) { - if !t.Cfg.BloomCompactor.Enabled { - return nil, nil - } - logger := log.With(util_log.Logger, "component", "bloom-compactor") - - return bloomcompactor.New( - t.Cfg.BloomCompactor, - t.Cfg.SchemaConfig, - t.Cfg.StorageConfig, - t.ClientMetrics, - t.Store, - t.bloomCompactorRingManager.Ring, - t.bloomCompactorRingManager.RingLifecycler, - t.Overrides, - t.BloomStore, - logger, - prometheus.DefaultRegisterer, - ) -} - -func (t *Loki) initBloomCompactorRing() (services.Service, error) { - if !t.Cfg.BloomCompactor.Enabled { - return nil, nil - } - t.Cfg.BloomCompactor.Ring.ListenPort = t.Cfg.Server.GRPCListenPort - - // is LegacyMode needed? - // legacyReadMode := t.Cfg.LegacyReadTarget && t.isModuleActive(Read) - - rm, err := lokiring.NewRingManager(bloomCompactorRingKey, lokiring.ServerMode, t.Cfg.BloomCompactor.Ring, 1, t.Cfg.BloomCompactor.Ring.NumTokens, util_log.Logger, prometheus.DefaultRegisterer) - if err != nil { - return nil, gerrors.Wrap(err, "error initializing bloom-compactor ring manager") - } - - t.bloomCompactorRingManager = rm - - t.Server.HTTP.Path("/bloomcompactor/ring").Methods("GET", "POST").Handler(t.bloomCompactorRingManager) - - if t.Cfg.InternalServer.Enable { - t.InternalServer.HTTP.Path("/bloomcompactor/ring").Methods("GET", "POST").Handler(t.bloomCompactorRingManager) - } - - return t.bloomCompactorRingManager, nil -} - func (t *Loki) initBloomPlanner() (services.Service, error) { if !t.Cfg.BloomBuild.Enabled { return nil, nil diff --git a/pkg/loki/modules_test.go b/pkg/loki/modules_test.go index 8c27c851d33e0..64241443d3439 100644 --- a/pkg/loki/modules_test.go +++ b/pkg/loki/modules_test.go @@ -410,7 +410,6 @@ func minimalWorkingConfig(t *testing.T, dir, target string, cfgTransformers ...f cfg.Distributor.DistributorRing.InstanceAddr = localhost cfg.IndexGateway.Mode = indexgateway.SimpleMode cfg.IndexGateway.Ring.InstanceAddr = localhost - cfg.BloomCompactor.Ring.InstanceAddr = localhost cfg.CompactorConfig.CompactorRing.InstanceAddr = localhost cfg.CompactorConfig.WorkingDirectory = filepath.Join(dir, "compactor") diff --git a/pkg/storage/bloom/v1/block.go b/pkg/storage/bloom/v1/block.go index 5ad4adbe7a0d6..863bdf12a9835 100644 --- a/pkg/storage/bloom/v1/block.go +++ b/pkg/storage/bloom/v1/block.go @@ -116,7 +116,7 @@ type BlockQuerier struct { // whether the underlying byte slice of the bloom page will be returned to the // pool for efficiency or not. Returning to the pool can only safely be used // when the underlying bloom bytes don't escape the decoder, i.e. when loading -// blooms for querying (bloom-gateway), but not for writing (bloom-compactor). +// blooms for querying (bloom-gateway), but not for writing (bloom-builder). // Therefore, when calling NewBlockQuerier on the write path, you should always // pass the SimpleHeapAllocator implementation of the Allocator interface. func NewBlockQuerier(b *Block, alloc mempool.Allocator, maxPageSize int) *BlockQuerier { diff --git a/pkg/storage/bloom/v1/bloom.go b/pkg/storage/bloom/v1/bloom.go index cf7053b07308a..49f1e06e36748 100644 --- a/pkg/storage/bloom/v1/bloom.go +++ b/pkg/storage/bloom/v1/bloom.go @@ -167,7 +167,7 @@ type BloomPageDecoder struct { // perf optimization. // This can only safely be used when the underlying bloom // bytes don't escape the decoder: -// on reads in the bloom-gw but not in the bloom-compactor +// on reads in the bloom-gw but not in the bloom-builder func (d *BloomPageDecoder) Relinquish(alloc mempool.Allocator) { if d == nil { return diff --git a/pkg/util/limiter/combined_limits.go b/pkg/util/limiter/combined_limits.go index 92caf2c19d681..3ea2a230634e8 100644 --- a/pkg/util/limiter/combined_limits.go +++ b/pkg/util/limiter/combined_limits.go @@ -3,7 +3,6 @@ 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" "github.com/grafana/loki/v3/pkg/distributor" @@ -27,7 +26,6 @@ type CombinedLimits interface { storage.StoreLimits indexgateway.Limits bloomgateway.Limits - bloomcompactor.Limits bloomplanner.Limits bloombuilder.Limits } diff --git a/pkg/validation/exporter_test.go b/pkg/validation/exporter_test.go index 59b4537533e53..45484bbc13b08 100644 --- a/pkg/validation/exporter_test.go +++ b/pkg/validation/exporter_test.go @@ -35,8 +35,8 @@ func TestOverridesExporter_noConfig(t *testing.T) { func TestOverridesExporter_withConfig(t *testing.T) { tenantLimits := map[string]*Limits{ "tenant-a": { - MaxQueriersPerTenant: 5, - BloomCompactorEnabled: true, + MaxQueriersPerTenant: 5, + BloomCreationEnabled: true, }, } overrides, _ := NewOverrides(Limits{}, newMockTenantLimits(tenantLimits)) diff --git a/pkg/validation/limits.go b/pkg/validation/limits.go index 1ae6ce3a47c63..3e44dc2047773 100644 --- a/pkg/validation/limits.go +++ b/pkg/validation/limits.go @@ -59,8 +59,8 @@ const ( defaultMaxStructuredMetadataSize = "64kb" defaultMaxStructuredMetadataCount = 128 - defaultBloomCompactorMaxBlockSize = "200MB" - defaultBloomCompactorMaxBloomSize = "128MB" + defaultBloomBuildMaxBlockSize = "200MB" + defaultBloomBuildMaxBloomSize = "128MB" defaultBlockedIngestionStatusCode = 260 // 260 is a custom status code to indicate blocked ingestion ) @@ -203,21 +203,19 @@ type Limits struct { BloomGatewayEnabled bool `yaml:"bloom_gateway_enable_filtering" json:"bloom_gateway_enable_filtering" category:"experimental"` BloomGatewayCacheKeyInterval time.Duration `yaml:"bloom_gateway_cache_key_interval" json:"bloom_gateway_cache_key_interval" category:"experimental"` - BloomCompactorShardSize int `yaml:"bloom_compactor_shard_size" json:"bloom_compactor_shard_size" category:"experimental"` - BloomCompactorEnabled bool `yaml:"bloom_compactor_enable_compaction" json:"bloom_compactor_enable_compaction" category:"experimental"` - 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"` + BloomBuildMaxBuilders int `yaml:"bloom_build_max_builders" json:"bloom_build_max_builders" category:"experimental"` + BloomBuildTaskMaxRetries int `yaml:"bloom_build_task_max_retries" json:"bloom_build_task_max_retries" category:"experimental"` + BloomBuilderResponseTimeout time.Duration `yaml:"bloom_build_builder_response_timeout" json:"bloom_build_builder_response_timeout" 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"` - BuilderResponseTimeout time.Duration `yaml:"bloom_build_builder_response_timeout" json:"bloom_build_builder_response_timeout" category:"experimental"` - BloomTaskMaxRetries int `yaml:"bloom_build_task_max_retries" json:"bloom_build_task_max_retries" 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"` + 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"` + BloomBlockEncoding string `yaml:"bloom_block_encoding" json:"bloom_block_encoding" 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"` - BloomBlockEncoding string `yaml:"bloom_block_encoding" json:"bloom_block_encoding" category:"experimental"` + BloomMaxBlockSize flagext.ByteSize `yaml:"bloom_max_block_size" json:"bloom_max_block_size" category:"experimental"` + BloomMaxBloomSize flagext.ByteSize `yaml:"bloom_max_bloom_size" json:"bloom_max_bloom_size" category:"experimental"` AllowStructuredMetadata bool `yaml:"allow_structured_metadata,omitempty" json:"allow_structured_metadata,omitempty" doc:"description=Allow user to send structured metadata in push payload."` MaxStructuredMetadataSize flagext.ByteSize `yaml:"max_structured_metadata_size" json:"max_structured_metadata_size" doc:"description=Maximum size accepted for structured metadata per log line."` @@ -377,33 +375,32 @@ func (l *Limits) RegisterFlags(f *flag.FlagSet) { f.IntVar(&l.BloomGatewayShardSize, "bloom-gateway.shard-size", 0, "Experimental. The shard size defines how many bloom gateways should be used by a tenant for querying.") f.BoolVar(&l.BloomGatewayEnabled, "bloom-gateway.enable-filtering", false, "Experimental. Whether to use the bloom gateway component in the read path to filter chunks.") - - f.IntVar(&l.BloomCompactorShardSize, "bloom-compactor.shard-size", 0, "Experimental. The shard size defines how many bloom compactors should be used by a tenant when computing blooms. If it's set to 0, shuffle sharding is disabled.") - f.BoolVar(&l.BloomCompactorEnabled, "bloom-compactor.enable-compaction", false, "Experimental. Whether to compact chunks into bloom filters.") - f.IntVar(&l.BloomNGramLength, "bloom-compactor.ngram-length", 4, "Experimental. Length of the n-grams created when computing blooms from log lines.") - f.IntVar(&l.BloomNGramSkip, "bloom-compactor.ngram-skip", 1, "Experimental. Skip factor for the n-grams created when computing blooms from log lines.") - f.Float64Var(&l.BloomFalsePositiveRate, "bloom-compactor.false-positive-rate", 0.01, "Experimental. Scalable Bloom Filter desired false-positive rate.") - f.StringVar(&l.BloomBlockEncoding, "bloom-compactor.block-encoding", "none", "Experimental. Compression algorithm for bloom block pages.") f.DurationVar(&l.BloomGatewayCacheKeyInterval, "bloom-gateway.cache-key-interval", 15*time.Minute, "Experimental. Interval for computing the cache key in the Bloom Gateway.") - _ = l.BloomCompactorMaxBlockSize.Set(defaultBloomCompactorMaxBlockSize) - f.Var(&l.BloomCompactorMaxBlockSize, "bloom-compactor.max-block-size", + + f.IntVar(&l.BloomNGramLength, "bloom-build.ngram-length", 4, "Experimental. Length of the n-grams created when computing blooms from log lines.") + f.IntVar(&l.BloomNGramSkip, "bloom-build.ngram-skip", 1, "Experimental. Skip factor for the n-grams created when computing blooms from log lines.") + f.Float64Var(&l.BloomFalsePositiveRate, "bloom-build.false-positive-rate", 0.01, "Experimental. Scalable Bloom Filter desired false-positive rate.") + f.StringVar(&l.BloomBlockEncoding, "bloom-build.block-encoding", "none", "Experimental. Compression algorithm for bloom block pages.") + + _ = l.BloomMaxBlockSize.Set(defaultBloomBuildMaxBlockSize) + f.Var(&l.BloomMaxBlockSize, "bloom-build.max-block-size", fmt.Sprintf( "Experimental. The maximum bloom block size. A value of 0 sets an unlimited size. Default is %s. The actual block size might exceed this limit since blooms will be added to blocks until the block exceeds the maximum block size.", - defaultBloomCompactorMaxBlockSize, + defaultBloomBuildMaxBlockSize, ), ) 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.") - f.DurationVar(&l.BuilderResponseTimeout, "bloom-build.builder-response-timeout", 0, "Experimental. Timeout for a builder to finish a task. If a builder does not respond within this time, it is considered failed and the task will be requeued. 0 disables the timeout.") - f.IntVar(&l.BloomTaskMaxRetries, "bloom-build.task-max-retries", 3, "Experimental. Maximum number of retries for a failed task. If a task fails more than this number of times, it is considered failed and will not be retried. A value of 0 disables this limit.") + f.DurationVar(&l.BloomBuilderResponseTimeout, "bloom-build.builder-response-timeout", 0, "Experimental. Timeout for a builder to finish a task. If a builder does not respond within this time, it is considered failed and the task will be requeued. 0 disables the timeout.") + f.IntVar(&l.BloomBuildTaskMaxRetries, "bloom-build.task-max-retries", 3, "Experimental. Maximum number of retries for a failed task. If a task fails more than this number of times, it is considered failed and will not be retried. A value of 0 disables this limit.") - _ = l.BloomCompactorMaxBloomSize.Set(defaultBloomCompactorMaxBloomSize) - f.Var(&l.BloomCompactorMaxBloomSize, "bloom-compactor.max-bloom-size", + _ = l.BloomMaxBloomSize.Set(defaultBloomBuildMaxBloomSize) + f.Var(&l.BloomMaxBloomSize, "bloom-build.max-bloom-size", fmt.Sprintf( "Experimental. The maximum bloom size per log stream. A log stream whose generated bloom filter exceeds this size will be discarded. A value of 0 sets an unlimited size. Default is %s.", - defaultBloomCompactorMaxBloomSize, + defaultBloomBuildMaxBloomSize, ), ) @@ -991,14 +988,6 @@ func (o *Overrides) BloomGatewayEnabled(userID string) bool { return o.getOverridesForUser(userID).BloomGatewayEnabled } -func (o *Overrides) BloomCompactorShardSize(userID string) int { - return o.getOverridesForUser(userID).BloomCompactorShardSize -} - -func (o *Overrides) BloomCompactorEnabled(userID string) bool { - return o.getOverridesForUser(userID).BloomCompactorEnabled -} - func (o *Overrides) BloomCreationEnabled(userID string) bool { return o.getOverridesForUser(userID).BloomCreationEnabled } @@ -1012,11 +1001,11 @@ func (o *Overrides) BloomBuildMaxBuilders(userID string) int { } func (o *Overrides) BuilderResponseTimeout(userID string) time.Duration { - return o.getOverridesForUser(userID).BuilderResponseTimeout + return o.getOverridesForUser(userID).BloomBuilderResponseTimeout } func (o *Overrides) BloomTaskMaxRetries(userID string) int { - return o.getOverridesForUser(userID).BloomTaskMaxRetries + return o.getOverridesForUser(userID).BloomBuildTaskMaxRetries } func (o *Overrides) BloomNGramLength(userID string) int { @@ -1027,12 +1016,12 @@ func (o *Overrides) BloomNGramSkip(userID string) int { return o.getOverridesForUser(userID).BloomNGramSkip } -func (o *Overrides) BloomCompactorMaxBlockSize(userID string) int { - return o.getOverridesForUser(userID).BloomCompactorMaxBlockSize.Val() +func (o *Overrides) BloomMaxBlockSize(userID string) int { + return o.getOverridesForUser(userID).BloomMaxBlockSize.Val() } -func (o *Overrides) BloomCompactorMaxBloomSize(userID string) int { - return o.getOverridesForUser(userID).BloomCompactorMaxBloomSize.Val() +func (o *Overrides) BloomMaxBloomSize(userID string) int { + return o.getOverridesForUser(userID).BloomMaxBloomSize.Val() } func (o *Overrides) BloomFalsePositiveRate(userID string) float64 { diff --git a/tools/doc-generator/parse/root_blocks.go b/tools/doc-generator/parse/root_blocks.go index 85e81705848cc..1bfcc57bc8965 100644 --- a/tools/doc-generator/parse/root_blocks.go +++ b/tools/doc-generator/parse/root_blocks.go @@ -15,7 +15,7 @@ import ( "golang.org/x/exp/slices" "github.com/grafana/loki/v3/pkg/analytics" - "github.com/grafana/loki/v3/pkg/bloomcompactor" + "github.com/grafana/loki/v3/pkg/bloombuild" "github.com/grafana/loki/v3/pkg/bloomgateway" "github.com/grafana/loki/v3/pkg/compactor" "github.com/grafana/loki/v3/pkg/distributor" @@ -123,16 +123,16 @@ var ( StructType: []reflect.Type{reflect.TypeOf(compactor.Config{})}, Desc: "The compactor block configures the compactor component, which compacts index shards for performance.", }, - { - Name: "bloom_compactor", - StructType: []reflect.Type{reflect.TypeOf(bloomcompactor.Config{})}, - Desc: "Experimental: The bloom_compactor block configures the Loki bloom compactor server, responsible for compacting stream indexes into bloom filters and merging them as bloom blocks.", - }, { Name: "bloom_gateway", StructType: []reflect.Type{reflect.TypeOf(bloomgateway.Config{})}, Desc: "Experimental: The bloom_gateway block configures the Loki bloom gateway server, responsible for serving queries for filtering chunks based on filter expressions.", }, + { + Name: "bloom_build", + StructType: []reflect.Type{reflect.TypeOf(bloombuild.Config{})}, + Desc: "Experimental: The bloom_build block configures the Loki bloom planner and builder servers, responsible for building bloom filters.", + }, { Name: "limits_config", StructType: []reflect.Type{reflect.TypeOf(validation.Limits{})},