From 80b2d5ccd88af1071a3d2fd45b8da8603450bcc6 Mon Sep 17 00:00:00 2001 From: Justin Jung Date: Tue, 25 Mar 2025 13:10:22 -0700 Subject: [PATCH 01/29] Add resource based throttling to ingesters and store gateways Signed-off-by: Justin Jung --- pkg/configs/resources.go | 26 +++ pkg/cortex/cortex.go | 8 + pkg/cortex/cortex_test.go | 12 +- pkg/cortex/modules.go | 22 +- pkg/frontend/transport/handler.go | 12 +- pkg/frontend/transport/handler_test.go | 18 ++ pkg/ingester/ingester.go | 13 +- pkg/ingester/ingester_test.go | 82 +++++++- pkg/querier/blocks_store_queryable.go | 7 + pkg/querier/blocks_store_queryable_test.go | 30 +++ pkg/storegateway/gateway.go | 32 ++- pkg/storegateway/gateway_test.go | 87 +++++++- pkg/util/resource/monitor.go | 232 +++++++++++++++++++++ pkg/util/resource/monitor_test.go | 1 + 14 files changed, 557 insertions(+), 25 deletions(-) create mode 100644 pkg/configs/resources.go create mode 100644 pkg/util/resource/monitor.go create mode 100644 pkg/util/resource/monitor_test.go diff --git a/pkg/configs/resources.go b/pkg/configs/resources.go new file mode 100644 index 00000000000..f36bd952080 --- /dev/null +++ b/pkg/configs/resources.go @@ -0,0 +1,26 @@ +package configs + +import ( + "errors" + "flag" +) + +var ErrInvalidResourceThreshold = errors.New("invalid resource utilization threshold, it must be between 0 and 1") + +type Resources struct { + CPU float64 `yaml:"cpu"` + Heap float64 `yaml:"heap"` +} + +func (cfg *Resources) RegisterFlags(f *flag.FlagSet) { + f.Float64Var(&cfg.CPU, "resource-thresholds.cpu", 0, "Utilization threshold for CPU, between 0 and 1. 0 to disable.") + f.Float64Var(&cfg.Heap, "resource-thresholds.heap", 0, "Utilization threshold for heap, between 0 and 1. 0 to disable.") +} + +func (cfg *Resources) Validate() error { + if cfg.CPU > 1 || cfg.CPU < 0 || cfg.Heap > 1 || cfg.Heap < 0 { + return ErrInvalidResourceThreshold + } + + return nil +} diff --git a/pkg/cortex/cortex.go b/pkg/cortex/cortex.go index 00b67879f06..aba1caa23f6 100644 --- a/pkg/cortex/cortex.go +++ b/pkg/cortex/cortex.go @@ -22,6 +22,7 @@ import ( "gopkg.in/yaml.v2" "github.com/cortexproject/cortex/pkg/util/grpcclient" + "github.com/cortexproject/cortex/pkg/util/resource" "github.com/cortexproject/cortex/pkg/alertmanager" "github.com/cortexproject/cortex/pkg/alertmanager/alertstore" @@ -122,6 +123,7 @@ type Config struct { RuntimeConfig runtimeconfig.Config `yaml:"runtime_config"` MemberlistKV memberlist.KVConfig `yaml:"memberlist"` QueryScheduler scheduler.Config `yaml:"query_scheduler"` + ResourceThresholds configs.Resources `yaml:"resource_thresholds"` Tracing tracing.Config `yaml:"tracing"` } @@ -170,6 +172,7 @@ func (c *Config) RegisterFlags(f *flag.FlagSet) { c.MemberlistKV.RegisterFlags(f) c.QueryScheduler.RegisterFlags(f) c.Tracing.RegisterFlags(f) + c.ResourceThresholds.RegisterFlags(f) } // Validate the cortex config and returns an error if the validation @@ -237,6 +240,10 @@ func (c *Config) Validate(log log.Logger) error { return errors.Wrap(err, "invalid tracing config") } + if err := c.ResourceThresholds.Validate(); err != nil { + return errors.Wrap(err, "invalid resource_thresholds config") + } + return nil } @@ -315,6 +322,7 @@ type Cortex struct { MetadataQuerier querier.MetadataQuerier QuerierEngine promql.QueryEngine QueryFrontendTripperware tripperware.Tripperware + ResourceMonitor *resource.Monitor Ruler *ruler.Ruler RulerStorage rulestore.RuleStore diff --git a/pkg/cortex/cortex_test.go b/pkg/cortex/cortex_test.go index bac7c0021c6..8fe169050d5 100644 --- a/pkg/cortex/cortex_test.go +++ b/pkg/cortex/cortex_test.go @@ -22,6 +22,7 @@ import ( "github.com/cortexproject/cortex/pkg/alertmanager" "github.com/cortexproject/cortex/pkg/alertmanager/alertstore" + "github.com/cortexproject/cortex/pkg/configs" "github.com/cortexproject/cortex/pkg/cortex/storage" "github.com/cortexproject/cortex/pkg/frontend/v1/frontendv1pb" "github.com/cortexproject/cortex/pkg/ingester" @@ -165,11 +166,20 @@ func TestConfigValidation(t *testing.T) { }, expectedError: errInvalidHTTPPrefix, }, + { + name: "should fail validation for resource thresholds", + getTestConfig: func() *Config { + configuration := newDefaultConfig() + configuration.ResourceThresholds.CPU = 10 + return configuration + }, + expectedError: configs.ErrInvalidResourceThreshold, + }, } { t.Run(tc.name, func(t *testing.T) { err := tc.getTestConfig().Validate(nil) if tc.expectedError != nil { - require.Equal(t, tc.expectedError, err) + require.ErrorContains(t, err, tc.expectedError.Error()) } else { require.NoError(t, err) } diff --git a/pkg/cortex/modules.go b/pkg/cortex/modules.go index 091297732d7..8d8d6d920c5 100644 --- a/pkg/cortex/modules.go +++ b/pkg/cortex/modules.go @@ -51,6 +51,7 @@ import ( "github.com/cortexproject/cortex/pkg/util/grpcclient" util_log "github.com/cortexproject/cortex/pkg/util/log" "github.com/cortexproject/cortex/pkg/util/modules" + "github.com/cortexproject/cortex/pkg/util/resource" "github.com/cortexproject/cortex/pkg/util/runtimeconfig" "github.com/cortexproject/cortex/pkg/util/services" "github.com/cortexproject/cortex/pkg/util/validation" @@ -86,6 +87,7 @@ const ( Purger string = "purger" QueryScheduler string = "query-scheduler" TenantFederation string = "tenant-federation" + ResourceMonitor string = "resource-monitor" All string = "all" ) @@ -441,7 +443,7 @@ func (t *Cortex) initIngesterService() (serv services.Service, err error) { t.Cfg.Ingester.QueryIngestersWithin = t.Cfg.Querier.QueryIngestersWithin t.tsdbIngesterConfig() - t.Ingester, err = ingester.New(t.Cfg.Ingester, t.Overrides, prometheus.DefaultRegisterer, util_log.Logger) + t.Ingester, err = ingester.New(t.Cfg.Ingester, t.Overrides, prometheus.DefaultRegisterer, util_log.Logger, t.ResourceMonitor) if err != nil { return } @@ -705,7 +707,7 @@ func (t *Cortex) initCompactor() (serv services.Service, err error) { func (t *Cortex) initStoreGateway() (serv services.Service, err error) { t.Cfg.StoreGateway.ShardingRing.ListenPort = t.Cfg.Server.GRPCListenPort - t.StoreGateway, err = storegateway.NewStoreGateway(t.Cfg.StoreGateway, t.Cfg.BlocksStorage, t.Overrides, t.Cfg.Server.LogLevel, util_log.Logger, prometheus.DefaultRegisterer) + t.StoreGateway, err = storegateway.NewStoreGateway(t.Cfg.StoreGateway, t.Cfg.BlocksStorage, t.Overrides, t.Cfg.Server.LogLevel, util_log.Logger, prometheus.DefaultRegisterer, t.ResourceMonitor) if err != nil { return nil, err } @@ -765,11 +767,23 @@ func (t *Cortex) initQueryScheduler() (services.Service, error) { return s, nil } +func (t *Cortex) initResourceMonitor() (services.Service, error) { + if t.Cfg.ResourceThresholds.CPU <= 0 && t.Cfg.ResourceThresholds.Heap <= 0 { + return nil, nil + } + + var err error + t.ResourceMonitor, err = resource.NewMonitor(t.Cfg.ResourceThresholds, prometheus.DefaultRegisterer) + + return t.ResourceMonitor, err +} + func (t *Cortex) setupModuleManager() error { mm := modules.NewManager(util_log.Logger) // Register all modules here. // RegisterModule(name string, initFn func()(services.Service, error)) + mm.RegisterModule(ResourceMonitor, t.initResourceMonitor) mm.RegisterModule(Server, t.initServer, modules.UserInvisibleModule) mm.RegisterModule(API, t.initAPI, modules.UserInvisibleModule) mm.RegisterModule(RuntimeConfig, t.initRuntimeConfig, modules.UserInvisibleModule) @@ -811,7 +825,7 @@ func (t *Cortex) setupModuleManager() error { Distributor: {DistributorService, API, GrpcClientService}, DistributorService: {Ring, Overrides}, Ingester: {IngesterService, Overrides, API}, - IngesterService: {Overrides, RuntimeConfig, MemberlistKV}, + IngesterService: {Overrides, RuntimeConfig, MemberlistKV, ResourceMonitor}, Flusher: {Overrides, API}, Queryable: {Overrides, DistributorService, Overrides, Ring, API, StoreQueryable, MemberlistKV}, Querier: {TenantFederation}, @@ -824,7 +838,7 @@ func (t *Cortex) setupModuleManager() error { Configs: {API}, AlertManager: {API, MemberlistKV, Overrides}, Compactor: {API, MemberlistKV, Overrides}, - StoreGateway: {API, Overrides, MemberlistKV}, + StoreGateway: {API, Overrides, MemberlistKV, ResourceMonitor}, TenantDeletion: {API, Overrides}, Purger: {TenantDeletion}, TenantFederation: {Queryable}, diff --git a/pkg/frontend/transport/handler.go b/pkg/frontend/transport/handler.go index 718260beefb..e73419ac1a3 100644 --- a/pkg/frontend/transport/handler.go +++ b/pkg/frontend/transport/handler.go @@ -28,6 +28,7 @@ import ( "github.com/cortexproject/cortex/pkg/util" util_api "github.com/cortexproject/cortex/pkg/util/api" util_log "github.com/cortexproject/cortex/pkg/util/log" + "github.com/cortexproject/cortex/pkg/util/resource" ) const ( @@ -49,6 +50,7 @@ const ( reasonRequestBodySizeExceeded = "request_body_size_exceeded" reasonResponseBodySizeExceeded = "response_body_size_exceeded" reasonTooManyRequests = "too_many_requests" + reasonResourceExhausted = "resource_exhausted" reasonTimeRangeExceeded = "time_range_exceeded" reasonTooManySamples = "too_many_samples" reasonSeriesFetched = "series_fetched" @@ -488,13 +490,19 @@ func (f *Handler) reportQueryStats(r *http.Request, source, userID string, query } } + // We are unable to use errors.As to compare since body string from the http response is wrapped as an error var reason string + errMsg := error.Error() if statusCode == http.StatusTooManyRequests { - reason = reasonTooManyRequests + var resourceExhaustedErr resource.ExhaustedError + if strings.Contains(errMsg, resourceExhaustedErr.Error()) { + reason = reasonResourceExhausted + } else { + reason = reasonTooManyRequests + } } else if statusCode == http.StatusRequestEntityTooLarge { reason = reasonResponseBodySizeExceeded } else if statusCode == http.StatusUnprocessableEntity { - errMsg := error.Error() if strings.Contains(errMsg, limitTooManySamples) { reason = reasonTooManySamples } else if strings.Contains(errMsg, limitTimeRangeExceeded) { diff --git a/pkg/frontend/transport/handler_test.go b/pkg/frontend/transport/handler_test.go index a7b91ad2407..ee23fc078b0 100644 --- a/pkg/frontend/transport/handler_test.go +++ b/pkg/frontend/transport/handler_test.go @@ -30,6 +30,7 @@ import ( "github.com/cortexproject/cortex/pkg/tenant" util_api "github.com/cortexproject/cortex/pkg/util/api" util_log "github.com/cortexproject/cortex/pkg/util/log" + "github.com/cortexproject/cortex/pkg/util/resource" ) type roundTripperFunc func(*http.Request) (*http.Response, error) @@ -380,6 +381,23 @@ func TestHandler_ServeHTTP(t *testing.T) { }, expectedStatusCode: http.StatusUnprocessableEntity, }, + { + name: "test handler with reasonResourceExhausted", + cfg: HandlerConfig{QueryStatsEnabled: true}, + expectedMetrics: 6, + roundTripperFunc: roundTripperFunc(func(req *http.Request) (*http.Response, error) { + resourceExhaustedErr := &resource.ExhaustedError{} + return &http.Response{ + StatusCode: http.StatusTooManyRequests, + Body: io.NopCloser(strings.NewReader(resourceExhaustedErr.Error())), + }, nil + }), + additionalMetricsCheckFunc: func(h *Handler) { + v := promtest.ToFloat64(h.rejectedQueries.WithLabelValues(reasonResourceExhausted, tripperware.SourceAPI, userID)) + assert.Equal(t, float64(1), v) + }, + expectedStatusCode: http.StatusTooManyRequests, + }, } { t.Run(tt.name, func(t *testing.T) { reg := prometheus.NewPedanticRegistry() diff --git a/pkg/ingester/ingester.go b/pkg/ingester/ingester.go index ea8ca3c6f12..c23199c2f28 100644 --- a/pkg/ingester/ingester.go +++ b/pkg/ingester/ingester.go @@ -58,6 +58,7 @@ import ( "github.com/cortexproject/cortex/pkg/util/extract" logutil "github.com/cortexproject/cortex/pkg/util/log" util_math "github.com/cortexproject/cortex/pkg/util/math" + "github.com/cortexproject/cortex/pkg/util/resource" "github.com/cortexproject/cortex/pkg/util/services" "github.com/cortexproject/cortex/pkg/util/spanlogger" "github.com/cortexproject/cortex/pkg/util/validation" @@ -231,6 +232,7 @@ type Ingester struct { lifecycler *ring.Lifecycler limits *validation.Overrides limiter *Limiter + resourceMonitor resource.IMonitor subservicesWatcher *services.FailureWatcher stoppedMtx sync.RWMutex // protects stopped @@ -699,7 +701,7 @@ func newTSDBState(bucketClient objstore.Bucket, registerer prometheus.Registerer } // New returns a new Ingester that uses Cortex block storage instead of chunks storage. -func New(cfg Config, limits *validation.Overrides, registerer prometheus.Registerer, logger log.Logger) (*Ingester, error) { +func New(cfg Config, limits *validation.Overrides, registerer prometheus.Registerer, logger log.Logger, resourceMonitor resource.IMonitor) (*Ingester, error) { defaultInstanceLimits = &cfg.DefaultLimits if cfg.ingesterClientFactory == nil { cfg.ingesterClientFactory = client.MakeIngesterClient @@ -719,6 +721,7 @@ func New(cfg Config, limits *validation.Overrides, registerer prometheus.Registe ingestionRate: util_math.NewEWMARate(0.2, instanceIngestionRateTickInterval), expandedPostingsCacheFactory: cortex_tsdb.NewExpandedPostingsCacheFactory(cfg.BlocksStorageConfig.TSDB.PostingsCache), matchersCache: storecache.NoopMatchersCache, + resourceMonitor: resourceMonitor, } if cfg.MatchersCacheMaxItems > 0 { @@ -2152,6 +2155,14 @@ func (i *Ingester) trackInflightQueryRequest() (func(), error) { } i.maxInflightQueryRequests.Track(i.inflightQueryRequests.Inc()) + + if i.resourceMonitor != nil { + if resourceName, threshold, utilization, err := i.resourceMonitor.CheckResourceUtilization(); err != nil { + level.Warn(i.logger).Log("msg", "resource threshold breached", "resource", resourceName, "threshold", threshold, "utilization", utilization) + return nil, errors.Wrapf(err, "failed to query") + } + } + return func() { i.inflightQueryRequests.Dec() }, nil diff --git a/pkg/ingester/ingester_test.go b/pkg/ingester/ingester_test.go index 65587587a69..575b20417d2 100644 --- a/pkg/ingester/ingester_test.go +++ b/pkg/ingester/ingester_test.go @@ -58,6 +58,7 @@ import ( "github.com/cortexproject/cortex/pkg/storage/tsdb/bucketindex" "github.com/cortexproject/cortex/pkg/util" "github.com/cortexproject/cortex/pkg/util/chunkcompat" + "github.com/cortexproject/cortex/pkg/util/resource" "github.com/cortexproject/cortex/pkg/util/services" "github.com/cortexproject/cortex/pkg/util/test" "github.com/cortexproject/cortex/pkg/util/validation" @@ -3121,6 +3122,61 @@ func TestIngester_Query_ShouldNotCreateTSDBIfDoesNotExists(t *testing.T) { assert.False(t, tsdbCreated) } +func Test_Ingester_Query_ResourceThresholdBreached(t *testing.T) { + series := []struct { + lbls labels.Labels + value float64 + timestamp int64 + }{ + {labels.Labels{{Name: labels.MetricName, Value: "test_1"}, {Name: "route", Value: "get_user"}, {Name: "status", Value: "200"}}, 1, 100000}, + } + + mockErr := fmt.Errorf("resource monitor error") + resourceMonitor := testResourceMonitor{ + err: mockErr, + } + i, err := prepareIngesterWithResourceMonitor(t, &resourceMonitor) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) + defer services.StopAndAwaitTerminated(context.Background(), i) //nolint:errcheck + + // Wait until it's ACTIVE + test.Poll(t, 1*time.Second, ring.ACTIVE, func() interface{} { + return i.lifecycler.GetState() + }) + + // Push series + ctx := user.InjectOrgID(context.Background(), "test") + + for _, series := range series { + req, _ := mockWriteRequest(t, series.lbls, series.value, series.timestamp) + _, err := i.Push(ctx, req) + require.NoError(t, err) + } + + rreq := &client.QueryRequest{} + s := &mockQueryStreamServer{ctx: ctx} + err = i.QueryStream(rreq, s) + require.Error(t, err) + require.ErrorContains(t, err, mockErr.Error()) +} + +type testResourceMonitor struct { + err error +} + +func (t *testResourceMonitor) GetCPUUtilization() float64 { + return 0 +} + +func (t *testResourceMonitor) GetHeapUtilization() float64 { + return 0 +} + +func (t *testResourceMonitor) CheckResourceUtilization() (string, float64, float64, error) { + return "", 0, 0, t.err +} + func TestIngester_LabelValues_ShouldNotCreateTSDBIfDoesNotExists(t *testing.T) { i, err := prepareIngesterWithBlocksStorage(t, defaultIngesterTestConfig(t), prometheus.NewRegistry()) require.NoError(t, err) @@ -3996,12 +4052,32 @@ func prepareIngesterWithBlocksStorageAndLimits(t testing.TB, ingesterCfg Config, ingesterCfg.BlocksStorageConfig.Bucket.Filesystem.Directory = bucketDir ingesterCfg.BlocksStorageConfig.TSDB.EnableNativeHistograms = nativeHistograms - ingester, err := New(ingesterCfg, overrides, registerer, log.NewNopLogger()) + ingester, err := New(ingesterCfg, overrides, registerer, log.NewNopLogger(), nil) + if err != nil { + return nil, err + } + + return ingester, nil +} + +func prepareIngesterWithResourceMonitor(t testing.TB, resourceMonitor resource.IMonitor) (*Ingester, error) { + dataDir := t.TempDir() + bucketDir := t.TempDir() + + ingesterCfg := defaultIngesterTestConfig(t) + ingesterCfg.BlocksStorageConfig.TSDB.Dir = dataDir + ingesterCfg.BlocksStorageConfig.Bucket.Backend = "filesystem" + ingesterCfg.BlocksStorageConfig.Bucket.Filesystem.Directory = bucketDir + + overrides, _ := validation.NewOverrides(defaultLimitsTestConfig(), nil) + + ingester, err := New(ingesterCfg, overrides, prometheus.NewRegistry(), log.NewNopLogger(), resourceMonitor) if err != nil { return nil, err } return ingester, nil + } func TestIngester_OpenExistingTSDBOnStartup(t *testing.T) { @@ -4159,7 +4235,7 @@ func TestIngester_OpenExistingTSDBOnStartup(t *testing.T) { // setup the tsdbs dir testData.setup(t, tempDir) - ingester, err := New(ingesterCfg, overrides, prometheus.NewRegistry(), log.NewNopLogger()) + ingester, err := New(ingesterCfg, overrides, prometheus.NewRegistry(), log.NewNopLogger(), nil) require.NoError(t, err) startErr := services.StartAndAwaitRunning(context.Background(), ingester) @@ -5368,7 +5444,7 @@ func TestHeadCompactionOnStartup(t *testing.T) { ingesterCfg.BlocksStorageConfig.Bucket.S3.Endpoint = "localhost" ingesterCfg.BlocksStorageConfig.TSDB.Retention = 2 * 24 * time.Hour // Make sure that no newly created blocks are deleted. - ingester, err := New(ingesterCfg, overrides, prometheus.NewRegistry(), log.NewNopLogger()) + ingester, err := New(ingesterCfg, overrides, prometheus.NewRegistry(), log.NewNopLogger(), nil) require.NoError(t, err) require.NoError(t, services.StartAndAwaitRunning(context.Background(), ingester)) diff --git a/pkg/querier/blocks_store_queryable.go b/pkg/querier/blocks_store_queryable.go index c7953482451..4edeca79058 100644 --- a/pkg/querier/blocks_store_queryable.go +++ b/pkg/querier/blocks_store_queryable.go @@ -51,6 +51,7 @@ import ( "github.com/cortexproject/cortex/pkg/util/limiter" util_log "github.com/cortexproject/cortex/pkg/util/log" "github.com/cortexproject/cortex/pkg/util/multierror" + "github.com/cortexproject/cortex/pkg/util/resource" "github.com/cortexproject/cortex/pkg/util/services" "github.com/cortexproject/cortex/pkg/util/spanlogger" "github.com/cortexproject/cortex/pkg/util/validation" @@ -1203,6 +1204,12 @@ func countSamplesAndChunks(series ...*storepb.Series) (samplesCount, chunksCount // only retry connection issues func isRetryableError(err error) bool { + // retry upon resource exhaustion error from resource monitor + var resourceExhaustedErr *resource.ExhaustedError + if errors.As(err, &resourceExhaustedErr) { + return true + } + switch status.Code(err) { case codes.Unavailable: return true diff --git a/pkg/querier/blocks_store_queryable_test.go b/pkg/querier/blocks_store_queryable_test.go index 162e606d4ff..43c11a9f070 100644 --- a/pkg/querier/blocks_store_queryable_test.go +++ b/pkg/querier/blocks_store_queryable_test.go @@ -44,6 +44,7 @@ import ( "github.com/cortexproject/cortex/pkg/util" "github.com/cortexproject/cortex/pkg/util/limiter" util_log "github.com/cortexproject/cortex/pkg/util/log" + "github.com/cortexproject/cortex/pkg/util/resource" "github.com/cortexproject/cortex/pkg/util/services" "github.com/cortexproject/cortex/pkg/util/validation" ) @@ -1518,6 +1519,35 @@ func TestBlocksStoreQuerier_Select(t *testing.T) { limits: &blocksStoreLimitsMock{}, expectedErr: errors.Wrapf(status.Error(codes.ResourceExhausted, "some other resource"), "failed to fetch series from 1.1.1.1"), }, + "multiple store-gateways has the block, but one of them had resource exhausted error from resource monitor": { + finderResult: bucketindex.Blocks{ + &bucketindex.Block{ID: block1}, + }, + storeSetResponses: []interface{}{ + map[BlocksStoreClient][]ulid.ULID{ + &storeGatewayClientMock{ + remoteAddr: "1.1.1.1", + mockedSeriesErr: &resource.ExhaustedError{}, + }: {block1}, + }, + map[BlocksStoreClient][]ulid.ULID{ + &storeGatewayClientMock{remoteAddr: "2.2.2.2", mockedSeriesResponses: []*storepb.SeriesResponse{ + mockSeriesResponse(labels.Labels{metricNameLabel, series1Label}, []cortexpb.Sample{{Value: 2, TimestampMs: minT}}, nil, nil), + mockHintsResponse(block1), + }}: {block1}, + }, + }, + limits: &blocksStoreLimitsMock{}, + queryLimiter: noOpQueryLimiter, + expectedSeries: []seriesResult{ + { + lbls: labels.New(metricNameLabel, series1Label), + values: []valueResult{ + {t: minT, v: 2}, + }, + }, + }, + }, } for testName, testData := range tests { diff --git a/pkg/storegateway/gateway.go b/pkg/storegateway/gateway.go index c043adee181..f2e4e140923 100644 --- a/pkg/storegateway/gateway.go +++ b/pkg/storegateway/gateway.go @@ -25,6 +25,7 @@ import ( "github.com/cortexproject/cortex/pkg/storegateway/storegatewaypb" "github.com/cortexproject/cortex/pkg/util" "github.com/cortexproject/cortex/pkg/util/flagext" + "github.com/cortexproject/cortex/pkg/util/resource" "github.com/cortexproject/cortex/pkg/util/services" "github.com/cortexproject/cortex/pkg/util/validation" ) @@ -117,10 +118,12 @@ type StoreGateway struct { subservices *services.Manager subservicesWatcher *services.FailureWatcher + resourceMonitor resource.IMonitor + bucketSync *prometheus.CounterVec } -func NewStoreGateway(gatewayCfg Config, storageCfg cortex_tsdb.BlocksStorageConfig, limits *validation.Overrides, logLevel logging.Level, logger log.Logger, reg prometheus.Registerer) (*StoreGateway, error) { +func NewStoreGateway(gatewayCfg Config, storageCfg cortex_tsdb.BlocksStorageConfig, limits *validation.Overrides, logLevel logging.Level, logger log.Logger, reg prometheus.Registerer, resourceMonitor *resource.Monitor) (*StoreGateway, error) { var ringStore kv.Client bucketClient, err := createBucketClient(storageCfg, gatewayCfg.HedgedRequest.GetHedgedRoundTripper(), logger, reg) @@ -140,10 +143,10 @@ func NewStoreGateway(gatewayCfg Config, storageCfg cortex_tsdb.BlocksStorageConf } } - return newStoreGateway(gatewayCfg, storageCfg, bucketClient, ringStore, limits, logLevel, logger, reg) + return newStoreGateway(gatewayCfg, storageCfg, bucketClient, ringStore, limits, logLevel, logger, reg, resourceMonitor) } -func newStoreGateway(gatewayCfg Config, storageCfg cortex_tsdb.BlocksStorageConfig, bucketClient objstore.InstrumentedBucket, ringStore kv.Client, limits *validation.Overrides, logLevel logging.Level, logger log.Logger, reg prometheus.Registerer) (*StoreGateway, error) { +func newStoreGateway(gatewayCfg Config, storageCfg cortex_tsdb.BlocksStorageConfig, bucketClient objstore.InstrumentedBucket, ringStore kv.Client, limits *validation.Overrides, logLevel logging.Level, logger log.Logger, reg prometheus.Registerer, resourceMonitor resource.IMonitor) (*StoreGateway, error) { var err error g := &StoreGateway{ @@ -154,6 +157,7 @@ func newStoreGateway(gatewayCfg Config, storageCfg cortex_tsdb.BlocksStorageConf Name: "cortex_storegateway_bucket_sync_total", Help: "Total number of times the bucket sync operation triggered.", }, []string{"reason"}), + resourceMonitor: resourceMonitor, } allowedTenants := util.NewAllowedTenants(gatewayCfg.EnabledTenants, gatewayCfg.DisabledTenants) @@ -381,19 +385,41 @@ func (g *StoreGateway) syncStores(ctx context.Context, reason string) { } func (g *StoreGateway) Series(req *storepb.SeriesRequest, srv storegatewaypb.StoreGateway_SeriesServer) error { + if err := g.checkResourceUtilization(); err != nil { + return err + } return g.stores.Series(req, srv) } // LabelNames implements the Storegateway proto service. func (g *StoreGateway) LabelNames(ctx context.Context, req *storepb.LabelNamesRequest) (*storepb.LabelNamesResponse, error) { + if err := g.checkResourceUtilization(); err != nil { + return nil, err + } return g.stores.LabelNames(ctx, req) } // LabelValues implements the Storegateway proto service. func (g *StoreGateway) LabelValues(ctx context.Context, req *storepb.LabelValuesRequest) (*storepb.LabelValuesResponse, error) { + if err := g.checkResourceUtilization(); err != nil { + return nil, err + } return g.stores.LabelValues(ctx, req) } +func (g *StoreGateway) checkResourceUtilization() error { + if g.resourceMonitor == nil { + return nil + } + + if resourceName, threshold, utilization, err := g.resourceMonitor.CheckResourceUtilization(); err != nil { + level.Warn(g.logger).Log("msg", "resource threshold breached", "resource", resourceName, "threshold", threshold, "utilization", utilization) + return errors.Wrapf(err, "failed to query") + } + + return nil +} + func (g *StoreGateway) OnRingInstanceRegister(lc *ring.BasicLifecycler, ringDesc ring.Desc, instanceExists bool, instanceID string, instanceDesc ring.InstanceDesc) (ring.InstanceState, ring.Tokens) { // When we initialize the store-gateway instance in the ring we want to start from // a clean situation, so whatever is the state we set it JOINING, while we keep existing diff --git a/pkg/storegateway/gateway_test.go b/pkg/storegateway/gateway_test.go index d35f3677b4b..1524008b818 100644 --- a/pkg/storegateway/gateway_test.go +++ b/pkg/storegateway/gateway_test.go @@ -150,7 +150,7 @@ func TestStoreGateway_InitialSyncWithDefaultShardingEnabled(t *testing.T) { })) } - g, err := newStoreGateway(gatewayCfg, storageCfg, bucketClient, ringStore, defaultLimitsOverrides(t), mockLoggingLevel(), log.NewNopLogger(), nil) + g, err := newStoreGateway(gatewayCfg, storageCfg, bucketClient, ringStore, defaultLimitsOverrides(t), mockLoggingLevel(), log.NewNopLogger(), nil, nil) require.NoError(t, err) defer services.StopAndAwaitTerminated(ctx, g) //nolint:errcheck assert.False(t, g.ringLifecycler.IsRegistered()) @@ -192,7 +192,7 @@ func TestStoreGateway_InitialSyncWithShardingDisabled(t *testing.T) { storageCfg := mockStorageConfig(t) bucketClient := &bucket.ClientMock{} - g, err := newStoreGateway(gatewayCfg, storageCfg, bucketClient, nil, defaultLimitsOverrides(t), mockLoggingLevel(), log.NewNopLogger(), nil) + g, err := newStoreGateway(gatewayCfg, storageCfg, bucketClient, nil, defaultLimitsOverrides(t), mockLoggingLevel(), log.NewNopLogger(), nil, nil) require.NoError(t, err) defer services.StopAndAwaitTerminated(ctx, g) //nolint:errcheck @@ -219,7 +219,7 @@ func TestStoreGateway_InitialSyncFailure(t *testing.T) { bucketClient := &bucket.ClientMock{} - g, err := newStoreGateway(gatewayCfg, storageCfg, bucketClient, ringStore, defaultLimitsOverrides(t), mockLoggingLevel(), log.NewNopLogger(), nil) + g, err := newStoreGateway(gatewayCfg, storageCfg, bucketClient, ringStore, defaultLimitsOverrides(t), mockLoggingLevel(), log.NewNopLogger(), nil, nil) require.NoError(t, err) bucketClient.MockIter("", []string{}, errors.New("network error")) @@ -358,7 +358,7 @@ func TestStoreGateway_InitialSyncWithWaitRingStability(t *testing.T) { require.NoError(t, err) reg := prometheus.NewPedanticRegistry() - g, err := newStoreGateway(gatewayCfg, storageCfg, bucketClient, ringStore, overrides, mockLoggingLevel(), log.NewNopLogger(), reg) + g, err := newStoreGateway(gatewayCfg, storageCfg, bucketClient, ringStore, overrides, mockLoggingLevel(), log.NewNopLogger(), reg, nil) require.NoError(t, err) defer services.StopAndAwaitTerminated(ctx, g) //nolint:errcheck @@ -460,7 +460,7 @@ func TestStoreGateway_BlocksSyncWithDefaultSharding_RingTopologyChangedAfterScal require.NoError(t, err) reg := prometheus.NewPedanticRegistry() - g, err := newStoreGateway(gatewayCfg, storageCfg, bucketClient, ringStore, overrides, mockLoggingLevel(), log.NewNopLogger(), reg) + g, err := newStoreGateway(gatewayCfg, storageCfg, bucketClient, ringStore, overrides, mockLoggingLevel(), log.NewNopLogger(), reg, nil) require.NoError(t, err) return g, instanceID, reg @@ -604,7 +604,7 @@ func TestStoreGateway_ShouldSupportLoadRingTokensFromFile(t *testing.T) { bucketClient := &bucket.ClientMock{} bucketClient.MockIter("", []string{}, nil) - g, err := newStoreGateway(gatewayCfg, storageCfg, bucketClient, ringStore, defaultLimitsOverrides(t), mockLoggingLevel(), log.NewNopLogger(), nil) + g, err := newStoreGateway(gatewayCfg, storageCfg, bucketClient, ringStore, defaultLimitsOverrides(t), mockLoggingLevel(), log.NewNopLogger(), nil, nil) require.NoError(t, err) defer services.StopAndAwaitTerminated(ctx, g) //nolint:errcheck assert.False(t, g.ringLifecycler.IsRegistered()) @@ -814,7 +814,7 @@ func TestStoreGateway_SyncOnRingTopologyChanged(t *testing.T) { bucketClient := &bucket.ClientMock{} bucketClient.MockIter("", []string{}, nil) - g, err := newStoreGateway(gatewayCfg, storageCfg, bucketClient, ringStore, defaultLimitsOverrides(t), mockLoggingLevel(), log.NewNopLogger(), reg) + g, err := newStoreGateway(gatewayCfg, storageCfg, bucketClient, ringStore, defaultLimitsOverrides(t), mockLoggingLevel(), log.NewNopLogger(), reg, nil) require.NoError(t, err) // Store the initial ring state before starting the gateway. @@ -876,7 +876,7 @@ func TestStoreGateway_RingLifecyclerShouldAutoForgetUnhealthyInstances(t *testin bucketClient := &bucket.ClientMock{} bucketClient.MockIter("", []string{}, nil) - g, err := newStoreGateway(gatewayCfg, storageCfg, bucketClient, ringStore, defaultLimitsOverrides(t), mockLoggingLevel(), log.NewNopLogger(), nil) + g, err := newStoreGateway(gatewayCfg, storageCfg, bucketClient, ringStore, defaultLimitsOverrides(t), mockLoggingLevel(), log.NewNopLogger(), nil, nil) require.NoError(t, err) require.NoError(t, services.StartAndAwaitRunning(ctx, g)) defer services.StopAndAwaitTerminated(ctx, g) //nolint:errcheck @@ -960,7 +960,7 @@ func TestStoreGateway_SeriesQueryingShouldRemoveExternalLabels(t *testing.T) { storageCfg := mockStorageConfig(t) storageCfg.BucketStore.BucketIndex.Enabled = bucketIndexEnabled - g, err := newStoreGateway(gatewayCfg, storageCfg, objstore.WithNoopInstr(bucketClient), nil, defaultLimitsOverrides(t), mockLoggingLevel(), logger, nil) + g, err := newStoreGateway(gatewayCfg, storageCfg, objstore.WithNoopInstr(bucketClient), nil, defaultLimitsOverrides(t), mockLoggingLevel(), logger, nil, nil) require.NoError(t, err) require.NoError(t, services.StartAndAwaitRunning(ctx, g)) defer services.StopAndAwaitTerminated(ctx, g) //nolint:errcheck @@ -1059,7 +1059,7 @@ func TestStoreGateway_SeriesQueryingShouldEnforceMaxChunksPerQueryLimit(t *testi gatewayCfg.ShardingEnabled = false storageCfg := mockStorageConfig(t) - g, err := newStoreGateway(gatewayCfg, storageCfg, objstore.WithNoopInstr(bucketClient), nil, overrides, mockLoggingLevel(), logger, nil) + g, err := newStoreGateway(gatewayCfg, storageCfg, objstore.WithNoopInstr(bucketClient), nil, overrides, mockLoggingLevel(), logger, nil, nil) require.NoError(t, err) require.NoError(t, services.StartAndAwaitRunning(ctx, g)) defer services.StopAndAwaitTerminated(ctx, g) //nolint:errcheck @@ -1148,7 +1148,7 @@ func TestStoreGateway_SeriesQueryingShouldEnforceMaxSeriesPerQueryLimit(t *testi gatewayCfg.ShardingEnabled = false storageCfg := mockStorageConfig(t) - g, err := newStoreGateway(gatewayCfg, storageCfg, objstore.WithNoopInstr(bucketClient), nil, overrides, mockLoggingLevel(), logger, nil) + g, err := newStoreGateway(gatewayCfg, storageCfg, objstore.WithNoopInstr(bucketClient), nil, overrides, mockLoggingLevel(), logger, nil, nil) require.NoError(t, err) require.NoError(t, services.StartAndAwaitRunning(ctx, g)) defer services.StopAndAwaitTerminated(ctx, g) //nolint:errcheck @@ -1176,6 +1176,71 @@ func TestStoreGateway_SeriesQueryingShouldEnforceMaxSeriesPerQueryLimit(t *testi } } +func TestStoreGateway_SeriesThrottledByResourceMonitor(t *testing.T) { + ctx := context.Background() + logger := log.NewNopLogger() + userID := "user-1" + + storageDir, err := os.MkdirTemp(os.TempDir(), "") + require.NoError(t, err) + defer os.RemoveAll(storageDir) //nolint:errcheck + + now := time.Now() + minT := now.Add(-1*time.Hour).Unix() * 1000 + maxT := now.Unix() * 1000 + mockTSDB(t, path.Join(storageDir, userID), 1, 0, minT, maxT) + + bucketClient, err := filesystem.NewBucketClient(filesystem.Config{Directory: storageDir}) + require.NoError(t, err) + + req := &storepb.SeriesRequest{ + MinTime: minT, + MaxTime: maxT, + Matchers: []storepb.LabelMatcher{ + {Type: storepb.LabelMatcher_RE, Name: "__name__", Value: ".*"}, + }, + } + + overrides, err := validation.NewOverrides(defaultLimitsConfig(), nil) + require.NoError(t, err) + + // Create a store-gateway used to query back the series from the blocks. + gatewayCfg := mockGatewayConfig() + gatewayCfg.ShardingEnabled = false + storageCfg := mockStorageConfig(t) + + mockErr := fmt.Errorf("resource monitor error") + resourceMonitor := testResourceMonitor{ + err: mockErr, + } + + g, err := newStoreGateway(gatewayCfg, storageCfg, objstore.WithNoopInstr(bucketClient), nil, overrides, mockLoggingLevel(), logger, nil, &resourceMonitor) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(ctx, g)) + defer services.StopAndAwaitTerminated(ctx, g) //nolint:errcheck + + srv := newBucketStoreSeriesServer(setUserIDToGRPCContext(ctx, userID)) + err = g.Series(req, srv) + require.Error(t, err) + require.ErrorContains(t, err, mockErr.Error()) +} + +type testResourceMonitor struct { + err error +} + +func (t *testResourceMonitor) GetCPUUtilization() float64 { + return 0 +} + +func (t *testResourceMonitor) GetHeapUtilization() float64 { + return 0 +} + +func (t *testResourceMonitor) CheckResourceUtilization() (string, float64, float64, error) { + return "", 0, 0, t.err +} + func mockGatewayConfig() Config { cfg := Config{} flagext.DefaultValues(&cfg) diff --git a/pkg/util/resource/monitor.go b/pkg/util/resource/monitor.go new file mode 100644 index 00000000000..a6197500bd4 --- /dev/null +++ b/pkg/util/resource/monitor.go @@ -0,0 +1,232 @@ +package resource + +import ( + "context" + "fmt" + "net/http" + "runtime" + "runtime/debug" + "runtime/metrics" + "sync" + "time" + + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + "github.com/prometheus/procfs" + "github.com/weaveworks/common/httpgrpc" + + "github.com/cortexproject/cortex/pkg/configs" + "github.com/cortexproject/cortex/pkg/util/services" +) + +type ExhaustedError struct{} + +func (e *ExhaustedError) Error() string { + return "resource exhausted" +} + +var ErrResourceExhausted = httpgrpc.Errorf(http.StatusTooManyRequests, "resource exhausted") + +const heapMetricName = "/memory/classes/heap/objects:bytes" +const monitorInterval = time.Second +const dataPointsToAvg = 30 + +type IScanner interface { + Scan() (Stats, error) +} + +type Scanner struct { + proc procfs.Proc + metricSamples []metrics.Sample +} + +type Stats struct { + cpu float64 + heap uint64 +} + +func NewScanner() (*Scanner, error) { + proc, err := procfs.Self() + if err != nil { + return nil, errors.Wrap(err, "error reading proc directory") + } + + metricSamples := make([]metrics.Sample, 1) + metricSamples[0].Name = heapMetricName + metrics.Read(metricSamples) + + for _, sample := range metricSamples { + if sample.Value.Kind() == metrics.KindBad { + return nil, fmt.Errorf("metric %s is not supported", sample.Name) + } + } + + return &Scanner{ + proc: proc, + metricSamples: metricSamples, + }, nil +} + +func (s *Scanner) Scan() (Stats, error) { + stat, err := s.proc.Stat() + if err != nil { + return Stats{}, err + } + + metrics.Read(s.metricSamples) + + return Stats{ + cpu: stat.CPUTime(), + heap: s.metricSamples[0].Value.Uint64(), + }, nil +} + +type IMonitor interface { + GetCPUUtilization() float64 + GetHeapUtilization() float64 + CheckResourceUtilization() (string, float64, float64, error) +} + +type Monitor struct { + services.Service + + scanner IScanner + + containerLimit configs.Resources + utilization configs.Resources + thresholds configs.Resources + + // Variables to calculate average CPU utilization + index int + cpuRates []float64 + cpuIntervals []float64 + totalCPU float64 + totalInterval float64 + lastCPU float64 + lastUpdate time.Time + + lock sync.RWMutex +} + +func NewMonitor(thresholds configs.Resources, registerer prometheus.Registerer) (*Monitor, error) { + m := &Monitor{ + thresholds: thresholds, + lock: sync.RWMutex{}, + } + + m.containerLimit.CPU = float64(runtime.GOMAXPROCS(0)) + m.containerLimit.Heap = float64(debug.SetMemoryLimit(-1)) + + var err error + m.scanner, err = NewScanner() + if err != nil { + return nil, err + } + + m.cpuRates = make([]float64, dataPointsToAvg) + m.cpuIntervals = make([]float64, dataPointsToAvg) + m.Service = services.NewBasicService(nil, m.running, nil) + + promauto.With(registerer).NewGaugeFunc(prometheus.GaugeOpts{ + Name: "cortex_resource_utilization", + ConstLabels: map[string]string{"resource": "cpu"}, + }, m.GetCPUUtilization) + promauto.With(registerer).NewGaugeFunc(prometheus.GaugeOpts{ + Name: "cortex_resource_utilization", + ConstLabels: map[string]string{"resource": "heap"}, + }, m.GetHeapUtilization) + promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ + Name: "cortex_resource_threshold", + ConstLabels: map[string]string{"resource": "cpu"}, + }).Set(thresholds.CPU) + promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ + Name: "cortex_resource_threshold", + ConstLabels: map[string]string{"resource": "heap"}, + }).Set(thresholds.Heap) + + return m, nil +} + +func (m *Monitor) running(ctx context.Context) error { + ticker := time.NewTicker(monitorInterval) + defer ticker.Stop() + + for { + select { + case <-ctx.Done(): + return nil + + case <-ticker.C: + stats, err := m.scanner.Scan() + if err != nil { + return errors.Wrap(err, "error scanning metrics") + } + + m.storeCPUUtilization(stats) + m.storeHeapUtilization(stats) + } + } +} + +func (m *Monitor) storeCPUUtilization(stats Stats) { + m.lock.Lock() + defer m.lock.Unlock() + + now := time.Now() + + m.totalCPU -= m.cpuRates[m.index] + m.totalInterval -= m.cpuIntervals[m.index] + + m.cpuRates[m.index] = stats.cpu - m.lastCPU + m.cpuIntervals[m.index] = now.Sub(m.lastUpdate).Seconds() + + m.totalCPU += m.cpuRates[m.index] + m.totalInterval += m.cpuIntervals[m.index] + + m.lastCPU = stats.cpu + m.lastUpdate = now + m.index = (m.index + 1) % dataPointsToAvg + + if m.totalInterval > 0 && m.containerLimit.CPU > 0 { + m.utilization.CPU = m.totalCPU / m.totalInterval / m.containerLimit.CPU + } +} + +func (m *Monitor) storeHeapUtilization(stats Stats) { + m.lock.Lock() + defer m.lock.Unlock() + + m.utilization.Heap = float64(stats.heap) / m.containerLimit.Heap +} + +func (m *Monitor) GetCPUUtilization() float64 { + m.lock.RLock() + defer m.lock.RUnlock() + + return m.utilization.CPU +} + +func (m *Monitor) GetHeapUtilization() float64 { + m.lock.RLock() + defer m.lock.RUnlock() + + return m.utilization.Heap +} + +func (m *Monitor) CheckResourceUtilization() (string, float64, float64, error) { + cpu := m.GetCPUUtilization() + heap := m.GetHeapUtilization() + + if m.thresholds.CPU > 0 && cpu > m.thresholds.CPU { + err := ExhaustedError{} + return "cpu", m.thresholds.CPU, cpu, httpgrpc.Errorf(http.StatusTooManyRequests, err.Error()) + } + + if m.thresholds.Heap > 0 && heap > m.thresholds.Heap { + err := ExhaustedError{} + return "heap", m.thresholds.Heap, heap, httpgrpc.Errorf(http.StatusTooManyRequests, err.Error()) + } + + return "", 0, 0, nil +} diff --git a/pkg/util/resource/monitor_test.go b/pkg/util/resource/monitor_test.go new file mode 100644 index 00000000000..958e3540f76 --- /dev/null +++ b/pkg/util/resource/monitor_test.go @@ -0,0 +1 @@ +package resource From 21218458355ef636def529763e427af6717422d3 Mon Sep 17 00:00:00 2001 From: Justin Jung Date: Tue, 25 Mar 2025 13:28:06 -0700 Subject: [PATCH 02/29] doc Signed-off-by: Justin Jung --- docs/configuration/config-file-reference.md | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/docs/configuration/config-file-reference.md b/docs/configuration/config-file-reference.md index 1b9a03b47b4..691ace2c503 100644 --- a/docs/configuration/config-file-reference.md +++ b/docs/configuration/config-file-reference.md @@ -271,6 +271,15 @@ query_scheduler: # CLI flag: -query-scheduler.grpc-client-config.connect-timeout [connect_timeout: | default = 5s] +resource_thresholds: + # Utilization threshold for CPU, between 0 and 1. 0 to disable. + # CLI flag: -resource-thresholds.cpu + [cpu: | default = 0] + + # Utilization threshold for heap, between 0 and 1. 0 to disable. + # CLI flag: -resource-thresholds.heap + [heap: | default = 0] + # The tracing_config configures backends cortex uses. [tracing: ] ``` From 2b168fcc05cd2752c76aa61b6b73ccf69b2047a2 Mon Sep 17 00:00:00 2001 From: Justin Jung Date: Tue, 25 Mar 2025 13:39:26 -0700 Subject: [PATCH 03/29] Add automaxprocs Signed-off-by: Justin Jung --- go.mod | 3 +- go.sum | 4 + pkg/util/resource/monitor.go | 1 + vendor/go.uber.org/automaxprocs/.codecov.yml | 14 ++ vendor/go.uber.org/automaxprocs/.gitignore | 33 ++++ vendor/go.uber.org/automaxprocs/CHANGELOG.md | 52 ++++++ .../automaxprocs/CODE_OF_CONDUCT.md | 75 ++++++++ .../go.uber.org/automaxprocs/CONTRIBUTING.md | 81 ++++++++ vendor/go.uber.org/automaxprocs/LICENSE | 19 ++ vendor/go.uber.org/automaxprocs/Makefile | 46 +++++ vendor/go.uber.org/automaxprocs/README.md | 71 +++++++ .../go.uber.org/automaxprocs/automaxprocs.go | 33 ++++ .../automaxprocs/internal/cgroups/cgroup.go | 79 ++++++++ .../automaxprocs/internal/cgroups/cgroups.go | 118 ++++++++++++ .../automaxprocs/internal/cgroups/cgroups2.go | 176 ++++++++++++++++++ .../automaxprocs/internal/cgroups/doc.go | 23 +++ .../automaxprocs/internal/cgroups/errors.go | 52 ++++++ .../internal/cgroups/mountpoint.go | 171 +++++++++++++++++ .../automaxprocs/internal/cgroups/subsys.go | 103 ++++++++++ .../internal/runtime/cpu_quota_linux.go | 75 ++++++++ .../internal/runtime/cpu_quota_unsupported.go | 31 +++ .../automaxprocs/internal/runtime/runtime.go | 40 ++++ .../automaxprocs/maxprocs/maxprocs.go | 139 ++++++++++++++ .../automaxprocs/maxprocs/version.go | 24 +++ vendor/modules.txt | 6 + 25 files changed, 1468 insertions(+), 1 deletion(-) create mode 100644 vendor/go.uber.org/automaxprocs/.codecov.yml create mode 100644 vendor/go.uber.org/automaxprocs/.gitignore create mode 100644 vendor/go.uber.org/automaxprocs/CHANGELOG.md create mode 100644 vendor/go.uber.org/automaxprocs/CODE_OF_CONDUCT.md create mode 100644 vendor/go.uber.org/automaxprocs/CONTRIBUTING.md create mode 100644 vendor/go.uber.org/automaxprocs/LICENSE create mode 100644 vendor/go.uber.org/automaxprocs/Makefile create mode 100644 vendor/go.uber.org/automaxprocs/README.md create mode 100644 vendor/go.uber.org/automaxprocs/automaxprocs.go create mode 100644 vendor/go.uber.org/automaxprocs/internal/cgroups/cgroup.go create mode 100644 vendor/go.uber.org/automaxprocs/internal/cgroups/cgroups.go create mode 100644 vendor/go.uber.org/automaxprocs/internal/cgroups/cgroups2.go create mode 100644 vendor/go.uber.org/automaxprocs/internal/cgroups/doc.go create mode 100644 vendor/go.uber.org/automaxprocs/internal/cgroups/errors.go create mode 100644 vendor/go.uber.org/automaxprocs/internal/cgroups/mountpoint.go create mode 100644 vendor/go.uber.org/automaxprocs/internal/cgroups/subsys.go create mode 100644 vendor/go.uber.org/automaxprocs/internal/runtime/cpu_quota_linux.go create mode 100644 vendor/go.uber.org/automaxprocs/internal/runtime/cpu_quota_unsupported.go create mode 100644 vendor/go.uber.org/automaxprocs/internal/runtime/runtime.go create mode 100644 vendor/go.uber.org/automaxprocs/maxprocs/maxprocs.go create mode 100644 vendor/go.uber.org/automaxprocs/maxprocs/version.go diff --git a/go.mod b/go.mod index 8565daef850..4b108fa3602 100644 --- a/go.mod +++ b/go.mod @@ -81,9 +81,11 @@ require ( github.com/google/go-cmp v0.6.0 github.com/hashicorp/golang-lru/v2 v2.0.7 github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 + github.com/prometheus/procfs v0.15.1 github.com/sercand/kuberesolver/v5 v5.1.1 github.com/tjhop/slog-gokit v0.1.2 go.opentelemetry.io/collector/pdata v1.24.0 + go.uber.org/automaxprocs v1.6.0 google.golang.org/protobuf v1.36.4 ) @@ -199,7 +201,6 @@ require ( github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect github.com/prometheus-community/prom-label-proxy v0.8.1-0.20240127162815-c1195f9aabc0 // indirect github.com/prometheus/exporter-toolkit v0.13.2 // indirect - github.com/prometheus/procfs v0.15.1 // indirect github.com/prometheus/sigv4 v0.1.1 // indirect github.com/redis/rueidis v1.0.45-alpha.1 // indirect github.com/rs/cors v1.11.1 // indirect diff --git a/go.sum b/go.sum index 5d7538dbf4a..c70528d0cbd 100644 --- a/go.sum +++ b/go.sum @@ -1559,6 +1559,8 @@ github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 h1:Jamvg5psRI github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/posener/complete v1.1.1/go.mod h1:em0nMJCgc9GFtwrmVmEMR/ZL6WyhyjMBndrE9hABlRI= github.com/posener/complete v1.2.3/go.mod h1:WZIdtGGp+qx0sLrYKtIRAruyNpv6hFCicSgv7Sy7s/s= +github.com/prashantv/gostub v1.1.0 h1:BTyx3RfQjRHnUWaGF9oQos79AlQ5k8WNktv7VGvVH4g= +github.com/prashantv/gostub v1.1.0/go.mod h1:A5zLQHz7ieHGG7is6LLXLz7I8+3LZzsrV0P1IAHhP5U= github.com/prometheus-community/prom-label-proxy v0.8.1-0.20240127162815-c1195f9aabc0 h1:owfYHh79h8Y5HvNMGyww+DaVwo10CKiRW1RQrrZzIwg= github.com/prometheus-community/prom-label-proxy v0.8.1-0.20240127162815-c1195f9aabc0/go.mod h1:rT989D4UtOcfd9tVqIZRVIM8rkg+9XbreBjFNEKXvVI= github.com/prometheus/alertmanager v0.28.1 h1:BK5pCoAtaKg01BYRUJhEDV1tqJMEtYBGzPw8QdvnnvA= @@ -1810,6 +1812,8 @@ go.opentelemetry.io/proto/otlp v1.5.0/go.mod h1:keN8WnHxOy8PG0rQZjJJ5A2ebUoafqWp go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= go.uber.org/atomic v1.11.0 h1:ZvwS0R+56ePWxUNi+Atn9dWONBPp/AUETXlHW0DxSjE= go.uber.org/atomic v1.11.0/go.mod h1:LUxbIzbOniOlMKjJjyPfpl4v+PKK2cNJn91OQbhoJI0= +go.uber.org/automaxprocs v1.6.0 h1:O3y2/QNTOdbF+e/dpXNNW7Rx2hZ4sTIPyybbxyNqTUs= +go.uber.org/automaxprocs v1.6.0/go.mod h1:ifeIMSnPZuznNm6jmdzmU3/bfk01Fe2fotchwEFJ8r8= go.uber.org/goleak v1.1.10/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/goleak v1.3.0 h1:2K3zAYmnTNqV73imy9J1T3WC+gmCePx2hEGkimedGto= go.uber.org/goleak v1.3.0/go.mod h1:CoHD4mav9JJNrW/WLlf7HGZPjdw8EucARQHekz1X6bE= diff --git a/pkg/util/resource/monitor.go b/pkg/util/resource/monitor.go index a6197500bd4..fb51356359c 100644 --- a/pkg/util/resource/monitor.go +++ b/pkg/util/resource/monitor.go @@ -15,6 +15,7 @@ import ( "github.com/prometheus/client_golang/prometheus/promauto" "github.com/prometheus/procfs" "github.com/weaveworks/common/httpgrpc" + _ "go.uber.org/automaxprocs" "github.com/cortexproject/cortex/pkg/configs" "github.com/cortexproject/cortex/pkg/util/services" diff --git a/vendor/go.uber.org/automaxprocs/.codecov.yml b/vendor/go.uber.org/automaxprocs/.codecov.yml new file mode 100644 index 00000000000..9a2ed4a9969 --- /dev/null +++ b/vendor/go.uber.org/automaxprocs/.codecov.yml @@ -0,0 +1,14 @@ +coverage: + range: 80..100 + round: down + precision: 2 + + status: + project: # measuring the overall project coverage + default: # context, you can create multiple ones with custom titles + enabled: yes # must be yes|true to enable this status + target: 90% # specify the target coverage for each commit status + # option: "auto" (must increase from parent commit or pull request base) + # option: "X%" a static target percentage to hit + if_not_found: success # if parent is not found report status as success, error, or failure + if_ci_failed: error # if ci fails report status as success, error, or failure diff --git a/vendor/go.uber.org/automaxprocs/.gitignore b/vendor/go.uber.org/automaxprocs/.gitignore new file mode 100644 index 00000000000..dd7bcf5130b --- /dev/null +++ b/vendor/go.uber.org/automaxprocs/.gitignore @@ -0,0 +1,33 @@ +# Compiled Object files, Static and Dynamic libs (Shared Objects) +*.o +*.a +*.so + +# Folders +_obj +_test +vendor + +# Architecture specific extensions/prefixes +*.[568vq] +[568vq].out + +*.cgo1.go +*.cgo2.c +_cgo_defun.c +_cgo_gotypes.go +_cgo_export.* + +_testmain.go + +*.exe +*.test +*.prof +*.pprof +*.out +*.log +coverage.txt + +/bin +cover.out +cover.html diff --git a/vendor/go.uber.org/automaxprocs/CHANGELOG.md b/vendor/go.uber.org/automaxprocs/CHANGELOG.md new file mode 100644 index 00000000000..f421056ae82 --- /dev/null +++ b/vendor/go.uber.org/automaxprocs/CHANGELOG.md @@ -0,0 +1,52 @@ +# Changelog + +## v1.6.0 (2024-07-24) + +- Add RoundQuotaFunc option that allows configuration of rounding + behavior for floating point CPU quota. + +## v1.5.3 (2023-07-19) + +- Fix mountinfo parsing when super options have fields with spaces. +- Fix division by zero while parsing cgroups. + +## v1.5.2 (2023-03-16) + +- Support child control cgroups +- Fix file descriptor leak +- Update dependencies + +## v1.5.1 (2022-04-06) + +- Fix cgroups v2 mountpoint detection. + +## v1.5.0 (2022-04-05) + +- Add support for cgroups v2. + +Thanks to @emadolsky for their contribution to this release. + +## v1.4.0 (2021-02-01) + +- Support colons in cgroup names. +- Remove linters from runtime dependencies. + +## v1.3.0 (2020-01-23) + +- Migrate to Go modules. + +## v1.2.0 (2018-02-22) + +- Fixed quota clamping to always round down rather than up; Rather than + guaranteeing constant throttling at saturation, instead assume that the + fractional CPU was added as a hedge for factors outside of Go's scheduler. + +## v1.1.0 (2017-11-10) + +- Log the new value of `GOMAXPROCS` rather than the current value. +- Make logs more explicit about whether `GOMAXPROCS` was modified or not. +- Allow customization of the minimum `GOMAXPROCS`, and modify default from 2 to 1. + +## v1.0.0 (2017-08-09) + +- Initial release. diff --git a/vendor/go.uber.org/automaxprocs/CODE_OF_CONDUCT.md b/vendor/go.uber.org/automaxprocs/CODE_OF_CONDUCT.md new file mode 100644 index 00000000000..e327d9aa5cd --- /dev/null +++ b/vendor/go.uber.org/automaxprocs/CODE_OF_CONDUCT.md @@ -0,0 +1,75 @@ +# Contributor Covenant Code of Conduct + +## Our Pledge + +In the interest of fostering an open and welcoming environment, we as +contributors and maintainers pledge to making participation in our project and +our community a harassment-free experience for everyone, regardless of age, +body size, disability, ethnicity, gender identity and expression, level of +experience, nationality, personal appearance, race, religion, or sexual +identity and orientation. + +## Our Standards + +Examples of behavior that contributes to creating a positive environment +include: + +* Using welcoming and inclusive language +* Being respectful of differing viewpoints and experiences +* Gracefully accepting constructive criticism +* Focusing on what is best for the community +* Showing empathy towards other community members + +Examples of unacceptable behavior by participants include: + +* The use of sexualized language or imagery and unwelcome sexual attention or + advances +* Trolling, insulting/derogatory comments, and personal or political attacks +* Public or private harassment +* Publishing others' private information, such as a physical or electronic + address, without explicit permission +* Other conduct which could reasonably be considered inappropriate in a + professional setting + +## Our Responsibilities + +Project maintainers are responsible for clarifying the standards of acceptable +behavior and are expected to take appropriate and fair corrective action in +response to any instances of unacceptable behavior. + +Project maintainers have the right and responsibility to remove, edit, or +reject comments, commits, code, wiki edits, issues, and other contributions +that are not aligned to this Code of Conduct, or to ban temporarily or +permanently any contributor for other behaviors that they deem inappropriate, +threatening, offensive, or harmful. + +## Scope + +This Code of Conduct applies both within project spaces and in public spaces +when an individual is representing the project or its community. Examples of +representing a project or community include using an official project e-mail +address, posting via an official social media account, or acting as an +appointed representative at an online or offline event. Representation of a +project may be further defined and clarified by project maintainers. + +## Enforcement + +Instances of abusive, harassing, or otherwise unacceptable behavior may be +reported by contacting the project team at oss-conduct@uber.com. The project +team will review and investigate all complaints, and will respond in a way +that it deems appropriate to the circumstances. The project team is obligated +to maintain confidentiality with regard to the reporter of an incident. +Further details of specific enforcement policies may be posted separately. + +Project maintainers who do not follow or enforce the Code of Conduct in good +faith may face temporary or permanent repercussions as determined by other +members of the project's leadership. + +## Attribution + +This Code of Conduct is adapted from the [Contributor Covenant][homepage], +version 1.4, available at +[http://contributor-covenant.org/version/1/4][version]. + +[homepage]: http://contributor-covenant.org +[version]: http://contributor-covenant.org/version/1/4/ diff --git a/vendor/go.uber.org/automaxprocs/CONTRIBUTING.md b/vendor/go.uber.org/automaxprocs/CONTRIBUTING.md new file mode 100644 index 00000000000..2b6a6040d78 --- /dev/null +++ b/vendor/go.uber.org/automaxprocs/CONTRIBUTING.md @@ -0,0 +1,81 @@ +# Contributing + +We'd love your help improving this package! + +If you'd like to add new exported APIs, please [open an issue][open-issue] +describing your proposal — discussing API changes ahead of time makes +pull request review much smoother. In your issue, pull request, and any other +communications, please remember to treat your fellow contributors with +respect! We take our [code of conduct](CODE_OF_CONDUCT.md) seriously. + +Note that you'll need to sign [Uber's Contributor License Agreement][cla] +before we can accept any of your contributions. If necessary, a bot will remind +you to accept the CLA when you open your pull request. + +## Setup + +[Fork][fork], then clone the repository: + +``` +mkdir -p $GOPATH/src/go.uber.org +cd $GOPATH/src/go.uber.org +git clone git@github.com:your_github_username/automaxprocs.git +cd automaxprocs +git remote add upstream https://github.com/uber-go/automaxprocs.git +git fetch upstream +``` + +Install the test dependencies: + +``` +make dependencies +``` + +Make sure that the tests and the linters pass: + +``` +make test +make lint +``` + +If you're not using the minor version of Go specified in the Makefile's +`LINTABLE_MINOR_VERSIONS` variable, `make lint` doesn't do anything. This is +fine, but it means that you'll only discover lint failures after you open your +pull request. + +## Making Changes + +Start by creating a new branch for your changes: + +``` +cd $GOPATH/src/go.uber.org/automaxprocs +git checkout master +git fetch upstream +git rebase upstream/master +git checkout -b cool_new_feature +``` + +Make your changes, then ensure that `make lint` and `make test` still pass. If +you're satisfied with your changes, push them to your fork. + +``` +git push origin cool_new_feature +``` + +Then use the GitHub UI to open a pull request. + +At this point, you're waiting on us to review your changes. We *try* to respond +to issues and pull requests within a few business days, and we may suggest some +improvements or alternatives. Once your changes are approved, one of the +project maintainers will merge them. + +We're much more likely to approve your changes if you: + +* Add tests for new functionality. +* Write a [good commit message][commit-message]. +* Maintain backward compatibility. + +[fork]: https://github.com/uber-go/automaxprocs/fork +[open-issue]: https://github.com/uber-go/automaxprocs/issues/new +[cla]: https://cla-assistant.io/uber-go/automaxprocs +[commit-message]: http://tbaggery.com/2008/04/19/a-note-about-git-commit-messages.html diff --git a/vendor/go.uber.org/automaxprocs/LICENSE b/vendor/go.uber.org/automaxprocs/LICENSE new file mode 100644 index 00000000000..20dcf51d96d --- /dev/null +++ b/vendor/go.uber.org/automaxprocs/LICENSE @@ -0,0 +1,19 @@ +Copyright (c) 2017 Uber Technologies, Inc. + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. \ No newline at end of file diff --git a/vendor/go.uber.org/automaxprocs/Makefile b/vendor/go.uber.org/automaxprocs/Makefile new file mode 100644 index 00000000000..1642b714801 --- /dev/null +++ b/vendor/go.uber.org/automaxprocs/Makefile @@ -0,0 +1,46 @@ +export GOBIN ?= $(shell pwd)/bin + +GO_FILES := $(shell \ + find . '(' -path '*/.*' -o -path './vendor' ')' -prune \ + -o -name '*.go' -print | cut -b3-) + +GOLINT = $(GOBIN)/golint +STATICCHECK = $(GOBIN)/staticcheck + +.PHONY: build +build: + go build ./... + +.PHONY: install +install: + go mod download + +.PHONY: test +test: + go test -race ./... + +.PHONY: cover +cover: + go test -coverprofile=cover.out -covermode=atomic -coverpkg=./... ./... + go tool cover -html=cover.out -o cover.html + +$(GOLINT): tools/go.mod + cd tools && go install golang.org/x/lint/golint + +$(STATICCHECK): tools/go.mod + cd tools && go install honnef.co/go/tools/cmd/staticcheck@2023.1.2 + +.PHONY: lint +lint: $(GOLINT) $(STATICCHECK) + @rm -rf lint.log + @echo "Checking gofmt" + @gofmt -d -s $(GO_FILES) 2>&1 | tee lint.log + @echo "Checking go vet" + @go vet ./... 2>&1 | tee -a lint.log + @echo "Checking golint" + @$(GOLINT) ./... | tee -a lint.log + @echo "Checking staticcheck" + @$(STATICCHECK) ./... 2>&1 | tee -a lint.log + @echo "Checking for license headers..." + @./.build/check_license.sh | tee -a lint.log + @[ ! -s lint.log ] diff --git a/vendor/go.uber.org/automaxprocs/README.md b/vendor/go.uber.org/automaxprocs/README.md new file mode 100644 index 00000000000..bfed32adae8 --- /dev/null +++ b/vendor/go.uber.org/automaxprocs/README.md @@ -0,0 +1,71 @@ +# automaxprocs [![GoDoc][doc-img]][doc] [![Build Status][ci-img]][ci] [![Coverage Status][cov-img]][cov] + +Automatically set `GOMAXPROCS` to match Linux container CPU quota. + +## Installation + +`go get -u go.uber.org/automaxprocs` + +## Quick Start + +```go +import _ "go.uber.org/automaxprocs" + +func main() { + // Your application logic here. +} +``` + +# Performance +Data measured from Uber's internal load balancer. We ran the load balancer with 200% CPU quota (i.e., 2 cores): + +| GOMAXPROCS | RPS | P50 (ms) | P99.9 (ms) | +| ------------------ | --------- | -------- | ---------- | +| 1 | 28,893.18 | 1.46 | 19.70 | +| 2 (equal to quota) | 44,715.07 | 0.84 | 26.38 | +| 3 | 44,212.93 | 0.66 | 30.07 | +| 4 | 41,071.15 | 0.57 | 42.94 | +| 8 | 33,111.69 | 0.43 | 64.32 | +| Default (24) | 22,191.40 | 0.45 | 76.19 | + +When `GOMAXPROCS` is increased above the CPU quota, we see P50 decrease slightly, but see significant increases to P99. We also see that the total RPS handled also decreases. + +When `GOMAXPROCS` is higher than the CPU quota allocated, we also saw significant throttling: + +``` +$ cat /sys/fs/cgroup/cpu,cpuacct/system.slice/[...]/cpu.stat +nr_periods 42227334 +nr_throttled 131923 +throttled_time 88613212216618 +``` + +Once `GOMAXPROCS` was reduced to match the CPU quota, we saw no CPU throttling. + +## Development Status: Stable + +All APIs are finalized, and no breaking changes will be made in the 1.x series +of releases. Users of semver-aware dependency management systems should pin +automaxprocs to `^1`. + +## Contributing + +We encourage and support an active, healthy community of contributors — +including you! Details are in the [contribution guide](CONTRIBUTING.md) and +the [code of conduct](CODE_OF_CONDUCT.md). The automaxprocs maintainers keep +an eye on issues and pull requests, but you can also report any negative +conduct to oss-conduct@uber.com. That email list is a private, safe space; +even the automaxprocs maintainers don't have access, so don't hesitate to hold +us to a high standard. + +
+ +Released under the [MIT License](LICENSE). + +[doc-img]: https://godoc.org/go.uber.org/automaxprocs?status.svg +[doc]: https://godoc.org/go.uber.org/automaxprocs +[ci-img]: https://github.com/uber-go/automaxprocs/actions/workflows/go.yml/badge.svg +[ci]: https://github.com/uber-go/automaxprocs/actions/workflows/go.yml +[cov-img]: https://codecov.io/gh/uber-go/automaxprocs/branch/master/graph/badge.svg +[cov]: https://codecov.io/gh/uber-go/automaxprocs + + diff --git a/vendor/go.uber.org/automaxprocs/automaxprocs.go b/vendor/go.uber.org/automaxprocs/automaxprocs.go new file mode 100644 index 00000000000..69946a3e1fd --- /dev/null +++ b/vendor/go.uber.org/automaxprocs/automaxprocs.go @@ -0,0 +1,33 @@ +// Copyright (c) 2017 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Package automaxprocs automatically sets GOMAXPROCS to match the Linux +// container CPU quota, if any. +package automaxprocs // import "go.uber.org/automaxprocs" + +import ( + "log" + + "go.uber.org/automaxprocs/maxprocs" +) + +func init() { + maxprocs.Set(maxprocs.Logger(log.Printf)) +} diff --git a/vendor/go.uber.org/automaxprocs/internal/cgroups/cgroup.go b/vendor/go.uber.org/automaxprocs/internal/cgroups/cgroup.go new file mode 100644 index 00000000000..fe4ecf561e2 --- /dev/null +++ b/vendor/go.uber.org/automaxprocs/internal/cgroups/cgroup.go @@ -0,0 +1,79 @@ +// Copyright (c) 2017 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +//go:build linux +// +build linux + +package cgroups + +import ( + "bufio" + "io" + "os" + "path/filepath" + "strconv" +) + +// CGroup represents the data structure for a Linux control group. +type CGroup struct { + path string +} + +// NewCGroup returns a new *CGroup from a given path. +func NewCGroup(path string) *CGroup { + return &CGroup{path: path} +} + +// Path returns the path of the CGroup*. +func (cg *CGroup) Path() string { + return cg.path +} + +// ParamPath returns the path of the given cgroup param under itself. +func (cg *CGroup) ParamPath(param string) string { + return filepath.Join(cg.path, param) +} + +// readFirstLine reads the first line from a cgroup param file. +func (cg *CGroup) readFirstLine(param string) (string, error) { + paramFile, err := os.Open(cg.ParamPath(param)) + if err != nil { + return "", err + } + defer paramFile.Close() + + scanner := bufio.NewScanner(paramFile) + if scanner.Scan() { + return scanner.Text(), nil + } + if err := scanner.Err(); err != nil { + return "", err + } + return "", io.ErrUnexpectedEOF +} + +// readInt parses the first line from a cgroup param file as int. +func (cg *CGroup) readInt(param string) (int, error) { + text, err := cg.readFirstLine(param) + if err != nil { + return 0, err + } + return strconv.Atoi(text) +} diff --git a/vendor/go.uber.org/automaxprocs/internal/cgroups/cgroups.go b/vendor/go.uber.org/automaxprocs/internal/cgroups/cgroups.go new file mode 100644 index 00000000000..e89f5436028 --- /dev/null +++ b/vendor/go.uber.org/automaxprocs/internal/cgroups/cgroups.go @@ -0,0 +1,118 @@ +// Copyright (c) 2017 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +//go:build linux +// +build linux + +package cgroups + +const ( + // _cgroupFSType is the Linux CGroup file system type used in + // `/proc/$PID/mountinfo`. + _cgroupFSType = "cgroup" + // _cgroupSubsysCPU is the CPU CGroup subsystem. + _cgroupSubsysCPU = "cpu" + // _cgroupSubsysCPUAcct is the CPU accounting CGroup subsystem. + _cgroupSubsysCPUAcct = "cpuacct" + // _cgroupSubsysCPUSet is the CPUSet CGroup subsystem. + _cgroupSubsysCPUSet = "cpuset" + // _cgroupSubsysMemory is the Memory CGroup subsystem. + _cgroupSubsysMemory = "memory" + + // _cgroupCPUCFSQuotaUsParam is the file name for the CGroup CFS quota + // parameter. + _cgroupCPUCFSQuotaUsParam = "cpu.cfs_quota_us" + // _cgroupCPUCFSPeriodUsParam is the file name for the CGroup CFS period + // parameter. + _cgroupCPUCFSPeriodUsParam = "cpu.cfs_period_us" +) + +const ( + _procPathCGroup = "/proc/self/cgroup" + _procPathMountInfo = "/proc/self/mountinfo" +) + +// CGroups is a map that associates each CGroup with its subsystem name. +type CGroups map[string]*CGroup + +// NewCGroups returns a new *CGroups from given `mountinfo` and `cgroup` files +// under for some process under `/proc` file system (see also proc(5) for more +// information). +func NewCGroups(procPathMountInfo, procPathCGroup string) (CGroups, error) { + cgroupSubsystems, err := parseCGroupSubsystems(procPathCGroup) + if err != nil { + return nil, err + } + + cgroups := make(CGroups) + newMountPoint := func(mp *MountPoint) error { + if mp.FSType != _cgroupFSType { + return nil + } + + for _, opt := range mp.SuperOptions { + subsys, exists := cgroupSubsystems[opt] + if !exists { + continue + } + + cgroupPath, err := mp.Translate(subsys.Name) + if err != nil { + return err + } + cgroups[opt] = NewCGroup(cgroupPath) + } + + return nil + } + + if err := parseMountInfo(procPathMountInfo, newMountPoint); err != nil { + return nil, err + } + return cgroups, nil +} + +// NewCGroupsForCurrentProcess returns a new *CGroups instance for the current +// process. +func NewCGroupsForCurrentProcess() (CGroups, error) { + return NewCGroups(_procPathMountInfo, _procPathCGroup) +} + +// CPUQuota returns the CPU quota applied with the CPU cgroup controller. +// It is a result of `cpu.cfs_quota_us / cpu.cfs_period_us`. If the value of +// `cpu.cfs_quota_us` was not set (-1), the method returns `(-1, nil)`. +func (cg CGroups) CPUQuota() (float64, bool, error) { + cpuCGroup, exists := cg[_cgroupSubsysCPU] + if !exists { + return -1, false, nil + } + + cfsQuotaUs, err := cpuCGroup.readInt(_cgroupCPUCFSQuotaUsParam) + if defined := cfsQuotaUs > 0; err != nil || !defined { + return -1, defined, err + } + + cfsPeriodUs, err := cpuCGroup.readInt(_cgroupCPUCFSPeriodUsParam) + if defined := cfsPeriodUs > 0; err != nil || !defined { + return -1, defined, err + } + + return float64(cfsQuotaUs) / float64(cfsPeriodUs), true, nil +} diff --git a/vendor/go.uber.org/automaxprocs/internal/cgroups/cgroups2.go b/vendor/go.uber.org/automaxprocs/internal/cgroups/cgroups2.go new file mode 100644 index 00000000000..78556062fe2 --- /dev/null +++ b/vendor/go.uber.org/automaxprocs/internal/cgroups/cgroups2.go @@ -0,0 +1,176 @@ +// Copyright (c) 2022 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +//go:build linux +// +build linux + +package cgroups + +import ( + "bufio" + "errors" + "fmt" + "io" + "os" + "path" + "strconv" + "strings" +) + +const ( + // _cgroupv2CPUMax is the file name for the CGroup-V2 CPU max and period + // parameter. + _cgroupv2CPUMax = "cpu.max" + // _cgroupFSType is the Linux CGroup-V2 file system type used in + // `/proc/$PID/mountinfo`. + _cgroupv2FSType = "cgroup2" + + _cgroupv2MountPoint = "/sys/fs/cgroup" + + _cgroupV2CPUMaxDefaultPeriod = 100000 + _cgroupV2CPUMaxQuotaMax = "max" +) + +const ( + _cgroupv2CPUMaxQuotaIndex = iota + _cgroupv2CPUMaxPeriodIndex +) + +// ErrNotV2 indicates that the system is not using cgroups2. +var ErrNotV2 = errors.New("not using cgroups2") + +// CGroups2 provides access to cgroups data for systems using cgroups2. +type CGroups2 struct { + mountPoint string + groupPath string + cpuMaxFile string +} + +// NewCGroups2ForCurrentProcess builds a CGroups2 for the current process. +// +// This returns ErrNotV2 if the system is not using cgroups2. +func NewCGroups2ForCurrentProcess() (*CGroups2, error) { + return newCGroups2From(_procPathMountInfo, _procPathCGroup) +} + +func newCGroups2From(mountInfoPath, procPathCGroup string) (*CGroups2, error) { + isV2, err := isCGroupV2(mountInfoPath) + if err != nil { + return nil, err + } + + if !isV2 { + return nil, ErrNotV2 + } + + subsystems, err := parseCGroupSubsystems(procPathCGroup) + if err != nil { + return nil, err + } + + // Find v2 subsystem by looking for the `0` id + var v2subsys *CGroupSubsys + for _, subsys := range subsystems { + if subsys.ID == 0 { + v2subsys = subsys + break + } + } + + if v2subsys == nil { + return nil, ErrNotV2 + } + + return &CGroups2{ + mountPoint: _cgroupv2MountPoint, + groupPath: v2subsys.Name, + cpuMaxFile: _cgroupv2CPUMax, + }, nil +} + +func isCGroupV2(procPathMountInfo string) (bool, error) { + var ( + isV2 bool + newMountPoint = func(mp *MountPoint) error { + isV2 = isV2 || (mp.FSType == _cgroupv2FSType && mp.MountPoint == _cgroupv2MountPoint) + return nil + } + ) + + if err := parseMountInfo(procPathMountInfo, newMountPoint); err != nil { + return false, err + } + + return isV2, nil +} + +// CPUQuota returns the CPU quota applied with the CPU cgroup2 controller. +// It is a result of reading cpu quota and period from cpu.max file. +// It will return `cpu.max / cpu.period`. If cpu.max is set to max, it returns +// (-1, false, nil) +func (cg *CGroups2) CPUQuota() (float64, bool, error) { + cpuMaxParams, err := os.Open(path.Join(cg.mountPoint, cg.groupPath, cg.cpuMaxFile)) + if err != nil { + if os.IsNotExist(err) { + return -1, false, nil + } + return -1, false, err + } + defer cpuMaxParams.Close() + + scanner := bufio.NewScanner(cpuMaxParams) + if scanner.Scan() { + fields := strings.Fields(scanner.Text()) + if len(fields) == 0 || len(fields) > 2 { + return -1, false, fmt.Errorf("invalid format") + } + + if fields[_cgroupv2CPUMaxQuotaIndex] == _cgroupV2CPUMaxQuotaMax { + return -1, false, nil + } + + max, err := strconv.Atoi(fields[_cgroupv2CPUMaxQuotaIndex]) + if err != nil { + return -1, false, err + } + + var period int + if len(fields) == 1 { + period = _cgroupV2CPUMaxDefaultPeriod + } else { + period, err = strconv.Atoi(fields[_cgroupv2CPUMaxPeriodIndex]) + if err != nil { + return -1, false, err + } + + if period == 0 { + return -1, false, errors.New("zero value for period is not allowed") + } + } + + return float64(max) / float64(period), true, nil + } + + if err := scanner.Err(); err != nil { + return -1, false, err + } + + return 0, false, io.ErrUnexpectedEOF +} diff --git a/vendor/go.uber.org/automaxprocs/internal/cgroups/doc.go b/vendor/go.uber.org/automaxprocs/internal/cgroups/doc.go new file mode 100644 index 00000000000..113555f63da --- /dev/null +++ b/vendor/go.uber.org/automaxprocs/internal/cgroups/doc.go @@ -0,0 +1,23 @@ +// Copyright (c) 2017 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Package cgroups provides utilities to access Linux control group (CGroups) +// parameters (CPU quota, for example) for a given process. +package cgroups diff --git a/vendor/go.uber.org/automaxprocs/internal/cgroups/errors.go b/vendor/go.uber.org/automaxprocs/internal/cgroups/errors.go new file mode 100644 index 00000000000..94ac75a46e8 --- /dev/null +++ b/vendor/go.uber.org/automaxprocs/internal/cgroups/errors.go @@ -0,0 +1,52 @@ +// Copyright (c) 2017 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +//go:build linux +// +build linux + +package cgroups + +import "fmt" + +type cgroupSubsysFormatInvalidError struct { + line string +} + +type mountPointFormatInvalidError struct { + line string +} + +type pathNotExposedFromMountPointError struct { + mountPoint string + root string + path string +} + +func (err cgroupSubsysFormatInvalidError) Error() string { + return fmt.Sprintf("invalid format for CGroupSubsys: %q", err.line) +} + +func (err mountPointFormatInvalidError) Error() string { + return fmt.Sprintf("invalid format for MountPoint: %q", err.line) +} + +func (err pathNotExposedFromMountPointError) Error() string { + return fmt.Sprintf("path %q is not a descendant of mount point root %q and cannot be exposed from %q", err.path, err.root, err.mountPoint) +} diff --git a/vendor/go.uber.org/automaxprocs/internal/cgroups/mountpoint.go b/vendor/go.uber.org/automaxprocs/internal/cgroups/mountpoint.go new file mode 100644 index 00000000000..f3877f78aa6 --- /dev/null +++ b/vendor/go.uber.org/automaxprocs/internal/cgroups/mountpoint.go @@ -0,0 +1,171 @@ +// Copyright (c) 2017 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +//go:build linux +// +build linux + +package cgroups + +import ( + "bufio" + "os" + "path/filepath" + "strconv" + "strings" +) + +const ( + _mountInfoSep = " " + _mountInfoOptsSep = "," + _mountInfoOptionalFieldsSep = "-" +) + +const ( + _miFieldIDMountID = iota + _miFieldIDParentID + _miFieldIDDeviceID + _miFieldIDRoot + _miFieldIDMountPoint + _miFieldIDOptions + _miFieldIDOptionalFields + + _miFieldCountFirstHalf +) + +const ( + _miFieldOffsetFSType = iota + _miFieldOffsetMountSource + _miFieldOffsetSuperOptions + + _miFieldCountSecondHalf +) + +const _miFieldCountMin = _miFieldCountFirstHalf + _miFieldCountSecondHalf + +// MountPoint is the data structure for the mount points in +// `/proc/$PID/mountinfo`. See also proc(5) for more information. +type MountPoint struct { + MountID int + ParentID int + DeviceID string + Root string + MountPoint string + Options []string + OptionalFields []string + FSType string + MountSource string + SuperOptions []string +} + +// NewMountPointFromLine parses a line read from `/proc/$PID/mountinfo` and +// returns a new *MountPoint. +func NewMountPointFromLine(line string) (*MountPoint, error) { + fields := strings.Split(line, _mountInfoSep) + + if len(fields) < _miFieldCountMin { + return nil, mountPointFormatInvalidError{line} + } + + mountID, err := strconv.Atoi(fields[_miFieldIDMountID]) + if err != nil { + return nil, err + } + + parentID, err := strconv.Atoi(fields[_miFieldIDParentID]) + if err != nil { + return nil, err + } + + for i, field := range fields[_miFieldIDOptionalFields:] { + if field == _mountInfoOptionalFieldsSep { + // End of optional fields. + fsTypeStart := _miFieldIDOptionalFields + i + 1 + + // Now we know where the optional fields end, split the line again with a + // limit to avoid issues with spaces in super options as present on WSL. + fields = strings.SplitN(line, _mountInfoSep, fsTypeStart+_miFieldCountSecondHalf) + if len(fields) != fsTypeStart+_miFieldCountSecondHalf { + return nil, mountPointFormatInvalidError{line} + } + + miFieldIDFSType := _miFieldOffsetFSType + fsTypeStart + miFieldIDMountSource := _miFieldOffsetMountSource + fsTypeStart + miFieldIDSuperOptions := _miFieldOffsetSuperOptions + fsTypeStart + + return &MountPoint{ + MountID: mountID, + ParentID: parentID, + DeviceID: fields[_miFieldIDDeviceID], + Root: fields[_miFieldIDRoot], + MountPoint: fields[_miFieldIDMountPoint], + Options: strings.Split(fields[_miFieldIDOptions], _mountInfoOptsSep), + OptionalFields: fields[_miFieldIDOptionalFields:(fsTypeStart - 1)], + FSType: fields[miFieldIDFSType], + MountSource: fields[miFieldIDMountSource], + SuperOptions: strings.Split(fields[miFieldIDSuperOptions], _mountInfoOptsSep), + }, nil + } + } + + return nil, mountPointFormatInvalidError{line} +} + +// Translate converts an absolute path inside the *MountPoint's file system to +// the host file system path in the mount namespace the *MountPoint belongs to. +func (mp *MountPoint) Translate(absPath string) (string, error) { + relPath, err := filepath.Rel(mp.Root, absPath) + + if err != nil { + return "", err + } + if relPath == ".." || strings.HasPrefix(relPath, "../") { + return "", pathNotExposedFromMountPointError{ + mountPoint: mp.MountPoint, + root: mp.Root, + path: absPath, + } + } + + return filepath.Join(mp.MountPoint, relPath), nil +} + +// parseMountInfo parses procPathMountInfo (usually at `/proc/$PID/mountinfo`) +// and yields parsed *MountPoint into newMountPoint. +func parseMountInfo(procPathMountInfo string, newMountPoint func(*MountPoint) error) error { + mountInfoFile, err := os.Open(procPathMountInfo) + if err != nil { + return err + } + defer mountInfoFile.Close() + + scanner := bufio.NewScanner(mountInfoFile) + + for scanner.Scan() { + mountPoint, err := NewMountPointFromLine(scanner.Text()) + if err != nil { + return err + } + if err := newMountPoint(mountPoint); err != nil { + return err + } + } + + return scanner.Err() +} diff --git a/vendor/go.uber.org/automaxprocs/internal/cgroups/subsys.go b/vendor/go.uber.org/automaxprocs/internal/cgroups/subsys.go new file mode 100644 index 00000000000..cddc3eaec39 --- /dev/null +++ b/vendor/go.uber.org/automaxprocs/internal/cgroups/subsys.go @@ -0,0 +1,103 @@ +// Copyright (c) 2017 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +//go:build linux +// +build linux + +package cgroups + +import ( + "bufio" + "os" + "strconv" + "strings" +) + +const ( + _cgroupSep = ":" + _cgroupSubsysSep = "," +) + +const ( + _csFieldIDID = iota + _csFieldIDSubsystems + _csFieldIDName + _csFieldCount +) + +// CGroupSubsys represents the data structure for entities in +// `/proc/$PID/cgroup`. See also proc(5) for more information. +type CGroupSubsys struct { + ID int + Subsystems []string + Name string +} + +// NewCGroupSubsysFromLine returns a new *CGroupSubsys by parsing a string in +// the format of `/proc/$PID/cgroup` +func NewCGroupSubsysFromLine(line string) (*CGroupSubsys, error) { + fields := strings.SplitN(line, _cgroupSep, _csFieldCount) + + if len(fields) != _csFieldCount { + return nil, cgroupSubsysFormatInvalidError{line} + } + + id, err := strconv.Atoi(fields[_csFieldIDID]) + if err != nil { + return nil, err + } + + cgroup := &CGroupSubsys{ + ID: id, + Subsystems: strings.Split(fields[_csFieldIDSubsystems], _cgroupSubsysSep), + Name: fields[_csFieldIDName], + } + + return cgroup, nil +} + +// parseCGroupSubsystems parses procPathCGroup (usually at `/proc/$PID/cgroup`) +// and returns a new map[string]*CGroupSubsys. +func parseCGroupSubsystems(procPathCGroup string) (map[string]*CGroupSubsys, error) { + cgroupFile, err := os.Open(procPathCGroup) + if err != nil { + return nil, err + } + defer cgroupFile.Close() + + scanner := bufio.NewScanner(cgroupFile) + subsystems := make(map[string]*CGroupSubsys) + + for scanner.Scan() { + cgroup, err := NewCGroupSubsysFromLine(scanner.Text()) + if err != nil { + return nil, err + } + for _, subsys := range cgroup.Subsystems { + subsystems[subsys] = cgroup + } + } + + if err := scanner.Err(); err != nil { + return nil, err + } + + return subsystems, nil +} diff --git a/vendor/go.uber.org/automaxprocs/internal/runtime/cpu_quota_linux.go b/vendor/go.uber.org/automaxprocs/internal/runtime/cpu_quota_linux.go new file mode 100644 index 00000000000..f9057fd2731 --- /dev/null +++ b/vendor/go.uber.org/automaxprocs/internal/runtime/cpu_quota_linux.go @@ -0,0 +1,75 @@ +// Copyright (c) 2017 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +//go:build linux +// +build linux + +package runtime + +import ( + "errors" + + cg "go.uber.org/automaxprocs/internal/cgroups" +) + +// CPUQuotaToGOMAXPROCS converts the CPU quota applied to the calling process +// to a valid GOMAXPROCS value. The quota is converted from float to int using round. +// If round == nil, DefaultRoundFunc is used. +func CPUQuotaToGOMAXPROCS(minValue int, round func(v float64) int) (int, CPUQuotaStatus, error) { + if round == nil { + round = DefaultRoundFunc + } + cgroups, err := _newQueryer() + if err != nil { + return -1, CPUQuotaUndefined, err + } + + quota, defined, err := cgroups.CPUQuota() + if !defined || err != nil { + return -1, CPUQuotaUndefined, err + } + + maxProcs := round(quota) + if minValue > 0 && maxProcs < minValue { + return minValue, CPUQuotaMinUsed, nil + } + return maxProcs, CPUQuotaUsed, nil +} + +type queryer interface { + CPUQuota() (float64, bool, error) +} + +var ( + _newCgroups2 = cg.NewCGroups2ForCurrentProcess + _newCgroups = cg.NewCGroupsForCurrentProcess + _newQueryer = newQueryer +) + +func newQueryer() (queryer, error) { + cgroups, err := _newCgroups2() + if err == nil { + return cgroups, nil + } + if errors.Is(err, cg.ErrNotV2) { + return _newCgroups() + } + return nil, err +} diff --git a/vendor/go.uber.org/automaxprocs/internal/runtime/cpu_quota_unsupported.go b/vendor/go.uber.org/automaxprocs/internal/runtime/cpu_quota_unsupported.go new file mode 100644 index 00000000000..e74701508ed --- /dev/null +++ b/vendor/go.uber.org/automaxprocs/internal/runtime/cpu_quota_unsupported.go @@ -0,0 +1,31 @@ +// Copyright (c) 2017 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +//go:build !linux +// +build !linux + +package runtime + +// CPUQuotaToGOMAXPROCS converts the CPU quota applied to the calling process +// to a valid GOMAXPROCS value. This is Linux-specific and not supported in the +// current OS. +func CPUQuotaToGOMAXPROCS(_ int, _ func(v float64) int) (int, CPUQuotaStatus, error) { + return -1, CPUQuotaUndefined, nil +} diff --git a/vendor/go.uber.org/automaxprocs/internal/runtime/runtime.go b/vendor/go.uber.org/automaxprocs/internal/runtime/runtime.go new file mode 100644 index 00000000000..f8a2834ac00 --- /dev/null +++ b/vendor/go.uber.org/automaxprocs/internal/runtime/runtime.go @@ -0,0 +1,40 @@ +// Copyright (c) 2017 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package runtime + +import "math" + +// CPUQuotaStatus presents the status of how CPU quota is used +type CPUQuotaStatus int + +const ( + // CPUQuotaUndefined is returned when CPU quota is undefined + CPUQuotaUndefined CPUQuotaStatus = iota + // CPUQuotaUsed is returned when a valid CPU quota can be used + CPUQuotaUsed + // CPUQuotaMinUsed is returned when CPU quota is smaller than the min value + CPUQuotaMinUsed +) + +// DefaultRoundFunc is the default function to convert CPU quota from float to int. It rounds the value down (floor). +func DefaultRoundFunc(v float64) int { + return int(math.Floor(v)) +} diff --git a/vendor/go.uber.org/automaxprocs/maxprocs/maxprocs.go b/vendor/go.uber.org/automaxprocs/maxprocs/maxprocs.go new file mode 100644 index 00000000000..e561fe60b20 --- /dev/null +++ b/vendor/go.uber.org/automaxprocs/maxprocs/maxprocs.go @@ -0,0 +1,139 @@ +// Copyright (c) 2017 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +// Package maxprocs lets Go programs easily configure runtime.GOMAXPROCS to +// match the configured Linux CPU quota. Unlike the top-level automaxprocs +// package, it lets the caller configure logging and handle errors. +package maxprocs // import "go.uber.org/automaxprocs/maxprocs" + +import ( + "os" + "runtime" + + iruntime "go.uber.org/automaxprocs/internal/runtime" +) + +const _maxProcsKey = "GOMAXPROCS" + +func currentMaxProcs() int { + return runtime.GOMAXPROCS(0) +} + +type config struct { + printf func(string, ...interface{}) + procs func(int, func(v float64) int) (int, iruntime.CPUQuotaStatus, error) + minGOMAXPROCS int + roundQuotaFunc func(v float64) int +} + +func (c *config) log(fmt string, args ...interface{}) { + if c.printf != nil { + c.printf(fmt, args...) + } +} + +// An Option alters the behavior of Set. +type Option interface { + apply(*config) +} + +// Logger uses the supplied printf implementation for log output. By default, +// Set doesn't log anything. +func Logger(printf func(string, ...interface{})) Option { + return optionFunc(func(cfg *config) { + cfg.printf = printf + }) +} + +// Min sets the minimum GOMAXPROCS value that will be used. +// Any value below 1 is ignored. +func Min(n int) Option { + return optionFunc(func(cfg *config) { + if n >= 1 { + cfg.minGOMAXPROCS = n + } + }) +} + +// RoundQuotaFunc sets the function that will be used to covert the CPU quota from float to int. +func RoundQuotaFunc(rf func(v float64) int) Option { + return optionFunc(func(cfg *config) { + cfg.roundQuotaFunc = rf + }) +} + +type optionFunc func(*config) + +func (of optionFunc) apply(cfg *config) { of(cfg) } + +// Set GOMAXPROCS to match the Linux container CPU quota (if any), returning +// any error encountered and an undo function. +// +// Set is a no-op on non-Linux systems and in Linux environments without a +// configured CPU quota. +func Set(opts ...Option) (func(), error) { + cfg := &config{ + procs: iruntime.CPUQuotaToGOMAXPROCS, + roundQuotaFunc: iruntime.DefaultRoundFunc, + minGOMAXPROCS: 1, + } + for _, o := range opts { + o.apply(cfg) + } + + undoNoop := func() { + cfg.log("maxprocs: No GOMAXPROCS change to reset") + } + + // Honor the GOMAXPROCS environment variable if present. Otherwise, amend + // `runtime.GOMAXPROCS()` with the current process' CPU quota if the OS is + // Linux, and guarantee a minimum value of 1. The minimum guaranteed value + // can be overridden using `maxprocs.Min()`. + if max, exists := os.LookupEnv(_maxProcsKey); exists { + cfg.log("maxprocs: Honoring GOMAXPROCS=%q as set in environment", max) + return undoNoop, nil + } + + maxProcs, status, err := cfg.procs(cfg.minGOMAXPROCS, cfg.roundQuotaFunc) + if err != nil { + return undoNoop, err + } + + if status == iruntime.CPUQuotaUndefined { + cfg.log("maxprocs: Leaving GOMAXPROCS=%v: CPU quota undefined", currentMaxProcs()) + return undoNoop, nil + } + + prev := currentMaxProcs() + undo := func() { + cfg.log("maxprocs: Resetting GOMAXPROCS to %v", prev) + runtime.GOMAXPROCS(prev) + } + + switch status { + case iruntime.CPUQuotaMinUsed: + cfg.log("maxprocs: Updating GOMAXPROCS=%v: using minimum allowed GOMAXPROCS", maxProcs) + case iruntime.CPUQuotaUsed: + cfg.log("maxprocs: Updating GOMAXPROCS=%v: determined from CPU quota", maxProcs) + } + + runtime.GOMAXPROCS(maxProcs) + return undo, nil +} diff --git a/vendor/go.uber.org/automaxprocs/maxprocs/version.go b/vendor/go.uber.org/automaxprocs/maxprocs/version.go new file mode 100644 index 00000000000..cc7fc5aee12 --- /dev/null +++ b/vendor/go.uber.org/automaxprocs/maxprocs/version.go @@ -0,0 +1,24 @@ +// Copyright (c) 2017 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package maxprocs + +// Version is the current package version. +const Version = "1.6.0" diff --git a/vendor/modules.txt b/vendor/modules.txt index fcc15c37780..9ea5e6605ec 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -1326,6 +1326,12 @@ go.opentelemetry.io/proto/otlp/trace/v1 # go.uber.org/atomic v1.11.0 ## explicit; go 1.18 go.uber.org/atomic +# go.uber.org/automaxprocs v1.6.0 +## explicit; go 1.20 +go.uber.org/automaxprocs +go.uber.org/automaxprocs/internal/cgroups +go.uber.org/automaxprocs/internal/runtime +go.uber.org/automaxprocs/maxprocs # go.uber.org/goleak v1.3.0 ## explicit; go 1.20 go.uber.org/goleak From 56f8e57a58cce7b13f082a1a6319b8320cfbb0ed Mon Sep 17 00:00:00 2001 From: Justin Jung Date: Tue, 25 Mar 2025 13:56:35 -0700 Subject: [PATCH 04/29] nit Signed-off-by: Justin Jung --- cmd/cortex/main.go | 1 + docs/configuration/config-file-reference.md | 4 ++-- pkg/configs/resources.go | 4 ++-- pkg/util/resource/monitor.go | 1 - 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/cmd/cortex/main.go b/cmd/cortex/main.go index a5742d0f2ed..c2702d1338a 100644 --- a/cmd/cortex/main.go +++ b/cmd/cortex/main.go @@ -18,6 +18,7 @@ import ( "github.com/prometheus/client_golang/prometheus" collectorversion "github.com/prometheus/client_golang/prometheus/collectors/version" "github.com/prometheus/common/version" + _ "go.uber.org/automaxprocs" "gopkg.in/yaml.v2" "github.com/cortexproject/cortex/pkg/cortex" diff --git a/docs/configuration/config-file-reference.md b/docs/configuration/config-file-reference.md index 691ace2c503..2d54d4430ff 100644 --- a/docs/configuration/config-file-reference.md +++ b/docs/configuration/config-file-reference.md @@ -272,11 +272,11 @@ query_scheduler: [connect_timeout: | default = 5s] resource_thresholds: - # Utilization threshold for CPU, between 0 and 1. 0 to disable. + # Utilization threshold for CPU in percentage, between 0 and 1. 0 to disable. # CLI flag: -resource-thresholds.cpu [cpu: | default = 0] - # Utilization threshold for heap, between 0 and 1. 0 to disable. + # Utilization threshold for heap in percentage, between 0 and 1. 0 to disable. # CLI flag: -resource-thresholds.heap [heap: | default = 0] diff --git a/pkg/configs/resources.go b/pkg/configs/resources.go index f36bd952080..50ab6c4bc37 100644 --- a/pkg/configs/resources.go +++ b/pkg/configs/resources.go @@ -13,8 +13,8 @@ type Resources struct { } func (cfg *Resources) RegisterFlags(f *flag.FlagSet) { - f.Float64Var(&cfg.CPU, "resource-thresholds.cpu", 0, "Utilization threshold for CPU, between 0 and 1. 0 to disable.") - f.Float64Var(&cfg.Heap, "resource-thresholds.heap", 0, "Utilization threshold for heap, between 0 and 1. 0 to disable.") + f.Float64Var(&cfg.CPU, "resource-thresholds.cpu", 0, "Utilization threshold for CPU in percentage, between 0 and 1. 0 to disable.") + f.Float64Var(&cfg.Heap, "resource-thresholds.heap", 0, "Utilization threshold for heap in percentage, between 0 and 1. 0 to disable.") } func (cfg *Resources) Validate() error { diff --git a/pkg/util/resource/monitor.go b/pkg/util/resource/monitor.go index fb51356359c..a6197500bd4 100644 --- a/pkg/util/resource/monitor.go +++ b/pkg/util/resource/monitor.go @@ -15,7 +15,6 @@ import ( "github.com/prometheus/client_golang/prometheus/promauto" "github.com/prometheus/procfs" "github.com/weaveworks/common/httpgrpc" - _ "go.uber.org/automaxprocs" "github.com/cortexproject/cortex/pkg/configs" "github.com/cortexproject/cortex/pkg/util/services" From 9efbbd9254dc4093ec03ab2208279aca9996bcfe Mon Sep 17 00:00:00 2001 From: Justin Jung Date: Wed, 26 Mar 2025 08:10:06 -0700 Subject: [PATCH 05/29] Add test for monitor Signed-off-by: Justin Jung --- pkg/cortex/modules.go | 16 ++++- pkg/util/resource/monitor.go | 37 ++++++----- pkg/util/resource/monitor_test.go | 104 ++++++++++++++++++++++++++++++ 3 files changed, 136 insertions(+), 21 deletions(-) diff --git a/pkg/cortex/modules.go b/pkg/cortex/modules.go index 8d8d6d920c5..e30e5d058bc 100644 --- a/pkg/cortex/modules.go +++ b/pkg/cortex/modules.go @@ -6,6 +6,8 @@ import ( "fmt" "log/slog" "net/http" + "runtime" + "runtime/debug" "github.com/go-kit/log/level" "github.com/opentracing-contrib/go-stdlib/nethttp" @@ -27,6 +29,7 @@ import ( "github.com/cortexproject/cortex/pkg/alertmanager/alertstore" "github.com/cortexproject/cortex/pkg/api" "github.com/cortexproject/cortex/pkg/compactor" + "github.com/cortexproject/cortex/pkg/configs" configAPI "github.com/cortexproject/cortex/pkg/configs/api" "github.com/cortexproject/cortex/pkg/configs/db" "github.com/cortexproject/cortex/pkg/distributor" @@ -772,8 +775,17 @@ func (t *Cortex) initResourceMonitor() (services.Service, error) { return nil, nil } - var err error - t.ResourceMonitor, err = resource.NewMonitor(t.Cfg.ResourceThresholds, prometheus.DefaultRegisterer) + scanner, err := resource.NewScanner() + if err != nil { + return nil, err + } + + limits := configs.Resources{ + CPU: float64(runtime.GOMAXPROCS(0)), + Heap: float64(debug.SetMemoryLimit(-1)), + } + + t.ResourceMonitor, err = resource.NewMonitor(t.Cfg.ResourceThresholds, limits, scanner, prometheus.DefaultRegisterer) return t.ResourceMonitor, err } diff --git a/pkg/util/resource/monitor.go b/pkg/util/resource/monitor.go index a6197500bd4..ab6a2d267ce 100644 --- a/pkg/util/resource/monitor.go +++ b/pkg/util/resource/monitor.go @@ -4,8 +4,6 @@ import ( "context" "fmt" "net/http" - "runtime" - "runtime/debug" "runtime/metrics" "sync" "time" @@ -26,8 +24,6 @@ func (e *ExhaustedError) Error() string { return "resource exhausted" } -var ErrResourceExhausted = httpgrpc.Errorf(http.StatusTooManyRequests, "resource exhausted") - const heapMetricName = "/memory/classes/heap/objects:bytes" const monitorInterval = time.Second const dataPointsToAvg = 30 @@ -109,23 +105,18 @@ type Monitor struct { lock sync.RWMutex } -func NewMonitor(thresholds configs.Resources, registerer prometheus.Registerer) (*Monitor, error) { +func NewMonitor(thresholds configs.Resources, limits configs.Resources, scanner IScanner, registerer prometheus.Registerer) (*Monitor, error) { m := &Monitor{ - thresholds: thresholds, - lock: sync.RWMutex{}, - } + thresholds: thresholds, + containerLimit: limits, + scanner: scanner, - m.containerLimit.CPU = float64(runtime.GOMAXPROCS(0)) - m.containerLimit.Heap = float64(debug.SetMemoryLimit(-1)) + cpuRates: make([]float64, dataPointsToAvg), + cpuIntervals: make([]float64, dataPointsToAvg), - var err error - m.scanner, err = NewScanner() - if err != nil { - return nil, err + lock: sync.RWMutex{}, } - m.cpuRates = make([]float64, dataPointsToAvg) - m.cpuIntervals = make([]float64, dataPointsToAvg) m.Service = services.NewBasicService(nil, m.running, nil) promauto.With(registerer).NewGaugeFunc(prometheus.GaugeOpts{ @@ -175,6 +166,12 @@ func (m *Monitor) storeCPUUtilization(stats Stats) { now := time.Now() + if m.lastUpdate.IsZero() { + m.lastCPU = stats.cpu + m.lastUpdate = now + return + } + m.totalCPU -= m.cpuRates[m.index] m.totalInterval -= m.cpuIntervals[m.index] @@ -197,7 +194,9 @@ func (m *Monitor) storeHeapUtilization(stats Stats) { m.lock.Lock() defer m.lock.Unlock() - m.utilization.Heap = float64(stats.heap) / m.containerLimit.Heap + if m.containerLimit.Heap > 0 { + m.utilization.Heap = float64(stats.heap) / m.containerLimit.Heap + } } func (m *Monitor) GetCPUUtilization() float64 { @@ -220,12 +219,12 @@ func (m *Monitor) CheckResourceUtilization() (string, float64, float64, error) { if m.thresholds.CPU > 0 && cpu > m.thresholds.CPU { err := ExhaustedError{} - return "cpu", m.thresholds.CPU, cpu, httpgrpc.Errorf(http.StatusTooManyRequests, err.Error()) + return "cpu", m.thresholds.CPU, cpu, httpgrpc.Errorf(http.StatusTooManyRequests, "%s", err.Error()) } if m.thresholds.Heap > 0 && heap > m.thresholds.Heap { err := ExhaustedError{} - return "heap", m.thresholds.Heap, heap, httpgrpc.Errorf(http.StatusTooManyRequests, err.Error()) + return "heap", m.thresholds.Heap, heap, httpgrpc.Errorf(http.StatusTooManyRequests, "%s", err.Error()) } return "", 0, 0, nil diff --git a/pkg/util/resource/monitor_test.go b/pkg/util/resource/monitor_test.go index 958e3540f76..b7683bb06dc 100644 --- a/pkg/util/resource/monitor_test.go +++ b/pkg/util/resource/monitor_test.go @@ -1 +1,105 @@ package resource + +import ( + "context" + "fmt" + "testing" + "time" + + "github.com/stretchr/testify/require" + + "github.com/cortexproject/cortex/pkg/configs" + "github.com/cortexproject/cortex/pkg/util/services" +) + +func TestMonitor(t *testing.T) { + type test struct { + limit float64 + val uint64 + threshold float64 + utilization float64 + throwErr bool + } + + tests := map[string]test{ + "should not throw error if below threshold": { + limit: 10, + val: 1, + utilization: 0.1, + threshold: 0.2, + throwErr: false, + }, + "should throw error if above threshold": { + limit: 10, + val: 5, + utilization: 0.5, + threshold: 0.2, + throwErr: true, + }, + "should not throw error if limit is 0": { + limit: 0, + val: 5, + utilization: 0, + threshold: 0.2, + throwErr: false, + }, + } + + for _, tc := range tests { + limits := configs.Resources{Heap: tc.limit} + thresholds := configs.Resources{Heap: tc.threshold} + scanner := mockScanner{Heap: tc.val} + + monitor, err := NewMonitor(thresholds, limits, &scanner, nil) + require.NoError(t, err) + require.NoError(t, monitor.StartAsync(context.Background())) + t.Cleanup(func() { + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), monitor)) + }) + + time.Sleep(2 * time.Second) // let scanner store values + + require.Equal(t, tc.utilization, monitor.GetHeapUtilization()) + _, _, _, err = monitor.CheckResourceUtilization() + + if tc.throwErr { + exhaustedErr := &ExhaustedError{} + require.ErrorContains(t, err, exhaustedErr.Error()) + } else { + require.NoError(t, err) + } + } +} + +func TestMonitor_GetCPUUtilization(t *testing.T) { + limits := configs.Resources{CPU: 10} + thresholds := configs.Resources{} + scanner := mockScanner{CPU: 0} + + monitor, err := NewMonitor(thresholds, limits, &scanner, nil) + require.NoError(t, err) + require.NoError(t, monitor.StartAsync(context.Background())) + t.Cleanup(func() { + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), monitor)) + }) + + for i := 0; i < 10; i++ { + scanner.CPU++ + time.Sleep(time.Second) // let scanner store values + } + + fmt.Println(monitor.GetCPUUtilization()) + require.InDelta(t, 0.1, monitor.GetCPUUtilization(), 1e-5) +} + +type mockScanner struct { + CPU float64 + Heap uint64 +} + +func (m *mockScanner) Scan() (Stats, error) { + return Stats{ + cpu: m.CPU, + heap: m.Heap, + }, nil +} From 30bbd3dfc0e8fc403a65d16192806a2f136a931f Mon Sep 17 00:00:00 2001 From: Justin Jung Date: Wed, 26 Mar 2025 09:46:55 -0700 Subject: [PATCH 06/29] fix tests Signed-off-by: Justin Jung --- pkg/frontend/transport/handler.go | 60 ++++++++++++++++--------------- pkg/util/resource/monitor_test.go | 22 ------------ 2 files changed, 31 insertions(+), 51 deletions(-) diff --git a/pkg/frontend/transport/handler.go b/pkg/frontend/transport/handler.go index e73419ac1a3..045a1b95059 100644 --- a/pkg/frontend/transport/handler.go +++ b/pkg/frontend/transport/handler.go @@ -490,37 +490,39 @@ func (f *Handler) reportQueryStats(r *http.Request, source, userID string, query } } - // We are unable to use errors.As to compare since body string from the http response is wrapped as an error var reason string - errMsg := error.Error() - if statusCode == http.StatusTooManyRequests { - var resourceExhaustedErr resource.ExhaustedError - if strings.Contains(errMsg, resourceExhaustedErr.Error()) { - reason = reasonResourceExhausted - } else { - reason = reasonTooManyRequests - } - } else if statusCode == http.StatusRequestEntityTooLarge { + if statusCode == http.StatusRequestEntityTooLarge { reason = reasonResponseBodySizeExceeded - } else if statusCode == http.StatusUnprocessableEntity { - if strings.Contains(errMsg, limitTooManySamples) { - reason = reasonTooManySamples - } else if strings.Contains(errMsg, limitTimeRangeExceeded) { - reason = reasonTimeRangeExceeded - } else if strings.Contains(errMsg, limitSeriesFetched) { - reason = reasonSeriesFetched - } else if strings.Contains(errMsg, limitChunksFetched) { - reason = reasonChunksFetched - } else if strings.Contains(errMsg, limitChunkBytesFetched) { - reason = reasonChunkBytesFetched - } else if strings.Contains(errMsg, limitDataBytesFetched) { - reason = reasonDataBytesFetched - } else if strings.Contains(errMsg, limitSeriesStoreGateway) { - reason = reasonSeriesLimitStoreGateway - } else if strings.Contains(errMsg, limitChunksStoreGateway) { - reason = reasonChunksLimitStoreGateway - } else if strings.Contains(errMsg, limitBytesStoreGateway) { - reason = reasonBytesLimitStoreGateway + } else if error != nil { + // We are unable to use errors.As to compare since body string from the http response is wrapped as an error + errMsg := error.Error() + if statusCode == http.StatusTooManyRequests { + var resourceExhaustedErr resource.ExhaustedError + if strings.Contains(errMsg, resourceExhaustedErr.Error()) { + reason = reasonResourceExhausted + } else { + reason = reasonTooManyRequests + } + } else if statusCode == http.StatusUnprocessableEntity { + if strings.Contains(errMsg, limitTooManySamples) { + reason = reasonTooManySamples + } else if strings.Contains(errMsg, limitTimeRangeExceeded) { + reason = reasonTimeRangeExceeded + } else if strings.Contains(errMsg, limitSeriesFetched) { + reason = reasonSeriesFetched + } else if strings.Contains(errMsg, limitChunksFetched) { + reason = reasonChunksFetched + } else if strings.Contains(errMsg, limitChunkBytesFetched) { + reason = reasonChunkBytesFetched + } else if strings.Contains(errMsg, limitDataBytesFetched) { + reason = reasonDataBytesFetched + } else if strings.Contains(errMsg, limitSeriesStoreGateway) { + reason = reasonSeriesLimitStoreGateway + } else if strings.Contains(errMsg, limitChunksStoreGateway) { + reason = reasonChunksLimitStoreGateway + } else if strings.Contains(errMsg, limitBytesStoreGateway) { + reason = reasonBytesLimitStoreGateway + } } } if len(reason) > 0 { diff --git a/pkg/util/resource/monitor_test.go b/pkg/util/resource/monitor_test.go index b7683bb06dc..5d79ac56c0d 100644 --- a/pkg/util/resource/monitor_test.go +++ b/pkg/util/resource/monitor_test.go @@ -2,7 +2,6 @@ package resource import ( "context" - "fmt" "testing" "time" @@ -71,27 +70,6 @@ func TestMonitor(t *testing.T) { } } -func TestMonitor_GetCPUUtilization(t *testing.T) { - limits := configs.Resources{CPU: 10} - thresholds := configs.Resources{} - scanner := mockScanner{CPU: 0} - - monitor, err := NewMonitor(thresholds, limits, &scanner, nil) - require.NoError(t, err) - require.NoError(t, monitor.StartAsync(context.Background())) - t.Cleanup(func() { - require.NoError(t, services.StopAndAwaitTerminated(context.Background(), monitor)) - }) - - for i := 0; i < 10; i++ { - scanner.CPU++ - time.Sleep(time.Second) // let scanner store values - } - - fmt.Println(monitor.GetCPUUtilization()) - require.InDelta(t, 0.1, monitor.GetCPUUtilization(), 1e-5) -} - type mockScanner struct { CPU float64 Heap uint64 From fa56e65b4438afa3399de54e77d0bf99ee7ada7d Mon Sep 17 00:00:00 2001 From: Justin Jung Date: Wed, 26 Mar 2025 10:01:33 -0700 Subject: [PATCH 07/29] changelog Signed-off-by: Justin Jung --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 1cf3e1ed8d9..0335e4e3f68 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -6,6 +6,7 @@ * [FEATURE] Querier/Ruler: Add `query_partial_data` and `rules_partial_data` limits to allow queries/rules to be evaluated with data from a single zone, if other zones are not available. #6526 * [FEATURE] Update prometheus alertmanager version to v0.28.0 and add new integration msteamsv2, jira, and rocketchat. #6590 * [FEATURE] Ingester: Add a `-ingester.enable-ooo-native-histograms` flag to enable out-of-order native histogram ingestion per tenant. It only takes effect when `-blocks-storage.tsdb.enable-native-histograms=true` and `-ingester.out-of-order-time-window` > 0. It is applied after the restart if it is changed at runtime through the runtime config. #6626 +* [FEATURE] Ingester/StoreGateway: Add `resource-thresholds` in ingesters and store gateways to throttle query requests when the pods are under resource pressure. #6674 * [ENHANCEMENT] Querier: limit label APIs to query only ingesters if `start` param is not been specified. #6618 * [ENHANCEMENT] Alertmanager: Add new limits `-alertmanager.max-silences-count` and `-alertmanager.max-silences-size-bytes` for limiting silences per tenant. #6605 * [ENHANCEMENT] Update prometheus version to v3.1.0. #6583 From 5cccd60e4c049bf52d457492c5413e0bbed1f2df Mon Sep 17 00:00:00 2001 From: Justin Jung Date: Wed, 26 Mar 2025 14:24:00 -0700 Subject: [PATCH 08/29] fix test Signed-off-by: Justin Jung --- pkg/ingester/ingester.go | 1 + pkg/storegateway/gateway.go | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/pkg/ingester/ingester.go b/pkg/ingester/ingester.go index bf0cdfe3d4b..e325f1c5fe1 100644 --- a/pkg/ingester/ingester.go +++ b/pkg/ingester/ingester.go @@ -2156,6 +2156,7 @@ func (i *Ingester) trackInflightQueryRequest() (func(), error) { i.maxInflightQueryRequests.Track(i.inflightQueryRequests.Inc()) + //if _, ok := i.resourceMonitor.(*resource.Monitor); ok { if i.resourceMonitor != nil { if resourceName, threshold, utilization, err := i.resourceMonitor.CheckResourceUtilization(); err != nil { level.Warn(i.logger).Log("msg", "resource threshold breached", "resource", resourceName, "threshold", threshold, "utilization", utilization) diff --git a/pkg/storegateway/gateway.go b/pkg/storegateway/gateway.go index f2e4e140923..348c74793ee 100644 --- a/pkg/storegateway/gateway.go +++ b/pkg/storegateway/gateway.go @@ -408,7 +408,7 @@ func (g *StoreGateway) LabelValues(ctx context.Context, req *storepb.LabelValues } func (g *StoreGateway) checkResourceUtilization() error { - if g.resourceMonitor == nil { + if _, ok := g.resourceMonitor.(*resource.Monitor); !ok { return nil } From 6e37330d52929a9aa456f98dc4549ed5d4927145 Mon Sep 17 00:00:00 2001 From: Justin Jung Date: Wed, 26 Mar 2025 15:46:30 -0700 Subject: [PATCH 09/29] remove interface Signed-off-by: Justin Jung --- pkg/ingester/ingester.go | 5 ++-- pkg/ingester/ingester_test.go | 43 +++++++++++++++++-------------- pkg/storegateway/gateway.go | 6 ++--- pkg/storegateway/gateway_test.go | 39 +++++++++++++++------------- pkg/util/resource/monitor.go | 36 +++++++++++--------------- pkg/util/resource/monitor_test.go | 4 +-- 6 files changed, 67 insertions(+), 66 deletions(-) diff --git a/pkg/ingester/ingester.go b/pkg/ingester/ingester.go index e325f1c5fe1..416d7420fff 100644 --- a/pkg/ingester/ingester.go +++ b/pkg/ingester/ingester.go @@ -232,7 +232,7 @@ type Ingester struct { lifecycler *ring.Lifecycler limits *validation.Overrides limiter *Limiter - resourceMonitor resource.IMonitor + resourceMonitor *resource.Monitor subservicesWatcher *services.FailureWatcher stoppedMtx sync.RWMutex // protects stopped @@ -701,7 +701,7 @@ func newTSDBState(bucketClient objstore.Bucket, registerer prometheus.Registerer } // New returns a new Ingester that uses Cortex block storage instead of chunks storage. -func New(cfg Config, limits *validation.Overrides, registerer prometheus.Registerer, logger log.Logger, resourceMonitor resource.IMonitor) (*Ingester, error) { +func New(cfg Config, limits *validation.Overrides, registerer prometheus.Registerer, logger log.Logger, resourceMonitor *resource.Monitor) (*Ingester, error) { defaultInstanceLimits = &cfg.DefaultLimits if cfg.ingesterClientFactory == nil { cfg.ingesterClientFactory = client.MakeIngesterClient @@ -2156,7 +2156,6 @@ func (i *Ingester) trackInflightQueryRequest() (func(), error) { i.maxInflightQueryRequests.Track(i.inflightQueryRequests.Inc()) - //if _, ok := i.resourceMonitor.(*resource.Monitor); ok { if i.resourceMonitor != nil { if resourceName, threshold, utilization, err := i.resourceMonitor.CheckResourceUtilization(); err != nil { level.Warn(i.logger).Log("msg", "resource threshold breached", "resource", resourceName, "threshold", threshold, "utilization", utilization) diff --git a/pkg/ingester/ingester_test.go b/pkg/ingester/ingester_test.go index bbf5547ef7c..4d75bc8372a 100644 --- a/pkg/ingester/ingester_test.go +++ b/pkg/ingester/ingester_test.go @@ -49,6 +49,7 @@ import ( "github.com/cortexproject/cortex/pkg/chunk" "github.com/cortexproject/cortex/pkg/chunk/encoding" + "github.com/cortexproject/cortex/pkg/configs" "github.com/cortexproject/cortex/pkg/cortexpb" "github.com/cortexproject/cortex/pkg/ingester/client" "github.com/cortexproject/cortex/pkg/querier/batch" @@ -3068,11 +3069,14 @@ func Test_Ingester_Query_ResourceThresholdBreached(t *testing.T) { {labels.Labels{{Name: labels.MetricName, Value: "test_1"}, {Name: "route", Value: "get_user"}, {Name: "status", Value: "200"}}, 1, 100000}, } - mockErr := fmt.Errorf("resource monitor error") - resourceMonitor := testResourceMonitor{ - err: mockErr, - } - i, err := prepareIngesterWithResourceMonitor(t, &resourceMonitor) + thresholds := configs.Resources{Heap: 0.1} + limits := configs.Resources{Heap: 10} + resourceMonitor, err := resource.NewMonitor(thresholds, limits, &mockResourceScanner{ + heap: uint64(5), + }, prometheus.NewRegistry()) + require.NoError(t, err) + + i, err := prepareIngesterWithResourceMonitor(t, resourceMonitor) require.NoError(t, err) require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) defer services.StopAndAwaitTerminated(context.Background(), i) //nolint:errcheck @@ -3095,23 +3099,18 @@ func Test_Ingester_Query_ResourceThresholdBreached(t *testing.T) { s := &mockQueryStreamServer{ctx: ctx} err = i.QueryStream(rreq, s) require.Error(t, err) - require.ErrorContains(t, err, mockErr.Error()) + exhaustedErr := resource.ExhaustedError{} + require.ErrorContains(t, err, exhaustedErr.Error()) } -type testResourceMonitor struct { - err error +type mockResourceScanner struct { + heap uint64 } -func (t *testResourceMonitor) GetCPUUtilization() float64 { - return 0 -} - -func (t *testResourceMonitor) GetHeapUtilization() float64 { - return 0 -} - -func (t *testResourceMonitor) CheckResourceUtilization() (string, float64, float64, error) { - return "", 0, 0, t.err +func (m *mockResourceScanner) Scan() (resource.Stats, error) { + return resource.Stats{ + Heap: m.heap, + }, nil } func TestIngester_LabelValues_ShouldNotCreateTSDBIfDoesNotExists(t *testing.T) { @@ -3997,7 +3996,7 @@ func prepareIngesterWithBlocksStorageAndLimits(t testing.TB, ingesterCfg Config, return ingester, nil } -func prepareIngesterWithResourceMonitor(t testing.TB, resourceMonitor resource.IMonitor) (*Ingester, error) { +func prepareIngesterWithResourceMonitor(t testing.TB, resourceMonitor *resource.Monitor) (*Ingester, error) { dataDir := t.TempDir() bucketDir := t.TempDir() @@ -4008,6 +4007,12 @@ func prepareIngesterWithResourceMonitor(t testing.TB, resourceMonitor resource.I overrides, _ := validation.NewOverrides(defaultLimitsTestConfig(), nil) + err := resourceMonitor.StartAsync(context.Background()) + if err != nil { + return nil, err + } + time.Sleep(time.Second) + ingester, err := New(ingesterCfg, overrides, prometheus.NewRegistry(), log.NewNopLogger(), resourceMonitor) if err != nil { return nil, err diff --git a/pkg/storegateway/gateway.go b/pkg/storegateway/gateway.go index 348c74793ee..848bc35548c 100644 --- a/pkg/storegateway/gateway.go +++ b/pkg/storegateway/gateway.go @@ -118,7 +118,7 @@ type StoreGateway struct { subservices *services.Manager subservicesWatcher *services.FailureWatcher - resourceMonitor resource.IMonitor + resourceMonitor *resource.Monitor bucketSync *prometheus.CounterVec } @@ -146,7 +146,7 @@ func NewStoreGateway(gatewayCfg Config, storageCfg cortex_tsdb.BlocksStorageConf return newStoreGateway(gatewayCfg, storageCfg, bucketClient, ringStore, limits, logLevel, logger, reg, resourceMonitor) } -func newStoreGateway(gatewayCfg Config, storageCfg cortex_tsdb.BlocksStorageConfig, bucketClient objstore.InstrumentedBucket, ringStore kv.Client, limits *validation.Overrides, logLevel logging.Level, logger log.Logger, reg prometheus.Registerer, resourceMonitor resource.IMonitor) (*StoreGateway, error) { +func newStoreGateway(gatewayCfg Config, storageCfg cortex_tsdb.BlocksStorageConfig, bucketClient objstore.InstrumentedBucket, ringStore kv.Client, limits *validation.Overrides, logLevel logging.Level, logger log.Logger, reg prometheus.Registerer, resourceMonitor *resource.Monitor) (*StoreGateway, error) { var err error g := &StoreGateway{ @@ -408,7 +408,7 @@ func (g *StoreGateway) LabelValues(ctx context.Context, req *storepb.LabelValues } func (g *StoreGateway) checkResourceUtilization() error { - if _, ok := g.resourceMonitor.(*resource.Monitor); !ok { + if g.resourceMonitor == nil { return nil } diff --git a/pkg/storegateway/gateway_test.go b/pkg/storegateway/gateway_test.go index 1524008b818..4bc21e64dc6 100644 --- a/pkg/storegateway/gateway_test.go +++ b/pkg/storegateway/gateway_test.go @@ -32,6 +32,7 @@ import ( "github.com/thanos-io/thanos/pkg/store/storepb" "google.golang.org/grpc/status" + "github.com/cortexproject/cortex/pkg/configs" "github.com/cortexproject/cortex/pkg/ring" "github.com/cortexproject/cortex/pkg/ring/kv/consul" "github.com/cortexproject/cortex/pkg/storage/bucket" @@ -41,6 +42,7 @@ import ( cortex_testutil "github.com/cortexproject/cortex/pkg/storage/tsdb/testutil" "github.com/cortexproject/cortex/pkg/util" "github.com/cortexproject/cortex/pkg/util/flagext" + "github.com/cortexproject/cortex/pkg/util/resource" "github.com/cortexproject/cortex/pkg/util/services" "github.com/cortexproject/cortex/pkg/util/test" "github.com/cortexproject/cortex/pkg/util/validation" @@ -1209,12 +1211,18 @@ func TestStoreGateway_SeriesThrottledByResourceMonitor(t *testing.T) { gatewayCfg.ShardingEnabled = false storageCfg := mockStorageConfig(t) - mockErr := fmt.Errorf("resource monitor error") - resourceMonitor := testResourceMonitor{ - err: mockErr, - } + thresholds := configs.Resources{Heap: 0.1} + limits := configs.Resources{Heap: 10} + resourceMonitor, err := resource.NewMonitor(thresholds, limits, &mockResourceScanner{ + heap: uint64(5), + }, prometheus.NewRegistry()) + require.NoError(t, err) + + err = resourceMonitor.StartAsync(context.Background()) + require.NoError(t, err) + time.Sleep(time.Second) - g, err := newStoreGateway(gatewayCfg, storageCfg, objstore.WithNoopInstr(bucketClient), nil, overrides, mockLoggingLevel(), logger, nil, &resourceMonitor) + g, err := newStoreGateway(gatewayCfg, storageCfg, objstore.WithNoopInstr(bucketClient), nil, overrides, mockLoggingLevel(), logger, nil, resourceMonitor) require.NoError(t, err) require.NoError(t, services.StartAndAwaitRunning(ctx, g)) defer services.StopAndAwaitTerminated(ctx, g) //nolint:errcheck @@ -1222,23 +1230,18 @@ func TestStoreGateway_SeriesThrottledByResourceMonitor(t *testing.T) { srv := newBucketStoreSeriesServer(setUserIDToGRPCContext(ctx, userID)) err = g.Series(req, srv) require.Error(t, err) - require.ErrorContains(t, err, mockErr.Error()) -} - -type testResourceMonitor struct { - err error -} - -func (t *testResourceMonitor) GetCPUUtilization() float64 { - return 0 + exhaustedErr := resource.ExhaustedError{} + require.ErrorContains(t, err, exhaustedErr.Error()) } -func (t *testResourceMonitor) GetHeapUtilization() float64 { - return 0 +type mockResourceScanner struct { + heap uint64 } -func (t *testResourceMonitor) CheckResourceUtilization() (string, float64, float64, error) { - return "", 0, 0, t.err +func (m *mockResourceScanner) Scan() (resource.Stats, error) { + return resource.Stats{ + Heap: m.heap, + }, nil } func mockGatewayConfig() Config { diff --git a/pkg/util/resource/monitor.go b/pkg/util/resource/monitor.go index ab6a2d267ce..7ea4e4793bb 100644 --- a/pkg/util/resource/monitor.go +++ b/pkg/util/resource/monitor.go @@ -24,7 +24,7 @@ func (e *ExhaustedError) Error() string { return "resource exhausted" } -const heapMetricName = "/memory/classes/heap/objects:bytes" +const heapMetricName = "/memory/classes/Heap/objects:bytes" const monitorInterval = time.Second const dataPointsToAvg = 30 @@ -38,8 +38,8 @@ type Scanner struct { } type Stats struct { - cpu float64 - heap uint64 + CPU float64 + Heap uint64 } func NewScanner() (*Scanner, error) { @@ -73,17 +73,11 @@ func (s *Scanner) Scan() (Stats, error) { metrics.Read(s.metricSamples) return Stats{ - cpu: stat.CPUTime(), - heap: s.metricSamples[0].Value.Uint64(), + CPU: stat.CPUTime(), + Heap: s.metricSamples[0].Value.Uint64(), }, nil } -type IMonitor interface { - GetCPUUtilization() float64 - GetHeapUtilization() float64 - CheckResourceUtilization() (string, float64, float64, error) -} - type Monitor struct { services.Service @@ -121,19 +115,19 @@ func NewMonitor(thresholds configs.Resources, limits configs.Resources, scanner promauto.With(registerer).NewGaugeFunc(prometheus.GaugeOpts{ Name: "cortex_resource_utilization", - ConstLabels: map[string]string{"resource": "cpu"}, + ConstLabels: map[string]string{"resource": "CPU"}, }, m.GetCPUUtilization) promauto.With(registerer).NewGaugeFunc(prometheus.GaugeOpts{ Name: "cortex_resource_utilization", - ConstLabels: map[string]string{"resource": "heap"}, + ConstLabels: map[string]string{"resource": "Heap"}, }, m.GetHeapUtilization) promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ Name: "cortex_resource_threshold", - ConstLabels: map[string]string{"resource": "cpu"}, + ConstLabels: map[string]string{"resource": "CPU"}, }).Set(thresholds.CPU) promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ Name: "cortex_resource_threshold", - ConstLabels: map[string]string{"resource": "heap"}, + ConstLabels: map[string]string{"resource": "Heap"}, }).Set(thresholds.Heap) return m, nil @@ -167,7 +161,7 @@ func (m *Monitor) storeCPUUtilization(stats Stats) { now := time.Now() if m.lastUpdate.IsZero() { - m.lastCPU = stats.cpu + m.lastCPU = stats.CPU m.lastUpdate = now return } @@ -175,13 +169,13 @@ func (m *Monitor) storeCPUUtilization(stats Stats) { m.totalCPU -= m.cpuRates[m.index] m.totalInterval -= m.cpuIntervals[m.index] - m.cpuRates[m.index] = stats.cpu - m.lastCPU + m.cpuRates[m.index] = stats.CPU - m.lastCPU m.cpuIntervals[m.index] = now.Sub(m.lastUpdate).Seconds() m.totalCPU += m.cpuRates[m.index] m.totalInterval += m.cpuIntervals[m.index] - m.lastCPU = stats.cpu + m.lastCPU = stats.CPU m.lastUpdate = now m.index = (m.index + 1) % dataPointsToAvg @@ -195,7 +189,7 @@ func (m *Monitor) storeHeapUtilization(stats Stats) { defer m.lock.Unlock() if m.containerLimit.Heap > 0 { - m.utilization.Heap = float64(stats.heap) / m.containerLimit.Heap + m.utilization.Heap = float64(stats.Heap) / m.containerLimit.Heap } } @@ -219,12 +213,12 @@ func (m *Monitor) CheckResourceUtilization() (string, float64, float64, error) { if m.thresholds.CPU > 0 && cpu > m.thresholds.CPU { err := ExhaustedError{} - return "cpu", m.thresholds.CPU, cpu, httpgrpc.Errorf(http.StatusTooManyRequests, "%s", err.Error()) + return "CPU", m.thresholds.CPU, cpu, httpgrpc.Errorf(http.StatusTooManyRequests, "%s", err.Error()) } if m.thresholds.Heap > 0 && heap > m.thresholds.Heap { err := ExhaustedError{} - return "heap", m.thresholds.Heap, heap, httpgrpc.Errorf(http.StatusTooManyRequests, "%s", err.Error()) + return "Heap", m.thresholds.Heap, heap, httpgrpc.Errorf(http.StatusTooManyRequests, "%s", err.Error()) } return "", 0, 0, nil diff --git a/pkg/util/resource/monitor_test.go b/pkg/util/resource/monitor_test.go index 5d79ac56c0d..0c76cb59f94 100644 --- a/pkg/util/resource/monitor_test.go +++ b/pkg/util/resource/monitor_test.go @@ -77,7 +77,7 @@ type mockScanner struct { func (m *mockScanner) Scan() (Stats, error) { return Stats{ - cpu: m.CPU, - heap: m.Heap, + CPU: m.CPU, + Heap: m.Heap, }, nil } From 08a6adf128171774f18a76abe9276e9913c10e58 Mon Sep 17 00:00:00 2001 From: Justin Jung Date: Mon, 31 Mar 2025 15:44:37 -0700 Subject: [PATCH 10/29] address comments Signed-off-by: Justin Jung --- CHANGELOG.md | 1 - docs/configuration/config-file-reference.md | 4 ++ docs/guides/resource-based-throttling.md | 55 +++++++++++++++++++++ pkg/configs/resources.go | 4 +- pkg/cortex/modules.go | 4 ++ pkg/util/resource/monitor.go | 19 +++++-- 6 files changed, 80 insertions(+), 7 deletions(-) create mode 100644 docs/guides/resource-based-throttling.md diff --git a/CHANGELOG.md b/CHANGELOG.md index ba4b696d501..281490846cb 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -5,7 +5,6 @@ * [FEATURE] Query Frontend: Add dynamic interval size for query splitting. This is enabled by configuring experimental flags `querier.max-shards-per-query` and/or `querier.max-fetched-data-duration-per-query`. The split interval size is dynamically increased to maintain a number of shards and total duration fetched below the configured values. #6458 * [FEATURE] Querier/Ruler: Add `query_partial_data` and `rules_partial_data` limits to allow queries/rules to be evaluated with data from a single zone, if other zones are not available. #6526 * [FEATURE] Update prometheus alertmanager version to v0.28.0 and add new integration msteamsv2, jira, and rocketchat. #6590 -* [FEATURE] Ingester: Add a `-ingester.enable-ooo-native-histograms` flag to enable out-of-order native histogram ingestion per tenant. It only takes effect when `-blocks-storage.tsdb.enable-native-histograms=true` and `-ingester.out-of-order-time-window` > 0. It is applied after the restart if it is changed at runtime through the runtime config. #6626 * [FEATURE] Ingester/StoreGateway: Add `resource-thresholds` in ingesters and store gateways to throttle query requests when the pods are under resource pressure. #6674 * [FEATURE] Ingester: Support out-of-order native histogram ingestion. It automatically enabled when `-ingester.out-of-order-time-window > 0` and `-blocks-storage.tsdb.enable-native-histograms=true`. #6626 #6663 * [ENHANCEMENT] Alertmanager: Add nflog and silences maintenance metrics. #6659 diff --git a/docs/configuration/config-file-reference.md b/docs/configuration/config-file-reference.md index 211b19ca634..b6ba3225b07 100644 --- a/docs/configuration/config-file-reference.md +++ b/docs/configuration/config-file-reference.md @@ -273,10 +273,14 @@ query_scheduler: resource_thresholds: # Utilization threshold for CPU in percentage, between 0 and 1. 0 to disable. + # The CPU utilization metric is from github.com/prometheus/procfs, which is a + # close estimate. Applicable to ingesters and store-gateways only. # CLI flag: -resource-thresholds.cpu [cpu: | default = 0] # Utilization threshold for heap in percentage, between 0 and 1. 0 to disable. + # The heap utilization metric is from runtime/metrics, which is a close + # estimate. Applicable to ingesters and store-gateways only. # CLI flag: -resource-thresholds.heap [heap: | default = 0] diff --git a/docs/guides/resource-based-throttling.md b/docs/guides/resource-based-throttling.md new file mode 100644 index 00000000000..71c4f88dc7a --- /dev/null +++ b/docs/guides/resource-based-throttling.md @@ -0,0 +1,55 @@ +--- +title: "Protecting Cortex from Heavy Queries" +linkTitle: "Protecting Cortex from Heavy Queries" +weight: 11 +slug: protecting-cortex-from-heavy-queries +--- + +PromQL is powerful, and is able to result in query requests that have very wide range of data fetched and samples processed. Heavy queries can cause: + +1. CPU on any query component to be partially exhausted, increasing latency and causing incoming queries to queue up with high chance of time-out. +2. CPU on any query component to be fully exhausted, causing GC to slow down leading to the pod being out-of-memory and killed. +3. Heap memory on any query component to be exhausted, leading to the pod being out-of-memory and killed. + +It's important to protect Cortex components by setting appropriate limits and throttling configurations based on your infrastructure and data ingested by the customers. + +## Static limits + +There are number of static limits that you could configure to block heavy queries from running. + +### Max outstanding requests per tenant + +See https://cortexmetrics.io/docs/configuration/configuration-file/#query_frontend_config:~:text=max_outstanding_requests_per_tenant for details. + +### Max data bytes fetched per (sharded) query + +See https://cortexmetrics.io/docs/configuration/configuration-file/#query_frontend_config:~:text=max_fetched_data_bytes_per_query for details. + +### Max series fetched per (sharded) query + +See https://cortexmetrics.io/docs/configuration/configuration-file/#query_frontend_config:~:text=max_fetched_series_per_query for details. + +### Max chunks fetched per (sharded) query + +See https://cortexmetrics.io/docs/configuration/configuration-file/#query_frontend_config:~:text=max_fetched_chunk_bytes_per_query for details. + +### Max samples fetched per (sharded) query + +See https://cortexmetrics.io/docs/configuration/configuration-file/#querier_config:~:text=max_samples for details. + +## Resource-based throttling + +Although the static limits are able to protect Cortex components from specific query patterns, they are not generic enough to cover different combinations of bad query patterns. For example, what if the query fetches relatively large postings, series and chunks that are slightly below the individual limits? For a more generic solution, you can enable resource-based throttling by setting CPU and heap utilization thresholds. + +Currently, it only throttles incoming query requests with error code 429 (too many requests) when the resource usage breaches the configured thresholds. + +For example, the following configuration will start throttling query requests if either CPU or heap utilization is above 80%, leaving 20% of room for inflight requests. + +``` +target: ingester +resource_thresholds: + cpu: 0.8 + heap: 0.8 +``` + +See https://cortexmetrics.io/docs/configuration/configuration-file/#generic-placeholders:~:text=resource_thresholds for details. \ No newline at end of file diff --git a/pkg/configs/resources.go b/pkg/configs/resources.go index 50ab6c4bc37..7990dce1432 100644 --- a/pkg/configs/resources.go +++ b/pkg/configs/resources.go @@ -13,8 +13,8 @@ type Resources struct { } func (cfg *Resources) RegisterFlags(f *flag.FlagSet) { - f.Float64Var(&cfg.CPU, "resource-thresholds.cpu", 0, "Utilization threshold for CPU in percentage, between 0 and 1. 0 to disable.") - f.Float64Var(&cfg.Heap, "resource-thresholds.heap", 0, "Utilization threshold for heap in percentage, between 0 and 1. 0 to disable.") + f.Float64Var(&cfg.CPU, "resource-thresholds.cpu", 0, "Utilization threshold for CPU in percentage, between 0 and 1. 0 to disable. The CPU utilization metric is from github.com/prometheus/procfs, which is a close estimate. Applicable to ingesters and store-gateways only.") + f.Float64Var(&cfg.Heap, "resource-thresholds.heap", 0, "Utilization threshold for heap in percentage, between 0 and 1. 0 to disable. The heap utilization metric is from runtime/metrics, which is a close estimate. Applicable to ingesters and store-gateways only.") } func (cfg *Resources) Validate() error { diff --git a/pkg/cortex/modules.go b/pkg/cortex/modules.go index e30e5d058bc..dd8e876e78f 100644 --- a/pkg/cortex/modules.go +++ b/pkg/cortex/modules.go @@ -777,6 +777,10 @@ func (t *Cortex) initResourceMonitor() (services.Service, error) { scanner, err := resource.NewScanner() if err != nil { + if errors.As(err, resource.UnsupportedOSError{}) { + level.Warn(util_log.Logger).Log("msg", "Skipping resource monitor", "err", err.Error()) + return nil, nil + } return nil, err } diff --git a/pkg/util/resource/monitor.go b/pkg/util/resource/monitor.go index 7ea4e4793bb..e1e2b014fc9 100644 --- a/pkg/util/resource/monitor.go +++ b/pkg/util/resource/monitor.go @@ -4,6 +4,7 @@ import ( "context" "fmt" "net/http" + "runtime" "runtime/metrics" "sync" "time" @@ -24,6 +25,12 @@ func (e *ExhaustedError) Error() string { return "resource exhausted" } +type UnsupportedOSError struct{} + +func (e *UnsupportedOSError) Error() string { + return "resource scanner is only supported in linux" +} + const heapMetricName = "/memory/classes/Heap/objects:bytes" const monitorInterval = time.Second const dataPointsToAvg = 30 @@ -43,6 +50,10 @@ type Stats struct { } func NewScanner() (*Scanner, error) { + if runtime.GOOS != "linux" { + return nil, &UnsupportedOSError{} + } + proc, err := procfs.Self() if err != nil { return nil, errors.Wrap(err, "error reading proc directory") @@ -89,8 +100,8 @@ type Monitor struct { // Variables to calculate average CPU utilization index int - cpuRates []float64 - cpuIntervals []float64 + cpuRates [dataPointsToAvg]float64 + cpuIntervals [dataPointsToAvg]float64 totalCPU float64 totalInterval float64 lastCPU float64 @@ -105,8 +116,8 @@ func NewMonitor(thresholds configs.Resources, limits configs.Resources, scanner containerLimit: limits, scanner: scanner, - cpuRates: make([]float64, dataPointsToAvg), - cpuIntervals: make([]float64, dataPointsToAvg), + cpuRates: [dataPointsToAvg]float64{}, + cpuIntervals: [dataPointsToAvg]float64{}, lock: sync.RWMutex{}, } From 067478bb73e64422acb032459411e05fbce70323 Mon Sep 17 00:00:00 2001 From: Justin Jung Date: Mon, 31 Mar 2025 15:47:49 -0700 Subject: [PATCH 11/29] rename doc Signed-off-by: Justin Jung --- ...ased-throttling.md => protecting-cortex-from-heavy-queries.md} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename docs/guides/{resource-based-throttling.md => protecting-cortex-from-heavy-queries.md} (100%) diff --git a/docs/guides/resource-based-throttling.md b/docs/guides/protecting-cortex-from-heavy-queries.md similarity index 100% rename from docs/guides/resource-based-throttling.md rename to docs/guides/protecting-cortex-from-heavy-queries.md From 18fdf37ea24e71c550f2d26bcd7559a563c0a3a4 Mon Sep 17 00:00:00 2001 From: Justin Jung Date: Wed, 9 Apr 2025 18:40:11 -0700 Subject: [PATCH 12/29] Make monitor more generic + separate scanners Signed-off-by: Justin Jung --- pkg/configs/instance_limits.go | 32 +++ pkg/configs/resources.go | 26 --- pkg/cortex/cortex.go | 27 ++- pkg/cortex/cortex_test.go | 8 +- pkg/cortex/modules.go | 25 ++- pkg/frontend/transport/handler.go | 53 +++-- pkg/frontend/transport/handler_test.go | 4 +- pkg/ingester/ingester.go | 48 +++-- pkg/ingester/instance_limits.go | 38 +++- pkg/storegateway/gateway.go | 33 +++- pkg/util/limiter/resource_based_limiter.go | 46 +++++ pkg/util/resource/monitor.go | 215 ++++++++------------- pkg/util/resource/scanner.go | 43 +++++ pkg/util/resource/scanner_darwin.go | 14 ++ pkg/util/resource/scanner_linux.go | 30 +++ 15 files changed, 397 insertions(+), 245 deletions(-) create mode 100644 pkg/configs/instance_limits.go delete mode 100644 pkg/configs/resources.go create mode 100644 pkg/util/limiter/resource_based_limiter.go create mode 100644 pkg/util/resource/scanner.go create mode 100644 pkg/util/resource/scanner_darwin.go create mode 100644 pkg/util/resource/scanner_linux.go diff --git a/pkg/configs/instance_limits.go b/pkg/configs/instance_limits.go new file mode 100644 index 00000000000..bb4ac4bc9fd --- /dev/null +++ b/pkg/configs/instance_limits.go @@ -0,0 +1,32 @@ +package configs + +import ( + "errors" + "flag" + "strings" + + "github.com/cortexproject/cortex/pkg/util/flagext" + "github.com/cortexproject/cortex/pkg/util/resource" +) + +type InstanceLimits struct { + CPUUtilization float64 `yaml:"cpu_utilization"` + HeapUtilization float64 `yaml:"heap_utilization"` +} + +func (cfg *InstanceLimits) RegisterFlagsWithPrefix(f *flag.FlagSet, prefix string) { + f.Float64Var(&cfg.CPUUtilization, prefix+"instance-limits.cpu-utilization", 0, "Max CPU utilization that this ingester can reach before rejecting new query request (across all tenants) in percentage, between 0 and 1. 0 = unlimited.") + f.Float64Var(&cfg.HeapUtilization, prefix+"instance-limits.heap-utilization", 0, "Max heap utilization that this ingester can reach before rejecting new query request (across all tenants) in percentage, between 0 and 1. 0 = unlimited.") +} + +func (cfg *InstanceLimits) Validate(monitoredResources flagext.StringSliceCSV) error { + if cfg.CPUUtilization > 0 && !strings.Contains(monitoredResources.String(), string(resource.CPU)) { + return errors.New("monitored_resources config must include \"cpu\" as well") + } + + if cfg.HeapUtilization > 0 && !strings.Contains(monitoredResources.String(), string(resource.Heap)) { + return errors.New("monitored_resources config must include \"heap\" as well") + } + + return nil +} diff --git a/pkg/configs/resources.go b/pkg/configs/resources.go deleted file mode 100644 index 7990dce1432..00000000000 --- a/pkg/configs/resources.go +++ /dev/null @@ -1,26 +0,0 @@ -package configs - -import ( - "errors" - "flag" -) - -var ErrInvalidResourceThreshold = errors.New("invalid resource utilization threshold, it must be between 0 and 1") - -type Resources struct { - CPU float64 `yaml:"cpu"` - Heap float64 `yaml:"heap"` -} - -func (cfg *Resources) RegisterFlags(f *flag.FlagSet) { - f.Float64Var(&cfg.CPU, "resource-thresholds.cpu", 0, "Utilization threshold for CPU in percentage, between 0 and 1. 0 to disable. The CPU utilization metric is from github.com/prometheus/procfs, which is a close estimate. Applicable to ingesters and store-gateways only.") - f.Float64Var(&cfg.Heap, "resource-thresholds.heap", 0, "Utilization threshold for heap in percentage, between 0 and 1. 0 to disable. The heap utilization metric is from runtime/metrics, which is a close estimate. Applicable to ingesters and store-gateways only.") -} - -func (cfg *Resources) Validate() error { - if cfg.CPU > 1 || cfg.CPU < 0 || cfg.Heap > 1 || cfg.Heap < 0 { - return ErrInvalidResourceThreshold - } - - return nil -} diff --git a/pkg/cortex/cortex.go b/pkg/cortex/cortex.go index aba1caa23f6..9b7ee5d8ab4 100644 --- a/pkg/cortex/cortex.go +++ b/pkg/cortex/cortex.go @@ -89,10 +89,11 @@ var ( // Config is the root config for Cortex. type Config struct { - Target flagext.StringSliceCSV `yaml:"target"` - AuthEnabled bool `yaml:"auth_enabled"` - PrintConfig bool `yaml:"-"` - HTTPPrefix string `yaml:"http_prefix"` + Target flagext.StringSliceCSV `yaml:"target"` + AuthEnabled bool `yaml:"auth_enabled"` + PrintConfig bool `yaml:"-"` + HTTPPrefix string `yaml:"http_prefix"` + MonitoredResources flagext.StringSliceCSV `yaml:"monitored_resources"` ExternalQueryable prom_storage.Queryable `yaml:"-"` ExternalPusher ruler.Pusher `yaml:"-"` @@ -123,7 +124,6 @@ type Config struct { RuntimeConfig runtimeconfig.Config `yaml:"runtime_config"` MemberlistKV memberlist.KVConfig `yaml:"memberlist"` QueryScheduler scheduler.Config `yaml:"query_scheduler"` - ResourceThresholds configs.Resources `yaml:"resource_thresholds"` Tracing tracing.Config `yaml:"tracing"` } @@ -145,6 +145,10 @@ func (c *Config) RegisterFlags(f *flag.FlagSet) { f.BoolVar(&c.PrintConfig, "print.config", false, "Print the config and exit.") f.StringVar(&c.HTTPPrefix, "http.prefix", "/api/prom", "HTTP path prefix for Cortex API.") + c.MonitoredResources = []string{} + f.Var(&c.MonitoredResources, "monitored_resources", "Comma-separated list of resources to monitor. "+ + "Supported values are cpu and heap. Empty string to disable.") + c.API.RegisterFlags(f) c.registerServerFlagsWithChangedDefaultValues(f) c.Distributor.RegisterFlags(f) @@ -172,7 +176,6 @@ func (c *Config) RegisterFlags(f *flag.FlagSet) { c.MemberlistKV.RegisterFlags(f) c.QueryScheduler.RegisterFlags(f) c.Tracing.RegisterFlags(f) - c.ResourceThresholds.RegisterFlags(f) } // Validate the cortex config and returns an error if the validation @@ -219,7 +222,7 @@ func (c *Config) Validate(log log.Logger) error { if err := c.QueryRange.Validate(c.Querier); err != nil { return errors.Wrap(err, "invalid query_range config") } - if err := c.StoreGateway.Validate(c.LimitsConfig); err != nil { + if err := c.StoreGateway.Validate(c.LimitsConfig, c.MonitoredResources); err != nil { return errors.Wrap(err, "invalid store-gateway config") } if err := c.Compactor.Validate(c.LimitsConfig); err != nil { @@ -232,7 +235,7 @@ func (c *Config) Validate(log log.Logger) error { return errors.Wrap(err, "invalid alertmanager config") } - if err := c.Ingester.Validate(); err != nil { + if err := c.Ingester.Validate(c.MonitoredResources); err != nil { return errors.Wrap(err, "invalid ingester config") } @@ -240,8 +243,12 @@ func (c *Config) Validate(log log.Logger) error { return errors.Wrap(err, "invalid tracing config") } - if err := c.ResourceThresholds.Validate(); err != nil { - return errors.Wrap(err, "invalid resource_thresholds config") + for _, r := range c.MonitoredResources { + switch resource.Type(r) { + case resource.CPU, resource.Heap: + default: + return fmt.Errorf("unsupported resource type to monitor: %s", r) + } } return nil diff --git a/pkg/cortex/cortex_test.go b/pkg/cortex/cortex_test.go index 8fe169050d5..551ba62a987 100644 --- a/pkg/cortex/cortex_test.go +++ b/pkg/cortex/cortex_test.go @@ -4,6 +4,7 @@ import ( "bytes" "context" "flag" + "fmt" "io" "net" "os" @@ -22,7 +23,6 @@ import ( "github.com/cortexproject/cortex/pkg/alertmanager" "github.com/cortexproject/cortex/pkg/alertmanager/alertstore" - "github.com/cortexproject/cortex/pkg/configs" "github.com/cortexproject/cortex/pkg/cortex/storage" "github.com/cortexproject/cortex/pkg/frontend/v1/frontendv1pb" "github.com/cortexproject/cortex/pkg/ingester" @@ -167,13 +167,13 @@ func TestConfigValidation(t *testing.T) { expectedError: errInvalidHTTPPrefix, }, { - name: "should fail validation for resource thresholds", + name: "should fail validation for invalid resource to monitor", getTestConfig: func() *Config { configuration := newDefaultConfig() - configuration.ResourceThresholds.CPU = 10 + configuration.MonitoredResources = []string{"wrong"} return configuration }, - expectedError: configs.ErrInvalidResourceThreshold, + expectedError: fmt.Errorf("unsupported resource type to monitor: %s", "wrong"), }, } { t.Run(tc.name, func(t *testing.T) { diff --git a/pkg/cortex/modules.go b/pkg/cortex/modules.go index dd8e876e78f..3f6c06b79d4 100644 --- a/pkg/cortex/modules.go +++ b/pkg/cortex/modules.go @@ -29,7 +29,6 @@ import ( "github.com/cortexproject/cortex/pkg/alertmanager/alertstore" "github.com/cortexproject/cortex/pkg/api" "github.com/cortexproject/cortex/pkg/compactor" - "github.com/cortexproject/cortex/pkg/configs" configAPI "github.com/cortexproject/cortex/pkg/configs/api" "github.com/cortexproject/cortex/pkg/configs/db" "github.com/cortexproject/cortex/pkg/distributor" @@ -771,26 +770,26 @@ func (t *Cortex) initQueryScheduler() (services.Service, error) { } func (t *Cortex) initResourceMonitor() (services.Service, error) { - if t.Cfg.ResourceThresholds.CPU <= 0 && t.Cfg.ResourceThresholds.Heap <= 0 { + if len(t.Cfg.MonitoredResources) == 0 { return nil, nil } - scanner, err := resource.NewScanner() - if err != nil { - if errors.As(err, resource.UnsupportedOSError{}) { - level.Warn(util_log.Logger).Log("msg", "Skipping resource monitor", "err", err.Error()) - return nil, nil + containerLimits := make(map[resource.Type]float64) + for _, res := range t.Cfg.MonitoredResources { + switch resource.Type(res) { + case resource.CPU: + containerLimits[resource.Type(res)] = float64(runtime.GOMAXPROCS(0)) + case resource.Heap: + containerLimits[resource.Type(res)] = float64(debug.SetMemoryLimit(-1)) } - return nil, err } - limits := configs.Resources{ - CPU: float64(runtime.GOMAXPROCS(0)), - Heap: float64(debug.SetMemoryLimit(-1)), + var err error + t.ResourceMonitor, err = resource.NewMonitor(containerLimits, prometheus.DefaultRegisterer) + if t.ResourceMonitor != nil { + util_log.WarnExperimentalUse("resource monitor") } - t.ResourceMonitor, err = resource.NewMonitor(t.Cfg.ResourceThresholds, limits, scanner, prometheus.DefaultRegisterer) - return t.ResourceMonitor, err } diff --git a/pkg/frontend/transport/handler.go b/pkg/frontend/transport/handler.go index 045a1b95059..11766278dce 100644 --- a/pkg/frontend/transport/handler.go +++ b/pkg/frontend/transport/handler.go @@ -491,38 +491,33 @@ func (f *Handler) reportQueryStats(r *http.Request, source, userID string, query } var reason string - if statusCode == http.StatusRequestEntityTooLarge { + if statusCode == http.StatusTooManyRequests { + reason = reasonTooManyRequests + } else if statusCode == http.StatusRequestEntityTooLarge { reason = reasonResponseBodySizeExceeded - } else if error != nil { + } else if statusCode == http.StatusUnprocessableEntity && error != nil { // We are unable to use errors.As to compare since body string from the http response is wrapped as an error errMsg := error.Error() - if statusCode == http.StatusTooManyRequests { - var resourceExhaustedErr resource.ExhaustedError - if strings.Contains(errMsg, resourceExhaustedErr.Error()) { - reason = reasonResourceExhausted - } else { - reason = reasonTooManyRequests - } - } else if statusCode == http.StatusUnprocessableEntity { - if strings.Contains(errMsg, limitTooManySamples) { - reason = reasonTooManySamples - } else if strings.Contains(errMsg, limitTimeRangeExceeded) { - reason = reasonTimeRangeExceeded - } else if strings.Contains(errMsg, limitSeriesFetched) { - reason = reasonSeriesFetched - } else if strings.Contains(errMsg, limitChunksFetched) { - reason = reasonChunksFetched - } else if strings.Contains(errMsg, limitChunkBytesFetched) { - reason = reasonChunkBytesFetched - } else if strings.Contains(errMsg, limitDataBytesFetched) { - reason = reasonDataBytesFetched - } else if strings.Contains(errMsg, limitSeriesStoreGateway) { - reason = reasonSeriesLimitStoreGateway - } else if strings.Contains(errMsg, limitChunksStoreGateway) { - reason = reasonChunksLimitStoreGateway - } else if strings.Contains(errMsg, limitBytesStoreGateway) { - reason = reasonBytesLimitStoreGateway - } + if strings.Contains(errMsg, limitTooManySamples) { + reason = reasonTooManySamples + } else if strings.Contains(errMsg, limitTimeRangeExceeded) { + reason = reasonTimeRangeExceeded + } else if strings.Contains(errMsg, limitSeriesFetched) { + reason = reasonSeriesFetched + } else if strings.Contains(errMsg, limitChunksFetched) { + reason = reasonChunksFetched + } else if strings.Contains(errMsg, limitChunkBytesFetched) { + reason = reasonChunkBytesFetched + } else if strings.Contains(errMsg, limitDataBytesFetched) { + reason = reasonDataBytesFetched + } else if strings.Contains(errMsg, limitSeriesStoreGateway) { + reason = reasonSeriesLimitStoreGateway + } else if strings.Contains(errMsg, limitChunksStoreGateway) { + reason = reasonChunksLimitStoreGateway + } else if strings.Contains(errMsg, limitBytesStoreGateway) { + reason = reasonBytesLimitStoreGateway + } else if strings.Contains(errMsg, resource.ExhaustedErrorStr) { + reason = reasonResourceExhausted } } if len(reason) > 0 { diff --git a/pkg/frontend/transport/handler_test.go b/pkg/frontend/transport/handler_test.go index ee23fc078b0..3dc1c2a3c7a 100644 --- a/pkg/frontend/transport/handler_test.go +++ b/pkg/frontend/transport/handler_test.go @@ -388,7 +388,7 @@ func TestHandler_ServeHTTP(t *testing.T) { roundTripperFunc: roundTripperFunc(func(req *http.Request) (*http.Response, error) { resourceExhaustedErr := &resource.ExhaustedError{} return &http.Response{ - StatusCode: http.StatusTooManyRequests, + StatusCode: http.StatusUnprocessableEntity, Body: io.NopCloser(strings.NewReader(resourceExhaustedErr.Error())), }, nil }), @@ -396,7 +396,7 @@ func TestHandler_ServeHTTP(t *testing.T) { v := promtest.ToFloat64(h.rejectedQueries.WithLabelValues(reasonResourceExhausted, tripperware.SourceAPI, userID)) assert.Equal(t, float64(1), v) }, - expectedStatusCode: http.StatusTooManyRequests, + expectedStatusCode: http.StatusUnprocessableEntity, }, } { t.Run(tt.name, func(t *testing.T) { diff --git a/pkg/ingester/ingester.go b/pkg/ingester/ingester.go index 416d7420fff..fb8285d5d91 100644 --- a/pkg/ingester/ingester.go +++ b/pkg/ingester/ingester.go @@ -56,6 +56,8 @@ import ( "github.com/cortexproject/cortex/pkg/util" "github.com/cortexproject/cortex/pkg/util/concurrency" "github.com/cortexproject/cortex/pkg/util/extract" + "github.com/cortexproject/cortex/pkg/util/flagext" + "github.com/cortexproject/cortex/pkg/util/limiter" logutil "github.com/cortexproject/cortex/pkg/util/log" util_math "github.com/cortexproject/cortex/pkg/util/math" "github.com/cortexproject/cortex/pkg/util/resource" @@ -168,23 +170,17 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) { f.DurationVar(&cfg.ActiveSeriesMetricsIdleTimeout, "ingester.active-series-metrics-idle-timeout", 10*time.Minute, "After what time a series is considered to be inactive.") f.BoolVar(&cfg.UploadCompactedBlocksEnabled, "ingester.upload-compacted-blocks-enabled", true, "Enable uploading compacted blocks.") - f.Float64Var(&cfg.DefaultLimits.MaxIngestionRate, "ingester.instance-limits.max-ingestion-rate", 0, "Max ingestion rate (samples/sec) that ingester will accept. This limit is per-ingester, not per-tenant. Additional push requests will be rejected. Current ingestion rate is computed as exponentially weighted moving average, updated every second. This limit only works when using blocks engine. 0 = unlimited.") - f.Int64Var(&cfg.DefaultLimits.MaxInMemoryTenants, "ingester.instance-limits.max-tenants", 0, "Max users that this ingester can hold. Requests from additional users will be rejected. This limit only works when using blocks engine. 0 = unlimited.") - f.Int64Var(&cfg.DefaultLimits.MaxInMemorySeries, "ingester.instance-limits.max-series", 0, "Max series that this ingester can hold (across all tenants). Requests to create additional series will be rejected. This limit only works when using blocks engine. 0 = unlimited.") - f.Int64Var(&cfg.DefaultLimits.MaxInflightPushRequests, "ingester.instance-limits.max-inflight-push-requests", 0, "Max inflight push requests that this ingester can handle (across all tenants). Additional requests will be rejected. 0 = unlimited.") - f.Int64Var(&cfg.DefaultLimits.MaxInflightQueryRequests, "ingester.instance-limits.max-inflight-query-requests", 0, "Max inflight query requests that this ingester can handle (across all tenants). Additional requests will be rejected. 0 = unlimited.") - f.StringVar(&cfg.IgnoreSeriesLimitForMetricNames, "ingester.ignore-series-limit-for-metric-names", "", "Comma-separated list of metric names, for which -ingester.max-series-per-metric and -ingester.max-global-series-per-metric limits will be ignored. Does not affect max-series-per-user or max-global-series-per-metric limits.") - f.StringVar(&cfg.AdminLimitMessage, "ingester.admin-limit-message", "please contact administrator to raise it", "Customize the message contained in limit errors") - f.BoolVar(&cfg.LabelsStringInterningEnabled, "ingester.labels-string-interning-enabled", false, "Experimental: Enable string interning for metrics labels.") f.BoolVar(&cfg.DisableChunkTrimming, "ingester.disable-chunk-trimming", false, "Disable trimming of matching series chunks based on query Start and End time. When disabled, the result may contain samples outside the queried time range but select performances may be improved. Note that certain query results might change by changing this option.") f.IntVar(&cfg.MatchersCacheMaxItems, "ingester.matchers-cache-max-items", 0, "Maximum number of entries in the regex matchers cache. 0 to disable.") + + cfg.DefaultLimits.RegisterFlagsWithPrefix(f, "ingester.") } -func (cfg *Config) Validate() error { +func (cfg *Config) Validate(monitoredResources flagext.StringSliceCSV) error { if err := cfg.LifecyclerConfig.Validate(); err != nil { return err } @@ -193,6 +189,10 @@ func (cfg *Config) Validate() error { logutil.WarnExperimentalUse("String interning for metrics labels Enabled") } + if err := cfg.DefaultLimits.Validate(monitoredResources); err != nil { + return err + } + return nil } @@ -229,11 +229,11 @@ type Ingester struct { logger log.Logger - lifecycler *ring.Lifecycler - limits *validation.Overrides - limiter *Limiter - resourceMonitor *resource.Monitor - subservicesWatcher *services.FailureWatcher + lifecycler *ring.Lifecycler + limits *validation.Overrides + limiter *Limiter + resourceBasedLimiter *limiter.ResourceBasedLimiter + subservicesWatcher *services.FailureWatcher stoppedMtx sync.RWMutex // protects stopped stopped bool // protected by stoppedMtx @@ -721,7 +721,6 @@ func New(cfg Config, limits *validation.Overrides, registerer prometheus.Registe ingestionRate: util_math.NewEWMARate(0.2, instanceIngestionRateTickInterval), expandedPostingsCacheFactory: cortex_tsdb.NewExpandedPostingsCacheFactory(cfg.BlocksStorageConfig.TSDB.PostingsCache), matchersCache: storecache.NoopMatchersCache, - resourceMonitor: resourceMonitor, } if cfg.MatchersCacheMaxItems > 0 { @@ -783,6 +782,17 @@ func New(cfg Config, limits *validation.Overrides, registerer prometheus.Registe i.TSDBState.compactionIdleTimeout = util.DurationWithPositiveJitter(i.cfg.BlocksStorageConfig.TSDB.HeadCompactionIdleTimeout, compactionIdleTimeoutJitter) level.Info(i.logger).Log("msg", "TSDB idle compaction timeout set", "timeout", i.TSDBState.compactionIdleTimeout) + if resourceMonitor != nil { + resourceLimits := make(map[resource.Type]float64) + if cfg.DefaultLimits.CPUUtilization > 0 { + resourceLimits[resource.CPU] = cfg.DefaultLimits.CPUUtilization + } + if cfg.DefaultLimits.HeapUtilization > 0 { + resourceLimits[resource.Heap] = cfg.DefaultLimits.HeapUtilization + } + i.resourceBasedLimiter = limiter.NewResourceBasedLimiter(resourceMonitor, resourceLimits) + } + i.BasicService = services.NewBasicService(i.starting, i.updateLoop, i.stopping) return i, nil } @@ -2156,10 +2166,10 @@ func (i *Ingester) trackInflightQueryRequest() (func(), error) { i.maxInflightQueryRequests.Track(i.inflightQueryRequests.Inc()) - if i.resourceMonitor != nil { - if resourceName, threshold, utilization, err := i.resourceMonitor.CheckResourceUtilization(); err != nil { - level.Warn(i.logger).Log("msg", "resource threshold breached", "resource", resourceName, "threshold", threshold, "utilization", utilization) - return nil, errors.Wrapf(err, "failed to query") + if i.resourceBasedLimiter != nil { + if err := i.resourceBasedLimiter.AcceptNewRequest(); err != nil { + level.Warn(i.logger).Log("msg", "failed to accept request", "err", err) + return nil, fmt.Errorf("failed to query: %s", limiter.ErrResourceLimitReachedStr) } } diff --git a/pkg/ingester/instance_limits.go b/pkg/ingester/instance_limits.go index d3b4671152b..8f642c5a732 100644 --- a/pkg/ingester/instance_limits.go +++ b/pkg/ingester/instance_limits.go @@ -1,6 +1,15 @@ package ingester -import "github.com/pkg/errors" +import ( + "flag" + "strings" + + "github.com/pkg/errors" + + "github.com/cortexproject/cortex/pkg/configs" + "github.com/cortexproject/cortex/pkg/util/flagext" + "github.com/cortexproject/cortex/pkg/util/resource" +) var ( // We don't include values in the message to avoid leaking Cortex cluster configuration to users. @@ -11,9 +20,10 @@ var ( errTooManyInflightQueryRequests = errors.New("cannot push: too many inflight query requests in ingester") ) -// InstanceLimits describes limits used by ingester. Reaching any of these will result in Push method to return -// (internal) error. +// InstanceLimits describes limits used by ingester. Reaching any of these will result in error response to the call. type InstanceLimits struct { + *configs.InstanceLimits + MaxIngestionRate float64 `yaml:"max_ingestion_rate"` MaxInMemoryTenants int64 `yaml:"max_tenants"` MaxInMemorySeries int64 `yaml:"max_series"` @@ -24,6 +34,28 @@ type InstanceLimits struct { // Sets default limit values for unmarshalling. var defaultInstanceLimits *InstanceLimits +func (cfg *InstanceLimits) RegisterFlagsWithPrefix(f *flag.FlagSet, prefix string) { + f.Float64Var(&cfg.MaxIngestionRate, prefix+"instance-limits.max-ingestion-rate", 0, "Max ingestion rate (samples/sec) that ingester will accept. This limit is per-ingester, not per-tenant. Additional push requests will be rejected. Current ingestion rate is computed as exponentially weighted moving average, updated every second. This limit only works when using blocks engine. 0 = unlimited.") + f.Int64Var(&cfg.MaxInMemoryTenants, prefix+"instance-limits.max-tenants", 0, "Max users that this ingester can hold. Requests from additional users will be rejected. This limit only works when using blocks engine. 0 = unlimited.") + f.Int64Var(&cfg.MaxInMemorySeries, prefix+"instance-limits.max-series", 0, "Max series that this ingester can hold (across all tenants). Requests to create additional series will be rejected. This limit only works when using blocks engine. 0 = unlimited.") + f.Int64Var(&cfg.MaxInflightPushRequests, prefix+"instance-limits.max-inflight-push-requests", 0, "Max inflight push requests that this ingester can handle (across all tenants). Additional requests will be rejected. 0 = unlimited.") + f.Int64Var(&cfg.MaxInflightQueryRequests, prefix+"instance-limits.max-inflight-query-requests", 0, "Max inflight query requests that this ingester can handle (across all tenants). Additional requests will be rejected. 0 = unlimited.") + f.Float64Var(&cfg.CPUUtilization, prefix+"instance-limits.cpu-utilization", 0, "Max CPU utilization that this ingester can reach before rejecting new query request (across all tenants) in percentage, between 0 and 1. 0 = unlimited.") + f.Float64Var(&cfg.HeapUtilization, prefix+"instance-limits.heap-utilization", 0, "Max heap utilization that this ingester can reach before rejecting new query request (across all tenants) in percentage, between 0 and 1. 0 = unlimited.") +} + +func (cfg *InstanceLimits) Validate(monitoredResources flagext.StringSliceCSV) error { + if cfg.CPUUtilization > 0 && !strings.Contains(monitoredResources.String(), string(resource.CPU)) { + return errors.New("monitored_resources config must include \"cpu\" as well") + } + + if cfg.HeapUtilization > 0 && !strings.Contains(monitoredResources.String(), string(resource.Heap)) { + return errors.New("monitored_resources config must include \"heap\" as well") + } + + return nil +} + // UnmarshalYAML implements the yaml.Unmarshaler interface. If give func (l *InstanceLimits) UnmarshalYAML(unmarshal func(interface{}) error) error { if defaultInstanceLimits != nil { diff --git a/pkg/storegateway/gateway.go b/pkg/storegateway/gateway.go index 848bc35548c..cbc98c7246e 100644 --- a/pkg/storegateway/gateway.go +++ b/pkg/storegateway/gateway.go @@ -18,6 +18,7 @@ import ( "github.com/thanos-io/thanos/pkg/store/storepb" "github.com/weaveworks/common/logging" + "github.com/cortexproject/cortex/pkg/configs" "github.com/cortexproject/cortex/pkg/ring" "github.com/cortexproject/cortex/pkg/ring/kv" "github.com/cortexproject/cortex/pkg/storage/bucket" @@ -25,6 +26,7 @@ import ( "github.com/cortexproject/cortex/pkg/storegateway/storegatewaypb" "github.com/cortexproject/cortex/pkg/util" "github.com/cortexproject/cortex/pkg/util/flagext" + util_limiter "github.com/cortexproject/cortex/pkg/util/limiter" "github.com/cortexproject/cortex/pkg/util/resource" "github.com/cortexproject/cortex/pkg/util/services" "github.com/cortexproject/cortex/pkg/util/validation" @@ -65,6 +67,8 @@ type Config struct { EnabledTenants flagext.StringSliceCSV `yaml:"enabled_tenants"` DisabledTenants flagext.StringSliceCSV `yaml:"disabled_tenants"` + InstanceLimits configs.InstanceLimits `yaml:"instance_limits"` + // Hedged Request HedgedRequest bucket.HedgedRequestConfig `yaml:"hedged_request"` } @@ -78,10 +82,11 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet) { f.Var(&cfg.EnabledTenants, "store-gateway.enabled-tenants", "Comma separated list of tenants whose store metrics this storegateway can process. If specified, only these tenants will be handled by storegateway, otherwise this storegateway will be enabled for all the tenants in the store-gateway cluster.") f.Var(&cfg.DisabledTenants, "store-gateway.disabled-tenants", "Comma separated list of tenants whose store metrics this storegateway cannot process. If specified, a storegateway that would normally pick the specified tenant(s) for processing will ignore them instead.") cfg.HedgedRequest.RegisterFlagsWithPrefix(f, "store-gateway.") + cfg.InstanceLimits.RegisterFlagsWithPrefix(f, "store-gateway.") } // Validate the Config. -func (cfg *Config) Validate(limits validation.Limits) error { +func (cfg *Config) Validate(limits validation.Limits, monitoredResources flagext.StringSliceCSV) error { if cfg.ShardingEnabled { if !util.StringsContain(supportedShardingStrategies, cfg.ShardingStrategy) { return errInvalidShardingStrategy @@ -96,6 +101,10 @@ func (cfg *Config) Validate(limits validation.Limits) error { return err } + if err := cfg.InstanceLimits.Validate(monitoredResources); err != nil { + return err + } + return nil } @@ -118,7 +127,7 @@ type StoreGateway struct { subservices *services.Manager subservicesWatcher *services.FailureWatcher - resourceMonitor *resource.Monitor + resourceBasedLimiter *util_limiter.ResourceBasedLimiter bucketSync *prometheus.CounterVec } @@ -157,7 +166,6 @@ func newStoreGateway(gatewayCfg Config, storageCfg cortex_tsdb.BlocksStorageConf Name: "cortex_storegateway_bucket_sync_total", Help: "Total number of times the bucket sync operation triggered.", }, []string{"reason"}), - resourceMonitor: resourceMonitor, } allowedTenants := util.NewAllowedTenants(gatewayCfg.EnabledTenants, gatewayCfg.DisabledTenants) @@ -233,6 +241,17 @@ func newStoreGateway(gatewayCfg Config, storageCfg cortex_tsdb.BlocksStorageConf return nil, errors.Wrap(err, "create bucket stores") } + if resourceMonitor != nil { + resourceLimits := make(map[resource.Type]float64) + if gatewayCfg.InstanceLimits.CPUUtilization > 0 { + resourceLimits[resource.CPU] = gatewayCfg.InstanceLimits.CPUUtilization + } + if gatewayCfg.InstanceLimits.HeapUtilization > 0 { + resourceLimits[resource.Heap] = gatewayCfg.InstanceLimits.HeapUtilization + } + g.resourceBasedLimiter = util_limiter.NewResourceBasedLimiter(resourceMonitor, resourceLimits) + } + g.Service = services.NewBasicService(g.starting, g.running, g.stopping) return g, nil @@ -408,13 +427,13 @@ func (g *StoreGateway) LabelValues(ctx context.Context, req *storepb.LabelValues } func (g *StoreGateway) checkResourceUtilization() error { - if g.resourceMonitor == nil { + if g.resourceBasedLimiter == nil { return nil } - if resourceName, threshold, utilization, err := g.resourceMonitor.CheckResourceUtilization(); err != nil { - level.Warn(g.logger).Log("msg", "resource threshold breached", "resource", resourceName, "threshold", threshold, "utilization", utilization) - return errors.Wrapf(err, "failed to query") + if err := g.resourceBasedLimiter.AcceptNewRequest(); err != nil { + level.Warn(g.logger).Log("msg", "failed to accept request", "err", err) + return fmt.Errorf("failed to query: %s", util_limiter.ErrResourceLimitReachedStr) } return nil diff --git a/pkg/util/limiter/resource_based_limiter.go b/pkg/util/limiter/resource_based_limiter.go new file mode 100644 index 00000000000..fe8cf86a4e3 --- /dev/null +++ b/pkg/util/limiter/resource_based_limiter.go @@ -0,0 +1,46 @@ +package limiter + +import ( + "fmt" + + "github.com/cortexproject/cortex/pkg/util/resource" +) + +const ErrResourceLimitReachedStr = "resource limit reached" + +type ResourceLimitReachedError struct{} + +func (e *ResourceLimitReachedError) Error() string { + return ErrResourceLimitReachedStr +} + +type ResourceBasedLimiter struct { + resourceMonitor resource.IMonitor + limits map[resource.Type]float64 +} + +func NewResourceBasedLimiter(resourceMonitor resource.IMonitor, limits map[resource.Type]float64) *ResourceBasedLimiter { + return &ResourceBasedLimiter{ + resourceMonitor: resourceMonitor, + limits: limits, + } +} + +func (l *ResourceBasedLimiter) AcceptNewRequest() error { + for resType, limit := range l.limits { + var utilization float64 + + switch resType { + case resource.CPU: + utilization = l.resourceMonitor.GetCPUUtilization() + case resource.Heap: + utilization = l.resourceMonitor.GetHeapUtilization() + } + + if utilization >= limit { + return fmt.Errorf("%s utilization limit reached (limit: %.3f, utilization: %.3f)", resType, limit, utilization) + } + } + + return nil +} diff --git a/pkg/util/resource/monitor.go b/pkg/util/resource/monitor.go index e1e2b014fc9..700641792e1 100644 --- a/pkg/util/resource/monitor.go +++ b/pkg/util/resource/monitor.go @@ -3,100 +3,45 @@ package resource import ( "context" "fmt" - "net/http" - "runtime" - "runtime/metrics" "sync" "time" - "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" - "github.com/prometheus/procfs" - "github.com/weaveworks/common/httpgrpc" - "github.com/cortexproject/cortex/pkg/configs" "github.com/cortexproject/cortex/pkg/util/services" ) -type ExhaustedError struct{} +const ( + ExhaustedErrorStr = "resource exhausted" + UnsupportedOSErrorStr = "resource scanner is only supported in linux" -func (e *ExhaustedError) Error() string { - return "resource exhausted" -} - -type UnsupportedOSError struct{} - -func (e *UnsupportedOSError) Error() string { - return "resource scanner is only supported in linux" -} - -const heapMetricName = "/memory/classes/Heap/objects:bytes" -const monitorInterval = time.Second -const dataPointsToAvg = 30 - -type IScanner interface { - Scan() (Stats, error) -} - -type Scanner struct { - proc procfs.Proc - metricSamples []metrics.Sample -} - -type Stats struct { - CPU float64 - Heap uint64 -} + CPU Type = "cpu" + Heap Type = "heap" -func NewScanner() (*Scanner, error) { - if runtime.GOOS != "linux" { - return nil, &UnsupportedOSError{} - } - - proc, err := procfs.Self() - if err != nil { - return nil, errors.Wrap(err, "error reading proc directory") - } + monitorInterval = time.Second + dataPointsToAvg = 30 +) - metricSamples := make([]metrics.Sample, 1) - metricSamples[0].Name = heapMetricName - metrics.Read(metricSamples) +type Type string - for _, sample := range metricSamples { - if sample.Value.Kind() == metrics.KindBad { - return nil, fmt.Errorf("metric %s is not supported", sample.Name) - } - } +type UnsupportedOSError struct{} - return &Scanner{ - proc: proc, - metricSamples: metricSamples, - }, nil +func (e *UnsupportedOSError) Error() string { + return UnsupportedOSErrorStr } -func (s *Scanner) Scan() (Stats, error) { - stat, err := s.proc.Stat() - if err != nil { - return Stats{}, err - } - - metrics.Read(s.metricSamples) - - return Stats{ - CPU: stat.CPUTime(), - Heap: s.metricSamples[0].Value.Uint64(), - }, nil +type IMonitor interface { + GetCPUUtilization() float64 + GetHeapUtilization() float64 } type Monitor struct { services.Service - scanner IScanner - - containerLimit configs.Resources - utilization configs.Resources - thresholds configs.Resources + scanners map[Type]scanner + containerLimit map[Type]float64 + utilization map[Type]float64 // Variables to calculate average CPU utilization index int @@ -110,11 +55,10 @@ type Monitor struct { lock sync.RWMutex } -func NewMonitor(thresholds configs.Resources, limits configs.Resources, scanner IScanner, registerer prometheus.Registerer) (*Monitor, error) { +func NewMonitor(limits map[Type]float64, registerer prometheus.Registerer) (*Monitor, error) { m := &Monitor{ - thresholds: thresholds, containerLimit: limits, - scanner: scanner, + scanners: make(map[Type]scanner), cpuRates: [dataPointsToAvg]float64{}, cpuIntervals: [dataPointsToAvg]float64{}, @@ -124,22 +68,40 @@ func NewMonitor(thresholds configs.Resources, limits configs.Resources, scanner m.Service = services.NewBasicService(nil, m.running, nil) - promauto.With(registerer).NewGaugeFunc(prometheus.GaugeOpts{ - Name: "cortex_resource_utilization", - ConstLabels: map[string]string{"resource": "CPU"}, - }, m.GetCPUUtilization) - promauto.With(registerer).NewGaugeFunc(prometheus.GaugeOpts{ - Name: "cortex_resource_utilization", - ConstLabels: map[string]string{"resource": "Heap"}, - }, m.GetHeapUtilization) - promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ - Name: "cortex_resource_threshold", - ConstLabels: map[string]string{"resource": "CPU"}, - }).Set(thresholds.CPU) - promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ - Name: "cortex_resource_threshold", - ConstLabels: map[string]string{"resource": "Heap"}, - }).Set(thresholds.Heap) + for resType, limit := range limits { + var scannerFunc func() (scanner, error) + var gaugeFunc func() float64 + + switch resType { + case CPU: + scannerFunc = newCPUScanner + gaugeFunc = m.GetCPUUtilization + case Heap: + scannerFunc = newHeapScanner + gaugeFunc = m.GetHeapUtilization + } + + s, err := scannerFunc() + if err != nil { + return nil, err + } + m.scanners[resType] = s + m.containerLimit[resType] = limit + + promauto.With(registerer).NewGaugeFunc(prometheus.GaugeOpts{ + Name: "cortex_resource_utilization", + ConstLabels: map[string]string{"resource": string(resType)}, + }, gaugeFunc) + } + + //promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ + // Name: "cortex_resource_threshold", + // ConstLabels: map[string]string{"resource": "CPU"}, + //}).Set(thresholds.CPU) + //promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ + // Name: "cortex_resource_threshold", + // ConstLabels: map[string]string{"resource": "Heap"}, + //}).Set(thresholds.Heap) return m, nil } @@ -154,25 +116,31 @@ func (m *Monitor) running(ctx context.Context) error { return nil case <-ticker.C: - stats, err := m.scanner.Scan() - if err != nil { - return errors.Wrap(err, "error scanning metrics") + for resType, scanner := range m.scanners { + val, err := scanner.scan() + if err != nil { + return fmt.Errorf("error scanning resource %s", resType) + } + + switch resType { + case CPU: + m.storeCPUUtilization(val) + case Heap: + m.storeHeapUtilization(val) + } } - - m.storeCPUUtilization(stats) - m.storeHeapUtilization(stats) } } } -func (m *Monitor) storeCPUUtilization(stats Stats) { +func (m *Monitor) storeCPUUtilization(cpuTime float64) { m.lock.Lock() defer m.lock.Unlock() now := time.Now() if m.lastUpdate.IsZero() { - m.lastCPU = stats.CPU + m.lastCPU = cpuTime m.lastUpdate = now return } @@ -180,27 +148,18 @@ func (m *Monitor) storeCPUUtilization(stats Stats) { m.totalCPU -= m.cpuRates[m.index] m.totalInterval -= m.cpuIntervals[m.index] - m.cpuRates[m.index] = stats.CPU - m.lastCPU + m.cpuRates[m.index] = cpuTime - m.lastCPU m.cpuIntervals[m.index] = now.Sub(m.lastUpdate).Seconds() m.totalCPU += m.cpuRates[m.index] m.totalInterval += m.cpuIntervals[m.index] - m.lastCPU = stats.CPU + m.lastCPU = cpuTime m.lastUpdate = now m.index = (m.index + 1) % dataPointsToAvg - if m.totalInterval > 0 && m.containerLimit.CPU > 0 { - m.utilization.CPU = m.totalCPU / m.totalInterval / m.containerLimit.CPU - } -} - -func (m *Monitor) storeHeapUtilization(stats Stats) { - m.lock.Lock() - defer m.lock.Unlock() - - if m.containerLimit.Heap > 0 { - m.utilization.Heap = float64(stats.Heap) / m.containerLimit.Heap + if m.totalInterval > 0 && m.containerLimit[CPU] > 0 { + m.utilization[CPU] = m.totalCPU / m.totalInterval / m.containerLimit[CPU] } } @@ -208,29 +167,21 @@ func (m *Monitor) GetCPUUtilization() float64 { m.lock.RLock() defer m.lock.RUnlock() - return m.utilization.CPU + return m.utilization[CPU] } -func (m *Monitor) GetHeapUtilization() float64 { - m.lock.RLock() - defer m.lock.RUnlock() - - return m.utilization.Heap -} - -func (m *Monitor) CheckResourceUtilization() (string, float64, float64, error) { - cpu := m.GetCPUUtilization() - heap := m.GetHeapUtilization() +func (m *Monitor) storeHeapUtilization(val float64) { + m.lock.Lock() + defer m.lock.Unlock() - if m.thresholds.CPU > 0 && cpu > m.thresholds.CPU { - err := ExhaustedError{} - return "CPU", m.thresholds.CPU, cpu, httpgrpc.Errorf(http.StatusTooManyRequests, "%s", err.Error()) + if m.containerLimit[Heap] > 0 { + m.utilization[Heap] = val / m.containerLimit[Heap] } +} - if m.thresholds.Heap > 0 && heap > m.thresholds.Heap { - err := ExhaustedError{} - return "Heap", m.thresholds.Heap, heap, httpgrpc.Errorf(http.StatusTooManyRequests, "%s", err.Error()) - } +func (m *Monitor) GetHeapUtilization() float64 { + m.lock.RLock() + defer m.lock.RUnlock() - return "", 0, 0, nil + return m.utilization[Heap] } diff --git a/pkg/util/resource/scanner.go b/pkg/util/resource/scanner.go new file mode 100644 index 00000000000..1d4f0906c70 --- /dev/null +++ b/pkg/util/resource/scanner.go @@ -0,0 +1,43 @@ +package resource + +import ( + "fmt" + "runtime/metrics" +) + +const ( + heapMetricName = "/memory/classes/Heap/objects:bytes" +) + +type scanner interface { + scan() (float64, error) +} + +type noopScanner struct{} + +func (s *noopScanner) scan() (float64, error) { + return 0, nil +} + +type heapScanner struct { + metricSamples []metrics.Sample +} + +func newHeapScanner() (scanner, error) { + metricSamples := make([]metrics.Sample, 1) + metricSamples[0].Name = heapMetricName + metrics.Read(metricSamples) + + for _, sample := range metricSamples { + if sample.Value.Kind() == metrics.KindBad { + return nil, fmt.Errorf("metric %s is not supported", sample.Name) + } + } + + return &heapScanner{metricSamples: metricSamples}, nil +} + +func (s *heapScanner) scan() (float64, error) { + metrics.Read(s.metricSamples) + return s.metricSamples[0].Value.Float64(), nil +} diff --git a/pkg/util/resource/scanner_darwin.go b/pkg/util/resource/scanner_darwin.go new file mode 100644 index 00000000000..1d4aedfb555 --- /dev/null +++ b/pkg/util/resource/scanner_darwin.go @@ -0,0 +1,14 @@ +//go:build darwin + +package resource + +import ( + "github.com/go-kit/log/level" + + "github.com/cortexproject/cortex/pkg/util/log" +) + +func newCPUScanner() (scanner, error) { + level.Warn(log.Logger).Log("msg", "CPU scanner not supported in darwin.") + return &noopScanner{}, nil +} diff --git a/pkg/util/resource/scanner_linux.go b/pkg/util/resource/scanner_linux.go new file mode 100644 index 00000000000..7746706d432 --- /dev/null +++ b/pkg/util/resource/scanner_linux.go @@ -0,0 +1,30 @@ +//go:build linux + +package resource + +import ( + "github.com/pkg/errors" + "github.com/prometheus/procfs" +) + +type cpuScanner struct { + proc procfs.Proc +} + +func newCPUScanner() (scanner, error) { + proc, err := procfs.Self() + if err != nil { + return nil, errors.Wrap(err, "error reading proc directory") + } + + return &cpuScanner{proc: proc}, nil +} + +func (s *cpuScanner) scan() (float64, error) { + stat, err := s.proc.Stat() + if err != nil { + return 0, err + } + + return stat.CPUTime(), nil +} From aa811555f6b603215bf264ee8911b2396e73f094 Mon Sep 17 00:00:00 2001 From: Justin Jung Date: Wed, 9 Apr 2025 18:48:00 -0700 Subject: [PATCH 13/29] fix tests Signed-off-by: Justin Jung --- docs/blocks-storage/store-gateway.md | 13 +++++++ docs/configuration/config-file-reference.md | 42 ++++++++++++++------- pkg/frontend/transport/handler.go | 4 +- pkg/frontend/transport/handler_test.go | 6 +-- pkg/ingester/instance_limits.go | 2 +- pkg/querier/blocks_store_queryable.go | 3 +- pkg/util/resource/monitor.go | 9 ----- 7 files changed, 49 insertions(+), 30 deletions(-) diff --git a/docs/blocks-storage/store-gateway.md b/docs/blocks-storage/store-gateway.md index 31005f0eaec..836735d26a4 100644 --- a/docs/blocks-storage/store-gateway.md +++ b/docs/blocks-storage/store-gateway.md @@ -349,6 +349,19 @@ store_gateway: # CLI flag: -store-gateway.disabled-tenants [disabled_tenants: | default = ""] + instance_limits: + # Max CPU utilization that this ingester can reach before rejecting new + # query request (across all tenants) in percentage, between 0 and 1. 0 = + # unlimited. + # CLI flag: -store-gateway.instance-limits.cpu-utilization + [cpu_utilization: | default = 0] + + # Max heap utilization that this ingester can reach before rejecting new + # query request (across all tenants) in percentage, between 0 and 1. 0 = + # unlimited. + # CLI flag: -store-gateway.instance-limits.heap-utilization + [heap_utilization: | default = 0] + hedged_request: # If true, hedged requests are applied to object store calls. It can help # with reducing tail latency. diff --git a/docs/configuration/config-file-reference.md b/docs/configuration/config-file-reference.md index b6ba3225b07..e654c0ff159 100644 --- a/docs/configuration/config-file-reference.md +++ b/docs/configuration/config-file-reference.md @@ -68,6 +68,11 @@ Where default_value is the value to use if the environment variable is undefined # CLI flag: -http.prefix [http_prefix: | default = "/api/prom"] +# Comma-separated list of resources to monitor. Supported values are cpu and +# heap. Empty string to disable. +# CLI flag: -monitored_resources +[monitored_resources: | default = ""] + api: # Use GZIP compression for API responses. Some endpoints serve large YAML or # JSON blobs which can benefit from compression. @@ -271,19 +276,6 @@ query_scheduler: # CLI flag: -query-scheduler.grpc-client-config.connect-timeout [connect_timeout: | default = 5s] -resource_thresholds: - # Utilization threshold for CPU in percentage, between 0 and 1. 0 to disable. - # The CPU utilization metric is from github.com/prometheus/procfs, which is a - # close estimate. Applicable to ingesters and store-gateways only. - # CLI flag: -resource-thresholds.cpu - [cpu: | default = 0] - - # Utilization threshold for heap in percentage, between 0 and 1. 0 to disable. - # The heap utilization metric is from runtime/metrics, which is a close - # estimate. Applicable to ingesters and store-gateways only. - # CLI flag: -resource-thresholds.heap - [heap: | default = 0] - # The tracing_config configures backends cortex uses. [tracing: ] ``` @@ -3210,6 +3202,19 @@ lifecycler: [upload_compacted_blocks_enabled: | default = true] instance_limits: + instancelimits: + # Max CPU utilization that this ingester can reach before rejecting new + # query request (across all tenants) in percentage, between 0 and 1. 0 = + # unlimited. + # CLI flag: -ingester.instance-limits.cpu-utilization + [cpu_utilization: | default = 0] + + # Max heap utilization that this ingester can reach before rejecting new + # query request (across all tenants) in percentage, between 0 and 1. 0 = + # unlimited. + # CLI flag: -ingester.instance-limits.heap-utilization + [heap_utilization: | default = 0] + # Max ingestion rate (samples/sec) that ingester will accept. This limit is # per-ingester, not per-tenant. Additional push requests will be rejected. # Current ingestion rate is computed as exponentially weighted moving average, @@ -5863,6 +5868,17 @@ sharding_ring: # CLI flag: -store-gateway.disabled-tenants [disabled_tenants: | default = ""] +instance_limits: + # Max CPU utilization that this ingester can reach before rejecting new query + # request (across all tenants) in percentage, between 0 and 1. 0 = unlimited. + # CLI flag: -store-gateway.instance-limits.cpu-utilization + [cpu_utilization: | default = 0] + + # Max heap utilization that this ingester can reach before rejecting new query + # request (across all tenants) in percentage, between 0 and 1. 0 = unlimited. + # CLI flag: -store-gateway.instance-limits.heap-utilization + [heap_utilization: | default = 0] + hedged_request: # If true, hedged requests are applied to object store calls. It can help with # reducing tail latency. diff --git a/pkg/frontend/transport/handler.go b/pkg/frontend/transport/handler.go index 11766278dce..d9ede2770a7 100644 --- a/pkg/frontend/transport/handler.go +++ b/pkg/frontend/transport/handler.go @@ -27,8 +27,8 @@ import ( "github.com/cortexproject/cortex/pkg/tenant" "github.com/cortexproject/cortex/pkg/util" util_api "github.com/cortexproject/cortex/pkg/util/api" + "github.com/cortexproject/cortex/pkg/util/limiter" util_log "github.com/cortexproject/cortex/pkg/util/log" - "github.com/cortexproject/cortex/pkg/util/resource" ) const ( @@ -516,7 +516,7 @@ func (f *Handler) reportQueryStats(r *http.Request, source, userID string, query reason = reasonChunksLimitStoreGateway } else if strings.Contains(errMsg, limitBytesStoreGateway) { reason = reasonBytesLimitStoreGateway - } else if strings.Contains(errMsg, resource.ExhaustedErrorStr) { + } else if strings.Contains(errMsg, limiter.ErrResourceLimitReachedStr) { reason = reasonResourceExhausted } } diff --git a/pkg/frontend/transport/handler_test.go b/pkg/frontend/transport/handler_test.go index 3dc1c2a3c7a..5159dd42beb 100644 --- a/pkg/frontend/transport/handler_test.go +++ b/pkg/frontend/transport/handler_test.go @@ -29,8 +29,8 @@ import ( "github.com/cortexproject/cortex/pkg/querier/tripperware" "github.com/cortexproject/cortex/pkg/tenant" util_api "github.com/cortexproject/cortex/pkg/util/api" + "github.com/cortexproject/cortex/pkg/util/limiter" util_log "github.com/cortexproject/cortex/pkg/util/log" - "github.com/cortexproject/cortex/pkg/util/resource" ) type roundTripperFunc func(*http.Request) (*http.Response, error) @@ -386,10 +386,10 @@ func TestHandler_ServeHTTP(t *testing.T) { cfg: HandlerConfig{QueryStatsEnabled: true}, expectedMetrics: 6, roundTripperFunc: roundTripperFunc(func(req *http.Request) (*http.Response, error) { - resourceExhaustedErr := &resource.ExhaustedError{} + resourceLimitReachedErr := &limiter.ResourceLimitReachedError{} return &http.Response{ StatusCode: http.StatusUnprocessableEntity, - Body: io.NopCloser(strings.NewReader(resourceExhaustedErr.Error())), + Body: io.NopCloser(strings.NewReader(resourceLimitReachedErr.Error())), }, nil }), additionalMetricsCheckFunc: func(h *Handler) { diff --git a/pkg/ingester/instance_limits.go b/pkg/ingester/instance_limits.go index 8f642c5a732..ed0454a8129 100644 --- a/pkg/ingester/instance_limits.go +++ b/pkg/ingester/instance_limits.go @@ -22,7 +22,7 @@ var ( // InstanceLimits describes limits used by ingester. Reaching any of these will result in error response to the call. type InstanceLimits struct { - *configs.InstanceLimits + configs.InstanceLimits MaxIngestionRate float64 `yaml:"max_ingestion_rate"` MaxInMemoryTenants int64 `yaml:"max_tenants"` diff --git a/pkg/querier/blocks_store_queryable.go b/pkg/querier/blocks_store_queryable.go index 4edeca79058..a5647e55451 100644 --- a/pkg/querier/blocks_store_queryable.go +++ b/pkg/querier/blocks_store_queryable.go @@ -51,7 +51,6 @@ import ( "github.com/cortexproject/cortex/pkg/util/limiter" util_log "github.com/cortexproject/cortex/pkg/util/log" "github.com/cortexproject/cortex/pkg/util/multierror" - "github.com/cortexproject/cortex/pkg/util/resource" "github.com/cortexproject/cortex/pkg/util/services" "github.com/cortexproject/cortex/pkg/util/spanlogger" "github.com/cortexproject/cortex/pkg/util/validation" @@ -1205,7 +1204,7 @@ func countSamplesAndChunks(series ...*storepb.Series) (samplesCount, chunksCount // only retry connection issues func isRetryableError(err error) bool { // retry upon resource exhaustion error from resource monitor - var resourceExhaustedErr *resource.ExhaustedError + var resourceExhaustedErr *limiter.ResourceLimitReachedError if errors.As(err, &resourceExhaustedErr) { return true } diff --git a/pkg/util/resource/monitor.go b/pkg/util/resource/monitor.go index 700641792e1..fc07b27bfdf 100644 --- a/pkg/util/resource/monitor.go +++ b/pkg/util/resource/monitor.go @@ -13,9 +13,6 @@ import ( ) const ( - ExhaustedErrorStr = "resource exhausted" - UnsupportedOSErrorStr = "resource scanner is only supported in linux" - CPU Type = "cpu" Heap Type = "heap" @@ -25,12 +22,6 @@ const ( type Type string -type UnsupportedOSError struct{} - -func (e *UnsupportedOSError) Error() string { - return UnsupportedOSErrorStr -} - type IMonitor interface { GetCPUUtilization() float64 GetHeapUtilization() float64 From a528a7a1d350178c53cc520568db163bc095e783 Mon Sep 17 00:00:00 2001 From: Justin Jung Date: Wed, 9 Apr 2025 22:54:22 -0700 Subject: [PATCH 14/29] fix more tests Signed-off-by: Justin Jung --- pkg/ingester/ingester_test.go | 60 +++++++--------------- pkg/querier/blocks_store_queryable_test.go | 3 +- pkg/storegateway/gateway_test.go | 48 +++++++++-------- 3 files changed, 42 insertions(+), 69 deletions(-) diff --git a/pkg/ingester/ingester_test.go b/pkg/ingester/ingester_test.go index 4d75bc8372a..92f07822923 100644 --- a/pkg/ingester/ingester_test.go +++ b/pkg/ingester/ingester_test.go @@ -49,7 +49,6 @@ import ( "github.com/cortexproject/cortex/pkg/chunk" "github.com/cortexproject/cortex/pkg/chunk/encoding" - "github.com/cortexproject/cortex/pkg/configs" "github.com/cortexproject/cortex/pkg/cortexpb" "github.com/cortexproject/cortex/pkg/ingester/client" "github.com/cortexproject/cortex/pkg/querier/batch" @@ -59,6 +58,7 @@ import ( "github.com/cortexproject/cortex/pkg/storage/tsdb/bucketindex" "github.com/cortexproject/cortex/pkg/util" "github.com/cortexproject/cortex/pkg/util/chunkcompat" + "github.com/cortexproject/cortex/pkg/util/limiter" "github.com/cortexproject/cortex/pkg/util/resource" "github.com/cortexproject/cortex/pkg/util/services" "github.com/cortexproject/cortex/pkg/util/test" @@ -3069,18 +3069,17 @@ func Test_Ingester_Query_ResourceThresholdBreached(t *testing.T) { {labels.Labels{{Name: labels.MetricName, Value: "test_1"}, {Name: "route", Value: "get_user"}, {Name: "status", Value: "200"}}, 1, 100000}, } - thresholds := configs.Resources{Heap: 0.1} - limits := configs.Resources{Heap: 10} - resourceMonitor, err := resource.NewMonitor(thresholds, limits, &mockResourceScanner{ - heap: uint64(5), - }, prometheus.NewRegistry()) - require.NoError(t, err) - - i, err := prepareIngesterWithResourceMonitor(t, resourceMonitor) + i, err := prepareIngesterWithBlocksStorage(t, defaultIngesterTestConfig(t), prometheus.NewRegistry()) require.NoError(t, err) require.NoError(t, services.StartAndAwaitRunning(context.Background(), i)) defer services.StopAndAwaitTerminated(context.Background(), i) //nolint:errcheck + limits := map[resource.Type]float64{ + resource.CPU: 0.5, + resource.Heap: 0.5, + } + i.resourceBasedLimiter = limiter.NewResourceBasedLimiter(&mockResourceMonitor{cpu: 0.4, heap: 0.6}, limits) + // Wait until it's ACTIVE test.Poll(t, 1*time.Second, ring.ACTIVE, func() interface{} { return i.lifecycler.GetState() @@ -3099,20 +3098,10 @@ func Test_Ingester_Query_ResourceThresholdBreached(t *testing.T) { s := &mockQueryStreamServer{ctx: ctx} err = i.QueryStream(rreq, s) require.Error(t, err) - exhaustedErr := resource.ExhaustedError{} + exhaustedErr := limiter.ResourceLimitReachedError{} require.ErrorContains(t, err, exhaustedErr.Error()) } -type mockResourceScanner struct { - heap uint64 -} - -func (m *mockResourceScanner) Scan() (resource.Stats, error) { - return resource.Stats{ - Heap: m.heap, - }, nil -} - func TestIngester_LabelValues_ShouldNotCreateTSDBIfDoesNotExists(t *testing.T) { i, err := prepareIngesterWithBlocksStorage(t, defaultIngesterTestConfig(t), prometheus.NewRegistry()) require.NoError(t, err) @@ -3996,30 +3985,17 @@ func prepareIngesterWithBlocksStorageAndLimits(t testing.TB, ingesterCfg Config, return ingester, nil } -func prepareIngesterWithResourceMonitor(t testing.TB, resourceMonitor *resource.Monitor) (*Ingester, error) { - dataDir := t.TempDir() - bucketDir := t.TempDir() - - ingesterCfg := defaultIngesterTestConfig(t) - ingesterCfg.BlocksStorageConfig.TSDB.Dir = dataDir - ingesterCfg.BlocksStorageConfig.Bucket.Backend = "filesystem" - ingesterCfg.BlocksStorageConfig.Bucket.Filesystem.Directory = bucketDir - - overrides, _ := validation.NewOverrides(defaultLimitsTestConfig(), nil) - - err := resourceMonitor.StartAsync(context.Background()) - if err != nil { - return nil, err - } - time.Sleep(time.Second) - - ingester, err := New(ingesterCfg, overrides, prometheus.NewRegistry(), log.NewNopLogger(), resourceMonitor) - if err != nil { - return nil, err - } +type mockResourceMonitor struct { + cpu float64 + heap float64 +} - return ingester, nil +func (m *mockResourceMonitor) GetCPUUtilization() float64 { + return m.cpu +} +func (m *mockResourceMonitor) GetHeapUtilization() float64 { + return m.heap } func TestIngester_OpenExistingTSDBOnStartup(t *testing.T) { diff --git a/pkg/querier/blocks_store_queryable_test.go b/pkg/querier/blocks_store_queryable_test.go index 43c11a9f070..3ca5c03fee2 100644 --- a/pkg/querier/blocks_store_queryable_test.go +++ b/pkg/querier/blocks_store_queryable_test.go @@ -44,7 +44,6 @@ import ( "github.com/cortexproject/cortex/pkg/util" "github.com/cortexproject/cortex/pkg/util/limiter" util_log "github.com/cortexproject/cortex/pkg/util/log" - "github.com/cortexproject/cortex/pkg/util/resource" "github.com/cortexproject/cortex/pkg/util/services" "github.com/cortexproject/cortex/pkg/util/validation" ) @@ -1527,7 +1526,7 @@ func TestBlocksStoreQuerier_Select(t *testing.T) { map[BlocksStoreClient][]ulid.ULID{ &storeGatewayClientMock{ remoteAddr: "1.1.1.1", - mockedSeriesErr: &resource.ExhaustedError{}, + mockedSeriesErr: &limiter.ResourceLimitReachedError{}, }: {block1}, }, map[BlocksStoreClient][]ulid.ULID{ diff --git a/pkg/storegateway/gateway_test.go b/pkg/storegateway/gateway_test.go index 4bc21e64dc6..b60b585b8d5 100644 --- a/pkg/storegateway/gateway_test.go +++ b/pkg/storegateway/gateway_test.go @@ -32,7 +32,6 @@ import ( "github.com/thanos-io/thanos/pkg/store/storepb" "google.golang.org/grpc/status" - "github.com/cortexproject/cortex/pkg/configs" "github.com/cortexproject/cortex/pkg/ring" "github.com/cortexproject/cortex/pkg/ring/kv/consul" "github.com/cortexproject/cortex/pkg/storage/bucket" @@ -42,6 +41,7 @@ import ( cortex_testutil "github.com/cortexproject/cortex/pkg/storage/tsdb/testutil" "github.com/cortexproject/cortex/pkg/util" "github.com/cortexproject/cortex/pkg/util/flagext" + util_limiter "github.com/cortexproject/cortex/pkg/util/limiter" "github.com/cortexproject/cortex/pkg/util/resource" "github.com/cortexproject/cortex/pkg/util/services" "github.com/cortexproject/cortex/pkg/util/test" @@ -91,7 +91,7 @@ func TestConfig_Validate(t *testing.T) { flagext.DefaultValues(cfg, limits) testData.setup(cfg, limits) - assert.Equal(t, testData.expected, cfg.Validate(*limits)) + assert.Equal(t, testData.expected, cfg.Validate(*limits, nil)) }) } } @@ -1211,39 +1211,24 @@ func TestStoreGateway_SeriesThrottledByResourceMonitor(t *testing.T) { gatewayCfg.ShardingEnabled = false storageCfg := mockStorageConfig(t) - thresholds := configs.Resources{Heap: 0.1} - limits := configs.Resources{Heap: 10} - resourceMonitor, err := resource.NewMonitor(thresholds, limits, &mockResourceScanner{ - heap: uint64(5), - }, prometheus.NewRegistry()) - require.NoError(t, err) - - err = resourceMonitor.StartAsync(context.Background()) - require.NoError(t, err) - time.Sleep(time.Second) - - g, err := newStoreGateway(gatewayCfg, storageCfg, objstore.WithNoopInstr(bucketClient), nil, overrides, mockLoggingLevel(), logger, nil, resourceMonitor) + g, err := newStoreGateway(gatewayCfg, storageCfg, objstore.WithNoopInstr(bucketClient), nil, overrides, mockLoggingLevel(), logger, nil, nil) require.NoError(t, err) require.NoError(t, services.StartAndAwaitRunning(ctx, g)) defer services.StopAndAwaitTerminated(ctx, g) //nolint:errcheck + limits := map[resource.Type]float64{ + resource.CPU: 0.5, + resource.Heap: 0.5, + } + g.resourceBasedLimiter = util_limiter.NewResourceBasedLimiter(&mockResourceMonitor{cpu: 0.4, heap: 0.6}, limits) + srv := newBucketStoreSeriesServer(setUserIDToGRPCContext(ctx, userID)) err = g.Series(req, srv) require.Error(t, err) - exhaustedErr := resource.ExhaustedError{} + exhaustedErr := util_limiter.ResourceLimitReachedError{} require.ErrorContains(t, err, exhaustedErr.Error()) } -type mockResourceScanner struct { - heap uint64 -} - -func (m *mockResourceScanner) Scan() (resource.Stats, error) { - return resource.Stats{ - Heap: m.heap, - }, nil -} - func mockGatewayConfig() Config { cfg := Config{} flagext.DefaultValues(&cfg) @@ -1256,6 +1241,19 @@ func mockGatewayConfig() Config { return cfg } +type mockResourceMonitor struct { + cpu float64 + heap float64 +} + +func (m *mockResourceMonitor) GetCPUUtilization() float64 { + return m.cpu +} + +func (m *mockResourceMonitor) GetHeapUtilization() float64 { + return m.heap +} + func mockStorageConfig(t *testing.T) cortex_tsdb.BlocksStorageConfig { cfg := cortex_tsdb.BlocksStorageConfig{} flagext.DefaultValues(&cfg) From 42e52b3c21c63a0026bafd18e0574d0a9961c200 Mon Sep 17 00:00:00 2001 From: Justin Jung Date: Wed, 9 Apr 2025 23:02:34 -0700 Subject: [PATCH 15/29] remove monitor_test.go Signed-off-by: Justin Jung --- pkg/util/resource/monitor_test.go | 83 ------------------------------- 1 file changed, 83 deletions(-) delete mode 100644 pkg/util/resource/monitor_test.go diff --git a/pkg/util/resource/monitor_test.go b/pkg/util/resource/monitor_test.go deleted file mode 100644 index 0c76cb59f94..00000000000 --- a/pkg/util/resource/monitor_test.go +++ /dev/null @@ -1,83 +0,0 @@ -package resource - -import ( - "context" - "testing" - "time" - - "github.com/stretchr/testify/require" - - "github.com/cortexproject/cortex/pkg/configs" - "github.com/cortexproject/cortex/pkg/util/services" -) - -func TestMonitor(t *testing.T) { - type test struct { - limit float64 - val uint64 - threshold float64 - utilization float64 - throwErr bool - } - - tests := map[string]test{ - "should not throw error if below threshold": { - limit: 10, - val: 1, - utilization: 0.1, - threshold: 0.2, - throwErr: false, - }, - "should throw error if above threshold": { - limit: 10, - val: 5, - utilization: 0.5, - threshold: 0.2, - throwErr: true, - }, - "should not throw error if limit is 0": { - limit: 0, - val: 5, - utilization: 0, - threshold: 0.2, - throwErr: false, - }, - } - - for _, tc := range tests { - limits := configs.Resources{Heap: tc.limit} - thresholds := configs.Resources{Heap: tc.threshold} - scanner := mockScanner{Heap: tc.val} - - monitor, err := NewMonitor(thresholds, limits, &scanner, nil) - require.NoError(t, err) - require.NoError(t, monitor.StartAsync(context.Background())) - t.Cleanup(func() { - require.NoError(t, services.StopAndAwaitTerminated(context.Background(), monitor)) - }) - - time.Sleep(2 * time.Second) // let scanner store values - - require.Equal(t, tc.utilization, monitor.GetHeapUtilization()) - _, _, _, err = monitor.CheckResourceUtilization() - - if tc.throwErr { - exhaustedErr := &ExhaustedError{} - require.ErrorContains(t, err, exhaustedErr.Error()) - } else { - require.NoError(t, err) - } - } -} - -type mockScanner struct { - CPU float64 - Heap uint64 -} - -func (m *mockScanner) Scan() (Stats, error) { - return Stats{ - CPU: m.CPU, - Heap: m.Heap, - }, nil -} From 50993e1730a9a5f17ca443abf6b6a686d62a99f3 Mon Sep 17 00:00:00 2001 From: Justin Jung Date: Wed, 9 Apr 2025 23:10:12 -0700 Subject: [PATCH 16/29] move noop scanner to darwin scanner Signed-off-by: Justin Jung --- pkg/util/resource/scanner.go | 6 ------ pkg/util/resource/scanner_darwin.go | 6 ++++++ 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/pkg/util/resource/scanner.go b/pkg/util/resource/scanner.go index 1d4f0906c70..65517243477 100644 --- a/pkg/util/resource/scanner.go +++ b/pkg/util/resource/scanner.go @@ -13,12 +13,6 @@ type scanner interface { scan() (float64, error) } -type noopScanner struct{} - -func (s *noopScanner) scan() (float64, error) { - return 0, nil -} - type heapScanner struct { metricSamples []metrics.Sample } diff --git a/pkg/util/resource/scanner_darwin.go b/pkg/util/resource/scanner_darwin.go index 1d4aedfb555..3d6231124d3 100644 --- a/pkg/util/resource/scanner_darwin.go +++ b/pkg/util/resource/scanner_darwin.go @@ -8,6 +8,12 @@ import ( "github.com/cortexproject/cortex/pkg/util/log" ) +type noopScanner struct{} + +func (s *noopScanner) scan() (float64, error) { + return 0, nil +} + func newCPUScanner() (scanner, error) { level.Warn(log.Logger).Log("msg", "CPU scanner not supported in darwin.") return &noopScanner{}, nil From e56431ea4b264eb6d837c9bd24b0352301e8d572 Mon Sep 17 00:00:00 2001 From: Justin Jung Date: Thu, 10 Apr 2025 10:12:01 -0700 Subject: [PATCH 17/29] doc update Signed-off-by: Justin Jung --- CHANGELOG.md | 2 +- docs/blocks-storage/store-gateway.md | 8 ++--- docs/configuration/config-file-reference.md | 31 +++++++++---------- docs/configuration/v1-guarantees.md | 5 +++ .../protecting-cortex-from-heavy-queries.md | 11 ++++--- pkg/configs/instance_limits.go | 4 +-- pkg/cortex/cortex.go | 4 +-- pkg/ingester/instance_limits.go | 14 ++------- 8 files changed, 37 insertions(+), 42 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 281490846cb..5581c58277b 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -5,7 +5,7 @@ * [FEATURE] Query Frontend: Add dynamic interval size for query splitting. This is enabled by configuring experimental flags `querier.max-shards-per-query` and/or `querier.max-fetched-data-duration-per-query`. The split interval size is dynamically increased to maintain a number of shards and total duration fetched below the configured values. #6458 * [FEATURE] Querier/Ruler: Add `query_partial_data` and `rules_partial_data` limits to allow queries/rules to be evaluated with data from a single zone, if other zones are not available. #6526 * [FEATURE] Update prometheus alertmanager version to v0.28.0 and add new integration msteamsv2, jira, and rocketchat. #6590 -* [FEATURE] Ingester/StoreGateway: Add `resource-thresholds` in ingesters and store gateways to throttle query requests when the pods are under resource pressure. #6674 +* [FEATURE] Ingester/StoreGateway: Add `monitored_resources` config + ResourceBasedLimiter in ingesters and store gateways. #6674 * [FEATURE] Ingester: Support out-of-order native histogram ingestion. It automatically enabled when `-ingester.out-of-order-time-window > 0` and `-blocks-storage.tsdb.enable-native-histograms=true`. #6626 #6663 * [ENHANCEMENT] Alertmanager: Add nflog and silences maintenance metrics. #6659 * [ENHANCEMENT] Querier: limit label APIs to query only ingesters if `start` param is not been specified. #6618 diff --git a/docs/blocks-storage/store-gateway.md b/docs/blocks-storage/store-gateway.md index 836735d26a4..446ab92f8c5 100644 --- a/docs/blocks-storage/store-gateway.md +++ b/docs/blocks-storage/store-gateway.md @@ -351,14 +351,14 @@ store_gateway: instance_limits: # Max CPU utilization that this ingester can reach before rejecting new - # query request (across all tenants) in percentage, between 0 and 1. 0 = - # unlimited. + # query request (across all tenants) in percentage, between 0 and 1. + # monitored_resources config must include the resource type. 0 to disable. # CLI flag: -store-gateway.instance-limits.cpu-utilization [cpu_utilization: | default = 0] # Max heap utilization that this ingester can reach before rejecting new - # query request (across all tenants) in percentage, between 0 and 1. 0 = - # unlimited. + # query request (across all tenants) in percentage, between 0 and 1. + # monitored_resources config must include the resource type. 0 to disable. # CLI flag: -store-gateway.instance-limits.heap-utilization [heap_utilization: | default = 0] diff --git a/docs/configuration/config-file-reference.md b/docs/configuration/config-file-reference.md index e654c0ff159..728d37db8cb 100644 --- a/docs/configuration/config-file-reference.md +++ b/docs/configuration/config-file-reference.md @@ -69,8 +69,8 @@ Where default_value is the value to use if the environment variable is undefined [http_prefix: | default = "/api/prom"] # Comma-separated list of resources to monitor. Supported values are cpu and -# heap. Empty string to disable. -# CLI flag: -monitored_resources +# heap, which tracks metrics that are close estimates. Empty string to disable. +# CLI flag: -monitored.resources [monitored_resources: | default = ""] api: @@ -3202,18 +3202,15 @@ lifecycler: [upload_compacted_blocks_enabled: | default = true] instance_limits: - instancelimits: - # Max CPU utilization that this ingester can reach before rejecting new - # query request (across all tenants) in percentage, between 0 and 1. 0 = - # unlimited. - # CLI flag: -ingester.instance-limits.cpu-utilization - [cpu_utilization: | default = 0] - - # Max heap utilization that this ingester can reach before rejecting new - # query request (across all tenants) in percentage, between 0 and 1. 0 = - # unlimited. - # CLI flag: -ingester.instance-limits.heap-utilization - [heap_utilization: | default = 0] + # Max CPU utilization that this ingester can reach before rejecting new query + # request (across all tenants) in percentage, between 0 and 1. 0 = unlimited. + # CLI flag: -ingester.instance-limits.cpu-utilization + [cpu_utilization: | default = 0] + + # Max heap utilization that this ingester can reach before rejecting new query + # request (across all tenants) in percentage, between 0 and 1. 0 = unlimited. + # CLI flag: -ingester.instance-limits.heap-utilization + [heap_utilization: | default = 0] # Max ingestion rate (samples/sec) that ingester will accept. This limit is # per-ingester, not per-tenant. Additional push requests will be rejected. @@ -5870,12 +5867,14 @@ sharding_ring: instance_limits: # Max CPU utilization that this ingester can reach before rejecting new query - # request (across all tenants) in percentage, between 0 and 1. 0 = unlimited. + # request (across all tenants) in percentage, between 0 and 1. + # monitored_resources config must include the resource type. 0 to disable. # CLI flag: -store-gateway.instance-limits.cpu-utilization [cpu_utilization: | default = 0] # Max heap utilization that this ingester can reach before rejecting new query - # request (across all tenants) in percentage, between 0 and 1. 0 = unlimited. + # request (across all tenants) in percentage, between 0 and 1. + # monitored_resources config must include the resource type. 0 to disable. # CLI flag: -store-gateway.instance-limits.heap-utilization [heap_utilization: | default = 0] diff --git a/docs/configuration/v1-guarantees.md b/docs/configuration/v1-guarantees.md index bf3beed915e..0922b47859f 100644 --- a/docs/configuration/v1-guarantees.md +++ b/docs/configuration/v1-guarantees.md @@ -123,3 +123,8 @@ Currently experimental features are: - Query-frontend: dynamic query splits - `querier.max-shards-per-query` (int) CLI flag - `querier.max-fetched-data-duration-per-query` (duration) CLI flag +- Ingester/Store-Gateway: Resource-based throttling + - `-ingester.instance-limits.cpu-utilization` + - `-ingester.instance-limits.heap-utilization` + - `-store-gateway.instance-limits.cpu-utilization` + - `-store-gateway.instance-limits.heap-utilization` \ No newline at end of file diff --git a/docs/guides/protecting-cortex-from-heavy-queries.md b/docs/guides/protecting-cortex-from-heavy-queries.md index 71c4f88dc7a..4f76ef18824 100644 --- a/docs/guides/protecting-cortex-from-heavy-queries.md +++ b/docs/guides/protecting-cortex-from-heavy-queries.md @@ -37,7 +37,7 @@ See https://cortexmetrics.io/docs/configuration/configuration-file/#query_fronte See https://cortexmetrics.io/docs/configuration/configuration-file/#querier_config:~:text=max_samples for details. -## Resource-based throttling +## Resource-based throttling (Experimental) Although the static limits are able to protect Cortex components from specific query patterns, they are not generic enough to cover different combinations of bad query patterns. For example, what if the query fetches relatively large postings, series and chunks that are slightly below the individual limits? For a more generic solution, you can enable resource-based throttling by setting CPU and heap utilization thresholds. @@ -47,9 +47,10 @@ For example, the following configuration will start throttling query requests if ``` target: ingester -resource_thresholds: - cpu: 0.8 - heap: 0.8 +monitored_resources: cpu,heap +instance_limits: + cpu_utilization: 0.8 + heap_utilization: 0.8 ``` -See https://cortexmetrics.io/docs/configuration/configuration-file/#generic-placeholders:~:text=resource_thresholds for details. \ No newline at end of file +See https://cortexmetrics.io/docs/configuration/configuration-file/:~:text=instance_limits for details. \ No newline at end of file diff --git a/pkg/configs/instance_limits.go b/pkg/configs/instance_limits.go index bb4ac4bc9fd..51164bd75fa 100644 --- a/pkg/configs/instance_limits.go +++ b/pkg/configs/instance_limits.go @@ -15,8 +15,8 @@ type InstanceLimits struct { } func (cfg *InstanceLimits) RegisterFlagsWithPrefix(f *flag.FlagSet, prefix string) { - f.Float64Var(&cfg.CPUUtilization, prefix+"instance-limits.cpu-utilization", 0, "Max CPU utilization that this ingester can reach before rejecting new query request (across all tenants) in percentage, between 0 and 1. 0 = unlimited.") - f.Float64Var(&cfg.HeapUtilization, prefix+"instance-limits.heap-utilization", 0, "Max heap utilization that this ingester can reach before rejecting new query request (across all tenants) in percentage, between 0 and 1. 0 = unlimited.") + f.Float64Var(&cfg.CPUUtilization, prefix+"instance-limits.cpu-utilization", 0, "EXPERIMENTAL: Max CPU utilization that this ingester can reach before rejecting new query request (across all tenants) in percentage, between 0 and 1. monitored_resources config must include the resource type. 0 to disable.") + f.Float64Var(&cfg.HeapUtilization, prefix+"instance-limits.heap-utilization", 0, "EXPERIMENTAL: Max heap utilization that this ingester can reach before rejecting new query request (across all tenants) in percentage, between 0 and 1. monitored_resources config must include the resource type. 0 to disable.") } func (cfg *InstanceLimits) Validate(monitoredResources flagext.StringSliceCSV) error { diff --git a/pkg/cortex/cortex.go b/pkg/cortex/cortex.go index 9b7ee5d8ab4..858f51499b0 100644 --- a/pkg/cortex/cortex.go +++ b/pkg/cortex/cortex.go @@ -146,8 +146,8 @@ func (c *Config) RegisterFlags(f *flag.FlagSet) { f.StringVar(&c.HTTPPrefix, "http.prefix", "/api/prom", "HTTP path prefix for Cortex API.") c.MonitoredResources = []string{} - f.Var(&c.MonitoredResources, "monitored_resources", "Comma-separated list of resources to monitor. "+ - "Supported values are cpu and heap. Empty string to disable.") + f.Var(&c.MonitoredResources, "monitored.resources", "Comma-separated list of resources to monitor. "+ + "Supported values are cpu and heap, which tracks metrics that are close estimates. Empty string to disable.") c.API.RegisterFlags(f) c.registerServerFlagsWithChangedDefaultValues(f) diff --git a/pkg/ingester/instance_limits.go b/pkg/ingester/instance_limits.go index ed0454a8129..c0839e9270a 100644 --- a/pkg/ingester/instance_limits.go +++ b/pkg/ingester/instance_limits.go @@ -2,13 +2,11 @@ package ingester import ( "flag" - "strings" "github.com/pkg/errors" "github.com/cortexproject/cortex/pkg/configs" "github.com/cortexproject/cortex/pkg/util/flagext" - "github.com/cortexproject/cortex/pkg/util/resource" ) var ( @@ -22,7 +20,7 @@ var ( // InstanceLimits describes limits used by ingester. Reaching any of these will result in error response to the call. type InstanceLimits struct { - configs.InstanceLimits + configs.InstanceLimits `yaml:",inline"` MaxIngestionRate float64 `yaml:"max_ingestion_rate"` MaxInMemoryTenants int64 `yaml:"max_tenants"` @@ -45,15 +43,7 @@ func (cfg *InstanceLimits) RegisterFlagsWithPrefix(f *flag.FlagSet, prefix strin } func (cfg *InstanceLimits) Validate(monitoredResources flagext.StringSliceCSV) error { - if cfg.CPUUtilization > 0 && !strings.Contains(monitoredResources.String(), string(resource.CPU)) { - return errors.New("monitored_resources config must include \"cpu\" as well") - } - - if cfg.HeapUtilization > 0 && !strings.Contains(monitoredResources.String(), string(resource.Heap)) { - return errors.New("monitored_resources config must include \"heap\" as well") - } - - return nil + return cfg.InstanceLimits.Validate(monitoredResources) } // UnmarshalYAML implements the yaml.Unmarshaler interface. If give From eae4df77767e728b6d03d8fff14d07bfec425db6 Mon Sep 17 00:00:00 2001 From: Justin Jung Date: Thu, 10 Apr 2025 10:19:28 -0700 Subject: [PATCH 18/29] doc Signed-off-by: Justin Jung --- docs/configuration/config-file-reference.md | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/docs/configuration/config-file-reference.md b/docs/configuration/config-file-reference.md index 728d37db8cb..06881f4c499 100644 --- a/docs/configuration/config-file-reference.md +++ b/docs/configuration/config-file-reference.md @@ -5866,15 +5866,17 @@ sharding_ring: [disabled_tenants: | default = ""] instance_limits: - # Max CPU utilization that this ingester can reach before rejecting new query - # request (across all tenants) in percentage, between 0 and 1. - # monitored_resources config must include the resource type. 0 to disable. + # EXPERIMENTAL: Max CPU utilization that this ingester can reach before + # rejecting new query request (across all tenants) in percentage, between 0 + # and 1. monitored_resources config must include the resource type. 0 to + # disable. # CLI flag: -store-gateway.instance-limits.cpu-utilization [cpu_utilization: | default = 0] - # Max heap utilization that this ingester can reach before rejecting new query - # request (across all tenants) in percentage, between 0 and 1. - # monitored_resources config must include the resource type. 0 to disable. + # EXPERIMENTAL: Max heap utilization that this ingester can reach before + # rejecting new query request (across all tenants) in percentage, between 0 + # and 1. monitored_resources config must include the resource type. 0 to + # disable. # CLI flag: -store-gateway.instance-limits.heap-utilization [heap_utilization: | default = 0] From fd19f5cd32c86559d968e18b95b3ea12ad3665aa Mon Sep 17 00:00:00 2001 From: Justin Jung Date: Thu, 10 Apr 2025 10:27:17 -0700 Subject: [PATCH 19/29] lint Signed-off-by: Justin Jung --- docs/blocks-storage/store-gateway.md | 14 ++++++++------ docs/configuration/config-file-reference.md | 3 ++- pkg/cortex/cortex.go | 3 ++- 3 files changed, 12 insertions(+), 8 deletions(-) diff --git a/docs/blocks-storage/store-gateway.md b/docs/blocks-storage/store-gateway.md index 446ab92f8c5..58b4c08eb3b 100644 --- a/docs/blocks-storage/store-gateway.md +++ b/docs/blocks-storage/store-gateway.md @@ -350,15 +350,17 @@ store_gateway: [disabled_tenants: | default = ""] instance_limits: - # Max CPU utilization that this ingester can reach before rejecting new - # query request (across all tenants) in percentage, between 0 and 1. - # monitored_resources config must include the resource type. 0 to disable. + # EXPERIMENTAL: Max CPU utilization that this ingester can reach before + # rejecting new query request (across all tenants) in percentage, between 0 + # and 1. monitored_resources config must include the resource type. 0 to + # disable. # CLI flag: -store-gateway.instance-limits.cpu-utilization [cpu_utilization: | default = 0] - # Max heap utilization that this ingester can reach before rejecting new - # query request (across all tenants) in percentage, between 0 and 1. - # monitored_resources config must include the resource type. 0 to disable. + # EXPERIMENTAL: Max heap utilization that this ingester can reach before + # rejecting new query request (across all tenants) in percentage, between 0 + # and 1. monitored_resources config must include the resource type. 0 to + # disable. # CLI flag: -store-gateway.instance-limits.heap-utilization [heap_utilization: | default = 0] diff --git a/docs/configuration/config-file-reference.md b/docs/configuration/config-file-reference.md index 06881f4c499..bcea8943f9a 100644 --- a/docs/configuration/config-file-reference.md +++ b/docs/configuration/config-file-reference.md @@ -69,7 +69,8 @@ Where default_value is the value to use if the environment variable is undefined [http_prefix: | default = "/api/prom"] # Comma-separated list of resources to monitor. Supported values are cpu and -# heap, which tracks metrics that are close estimates. Empty string to disable. +# heap, which tracks metrics from github.com/prometheus/procfs and +# runtime/metrics that are close estimates. Empty string to disable. # CLI flag: -monitored.resources [monitored_resources: | default = ""] diff --git a/pkg/cortex/cortex.go b/pkg/cortex/cortex.go index 858f51499b0..694f9da83b8 100644 --- a/pkg/cortex/cortex.go +++ b/pkg/cortex/cortex.go @@ -147,7 +147,8 @@ func (c *Config) RegisterFlags(f *flag.FlagSet) { c.MonitoredResources = []string{} f.Var(&c.MonitoredResources, "monitored.resources", "Comma-separated list of resources to monitor. "+ - "Supported values are cpu and heap, which tracks metrics that are close estimates. Empty string to disable.") + "Supported values are cpu and heap, which tracks metrics from github.com/prometheus/procfs and runtime/metrics "+ + "that are close estimates. Empty string to disable.") c.API.RegisterFlags(f) c.registerServerFlagsWithChangedDefaultValues(f) From f588d945bd36b5944c8b3f4c5549c110caeeed1e Mon Sep 17 00:00:00 2001 From: Justin Jung Date: Thu, 10 Apr 2025 10:46:04 -0700 Subject: [PATCH 20/29] add debugging log on unsupported resource type Signed-off-by: Justin Jung --- pkg/cortex/cortex.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/cortex/cortex.go b/pkg/cortex/cortex.go index 694f9da83b8..8edd75d4751 100644 --- a/pkg/cortex/cortex.go +++ b/pkg/cortex/cortex.go @@ -248,7 +248,7 @@ func (c *Config) Validate(log log.Logger) error { switch resource.Type(r) { case resource.CPU, resource.Heap: default: - return fmt.Errorf("unsupported resource type to monitor: %s", r) + return fmt.Errorf("unsupported resource type to monitor: %s from list [%s]", r, c.MonitoredResources) } } From 6138a9df384bcbc7332f31df83ec89deb689f983 Mon Sep 17 00:00:00 2001 From: Justin Jung Date: Thu, 10 Apr 2025 11:14:07 -0700 Subject: [PATCH 21/29] test Signed-off-by: Justin Jung --- pkg/cortex/cortex.go | 4 +++- pkg/cortex/cortex_test.go | 9 +++++++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/pkg/cortex/cortex.go b/pkg/cortex/cortex.go index 8edd75d4751..a782ed4d7b8 100644 --- a/pkg/cortex/cortex.go +++ b/pkg/cortex/cortex.go @@ -248,7 +248,9 @@ func (c *Config) Validate(log log.Logger) error { switch resource.Type(r) { case resource.CPU, resource.Heap: default: - return fmt.Errorf("unsupported resource type to monitor: %s from list [%s]", r, c.MonitoredResources) + if len(r) > 0 { + return fmt.Errorf("unsupported resource type to monitor: %s", r) + } } } diff --git a/pkg/cortex/cortex_test.go b/pkg/cortex/cortex_test.go index 551ba62a987..74bf0750a33 100644 --- a/pkg/cortex/cortex_test.go +++ b/pkg/cortex/cortex_test.go @@ -175,6 +175,15 @@ func TestConfigValidation(t *testing.T) { }, expectedError: fmt.Errorf("unsupported resource type to monitor: %s", "wrong"), }, + { + name: "should not fail validation for valid resources to monitor", + getTestConfig: func() *Config { + configuration := newDefaultConfig() + configuration.MonitoredResources = []string{"cpu", "heap"} + return configuration + }, + expectedError: nil, + }, } { t.Run(tc.name, func(t *testing.T) { err := tc.getTestConfig().Validate(nil) From 7bd7ab984ed7fda3abd37ee47709d0aa48cded8b Mon Sep 17 00:00:00 2001 From: Justin Jung Date: Thu, 10 Apr 2025 12:15:11 -0700 Subject: [PATCH 22/29] add more error handling + resource_based_limiter_limit metric Signed-off-by: Justin Jung --- pkg/ingester/ingester.go | 5 ++++- pkg/storegateway/gateway.go | 5 ++++- pkg/storegateway/gateway_test.go | 2 +- pkg/util/limiter/resource_based_limiter.go | 19 +++++++++++++++++-- pkg/util/resource/monitor.go | 15 ++++----------- 5 files changed, 30 insertions(+), 16 deletions(-) diff --git a/pkg/ingester/ingester.go b/pkg/ingester/ingester.go index fb8285d5d91..b0239e6bd35 100644 --- a/pkg/ingester/ingester.go +++ b/pkg/ingester/ingester.go @@ -790,7 +790,10 @@ func New(cfg Config, limits *validation.Overrides, registerer prometheus.Registe if cfg.DefaultLimits.HeapUtilization > 0 { resourceLimits[resource.Heap] = cfg.DefaultLimits.HeapUtilization } - i.resourceBasedLimiter = limiter.NewResourceBasedLimiter(resourceMonitor, resourceLimits) + i.resourceBasedLimiter, err = limiter.NewResourceBasedLimiter(resourceMonitor, resourceLimits, registerer) + if err != nil { + return nil, errors.Wrap(err, "error creating resource based limiter") + } } i.BasicService = services.NewBasicService(i.starting, i.updateLoop, i.stopping) diff --git a/pkg/storegateway/gateway.go b/pkg/storegateway/gateway.go index cbc98c7246e..068eb0dfab0 100644 --- a/pkg/storegateway/gateway.go +++ b/pkg/storegateway/gateway.go @@ -249,7 +249,10 @@ func newStoreGateway(gatewayCfg Config, storageCfg cortex_tsdb.BlocksStorageConf if gatewayCfg.InstanceLimits.HeapUtilization > 0 { resourceLimits[resource.Heap] = gatewayCfg.InstanceLimits.HeapUtilization } - g.resourceBasedLimiter = util_limiter.NewResourceBasedLimiter(resourceMonitor, resourceLimits) + g.resourceBasedLimiter, err = util_limiter.NewResourceBasedLimiter(resourceMonitor, resourceLimits, reg) + if err != nil { + return nil, errors.Wrap(err, "error creating resource based limiter") + } } g.Service = services.NewBasicService(g.starting, g.running, g.stopping) diff --git a/pkg/storegateway/gateway_test.go b/pkg/storegateway/gateway_test.go index b60b585b8d5..374d7a1d3ec 100644 --- a/pkg/storegateway/gateway_test.go +++ b/pkg/storegateway/gateway_test.go @@ -1220,7 +1220,7 @@ func TestStoreGateway_SeriesThrottledByResourceMonitor(t *testing.T) { resource.CPU: 0.5, resource.Heap: 0.5, } - g.resourceBasedLimiter = util_limiter.NewResourceBasedLimiter(&mockResourceMonitor{cpu: 0.4, heap: 0.6}, limits) + g.resourceBasedLimiter, err = util_limiter.NewResourceBasedLimiter(&mockResourceMonitor{cpu: 0.4, heap: 0.6}, limits, nil) srv := newBucketStoreSeriesServer(setUserIDToGRPCContext(ctx, userID)) err = g.Series(req, srv) diff --git a/pkg/util/limiter/resource_based_limiter.go b/pkg/util/limiter/resource_based_limiter.go index fe8cf86a4e3..52ea0621a10 100644 --- a/pkg/util/limiter/resource_based_limiter.go +++ b/pkg/util/limiter/resource_based_limiter.go @@ -3,6 +3,9 @@ package limiter import ( "fmt" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + "github.com/cortexproject/cortex/pkg/util/resource" ) @@ -19,11 +22,23 @@ type ResourceBasedLimiter struct { limits map[resource.Type]float64 } -func NewResourceBasedLimiter(resourceMonitor resource.IMonitor, limits map[resource.Type]float64) *ResourceBasedLimiter { +func NewResourceBasedLimiter(resourceMonitor resource.IMonitor, limits map[resource.Type]float64, registerer prometheus.Registerer) (*ResourceBasedLimiter, error) { + for resType, limit := range limits { + switch resType { + case resource.CPU, resource.Heap: + promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ + Name: "cortex_resource_based_limiter_limit", + ConstLabels: map[string]string{"resource": string(resType)}, + }).Set(limit) + default: + return nil, fmt.Errorf("unsupported resource type: [%s]", resType) + } + } + return &ResourceBasedLimiter{ resourceMonitor: resourceMonitor, limits: limits, - } + }, nil } func (l *ResourceBasedLimiter) AcceptNewRequest() error { diff --git a/pkg/util/resource/monitor.go b/pkg/util/resource/monitor.go index fc07b27bfdf..057fe4e1c87 100644 --- a/pkg/util/resource/monitor.go +++ b/pkg/util/resource/monitor.go @@ -16,8 +16,8 @@ const ( CPU Type = "cpu" Heap Type = "heap" - monitorInterval = time.Second - dataPointsToAvg = 30 + monitorInterval = 100 * time.Millisecond + dataPointsToAvg = 50 ) type Type string @@ -70,6 +70,8 @@ func NewMonitor(limits map[Type]float64, registerer prometheus.Registerer) (*Mon case Heap: scannerFunc = newHeapScanner gaugeFunc = m.GetHeapUtilization + default: + return nil, fmt.Errorf("no scanner available for resource type: [%s]", resType) } s, err := scannerFunc() @@ -85,15 +87,6 @@ func NewMonitor(limits map[Type]float64, registerer prometheus.Registerer) (*Mon }, gaugeFunc) } - //promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ - // Name: "cortex_resource_threshold", - // ConstLabels: map[string]string{"resource": "CPU"}, - //}).Set(thresholds.CPU) - //promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ - // Name: "cortex_resource_threshold", - // ConstLabels: map[string]string{"resource": "Heap"}, - //}).Set(thresholds.Heap) - return m, nil } From 6da53e9f133b822e6db98f1987615cb71b0a3012 Mon Sep 17 00:00:00 2001 From: Justin Jung Date: Thu, 10 Apr 2025 12:21:12 -0700 Subject: [PATCH 23/29] fix test Signed-off-by: Justin Jung --- pkg/ingester/ingester_test.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pkg/ingester/ingester_test.go b/pkg/ingester/ingester_test.go index 92f07822923..becfd5c9c2c 100644 --- a/pkg/ingester/ingester_test.go +++ b/pkg/ingester/ingester_test.go @@ -3078,7 +3078,8 @@ func Test_Ingester_Query_ResourceThresholdBreached(t *testing.T) { resource.CPU: 0.5, resource.Heap: 0.5, } - i.resourceBasedLimiter = limiter.NewResourceBasedLimiter(&mockResourceMonitor{cpu: 0.4, heap: 0.6}, limits) + i.resourceBasedLimiter, err = limiter.NewResourceBasedLimiter(&mockResourceMonitor{cpu: 0.4, heap: 0.6}, limits, nil) + require.NoError(t, err) // Wait until it's ACTIVE test.Poll(t, 1*time.Second, ring.ACTIVE, func() interface{} { From a8d42183763cf6d162c256352ab174963a60aaeb Mon Sep 17 00:00:00 2001 From: Justin Jung Date: Thu, 10 Apr 2025 13:46:49 -0700 Subject: [PATCH 24/29] fix test Signed-off-by: Justin Jung --- pkg/storegateway/gateway_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/pkg/storegateway/gateway_test.go b/pkg/storegateway/gateway_test.go index 374d7a1d3ec..be17e58e8ad 100644 --- a/pkg/storegateway/gateway_test.go +++ b/pkg/storegateway/gateway_test.go @@ -1221,6 +1221,7 @@ func TestStoreGateway_SeriesThrottledByResourceMonitor(t *testing.T) { resource.Heap: 0.5, } g.resourceBasedLimiter, err = util_limiter.NewResourceBasedLimiter(&mockResourceMonitor{cpu: 0.4, heap: 0.6}, limits, nil) + require.NoError(t, err) srv := newBucketStoreSeriesServer(setUserIDToGRPCContext(ctx, userID)) err = g.Series(req, srv) From d6d3839c12ed9aa822a1f9b675678e18a53d83c2 Mon Sep 17 00:00:00 2001 From: Justin Jung Date: Thu, 10 Apr 2025 14:11:16 -0700 Subject: [PATCH 25/29] update changelog Signed-off-by: Justin Jung --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 5581c58277b..9d3d99a2466 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -5,7 +5,7 @@ * [FEATURE] Query Frontend: Add dynamic interval size for query splitting. This is enabled by configuring experimental flags `querier.max-shards-per-query` and/or `querier.max-fetched-data-duration-per-query`. The split interval size is dynamically increased to maintain a number of shards and total duration fetched below the configured values. #6458 * [FEATURE] Querier/Ruler: Add `query_partial_data` and `rules_partial_data` limits to allow queries/rules to be evaluated with data from a single zone, if other zones are not available. #6526 * [FEATURE] Update prometheus alertmanager version to v0.28.0 and add new integration msteamsv2, jira, and rocketchat. #6590 -* [FEATURE] Ingester/StoreGateway: Add `monitored_resources` config + ResourceBasedLimiter in ingesters and store gateways. #6674 +* [FEATURE] Ingester/StoreGateway: Add `ResourceMonitor` module in Cortex, and add `ResourceBasedLimiter` in Ingesters and StoreGateways. #6674 * [FEATURE] Ingester: Support out-of-order native histogram ingestion. It automatically enabled when `-ingester.out-of-order-time-window > 0` and `-blocks-storage.tsdb.enable-native-histograms=true`. #6626 #6663 * [ENHANCEMENT] Alertmanager: Add nflog and silences maintenance metrics. #6659 * [ENHANCEMENT] Querier: limit label APIs to query only ingesters if `start` param is not been specified. #6618 From c68bbd284103fa4f64fd1fc315f546cb214b7e45 Mon Sep 17 00:00:00 2001 From: Justin Jung Date: Tue, 15 Apr 2025 14:15:11 -0700 Subject: [PATCH 26/29] Move noopScanner to scanner.go and fix RegisterFlagsWithPrefix Signed-off-by: Justin Jung --- docs/configuration/config-file-reference.md | 12 ++++++++---- pkg/ingester/instance_limits.go | 3 +-- pkg/util/resource/scanner.go | 6 ++++++ pkg/util/resource/scanner_darwin.go | 6 ------ pkg/util/resource/scanner_test.go | 14 ++++++++++++++ 5 files changed, 29 insertions(+), 12 deletions(-) create mode 100644 pkg/util/resource/scanner_test.go diff --git a/docs/configuration/config-file-reference.md b/docs/configuration/config-file-reference.md index bcea8943f9a..20a501a489f 100644 --- a/docs/configuration/config-file-reference.md +++ b/docs/configuration/config-file-reference.md @@ -3203,13 +3203,17 @@ lifecycler: [upload_compacted_blocks_enabled: | default = true] instance_limits: - # Max CPU utilization that this ingester can reach before rejecting new query - # request (across all tenants) in percentage, between 0 and 1. 0 = unlimited. + # EXPERIMENTAL: Max CPU utilization that this ingester can reach before + # rejecting new query request (across all tenants) in percentage, between 0 + # and 1. monitored_resources config must include the resource type. 0 to + # disable. # CLI flag: -ingester.instance-limits.cpu-utilization [cpu_utilization: | default = 0] - # Max heap utilization that this ingester can reach before rejecting new query - # request (across all tenants) in percentage, between 0 and 1. 0 = unlimited. + # EXPERIMENTAL: Max heap utilization that this ingester can reach before + # rejecting new query request (across all tenants) in percentage, between 0 + # and 1. monitored_resources config must include the resource type. 0 to + # disable. # CLI flag: -ingester.instance-limits.heap-utilization [heap_utilization: | default = 0] diff --git a/pkg/ingester/instance_limits.go b/pkg/ingester/instance_limits.go index c0839e9270a..334d1250e60 100644 --- a/pkg/ingester/instance_limits.go +++ b/pkg/ingester/instance_limits.go @@ -38,8 +38,7 @@ func (cfg *InstanceLimits) RegisterFlagsWithPrefix(f *flag.FlagSet, prefix strin f.Int64Var(&cfg.MaxInMemorySeries, prefix+"instance-limits.max-series", 0, "Max series that this ingester can hold (across all tenants). Requests to create additional series will be rejected. This limit only works when using blocks engine. 0 = unlimited.") f.Int64Var(&cfg.MaxInflightPushRequests, prefix+"instance-limits.max-inflight-push-requests", 0, "Max inflight push requests that this ingester can handle (across all tenants). Additional requests will be rejected. 0 = unlimited.") f.Int64Var(&cfg.MaxInflightQueryRequests, prefix+"instance-limits.max-inflight-query-requests", 0, "Max inflight query requests that this ingester can handle (across all tenants). Additional requests will be rejected. 0 = unlimited.") - f.Float64Var(&cfg.CPUUtilization, prefix+"instance-limits.cpu-utilization", 0, "Max CPU utilization that this ingester can reach before rejecting new query request (across all tenants) in percentage, between 0 and 1. 0 = unlimited.") - f.Float64Var(&cfg.HeapUtilization, prefix+"instance-limits.heap-utilization", 0, "Max heap utilization that this ingester can reach before rejecting new query request (across all tenants) in percentage, between 0 and 1. 0 = unlimited.") + cfg.InstanceLimits.RegisterFlagsWithPrefix(f, prefix) } func (cfg *InstanceLimits) Validate(monitoredResources flagext.StringSliceCSV) error { diff --git a/pkg/util/resource/scanner.go b/pkg/util/resource/scanner.go index 65517243477..1d4f0906c70 100644 --- a/pkg/util/resource/scanner.go +++ b/pkg/util/resource/scanner.go @@ -13,6 +13,12 @@ type scanner interface { scan() (float64, error) } +type noopScanner struct{} + +func (s *noopScanner) scan() (float64, error) { + return 0, nil +} + type heapScanner struct { metricSamples []metrics.Sample } diff --git a/pkg/util/resource/scanner_darwin.go b/pkg/util/resource/scanner_darwin.go index 3d6231124d3..1d4aedfb555 100644 --- a/pkg/util/resource/scanner_darwin.go +++ b/pkg/util/resource/scanner_darwin.go @@ -8,12 +8,6 @@ import ( "github.com/cortexproject/cortex/pkg/util/log" ) -type noopScanner struct{} - -func (s *noopScanner) scan() (float64, error) { - return 0, nil -} - func newCPUScanner() (scanner, error) { level.Warn(log.Logger).Log("msg", "CPU scanner not supported in darwin.") return &noopScanner{}, nil diff --git a/pkg/util/resource/scanner_test.go b/pkg/util/resource/scanner_test.go new file mode 100644 index 00000000000..0e632d21e7f --- /dev/null +++ b/pkg/util/resource/scanner_test.go @@ -0,0 +1,14 @@ +package resource + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func Test_NoopScanner(t *testing.T) { + s := noopScanner{} + val, err := s.scan() + require.NoError(t, err) + require.Zero(t, val) +} From 025a93aa70c101157216c824add2d7c30d49d04e Mon Sep 17 00:00:00 2001 From: Justin Jung Date: Wed, 16 Apr 2025 11:53:27 -0700 Subject: [PATCH 27/29] Add limit breached metric + wrap error with 429 Signed-off-by: Justin Jung --- pkg/ingester/ingester.go | 2 +- pkg/storegateway/gateway.go | 3 ++- pkg/util/limiter/resource_based_limiter.go | 13 +++++++++++-- 3 files changed, 14 insertions(+), 4 deletions(-) diff --git a/pkg/ingester/ingester.go b/pkg/ingester/ingester.go index b0239e6bd35..6225b5eee75 100644 --- a/pkg/ingester/ingester.go +++ b/pkg/ingester/ingester.go @@ -2172,7 +2172,7 @@ func (i *Ingester) trackInflightQueryRequest() (func(), error) { if i.resourceBasedLimiter != nil { if err := i.resourceBasedLimiter.AcceptNewRequest(); err != nil { level.Warn(i.logger).Log("msg", "failed to accept request", "err", err) - return nil, fmt.Errorf("failed to query: %s", limiter.ErrResourceLimitReachedStr) + return nil, httpgrpc.Errorf(http.StatusTooManyRequests, "failed to query: %s", limiter.ErrResourceLimitReachedStr) } } diff --git a/pkg/storegateway/gateway.go b/pkg/storegateway/gateway.go index 068eb0dfab0..9f4e7467329 100644 --- a/pkg/storegateway/gateway.go +++ b/pkg/storegateway/gateway.go @@ -16,6 +16,7 @@ import ( "github.com/thanos-io/objstore" "github.com/thanos-io/thanos/pkg/extprom" "github.com/thanos-io/thanos/pkg/store/storepb" + "github.com/weaveworks/common/httpgrpc" "github.com/weaveworks/common/logging" "github.com/cortexproject/cortex/pkg/configs" @@ -436,7 +437,7 @@ func (g *StoreGateway) checkResourceUtilization() error { if err := g.resourceBasedLimiter.AcceptNewRequest(); err != nil { level.Warn(g.logger).Log("msg", "failed to accept request", "err", err) - return fmt.Errorf("failed to query: %s", util_limiter.ErrResourceLimitReachedStr) + return httpgrpc.Errorf(http.StatusTooManyRequests, "failed to query: %s", util_limiter.ErrResourceLimitReachedStr) } return nil diff --git a/pkg/util/limiter/resource_based_limiter.go b/pkg/util/limiter/resource_based_limiter.go index 52ea0621a10..d0272a9994c 100644 --- a/pkg/util/limiter/resource_based_limiter.go +++ b/pkg/util/limiter/resource_based_limiter.go @@ -18,8 +18,9 @@ func (e *ResourceLimitReachedError) Error() string { } type ResourceBasedLimiter struct { - resourceMonitor resource.IMonitor - limits map[resource.Type]float64 + resourceMonitor resource.IMonitor + limits map[resource.Type]float64 + limitBreachedCount *prometheus.CounterVec } func NewResourceBasedLimiter(resourceMonitor resource.IMonitor, limits map[resource.Type]float64, registerer prometheus.Registerer) (*ResourceBasedLimiter, error) { @@ -38,6 +39,13 @@ func NewResourceBasedLimiter(resourceMonitor resource.IMonitor, limits map[resou return &ResourceBasedLimiter{ resourceMonitor: resourceMonitor, limits: limits, + limitBreachedCount: promauto.With(registerer).NewCounterVec( + prometheus.CounterOpts{ + Name: "cortex_resource_based_limiter_limit_breached", + Help: "The total number of times resource based limiter was throttled.", + }, + []string{"resource"}, + ), }, nil } @@ -53,6 +61,7 @@ func (l *ResourceBasedLimiter) AcceptNewRequest() error { } if utilization >= limit { + l.limitBreachedCount.WithLabelValues(string(resType)).Inc() return fmt.Errorf("%s utilization limit reached (limit: %.3f, utilization: %.3f)", resType, limit, utilization) } } From 6ffef630b532c3bb79d2ce8b88c77db3b7da0cc6 Mon Sep 17 00:00:00 2001 From: Justin Jung Date: Wed, 16 Apr 2025 16:03:36 -0700 Subject: [PATCH 28/29] Add more validation and test on instance_limits Signed-off-by: Justin Jung --- pkg/configs/instance_limits.go | 8 +++ pkg/configs/instance_limits_test.go | 64 ++++++++++++++++++++++ pkg/ingester/ingester.go | 2 +- pkg/ingester/ingester_test.go | 2 +- pkg/storegateway/gateway.go | 2 +- pkg/storegateway/gateway_test.go | 2 +- pkg/util/limiter/resource_based_limiter.go | 10 ++-- 7 files changed, 82 insertions(+), 8 deletions(-) create mode 100644 pkg/configs/instance_limits_test.go diff --git a/pkg/configs/instance_limits.go b/pkg/configs/instance_limits.go index 51164bd75fa..8273400847c 100644 --- a/pkg/configs/instance_limits.go +++ b/pkg/configs/instance_limits.go @@ -20,10 +20,18 @@ func (cfg *InstanceLimits) RegisterFlagsWithPrefix(f *flag.FlagSet, prefix strin } func (cfg *InstanceLimits) Validate(monitoredResources flagext.StringSliceCSV) error { + if cfg.CPUUtilization > 1 || cfg.CPUUtilization < 0 { + return errors.New("cpu_utilization must be between 0 and 1") + } + if cfg.CPUUtilization > 0 && !strings.Contains(monitoredResources.String(), string(resource.CPU)) { return errors.New("monitored_resources config must include \"cpu\" as well") } + if cfg.HeapUtilization > 1 || cfg.HeapUtilization < 0 { + return errors.New("heap_utilization must be between 0 and 1") + } + if cfg.HeapUtilization > 0 && !strings.Contains(monitoredResources.String(), string(resource.Heap)) { return errors.New("monitored_resources config must include \"heap\" as well") } diff --git a/pkg/configs/instance_limits_test.go b/pkg/configs/instance_limits_test.go new file mode 100644 index 00000000000..a1d3686dba6 --- /dev/null +++ b/pkg/configs/instance_limits_test.go @@ -0,0 +1,64 @@ +package configs + +import ( + "errors" + "testing" + + "github.com/stretchr/testify/require" +) + +func Test_Validate(t *testing.T) { + for name, tc := range map[string]struct { + instanceLimits InstanceLimits + monitoredResources []string + err error + }{ + "correct config should pass validation": { + instanceLimits: InstanceLimits{ + CPUUtilization: 0.5, + HeapUtilization: 0.5, + }, + monitoredResources: []string{"cpu", "heap"}, + err: nil, + }, + "utilization config less than 0 should fail validation": { + instanceLimits: InstanceLimits{ + CPUUtilization: -0.5, + HeapUtilization: 0.5, + }, + monitoredResources: []string{"cpu", "heap"}, + err: errors.New("cpu_utilization must be between 0 and 1"), + }, + "utilization config greater than 1 should fail validation": { + instanceLimits: InstanceLimits{ + CPUUtilization: 0.5, + HeapUtilization: 1.5, + }, + monitoredResources: []string{"cpu", "heap"}, + err: errors.New("heap_utilization must be between 0 and 1"), + }, + "missing cpu in monitored_resources config should fail validation": { + instanceLimits: InstanceLimits{ + CPUUtilization: 0.5, + }, + monitoredResources: []string{"heap"}, + err: errors.New("monitored_resources config must include \"cpu\" as well"), + }, + "missing heap in monitored_resources config should fail validation": { + instanceLimits: InstanceLimits{ + HeapUtilization: 0.5, + }, + monitoredResources: []string{"cpu"}, + err: errors.New("monitored_resources config must include \"heap\" as well"), + }, + } { + t.Run(name, func(t *testing.T) { + err := tc.instanceLimits.Validate(tc.monitoredResources) + if tc.err != nil { + require.Errorf(t, err, tc.err.Error()) + } else { + require.NoError(t, err) + } + }) + } +} diff --git a/pkg/ingester/ingester.go b/pkg/ingester/ingester.go index 6225b5eee75..7eac9dbead3 100644 --- a/pkg/ingester/ingester.go +++ b/pkg/ingester/ingester.go @@ -790,7 +790,7 @@ func New(cfg Config, limits *validation.Overrides, registerer prometheus.Registe if cfg.DefaultLimits.HeapUtilization > 0 { resourceLimits[resource.Heap] = cfg.DefaultLimits.HeapUtilization } - i.resourceBasedLimiter, err = limiter.NewResourceBasedLimiter(resourceMonitor, resourceLimits, registerer) + i.resourceBasedLimiter, err = limiter.NewResourceBasedLimiter(resourceMonitor, resourceLimits, registerer, "ingester") if err != nil { return nil, errors.Wrap(err, "error creating resource based limiter") } diff --git a/pkg/ingester/ingester_test.go b/pkg/ingester/ingester_test.go index becfd5c9c2c..35b28fbb35f 100644 --- a/pkg/ingester/ingester_test.go +++ b/pkg/ingester/ingester_test.go @@ -3078,7 +3078,7 @@ func Test_Ingester_Query_ResourceThresholdBreached(t *testing.T) { resource.CPU: 0.5, resource.Heap: 0.5, } - i.resourceBasedLimiter, err = limiter.NewResourceBasedLimiter(&mockResourceMonitor{cpu: 0.4, heap: 0.6}, limits, nil) + i.resourceBasedLimiter, err = limiter.NewResourceBasedLimiter(&mockResourceMonitor{cpu: 0.4, heap: 0.6}, limits, nil, "ingester") require.NoError(t, err) // Wait until it's ACTIVE diff --git a/pkg/storegateway/gateway.go b/pkg/storegateway/gateway.go index 9f4e7467329..835e95e8912 100644 --- a/pkg/storegateway/gateway.go +++ b/pkg/storegateway/gateway.go @@ -250,7 +250,7 @@ func newStoreGateway(gatewayCfg Config, storageCfg cortex_tsdb.BlocksStorageConf if gatewayCfg.InstanceLimits.HeapUtilization > 0 { resourceLimits[resource.Heap] = gatewayCfg.InstanceLimits.HeapUtilization } - g.resourceBasedLimiter, err = util_limiter.NewResourceBasedLimiter(resourceMonitor, resourceLimits, reg) + g.resourceBasedLimiter, err = util_limiter.NewResourceBasedLimiter(resourceMonitor, resourceLimits, reg, "store-gateway") if err != nil { return nil, errors.Wrap(err, "error creating resource based limiter") } diff --git a/pkg/storegateway/gateway_test.go b/pkg/storegateway/gateway_test.go index be17e58e8ad..ca35034bd43 100644 --- a/pkg/storegateway/gateway_test.go +++ b/pkg/storegateway/gateway_test.go @@ -1220,7 +1220,7 @@ func TestStoreGateway_SeriesThrottledByResourceMonitor(t *testing.T) { resource.CPU: 0.5, resource.Heap: 0.5, } - g.resourceBasedLimiter, err = util_limiter.NewResourceBasedLimiter(&mockResourceMonitor{cpu: 0.4, heap: 0.6}, limits, nil) + g.resourceBasedLimiter, err = util_limiter.NewResourceBasedLimiter(&mockResourceMonitor{cpu: 0.4, heap: 0.6}, limits, nil, "store-gateway") require.NoError(t, err) srv := newBucketStoreSeriesServer(setUserIDToGRPCContext(ctx, userID)) diff --git a/pkg/util/limiter/resource_based_limiter.go b/pkg/util/limiter/resource_based_limiter.go index d0272a9994c..c1da4a432dd 100644 --- a/pkg/util/limiter/resource_based_limiter.go +++ b/pkg/util/limiter/resource_based_limiter.go @@ -23,13 +23,14 @@ type ResourceBasedLimiter struct { limitBreachedCount *prometheus.CounterVec } -func NewResourceBasedLimiter(resourceMonitor resource.IMonitor, limits map[resource.Type]float64, registerer prometheus.Registerer) (*ResourceBasedLimiter, error) { +func NewResourceBasedLimiter(resourceMonitor resource.IMonitor, limits map[resource.Type]float64, registerer prometheus.Registerer, component string) (*ResourceBasedLimiter, error) { for resType, limit := range limits { switch resType { case resource.CPU, resource.Heap: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ Name: "cortex_resource_based_limiter_limit", - ConstLabels: map[string]string{"resource": string(resType)}, + Help: "Limit set for the resource utilization.", + ConstLabels: map[string]string{"component": component}, }).Set(limit) default: return nil, fmt.Errorf("unsupported resource type: [%s]", resType) @@ -41,8 +42,9 @@ func NewResourceBasedLimiter(resourceMonitor resource.IMonitor, limits map[resou limits: limits, limitBreachedCount: promauto.With(registerer).NewCounterVec( prometheus.CounterOpts{ - Name: "cortex_resource_based_limiter_limit_breached", - Help: "The total number of times resource based limiter was throttled.", + Name: "cortex_resource_based_limiter_limit_breached", + Help: "The total number of times resource based limiter was throttled.", + ConstLabels: map[string]string{"component": component}, }, []string{"resource"}, ), From 7808940ed3241af8dcbdcc68c1d274a5fe042492 Mon Sep 17 00:00:00 2001 From: Justin Jung Date: Thu, 17 Apr 2025 16:57:06 -0700 Subject: [PATCH 29/29] Added _total to counter metric Signed-off-by: Justin Jung --- pkg/util/limiter/resource_based_limiter.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/util/limiter/resource_based_limiter.go b/pkg/util/limiter/resource_based_limiter.go index c1da4a432dd..40e4768cd0c 100644 --- a/pkg/util/limiter/resource_based_limiter.go +++ b/pkg/util/limiter/resource_based_limiter.go @@ -42,8 +42,8 @@ func NewResourceBasedLimiter(resourceMonitor resource.IMonitor, limits map[resou limits: limits, limitBreachedCount: promauto.With(registerer).NewCounterVec( prometheus.CounterOpts{ - Name: "cortex_resource_based_limiter_limit_breached", - Help: "The total number of times resource based limiter was throttled.", + Name: "cortex_resource_based_limiter_throttled_total", + Help: "The total number of times resource based limiter throttled.", ConstLabels: map[string]string{"component": component}, }, []string{"resource"},