Skip to content

Commit cfe7ac3

Browse files
authored
Add distributor per labelset received samples metric (#6443)
* add distributor per labelset ingestion rate metric Signed-off-by: Ben Ye <[email protected]> * changelog Signed-off-by: Ben Ye <[email protected]> * update metrics synchronously Signed-off-by: Ben Ye <[email protected]> * allocate label set counter only if there are matching limtis for the series Signed-off-by: Ben Ye <[email protected]> * fix counter initialization Signed-off-by: Ben Ye <[email protected]> * fix lock on active users Signed-off-by: Ben Ye <[email protected]> --------- Signed-off-by: Ben Ye <[email protected]>
1 parent 0b4041b commit cfe7ac3

File tree

7 files changed

+418
-1
lines changed

7 files changed

+418
-1
lines changed

CHANGELOG.md

+1
Original file line numberDiff line numberDiff line change
@@ -51,6 +51,7 @@
5151
* [ENHANCEMENT] Distributor: Add a new `-distributor.num-push-workers` flag to use a goroutine worker pool when sending data from distributor to ingesters. #6406
5252
* [ENHANCEMENT] Ingester: If a limit per label set entry doesn't have any label, use it as the default partition to catch all series that doesn't match any other label sets entries. #6435
5353
* [ENHANCEMENT] Querier: Add new `cortex_querier_codec_response_size` metric to track the size of the encoded query responses from queriers. #6444
54+
* [ENHANCEMENT] Distributor: Added `cortex_distributor_received_samples_per_labelset_total` metric to calculate ingestion rate per label set. #6443
5455
* [BUGFIX] Runtime-config: Handle absolute file paths when working directory is not / #6224
5556
* [BUGFIX] Ruler: Allow rule evaluation to complete during shutdown. #6326
5657
* [BUGFIX] Ring: update ring with new ip address when instance is lost, rejoins, but heartbeat is disabled. #6271

pkg/distributor/distributor.go

+62-1
Original file line numberDiff line numberDiff line change
@@ -64,6 +64,8 @@ const (
6464

6565
clearStaleIngesterMetricsInterval = time.Minute
6666

67+
labelSetMetricsTickInterval = 30 * time.Second
68+
6769
// mergeSlicesParallelism is a constant of how much go routines we should use to merge slices, and
6870
// it was based on empirical observation: See BenchmarkMergeSlicesParallel
6971
mergeSlicesParallelism = 8
@@ -107,6 +109,7 @@ type Distributor struct {
107109
// Metrics
108110
queryDuration *instrument.HistogramCollector
109111
receivedSamples *prometheus.CounterVec
112+
receivedSamplesPerLabelSet *prometheus.CounterVec
110113
receivedExemplars *prometheus.CounterVec
111114
receivedMetadata *prometheus.CounterVec
112115
incomingSamples *prometheus.CounterVec
@@ -125,6 +128,9 @@ type Distributor struct {
125128
validateMetrics *validation.ValidateMetrics
126129

127130
asyncExecutor util.AsyncExecutor
131+
132+
// Map to track label sets from user.
133+
labelSetTracker *labelSetTracker
128134
}
129135

130136
// Config contains the configuration required to
@@ -302,6 +308,11 @@ func New(cfg Config, clientConfig ingester_client.Config, limits *validation.Ove
302308
Name: "distributor_received_samples_total",
303309
Help: "The total number of received samples, excluding rejected and deduped samples.",
304310
}, []string{"user", "type"}),
311+
receivedSamplesPerLabelSet: promauto.With(reg).NewCounterVec(prometheus.CounterOpts{
312+
Namespace: "cortex",
313+
Name: "distributor_received_samples_per_labelset_total",
314+
Help: "The total number of received samples per label set, excluding rejected and deduped samples.",
315+
}, []string{"user", "type", "labelset"}),
305316
receivedExemplars: promauto.With(reg).NewCounterVec(prometheus.CounterOpts{
306317
Namespace: "cortex",
307318
Name: "distributor_received_exemplars_total",
@@ -377,6 +388,8 @@ func New(cfg Config, clientConfig ingester_client.Config, limits *validation.Ove
377388
asyncExecutor: util.NewNoOpExecutor(),
378389
}
379390

391+
d.labelSetTracker = newLabelSetTracker(d.receivedSamplesPerLabelSet)
392+
380393
if cfg.NumPushWorkers > 0 {
381394
util_log.WarnExperimentalUse("Distributor: using goroutine worker pool")
382395
d.asyncExecutor = util.NewWorkerPool("distributor", cfg.NumPushWorkers, reg)
@@ -449,6 +462,9 @@ func (d *Distributor) running(ctx context.Context) error {
449462
staleIngesterMetricTicker := time.NewTicker(clearStaleIngesterMetricsInterval)
450463
defer staleIngesterMetricTicker.Stop()
451464

465+
labelSetMetricsTicker := time.NewTicker(labelSetMetricsTickInterval)
466+
defer labelSetMetricsTicker.Stop()
467+
452468
for {
453469
select {
454470
case <-ctx.Done():
@@ -460,6 +476,9 @@ func (d *Distributor) running(ctx context.Context) error {
460476
case <-staleIngesterMetricTicker.C:
461477
d.cleanStaleIngesterMetrics()
462478

479+
case <-labelSetMetricsTicker.C:
480+
d.updateLabelSetMetrics()
481+
463482
case err := <-d.subservicesWatcher.Chan():
464483
return errors.Wrap(err, "distributor subservice failed")
465484
}
@@ -486,6 +505,10 @@ func (d *Distributor) cleanupInactiveUser(userID string) {
486505
level.Warn(d.log).Log("msg", "failed to remove cortex_distributor_deduped_samples_total metric for user", "user", userID, "err", err)
487506
}
488507

508+
if err := util.DeleteMatchingLabels(d.receivedSamplesPerLabelSet, map[string]string{"user": userID}); err != nil {
509+
level.Warn(d.log).Log("msg", "failed to remove cortex_distributor_received_samples_per_labelset_total metric for user", "user", userID, "err", err)
510+
}
511+
489512
validation.DeletePerUserValidationMetrics(d.validateMetrics, userID, d.log)
490513
}
491514

@@ -777,6 +800,19 @@ func (d *Distributor) Push(ctx context.Context, req *cortexpb.WriteRequest) (*co
777800
return &cortexpb.WriteResponse{}, firstPartialErr
778801
}
779802

803+
func (d *Distributor) updateLabelSetMetrics() {
804+
activeUserSet := make(map[string]map[uint64]struct{})
805+
for _, user := range d.activeUsers.ActiveUsers() {
806+
limits := d.limits.LimitsPerLabelSet(user)
807+
activeUserSet[user] = make(map[uint64]struct{}, len(limits))
808+
for _, l := range limits {
809+
activeUserSet[user][l.Hash] = struct{}{}
810+
}
811+
}
812+
813+
d.labelSetTracker.updateMetrics(activeUserSet)
814+
}
815+
780816
func (d *Distributor) cleanStaleIngesterMetrics() {
781817
healthy, unhealthy, err := d.ingestersRing.GetAllInstanceDescs(ring.WriteNoExtend)
782818
if err != nil {
@@ -888,8 +924,12 @@ func (d *Distributor) prepareSeriesKeys(ctx context.Context, req *cortexpb.Write
888924
validatedFloatSamples := 0
889925
validatedHistogramSamples := 0
890926
validatedExemplars := 0
927+
limitsPerLabelSet := d.limits.LimitsPerLabelSet(userID)
891928

892-
var firstPartialErr error
929+
var (
930+
labelSetCounters map[uint64]*samplesLabelSetEntry
931+
firstPartialErr error
932+
)
893933

894934
latestSampleTimestampMs := int64(0)
895935
defer func() {
@@ -1005,12 +1045,33 @@ func (d *Distributor) prepareSeriesKeys(ctx context.Context, req *cortexpb.Write
10051045
continue
10061046
}
10071047

1048+
matchedLabelSetLimits := validation.LimitsPerLabelSetsForSeries(limitsPerLabelSet, cortexpb.FromLabelAdaptersToLabels(validatedSeries.Labels))
1049+
if len(matchedLabelSetLimits) > 0 && labelSetCounters == nil {
1050+
// TODO: use pool.
1051+
labelSetCounters = make(map[uint64]*samplesLabelSetEntry, len(matchedLabelSetLimits))
1052+
}
1053+
for _, l := range matchedLabelSetLimits {
1054+
if c, exists := labelSetCounters[l.Hash]; exists {
1055+
c.floatSamples += int64(len(ts.Samples))
1056+
c.histogramSamples += int64(len(ts.Histograms))
1057+
} else {
1058+
labelSetCounters[l.Hash] = &samplesLabelSetEntry{
1059+
floatSamples: int64(len(ts.Samples)),
1060+
histogramSamples: int64(len(ts.Histograms)),
1061+
labels: l.LabelSet,
1062+
}
1063+
}
1064+
}
1065+
10081066
seriesKeys = append(seriesKeys, key)
10091067
validatedTimeseries = append(validatedTimeseries, validatedSeries)
10101068
validatedFloatSamples += len(ts.Samples)
10111069
validatedHistogramSamples += len(ts.Histograms)
10121070
validatedExemplars += len(ts.Exemplars)
10131071
}
1072+
for h, counter := range labelSetCounters {
1073+
d.labelSetTracker.increaseSamplesLabelSet(userID, h, counter.labels, counter.floatSamples, counter.histogramSamples)
1074+
}
10141075
return seriesKeys, validatedTimeseries, validatedFloatSamples, validatedHistogramSamples, validatedExemplars, firstPartialErr, nil
10151076
}
10161077

pkg/distributor/distributor_test.go

+118
Original file line numberDiff line numberDiff line change
@@ -420,6 +420,7 @@ func TestDistributor_MetricsCleanup(t *testing.T) {
420420
"cortex_distributor_metadata_in_total",
421421
"cortex_distributor_non_ha_samples_received_total",
422422
"cortex_distributor_latest_seen_sample_timestamp_seconds",
423+
"cortex_distributor_received_samples_per_labelset_total",
423424
}
424425

425426
allMetrics := append(removedMetrics, permanentMetrics...)
@@ -438,6 +439,8 @@ func TestDistributor_MetricsCleanup(t *testing.T) {
438439
d.nonHASamples.WithLabelValues("userA").Add(5)
439440
d.dedupedSamples.WithLabelValues("userA", "cluster1").Inc() // We cannot clean this metric
440441
d.latestSeenSampleTimestampPerUser.WithLabelValues("userA").Set(1111)
442+
d.receivedSamplesPerLabelSet.WithLabelValues("userA", sampleMetricTypeFloat, "{}").Add(5)
443+
d.receivedSamplesPerLabelSet.WithLabelValues("userA", sampleMetricTypeHistogram, "{}").Add(10)
441444

442445
h, _, _ := r.GetAllInstanceDescs(ring.WriteNoExtend)
443446
ingId0, _ := r.GetInstanceIdByAddr(h[0].Addr)
@@ -473,6 +476,11 @@ func TestDistributor_MetricsCleanup(t *testing.T) {
473476
cortex_distributor_received_metadata_total{user="userA"} 5
474477
cortex_distributor_received_metadata_total{user="userB"} 10
475478
479+
# HELP cortex_distributor_received_samples_per_labelset_total The total number of received samples per label set, excluding rejected and deduped samples.
480+
# TYPE cortex_distributor_received_samples_per_labelset_total counter
481+
cortex_distributor_received_samples_per_labelset_total{labelset="{}",type="float",user="userA"} 5
482+
cortex_distributor_received_samples_per_labelset_total{labelset="{}",type="histogram",user="userA"} 10
483+
476484
# HELP cortex_distributor_received_samples_total The total number of received samples, excluding rejected and deduped samples.
477485
# TYPE cortex_distributor_received_samples_total counter
478486
cortex_distributor_received_samples_total{type="float",user="userA"} 5
@@ -4081,6 +4089,116 @@ func TestDistributor_Push_RelabelDropWillExportMetricOfDroppedSamples(t *testing
40814089
}
40824090
}
40834091

4092+
func TestDistributor_PushLabelSetMetrics(t *testing.T) {
4093+
t.Parallel()
4094+
inputSeries := []labels.Labels{
4095+
{
4096+
{Name: "__name__", Value: "foo"},
4097+
{Name: "cluster", Value: "one"},
4098+
},
4099+
{
4100+
{Name: "__name__", Value: "bar"},
4101+
{Name: "cluster", Value: "one"},
4102+
},
4103+
{
4104+
{Name: "__name__", Value: "bar"},
4105+
{Name: "cluster", Value: "two"},
4106+
},
4107+
{
4108+
{Name: "__name__", Value: "foo"},
4109+
{Name: "cluster", Value: "three"},
4110+
},
4111+
}
4112+
4113+
var err error
4114+
var limits validation.Limits
4115+
flagext.DefaultValues(&limits)
4116+
limits.LimitsPerLabelSet = []validation.LimitsPerLabelSet{
4117+
{Hash: 0, LabelSet: labels.FromStrings("cluster", "one")},
4118+
{Hash: 1, LabelSet: labels.FromStrings("cluster", "two")},
4119+
{Hash: 2, LabelSet: labels.EmptyLabels()},
4120+
}
4121+
4122+
ds, _, regs, _ := prepare(t, prepConfig{
4123+
numIngesters: 2,
4124+
happyIngesters: 2,
4125+
numDistributors: 1,
4126+
shardByAllLabels: true,
4127+
limits: &limits,
4128+
})
4129+
reg := regs[0]
4130+
4131+
// Push the series to the distributor
4132+
id := "user"
4133+
req := mockWriteRequest(inputSeries, 1, 1, false)
4134+
ctx := user.InjectOrgID(context.Background(), id)
4135+
_, err = ds[0].Push(ctx, req)
4136+
require.NoError(t, err)
4137+
4138+
require.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(`
4139+
# HELP cortex_distributor_received_samples_per_labelset_total The total number of received samples per label set, excluding rejected and deduped samples.
4140+
# TYPE cortex_distributor_received_samples_per_labelset_total counter
4141+
cortex_distributor_received_samples_per_labelset_total{labelset="{cluster=\"one\"}",type="float",user="user"} 2
4142+
cortex_distributor_received_samples_per_labelset_total{labelset="{cluster=\"two\"}",type="float",user="user"} 1
4143+
cortex_distributor_received_samples_per_labelset_total{labelset="{}",type="float",user="user"} 1
4144+
`), "cortex_distributor_received_samples_per_labelset_total"))
4145+
4146+
// Push more series.
4147+
inputSeries = []labels.Labels{
4148+
{
4149+
{Name: "__name__", Value: "baz"},
4150+
{Name: "cluster", Value: "two"},
4151+
},
4152+
{
4153+
{Name: "__name__", Value: "foo"},
4154+
{Name: "cluster", Value: "four"},
4155+
},
4156+
}
4157+
// Write the same request twice for different users.
4158+
req = mockWriteRequest(inputSeries, 1, 1, false)
4159+
ctx2 := user.InjectOrgID(context.Background(), "user2")
4160+
_, err = ds[0].Push(ctx, req)
4161+
require.NoError(t, err)
4162+
req = mockWriteRequest(inputSeries, 1, 1, false)
4163+
_, err = ds[0].Push(ctx2, req)
4164+
require.NoError(t, err)
4165+
require.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(`
4166+
# HELP cortex_distributor_received_samples_per_labelset_total The total number of received samples per label set, excluding rejected and deduped samples.
4167+
# TYPE cortex_distributor_received_samples_per_labelset_total counter
4168+
cortex_distributor_received_samples_per_labelset_total{labelset="{cluster=\"one\"}",type="float",user="user"} 2
4169+
cortex_distributor_received_samples_per_labelset_total{labelset="{cluster=\"two\"}",type="float",user="user"} 2
4170+
cortex_distributor_received_samples_per_labelset_total{labelset="{cluster=\"two\"}",type="float",user="user2"} 1
4171+
cortex_distributor_received_samples_per_labelset_total{labelset="{}",type="float",user="user"} 2
4172+
cortex_distributor_received_samples_per_labelset_total{labelset="{}",type="float",user="user2"} 1
4173+
`), "cortex_distributor_received_samples_per_labelset_total"))
4174+
4175+
// Remove existing limits and add new limits
4176+
limits.LimitsPerLabelSet = []validation.LimitsPerLabelSet{
4177+
{Hash: 3, LabelSet: labels.FromStrings("cluster", "three")},
4178+
{Hash: 4, LabelSet: labels.FromStrings("cluster", "four")},
4179+
{Hash: 2, LabelSet: labels.EmptyLabels()},
4180+
}
4181+
ds[0].limits, err = validation.NewOverrides(limits, nil)
4182+
require.NoError(t, err)
4183+
ds[0].updateLabelSetMetrics()
4184+
// Old label set metrics are removed. New label set metrics will be added when
4185+
// new requests come in.
4186+
require.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(`
4187+
# HELP cortex_distributor_received_samples_per_labelset_total The total number of received samples per label set, excluding rejected and deduped samples.
4188+
# TYPE cortex_distributor_received_samples_per_labelset_total counter
4189+
cortex_distributor_received_samples_per_labelset_total{labelset="{}",type="float",user="user"} 2
4190+
cortex_distributor_received_samples_per_labelset_total{labelset="{}",type="float",user="user2"} 1
4191+
`), "cortex_distributor_received_samples_per_labelset_total"))
4192+
4193+
// Metrics from `user` got removed but `user2` metric should remain.
4194+
ds[0].cleanupInactiveUser(id)
4195+
require.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(`
4196+
# HELP cortex_distributor_received_samples_per_labelset_total The total number of received samples per label set, excluding rejected and deduped samples.
4197+
# TYPE cortex_distributor_received_samples_per_labelset_total counter
4198+
cortex_distributor_received_samples_per_labelset_total{labelset="{}",type="float",user="user2"} 1
4199+
`), "cortex_distributor_received_samples_per_labelset_total"))
4200+
}
4201+
40844202
func countMockIngestersCalls(ingesters []*mockIngester, name string) int {
40854203
count := 0
40864204
for i := 0; i < len(ingesters); i++ {

pkg/distributor/metrics.go

+95
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,95 @@
1+
package distributor
2+
3+
import (
4+
"sync"
5+
6+
"github.com./prometheus/client_golang/prometheus"
7+
"github.com./prometheus/common/model"
8+
"github.com./prometheus/prometheus/model/labels"
9+
10+
"github.com./cortexproject/cortex/pkg/util"
11+
)
12+
13+
const (
14+
numMetricShards = 128
15+
)
16+
17+
type labelSetTracker struct {
18+
receivedSamplesPerLabelSet *prometheus.CounterVec
19+
20+
shards []*labelSetCounterShard
21+
}
22+
23+
func newLabelSetTracker(receivedSamplesPerLabelSet *prometheus.CounterVec) *labelSetTracker {
24+
shards := make([]*labelSetCounterShard, 0, numMetricShards)
25+
for i := 0; i < numMetricShards; i++ {
26+
shards = append(shards, &labelSetCounterShard{
27+
RWMutex: &sync.RWMutex{},
28+
userLabelSets: map[string]map[uint64]labels.Labels{},
29+
})
30+
}
31+
return &labelSetTracker{shards: shards, receivedSamplesPerLabelSet: receivedSamplesPerLabelSet}
32+
}
33+
34+
type labelSetCounterShard struct {
35+
*sync.RWMutex
36+
userLabelSets map[string]map[uint64]labels.Labels
37+
}
38+
39+
type samplesLabelSetEntry struct {
40+
floatSamples int64
41+
histogramSamples int64
42+
labels labels.Labels
43+
}
44+
45+
func (m *labelSetTracker) increaseSamplesLabelSet(userId string, hash uint64, labelSet labels.Labels, floatSamples, histogramSamples int64) {
46+
s := m.shards[util.HashFP(model.Fingerprint(hash))%numMetricShards]
47+
s.Lock()
48+
if userEntry, ok := s.userLabelSets[userId]; ok {
49+
if _, ok2 := userEntry[hash]; !ok2 {
50+
userEntry[hash] = labelSet
51+
}
52+
} else {
53+
s.userLabelSets[userId] = map[uint64]labels.Labels{hash: labelSet}
54+
}
55+
// Unlock before we update metrics.
56+
s.Unlock()
57+
58+
labelSetStr := labelSet.String()
59+
if floatSamples > 0 {
60+
m.receivedSamplesPerLabelSet.WithLabelValues(userId, sampleMetricTypeFloat, labelSetStr).Add(float64(floatSamples))
61+
}
62+
if histogramSamples > 0 {
63+
m.receivedSamplesPerLabelSet.WithLabelValues(userId, sampleMetricTypeHistogram, labelSetStr).Add(float64(histogramSamples))
64+
}
65+
}
66+
67+
// Clean up dangling user and label set from the tracker as well as metrics.
68+
func (m *labelSetTracker) updateMetrics(userSet map[string]map[uint64]struct{}) {
69+
for i := 0; i < numMetricShards; i++ {
70+
shard := m.shards[i]
71+
shard.Lock()
72+
73+
for user, userEntry := range shard.userLabelSets {
74+
limits, ok := userSet[user]
75+
if !ok {
76+
// If user is removed, we will delete user metrics in cleanupInactiveUser loop
77+
// so skip deleting metrics here.
78+
delete(shard.userLabelSets, user)
79+
continue
80+
}
81+
for h, lbls := range userEntry {
82+
// This limit no longer exists.
83+
if _, ok := limits[h]; !ok {
84+
delete(userEntry, h)
85+
labelSetStr := lbls.String()
86+
m.receivedSamplesPerLabelSet.DeleteLabelValues(user, sampleMetricTypeFloat, labelSetStr)
87+
m.receivedSamplesPerLabelSet.DeleteLabelValues(user, sampleMetricTypeHistogram, labelSetStr)
88+
continue
89+
}
90+
}
91+
}
92+
93+
shard.Unlock()
94+
}
95+
}

0 commit comments

Comments
 (0)