[vmagent] added ingestion rate limiting with new flag -maxIngestionRate (#5900)

* [vmagent] added ingestion rate limiting with new flag `-maxIngestionRate`. This flag can be used to limit the number of samples ingested by vmagent per second. If the limit is exceeded, the ingestion rate will be throttled.

* fix changelog

* fix review comment
This commit is contained in:
Alexander Marshalov 2024-03-21 17:14:49 +01:00 committed by GitHub
parent db3709c87d
commit 02bccd1eb9
No known key found for this signature in database
GPG key ID: B5690EEEBB952194
4 changed files with 48 additions and 13 deletions

View file

@ -91,7 +91,7 @@ type client struct {
authCfg *promauth.Config authCfg *promauth.Config
awsCfg *awsapi.Config awsCfg *awsapi.Config
rl rateLimiter rl *rateLimiter
bytesSent *metrics.Counter bytesSent *metrics.Counter
blocksSent *metrics.Counter blocksSent *metrics.Counter
@ -177,12 +177,11 @@ func newHTTPClient(argIdx int, remoteWriteURL, sanitizedURL string, fq *persiste
} }
func (c *client) init(argIdx, concurrency int, sanitizedURL string) { func (c *client) init(argIdx, concurrency int, sanitizedURL string) {
limitReached := metrics.GetOrCreateCounter(fmt.Sprintf(`vmagent_remotewrite_rate_limit_reached_total{url=%q}`, c.sanitizedURL))
if bytesPerSec := rateLimit.GetOptionalArg(argIdx); bytesPerSec > 0 { if bytesPerSec := rateLimit.GetOptionalArg(argIdx); bytesPerSec > 0 {
logger.Infof("applying %d bytes per second rate limit for -remoteWrite.url=%q", bytesPerSec, sanitizedURL) logger.Infof("applying %d bytes per second rate limit for -remoteWrite.url=%q", bytesPerSec, sanitizedURL)
c.rl.perSecondLimit = int64(bytesPerSec) c.rl = newRateLimiter(time.Second, int64(bytesPerSec), limitReached, c.stopCh)
} }
c.rl.limitReached = metrics.GetOrCreateCounter(fmt.Sprintf(`vmagent_remotewrite_rate_limit_reached_total{url=%q}`, c.sanitizedURL))
c.bytesSent = metrics.GetOrCreateCounter(fmt.Sprintf(`vmagent_remotewrite_bytes_sent_total{url=%q}`, c.sanitizedURL)) c.bytesSent = metrics.GetOrCreateCounter(fmt.Sprintf(`vmagent_remotewrite_bytes_sent_total{url=%q}`, c.sanitizedURL))
c.blocksSent = metrics.GetOrCreateCounter(fmt.Sprintf(`vmagent_remotewrite_blocks_sent_total{url=%q}`, c.sanitizedURL)) c.blocksSent = metrics.GetOrCreateCounter(fmt.Sprintf(`vmagent_remotewrite_blocks_sent_total{url=%q}`, c.sanitizedURL))
c.rateLimit = metrics.GetOrCreateGauge(fmt.Sprintf(`vmagent_remotewrite_rate_limit{url=%q}`, c.sanitizedURL), func() float64 { c.rateLimit = metrics.GetOrCreateGauge(fmt.Sprintf(`vmagent_remotewrite_rate_limit{url=%q}`, c.sanitizedURL), func() float64 {
@ -396,7 +395,7 @@ func (c *client) newRequest(url string, body []byte) (*http.Request, error) {
// The function returns false only if c.stopCh is closed. // The function returns false only if c.stopCh is closed.
// Otherwise it tries sending the block to remote storage indefinitely. // Otherwise it tries sending the block to remote storage indefinitely.
func (c *client) sendBlockHTTP(block []byte) bool { func (c *client) sendBlockHTTP(block []byte) bool {
c.rl.register(len(block), c.stopCh) c.rl.register(len(block))
maxRetryDuration := timeutil.AddJitterToDuration(time.Minute) maxRetryDuration := timeutil.AddJitterToDuration(time.Minute)
retryDuration := timeutil.AddJitterToDuration(time.Second) retryDuration := timeutil.AddJitterToDuration(time.Second)
retriesCount := 0 retriesCount := 0
@ -481,22 +480,37 @@ again:
var remoteWriteRejectedLogger = logger.WithThrottler("remoteWriteRejected", 5*time.Second) var remoteWriteRejectedLogger = logger.WithThrottler("remoteWriteRejected", 5*time.Second)
type rateLimiter struct { type rateLimiter struct {
perSecondLimit int64 interval time.Duration
perIntervalLimit int64
stopCh <-chan struct{}
// mu protects budget and deadline from concurrent access. // mu protects budget and deadline from concurrent access.
mu sync.Mutex mu sync.Mutex
// The current budget. It is increased by perSecondLimit every second. // The current budget. It is increased by perIntervalLimit every interval.
budget int64 budget int64
// The next deadline for increasing the budget by perSecondLimit // The next deadline for increasing the budget by perIntervalLimit
deadline time.Time deadline time.Time
limitReached *metrics.Counter limitReached *metrics.Counter
} }
func (rl *rateLimiter) register(dataLen int, stopCh <-chan struct{}) { func newRateLimiter(interval time.Duration, perIntervalLimit int64, limitReached *metrics.Counter, stopCh <-chan struct{}) *rateLimiter {
limit := rl.perSecondLimit return &rateLimiter{
interval: interval,
perIntervalLimit: perIntervalLimit,
stopCh: stopCh,
limitReached: limitReached,
}
}
func (rl *rateLimiter) register(count int) {
if rl == nil {
return
}
limit := rl.perIntervalLimit
if limit <= 0 { if limit <= 0 {
return return
} }
@ -509,7 +523,7 @@ func (rl *rateLimiter) register(dataLen int, stopCh <-chan struct{}) {
rl.limitReached.Inc() rl.limitReached.Inc()
t := timerpool.Get(d) t := timerpool.Get(d)
select { select {
case <-stopCh: case <-rl.stopCh:
timerpool.Put(t) timerpool.Put(t)
return return
case <-t.C: case <-t.C:
@ -517,7 +531,7 @@ func (rl *rateLimiter) register(dataLen int, stopCh <-chan struct{}) {
} }
} }
rl.budget += limit rl.budget += limit
rl.deadline = time.Now().Add(time.Second) rl.deadline = time.Now().Add(rl.interval)
} }
rl.budget -= int64(dataLen) rl.budget -= int64(count)
} }

View file

@ -80,6 +80,8 @@ var (
"Excess series are logged and dropped. This can be useful for limiting series cardinality. See https://docs.victoriametrics.com/vmagent.html#cardinality-limiter") "Excess series are logged and dropped. This can be useful for limiting series cardinality. See https://docs.victoriametrics.com/vmagent.html#cardinality-limiter")
maxDailySeries = flag.Int("remoteWrite.maxDailySeries", 0, "The maximum number of unique series vmagent can send to remote storage systems during the last 24 hours. "+ maxDailySeries = flag.Int("remoteWrite.maxDailySeries", 0, "The maximum number of unique series vmagent can send to remote storage systems during the last 24 hours. "+
"Excess series are logged and dropped. This can be useful for limiting series churn rate. See https://docs.victoriametrics.com/vmagent.html#cardinality-limiter") "Excess series are logged and dropped. This can be useful for limiting series churn rate. See https://docs.victoriametrics.com/vmagent.html#cardinality-limiter")
maxIngestionRate = flag.Int("maxIngestionRate", 0, "The maximum number of samples vmagent can receive per second. "+
"If the limit is exceeded, the ingestion rate will be throttled.")
streamAggrConfig = flagutil.NewArrayString("remoteWrite.streamAggr.config", "Optional path to file with stream aggregation config. "+ streamAggrConfig = flagutil.NewArrayString("remoteWrite.streamAggr.config", "Optional path to file with stream aggregation config. "+
"See https://docs.victoriametrics.com/stream-aggregation.html . "+ "See https://docs.victoriametrics.com/stream-aggregation.html . "+
@ -178,6 +180,12 @@ func Init() {
return float64(dailySeriesLimiter.CurrentItems()) return float64(dailySeriesLimiter.CurrentItems())
}) })
} }
if *maxIngestionRate > 0 {
// Start ingestion rate limiter.
ingestionRateLimitReached = metrics.NewCounter(`vmagent_max_ingestion_rate_limit_reached_total`)
ingestionRateLimiterStopCh = make(chan struct{})
ingestionRateLimiter = newRateLimiter(time.Second, int64(*maxIngestionRate), ingestionRateLimitReached, ingestionRateLimiterStopCh)
}
if *queues > maxQueues { if *queues > maxQueues {
*queues = maxQueues *queues = maxQueues
} }
@ -346,6 +354,9 @@ var configReloaderWG sync.WaitGroup
// //
// It is expected that nobody calls TryPush during and after the call to this func. // It is expected that nobody calls TryPush during and after the call to this func.
func Stop() { func Stop() {
if ingestionRateLimiterStopCh != nil {
close(ingestionRateLimiterStopCh)
}
close(configReloaderStopCh) close(configReloaderStopCh)
configReloaderWG.Wait() configReloaderWG.Wait()
@ -468,6 +479,9 @@ func tryPush(at *auth.Token, wr *prompbmarshal.WriteRequest, dropSamplesOnFailur
break break
} }
} }
ingestionRateLimiter.register(samplesCount)
tssBlock := tss tssBlock := tss
if i < len(tss) { if i < len(tss) {
tssBlock = tss[:i] tssBlock = tss[:i]
@ -612,6 +626,10 @@ func limitSeriesCardinality(tss []prompbmarshal.TimeSeries) []prompbmarshal.Time
} }
var ( var (
ingestionRateLimiter *rateLimiter
ingestionRateLimiterStopCh chan struct{}
ingestionRateLimitReached *metrics.Counter
hourlySeriesLimiter *bloomfilter.Limiter hourlySeriesLimiter *bloomfilter.Limiter
dailySeriesLimiter *bloomfilter.Limiter dailySeriesLimiter *bloomfilter.Limiter

View file

@ -85,6 +85,7 @@ Released at 2024-03-01
* FEATURE: [vmui](https://docs.victoriametrics.com/#vmui): allow filling gaps on graphs with interpolated lines as Grafana does. See [this feature request](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/5152) and [this pull request](https://github.com/VictoriaMetrics/VictoriaMetrics/pull/5862). * FEATURE: [vmui](https://docs.victoriametrics.com/#vmui): allow filling gaps on graphs with interpolated lines as Grafana does. See [this feature request](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/5152) and [this pull request](https://github.com/VictoriaMetrics/VictoriaMetrics/pull/5862).
* FEATURE: [vmalert](https://docs.victoriametrics.com/#vmalert): support filtering by group, rule or labels in [vmalert's UI](https://docs.victoriametrics.com/vmalert/#web) for `/groups` and `/alerts` pages. See [the pull request](https://github.com/VictoriaMetrics/VictoriaMetrics/pull/5791) by @victoramsantos. * FEATURE: [vmalert](https://docs.victoriametrics.com/#vmalert): support filtering by group, rule or labels in [vmalert's UI](https://docs.victoriametrics.com/vmalert/#web) for `/groups` and `/alerts` pages. See [the pull request](https://github.com/VictoriaMetrics/VictoriaMetrics/pull/5791) by @victoramsantos.
* FEATURE: [docker-compose](https://github.com/VictoriaMetrics/VictoriaMetrics/tree/master/deployment/docker#docker-compose-environment-for-victoriametrics): create a separate [docker-compose environment](https://github.com/VictoriaMetrics/VictoriaMetrics/blob/master/deployment/docker/docker-compose-victorialogs.yml) for VictoriaLogs installation, including fluentbit and [VictoriaLogs Grafana datasource](https://github.com/VictoriaMetrics/victorialogs-datasource). See [these docs](https://github.com/VictoriaMetrics/VictoriaMetrics/tree/master/deployment/docker#victorialogs-server) for details. * FEATURE: [docker-compose](https://github.com/VictoriaMetrics/VictoriaMetrics/tree/master/deployment/docker#docker-compose-environment-for-victoriametrics): create a separate [docker-compose environment](https://github.com/VictoriaMetrics/VictoriaMetrics/blob/master/deployment/docker/docker-compose-victorialogs.yml) for VictoriaLogs installation, including fluentbit and [VictoriaLogs Grafana datasource](https://github.com/VictoriaMetrics/victorialogs-datasource). See [these docs](https://github.com/VictoriaMetrics/VictoriaMetrics/tree/master/deployment/docker#victorialogs-server) for details.
* FEATURE: [vmagent](https://docs.victoriametrics.com/vmagent.html): add ingestion rate limiting with new flag `-maxIngestionRate`. This flag can be used to limit the number of samples ingested by vmagent per second.
* FEATURE: [vmbackupmanager](https://docs.victoriametrics.com/vmbackupmanager/): wait for up 30 seconds before making a [snapshot](https://docs.victoriametrics.com/#how-to-work-with-snapshots) for backup if `vmstorage` is temporarily unavailalbe. This should prevent from `vmbackupmanager` termination in this case. See [this feature request](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/5859). * FEATURE: [vmbackupmanager](https://docs.victoriametrics.com/vmbackupmanager/): wait for up 30 seconds before making a [snapshot](https://docs.victoriametrics.com/#how-to-work-with-snapshots) for backup if `vmstorage` is temporarily unavailalbe. This should prevent from `vmbackupmanager` termination in this case. See [this feature request](https://github.com/VictoriaMetrics/VictoriaMetrics/issues/5859).
* BUGFIX: downgrade Go builder from `1.22.0` to `1.21.7`, since `1.22.0` contains [the bug](https://github.com/golang/go/issues/65705), which can lead to deadlocked HTTP connections to remote storage systems, scrape targets and service discovery endpoints at [vmagent](https://docs.victoriametrics.com/vmagent/). This may result in incorrect service discovery, target scraping and failed sending samples to remote storage. * BUGFIX: downgrade Go builder from `1.22.0` to `1.21.7`, since `1.22.0` contains [the bug](https://github.com/golang/go/issues/65705), which can lead to deadlocked HTTP connections to remote storage systems, scrape targets and service discovery endpoints at [vmagent](https://docs.victoriametrics.com/vmagent/). This may result in incorrect service discovery, target scraping and failed sending samples to remote storage.

View file

@ -1803,6 +1803,8 @@ See the docs at https://docs.victoriametrics.com/vmagent.html .
Per-second limit on the number of WARN messages. If more than the given number of warns are emitted per second, then the remaining warns are suppressed. Zero values disable the rate limit Per-second limit on the number of WARN messages. If more than the given number of warns are emitted per second, then the remaining warns are suppressed. Zero values disable the rate limit
-maxConcurrentInserts int -maxConcurrentInserts int
The maximum number of concurrent insert requests. Default value should work for most cases, since it minimizes the memory usage. The default value can be increased when clients send data over slow networks. See also -insert.maxQueueDuration (default 32) The maximum number of concurrent insert requests. Default value should work for most cases, since it minimizes the memory usage. The default value can be increased when clients send data over slow networks. See also -insert.maxQueueDuration (default 32)
-maxIngestionRate int
The maximum number of samples vmagent can receive per second. If the limit is exceeded, the ingestion rate will be throttled (default 0)
-maxInsertRequestSize size -maxInsertRequestSize size
The maximum size in bytes of a single Prometheus remote_write API request The maximum size in bytes of a single Prometheus remote_write API request
Supports the following optional suffixes for size values: KB, MB, GB, TB, KiB, MiB, GiB, TiB (default 33554432) Supports the following optional suffixes for size values: KB, MB, GB, TB, KiB, MiB, GiB, TiB (default 33554432)