Skip to content

Commit b6816e3

Browse files
authored
feat(distributor): Relabel profiles at ingest (#3369)
* Relabel profiles at ingest This is an important operational tool: It allows us to deal with problems in the ingested profile labels. The relabeling rules cover both series label and also sample labes. By default we supply relabeling rules, which do remove the godeltaprof_ prefix from profiles generated by pyroscope-go. * Merge samples with identical stacktraces I have also added the most common test cases I could think of.
1 parent 359ecf1 commit b6816e3

File tree

13 files changed

+1614
-77
lines changed

13 files changed

+1614
-77
lines changed

pkg/distributor/distributor.go

Lines changed: 194 additions & 33 deletions
Original file line numberDiff line numberDiff line change
@@ -12,6 +12,7 @@ import (
1212
"sort"
1313
"sync"
1414
"time"
15+
"unsafe"
1516

1617
"connectrpc.com/connect"
1718
"github.com/dustin/go-humanize"
@@ -31,14 +32,15 @@ import (
3132
"github.com/prometheus/common/model"
3233
"go.uber.org/atomic"
3334

34-
googlev1 "github.com/grafana/pyroscope/api/gen/proto/go/google/v1"
35+
profilev1 "github.com/grafana/pyroscope/api/gen/proto/go/google/v1"
3536
pushv1 "github.com/grafana/pyroscope/api/gen/proto/go/push/v1"
3637
typesv1 "github.com/grafana/pyroscope/api/gen/proto/go/types/v1"
3738
connectapi "github.com/grafana/pyroscope/pkg/api/connect"
3839
"github.com/grafana/pyroscope/pkg/clientpool"
3940
"github.com/grafana/pyroscope/pkg/distributor/aggregator"
4041
distributormodel "github.com/grafana/pyroscope/pkg/distributor/model"
4142
phlaremodel "github.com/grafana/pyroscope/pkg/model"
43+
"github.com/grafana/pyroscope/pkg/model/relabel"
4244
"github.com/grafana/pyroscope/pkg/pprof"
4345
"github.com/grafana/pyroscope/pkg/slices"
4446
"github.com/grafana/pyroscope/pkg/tenant"
@@ -123,6 +125,7 @@ type Limits interface {
123125
MaxProfileSymbolValueLength(tenantID string) int
124126
MaxSessionsPerSeries(tenantID string) int
125127
EnforceLabelsOrder(tenantID string) bool
128+
IngestionRelabelingRules(tenantID string) []*relabel.Config
126129
validation.ProfileValidationLimits
127130
aggregator.Limits
128131
}
@@ -399,8 +402,11 @@ func (d *Distributor) sendRequests(ctx context.Context, req *distributormodel.Pu
399402
series.Labels = d.limitMaxSessionsPerSeries(maxSessionsPerSeries, series.Labels)
400403
}
401404

402-
// Next we split profiles by labels.
403-
profileSeries := extractSampleSeries(req)
405+
// Next we split profiles by labels and apply relabel rules.
406+
profileSeries, bytesRelabelDropped, profilesRelabelDropped := extractSampleSeries(req, d.limits.IngestionRelabelingRules(tenantID))
407+
validation.DiscardedBytes.WithLabelValues(string(validation.RelabelRules), tenantID).Add(bytesRelabelDropped)
408+
validation.DiscardedProfiles.WithLabelValues(string(validation.RelabelRules), tenantID).Add(profilesRelabelDropped)
409+
404410
// Filter our series and profiles without samples.
405411
for _, series := range profileSeries {
406412
series.Samples = slices.RemoveInPlace(series.Samples, func(sample *distributormodel.ProfileSample, _ int) bool {
@@ -491,8 +497,20 @@ func (d *Distributor) sendRequests(ctx context.Context, req *distributormodel.Pu
491497
}
492498
}
493499

500+
// sampleSize returns the size of a samples in bytes.
501+
func sampleSize(stringTable []string, samplesSlice []*profilev1.Sample) int64 {
502+
var size int64
503+
for _, s := range samplesSlice {
504+
size += int64(s.SizeVT())
505+
for _, l := range s.Label {
506+
size += int64(len(stringTable[l.Key]) + len(stringTable[l.Str]) + len(stringTable[l.NumUnit]))
507+
}
508+
}
509+
return size
510+
}
511+
494512
// profileSizeBytes returns the size of symbols and samples in bytes.
495-
func profileSizeBytes(p *googlev1.Profile) (symbols, samples int64) {
513+
func profileSizeBytes(p *profilev1.Profile) (symbols, samples int64) {
496514
fullSize := p.SizeVT()
497515
// remove samples
498516
samplesSlice := p.Sample
@@ -516,7 +534,7 @@ func profileSizeBytes(p *googlev1.Profile) (symbols, samples int64) {
516534
return
517535
}
518536

519-
func (d *Distributor) maybeAggregate(tenantID string, labels phlaremodel.Labels, profile *googlev1.Profile) (func() (*pprof.ProfileMerge, error), bool, error) {
537+
func (d *Distributor) maybeAggregate(tenantID string, labels phlaremodel.Labels, profile *profilev1.Profile) (func() (*pprof.ProfileMerge, error), bool, error) {
520538
a, ok := d.aggregator.AggregatorForTenant(tenantID)
521539
if !ok {
522540
return nil, false, nil
@@ -534,7 +552,7 @@ func (d *Distributor) maybeAggregate(tenantID string, labels phlaremodel.Labels,
534552
return r.Handler(), true, nil
535553
}
536554

537-
func mergeProfile(profile *googlev1.Profile) aggregator.AggregateFn[*pprof.ProfileMerge] {
555+
func mergeProfile(profile *profilev1.Profile) aggregator.AggregateFn[*pprof.ProfileMerge] {
538556
return func(m *pprof.ProfileMerge) (*pprof.ProfileMerge, error) {
539557
if m == nil {
540558
m = new(pprof.ProfileMerge)
@@ -633,7 +651,106 @@ func (d *Distributor) HealthyInstancesCount() int {
633651
return int(d.healthyInstancesCount.Load())
634652
}
635653

636-
func extractSampleSeries(req *distributormodel.PushRequest) []*distributormodel.ProfileSeries {
654+
type sampleKey struct {
655+
stacktrace string
656+
// note this is an index into the string table, rather than span ID
657+
spanIDIdx int64
658+
}
659+
660+
func sampleKeyFromSample(stringTable []string, s *profilev1.Sample) sampleKey {
661+
var k sampleKey
662+
663+
// populate spanID if present
664+
for _, l := range s.Label {
665+
if stringTable[int(l.Key)] == pprof.SpanIDLabelName {
666+
k.spanIDIdx = l.Str
667+
}
668+
}
669+
if len(s.LocationId) > 0 {
670+
k.stacktrace = unsafe.String(
671+
(*byte)(unsafe.Pointer(&s.LocationId[0])),
672+
len(s.LocationId)*8,
673+
)
674+
}
675+
return k
676+
}
677+
678+
type lazyGroup struct {
679+
sampleGroup pprof.SampleGroup
680+
// The map is only initialized when the group is being modified. Key is the
681+
// string representation (unsafe) of the sample stack trace and its potential
682+
// span ID.
683+
sampleMap map[sampleKey]*profilev1.Sample
684+
labels phlaremodel.Labels
685+
}
686+
687+
func (g *lazyGroup) addSampleGroup(stringTable []string, sg pprof.SampleGroup) {
688+
if len(g.sampleGroup.Samples) == 0 {
689+
g.sampleGroup = sg
690+
return
691+
}
692+
693+
// If the group is already initialized, we need to merge the samples.
694+
if g.sampleMap == nil {
695+
g.sampleMap = make(map[sampleKey]*profilev1.Sample)
696+
for _, s := range g.sampleGroup.Samples {
697+
g.sampleMap[sampleKeyFromSample(stringTable, s)] = s
698+
}
699+
}
700+
701+
for _, s := range sg.Samples {
702+
k := sampleKeyFromSample(stringTable, s)
703+
if _, ok := g.sampleMap[k]; !ok {
704+
g.sampleGroup.Samples = append(g.sampleGroup.Samples, s)
705+
g.sampleMap[k] = s
706+
} else {
707+
// merge the samples
708+
for idx := range s.Value {
709+
g.sampleMap[k].Value[idx] += s.Value[idx]
710+
}
711+
}
712+
}
713+
}
714+
715+
type groupsWithFingerprints struct {
716+
m map[uint64][]lazyGroup
717+
order []uint64
718+
}
719+
720+
func newGroupsWithFingerprints() *groupsWithFingerprints {
721+
return &groupsWithFingerprints{
722+
m: make(map[uint64][]lazyGroup),
723+
}
724+
}
725+
726+
func (g *groupsWithFingerprints) add(stringTable []string, lbls phlaremodel.Labels, group pprof.SampleGroup) {
727+
fp := lbls.Hash()
728+
idxs, ok := g.m[fp]
729+
if ok {
730+
// fingerprint matches, check if the labels are the same
731+
for _, idx := range idxs {
732+
if phlaremodel.CompareLabelPairs(idx.labels, lbls) == 0 {
733+
// append samples to the group
734+
idx.addSampleGroup(stringTable, group)
735+
return
736+
}
737+
}
738+
} else {
739+
g.order = append(g.order, fp)
740+
}
741+
742+
// add the labels to the list
743+
g.m[fp] = append(g.m[fp], lazyGroup{
744+
sampleGroup: group,
745+
labels: lbls,
746+
})
747+
}
748+
749+
func extractSampleSeries(req *distributormodel.PushRequest, relabelRules []*relabel.Config) (result []*distributormodel.ProfileSeries, bytesRelabelDropped, profilesRelabelDropped float64) {
750+
var (
751+
lblbuilder = phlaremodel.NewLabelsBuilder(phlaremodel.EmptyLabels())
752+
)
753+
637754
profileSeries := make([]*distributormodel.ProfileSeries, 0, len(req.Series))
638755
for _, series := range req.Series {
639756
s := &distributormodel.ProfileSeries{
@@ -643,33 +760,74 @@ func extractSampleSeries(req *distributormodel.PushRequest) []*distributormodel.
643760
for _, raw := range series.Samples {
644761
pprof.RenameLabel(raw.Profile.Profile, pprof.ProfileIDLabelName, pprof.SpanIDLabelName)
645762
groups := pprof.GroupSamplesWithoutLabels(raw.Profile.Profile, pprof.SpanIDLabelName)
763+
646764
if len(groups) == 0 || (len(groups) == 1 && len(groups[0].Labels) == 0) {
647765
// No sample labels in the profile.
766+
767+
// relabel the labels of the series
768+
lblbuilder.Reset(series.Labels)
769+
if len(relabelRules) > 0 {
770+
keep := relabel.ProcessBuilder(lblbuilder, relabelRules...)
771+
if !keep {
772+
bytesRelabelDropped += float64(raw.Profile.SizeVT())
773+
profilesRelabelDropped++ // in this case we dropped a whole profile
774+
continue
775+
}
776+
}
777+
778+
// Copy over the labels from the builder
779+
s.Labels = lblbuilder.Labels()
780+
648781
// We do not modify the request.
649782
s.Samples = append(s.Samples, raw)
783+
650784
continue
651785
}
652-
e := pprof.NewSampleExporter(raw.Profile.Profile)
786+
787+
// iterate through groups relabel them and find relevant overlapping labelsets
788+
groupsKept := newGroupsWithFingerprints()
653789
for _, group := range groups {
654-
// exportSamples creates a new profile with the samples provided.
655-
// The samples are obtained via GroupSamples call, which means
656-
// the underlying capacity is referenced by the source profile.
657-
// Therefore, the slice has to be copied and samples zeroed to
658-
// avoid ownership issues.
659-
profile := exportSamples(e, group.Samples)
660-
// Note that group.Labels reference strings from the source profile.
661-
labels := mergeSeriesAndSampleLabels(raw.Profile.Profile, series.Labels, group.Labels)
662-
profileSeries = append(profileSeries, &distributormodel.ProfileSeries{
663-
Labels: labels,
664-
Samples: []*distributormodel.ProfileSample{{Profile: profile}},
665-
})
790+
lblbuilder.Reset(series.Labels)
791+
addSampleLabelsToLabelsBuilder(lblbuilder, raw.Profile.Profile, group.Labels)
792+
if len(relabelRules) > 0 {
793+
keep := relabel.ProcessBuilder(lblbuilder, relabelRules...)
794+
if !keep {
795+
bytesRelabelDropped += float64(sampleSize(raw.Profile.Profile.StringTable, group.Samples))
796+
continue
797+
}
798+
}
799+
800+
// add the group to the list
801+
groupsKept.add(raw.Profile.StringTable, lblbuilder.Labels(), group)
802+
}
803+
804+
if len(groupsKept.m) == 0 {
805+
// no groups kept, count the whole profile as dropped
806+
profilesRelabelDropped++
807+
continue
808+
}
809+
810+
e := pprof.NewSampleExporter(raw.Profile.Profile)
811+
for _, idx := range groupsKept.order {
812+
for _, group := range groupsKept.m[idx] {
813+
// exportSamples creates a new profile with the samples provided.
814+
// The samples are obtained via GroupSamples call, which means
815+
// the underlying capacity is referenced by the source profile.
816+
// Therefore, the slice has to be copied and samples zeroed to
817+
// avoid ownership issues.
818+
profile := exportSamples(e, group.sampleGroup.Samples)
819+
profileSeries = append(profileSeries, &distributormodel.ProfileSeries{
820+
Labels: group.labels,
821+
Samples: []*distributormodel.ProfileSample{{Profile: profile}},
822+
})
823+
}
666824
}
667825
}
668826
if len(s.Samples) > 0 {
669827
profileSeries = append(profileSeries, s)
670828
}
671829
}
672-
return profileSeries
830+
return profileSeries, bytesRelabelDropped, profilesRelabelDropped
673831
}
674832

675833
func (d *Distributor) limitMaxSessionsPerSeries(maxSessionsPerSeries int, labels phlaremodel.Labels) phlaremodel.Labels {
@@ -712,22 +870,25 @@ func (d *Distributor) rateLimit(tenantID string, req *distributormodel.PushReque
712870
return nil
713871
}
714872

715-
// mergeSeriesAndSampleLabels merges sample labels with
716-
// series labels. Series labels take precedence.
717-
func mergeSeriesAndSampleLabels(p *googlev1.Profile, sl []*typesv1.LabelPair, pl []*googlev1.Label) []*typesv1.LabelPair {
718-
m := phlaremodel.Labels(sl).Clone()
873+
// addSampleLabelsToLabelsBuilder: adds sample label that don't exists yet on the profile builder. So the existing labels take precedence.
874+
func addSampleLabelsToLabelsBuilder(b *phlaremodel.LabelsBuilder, p *profilev1.Profile, pl []*profilev1.Label) {
875+
var name string
719876
for _, l := range pl {
720-
m = append(m, &typesv1.LabelPair{
721-
Name: p.StringTable[l.Key],
722-
Value: p.StringTable[l.Str],
723-
})
877+
name = p.StringTable[l.Key]
878+
if l.Str <= 0 {
879+
// skip if label value is not a string
880+
continue
881+
}
882+
if b.Get(name) != "" {
883+
// do nothing if label name already exists
884+
continue
885+
}
886+
b.Set(name, p.StringTable[l.Str])
724887
}
725-
sort.Stable(m)
726-
return m.Unique()
727888
}
728889

729-
func exportSamples(e *pprof.SampleExporter, samples []*googlev1.Sample) *pprof.Profile {
730-
samplesCopy := make([]*googlev1.Sample, len(samples))
890+
func exportSamples(e *pprof.SampleExporter, samples []*profilev1.Sample) *pprof.Profile {
891+
samplesCopy := make([]*profilev1.Sample, len(samples))
731892
copy(samplesCopy, samples)
732893
slices.Clear(samples)
733894
n := pprof.NewProfile()

0 commit comments

Comments
 (0)