diff --git a/.chloggen/create_create-time-series-prw.yaml b/.chloggen/create_create-time-series-prw.yaml new file mode 100644 index 000000000000..f59b6a8cbcaa --- /dev/null +++ b/.chloggen/create_create-time-series-prw.yaml @@ -0,0 +1,17 @@ +# One of 'breaking', 'deprecation', 'new_component', 'enhancement', 'bug_fix' +change_type: bug_fix + +# 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: Export `_created` metric for Summary, Histogram and Monotonic Sum metric points if `StartTimeUnixNano` is set. + +# One or more tracking issues related to the change +issues: [17412, 12426] + +# (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: + The export of tbe `_created` metric is configurable in the Prometheus remote write exporter. It is disabled by default. diff --git a/exporter/prometheusremotewriteexporter/README.md b/exporter/prometheusremotewriteexporter/README.md index c6b06581efe9..75645382bf88 100644 --- a/exporter/prometheusremotewriteexporter/README.md +++ b/exporter/prometheusremotewriteexporter/README.md @@ -50,6 +50,10 @@ The following settings can be optionally configured: - `enabled` (default = false): If `enabled` is `true`, all the resource attributes will be converted to metric labels by default. - `target_info`: customize `target_info` metric - `enabled` (default = true): If `enabled` is `true`, a `target_info` metric will be generated for each resource metric (see https://github.com/open-telemetry/opentelemetry-specification/pull/2381). +- `export_created_metric`: + - `enabled` (default = false): If `enabled` is `true`, a `_created` metric is + exported for Summary, Histogram, and Monotonic Sum metric points if + `StartTimeUnixNano` is set. Example: diff --git a/exporter/prometheusremotewriteexporter/config.go b/exporter/prometheusremotewriteexporter/config.go index 35a7235cd626..c27db8c35255 100644 --- a/exporter/prometheusremotewriteexporter/config.go +++ b/exporter/prometheusremotewriteexporter/config.go @@ -50,6 +50,14 @@ type Config struct { // TargetInfo allows customizing the target_info metric TargetInfo *TargetInfo `mapstructure:"target_info,omitempty"` + + // CreatedMetric allows customizing creation of _created metrics + CreatedMetric *CreatedMetric `mapstructure:"export_created_metric,omitempty"` +} + +type CreatedMetric struct { + // Enabled if true the _created metrics could be exported + Enabled bool `mapstructure:"enabled"` } type TargetInfo struct { @@ -95,5 +103,10 @@ func (cfg *Config) Validate() error { Enabled: true, } } + if cfg.CreatedMetric == nil { + cfg.CreatedMetric = &CreatedMetric{ + Enabled: false, + } + } return nil } diff --git a/exporter/prometheusremotewriteexporter/config_test.go b/exporter/prometheusremotewriteexporter/config_test.go index 73440c9887b6..766ae5c9c68a 100644 --- a/exporter/prometheusremotewriteexporter/config_test.go +++ b/exporter/prometheusremotewriteexporter/config_test.go @@ -82,6 +82,7 @@ func TestLoadConfig(t *testing.T) { TargetInfo: &TargetInfo{ Enabled: true, }, + CreatedMetric: &CreatedMetric{Enabled: true}, }, }, { diff --git a/exporter/prometheusremotewriteexporter/exporter.go b/exporter/prometheusremotewriteexporter/exporter.go index e806d460f379..90be86f614c8 100644 --- a/exporter/prometheusremotewriteexporter/exporter.go +++ b/exporter/prometheusremotewriteexporter/exporter.go @@ -44,19 +44,17 @@ const maxBatchByteSize = 3000000 // prwExporter converts OTLP metrics to Prometheus remote write TimeSeries and sends them to a remote endpoint. type prwExporter struct { - namespace string - externalLabels map[string]string - endpointURL *url.URL - client *http.Client - wg *sync.WaitGroup - closeChan chan struct{} - concurrency int - userAgentHeader string - clientSettings *confighttp.HTTPClientSettings - settings component.TelemetrySettings - disableTargetInfo bool - - wal *prweWAL + endpointURL *url.URL + client *http.Client + wg *sync.WaitGroup + closeChan chan struct{} + concurrency int + userAgentHeader string + clientSettings *confighttp.HTTPClientSettings + settings component.TelemetrySettings + + wal *prweWAL + exporterSettings prometheusremotewrite.Settings } // newPRWExporter initializes a new prwExporter instance and sets fields accordingly. @@ -74,16 +72,19 @@ func newPRWExporter(cfg *Config, set exporter.CreateSettings) (*prwExporter, err userAgentHeader := fmt.Sprintf("%s/%s", strings.ReplaceAll(strings.ToLower(set.BuildInfo.Description), " ", "-"), set.BuildInfo.Version) prwe := &prwExporter{ - namespace: cfg.Namespace, - externalLabels: sanitizedLabels, - endpointURL: endpointURL, - wg: new(sync.WaitGroup), - closeChan: make(chan struct{}), - userAgentHeader: userAgentHeader, - concurrency: cfg.RemoteWriteQueue.NumConsumers, - clientSettings: &cfg.HTTPClientSettings, - settings: set.TelemetrySettings, - disableTargetInfo: !cfg.TargetInfo.Enabled, + endpointURL: endpointURL, + wg: new(sync.WaitGroup), + closeChan: make(chan struct{}), + userAgentHeader: userAgentHeader, + concurrency: cfg.RemoteWriteQueue.NumConsumers, + clientSettings: &cfg.HTTPClientSettings, + settings: set.TelemetrySettings, + exporterSettings: prometheusremotewrite.Settings{ + Namespace: cfg.Namespace, + ExternalLabels: sanitizedLabels, + DisableTargetInfo: !cfg.TargetInfo.Enabled, + ExportCreatedMetric: cfg.CreatedMetric.Enabled, + }, } if cfg.WAL == nil { return prwe, nil @@ -136,7 +137,7 @@ func (prwe *prwExporter) PushMetrics(ctx context.Context, md pmetric.Metrics) er case <-prwe.closeChan: return errors.New("shutdown has been called") default: - tsMap, err := prometheusremotewrite.FromMetrics(md, prometheusremotewrite.Settings{Namespace: prwe.namespace, ExternalLabels: prwe.externalLabels, DisableTargetInfo: prwe.disableTargetInfo}) + tsMap, err := prometheusremotewrite.FromMetrics(md, prwe.exporterSettings) if err != nil { err = consumererror.NewPermanent(err) } diff --git a/exporter/prometheusremotewriteexporter/exporter_test.go b/exporter/prometheusremotewriteexporter/exporter_test.go index 12bcdd23fd1a..f23f533fa874 100644 --- a/exporter/prometheusremotewriteexporter/exporter_test.go +++ b/exporter/prometheusremotewriteexporter/exporter_test.go @@ -52,6 +52,9 @@ func Test_NewPRWExporter(t *testing.T) { TargetInfo: &TargetInfo{ Enabled: true, }, + CreatedMetric: &CreatedMetric{ + Enabled: false, + }, } buildInfo := component.BuildInfo{ Description: "OpenTelemetry Collector", @@ -124,9 +127,8 @@ func Test_NewPRWExporter(t *testing.T) { } assert.NoError(t, err) require.NotNil(t, prwe) - assert.NotNil(t, prwe.namespace) + assert.NotNil(t, prwe.exporterSettings) assert.NotNil(t, prwe.endpointURL) - assert.NotNil(t, prwe.externalLabels) assert.NotNil(t, prwe.closeChan) assert.NotNil(t, prwe.wg) assert.NotNil(t, prwe.userAgentHeader) @@ -145,6 +147,9 @@ func Test_Start(t *testing.T) { TargetInfo: &TargetInfo{ Enabled: true, }, + CreatedMetric: &CreatedMetric{ + Enabled: false, + }, } buildInfo := component.BuildInfo{ Description: "OpenTelemetry Collector", @@ -472,7 +477,7 @@ func Test_PushMetrics(t *testing.T) { name: "intSum_case", metrics: intSumBatch, reqTestFunc: checkFunc, - expectedTimeSeries: 3, + expectedTimeSeries: 5, httpResponseCode: http.StatusAccepted, }, { @@ -685,6 +690,9 @@ func Test_PushMetrics(t *testing.T) { TargetInfo: &TargetInfo{ Enabled: true, }, + CreatedMetric: &CreatedMetric{ + Enabled: true, + }, } if useWAL { @@ -867,6 +875,9 @@ func TestWALOnExporterRoundTrip(t *testing.T) { TargetInfo: &TargetInfo{ Enabled: true, }, + CreatedMetric: &CreatedMetric{ + Enabled: false, + }, } set := exportertest.NewNopCreateSettings() diff --git a/exporter/prometheusremotewriteexporter/factory.go b/exporter/prometheusremotewriteexporter/factory.go index bb753069db95..f90dc08f3d87 100644 --- a/exporter/prometheusremotewriteexporter/factory.go +++ b/exporter/prometheusremotewriteexporter/factory.go @@ -111,5 +111,8 @@ func createDefaultConfig() component.Config { TargetInfo: &TargetInfo{ Enabled: true, }, + CreatedMetric: &CreatedMetric{ + Enabled: false, + }, } } diff --git a/exporter/prometheusremotewriteexporter/testdata/config.yaml b/exporter/prometheusremotewriteexporter/testdata/config.yaml index 2d3e21b57ad4..17df70aa8cbf 100644 --- a/exporter/prometheusremotewriteexporter/testdata/config.yaml +++ b/exporter/prometheusremotewriteexporter/testdata/config.yaml @@ -19,6 +19,8 @@ prometheusremotewrite/2: key2: value2 resource_to_telemetry_conversion: enabled: true + export_created_metric: + enabled: true remote_write_queue: queue_size: 2000 num_consumers: 10 diff --git a/pkg/translator/prometheusremotewrite/helper.go b/pkg/translator/prometheusremotewrite/helper.go index 95e23c20eece..600f4f7900cc 100644 --- a/pkg/translator/prometheusremotewrite/helper.go +++ b/pkg/translator/prometheusremotewrite/helper.go @@ -37,13 +37,14 @@ import ( ) const ( - nameStr = "__name__" - sumStr = "_sum" - countStr = "_count" - bucketStr = "_bucket" - leStr = "le" - quantileStr = "quantile" - pInfStr = "+Inf" + nameStr = "__name__" + sumStr = "_sum" + countStr = "_count" + bucketStr = "_bucket" + leStr = "le" + quantileStr = "quantile" + pInfStr = "+Inf" + createdSuffix = "_created" // maxExemplarRunes is the maximum number of UTF-8 exemplar characters // according to the prometheus specification // https://github.com/OpenObservability/OpenMetrics/blob/main/specification/OpenMetrics.md#exemplars @@ -277,12 +278,31 @@ func addSingleNumberDataPoint(pt pmetric.NumberDataPoint, resource pcommon.Resou sample.Value = math.Float64frombits(value.StaleNaN) } addSample(tsMap, sample, labels, metric.Type().String()) + + // add _created time series if needed + if settings.ExportCreatedMetric && isMonotonicSum(metric) { + startTimestamp := pt.StartTimestamp() + if startTimestamp != 0 { + createdLabels := createAttributes( + resource, + pt.Attributes(), + settings.ExternalLabels, + nameStr, + name+createdSuffix, + ) + addCreatedTimeSeriesIfNeeded(tsMap, createdLabels, startTimestamp, metric.Type().String()) + } + } +} + +func isMonotonicSum(metric pmetric.Metric) bool { + return metric.Type() == pmetric.MetricTypeSum && metric.Sum().IsMonotonic() } // addSingleHistogramDataPoint converts pt to 2 + min(len(ExplicitBounds), len(BucketCount)) + 1 samples. It // ignore extra buckets if len(ExplicitBounds) > len(BucketCounts) func addSingleHistogramDataPoint(pt pmetric.HistogramDataPoint, resource pcommon.Resource, metric pmetric.Metric, settings Settings, tsMap map[string]*prompb.TimeSeries) { - time := convertTimeStamp(pt.Timestamp()) + timestamp := convertTimeStamp(pt.Timestamp()) // sum, count, and buckets of the histogram should append suffix to baseName baseName := prometheustranslator.BuildPromCompliantName(metric, settings.Namespace) @@ -292,7 +312,7 @@ func addSingleHistogramDataPoint(pt pmetric.HistogramDataPoint, resource pcommon // treat sum as a sample in an individual TimeSeries sum := &prompb.Sample{ Value: pt.Sum(), - Timestamp: time, + Timestamp: timestamp, } if pt.Flags().NoRecordedValue() { sum.Value = math.Float64frombits(value.StaleNaN) @@ -300,12 +320,13 @@ func addSingleHistogramDataPoint(pt pmetric.HistogramDataPoint, resource pcommon sumlabels := createAttributes(resource, pt.Attributes(), settings.ExternalLabels, nameStr, baseName+sumStr) addSample(tsMap, sum, sumlabels, metric.Type().String()) + } // treat count as a sample in an individual TimeSeries count := &prompb.Sample{ Value: float64(pt.Count()), - Timestamp: time, + Timestamp: timestamp, } if pt.Flags().NoRecordedValue() { count.Value = math.Float64frombits(value.StaleNaN) @@ -327,7 +348,7 @@ func addSingleHistogramDataPoint(pt pmetric.HistogramDataPoint, resource pcommon cumulativeCount += pt.BucketCounts().At(i) bucket := &prompb.Sample{ Value: float64(cumulativeCount), - Timestamp: time, + Timestamp: timestamp, } if pt.Flags().NoRecordedValue() { bucket.Value = math.Float64frombits(value.StaleNaN) @@ -340,7 +361,7 @@ func addSingleHistogramDataPoint(pt pmetric.HistogramDataPoint, resource pcommon } // add le=+Inf bucket infBucket := &prompb.Sample{ - Timestamp: time, + Timestamp: timestamp, } if pt.Flags().NoRecordedValue() { infBucket.Value = math.Float64frombits(value.StaleNaN) @@ -352,6 +373,19 @@ func addSingleHistogramDataPoint(pt pmetric.HistogramDataPoint, resource pcommon bucketBounds = append(bucketBounds, bucketBoundsData{sig: sig, bound: math.Inf(1)}) addExemplars(tsMap, promExemplars, bucketBounds) + + // add _created time series if needed + startTimestamp := pt.StartTimestamp() + if settings.ExportCreatedMetric && startTimestamp != 0 { + createdLabels := createAttributes( + resource, + pt.Attributes(), + settings.ExternalLabels, + nameStr, + baseName+createdSuffix, + ) + addCreatedTimeSeriesIfNeeded(tsMap, createdLabels, startTimestamp, metric.Type().String()) + } } type exemplarType interface { @@ -458,13 +492,13 @@ func maxTimestamp(a, b pcommon.Timestamp) pcommon.Timestamp { // addSingleSummaryDataPoint converts pt to len(QuantileValues) + 2 samples. func addSingleSummaryDataPoint(pt pmetric.SummaryDataPoint, resource pcommon.Resource, metric pmetric.Metric, settings Settings, tsMap map[string]*prompb.TimeSeries) { - time := convertTimeStamp(pt.Timestamp()) + timestamp := convertTimeStamp(pt.Timestamp()) // sum and count of the summary should append suffix to baseName baseName := prometheustranslator.BuildPromCompliantName(metric, settings.Namespace) // treat sum as a sample in an individual TimeSeries sum := &prompb.Sample{ Value: pt.Sum(), - Timestamp: time, + Timestamp: timestamp, } if pt.Flags().NoRecordedValue() { sum.Value = math.Float64frombits(value.StaleNaN) @@ -475,7 +509,7 @@ func addSingleSummaryDataPoint(pt pmetric.SummaryDataPoint, resource pcommon.Res // treat count as a sample in an individual TimeSeries count := &prompb.Sample{ Value: float64(pt.Count()), - Timestamp: time, + Timestamp: timestamp, } if pt.Flags().NoRecordedValue() { count.Value = math.Float64frombits(value.StaleNaN) @@ -488,7 +522,7 @@ func addSingleSummaryDataPoint(pt pmetric.SummaryDataPoint, resource pcommon.Res qt := pt.QuantileValues().At(i) quantile := &prompb.Sample{ Value: qt.Value(), - Timestamp: time, + Timestamp: timestamp, } if pt.Flags().NoRecordedValue() { quantile.Value = math.Float64frombits(value.StaleNaN) @@ -497,6 +531,40 @@ func addSingleSummaryDataPoint(pt pmetric.SummaryDataPoint, resource pcommon.Res qtlabels := createAttributes(resource, pt.Attributes(), settings.ExternalLabels, nameStr, baseName, quantileStr, percentileStr) addSample(tsMap, quantile, qtlabels, metric.Type().String()) } + + // add _created time series if needed + startTimestamp := pt.StartTimestamp() + if settings.ExportCreatedMetric && startTimestamp != 0 { + createdLabels := createAttributes( + resource, + pt.Attributes(), + settings.ExternalLabels, + nameStr, + baseName+createdSuffix, + ) + addCreatedTimeSeriesIfNeeded(tsMap, createdLabels, startTimestamp, metric.Type().String()) + } +} + +// addCreatedTimeSeriesIfNeeded adds {name}_created time series with a single +// sample. If the series exists, then new samples won't be added. +func addCreatedTimeSeriesIfNeeded( + series map[string]*prompb.TimeSeries, + labels []prompb.Label, + startTimestamp pcommon.Timestamp, + metricType string, +) { + sig := timeSeriesSignature(metricType, &labels) + if _, ok := series[sig]; !ok { + series[sig] = &prompb.TimeSeries{ + Labels: labels, + Samples: []prompb.Sample{ + { // convert ns to ms + Value: float64(convertTimeStamp(startTimestamp)), + }, + }, + } + } } // addResourceTargetInfo converts the resource to the target info metric diff --git a/pkg/translator/prometheusremotewrite/helper_test.go b/pkg/translator/prometheusremotewrite/helper_test.go index 199f00d3717b..62352d529dfc 100644 --- a/pkg/translator/prometheusremotewrite/helper_test.go +++ b/pkg/translator/prometheusremotewrite/helper_test.go @@ -19,6 +19,7 @@ import ( "testing" "time" + "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/timestamp" "github.com/prometheus/prometheus/prompb" "github.com/stretchr/testify/assert" @@ -651,3 +652,345 @@ func TestMostRecentTimestampInMetric(t *testing.T) { }) } } + +func TestAddSingleNumberDataPoint(t *testing.T) { + ts := pcommon.Timestamp(time.Now().UnixNano()) + tests := []struct { + name string + metric func() pmetric.Metric + want func() map[string]*prompb.TimeSeries + }{ + { + name: "monotonic cumulative sum with start timestamp", + metric: func() pmetric.Metric { + metric := pmetric.NewMetric() + metric.SetName("test_sum") + metric.SetEmptySum().SetAggregationTemporality(pmetric.AggregationTemporalityCumulative) + metric.SetEmptySum().SetIsMonotonic(true) + + dp := metric.Sum().DataPoints().AppendEmpty() + dp.SetDoubleValue(1) + dp.SetTimestamp(ts) + dp.SetStartTimestamp(ts) + + return metric + }, + want: func() map[string]*prompb.TimeSeries { + labels := []prompb.Label{ + {Name: model.MetricNameLabel, Value: "test_sum"}, + } + createdLabels := []prompb.Label{ + {Name: model.MetricNameLabel, Value: "test_sum" + createdSuffix}, + } + return map[string]*prompb.TimeSeries{ + timeSeriesSignature(pmetric.MetricTypeSum.String(), &labels): { + Labels: labels, + Samples: []prompb.Sample{ + {Value: 1, Timestamp: convertTimeStamp(ts)}, + }, + }, + timeSeriesSignature(pmetric.MetricTypeSum.String(), &createdLabels): { + Labels: createdLabels, + Samples: []prompb.Sample{ + {Value: float64(convertTimeStamp(ts))}, + }, + }, + } + }, + }, + { + name: "monotonic cumulative sum with no start time", + metric: func() pmetric.Metric { + metric := pmetric.NewMetric() + metric.SetName("test_sum") + metric.SetEmptySum().SetAggregationTemporality(pmetric.AggregationTemporalityCumulative) + metric.SetEmptySum().SetIsMonotonic(true) + + dp := metric.Sum().DataPoints().AppendEmpty() + dp.SetTimestamp(ts) + + return metric + }, + want: func() map[string]*prompb.TimeSeries { + labels := []prompb.Label{ + {Name: model.MetricNameLabel, Value: "test_sum"}, + } + return map[string]*prompb.TimeSeries{ + timeSeriesSignature(pmetric.MetricTypeSum.String(), &labels): { + Labels: labels, + Samples: []prompb.Sample{ + {Value: 0, Timestamp: convertTimeStamp(ts)}, + }, + }, + } + }, + }, + { + name: "non-monotonic cumulative sum with start time", + metric: func() pmetric.Metric { + metric := pmetric.NewMetric() + metric.SetName("test_sum") + metric.SetEmptySum().SetAggregationTemporality(pmetric.AggregationTemporalityCumulative) + metric.SetEmptySum().SetIsMonotonic(false) + + dp := metric.Sum().DataPoints().AppendEmpty() + dp.SetTimestamp(ts) + + return metric + }, + want: func() map[string]*prompb.TimeSeries { + labels := []prompb.Label{ + {Name: model.MetricNameLabel, Value: "test_sum"}, + } + return map[string]*prompb.TimeSeries{ + timeSeriesSignature(pmetric.MetricTypeSum.String(), &labels): { + Labels: labels, + Samples: []prompb.Sample{ + {Value: 0, Timestamp: convertTimeStamp(ts)}, + }, + }, + } + }, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + metric := tt.metric() + + got := make(map[string]*prompb.TimeSeries) + for x := 0; x < metric.Sum().DataPoints().Len(); x++ { + addSingleNumberDataPoint( + metric.Sum().DataPoints().At(x), + pcommon.NewResource(), + metric, + Settings{ + ExportCreatedMetric: true, + }, + got, + ) + } + + assert.Equal(t, tt.want(), got) + }) + } +} + +func TestAddSingleSummaryDataPoint(t *testing.T) { + ts := pcommon.Timestamp(time.Now().UnixNano()) + tests := []struct { + name string + metric func() pmetric.Metric + want func() map[string]*prompb.TimeSeries + }{ + { + name: "summary with start time", + metric: func() pmetric.Metric { + metric := pmetric.NewMetric() + metric.SetName("test_summary") + metric.SetEmptySummary() + + dp := metric.Summary().DataPoints().AppendEmpty() + dp.SetTimestamp(ts) + dp.SetStartTimestamp(ts) + + return metric + }, + want: func() map[string]*prompb.TimeSeries { + labels := []prompb.Label{ + {Name: model.MetricNameLabel, Value: "test_summary" + countStr}, + } + createdLabels := []prompb.Label{ + {Name: model.MetricNameLabel, Value: "test_summary" + createdSuffix}, + } + sumLabels := []prompb.Label{ + {Name: model.MetricNameLabel, Value: "test_summary" + sumStr}, + } + return map[string]*prompb.TimeSeries{ + timeSeriesSignature(pmetric.MetricTypeSummary.String(), &labels): { + Labels: labels, + Samples: []prompb.Sample{ + {Value: 0, Timestamp: convertTimeStamp(ts)}, + }, + }, + timeSeriesSignature(pmetric.MetricTypeSummary.String(), &sumLabels): { + Labels: sumLabels, + Samples: []prompb.Sample{ + {Value: 0, Timestamp: convertTimeStamp(ts)}, + }, + }, + timeSeriesSignature(pmetric.MetricTypeSummary.String(), &createdLabels): { + Labels: createdLabels, + Samples: []prompb.Sample{ + {Value: float64(convertTimeStamp(ts))}, + }, + }, + } + }, + }, + { + name: "summary without start time", + metric: func() pmetric.Metric { + metric := pmetric.NewMetric() + metric.SetName("test_summary") + metric.SetEmptySummary() + + dp := metric.Summary().DataPoints().AppendEmpty() + dp.SetTimestamp(ts) + + return metric + }, + want: func() map[string]*prompb.TimeSeries { + labels := []prompb.Label{ + {Name: model.MetricNameLabel, Value: "test_summary" + countStr}, + } + sumLabels := []prompb.Label{ + {Name: model.MetricNameLabel, Value: "test_summary" + sumStr}, + } + return map[string]*prompb.TimeSeries{ + timeSeriesSignature(pmetric.MetricTypeSummary.String(), &labels): { + Labels: labels, + Samples: []prompb.Sample{ + {Value: 0, Timestamp: convertTimeStamp(ts)}, + }, + }, + timeSeriesSignature(pmetric.MetricTypeSummary.String(), &sumLabels): { + Labels: sumLabels, + Samples: []prompb.Sample{ + {Value: 0, Timestamp: convertTimeStamp(ts)}, + }, + }, + } + }, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + metric := tt.metric() + + got := make(map[string]*prompb.TimeSeries) + for x := 0; x < metric.Summary().DataPoints().Len(); x++ { + addSingleSummaryDataPoint( + metric.Summary().DataPoints().At(x), + pcommon.NewResource(), + metric, + Settings{ + ExportCreatedMetric: true, + }, + got, + ) + } + assert.Equal(t, tt.want(), got) + }) + } +} + +func TestAddSingleHistogramDataPoint(t *testing.T) { + ts := pcommon.Timestamp(time.Now().UnixNano()) + tests := []struct { + name string + metric func() pmetric.Metric + want func() map[string]*prompb.TimeSeries + }{ + { + name: "histogram with start time", + metric: func() pmetric.Metric { + metric := pmetric.NewMetric() + metric.SetName("test_hist") + metric.SetEmptyHistogram().SetAggregationTemporality(pmetric.AggregationTemporalityCumulative) + + pt := metric.Histogram().DataPoints().AppendEmpty() + pt.SetTimestamp(ts) + pt.SetStartTimestamp(ts) + + return metric + }, + want: func() map[string]*prompb.TimeSeries { + labels := []prompb.Label{ + {Name: model.MetricNameLabel, Value: "test_hist" + countStr}, + } + createdLabels := []prompb.Label{ + {Name: model.MetricNameLabel, Value: "test_hist" + createdSuffix}, + } + infLabels := []prompb.Label{ + {Name: model.MetricNameLabel, Value: "test_hist_bucket"}, + {Name: model.BucketLabel, Value: "+Inf"}, + } + return map[string]*prompb.TimeSeries{ + timeSeriesSignature(pmetric.MetricTypeHistogram.String(), &infLabels): { + Labels: infLabels, + Samples: []prompb.Sample{ + {Value: 0, Timestamp: convertTimeStamp(ts)}, + }, + }, + timeSeriesSignature(pmetric.MetricTypeHistogram.String(), &labels): { + Labels: labels, + Samples: []prompb.Sample{ + {Value: 0, Timestamp: convertTimeStamp(ts)}, + }, + }, + timeSeriesSignature(pmetric.MetricTypeHistogram.String(), &createdLabels): { + Labels: createdLabels, + Samples: []prompb.Sample{ + {Value: float64(convertTimeStamp(ts))}, + }, + }, + } + }, + }, + { + name: "histogram without start time", + metric: func() pmetric.Metric { + metric := pmetric.NewMetric() + metric.SetName("test_hist") + metric.SetEmptyHistogram().SetAggregationTemporality(pmetric.AggregationTemporalityCumulative) + + pt := metric.Histogram().DataPoints().AppendEmpty() + pt.SetTimestamp(ts) + + return metric + }, + want: func() map[string]*prompb.TimeSeries { + labels := []prompb.Label{ + {Name: model.MetricNameLabel, Value: "test_hist" + countStr}, + } + infLabels := []prompb.Label{ + {Name: model.MetricNameLabel, Value: "test_hist_bucket"}, + {Name: model.BucketLabel, Value: "+Inf"}, + } + return map[string]*prompb.TimeSeries{ + timeSeriesSignature(pmetric.MetricTypeHistogram.String(), &infLabels): { + Labels: infLabels, + Samples: []prompb.Sample{ + {Value: 0, Timestamp: convertTimeStamp(ts)}, + }, + }, + timeSeriesSignature(pmetric.MetricTypeHistogram.String(), &labels): { + Labels: labels, + Samples: []prompb.Sample{ + {Value: 0, Timestamp: convertTimeStamp(ts)}, + }, + }, + } + }, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + metric := tt.metric() + + got := make(map[string]*prompb.TimeSeries) + for x := 0; x < metric.Histogram().DataPoints().Len(); x++ { + addSingleHistogramDataPoint( + metric.Histogram().DataPoints().At(x), + pcommon.NewResource(), + metric, + Settings{ + ExportCreatedMetric: true, + }, + got, + ) + } + assert.Equal(t, tt.want(), got) + }) + } +} diff --git a/pkg/translator/prometheusremotewrite/metrics_to_prw.go b/pkg/translator/prometheusremotewrite/metrics_to_prw.go index 052a62e8d0d9..912f3dfeccfa 100644 --- a/pkg/translator/prometheusremotewrite/metrics_to_prw.go +++ b/pkg/translator/prometheusremotewrite/metrics_to_prw.go @@ -27,9 +27,10 @@ import ( ) type Settings struct { - Namespace string - ExternalLabels map[string]string - DisableTargetInfo bool + Namespace string + ExternalLabels map[string]string + DisableTargetInfo bool + ExportCreatedMetric bool } // FromMetrics converts pmetric.Metrics to prometheus remote write format.