Skip to content

Commit

Permalink
Add distributor limits (grafana/phlare#510)
Browse files Browse the repository at this point in the history
This PR introduce limits and runtime config with it to allow reloading limits per tenants.

For now only limits of ingestion are implemented but I'm planning to follow with more.

The overrides exporter is also added to export those limits as Prometheus metrics.

Usage:

`phlare -runtime-config.file=./overrides.yaml --config.file=./phlare.yaml`

```
cat overrides.yaml
overrides:
  "anonymous":
    ingestion_rate_mb: 0.1
    ingestion_burst_size_mb: 0.001
```

A lot of this code is adapted from Loki and Mimir.
  • Loading branch information
cyriltovena authored Feb 20, 2023
1 parent 5f5223a commit 53e7117
Show file tree
Hide file tree
Showing 29 changed files with 2,567 additions and 125 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -167,6 +167,60 @@ client:
# The frontend_worker block configures the frontend-worker.
[frontend_worker: <frontend_worker>]

limits:
# Per-tenant ingestion rate limit in sample size per second. Units in MB.
# CLI flag: -distributor.ingestion-rate-limit-mb
[ingestion_rate_mb: <float> | default = 4]

# Per-tenant allowed ingestion burst size (in sample size). Units in MB. The
# burst size refers to the per-distributor local rate limiter, and should be
# set at least to the maximum profile size expected in a single push request.
# CLI flag: -distributor.ingestion-burst-size-mb
[ingestion_burst_size_mb: <float> | default = 2]

# Maximum length accepted for label names.
# CLI flag: -validation.max-length-label-name
[max_label_name_length: <int> | default = 1024]

# Maximum length accepted for label value. This setting also applies to the
# metric name.
# CLI flag: -validation.max-length-label-value
[max_label_value_length: <int> | default = 2048]

# Maximum number of label names per series.
# CLI flag: -validation.max-label-names-per-series
[max_label_names_per_series: <int> | default = 30]

# Maximum number of active series of profiles per tenant, per ingester. 0 to
# disable.
# CLI flag: -ingester.max-series-per-tenant
[max_series_per_tenant: <int> | default = 0]

# Maximum number of active series of profiles per tenant, across the cluster.
# 0 to disable. When the global limit is enabled, each ingester is configured
# with a dynamic local limit based on the replication factor and the current
# number of healthy ingesters, and is kept updated whenever the number of
# ingesters change.
# CLI flag: -ingester.max-global-series-per-tenant
[max_global_series_per_user: <int> | default = 5000]

# Limit how far back in profiling data can be queried, up until lookback
# duration ago. This limit is enforced in the query frontend. If the requested
# time range is outside the allowed range, the request will not fail, but will
# be modified to only query data within the allowed time range. The default
# value of 0 does not set a limit.
# CLI flag: -querier.max-query-lookback
[max_query_lookback: <duration> | default = 0s]

# The limit to length of queries. 0 to disable.
# CLI flag: -querier.max-query-length
[max_query_length: <duration> | default = 30d1h]

# Maximum number of queries that will be scheduled in parallel by the
# frontend.
# CLI flag: -querier.max-query-parallelism
[max_query_parallelism: <int> | default = 32]

# The query_scheduler block configures the query-scheduler.
[query_scheduler: <query_scheduler>]

Expand Down Expand Up @@ -194,6 +248,16 @@ tracing:
# CLI flag: -tracing.enabled
[enabled: <boolean> | default = true]

runtime_config:
# How often to check runtime config files.
# CLI flag: -runtime-config.reload-period
[period: <duration> | default = 10s]

# Comma separated list of yaml files with the configuration that can be
# updated at runtime. Runtime config files will be merged from left to right.
# CLI flag: -runtime-config.file
[file: <string> | default = ""]

storage:
# Backend storage to use. Supported backends are: s3, gcs, azure, swift,
# filesystem, cos.
Expand Down
2 changes: 1 addition & 1 deletion go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,7 @@ require (
golang.org/x/sync v0.1.0
golang.org/x/sys v0.4.0
golang.org/x/text v0.6.0
golang.org/x/time v0.3.0
google.golang.org/genproto v0.0.0-20221227171554-f9683d7f8bef
google.golang.org/grpc v1.51.0
google.golang.org/protobuf v1.28.1
Expand Down Expand Up @@ -235,7 +236,6 @@ require (
golang.org/x/mod v0.7.0 // indirect
golang.org/x/oauth2 v0.3.0 // indirect
golang.org/x/term v0.4.0 // indirect
golang.org/x/time v0.3.0 // indirect
golang.org/x/tools v0.4.0 // indirect
golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect
google.golang.org/api v0.105.0 // indirect
Expand Down
170 changes: 155 additions & 15 deletions pkg/distributor/distributor.go
Original file line number Diff line number Diff line change
Expand Up @@ -6,12 +6,16 @@ import (
"flag"
"fmt"
"hash/fnv"
"net/http"
"strconv"
"time"

"github.com/bufbuild/connect-go"
"github.com/dustin/go-humanize"
"github.com/go-kit/log"
"github.com/google/uuid"
"github.com/grafana/dskit/kv"
"github.com/grafana/dskit/limiter"
"github.com/grafana/dskit/ring"
ring_client "github.com/grafana/dskit/ring/client"
"github.com/grafana/dskit/services"
Expand All @@ -29,12 +33,23 @@ import (
"github.com/grafana/phlare/pkg/pprof"
"github.com/grafana/phlare/pkg/tenant"
"github.com/grafana/phlare/pkg/usagestats"
"github.com/grafana/phlare/pkg/util"
"github.com/grafana/phlare/pkg/validation"
)

type PushClient interface {
Push(context.Context, *connect.Request[pushv1.PushRequest]) (*connect.Response[pushv1.PushResponse], error)
}

const (
// distributorRingKey is the key under which we store the distributors ring in the KVStore.
distributorRingKey = "distributor"

// ringAutoForgetUnhealthyPeriods is how many consecutive timeout periods an unhealthy instance
// in the ring will be automatically removed after.
ringAutoForgetUnhealthyPeriods = 10
)

// todo: move to non global metrics.
var (
clients = promauto.NewGauge(prometheus.GaugeOpts{
Expand All @@ -52,12 +67,16 @@ var (
type Config struct {
PushTimeout time.Duration
PoolConfig clientpool.PoolConfig `yaml:"pool_config,omitempty"`

// Distributors ring
DistributorRing RingConfig `yaml:"ring" doc:"hidden"`
}

// RegisterFlags registers distributor-related flags.
func (cfg *Config) RegisterFlags(fs *flag.FlagSet) {
cfg.PoolConfig.RegisterFlagsWithPrefix("distributor", fs)
fs.DurationVar(&cfg.PushTimeout, "distributor.push.timeout", 5*time.Second, "Timeout when pushing data to ingester.")
cfg.DistributorRing.RegisterFlags(fs)
}

// Distributor coordinates replicates and distribution of log streams.
Expand All @@ -66,30 +85,64 @@ type Distributor struct {
logger log.Logger

cfg Config
limits Limits
ingestersRing ring.ReadRing
pool *ring_client.Pool

// The global rate limiter requires a distributors ring to count
// the number of healthy instances
distributorsLifecycler *ring.BasicLifecycler
distributorsRing *ring.Ring
healthyInstancesCount *atomic.Uint32
ingestionRateLimiter *limiter.RateLimiter

subservices *services.Manager
subservicesWatcher *services.FailureWatcher

metrics *metrics
}

func New(cfg Config, ingestersRing ring.ReadRing, factory ring_client.PoolFactory, reg prometheus.Registerer, logger log.Logger, clientsOptions ...connect.ClientOption) (*Distributor, error) {
type Limits interface {
IngestionRateBytes(tenantID string) float64
IngestionBurstSizeBytes(tenantID string) int
MaxLabelNameLength(userID string) int
MaxLabelValueLength(userID string) int
MaxLabelNamesPerSeries(userID string) int
}

func New(cfg Config, ingestersRing ring.ReadRing, factory ring_client.PoolFactory, limits Limits, reg prometheus.Registerer, logger log.Logger, clientsOptions ...connect.ClientOption) (*Distributor, error) {
d := &Distributor{
cfg: cfg,
logger: logger,
ingestersRing: ingestersRing,
pool: clientpool.NewPool(cfg.PoolConfig, ingestersRing, factory, clients, logger, clientsOptions...),
metrics: newMetrics(reg),
cfg: cfg,
logger: logger,
ingestersRing: ingestersRing,
pool: clientpool.NewPool(cfg.PoolConfig, ingestersRing, factory, clients, logger, clientsOptions...),
metrics: newMetrics(reg),
healthyInstancesCount: atomic.NewUint32(0),
limits: limits,
}
var err error
d.subservices, err = services.NewManager(d.pool)

subservices := []services.Service(nil)
subservices = append(subservices, d.pool)

distributorsRing, distributorsLifecycler, err := newRingAndLifecycler(cfg.DistributorRing, d.healthyInstancesCount, logger, reg)
if err != nil {
return nil, err
}

subservices = append(subservices, distributorsLifecycler, distributorsRing)

d.ingestionRateLimiter = limiter.NewRateLimiter(newGlobalRateStrategy(newIngestionRateStrategy(limits), d), 10*time.Second)
d.distributorsLifecycler = distributorsLifecycler
d.distributorsRing = distributorsRing

d.subservices, err = services.NewManager(subservices...)
if err != nil {
return nil, errors.Wrap(err, "services manager")
}
d.subservicesWatcher = services.NewFailureWatcher()
d.subservicesWatcher.WatchManager(d.subservices)

d.Service = services.NewBasicService(d.starting, d.running, d.stopping)
rfStats.Set(int64(ingestersRing.ReplicationFactor()))
return d, nil
Expand All @@ -115,29 +168,37 @@ func (d *Distributor) stopping(_ error) error {
func (d *Distributor) Push(ctx context.Context, req *connect.Request[pushv1.PushRequest]) (*connect.Response[pushv1.PushResponse], error) {
tenantID, err := tenant.ExtractTenantIDFromContext(ctx)
if err != nil {
return nil, connect.NewError(connect.CodeInvalidArgument, err)
return nil, connect.NewError(connect.CodeUnauthenticated, err)
}
var (
keys = make([]uint32, 0, len(req.Msg.Series))
profiles = make([]*profileTracker, 0, len(req.Msg.Series))
keys = make([]uint32, 0, len(req.Msg.Series))
profiles = make([]*profileTracker, 0, len(req.Msg.Series))
totalPushUncompressedBytes int64
totalProfiles int64
)

for _, series := range req.Msg.Series {
// include the labels in the size calculation
for _, lbs := range series.Labels {
totalPushUncompressedBytes += int64(len(lbs.Name))
totalPushUncompressedBytes += int64(len(lbs.Value))
}
keys = append(keys, TokenFor(tenantID, labelsString(series.Labels)))
profName := phlaremodel.Labels(series.Labels).Get(scrape.ProfileName)
for _, raw := range series.Samples {
usagestats.NewCounter(fmt.Sprintf("distributor_profile_type_%s_received", profName)).Inc(1)
profileReceivedStats.Inc(1)
bytesReceivedTotalStats.Inc(int64(len(raw.RawProfile)))
bytesReceivedStats.Record(float64(len(raw.RawProfile)))
d.metrics.receivedCompressedBytes.WithLabelValues(profName).Observe(float64(len(raw.RawProfile)))
totalProfiles++
d.metrics.receivedCompressedBytes.WithLabelValues(profName, tenantID).Observe(float64(len(raw.RawProfile)))
p, err := pprof.RawFromBytes(raw.RawProfile)
if err != nil {
return nil, err
return nil, connect.NewError(connect.CodeInvalidArgument, err)
}
d.metrics.receivedDecompressedBytes.WithLabelValues(profName).Observe(float64(p.SizeBytes()))
d.metrics.receivedSamples.WithLabelValues(profName).Observe(float64(len(p.Sample)))

d.metrics.receivedDecompressedBytes.WithLabelValues(profName, tenantID).Observe(float64(p.SizeBytes()))
d.metrics.receivedSamples.WithLabelValues(profName, tenantID).Observe(float64(len(p.Sample)))
totalPushUncompressedBytes += int64(p.SizeBytes())
p.Normalize()

// zip the data back into the buffer
Expand All @@ -153,6 +214,24 @@ func (d *Distributor) Push(ctx context.Context, req *connect.Request[pushv1.Push
profiles = append(profiles, &profileTracker{profile: series})
}

// validate the request
for _, series := range req.Msg.Series {
if err := validation.ValidateLabels(d.limits, tenantID, series.Labels); err != nil {
validation.DiscardedProfiles.WithLabelValues(string(validation.ReasonOf(err)), tenantID).Add(float64(totalProfiles))
validation.DiscardedBytes.WithLabelValues(string(validation.ReasonOf(err)), tenantID).Add(float64(totalPushUncompressedBytes))
return nil, connect.NewError(connect.CodeInvalidArgument, err)
}
}

// rate limit the request
if !d.ingestionRateLimiter.AllowN(time.Now(), tenantID, int(totalPushUncompressedBytes)) {
validation.DiscardedProfiles.WithLabelValues(string(validation.RateLimited), tenantID).Add(float64(totalProfiles))
validation.DiscardedBytes.WithLabelValues(string(validation.RateLimited), tenantID).Add(float64(totalPushUncompressedBytes))
return nil, connect.NewError(connect.CodeResourceExhausted,
fmt.Errorf("push rate limit (%s) exceeded while adding %s", humanize.Bytes(uint64(d.limits.IngestionRateBytes(tenantID))), humanize.Bytes(uint64(totalPushUncompressedBytes))),
)
}

const maxExpectedReplicationSet = 5 // typical replication factor 3 plus one for inactive plus one for luck
var descs [maxExpectedReplicationSet]ring.InstanceDesc

Expand Down Expand Up @@ -245,6 +324,35 @@ func (d *Distributor) sendProfilesErr(ctx context.Context, ingester ring.Instanc
return err
}

func (d *Distributor) ServeHTTP(w http.ResponseWriter, req *http.Request) {
if d.distributorsRing != nil {
d.distributorsRing.ServeHTTP(w, req)
} else {
ringNotEnabledPage := `
<!DOCTYPE html>
<html>
<head>
<meta charset="UTF-8">
<title>Distributor Status</title>
</head>
<body>
<h1>Distributor Status</h1>
<p>Distributor is not running with global limits enabled</p>
</body>
</html>`
util.WriteHTMLResponse(w, ringNotEnabledPage)
}
}

// HealthyInstancesCount implements the ReadLifecycler interface
//
// We use a ring lifecycler delegate to count the number of members of the
// ring. The count is then used to enforce rate limiting correctly for each
// distributor. $EFFECTIVE_RATE_LIMIT = $GLOBAL_RATE_LIMIT / $NUM_INSTANCES
func (d *Distributor) HealthyInstancesCount() int {
return int(d.healthyInstancesCount.Load())
}

type profileTracker struct {
profile *pushv1.RawProfileSeries
minSuccess int
Expand Down Expand Up @@ -283,3 +391,35 @@ func TokenFor(tenantID, labels string) uint32 {
_, _ = h.Write([]byte(labels))
return h.Sum32()
}

// newRingAndLifecycler creates a new distributor ring and lifecycler with all required lifecycler delegates
func newRingAndLifecycler(cfg RingConfig, instanceCount *atomic.Uint32, logger log.Logger, reg prometheus.Registerer) (*ring.Ring, *ring.BasicLifecycler, error) {
reg = prometheus.WrapRegistererWithPrefix("phlare_", reg)
kvStore, err := kv.NewClient(cfg.KVStore, ring.GetCodec(), kv.RegistererWithKVName(reg, "distributor-lifecycler"), logger)
if err != nil {
return nil, nil, errors.Wrap(err, "failed to initialize distributors' KV store")
}

lifecyclerCfg, err := cfg.ToBasicLifecyclerConfig(logger)
if err != nil {
return nil, nil, errors.Wrap(err, "failed to build distributors' lifecycler config")
}

var delegate ring.BasicLifecyclerDelegate
delegate = ring.NewInstanceRegisterDelegate(ring.ACTIVE, lifecyclerCfg.NumTokens)
delegate = newHealthyInstanceDelegate(instanceCount, cfg.HeartbeatTimeout, delegate)
delegate = ring.NewLeaveOnStoppingDelegate(delegate, logger)
delegate = ring.NewAutoForgetDelegate(ringAutoForgetUnhealthyPeriods*cfg.HeartbeatTimeout, delegate, logger)

distributorsLifecycler, err := ring.NewBasicLifecycler(lifecyclerCfg, "distributor", distributorRingKey, kvStore, delegate, logger, reg)
if err != nil {
return nil, nil, errors.Wrap(err, "failed to initialize distributors' lifecycler")
}

distributorsRing, err := ring.New(cfg.ToRingConfig(), "distributor", distributorRingKey, logger, reg)
if err != nil {
return nil, nil, errors.Wrap(err, "failed to initialize distributors' ring client")
}

return distributorsRing, distributorsLifecycler, nil
}
Loading

0 comments on commit 53e7117

Please sign in to comment.