Like Prometheus, but for logs.
You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
loki/pkg/querier/queryrange/limits_test.go

674 lines
18 KiB

package queryrange
import (
"context"
"fmt"
"net/http"
"sync"
"testing"
"time"
"github.com/grafana/dskit/user"
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/model/labels"
"github.com/prometheus/prometheus/promql"
"github.com/stretchr/testify/require"
"go.uber.org/atomic"
"gopkg.in/yaml.v2"
"github.com/grafana/loki/pkg/logproto"
"github.com/grafana/loki/pkg/logqlmodel"
"github.com/grafana/loki/pkg/querier/queryrange/queryrangebase"
"github.com/grafana/loki/pkg/storage/config"
util_log "github.com/grafana/loki/pkg/util/log"
"github.com/grafana/loki/pkg/util/marshal"
Pass engine opts down to middlewares (#9130) **What this PR does / why we need it**: The following middlewares in the query frontend uses a downstream engine: - `NewQuerySizeLimiterMiddleware` and `NewQuerierSizeLimiterMiddleware` - `NewQueryShardMiddleware` - `NewSplitByRangeMiddleware` These were all creating the downstream engine as follows: ```go logql.NewDownstreamEngine(logql.EngineOpts{LogExecutingQuery: false}, DownstreamHandler{next: next, limits: limits}, limits, logger), ``` As can be seen, the [engine options configured in Loki][1] were not being used at all. In the case of `NewQuerySizeLimiterMiddleware`, `NewQuerierSizeLimiterMiddleware` and `NewQueryShardMiddleware`, the downstream engine was created to get the `MaxLookBackPeriod`. When creating a new Downstream Engine as above, the `MaxLookBackPeriod` [would always be the default][2] (30 seconds). This PR fixes this by passing down the engine config to these middlewares, so this config is used to create the new downstream engines. **Which issue(s) this PR fixes**: Adresses some pending tasks from https://github.com/grafana/loki/pull/8670#issuecomment-1507031976. **Special notes for your reviewer**: **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] 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]: https://github.com/grafana/loki/blob/1bcf683513b5e11fb1dcc5625761b9760cfbd958/pkg/querier/querier.go#L52 [2]: https://github.com/grafana/loki/blob/edc6b0bff76714ff584c20d7ff0235461a4f4a88/pkg/logql/engine.go#L136-L140
3 years ago
"github.com/grafana/loki/pkg/util/math"
)
func TestLimits(t *testing.T) {
l := fakeLimits{
splits: map[string]time.Duration{"a": time.Minute},
}
wrapped := WithSplitByLimits(l, time.Hour)
// Test default
require.Equal(t, wrapped.QuerySplitDuration("b"), time.Hour)
// Ensure we override the underlying implementation
require.Equal(t, wrapped.QuerySplitDuration("a"), time.Hour)
r := &LokiRequest{
Query: "qry",
StartTs: time.Now(),
Step: int64(time.Minute / time.Millisecond),
}
require.Equal(
t,
fmt.Sprintf("%s:%s:%d:%d:%d", "a", r.GetQuery(), r.GetStep(), r.GetStart()/int64(time.Hour/time.Millisecond), int64(time.Hour)),
cacheKeyLimits{wrapped, nil}.GenerateCacheKey(context.Background(), "a", r),
)
}
func Test_seriesLimiter(t *testing.T) {
cfg := testConfig
cfg.CacheResults = false
Stats cache can be configured independently (#9535) **What this PR does / why we need it**: Before this PR, the index stats cache would use the same config as the query results cache. This was a limitation since: 1. We would not be able to point to a different cache for storing the index stats if needed. 2. We would not be able to add specific settings for this cache, without adding it to the results cache. In this PR, we refactor the index stats cache config to be independently configurable. Note that if it's not configured, it will try to use the results cache settings. **Which issue(s) this PR fixes**: This is needed for: - https://github.com/grafana/loki/pull/9537 - https://github.com/grafana/loki/pull/9536 **Special notes for your reviewer**: - This PR also refactors all the tripperwares in rountrip.go to reuse the same stats tripperware instead of each one creating their own. - Configuring a new cache in rountrip.go is a requirement for https://github.com/grafana/loki/pull/9536 so the stats summary can distinguish before the stats cache and the results cache. **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` - [ ] For Helm chart changes bump the Helm chart version in `production/helm/loki/Chart.yaml` and update `production/helm/loki/CHANGELOG.md` and `production/helm/loki/README.md`. [Example PR](https://github.com/grafana/loki/commit/d10549e3ece02120974929894ee333d07755d213)
3 years ago
cfg.CacheIndexStatsResults = false
// split in 7 with 2 in // max.
l := WithSplitByLimits(fakeLimits{maxSeries: 1, maxQueryParallelism: 2}, time.Hour)
Pass engine opts down to middlewares (#9130) **What this PR does / why we need it**: The following middlewares in the query frontend uses a downstream engine: - `NewQuerySizeLimiterMiddleware` and `NewQuerierSizeLimiterMiddleware` - `NewQueryShardMiddleware` - `NewSplitByRangeMiddleware` These were all creating the downstream engine as follows: ```go logql.NewDownstreamEngine(logql.EngineOpts{LogExecutingQuery: false}, DownstreamHandler{next: next, limits: limits}, limits, logger), ``` As can be seen, the [engine options configured in Loki][1] were not being used at all. In the case of `NewQuerySizeLimiterMiddleware`, `NewQuerierSizeLimiterMiddleware` and `NewQueryShardMiddleware`, the downstream engine was created to get the `MaxLookBackPeriod`. When creating a new Downstream Engine as above, the `MaxLookBackPeriod` [would always be the default][2] (30 seconds). This PR fixes this by passing down the engine config to these middlewares, so this config is used to create the new downstream engines. **Which issue(s) this PR fixes**: Adresses some pending tasks from https://github.com/grafana/loki/pull/8670#issuecomment-1507031976. **Special notes for your reviewer**: **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] 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]: https://github.com/grafana/loki/blob/1bcf683513b5e11fb1dcc5625761b9760cfbd958/pkg/querier/querier.go#L52 [2]: https://github.com/grafana/loki/blob/edc6b0bff76714ff584c20d7ff0235461a4f4a88/pkg/logql/engine.go#L136-L140
3 years ago
tpw, stopper, err := NewTripperware(cfg, testEngineOpts, util_log.Logger, l, config.SchemaConfig{
Configs: testSchemas,
}, nil, false, nil)
if stopper != nil {
defer stopper.Stop()
}
require.NoError(t, err)
lreq := &LokiRequest{
Query: `rate({app="foo"} |= "foo"[1m])`,
Limit: 1000,
Step: 30000, // 30sec
StartTs: testTime.Add(-6 * time.Hour),
EndTs: testTime,
Direction: logproto.FORWARD,
Path: "/query_range",
}
ctx := user.InjectOrgID(context.Background(), "1")
req, err := DefaultCodec.EncodeRequest(ctx, lreq)
require.NoError(t, err)
req = req.WithContext(ctx)
err = user.InjectOrgIDIntoHTTPRequest(ctx, req)
require.NoError(t, err)
rt, err := newfakeRoundTripper()
require.NoError(t, err)
defer rt.Close()
count, h := promqlResult(matrix)
rt.setHandler(h)
_, err = tpw(rt).RoundTrip(req)
require.NoError(t, err)
require.Equal(t, 7, *count)
// 2 series should not be allowed.
c := new(int)
m := &sync.Mutex{}
h = http.HandlerFunc(func(rw http.ResponseWriter, r *http.Request) {
m.Lock()
defer m.Unlock()
defer func() {
*c++
}()
// first time returns a single series
if *c == 0 {
if err := marshal.WriteQueryResponseJSON(logqlmodel.Result{Data: matrix}, rw); err != nil {
panic(err)
}
return
}
// second time returns a different series.
if err := marshal.WriteQueryResponseJSON(logqlmodel.Result{
Data: promql.Matrix{
{
Floats: []promql.FPoint{
{
T: toMs(testTime.Add(-4 * time.Hour)),
F: 0.013333333333333334,
},
},
Metric: []labels.Label{
{
Name: "filename",
Value: `/var/hostlog/apport.log`,
},
{
Name: "job",
Value: "anotherjob",
},
},
},
},
}, rw); err != nil {
panic(err)
}
})
rt.setHandler(h)
_, err = tpw(rt).RoundTrip(req)
require.Error(t, err)
require.LessOrEqual(t, *c, 4)
}
func Test_MaxQueryParallelism(t *testing.T) {
maxQueryParallelism := 2
f, err := newfakeRoundTripper()
require.Nil(t, err)
var count atomic.Int32
var max atomic.Int32
f.setHandler(http.HandlerFunc(func(rw http.ResponseWriter, r *http.Request) {
cur := count.Inc()
if cur > max.Load() {
max.Store(cur)
}
defer count.Dec()
// simulate some work
time.Sleep(20 * time.Millisecond)
}))
ctx := user.InjectOrgID(context.Background(), "foo")
r, err := http.NewRequestWithContext(ctx, "GET", "/query_range", http.NoBody)
require.Nil(t, err)
_, _ = NewLimitedRoundTripper(f, DefaultCodec, fakeLimits{maxQueryParallelism: maxQueryParallelism},
testSchemas,
queryrangebase.MiddlewareFunc(func(next queryrangebase.Handler) queryrangebase.Handler {
return queryrangebase.HandlerFunc(func(c context.Context, r queryrangebase.Request) (queryrangebase.Response, error) {
var wg sync.WaitGroup
for i := 0; i < 10; i++ {
wg.Add(1)
go func() {
defer wg.Done()
_, _ = next.Do(c, &LokiRequest{})
}()
}
wg.Wait()
return nil, nil
})
}),
).RoundTrip(r)
maxFound := int(max.Load())
require.LessOrEqual(t, maxFound, maxQueryParallelism, "max query parallelism: ", maxFound, " went over the configured one:", maxQueryParallelism)
}
func Test_MaxQueryParallelismLateScheduling(t *testing.T) {
maxQueryParallelism := 2
f, err := newfakeRoundTripper()
require.Nil(t, err)
f.setHandler(http.HandlerFunc(func(rw http.ResponseWriter, r *http.Request) {
// simulate some work
time.Sleep(20 * time.Millisecond)
}))
ctx := user.InjectOrgID(context.Background(), "foo")
r, err := http.NewRequestWithContext(ctx, "GET", "/query_range", http.NoBody)
require.Nil(t, err)
_, _ = NewLimitedRoundTripper(f, DefaultCodec, fakeLimits{maxQueryParallelism: maxQueryParallelism},
testSchemas,
queryrangebase.MiddlewareFunc(func(next queryrangebase.Handler) queryrangebase.Handler {
return queryrangebase.HandlerFunc(func(c context.Context, r queryrangebase.Request) (queryrangebase.Response, error) {
for i := 0; i < 10; i++ {
go func() {
_, _ = next.Do(c, &LokiRequest{})
}()
}
return nil, nil
})
}),
).RoundTrip(r)
}
func Test_MaxQueryParallelismDisable(t *testing.T) {
maxQueryParallelism := 0
f, err := newfakeRoundTripper()
require.Nil(t, err)
f.setHandler(http.HandlerFunc(func(rw http.ResponseWriter, r *http.Request) {
// simulate some work
time.Sleep(20 * time.Millisecond)
}))
ctx := user.InjectOrgID(context.Background(), "foo")
r, err := http.NewRequestWithContext(ctx, "GET", "/query_range", http.NoBody)
require.Nil(t, err)
_, err = NewLimitedRoundTripper(f, DefaultCodec, fakeLimits{maxQueryParallelism: maxQueryParallelism},
testSchemas,
queryrangebase.MiddlewareFunc(func(next queryrangebase.Handler) queryrangebase.Handler {
return queryrangebase.HandlerFunc(func(c context.Context, r queryrangebase.Request) (queryrangebase.Response, error) {
for i := 0; i < 10; i++ {
go func() {
_, _ = next.Do(c, &LokiRequest{})
}()
}
return nil, nil
})
}),
).RoundTrip(r)
require.Error(t, err)
}
func Test_MaxQueryLookBack(t *testing.T) {
Pass engine opts down to middlewares (#9130) **What this PR does / why we need it**: The following middlewares in the query frontend uses a downstream engine: - `NewQuerySizeLimiterMiddleware` and `NewQuerierSizeLimiterMiddleware` - `NewQueryShardMiddleware` - `NewSplitByRangeMiddleware` These were all creating the downstream engine as follows: ```go logql.NewDownstreamEngine(logql.EngineOpts{LogExecutingQuery: false}, DownstreamHandler{next: next, limits: limits}, limits, logger), ``` As can be seen, the [engine options configured in Loki][1] were not being used at all. In the case of `NewQuerySizeLimiterMiddleware`, `NewQuerierSizeLimiterMiddleware` and `NewQueryShardMiddleware`, the downstream engine was created to get the `MaxLookBackPeriod`. When creating a new Downstream Engine as above, the `MaxLookBackPeriod` [would always be the default][2] (30 seconds). This PR fixes this by passing down the engine config to these middlewares, so this config is used to create the new downstream engines. **Which issue(s) this PR fixes**: Adresses some pending tasks from https://github.com/grafana/loki/pull/8670#issuecomment-1507031976. **Special notes for your reviewer**: **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] 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]: https://github.com/grafana/loki/blob/1bcf683513b5e11fb1dcc5625761b9760cfbd958/pkg/querier/querier.go#L52 [2]: https://github.com/grafana/loki/blob/edc6b0bff76714ff584c20d7ff0235461a4f4a88/pkg/logql/engine.go#L136-L140
3 years ago
tpw, stopper, err := NewTripperware(testConfig, testEngineOpts, util_log.Logger, fakeLimits{
maxQueryLookback: 1 * time.Hour,
maxQueryParallelism: 1,
}, config.SchemaConfig{
Configs: testSchemas,
}, nil, false, nil)
if stopper != nil {
defer stopper.Stop()
}
require.NoError(t, err)
rt, err := newfakeRoundTripper()
require.NoError(t, err)
defer rt.Close()
lreq := &LokiRequest{
Query: `{app="foo"} |= "foo"`,
Limit: 10000,
StartTs: testTime.Add(-6 * time.Hour),
EndTs: testTime,
Direction: logproto.FORWARD,
Path: "/loki/api/v1/query_range",
}
ctx := user.InjectOrgID(context.Background(), "1")
req, err := DefaultCodec.EncodeRequest(ctx, lreq)
require.NoError(t, err)
req = req.WithContext(ctx)
err = user.InjectOrgIDIntoHTTPRequest(ctx, req)
require.NoError(t, err)
_, err = tpw(rt).RoundTrip(req)
require.NoError(t, err)
}
func Test_GenerateCacheKey_NoDivideZero(t *testing.T) {
l := cacheKeyLimits{WithSplitByLimits(nil, 0), nil}
start := time.Now()
r := &LokiRequest{
Query: "qry",
StartTs: start,
Step: int64(time.Minute / time.Millisecond),
}
require.Equal(
t,
fmt.Sprintf("foo:qry:%d:0:0", r.GetStep()),
l.GenerateCacheKey(context.Background(), "foo", r),
)
}
func Test_WeightedParallelism(t *testing.T) {
limits := &fakeLimits{
tsdbMaxQueryParallelism: 100,
maxQueryParallelism: 10,
}
for _, cfgs := range []struct {
desc string
periods string
}{
{
desc: "end configs",
periods: `
- from: "2022-01-01"
store: boltdb-shipper
object_store: gcs
schema: v12
- from: "2022-01-02"
store: tsdb
object_store: gcs
schema: v12
`,
},
{
// Add another test that wraps the tested period configs with other unused configs
// to ensure we bounds-test properly
desc: "middle configs",
periods: `
- from: "2021-01-01"
store: boltdb-shipper
object_store: gcs
schema: v12
- from: "2022-01-01"
store: boltdb-shipper
object_store: gcs
schema: v12
- from: "2022-01-02"
store: tsdb
object_store: gcs
schema: v12
- from: "2023-01-02"
store: tsdb
object_store: gcs
schema: v12
`,
},
} {
var confs []config.PeriodConfig
require.Nil(t, yaml.Unmarshal([]byte(cfgs.periods), &confs))
parsed, err := time.Parse("2006-01-02", "2022-01-02")
borderTime := model.TimeFromUnix(parsed.Unix())
require.Nil(t, err)
for _, tc := range []struct {
desc string
start, end model.Time
exp int
}{
{
desc: "50% each",
start: borderTime.Add(-time.Hour),
end: borderTime.Add(time.Hour),
exp: 55,
},
{
desc: "75/25 split",
start: borderTime.Add(-3 * time.Hour),
end: borderTime.Add(time.Hour),
exp: 32,
},
{
desc: "start==end",
start: borderTime.Add(time.Hour),
end: borderTime.Add(time.Hour),
exp: 100,
},
} {
t.Run(cfgs.desc+tc.desc, func(t *testing.T) {
require.Equal(t, tc.exp, WeightedParallelism(context.Background(), confs, "fake", limits, tc.start, tc.end))
})
}
}
}
func Test_WeightedParallelism_DivideByZeroError(t *testing.T) {
t.Run("query end before start", func(t *testing.T) {
parsed, err := time.Parse("2006-01-02", "2022-01-02")
require.NoError(t, err)
borderTime := model.TimeFromUnix(parsed.Unix())
confs := []config.PeriodConfig{
{
From: config.DayTime{
Time: borderTime.Add(-1 * time.Hour),
},
IndexType: config.TSDBType,
},
}
result := WeightedParallelism(context.Background(), confs, "fake", &fakeLimits{tsdbMaxQueryParallelism: 50}, borderTime, borderTime.Add(-1*time.Hour))
require.Equal(t, 1, result)
})
t.Run("negative start and end time", func(t *testing.T) {
parsed, err := time.Parse("2006-01-02", "2022-01-02")
require.NoError(t, err)
borderTime := model.TimeFromUnix(parsed.Unix())
confs := []config.PeriodConfig{
{
From: config.DayTime{
Time: borderTime.Add(-1 * time.Hour),
},
IndexType: config.TSDBType,
},
}
result := WeightedParallelism(context.Background(), confs, "fake", &fakeLimits{maxQueryParallelism: 50}, -100, -50)
require.Equal(t, 1, result)
})
t.Run("query start and end time before config start", func(t *testing.T) {
parsed, err := time.Parse("2006-01-02", "2022-01-02")
require.NoError(t, err)
borderTime := model.TimeFromUnix(parsed.Unix())
confs := []config.PeriodConfig{
{
From: config.DayTime{
Time: borderTime.Add(-1 * time.Hour),
},
IndexType: config.TSDBType,
},
}
result := WeightedParallelism(context.Background(), confs, "fake", &fakeLimits{maxQueryParallelism: 50}, confs[0].From.Add(-24*time.Hour), confs[0].From.Add(-12*time.Hour))
require.Equal(t, 1, result)
})
}
Max bytes read limit (#8670) **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]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L179-L186 [2]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L72 [3]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L124 [4]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L163 [5]: https://github.com/grafana/loki/blob/f422e0a52b743a11209b8276510feb2ab8241486/pkg/querier/queryrange/roundtrip.go#L521
3 years ago
func getFakeStatsHandler(retBytes uint64) (queryrangebase.Handler, *int, error) {
fakeRT, err := newfakeRoundTripper()
if err != nil {
return nil, nil, err
}
count, statsHandler := indexStatsResult(logproto.IndexStatsResponse{Bytes: retBytes})
fakeRT.setHandler(statsHandler)
return queryrangebase.NewRoundTripperHandler(fakeRT, DefaultCodec), count, nil
Max bytes read limit (#8670) **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]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L179-L186 [2]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L72 [3]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L124 [4]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L163 [5]: https://github.com/grafana/loki/blob/f422e0a52b743a11209b8276510feb2ab8241486/pkg/querier/queryrange/roundtrip.go#L521
3 years ago
}
func Test_MaxQuerySize(t *testing.T) {
const statsBytes = 1000
schemas := []config.PeriodConfig{
{
// BoltDB -> Time -4 days
From: config.DayTime{Time: model.TimeFromUnix(testTime.Add(-96 * time.Hour).Unix())},
IndexType: config.BoltDBShipperType,
},
{
// TSDB -> Time -2 days
From: config.DayTime{Time: model.TimeFromUnix(testTime.Add(-48 * time.Hour).Unix())},
IndexType: config.TSDBType,
},
}
for _, tc := range []struct {
desc string
schema string
query string
queryRange time.Duration
queryStart time.Time
queryEnd time.Time
limits Limits
shouldErr bool
expectedQueryStatsHits int
expectedQuerierStatsHits int
}{
{
desc: "No TSDB",
schema: config.BoltDBShipperType,
query: `{app="foo"} |= "foo"`,
queryRange: 1 * time.Hour,
queryStart: testTime.Add(-96 * time.Hour),
queryEnd: testTime.Add(-90 * time.Hour),
limits: fakeLimits{
maxQueryBytesRead: 1,
maxQuerierBytesRead: 1,
},
shouldErr: false,
expectedQueryStatsHits: 0,
expectedQuerierStatsHits: 0,
},
{
desc: "Unlimited",
query: `{app="foo"} |= "foo"`,
queryStart: testTime.Add(-48 * time.Hour),
queryEnd: testTime,
limits: fakeLimits{
maxQueryBytesRead: 0,
maxQuerierBytesRead: 0,
},
shouldErr: false,
expectedQueryStatsHits: 0,
expectedQuerierStatsHits: 0,
},
{
desc: "1 hour range",
query: `{app="foo"} |= "foo"`,
queryStart: testTime.Add(-1 * time.Hour),
queryEnd: testTime,
limits: fakeLimits{
maxQueryBytesRead: statsBytes,
maxQuerierBytesRead: statsBytes,
},
shouldErr: false,
// [testTime-1h, testTime)
expectedQueryStatsHits: 1,
expectedQuerierStatsHits: 1,
},
{
desc: "Query size too big",
query: `{app="foo"} |= "foo"`,
queryStart: testTime.Add(-1 * time.Hour),
queryEnd: testTime,
limits: fakeLimits{
maxQueryBytesRead: statsBytes - 1,
maxQuerierBytesRead: statsBytes,
},
shouldErr: true,
expectedQueryStatsHits: 1,
expectedQuerierStatsHits: 0,
},
{
desc: "Querier size too big",
query: `{app="foo"} |= "foo"`,
queryStart: testTime.Add(-1 * time.Hour),
queryEnd: testTime,
limits: fakeLimits{
maxQueryBytesRead: statsBytes,
maxQuerierBytesRead: statsBytes - 1,
},
shouldErr: true,
expectedQueryStatsHits: 1,
expectedQuerierStatsHits: 1,
},
{
desc: "Multi-matchers with offset",
query: `sum_over_time ({app="foo"} |= "foo" | unwrap foo [5m] ) / sum_over_time ({app="bar"} |= "bar" | unwrap bar [5m] offset 1h)`,
queryStart: testTime.Add(-1 * time.Hour),
queryEnd: testTime,
limits: fakeLimits{
maxQueryBytesRead: statsBytes,
maxQuerierBytesRead: statsBytes,
},
shouldErr: false,
// *2 since we have two matcher groups
expectedQueryStatsHits: 1 * 2,
expectedQuerierStatsHits: 1 * 2,
},
} {
t.Run(tc.desc, func(t *testing.T) {
queryStatsHandler, queryStatsHits, err := getFakeStatsHandler(uint64(statsBytes / math.Max(tc.expectedQueryStatsHits, 1)))
require.NoError(t, err)
querierStatsHandler, querierStatsHits, err := getFakeStatsHandler(uint64(statsBytes / math.Max(tc.expectedQuerierStatsHits, 1)))
require.NoError(t, err)
fakeRT, err := newfakeRoundTripper()
require.NoError(t, err)
_, promHandler := promqlResult(matrix)
fakeRT.setHandler(promHandler)
lokiReq := &LokiRequest{
Query: tc.query,
Limit: 1000,
StartTs: tc.queryStart,
EndTs: tc.queryEnd,
Direction: logproto.FORWARD,
Path: "/query_range",
}
ctx := user.InjectOrgID(context.Background(), "foo")
req, err := DefaultCodec.EncodeRequest(ctx, lokiReq)
Max bytes read limit (#8670) **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]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L179-L186 [2]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L72 [3]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L124 [4]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L163 [5]: https://github.com/grafana/loki/blob/f422e0a52b743a11209b8276510feb2ab8241486/pkg/querier/queryrange/roundtrip.go#L521
3 years ago
require.NoError(t, err)
req = req.WithContext(ctx)
err = user.InjectOrgIDIntoHTTPRequest(ctx, req)
require.NoError(t, err)
middlewares := []queryrangebase.Middleware{
Pass engine opts down to middlewares (#9130) **What this PR does / why we need it**: The following middlewares in the query frontend uses a downstream engine: - `NewQuerySizeLimiterMiddleware` and `NewQuerierSizeLimiterMiddleware` - `NewQueryShardMiddleware` - `NewSplitByRangeMiddleware` These were all creating the downstream engine as follows: ```go logql.NewDownstreamEngine(logql.EngineOpts{LogExecutingQuery: false}, DownstreamHandler{next: next, limits: limits}, limits, logger), ``` As can be seen, the [engine options configured in Loki][1] were not being used at all. In the case of `NewQuerySizeLimiterMiddleware`, `NewQuerierSizeLimiterMiddleware` and `NewQueryShardMiddleware`, the downstream engine was created to get the `MaxLookBackPeriod`. When creating a new Downstream Engine as above, the `MaxLookBackPeriod` [would always be the default][2] (30 seconds). This PR fixes this by passing down the engine config to these middlewares, so this config is used to create the new downstream engines. **Which issue(s) this PR fixes**: Adresses some pending tasks from https://github.com/grafana/loki/pull/8670#issuecomment-1507031976. **Special notes for your reviewer**: **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] 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]: https://github.com/grafana/loki/blob/1bcf683513b5e11fb1dcc5625761b9760cfbd958/pkg/querier/querier.go#L52 [2]: https://github.com/grafana/loki/blob/edc6b0bff76714ff584c20d7ff0235461a4f4a88/pkg/logql/engine.go#L136-L140
3 years ago
NewQuerySizeLimiterMiddleware(schemas, testEngineOpts, util_log.Logger, tc.limits, queryStatsHandler),
NewQuerierSizeLimiterMiddleware(schemas, testEngineOpts, util_log.Logger, tc.limits, querierStatsHandler),
Max bytes read limit (#8670) **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]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L179-L186 [2]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L72 [3]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L124 [4]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L163 [5]: https://github.com/grafana/loki/blob/f422e0a52b743a11209b8276510feb2ab8241486/pkg/querier/queryrange/roundtrip.go#L521
3 years ago
}
_, err = queryrangebase.NewRoundTripper(fakeRT, DefaultCodec, nil, middlewares...).RoundTrip(req)
Max bytes read limit (#8670) **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]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L179-L186 [2]: https://github.com/grafana/loki/blob/ff847305afaf7de5eb56436f3683773e88701075/pkg/querier/queryrange/shard_resolver.go#L72 [3]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L124 [4]: https://github.com/grafana/loki/blob/3d2fff3a2d416a48a73346a53ba7499b0eeb67f7/pkg/querier/queryrange/queryrangebase/roundtrip.go#L163 [5]: https://github.com/grafana/loki/blob/f422e0a52b743a11209b8276510feb2ab8241486/pkg/querier/queryrange/roundtrip.go#L521
3 years ago
if tc.shouldErr {
require.Error(t, err)
} else {
require.NoError(t, err)
}
require.Equal(t, tc.expectedQueryStatsHits, *queryStatsHits)
require.Equal(t, tc.expectedQuerierStatsHits, *querierStatsHits)
})
}
}
Pass engine opts down to middlewares (#9130) **What this PR does / why we need it**: The following middlewares in the query frontend uses a downstream engine: - `NewQuerySizeLimiterMiddleware` and `NewQuerierSizeLimiterMiddleware` - `NewQueryShardMiddleware` - `NewSplitByRangeMiddleware` These were all creating the downstream engine as follows: ```go logql.NewDownstreamEngine(logql.EngineOpts{LogExecutingQuery: false}, DownstreamHandler{next: next, limits: limits}, limits, logger), ``` As can be seen, the [engine options configured in Loki][1] were not being used at all. In the case of `NewQuerySizeLimiterMiddleware`, `NewQuerierSizeLimiterMiddleware` and `NewQueryShardMiddleware`, the downstream engine was created to get the `MaxLookBackPeriod`. When creating a new Downstream Engine as above, the `MaxLookBackPeriod` [would always be the default][2] (30 seconds). This PR fixes this by passing down the engine config to these middlewares, so this config is used to create the new downstream engines. **Which issue(s) this PR fixes**: Adresses some pending tasks from https://github.com/grafana/loki/pull/8670#issuecomment-1507031976. **Special notes for your reviewer**: **Checklist** - [x] Reviewed the [`CONTRIBUTING.md`](https://github.com/grafana/loki/blob/main/CONTRIBUTING.md) guide (**required**) - [ ] 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]: https://github.com/grafana/loki/blob/1bcf683513b5e11fb1dcc5625761b9760cfbd958/pkg/querier/querier.go#L52 [2]: https://github.com/grafana/loki/blob/edc6b0bff76714ff584c20d7ff0235461a4f4a88/pkg/logql/engine.go#L136-L140
3 years ago
func Test_MaxQuerySize_MaxLookBackPeriod(t *testing.T) {
engineOpts := testEngineOpts
engineOpts.MaxLookBackPeriod = 1 * time.Hour
lim := fakeLimits{
maxQueryBytesRead: 1 << 10,
maxQuerierBytesRead: 1 << 10,
}
statsHandler := queryrangebase.HandlerFunc(func(_ context.Context, req queryrangebase.Request) (queryrangebase.Response, error) {
// This is the actual check that we're testing.
require.Equal(t, testTime.Add(-engineOpts.MaxLookBackPeriod).UnixMilli(), req.GetStart())
return &IndexStatsResponse{
Response: &logproto.IndexStatsResponse{
Bytes: 1 << 10,
},
}, nil
})
for _, tc := range []struct {
desc string
middleware queryrangebase.Middleware
}{
{
desc: "QuerySizeLimiter",
middleware: NewQuerySizeLimiterMiddleware(testSchemasTSDB, engineOpts, util_log.Logger, lim, statsHandler),
},
{
desc: "QuerierSizeLimiter",
middleware: NewQuerierSizeLimiterMiddleware(testSchemasTSDB, engineOpts, util_log.Logger, lim, statsHandler),
},
} {
t.Run(tc.desc, func(t *testing.T) {
lokiReq := &LokiInstantRequest{
Query: `{cluster="dev-us-central-0"}`,
Limit: 1000,
TimeTs: testTime,
Direction: logproto.FORWARD,
Path: "/loki/api/v1/query",
}
handler := tc.middleware.Wrap(
queryrangebase.HandlerFunc(func(_ context.Context, req queryrangebase.Request) (queryrangebase.Response, error) {
return &LokiResponse{}, nil
}),
)
ctx := user.InjectOrgID(context.Background(), "foo")
_, err := handler.Do(ctx, lokiReq)
require.NoError(t, err)
})
}
}