Define `RequiredLabels` query limit. (#8851)

**What this PR does / why we need it**:
Some end-users can impose great workload on a cluster by selecting too
many streams in their queries. We should be able to limit them.

Therefore we introduce a new limit `RequiredLabelMatchers` which list
label names that must be included in the stream selectors.

The implementation follows the same approach as for max query limit.

**Which issue(s) this PR fixes**:
Fixes #8745

**Checklist**
- [ ] 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`
pull/8670/head^2
Karsten Jeschkies 2 years ago committed by GitHub
parent 4e893a0a88
commit 94725e7908
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 6
      CHANGELOG.md
  2. 3
      docs/sources/configuration/_index.md
  3. 4
      pkg/logcli/client/file.go
  4. 5
      pkg/logql/limits.go
  5. 3
      pkg/logql/log/parser_hints.go
  6. 52
      pkg/querier/queryrange/limits.go
  7. 47
      pkg/querier/queryrange/roundtrip.go
  8. 65
      pkg/querier/queryrange/roundtrip_test.go
  9. 25
      pkg/util/querylimits/limiter.go
  10. 23
      pkg/util/querylimits/limiter_test.go
  11. 1
      pkg/util/querylimits/middleware_test.go
  12. 1
      pkg/util/querylimits/propagation.go
  13. 6
      pkg/validation/limits.go

@ -1,5 +1,11 @@
## Main/Unreleased
#### Loki
##### Enhancements
* [8851](https://github.com/grafana/loki/pull/8851) **jeschkies**: Introduce limit to require a set of labels for selecting streams.
### All Changes
## 2.8.0 (2023-03-??)

@ -2480,6 +2480,9 @@ shard_streams:
[desired_rate: <int>]
[blocked_queries: <blocked_query...>]
# Define a list of required selector labels.
[required_label_matchers: <list of strings>]
```
### frontend_worker

@ -198,6 +198,10 @@ func (l *limiter) BlockedQueries(ctx context.Context, userID string) []*validati
return []*validation.BlockedQuery{}
}
func (l *limiter) RequiredLabels(ctx context.Context, userID string) []string {
return nil
}
type querier struct {
r io.Reader
labels labels.Labels

@ -23,6 +23,7 @@ type fakeLimits struct {
maxSeries int
timeout time.Duration
blockedQueries []*validation.BlockedQuery
requiredLabels []string
}
func (f fakeLimits) MaxQuerySeries(ctx context.Context, userID string) int {
@ -36,3 +37,7 @@ func (f fakeLimits) QueryTimeout(ctx context.Context, userID string) time.Durati
func (f fakeLimits) BlockedQueries(ctx context.Context, userID string) []*validation.BlockedQuery {
return f.blockedQueries
}
func (f fakeLimits) RequiredLabels(ctx context.Context, userID string) []string {
return f.requiredLabels
}

@ -1,8 +1,9 @@
package log
import (
"github.com/grafana/loki/pkg/logqlmodel"
"strings"
"github.com/grafana/loki/pkg/logqlmodel"
)
var noParserHints = &Hints{}

@ -5,6 +5,7 @@ import (
"fmt"
"net/http"
"sort"
"strings"
"sync"
"time"
@ -12,6 +13,7 @@ import (
"github.com/grafana/dskit/tenant"
"github.com/opentracing/opentracing-go"
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/model/labels"
"github.com/prometheus/prometheus/model/timestamp"
"github.com/weaveworks/common/httpgrpc"
"github.com/weaveworks/common/user"
@ -27,7 +29,9 @@ import (
)
const (
limitErrTmpl = "maximum of series (%d) reached for a single query"
limitErrTmpl = "maximum of series (%d) reached for a single query"
maxSeriesErrTmpl = "max entries limit per query exceeded, limit > max_entries_limit (%d > %d)"
requiredLabelsErrTmpl = "stream selector is missing required matchers [%s], labels present in the query were [%s]"
)
var (
@ -45,6 +49,7 @@ type Limits interface {
// TSDBMaxQueryParallelism returns the limit to the number of split queries the
// frontend will process in parallel for TSDB queries.
TSDBMaxQueryParallelism(context.Context, string) int
RequiredLabels(context.Context, string) []string
}
type limits struct {
@ -508,3 +513,48 @@ func MinWeightedParallelism(ctx context.Context, tenantIDs []string, configs []c
)
})
}
// validates log entries limits
func validateMaxEntriesLimits(req *http.Request, reqLimit uint32, limits Limits) error {
tenantIDs, err := tenant.TenantIDs(req.Context())
if err != nil {
return httpgrpc.Errorf(http.StatusBadRequest, err.Error())
}
maxEntriesCapture := func(id string) int { return limits.MaxEntriesLimitPerQuery(req.Context(), id) }
maxEntriesLimit := validation.SmallestPositiveNonZeroIntPerTenant(tenantIDs, maxEntriesCapture)
if int(reqLimit) > maxEntriesLimit && maxEntriesLimit != 0 {
return fmt.Errorf(maxSeriesErrTmpl, reqLimit, maxEntriesLimit)
}
return nil
}
func validateMatchers(req *http.Request, limits Limits, matchers []*labels.Matcher) error {
tenants, err := tenant.TenantIDs(req.Context())
if err != nil {
return err
}
actual := make(map[string]struct{}, len(matchers))
var present []string
for _, m := range matchers {
actual[m.Name] = struct{}{}
present = append(present, m.Name)
}
for _, tenant := range tenants {
required := limits.RequiredLabels(req.Context(), tenant)
var missing []string
for _, label := range required {
if _, found := actual[label]; !found {
missing = append(missing, label)
}
}
if len(missing) > 0 {
return fmt.Errorf(requiredLabelsErrTmpl, strings.Join(missing, ", "), strings.Join(present, ", "))
}
}
return nil
}

@ -14,8 +14,6 @@ import (
"github.com/prometheus/prometheus/model/labels"
"github.com/weaveworks/common/httpgrpc"
"github.com/grafana/dskit/tenant"
"github.com/grafana/loki/pkg/loghttp"
"github.com/grafana/loki/pkg/logql"
"github.com/grafana/loki/pkg/logql/syntax"
@ -24,7 +22,6 @@ import (
"github.com/grafana/loki/pkg/storage/chunk/cache"
"github.com/grafana/loki/pkg/storage/config"
logutil "github.com/grafana/loki/pkg/util/log"
"github.com/grafana/loki/pkg/util/validation"
)
// Config is the configuration for the queryrange tripperware
@ -137,6 +134,7 @@ func newRoundTripper(logger log.Logger, next, limited, log, metric, series, labe
}
func (r roundTripper) RoundTrip(req *http.Request) (*http.Response, error) {
logger := logutil.WithContext(req.Context(), r.logger)
err := req.ParseForm()
if err != nil {
return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error())
@ -154,10 +152,21 @@ func (r roundTripper) RoundTrip(req *http.Request) (*http.Response, error) {
}
queryHash := logql.HashedQuery(rangeQuery.Query)
level.Info(logutil.WithContext(req.Context(), r.logger)).Log("msg", "executing query", "type", "range", "query", rangeQuery.Query, "length", rangeQuery.End.Sub(rangeQuery.Start), "step", rangeQuery.Step, "query_hash", queryHash)
level.Info(logger).Log("msg", "executing query", "type", "range", "query", rangeQuery.Query, "length", rangeQuery.End.Sub(rangeQuery.Start), "step", rangeQuery.Step, "query_hash", queryHash)
switch e := expr.(type) {
case syntax.SampleExpr:
// The error will be handled later.
groups, err := e.MatcherGroups()
if err != nil {
level.Warn(logger).Log("msg", "unexpected matcher groups error in roundtripper", "err", err)
}
for _, g := range groups {
if err := validateMatchers(req, r.limits, g.Matchers); err != nil {
return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error())
}
}
return r.metric.RoundTrip(req)
case syntax.LogSelectorExpr:
// Note, this function can mutate the request
@ -166,8 +175,13 @@ func (r roundTripper) RoundTrip(req *http.Request) (*http.Response, error) {
return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error())
}
if err := validateMaxEntriesLimits(req, rangeQuery.Limit, r.limits); err != nil {
return nil, err
return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error())
}
if err := validateMatchers(req, r.limits, e.Matchers()); err != nil {
return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error())
}
// Only filter expressions are query sharded
if !expr.HasFilter() {
return r.limited.RoundTrip(req)
@ -183,7 +197,7 @@ func (r roundTripper) RoundTrip(req *http.Request) (*http.Response, error) {
return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error())
}
level.Info(logutil.WithContext(req.Context(), r.logger)).Log("msg", "executing query", "type", "series", "match", logql.PrintMatches(sr.Groups), "length", sr.End.Sub(sr.Start))
level.Info(logger).Log("msg", "executing query", "type", "series", "match", logql.PrintMatches(sr.Groups), "length", sr.End.Sub(sr.Start))
return r.series.RoundTrip(req)
case LabelNamesOp:
@ -192,7 +206,7 @@ func (r roundTripper) RoundTrip(req *http.Request) (*http.Response, error) {
return nil, httpgrpc.Errorf(http.StatusBadRequest, err.Error())
}
level.Info(logutil.WithContext(req.Context(), r.logger)).Log("msg", "executing query", "type", "labels", "label", lr.Name, "length", lr.End.Sub(*lr.Start))
level.Info(logger).Log("msg", "executing query", "type", "labels", "label", lr.Name, "length", lr.End.Sub(*lr.Start))
return r.labels.RoundTrip(req)
case InstantQueryOp:
@ -206,7 +220,7 @@ func (r roundTripper) RoundTrip(req *http.Request) (*http.Response, error) {
}
queryHash := logql.HashedQuery(instantQuery.Query)
level.Info(logutil.WithContext(req.Context(), r.logger)).Log("msg", "executing query", "type", "instant", "query", instantQuery.Query, "query_hash", queryHash)
level.Info(logger).Log("msg", "executing query", "type", "instant", "query", instantQuery.Query, "query_hash", queryHash)
switch expr.(type) {
case syntax.SampleExpr:
@ -238,23 +252,6 @@ func transformRegexQuery(req *http.Request, expr syntax.LogSelectorExpr) (syntax
return expr, nil
}
// validates log entries limits
func validateMaxEntriesLimits(req *http.Request, reqLimit uint32, limits Limits) error {
tenantIDs, err := tenant.TenantIDs(req.Context())
if err != nil {
return httpgrpc.Errorf(http.StatusBadRequest, err.Error())
}
maxEntriesCapture := func(id string) int { return limits.MaxEntriesLimitPerQuery(req.Context(), id) }
maxEntriesLimit := validation.SmallestPositiveNonZeroIntPerTenant(tenantIDs, maxEntriesCapture)
if int(reqLimit) > maxEntriesLimit && maxEntriesLimit != 0 {
return httpgrpc.Errorf(http.StatusBadRequest,
"max entries limit per query exceeded, limit > max_entries_limit (%d > %d)", reqLimit, maxEntriesLimit)
}
return nil
}
const (
InstantQueryOp = "instant_query"
QueryRangeOp = "query_range"

@ -470,8 +470,8 @@ func TestPostQueries(t *testing.T) {
require.NoError(t, err)
}
func TestEntriesLimitsTripperware(t *testing.T) {
tpw, stopper, err := NewTripperware(testConfig, util_log.Logger, fakeLimits{maxEntriesLimitPerQuery: 5000}, config.SchemaConfig{Configs: testSchemas}, nil, false, nil)
func TestTripperware_EntriesLimit(t *testing.T) {
tpw, stopper, err := NewTripperware(testConfig, util_log.Logger, fakeLimits{maxEntriesLimitPerQuery: 5000, maxQueryParallelism: 1}, config.SchemaConfig{Configs: testSchemas}, nil, false, nil)
if stopper != nil {
defer stopper.Stop()
}
@ -501,6 +501,62 @@ func TestEntriesLimitsTripperware(t *testing.T) {
require.Equal(t, httpgrpc.Errorf(http.StatusBadRequest, "max entries limit per query exceeded, limit > max_entries_limit (10000 > 5000)"), err)
}
func TestTripperware_RequiredLabels(t *testing.T) {
const noErr = ""
for _, test := range []struct {
qs string
expectedError string
response parser.Value
}{
{`avg(count_over_time({app=~"foo|bar"} |~".+bar" [1m]))`, noErr, vector},
{`count_over_time({app="foo"}[1m]) / count_over_time({app="bar"}[1m] offset 1m)`, noErr, vector},
{`count_over_time({app="foo"}[1m]) / count_over_time({pod="bar"}[1m] offset 1m)`, "stream selector is missing required matchers [app], labels present in the query were [pod]", nil},
{`avg(count_over_time({pod=~"foo|bar"} |~".+bar" [1m]))`, "stream selector is missing required matchers [app], labels present in the query were [pod]", nil},
{`{app="foo", pod="bar"}`, noErr, streams},
{`{pod="bar"} |= "foo" |~ ".+bar"`, "stream selector is missing required matchers [app], labels present in the query were [pod]", nil},
} {
t.Run(test.qs, func(t *testing.T) {
limits := fakeLimits{maxEntriesLimitPerQuery: 5000, maxQueryParallelism: 1, requiredLabels: []string{"app"}}
tpw, stopper, err := NewTripperware(testConfig, util_log.Logger, limits, 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()
_, h := promqlResult(test.response)
rt.setHandler(h)
lreq := &LokiRequest{
Query: test.qs,
Limit: 1000,
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 := LokiCodec.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)
if test.expectedError != "" {
require.Equal(t, httpgrpc.Errorf(http.StatusBadRequest, test.expectedError), err)
} else {
require.NoError(t, err)
}
})
}
}
func Test_getOperation(t *testing.T) {
cases := []struct {
name string
@ -582,6 +638,7 @@ type fakeLimits struct {
splits map[string]time.Duration
minShardingLookback time.Duration
queryTimeout time.Duration
requiredLabels []string
}
func (f fakeLimits) QuerySplitDuration(key string) time.Duration {
@ -634,6 +691,10 @@ func (f fakeLimits) BlockedQueries(context.Context, string) []*validation.Blocke
return []*validation.BlockedQuery{}
}
func (f fakeLimits) RequiredLabels(context.Context, string) []string {
return f.requiredLabels
}
func counter() (*int, http.Handler) {
count := 0
var lock sync.Mutex

@ -67,3 +67,28 @@ func (l *Limiter) QueryTimeout(ctx context.Context, userID string) time.Duration
}
return time.Duration(requestLimits.QueryTimeout)
}
func (l *Limiter) RequiredLabels(ctx context.Context, userID string) []string {
original := l.CombinedLimits.RequiredLabels(ctx, userID)
requestLimits := ExtractQueryLimitsContext(ctx)
if requestLimits == nil {
return original
}
// The most restricting is a union of both slices
unionMap := make(map[string]struct{})
for _, label := range original {
unionMap[label] = struct{}{}
}
for _, label := range requestLimits.RequiredLabels {
unionMap[label] = struct{}{}
}
union := make([]string, 0, len(unionMap))
for label := range unionMap {
union = append(union, label)
}
return union
}

@ -38,6 +38,7 @@ func TestLimiter_Defaults(t *testing.T) {
MaxQueryLookback: model.Duration(30 * time.Second),
MaxQueryLength: model.Duration(30 * time.Second),
MaxEntriesLimitPerQuery: 10,
RequiredLabels: []string{"foo", "bar"},
}
overrides, _ := validation.NewOverrides(validation.Limits{}, newMockTenantLimits(tLimits))
@ -48,6 +49,7 @@ func TestLimiter_Defaults(t *testing.T) {
MaxQueryLookback: model.Duration(30 * time.Second),
MaxEntriesLimitPerQuery: 10,
QueryTimeout: model.Duration(30 * time.Second),
RequiredLabels: []string{"foo", "bar"},
}
ctx := context.Background()
queryLookback := l.MaxQueryLookback(ctx, "fake")
@ -66,6 +68,7 @@ func TestLimiter_Defaults(t *testing.T) {
MaxQueryLookback: model.Duration(30 * time.Second),
MaxEntriesLimitPerQuery: 10,
QueryTimeout: model.Duration(29 * time.Second),
RequiredLabels: []string{"foo", "bar"},
}
{
ctx2 := InjectQueryLimitsContext(context.Background(), limits)
@ -138,3 +141,23 @@ func TestLimiter_AcceptLowerLimits(t *testing.T) {
require.Equal(t, limits.MaxEntriesLimitPerQuery, l.MaxEntriesLimitPerQuery(ctx, "fake"))
require.Equal(t, time.Duration(limits.QueryTimeout), l.QueryTimeout(ctx, "fake"))
}
func TestLimiter_MergeLimits(t *testing.T) {
// some fake tenant
tLimits := make(map[string]*validation.Limits)
tLimits["fake"] = &validation.Limits{
RequiredLabels: []string{"one", "two"},
}
overrides, _ := validation.NewOverrides(validation.Limits{}, newMockTenantLimits(tLimits))
l := NewLimiter(log.NewNopLogger(), overrides)
limits := QueryLimits{
RequiredLabels: []string{"one", "three"},
}
require.ElementsMatch(t, []string{"one", "two"}, l.RequiredLabels(context.Background(), "fake"))
ctx := InjectQueryLimitsContext(context.Background(), limits)
require.ElementsMatch(t, []string{"one", "two", "three"}, l.RequiredLabels(ctx, "fake"))
}

@ -31,6 +31,7 @@ func Test_MiddlewareWithHeader(t *testing.T) {
model.Duration(1 * time.Second),
1,
model.Duration(1 * time.Second),
[]string{"foo", "bar"},
}
nextHandler := http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {

@ -24,6 +24,7 @@ type QueryLimits struct {
MaxQueryLookback model.Duration `json:"maxQueryLookback,omitempty"`
MaxEntriesLimitPerQuery int `json:"maxEntriesLimitPerQuery,omitempty"`
QueryTimeout model.Duration `json:"queryTimeout,omitempty"`
RequiredLabels []string `json:"requiredLabels,omitempty"`
}
func UnmarshalQueryLimits(data []byte) (*QueryLimits, error) {

@ -166,6 +166,8 @@ type Limits struct {
ShardStreams *shardstreams.Config `yaml:"shard_streams" json:"shard_streams"`
BlockedQueries []*validation.BlockedQuery `yaml:"blocked_queries,omitempty" json:"blocked_queries,omitempty"`
RequiredLabels []string `yaml:"required_label_matchers,omitempty" json:"required_label_matchers,omitempty" doc:"description=Define a list of required selector labels."`
}
type StreamRetention struct {
@ -672,6 +674,10 @@ func (o *Overrides) BlockedQueries(ctx context.Context, userID string) []*valida
return o.getOverridesForUser(userID).BlockedQueries
}
func (o *Overrides) RequiredLabels(ctx context.Context, userID string) []string {
return o.getOverridesForUser(userID).RequiredLabels
}
func (o *Overrides) DefaultLimits() *Limits {
return o.defaultLimits
}

Loading…
Cancel
Save