diff --git a/.github/workflows/checks.yml b/.github/workflows/checks.yml new file mode 100644 index 000000000000..987df91c8c3e --- /dev/null +++ b/.github/workflows/checks.yml @@ -0,0 +1,20 @@ +name: Checks +on: [push] +jobs: + checks: + runs-on: ubuntu-latest + env: + BUILD_IN_CONTAINER: false + container: + image: grafana/loki-build-image:0.32.0 + steps: + - uses: actions/checkout@v4 + - run: git config --global --add safe.directory "$GITHUB_WORKSPACE" + - run: make lint + - run: make check-doc + - run: make check-mod + - run: make validate-example-configs + - run: make check-example-config-doc + - run: make check-drone-drift + - run: make check-generated-files + - run: make test diff --git a/CHANGELOG.md b/CHANGELOG.md index bd2ae7c12de5..4002d76c032e 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -42,6 +42,7 @@ * [10956](https://github.com/grafana/loki/pull/10956) **jeschkies** do not wrap requests but send pure Protobuf from frontend v2 via scheduler to querier when `-frontend.encoding=protobuf`. * [10417](https://github.com/grafana/loki/pull/10417) **jeschkies** shard `quantile_over_time` range queries using probabilistic data structures. * [11284](https://github.com/grafana/loki/pull/11284) **ashwanthgoli** Config: Adds `frontend.max-query-capacity` to tune per-tenant query capacity. +* [11539](https://github.com/grafana/loki/pull/11539) **kaviraj,ashwanthgoli** Support caching /series and /labels query results * [11545](https://github.com/grafana/loki/pull/11545) **dannykopping** Force correct memcached timeout when fetching chunks. ##### Fixes diff --git a/cmd/loki/loki-local-with-memcached.yaml b/cmd/loki/loki-local-with-memcached.yaml new file mode 100644 index 000000000000..d1b0ae1c2493 --- /dev/null +++ b/cmd/loki/loki-local-with-memcached.yaml @@ -0,0 +1,87 @@ +auth_enabled: false + +server: + http_listen_port: 3100 + grpc_listen_port: 9096 + +common: + instance_addr: 127.0.0.1 + path_prefix: /tmp/loki + storage: + filesystem: + chunks_directory: /tmp/loki/chunks + rules_directory: /tmp/loki/rules + replication_factor: 1 + ring: + kvstore: + store: inmemory + +query_range: + align_queries_with_step: true + cache_index_stats_results: true + cache_results: true + cache_volume_results: true + cache_series_results: true + series_results_cache: + cache: + default_validity: 12h + memcached_client: + consistent_hash: true + addresses: "dns+localhost:11211" + max_idle_conns: 16 + timeout: 500ms + update_interval: 1m + index_stats_results_cache: + cache: + default_validity: 12h + memcached_client: + consistent_hash: true + addresses: "dns+localhost:11211" + max_idle_conns: 16 + timeout: 500ms + update_interval: 1m + max_retries: 5 + results_cache: + cache: + default_validity: 12h + memcached_client: + consistent_hash: true + addresses: "dns+localhost:11211" + max_idle_conns: 16 + timeout: 500ms + update_interval: 1m + volume_results_cache: + cache: + default_validity: 12h + memcached_client: + consistent_hash: true + addresses: "dns+localhost:11211" + max_idle_conns: 16 + timeout: 500ms + update_interval: 1m + +schema_config: + configs: + - from: 2020-10-24 + store: tsdb + object_store: filesystem + schema: v12 + index: + prefix: index_ + period: 24h + +ruler: + alertmanager_url: http://localhost:9093 + +# By default, Loki will send anonymous, but uniquely-identifiable usage and configuration +# analytics to Grafana Labs. These statistics are sent to https://stats.grafana.org/ +# +# Statistics help us better understand how Loki is used, and they show us performance +# levels for most users. This helps us prioritize features and documentation. +# For more information on what's sent, look at +# https://github.com/grafana/loki/blob/main/pkg/analytics/stats.go +# Refer to the buildReport method to see what goes into a report. +# +# If you would like to disable reporting, uncomment the following lines: +#analytics: +# reporting_enabled: false diff --git a/docs/sources/configure/_index.md b/docs/sources/configure/_index.md index 2b170dd8db69..e2185c19474f 100644 --- a/docs/sources/configure/_index.md +++ b/docs/sources/configure/_index.md @@ -880,6 +880,40 @@ volume_results_cache: # compression. Supported values are: 'snappy' and ''. # CLI flag: -frontend.volume-results-cache.compression [compression: | default = ""] + +# Cache series query results. +# CLI flag: -querier.cache-series-results +[cache_series_results: | default = false] + +# If series_results_cache is not configured and cache_series_results is true, +# the config for the results cache is used. +series_results_cache: + # The cache block configures the cache backend. + # The CLI flags prefix for this block configuration is: + # frontend.series-results-cache + [cache: ] + + # Use compression in cache. The default is an empty value '', which disables + # compression. Supported values are: 'snappy' and ''. + # CLI flag: -frontend.series-results-cache.compression + [compression: | default = ""] + +# Cache label query results. +# CLI flag: -querier.cache-label-results +[cache_label_results: | default = false] + +# If label_results_cache is not configured and cache_label_results is true, the +# config for the results cache is used. +label_results_cache: + # The cache block configures the cache backend. + # The CLI flags prefix for this block configuration is: + # frontend.label-results-cache + [cache: ] + + # Use compression in cache. The default is an empty value '', which disables + # compression. Supported values are: 'snappy' and ''. + # CLI flag: -frontend.label-results-cache.compression + [compression: | default = ""] ``` ### ruler @@ -2844,6 +2878,12 @@ The `limits_config` block configures global and per-tenant limits in Loki. # CLI flag: -querier.split-queries-by-interval [split_queries_by_interval: | default = 1h] +# Split metadata queries by a time interval and execute in parallel. The value 0 +# disables splitting metadata queries by time. This also determines how cache +# keys are chosen when label/series result caching is enabled. +# CLI flag: -querier.split-metadata-queries-by-interval +[split_metadata_queries_by_interval: | default = 1d] + # Limit queries that can be sharded. Queries within the time range of now and # now minus this sharding lookback are not sharded. The default value of 0s # disables the lookback, causing sharding of all queries at all times. @@ -4276,6 +4316,8 @@ The cache block configures the cache backend. The supported CLI flags `` - `bloom-gateway-client.cache` - `frontend` - `frontend.index-stats-results-cache` +- `frontend.label-results-cache` +- `frontend.series-results-cache` - `frontend.volume-results-cache` - `store.chunks-cache` - `store.index-cache-read` diff --git a/docs/sources/send-data/promtail/cloud/ecs/_index.md b/docs/sources/send-data/promtail/cloud/ecs/_index.md index 90682f265ded..e9eaca48f52a 100644 --- a/docs/sources/send-data/promtail/cloud/ecs/_index.md +++ b/docs/sources/send-data/promtail/cloud/ecs/_index.md @@ -88,13 +88,13 @@ Our [task definition][task] will be made of two containers, the [Firelens][Firel Let's download the task definition, we'll go through the most important parts. ```bash -curl https://raw.githubusercontent.com/grafana/loki/main/docs/sources/clients/aws/ecs/ecs-task.json > ecs-task.json +curl https://raw.githubusercontent.com/grafana/loki/main/docs/sources/send-data/promtail/cloud/ecs/ecs-task.json > ecs-task.json ``` ```json { "essential": true, - "image": "grafana/fluent-bit-plugin-loki:2.0.0-amd64", + "image": "grafana/fluent-bit-plugin-loki:2.9.3-amd64", "name": "log_router", "firelensConfiguration": { "type": "fluentbit", diff --git a/docs/sources/send-data/promtail/cloud/ecs/ecs-task.json b/docs/sources/send-data/promtail/cloud/ecs/ecs-task.json index d9f7b9b8d670..dd2db2251825 100644 --- a/docs/sources/send-data/promtail/cloud/ecs/ecs-task.json +++ b/docs/sources/send-data/promtail/cloud/ecs/ecs-task.json @@ -2,7 +2,7 @@ "containerDefinitions": [ { "essential": true, - "image": "grafana/fluent-bit-plugin-loki:1.6.0-amd64", + "image": "grafana/fluent-bit-plugin-loki:2.9.3-amd64", "name": "log_router", "firelensConfiguration": { "type": "fluentbit", diff --git a/pkg/bloomcompactor/bloomcompactor.go b/pkg/bloomcompactor/bloomcompactor.go index c8d4ba49ba3c..06cdf5c6f93a 100644 --- a/pkg/bloomcompactor/bloomcompactor.go +++ b/pkg/bloomcompactor/bloomcompactor.go @@ -465,10 +465,10 @@ func (c *Compactor) runCompact(ctx context.Context, logger log.Logger, job Job, } metaSearchParams := bloomshipper.MetaSearchParams{ TenantID: job.tenantID, - MinFingerprint: uint64(job.minFp), - MaxFingerprint: uint64(job.maxFp), - StartTimestamp: int64(job.from), - EndTimestamp: int64(job.through), + MinFingerprint: job.minFp, + MaxFingerprint: job.maxFp, + StartTimestamp: job.from, + EndTimestamp: job.through, } var metas []bloomshipper.Meta //TODO Configure pool for these to avoid allocations diff --git a/pkg/bloomcompactor/chunkcompactor.go b/pkg/bloomcompactor/chunkcompactor.go index a949f26452d9..744a38b1ad5a 100644 --- a/pkg/bloomcompactor/chunkcompactor.go +++ b/pkg/bloomcompactor/chunkcompactor.go @@ -135,8 +135,8 @@ func buildBlockFromBlooms( TableName: job.tableName, MinFingerprint: uint64(job.minFp), MaxFingerprint: uint64(job.maxFp), - StartTimestamp: int64(job.from), - EndTimestamp: int64(job.through), + StartTimestamp: job.from, + EndTimestamp: job.through, Checksum: checksum, }, IndexPath: job.indexPath, @@ -148,7 +148,7 @@ func buildBlockFromBlooms( } func createLocalDirName(workingDir string, job Job) string { - dir := fmt.Sprintf("bloomBlock-%s-%s-%s-%s-%s-%s", job.tableName, job.tenantID, job.minFp, job.maxFp, job.from, job.through) + dir := fmt.Sprintf("bloomBlock-%s-%s-%s-%s-%d-%d", job.tableName, job.tenantID, job.minFp, job.maxFp, job.from, job.through) return filepath.Join(workingDir, dir) } diff --git a/pkg/bloomcompactor/chunkcompactor_test.go b/pkg/bloomcompactor/chunkcompactor_test.go index 4d19f24417d4..a89e4e967a1d 100644 --- a/pkg/bloomcompactor/chunkcompactor_test.go +++ b/pkg/bloomcompactor/chunkcompactor_test.go @@ -121,8 +121,8 @@ func TestChunkCompactor_CompactNewChunks(t *testing.T) { require.Equal(t, job.tableName, compactedBlock.TableName) require.Equal(t, uint64(fp1), compactedBlock.MinFingerprint) require.Equal(t, uint64(fp2), compactedBlock.MaxFingerprint) - require.Equal(t, chunkRef1.MinTime, compactedBlock.StartTimestamp) - require.Equal(t, chunkRef2.MaxTime, compactedBlock.EndTimestamp) + require.Equal(t, model.Time(chunkRef1.MinTime), compactedBlock.StartTimestamp) + require.Equal(t, model.Time(chunkRef2.MaxTime), compactedBlock.EndTimestamp) require.Equal(t, indexPath, compactedBlock.IndexPath) } diff --git a/pkg/bloomcompactor/mergecompactor.go b/pkg/bloomcompactor/mergecompactor.go index 94682579ac9e..0cf55cef86a7 100644 --- a/pkg/bloomcompactor/mergecompactor.go +++ b/pkg/bloomcompactor/mergecompactor.go @@ -137,8 +137,8 @@ func mergeCompactChunks(logger log.Logger, TableName: job.tableName, MinFingerprint: uint64(job.minFp), MaxFingerprint: uint64(job.maxFp), - StartTimestamp: int64(job.from), - EndTimestamp: int64(job.through), + StartTimestamp: job.from, + EndTimestamp: job.through, Checksum: checksum, }, IndexPath: job.indexPath, diff --git a/pkg/bloomgateway/bloomgateway_test.go b/pkg/bloomgateway/bloomgateway_test.go index f24b8bc8a4e2..b34e3d55852a 100644 --- a/pkg/bloomgateway/bloomgateway_test.go +++ b/pkg/bloomgateway/bloomgateway_test.go @@ -75,7 +75,13 @@ func TestBloomGateway_StartStopService(t *testing.T) { t.Cleanup(cm.Unregister) p := config.PeriodConfig{ - From: parseDayTime("2023-09-01"), + From: parseDayTime("2023-09-01"), + IndexTables: config.IndexPeriodicTableConfig{ + PeriodicTableConfig: config.PeriodicTableConfig{ + Prefix: "index_", + Period: 24 * time.Hour, + }, + }, IndexType: config.TSDBType, ObjectType: config.StorageTypeFileSystem, Schema: "v13", @@ -137,7 +143,13 @@ func TestBloomGateway_FilterChunkRefs(t *testing.T) { t.Cleanup(cm.Unregister) p := config.PeriodConfig{ - From: parseDayTime("2023-09-01"), + From: parseDayTime("2023-09-01"), + IndexTables: config.IndexPeriodicTableConfig{ + PeriodicTableConfig: config.PeriodicTableConfig{ + Prefix: "index_", + Period: 24 * time.Hour, + }, + }, IndexType: config.TSDBType, ObjectType: config.StorageTypeFileSystem, Schema: "v13", diff --git a/pkg/distributor/distributor.go b/pkg/distributor/distributor.go index 98dde0b91515..a5229b0ca149 100644 --- a/pkg/distributor/distributor.go +++ b/pkg/distributor/distributor.go @@ -420,7 +420,7 @@ func (d *Distributor) Push(ctx context.Context, req *logproto.PushRequest) (*log // Nil check for performance reasons, to avoid dynamic lookup and/or no-op // function calls that cannot be inlined. if d.tee != nil { - d.tee.Duplicate(streams) + d.tee.Duplicate(tenantID, streams) } const maxExpectedReplicationSet = 5 // typical replication factor 3 plus one for inactive plus one for luck diff --git a/pkg/distributor/distributor_test.go b/pkg/distributor/distributor_test.go index 5a03fe98e94c..71830b4be4d2 100644 --- a/pkg/distributor/distributor_test.go +++ b/pkg/distributor/distributor_test.go @@ -1251,12 +1251,14 @@ func (s *fakeRateStore) RateFor(_ string, _ uint64) (int64, float64) { type mockTee struct { mu sync.Mutex duplicated [][]KeyedStream + tenant string } -func (mt *mockTee) Duplicate(streams []KeyedStream) { +func (mt *mockTee) Duplicate(tenant string, streams []KeyedStream) { mt.mu.Lock() defer mt.mu.Unlock() mt.duplicated = append(mt.duplicated, streams) + mt.tenant = tenant } func TestDistributorTee(t *testing.T) { @@ -1307,5 +1309,7 @@ func TestDistributorTee(t *testing.T) { for j, streams := range td.Streams { assert.Equal(t, tee.duplicated[i][j].Stream.Entries, streams.Entries) } + + require.Equal(t, "test", tee.tenant) } } diff --git a/pkg/distributor/tee.go b/pkg/distributor/tee.go index 9ac48083956e..460f9622b2ea 100644 --- a/pkg/distributor/tee.go +++ b/pkg/distributor/tee.go @@ -1,6 +1,6 @@ package distributor -// Tee imlpementations can duplicate the log streams to another endpoint. +// Tee implementations can duplicate the log streams to another endpoint. type Tee interface { - Duplicate([]KeyedStream) + Duplicate(tenant string, streams []KeyedStream) } diff --git a/pkg/logql/downstream.go b/pkg/logql/downstream.go index 27cb3e849fa2..b7d37390d11f 100644 --- a/pkg/logql/downstream.go +++ b/pkg/logql/downstream.go @@ -235,6 +235,12 @@ type DownstreamQuery struct { Params Params } +type Resp struct { + I int + Res logqlmodel.Result + Err error +} + // Downstreamer is an interface for deferring responsibility for query execution. // It is decoupled from but consumed by a downStreamEvaluator to dispatch ASTs. type Downstreamer interface { @@ -375,24 +381,18 @@ func (ev *DownstreamEvaluator) NewStepEvaluator( results, err := ev.Downstream(ctx, queries) if err != nil { - return nil, fmt.Errorf("error running quantile sketch downstream query: %w", err) + return nil, err } - xs := make([]StepEvaluator, 0, len(queries)) - for _, res := range results { - if res.Data.Type() != QuantileSketchMatrixType { - return nil, fmt.Errorf("unexpected matrix data type: got (%s), want (%s)", res.Data.Type(), QuantileSketchMatrixType) - } - data, ok := res.Data.(ProbabilisticQuantileMatrix) - if !ok { - return nil, fmt.Errorf("unexpected matrix type: got (%T), want (ProbabilisticQuantileMatrix)", res.Data) - } - stepper := NewQuantileSketchMatrixStepEvaluator(data, params) - xs = append(xs, stepper) + if len(results) != 1 { + return nil, fmt.Errorf("unexpected results length for sharded quantile: got (%d), want (1)", len(results)) } - inner := NewQuantileSketchMergeStepEvaluator(xs) - + matrix, ok := results[0].Data.(ProbabilisticQuantileMatrix) + if !ok { + return nil, fmt.Errorf("unexpected matrix type: got (%T), want (ProbabilisticQuantileMatrix)", results[0].Data) + } + inner := NewQuantileSketchMatrixStepEvaluator(matrix, params) return NewQuantileSketchVectorStepEvaluator(inner, *e.quantile), nil default: diff --git a/pkg/logql/metrics.go b/pkg/logql/metrics.go index e86a85ba7bc9..0a24e7414536 100644 --- a/pkg/logql/metrics.go +++ b/pkg/logql/metrics.go @@ -222,6 +222,10 @@ func RecordLabelQueryMetrics( "query", query, "query_hash", util.HashedQuery(query), "total_entries", stats.Summary.TotalEntriesReturned, + "cache_label_results_req", stats.Caches.LabelResult.EntriesRequested, + "cache_label_results_hit", stats.Caches.LabelResult.EntriesFound, + "cache_label_results_stored", stats.Caches.LabelResult.EntriesStored, + "cache_label_results_download_time", stats.Caches.LabelResult.CacheDownloadTime(), ) execLatency.WithLabelValues(status, queryType, "").Observe(stats.Summary.ExecTime) @@ -272,7 +276,12 @@ func RecordSeriesQueryMetrics(ctx context.Context, log log.Logger, start, end ti "status", status, "match", PrintMatches(match), "query_hash", util.HashedQuery(PrintMatches(match)), - "total_entries", stats.Summary.TotalEntriesReturned) + "total_entries", stats.Summary.TotalEntriesReturned, + "cache_series_results_req", stats.Caches.SeriesResult.EntriesRequested, + "cache_series_results_hit", stats.Caches.SeriesResult.EntriesFound, + "cache_series_results_stored", stats.Caches.SeriesResult.EntriesStored, + "cache_series_results_download_time", stats.Caches.SeriesResult.CacheDownloadTime(), + ) if shard != nil { logValues = append(logValues, diff --git a/pkg/logql/metrics_test.go b/pkg/logql/metrics_test.go index 6d07040bb802..efaead9afd0d 100644 --- a/pkg/logql/metrics_test.go +++ b/pkg/logql/metrics_test.go @@ -106,10 +106,18 @@ func TestLogLabelsQuery(t *testing.T) { TotalBytesProcessed: 100000, TotalEntriesReturned: 12, }, + Caches: stats.Caches{ + LabelResult: stats.Cache{ + EntriesRequested: 2, + EntriesFound: 1, + EntriesStored: 1, + DownloadTime: 80, + }, + }, }) require.Regexp(t, fmt.Sprintf( - "level=info org_id=foo traceID=%s sampled=true latency=slow query_type=labels splits=0 start=.* end=.* start_delta=1h0m0.* end_delta=.* length=1h0m0s duration=25.25s status=200 label=foo query= query_hash=2166136261 total_entries=12\n", + "level=info org_id=foo traceID=%s sampled=true latency=slow query_type=labels splits=0 start=.* end=.* start_delta=1h0m0.* end_delta=.* length=1h0m0s duration=25.25s status=200 label=foo query= query_hash=2166136261 total_entries=12 cache_label_results_req=2 cache_label_results_hit=1 cache_label_results_stored=1 cache_label_results_download_time=80ns\n", sp.Context().(jaeger.SpanContext).SpanID().String(), ), buf.String()) @@ -132,10 +140,18 @@ func TestLogSeriesQuery(t *testing.T) { TotalBytesProcessed: 100000, TotalEntriesReturned: 10, }, + Caches: stats.Caches{ + SeriesResult: stats.Cache{ + EntriesRequested: 2, + EntriesFound: 1, + EntriesStored: 1, + DownloadTime: 80, + }, + }, }) require.Regexp(t, fmt.Sprintf( - "level=info org_id=foo traceID=%s sampled=true latency=slow query_type=series splits=0 start=.* end=.* start_delta=1h0m0.* end_delta=.* length=1h0m0s duration=25.25s status=200 match=\"{container_name=.*\"}:{app=.*}\" query_hash=23523089 total_entries=10\n", + "level=info org_id=foo traceID=%s sampled=true latency=slow query_type=series splits=0 start=.* end=.* start_delta=1h0m0.* end_delta=.* length=1h0m0s duration=25.25s status=200 match=\"{container_name=.*\"}:{app=.*}\" query_hash=23523089 total_entries=10 cache_series_results_req=2 cache_series_results_hit=1 cache_series_results_stored=1 cache_series_results_download_time=80ns\n", sp.Context().(jaeger.SpanContext).SpanID().String(), ), buf.String()) diff --git a/pkg/logql/quantile_over_time_sketch.go b/pkg/logql/quantile_over_time_sketch.go index 121a3e551133..3d469de4078d 100644 --- a/pkg/logql/quantile_over_time_sketch.go +++ b/pkg/logql/quantile_over_time_sketch.go @@ -3,6 +3,7 @@ package logql import ( "fmt" "math" + "sync" "time" "github.com/prometheus/prometheus/model/labels" @@ -23,9 +24,17 @@ const ( type ProbabilisticQuantileVector []ProbabilisticQuantileSample type ProbabilisticQuantileMatrix []ProbabilisticQuantileVector +var streamHashPool = sync.Pool{ + New: func() interface{} { return make(map[uint64]int) }, +} + func (q ProbabilisticQuantileVector) Merge(right ProbabilisticQuantileVector) (ProbabilisticQuantileVector, error) { // labels hash to vector index map - groups := make(map[uint64]int) + groups := streamHashPool.Get().(map[uint64]int) + defer func() { + clear(groups) + streamHashPool.Put(groups) + }() for i, sample := range q { groups[sample.Metric.Hash()] = i } @@ -80,6 +89,21 @@ func (ProbabilisticQuantileMatrix) String() string { return "QuantileSketchMatrix()" } +func (m ProbabilisticQuantileMatrix) Merge(right ProbabilisticQuantileMatrix) (ProbabilisticQuantileMatrix, error) { + if len(m) != len(right) { + return nil, fmt.Errorf("failed to merge probabilistic quantile matrix: lengths differ %d!=%d", len(m), len(right)) + } + var err error + for i, vec := range m { + m[i], err = vec.Merge(right[i]) + if err != nil { + return nil, fmt.Errorf("failed to merge probabilistic quantile matrix: %w", err) + } + } + + return m, nil +} + func (ProbabilisticQuantileMatrix) Type() promql_parser.ValueType { return QuantileSketchMatrixType } func (m ProbabilisticQuantileMatrix) Release() { @@ -398,6 +422,9 @@ func NewQuantileSketchVectorStepEvaluator(inner StepEvaluator, quantile float64) func (e *QuantileSketchVectorStepEvaluator) Next() (bool, int64, StepResult) { ok, ts, r := e.inner.Next() + if !ok { + return false, 0, SampleVector{} + } quantileSketchVec := r.QuantileSketchVec() vec := make(promql.Vector, len(quantileSketchVec)) diff --git a/pkg/logql/test_utils.go b/pkg/logql/test_utils.go index b979dedb4232..82442e09bf60 100644 --- a/pkg/logql/test_utils.go +++ b/pkg/logql/test_utils.go @@ -225,6 +225,16 @@ func (m MockDownstreamer) Downstream(ctx context.Context, queries []DownstreamQu results = append(results, res) } + + if matrix, ok := results[0].Data.(ProbabilisticQuantileMatrix); ok { + if len(results) == 1 { + return results, nil + } + for _, m := range results[1:] { + matrix, _ = matrix.Merge(m.Data.(ProbabilisticQuantileMatrix)) + } + return []logqlmodel.Result{{Data: matrix}}, nil + } return results, nil } diff --git a/pkg/logqlmodel/stats/context.go b/pkg/logqlmodel/stats/context.go index 187bd88763a2..518e7effb59e 100644 --- a/pkg/logqlmodel/stats/context.go +++ b/pkg/logqlmodel/stats/context.go @@ -61,6 +61,8 @@ const ( StatsResultCache = "stats-result" VolumeResultCache = "volume-result" WriteDedupeCache = "write-dedupe" + SeriesResultCache = "series-result" + LabelResultCache = "label-result" BloomFilterCache = "bloom-filter" BloomBlocksCache = "bloom-blocks" ) @@ -100,6 +102,8 @@ func (c *Context) Caches() Caches { Result: c.caches.Result, StatsResult: c.caches.StatsResult, VolumeResult: c.caches.VolumeResult, + SeriesResult: c.caches.SeriesResult, + LabelResult: c.caches.LabelResult, } } @@ -215,6 +219,8 @@ func (c *Caches) Merge(m Caches) { c.Result.Merge(m.Result) c.StatsResult.Merge(m.StatsResult) c.VolumeResult.Merge(m.VolumeResult) + c.SeriesResult.Merge(m.SeriesResult) + c.LabelResult.Merge(m.LabelResult) } func (c *Cache) Merge(m Cache) { @@ -444,6 +450,10 @@ func (c *Context) getCacheStatsByType(t CacheType) *Cache { stats = &c.caches.StatsResult case VolumeResultCache: stats = &c.caches.VolumeResult + case SeriesResultCache: + stats = &c.caches.SeriesResult + case LabelResultCache: + stats = &c.caches.LabelResult default: return nil } @@ -526,6 +536,18 @@ func (c Caches) Log(log log.Logger) { "Cache.VolumeResult.EntriesStored", c.VolumeResult.EntriesStored, "Cache.VolumeResult.BytesSent", humanize.Bytes(uint64(c.VolumeResult.BytesSent)), "Cache.VolumeResult.BytesReceived", humanize.Bytes(uint64(c.VolumeResult.BytesReceived)), + "Cache.SeriesResult.Requests", c.SeriesResult.Requests, + "Cache.SeriesResult.EntriesRequested", c.SeriesResult.EntriesRequested, + "Cache.SeriesResult.EntriesFound", c.SeriesResult.EntriesFound, + "Cache.SeriesResult.EntriesStored", c.SeriesResult.EntriesStored, + "Cache.SeriesResult.BytesSent", humanize.Bytes(uint64(c.SeriesResult.BytesSent)), + "Cache.SeriesResult.BytesReceived", humanize.Bytes(uint64(c.SeriesResult.BytesReceived)), + "Cache.LabelResult.Requests", c.LabelResult.Requests, + "Cache.LabelResult.EntriesRequested", c.LabelResult.EntriesRequested, + "Cache.LabelResult.EntriesFound", c.LabelResult.EntriesFound, + "Cache.LabelResult.EntriesStored", c.LabelResult.EntriesStored, + "Cache.LabelResult.BytesSent", humanize.Bytes(uint64(c.LabelResult.BytesSent)), + "Cache.LabelResult.BytesReceived", humanize.Bytes(uint64(c.LabelResult.BytesReceived)), "Cache.Result.DownloadTime", c.Result.CacheDownloadTime(), "Cache.Result.Requests", c.Result.Requests, "Cache.Result.EntriesRequested", c.Result.EntriesRequested, diff --git a/pkg/logqlmodel/stats/stats.pb.go b/pkg/logqlmodel/stats/stats.pb.go index af008968ebd4..7d2df4df3323 100644 --- a/pkg/logqlmodel/stats/stats.pb.go +++ b/pkg/logqlmodel/stats/stats.pb.go @@ -100,6 +100,8 @@ type Caches struct { Result Cache `protobuf:"bytes,3,opt,name=result,proto3" json:"result"` StatsResult Cache `protobuf:"bytes,4,opt,name=statsResult,proto3" json:"statsResult"` VolumeResult Cache `protobuf:"bytes,5,opt,name=volumeResult,proto3" json:"volumeResult"` + SeriesResult Cache `protobuf:"bytes,6,opt,name=seriesResult,proto3" json:"seriesResult"` + LabelResult Cache `protobuf:"bytes,7,opt,name=labelResult,proto3" json:"labelResult"` } func (m *Caches) Reset() { *m = Caches{} } @@ -169,6 +171,20 @@ func (m *Caches) GetVolumeResult() Cache { return Cache{} } +func (m *Caches) GetSeriesResult() Cache { + if m != nil { + return m.SeriesResult + } + return Cache{} +} + +func (m *Caches) GetLabelResult() Cache { + if m != nil { + return m.LabelResult + } + return Cache{} +} + // Summary is the summary of a query statistics. type Summary struct { // Total bytes processed per second. @@ -749,80 +765,82 @@ func init() { func init() { proto.RegisterFile("pkg/logqlmodel/stats/stats.proto", fileDescriptor_6cdfe5d2aea33ebb) } var fileDescriptor_6cdfe5d2aea33ebb = []byte{ - // 1163 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x57, 0x4d, 0x6f, 0xe4, 0x44, - 0x13, 0x1e, 0x27, 0xaf, 0x67, 0xb2, 0x9d, 0xcf, 0xed, 0x64, 0xdf, 0xf5, 0x82, 0x64, 0x87, 0x81, - 0x15, 0x41, 0xa0, 0x8c, 0xf8, 0x90, 0x10, 0x88, 0x95, 0x90, 0xb3, 0x44, 0x8a, 0xb4, 0x2b, 0x42, - 0x05, 0x2e, 0xdc, 0x1c, 0xbb, 0x33, 0x63, 0xc5, 0x63, 0x4f, 0xec, 0xf6, 0xb2, 0x39, 0xc1, 0x4f, - 0xe0, 0x67, 0x70, 0xe1, 0x80, 0x38, 0x21, 0xf1, 0x03, 0xf6, 0x98, 0xe3, 0x9e, 0x2c, 0x32, 0xb9, - 0x20, 0x9f, 0x56, 0xe2, 0x8e, 0x50, 0x57, 0xf7, 0xf8, 0x6b, 0x3c, 0xda, 0x5c, 0xc6, 0x5d, 0x4f, - 0x3d, 0x4f, 0xf5, 0x67, 0x55, 0xf7, 0x90, 0xdd, 0xc9, 0xf9, 0x70, 0x10, 0x44, 0xc3, 0x8b, 0x60, - 0x1c, 0x79, 0x2c, 0x18, 0x24, 0xdc, 0xe1, 0x89, 0xfc, 0xdd, 0x9f, 0xc4, 0x11, 0x8f, 0xa8, 0x8e, - 0xc6, 0x1b, 0x3b, 0xc3, 0x68, 0x18, 0x21, 0x32, 0x10, 0x2d, 0xe9, 0xec, 0xff, 0xa3, 0x91, 0x2e, - 0xb0, 0x24, 0x0d, 0x38, 0xfd, 0x8c, 0xf4, 0x92, 0x74, 0x3c, 0x76, 0xe2, 0x4b, 0x43, 0xdb, 0xd5, - 0xf6, 0x56, 0x3f, 0xda, 0xd8, 0x97, 0x61, 0x4e, 0x24, 0x6a, 0x6f, 0xbe, 0xc8, 0xac, 0x4e, 0x9e, - 0x59, 0x33, 0x1a, 0xcc, 0x1a, 0x42, 0x7a, 0x91, 0xb2, 0xd8, 0x67, 0xb1, 0xb1, 0x54, 0x93, 0x7e, - 0x23, 0xd1, 0x52, 0xaa, 0x68, 0x30, 0x6b, 0xd0, 0x47, 0x64, 0xc5, 0x0f, 0x87, 0x2c, 0xe1, 0x2c, - 0x36, 0x96, 0x51, 0xbb, 0xa9, 0xb4, 0x47, 0x0a, 0xb6, 0xb7, 0x94, 0xb8, 0x20, 0x42, 0xd1, 0xa2, - 0x9f, 0x90, 0xae, 0xeb, 0xb8, 0x23, 0x96, 0x18, 0xff, 0x43, 0xf1, 0xba, 0x12, 0x1f, 0x20, 0x68, - 0xaf, 0x2b, 0xa9, 0x8e, 0x24, 0x50, 0xdc, 0xfe, 0x6f, 0x4b, 0xa4, 0x2b, 0x19, 0xf4, 0x43, 0xa2, - 0xbb, 0xa3, 0x34, 0x3c, 0x57, 0x73, 0x5e, 0xab, 0xea, 0x2b, 0x72, 0x41, 0x01, 0xf9, 0x11, 0x12, - 0x3f, 0xf4, 0xd8, 0x73, 0x35, 0xd7, 0x05, 0x12, 0xa4, 0x80, 0xfc, 0x88, 0x61, 0xc6, 0xb8, 0xca, - 0x6a, 0x8e, 0x75, 0xcd, 0x86, 0xd2, 0x28, 0x0e, 0xa8, 0x2f, 0x3d, 0x20, 0xab, 0x48, 0x93, 0x1b, - 0xa4, 0x66, 0x58, 0x97, 0x6e, 0x2b, 0x69, 0x95, 0x08, 0x55, 0x83, 0x1e, 0x92, 0xb5, 0x67, 0x51, - 0x90, 0x8e, 0x99, 0x8a, 0xa2, 0xb7, 0x44, 0xd9, 0x51, 0x51, 0x6a, 0x4c, 0xa8, 0x59, 0xfd, 0x3f, - 0xba, 0xa4, 0xa7, 0x4e, 0x02, 0xfd, 0x8e, 0xdc, 0x3f, 0xbd, 0xe4, 0x2c, 0x39, 0x8e, 0x23, 0x97, - 0x25, 0x09, 0xf3, 0x8e, 0x59, 0x7c, 0xc2, 0xdc, 0x28, 0xf4, 0x70, 0x19, 0x97, 0xed, 0x37, 0xf3, - 0xcc, 0x5a, 0x44, 0x81, 0x45, 0x0e, 0x11, 0x36, 0xf0, 0xc3, 0xd6, 0xb0, 0x4b, 0x65, 0xd8, 0x05, - 0x14, 0x58, 0xe4, 0xa0, 0x47, 0x64, 0x9b, 0x47, 0xdc, 0x09, 0xec, 0x5a, 0xb7, 0xb8, 0x13, 0xcb, - 0xf6, 0xfd, 0x3c, 0xb3, 0xda, 0xdc, 0xd0, 0x06, 0x16, 0xa1, 0x9e, 0xd4, 0xba, 0xc2, 0x9d, 0xa9, - 0x86, 0xaa, 0xbb, 0xa1, 0x0d, 0xa4, 0x7b, 0x64, 0x85, 0x3d, 0x67, 0xee, 0xb7, 0xfe, 0x98, 0xe1, - 0x9e, 0x68, 0xf6, 0x9a, 0x38, 0xe3, 0x33, 0x0c, 0x8a, 0x16, 0x7d, 0x9f, 0xdc, 0xb9, 0x48, 0x59, - 0xca, 0x90, 0xda, 0x45, 0xea, 0x7a, 0x9e, 0x59, 0x25, 0x08, 0x65, 0x93, 0xee, 0x13, 0x92, 0xa4, - 0xa7, 0x32, 0xbb, 0x12, 0xa3, 0x87, 0x03, 0xdb, 0xc8, 0x33, 0xab, 0x82, 0x42, 0xa5, 0x4d, 0x9f, - 0x90, 0x1d, 0x1c, 0xdd, 0x57, 0x21, 0x47, 0x1f, 0xe3, 0x69, 0x1c, 0x32, 0xcf, 0x58, 0x41, 0xa5, - 0x91, 0x67, 0x56, 0xab, 0x1f, 0x5a, 0x51, 0xda, 0x27, 0xdd, 0x64, 0x12, 0xf8, 0x3c, 0x31, 0xee, - 0xa0, 0x9e, 0x88, 0x53, 0x2d, 0x11, 0x50, 0x5f, 0xe4, 0x8c, 0x9c, 0xd8, 0x4b, 0x0c, 0x52, 0xe1, - 0x20, 0x02, 0xea, 0x5b, 0x8c, 0xea, 0x38, 0x4a, 0xf8, 0xa1, 0x1f, 0x70, 0x16, 0xe3, 0xea, 0x19, - 0xab, 0x8d, 0x51, 0x35, 0xfc, 0xd0, 0x8a, 0xd2, 0x1f, 0xc9, 0x43, 0xc4, 0x4f, 0x78, 0x9c, 0xba, - 0x3c, 0x8d, 0x99, 0xf7, 0x94, 0x71, 0xc7, 0x73, 0xb8, 0xd3, 0x38, 0x12, 0x6b, 0x18, 0xfe, 0xbd, - 0x3c, 0xb3, 0x6e, 0x27, 0x80, 0xdb, 0xd1, 0xfa, 0x5f, 0x90, 0x9e, 0xaa, 0x84, 0xa2, 0x78, 0x24, - 0x3c, 0x8a, 0x59, 0xa3, 0xde, 0x9c, 0x08, 0xac, 0x2c, 0x1e, 0x48, 0x01, 0xf9, 0xe9, 0xff, 0xba, - 0x44, 0x56, 0x8e, 0xca, 0x82, 0xb7, 0x86, 0x7d, 0x02, 0x13, 0x99, 0x2b, 0xf3, 0x4d, 0xb7, 0xb7, - 0x44, 0xf2, 0x56, 0x71, 0xa8, 0x59, 0xf4, 0x90, 0x50, 0xb4, 0x0f, 0x44, 0x01, 0x4b, 0x9e, 0x3a, - 0x1c, 0xb5, 0x32, 0xa9, 0xfe, 0x9f, 0x67, 0x56, 0x8b, 0x17, 0x5a, 0xb0, 0xa2, 0x77, 0x1b, 0xed, - 0x44, 0xe5, 0x50, 0xd9, 0xbb, 0xc2, 0xa1, 0x66, 0xd1, 0xcf, 0xc9, 0x46, 0x99, 0x01, 0x27, 0x2c, - 0xe4, 0x2a, 0x61, 0x68, 0x9e, 0x59, 0x0d, 0x0f, 0x34, 0xec, 0x72, 0xbd, 0xf4, 0x5b, 0xaf, 0xd7, - 0x9f, 0xcb, 0x44, 0x47, 0x7f, 0xd1, 0xb1, 0x9c, 0x04, 0xb0, 0x33, 0x55, 0x9e, 0xca, 0x8e, 0x0b, - 0x0f, 0x34, 0x6c, 0xfa, 0x35, 0xb9, 0x57, 0x41, 0x1e, 0x47, 0x3f, 0x84, 0x41, 0xe4, 0x78, 0xc5, - 0xaa, 0x3d, 0xc8, 0x33, 0xab, 0x9d, 0x00, 0xed, 0xb0, 0xd8, 0x03, 0xb7, 0x86, 0x61, 0x3e, 0x2f, - 0x97, 0x7b, 0x30, 0xef, 0x85, 0x16, 0x8c, 0xba, 0xe4, 0x81, 0x48, 0xde, 0x4b, 0x60, 0x67, 0x2c, - 0x66, 0xa1, 0xcb, 0xbc, 0xf2, 0xfc, 0x19, 0xeb, 0xbb, 0xda, 0xde, 0x8a, 0xfd, 0x30, 0xcf, 0xac, - 0xb7, 0x16, 0x92, 0x66, 0x87, 0x14, 0x16, 0xc7, 0x29, 0xaf, 0xc5, 0xc6, 0xa5, 0x23, 0xb0, 0x05, - 0xd7, 0xe2, 0x6c, 0x7e, 0xc0, 0xce, 0x92, 0x43, 0xc6, 0xdd, 0x51, 0x51, 0xda, 0xaa, 0xf3, 0xab, - 0x79, 0xa1, 0x05, 0xeb, 0xff, 0xae, 0x13, 0x1d, 0xfb, 0x11, 0xdb, 0x37, 0x62, 0x8e, 0x27, 0x3b, - 0x15, 0x19, 0x55, 0x3d, 0x37, 0x75, 0x0f, 0x34, 0xec, 0x9a, 0x56, 0xd6, 0x0e, 0xbd, 0x45, 0x2b, - 0xab, 0x46, 0xc3, 0xa6, 0x07, 0xe4, 0xae, 0xc7, 0xdc, 0x68, 0x3c, 0x89, 0x31, 0x7d, 0x65, 0xd7, - 0x5d, 0x94, 0xdf, 0xcb, 0x33, 0x6b, 0xde, 0x09, 0xf3, 0x50, 0x33, 0x88, 0x1c, 0x43, 0xaf, 0x3d, - 0x88, 0x1c, 0xc6, 0x3c, 0x44, 0x1f, 0x91, 0xcd, 0xe6, 0x38, 0x64, 0x61, 0xde, 0xce, 0x33, 0xab, - 0xe9, 0x82, 0x26, 0x20, 0xe4, 0x78, 0x16, 0x1f, 0xa7, 0x93, 0xc0, 0x77, 0x1d, 0x21, 0xbf, 0x53, - 0xca, 0x1b, 0x2e, 0x68, 0x02, 0x42, 0x3e, 0x69, 0x14, 0x60, 0x52, 0xca, 0x1b, 0x2e, 0x68, 0x02, - 0x74, 0x42, 0x76, 0x8b, 0x85, 0x5d, 0x50, 0x22, 0x55, 0x41, 0x7f, 0x27, 0xcf, 0xac, 0xd7, 0x72, - 0xe1, 0xb5, 0x0c, 0x7a, 0x49, 0xde, 0xae, 0xae, 0xe1, 0xa2, 0x4e, 0x65, 0x99, 0x7f, 0x37, 0xcf, - 0xac, 0xdb, 0xd0, 0xe1, 0x36, 0xa4, 0xfe, 0xbf, 0x4b, 0x44, 0xc7, 0xc7, 0x94, 0xa8, 0x91, 0x4c, - 0x5e, 0x8b, 0x87, 0x51, 0x1a, 0xd6, 0x2a, 0x74, 0x15, 0x87, 0x9a, 0x45, 0xbf, 0x24, 0x5b, 0x6c, - 0x76, 0x99, 0x5e, 0xa4, 0xa2, 0xd6, 0xcb, 0x4a, 0xa3, 0xdb, 0x3b, 0x79, 0x66, 0xcd, 0xf9, 0x60, - 0x0e, 0xa1, 0x9f, 0x92, 0x75, 0x85, 0x61, 0xf1, 0x93, 0x0f, 0x1c, 0xdd, 0xbe, 0x9b, 0x67, 0x56, - 0xdd, 0x01, 0x75, 0x53, 0x08, 0xf1, 0x45, 0x06, 0xcc, 0x65, 0xfe, 0xb3, 0xe2, 0x39, 0x83, 0xc2, - 0x9a, 0x03, 0xea, 0xa6, 0x78, 0x98, 0x20, 0x80, 0x25, 0x5d, 0xa6, 0x17, 0x3e, 0x4c, 0x0a, 0x10, - 0xca, 0xa6, 0x78, 0xef, 0xc4, 0x72, 0xac, 0x32, 0x97, 0x74, 0xf9, 0xde, 0x99, 0x61, 0x50, 0xb4, - 0xc4, 0x02, 0x7a, 0xd5, 0x12, 0xd9, 0x2b, 0x2f, 0x99, 0x2a, 0x0e, 0x35, 0xcb, 0x3e, 0xbd, 0xba, - 0x36, 0x3b, 0x2f, 0xaf, 0xcd, 0xce, 0xab, 0x6b, 0x53, 0xfb, 0x69, 0x6a, 0x6a, 0xbf, 0x4c, 0x4d, - 0xed, 0xc5, 0xd4, 0xd4, 0xae, 0xa6, 0xa6, 0xf6, 0xd7, 0xd4, 0xd4, 0xfe, 0x9e, 0x9a, 0x9d, 0x57, - 0x53, 0x53, 0xfb, 0xf9, 0xc6, 0xec, 0x5c, 0xdd, 0x98, 0x9d, 0x97, 0x37, 0x66, 0xe7, 0xfb, 0x0f, - 0x86, 0x3e, 0x1f, 0xa5, 0xa7, 0xfb, 0x6e, 0x34, 0x1e, 0x0c, 0x63, 0xe7, 0xcc, 0x09, 0x9d, 0x41, - 0x10, 0x9d, 0xfb, 0x83, 0xb6, 0xbf, 0x55, 0xa7, 0x5d, 0xfc, 0xd3, 0xf4, 0xf1, 0x7f, 0x01, 0x00, - 0x00, 0xff, 0xff, 0x02, 0xba, 0xb6, 0x0a, 0x75, 0x0d, 0x00, 0x00, + // 1193 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x57, 0xcf, 0x6f, 0xdc, 0xc4, + 0x17, 0x5f, 0x67, 0xbf, 0xde, 0x4d, 0xa7, 0xf9, 0xd5, 0x49, 0xfa, 0xed, 0x16, 0x24, 0x3b, 0x2c, + 0x54, 0x04, 0x81, 0xb2, 0xe2, 0x87, 0x84, 0x40, 0x54, 0x42, 0x4e, 0x89, 0x14, 0xa9, 0x15, 0xe1, + 0x05, 0x2e, 0xdc, 0xbc, 0xf6, 0x64, 0xd7, 0x8a, 0xd7, 0xde, 0xf8, 0x47, 0x69, 0x4e, 0xf0, 0x27, + 0xf0, 0x67, 0x70, 0xe1, 0xc4, 0x09, 0x09, 0x71, 0xee, 0x31, 0xc7, 0x9e, 0x2c, 0xb2, 0xb9, 0x20, + 0x9f, 0x2a, 0x71, 0x47, 0x68, 0xde, 0xcc, 0xda, 0x1e, 0xaf, 0xb7, 0xcd, 0x65, 0x3d, 0xef, 0xf3, + 0x3e, 0x9f, 0x37, 0x3f, 0xdf, 0x9b, 0x59, 0xb2, 0x3b, 0x3d, 0x1b, 0x0d, 0xfc, 0x70, 0x74, 0xee, + 0x4f, 0x42, 0x97, 0xf9, 0x83, 0x38, 0xb1, 0x93, 0x58, 0xfc, 0xee, 0x4f, 0xa3, 0x30, 0x09, 0xa9, + 0x8e, 0xc6, 0x1b, 0x3b, 0xa3, 0x70, 0x14, 0x22, 0x32, 0xe0, 0x2d, 0xe1, 0xec, 0xff, 0xa3, 0x91, + 0x0e, 0xb0, 0x38, 0xf5, 0x13, 0xfa, 0x19, 0xe9, 0xc6, 0xe9, 0x64, 0x62, 0x47, 0x17, 0x3d, 0x6d, + 0x57, 0xdb, 0xbb, 0xfd, 0xd1, 0xc6, 0xbe, 0x08, 0x73, 0x22, 0x50, 0x6b, 0xf3, 0x79, 0x66, 0xb6, + 0xf2, 0xcc, 0x9c, 0xd3, 0x60, 0xde, 0xe0, 0xd2, 0xf3, 0x94, 0x45, 0x1e, 0x8b, 0x7a, 0x2b, 0x8a, + 0xf4, 0x1b, 0x81, 0x96, 0x52, 0x49, 0x83, 0x79, 0x83, 0x3e, 0x24, 0xab, 0x5e, 0x30, 0x62, 0x71, + 0xc2, 0xa2, 0x5e, 0x1b, 0xb5, 0x9b, 0x52, 0x7b, 0x24, 0x61, 0x6b, 0x4b, 0x8a, 0x0b, 0x22, 0x14, + 0x2d, 0xfa, 0x09, 0xe9, 0x38, 0xb6, 0x33, 0x66, 0x71, 0xef, 0x7f, 0x28, 0x5e, 0x97, 0xe2, 0x03, + 0x04, 0xad, 0x75, 0x29, 0xd5, 0x91, 0x04, 0x92, 0xdb, 0xff, 0xb3, 0x4d, 0x3a, 0x82, 0x41, 0x3f, + 0x24, 0xba, 0x33, 0x4e, 0x83, 0x33, 0x39, 0xe7, 0xb5, 0xaa, 0xbe, 0x22, 0xe7, 0x14, 0x10, 0x1f, + 0x2e, 0xf1, 0x02, 0x97, 0x3d, 0x93, 0x73, 0x5d, 0x22, 0x41, 0x0a, 0x88, 0x0f, 0x1f, 0x66, 0x84, + 0xab, 0x2c, 0xe7, 0xa8, 0x6a, 0x36, 0xa4, 0x46, 0x72, 0x40, 0x7e, 0xe9, 0x01, 0xb9, 0x8d, 0x34, + 0xb1, 0x41, 0x72, 0x86, 0xaa, 0x74, 0x5b, 0x4a, 0xab, 0x44, 0xa8, 0x1a, 0xf4, 0x90, 0xac, 0x3d, + 0x0d, 0xfd, 0x74, 0xc2, 0x64, 0x14, 0xbd, 0x21, 0xca, 0x8e, 0x8c, 0xa2, 0x30, 0x41, 0xb1, 0x78, + 0x9c, 0x98, 0x6f, 0xd9, 0x7c, 0x34, 0x9d, 0x57, 0xc5, 0xa9, 0x32, 0x41, 0xb1, 0xf8, 0xa4, 0x7c, + 0x7b, 0xc8, 0x7c, 0x19, 0xa6, 0xfb, 0xaa, 0x49, 0x55, 0x88, 0x50, 0x35, 0xfa, 0xbf, 0x77, 0x48, + 0x57, 0x1e, 0x4b, 0xfa, 0x1d, 0xb9, 0x37, 0xbc, 0x48, 0x58, 0x7c, 0x1c, 0x85, 0x0e, 0x8b, 0x63, + 0xe6, 0x1e, 0xb3, 0xe8, 0x84, 0x39, 0x61, 0xe0, 0xe2, 0x9e, 0xb6, 0xad, 0x37, 0xf3, 0xcc, 0x5c, + 0x46, 0x81, 0x65, 0x0e, 0x1e, 0xd6, 0xf7, 0x82, 0xc6, 0xb0, 0x2b, 0x65, 0xd8, 0x25, 0x14, 0x58, + 0xe6, 0xa0, 0x47, 0x64, 0x3b, 0x09, 0x13, 0xdb, 0xb7, 0x94, 0x6e, 0xf1, 0x58, 0xb4, 0xad, 0x7b, + 0x79, 0x66, 0x36, 0xb9, 0xa1, 0x09, 0x2c, 0x42, 0x3d, 0x56, 0xba, 0xc2, 0x63, 0x52, 0x0d, 0xa5, + 0xba, 0xa1, 0x09, 0xa4, 0x7b, 0x64, 0x95, 0x3d, 0x63, 0xce, 0xb7, 0xde, 0x84, 0xe1, 0x01, 0xd1, + 0xac, 0x35, 0x9e, 0x70, 0x73, 0x0c, 0x8a, 0x16, 0x7d, 0x9f, 0xdc, 0x3a, 0x4f, 0x59, 0xca, 0x90, + 0xda, 0x41, 0xea, 0x7a, 0x9e, 0x99, 0x25, 0x08, 0x65, 0x93, 0xee, 0x13, 0x12, 0xa7, 0x43, 0x91, + 0xea, 0x31, 0x6e, 0x75, 0xdb, 0xda, 0xc8, 0x33, 0xb3, 0x82, 0x42, 0xa5, 0x4d, 0x1f, 0x93, 0x1d, + 0x1c, 0xdd, 0x57, 0x41, 0x22, 0x4e, 0x4c, 0x92, 0x46, 0x01, 0x73, 0x7b, 0xab, 0xa8, 0xec, 0xe5, + 0x99, 0xd9, 0xe8, 0x87, 0x46, 0x94, 0xf6, 0x49, 0x27, 0x9e, 0xfa, 0x5e, 0x12, 0xf7, 0x6e, 0xa1, + 0x9e, 0xf0, 0x14, 0x13, 0x08, 0xc8, 0x2f, 0x72, 0xc6, 0x76, 0xe4, 0xc6, 0x3d, 0x52, 0xe1, 0x20, + 0x02, 0xf2, 0x5b, 0x8c, 0xea, 0x38, 0x8c, 0x93, 0x43, 0xcf, 0x4f, 0x58, 0x84, 0xab, 0xd7, 0xbb, + 0x5d, 0x1b, 0x55, 0xcd, 0x0f, 0x8d, 0x28, 0xfd, 0x91, 0x3c, 0x40, 0xfc, 0x24, 0x89, 0x52, 0x27, + 0x49, 0x23, 0xe6, 0x3e, 0x61, 0x89, 0xed, 0xda, 0x89, 0x5d, 0x3b, 0x12, 0x6b, 0x18, 0xfe, 0xbd, + 0x3c, 0x33, 0x6f, 0x26, 0x80, 0x9b, 0xd1, 0xfa, 0x5f, 0x90, 0xae, 0x2c, 0xcb, 0xbc, 0x92, 0xc5, + 0x49, 0x18, 0xb1, 0x5a, 0xf1, 0x3b, 0xe1, 0x58, 0x59, 0xc9, 0x90, 0x02, 0xe2, 0xd3, 0xff, 0x75, + 0x85, 0xac, 0x1e, 0x95, 0xd5, 0x77, 0x0d, 0xfb, 0x04, 0xc6, 0xf3, 0x56, 0xe4, 0x9b, 0x6e, 0x6d, + 0xf1, 0x0a, 0x50, 0xc5, 0x41, 0xb1, 0xe8, 0x21, 0xa1, 0x68, 0x1f, 0xf0, 0x6a, 0x1a, 0x3f, 0xb1, + 0x13, 0xd4, 0x8a, 0xa4, 0xfa, 0x7f, 0x9e, 0x99, 0x0d, 0x5e, 0x68, 0xc0, 0x8a, 0xde, 0x2d, 0xb4, + 0x63, 0x99, 0x43, 0x65, 0xef, 0x12, 0x07, 0xc5, 0xa2, 0x9f, 0x93, 0x8d, 0x32, 0x03, 0x4e, 0x58, + 0x90, 0xc8, 0x84, 0xa1, 0x79, 0x66, 0xd6, 0x3c, 0x50, 0xb3, 0xcb, 0xf5, 0xd2, 0x6f, 0xbc, 0x5e, + 0x7f, 0xb4, 0x89, 0x8e, 0xfe, 0xa2, 0x63, 0x31, 0x09, 0x60, 0xa7, 0xb2, 0x3c, 0x95, 0x1d, 0x17, + 0x1e, 0xa8, 0xd9, 0xf4, 0x6b, 0x72, 0xb7, 0x82, 0x3c, 0x0a, 0x7f, 0x08, 0xfc, 0xd0, 0x76, 0x8b, + 0x55, 0xbb, 0x9f, 0x67, 0x66, 0x33, 0x01, 0x9a, 0x61, 0xbe, 0x07, 0x8e, 0x82, 0x61, 0x3e, 0xb7, + 0xcb, 0x3d, 0x58, 0xf4, 0x42, 0x03, 0x46, 0x1d, 0x72, 0x9f, 0x27, 0xef, 0x05, 0xb0, 0x53, 0x16, + 0xb1, 0xc0, 0x61, 0x6e, 0x79, 0xfe, 0x7a, 0xeb, 0xbb, 0xda, 0xde, 0xaa, 0xf5, 0x20, 0xcf, 0xcc, + 0xb7, 0x96, 0x92, 0xe6, 0x87, 0x14, 0x96, 0xc7, 0x29, 0xef, 0xe8, 0xda, 0x0d, 0xc8, 0xb1, 0x25, + 0x77, 0xf4, 0x7c, 0x7e, 0xc0, 0x4e, 0xe3, 0x43, 0x96, 0x38, 0xe3, 0xa2, 0xb4, 0x55, 0xe7, 0xa7, + 0x78, 0xa1, 0x01, 0xeb, 0xff, 0xa6, 0x13, 0x1d, 0xfb, 0xe1, 0xdb, 0x37, 0x66, 0xb6, 0x2b, 0x3a, + 0xe5, 0x19, 0x55, 0x3d, 0x37, 0xaa, 0x07, 0x6a, 0xb6, 0xa2, 0x15, 0xb5, 0x43, 0x6f, 0xd0, 0x8a, + 0xaa, 0x51, 0xb3, 0xe9, 0x01, 0xb9, 0xe3, 0x32, 0x27, 0x9c, 0x4c, 0x23, 0x4c, 0x5f, 0xd1, 0x75, + 0x07, 0xe5, 0x77, 0xf3, 0xcc, 0x5c, 0x74, 0xc2, 0x22, 0x54, 0x0f, 0x22, 0xc6, 0xd0, 0x6d, 0x0e, + 0x22, 0x86, 0xb1, 0x08, 0xd1, 0x87, 0x64, 0xb3, 0x3e, 0x0e, 0x51, 0x98, 0xb7, 0xf3, 0xcc, 0xac, + 0xbb, 0xa0, 0x0e, 0x70, 0x39, 0x9e, 0xc5, 0x47, 0xe9, 0xd4, 0xf7, 0x1c, 0x9b, 0xcb, 0x6f, 0x95, + 0xf2, 0x9a, 0x0b, 0xea, 0x00, 0x97, 0x4f, 0x6b, 0x05, 0x98, 0x94, 0xf2, 0x9a, 0x0b, 0xea, 0x00, + 0x9d, 0x92, 0xdd, 0x62, 0x61, 0x97, 0x94, 0x48, 0x59, 0xd0, 0xdf, 0xc9, 0x33, 0xf3, 0xb5, 0x5c, + 0x78, 0x2d, 0x83, 0x5e, 0x90, 0xb7, 0xab, 0x6b, 0xb8, 0xac, 0x53, 0x51, 0xe6, 0xdf, 0xcd, 0x33, + 0xf3, 0x26, 0x74, 0xb8, 0x09, 0xa9, 0xff, 0xef, 0x0a, 0xd1, 0xf1, 0x29, 0xc5, 0x6b, 0x24, 0x13, + 0xd7, 0xe2, 0x61, 0x98, 0x06, 0x4a, 0x85, 0xae, 0xe2, 0xa0, 0x58, 0xf4, 0x4b, 0xb2, 0xc5, 0xe6, + 0x97, 0xe9, 0x79, 0xca, 0x6b, 0xbd, 0xa8, 0x34, 0xba, 0xb5, 0x93, 0x67, 0xe6, 0x82, 0x0f, 0x16, + 0x10, 0xfa, 0x29, 0x59, 0x97, 0x18, 0x16, 0x3f, 0xf1, 0xc0, 0xd1, 0xad, 0x3b, 0x79, 0x66, 0xaa, + 0x0e, 0x50, 0x4d, 0x2e, 0xc4, 0x17, 0x19, 0x30, 0x87, 0x79, 0x4f, 0x8b, 0xe7, 0x0c, 0x0a, 0x15, + 0x07, 0xa8, 0x26, 0x7f, 0x98, 0x20, 0x80, 0x25, 0x5d, 0xa4, 0x17, 0x3e, 0x4c, 0x0a, 0x10, 0xca, + 0x26, 0x7f, 0xef, 0x44, 0x62, 0xac, 0x22, 0x97, 0x74, 0xf1, 0xde, 0x99, 0x63, 0x50, 0xb4, 0xf8, + 0x02, 0xba, 0xd5, 0x12, 0xd9, 0x2d, 0x2f, 0x99, 0x2a, 0x0e, 0x8a, 0x65, 0x0d, 0x2f, 0xaf, 0x8c, + 0xd6, 0x8b, 0x2b, 0xa3, 0xf5, 0xf2, 0xca, 0xd0, 0x7e, 0x9a, 0x19, 0xda, 0x2f, 0x33, 0x43, 0x7b, + 0x3e, 0x33, 0xb4, 0xcb, 0x99, 0xa1, 0xfd, 0x35, 0x33, 0xb4, 0xbf, 0x67, 0x46, 0xeb, 0xe5, 0xcc, + 0xd0, 0x7e, 0xbe, 0x36, 0x5a, 0x97, 0xd7, 0x46, 0xeb, 0xc5, 0xb5, 0xd1, 0xfa, 0xfe, 0x83, 0x91, + 0x97, 0x8c, 0xd3, 0xe1, 0xbe, 0x13, 0x4e, 0x06, 0xa3, 0xc8, 0x3e, 0xb5, 0x03, 0x7b, 0xe0, 0x87, + 0x67, 0xde, 0xa0, 0xe9, 0x3f, 0xde, 0xb0, 0x83, 0xff, 0xe0, 0x3e, 0xfe, 0x2f, 0x00, 0x00, 0xff, + 0xff, 0x99, 0x60, 0xf2, 0x04, 0x02, 0x0e, 0x00, 0x00, } func (this *Result) Equal(that interface{}) bool { @@ -892,6 +910,12 @@ func (this *Caches) Equal(that interface{}) bool { if !this.VolumeResult.Equal(&that1.VolumeResult) { return false } + if !this.SeriesResult.Equal(&that1.SeriesResult) { + return false + } + if !this.LabelResult.Equal(&that1.LabelResult) { + return false + } return true } func (this *Summary) Equal(that interface{}) bool { @@ -1157,13 +1181,15 @@ func (this *Caches) GoString() string { if this == nil { return "nil" } - s := make([]string, 0, 9) + s := make([]string, 0, 11) s = append(s, "&stats.Caches{") s = append(s, "Chunk: "+strings.Replace(this.Chunk.GoString(), `&`, ``, 1)+",\n") s = append(s, "Index: "+strings.Replace(this.Index.GoString(), `&`, ``, 1)+",\n") s = append(s, "Result: "+strings.Replace(this.Result.GoString(), `&`, ``, 1)+",\n") s = append(s, "StatsResult: "+strings.Replace(this.StatsResult.GoString(), `&`, ``, 1)+",\n") s = append(s, "VolumeResult: "+strings.Replace(this.VolumeResult.GoString(), `&`, ``, 1)+",\n") + s = append(s, "SeriesResult: "+strings.Replace(this.SeriesResult.GoString(), `&`, ``, 1)+",\n") + s = append(s, "LabelResult: "+strings.Replace(this.LabelResult.GoString(), `&`, ``, 1)+",\n") s = append(s, "}") return strings.Join(s, "") } @@ -1352,6 +1378,26 @@ func (m *Caches) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + { + size, err := m.LabelResult.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintStats(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x3a + { + size, err := m.SeriesResult.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintStats(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x32 { size, err := m.VolumeResult.MarshalToSizedBuffer(dAtA[:i]) if err != nil { @@ -1809,6 +1855,10 @@ func (m *Caches) Size() (n int) { n += 1 + l + sovStats(uint64(l)) l = m.VolumeResult.Size() n += 1 + l + sovStats(uint64(l)) + l = m.SeriesResult.Size() + n += 1 + l + sovStats(uint64(l)) + l = m.LabelResult.Size() + n += 1 + l + sovStats(uint64(l)) return n } @@ -2012,6 +2062,8 @@ func (this *Caches) String() string { `Result:` + strings.Replace(strings.Replace(this.Result.String(), "Cache", "Cache", 1), `&`, ``, 1) + `,`, `StatsResult:` + strings.Replace(strings.Replace(this.StatsResult.String(), "Cache", "Cache", 1), `&`, ``, 1) + `,`, `VolumeResult:` + strings.Replace(strings.Replace(this.VolumeResult.String(), "Cache", "Cache", 1), `&`, ``, 1) + `,`, + `SeriesResult:` + strings.Replace(strings.Replace(this.SeriesResult.String(), "Cache", "Cache", 1), `&`, ``, 1) + `,`, + `LabelResult:` + strings.Replace(strings.Replace(this.LabelResult.String(), "Cache", "Cache", 1), `&`, ``, 1) + `,`, `}`, }, "") return s @@ -2497,6 +2549,72 @@ func (m *Caches) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SeriesResult", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStats + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthStats + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthStats + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.SeriesResult.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 7: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field LabelResult", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStats + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthStats + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthStats + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.LabelResult.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipStats(dAtA[iNdEx:]) diff --git a/pkg/logqlmodel/stats/stats.proto b/pkg/logqlmodel/stats/stats.proto index 3aeaab7d0c9a..0be2f2334a00 100644 --- a/pkg/logqlmodel/stats/stats.proto +++ b/pkg/logqlmodel/stats/stats.proto @@ -49,6 +49,14 @@ message Caches { (gogoproto.nullable) = false, (gogoproto.jsontag) = "volumeResult" ]; + Cache seriesResult = 6 [ + (gogoproto.nullable) = false, + (gogoproto.jsontag) = "seriesResult" + ]; + Cache labelResult = 7 [ + (gogoproto.nullable) = false, + (gogoproto.jsontag) = "labelResult" + ]; } // Summary is the summary of a query statistics. diff --git a/pkg/loki/config_wrapper.go b/pkg/loki/config_wrapper.go index 41a87775a9ec..9817c04afdc5 100644 --- a/pkg/loki/config_wrapper.go +++ b/pkg/loki/config_wrapper.go @@ -619,14 +619,32 @@ func applyEmbeddedCacheConfig(r *ConfigWrapper) { indexStatsCacheConfig := r.QueryRange.StatsCacheConfig.CacheConfig if !cache.IsCacheConfigured(indexStatsCacheConfig) { + prefix := indexStatsCacheConfig.Prefix // We use the same config as the query range results cache. r.QueryRange.StatsCacheConfig.CacheConfig = r.QueryRange.ResultsCacheConfig.CacheConfig + r.QueryRange.StatsCacheConfig.CacheConfig.Prefix = prefix } volumeCacheConfig := r.QueryRange.VolumeCacheConfig.CacheConfig if !cache.IsCacheConfigured(volumeCacheConfig) { + prefix := volumeCacheConfig.Prefix // We use the same config as the query range results cache. r.QueryRange.VolumeCacheConfig.CacheConfig = r.QueryRange.ResultsCacheConfig.CacheConfig + r.QueryRange.VolumeCacheConfig.CacheConfig.Prefix = prefix + } + + seriesCacheConfig := r.QueryRange.SeriesCacheConfig.CacheConfig + if !cache.IsCacheConfigured(seriesCacheConfig) { + prefix := seriesCacheConfig.Prefix + r.QueryRange.SeriesCacheConfig.CacheConfig = r.QueryRange.ResultsCacheConfig.CacheConfig + r.QueryRange.SeriesCacheConfig.CacheConfig.Prefix = prefix + } + + labelsCacheConfig := r.QueryRange.LabelsCacheConfig.CacheConfig + if !cache.IsCacheConfigured(labelsCacheConfig) { + prefix := labelsCacheConfig.Prefix + r.QueryRange.LabelsCacheConfig.CacheConfig = r.QueryRange.ResultsCacheConfig.CacheConfig + r.QueryRange.LabelsCacheConfig.CacheConfig.Prefix = prefix } } diff --git a/pkg/loki/config_wrapper_test.go b/pkg/loki/config_wrapper_test.go index 7b3e8c9fff78..866079b71f60 100644 --- a/pkg/loki/config_wrapper_test.go +++ b/pkg/loki/config_wrapper_test.go @@ -937,6 +937,7 @@ query_range: config, _, _ := configWrapperFromYAML(t, configFileString, nil) assert.EqualValues(t, config.QueryRange.StatsCacheConfig.CacheConfig.Redis.Endpoint, "endpoint.redis.org") + assert.EqualValues(t, "frontend.index-stats-results-cache.", config.QueryRange.StatsCacheConfig.CacheConfig.Prefix) assert.False(t, config.QueryRange.StatsCacheConfig.CacheConfig.EmbeddedCache.Enabled) }) @@ -950,17 +951,20 @@ query_range: config, _, _ := configWrapperFromYAML(t, configFileString, nil) assert.EqualValues(t, "memcached.host.org", config.QueryRange.StatsCacheConfig.CacheConfig.MemcacheClient.Host) + assert.EqualValues(t, "frontend.index-stats-results-cache.", config.QueryRange.StatsCacheConfig.CacheConfig.Prefix) assert.False(t, config.QueryRange.StatsCacheConfig.CacheConfig.EmbeddedCache.Enabled) }) t.Run("embedded cache is enabled by default if no other cache is set", func(t *testing.T) { config, _, _ := configWrapperFromYAML(t, minimalConfig, nil) + assert.EqualValues(t, "frontend.index-stats-results-cache.", config.QueryRange.StatsCacheConfig.CacheConfig.Prefix) assert.True(t, config.QueryRange.StatsCacheConfig.CacheConfig.EmbeddedCache.Enabled) }) t.Run("gets results cache config if not configured directly", func(t *testing.T) { config, _, _ := configWrapperFromYAML(t, defaultResulsCacheString, nil) assert.EqualValues(t, "memcached.host.org", config.QueryRange.StatsCacheConfig.CacheConfig.MemcacheClient.Host) + assert.EqualValues(t, "frontend.index-stats-results-cache.", config.QueryRange.StatsCacheConfig.CacheConfig.Prefix) assert.False(t, config.QueryRange.StatsCacheConfig.CacheConfig.EmbeddedCache.Enabled) }) }) @@ -976,6 +980,7 @@ query_range: config, _, _ := configWrapperFromYAML(t, configFileString, nil) assert.EqualValues(t, config.QueryRange.VolumeCacheConfig.CacheConfig.Redis.Endpoint, "endpoint.redis.org") + assert.EqualValues(t, "frontend.volume-results-cache.", config.QueryRange.VolumeCacheConfig.CacheConfig.Prefix) assert.False(t, config.QueryRange.VolumeCacheConfig.CacheConfig.EmbeddedCache.Enabled) }) @@ -989,20 +994,109 @@ query_range: config, _, _ := configWrapperFromYAML(t, configFileString, nil) assert.EqualValues(t, "memcached.host.org", config.QueryRange.VolumeCacheConfig.CacheConfig.MemcacheClient.Host) + assert.EqualValues(t, "frontend.volume-results-cache.", config.QueryRange.VolumeCacheConfig.CacheConfig.Prefix) assert.False(t, config.QueryRange.VolumeCacheConfig.CacheConfig.EmbeddedCache.Enabled) }) t.Run("embedded cache is enabled by default if no other cache is set", func(t *testing.T) { config, _, _ := configWrapperFromYAML(t, minimalConfig, nil) + assert.EqualValues(t, "frontend.volume-results-cache.", config.QueryRange.VolumeCacheConfig.CacheConfig.Prefix) assert.True(t, config.QueryRange.VolumeCacheConfig.CacheConfig.EmbeddedCache.Enabled) }) t.Run("gets results cache config if not configured directly", func(t *testing.T) { config, _, _ := configWrapperFromYAML(t, defaultResulsCacheString, nil) assert.EqualValues(t, "memcached.host.org", config.QueryRange.VolumeCacheConfig.CacheConfig.MemcacheClient.Host) + assert.EqualValues(t, "frontend.volume-results-cache.", config.QueryRange.VolumeCacheConfig.CacheConfig.Prefix) assert.False(t, config.QueryRange.VolumeCacheConfig.CacheConfig.EmbeddedCache.Enabled) }) }) + + t.Run("for the series results cache config", func(t *testing.T) { + t.Run("no embedded cache enabled by default if Redis is set", func(t *testing.T) { + configFileString := `--- +query_range: + series_results_cache: + cache: + redis: + endpoint: endpoint.redis.org` + + config, _, _ := configWrapperFromYAML(t, configFileString, nil) + assert.EqualValues(t, "endpoint.redis.org", config.QueryRange.SeriesCacheConfig.CacheConfig.Redis.Endpoint) + assert.EqualValues(t, "frontend.series-results-cache.", config.QueryRange.SeriesCacheConfig.CacheConfig.Prefix) + assert.False(t, config.QueryRange.SeriesCacheConfig.CacheConfig.EmbeddedCache.Enabled) + }) + + t.Run("no embedded cache enabled by default if Memcache is set", func(t *testing.T) { + configFileString := `--- +query_range: + series_results_cache: + cache: + memcached_client: + host: memcached.host.org` + + config, _, _ := configWrapperFromYAML(t, configFileString, nil) + assert.EqualValues(t, "memcached.host.org", config.QueryRange.SeriesCacheConfig.CacheConfig.MemcacheClient.Host) + assert.EqualValues(t, "frontend.series-results-cache.", config.QueryRange.SeriesCacheConfig.CacheConfig.Prefix) + assert.False(t, config.QueryRange.SeriesCacheConfig.CacheConfig.EmbeddedCache.Enabled) + }) + + t.Run("embedded cache is enabled by default if no other cache is set", func(t *testing.T) { + config, _, _ := configWrapperFromYAML(t, minimalConfig, nil) + assert.True(t, config.QueryRange.SeriesCacheConfig.CacheConfig.EmbeddedCache.Enabled) + assert.EqualValues(t, "frontend.series-results-cache.", config.QueryRange.SeriesCacheConfig.CacheConfig.Prefix) + }) + + t.Run("gets results cache config if not configured directly", func(t *testing.T) { + config, _, _ := configWrapperFromYAML(t, defaultResulsCacheString, nil) + assert.EqualValues(t, "memcached.host.org", config.QueryRange.SeriesCacheConfig.CacheConfig.MemcacheClient.Host) + assert.EqualValues(t, "frontend.series-results-cache.", config.QueryRange.SeriesCacheConfig.CacheConfig.Prefix) + assert.False(t, config.QueryRange.SeriesCacheConfig.CacheConfig.EmbeddedCache.Enabled) + }) + }) + + t.Run("for the labels results cache config", func(t *testing.T) { + t.Run("no embedded cache enabled by default if Redis is set", func(t *testing.T) { + configFileString := `--- +query_range: + label_results_cache: + cache: + redis: + endpoint: endpoint.redis.org` + + config, _, _ := configWrapperFromYAML(t, configFileString, nil) + assert.EqualValues(t, "endpoint.redis.org", config.QueryRange.LabelsCacheConfig.CacheConfig.Redis.Endpoint) + assert.EqualValues(t, "frontend.label-results-cache.", config.QueryRange.LabelsCacheConfig.CacheConfig.Prefix) + assert.False(t, config.QueryRange.LabelsCacheConfig.CacheConfig.EmbeddedCache.Enabled) + }) + + t.Run("no embedded cache enabled by default if Memcache is set", func(t *testing.T) { + configFileString := `--- +query_range: + label_results_cache: + cache: + memcached_client: + host: memcached.host.org` + + config, _, _ := configWrapperFromYAML(t, configFileString, nil) + assert.EqualValues(t, "memcached.host.org", config.QueryRange.LabelsCacheConfig.CacheConfig.MemcacheClient.Host) + assert.EqualValues(t, "frontend.label-results-cache.", config.QueryRange.LabelsCacheConfig.CacheConfig.Prefix) + assert.False(t, config.QueryRange.LabelsCacheConfig.CacheConfig.EmbeddedCache.Enabled) + }) + + t.Run("embedded cache is enabled by default if no other cache is set", func(t *testing.T) { + config, _, _ := configWrapperFromYAML(t, minimalConfig, nil) + assert.True(t, config.QueryRange.LabelsCacheConfig.CacheConfig.EmbeddedCache.Enabled) + assert.EqualValues(t, "frontend.label-results-cache.", config.QueryRange.LabelsCacheConfig.CacheConfig.Prefix) + }) + + t.Run("gets results cache config if not configured directly", func(t *testing.T) { + config, _, _ := configWrapperFromYAML(t, defaultResulsCacheString, nil) + assert.EqualValues(t, "memcached.host.org", config.QueryRange.LabelsCacheConfig.CacheConfig.MemcacheClient.Host) + assert.EqualValues(t, "frontend.label-results-cache.", config.QueryRange.LabelsCacheConfig.CacheConfig.Prefix) + assert.False(t, config.QueryRange.LabelsCacheConfig.CacheConfig.EmbeddedCache.Enabled) + }) + }) } func TestDefaultUnmarshal(t *testing.T) { diff --git a/pkg/querier/queryrange/codec.go b/pkg/querier/queryrange/codec.go index d5a324779bf2..205cae586e09 100644 --- a/pkg/querier/queryrange/codec.go +++ b/pkg/querier/queryrange/codec.go @@ -231,10 +231,8 @@ func (r *LabelRequest) GetStep() int64 { func (r *LabelRequest) WithStartEnd(s, e time.Time) queryrangebase.Request { clone := *r - tmp := s - clone.Start = &tmp - tmp = e - clone.End = &tmp + clone.Start = &s + clone.End = &e return &clone } @@ -863,15 +861,17 @@ func decodeResponseJSONFrom(buf []byte, req queryrangebase.Request, headers http switch req := req.(type) { case *LokiSeriesRequest: - resp := &LokiSeriesResponse{ - Version: uint32(loghttp.GetVersion(req.Path)), - Headers: httpResponseHeadersToPromResponseHeaders(headers), - } + var resp LokiSeriesResponse if err := json.Unmarshal(buf, &resp); err != nil { return nil, httpgrpc.Errorf(http.StatusInternalServerError, "error decoding response: %v", err) } - return resp, nil + return &LokiSeriesResponse{ + Status: resp.Status, + Version: uint32(loghttp.GetVersion(req.Path)), + Headers: httpResponseHeadersToPromResponseHeaders(headers), + Data: resp.Data, + }, nil case *LabelRequest: var resp loghttp.LabelResponse if err := json.Unmarshal(buf, &resp); err != nil { @@ -1206,6 +1206,7 @@ func (Codec) MergeResponse(responses ...queryrangebase.Response) (queryrangebase Status: lokiSeriesRes.Status, Version: lokiSeriesRes.Version, Data: lokiSeriesData, + Headers: lokiSeriesRes.Headers, Statistics: mergedStats, }, nil case *LokiSeriesResponseView: @@ -1240,6 +1241,7 @@ func (Codec) MergeResponse(responses ...queryrangebase.Response) (queryrangebase return &LokiLabelNamesResponse{ Status: labelNameRes.Status, Version: labelNameRes.Version, + Headers: labelNameRes.Headers, Data: names, Statistics: mergedStats, }, nil diff --git a/pkg/querier/queryrange/codec_test.go b/pkg/querier/queryrange/codec_test.go index 215590f26776..936a702662c4 100644 --- a/pkg/querier/queryrange/codec_test.go +++ b/pkg/querier/queryrange/codec_test.go @@ -1612,6 +1612,24 @@ var ( "requests": 0, "downloadTime": 0 }, + "seriesResult": { + "entriesFound": 0, + "entriesRequested": 0, + "entriesStored": 0, + "bytesReceived": 0, + "bytesSent": 0, + "requests": 0, + "downloadTime": 0 + }, + "labelResult": { + "entriesFound": 0, + "entriesRequested": 0, + "entriesStored": 0, + "bytesReceived": 0, + "bytesSent": 0, + "requests": 0, + "downloadTime": 0 + }, "volumeResult": { "entriesFound": 0, "entriesRequested": 0, @@ -2006,6 +2024,8 @@ var ( Index: stats.Cache{}, StatsResult: stats.Cache{}, VolumeResult: stats.Cache{}, + SeriesResult: stats.Cache{}, + LabelResult: stats.Cache{}, Result: stats.Cache{}, }, } diff --git a/pkg/querier/queryrange/downstreamer.go b/pkg/querier/queryrange/downstreamer.go index 860aa980fb30..d8514e8a4ee7 100644 --- a/pkg/querier/queryrange/downstreamer.go +++ b/pkg/querier/queryrange/downstreamer.go @@ -9,6 +9,7 @@ import ( "time" "github.com/go-kit/log/level" + "github.com/grafana/dskit/concurrency" "github.com/grafana/dskit/tenant" "github.com/opentracing/opentracing-go" "github.com/prometheus/prometheus/model/labels" @@ -125,58 +126,46 @@ func (in instance) For( queries []logql.DownstreamQuery, fn func(logql.DownstreamQuery) (logqlmodel.Result, error), ) ([]logqlmodel.Result, error) { - type resp struct { - i int - res logqlmodel.Result - err error - } - ctx, cancel := context.WithCancel(ctx) defer cancel() - ch := make(chan resp) - // Make one goroutine to dispatch the other goroutines, bounded by instance parallelism + ch := make(chan logql.Resp) + + // ForEachJob blocks until all are done. However, we want to process the + // results as they come in. That's why we start everything in another + // gorouting. go func() { - for i := 0; i < len(queries); i++ { + err := concurrency.ForEachJob(ctx, len(queries), in.parallelism, func(ctx context.Context, i int) error { + res, err := fn(queries[i]) + response := logql.Resp{ + I: i, + Res: res, + Err: err, + } + + // Feed the result into the channel unless the work has completed. select { case <-ctx.Done(): - break - case <-in.locks: - go func(i int) { - // release lock back into pool - defer func() { - in.locks <- struct{}{} - }() - - res, err := fn(queries[i]) - response := resp{ - i: i, - res: res, - err: err, - } - - // Feed the result into the channel unless the work has completed. - select { - case <-ctx.Done(): - case ch <- response: - } - }(i) + case ch <- response: + } + return err + }) + if err != nil { + ch <- logql.Resp{ + I: -1, + Err: err, } } + close(ch) }() acc := newDownstreamAccumulator(queries[0].Params, len(queries)) - for i := 0; i < len(queries); i++ { - select { - case <-ctx.Done(): - return nil, ctx.Err() - case resp := <-ch: - if resp.err != nil { - return nil, resp.err - } - if err := acc.Accumulate(ctx, resp.i, resp.res); err != nil { - return nil, err - } + for resp := range ch { + if resp.Err != nil { + return nil, resp.Err + } + if err := acc.Accumulate(ctx, resp.I, resp.Res); err != nil { + return nil, err } } return acc.Result(), nil @@ -222,8 +211,8 @@ func sampleStreamToVector(streams []queryrangebase.SampleStream) parser.Value { return xs } -// downstreamAccumulator is one of two variants: -// a logsAccumulator or a bufferedAccumulator. +// downstreamAccumulator is one of three variants: +// a logsAccumulator, a bufferedAccumulator, or a quantileSketchAccumulator. // Which variant is detected on the first call to Accumulate. // Metric queries, which are generally small payloads, are buffered // since the memory overhead is negligible. @@ -232,6 +221,7 @@ func sampleStreamToVector(streams []queryrangebase.SampleStream) parser.Value { // accumulate the results into a logsAccumulator, discarding values // over the limit to keep memory pressure down while other subqueries // are executing. +// Sharded probabilistic quantile query results are merged as they come in. type downstreamAccumulator struct { acc resultAccumulator params logql.Params @@ -248,7 +238,8 @@ func newDownstreamAccumulator(params logql.Params, nQueries int) *downstreamAccu } func (a *downstreamAccumulator) build(acc logqlmodel.Result) { - if acc.Data.Type() == logqlmodel.ValueTypeStreams { + switch acc.Data.Type() { + case logqlmodel.ValueTypeStreams: // the stream accumulator stores a heap with reversed order // from the results we expect, so we need to reverse the direction @@ -258,8 +249,9 @@ func (a *downstreamAccumulator) build(acc logqlmodel.Result) { } a.acc = newStreamAccumulator(direction, int(a.params.Limit())) - - } else { + case logql.QuantileSketchMatrixType: + a.acc = newQuantileSketchAccumulator() + default: a.acc = &bufferedAccumulator{ results: make([]logqlmodel.Result, a.n), } @@ -297,6 +289,36 @@ func (a *bufferedAccumulator) Result() []logqlmodel.Result { return a.results } +type quantileSketchAccumulator struct { + matrix logql.ProbabilisticQuantileMatrix +} + +func newQuantileSketchAccumulator() *quantileSketchAccumulator { + return &quantileSketchAccumulator{} +} + +func (a *quantileSketchAccumulator) Accumulate(res logqlmodel.Result, _ int) error { + if res.Data.Type() != logql.QuantileSketchMatrixType { + return fmt.Errorf("unexpected matrix data type: got (%s), want (%s)", res.Data.Type(), logql.QuantileSketchMatrixType) + } + data, ok := res.Data.(logql.ProbabilisticQuantileMatrix) + if !ok { + return fmt.Errorf("unexpected matrix type: got (%T), want (ProbabilisticQuantileMatrix)", res.Data) + } + if a.matrix == nil { + a.matrix = data + return nil + } + + var err error + a.matrix, err = a.matrix.Merge(data) + return err +} + +func (a *quantileSketchAccumulator) Result() []logqlmodel.Result { + return []logqlmodel.Result{{Data: a.matrix}} +} + // heap impl for keeping only the top n results across m streams // importantly, accumulatedStreams is _bounded_, so it will only // store the top `limit` results across all streams. diff --git a/pkg/querier/queryrange/downstreamer_test.go b/pkg/querier/queryrange/downstreamer_test.go index e453f03d9a3e..007166c30c30 100644 --- a/pkg/querier/queryrange/downstreamer_test.go +++ b/pkg/querier/queryrange/downstreamer_test.go @@ -4,6 +4,7 @@ import ( "context" "errors" "fmt" + "math/rand" "strconv" "strings" "sync" @@ -18,6 +19,7 @@ import ( "github.com/grafana/loki/pkg/logproto" "github.com/grafana/loki/pkg/logql" + "github.com/grafana/loki/pkg/logql/sketch" "github.com/grafana/loki/pkg/logql/syntax" "github.com/grafana/loki/pkg/logqlmodel" "github.com/grafana/loki/pkg/logqlmodel/stats" @@ -598,3 +600,93 @@ func TestDownstreamAccumulatorMultiMerge(t *testing.T) { }) } } + +func BenchmarkAccumulator(b *testing.B) { + + // dummy params. Only need to populate direction & limit + lim := 30 + params, err := logql.NewLiteralParams( + `{app="foo"}`, time.Time{}, time.Time{}, 0, 0, logproto.BACKWARD, uint32(lim), nil, + ) + require.NoError(b, err) + + for acc, tc := range map[string]struct { + results []logqlmodel.Result + params logql.Params + }{ + "streams": { + newStreamResults(), + params, + }, + "quantile sketches": { + newQuantileSketchResults(), + params, + }, + } { + b.Run(acc, func(b *testing.B) { + b.ResetTimer() + b.ReportAllocs() + for n := 0; n < b.N; n++ { + + acc := newDownstreamAccumulator(params, len(tc.results)) + for i, r := range tc.results { + err := acc.Accumulate(context.Background(), i, r) + require.Nil(b, err) + } + + acc.Result() + } + }) + } +} + +func newStreamResults() []logqlmodel.Result { + nQueries := 50 + delta := 100 // 10 entries per stream, 1s apart + streamsPerQuery := 50 + + results := make([]logqlmodel.Result, nQueries) + for i := 0; i < nQueries; i++ { + start := i * delta + end := start + delta + streams := newStreams(time.Unix(int64(start), 0), time.Unix(int64(end), 0), time.Second, streamsPerQuery, logproto.BACKWARD) + var res logqlmodel.Streams + for i := range streams { + res = append(res, *streams[i]) + } + results[i] = logqlmodel.Result{Data: res} + + } + + return results +} + +func newQuantileSketchResults() []logqlmodel.Result { + results := make([]logqlmodel.Result, 100) + + for r := range results { + vectors := make([]logql.ProbabilisticQuantileVector, 10) + for i := range vectors { + vectors[i] = make(logql.ProbabilisticQuantileVector, 10) + for j := range vectors[i] { + vectors[i][j] = logql.ProbabilisticQuantileSample{ + T: int64(i), + F: newRandomSketch(), + Metric: []labels.Label{{Name: "foo", Value: fmt.Sprintf("bar-%d", j)}}, + } + } + } + results[r] = logqlmodel.Result{Data: logql.ProbabilisticQuantileMatrix(vectors)} + } + + return results +} + +func newRandomSketch() sketch.QuantileSketch { + r := rand.New(rand.NewSource(42)) + s := sketch.NewDDSketch() + for i := 0; i < 1000; i++ { + _ = s.Add(r.Float64()) + } + return s +} diff --git a/pkg/querier/queryrange/labels_cache.go b/pkg/querier/queryrange/labels_cache.go new file mode 100644 index 000000000000..1e0dd225fa7b --- /dev/null +++ b/pkg/querier/queryrange/labels_cache.go @@ -0,0 +1,99 @@ +package queryrange + +import ( + "context" + "flag" + "fmt" + "time" + + "github.com/go-kit/log" + + "github.com/grafana/loki/pkg/querier/queryrange/queryrangebase" + "github.com/grafana/loki/pkg/storage/chunk/cache" + "github.com/grafana/loki/pkg/storage/chunk/cache/resultscache" +) + +type cacheKeyLabels struct { + Limits + transformer UserIDTransformer +} + +// GenerateCacheKey generates a cache key based on the userID, split duration and the interval of the request. +// It also includes the label name and the provided query for label values request. +func (i cacheKeyLabels) GenerateCacheKey(ctx context.Context, userID string, r resultscache.Request) string { + lr := r.(*LabelRequest) + split := i.MetadataQuerySplitDuration(userID) + + var currentInterval int64 + if denominator := int64(split / time.Millisecond); denominator > 0 { + currentInterval = lr.GetStart().UnixMilli() / denominator + } + + if i.transformer != nil { + userID = i.transformer(ctx, userID) + } + + if lr.GetValues() { + return fmt.Sprintf("labelvalues:%s:%s:%s:%d:%d", userID, lr.GetName(), lr.GetQuery(), currentInterval, split) + } + + return fmt.Sprintf("labels:%s:%d:%d", userID, currentInterval, split) +} + +type labelsExtractor struct{} + +// Extract extracts the labels response for the specific time range. +// It is a no-op since it is not possible to partition the labels data by time range as it is just a slice of strings. +func (p labelsExtractor) Extract(_, _ int64, res resultscache.Response, _, _ int64) resultscache.Response { + return res +} + +func (p labelsExtractor) ResponseWithoutHeaders(resp queryrangebase.Response) queryrangebase.Response { + labelsResp := resp.(*LokiLabelNamesResponse) + return &LokiLabelNamesResponse{ + Status: labelsResp.Status, + Data: labelsResp.Data, + Version: labelsResp.Version, + Statistics: labelsResp.Statistics, + } +} + +type LabelsCacheConfig struct { + queryrangebase.ResultsCacheConfig `yaml:",inline"` +} + +// RegisterFlags registers flags. +func (cfg *LabelsCacheConfig) RegisterFlags(f *flag.FlagSet) { + cfg.RegisterFlagsWithPrefix(f, "frontend.label-results-cache.") +} + +func (cfg *LabelsCacheConfig) Validate() error { + return cfg.ResultsCacheConfig.Validate() +} + +func NewLabelsCacheMiddleware( + logger log.Logger, + limits Limits, + merger queryrangebase.Merger, + c cache.Cache, + cacheGenNumberLoader queryrangebase.CacheGenNumberLoader, + shouldCache queryrangebase.ShouldCacheFn, + parallelismForReq queryrangebase.ParallelismForReqFn, + retentionEnabled bool, + transformer UserIDTransformer, + metrics *queryrangebase.ResultsCacheMetrics, +) (queryrangebase.Middleware, error) { + return queryrangebase.NewResultsCacheMiddleware( + logger, + c, + cacheKeyLabels{limits, transformer}, + limits, + merger, + labelsExtractor{}, + cacheGenNumberLoader, + shouldCache, + parallelismForReq, + retentionEnabled, + metrics, + ) +} diff --git a/pkg/querier/queryrange/labels_cache_test.go b/pkg/querier/queryrange/labels_cache_test.go new file mode 100644 index 000000000000..73ab9ad8f4f8 --- /dev/null +++ b/pkg/querier/queryrange/labels_cache_test.go @@ -0,0 +1,251 @@ +package queryrange + +import ( + "context" + "fmt" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/grafana/dskit/user" + "github.com/stretchr/testify/require" + + "github.com/grafana/loki/pkg/loghttp" + "github.com/grafana/loki/pkg/logproto" + "github.com/grafana/loki/pkg/logqlmodel/stats" + "github.com/grafana/loki/pkg/querier/queryrange/queryrangebase" + "github.com/grafana/loki/pkg/storage/chunk/cache" + "github.com/grafana/loki/pkg/util" +) + +func TestCacheKeyLabels_GenerateCacheKey(t *testing.T) { + k := cacheKeyLabels{ + transformer: nil, + Limits: fakeLimits{ + metadataSplitDuration: map[string]time.Duration{ + "fake": time.Hour, + }, + }, + } + + from, through := util.RoundToMilliseconds(testTime, testTime.Add(1*time.Hour)) + start := from.Time() + end := through.Time() + + req := LabelRequest{ + LabelRequest: logproto.LabelRequest{ + Start: &start, + End: &end, + }, + } + + expectedInterval := testTime.UnixMilli() / time.Hour.Milliseconds() + + t.Run("labels", func(t *testing.T) { + require.Equal(t, fmt.Sprintf(`labels:fake:%d:%d`, expectedInterval, time.Hour.Nanoseconds()), k.GenerateCacheKey(context.Background(), "fake", &req)) + }) + + t.Run("label values", func(t *testing.T) { + req := req + req.Name = "foo" + req.Values = true + require.Equal(t, fmt.Sprintf(`labelvalues:fake:foo::%d:%d`, expectedInterval, time.Hour.Nanoseconds()), k.GenerateCacheKey(context.Background(), "fake", &req)) + + req.Query = `{cluster="eu-west1"}` + require.Equal(t, fmt.Sprintf(`labelvalues:fake:foo:{cluster="eu-west1"}:%d:%d`, expectedInterval, time.Hour.Nanoseconds()), k.GenerateCacheKey(context.Background(), "fake", &req)) + }) +} + +func TestLabelsCache(t *testing.T) { + setupCacheMW := func() queryrangebase.Middleware { + cacheMiddleware, err := NewLabelsCacheMiddleware( + log.NewNopLogger(), + fakeLimits{ + metadataSplitDuration: map[string]time.Duration{ + "fake": 24 * time.Hour, + }, + }, + DefaultCodec, + cache.NewMockCache(), + nil, + nil, + func(_ context.Context, _ []string, _ queryrangebase.Request) int { + return 1 + }, + false, + nil, + nil, + ) + require.NoError(t, err) + return cacheMiddleware + } + + cacheMiddleware := setupCacheMW() + for _, values := range []bool{false, true} { + prefix := "labels" + if values { + prefix = "label values" + } + t.Run(prefix+": cache the response for the same request", func(t *testing.T) { + start := testTime.Truncate(time.Millisecond) + end := start.Add(time.Hour) + + labelsReq := LabelRequest{ + LabelRequest: logproto.LabelRequest{ + Start: &start, + End: &end, + }, + } + + if values { + labelsReq.Values = true + labelsReq.Name = "foo" + labelsReq.Query = `{cluster="eu-west1"}` + } + + labelsResp := &LokiLabelNamesResponse{ + Status: "success", + Version: uint32(loghttp.VersionV1), + Data: []string{"bar", "buzz"}, + Statistics: stats.Result{ + Summary: stats.Summary{ + Splits: 1, + }, + }, + } + + called := 0 + handler := cacheMiddleware.Wrap(queryrangebase.HandlerFunc(func(_ context.Context, r queryrangebase.Request) (queryrangebase.Response, error) { + called++ + + // should request the entire length with no partitioning as nothing is cached yet. + require.Equal(t, labelsReq.GetStart(), r.GetStart()) + require.Equal(t, labelsReq.GetEnd(), r.GetEnd()) + + got := r.(*LabelRequest) + require.Equal(t, labelsReq.GetName(), got.GetName()) + require.Equal(t, labelsReq.GetValues(), got.GetValues()) + require.Equal(t, labelsReq.GetQuery(), got.GetQuery()) + + return labelsResp, nil + })) + + ctx := user.InjectOrgID(context.Background(), "fake") + got, err := handler.Do(ctx, &labelsReq) + require.NoError(t, err) + require.Equal(t, 1, called) // called actual handler, as not cached. + require.Equal(t, labelsResp, got) + + // Doing same request again shouldn't change anything. + called = 0 + got, err = handler.Do(ctx, &labelsReq) + require.NoError(t, err) + require.Equal(t, 0, called) + require.Equal(t, labelsResp, got) + }) + } + + // reset cacheMiddleware + cacheMiddleware = setupCacheMW() + for _, values := range []bool{false, true} { + prefix := "labels" + if values { + prefix = "label values" + } + t.Run(prefix+": a new request with overlapping time range should reuse part of the previous request for the overlap", func(t *testing.T) { + cacheMiddleware := setupCacheMW() + + start := testTime.Truncate(time.Millisecond) + end := start.Add(time.Hour) + + labelsReq1 := LabelRequest{ + LabelRequest: logproto.LabelRequest{ + Start: &start, + End: &end, + }, + } + + if values { + labelsReq1.Values = true + labelsReq1.Name = "foo" + labelsReq1.Query = `{cluster="eu-west1"}` + } + + labelsResp1 := &LokiLabelNamesResponse{ + Status: "success", + Version: uint32(loghttp.VersionV1), + Data: []string{"bar", "buzz"}, + Statistics: stats.Result{ + Summary: stats.Summary{ + Splits: 1, + }, + }, + } + + called := 0 + handler := cacheMiddleware.Wrap(queryrangebase.HandlerFunc(func(_ context.Context, r queryrangebase.Request) (queryrangebase.Response, error) { + called++ + + // should request the entire length with no partitioning as nothing is cached yet. + require.Equal(t, labelsReq1.GetStart(), r.GetStart()) + require.Equal(t, labelsReq1.GetEnd(), r.GetEnd()) + + got := r.(*LabelRequest) + require.Equal(t, labelsReq1.GetName(), got.GetName()) + require.Equal(t, labelsReq1.GetValues(), got.GetValues()) + require.Equal(t, labelsReq1.GetQuery(), got.GetQuery()) + + return labelsResp1, nil + })) + + ctx := user.InjectOrgID(context.Background(), "fake") + got, err := handler.Do(ctx, &labelsReq1) + require.NoError(t, err) + require.Equal(t, 1, called) + require.Equal(t, labelsResp1, got) + + labelsReq2 := labelsReq1.WithStartEnd(labelsReq1.GetStart().Add(15*time.Minute), labelsReq1.GetEnd().Add(15*time.Minute)) + + called = 0 + handler = cacheMiddleware.Wrap(queryrangebase.HandlerFunc(func(_ context.Context, r queryrangebase.Request) (queryrangebase.Response, error) { + called++ + + // make downstream request only for the non-overlapping portion of the query. + require.Equal(t, labelsReq1.GetEnd(), r.GetStart()) + require.Equal(t, labelsReq2.GetEnd(), r.GetEnd()) + + got := r.(*LabelRequest) + require.Equal(t, labelsReq1.GetName(), got.GetName()) + require.Equal(t, labelsReq1.GetValues(), got.GetValues()) + require.Equal(t, labelsReq1.GetQuery(), got.GetQuery()) + + return &LokiLabelNamesResponse{ + Status: "success", + Version: uint32(loghttp.VersionV1), + Data: []string{"fizz"}, + Statistics: stats.Result{ + Summary: stats.Summary{ + Splits: 1, + }, + }, + }, nil + })) + + got, err = handler.Do(ctx, labelsReq2) + require.NoError(t, err) + require.Equal(t, 1, called) + // two splits as we merge the results from the extent and downstream request + labelsResp1.Statistics.Summary.Splits = 2 + require.Equal(t, &LokiLabelNamesResponse{ + Status: "success", + Version: uint32(loghttp.VersionV1), + Data: []string{"bar", "buzz", "fizz"}, + Statistics: stats.Result{ + Summary: stats.Summary{ + Splits: 2, + }, + }, + }, got) + }) + } +} diff --git a/pkg/querier/queryrange/limits/definitions.go b/pkg/querier/queryrange/limits/definitions.go index bc8f7d0ec94b..bd84e144fa47 100644 --- a/pkg/querier/queryrange/limits/definitions.go +++ b/pkg/querier/queryrange/limits/definitions.go @@ -14,6 +14,7 @@ type Limits interface { queryrangebase.Limits logql.Limits QuerySplitDuration(string) time.Duration + MetadataQuerySplitDuration(string) time.Duration MaxQuerySeries(context.Context, string) int MaxEntriesLimitPerQuery(context.Context, string) int MinShardingLookback(string) time.Duration diff --git a/pkg/querier/queryrange/limits_test.go b/pkg/querier/queryrange/limits_test.go index efc9b030f7f8..3b82c1dc9eab 100644 --- a/pkg/querier/queryrange/limits_test.go +++ b/pkg/querier/queryrange/limits_test.go @@ -29,7 +29,7 @@ import ( func TestLimits(t *testing.T) { l := fakeLimits{ - splits: map[string]time.Duration{"a": time.Minute}, + splitDuration: map[string]time.Duration{"a": time.Minute}, } wrapped := WithSplitByLimits(l, time.Hour) diff --git a/pkg/querier/queryrange/log_result_cache_test.go b/pkg/querier/queryrange/log_result_cache_test.go index 815e24b77ea9..5d67be33b84f 100644 --- a/pkg/querier/queryrange/log_result_cache_test.go +++ b/pkg/querier/queryrange/log_result_cache_test.go @@ -32,7 +32,7 @@ func Test_LogResultCacheSameRange(t *testing.T) { lrc = NewLogResultCache( log.NewNopLogger(), fakeLimits{ - splits: map[string]time.Duration{"foo": time.Minute}, + splitDuration: map[string]time.Duration{"foo": time.Minute}, }, cache.NewMockCache(), nil, @@ -74,7 +74,7 @@ func Test_LogResultCacheSameRangeNonEmpty(t *testing.T) { lrc = NewLogResultCache( log.NewNopLogger(), fakeLimits{ - splits: map[string]time.Duration{"foo": time.Minute}, + splitDuration: map[string]time.Duration{"foo": time.Minute}, }, cache.NewMockCache(), nil, @@ -122,7 +122,7 @@ func Test_LogResultCacheSmallerRange(t *testing.T) { lrc = NewLogResultCache( log.NewNopLogger(), fakeLimits{ - splits: map[string]time.Duration{"foo": time.Minute}, + splitDuration: map[string]time.Duration{"foo": time.Minute}, }, cache.NewMockCache(), nil, @@ -172,7 +172,7 @@ func Test_LogResultCacheDifferentRange(t *testing.T) { lrc = NewLogResultCache( log.NewNopLogger(), fakeLimits{ - splits: map[string]time.Duration{"foo": time.Minute}, + splitDuration: map[string]time.Duration{"foo": time.Minute}, }, cache.NewMockCache(), nil, @@ -248,7 +248,7 @@ func Test_LogResultCacheDifferentRangeNonEmpty(t *testing.T) { lrc = NewLogResultCache( log.NewNopLogger(), fakeLimits{ - splits: map[string]time.Duration{"foo": time.Minute}, + splitDuration: map[string]time.Duration{"foo": time.Minute}, }, cache.NewMockCache(), nil, @@ -335,7 +335,7 @@ func Test_LogResultCacheDifferentRangeNonEmptyAndEmpty(t *testing.T) { lrc = NewLogResultCache( log.NewNopLogger(), fakeLimits{ - splits: map[string]time.Duration{"foo": time.Minute}, + splitDuration: map[string]time.Duration{"foo": time.Minute}, }, cache.NewMockCache(), nil, @@ -445,7 +445,7 @@ func Test_LogResultNonOverlappingCache(t *testing.T) { lrc = NewLogResultCache( log.NewNopLogger(), fakeLimits{ - splits: map[string]time.Duration{"foo": time.Minute}, + splitDuration: map[string]time.Duration{"foo": time.Minute}, }, mockCache, nil, diff --git a/pkg/querier/queryrange/prometheus_test.go b/pkg/querier/queryrange/prometheus_test.go index c0529906241a..98eb563ca7bd 100644 --- a/pkg/querier/queryrange/prometheus_test.go +++ b/pkg/querier/queryrange/prometheus_test.go @@ -85,6 +85,24 @@ var emptyStats = `"stats": { "requests": 0, "downloadTime": 0 }, + "seriesResult": { + "entriesFound": 0, + "entriesRequested": 0, + "entriesStored": 0, + "bytesReceived": 0, + "bytesSent": 0, + "requests": 0, + "downloadTime": 0 + }, + "labelResult": { + "entriesFound": 0, + "entriesRequested": 0, + "entriesStored": 0, + "bytesReceived": 0, + "bytesSent": 0, + "requests": 0, + "downloadTime": 0 + }, "volumeResult": { "entriesFound": 0, "entriesRequested": 0, diff --git a/pkg/querier/queryrange/roundtrip.go b/pkg/querier/queryrange/roundtrip.go index c03d459ba9b2..5f0aef4a1ab4 100644 --- a/pkg/querier/queryrange/roundtrip.go +++ b/pkg/querier/queryrange/roundtrip.go @@ -3,6 +3,7 @@ package queryrange import ( "context" "flag" + "fmt" "net/http" "strings" "time" @@ -28,6 +29,19 @@ import ( logutil "github.com/grafana/loki/pkg/util/log" ) +const ( + // Parallelize the index stats requests, so it doesn't send a huge request to a single index-gw (i.e. {app=~".+"} for 30d). + // Indices are sharded by 24 hours, so we split the stats request in 24h intervals. + indexStatsQuerySplitInterval = 24 * time.Hour + + // Limited queries only need to fetch up to the requested line limit worth of logs, + // Our defaults for splitting and parallelism are much too aggressive for large customers and result in + // potentially GB of logs being returned by all the shards and splits which will overwhelm the frontend + // Therefore we force max parallelism to `1` so that these queries are executed sequentially. + // Below we also fix the number of shards to a static number. + limitedQuerySplits = 1 +) + // Config is the configuration for the queryrange tripperware type Config struct { base.Config `yaml:",inline"` @@ -36,6 +50,10 @@ type Config struct { StatsCacheConfig IndexStatsCacheConfig `yaml:"index_stats_results_cache" doc:"description=If a cache config is not specified and cache_index_stats_results is true, the config for the results cache is used."` CacheVolumeResults bool `yaml:"cache_volume_results"` VolumeCacheConfig VolumeCacheConfig `yaml:"volume_results_cache" doc:"description=If a cache config is not specified and cache_volume_results is true, the config for the results cache is used."` + CacheSeriesResults bool `yaml:"cache_series_results"` + SeriesCacheConfig SeriesCacheConfig `yaml:"series_results_cache" doc:"description=If series_results_cache is not configured and cache_series_results is true, the config for the results cache is used."` + CacheLabelResults bool `yaml:"cache_label_results"` + LabelsCacheConfig LabelsCacheConfig `yaml:"label_results_cache" doc:"description=If label_results_cache is not configured and cache_label_results is true, the config for the results cache is used."` } // RegisterFlags adds the flags required to configure this flag set. @@ -45,6 +63,10 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) { cfg.StatsCacheConfig.RegisterFlags(f) f.BoolVar(&cfg.CacheVolumeResults, "querier.cache-volume-results", false, "Cache volume query results.") cfg.VolumeCacheConfig.RegisterFlags(f) + f.BoolVar(&cfg.CacheSeriesResults, "querier.cache-series-results", false, "Cache series query results.") + cfg.SeriesCacheConfig.RegisterFlags(f) + f.BoolVar(&cfg.CacheLabelResults, "querier.cache-label-results", false, "Cache label query results.") + cfg.LabelsCacheConfig.RegisterFlags(f) } // Validate validates the config. @@ -112,6 +134,8 @@ func NewMiddleware( resultsCache cache.Cache statsCache cache.Cache volumeCache cache.Cache + seriesCache cache.Cache + labelsCache cache.Cache err error ) @@ -123,28 +147,28 @@ func NewMiddleware( } if cfg.CacheIndexStatsResults { - // If the stats cache is not configured, use the results cache config. - cacheCfg := cfg.StatsCacheConfig.ResultsCacheConfig - if !cache.IsCacheConfigured(cacheCfg.CacheConfig) { - level.Debug(log).Log("msg", "using results cache config for stats cache") - cacheCfg = cfg.ResultsCacheConfig + statsCache, err = newResultsCacheFromConfig(cfg.StatsCacheConfig.ResultsCacheConfig, registerer, log, stats.StatsResultCache) + if err != nil { + return nil, nil, err } + } - statsCache, err = newResultsCacheFromConfig(cacheCfg, registerer, log, stats.StatsResultCache) + if cfg.CacheVolumeResults { + volumeCache, err = newResultsCacheFromConfig(cfg.VolumeCacheConfig.ResultsCacheConfig, registerer, log, stats.VolumeResultCache) if err != nil { return nil, nil, err } } - if cfg.CacheVolumeResults { - // If the volume cache is not configured, use the results cache config. - cacheCfg := cfg.VolumeCacheConfig.ResultsCacheConfig - if !cache.IsCacheConfigured(cacheCfg.CacheConfig) { - level.Debug(log).Log("msg", "using results cache config for volume cache") - cacheCfg = cfg.ResultsCacheConfig + if cfg.CacheSeriesResults { + seriesCache, err = newResultsCacheFromConfig(cfg.SeriesCacheConfig.ResultsCacheConfig, registerer, log, stats.SeriesResultCache) + if err != nil { + return nil, nil, err } + } - volumeCache, err = newResultsCacheFromConfig(cacheCfg, registerer, log, stats.VolumeResultCache) + if cfg.CacheLabelResults { + labelsCache, err = newResultsCacheFromConfig(cfg.LabelsCacheConfig.ResultsCacheConfig, registerer, log, stats.LabelResultCache) if err != nil { return nil, nil, err } @@ -176,12 +200,12 @@ func NewMiddleware( return nil, nil, err } - seriesTripperware, err := NewSeriesTripperware(cfg, log, limits, metrics, schema, DefaultCodec, metricsNamespace) + seriesTripperware, err := NewSeriesTripperware(cfg, log, limits, metrics, schema, codec, seriesCache, cacheGenNumLoader, retentionEnabled, metricsNamespace) if err != nil { return nil, nil, err } - labelsTripperware, err := NewLabelsTripperware(cfg, log, limits, codec, metrics, schema, metricsNamespace) + labelsTripperware, err := NewLabelsTripperware(cfg, log, limits, codec, labelsCache, cacheGenNumLoader, retentionEnabled, metrics, schema, metricsNamespace) if err != nil { return nil, nil, err } @@ -475,12 +499,7 @@ func NewLimitedTripperware( NewLimitsMiddleware(limits), NewQuerySizeLimiterMiddleware(schema.Configs, engineOpts, log, limits, statsHandler), base.InstrumentMiddleware("split_by_interval", metrics.InstrumentMiddlewareMetrics), - // Limited queries only need to fetch up to the requested line limit worth of logs, - // Our defaults for splitting and parallelism are much too aggressive for large customers and result in - // potentially GB of logs being returned by all the shards and splits which will overwhelm the frontend - // Therefore we force max parallelism to one so that these queries are executed sequentially. - // Below we also fix the number of shards to a static number. - SplitByIntervalMiddleware(schema.Configs, WithMaxParallelism(limits, 1), merger, splitByTime, metrics.SplitByMetrics), + SplitByIntervalMiddleware(schema.Configs, WithMaxParallelism(limits, limitedQuerySplits), merger, splitByTime, metrics.SplitByMetrics), NewQuerierSizeLimiterMiddleware(schema.Configs, engineOpts, log, limits, statsHandler), } @@ -499,16 +518,56 @@ func NewSeriesTripperware( metrics *Metrics, schema config.SchemaConfig, merger base.Merger, + c cache.Cache, + cacheGenNumLoader base.CacheGenNumberLoader, + retentionEnabled bool, metricsNamespace string, ) (base.Middleware, error) { + var cacheMiddleware base.Middleware + if cfg.CacheSeriesResults { + var err error + cacheMiddleware, err = NewSeriesCacheMiddleware( + log, + limits, + merger, + c, + cacheGenNumLoader, + func(_ context.Context, r base.Request) bool { + return !r.GetCachingOptions().Disabled + }, + func(ctx context.Context, tenantIDs []string, r base.Request) int { + return MinWeightedParallelism( + ctx, + tenantIDs, + schema.Configs, + limits, + model.Time(r.GetStart().UnixMilli()), + model.Time(r.GetEnd().UnixMilli()), + ) + }, + retentionEnabled, + cfg.Transformer, + metrics.ResultsCacheMetrics, + ) + if err != nil { + return nil, fmt.Errorf("failed to create series cache middleware: %w", err) + } + } + queryRangeMiddleware := []base.Middleware{ StatsCollectorMiddleware(), NewLimitsMiddleware(limits), base.InstrumentMiddleware("split_by_interval", metrics.InstrumentMiddlewareMetrics), - // The Series API needs to pull one chunk per series to extract the label set, which is much cheaper than iterating through all matching chunks. - // Force a 24 hours split by for series API, this will be more efficient with our static daily bucket storage. - // This would avoid queriers downloading chunks for same series over and over again for serving smaller queries. - SplitByIntervalMiddleware(schema.Configs, WithSplitByLimits(limits, 24*time.Hour), merger, splitByTime, metrics.SplitByMetrics), + SplitByIntervalMiddleware(schema.Configs, limits, merger, splitByTime, metrics.SplitByMetrics), + } + + if cfg.CacheSeriesResults { + queryRangeMiddleware = append( + queryRangeMiddleware, + base.InstrumentMiddleware("series_results_cache", metrics.InstrumentMiddlewareMetrics), + cacheMiddleware, + ) + } if cfg.MaxRetries > 0 { @@ -542,17 +601,58 @@ func NewLabelsTripperware( log log.Logger, limits Limits, merger base.Merger, + c cache.Cache, + cacheGenNumLoader base.CacheGenNumberLoader, + retentionEnabled bool, metrics *Metrics, schema config.SchemaConfig, metricsNamespace string, ) (base.Middleware, error) { + var cacheMiddleware base.Middleware + if cfg.CacheLabelResults { + var err error + cacheMiddleware, err = NewLabelsCacheMiddleware( + log, + limits, + merger, + c, + cacheGenNumLoader, + func(_ context.Context, r base.Request) bool { + return !r.GetCachingOptions().Disabled + }, + func(ctx context.Context, tenantIDs []string, r base.Request) int { + return MinWeightedParallelism( + ctx, + tenantIDs, + schema.Configs, + limits, + model.Time(r.GetStart().UnixMilli()), + model.Time(r.GetEnd().UnixMilli()), + ) + }, + retentionEnabled, + cfg.Transformer, + metrics.ResultsCacheMetrics, + ) + if err != nil { + return nil, fmt.Errorf("failed to create labels cache middleware: %w", err) + } + } + queryRangeMiddleware := []base.Middleware{ StatsCollectorMiddleware(), NewLimitsMiddleware(limits), base.InstrumentMiddleware("split_by_interval", metrics.InstrumentMiddlewareMetrics), - // Force a 24 hours split by for labels API, this will be more efficient with our static daily bucket storage. - // This is because the labels API is an index-only operation. - SplitByIntervalMiddleware(schema.Configs, WithSplitByLimits(limits, 24*time.Hour), merger, splitByTime, metrics.SplitByMetrics), + SplitByIntervalMiddleware(schema.Configs, limits, merger, splitByTime, metrics.SplitByMetrics), + } + + if cfg.CacheLabelResults { + queryRangeMiddleware = append( + queryRangeMiddleware, + base.InstrumentMiddleware("label_results_cache", metrics.InstrumentMiddlewareMetrics), + cacheMiddleware, + ) + } if cfg.MaxRetries > 0 { @@ -874,9 +974,7 @@ func NewIndexStatsTripperware( metrics *Metrics, metricsNamespace string, ) (base.Middleware, error) { - // Parallelize the index stats requests, so it doesn't send a huge request to a single index-gw (i.e. {app=~".+"} for 30d). - // Indices are sharded by 24 hours, so we split the stats request in 24h intervals. - limits = WithSplitByLimits(limits, 24*time.Hour) + limits = WithSplitByLimits(limits, indexStatsQuerySplitInterval) var cacheMiddleware base.Middleware if cfg.CacheIndexStatsResults { diff --git a/pkg/querier/queryrange/roundtrip_test.go b/pkg/querier/queryrange/roundtrip_test.go index c0b05103ded3..883f9b14226b 100644 --- a/pkg/querier/queryrange/roundtrip_test.go +++ b/pkg/querier/queryrange/roundtrip_test.go @@ -396,7 +396,14 @@ func TestInstantQueryTripperware(t *testing.T) { } func TestSeriesTripperware(t *testing.T) { - tpw, stopper, err := NewMiddleware(testConfig, testEngineOpts, util_log.Logger, fakeLimits{maxQueryLength: 48 * time.Hour, maxQueryParallelism: 1}, config.SchemaConfig{Configs: testSchemas}, nil, false, nil, constants.Loki) + l := fakeLimits{ + maxQueryLength: 48 * time.Hour, + maxQueryParallelism: 1, + metadataSplitDuration: map[string]time.Duration{ + "1": 24 * time.Hour, + }, + } + tpw, stopper, err := NewMiddleware(testConfig, testEngineOpts, util_log.Logger, l, config.SchemaConfig{Configs: testSchemas}, nil, false, nil, constants.Loki) if stopper != nil { defer stopper.Stop() } @@ -427,7 +434,14 @@ func TestSeriesTripperware(t *testing.T) { } func TestLabelsTripperware(t *testing.T) { - tpw, stopper, err := NewMiddleware(testConfig, testEngineOpts, util_log.Logger, fakeLimits{maxQueryLength: 48 * time.Hour, maxQueryParallelism: 1}, config.SchemaConfig{Configs: testSchemas}, nil, false, nil, constants.Loki) + l := fakeLimits{ + maxQueryLength: 48 * time.Hour, + maxQueryParallelism: 1, + metadataSplitDuration: map[string]time.Duration{ + "1": 24 * time.Hour, + }, + } + tpw, stopper, err := NewMiddleware(testConfig, testEngineOpts, util_log.Logger, l, config.SchemaConfig{Configs: testSchemas}, nil, false, nil, constants.Loki) if stopper != nil { defer stopper.Stop() } @@ -679,7 +693,7 @@ func TestNewTripperware_Caches(t *testing.T) { err: "", }, { - name: "results cache enabled, stats cache disabled", + name: "results cache enabled", config: Config{ Config: base.Config{ CacheResults: true, @@ -694,34 +708,32 @@ func TestNewTripperware_Caches(t *testing.T) { }, }, }, - CacheIndexStatsResults: false, }, numCaches: 1, err: "", }, { - name: "results cache enabled, stats cache enabled", + name: "stats cache enabled", config: Config{ - Config: base.Config{ - CacheResults: true, + CacheIndexStatsResults: true, + StatsCacheConfig: IndexStatsCacheConfig{ ResultsCacheConfig: base.ResultsCacheConfig{ Config: resultscache.Config{ CacheConfig: cache.Config{ EmbeddedCache: cache.EmbeddedCacheConfig{ - MaxSizeMB: 1, Enabled: true, + MaxSizeMB: 1000, }, }, }, }, }, - CacheIndexStatsResults: true, }, - numCaches: 2, + numCaches: 1, err: "", }, { - name: "results cache enabled, stats cache enabled but different", + name: "results cache enabled, stats cache enabled", config: Config{ Config: base.Config{ CacheResults: true, @@ -763,11 +775,8 @@ func TestNewTripperware_Caches(t *testing.T) { err: fmt.Sprintf("%s cache is not configured", stats.ResultCache), }, { - name: "results cache disabled, stats cache enabled (no config provided)", + name: "stats cache enabled (no config provided)", config: Config{ - Config: base.Config{ - CacheResults: false, - }, CacheIndexStatsResults: true, }, numCaches: 0, @@ -1234,7 +1243,8 @@ type fakeLimits struct { maxQueryLookback time.Duration maxEntriesLimitPerQuery int maxSeries int - splits map[string]time.Duration + splitDuration map[string]time.Duration + metadataSplitDuration map[string]time.Duration minShardingLookback time.Duration queryTimeout time.Duration requiredLabels []string @@ -1246,10 +1256,17 @@ type fakeLimits struct { } func (f fakeLimits) QuerySplitDuration(key string) time.Duration { - if f.splits == nil { + if f.splitDuration == nil { + return 0 + } + return f.splitDuration[key] +} + +func (f fakeLimits) MetadataQuerySplitDuration(key string) time.Duration { + if f.metadataSplitDuration == nil { return 0 } - return f.splits[key] + return f.metadataSplitDuration[key] } func (f fakeLimits) MaxQueryLength(context.Context, string) time.Duration { diff --git a/pkg/querier/queryrange/series_cache.go b/pkg/querier/queryrange/series_cache.go new file mode 100644 index 000000000000..9ad67f70acf5 --- /dev/null +++ b/pkg/querier/queryrange/series_cache.go @@ -0,0 +1,100 @@ +package queryrange + +import ( + "context" + "flag" + "fmt" + "sort" + strings "strings" + "time" + + "github.com/go-kit/log" + + "github.com/grafana/loki/pkg/querier/queryrange/queryrangebase" + "github.com/grafana/loki/pkg/storage/chunk/cache" + "github.com/grafana/loki/pkg/storage/chunk/cache/resultscache" +) + +type cacheKeySeries struct { + Limits + transformer UserIDTransformer +} + +// GenerateCacheKey generates a cache key based on the userID, matchers, split duration and the interval of the request. +func (i cacheKeySeries) GenerateCacheKey(ctx context.Context, userID string, r resultscache.Request) string { + sr := r.(*LokiSeriesRequest) + split := i.MetadataQuerySplitDuration(userID) + + var currentInterval int64 + if denominator := int64(split / time.Millisecond); denominator > 0 { + currentInterval = sr.GetStart().UnixMilli() / denominator + } + + if i.transformer != nil { + userID = i.transformer(ctx, userID) + } + + matchers := sr.GetMatch() + sort.Strings(matchers) + matcherStr := strings.Join(matchers, ",") + + return fmt.Sprintf("series:%s:%s:%d:%d", userID, matcherStr, currentInterval, split) +} + +type seriesExtractor struct{} + +// Extract extracts the series response for the specific time range. +// It is a no-op since it is not possible to partition the series data by time range as it is just a list of kv pairs. +func (p seriesExtractor) Extract(_, _ int64, res resultscache.Response, _, _ int64) resultscache.Response { + return res +} + +func (p seriesExtractor) ResponseWithoutHeaders(resp queryrangebase.Response) queryrangebase.Response { + seriesRes := resp.(*LokiSeriesResponse) + return &LokiSeriesResponse{ + Data: seriesRes.Data, + Status: seriesRes.Status, + Version: seriesRes.Version, + Statistics: seriesRes.Statistics, + } +} + +type SeriesCacheConfig struct { + queryrangebase.ResultsCacheConfig `yaml:",inline"` +} + +// RegisterFlags registers flags. +func (cfg *SeriesCacheConfig) RegisterFlags(f *flag.FlagSet) { + cfg.RegisterFlagsWithPrefix(f, "frontend.series-results-cache.") +} + +func (cfg *SeriesCacheConfig) Validate() error { + return cfg.ResultsCacheConfig.Validate() +} + +func NewSeriesCacheMiddleware( + logger log.Logger, + limits Limits, + merger queryrangebase.Merger, + c cache.Cache, + cacheGenNumberLoader queryrangebase.CacheGenNumberLoader, + shouldCache queryrangebase.ShouldCacheFn, + parallelismForReq queryrangebase.ParallelismForReqFn, + retentionEnabled bool, + transformer UserIDTransformer, + metrics *queryrangebase.ResultsCacheMetrics, +) (queryrangebase.Middleware, error) { + return queryrangebase.NewResultsCacheMiddleware( + logger, + c, + cacheKeySeries{limits, transformer}, + limits, + merger, + seriesExtractor{}, + cacheGenNumberLoader, + shouldCache, + parallelismForReq, + retentionEnabled, + metrics, + ) +} diff --git a/pkg/querier/queryrange/series_cache_test.go b/pkg/querier/queryrange/series_cache_test.go new file mode 100644 index 000000000000..abe992001217 --- /dev/null +++ b/pkg/querier/queryrange/series_cache_test.go @@ -0,0 +1,314 @@ +package queryrange + +import ( + "context" + "fmt" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/grafana/dskit/user" + "github.com/stretchr/testify/require" + + "github.com/grafana/loki/pkg/loghttp" + + "github.com/grafana/loki/pkg/logproto" + "github.com/grafana/loki/pkg/logqlmodel/stats" + "github.com/grafana/loki/pkg/querier/queryrange/queryrangebase" + "github.com/grafana/loki/pkg/storage/chunk/cache" + "github.com/grafana/loki/pkg/util" +) + +var ( + seriesAPIPath = "/loki/api/v1/series" +) + +func TestCacheKeySeries_GenerateCacheKey(t *testing.T) { + k := cacheKeySeries{ + transformer: nil, + Limits: fakeLimits{ + metadataSplitDuration: map[string]time.Duration{ + "fake": time.Hour, + }, + }, + } + + from, through := util.RoundToMilliseconds(testTime, testTime.Add(1*time.Hour)) + req := &LokiSeriesRequest{ + StartTs: from.Time(), + EndTs: through.Time(), + Match: []string{`{namespace="prod"}`, `{service="foo"}`}, + Path: seriesAPIPath, + } + + expectedInterval := testTime.UnixMilli() / time.Hour.Milliseconds() + require.Equal(t, fmt.Sprintf(`series:fake:{namespace="prod"},{service="foo"}:%d:%d`, expectedInterval, time.Hour.Nanoseconds()), k.GenerateCacheKey(context.Background(), "fake", req)) + + t.Run("same set of matchers in any order should result in the same cache key", func(t *testing.T) { + from, through := util.RoundToMilliseconds(testTime, testTime.Add(1*time.Hour)) + + for _, matchers := range [][]string{ + {`{cluster="us-central"}`, `{namespace="prod"}`, `{service=~"foo.*"}`}, + {`{namespace="prod"}`, `{service=~"foo.*"}`, `{cluster="us-central"}`}, + } { + req := &LokiSeriesRequest{ + StartTs: from.Time(), + EndTs: through.Time(), + Match: matchers, + Path: seriesAPIPath, + } + expectedInterval := testTime.UnixMilli() / time.Hour.Milliseconds() + require.Equal(t, fmt.Sprintf(`series:fake:{cluster="us-central"},{namespace="prod"},{service=~"foo.*"}:%d:%d`, expectedInterval, time.Hour.Nanoseconds()), k.GenerateCacheKey(context.Background(), "fake", req)) + } + + }) +} + +func TestSeriesCache(t *testing.T) { + setupCacheMW := func() queryrangebase.Middleware { + cacheMiddleware, err := NewSeriesCacheMiddleware( + log.NewNopLogger(), + fakeLimits{ + metadataSplitDuration: map[string]time.Duration{ + "fake": 24 * time.Hour, + }, + }, + DefaultCodec, + cache.NewMockCache(), + nil, + nil, + func(_ context.Context, _ []string, _ queryrangebase.Request) int { + return 1 + }, + false, + nil, + nil, + ) + require.NoError(t, err) + + return cacheMiddleware + } + + t.Run("caches the response for the same request", func(t *testing.T) { + cacheMiddleware := setupCacheMW() + from, through := util.RoundToMilliseconds(testTime, testTime.Add(1*time.Hour)) + + seriesReq := &LokiSeriesRequest{ + StartTs: from.Time(), + EndTs: through.Time(), + Match: []string{`{namespace=~".*"}`}, + Path: seriesAPIPath, + } + + seriesResp := &LokiSeriesResponse{ + Status: "success", + Version: uint32(loghttp.VersionV1), + Data: []logproto.SeriesIdentifier{ + { + Labels: []logproto.SeriesIdentifier_LabelsEntry{{Key: "cluster", Value: "eu-west"}, {Key: "namespace", Value: "prod"}}, + }, + }, + Statistics: stats.Result{ + Summary: stats.Summary{ + Splits: 1, + }, + }, + } + + called := 0 + handler := cacheMiddleware.Wrap(queryrangebase.HandlerFunc(func(_ context.Context, r queryrangebase.Request) (queryrangebase.Response, error) { + called++ + + // should request the entire length with no partitioning as nothing is cached yet. + require.Equal(t, seriesReq.GetStart(), r.GetStart()) + require.Equal(t, seriesReq.GetEnd(), r.GetEnd()) + + return seriesResp, nil + })) + + ctx := user.InjectOrgID(context.Background(), "fake") + got, err := handler.Do(ctx, seriesReq) + require.NoError(t, err) + require.Equal(t, 1, called) // called actual handler, as not cached. + require.Equal(t, seriesResp, got) + + // Doing same request again shouldn't change anything. + called = 0 + got, err = handler.Do(ctx, seriesReq) + require.NoError(t, err) + require.Equal(t, 0, called) + require.Equal(t, seriesResp, got) + }) + + t.Run("a new request with overlapping time range should reuse part of the previous request for the overlap", func(t *testing.T) { + cacheMiddleware := setupCacheMW() + + from, through := util.RoundToMilliseconds(testTime, testTime.Add(1*time.Hour)) + req1 := &LokiSeriesRequest{ + StartTs: from.Time(), + EndTs: through.Time(), + Match: []string{`{namespace=~".*"}`}, + Path: seriesAPIPath, + } + resp1 := &LokiSeriesResponse{ + Status: "success", + Version: uint32(loghttp.VersionV1), + Data: []logproto.SeriesIdentifier{ + { + Labels: []logproto.SeriesIdentifier_LabelsEntry{{Key: "cluster", Value: "us-central"}, {Key: "namespace", Value: "dev"}}, + }, + { + Labels: []logproto.SeriesIdentifier_LabelsEntry{{Key: "cluster", Value: "eu-west"}, {Key: "namespace", Value: "prod"}}, + }, + }, + Statistics: stats.Result{ + Summary: stats.Summary{ + Splits: 1, + }, + }, + } + + called := 0 + handler := cacheMiddleware.Wrap(queryrangebase.HandlerFunc(func(_ context.Context, r queryrangebase.Request) (queryrangebase.Response, error) { + called++ + + // should request the entire length with no partitioning as nothing is cached yet. + require.Equal(t, req1.GetStart(), r.GetStart()) + require.Equal(t, req1.GetEnd(), r.GetEnd()) + + return resp1, nil + })) + + ctx := user.InjectOrgID(context.Background(), "fake") + got, err := handler.Do(ctx, req1) + require.NoError(t, err) + require.Equal(t, 1, called) + require.Equal(t, resp1, got) + + req2 := req1.WithStartEnd(req1.GetStart().Add(15*time.Minute), req1.GetEnd().Add(15*time.Minute)) + + called = 0 + handler = cacheMiddleware.Wrap(queryrangebase.HandlerFunc(func(_ context.Context, r queryrangebase.Request) (queryrangebase.Response, error) { + called++ + + // make downstream request only for the non-overlapping portion of the query. + require.Equal(t, req1.GetEnd(), r.GetStart()) + require.Equal(t, req1.GetEnd().Add(15*time.Minute), r.GetEnd()) + + return &LokiSeriesResponse{ + Status: "success", + Version: uint32(loghttp.VersionV1), + Data: []logproto.SeriesIdentifier{ + { + Labels: []logproto.SeriesIdentifier_LabelsEntry{{Key: "cluster", Value: "us-central"}, {Key: "namespace", Value: "prod"}}, + }, + }, + Statistics: stats.Result{ + Summary: stats.Summary{ + Splits: 1, + }, + }, + }, nil + })) + + got, err = handler.Do(ctx, req2) + require.NoError(t, err) + require.Equal(t, 1, called) + // two splits as we merge the results from the extent and downstream request + resp1.Statistics.Summary.Splits = 2 + require.Equal(t, &LokiSeriesResponse{ + Status: "success", + Version: uint32(loghttp.VersionV1), + Data: []logproto.SeriesIdentifier{ + { + Labels: []logproto.SeriesIdentifier_LabelsEntry{{Key: "cluster", Value: "us-central"}, {Key: "namespace", Value: "dev"}}, + }, + { + Labels: []logproto.SeriesIdentifier_LabelsEntry{{Key: "cluster", Value: "eu-west"}, {Key: "namespace", Value: "prod"}}, + }, + { + Labels: []logproto.SeriesIdentifier_LabelsEntry{{Key: "cluster", Value: "us-central"}, {Key: "namespace", Value: "prod"}}, + }, + }, + Statistics: stats.Result{ + Summary: stats.Summary{ + Splits: 2, + }, + }, + }, got) + }) + + t.Run("caches are only valid for the same request parameters", func(t *testing.T) { + cacheMiddleware := setupCacheMW() + + from, through := util.RoundToMilliseconds(testTime, testTime.Add(1*time.Hour)) + seriesReq := &LokiSeriesRequest{ + StartTs: from.Time(), + EndTs: through.Time(), + Match: []string{`{namespace=~".*"}`}, + Path: seriesAPIPath, + } + seriesResp := &LokiSeriesResponse{ + Status: "success", + Version: uint32(loghttp.VersionV1), + Data: []logproto.SeriesIdentifier{ + { + Labels: []logproto.SeriesIdentifier_LabelsEntry{{Key: "cluster", Value: "eu-west"}, {Key: "namespace", Value: "prod"}}, + }, + }, + Statistics: stats.Result{ + Summary: stats.Summary{ + Splits: 1, + }, + }, + } + + called := 0 + handler := cacheMiddleware.Wrap(queryrangebase.HandlerFunc(func(_ context.Context, r queryrangebase.Request) (queryrangebase.Response, error) { + called++ + + // should request the entire length as none of the subsequent queries hit the cache. + require.Equal(t, seriesReq.GetStart(), r.GetStart()) + require.Equal(t, seriesReq.GetEnd(), r.GetEnd()) + return seriesResp, nil + })) + + // initial call to fill cache + ctx := user.InjectOrgID(context.Background(), "fake") + _, err := handler.Do(ctx, seriesReq) + require.NoError(t, err) + require.Equal(t, 1, called) + + type testCase struct { + fn func(*LokiSeriesRequest) + user string + } + testCases := map[string]testCase{ + "different match": { + fn: func(req *LokiSeriesRequest) { + req.Match = append(req.Match, `{foo="bar"}`) + }, + }, + "different user": { + user: "fake2s", + }, + } + + for name, tc := range testCases { + called = 0 + seriesReq := seriesReq + + if tc.fn != nil { + tc.fn(seriesReq) + } + + if tc.user != "" { + ctx = user.InjectOrgID(context.Background(), tc.user) + } + + _, err = handler.Do(ctx, seriesReq) + require.NoError(t, err) + require.Equal(t, 1, called, name) + } + }) +} diff --git a/pkg/querier/queryrange/split_by_interval.go b/pkg/querier/queryrange/split_by_interval.go index d568fe65ddde..9e2eda4b1942 100644 --- a/pkg/querier/queryrange/split_by_interval.go +++ b/pkg/querier/queryrange/split_by_interval.go @@ -184,7 +184,14 @@ func (h *splitByInterval) Do(ctx context.Context, r queryrangebase.Request) (que return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error()) } - interval := validation.MaxDurationOrZeroPerTenant(tenantIDs, h.limits.QuerySplitDuration) + var interval time.Duration + switch r.(type) { + case *LokiSeriesRequest, *LabelRequest: + interval = validation.MaxDurationOrZeroPerTenant(tenantIDs, h.limits.MetadataQuerySplitDuration) + default: + interval = validation.MaxDurationOrZeroPerTenant(tenantIDs, h.limits.QuerySplitDuration) + } + // skip split by if unset if interval == 0 { return h.next.Do(ctx, r) diff --git a/pkg/querier/queryrange/split_by_interval_test.go b/pkg/querier/queryrange/split_by_interval_test.go index 58b78b820a51..b236b88fb4d5 100644 --- a/pkg/querier/queryrange/split_by_interval_test.go +++ b/pkg/querier/queryrange/split_by_interval_test.go @@ -828,7 +828,12 @@ func Test_series_splitByInterval_Do(t *testing.T) { }, nil }) - l := WithSplitByLimits(fakeLimits{maxQueryParallelism: 1}, time.Hour) + l := fakeLimits{ + maxQueryParallelism: 1, + metadataSplitDuration: map[string]time.Duration{ + "1": time.Hour, + }, + } split := SplitByIntervalMiddleware( testSchemas, l, diff --git a/pkg/querier/queryrange/split_by_range_test.go b/pkg/querier/queryrange/split_by_range_test.go index ef25e3f910fb..b1687611abc1 100644 --- a/pkg/querier/queryrange/split_by_range_test.go +++ b/pkg/querier/queryrange/split_by_range_test.go @@ -21,7 +21,7 @@ func Test_RangeVectorSplit(t *testing.T) { srm := NewSplitByRangeMiddleware(log.NewNopLogger(), testEngineOpts, fakeLimits{ maxSeries: 10000, queryTimeout: time.Second, - splits: map[string]time.Duration{ + splitDuration: map[string]time.Duration{ "tenant": time.Minute, }, }, nilShardingMetrics) diff --git a/pkg/storage/stores/shipper/bloomshipper/client.go b/pkg/storage/stores/shipper/bloomshipper/client.go index 5636d1916f18..50b26d57a3a7 100644 --- a/pkg/storage/stores/shipper/bloomshipper/client.go +++ b/pkg/storage/stores/shipper/bloomshipper/client.go @@ -33,15 +33,15 @@ type Ref struct { TenantID string TableName string MinFingerprint, MaxFingerprint uint64 - StartTimestamp, EndTimestamp int64 + StartTimestamp, EndTimestamp model.Time Checksum uint32 } // Cmp returns the fingerprint's position relative to the bounds -func (b Ref) Cmp(fp uint64) v1.BoundsCheck { - if fp < b.MinFingerprint { +func (r Ref) Cmp(fp uint64) v1.BoundsCheck { + if fp < r.MinFingerprint { return v1.Before - } else if fp > b.MaxFingerprint { + } else if fp > r.MaxFingerprint { return v1.After } return v1.Overlap @@ -67,11 +67,9 @@ type Meta struct { } type MetaSearchParams struct { - TenantID string - MinFingerprint uint64 - MaxFingerprint uint64 - StartTimestamp int64 - EndTimestamp int64 + TenantID string + MinFingerprint, MaxFingerprint model.Fingerprint + StartTimestamp, EndTimestamp model.Time } type MetaClient interface { @@ -128,9 +126,7 @@ type BloomClient struct { } func (b *BloomClient) GetMetas(ctx context.Context, params MetaSearchParams) ([]Meta, error) { - start := model.TimeFromUnix(params.StartTimestamp) - end := model.TimeFromUnix(params.EndTimestamp) - tablesByPeriod := tablesByPeriod(b.periodicConfigs, start, end) + tablesByPeriod := tablesByPeriod(b.periodicConfigs, params.StartTimestamp, params.EndTimestamp) var metas []Meta for periodFrom, tables := range tablesByPeriod { @@ -146,8 +142,8 @@ func (b *BloomClient) GetMetas(ctx context.Context, params MetaSearchParams) ([] if err != nil { return nil, err } - if metaRef.MaxFingerprint < params.MinFingerprint || params.MaxFingerprint < metaRef.MinFingerprint || - metaRef.StartTimestamp < params.StartTimestamp || params.EndTimestamp < metaRef.EndTimestamp { + if metaRef.MaxFingerprint < uint64(params.MinFingerprint) || uint64(params.MaxFingerprint) < metaRef.MinFingerprint || + metaRef.StartTimestamp.Before(params.StartTimestamp) || metaRef.EndTimestamp.After(params.EndTimestamp) { continue } meta, err := b.downloadMeta(ctx, metaRef, periodClient) @@ -176,24 +172,23 @@ func (b *BloomClient) PutMeta(ctx context.Context, meta Meta) error { func createBlockObjectKey(meta Ref) string { blockParentFolder := fmt.Sprintf("%x-%x", meta.MinFingerprint, meta.MaxFingerprint) - filename := fmt.Sprintf("%v-%v-%x", meta.StartTimestamp, meta.EndTimestamp, meta.Checksum) + filename := fmt.Sprintf("%d-%d-%x", meta.StartTimestamp, meta.EndTimestamp, meta.Checksum) return strings.Join([]string{rootFolder, meta.TableName, meta.TenantID, bloomsFolder, blockParentFolder, filename}, delimiter) } func createMetaObjectKey(meta Ref) string { - filename := fmt.Sprintf("%x-%x-%v-%v-%x", meta.MinFingerprint, meta.MaxFingerprint, meta.StartTimestamp, meta.EndTimestamp, meta.Checksum) + filename := fmt.Sprintf("%x-%x-%d-%d-%x", meta.MinFingerprint, meta.MaxFingerprint, meta.StartTimestamp, meta.EndTimestamp, meta.Checksum) return strings.Join([]string{rootFolder, meta.TableName, meta.TenantID, metasFolder, filename}, delimiter) } -func findPeriod(configs []config.PeriodConfig, timestamp int64) (config.DayTime, error) { - ts := model.TimeFromUnix(timestamp) +func findPeriod(configs []config.PeriodConfig, ts model.Time) (config.DayTime, error) { for i := len(configs) - 1; i >= 0; i-- { periodConfig := configs[i] if periodConfig.From.Before(ts) || periodConfig.From.Equal(ts) { return periodConfig.From, nil } } - return config.DayTime{}, fmt.Errorf("can not find period for timestamp %d", timestamp) + return config.DayTime{}, fmt.Errorf("can not find period for timestamp %d", ts) } func (b *BloomClient) DeleteMeta(ctx context.Context, meta Meta) error { @@ -289,7 +284,6 @@ func (b *BloomClient) downloadMeta(ctx context.Context, metaRef MetaRef, client return meta, nil } -// todo cover with tests func createMetaRef(objectKey string, tenantID string, tableName string) (MetaRef, error) { fileName := objectKey[strings.LastIndex(objectKey, delimiter)+1:] parts := strings.Split(fileName, fileNamePartDelimiter) @@ -323,8 +317,8 @@ func createMetaRef(objectKey string, tenantID string, tableName string) (MetaRef TableName: tableName, MinFingerprint: minFingerprint, MaxFingerprint: maxFingerprint, - StartTimestamp: startTimestamp, - EndTimestamp: endTimestamp, + StartTimestamp: model.Time(startTimestamp), + EndTimestamp: model.Time(endTimestamp), Checksum: uint32(checksum), }, FilePath: objectKey, @@ -354,9 +348,9 @@ func tablesByPeriod(periodicConfigs []config.PeriodConfig, start, end model.Time func tablesForRange(periodConfig config.PeriodConfig, from, to int64) []string { interval := periodConfig.IndexTables.Period - intervalSeconds := interval.Seconds() - lower := from / int64(intervalSeconds) - upper := to / int64(intervalSeconds) + step := int64(interval.Seconds()) + lower := from / step + upper := to / step tables := make([]string, 0, 1+upper-lower) prefix := periodConfig.IndexTables.Prefix for i := lower; i <= upper; i++ { diff --git a/pkg/storage/stores/shipper/bloomshipper/client_test.go b/pkg/storage/stores/shipper/bloomshipper/client_test.go index 7267856a4315..d6043febb48c 100644 --- a/pkg/storage/stores/shipper/bloomshipper/client_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/client_test.go @@ -13,7 +13,7 @@ import ( "testing" "time" - aws_io "github.com/aws/smithy-go/io" + awsio "github.com/aws/smithy-go/io" "github.com/google/uuid" "github.com/prometheus/common/model" "github.com/stretchr/testify/require" @@ -28,9 +28,24 @@ const ( var ( // table 19627 - fixedDay = model.TimeFromUnix(time.Date(2023, time.September, 27, 0, 0, 0, 0, time.UTC).Unix()) + fixedDay = Date(2023, time.September, 27, 0, 0, 0) ) +func Date(year int, month time.Month, day, hour, min, sec int) model.Time { + date := time.Date(year, month, day, hour, min, sec, 0, time.UTC) + return model.TimeFromUnixNano(date.UnixNano()) +} + +func parseDayTime(s string) config.DayTime { + t, err := time.Parse("2006-01-02", s) + if err != nil { + panic(err) + } + return config.DayTime{ + Time: model.TimeFromUnix(t.Unix()), + } +} + func Test_BloomClient_GetMetas(t *testing.T) { shipper := createClient(t) @@ -57,8 +72,8 @@ func Test_BloomClient_GetMetas(t *testing.T) { TenantID: "tenantA", MinFingerprint: 50, MaxFingerprint: 150, - StartTimestamp: fixedDay.Add(-6 * day).Unix(), - EndTimestamp: fixedDay.Add(-1*day - 1*time.Hour).Unix(), + StartTimestamp: fixedDay.Add(-6 * day), + EndTimestamp: fixedDay.Add(-1*day - 1*time.Hour), }) require.NoError(t, err) require.ElementsMatch(t, expected, actual) @@ -75,26 +90,26 @@ func Test_BloomClient_PutMeta(t *testing.T) { "first-period-19621", 0xff, 0xfff, - time.Date(2023, time.September, 21, 5, 0, 0, 0, time.UTC).Unix(), - time.Date(2023, time.September, 21, 6, 0, 0, 0, time.UTC).Unix(), + Date(2023, time.September, 21, 5, 0, 0), + Date(2023, time.September, 21, 6, 0, 0), 0xaaa, "ignored-file-path-during-uploading", ), expectedStorage: "folder-1", - expectedFilePath: "bloom/first-period-19621/tenantA/metas/ff-fff-1695272400-1695276000-aaa", + expectedFilePath: "bloom/first-period-19621/tenantA/metas/ff-fff-1695272400000-1695276000000-aaa", }, "expected meta to be uploaded to the second folder": { source: createMetaEntity("tenantA", "second-period-19625", 200, 300, - time.Date(2023, time.September, 25, 0, 0, 0, 0, time.UTC).Unix(), - time.Date(2023, time.September, 25, 1, 0, 0, 0, time.UTC).Unix(), + Date(2023, time.September, 25, 0, 0, 0), + Date(2023, time.September, 25, 1, 0, 0), 0xbbb, "ignored-file-path-during-uploading", ), expectedStorage: "folder-2", - expectedFilePath: "bloom/second-period-19625/tenantA/metas/c8-12c-1695600000-1695603600-bbb", + expectedFilePath: "bloom/second-period-19625/tenantA/metas/c8-12c-1695600000000-1695603600000-bbb", }, } for name, data := range tests { @@ -131,26 +146,26 @@ func Test_BloomClient_DeleteMeta(t *testing.T) { "first-period-19621", 0xff, 0xfff, - time.Date(2023, time.September, 21, 5, 0, 0, 0, time.UTC).Unix(), - time.Date(2023, time.September, 21, 6, 0, 0, 0, time.UTC).Unix(), + Date(2023, time.September, 21, 5, 0, 0), + Date(2023, time.September, 21, 6, 0, 0), 0xaaa, "ignored-file-path-during-uploading", ), expectedStorage: "folder-1", - expectedFilePath: "bloom/first-period-19621/tenantA/metas/ff-fff-1695272400-1695276000-aaa", + expectedFilePath: "bloom/first-period-19621/tenantA/metas/ff-fff-1695272400000-1695276000000-aaa", }, "expected meta to be delete from the second folder": { source: createMetaEntity("tenantA", "second-period-19625", 200, 300, - time.Date(2023, time.September, 25, 0, 0, 0, 0, time.UTC).Unix(), - time.Date(2023, time.September, 25, 1, 0, 0, 0, time.UTC).Unix(), + Date(2023, time.September, 25, 0, 0, 0), + Date(2023, time.September, 25, 1, 0, 0), 0xbbb, "ignored-file-path-during-uploading", ), expectedStorage: "folder-2", - expectedFilePath: "bloom/second-period-19625/tenantA/metas/c8-12c-1695600000-1695603600-bbb", + expectedFilePath: "bloom/second-period-19625/tenantA/metas/c8-12c-1695600000000-1695603600000-bbb", }, } for name, data := range tests { @@ -175,10 +190,10 @@ func Test_BloomClient_DeleteMeta(t *testing.T) { func Test_BloomClient_GetBlocks(t *testing.T) { bloomClient := createClient(t) fsNamedStores := bloomClient.storageConfig.NamedStores.Filesystem - firstBlockPath := "bloom/first-period-19621/tenantA/blooms/eeee-ffff/1695272400-1695276000-1" + firstBlockPath := "bloom/first-period-19621/tenantA/blooms/eeee-ffff/1695272400000-1695276000000-1" firstBlockFullPath := filepath.Join(fsNamedStores["folder-1"].Directory, firstBlockPath) firstBlockData := createBlockFile(t, firstBlockFullPath) - secondBlockPath := "bloom/second-period-19624/tenantA/blooms/aaaa-bbbb/1695531600-1695535200-2" + secondBlockPath := "bloom/second-period-19624/tenantA/blooms/aaaa-bbbb/1695531600000-1695535200000-2" secondBlockFullPath := filepath.Join(fsNamedStores["folder-2"].Directory, secondBlockPath) secondBlockData := createBlockFile(t, secondBlockFullPath) require.FileExists(t, firstBlockFullPath) @@ -190,8 +205,8 @@ func Test_BloomClient_GetBlocks(t *testing.T) { TableName: "first-period-19621", MinFingerprint: 0xeeee, MaxFingerprint: 0xffff, - StartTimestamp: time.Date(2023, time.September, 21, 5, 0, 0, 0, time.UTC).Unix(), - EndTimestamp: time.Date(2023, time.September, 21, 6, 0, 0, 0, time.UTC).Unix(), + StartTimestamp: Date(2023, time.September, 21, 5, 0, 0), + EndTimestamp: Date(2023, time.September, 21, 6, 0, 0), Checksum: 1, }, BlockPath: firstBlockPath, @@ -202,8 +217,8 @@ func Test_BloomClient_GetBlocks(t *testing.T) { TableName: "second-period-19624", MinFingerprint: 0xaaaa, MaxFingerprint: 0xbbbb, - StartTimestamp: time.Date(2023, time.September, 24, 5, 0, 0, 0, time.UTC).Unix(), - EndTimestamp: time.Date(2023, time.September, 24, 6, 0, 0, 0, time.UTC).Unix(), + StartTimestamp: Date(2023, time.September, 24, 5, 0, 0), + EndTimestamp: Date(2023, time.September, 24, 6, 0, 0), Checksum: 2, }, BlockPath: secondBlockPath, @@ -232,13 +247,13 @@ func Test_BloomClient_PutBlocks(t *testing.T) { TableName: "first-period-19621", MinFingerprint: 0xeeee, MaxFingerprint: 0xffff, - StartTimestamp: time.Date(2023, time.September, 21, 5, 0, 0, 0, time.UTC).Unix(), - EndTimestamp: time.Date(2023, time.September, 21, 6, 0, 0, 0, time.UTC).Unix(), + StartTimestamp: Date(2023, time.September, 21, 5, 0, 0), + EndTimestamp: Date(2023, time.September, 21, 6, 0, 0), Checksum: 1, }, IndexPath: uuid.New().String(), }, - Data: aws_io.ReadSeekNopCloser{ReadSeeker: bytes.NewReader([]byte(blockForFirstFolderData))}, + Data: awsio.ReadSeekNopCloser{ReadSeeker: bytes.NewReader([]byte(blockForFirstFolderData))}, } blockForSecondFolderData := "data2" @@ -249,13 +264,13 @@ func Test_BloomClient_PutBlocks(t *testing.T) { TableName: "second-period-19624", MinFingerprint: 0xaaaa, MaxFingerprint: 0xbbbb, - StartTimestamp: time.Date(2023, time.September, 24, 5, 0, 0, 0, time.UTC).Unix(), - EndTimestamp: time.Date(2023, time.September, 24, 6, 0, 0, 0, time.UTC).Unix(), + StartTimestamp: Date(2023, time.September, 24, 5, 0, 0), + EndTimestamp: Date(2023, time.September, 24, 6, 0, 0), Checksum: 2, }, IndexPath: uuid.New().String(), }, - Data: aws_io.ReadSeekNopCloser{ReadSeeker: bytes.NewReader([]byte(blockForSecondFolderData))}, + Data: awsio.ReadSeekNopCloser{ReadSeeker: bytes.NewReader([]byte(blockForSecondFolderData))}, } results, err := bloomClient.PutBlocks(context.Background(), []Block{blockForFirstFolder, blockForSecondFolder}) @@ -263,7 +278,7 @@ func Test_BloomClient_PutBlocks(t *testing.T) { require.Len(t, results, 2) firstResultBlock := results[0] path := firstResultBlock.BlockPath - require.Equal(t, "bloom/first-period-19621/tenantA/blooms/eeee-ffff/1695272400-1695276000-1", path) + require.Equal(t, "bloom/first-period-19621/tenantA/blooms/eeee-ffff/1695272400000-1695276000000-1", path) require.Equal(t, blockForFirstFolder.TenantID, firstResultBlock.TenantID) require.Equal(t, blockForFirstFolder.TableName, firstResultBlock.TableName) require.Equal(t, blockForFirstFolder.MinFingerprint, firstResultBlock.MinFingerprint) @@ -281,7 +296,7 @@ func Test_BloomClient_PutBlocks(t *testing.T) { secondResultBlock := results[1] path = secondResultBlock.BlockPath - require.Equal(t, "bloom/second-period-19624/tenantA/blooms/aaaa-bbbb/1695531600-1695535200-2", path) + require.Equal(t, "bloom/second-period-19624/tenantA/blooms/aaaa-bbbb/1695531600000-1695535200000-2", path) require.Equal(t, blockForSecondFolder.TenantID, secondResultBlock.TenantID) require.Equal(t, blockForSecondFolder.TableName, secondResultBlock.TableName) require.Equal(t, blockForSecondFolder.MinFingerprint, secondResultBlock.MinFingerprint) @@ -302,9 +317,9 @@ func Test_BloomClient_PutBlocks(t *testing.T) { func Test_BloomClient_DeleteBlocks(t *testing.T) { bloomClient := createClient(t) fsNamedStores := bloomClient.storageConfig.NamedStores.Filesystem - block1Path := filepath.Join(fsNamedStores["folder-1"].Directory, "bloom/first-period-19621/tenantA/blooms/eeee-ffff/1695272400-1695276000-1") + block1Path := filepath.Join(fsNamedStores["folder-1"].Directory, "bloom/first-period-19621/tenantA/blooms/eeee-ffff/1695272400000-1695276000000-1") createBlockFile(t, block1Path) - block2Path := filepath.Join(fsNamedStores["folder-2"].Directory, "bloom/second-period-19624/tenantA/blooms/aaaa-bbbb/1695531600-1695535200-2") + block2Path := filepath.Join(fsNamedStores["folder-2"].Directory, "bloom/second-period-19624/tenantA/blooms/aaaa-bbbb/1695531600000-1695535200000-2") createBlockFile(t, block2Path) require.FileExists(t, block1Path) require.FileExists(t, block2Path) @@ -316,8 +331,8 @@ func Test_BloomClient_DeleteBlocks(t *testing.T) { TableName: "second-period-19624", MinFingerprint: 0xaaaa, MaxFingerprint: 0xbbbb, - StartTimestamp: time.Date(2023, time.September, 24, 5, 0, 0, 0, time.UTC).Unix(), - EndTimestamp: time.Date(2023, time.September, 24, 6, 0, 0, 0, time.UTC).Unix(), + StartTimestamp: Date(2023, time.September, 24, 5, 0, 0), + EndTimestamp: Date(2023, time.September, 24, 6, 0, 0), Checksum: 2, }, IndexPath: uuid.New().String(), @@ -328,8 +343,8 @@ func Test_BloomClient_DeleteBlocks(t *testing.T) { TableName: "first-period-19621", MinFingerprint: 0xeeee, MaxFingerprint: 0xffff, - StartTimestamp: time.Date(2023, time.September, 21, 5, 0, 0, 0, time.UTC).Unix(), - EndTimestamp: time.Date(2023, time.September, 21, 6, 0, 0, 0, time.UTC).Unix(), + StartTimestamp: Date(2023, time.September, 21, 5, 0, 0), + EndTimestamp: Date(2023, time.September, 21, 6, 0, 0), Checksum: 1, }, IndexPath: uuid.New().String(), @@ -500,7 +515,7 @@ func createPeriodConfigs() []config.PeriodConfig { { ObjectType: "folder-1", // from 2023-09-20: table range [19620:19623] - From: config.DayTime{Time: model.TimeFromUnix(time.Date(2023, time.September, 20, 0, 0, 0, 0, time.UTC).Unix())}, + From: parseDayTime("2023-09-20"), IndexTables: config.IndexPeriodicTableConfig{ PeriodicTableConfig: config.PeriodicTableConfig{ Period: day, @@ -510,7 +525,7 @@ func createPeriodConfigs() []config.PeriodConfig { { ObjectType: "folder-2", // from 2023-09-24: table range [19624:19627] - From: config.DayTime{Time: model.TimeFromUnix(time.Date(2023, time.September, 24, 0, 0, 0, 0, time.UTC).Unix())}, + From: parseDayTime("2023-09-24"), IndexTables: config.IndexPeriodicTableConfig{ PeriodicTableConfig: config.PeriodicTableConfig{ Period: day, @@ -522,15 +537,15 @@ func createPeriodConfigs() []config.PeriodConfig { } func createMetaInStorage(t *testing.T, folder string, tableName string, tenant string, minFingerprint uint64, maxFingerprint uint64, start model.Time) Meta { - startTimestamp := start.Unix() - endTimestamp := start.Add(12 * time.Hour).Unix() + end := start.Add(12 * time.Hour) metaChecksum := rand.Uint32() - metaFileName := fmt.Sprintf("%x-%x-%v-%v-%x", minFingerprint, maxFingerprint, startTimestamp, endTimestamp, metaChecksum) + // make sure this is equal to the createMetaObjectKey() + metaFileName := fmt.Sprintf("%x-%x-%d-%d-%x", minFingerprint, maxFingerprint, start, end, metaChecksum) metaFilePath := filepath.Join(rootFolder, tableName, tenant, metasFolder, metaFileName) err := os.MkdirAll(filepath.Join(folder, metaFilePath[:strings.LastIndex(metaFilePath, delimiter)]), 0700) require.NoError(t, err) - meta := createMetaEntity(tenant, tableName, minFingerprint, maxFingerprint, startTimestamp, endTimestamp, metaChecksum, metaFilePath) + meta := createMetaEntity(tenant, tableName, minFingerprint, maxFingerprint, start, end, metaChecksum, metaFilePath) metaFileContent, err := json.Marshal(meta) require.NoError(t, err) @@ -544,8 +559,8 @@ func createMetaEntity( tableName string, minFingerprint uint64, maxFingerprint uint64, - startTimestamp int64, - endTimestamp int64, + startTimestamp model.Time, + endTimestamp model.Time, metaChecksum uint32, metaFilePath string) Meta { return Meta{ diff --git a/pkg/storage/stores/shipper/bloomshipper/shipper.go b/pkg/storage/stores/shipper/bloomshipper/shipper.go index ee0665c4f6c3..d7038fc13761 100644 --- a/pkg/storage/stores/shipper/bloomshipper/shipper.go +++ b/pkg/storage/stores/shipper/bloomshipper/shipper.go @@ -4,11 +4,12 @@ import ( "cmp" "context" "fmt" - "time" + "math" "github.com/go-kit/log" "github.com/go-kit/log/level" "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/common/model" "golang.org/x/exp/slices" "github.com/grafana/loki/pkg/storage/stores/shipper/bloomshipper/config" @@ -39,10 +40,10 @@ func NewShipper(client Client, config config.Config, limits Limits, logger log.L }, nil } -func (s *Shipper) GetBlockRefs(ctx context.Context, tenantID string, from, through time.Time) ([]BlockRef, error) { +func (s *Shipper) GetBlockRefs(ctx context.Context, tenantID string, from, through model.Time) ([]BlockRef, error) { level.Debug(s.logger).Log("msg", "GetBlockRefs", "tenant", tenantID, "from", from, "through", through) - blockRefs, err := s.getActiveBlockRefs(ctx, tenantID, from.UnixNano(), through.UnixNano(), nil) + blockRefs, err := s.getActiveBlockRefs(ctx, tenantID, from, through, []uint64{0, math.MaxUint64}) if err != nil { return nil, fmt.Errorf("error fetching active block references : %w", err) } @@ -85,10 +86,10 @@ func runCallback(callback ForEachBlockCallback, block blockWithQuerier) error { return nil } -func (s *Shipper) ForEachBlock(ctx context.Context, tenantID string, from, through time.Time, fingerprints []uint64, callback ForEachBlockCallback) error { +func (s *Shipper) ForEachBlock(ctx context.Context, tenantID string, from, through model.Time, fingerprints []uint64, callback ForEachBlockCallback) error { level.Debug(s.logger).Log("msg", "ForEachBlock", "tenant", tenantID, "from", from, "through", through, "fingerprints", len(fingerprints)) - blockRefs, err := s.getActiveBlockRefs(ctx, tenantID, from.UnixNano(), through.UnixNano(), fingerprints) + blockRefs, err := s.getActiveBlockRefs(ctx, tenantID, from, through, fingerprints) if err != nil { return fmt.Errorf("error fetching active block references : %w", err) } @@ -111,12 +112,12 @@ func getFirstLast[T any](s []T) (T, T) { return s[0], s[len(s)-1] } -func (s *Shipper) getActiveBlockRefs(ctx context.Context, tenantID string, from, through int64, fingerprints []uint64) ([]BlockRef, error) { +func (s *Shipper) getActiveBlockRefs(ctx context.Context, tenantID string, from, through model.Time, fingerprints []uint64) ([]BlockRef, error) { minFingerprint, maxFingerprint := getFirstLast(fingerprints) metas, err := s.client.GetMetas(ctx, MetaSearchParams{ TenantID: tenantID, - MinFingerprint: minFingerprint, - MaxFingerprint: maxFingerprint, + MinFingerprint: model.Fingerprint(minFingerprint), + MaxFingerprint: model.Fingerprint(maxFingerprint), StartTimestamp: from, EndTimestamp: through, }) @@ -137,7 +138,7 @@ func (s *Shipper) getActiveBlockRefs(ctx context.Context, tenantID string, from, return activeBlocks, nil } -func (s *Shipper) findBlocks(metas []Meta, startTimestamp, endTimestamp int64, fingerprints []uint64) []BlockRef { +func (s *Shipper) findBlocks(metas []Meta, startTimestamp, endTimestamp model.Time, fingerprints []uint64) []BlockRef { outdatedBlocks := make(map[string]interface{}) for _, meta := range metas { for _, tombstone := range meta.Tombstones { @@ -175,7 +176,7 @@ func getPosition[S ~[]E, E cmp.Ordered](s S, v E) int { return len(s) } -func isOutsideRange(b *BlockRef, startTimestamp, endTimestamp int64, fingerprints []uint64) bool { +func isOutsideRange(b *BlockRef, startTimestamp, endTimestamp model.Time, fingerprints []uint64) bool { // First, check time range if b.EndTimestamp < startTimestamp || b.StartTimestamp > endTimestamp { return true diff --git a/pkg/storage/stores/shipper/bloomshipper/shipper_test.go b/pkg/storage/stores/shipper/bloomshipper/shipper_test.go index 17f21793680c..83c9379cd44c 100644 --- a/pkg/storage/stores/shipper/bloomshipper/shipper_test.go +++ b/pkg/storage/stores/shipper/bloomshipper/shipper_test.go @@ -5,6 +5,7 @@ import ( "math" "testing" + "github.com/prometheus/common/model" "github.com/stretchr/testify/require" ) @@ -190,8 +191,8 @@ func createBlockRef( TableName: "16600", MinFingerprint: minFingerprint, MaxFingerprint: maxFingerprint, - StartTimestamp: startTimestamp, - EndTimestamp: endTimestamp, + StartTimestamp: model.Time(startTimestamp), + EndTimestamp: model.Time(endTimestamp), Checksum: 0, }, // block path is unique, and it's used to distinguish the blocks so the rest of the fields might be skipped in this test diff --git a/pkg/storage/stores/shipper/bloomshipper/store.go b/pkg/storage/stores/shipper/bloomshipper/store.go index e24d7e35c412..06e1d7a4675b 100644 --- a/pkg/storage/stores/shipper/bloomshipper/store.go +++ b/pkg/storage/stores/shipper/bloomshipper/store.go @@ -13,8 +13,8 @@ import ( type ForEachBlockCallback func(bq *v1.BlockQuerier, minFp, maxFp uint64) error type ReadShipper interface { - GetBlockRefs(ctx context.Context, tenant string, from, through time.Time) ([]BlockRef, error) - ForEachBlock(ctx context.Context, tenant string, from, through time.Time, fingerprints []uint64, callback ForEachBlockCallback) error + GetBlockRefs(ctx context.Context, tenant string, from, through model.Time) ([]BlockRef, error) + ForEachBlock(ctx context.Context, tenant string, from, through model.Time, fingerprints []uint64, callback ForEachBlockCallback) error Fetch(ctx context.Context, tenant string, blocks []BlockRef, callback ForEachBlockCallback) error } @@ -52,7 +52,7 @@ func (bs *BloomStore) Stop() { // GetBlockRefs implements Store func (bs *BloomStore) GetBlockRefs(ctx context.Context, tenant string, from, through time.Time) ([]BlockRef, error) { - return bs.shipper.GetBlockRefs(ctx, tenant, from, through) + return bs.shipper.GetBlockRefs(ctx, tenant, toModelTime(from), toModelTime(through)) } // ForEach implements Store @@ -80,7 +80,7 @@ func (bs *BloomStore) GetBlockQueriersForBlockRefs(ctx context.Context, tenant s // BlockQueriers implements Store func (bs *BloomStore) GetBlockQueriers(ctx context.Context, tenant string, from, through time.Time, fingerprints []uint64) ([]BlockQuerierWithFingerprintRange, error) { bqs := make([]BlockQuerierWithFingerprintRange, 0, 32) - err := bs.shipper.ForEachBlock(ctx, tenant, from, through, fingerprints, func(bq *v1.BlockQuerier, minFp uint64, maxFp uint64) error { + err := bs.shipper.ForEachBlock(ctx, tenant, toModelTime(from), toModelTime(through), fingerprints, func(bq *v1.BlockQuerier, minFp uint64, maxFp uint64) error { bqs = append(bqs, BlockQuerierWithFingerprintRange{ BlockQuerier: bq, MinFp: model.Fingerprint(minFp), @@ -93,3 +93,7 @@ func (bs *BloomStore) GetBlockQueriers(ctx context.Context, tenant string, from, }) return bqs, err } + +func toModelTime(t time.Time) model.Time { + return model.TimeFromUnixNano(t.UnixNano()) +} diff --git a/pkg/util/marshal/legacy/marshal_test.go b/pkg/util/marshal/legacy/marshal_test.go index 79f40c8990f4..bc5ae29cd108 100644 --- a/pkg/util/marshal/legacy/marshal_test.go +++ b/pkg/util/marshal/legacy/marshal_test.go @@ -128,6 +128,24 @@ var queryTests = []struct { "requests": 0, "downloadTime": 0 }, + "seriesResult": { + "entriesFound": 0, + "entriesRequested": 0, + "entriesStored": 0, + "bytesReceived": 0, + "bytesSent": 0, + "requests": 0, + "downloadTime": 0 + }, + "labelResult": { + "entriesFound": 0, + "entriesRequested": 0, + "entriesStored": 0, + "bytesReceived": 0, + "bytesSent": 0, + "requests": 0, + "downloadTime": 0 + }, "volumeResult": { "entriesFound": 0, "entriesRequested": 0, diff --git a/pkg/util/marshal/marshal_test.go b/pkg/util/marshal/marshal_test.go index 6d816841daa1..87fe3fdca932 100644 --- a/pkg/util/marshal/marshal_test.go +++ b/pkg/util/marshal/marshal_test.go @@ -96,6 +96,24 @@ const emptyStats = `{ "requests": 0, "downloadTime": 0 }, + "seriesResult": { + "entriesFound": 0, + "entriesRequested": 0, + "entriesStored": 0, + "bytesReceived": 0, + "bytesSent": 0, + "requests": 0, + "downloadTime": 0 + }, + "labelResult": { + "entriesFound": 0, + "entriesRequested": 0, + "entriesStored": 0, + "bytesReceived": 0, + "bytesSent": 0, + "requests": 0, + "downloadTime": 0 + }, "volumeResult": { "entriesFound": 0, "entriesRequested": 0, diff --git a/pkg/validation/limits.go b/pkg/validation/limits.go index 92d05398065e..7f1f6ea0d734 100644 --- a/pkg/validation/limits.go +++ b/pkg/validation/limits.go @@ -104,12 +104,13 @@ type Limits struct { QueryTimeout model.Duration `yaml:"query_timeout" json:"query_timeout"` // Query frontend enforced limits. The default is actually parameterized by the queryrange config. - QuerySplitDuration model.Duration `yaml:"split_queries_by_interval" json:"split_queries_by_interval"` - MinShardingLookback model.Duration `yaml:"min_sharding_lookback" json:"min_sharding_lookback"` - MaxQueryBytesRead flagext.ByteSize `yaml:"max_query_bytes_read" json:"max_query_bytes_read"` - MaxQuerierBytesRead flagext.ByteSize `yaml:"max_querier_bytes_read" json:"max_querier_bytes_read"` - VolumeEnabled bool `yaml:"volume_enabled" json:"volume_enabled" doc:"description=Enable log-volume endpoints."` - VolumeMaxSeries int `yaml:"volume_max_series" json:"volume_max_series" doc:"description=The maximum number of aggregated series in a log-volume response"` + QuerySplitDuration model.Duration `yaml:"split_queries_by_interval" json:"split_queries_by_interval"` + MetadataQuerySplitDuration model.Duration `yaml:"split_metadata_queries_by_interval" json:"split_metadata_queries_by_interval"` + MinShardingLookback model.Duration `yaml:"min_sharding_lookback" json:"min_sharding_lookback"` + MaxQueryBytesRead flagext.ByteSize `yaml:"max_query_bytes_read" json:"max_query_bytes_read"` + MaxQuerierBytesRead flagext.ByteSize `yaml:"max_querier_bytes_read" json:"max_querier_bytes_read"` + VolumeEnabled bool `yaml:"volume_enabled" json:"volume_enabled" doc:"description=Enable log-volume endpoints."` + VolumeMaxSeries int `yaml:"volume_max_series" json:"volume_max_series" doc:"description=The maximum number of aggregated series in a log-volume response"` // Ruler defaults and limits. RulerMaxRulesPerRuleGroup int `yaml:"ruler_max_rules_per_rule_group" json:"ruler_max_rules_per_rule_group"` @@ -295,6 +296,9 @@ func (l *Limits) RegisterFlags(f *flag.FlagSet) { _ = l.QuerySplitDuration.Set("1h") f.Var(&l.QuerySplitDuration, "querier.split-queries-by-interval", "Split queries by a time interval and execute in parallel. The value 0 disables splitting by time. This also determines how cache keys are chosen when result caching is enabled.") + _ = l.MetadataQuerySplitDuration.Set("24h") + f.Var(&l.MetadataQuerySplitDuration, "querier.split-metadata-queries-by-interval", "Split metadata queries by a time interval and execute in parallel. The value 0 disables splitting metadata queries by time. This also determines how cache keys are chosen when label/series result caching is enabled.") + f.StringVar(&l.DeletionMode, "compactor.deletion-mode", "filter-and-delete", "Deletion mode. Can be one of 'disabled', 'filter-only', or 'filter-and-delete'. When set to 'filter-only' or 'filter-and-delete', and if retention_enabled is true, then the log entry deletion API endpoints are available.") // Deprecated @@ -565,6 +569,11 @@ func (o *Overrides) QuerySplitDuration(userID string) time.Duration { return time.Duration(o.getOverridesForUser(userID).QuerySplitDuration) } +// MetadataQuerySplitDuration returns the tenant specific metadata splitby interval applied in the query frontend. +func (o *Overrides) MetadataQuerySplitDuration(userID string) time.Duration { + return time.Duration(o.getOverridesForUser(userID).MetadataQuerySplitDuration) +} + // MaxQueryBytesRead returns the maximum bytes a query can read. func (o *Overrides) MaxQueryBytesRead(_ context.Context, userID string) int { return o.getOverridesForUser(userID).MaxQueryBytesRead.Val()