Skip to content

Commit b0039f7

Browse files
committed
max chunks per query limit shared between ingesters and storage gateways
Signed-off-by: Alan Protasio <[email protected]>
1 parent 5e50806 commit b0039f7

File tree

10 files changed

+53
-73
lines changed

10 files changed

+53
-73
lines changed

CHANGELOG.md

+1-1
Original file line numberDiff line numberDiff line change
@@ -45,7 +45,7 @@
4545
* [BUGFIX] Fixed cache fetch error on Redis Cluster. #4056
4646
* [BUGFIX] Ingester: fix issue where runtime limits erroneously override default limits. #4246
4747
* [BUGFIX] Ruler: fix startup in single-binary mode when the new `ruler_storage` is used. #4252
48-
48+
* [CHANGE] Querier / ruler: Migrate the -querier.max-fetched-chunks-per-query limit to the new QueryLimiter to limit the number of chunks returned as a sum of chunks returned from the ingester and storage gateway.
4949
## Blocksconvert
5050

5151
* [ENHANCEMENT] Scanner: add support for DynamoDB (v9 schema only). #3828

docs/configuration/config-file-reference.md

+3-5
Original file line numberDiff line numberDiff line change
@@ -4028,11 +4028,9 @@ The `limits_config` configures default and per-tenant limits imposed by Cortex s
40284028
[max_chunks_per_query: <int> | default = 2000000]
40294029
40304030
# Maximum number of chunks that can be fetched in a single query from ingesters
4031-
# and long-term storage: the total number of actual fetched chunks could be 2x
4032-
# the limit, being independently applied when querying ingesters and long-term
4033-
# storage. This limit is enforced in the ingester (if chunks streaming is
4034-
# enabled), querier, ruler and store-gateway. Takes precedence over the
4035-
# deprecated -store.query-chunk-limit. 0 to disable.
4031+
# and long-term storage. This limit is enforced in the ingester (if chunks
4032+
# streaming is enabled), querier, ruler and store-gateway. Takes precedence over
4033+
# the deprecated -store.query-chunk-limit. 0 to disable.
40364034
# CLI flag: -querier.max-fetched-chunks-per-query
40374035
[max_fetched_chunks_per_query: <int> | default = 0]
40384036

pkg/distributor/distributor_test.go

+4-2
Original file line numberDiff line numberDiff line change
@@ -912,6 +912,8 @@ func TestDistributor_QueryStream_ShouldReturnErrorIfMaxChunksPerQueryLimitIsReac
912912
shardByAllLabels: true,
913913
limits: limits,
914914
})
915+
916+
ctx = limiter.AddQueryLimiterToContext(ctx, limiter.NewQueryLimiter(0, 0, maxChunksLimit))
915917
defer stopAll(ds, r)
916918

917919
// Push a number of series below the max chunks limit. Each series has 1 sample,
@@ -957,7 +959,7 @@ func TestDistributor_QueryStream_ShouldReturnErrorIfMaxSeriesPerQueryLimitIsReac
957959
ctx := user.InjectOrgID(context.Background(), "user")
958960
limits := &validation.Limits{}
959961
flagext.DefaultValues(limits)
960-
ctx = limiter.AddQueryLimiterToContext(ctx, limiter.NewQueryLimiter(maxSeriesLimit, 0))
962+
ctx = limiter.AddQueryLimiterToContext(ctx, limiter.NewQueryLimiter(maxSeriesLimit, 0, 0))
961963

962964
// Prepare distributors.
963965
ds, _, r, _ := prepare(t, prepConfig{
@@ -1040,7 +1042,7 @@ func TestDistributor_QueryStream_ShouldReturnErrorIfMaxChunkBytesPerQueryLimitIs
10401042
var maxBytesLimit = (seriesToAdd) * responseChunkSize
10411043

10421044
// Update the limiter with the calculated limits.
1043-
ctx = limiter.AddQueryLimiterToContext(ctx, limiter.NewQueryLimiter(0, maxBytesLimit))
1045+
ctx = limiter.AddQueryLimiterToContext(ctx, limiter.NewQueryLimiter(0, maxBytesLimit, 0))
10441046

10451047
// Push a number of series below the max chunk bytes limit. Subtract one for the series added above.
10461048
writeReq = makeWriteRequest(0, seriesToAdd-1, 0)

pkg/distributor/query.go

+5-23
Original file line numberDiff line numberDiff line change
@@ -2,15 +2,13 @@ package distributor
22

33
import (
44
"context"
5-
"fmt"
65
"io"
76
"time"
87

98
"github.com/opentracing/opentracing-go"
109
"github.com/prometheus/common/model"
1110
"github.com/prometheus/prometheus/pkg/labels"
1211
"github.com/weaveworks/common/instrument"
13-
"go.uber.org/atomic"
1412

1513
"github.com/cortexproject/cortex/pkg/cortexpb"
1614
ingester_client "github.com/cortexproject/cortex/pkg/ingester/client"
@@ -23,10 +21,6 @@ import (
2321
"github.com/cortexproject/cortex/pkg/util/validation"
2422
)
2523

26-
var (
27-
errMaxChunksPerQueryLimit = "the query hit the max number of chunks limit while fetching chunks from ingesters for %s (limit: %d)"
28-
)
29-
3024
// Query multiple ingesters and returns a Matrix of samples.
3125
func (d *Distributor) Query(ctx context.Context, from, to model.Time, matchers ...*labels.Matcher) (model.Matrix, error) {
3226
var matrix model.Matrix
@@ -58,11 +52,6 @@ func (d *Distributor) Query(ctx context.Context, from, to model.Time, matchers .
5852
func (d *Distributor) QueryStream(ctx context.Context, from, to model.Time, matchers ...*labels.Matcher) (*ingester_client.QueryStreamResponse, error) {
5953
var result *ingester_client.QueryStreamResponse
6054
err := instrument.CollectedRequest(ctx, "Distributor.QueryStream", d.queryDuration, instrument.ErrorCode, func(ctx context.Context) error {
61-
userID, err := tenant.TenantID(ctx)
62-
if err != nil {
63-
return err
64-
}
65-
6655
req, err := ingester_client.ToQueryRequest(from, to, matchers)
6756
if err != nil {
6857
return err
@@ -73,7 +62,7 @@ func (d *Distributor) QueryStream(ctx context.Context, from, to model.Time, matc
7362
return err
7463
}
7564

76-
result, err = d.queryIngesterStream(ctx, userID, replicationSet, req)
65+
result, err = d.queryIngesterStream(ctx, replicationSet, req)
7766
if err != nil {
7867
return err
7968
}
@@ -186,10 +175,8 @@ func (d *Distributor) queryIngesters(ctx context.Context, replicationSet ring.Re
186175
}
187176

188177
// queryIngesterStream queries the ingesters using the new streaming API.
189-
func (d *Distributor) queryIngesterStream(ctx context.Context, userID string, replicationSet ring.ReplicationSet, req *ingester_client.QueryRequest) (*ingester_client.QueryStreamResponse, error) {
178+
func (d *Distributor) queryIngesterStream(ctx context.Context, replicationSet ring.ReplicationSet, req *ingester_client.QueryRequest) (*ingester_client.QueryStreamResponse, error) {
190179
var (
191-
chunksLimit = d.limits.MaxChunksPerQueryFromIngesters(userID)
192-
chunksCount = atomic.Int32{}
193180
queryLimiter = limiter.QueryLimiterFromContextWithFallback(ctx)
194181
)
195182

@@ -223,14 +210,9 @@ func (d *Distributor) queryIngesterStream(ctx context.Context, userID string, re
223210
}
224211

225212
// Enforce the max chunks limits.
226-
if chunksLimit > 0 {
227-
if count := int(chunksCount.Add(int32(resp.ChunksCount()))); count > chunksLimit {
228-
// We expect to be always able to convert the label matchers back to Prometheus ones.
229-
// In case we fail (unexpected) the error will not include the matchers, but the core
230-
// logic doesn't break.
231-
matchers, _ := ingester_client.FromLabelMatchers(req.Matchers)
232-
return nil, validation.LimitError(fmt.Sprintf(errMaxChunksPerQueryLimit, util.LabelMatchersToString(matchers), chunksLimit))
233-
}
213+
matchers, _ := ingester_client.FromLabelMatchers(req.Matchers)
214+
if chunkLimitErr := queryLimiter.AddChunks(resp.ChunksCount(), matchers); chunkLimitErr != nil {
215+
return nil, validation.LimitError(chunkLimitErr.Error())
234216
}
235217

236218
for _, series := range resp.Chunkseries {

pkg/querier/blocks_store_queryable.go

+5-18
Original file line numberDiff line numberDiff line change
@@ -54,8 +54,7 @@ const (
5454
)
5555

5656
var (
57-
errNoStoreGatewayAddress = errors.New("no store-gateway address configured")
58-
errMaxChunksPerQueryLimit = "the query hit the max number of chunks limit while fetching chunks from store-gateways for %s (limit: %d)"
57+
errNoStoreGatewayAddress = errors.New("no store-gateway address configured")
5958
)
6059

6160
// BlocksStoreSet is the interface used to get the clients to query series on a set of blocks.
@@ -403,14 +402,11 @@ func (q *blocksStoreQuerier) selectSorted(sp *storage.SelectHints, matchers ...*
403402
resSeriesSets = []storage.SeriesSet(nil)
404403
resWarnings = storage.Warnings(nil)
405404

406-
maxChunksLimit = q.limits.MaxChunksPerQueryFromStore(q.userID)
407-
leftChunksLimit = maxChunksLimit
408-
409405
resultMtx sync.Mutex
410406
)
411407

412408
queryFunc := func(clients map[BlocksStoreClient][]ulid.ULID, minT, maxT int64) ([]ulid.ULID, error) {
413-
seriesSets, queriedBlocks, warnings, numChunks, err := q.fetchSeriesFromStores(spanCtx, sp, clients, minT, maxT, matchers, convertedMatchers, maxChunksLimit, leftChunksLimit)
409+
seriesSets, queriedBlocks, warnings, _, err := q.fetchSeriesFromStores(spanCtx, sp, clients, minT, maxT, matchers, convertedMatchers)
414410
if err != nil {
415411
return nil, err
416412
}
@@ -420,11 +416,6 @@ func (q *blocksStoreQuerier) selectSorted(sp *storage.SelectHints, matchers ...*
420416
resSeriesSets = append(resSeriesSets, seriesSets...)
421417
resWarnings = append(resWarnings, warnings...)
422418

423-
// Given a single block is guaranteed to not be queried twice, we can safely decrease the number of
424-
// chunks we can still read before hitting the limit (max == 0 means disabled).
425-
if maxChunksLimit > 0 {
426-
leftChunksLimit -= numChunks
427-
}
428419
resultMtx.Unlock()
429420

430421
return queriedBlocks, nil
@@ -552,8 +543,6 @@ func (q *blocksStoreQuerier) fetchSeriesFromStores(
552543
maxT int64,
553544
matchers []*labels.Matcher,
554545
convertedMatchers []storepb.LabelMatcher,
555-
maxChunksLimit int,
556-
leftChunksLimit int,
557546
) ([]storage.SeriesSet, []ulid.ULID, storage.Warnings, int, error) {
558547
var (
559548
reqCtx = grpc_metadata.AppendToOutgoingContext(ctx, cortex_tsdb.TenantIDExternalLabel, q.userID)
@@ -620,12 +609,10 @@ func (q *blocksStoreQuerier) fetchSeriesFromStores(
620609
}
621610

622611
// Ensure the max number of chunks limit hasn't been reached (max == 0 means disabled).
623-
if maxChunksLimit > 0 {
624-
actual := numChunks.Add(int32(len(s.Chunks)))
625-
if actual > int32(leftChunksLimit) {
626-
return validation.LimitError(fmt.Sprintf(errMaxChunksPerQueryLimit, util.LabelMatchersToString(matchers), maxChunksLimit))
627-
}
612+
if chunkLimitErr := queryLimiter.AddChunks(len(s.Chunks), matchers); chunkLimitErr != nil {
613+
return validation.LimitError(chunkLimitErr.Error())
628614
}
615+
629616
chunksSize := 0
630617
for _, c := range s.Chunks {
631618
chunksSize += c.Size()

pkg/querier/blocks_store_queryable_test.go

+7-7
Original file line numberDiff line numberDiff line change
@@ -51,7 +51,7 @@ func TestBlocksStoreQuerier_Select(t *testing.T) {
5151
metricNameLabel = labels.Label{Name: labels.MetricName, Value: metricName}
5252
series1Label = labels.Label{Name: "series", Value: "1"}
5353
series2Label = labels.Label{Name: "series", Value: "2"}
54-
noOpQueryLimiter = limiter.NewQueryLimiter(0, 0)
54+
noOpQueryLimiter = limiter.NewQueryLimiter(0, 0, 0)
5555
)
5656

5757
type valueResult struct {
@@ -451,8 +451,8 @@ func TestBlocksStoreQuerier_Select(t *testing.T) {
451451
},
452452
},
453453
limits: &blocksStoreLimitsMock{maxChunksPerQuery: 1},
454-
queryLimiter: noOpQueryLimiter,
455-
expectedErr: validation.LimitError(fmt.Sprintf(errMaxChunksPerQueryLimit, fmt.Sprintf("{__name__=%q}", metricName), 1)),
454+
queryLimiter: limiter.NewQueryLimiter(0, 0, 1),
455+
expectedErr: validation.LimitError(fmt.Sprintf(limiter.ErrMaxChunksPerQueryLimit, fmt.Sprintf("{__name__=%q}", metricName), 1)),
456456
},
457457
"max chunks per query limit hit while fetching chunks during subsequent attempts": {
458458
finderResult: bucketindex.Blocks{
@@ -489,8 +489,8 @@ func TestBlocksStoreQuerier_Select(t *testing.T) {
489489
},
490490
},
491491
limits: &blocksStoreLimitsMock{maxChunksPerQuery: 3},
492-
queryLimiter: noOpQueryLimiter,
493-
expectedErr: validation.LimitError(fmt.Sprintf(errMaxChunksPerQueryLimit, fmt.Sprintf("{__name__=%q}", metricName), 3)),
492+
queryLimiter: limiter.NewQueryLimiter(0, 0, 3),
493+
expectedErr: validation.LimitError(fmt.Sprintf(limiter.ErrMaxChunksPerQueryLimit, fmt.Sprintf("{__name__=%q}", metricName), 3)),
494494
},
495495
"max series per query limit hit while fetching chunks": {
496496
finderResult: bucketindex.Blocks{
@@ -507,7 +507,7 @@ func TestBlocksStoreQuerier_Select(t *testing.T) {
507507
},
508508
},
509509
limits: &blocksStoreLimitsMock{},
510-
queryLimiter: limiter.NewQueryLimiter(1, 0),
510+
queryLimiter: limiter.NewQueryLimiter(1, 0, 0),
511511
expectedErr: validation.LimitError(fmt.Sprintf(limiter.ErrMaxSeriesHit, 1)),
512512
},
513513
"max chunk bytes per query limit hit while fetching chunks": {
@@ -525,7 +525,7 @@ func TestBlocksStoreQuerier_Select(t *testing.T) {
525525
},
526526
},
527527
limits: &blocksStoreLimitsMock{maxChunksPerQuery: 1},
528-
queryLimiter: limiter.NewQueryLimiter(0, 8),
528+
queryLimiter: limiter.NewQueryLimiter(0, 8, 0),
529529
expectedErr: validation.LimitError(fmt.Sprintf(limiter.ErrMaxChunkBytesHit, 8)),
530530
},
531531
}

pkg/querier/querier.go

+1-1
Original file line numberDiff line numberDiff line change
@@ -224,7 +224,7 @@ func NewQueryable(distributor QueryableWithFilter, stores []QueryableWithFilter,
224224
return nil, err
225225
}
226226

227-
ctx = limiter.AddQueryLimiterToContext(ctx, limiter.NewQueryLimiter(limits.MaxFetchedSeriesPerQuery(userID), limits.MaxFetchedChunkBytesPerQuery(userID)))
227+
ctx = limiter.AddQueryLimiterToContext(ctx, limiter.NewQueryLimiter(limits.MaxFetchedSeriesPerQuery(userID), limits.MaxFetchedChunkBytesPerQuery(userID), limits.MaxChunksPerQueryFromStore(userID)))
228228

229229
mint, maxt, err = validateQueryTimeRange(ctx, userID, mint, maxt, limits, cfg.MaxQueryIntoFuture)
230230
if err == errEmptyTimeRange {

pkg/util/limiter/query_limiter.go

+22-5
Original file line numberDiff line numberDiff line change
@@ -6,39 +6,45 @@ import (
66
"sync"
77

88
"github.com/prometheus/common/model"
9+
"github.com/prometheus/prometheus/pkg/labels"
910
"go.uber.org/atomic"
1011

1112
"github.com/cortexproject/cortex/pkg/cortexpb"
1213
"github.com/cortexproject/cortex/pkg/ingester/client"
14+
"github.com/cortexproject/cortex/pkg/util"
1315
)
1416

1517
type queryLimiterCtxKey struct{}
1618

1719
var (
18-
ctxKey = &queryLimiterCtxKey{}
19-
ErrMaxSeriesHit = "the query hit the max number of series limit (limit: %d series)"
20-
ErrMaxChunkBytesHit = "the query hit the aggregated chunks size limit (limit: %d bytes)"
20+
ctxKey = &queryLimiterCtxKey{}
21+
ErrMaxSeriesHit = "the query hit the max number of series limit (limit: %d series)"
22+
ErrMaxChunkBytesHit = "the query hit the aggregated chunks size limit (limit: %d bytes)"
23+
ErrMaxChunksPerQueryLimit = "the query hit the max number of chunks limit while fetching chunks from ingesters for %s (limit: %d)"
2124
)
2225

2326
type QueryLimiter struct {
2427
uniqueSeriesMx sync.Mutex
2528
uniqueSeries map[model.Fingerprint]struct{}
2629

2730
chunkBytesCount atomic.Int64
31+
chunkCount atomic.Int64
2832

2933
maxSeriesPerQuery int
3034
maxChunkBytesPerQuery int
35+
maxChunksPerQuery int
3136
}
3237

3338
// NewQueryLimiter makes a new per-query limiter. Each query limiter
3439
// is configured using the `maxSeriesPerQuery` limit.
35-
func NewQueryLimiter(maxSeriesPerQuery, maxChunkBytesPerQuery int) *QueryLimiter {
40+
func NewQueryLimiter(maxSeriesPerQuery, maxChunkBytesPerQuery int, maxChunksPerQuery int) *QueryLimiter {
3641
return &QueryLimiter{
3742
uniqueSeriesMx: sync.Mutex{},
3843
uniqueSeries: map[model.Fingerprint]struct{}{},
3944

4045
maxSeriesPerQuery: maxSeriesPerQuery,
4146
maxChunkBytesPerQuery: maxChunkBytesPerQuery,
47+
maxChunksPerQuery: maxChunksPerQuery,
4248
}
4349
}
4450

@@ -52,7 +58,7 @@ func QueryLimiterFromContextWithFallback(ctx context.Context) *QueryLimiter {
5258
ql, ok := ctx.Value(ctxKey).(*QueryLimiter)
5359
if !ok {
5460
// If there's no limiter return a new unlimited limiter as a fallback
55-
ql = NewQueryLimiter(0, 0)
61+
ql = NewQueryLimiter(0, 0, 0)
5662
}
5763
return ql
5864
}
@@ -93,3 +99,14 @@ func (ql *QueryLimiter) AddChunkBytes(chunkSizeInBytes int) error {
9399
}
94100
return nil
95101
}
102+
103+
func (ql *QueryLimiter) AddChunks(count int, matchers []*labels.Matcher) error {
104+
if ql.maxChunksPerQuery == 0 {
105+
return nil
106+
}
107+
108+
if ql.chunkCount.Add(int64(count)) > int64(ql.maxChunksPerQuery) {
109+
return fmt.Errorf(fmt.Sprintf(ErrMaxChunksPerQueryLimit, util.LabelMatchersToString(matchers), ql.maxChunksPerQuery))
110+
}
111+
return nil
112+
}

pkg/util/limiter/query_limiter_test.go

+4-4
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,7 @@ func TestQueryLimiter_AddSeries_ShouldReturnNoErrorOnLimitNotExceeded(t *testing
2525
labels.MetricName: metricName + "_2",
2626
"series2": "1",
2727
})
28-
limiter = NewQueryLimiter(100, 0)
28+
limiter = NewQueryLimiter(100, 0, 0)
2929
)
3030
err := limiter.AddSeries(cortexpb.FromLabelsToLabelAdapters(series1))
3131
assert.NoError(t, err)
@@ -53,7 +53,7 @@ func TestQueryLimiter_AddSeriers_ShouldReturnErrorOnLimitExceeded(t *testing.T)
5353
labels.MetricName: metricName + "_2",
5454
"series2": "1",
5555
})
56-
limiter = NewQueryLimiter(1, 0)
56+
limiter = NewQueryLimiter(1, 0, 0)
5757
)
5858
err := limiter.AddSeries(cortexpb.FromLabelsToLabelAdapters(series1))
5959
require.NoError(t, err)
@@ -62,7 +62,7 @@ func TestQueryLimiter_AddSeriers_ShouldReturnErrorOnLimitExceeded(t *testing.T)
6262
}
6363

6464
func TestQueryLimiter_AddChunkBytes(t *testing.T) {
65-
var limiter = NewQueryLimiter(0, 100)
65+
var limiter = NewQueryLimiter(0, 100, 0)
6666

6767
err := limiter.AddChunkBytes(100)
6868
require.NoError(t, err)
@@ -84,7 +84,7 @@ func BenchmarkQueryLimiter_AddSeries(b *testing.B) {
8484
}
8585
b.ResetTimer()
8686

87-
limiter := NewQueryLimiter(b.N+1, 0)
87+
limiter := NewQueryLimiter(b.N+1, 0, 0)
8888
for _, s := range series {
8989
err := limiter.AddSeries(cortexpb.FromLabelsToLabelAdapters(s))
9090
assert.NoError(b, err)

pkg/util/validation/limits.go

+1-7
Original file line numberDiff line numberDiff line change
@@ -147,7 +147,7 @@ func (l *Limits) RegisterFlags(f *flag.FlagSet) {
147147
f.IntVar(&l.MaxGlobalMetricsWithMetadataPerUser, "ingester.max-global-metadata-per-user", 0, "The maximum number of active metrics with metadata per user, across the cluster. 0 to disable. Supported only if -distributor.shard-by-all-labels is true.")
148148
f.IntVar(&l.MaxGlobalMetadataPerMetric, "ingester.max-global-metadata-per-metric", 0, "The maximum number of metadata per metric, across the cluster. 0 to disable.")
149149
f.IntVar(&l.MaxChunksPerQueryFromStore, "store.query-chunk-limit", 2e6, "Deprecated. Use -querier.max-fetched-chunks-per-query CLI flag and its respective YAML config option instead. Maximum number of chunks that can be fetched in a single query. This limit is enforced when fetching chunks from the long-term storage only. When running the Cortex chunks storage, this limit is enforced in the querier and ruler, while when running the Cortex blocks storage this limit is enforced in the querier, ruler and store-gateway. 0 to disable.")
150-
f.IntVar(&l.MaxChunksPerQuery, "querier.max-fetched-chunks-per-query", 0, "Maximum number of chunks that can be fetched in a single query from ingesters and long-term storage: the total number of actual fetched chunks could be 2x the limit, being independently applied when querying ingesters and long-term storage. This limit is enforced in the ingester (if chunks streaming is enabled), querier, ruler and store-gateway. Takes precedence over the deprecated -store.query-chunk-limit. 0 to disable.")
150+
f.IntVar(&l.MaxChunksPerQuery, "querier.max-fetched-chunks-per-query", 0, "Maximum number of chunks that can be fetched in a single query from ingesters and long-term storage. This limit is enforced in the ingester (if chunks streaming is enabled), querier, ruler and store-gateway. Takes precedence over the deprecated -store.query-chunk-limit. 0 to disable.")
151151
f.IntVar(&l.MaxFetchedSeriesPerQuery, "querier.max-fetched-series-per-query", 0, "The maximum number of unique series for which a query can fetch samples from each ingesters and blocks storage. This limit is enforced in the querier only when running Cortex with blocks storage. 0 to disable")
152152
f.IntVar(&l.MaxFetchedChunkBytesPerQuery, "querier.max-fetched-chunk-bytes-per-query", 0, "The maximum size of all chunks in bytes that a query can fetch from each ingester and storage. This limit is enforced in the querier and ruler only when running Cortex with blocks storage. 0 to disable.")
153153
f.Var(&l.MaxQueryLength, "store.max-query-length", "Limit the query time range (end - start time). This limit is enforced in the query-frontend (on the received query), in the querier (on the query possibly split by the query-frontend) and in the chunks storage. 0 to disable.")
@@ -388,12 +388,6 @@ func (o *Overrides) MaxChunksPerQueryFromStore(userID string) int {
388388
return o.getOverridesForUser(userID).MaxChunksPerQueryFromStore
389389
}
390390

391-
// MaxChunksPerQueryFromIngesters returns the maximum number of chunks allowed per query when fetching
392-
// chunks from ingesters.
393-
func (o *Overrides) MaxChunksPerQueryFromIngesters(userID string) int {
394-
return o.getOverridesForUser(userID).MaxChunksPerQuery
395-
}
396-
397391
// MaxFetchedSeriesPerQuery returns the maximum number of series allowed per query when fetching
398392
// chunks from ingesters and blocks storage.
399393
func (o *Overrides) MaxFetchedSeriesPerQuery(userID string) int {

0 commit comments

Comments
 (0)