feat: Introduce policy stream mapping (#15982)

**What this PR does / why we need it**:
Introduces the idea of policies to Loki, which are recognizable based on the given stream selectors.
This is an improved version of https://github.com/grafana/loki/pull/15561 and built on top of https://github.com/grafana/loki/pull/15875.
A policy mapping can be configured the following way:
```yaml
12345:
  policy_stream_mapping:
    policy6:
    - selector: `{env="prod"}`
      priority: 2
    - selector: `{env=~"prod|staging"}`
      priority: 1
    - selector: `{team="finance"}`
      priority: 4
    policy7:
    - selector: `{env=~"prod|dev"}`
      priority: 3
```
With that configuration, pushes to tenant `12345` with the labels `{env="prod", team="finance"}` would be assigned to policy6 because the third mapping for policy6 matches these labels and has higher priority than any other matching.
pull/15934/merge
Dylan Guedes 3 months ago committed by GitHub
parent 2587f3425d
commit 5c8e832260
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
  1. 14
      docs/sources/shared/configuration.md
  2. 1
      pkg/compactor/retention/expiration.go
  3. 9
      pkg/compactor/retention/expiration_test.go
  4. 48
      pkg/distributor/distributor.go
  5. 63
      pkg/distributor/distributor_test.go
  6. 40
      pkg/distributor/validation_metrics.go
  7. 44
      pkg/distributor/validator.go
  8. 4
      pkg/distributor/validator_test.go
  9. 11
      pkg/ingester/instance.go
  10. 9
      pkg/ingester/stream.go
  11. 47
      pkg/validation/ingestion_policies.go
  12. 105
      pkg/validation/ingestion_policies_test.go
  13. 23
      pkg/validation/limits.go
  14. 21
      pkg/validation/limits_test.go
  15. 4
      pkg/validation/validate.go

@ -3613,6 +3613,20 @@ otlp_config:
# CLI flag: -validation.enforced-labels
[enforced_labels: <list of strings> | default = []]
# Map of policies to stream selectors with a priority. Experimental.
# Example:
# policy_stream_mapping:
# finance:
# - selectors: ["{namespace="prod", container="billing"}"]
# priority: 2
# ops:
# - selectors: ["{namespace="prod", container="ops"}"]
# priority: 1
# staging:
# - selectors: ["{namespace="staging"}, {namespace="dev"}"]
# priority: 1
[policy_stream_mapping: <map of string to list of PriorityStreams>]
# The number of partitions a tenant's data should be sharded to when using kafka
# ingestion. Tenants are sharded across partitions using shuffle-sharding. 0
# disables shuffle sharding and tenant is sharded across all partitions.

@ -42,6 +42,7 @@ type Limits interface {
StreamRetention(userID string) []validation.StreamRetention
AllByUserID() map[string]*validation.Limits
DefaultLimits() *validation.Limits
PoliciesStreamMapping(userID string) validation.PolicyStreamMapping
}
func NewExpirationChecker(limits Limits) ExpirationChecker {

@ -13,8 +13,9 @@ import (
)
type retentionLimit struct {
retentionPeriod time.Duration
streamRetention []validation.StreamRetention
retentionPeriod time.Duration
streamRetention []validation.StreamRetention
policyStreamMapping validation.PolicyStreamMapping
}
func (r retentionLimit) convertToValidationLimit() *validation.Limits {
@ -33,6 +34,10 @@ func (f fakeLimits) RetentionPeriod(userID string) time.Duration {
return f.perTenant[userID].retentionPeriod
}
func (f fakeLimits) PoliciesStreamMapping(_ string) validation.PolicyStreamMapping {
return f.perTenant["user0"].policyStreamMapping
}
func (f fakeLimits) StreamRetention(userID string) []validation.StreamRetention {
return f.perTenant[userID].streamRetention
}

@ -528,14 +528,14 @@ func (d *Distributor) Push(ctx context.Context, req *logproto.PushRequest) (*log
d.truncateLines(validationContext, &stream)
var lbs labels.Labels
var retentionHours string
lbs, stream.Labels, stream.Hash, retentionHours, err = d.parseStreamLabels(validationContext, stream.Labels, stream)
var retentionHours, policy string
lbs, stream.Labels, stream.Hash, retentionHours, policy, err = d.parseStreamLabels(validationContext, stream.Labels, stream)
if err != nil {
d.writeFailuresManager.Log(tenantID, err)
validationErrors.Add(err)
validation.DiscardedSamples.WithLabelValues(validation.InvalidLabels, tenantID, retentionHours).Add(float64(len(stream.Entries)))
validation.DiscardedSamples.WithLabelValues(validation.InvalidLabels, tenantID, retentionHours, policy).Add(float64(len(stream.Entries)))
discardedBytes := util.EntriesTotalSize(stream.Entries)
validation.DiscardedBytes.WithLabelValues(validation.InvalidLabels, tenantID, retentionHours).Add(float64(discardedBytes))
validation.DiscardedBytes.WithLabelValues(validation.InvalidLabels, tenantID, retentionHours, policy).Add(float64(discardedBytes))
continue
}
@ -543,9 +543,9 @@ func (d *Distributor) Push(ctx context.Context, req *logproto.PushRequest) (*log
err := fmt.Errorf(validation.MissingEnforcedLabelsErrorMsg, strings.Join(lbsMissing, ","), tenantID)
d.writeFailuresManager.Log(tenantID, err)
validationErrors.Add(err)
validation.DiscardedSamples.WithLabelValues(validation.MissingEnforcedLabels, tenantID, retentionHours).Add(float64(len(stream.Entries)))
validation.DiscardedSamples.WithLabelValues(validation.MissingEnforcedLabels, tenantID, retentionHours, policy).Add(float64(len(stream.Entries)))
discardedBytes := util.EntriesTotalSize(stream.Entries)
validation.DiscardedBytes.WithLabelValues(validation.MissingEnforcedLabels, tenantID, retentionHours).Add(float64(discardedBytes))
validation.DiscardedBytes.WithLabelValues(validation.MissingEnforcedLabels, tenantID, retentionHours, policy).Add(float64(discardedBytes))
continue
}
@ -554,7 +554,7 @@ func (d *Distributor) Push(ctx context.Context, req *logproto.PushRequest) (*log
prevTs := stream.Entries[0].Timestamp
for _, entry := range stream.Entries {
if err := d.validator.ValidateEntry(ctx, validationContext, lbs, entry, retentionHours); err != nil {
if err := d.validator.ValidateEntry(ctx, validationContext, lbs, entry, retentionHours, policy); err != nil {
d.writeFailuresManager.Log(tenantID, err)
validationErrors.Add(err)
continue
@ -609,7 +609,7 @@ func (d *Distributor) Push(ctx context.Context, req *logproto.PushRequest) (*log
}
n++
validationContext.validationMetrics.compute(entry, retentionHours)
validationContext.validationMetrics.compute(entry, retentionHours, policy)
pushSize += len(entry.Line)
}
stream.Entries = stream.Entries[:n]
@ -647,10 +647,10 @@ func (d *Distributor) Push(ctx context.Context, req *logproto.PushRequest) (*log
return nil, httpgrpc.Errorf(retStatusCode, "%s", err.Error())
}
if !d.ingestionRateLimiter.AllowN(now, tenantID, validationContext.validationMetrics.lineSize) {
if !d.ingestionRateLimiter.AllowN(now, tenantID, validationContext.validationMetrics.aggregatedPushStats.lineSize) {
d.trackDiscardedData(ctx, req, validationContext, tenantID, validationContext.validationMetrics, validation.RateLimited)
err = fmt.Errorf(validation.RateLimitedErrorMsg, tenantID, int(d.ingestionRateLimiter.Limit(now, tenantID)), validationContext.validationMetrics.lineCount, validationContext.validationMetrics.lineSize)
err = fmt.Errorf(validation.RateLimitedErrorMsg, tenantID, int(d.ingestionRateLimiter.Limit(now, tenantID)), validationContext.validationMetrics.aggregatedPushStats.lineCount, validationContext.validationMetrics.aggregatedPushStats.lineSize)
d.writeFailuresManager.Log(tenantID, err)
// Return a 429 to indicate to the client they are being rate limited
return nil, httpgrpc.Errorf(http.StatusTooManyRequests, "%s", err.Error())
@ -787,14 +787,16 @@ func (d *Distributor) trackDiscardedData(
validationMetrics validationMetrics,
reason string,
) {
for retentionHours, count := range validationMetrics.lineCountPerRetentionHours {
validation.DiscardedSamples.WithLabelValues(reason, tenantID, retentionHours).Add(float64(count))
validation.DiscardedBytes.WithLabelValues(reason, tenantID, retentionHours).Add(float64(validationMetrics.lineSizePerRetentionHours[retentionHours]))
for policy, retentionToStats := range validationMetrics.policyPushStats {
for retentionHours, stats := range retentionToStats {
validation.DiscardedSamples.WithLabelValues(reason, tenantID, retentionHours, policy).Add(float64(stats.lineCount))
validation.DiscardedBytes.WithLabelValues(reason, tenantID, retentionHours, policy).Add(float64(stats.lineSize))
}
}
if d.usageTracker != nil {
for _, stream := range req.Streams {
lbs, _, _, _, err := d.parseStreamLabels(validationContext, stream.Labels, stream)
lbs, _, _, _, _, err := d.parseStreamLabels(validationContext, stream.Labels, stream)
if err != nil {
continue
}
@ -1173,28 +1175,32 @@ type labelData struct {
hash uint64
}
func (d *Distributor) parseStreamLabels(vContext validationContext, key string, stream logproto.Stream) (labels.Labels, string, uint64, string, error) {
func (d *Distributor) parseStreamLabels(vContext validationContext, key string, stream logproto.Stream) (labels.Labels, string, uint64, string, string, error) {
mapping := d.validator.Limits.PoliciesStreamMapping(vContext.userID)
if val, ok := d.labelCache.Get(key); ok {
retentionHours := d.tenantsRetention.RetentionHoursFor(vContext.userID, val.ls)
return val.ls, val.ls.String(), val.hash, retentionHours, nil
policy := mapping.PolicyFor(val.ls)
return val.ls, val.ls.String(), val.hash, retentionHours, policy, nil
}
ls, err := syntax.ParseLabels(key)
if err != nil {
tenantRetentionHours := d.tenantsRetention.RetentionHoursFor(vContext.userID, nil)
return nil, "", 0, tenantRetentionHours, fmt.Errorf(validation.InvalidLabelsErrorMsg, key, err)
retentionHours := d.tenantsRetention.RetentionHoursFor(vContext.userID, nil)
// TODO: check for global policy.
return nil, "", 0, retentionHours, mapping.PolicyFor(nil), fmt.Errorf(validation.InvalidLabelsErrorMsg, key, err)
}
policy := mapping.PolicyFor(ls)
retentionHours := d.tenantsRetention.RetentionHoursFor(vContext.userID, ls)
if err := d.validator.ValidateLabels(vContext, ls, stream, retentionHours); err != nil {
return nil, "", 0, retentionHours, err
if err := d.validator.ValidateLabels(vContext, ls, stream, retentionHours, policy); err != nil {
return nil, "", 0, retentionHours, policy, err
}
lsHash := ls.Hash()
d.labelCache.Add(key, labelData{ls, lsHash})
return ls, ls.String(), lsHash, retentionHours, nil
return ls, ls.String(), lsHash, retentionHours, policy, nil
}
// shardCountFor returns the right number of shards to be used by the given stream.

@ -1233,7 +1233,7 @@ func Benchmark_SortLabelsOnPush(b *testing.B) {
for n := 0; n < b.N; n++ {
stream := request.Streams[0]
stream.Labels = `{buzz="f", a="b"}`
_, _, _, _, err := d.parseStreamLabels(vCtx, stream.Labels, stream)
_, _, _, _, _, err := d.parseStreamLabels(vCtx, stream.Labels, stream)
if err != nil {
panic("parseStreamLabels fail,err:" + err.Error())
}
@ -1279,7 +1279,7 @@ func TestParseStreamLabels(t *testing.T) {
vCtx := d.validator.getValidationContextForTime(testTime, "123")
t.Run(tc.name, func(t *testing.T) {
lbs, lbsString, hash, _, err := d.parseStreamLabels(vCtx, tc.origLabels, logproto.Stream{
lbs, lbsString, hash, _, _, err := d.parseStreamLabels(vCtx, tc.origLabels, logproto.Stream{
Labels: tc.origLabels,
})
if tc.expectedErr != nil {
@ -2063,3 +2063,62 @@ func TestDistributor_StructuredMetadataSanitization(t *testing.T) {
assert.Equal(t, tc.numSanitizations, testutil.ToFloat64(distributors[0].tenantPushSanitizedStructuredMetadata.WithLabelValues("test")))
}
}
func BenchmarkDistributor_PushWithPolicies(b *testing.B) {
baselineLimits := &validation.Limits{}
flagext.DefaultValues(baselineLimits)
lbs := `{foo="bar", env="prod", daz="baz", container="loki", pod="loki-0"}`
b.Run("push without policies", func(b *testing.B) {
limits := baselineLimits
limits.PolicyStreamMapping = make(validation.PolicyStreamMapping)
distributors, _ := prepare(&testing.T{}, 1, 3, limits, nil)
req := makeWriteRequestWithLabels(10, 10, []string{lbs}, false, false, false)
b.ResetTimer()
for i := 0; i < b.N; i++ {
distributors[0].Push(ctx, req) //nolint:errcheck
}
})
for numPolicies := 1; numPolicies <= 100; numPolicies *= 10 {
b.Run(fmt.Sprintf("push with %d policies", numPolicies), func(b *testing.B) {
limits := baselineLimits
limits.PolicyStreamMapping = make(validation.PolicyStreamMapping)
for i := 1; i <= numPolicies; i++ {
limits.PolicyStreamMapping[fmt.Sprintf("policy%d", i)] = []*validation.PriorityStream{
{
Selector: `{foo="bar"}`, Priority: i,
},
}
}
req := makeWriteRequestWithLabels(10, 10, []string{lbs}, false, false, false)
distributors, _ := prepare(&testing.T{}, 1, 3, limits, nil)
b.ResetTimer()
for i := 0; i < b.N; i++ {
distributors[0].Push(ctx, req) //nolint:errcheck
}
})
}
for numMatchers := 1; numMatchers <= 100; numMatchers *= 10 {
b.Run(fmt.Sprintf("push with %d matchers", numMatchers), func(b *testing.B) {
limits := baselineLimits
limits.PolicyStreamMapping = make(validation.PolicyStreamMapping)
for i := 1; i <= numMatchers; i++ {
limits.PolicyStreamMapping["policy0"] = append(limits.PolicyStreamMapping["policy0"], &validation.PriorityStream{
Selector: `{foo="bar"}`,
Matchers: []*labels.Matcher{labels.MustNewMatcher(labels.MatchEqual, "foo", "bar")},
Priority: i,
})
}
req := makeWriteRequestWithLabels(10, 10, []string{lbs}, false, false, false)
distributors, _ := prepare(&testing.T{}, 1, 3, limits, nil)
b.ResetTimer()
for i := 0; i < b.N; i++ {
distributors[0].Push(ctx, req) //nolint:errcheck
}
})
}
}

@ -5,26 +5,40 @@ import (
"github.com/grafana/loki/v3/pkg/util"
)
type pushStats struct {
lineSize int
lineCount int
}
type validationMetrics struct {
lineSizePerRetentionHours map[string]int
lineCountPerRetentionHours map[string]int
lineSize int
lineCount int
tenantRetentionHours string
policyPushStats map[string]map[string]pushStats // policy -> retentionHours -> lineSize
tenantRetentionHours string
aggregatedPushStats pushStats
}
func newValidationMetrics(tenantRetentionHours string) validationMetrics {
return validationMetrics{
lineSizePerRetentionHours: make(map[string]int),
lineCountPerRetentionHours: make(map[string]int),
tenantRetentionHours: tenantRetentionHours,
policyPushStats: make(map[string]map[string]pushStats),
tenantRetentionHours: tenantRetentionHours,
}
}
func (v *validationMetrics) compute(entry logproto.Entry, retentionHours string) {
func (v *validationMetrics) compute(entry logproto.Entry, retentionHours string, policy string) {
if _, ok := v.policyPushStats[policy]; !ok {
v.policyPushStats[policy] = make(map[string]pushStats)
}
if _, ok := v.policyPushStats[policy][retentionHours]; !ok {
v.policyPushStats[policy][retentionHours] = pushStats{}
}
totalEntrySize := util.EntryTotalSize(&entry)
v.lineSizePerRetentionHours[retentionHours] += totalEntrySize
v.lineCountPerRetentionHours[retentionHours]++
v.lineSize += totalEntrySize
v.lineCount++
v.aggregatedPushStats.lineSize += totalEntrySize
v.aggregatedPushStats.lineCount++
stats := v.policyPushStats[policy][retentionHours]
stats.lineCount++
stats.lineSize += totalEntrySize
v.policyPushStats[policy][retentionHours] = stats
}

@ -93,7 +93,7 @@ func (v Validator) getValidationContextForTime(now time.Time, userID string) val
}
// ValidateEntry returns an error if the entry is invalid and report metrics for invalid entries accordingly.
func (v Validator) ValidateEntry(ctx context.Context, vCtx validationContext, labels labels.Labels, entry logproto.Entry, retentionHours string) error {
func (v Validator) ValidateEntry(ctx context.Context, vCtx validationContext, labels labels.Labels, entry logproto.Entry, retentionHours string, policy string) error {
ts := entry.Timestamp.UnixNano()
validation.LineLengthHist.Observe(float64(len(entry.Line)))
structuredMetadataCount := len(entry.StructuredMetadata)
@ -104,8 +104,8 @@ func (v Validator) ValidateEntry(ctx context.Context, vCtx validationContext, la
// Makes time string on the error message formatted consistently.
formatedEntryTime := entry.Timestamp.Format(timeFormat)
formatedRejectMaxAgeTime := time.Unix(0, vCtx.rejectOldSampleMaxAge).Format(timeFormat)
validation.DiscardedSamples.WithLabelValues(validation.GreaterThanMaxSampleAge, vCtx.userID, retentionHours).Inc()
validation.DiscardedBytes.WithLabelValues(validation.GreaterThanMaxSampleAge, vCtx.userID, retentionHours).Add(entrySize)
validation.DiscardedSamples.WithLabelValues(validation.GreaterThanMaxSampleAge, vCtx.userID, retentionHours, policy).Inc()
validation.DiscardedBytes.WithLabelValues(validation.GreaterThanMaxSampleAge, vCtx.userID, retentionHours, policy).Add(entrySize)
if v.usageTracker != nil {
v.usageTracker.DiscardedBytesAdd(ctx, vCtx.userID, validation.GreaterThanMaxSampleAge, labels, entrySize)
}
@ -114,8 +114,8 @@ func (v Validator) ValidateEntry(ctx context.Context, vCtx validationContext, la
if ts > vCtx.creationGracePeriod {
formatedEntryTime := entry.Timestamp.Format(timeFormat)
validation.DiscardedSamples.WithLabelValues(validation.TooFarInFuture, vCtx.userID, retentionHours).Inc()
validation.DiscardedBytes.WithLabelValues(validation.TooFarInFuture, vCtx.userID, retentionHours).Add(entrySize)
validation.DiscardedSamples.WithLabelValues(validation.TooFarInFuture, vCtx.userID, retentionHours, policy).Inc()
validation.DiscardedBytes.WithLabelValues(validation.TooFarInFuture, vCtx.userID, retentionHours, policy).Add(entrySize)
if v.usageTracker != nil {
v.usageTracker.DiscardedBytesAdd(ctx, vCtx.userID, validation.TooFarInFuture, labels, entrySize)
}
@ -127,8 +127,8 @@ func (v Validator) ValidateEntry(ctx context.Context, vCtx validationContext, la
// an orthogonal concept (we need not use ValidateLabels in this context)
// but the upstream cortex_validation pkg uses it, so we keep this
// for parity.
validation.DiscardedSamples.WithLabelValues(validation.LineTooLong, vCtx.userID, retentionHours).Inc()
validation.DiscardedBytes.WithLabelValues(validation.LineTooLong, vCtx.userID, retentionHours).Add(entrySize)
validation.DiscardedSamples.WithLabelValues(validation.LineTooLong, vCtx.userID, retentionHours, policy).Inc()
validation.DiscardedBytes.WithLabelValues(validation.LineTooLong, vCtx.userID, retentionHours, policy).Add(entrySize)
if v.usageTracker != nil {
v.usageTracker.DiscardedBytesAdd(ctx, vCtx.userID, validation.LineTooLong, labels, entrySize)
}
@ -137,8 +137,8 @@ func (v Validator) ValidateEntry(ctx context.Context, vCtx validationContext, la
if structuredMetadataCount > 0 {
if !vCtx.allowStructuredMetadata {
validation.DiscardedSamples.WithLabelValues(validation.DisallowedStructuredMetadata, vCtx.userID, retentionHours).Inc()
validation.DiscardedBytes.WithLabelValues(validation.DisallowedStructuredMetadata, vCtx.userID, retentionHours).Add(entrySize)
validation.DiscardedSamples.WithLabelValues(validation.DisallowedStructuredMetadata, vCtx.userID, retentionHours, policy).Inc()
validation.DiscardedBytes.WithLabelValues(validation.DisallowedStructuredMetadata, vCtx.userID, retentionHours, policy).Add(entrySize)
if v.usageTracker != nil {
v.usageTracker.DiscardedBytesAdd(ctx, vCtx.userID, validation.DisallowedStructuredMetadata, labels, entrySize)
}
@ -146,8 +146,8 @@ func (v Validator) ValidateEntry(ctx context.Context, vCtx validationContext, la
}
if maxSize := vCtx.maxStructuredMetadataSize; maxSize != 0 && structuredMetadataSizeBytes > maxSize {
validation.DiscardedSamples.WithLabelValues(validation.StructuredMetadataTooLarge, vCtx.userID, retentionHours).Inc()
validation.DiscardedBytes.WithLabelValues(validation.StructuredMetadataTooLarge, vCtx.userID, retentionHours).Add(entrySize)
validation.DiscardedSamples.WithLabelValues(validation.StructuredMetadataTooLarge, vCtx.userID, retentionHours, policy).Inc()
validation.DiscardedBytes.WithLabelValues(validation.StructuredMetadataTooLarge, vCtx.userID, retentionHours, policy).Add(entrySize)
if v.usageTracker != nil {
v.usageTracker.DiscardedBytesAdd(ctx, vCtx.userID, validation.StructuredMetadataTooLarge, labels, entrySize)
}
@ -155,8 +155,8 @@ func (v Validator) ValidateEntry(ctx context.Context, vCtx validationContext, la
}
if maxCount := vCtx.maxStructuredMetadataCount; maxCount != 0 && structuredMetadataCount > maxCount {
validation.DiscardedSamples.WithLabelValues(validation.StructuredMetadataTooMany, vCtx.userID, retentionHours).Inc()
validation.DiscardedBytes.WithLabelValues(validation.StructuredMetadataTooMany, vCtx.userID, retentionHours).Add(entrySize)
validation.DiscardedSamples.WithLabelValues(validation.StructuredMetadataTooMany, vCtx.userID, retentionHours, policy).Inc()
validation.DiscardedBytes.WithLabelValues(validation.StructuredMetadataTooMany, vCtx.userID, retentionHours, policy).Add(entrySize)
if v.usageTracker != nil {
v.usageTracker.DiscardedBytesAdd(ctx, vCtx.userID, validation.StructuredMetadataTooMany, labels, entrySize)
}
@ -168,9 +168,9 @@ func (v Validator) ValidateEntry(ctx context.Context, vCtx validationContext, la
}
// Validate labels returns an error if the labels are invalid
func (v Validator) ValidateLabels(ctx validationContext, ls labels.Labels, stream logproto.Stream, retentionHours string) error {
func (v Validator) ValidateLabels(ctx validationContext, ls labels.Labels, stream logproto.Stream, retentionHours, policy string) error {
if len(ls) == 0 {
validation.DiscardedSamples.WithLabelValues(validation.MissingLabels, ctx.userID, retentionHours).Inc()
validation.DiscardedSamples.WithLabelValues(validation.MissingLabels, ctx.userID, retentionHours, policy).Inc()
return fmt.Errorf(validation.MissingLabelsErrorMsg)
}
@ -187,20 +187,20 @@ func (v Validator) ValidateLabels(ctx validationContext, ls labels.Labels, strea
}
if numLabelNames > ctx.maxLabelNamesPerSeries {
updateMetrics(validation.MaxLabelNamesPerSeries, ctx.userID, stream, retentionHours)
updateMetrics(validation.MaxLabelNamesPerSeries, ctx.userID, stream, retentionHours, policy)
return fmt.Errorf(validation.MaxLabelNamesPerSeriesErrorMsg, stream.Labels, numLabelNames, ctx.maxLabelNamesPerSeries)
}
lastLabelName := ""
for _, l := range ls {
if len(l.Name) > ctx.maxLabelNameLength {
updateMetrics(validation.LabelNameTooLong, ctx.userID, stream, retentionHours)
updateMetrics(validation.LabelNameTooLong, ctx.userID, stream, retentionHours, policy)
return fmt.Errorf(validation.LabelNameTooLongErrorMsg, stream.Labels, l.Name)
} else if len(l.Value) > ctx.maxLabelValueLength {
updateMetrics(validation.LabelValueTooLong, ctx.userID, stream, retentionHours)
updateMetrics(validation.LabelValueTooLong, ctx.userID, stream, retentionHours, policy)
return fmt.Errorf(validation.LabelValueTooLongErrorMsg, stream.Labels, l.Value)
} else if cmp := strings.Compare(lastLabelName, l.Name); cmp == 0 {
updateMetrics(validation.DuplicateLabelNames, ctx.userID, stream, retentionHours)
updateMetrics(validation.DuplicateLabelNames, ctx.userID, stream, retentionHours, policy)
return fmt.Errorf(validation.DuplicateLabelNamesErrorMsg, stream.Labels, l.Name)
}
lastLabelName = l.Name
@ -217,8 +217,8 @@ func (v Validator) ShouldBlockIngestion(ctx validationContext, now time.Time) (b
return now.Before(ctx.blockIngestionUntil), ctx.blockIngestionUntil, ctx.blockIngestionStatusCode
}
func updateMetrics(reason, userID string, stream logproto.Stream, retentionHours string) {
validation.DiscardedSamples.WithLabelValues(reason, userID, retentionHours).Add(float64(len(stream.Entries)))
func updateMetrics(reason, userID string, stream logproto.Stream, retentionHours, policy string) {
validation.DiscardedSamples.WithLabelValues(reason, userID, retentionHours, policy).Add(float64(len(stream.Entries)))
bytes := util.EntriesTotalSize(stream.Entries)
validation.DiscardedBytes.WithLabelValues(reason, userID, retentionHours).Add(float64(bytes))
validation.DiscardedBytes.WithLabelValues(reason, userID, retentionHours, policy).Add(float64(bytes))
}

@ -133,7 +133,7 @@ func TestValidator_ValidateEntry(t *testing.T) {
assert.NoError(t, err)
retentionHours := util.RetentionHours(v.RetentionPeriod(tt.userID))
err = v.ValidateEntry(ctx, v.getValidationContextForTime(testTime, tt.userID), testStreamLabels, tt.entry, retentionHours)
err = v.ValidateEntry(ctx, v.getValidationContextForTime(testTime, tt.userID), testStreamLabels, tt.entry, retentionHours, "")
assert.Equal(t, tt.expected, err)
})
}
@ -232,7 +232,7 @@ func TestValidator_ValidateLabels(t *testing.T) {
v, err := NewValidator(o, nil)
assert.NoError(t, err)
err = v.ValidateLabels(v.getValidationContextForTime(testTime, tt.userID), mustParseLabels(tt.labels), logproto.Stream{Labels: tt.labels}, retentionHours)
err = v.ValidateLabels(v.getValidationContextForTime(testTime, tt.userID), mustParseLabels(tt.labels), logproto.Stream{Labels: tt.labels}, retentionHours, "")
assert.Equal(t, tt.expected, err)
})
}

@ -297,13 +297,14 @@ func (i *instance) createStream(ctx context.Context, pushReqStream logproto.Stre
}
retentionHours := util.RetentionHours(i.tenantsRetention.RetentionPeriodFor(i.instanceID, labels))
mapping := i.limiter.limits.PoliciesStreamMapping(i.instanceID)
policy := mapping.PolicyFor(labels)
if record != nil {
err = i.streamCountLimiter.AssertNewStreamAllowed(i.instanceID)
}
if err != nil {
return i.onStreamCreationError(ctx, pushReqStream, err, labels, retentionHours)
return i.onStreamCreationError(ctx, pushReqStream, err, labels, retentionHours, policy)
}
fp := i.getHashForLabels(labels)
@ -333,7 +334,7 @@ func (i *instance) createStream(ctx context.Context, pushReqStream logproto.Stre
return s, nil
}
func (i *instance) onStreamCreationError(ctx context.Context, pushReqStream logproto.Stream, err error, labels labels.Labels, retentionHours string) (*stream, error) {
func (i *instance) onStreamCreationError(ctx context.Context, pushReqStream logproto.Stream, err error, labels labels.Labels, retentionHours, policy string) (*stream, error) {
if i.configs.LogStreamCreation(i.instanceID) || i.cfg.KafkaIngestion.Enabled {
l := level.Debug(util_log.Logger)
@ -349,9 +350,9 @@ func (i *instance) onStreamCreationError(ctx context.Context, pushReqStream logp
)
}
validation.DiscardedSamples.WithLabelValues(validation.StreamLimit, i.instanceID, retentionHours).Add(float64(len(pushReqStream.Entries)))
validation.DiscardedSamples.WithLabelValues(validation.StreamLimit, i.instanceID, retentionHours, policy).Add(float64(len(pushReqStream.Entries)))
bytes := util.EntriesTotalSize(pushReqStream.Entries)
validation.DiscardedBytes.WithLabelValues(validation.StreamLimit, i.instanceID, retentionHours).Add(float64(bytes))
validation.DiscardedBytes.WithLabelValues(validation.StreamLimit, i.instanceID, retentionHours, policy).Add(float64(bytes))
if i.customStreamsTracker != nil {
i.customStreamsTracker.DiscardedBytesAdd(ctx, i.instanceID, validation.StreamLimit, labels, float64(bytes))
}

@ -84,6 +84,7 @@ type stream struct {
configs *runtime.TenantConfigs
retentionHours string
policy string
}
type chunkDesc struct {
@ -481,15 +482,15 @@ func (s *stream) reportMetrics(ctx context.Context, outOfOrderSamples, outOfOrde
if s.unorderedWrites {
name = validation.TooFarBehind
}
validation.DiscardedSamples.WithLabelValues(name, s.tenant, s.retentionHours).Add(float64(outOfOrderSamples))
validation.DiscardedBytes.WithLabelValues(name, s.tenant, s.retentionHours).Add(float64(outOfOrderBytes))
validation.DiscardedSamples.WithLabelValues(name, s.tenant, s.retentionHours, s.policy).Add(float64(outOfOrderSamples))
validation.DiscardedBytes.WithLabelValues(name, s.tenant, s.retentionHours, s.policy).Add(float64(outOfOrderBytes))
if usageTracker != nil {
usageTracker.DiscardedBytesAdd(ctx, s.tenant, name, s.labels, float64(outOfOrderBytes))
}
}
if rateLimitedSamples > 0 {
validation.DiscardedSamples.WithLabelValues(validation.StreamRateLimit, s.tenant, s.retentionHours).Add(float64(rateLimitedSamples))
validation.DiscardedBytes.WithLabelValues(validation.StreamRateLimit, s.tenant, s.retentionHours).Add(float64(rateLimitedBytes))
validation.DiscardedSamples.WithLabelValues(validation.StreamRateLimit, s.tenant, s.retentionHours, s.policy).Add(float64(rateLimitedSamples))
validation.DiscardedBytes.WithLabelValues(validation.StreamRateLimit, s.tenant, s.retentionHours, s.policy).Add(float64(rateLimitedBytes))
if usageTracker != nil {
usageTracker.DiscardedBytesAdd(ctx, s.tenant, validation.StreamRateLimit, s.labels, float64(rateLimitedBytes))
}

@ -0,0 +1,47 @@
package validation
import "github.com/prometheus/prometheus/model/labels"
type PriorityStream struct {
Priority int `yaml:"priority" json:"priority" doc:"description=The larger the value, the higher the priority."`
Selector string `yaml:"selector" json:"selector" doc:"description=Stream selector expression."`
Matchers []*labels.Matcher `yaml:"-" json:"-"` // populated during validation.
}
func (p *PriorityStream) Matches(lbs labels.Labels) bool {
for _, m := range p.Matchers {
if !m.Matches(lbs.Get(m.Name)) {
return false
}
}
return true
}
type PolicyStreamMapping map[string][]*PriorityStream
func (p *PolicyStreamMapping) PolicyFor(lbs labels.Labels) string {
var (
matchedPolicy *PriorityStream
found bool
matchedPolicyName string
)
for policyName, policyStreams := range *p {
for _, policyStream := range policyStreams {
if found && policyStream.Priority <= matchedPolicy.Priority {
// Even if a match occurs it won't have a higher priority than the current matched policy.
continue
}
if !policyStream.Matches(lbs) {
continue
}
found = true
matchedPolicy = policyStream
matchedPolicyName = policyName
}
}
return matchedPolicyName
}

@ -0,0 +1,105 @@
package validation
import (
"testing"
"github.com/prometheus/prometheus/model/labels"
"github.com/stretchr/testify/require"
)
func Test_PolicyStreamMapping_PolicyFor(t *testing.T) {
mapping := PolicyStreamMapping{
"policy1": []*PriorityStream{
{
Selector: `{foo="bar"}`,
Priority: 2,
Matchers: []*labels.Matcher{
labels.MustNewMatcher(labels.MatchEqual, "foo", "bar"),
},
},
},
"policy2": []*PriorityStream{
{
Selector: `{foo="bar", daz="baz"}`,
Priority: 1,
Matchers: []*labels.Matcher{
labels.MustNewMatcher(labels.MatchEqual, "foo", "bar"),
labels.MustNewMatcher(labels.MatchEqual, "daz", "baz"),
},
},
},
"policy3": []*PriorityStream{
{
Selector: `{qyx="qzx", qox="qox"}`,
Priority: 1,
Matchers: []*labels.Matcher{
labels.MustNewMatcher(labels.MatchEqual, "qyx", "qzx"),
labels.MustNewMatcher(labels.MatchEqual, "qox", "qox"),
},
},
},
"policy4": []*PriorityStream{
{
Selector: `{qyx="qzx", qox="qox"}`,
Priority: 1,
Matchers: []*labels.Matcher{
labels.MustNewMatcher(labels.MatchEqual, "qyx", "qzx"),
labels.MustNewMatcher(labels.MatchEqual, "qox", "qox"),
},
},
},
"policy5": []*PriorityStream{
{
Selector: `{qab=~"qzx.*"}`,
Priority: 1,
Matchers: []*labels.Matcher{
labels.MustNewMatcher(labels.MatchRegexp, "qab", "qzx.*"),
},
},
},
"policy6": []*PriorityStream{
{
Selector: `{env="prod"}`,
Priority: 2,
Matchers: []*labels.Matcher{
labels.MustNewMatcher(labels.MatchEqual, "env", "prod"),
},
},
{
Selector: `{env=~"prod|staging"}`,
Priority: 1,
Matchers: []*labels.Matcher{
labels.MustNewMatcher(labels.MatchRegexp, "env", "prod|staging"),
},
},
{
Selector: `{team="finance"}`,
Priority: 4,
Matchers: []*labels.Matcher{
labels.MustNewMatcher(labels.MatchEqual, "team", "finance"),
},
},
},
"policy7": []*PriorityStream{
{
Selector: `{env=~"prod|dev"}`,
Priority: 3,
Matchers: []*labels.Matcher{
labels.MustNewMatcher(labels.MatchRegexp, "env", "prod|dev"),
},
},
},
}
require.Equal(t, "policy1", mapping.PolicyFor(labels.FromStrings("foo", "bar")))
// matches both policy2 and policy1 but policy1 has higher priority.
require.Equal(t, "policy1", mapping.PolicyFor(labels.FromStrings("foo", "bar", "daz", "baz")))
// matches policy3 and policy4 but policy3 appears first.
require.Equal(t, "policy3", mapping.PolicyFor(labels.FromStrings("qyx", "qzx", "qox", "qox")))
// matches no policy.
require.Equal(t, "", mapping.PolicyFor(labels.FromStrings("foo", "fooz", "daz", "qux", "quux", "corge")))
// matches policy5 through regex.
require.Equal(t, "policy5", mapping.PolicyFor(labels.FromStrings("qab", "qzxqox")))
require.Equal(t, "policy6", mapping.PolicyFor(labels.FromStrings("env", "prod", "team", "finance")))
}

@ -227,9 +227,10 @@ type Limits struct {
OTLPConfig push.OTLPConfig `yaml:"otlp_config" json:"otlp_config" doc:"description=OTLP log ingestion configurations"`
GlobalOTLPConfig push.GlobalOTLPConfig `yaml:"-" json:"-"`
BlockIngestionUntil dskit_flagext.Time `yaml:"block_ingestion_until" json:"block_ingestion_until"`
BlockIngestionStatusCode int `yaml:"block_ingestion_status_code" json:"block_ingestion_status_code"`
EnforcedLabels []string `yaml:"enforced_labels" json:"enforced_labels" category:"experimental"`
BlockIngestionUntil dskit_flagext.Time `yaml:"block_ingestion_until" json:"block_ingestion_until"`
BlockIngestionStatusCode int `yaml:"block_ingestion_status_code" json:"block_ingestion_status_code"`
EnforcedLabels []string `yaml:"enforced_labels" json:"enforced_labels" category:"experimental"`
PolicyStreamMapping PolicyStreamMapping `yaml:"policy_stream_mapping" json:"policy_stream_mapping" category:"experimental" doc:"description=Map of policies to stream selectors with a priority. Experimental.\nExample:\npolicy_stream_mapping:\n finance:\n - selectors: [\"{namespace=\"prod\", container=\"billing\"}\"]\n priority: 2\n ops:\n - selectors: [\"{namespace=\"prod\", container=\"ops\"}\"]\n priority: 1\n staging:\n - selectors: [\"{namespace=\"staging\"}, {namespace=\"dev\"}\"]\n priority: 1"`
IngestionPartitionsTenantShardSize int `yaml:"ingestion_partitions_tenant_shard_size" json:"ingestion_partitions_tenant_shard_size" category:"experimental"`
@ -511,6 +512,18 @@ func (l *Limits) Validate() error {
}
}
if l.PolicyStreamMapping != nil {
for policyName, policyStreams := range l.PolicyStreamMapping {
for idx, policyStream := range policyStreams {
matchers, err := syntax.ParseMatchers(policyStream.Selector, true)
if err != nil {
return fmt.Errorf("invalid labels matchers for policy stream mapping: %w", err)
}
l.PolicyStreamMapping[policyName][idx].Matchers = matchers
}
}
}
if _, err := deletionmode.ParseMode(l.DeletionMode); err != nil {
return err
}
@ -1111,6 +1124,10 @@ func (o *Overrides) EnforcedLabels(userID string) []string {
return o.getOverridesForUser(userID).EnforcedLabels
}
func (o *Overrides) PoliciesStreamMapping(userID string) PolicyStreamMapping {
return o.getOverridesForUser(userID).PolicyStreamMapping
}
func (o *Overrides) ShardAggregations(userID string) []string {
return o.getOverridesForUser(userID).ShardAggregations
}

@ -226,8 +226,9 @@ ruler_remote_write_headers:
Selector: `{a="b"}`,
},
},
OTLPConfig: defaultOTLPConfig,
EnforcedLabels: []string{},
OTLPConfig: defaultOTLPConfig,
EnforcedLabels: []string{},
PolicyStreamMapping: PolicyStreamMapping{},
},
},
{
@ -246,8 +247,9 @@ ruler_remote_write_headers:
Selector: `{a="b"}`,
},
},
OTLPConfig: defaultOTLPConfig,
EnforcedLabels: []string{},
OTLPConfig: defaultOTLPConfig,
EnforcedLabels: []string{},
PolicyStreamMapping: PolicyStreamMapping{},
},
},
{
@ -272,6 +274,7 @@ retention_stream:
RulerRemoteWriteHeaders: OverwriteMarshalingStringMap{map[string]string{"a": "b"}},
OTLPConfig: defaultOTLPConfig,
EnforcedLabels: []string{},
PolicyStreamMapping: PolicyStreamMapping{},
},
},
{
@ -293,8 +296,9 @@ reject_old_samples: true
Selector: `{a="b"}`,
},
},
OTLPConfig: defaultOTLPConfig,
EnforcedLabels: []string{},
OTLPConfig: defaultOTLPConfig,
EnforcedLabels: []string{},
PolicyStreamMapping: PolicyStreamMapping{},
},
},
{
@ -317,8 +321,9 @@ query_timeout: 5m
Selector: `{a="b"}`,
},
},
OTLPConfig: defaultOTLPConfig,
EnforcedLabels: []string{},
OTLPConfig: defaultOTLPConfig,
EnforcedLabels: []string{},
PolicyStreamMapping: PolicyStreamMapping{},
},
},
} {

@ -115,7 +115,7 @@ var DiscardedBytes = promauto.NewCounterVec(
Name: "discarded_bytes_total",
Help: "The total number of bytes that were discarded.",
},
[]string{ReasonLabel, "tenant", "retention_hours"},
[]string{ReasonLabel, "tenant", "retention_hours", "policy"},
)
// DiscardedSamples is a metric of the number of discarded samples, by reason.
@ -125,7 +125,7 @@ var DiscardedSamples = promauto.NewCounterVec(
Name: "discarded_samples_total",
Help: "The total number of samples that were discarded.",
},
[]string{ReasonLabel, "tenant", "retention_hours"},
[]string{ReasonLabel, "tenant", "retention_hours", "policy"},
)
var LineLengthHist = promauto.NewHistogram(prometheus.HistogramOpts{

Loading…
Cancel
Save