From b880665a4dd2d0194558036d5ddc13098c7a0b90 Mon Sep 17 00:00:00 2001 From: huyan0 Date: Sun, 30 Aug 2020 22:15:11 -0500 Subject: [PATCH 1/2] refactor to convert from OTLP v0.5.0 --- .../prometheusremotewriteexporter/exporter.go | 141 ++-- .../exporter_test.go | 606 +++++++++--------- .../prometheusremotewriteexporter/helper.go | 227 ++++++- .../helper_test.go | 108 ++-- .../testutil_test.go | 478 ++++++++++---- 5 files changed, 951 insertions(+), 609 deletions(-) diff --git a/exporter/prometheusremotewriteexporter/exporter.go b/exporter/prometheusremotewriteexporter/exporter.go index 72858c16874..86d3b786f62 100644 --- a/exporter/prometheusremotewriteexporter/exporter.go +++ b/exporter/prometheusremotewriteexporter/exporter.go @@ -24,7 +24,6 @@ import ( "io" "net/http" "net/url" - "strconv" "sync" "github.com/gogo/protobuf/proto" @@ -34,8 +33,8 @@ import ( "go.opentelemetry.io/collector/component/componenterror" "go.opentelemetry.io/collector/consumer/pdata" "go.opentelemetry.io/collector/consumer/pdatautil" - otlp "go.opentelemetry.io/collector/internal/data/opentelemetry-proto-gen/metrics/v1old" - "go.opentelemetry.io/collector/internal/dataold" + "go.opentelemetry.io/collector/internal/data" + otlp "go.opentelemetry.io/collector/internal/data/opentelemetry-proto-gen/metrics/v1" ) // PrwExporter converts OTLP metrics to Prometheus remote write TimeSeries and sends them to a remote endpoint @@ -91,7 +90,7 @@ func (prwe *PrwExporter) PushMetrics(ctx context.Context, md pdata.Metrics) (int dropped := 0 errs := []error{} - resourceMetrics := dataold.MetricDataToOtlp(pdatautil.MetricsToOldInternalMetrics(md)) + resourceMetrics := data.MetricDataToOtlp(pdatautil.MetricsToInternalMetrics(md)) for _, resourceMetric := range resourceMetrics { if resourceMetric == nil { continue @@ -104,23 +103,23 @@ func (prwe *PrwExporter) PushMetrics(ctx context.Context, md pdata.Metrics) (int // TODO: decide if instrumentation library information should be exported as labels for _, metric := range instrumentationMetrics.Metrics { if metric == nil { + dropped++ continue } - // check for valid type and temporality combination - if ok := validateMetrics(metric.MetricDescriptor); !ok { + // check for valid type and temporality combination and for matching data field and type + if ok := validateMetrics(metric); !ok { dropped++ errs = append(errs, errors.New("invalid temporality and type combination")) continue } // handle individual metric based on type - switch metric.GetMetricDescriptor().GetType() { - case otlp.MetricDescriptor_MONOTONIC_INT64, otlp.MetricDescriptor_INT64, - otlp.MetricDescriptor_MONOTONIC_DOUBLE, otlp.MetricDescriptor_DOUBLE: + switch metric.Data.(type) { + case *otlp.Metric_DoubleSum, *otlp.Metric_IntSum, *otlp.Metric_DoubleGauge, *otlp.Metric_IntGauge: if err := prwe.handleScalarMetric(tsMap, metric); err != nil { dropped++ errs = append(errs, err) } - case otlp.MetricDescriptor_HISTOGRAM: + case *otlp.Metric_DoubleHistogram, *otlp.Metric_IntHistogram: if err := prwe.handleHistogramMetric(tsMap, metric); err != nil { dropped++ errs = append(errs, err) @@ -151,49 +150,38 @@ func (prwe *PrwExporter) PushMetrics(ctx context.Context, md pdata.Metrics) (int // tsMap and metric cannot be nil, and metric must have a non-nil descriptor func (prwe *PrwExporter) handleScalarMetric(tsMap map[string]*prompb.TimeSeries, metric *otlp.Metric) error { - mType := metric.MetricDescriptor.Type - - switch mType { + switch metric.Data.(type) { // int points - case otlp.MetricDescriptor_MONOTONIC_INT64, otlp.MetricDescriptor_INT64: - if metric.Int64DataPoints == nil { - return fmt.Errorf("nil data point field in metric %s", metric.GetMetricDescriptor().Name) + case *otlp.Metric_DoubleGauge: + if metric.GetDoubleGauge().GetDataPoints() == nil { + return fmt.Errorf("nil data point. %s is dropped", metric.GetName()) } - - for _, pt := range metric.Int64DataPoints { - // create parameters for addSample - name := getPromMetricName(metric.GetMetricDescriptor(), prwe.namespace) - labels := createLabelSet(pt.GetLabels(), nameStr, name) - sample := &prompb.Sample{ - Value: float64(pt.Value), - // convert ns to ms - Timestamp: convertTimeStamp(pt.TimeUnixNano), - } - - addSample(tsMap, sample, labels, metric.GetMetricDescriptor().GetType()) + for _, pt := range metric.GetDoubleGauge().GetDataPoints() { + addSingleDoubleDataPoint(pt, metric, prwe.namespace, tsMap) } - return nil - - // double points - case otlp.MetricDescriptor_MONOTONIC_DOUBLE, otlp.MetricDescriptor_DOUBLE: - if metric.DoubleDataPoints == nil { - return fmt.Errorf("nil data point field in metric %s", metric.GetMetricDescriptor().Name) + case *otlp.Metric_IntGauge: + if metric.GetIntGauge().GetDataPoints() == nil { + return fmt.Errorf("nil data point. %s is dropped", metric.GetName()) } - for _, pt := range metric.DoubleDataPoints { - - // create parameters for addSample - name := getPromMetricName(metric.GetMetricDescriptor(), prwe.namespace) - labels := createLabelSet(pt.GetLabels(), nameStr, name) - sample := &prompb.Sample{ - Value: pt.Value, - Timestamp: convertTimeStamp(pt.TimeUnixNano), - } - - addSample(tsMap, sample, labels, metric.GetMetricDescriptor().GetType()) + for _, pt := range metric.GetIntGauge().GetDataPoints() { + addSingleIntDataPoint(pt, metric, prwe.namespace, tsMap) + } + case *otlp.Metric_DoubleSum: + if metric.GetDoubleSum().GetDataPoints() == nil { + return fmt.Errorf("nil data point. %s is dropped", metric.GetName()) + } + for _, pt := range metric.GetDoubleSum().GetDataPoints() { + addSingleDoubleDataPoint(pt, metric, prwe.namespace, tsMap) + } + case *otlp.Metric_IntSum: + if metric.GetIntSum().GetDataPoints() == nil { + return fmt.Errorf("nil data point. %s is dropped", metric.GetName()) + } + for _, pt := range metric.GetIntSum().GetDataPoints() { + addSingleIntDataPoint(pt, metric, prwe.namespace, tsMap) } - return nil } - return errors.New("invalid metric type: wants int or double data points") + return nil } // handleHistogramMetric processes data points in a single OTLP histogram metric by mapping the sum, count and each @@ -201,58 +189,21 @@ func (prwe *PrwExporter) handleScalarMetric(tsMap map[string]*prompb.TimeSeries, // tsMap and metric cannot be nil. func (prwe *PrwExporter) handleHistogramMetric(tsMap map[string]*prompb.TimeSeries, metric *otlp.Metric) error { - if metric.HistogramDataPoints == nil { - return fmt.Errorf("nil data point field in metric %s", metric.GetMetricDescriptor().Name) - } - - for _, pt := range metric.HistogramDataPoints { - if pt == nil { - continue - } - time := convertTimeStamp(pt.TimeUnixNano) - mType := metric.GetMetricDescriptor().GetType() - - // sum, count, and buckets of the histogram should append suffix to baseName - baseName := getPromMetricName(metric.GetMetricDescriptor(), prwe.namespace) - - // treat sum as a sample in an individual TimeSeries - sum := &prompb.Sample{ - Value: pt.GetSum(), - Timestamp: time, + switch metric.Data.(type) { + case *otlp.Metric_IntHistogram: + if metric.GetIntHistogram().GetDataPoints() == nil { + return fmt.Errorf("nil data point. %s is dropped", metric.GetName()) } - sumlabels := createLabelSet(pt.GetLabels(), nameStr, baseName+sumStr) - addSample(tsMap, sum, sumlabels, mType) - - // treat count as a sample in an individual TimeSeries - count := &prompb.Sample{ - Value: float64(pt.GetCount()), - Timestamp: time, + for _, pt := range metric.GetIntHistogram().GetDataPoints() { + addSingleIntHistogramDataPoint(pt, metric, prwe.namespace, tsMap) } - countlabels := createLabelSet(pt.GetLabels(), nameStr, baseName+countStr) - addSample(tsMap, count, countlabels, mType) - - // count for +Inf bound - var totalCount uint64 - - // process each bucket - for le, bk := range pt.GetBuckets() { - bucket := &prompb.Sample{ - Value: float64(bk.Count), - Timestamp: time, - } - boundStr := strconv.FormatFloat(pt.GetExplicitBounds()[le], 'f', -1, 64) - labels := createLabelSet(pt.GetLabels(), nameStr, baseName+bucketStr, leStr, boundStr) - addSample(tsMap, bucket, labels, mType) - - totalCount += bk.GetCount() + case *otlp.Metric_DoubleHistogram: + if metric.GetDoubleHistogram().GetDataPoints() == nil { + return fmt.Errorf("nil data point. %s is dropped", metric.GetName()) } - // add le=+Inf bucket - infBucket := &prompb.Sample{ - Value: float64(totalCount), - Timestamp: time, + for _, pt := range metric.GetDoubleHistogram().GetDataPoints() { + addSingleDoubleHistogramDataPoint(pt, metric, prwe.namespace, tsMap) } - infLabels := createLabelSet(pt.GetLabels(), nameStr, baseName+bucketStr, leStr, pInfStr) - addSample(tsMap, infBucket, infLabels, mType) } return nil } diff --git a/exporter/prometheusremotewriteexporter/exporter_test.go b/exporter/prometheusremotewriteexporter/exporter_test.go index 6e64be00651..a1323022bf7 100644 --- a/exporter/prometheusremotewriteexporter/exporter_test.go +++ b/exporter/prometheusremotewriteexporter/exporter_test.go @@ -20,7 +20,6 @@ import ( "net/http" "net/http/httptest" "net/url" - "strconv" "sync" "testing" @@ -35,220 +34,11 @@ import ( "go.opentelemetry.io/collector/consumer/pdata" "go.opentelemetry.io/collector/consumer/pdatautil" "go.opentelemetry.io/collector/exporter/exporterhelper" - otlp "go.opentelemetry.io/collector/internal/data/opentelemetry-proto-gen/metrics/v1old" - "go.opentelemetry.io/collector/internal/dataold" - "go.opentelemetry.io/collector/internal/dataold/testdataold" + "go.opentelemetry.io/collector/internal/data" + otlp "go.opentelemetry.io/collector/internal/data/opentelemetry-proto-gen/metrics/v1" + "go.opentelemetry.io/collector/internal/data/testdata" ) -// Test_handleScalarMetric checks whether data points within a single scalar metric can be added to a map of -// TimeSeries correctly. -// Test cases are two data point belonging to the same TimeSeries, two data point belonging different TimeSeries, -// and nil data points case. -func Test_handleScalarMetric(t *testing.T) { - sameTs := map[string]*prompb.TimeSeries{ - // string signature of the data point is the key of the map - typeMonotonicInt64 + "-__name__-same_ts_int_points_total" + lb1Sig: getTimeSeries( - getPromLabels(label11, value11, label12, value12, nameStr, "same_ts_int_points_total"), - getSample(float64(intVal1), msTime1), - getSample(float64(intVal2), msTime1)), - } - differentTs := map[string]*prompb.TimeSeries{ - typeMonotonicDouble + "-__name__-different_ts_double_points_total" + lb1Sig: getTimeSeries( - getPromLabels(label11, value11, label12, value12, nameStr, "different_ts_double_points_total"), - getSample(floatVal1, msTime1)), - typeMonotonicDouble + "-__name__-different_ts_double_points_total" + lb2Sig: getTimeSeries( - getPromLabels(label21, value21, label22, value22, nameStr, "different_ts_double_points_total"), - getSample(floatVal2, msTime2)), - } - - tests := []struct { - name string - m *otlp.Metric - returnError bool - want map[string]*prompb.TimeSeries - }{ - { - "invalid_nil_array", - &otlp.Metric{ - MetricDescriptor: getDescriptor("invalid_nil_array", monotonicInt64Comb, validCombinations), - Int64DataPoints: nil, - DoubleDataPoints: nil, - HistogramDataPoints: nil, - SummaryDataPoints: nil, - }, - true, - map[string]*prompb.TimeSeries{}, - }, - { - "invalid_type_array", - &otlp.Metric{ - MetricDescriptor: getDescriptor("invalid_type_array", histogramComb, validCombinations), - Int64DataPoints: nil, - DoubleDataPoints: nil, - HistogramDataPoints: nil, - SummaryDataPoints: nil, - }, - true, - map[string]*prompb.TimeSeries{}, - }, - { - "same_ts_int_points", - &otlp.Metric{ - MetricDescriptor: getDescriptor("same_ts_int_points", monotonicInt64Comb, validCombinations), - Int64DataPoints: []*otlp.Int64DataPoint{ - getIntDataPoint(lbs1, intVal1, time1), - getIntDataPoint(lbs1, intVal2, time1), - }, - DoubleDataPoints: nil, - HistogramDataPoints: nil, - SummaryDataPoints: nil, - }, - false, - sameTs, - }, - { - "different_ts_double_points", - &otlp.Metric{ - MetricDescriptor: getDescriptor("different_ts_double_points", monotonicDoubleComb, validCombinations), - Int64DataPoints: nil, - DoubleDataPoints: []*otlp.DoubleDataPoint{ - getDoubleDataPoint(lbs1, floatVal1, time1), - getDoubleDataPoint(lbs2, floatVal2, time2), - }, - HistogramDataPoints: nil, - SummaryDataPoints: nil, - }, - false, - differentTs, - }, - } - // run tests - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - tsMap := map[string]*prompb.TimeSeries{} - prw := &PrwExporter{} - ok := prw.handleScalarMetric(tsMap, tt.m) - if tt.returnError { - assert.Error(t, ok) - return - } - assert.Exactly(t, len(tt.want), len(tsMap)) - for k, v := range tsMap { - require.NotNil(t, tt.want[k]) - assert.ElementsMatch(t, tt.want[k].Labels, v.Labels) - assert.ElementsMatch(t, tt.want[k].Samples, v.Samples) - } - }) - } -} - -// Test_handleHistogramMetric checks whether data points(sum, count, buckets) within a single Histogram metric can be -// added to a map of TimeSeries correctly. -// Test cases are a histogram data point with two buckets and nil data points case. -func Test_handleHistogramMetric(t *testing.T) { - sum := "sum" - count := "count" - bucket1 := "bucket1" - bucket2 := "bucket2" - bucketInf := "bucketInf" - histPoint := getHistogramDataPoint( - lbs1, - time1, - floatVal2, - uint64(intVal2), []float64{floatVal1, floatVal2}, - []uint64{uint64(intVal1), uint64(intVal1)}) - - // string signature of the data point is the key of the map - sigs := map[string]string{ - sum: typeHistogram + "-" + nameStr + "-" + name1 + "_sum" + lb1Sig, - count: typeHistogram + "-" + nameStr + "-" + name1 + "_count" + lb1Sig, - bucket1: typeHistogram + "-" + nameStr + "-" + name1 + "_bucket" + "-" + "le-" + - strconv.FormatFloat(floatVal1, 'f', -1, 64) + lb1Sig, - bucket2: typeHistogram + "-" + nameStr + "-" + name1 + "_bucket" + "-" + "le-" + - strconv.FormatFloat(floatVal2, 'f', -1, 64) + lb1Sig, - bucketInf: typeHistogram + "-" + nameStr + "-" + name1 + "_bucket" + "-" + "le-" + - "+Inf" + lb1Sig, - } - labels := map[string][]prompb.Label{ - sum: append(promLbs1, getPromLabels(nameStr, name1+"_sum")...), - count: append(promLbs1, getPromLabels(nameStr, name1+"_count")...), - bucket1: append(promLbs1, getPromLabels(nameStr, name1+"_bucket", "le", - strconv.FormatFloat(floatVal1, 'f', -1, 64))...), - bucket2: append(promLbs1, getPromLabels(nameStr, name1+"_bucket", "le", - strconv.FormatFloat(floatVal2, 'f', -1, 64))...), - bucketInf: append(promLbs1, getPromLabels(nameStr, name1+"_bucket", "le", - "+Inf")...), - } - tests := []struct { - name string - m otlp.Metric - returnError bool - want map[string]*prompb.TimeSeries - }{ - { - "invalid_type_array", - otlp.Metric{ - MetricDescriptor: getDescriptor("invalid_type_array", histogramComb, validCombinations), - Int64DataPoints: nil, - DoubleDataPoints: nil, - HistogramDataPoints: nil, - SummaryDataPoints: nil, - }, - true, - map[string]*prompb.TimeSeries{}, - }, - { - "hist_nil_pt", - otlp.Metric{ - MetricDescriptor: getDescriptor("hist_nil_pt", histogramComb, validCombinations), - Int64DataPoints: nil, - DoubleDataPoints: nil, - HistogramDataPoints: []*otlp.HistogramDataPoint{nil}, - SummaryDataPoints: nil, - }, - false, - map[string]*prompb.TimeSeries{}, - }, - { - "single_histogram_point", - otlp.Metric{ - MetricDescriptor: getDescriptor(name1+"", histogramComb, validCombinations), - Int64DataPoints: nil, - DoubleDataPoints: nil, - HistogramDataPoints: []*otlp.HistogramDataPoint{histPoint}, - SummaryDataPoints: nil, - }, - false, - map[string]*prompb.TimeSeries{ - sigs[sum]: getTimeSeries(labels[sum], getSample(floatVal2, msTime1)), - sigs[count]: getTimeSeries(labels[count], getSample(float64(intVal2), msTime1)), - sigs[bucket1]: getTimeSeries(labels[bucket1], getSample(float64(intVal1), msTime1)), - sigs[bucket2]: getTimeSeries(labels[bucket2], getSample(float64(intVal1), msTime1)), - sigs[bucketInf]: getTimeSeries(labels[bucketInf], getSample(float64(intVal2), msTime1)), - }, - }, - } - - // run tests - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - tsMap := map[string]*prompb.TimeSeries{} - prw := &PrwExporter{} - ok := prw.handleHistogramMetric(tsMap, &tt.m) - if tt.returnError { - assert.Error(t, ok) - return - } - assert.Exactly(t, len(tt.want), len(tsMap)) - for k, v := range tsMap { - require.NotNil(t, tt.want[k], k) - assert.ElementsMatch(t, tt.want[k].Labels, v.Labels) - assert.ElementsMatch(t, tt.want[k].Samples, v.Samples) - } - }) - } -} - // Test_ NewPrwExporter checks that a new exporter instance with non-nil fields is initialized func Test_NewPrwExporter(t *testing.T) { config := &Config{ @@ -326,7 +116,7 @@ func Test_Shutdown(t *testing.T) { go func() { defer wg.Done() _, ok := prwe.PushMetrics(context.Background(), - pdatautil.MetricsFromOldInternalMetrics(testdataold.GenerateMetricDataEmpty())) + pdatautil.MetricsFromInternalMetrics(testdata.GenerateMetricsEmpty())) errChan <- ok }() } @@ -444,64 +234,211 @@ func runExportPipeline(t *testing.T, ts *prompb.TimeSeries, endpoint *url.URL) e // expected func Test_PushMetrics(t *testing.T) { - noTempBatch := pdatautil.MetricsFromOldInternalMetrics((testdataold.GenerateMetricDataManyMetricsSameResource(10))) - invalidTypeBatch := pdatautil.MetricsFromOldInternalMetrics((testdataold.GenerateMetricDataMetricTypeInvalid())) - nilDescBatch := pdatautil.MetricsFromOldInternalMetrics((testdataold.GenerateMetricDataNilMetricDescriptor())) - - // 10 counter metrics, 2 points in each. Two TimeSeries in total - batch := testdataold.GenerateMetricDataManyMetricsSameResource(10) - setCumulative(&batch) - scalarBatch := pdatautil.MetricsFromOldInternalMetrics((batch)) + invalidTypeBatch := pdatautil.MetricsFromInternalMetrics((testdata.GenerateMetricsMetricTypeInvalid())) - nilBatch1 := testdataold.GenerateMetricDataManyMetricsSameResource(10) - nilBatch2 := testdataold.GenerateMetricDataManyMetricsSameResource(10) - nilBatch3 := testdataold.GenerateMetricDataManyMetricsSameResource(10) - nilBatch4 := testdataold.GenerateMetricDataManyMetricsSameResource(10) - nilBatch5 := testdataold.GenerateMetricDataOneEmptyResourceMetrics() - nilBatch6 := testdataold.GenerateMetricDataOneEmptyInstrumentationLibrary() - nilBatch7 := testdataold.GenerateMetricDataOneMetric() + nilBatch1 := testdata.GenerateMetricsOneEmptyResourceMetrics() + nilBatch2 := testdata.GenerateMetricsOneEmptyInstrumentationLibrary() + nilBatch3 := testdata.GenerateMetricsOneMetric() - nilResource := dataold.MetricDataToOtlp(nilBatch5) + nilResource := data.MetricDataToOtlp(nilBatch1) nilResource[0] = nil - nilResourceBatch := pdatautil.MetricsFromOldInternalMetrics(dataold.MetricDataFromOtlp(nilResource)) + nilResourceBatch := pdatautil.MetricsFromInternalMetrics(data.MetricDataFromOtlp(nilResource)) - nilInstrumentation := dataold.MetricDataToOtlp(nilBatch6) + nilInstrumentation := data.MetricDataToOtlp(nilBatch2) nilInstrumentation[0].InstrumentationLibraryMetrics[0] = nil - nilInstrumentationBatch := pdatautil.MetricsFromOldInternalMetrics(dataold.MetricDataFromOtlp(nilInstrumentation)) + nilInstrumentationBatch := pdatautil.MetricsFromInternalMetrics(data.MetricDataFromOtlp(nilInstrumentation)) - nilMetric := dataold.MetricDataToOtlp(nilBatch7) + nilMetric := data.MetricDataToOtlp(nilBatch3) nilMetric[0].InstrumentationLibraryMetrics[0].Metrics[0] = nil - nilMetricBatch := pdatautil.MetricsFromOldInternalMetrics(dataold.MetricDataFromOtlp(nilMetric)) - - setCumulative(&nilBatch1) - setCumulative(&nilBatch2) - setCumulative(&nilBatch3) - setCumulative(&nilBatch4) - - setDataPointToNil(&nilBatch1, typeMonotonicInt64) - setType(&nilBatch2, typeMonotonicDouble) - setType(&nilBatch3, typeHistogram) - setType(&nilBatch4, typeSummary) - - nilIntDataPointsBatch := pdatautil.MetricsFromOldInternalMetrics((nilBatch1)) - nilDoubleDataPointsBatch := pdatautil.MetricsFromOldInternalMetrics((nilBatch2)) - nilHistogramDataPointsBatch := pdatautil.MetricsFromOldInternalMetrics((nilBatch3)) - - hist := dataold.MetricDataToOtlp(testdataold.GenerateMetricDataOneMetric()) - hist[0].InstrumentationLibraryMetrics[0].Metrics[0] = &otlp.Metric{ - MetricDescriptor: getDescriptor("hist_test", histogramComb, validCombinations), - HistogramDataPoints: []*otlp.HistogramDataPoint{getHistogramDataPoint( - lbs1, - time1, - floatVal1, - uint64(intVal1), - []float64{floatVal1}, - []uint64{uint64(intVal1)}, - ), + nilMetricBatch := pdatautil.MetricsFromInternalMetrics(data.MetricDataFromOtlp(nilMetric)) + + // success cases + intSumMetric := testdata.GenerateMetricsManyMetricsSameResource(10) + intSumBatch := pdatautil.MetricsFromInternalMetrics(intSumMetric) + + doubleSumMetric := []*otlp.ResourceMetrics{ + { + InstrumentationLibraryMetrics: []*otlp.InstrumentationLibraryMetrics{ + { + Metrics: []*otlp.Metric{ + validMetrics1[validDoubleSum], + validMetrics2[validDoubleSum], + }, + }, + }, + }, + } + doubleSumBatch := pdatautil.MetricsFromInternalMetrics(data.MetricDataFromOtlp(doubleSumMetric)) + + intGaugeMetric := []*otlp.ResourceMetrics{ + { + InstrumentationLibraryMetrics: []*otlp.InstrumentationLibraryMetrics{ + { + Metrics: []*otlp.Metric{ + validMetrics1[validIntGauge], + validMetrics2[validIntGauge], + }, + }, + }, + }, + } + intGaugeBatch := pdatautil.MetricsFromInternalMetrics(data.MetricDataFromOtlp(intGaugeMetric)) + + doubleGaugeMetric := []*otlp.ResourceMetrics{ + { + InstrumentationLibraryMetrics: []*otlp.InstrumentationLibraryMetrics{ + { + Metrics: []*otlp.Metric{ + validMetrics1[validDoubleGauge], + validMetrics2[validDoubleGauge], + }, + }, + }, + }, + } + doubleGaugeBatch := pdatautil.MetricsFromInternalMetrics(data.MetricDataFromOtlp(doubleGaugeMetric)) + + intHistogramMetric := []*otlp.ResourceMetrics{ + { + InstrumentationLibraryMetrics: []*otlp.InstrumentationLibraryMetrics{ + { + Metrics: []*otlp.Metric{ + validMetrics1[validIntHistogram], + validMetrics2[validIntHistogram], + }, + }, + }, + }, + } + intHistogramBatch := pdatautil.MetricsFromInternalMetrics(data.MetricDataFromOtlp(intHistogramMetric)) + + doubleHistogramMetric := []*otlp.ResourceMetrics{ + { + InstrumentationLibraryMetrics: []*otlp.InstrumentationLibraryMetrics{ + { + Metrics: []*otlp.Metric{ + validMetrics1[validDoubleHistogram], + validMetrics2[validDoubleHistogram], + }, + }, + }, + }, + } + doubleHistogramBatch := pdatautil.MetricsFromInternalMetrics(data.MetricDataFromOtlp(doubleHistogramMetric)) + + // len(BucketCount) > len(ExplicitBounds) + unmatchedBoundBucketIntHistMetric := []*otlp.ResourceMetrics{ + { + InstrumentationLibraryMetrics: []*otlp.InstrumentationLibraryMetrics{ + { + Metrics: []*otlp.Metric{ + validMetrics2[unmatchedBoundBucketIntHist], + }, + }, + }, + }, + } + unmatchedBoundBucketIntHistBatch := pdatautil.MetricsFromInternalMetrics( + data.MetricDataFromOtlp(unmatchedBoundBucketIntHistMetric)) + + unmatchedBoundBucketDoubleHistMetric := []*otlp.ResourceMetrics{ + { + InstrumentationLibraryMetrics: []*otlp.InstrumentationLibraryMetrics{ + { + Metrics: []*otlp.Metric{ + validMetrics2[unmatchedBoundBucketDoubleHist], + }, + }, + }, }, } + unmatchedBoundBucketDoubleHistBatch := pdatautil.MetricsFromInternalMetrics( + data.MetricDataFromOtlp(unmatchedBoundBucketDoubleHistMetric)) + + // fail cases + nilDataPointIntGaugeMetric := []*otlp.ResourceMetrics{ + { + InstrumentationLibraryMetrics: []*otlp.InstrumentationLibraryMetrics{ + { + Metrics: []*otlp.Metric{ + errorMetrics[nilDataPointIntGauge], + }, + }, + }, + }, + } + nilDataPointIntGaugeBatch := pdatautil.MetricsFromInternalMetrics( + data.MetricDataFromOtlp(nilDataPointIntGaugeMetric)) + + nilDataPointDoubleGaugeMetric := []*otlp.ResourceMetrics{ + { + InstrumentationLibraryMetrics: []*otlp.InstrumentationLibraryMetrics{ + { + Metrics: []*otlp.Metric{ + errorMetrics[nilDataPointDoubleGauge], + }, + }, + }, + }, + } + nilDataPointDoubleGaugeBatch := pdatautil.MetricsFromInternalMetrics( + data.MetricDataFromOtlp(nilDataPointDoubleGaugeMetric)) + + nilDataPointIntSumMetric := []*otlp.ResourceMetrics{ + { + InstrumentationLibraryMetrics: []*otlp.InstrumentationLibraryMetrics{ + { + Metrics: []*otlp.Metric{ + errorMetrics[nilDataPointIntSum], + }, + }, + }, + }, + } + nilDataPointIntSumBatch := pdatautil.MetricsFromInternalMetrics(data.MetricDataFromOtlp(nilDataPointIntSumMetric)) + + nilDataPointDoubleSumMetric := []*otlp.ResourceMetrics{ + { + InstrumentationLibraryMetrics: []*otlp.InstrumentationLibraryMetrics{ + { + Metrics: []*otlp.Metric{ + errorMetrics[nilDataPointDoubleSum], + }, + }, + }, + }, + } + nilDataPointDoubleSumBatch := pdatautil.MetricsFromInternalMetrics( + data.MetricDataFromOtlp(nilDataPointDoubleSumMetric)) + + nilDataPointIntHistogramMetric := []*otlp.ResourceMetrics{ + { + InstrumentationLibraryMetrics: []*otlp.InstrumentationLibraryMetrics{ + { + Metrics: []*otlp.Metric{ + errorMetrics[nilDataPointIntHistogram], + }, + }, + }, + }, + } + nilDataPointIntHistogramBatch := pdatautil.MetricsFromInternalMetrics( + data.MetricDataFromOtlp(nilDataPointIntHistogramMetric)) + + nilDataPointDoubleHistogramMetric := []*otlp.ResourceMetrics{ + { + InstrumentationLibraryMetrics: []*otlp.InstrumentationLibraryMetrics{ + { + Metrics: []*otlp.Metric{ + errorMetrics[nilDataPointDoubleHistogram], + }, + }, + }, + }, + } + nilDataPointDoubleHistogramBatch := pdatautil.MetricsFromInternalMetrics( + data.MetricDataFromOtlp(nilDataPointDoubleHistogramMetric)) - histBatch := pdatautil.MetricsFromOldInternalMetrics((dataold.MetricDataFromOtlp(hist))) checkFunc := func(t *testing.T, r *http.Request, expected int) { body, err := ioutil.ReadAll(r.Body) if err != nil { @@ -520,13 +457,6 @@ func Test_PushMetrics(t *testing.T) { assert.EqualValues(t, expected, len(wr.Timeseries)) } - summary := dataold.MetricDataToOtlp(testdataold.GenerateMetricDataOneMetric()) - summary[0].InstrumentationLibraryMetrics[0].Metrics[0] = &otlp.Metric{ - MetricDescriptor: getDescriptor("summary_test", summaryComb, validCombinations), - SummaryDataPoints: []*otlp.SummaryDataPoint{}, - } - summaryBatch := pdatautil.MetricsFromOldInternalMetrics(dataold.MetricDataFromOtlp(summary)) - tests := []struct { name string md *pdata.Metrics @@ -545,15 +475,6 @@ func Test_PushMetrics(t *testing.T) { pdatautil.MetricCount(invalidTypeBatch), true, }, - { - "nil_desc_case", - &nilDescBatch, - nil, - 0, - http.StatusAccepted, - pdatautil.MetricCount(nilDescBatch), - true, - }, { "nil_resourece_case", &nilResourceBatch, @@ -582,82 +503,129 @@ func Test_PushMetrics(t *testing.T) { true, }, { - "nil_int_point_case", - &nilIntDataPointsBatch, - nil, + "intSum_case", + &intSumBatch, + checkFunc, + 2, + http.StatusAccepted, 0, + false, + }, + { + "doubleSum_case", + &doubleSumBatch, + checkFunc, + 2, http.StatusAccepted, - pdatautil.MetricCount(nilIntDataPointsBatch), - true, + 0, + false, }, { - "nil_double_point_case", - &nilDoubleDataPointsBatch, - nil, + "doubleGauge_case", + &doubleGaugeBatch, + checkFunc, + 2, + http.StatusAccepted, 0, + false, + }, + { + "intGauge_case", + &intGaugeBatch, + checkFunc, + 2, http.StatusAccepted, - pdatautil.MetricCount(nilDoubleDataPointsBatch), - true, + 0, + false, }, { - "nil_histogram_point_case", - &nilHistogramDataPointsBatch, - nil, + "intHistogram_case", + &intHistogramBatch, + checkFunc, + 12, + http.StatusAccepted, 0, + false, + }, + { + "doubleHistogram_case", + &doubleHistogramBatch, + checkFunc, + 12, http.StatusAccepted, - pdatautil.MetricCount(nilHistogramDataPointsBatch), - true, + 0, + false, }, { - "nil_histogram_point_case", - &nilHistogramDataPointsBatch, - nil, + "unmatchedBoundBucketIntHist_case", + &unmatchedBoundBucketIntHistBatch, + checkFunc, + 5, + http.StatusAccepted, + 0, + false, + }, + { + "unmatchedBoundBucketDoubleHist_case", + &unmatchedBoundBucketDoubleHistBatch, + checkFunc, + 5, + http.StatusAccepted, + 0, + false, + }, + { + "nilDataPointDoubleGauge_case", + &nilDataPointDoubleGaugeBatch, + checkFunc, 0, http.StatusAccepted, - pdatautil.MetricCount(nilHistogramDataPointsBatch), + pdatautil.MetricCount(nilDataPointDoubleGaugeBatch), true, }, { - "no_temp_case", - &noTempBatch, - nil, + "nilDataPointIntGauge_case", + &nilDataPointIntGaugeBatch, + checkFunc, 0, http.StatusAccepted, - pdatautil.MetricCount(noTempBatch), + pdatautil.MetricCount(nilDataPointIntGaugeBatch), true, }, { - "http_error_case", - &noTempBatch, - nil, + "nilDataPointDoubleSum_case", + &nilDataPointDoubleSumBatch, + checkFunc, 0, - http.StatusForbidden, - pdatautil.MetricCount(noTempBatch), + http.StatusAccepted, + pdatautil.MetricCount(nilDataPointDoubleSumBatch), true, }, { - "scalar_case", - &scalarBatch, + "nilDataPointIntSum_case", + &nilDataPointIntSumBatch, checkFunc, - 2, - http.StatusAccepted, 0, - false, + http.StatusAccepted, + pdatautil.MetricCount(nilDataPointIntSumBatch), + true, }, - {"histogram_case", - &histBatch, + { + "nilDataPointDoubleHistogram_case", + &nilDataPointDoubleHistogramBatch, checkFunc, - 4, - http.StatusAccepted, 0, - false, + http.StatusAccepted, + pdatautil.MetricCount(nilDataPointDoubleHistogramBatch), + true, }, - {"summary_case", - &summaryBatch, + { + "nilDataPointIntHistogram_case", + &nilDataPointIntHistogramBatch, checkFunc, 0, http.StatusAccepted, - pdatautil.MetricCount(summaryBatch), + pdatautil.MetricCount(nilDataPointIntHistogramBatch), true, }, } diff --git a/exporter/prometheusremotewriteexporter/helper.go b/exporter/prometheusremotewriteexporter/helper.go index a08e5c91f7b..0a59f48c0b0 100644 --- a/exporter/prometheusremotewriteexporter/helper.go +++ b/exporter/prometheusremotewriteexporter/helper.go @@ -18,14 +18,16 @@ import ( "errors" "log" "sort" + "strconv" "strings" "time" "unicode" "github.com/prometheus/prometheus/prompb" + "go.opentelemetry.io/collector/consumer/pdata" common "go.opentelemetry.io/collector/internal/data/opentelemetry-proto-gen/common/v1" - otlp "go.opentelemetry.io/collector/internal/data/opentelemetry-proto-gen/metrics/v1old" + otlp "go.opentelemetry.io/collector/internal/data/opentelemetry-proto-gen/metrics/v1" ) const ( @@ -47,34 +49,43 @@ func (a ByLabelName) Len() int { return len(a) } func (a ByLabelName) Less(i, j int) bool { return a[i].Name < a[j].Name } func (a ByLabelName) Swap(i, j int) { a[i], a[j] = a[j], a[i] } -// validateMetrics returns a bool representing whether the metric has a valid type and temporality combination. -func validateMetrics(desc *otlp.MetricDescriptor) bool { - - if desc == nil { +// validateMetrics returns a bool representing whether the metric has a valid type and temporality combination and a +// matching metric type and field +func validateMetrics(metric *otlp.Metric) bool { + if metric == nil || metric.Data == nil { return false } - - switch desc.GetType() { - case otlp.MetricDescriptor_MONOTONIC_DOUBLE, otlp.MetricDescriptor_MONOTONIC_INT64, - otlp.MetricDescriptor_HISTOGRAM, otlp.MetricDescriptor_SUMMARY: - return desc.GetTemporality() == otlp.MetricDescriptor_CUMULATIVE - case otlp.MetricDescriptor_INT64, otlp.MetricDescriptor_DOUBLE: - return true + switch metric.Data.(type) { + case *otlp.Metric_DoubleGauge: + return metric.GetDoubleGauge() != nil + case *otlp.Metric_IntGauge: + return metric.GetIntGauge() != nil + case *otlp.Metric_DoubleSum: + return metric.GetDoubleSum() != nil && metric.GetDoubleSum().GetAggregationTemporality() == + otlp.AggregationTemporality_AGGREGATION_TEMPORALITY_CUMULATIVE + case *otlp.Metric_IntSum: + return metric.GetIntSum() != nil && metric.GetIntSum().GetAggregationTemporality() == + otlp.AggregationTemporality_AGGREGATION_TEMPORALITY_CUMULATIVE + case *otlp.Metric_DoubleHistogram: + return metric.GetDoubleHistogram() != nil && metric.GetDoubleHistogram().GetAggregationTemporality() == + otlp.AggregationTemporality_AGGREGATION_TEMPORALITY_CUMULATIVE + case *otlp.Metric_IntHistogram: + return metric.GetIntHistogram() != nil && metric.GetIntHistogram().GetAggregationTemporality() == + otlp.AggregationTemporality_AGGREGATION_TEMPORALITY_CUMULATIVE } - return false } // addSample finds a TimeSeries in tsMap that corresponds to the label set labels, and add sample to the TimeSeries; it // creates a new TimeSeries in the map if not found. tsMap is unmodified if either of its parameters is nil. func addSample(tsMap map[string]*prompb.TimeSeries, sample *prompb.Sample, labels []prompb.Label, - ty otlp.MetricDescriptor_Type) { + metric *otlp.Metric) { if sample == nil || labels == nil || tsMap == nil { return } - sig := timeSeriesSignature(ty, &labels) + sig := timeSeriesSignature(metric, &labels) ts, ok := tsMap[sig] if ok { @@ -92,9 +103,9 @@ func addSample(tsMap map[string]*prompb.TimeSeries, sample *prompb.Sample, label // TYPE-label1-value1- ... -labelN-valueN // the label slice should not contain duplicate label names; this method sorts the slice by label name before creating // the signature. -func timeSeriesSignature(t otlp.MetricDescriptor_Type, labels *[]prompb.Label) string { +func timeSeriesSignature(metric *otlp.Metric, labels *[]prompb.Label) string { b := strings.Builder{} - b.WriteString(t.String()) + b.WriteString(getTypeString(metric)) sort.Sort(ByLabelName(*labels)) @@ -153,14 +164,16 @@ func createLabelSet(labels []*common.StringKeyValue, extras ...string) []prompb. // getPromMetricName creates a Prometheus metric name by attaching namespace prefix, and _total suffix for Monotonic // metrics. -func getPromMetricName(desc *otlp.MetricDescriptor, ns string) string { +func getPromMetricName(metric *otlp.Metric, ns string) string { - if desc == nil { + if metric == nil { return "" } - // whether _total suffix should be applied - isCounter := desc.Type == otlp.MetricDescriptor_MONOTONIC_INT64 || - desc.Type == otlp.MetricDescriptor_MONOTONIC_DOUBLE + + // if the metric is counter, _total suffix should be applied + _, isCounter1 := metric.Data.(*otlp.Metric_DoubleSum) + _, isCounter2 := metric.Data.(*otlp.Metric_IntSum) + isCounter := isCounter1 || isCounter2 b := strings.Builder{} @@ -169,7 +182,11 @@ func getPromMetricName(desc *otlp.MetricDescriptor, ns string) string { if b.Len() > 0 { b.WriteString(delimeter) } - b.WriteString(desc.GetName()) + name := metric.GetName() + b.WriteString(name) + + // do not add the total suffix if the metric name already ends in "total" + isCounter = isCounter && name[len(name)-len(totalStr):] != totalStr // Including units makes two metrics with the same name and label set belong to two different TimeSeries if the // units are different. @@ -238,3 +255,167 @@ func sanitizeRune(r rune) rune { // Everything else turns into an underscore return '_' } + +func getTypeString(metric *otlp.Metric) string { + switch metric.Data.(type) { + case *otlp.Metric_DoubleGauge: + return strconv.Itoa(int(pdata.MetricDataTypeDoubleGauge)) + case *otlp.Metric_IntGauge: + return strconv.Itoa(int(pdata.MetricDataTypeIntGauge)) + case *otlp.Metric_DoubleSum: + return strconv.Itoa(int(pdata.MetricDataTypeDoubleSum)) + case *otlp.Metric_IntSum: + return strconv.Itoa(int(pdata.MetricDataTypeIntSum)) + case *otlp.Metric_DoubleHistogram: + return strconv.Itoa(int(pdata.MetricDataTypeDoubleHistogram)) + case *otlp.Metric_IntHistogram: + return strconv.Itoa(int(pdata.MetricDataTypeIntHistogram)) + } + return "" +} + +// addSingleDoubleDataPoint converts the metric value stored in pt to a Prometheus sample, and add the sample +// to its corresponding time series in tsMap +func addSingleDoubleDataPoint(pt *otlp.DoubleDataPoint, metric *otlp.Metric, namespace string, + tsMap map[string]*prompb.TimeSeries) { + if pt == nil { + return + } + // create parameters for addSample + name := getPromMetricName(metric, namespace) + labels := createLabelSet(pt.GetLabels(), nameStr, name) + sample := &prompb.Sample{ + Value: pt.Value, + // convert ns to ms + Timestamp: convertTimeStamp(pt.TimeUnixNano), + } + addSample(tsMap, sample, labels, metric) +} + +// addSingleIntDataPoint converts the metric value stored in pt to a Prometheus sample, and add the sample +// to its corresponding time series in tsMap +func addSingleIntDataPoint(pt *otlp.IntDataPoint, metric *otlp.Metric, namespace string, + tsMap map[string]*prompb.TimeSeries) { + if pt == nil { + return + } + // create parameters for addSample + name := getPromMetricName(metric, namespace) + labels := createLabelSet(pt.GetLabels(), nameStr, name) + sample := &prompb.Sample{ + Value: float64(pt.Value), + // convert ns to ms + Timestamp: convertTimeStamp(pt.TimeUnixNano), + } + addSample(tsMap, sample, labels, metric) +} + +// addSingleIntHistogramDataPoint converts pt to 2 + min(len(ExplicitBounds), len(BucketCount)) + 1 samples. It +// ignore extra buckets if len(ExplicitBounds) > len(BucketCounts) +func addSingleIntHistogramDataPoint(pt *otlp.IntHistogramDataPoint, metric *otlp.Metric, namespace string, + tsMap map[string]*prompb.TimeSeries) { + if pt == nil { + return + } + time := convertTimeStamp(pt.TimeUnixNano) + // sum, count, and buckets of the histogram should append suffix to baseName + baseName := getPromMetricName(metric, namespace) + // treat sum as a sample in an individual TimeSeries + sum := &prompb.Sample{ + Value: float64(pt.GetSum()), + Timestamp: time, + } + + sumlabels := createLabelSet(pt.GetLabels(), nameStr, baseName+sumStr) + addSample(tsMap, sum, sumlabels, metric) + + // treat count as a sample in an individual TimeSeries + count := &prompb.Sample{ + Value: float64(pt.GetCount()), + Timestamp: time, + } + countlabels := createLabelSet(pt.GetLabels(), nameStr, baseName+countStr) + addSample(tsMap, count, countlabels, metric) + + // count for +Inf bound + var totalCount uint64 + + // process each bound, ignore extra bucket values + for index, bound := range pt.GetExplicitBounds() { + if index >= len(pt.GetBucketCounts()) { + break + } + bk := pt.GetBucketCounts()[index] + bucket := &prompb.Sample{ + Value: float64(bk), + Timestamp: time, + } + boundStr := strconv.FormatFloat(bound, 'f', -1, 64) + labels := createLabelSet(pt.GetLabels(), nameStr, baseName+bucketStr, leStr, boundStr) + addSample(tsMap, bucket, labels, metric) + + totalCount += bk + } + // add le=+Inf bucket + infBucket := &prompb.Sample{ + Value: float64(totalCount), + Timestamp: time, + } + infLabels := createLabelSet(pt.GetLabels(), nameStr, baseName+bucketStr, leStr, pInfStr) + addSample(tsMap, infBucket, infLabels, metric) +} + +// addSingleDoubleHistogramDataPoint converts pt to 2 + min(len(ExplicitBounds), len(BucketCount)) + 1 samples. It +//// ignore extra buckets if len(ExplicitBounds) > len(BucketCounts) +func addSingleDoubleHistogramDataPoint(pt *otlp.DoubleHistogramDataPoint, metric *otlp.Metric, namespace string, + tsMap map[string]*prompb.TimeSeries) { + if pt == nil { + return + } + time := convertTimeStamp(pt.TimeUnixNano) + // sum, count, and buckets of the histogram should append suffix to baseName + baseName := getPromMetricName(metric, namespace) + // treat sum as a sample in an individual TimeSeries + sum := &prompb.Sample{ + Value: pt.GetSum(), + Timestamp: time, + } + + sumlabels := createLabelSet(pt.GetLabels(), nameStr, baseName+sumStr) + addSample(tsMap, sum, sumlabels, metric) + + // treat count as a sample in an individual TimeSeries + count := &prompb.Sample{ + Value: float64(pt.GetCount()), + Timestamp: time, + } + countlabels := createLabelSet(pt.GetLabels(), nameStr, baseName+countStr) + addSample(tsMap, count, countlabels, metric) + + // count for +Inf bound + var totalCount uint64 + + // process each bound, ignore extra bucket values + for index, bound := range pt.GetExplicitBounds() { + if index >= len(pt.GetBucketCounts()) { + break + } + bk := pt.GetBucketCounts()[index] + bucket := &prompb.Sample{ + Value: float64(bk), + Timestamp: time, + } + boundStr := strconv.FormatFloat(bound, 'f', -1, 64) + labels := createLabelSet(pt.GetLabels(), nameStr, baseName+bucketStr, leStr, boundStr) + addSample(tsMap, bucket, labels, metric) + + totalCount += bk + } + // add le=+Inf bucket + infBucket := &prompb.Sample{ + Value: float64(totalCount), + Timestamp: time, + } + infLabels := createLabelSet(pt.GetLabels(), nameStr, baseName+bucketStr, leStr, pInfStr) + addSample(tsMap, infBucket, infLabels, metric) +} diff --git a/exporter/prometheusremotewriteexporter/helper_test.go b/exporter/prometheusremotewriteexporter/helper_test.go index f3ccaa777e4..c93626325c7 100644 --- a/exporter/prometheusremotewriteexporter/helper_test.go +++ b/exporter/prometheusremotewriteexporter/helper_test.go @@ -21,49 +21,52 @@ import ( "github.com/prometheus/prometheus/prompb" "github.com/stretchr/testify/assert" + "go.opentelemetry.io/collector/consumer/pdata" common "go.opentelemetry.io/collector/internal/data/opentelemetry-proto-gen/common/v1" - otlp "go.opentelemetry.io/collector/internal/data/opentelemetry-proto-gen/metrics/v1old" + otlp "go.opentelemetry.io/collector/internal/data/opentelemetry-proto-gen/metrics/v1" ) // Test_validateMetrics checks validateMetrics return true if a type and temporality combination is valid, false // otherwise. func Test_validateMetrics(t *testing.T) { + // define a single test type combTest struct { - name string - desc *otlp.MetricDescriptor - want bool + name string + metric *otlp.Metric + want bool } tests := []combTest{} // append true cases - for i := range validCombinations { - name := "valid_" + strconv.Itoa(i) - desc := getDescriptor(name, i, validCombinations) + for k, validMetric := range validMetrics1 { + name := "valid_" + k + tests = append(tests, combTest{ name, - desc, + validMetric, true, }) } - // append false cases - for i := range invalidCombinations { - name := "invalid_" + strconv.Itoa(i) - desc := getDescriptor(name, i, invalidCombinations) + + // append nil case + tests = append(tests, combTest{"invalid_nil", nil, false}) + + for k, invalidMetric := range invalidMetrics { + name := "valid_" + k + tests = append(tests, combTest{ name, - desc, + invalidMetric, false, }) } - // append nil case - tests = append(tests, combTest{"invalid_nil", nil, false}) // run tests for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - got := validateMetrics(tt.desc) + got := validateMetrics(tt.metric) assert.Equal(t, tt.want, got) }) } @@ -75,7 +78,7 @@ func Test_validateMetrics(t *testing.T) { // case. func Test_addSample(t *testing.T) { type testCase struct { - desc otlp.MetricDescriptor_Type + metric *otlp.Metric sample prompb.Sample labels []prompb.Label } @@ -90,13 +93,13 @@ func Test_addSample(t *testing.T) { "two_points_same_ts_same_metric", map[string]*prompb.TimeSeries{}, []testCase{ - {otlp.MetricDescriptor_INT64, - getSample(float64(intVal1), msTime1), + {validMetrics1[validDoubleGauge], + getSample(floatVal1, msTime1), promLbs1, }, { - otlp.MetricDescriptor_INT64, - getSample(float64(intVal2), msTime2), + validMetrics1[validDoubleGauge], + getSample(floatVal2, msTime2), promLbs1, }, }, @@ -106,11 +109,11 @@ func Test_addSample(t *testing.T) { "two_points_different_ts_same_metric", map[string]*prompb.TimeSeries{}, []testCase{ - {otlp.MetricDescriptor_INT64, + {validMetrics1[validIntGauge], getSample(float64(intVal1), msTime1), promLbs1, }, - {otlp.MetricDescriptor_INT64, + {validMetrics1[validIntGauge], getSample(float64(intVal1), msTime2), promLbs2, }, @@ -120,14 +123,14 @@ func Test_addSample(t *testing.T) { } t.Run("nil_case", func(t *testing.T) { tsMap := map[string]*prompb.TimeSeries{} - addSample(tsMap, nil, nil, 0) + addSample(tsMap, nil, nil, nil) assert.Exactly(t, tsMap, map[string]*prompb.TimeSeries{}) }) // run tests for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - addSample(tt.orig, &tt.testCase[0].sample, tt.testCase[0].labels, tt.testCase[0].desc) - addSample(tt.orig, &tt.testCase[1].sample, tt.testCase[1].labels, tt.testCase[1].desc) + addSample(tt.orig, &tt.testCase[0].sample, tt.testCase[0].labels, tt.testCase[0].metric) + addSample(tt.orig, &tt.testCase[1].sample, tt.testCase[1].labels, tt.testCase[1].metric) assert.Exactly(t, tt.want, tt.orig) }) } @@ -137,42 +140,42 @@ func Test_addSample(t *testing.T) { // metric type combination. func Test_timeSeriesSignature(t *testing.T) { tests := []struct { - name string - lbs []prompb.Label - desc otlp.MetricDescriptor_Type - want string + name string + lbs []prompb.Label + metric *otlp.Metric + want string }{ { "int64_signature", promLbs1, - otlp.MetricDescriptor_INT64, - typeInt64 + lb1Sig, + validMetrics1[validIntGauge], + strconv.Itoa(int(pdata.MetricDataTypeIntGauge)) + lb1Sig, }, { "histogram_signature", promLbs2, - otlp.MetricDescriptor_HISTOGRAM, - typeHistogram + lb2Sig, + validMetrics1[validIntHistogram], + strconv.Itoa(int(pdata.MetricDataTypeIntHistogram)) + lb2Sig, }, { "unordered_signature", getPromLabels(label22, value22, label21, value21), - otlp.MetricDescriptor_HISTOGRAM, - typeHistogram + lb2Sig, + validMetrics1[validIntHistogram], + strconv.Itoa(int(pdata.MetricDataTypeIntHistogram)) + lb2Sig, }, // descriptor type cannot be nil, as checked by validateMetrics { "nil_case", nil, - otlp.MetricDescriptor_HISTOGRAM, - typeHistogram, + validMetrics1[validIntHistogram], + strconv.Itoa(int(pdata.MetricDataTypeIntHistogram)), }, } // run tests for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - assert.EqualValues(t, tt.want, timeSeriesSignature(tt.desc, &tt.lbs)) + assert.EqualValues(t, tt.want, timeSeriesSignature(tt.metric, &tt.lbs)) }) } } @@ -236,10 +239,10 @@ func Test_createLabelSet(t *testing.T) { // invalid characters. func Test_getPromMetricName(t *testing.T) { tests := []struct { - name string - desc *otlp.MetricDescriptor - ns string - want string + name string + metric *otlp.Metric + ns string + want string }{ { "nil_case", @@ -249,34 +252,33 @@ func Test_getPromMetricName(t *testing.T) { }, { "normal_case", - getDescriptor(name1, histogramComb, validCombinations), + validMetrics1[validDoubleGauge], ns1, - "test_ns_valid_single_int_point", + "test_ns_" + validDoubleGauge, }, { "empty_namespace", - getDescriptor(name1, summaryComb, validCombinations), + validMetrics1[validDoubleGauge], "", - "valid_single_int_point", + validDoubleGauge, }, { "total_suffix", - getDescriptor(name1, monotonicInt64Comb, validCombinations), + validMetrics1[validIntSum], ns1, - "test_ns_valid_single_int_point_total", + "test_ns_" + validIntSum + delimeter + totalStr, }, { "dirty_string", - getDescriptor(name1+dirty1, monotonicInt64Comb, validCombinations), + validMetrics2[validIntGaugeDirty], "7" + ns1, - "key_7test_ns_valid_single_int_point__total", + "key_7test_ns__" + validIntGauge + "_", }, } // run tests for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - assert.Equal(t, tt.want, getPromMetricName(tt.desc, tt.ns)) + assert.Equal(t, tt.want, getPromMetricName(tt.metric, tt.ns)) }) } - } diff --git a/exporter/prometheusremotewriteexporter/testutil_test.go b/exporter/prometheusremotewriteexporter/testutil_test.go index f545b495c35..b90bd11932c 100644 --- a/exporter/prometheusremotewriteexporter/testutil_test.go +++ b/exporter/prometheusremotewriteexporter/testutil_test.go @@ -20,27 +20,15 @@ import ( "github.com/prometheus/prometheus/prompb" commonpb "go.opentelemetry.io/collector/internal/data/opentelemetry-proto-gen/common/v1" - otlp "go.opentelemetry.io/collector/internal/data/opentelemetry-proto-gen/metrics/v1old" - "go.opentelemetry.io/collector/internal/dataold" + otlp "go.opentelemetry.io/collector/internal/data/opentelemetry-proto-gen/metrics/v1" ) -type combination struct { - ty otlp.MetricDescriptor_Type - temp otlp.MetricDescriptor_Temporality -} - var ( time1 = uint64(time.Now().UnixNano()) time2 = uint64(time.Date(1970, 1, 0, 0, 0, 0, 0, time.UTC).UnixNano()) msTime1 = int64(time1 / uint64(int64(time.Millisecond)/int64(time.Nanosecond))) msTime2 = int64(time2 / uint64(int64(time.Millisecond)/int64(time.Nanosecond))) - typeInt64 = "INT64" - typeMonotonicInt64 = "MONOTONIC_INT64" - typeMonotonicDouble = "MONOTONIC_DOUBLE" - typeHistogram = "HISTOGRAM" - typeSummary = "SUMMARY" - label11 = "test_label11" value11 = "test_value11" label12 = "test_label12" @@ -71,48 +59,353 @@ var ( lb1Sig = "-" + label11 + "-" + value11 + "-" + label12 + "-" + value12 lb2Sig = "-" + label21 + "-" + value21 + "-" + label22 + "-" + value22 ns1 = "test_ns" - name1 = "valid_single_int_point" - monotonicInt64Comb = 0 - monotonicDoubleComb = 1 - histogramComb = 2 - summaryComb = 3 - validCombinations = []combination{ - {otlp.MetricDescriptor_MONOTONIC_INT64, otlp.MetricDescriptor_CUMULATIVE}, - {otlp.MetricDescriptor_MONOTONIC_DOUBLE, otlp.MetricDescriptor_CUMULATIVE}, - {otlp.MetricDescriptor_HISTOGRAM, otlp.MetricDescriptor_CUMULATIVE}, - {otlp.MetricDescriptor_SUMMARY, otlp.MetricDescriptor_CUMULATIVE}, - {otlp.MetricDescriptor_INT64, otlp.MetricDescriptor_DELTA}, - {otlp.MetricDescriptor_DOUBLE, otlp.MetricDescriptor_DELTA}, - {otlp.MetricDescriptor_INT64, otlp.MetricDescriptor_INSTANTANEOUS}, - {otlp.MetricDescriptor_DOUBLE, otlp.MetricDescriptor_INSTANTANEOUS}, - {otlp.MetricDescriptor_INT64, otlp.MetricDescriptor_CUMULATIVE}, - {otlp.MetricDescriptor_DOUBLE, otlp.MetricDescriptor_CUMULATIVE}, - } - invalidCombinations = []combination{ - {otlp.MetricDescriptor_MONOTONIC_INT64, otlp.MetricDescriptor_DELTA}, - {otlp.MetricDescriptor_MONOTONIC_DOUBLE, otlp.MetricDescriptor_DELTA}, - {otlp.MetricDescriptor_HISTOGRAM, otlp.MetricDescriptor_DELTA}, - {otlp.MetricDescriptor_SUMMARY, otlp.MetricDescriptor_DELTA}, - {otlp.MetricDescriptor_MONOTONIC_INT64, otlp.MetricDescriptor_DELTA}, - {otlp.MetricDescriptor_MONOTONIC_DOUBLE, otlp.MetricDescriptor_DELTA}, - {otlp.MetricDescriptor_HISTOGRAM, otlp.MetricDescriptor_DELTA}, - {otlp.MetricDescriptor_SUMMARY, otlp.MetricDescriptor_DELTA}, - {ty: otlp.MetricDescriptor_INVALID_TYPE}, - {temp: otlp.MetricDescriptor_INVALID_TEMPORALITY}, - {}, - } twoPointsSameTs = map[string]*prompb.TimeSeries{ - typeInt64 + "-" + label11 + "-" + value11 + "-" + label12 + "-" + value12: getTimeSeries(getPromLabels(label11, value11, label12, value12), + "2" + "-" + label11 + "-" + value11 + "-" + label12 + "-" + value12: getTimeSeries(getPromLabels(label11, value11, label12, value12), getSample(float64(intVal1), msTime1), getSample(float64(intVal2), msTime2)), } twoPointsDifferentTs = map[string]*prompb.TimeSeries{ - typeInt64 + "-" + label11 + "-" + value11 + "-" + label12 + "-" + value12: getTimeSeries(getPromLabels(label11, value11, label12, value12), + "1" + "-" + label11 + "-" + value11 + "-" + label12 + "-" + value12: getTimeSeries(getPromLabels(label11, value11, label12, value12), getSample(float64(intVal1), msTime1)), - typeInt64 + "-" + label21 + "-" + value21 + "-" + label22 + "-" + value22: getTimeSeries(getPromLabels(label21, value21, label22, value22), + "1" + "-" + label21 + "-" + value21 + "-" + label22 + "-" + value22: getTimeSeries(getPromLabels(label21, value21, label22, value22), getSample(float64(intVal1), msTime2)), } + bounds = []float64{0.1, 0.5, 0.99} + buckets = []uint64{1, 2, 3} + + validIntGauge = "valid_IntGauge" + validDoubleGauge = "valid_DoubleGauge" + validIntSum = "valid_IntSum" + validDoubleSum = "valid_DoubleSum" + validIntHistogram = "valid_IntHistogram" + validDoubleHistogram = "valid_DoubleHistogram" + + validIntGaugeDirty = "*valid_IntGauge$" + + unmatchedBoundBucketIntHist = "unmatchedBoundBucketIntHist" + unmatchedBoundBucketDoubleHist = "unmatchedBoundBucketDoubleHist" + + // valid metrics as input should not return error + validMetrics1 = map[string]*otlp.Metric{ + validIntGauge: { + Name: validIntGauge, + Data: &otlp.Metric_IntGauge{ + IntGauge: &otlp.IntGauge{ + DataPoints: []*otlp.IntDataPoint{ + getIntDataPoint(lbs1, intVal1, time1), + nil, + }, + }, + }, + }, + validDoubleGauge: { + Name: validDoubleGauge, + Data: &otlp.Metric_DoubleGauge{ + DoubleGauge: &otlp.DoubleGauge{ + DataPoints: []*otlp.DoubleDataPoint{ + getDoubleDataPoint(lbs1, floatVal1, time1), + nil, + }, + }, + }, + }, + validIntSum: { + Name: validIntSum, + Data: &otlp.Metric_IntSum{ + IntSum: &otlp.IntSum{ + DataPoints: []*otlp.IntDataPoint{ + getIntDataPoint(lbs1, intVal1, time1), + nil, + }, + AggregationTemporality: otlp.AggregationTemporality_AGGREGATION_TEMPORALITY_CUMULATIVE, + }, + }, + }, + validDoubleSum: { + Name: validDoubleSum, + Data: &otlp.Metric_DoubleSum{ + DoubleSum: &otlp.DoubleSum{ + DataPoints: []*otlp.DoubleDataPoint{ + getDoubleDataPoint(lbs1, floatVal1, time1), + nil, + }, + AggregationTemporality: otlp.AggregationTemporality_AGGREGATION_TEMPORALITY_CUMULATIVE, + }, + }, + }, + validIntHistogram: { + Name: validIntHistogram, + Data: &otlp.Metric_IntHistogram{ + IntHistogram: &otlp.IntHistogram{ + DataPoints: []*otlp.IntHistogramDataPoint{ + getIntHistogramDataPoint(lbs1, time1, floatVal1, uint64(intVal1), bounds, buckets), + nil, + }, + AggregationTemporality: otlp.AggregationTemporality_AGGREGATION_TEMPORALITY_CUMULATIVE, + }, + }, + }, + validDoubleHistogram: { + Name: validDoubleHistogram, + Data: &otlp.Metric_DoubleHistogram{ + DoubleHistogram: &otlp.DoubleHistogram{ + DataPoints: []*otlp.DoubleHistogramDataPoint{ + getDoubleHistogramDataPoint(lbs1, time1, floatVal1, uint64(intVal1), bounds, buckets), + nil, + }, + AggregationTemporality: otlp.AggregationTemporality_AGGREGATION_TEMPORALITY_CUMULATIVE, + }, + }, + }, + } + validMetrics2 = map[string]*otlp.Metric{ + validIntGauge: { + Name: validIntGauge, + Data: &otlp.Metric_IntGauge{ + IntGauge: &otlp.IntGauge{ + DataPoints: []*otlp.IntDataPoint{ + getIntDataPoint(lbs2, intVal2, time2), + }, + }, + }, + }, + validDoubleGauge: { + Name: validDoubleGauge, + Data: &otlp.Metric_DoubleGauge{ + DoubleGauge: &otlp.DoubleGauge{ + DataPoints: []*otlp.DoubleDataPoint{ + getDoubleDataPoint(lbs2, floatVal2, time2), + }, + }, + }, + }, + validIntSum: { + Name: validIntSum, + Data: &otlp.Metric_IntSum{ + IntSum: &otlp.IntSum{ + DataPoints: []*otlp.IntDataPoint{ + getIntDataPoint(lbs2, intVal2, time2), + }, + AggregationTemporality: otlp.AggregationTemporality_AGGREGATION_TEMPORALITY_CUMULATIVE, + }, + }, + }, + validDoubleSum: { + Name: validDoubleSum, + Data: &otlp.Metric_DoubleSum{ + DoubleSum: &otlp.DoubleSum{ + DataPoints: []*otlp.DoubleDataPoint{ + getDoubleDataPoint(lbs2, floatVal2, time2), + }, + AggregationTemporality: otlp.AggregationTemporality_AGGREGATION_TEMPORALITY_CUMULATIVE, + }, + }, + }, + validIntHistogram: { + Name: validIntHistogram, + Data: &otlp.Metric_IntHistogram{ + IntHistogram: &otlp.IntHistogram{ + DataPoints: []*otlp.IntHistogramDataPoint{ + getIntHistogramDataPoint(lbs2, time2, floatVal2, uint64(intVal2), bounds, buckets), + }, + AggregationTemporality: otlp.AggregationTemporality_AGGREGATION_TEMPORALITY_CUMULATIVE, + }, + }, + }, + validDoubleHistogram: { + Name: validDoubleHistogram, + Data: &otlp.Metric_DoubleHistogram{ + DoubleHistogram: &otlp.DoubleHistogram{ + DataPoints: []*otlp.DoubleHistogramDataPoint{ + getDoubleHistogramDataPoint(lbs2, time2, floatVal2, uint64(intVal2), bounds, buckets), + }, + AggregationTemporality: otlp.AggregationTemporality_AGGREGATION_TEMPORALITY_CUMULATIVE, + }, + }, + }, + validIntGaugeDirty: { + Name: validIntGaugeDirty, + Data: &otlp.Metric_IntGauge{ + IntGauge: &otlp.IntGauge{ + DataPoints: []*otlp.IntDataPoint{ + getIntDataPoint(lbs1, intVal1, time1), + nil, + }, + }, + }, + }, + unmatchedBoundBucketIntHist: { + Name: unmatchedBoundBucketIntHist, + Data: &otlp.Metric_IntHistogram{ + IntHistogram: &otlp.IntHistogram{ + DataPoints: []*otlp.IntHistogramDataPoint{ + { + ExplicitBounds: []float64{0.1, 0.2, 0.3}, + BucketCounts: []uint64{1, 2}, + }, + }, + AggregationTemporality: otlp.AggregationTemporality_AGGREGATION_TEMPORALITY_CUMULATIVE, + }, + }, + }, + unmatchedBoundBucketDoubleHist: { + Name: unmatchedBoundBucketDoubleHist, + Data: &otlp.Metric_DoubleHistogram{ + DoubleHistogram: &otlp.DoubleHistogram{ + DataPoints: []*otlp.DoubleHistogramDataPoint{ + { + ExplicitBounds: []float64{0.1, 0.2, 0.3}, + BucketCounts: []uint64{1, 2}, + }, + }, + AggregationTemporality: otlp.AggregationTemporality_AGGREGATION_TEMPORALITY_CUMULATIVE, + }, + }, + }, + } + + nilMetric = "nil" + empty = "empty" + + // Category 1: type and data field doesn't match + notMatchIntGauge = "noMatchIntGauge" + notMatchDoubleGauge = "notMatchDoubleGauge" + notMatchIntSum = "notMatchIntSum" + notMatchDoubleSum = "notMatchDoubleSum" + notMatchIntHistogram = "notMatchIntHistogram" + notMatchDoubleHistogram = "notMatchDoubleHistogram" + + // Category 2: invalid type and temporality combination + invalidIntSum = "invalidIntSum" + invalidDoubleSum = "invalidDoubleSum" + invalidIntHistogram = "invalidIntHistogram" + invalidDoubleHistogram = "invalidDoubleHistogram" + + //Category 3: nil data points + nilDataPointIntGauge = "nilDataPointIntGauge" + nilDataPointDoubleGauge = "nilDataPointDoubleGauge" + nilDataPointIntSum = "nilDataPointIntSum" + nilDataPointDoubleSum = "nilDataPointDoubleSum" + nilDataPointIntHistogram = "nilDataPointIntHistogram" + nilDataPointDoubleHistogram = "nilDataPointDoubleHistogram" + + // different metrics that will not pass validate metrics + invalidMetrics = map[string]*otlp.Metric{ + // nil + nilMetric: nil, + // Data = nil + empty: {}, + notMatchIntGauge: { + Name: notMatchIntGauge, + Data: &otlp.Metric_IntGauge{}, + }, + notMatchDoubleGauge: { + Name: notMatchDoubleGauge, + Data: &otlp.Metric_DoubleGauge{}, + }, + notMatchIntSum: { + Name: notMatchIntSum, + Data: &otlp.Metric_IntSum{}, + }, + notMatchDoubleSum: { + Name: notMatchDoubleSum, + Data: &otlp.Metric_DoubleSum{}, + }, + notMatchIntHistogram: { + Name: notMatchIntHistogram, + Data: &otlp.Metric_IntHistogram{}, + }, + notMatchDoubleHistogram: { + Name: notMatchDoubleHistogram, + Data: &otlp.Metric_DoubleHistogram{}, + }, + invalidIntSum: { + Name: invalidIntSum, + Data: &otlp.Metric_IntSum{ + IntSum: &otlp.IntSum{ + AggregationTemporality: otlp.AggregationTemporality_AGGREGATION_TEMPORALITY_DELTA, + }, + }, + }, + invalidDoubleSum: { + Name: invalidDoubleSum, + Data: &otlp.Metric_DoubleSum{ + DoubleSum: &otlp.DoubleSum{ + AggregationTemporality: otlp.AggregationTemporality_AGGREGATION_TEMPORALITY_DELTA, + }, + }, + }, + invalidIntHistogram: { + Name: invalidIntHistogram, + Data: &otlp.Metric_IntHistogram{ + IntHistogram: &otlp.IntHistogram{ + AggregationTemporality: otlp.AggregationTemporality_AGGREGATION_TEMPORALITY_DELTA, + }, + }, + }, + invalidDoubleHistogram: { + Name: invalidDoubleHistogram, + Data: &otlp.Metric_DoubleHistogram{ + DoubleHistogram: &otlp.DoubleHistogram{ + AggregationTemporality: otlp.AggregationTemporality_AGGREGATION_TEMPORALITY_DELTA, + }, + }, + }, + } + + // different metrics that will cause the exporter to return an error + errorMetrics = map[string]*otlp.Metric{ + + nilDataPointIntGauge: { + Name: nilDataPointIntGauge, + Data: &otlp.Metric_IntGauge{ + IntGauge: &otlp.IntGauge{DataPoints: nil}, + }, + }, + nilDataPointDoubleGauge: { + Name: nilDataPointDoubleGauge, + Data: &otlp.Metric_DoubleGauge{ + DoubleGauge: &otlp.DoubleGauge{DataPoints: nil}, + }, + }, + nilDataPointIntSum: { + Name: nilDataPointIntSum, + Data: &otlp.Metric_IntSum{ + IntSum: &otlp.IntSum{ + DataPoints: nil, + AggregationTemporality: otlp.AggregationTemporality_AGGREGATION_TEMPORALITY_CUMULATIVE, + }, + }, + }, + nilDataPointDoubleSum: { + Name: nilDataPointDoubleSum, + Data: &otlp.Metric_DoubleSum{ + DoubleSum: &otlp.DoubleSum{ + DataPoints: nil, + AggregationTemporality: otlp.AggregationTemporality_AGGREGATION_TEMPORALITY_CUMULATIVE, + }, + }, + }, + nilDataPointIntHistogram: { + Name: nilDataPointIntHistogram, + Data: &otlp.Metric_IntHistogram{ + IntHistogram: &otlp.IntHistogram{ + DataPoints: nil, + AggregationTemporality: otlp.AggregationTemporality_AGGREGATION_TEMPORALITY_CUMULATIVE, + }, + }, + }, + nilDataPointDoubleHistogram: { + Name: nilDataPointDoubleHistogram, + Data: &otlp.Metric_DoubleHistogram{ + DoubleHistogram: &otlp.DoubleHistogram{ + DataPoints: nil, + AggregationTemporality: otlp.AggregationTemporality_AGGREGATION_TEMPORALITY_CUMULATIVE, + }, + }, + }, + } ) // OTLP metrics @@ -128,18 +421,8 @@ func getLabels(labels ...string) []*commonpb.StringKeyValue { return set } -func getDescriptor(name string, i int, comb []combination) *otlp.MetricDescriptor { - return &otlp.MetricDescriptor{ - Name: name, - Description: "", - Unit: "", - Type: comb[i].ty, - Temporality: comb[i].temp, - } -} - -func getIntDataPoint(labels []*commonpb.StringKeyValue, value int64, ts uint64) *otlp.Int64DataPoint { - return &otlp.Int64DataPoint{ +func getIntDataPoint(labels []*commonpb.StringKeyValue, value int64, ts uint64) *otlp.IntDataPoint { + return &otlp.IntDataPoint{ Labels: labels, StartTimeUnixNano: 0, TimeUnixNano: ts, @@ -156,22 +439,29 @@ func getDoubleDataPoint(labels []*commonpb.StringKeyValue, value float64, ts uin } } -func getHistogramDataPoint(labels []*commonpb.StringKeyValue, ts uint64, sum float64, count uint64, bounds []float64, buckets []uint64) *otlp.HistogramDataPoint { - bks := []*otlp.HistogramDataPoint_Bucket{} - for _, c := range buckets { - bks = append(bks, &otlp.HistogramDataPoint_Bucket{ - Count: c, - Exemplar: nil, - }) - } - return &otlp.HistogramDataPoint{ +func getIntHistogramDataPoint(labels []*commonpb.StringKeyValue, ts uint64, sum float64, count uint64, bounds []float64, + buckets []uint64) *otlp.IntHistogramDataPoint { + return &otlp.IntHistogramDataPoint{ Labels: labels, StartTimeUnixNano: 0, TimeUnixNano: ts, Count: count, - Sum: sum, - Buckets: bks, + Sum: int64(sum), + BucketCounts: buckets, ExplicitBounds: bounds, + Exemplars: nil, + } +} + +func getDoubleHistogramDataPoint(labels []*commonpb.StringKeyValue, ts uint64, sum float64, count uint64, + bounds []float64, buckets []uint64) *otlp.DoubleHistogramDataPoint { + return &otlp.DoubleHistogramDataPoint{ + Labels: labels, + TimeUnixNano: ts, + Count: count, + Sum: sum, + BucketCounts: buckets, + ExplicitBounds: bounds, } } @@ -206,53 +496,3 @@ func getTimeSeries(labels []prompb.Label, samples ...prompb.Sample) *prompb.Time Samples: samples, } } - -func setCumulative(metricsData *dataold.MetricData) { - for _, r := range dataold.MetricDataToOtlp(*metricsData) { - for _, instMetrics := range r.InstrumentationLibraryMetrics { - for _, m := range instMetrics.Metrics { - m.MetricDescriptor.Temporality = otlp.MetricDescriptor_CUMULATIVE - } - } - } -} - -//setDataPointToNil is for creating the dataold.MetricData to test with -func setDataPointToNil(metricsData *dataold.MetricData, dataField string) { - for _, r := range dataold.MetricDataToOtlp(*metricsData) { - for _, instMetrics := range r.InstrumentationLibraryMetrics { - for _, m := range instMetrics.Metrics { - switch dataField { - case typeMonotonicInt64: - m.Int64DataPoints = nil - case typeMonotonicDouble: - m.DoubleDataPoints = nil - case typeHistogram: - m.HistogramDataPoints = nil - case typeSummary: - m.SummaryDataPoints = nil - } - } - } - } -} - -//setType is for creating the dataold.MetricData to test with -func setType(metricsData *dataold.MetricData, dataField string) { - for _, r := range dataold.MetricDataToOtlp(*metricsData) { - for _, instMetrics := range r.InstrumentationLibraryMetrics { - for _, m := range instMetrics.Metrics { - switch dataField { - case typeMonotonicInt64: - m.GetMetricDescriptor().Type = otlp.MetricDescriptor_MONOTONIC_INT64 - case typeMonotonicDouble: - m.GetMetricDescriptor().Type = otlp.MetricDescriptor_MONOTONIC_DOUBLE - case typeHistogram: - m.GetMetricDescriptor().Type = otlp.MetricDescriptor_HISTOGRAM - case typeSummary: - m.GetMetricDescriptor().Type = otlp.MetricDescriptor_SUMMARY - } - } - } - } -} From d007b8124c6f896b75f51af0a42658ed590032f5 Mon Sep 17 00:00:00 2001 From: huyan0 Date: Tue, 1 Sep 2020 13:59:42 -0500 Subject: [PATCH 2/2] fix type --- exporter/prometheusremotewriteexporter/helper.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/exporter/prometheusremotewriteexporter/helper.go b/exporter/prometheusremotewriteexporter/helper.go index 0a59f48c0b0..bf5eda1c2e6 100644 --- a/exporter/prometheusremotewriteexporter/helper.go +++ b/exporter/prometheusremotewriteexporter/helper.go @@ -366,7 +366,7 @@ func addSingleIntHistogramDataPoint(pt *otlp.IntHistogramDataPoint, metric *otlp } // addSingleDoubleHistogramDataPoint converts pt to 2 + min(len(ExplicitBounds), len(BucketCount)) + 1 samples. It -//// ignore extra buckets if len(ExplicitBounds) > len(BucketCounts) +// ignore extra buckets if len(ExplicitBounds) > len(BucketCounts) func addSingleDoubleHistogramDataPoint(pt *otlp.DoubleHistogramDataPoint, metric *otlp.Metric, namespace string, tsMap map[string]*prompb.TimeSeries) { if pt == nil {