diff --git a/.chloggen/prometheusremotewrite-optimize-createattributes.yaml b/.chloggen/prometheusremotewrite-optimize-createattributes.yaml new file mode 100644 index 000000000000..e98e09b17fd2 --- /dev/null +++ b/.chloggen/prometheusremotewrite-optimize-createattributes.yaml @@ -0,0 +1,27 @@ +# Use this changelog template to create an entry for release notes. + +# One of 'breaking', 'deprecation', 'new_component', 'enhancement', 'bug_fix' +change_type: 'breaking' + +# The name of the component, or a single word describing the area of concern, (e.g. filelogreceiver) +component: prometheusremotewriteexporter + +# A brief description of the change. Surround your text with quotes ("") if it needs to start with a backtick (`). +note: reduce allocations in createAttributes + +# Mandatory: One or more tracking issues related to the change. You can use the PR number here if no issue exists. +issues: [35184] + +# (Optional) One or more lines of additional information to render under the primary note. +# These lines will be padded with 2 spaces and then inserted directly into the document. +# Use pipe (|) for multiline entries. +subtext: + +# If your change doesn't affect end users or the exported elements of any package, +# you should instead start your pull request title with [chore] or use the "Skip Changelog" label. +# Optional: The change log or logs in which this entry should be included. +# e.g. '[user]' or '[user, api]' +# Include 'user' if the change is relevant to end users. +# Include 'api' if there is a change to a library API. +# Default: '[user]' +change_logs: [] diff --git a/exporter/prometheusremotewriteexporter/exporter.go b/exporter/prometheusremotewriteexporter/exporter.go index 74e1cff42b79..c11182169ebd 100644 --- a/exporter/prometheusremotewriteexporter/exporter.go +++ b/exporter/prometheusremotewriteexporter/exporter.go @@ -53,6 +53,12 @@ func (p *prwTelemetryOtel) recordTranslatedTimeSeries(ctx context.Context, numTS p.telemetryBuilder.ExporterPrometheusremotewriteTranslatedTimeSeries.Add(ctx, int64(numTS), metric.WithAttributes(p.otelAttrs...)) } +var converterPool = sync.Pool{ + New: func() any { + return prometheusremotewrite.NewPrometheusConverter() + }, +} + type buffer struct { protobuf *proto.Buffer snappy []byte @@ -191,8 +197,10 @@ func (prwe *prwExporter) PushMetrics(ctx context.Context, md pmetric.Metrics) er case <-prwe.closeChan: return errors.New("shutdown has been called") default: + converter := converterPool.Get().(*prometheusremotewrite.PrometheusConverter) + defer converterPool.Put(converter) - tsMap, err := prometheusremotewrite.FromMetrics(md, prwe.exporterSettings) + tsMap, err := converter.FromMetrics(md, prwe.exporterSettings) if err != nil { prwe.telemetry.recordTranslationFailure(ctx) prwe.settings.Logger.Debug("failed to translate metrics, exporting remaining metrics", zap.Error(err), zap.Int("translated", len(tsMap))) diff --git a/exporter/prometheusremotewriteexporter/exporter_concurrency_test.go b/exporter/prometheusremotewriteexporter/exporter_concurrency_test.go new file mode 100644 index 000000000000..31cae0afff75 --- /dev/null +++ b/exporter/prometheusremotewriteexporter/exporter_concurrency_test.go @@ -0,0 +1,153 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +//go:build !race +// +build !race + +// note: this test doesn't pass currently due to a race condition in batchTimeSeries +// WARNING: DATA RACE +// Write at 0x00c0001e9550 by goroutine 34: +// github.com/open-telemetry/opentelemetry-collector-contrib/exporter/prometheusremotewriteexporter.batchTimeSeries() +// helper.go:92 +0xf8b +// github.com/open-telemetry/opentelemetry-collector-contrib/exporter/prometheusremotewriteexporter.(*prwExporter).handleExport() +// exporter.go:240 +0xe4 +// github.com/open-telemetry/opentelemetry-collector-contrib/exporter/prometheusremotewriteexporter.(*prwExporter).PushMetrics() +// exporter.go:217 +0x70f +// github.com/open-telemetry/opentelemetry-collector-contrib/exporter/prometheusremotewriteexporter.Test_PushMetricsConcurrent.func3() +// exporter_test.go:905 +0x78 + +package prometheusremotewriteexporter + +import ( + "context" + "io" + "net/http" + "net/http/httptest" + "strconv" + "sync" + "testing" + "time" + + "github.com/gogo/protobuf/proto" + "github.com/golang/snappy" + "github.com/prometheus/prometheus/prompb" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "go.opentelemetry.io/collector/component/componenttest" + "go.opentelemetry.io/collector/config/confighttp" + "go.opentelemetry.io/collector/config/configretry" + "go.opentelemetry.io/collector/config/configtelemetry" + "go.opentelemetry.io/collector/exporter/exportertest" + "go.opentelemetry.io/collector/pdata/pmetric" + + "github.com/open-telemetry/opentelemetry-collector-contrib/internal/coreinternal/testdata" +) + +// Test everything works when there is more than one goroutine calling PushMetrics. +// Today we only use 1 worker per exporter, but the intention of this test is to future-proof in case it changes. +func Test_PushMetricsConcurrent(t *testing.T) { + n := 1000 + ms := make([]pmetric.Metrics, n) + testIDKey := "test_id" + for i := 0; i < n; i++ { + m := testdata.GenerateMetricsOneMetric() + dps := m.ResourceMetrics().At(0).ScopeMetrics().At(0).Metrics().At(0).Sum().DataPoints() + for j := 0; j < dps.Len(); j++ { + dp := dps.At(j) + dp.Attributes().PutInt(testIDKey, int64(i)) + } + ms[i] = m + } + received := make(map[int]prompb.TimeSeries) + var mu sync.Mutex + + server := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + body, err := io.ReadAll(r.Body) + if err != nil { + t.Fatal(err) + } + assert.NotNil(t, body) + // Receives the http requests and unzip, unmarshalls, and extracts TimeSeries + assert.Equal(t, "0.1.0", r.Header.Get("X-Prometheus-Remote-Write-Version")) + assert.Equal(t, "snappy", r.Header.Get("Content-Encoding")) + var unzipped []byte + + dest, err := snappy.Decode(unzipped, body) + assert.NoError(t, err) + + wr := &prompb.WriteRequest{} + ok := proto.Unmarshal(dest, wr) + assert.NoError(t, ok) + assert.Len(t, wr.Timeseries, 2) + ts := wr.Timeseries[0] + foundLabel := false + for _, label := range ts.Labels { + if label.Name == testIDKey { + id, err := strconv.Atoi(label.Value) + assert.NoError(t, err) + mu.Lock() + _, ok := received[id] + assert.False(t, ok) // fail if we already saw it + received[id] = ts + mu.Unlock() + foundLabel = true + break + } + } + assert.True(t, foundLabel) + w.WriteHeader(http.StatusOK) + })) + + defer server.Close() + + // Adjusted retry settings for faster testing + retrySettings := configretry.BackOffConfig{ + Enabled: true, + InitialInterval: 100 * time.Millisecond, // Shorter initial interval + MaxInterval: 1 * time.Second, // Shorter max interval + MaxElapsedTime: 2 * time.Second, // Shorter max elapsed time + } + clientConfig := confighttp.NewDefaultClientConfig() + clientConfig.Endpoint = server.URL + clientConfig.ReadBufferSize = 0 + clientConfig.WriteBufferSize = 512 * 1024 + cfg := &Config{ + Namespace: "", + ClientConfig: clientConfig, + MaxBatchSizeBytes: 3000000, + RemoteWriteQueue: RemoteWriteQueue{NumConsumers: 1}, + TargetInfo: &TargetInfo{ + Enabled: true, + }, + CreatedMetric: &CreatedMetric{ + Enabled: false, + }, + BackOffConfig: retrySettings, + } + + assert.NotNil(t, cfg) + set := exportertest.NewNopSettings() + set.MetricsLevel = configtelemetry.LevelBasic + + prwe, nErr := newPRWExporter(cfg, set) + + require.NoError(t, nErr) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + require.NoError(t, prwe.Start(ctx, componenttest.NewNopHost())) + defer func() { + require.NoError(t, prwe.Shutdown(ctx)) + }() + + var wg sync.WaitGroup + wg.Add(n) + for _, m := range ms { + go func() { + err := prwe.PushMetrics(ctx, m) + assert.NoError(t, err) + wg.Done() + }() + } + wg.Wait() + assert.Len(t, received, n) +} diff --git a/pkg/translator/prometheusremotewrite/helper.go b/pkg/translator/prometheusremotewrite/helper.go index b8c870d62a7d..f3f333d6b641 100644 --- a/pkg/translator/prometheusremotewrite/helper.go +++ b/pkg/translator/prometheusremotewrite/helper.go @@ -11,6 +11,7 @@ import ( "slices" "sort" "strconv" + "strings" "time" "unicode/utf8" @@ -104,47 +105,41 @@ var seps = []byte{'\xff'} // createAttributes creates a slice of Prometheus Labels with OTLP attributes and pairs of string values. // Unpaired string values are ignored. String pairs overwrite OTLP labels if collisions happen and // if logOnOverwrite is true, the overwrite is logged. Resulting label names are sanitized. -func createAttributes(resource pcommon.Resource, attributes pcommon.Map, externalLabels map[string]string, +func (c *PrometheusConverter) createAttributes(resource pcommon.Resource, attributes pcommon.Map, externalLabels map[string]string, ignoreAttrs []string, logOnOverwrite bool, extras ...string, ) []prompb.Label { resourceAttrs := resource.Attributes() serviceName, haveServiceName := resourceAttrs.Get(conventions.AttributeServiceName) instance, haveInstanceID := resourceAttrs.Get(conventions.AttributeServiceInstanceID) - // Calculate the maximum possible number of labels we could return so we can preallocate l - maxLabelCount := attributes.Len() + len(externalLabels) + len(extras)/2 - - if haveServiceName { - maxLabelCount++ - } - - if haveInstanceID { - maxLabelCount++ - } - // map ensures no duplicate label name - l := make(map[string]string, maxLabelCount) + l := c.labelsMap + clear(l) + + // store duplicate labels separately in a throwaway map + // assuming this is the less common case + collisions := make(map[string][]string) - // Ensure attributes are sorted by key for consistent merging of keys which - // collide when sanitized. - labels := make([]prompb.Label, 0, maxLabelCount) // XXX: Should we always drop service namespace/service name/service instance ID from the labels // (as they get mapped to other Prometheus labels)? attributes.Range(func(key string, value pcommon.Value) bool { if !slices.Contains(ignoreAttrs, key) { - labels = append(labels, prompb.Label{Name: key, Value: value.AsString()}) + finalKey := prometheustranslator.NormalizeLabel(key) + if _, alreadyExists := l[finalKey]; alreadyExists { + collisions[finalKey] = append(collisions[finalKey], value.AsString()) + } else { + l[finalKey] = value.AsString() + } } return true }) - sort.Stable(ByLabelName(labels)) - for _, label := range labels { - finalKey := prometheustranslator.NormalizeLabel(label.Name) - if existingValue, alreadyExists := l[finalKey]; alreadyExists { - l[finalKey] = existingValue + ";" + label.Value - } else { - l[finalKey] = label.Value - } + for key, values := range collisions { + values = append(values, l[key]) + // Ensure attributes are sorted by key for consistent merging of keys which + // collide when sanitized. + sort.Strings(values) + l[key] = strings.Join(values, ";") } // Map service.name + service.namespace to job @@ -184,12 +179,12 @@ func createAttributes(resource pcommon.Resource, attributes pcommon.Map, externa l[name] = extras[i+1] } - labels = labels[:0] + startIndex := len(c.labels) for k, v := range l { - labels = append(labels, prompb.Label{Name: k, Value: v}) + c.labels = append(c.labels, prompb.Label{Name: k, Value: v}) } - return labels + return c.labels[startIndex:] } // isValidAggregationTemporality checks whether an OTel metric has a valid @@ -209,13 +204,13 @@ func isValidAggregationTemporality(metric pmetric.Metric) bool { return false } -func (c *prometheusConverter) addHistogramDataPoints(dataPoints pmetric.HistogramDataPointSlice, +func (c *PrometheusConverter) addHistogramDataPoints(dataPoints pmetric.HistogramDataPointSlice, resource pcommon.Resource, settings Settings, baseName string, ) { for x := 0; x < dataPoints.Len(); x++ { pt := dataPoints.At(x) timestamp := convertTimeStamp(pt.Timestamp()) - baseLabels := createAttributes(resource, pt.Attributes(), settings.ExternalLabels, nil, false) + baseLabels := c.createAttributes(resource, pt.Attributes(), settings.ExternalLabels, nil, false) // If the sum is unset, it indicates the _sum metric point should be // omitted @@ -392,13 +387,13 @@ func maxTimestamp(a, b pcommon.Timestamp) pcommon.Timestamp { return b } -func (c *prometheusConverter) addSummaryDataPoints(dataPoints pmetric.SummaryDataPointSlice, resource pcommon.Resource, +func (c *PrometheusConverter) addSummaryDataPoints(dataPoints pmetric.SummaryDataPointSlice, resource pcommon.Resource, settings Settings, baseName string, ) { for x := 0; x < dataPoints.Len(); x++ { pt := dataPoints.At(x) timestamp := convertTimeStamp(pt.Timestamp()) - baseLabels := createAttributes(resource, pt.Attributes(), settings.ExternalLabels, nil, false) + baseLabels := c.createAttributes(resource, pt.Attributes(), settings.ExternalLabels, nil, false) // treat sum as a sample in an individual TimeSeries sum := &prompb.Sample{ @@ -466,7 +461,7 @@ func createLabels(name string, baseLabels []prompb.Label, extras ...string) []pr // getOrCreateTimeSeries returns the time series corresponding to the label set if existent, and false. // Otherwise it creates a new one and returns that, and true. -func (c *prometheusConverter) getOrCreateTimeSeries(lbls []prompb.Label) (*prompb.TimeSeries, bool) { +func (c *PrometheusConverter) getOrCreateTimeSeries(lbls []prompb.Label) (*prompb.TimeSeries, bool) { h := timeSeriesSignature(lbls) ts := c.unique[h] if ts != nil { @@ -502,7 +497,7 @@ func (c *prometheusConverter) getOrCreateTimeSeries(lbls []prompb.Label) (*promp // addTimeSeriesIfNeeded adds a corresponding time series if it doesn't already exist. // If the time series doesn't already exist, it gets added with startTimestamp for its value and timestamp for its timestamp, // both converted to milliseconds. -func (c *prometheusConverter) addTimeSeriesIfNeeded(lbls []prompb.Label, startTimestamp pcommon.Timestamp, timestamp pcommon.Timestamp) { +func (c *PrometheusConverter) addTimeSeriesIfNeeded(lbls []prompb.Label, startTimestamp pcommon.Timestamp, timestamp pcommon.Timestamp) { ts, created := c.getOrCreateTimeSeries(lbls) if created { ts.Samples = []prompb.Sample{ @@ -516,7 +511,7 @@ func (c *prometheusConverter) addTimeSeriesIfNeeded(lbls []prompb.Label, startTi } // addResourceTargetInfo converts the resource to the target info metric. -func addResourceTargetInfo(resource pcommon.Resource, settings Settings, timestamp pcommon.Timestamp, converter *prometheusConverter) { +func addResourceTargetInfo(resource pcommon.Resource, settings Settings, timestamp pcommon.Timestamp, converter *PrometheusConverter) { if settings.DisableTargetInfo || timestamp == 0 { return } @@ -544,7 +539,7 @@ func addResourceTargetInfo(resource pcommon.Resource, settings Settings, timesta name = settings.Namespace + "_" + name } - labels := createAttributes(resource, attributes, settings.ExternalLabels, identifyingAttrs, false, model.MetricNameLabel, name) + labels := converter.createAttributes(resource, attributes, settings.ExternalLabels, identifyingAttrs, false, model.MetricNameLabel, name) haveIdentifier := false for _, l := range labels { if l.Name == model.JobLabel || l.Name == model.InstanceLabel { diff --git a/pkg/translator/prometheusremotewrite/helper_test.go b/pkg/translator/prometheusremotewrite/helper_test.go index 12c0dcd978be..69373a0f172d 100644 --- a/pkg/translator/prometheusremotewrite/helper_test.go +++ b/pkg/translator/prometheusremotewrite/helper_test.go @@ -116,7 +116,7 @@ func TestPrometheusConverter_addSample(t *testing.T) { } t.Run("empty_case", func(t *testing.T) { - converter := newPrometheusConverter() + converter := NewPrometheusConverter() converter.addSample(nil, nil) assert.Empty(t, converter.unique) assert.Empty(t, converter.conflicts) @@ -160,7 +160,7 @@ func TestPrometheusConverter_addSample(t *testing.T) { } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - converter := newPrometheusConverter() + converter := NewPrometheusConverter() converter.addSample(&tt.testCase[0].sample, tt.testCase[0].labels) converter.addSample(&tt.testCase[1].sample, tt.testCase[1].labels) assert.Exactly(t, tt.want, converter.unique) @@ -360,8 +360,9 @@ func Test_createLabelSet(t *testing.T) { } // run tests for _, tt := range tests { + c := NewPrometheusConverter() t.Run(tt.name, func(t *testing.T) { - assert.ElementsMatch(t, tt.want, createAttributes(tt.resource, tt.orig, tt.externalLabels, nil, true, tt.extras...)) + assert.ElementsMatch(t, tt.want, c.createAttributes(tt.resource, tt.orig, tt.externalLabels, nil, true, tt.extras...)) }) } } @@ -376,10 +377,15 @@ func BenchmarkCreateAttributes(b *testing.B) { m.PutInt("test-int-key", 123) m.PutBool("test-bool-key", true) + c := NewPrometheusConverter() + // preallocate slice to simulate a fully-grown buffer + c.labels = make([]prompb.Label, 0, b.N*m.Len()) + b.ReportAllocs() b.ResetTimer() + for i := 0; i < b.N; i++ { - createAttributes(r, m, ext, nil, true) + c.createAttributes(r, m, ext, nil, true) } } @@ -440,7 +446,7 @@ func TestPrometheusConverter_addExemplars(t *testing.T) { // run tests for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - converter := &prometheusConverter{ + converter := &PrometheusConverter{ unique: tt.orig, } converter.addExemplars(tt.dataPoint, tt.bucketBounds) @@ -621,7 +627,7 @@ func TestAddResourceTargetInfo(t *testing.T) { }, } { t.Run(tc.desc, func(t *testing.T) { - converter := newPrometheusConverter() + converter := NewPrometheusConverter() addResourceTargetInfo(tc.resource, tc.settings, tc.timestamp, converter) @@ -810,7 +816,7 @@ func TestPrometheusConverter_AddSummaryDataPoints(t *testing.T) { defer testutil.SetFeatureGateForTest(t, exportCreatedMetricGate, oldValue) metric := tt.metric() - converter := newPrometheusConverter() + converter := NewPrometheusConverter() converter.addSummaryDataPoints( metric.Summary().DataPoints(), @@ -965,7 +971,7 @@ func TestPrometheusConverter_AddHistogramDataPoints(t *testing.T) { defer testutil.SetFeatureGateForTest(t, exportCreatedMetricGate, oldValue) metric := tt.metric() - converter := newPrometheusConverter() + converter := NewPrometheusConverter() converter.addHistogramDataPoints( metric.Histogram().DataPoints(), @@ -983,7 +989,7 @@ func TestPrometheusConverter_AddHistogramDataPoints(t *testing.T) { } func TestPrometheusConverter_getOrCreateTimeSeries(t *testing.T) { - converter := newPrometheusConverter() + converter := NewPrometheusConverter() lbls := []prompb.Label{ { Name: "key1", diff --git a/pkg/translator/prometheusremotewrite/histograms.go b/pkg/translator/prometheusremotewrite/histograms.go index 4ac4d58a96d4..e9c9de785489 100644 --- a/pkg/translator/prometheusremotewrite/histograms.go +++ b/pkg/translator/prometheusremotewrite/histograms.go @@ -16,12 +16,12 @@ import ( const defaultZeroThreshold = 1e-128 -func (c *prometheusConverter) addExponentialHistogramDataPoints(dataPoints pmetric.ExponentialHistogramDataPointSlice, +func (c *PrometheusConverter) addExponentialHistogramDataPoints(dataPoints pmetric.ExponentialHistogramDataPointSlice, resource pcommon.Resource, settings Settings, baseName string, ) error { for x := 0; x < dataPoints.Len(); x++ { pt := dataPoints.At(x) - lbls := createAttributes( + lbls := c.createAttributes( resource, pt.Attributes(), settings.ExternalLabels, diff --git a/pkg/translator/prometheusremotewrite/histograms_test.go b/pkg/translator/prometheusremotewrite/histograms_test.go index d2b3cba24ae2..1433ec09c554 100644 --- a/pkg/translator/prometheusremotewrite/histograms_test.go +++ b/pkg/translator/prometheusremotewrite/histograms_test.go @@ -738,7 +738,7 @@ func TestPrometheusConverter_addExponentialHistogramDataPoints(t *testing.T) { t.Run(tt.name, func(t *testing.T) { metric := tt.metric() - converter := newPrometheusConverter() + converter := NewPrometheusConverter() require.NoError(t, converter.addExponentialHistogramDataPoints( metric.ExponentialHistogram().DataPoints(), pcommon.NewResource(), diff --git a/pkg/translator/prometheusremotewrite/metrics_to_prw.go b/pkg/translator/prometheusremotewrite/metrics_to_prw.go index 65def1ccfa58..a8859370e225 100644 --- a/pkg/translator/prometheusremotewrite/metrics_to_prw.go +++ b/pkg/translator/prometheusremotewrite/metrics_to_prw.go @@ -27,8 +27,11 @@ type Settings struct { } // FromMetrics converts pmetric.Metrics to Prometheus remote write format. -func FromMetrics(md pmetric.Metrics, settings Settings) (map[string]*prompb.TimeSeries, error) { - c := newPrometheusConverter() +// Only call this function when the result of the previous call is no longer needed +// (i.e. has been exported out of process). The return values of this function depend on +// internal state which is reset on every call. +func (c *PrometheusConverter) FromMetrics(md pmetric.Metrics, settings Settings) (map[string]*prompb.TimeSeries, error) { + c.reset() errs := c.fromMetrics(md, settings) tss := c.timeSeries() out := make(map[string]*prompb.TimeSeries, len(tss)) @@ -39,21 +42,35 @@ func FromMetrics(md pmetric.Metrics, settings Settings) (map[string]*prompb.Time return out, errs } -// prometheusConverter converts from OTel write format to Prometheus write format. -type prometheusConverter struct { +// PrometheusConverter converts from OTel write format to Prometheus write format. +// Internally it keeps a buffer of labels to avoid expensive allocations, so it is +// best to keep it around for the lifetime of the Go process. Due to this shared +// state, PrometheusConverter is NOT thread-safe and is only intended to be used by +// a single go-routine at a time. +type PrometheusConverter struct { unique map[uint64]*prompb.TimeSeries conflicts map[uint64][]*prompb.TimeSeries + labels []prompb.Label + labelsMap map[string]string } -func newPrometheusConverter() *prometheusConverter { - return &prometheusConverter{ +func NewPrometheusConverter() *PrometheusConverter { + return &PrometheusConverter{ unique: map[uint64]*prompb.TimeSeries{}, conflicts: map[uint64][]*prompb.TimeSeries{}, + labelsMap: make(map[string]string), } } +func (c *PrometheusConverter) reset() { + clear(c.labels) + c.labels = c.labels[:0] + clear(c.unique) + clear(c.conflicts) +} + // fromMetrics converts pmetric.Metrics to Prometheus remote write format. -func (c *prometheusConverter) fromMetrics(md pmetric.Metrics, settings Settings) (errs error) { +func (c *PrometheusConverter) fromMetrics(md pmetric.Metrics, settings Settings) (errs error) { resourceMetricsSlice := md.ResourceMetrics() for i := 0; i < resourceMetricsSlice.Len(); i++ { resourceMetrics := resourceMetricsSlice.At(i) @@ -132,7 +149,7 @@ func (c *prometheusConverter) fromMetrics(md pmetric.Metrics, settings Settings) } // timeSeries returns a slice of the prompb.TimeSeries that were converted from OTel format. -func (c *prometheusConverter) timeSeries() []prompb.TimeSeries { +func (c *PrometheusConverter) timeSeries() []prompb.TimeSeries { conflicts := 0 for _, ts := range c.conflicts { conflicts += len(ts) @@ -164,7 +181,7 @@ func isSameMetric(ts *prompb.TimeSeries, lbls []prompb.Label) bool { // addExemplars adds exemplars for the dataPoint. For each exemplar, if it can find a bucket bound corresponding to its value, // the exemplar is added to the bucket bound's time series, provided that the time series' has samples. -func (c *prometheusConverter) addExemplars(dataPoint pmetric.HistogramDataPoint, bucketBounds []bucketBoundsData) { +func (c *PrometheusConverter) addExemplars(dataPoint pmetric.HistogramDataPoint, bucketBounds []bucketBoundsData) { if len(bucketBounds) == 0 { return } @@ -189,7 +206,7 @@ func (c *prometheusConverter) addExemplars(dataPoint pmetric.HistogramDataPoint, // If there is no corresponding TimeSeries already, it's created. // The corresponding TimeSeries is returned. // If either lbls is nil/empty or sample is nil, nothing is done. -func (c *prometheusConverter) addSample(sample *prompb.Sample, lbls []prompb.Label) *prompb.TimeSeries { +func (c *PrometheusConverter) addSample(sample *prompb.Sample, lbls []prompb.Label) *prompb.TimeSeries { if sample == nil || len(lbls) == 0 { // This shouldn't happen return nil diff --git a/pkg/translator/prometheusremotewrite/metrics_to_prw_test.go b/pkg/translator/prometheusremotewrite/metrics_to_prw_test.go index e79bd24f49d4..420e6afa70f0 100644 --- a/pkg/translator/prometheusremotewrite/metrics_to_prw_test.go +++ b/pkg/translator/prometheusremotewrite/metrics_to_prw_test.go @@ -36,8 +36,9 @@ func BenchmarkFromMetrics(b *testing.B) { b.Run(fmt.Sprintf("exemplars per series: %v", exemplarsPerSeries), func(b *testing.B) { payload := createExportRequest(resourceAttributeCount, histogramCount, nonHistogramCount, labelsPerMetric, exemplarsPerSeries, pcommon.Timestamp(uint64(time.Now().UnixNano()))) + c := NewPrometheusConverter() for i := 0; i < b.N; i++ { - tsMap, err := FromMetrics(payload.Metrics(), Settings{}) + tsMap, err := c.FromMetrics(payload.Metrics(), Settings{}) require.NoError(b, err) require.NotNil(b, tsMap) } @@ -73,10 +74,11 @@ func BenchmarkPrometheusConverter_FromMetrics(b *testing.B) { b.Run(fmt.Sprintf("exemplars per series: %v", exemplarsPerSeries), func(b *testing.B) { payload := createExportRequest(resourceAttributeCount, histogramCount, nonHistogramCount, labelsPerMetric, exemplarsPerSeries, pcommon.Timestamp(uint64(time.Now().UnixNano()))) + c := NewPrometheusConverter() for i := 0; i < b.N; i++ { - converter := newPrometheusConverter() - require.NoError(b, converter.fromMetrics(payload.Metrics(), Settings{})) - require.NotNil(b, converter.timeSeries()) + require.NoError(b, c.fromMetrics(payload.Metrics(), Settings{})) + require.NotNil(b, c.timeSeries()) + c.reset() } }) } diff --git a/pkg/translator/prometheusremotewrite/number_data_points.go b/pkg/translator/prometheusremotewrite/number_data_points.go index 51d3b9c9017a..65ea17217702 100644 --- a/pkg/translator/prometheusremotewrite/number_data_points.go +++ b/pkg/translator/prometheusremotewrite/number_data_points.go @@ -13,12 +13,12 @@ import ( "go.opentelemetry.io/collector/pdata/pmetric" ) -func (c *prometheusConverter) addGaugeNumberDataPoints(dataPoints pmetric.NumberDataPointSlice, +func (c *PrometheusConverter) addGaugeNumberDataPoints(dataPoints pmetric.NumberDataPointSlice, resource pcommon.Resource, settings Settings, name string, ) { for x := 0; x < dataPoints.Len(); x++ { pt := dataPoints.At(x) - labels := createAttributes( + labels := c.createAttributes( resource, pt.Attributes(), settings.ExternalLabels, @@ -44,12 +44,12 @@ func (c *prometheusConverter) addGaugeNumberDataPoints(dataPoints pmetric.Number } } -func (c *prometheusConverter) addSumNumberDataPoints(dataPoints pmetric.NumberDataPointSlice, +func (c *PrometheusConverter) addSumNumberDataPoints(dataPoints pmetric.NumberDataPointSlice, resource pcommon.Resource, metric pmetric.Metric, settings Settings, name string, ) { for x := 0; x < dataPoints.Len(); x++ { pt := dataPoints.At(x) - lbls := createAttributes( + lbls := c.createAttributes( resource, pt.Attributes(), settings.ExternalLabels, diff --git a/pkg/translator/prometheusremotewrite/number_data_points_test.go b/pkg/translator/prometheusremotewrite/number_data_points_test.go index 473fec8bd6e6..9d2c14c01ab7 100644 --- a/pkg/translator/prometheusremotewrite/number_data_points_test.go +++ b/pkg/translator/prometheusremotewrite/number_data_points_test.go @@ -51,7 +51,7 @@ func TestPrometheusConverter_addGaugeNumberDataPoints(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { metric := tt.metric() - converter := newPrometheusConverter() + converter := NewPrometheusConverter() converter.addGaugeNumberDataPoints( metric.Gauge().DataPoints(), @@ -226,7 +226,7 @@ func TestPrometheusConverter_addSumNumberDataPoints(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { metric := tt.metric() - converter := newPrometheusConverter() + converter := NewPrometheusConverter() converter.addSumNumberDataPoints( metric.Sum().DataPoints(), diff --git a/pkg/translator/prometheusremotewrite/number_data_points_v2.go b/pkg/translator/prometheusremotewrite/number_data_points_v2.go index 3509f7a95b97..566e70e33d48 100644 --- a/pkg/translator/prometheusremotewrite/number_data_points_v2.go +++ b/pkg/translator/prometheusremotewrite/number_data_points_v2.go @@ -19,7 +19,7 @@ func (c *prometheusConverterV2) addGaugeNumberDataPoints(dataPoints pmetric.Numb for x := 0; x < dataPoints.Len(); x++ { pt := dataPoints.At(x) - labels := createAttributes( + labels := NewPrometheusConverter().createAttributes( resource, pt.Attributes(), settings.ExternalLabels,