**What this PR does / why we need it**:
This PR implements two new per-tenant limits that are enforced on log
and metric queries (both range and instant) when TSDB is used:
- `max_query_bytes_read`: Refuse queries that would read more than the
configured bytes here. Overall limit regardless of splitting/sharding.
The goal is to refuse queries that would take too long. The default
value of 0 disables this limit.
- `max_querier_bytes_read`: Refuse queries in which any of their
subqueries after splitting and sharding would read more than the
configured bytes here. The goal is to avoid a querier from running a
query that would load too much data in memory and can potentially get
OOMed. The default value of 0 disables this limit.
These new limits can be configured per tenant and per query (see
https://github.com/grafana/loki/pull/8727).
The bytes a query would read are estimated through TSDB's index stats.
Even though they are not exact, they are good enough to have a rough
estimation of whether a query is too big to run or not. For more details
on this refer to this discussion in the PR:
https://github.com/grafana/loki/pull/8670#discussion_r1124858508.
Both limits are implemented in the frontend. Even though we considered
implementing `max_querier_bytes_read` in the querier, this way, the
limits for pre and post splitting/sharding queries are enforced close to
each other on the same component. Moreover, this way we can reduce the
number of index stats requests issued to the index gateways by reusing
the stats gathered while sharding the query.
With regard to how index stats requests are issued:
- We parallelize index stats requests by splitting them into queries
that span up to 24h since our indices are sharded by 24h periods. On top
of that, this prevents a single index gateway from processing a single
huge request like `{app=~".+"} for 30d`.
- If sharding is enabled and the query is shardable, for
`max_querier_bytes_read`, we re-use the stats requests issued by the
sharding ware. Specifically, we look at the [bytesPerShard][1] to
enforce this limit.
Note that once we merge this PR and enable these limits, the load of
index stats requests will increase substantially and we may discover
bottlenecks in our index gateways and TSDB. After speaking with @owen-d,
we think it should be fine as, if needed, we can scale up our index
gateways and support caching index stats requests.
Here's a demo of this working:
<img width="1647" alt="image"
src="https://user-images.githubusercontent.com/8354290/226918478-d4b6c2fd-de4d-478a-9c8b-e38fe148fa95.png">
<img width="1647" alt="image"
src="https://user-images.githubusercontent.com/8354290/226918798-a71b1db8-ea68-4d00-933b-e5eb1524d240.png">
**Which issue(s) this PR fixes**:
This PR addresses https://github.com/grafana/loki-private/issues/674.
**Special notes for your reviewer**:
- @jeschkies has reviewed the changes related to query-time limits.
- I've done some refactoring in this PR:
- Extracted logic to get stats for a set of matches into a new function
[getStatsForMatchers][2].
- Extracted the _Handler_ interface implementation for
[queryrangebase.roundTripper][3] into a new type
[queryrangebase.roundTripperHandler][4]. This is used to create the
handler that skips the rest of configured middlewares when sending an
index stat quests ([example][5]).
**Checklist**
- [x] Reviewed the
[`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md)
guide (**required**)
- [x] Documentation added
- [x] Tests updated
- [x] `CHANGELOG.md` updated
- [ ] Changes that require user attention or interaction to upgrade are
documented in `docs/sources/upgrading/_index.md`
[1]:
ff847305af/pkg/querier/queryrange/shard_resolver.go (L179-L186)
[2]:
ff847305af/pkg/querier/queryrange/shard_resolver.go (L72)
[3]:
3d2fff3a2d/pkg/querier/queryrange/queryrangebase/roundtrip.go (L124)
[4]:
3d2fff3a2d/pkg/querier/queryrange/queryrangebase/roundtrip.go (L163)
[5]:
f422e0a52b/pkg/querier/queryrange/roundtrip.go (L521)
* [6675](https://github.com/grafana/loki/pull/6675) **btaani**: Add logfmt expression parser for selective extraction of labels from logfmt formatted logs
* [8474](https://github.com/grafana/loki/pull/8474) **farodin91**: Add support for short-lived S3 session tokens
* [8774](https://github.com/grafana/loki/pull/8774) **slim-bean**: Add new logql template functions `bytes`, `duration`, `unixEpochMillis`, `unixEpochNanos`, `toDateInZone`, `b64Enc`, and `b64Dec`
* [8670](https://github.com/grafana/loki/pull/8670) **salvacorts** Introduce two new limits to refuse log and metric queries that would read too much data.
requiredLabelsErrTmpl="stream selector is missing required matchers [%s], labels present in the query were [%s]"
limErrQueryTooManyBytesTmpl="the query would read too many bytes (query: %s, limit: %s). Consider adding more specific stream selectors or reduce the time range of the query"
limErrQuerierTooManyBytesTmpl="query too large to execute on a single querier, either because parallelization is not enabled, the query is unshardable, or a shard query is too big to execute: (query: %s, limit: %s). Consider adding more specific stream selectors or reduce the time range of the query"
)
var(
@ -50,6 +57,8 @@ type Limits interface {
// frontend will process in parallel for TSDB queries.
level.Debug(log).Log("msg","Query is within limits","status","accepted","limit_name",q.guessLimitName(),"limit_bytes",maxBytesReadStr,"resolved_bytes",statsBytesStr)
level.Debug(ast.logger).Log("msg","Query is within limits","status","accepted","limit_name","MaxQuerierBytesRead","limit_bytes",maxBytesReadStr,"resolved_bytes",statsBytesStr)
f.Var(&l.MinShardingLookback,"frontend.min-sharding-lookback","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.")
f.Var(&l.MaxQueryBytesRead,"frontend.max-query-bytes-read","Max number of bytes a query can fetch. Enforced in log and metric queries only when TSDB is used. The default value of 0 disables this limit.")
f.Var(&l.MaxQuerierBytesRead,"frontend.max-querier-bytes-read","Max number of bytes a query can fetch after splitting and sharding. Enforced in log and metric queries only when TSDB is used. The default value of 0 disables this limit.")
_=l.MaxCacheFreshness.Set("1m")
f.Var(&l.MaxCacheFreshness,"frontend.max-cache-freshness","Most recent allowed cacheable result per-tenant, to prevent caching very recent results that might still be in flux.")