Skip to content

Commit 2b41aa3

Browse files
authored
Added -store.max-chunks-per-query limit support to blocks storage (#2852)
* Removed max-sample-count support and added -store.max-chunks-per-query limit support to blocks storage Signed-off-by: Marco Pracucci <[email protected]> * Added PR number to CHANGELOG Signed-off-by: Marco Pracucci <[email protected]> * Use atomic.LoadInt32() when reading back numChunks Signed-off-by: Marco Pracucci <[email protected]> * Use Uber atomic (safer and cleaner) instead of sync/atomic Signed-off-by: Marco Pracucci <[email protected]>
1 parent 6267964 commit 2b41aa3

File tree

9 files changed

+177
-32
lines changed

9 files changed

+177
-32
lines changed

CHANGELOG.md

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -10,6 +10,7 @@
1010
* [CHANGE] Metric `cortex_ingester_flush_reasons` has been renamed to `cortex_ingester_series_flushed_total`, and is now incremented during flush, not when series is enqueued for flushing. #2802
1111
* [CHANGE] Experimental Delete Series: Metric `cortex_purger_oldest_pending_delete_request_age_seconds` would track age of delete requests since they are over their cancellation period instead of their creation time. #2806
1212
* [CHANGE] Experimental TSDB: the store-gateway service is required in a Cortex cluster running with the experimental blocks storage. Removed the `-experimental.tsdb.store-gateway-enabled` CLI flag and `store_gateway_enabled` YAML config option. The store-gateway is now always enabled when the storage engine is `tsdb`. #2822
13+
* [CHANGE] Experimental TSDB: removed support for `-experimental.tsdb.bucket-store.max-sample-count` flag because the implementation was flawed. To limit the number of samples/chunks processed by a single query you can set `-store.query-chunk-limit`, which is now supported by the blocks storage too. #2852
1314
* [CHANGE] Ingester: Chunks flushed via /flush stay in memory until retention period is reached. This affects `cortex_ingester_memory_chunks` metric. #2778
1415
* [CHANGE] Querier: the error message returned when the query time range exceeds `-store.max-query-length` has changed from `invalid query, length > limit (X > Y)` to `the query time range exceeds the limit (query length: X, limit: Y)`. #2826
1516
* [CHANGE] KV: The `role` label which was a label of `multi` KV store client only has been added to metrics of every KV store client. If KV store client is not `multi`, then the value of `role` label is `primary`. #2837
@@ -38,6 +39,7 @@
3839
* [ENHANCEMENT] Experimental TSDB: Added `-experimental.tsdb.head-compaction-idle-timeout` option to force compaction of data in memory into a block. #2803
3940
* [ENHANCEMENT] Experimental TSDB: Added support for flushing blocks via `/flush`, `/shutdown` (previously these only worked for chunks storage) and by using `-experimental.tsdb.flush-blocks-on-shutdown` option. #2794
4041
* [ENHANCEMENT] Experimental TSDB: Added support to enforce max query time range length via `-store.max-query-length`. #2826
42+
* [ENHANCEMENT] Experimental TSDB: Added support to limit the max number of chunks that can be fetched from the long-term storage while executing a query. The limit is configurable via `-store.query-chunk-limit`. #2852
4143
* [ENHANCEMENT] Ingester: Added new metric `cortex_ingester_flush_series_in_progress` that reports number of ongoing flush-series operations. Useful when calling `/flush` handler: if `cortex_ingester_flush_queue_length + cortex_ingester_flush_series_in_progress` is 0, all flushes are finished. #2778
4244
* [ENHANCEMENT] Memberlist members can join cluster via SRV records. #2788
4345
* [ENHANCEMENT] Added configuration options for chunks s3 client. #2831

docs/configuration/config-file-reference.md

Lines changed: 1 addition & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -2669,7 +2669,7 @@ The `limits_config` configures default and per-tenant limits imposed by Cortex s
26692669
[max_global_metadata_per_metric: <int> | default = 0]
26702670
26712671
# Maximum number of chunks that can be fetched in a single query. This limit is
2672-
# ignored when running the Cortex blocks storage.
2672+
# enforced when fetching chunks from the long-term storage.
26732673
# CLI flag: -store.query-chunk-limit
26742674
[max_chunks_per_query: <int> | default = 2000000]
26752675
@@ -2968,11 +2968,6 @@ bucket_store:
29682968
# CLI flag: -experimental.tsdb.bucket-store.max-chunk-pool-bytes
29692969
[max_chunk_pool_bytes: <int> | default = 2147483648]
29702970
2971-
# Max number of samples per query when loading series from the long-term
2972-
# storage. 0 disables the limit.
2973-
# CLI flag: -experimental.tsdb.bucket-store.max-sample-count
2974-
[max_sample_count: <int> | default = 0]
2975-
29762971
# Max number of concurrent queries to execute against the long-term storage.
29772972
# The limit is shared across all tenants.
29782973
# CLI flag: -experimental.tsdb.bucket-store.max-concurrent

docs/operations/blocks-storage.md

Lines changed: 0 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -214,11 +214,6 @@ tsdb:
214214
# CLI flag: -experimental.tsdb.bucket-store.max-chunk-pool-bytes
215215
[max_chunk_pool_bytes: <int> | default = 2147483648]
216216
217-
# Max number of samples per query when loading series from the long-term
218-
# storage. 0 disables the limit.
219-
# CLI flag: -experimental.tsdb.bucket-store.max-sample-count
220-
[max_sample_count: <int> | default = 0]
221-
222217
# Max number of concurrent queries to execute against the long-term storage.
223218
# The limit is shared across all tenants.
224219
# CLI flag: -experimental.tsdb.bucket-store.max-concurrent

pkg/cortex/modules.go

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -205,7 +205,7 @@ func (t *Cortex) initStoreQueryables() (services.Service, error) {
205205
var servs []services.Service
206206

207207
//nolint:golint // I prefer this form over removing 'else', because it allows q to have smaller scope.
208-
if q, err := initQueryableForEngine(t.Cfg.Storage.Engine, t.Cfg, t.Store, prometheus.DefaultRegisterer); err != nil {
208+
if q, err := initQueryableForEngine(t.Cfg.Storage.Engine, t.Cfg, t.Store, t.Overrides, prometheus.DefaultRegisterer); err != nil {
209209
return nil, fmt.Errorf("failed to initialize querier for engine '%s': %v", t.Cfg.Storage.Engine, err)
210210
} else {
211211
t.StoreQueryables = append(t.StoreQueryables, querier.UseAlwaysQueryable(q))
@@ -219,7 +219,7 @@ func (t *Cortex) initStoreQueryables() (services.Service, error) {
219219
return nil, fmt.Errorf("second store engine used by querier '%s' must be different than primary engine '%s'", t.Cfg.Querier.SecondStoreEngine, t.Cfg.Storage.Engine)
220220
}
221221

222-
sq, err := initQueryableForEngine(t.Cfg.Querier.SecondStoreEngine, t.Cfg, t.Store, prometheus.DefaultRegisterer)
222+
sq, err := initQueryableForEngine(t.Cfg.Querier.SecondStoreEngine, t.Cfg, t.Store, t.Overrides, prometheus.DefaultRegisterer)
223223
if err != nil {
224224
return nil, fmt.Errorf("failed to initialize querier for engine '%s': %v", t.Cfg.Querier.SecondStoreEngine, err)
225225
}
@@ -245,7 +245,7 @@ func (t *Cortex) initStoreQueryables() (services.Service, error) {
245245
}
246246
}
247247

248-
func initQueryableForEngine(engine string, cfg Config, chunkStore chunk.Store, reg prometheus.Registerer) (prom_storage.Queryable, error) {
248+
func initQueryableForEngine(engine string, cfg Config, chunkStore chunk.Store, limits *validation.Overrides, reg prometheus.Registerer) (prom_storage.Queryable, error) {
249249
switch engine {
250250
case storage.StorageEngineChunks:
251251
if chunkStore == nil {
@@ -260,7 +260,7 @@ func initQueryableForEngine(engine string, cfg Config, chunkStore chunk.Store, r
260260
cfg.Querier.StoreGatewayAddresses = fmt.Sprintf("127.0.0.1:%d", cfg.Server.GRPCListenPort)
261261
}
262262

263-
return querier.NewBlocksStoreQueryableFromConfig(cfg.Querier, cfg.StoreGateway, cfg.TSDB, util.Logger, reg)
263+
return querier.NewBlocksStoreQueryableFromConfig(cfg.Querier, cfg.StoreGateway, cfg.TSDB, limits, util.Logger, reg)
264264

265265
default:
266266
return nil, fmt.Errorf("unknown storage engine '%s'", engine)
@@ -588,7 +588,7 @@ func (t *Cortex) setupModuleManager() error {
588588
Ingester: {Overrides, Store, API, RuntimeConfig, MemberlistKV},
589589
Flusher: {Store, API},
590590
Querier: {Overrides, Distributor, Store, Ring, API, StoreQueryable},
591-
StoreQueryable: {Store},
591+
StoreQueryable: {Overrides, Store},
592592
QueryFrontend: {API, Overrides, DeleteRequestsStore},
593593
TableManager: {API},
594594
Ruler: {Overrides, Distributor, Store, StoreQueryable},

pkg/querier/blocks_store_queryable.go

Lines changed: 64 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@ import (
2424
"github.com/thanos-io/thanos/pkg/store/hintspb"
2525
"github.com/thanos-io/thanos/pkg/store/storepb"
2626
"github.com/weaveworks/common/user"
27+
"go.uber.org/atomic"
2728
"golang.org/x/sync/errgroup"
2829
grpc_metadata "google.golang.org/grpc/metadata"
2930

@@ -46,7 +47,8 @@ const (
4647
)
4748

4849
var (
49-
errNoStoreGatewayAddress = errors.New("no store-gateway address configured")
50+
errNoStoreGatewayAddress = errors.New("no store-gateway address configured")
51+
errMaxChunksPerQueryLimit = "the query hit the max number of chunks limit while fetching chunks for %s (limit: %d)"
5052
)
5153

5254
// BlocksStoreSet is the interface used to get the clients to query series on a set of blocks.
@@ -78,6 +80,11 @@ type BlocksStoreClient interface {
7880
RemoteAddress() string
7981
}
8082

83+
// BlocksStoreLimits is the interface that should be implemented by the limits provider.
84+
type BlocksStoreLimits interface {
85+
MaxChunksPerQuery(userID string) int
86+
}
87+
8188
type blocksStoreQueryableMetrics struct {
8289
storesHit prometheus.Histogram
8390
refetches prometheus.Histogram
@@ -111,13 +118,14 @@ type BlocksStoreQueryable struct {
111118
logger log.Logger
112119
queryStoreAfter time.Duration
113120
metrics *blocksStoreQueryableMetrics
121+
limits BlocksStoreLimits
114122

115123
// Subservices manager.
116124
subservices *services.Manager
117125
subservicesWatcher *services.FailureWatcher
118126
}
119127

120-
func NewBlocksStoreQueryable(stores BlocksStoreSet, finder BlocksFinder, consistency *BlocksConsistencyChecker, queryStoreAfter time.Duration, logger log.Logger, reg prometheus.Registerer) (*BlocksStoreQueryable, error) {
128+
func NewBlocksStoreQueryable(stores BlocksStoreSet, finder BlocksFinder, consistency *BlocksConsistencyChecker, limits BlocksStoreLimits, queryStoreAfter time.Duration, logger log.Logger, reg prometheus.Registerer) (*BlocksStoreQueryable, error) {
121129
util.WarnExperimentalUse("Blocks storage engine")
122130

123131
manager, err := services.NewManager(stores, finder)
@@ -134,14 +142,15 @@ func NewBlocksStoreQueryable(stores BlocksStoreSet, finder BlocksFinder, consist
134142
subservices: manager,
135143
subservicesWatcher: services.NewFailureWatcher(),
136144
metrics: newBlocksStoreQueryableMetrics(reg),
145+
limits: limits,
137146
}
138147

139148
q.Service = services.NewBasicService(q.starting, q.running, q.stopping)
140149

141150
return q, nil
142151
}
143152

144-
func NewBlocksStoreQueryableFromConfig(querierCfg Config, gatewayCfg storegateway.Config, storageCfg cortex_tsdb.Config, logger log.Logger, reg prometheus.Registerer) (*BlocksStoreQueryable, error) {
153+
func NewBlocksStoreQueryableFromConfig(querierCfg Config, gatewayCfg storegateway.Config, storageCfg cortex_tsdb.Config, limits BlocksStoreLimits, logger log.Logger, reg prometheus.Registerer) (*BlocksStoreQueryable, error) {
145154
var stores BlocksStoreSet
146155

147156
bucketClient, err := cortex_tsdb.NewBucketClient(context.Background(), storageCfg, "querier", logger, reg)
@@ -209,7 +218,7 @@ func NewBlocksStoreQueryableFromConfig(querierCfg Config, gatewayCfg storegatewa
209218
reg,
210219
)
211220

212-
return NewBlocksStoreQueryable(stores, scanner, consistency, querierCfg.QueryStoreAfter, logger, reg)
221+
return NewBlocksStoreQueryable(stores, scanner, consistency, limits, querierCfg.QueryStoreAfter, logger, reg)
213222
}
214223

215224
func (q *BlocksStoreQueryable) starting(ctx context.Context) error {
@@ -256,6 +265,7 @@ func (q *BlocksStoreQueryable) Querier(ctx context.Context, mint, maxt int64) (s
256265
finder: q.finder,
257266
stores: q.stores,
258267
metrics: q.metrics,
268+
limits: q.limits,
259269
consistency: q.consistency,
260270
logger: q.logger,
261271
queryStoreAfter: q.queryStoreAfter,
@@ -270,6 +280,7 @@ type blocksStoreQuerier struct {
270280
stores BlocksStoreSet
271281
metrics *blocksStoreQueryableMetrics
272282
consistency *BlocksConsistencyChecker
283+
limits BlocksStoreLimits
273284
logger log.Logger
274285

275286
// If set, the querier manipulates the max time to not be greater than
@@ -357,6 +368,9 @@ func (q *blocksStoreQuerier) selectSorted(sp *storage.SelectHints, matchers ...*
357368
resSeriesSets = []storage.SeriesSet(nil)
358369
resWarnings = storage.Warnings(nil)
359370
resQueriedBlocks = []ulid.ULID(nil)
371+
372+
maxChunksLimit = q.limits.MaxChunksPerQuery(q.userID)
373+
leftChunksLimit = maxChunksLimit
360374
)
361375

362376
for attempt := 1; attempt <= maxFetchSeriesAttempts; attempt++ {
@@ -377,7 +391,7 @@ func (q *blocksStoreQuerier) selectSorted(sp *storage.SelectHints, matchers ...*
377391

378392
// Fetch series from stores. If an error occur we do not retry because retries
379393
// are only meant to cover missing blocks.
380-
seriesSets, queriedBlocks, warnings, err := q.fetchSeriesFromStores(spanCtx, clients, minT, maxT, convertedMatchers)
394+
seriesSets, queriedBlocks, warnings, numChunks, err := q.fetchSeriesFromStores(spanCtx, clients, minT, maxT, matchers, convertedMatchers, maxChunksLimit, leftChunksLimit)
381395
if err != nil {
382396
return storage.ErrSeriesSet(err)
383397
}
@@ -387,6 +401,12 @@ func (q *blocksStoreQuerier) selectSorted(sp *storage.SelectHints, matchers ...*
387401
resWarnings = append(resWarnings, warnings...)
388402
resQueriedBlocks = append(resQueriedBlocks, queriedBlocks...)
389403

404+
// Given a single block is guaranteed to not be queried twice, we can safely decrease the number of
405+
// chunks we can still read before hitting the limit (max == 0 means disabled).
406+
if maxChunksLimit > 0 {
407+
leftChunksLimit -= numChunks
408+
}
409+
390410
// Update the map of blocks we attempted to query.
391411
for client, blockIDs := range clients {
392412
touchedStores[client.RemoteAddress()] = struct{}{}
@@ -425,15 +445,19 @@ func (q *blocksStoreQuerier) fetchSeriesFromStores(
425445
clients map[BlocksStoreClient][]ulid.ULID,
426446
minT int64,
427447
maxT int64,
428-
matchers []storepb.LabelMatcher,
429-
) ([]storage.SeriesSet, []ulid.ULID, storage.Warnings, error) {
448+
matchers []*labels.Matcher,
449+
convertedMatchers []storepb.LabelMatcher,
450+
maxChunksLimit int,
451+
leftChunksLimit int,
452+
) ([]storage.SeriesSet, []ulid.ULID, storage.Warnings, int, error) {
430453
var (
431454
reqCtx = grpc_metadata.AppendToOutgoingContext(ctx, cortex_tsdb.TenantIDExternalLabel, q.userID)
432455
g, gCtx = errgroup.WithContext(reqCtx)
433456
mtx = sync.Mutex{}
434457
seriesSets = []storage.SeriesSet(nil)
435458
warnings = storage.Warnings(nil)
436459
queriedBlocks = []ulid.ULID(nil)
460+
numChunks = atomic.NewInt32(0)
437461
spanLog = spanlogger.FromContext(ctx)
438462
)
439463

@@ -444,7 +468,7 @@ func (q *blocksStoreQuerier) fetchSeriesFromStores(
444468
blockIDs := blockIDs
445469

446470
g.Go(func() error {
447-
req, err := createSeriesRequest(minT, maxT, matchers, blockIDs)
471+
req, err := createSeriesRequest(minT, maxT, convertedMatchers, blockIDs)
448472
if err != nil {
449473
return errors.Wrapf(err, "failed to create series request")
450474
}
@@ -459,6 +483,12 @@ func (q *blocksStoreQuerier) fetchSeriesFromStores(
459483
myQueriedBlocks := []ulid.ULID(nil)
460484

461485
for {
486+
// Ensure the context hasn't been canceled in the meanwhile (eg. an error occurred
487+
// in another goroutine).
488+
if gCtx.Err() != nil {
489+
return gCtx.Err()
490+
}
491+
462492
resp, err := stream.Recv()
463493
if err == io.EOF {
464494
break
@@ -470,6 +500,14 @@ func (q *blocksStoreQuerier) fetchSeriesFromStores(
470500
// Response may either contain series, warning or hints.
471501
if s := resp.GetSeries(); s != nil {
472502
mySeries = append(mySeries, s)
503+
504+
// Ensure the max number of chunks limit hasn't been reached (max == 0 means disabled).
505+
if maxChunksLimit > 0 {
506+
actual := numChunks.Add(int32(len(s.Chunks)))
507+
if actual > int32(leftChunksLimit) {
508+
return fmt.Errorf(errMaxChunksPerQueryLimit, convertMatchersToString(matchers), maxChunksLimit)
509+
}
510+
}
473511
}
474512

475513
if w := resp.GetWarning(); w != "" {
@@ -511,10 +549,10 @@ func (q *blocksStoreQuerier) fetchSeriesFromStores(
511549

512550
// Wait until all client requests complete.
513551
if err := g.Wait(); err != nil {
514-
return nil, nil, nil, err
552+
return nil, nil, nil, 0, err
515553
}
516554

517-
return seriesSets, queriedBlocks, warnings, nil
555+
return seriesSets, queriedBlocks, warnings, int(numChunks.Load()), nil
518556
}
519557

520558
func createSeriesRequest(minT, maxT int64, matchers []storepb.LabelMatcher, blockIDs []ulid.ULID) (*storepb.SeriesRequest, error) {
@@ -577,3 +615,19 @@ func countSeriesBytes(series []*storepb.Series) (count uint64) {
577615

578616
return count
579617
}
618+
619+
func convertMatchersToString(matchers []*labels.Matcher) string {
620+
out := strings.Builder{}
621+
out.WriteRune('{')
622+
623+
for idx, m := range matchers {
624+
if idx > 0 {
625+
out.WriteRune(',')
626+
}
627+
628+
out.WriteString(m.String())
629+
}
630+
631+
out.WriteRune('}')
632+
return out.String()
633+
}

0 commit comments

Comments
 (0)