From fd1a25b330fe7b5762f0b7aae6b794ec264239b7 Mon Sep 17 00:00:00 2001 From: Raphael Koh Date: Tue, 1 Dec 2020 19:58:15 -0500 Subject: [PATCH 1/3] Implement batching logic in EMF Exporter --- exporter/awsemfexporter/datapoint.go | 307 ++ exporter/awsemfexporter/datapoint_test.go | 715 +++++ exporter/awsemfexporter/emf_exporter.go | 59 +- exporter/awsemfexporter/emf_exporter_test.go | 71 +- exporter/awsemfexporter/groupedmetric.go | 93 + exporter/awsemfexporter/groupedmetric_test.go | 390 +++ exporter/awsemfexporter/metric_declaration.go | 60 +- .../awsemfexporter/metric_declaration_test.go | 197 +- exporter/awsemfexporter/metric_translator.go | 530 ++-- .../awsemfexporter/metric_translator_test.go | 2703 ++++++++--------- exporter/awsemfexporter/pusher_test.go | 2 +- exporter/awsemfexporter/util.go | 119 + exporter/awsemfexporter/util_test.go | 424 +++ 13 files changed, 3614 insertions(+), 2056 deletions(-) create mode 100644 exporter/awsemfexporter/datapoint.go create mode 100644 exporter/awsemfexporter/datapoint_test.go create mode 100644 exporter/awsemfexporter/groupedmetric.go create mode 100644 exporter/awsemfexporter/groupedmetric_test.go diff --git a/exporter/awsemfexporter/datapoint.go b/exporter/awsemfexporter/datapoint.go new file mode 100644 index 0000000000000..6b721152a7a94 --- /dev/null +++ b/exporter/awsemfexporter/datapoint.go @@ -0,0 +1,307 @@ +// Copyright 2020, OpenTelemetry Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package awsemfexporter + +import ( + "time" + + "go.opentelemetry.io/collector/consumer/pdata" + "go.uber.org/zap" + + "github.com/open-telemetry/opentelemetry-collector-contrib/exporter/awsemfexporter/mapwithexpiry" +) + +const ( + CleanInterval = 5 * time.Minute + MinTimeDiff = 50 * time.Millisecond // We assume 50 milli-seconds is the minimal gap between two collected data sample to be valid to calculate delta +) + +var currentState = mapwithexpiry.NewMapWithExpiry(CleanInterval) + +// DataPoint represents a processed metric data point +type DataPoint struct { + Value interface{} + Labels map[string]string +} + +// DataPoints is a wrapper interface for: +// - pdata.IntDataPointSlice +// - pdata.DoubleDataPointSlice +// - pdata.IntHistogramDataPointSlice +// - pdata.DoubleHistogramDataPointSlice +type DataPoints interface { + Len() int + // NOTE: At() is an expensive call as it calculates the metric's value + At(i int) DataPoint +} + +// rateCalculationMetadata contains the metadata required to perform rate calculation +type rateCalculationMetadata struct { + needsCalculateRate bool + rateKeyParams map[string]string + timestamp int64 +} + +// rateState stores a metric's value +type rateState struct { + value interface{} + timestamp int64 +} + +// IntDataPointSlice is a wrapper for pdata.IntDataPointSlice +type IntDataPointSlice struct { + instrumentationLibraryName string + rateCalculationMetadata + pdata.IntDataPointSlice +} + +// DoubleDataPointSlice is a wrapper for pdata.DoubleDataPointSlice +type DoubleDataPointSlice struct { + instrumentationLibraryName string + rateCalculationMetadata + pdata.DoubleDataPointSlice +} + +// DoubleHistogramDataPointSlice is a wrapper for pdata.DoubleHistogramDataPointSlice +type DoubleHistogramDataPointSlice struct { + instrumentationLibraryName string + pdata.DoubleHistogramDataPointSlice +} + +// DoubleSummaryDataPointSlice is a wrapper for pdata.DoubleSummaryDataPointSlice +type DoubleSummaryDataPointSlice struct { + instrumentationLibraryName string + pdata.DoubleSummaryDataPointSlice +} + +// At retrieves the IntDataPoint at the given index and performs rate calculation if necessary. +func (dps IntDataPointSlice) At(i int) DataPoint { + metric := dps.IntDataPointSlice.At(i) + labels := createLabels(metric.LabelsMap(), dps.instrumentationLibraryName) + + var metricVal interface{} + metricVal = metric.Value() + if dps.needsCalculateRate { + rateKey := createMetricKey(labels, dps.rateKeyParams) + metricVal = calculateRate(rateKey, metricVal, dps.timestamp) + } + + return DataPoint{ + Value: metricVal, + Labels: labels, + } +} + +// At retrieves the DoubleDataPoint at the given index and performs rate calculation if necessary. +func (dps DoubleDataPointSlice) At(i int) DataPoint { + metric := dps.DoubleDataPointSlice.At(i) + labels := createLabels(metric.LabelsMap(), dps.instrumentationLibraryName) + + var metricVal interface{} + metricVal = metric.Value() + if dps.needsCalculateRate { + rateKey := createMetricKey(labels, dps.rateKeyParams) + metricVal = calculateRate(rateKey, metricVal, dps.timestamp) + } + + return DataPoint{ + Value: metricVal, + Labels: labels, + } +} + +// At retrieves the DoubleHistogramDataPoint at the given index. +func (dps DoubleHistogramDataPointSlice) At(i int) DataPoint { + metric := dps.DoubleHistogramDataPointSlice.At(i) + labels := createLabels(metric.LabelsMap(), dps.instrumentationLibraryName) + + var minBound, maxBound float64 + bucketBounds := metric.ExplicitBounds() + if len(bucketBounds) > 0 { + minBound = bucketBounds[0] + maxBound = bucketBounds[len(bucketBounds)-1] + } + + return DataPoint{ + Value: &CWMetricStats{ + Min: minBound, + Max: maxBound, + Count: metric.Count(), + Sum: metric.Sum(), + }, + Labels: labels, + } +} + +// At retrieves the DoubleSummaryDataPoint at the given index. +func (dps DoubleSummaryDataPointSlice) At(i int) DataPoint { + metric := dps.DoubleSummaryDataPointSlice.At(i) + labels := createLabels(metric.LabelsMap(), dps.instrumentationLibraryName) + + metricVal := &CWMetricStats{ + Count: metric.Count(), + Sum: metric.Sum(), + } + if quantileValues := metric.QuantileValues(); quantileValues.Len() > 0 { + metricVal.Min = quantileValues.At(0).Value() + metricVal.Max = quantileValues.At(quantileValues.Len() - 1).Value() + } + + return DataPoint{ + Value: metricVal, + Labels: labels, + } +} + +// createLabels converts OTel StringMap labels to a map and optionally adds in the +// OTel instrumentation library name +func createLabels(labelsMap pdata.StringMap, instrLibName string) map[string]string { + labels := make(map[string]string, labelsMap.Len()+1) + labelsMap.ForEach(func(k, v string) { + labels[k] = v + }) + + // Add OTel instrumentation lib name as an additional label if it is defined + if instrLibName != noInstrumentationLibraryName { + labels[OTellibDimensionKey] = instrLibName + } + + return labels +} + +// calculateRate calculates the metric value's rate of change using valDelta / timeDelta. +func calculateRate(metricKey string, val interface{}, timestamp int64) (metricRate interface{}) { + // get previous Metric content from map. Need to lock the map until set the new state + currentState.Lock() + if state, ok := currentState.Get(metricKey); ok { + prevStats := state.(*rateState) + deltaTime := timestamp - prevStats.timestamp + var deltaVal interface{} + + if _, ok := val.(float64); ok { + if _, ok := prevStats.value.(int64); ok { + deltaVal = val.(float64) - float64(prevStats.value.(int64)) + } else { + deltaVal = val.(float64) - prevStats.value.(float64) + } + if deltaTime > MinTimeDiff.Milliseconds() && deltaVal.(float64) >= 0 { + metricRate = deltaVal.(float64) * 1e3 / float64(deltaTime) + } + } else { + if _, ok := prevStats.value.(float64); ok { + deltaVal = val.(int64) - int64(prevStats.value.(float64)) + } else { + deltaVal = val.(int64) - prevStats.value.(int64) + } + if deltaTime > MinTimeDiff.Milliseconds() && deltaVal.(int64) >= 0 { + metricRate = deltaVal.(int64) * 1e3 / deltaTime + } + } + } + content := &rateState{ + value: val, + timestamp: timestamp, + } + currentState.Set(metricKey, content) + currentState.Unlock() + if metricRate == nil { + if _, ok := val.(float64); ok { + metricRate = float64(0) + } else { + metricRate = int64(0) + } + } + return metricRate +} + +// getDataPoints retrieves data points from OT Metric. +func getDataPoints(pmd *pdata.Metric, metadata CWMetricMetadata, logger *zap.Logger) (dps DataPoints) { + if pmd == nil { + return + } + + rateKeyParams := map[string]string{ + (namespaceKey): metadata.Namespace, + (metricNameKey): pmd.Name(), + (logGroupKey): metadata.LogGroup, + (logStreamKey): metadata.LogStream, + } + + switch pmd.DataType() { + case pdata.MetricDataTypeIntGauge: + metric := pmd.IntGauge() + dps = IntDataPointSlice{ + metadata.InstrumentationLibraryName, + rateCalculationMetadata{ + false, + rateKeyParams, + metadata.Timestamp, + }, + metric.DataPoints(), + } + case pdata.MetricDataTypeDoubleGauge: + metric := pmd.DoubleGauge() + dps = DoubleDataPointSlice{ + metadata.InstrumentationLibraryName, + rateCalculationMetadata{ + false, + rateKeyParams, + metadata.Timestamp, + }, + metric.DataPoints(), + } + case pdata.MetricDataTypeIntSum: + metric := pmd.IntSum() + dps = IntDataPointSlice{ + metadata.InstrumentationLibraryName, + rateCalculationMetadata{ + metric.AggregationTemporality() == pdata.AggregationTemporalityCumulative, + rateKeyParams, + metadata.Timestamp, + }, + metric.DataPoints(), + } + case pdata.MetricDataTypeDoubleSum: + metric := pmd.DoubleSum() + dps = DoubleDataPointSlice{ + metadata.InstrumentationLibraryName, + rateCalculationMetadata{ + metric.AggregationTemporality() == pdata.AggregationTemporalityCumulative, + rateKeyParams, + metadata.Timestamp, + }, + metric.DataPoints(), + } + case pdata.MetricDataTypeDoubleHistogram: + metric := pmd.DoubleHistogram() + dps = DoubleHistogramDataPointSlice{ + metadata.InstrumentationLibraryName, + metric.DataPoints(), + } + case pdata.MetricDataTypeDoubleSummary: + metric := pmd.DoubleSummary() + dps = DoubleSummaryDataPointSlice{ + metadata.InstrumentationLibraryName, + metric.DataPoints(), + } + default: + logger.Warn("Unhandled metric data type.", + zap.String("DataType", pmd.DataType().String()), + zap.String("Name", pmd.Name()), + zap.String("Unit", pmd.Unit()), + ) + } + return +} diff --git a/exporter/awsemfexporter/datapoint_test.go b/exporter/awsemfexporter/datapoint_test.go new file mode 100644 index 0000000000000..70fab67625497 --- /dev/null +++ b/exporter/awsemfexporter/datapoint_test.go @@ -0,0 +1,715 @@ +// Copyright 2020, OpenTelemetry Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package awsemfexporter + +import ( + "reflect" + "testing" + "time" + + metricspb "github.com/census-instrumentation/opencensus-proto/gen-go/metrics/v1" + "github.com/golang/protobuf/ptypes/wrappers" + "github.com/stretchr/testify/assert" + "go.opentelemetry.io/collector/consumer/consumerdata" + "go.opentelemetry.io/collector/consumer/pdata" + "go.opentelemetry.io/collector/translator/internaldata" + "go.uber.org/zap" + "go.uber.org/zap/zapcore" + "go.uber.org/zap/zaptest/observer" +) + +func generateTestIntGauge(name string) *metricspb.Metric { + return &metricspb.Metric{ + MetricDescriptor: &metricspb.MetricDescriptor{ + Name: name, + Type: metricspb.MetricDescriptor_GAUGE_INT64, + Unit: "Count", + LabelKeys: []*metricspb.LabelKey{ + {Key: "label1"}, + }, + }, + Timeseries: []*metricspb.TimeSeries{ + { + LabelValues: []*metricspb.LabelValue{ + {Value: "value1", HasValue: true}, + }, + Points: []*metricspb.Point{ + { + Value: &metricspb.Point_Int64Value{ + Int64Value: 1, + }, + }, + }, + }, + }, + } +} + +func generateTestDoubleGauge(name string) *metricspb.Metric { + return &metricspb.Metric{ + MetricDescriptor: &metricspb.MetricDescriptor{ + Name: name, + Type: metricspb.MetricDescriptor_GAUGE_DOUBLE, + Unit: "Count", + LabelKeys: []*metricspb.LabelKey{ + {Key: "label1"}, + }, + }, + Timeseries: []*metricspb.TimeSeries{ + { + LabelValues: []*metricspb.LabelValue{ + {Value: "value1", HasValue: true}, + }, + Points: []*metricspb.Point{ + { + Value: &metricspb.Point_DoubleValue{ + DoubleValue: 0.1, + }, + }, + }, + }, + }, + } +} + +func generateTestIntSum(name string) *metricspb.Metric { + return &metricspb.Metric{ + MetricDescriptor: &metricspb.MetricDescriptor{ + Name: name, + Type: metricspb.MetricDescriptor_CUMULATIVE_INT64, + Unit: "Count", + LabelKeys: []*metricspb.LabelKey{ + {Key: "label1"}, + }, + }, + Timeseries: []*metricspb.TimeSeries{ + { + LabelValues: []*metricspb.LabelValue{ + {Value: "value1", HasValue: true}, + {Value: "value2", HasValue: true}, + }, + Points: []*metricspb.Point{ + { + Value: &metricspb.Point_Int64Value{ + Int64Value: 1, + }, + }, + }, + }, + }, + } +} + +func generateTestDoubleSum(name string) *metricspb.Metric { + return &metricspb.Metric{ + MetricDescriptor: &metricspb.MetricDescriptor{ + Name: name, + Type: metricspb.MetricDescriptor_CUMULATIVE_DOUBLE, + Unit: "Count", + LabelKeys: []*metricspb.LabelKey{ + {Key: "label1"}, + }, + }, + Timeseries: []*metricspb.TimeSeries{ + { + LabelValues: []*metricspb.LabelValue{ + {Value: "value1", HasValue: true}, + {Value: "value2", HasValue: true}, + }, + Points: []*metricspb.Point{ + { + Value: &metricspb.Point_DoubleValue{ + DoubleValue: 0.1, + }, + }, + }, + }, + }, + } +} + +func generateTestDoubleHistogram(name string) *metricspb.Metric { + return &metricspb.Metric{ + MetricDescriptor: &metricspb.MetricDescriptor{ + Name: name, + Type: metricspb.MetricDescriptor_CUMULATIVE_DISTRIBUTION, + Unit: "Seconds", + LabelKeys: []*metricspb.LabelKey{ + {Key: "label1"}, + }, + }, + Timeseries: []*metricspb.TimeSeries{ + { + LabelValues: []*metricspb.LabelValue{ + {Value: "value1", HasValue: true}, + {Value: "value2", HasValue: true}, + }, + Points: []*metricspb.Point{ + { + Value: &metricspb.Point_DistributionValue{ + DistributionValue: &metricspb.DistributionValue{ + Sum: 35.0, + Count: 18, + BucketOptions: &metricspb.DistributionValue_BucketOptions{ + Type: &metricspb.DistributionValue_BucketOptions_Explicit_{ + Explicit: &metricspb.DistributionValue_BucketOptions_Explicit{ + Bounds: []float64{0, 10}, + }, + }, + }, + Buckets: []*metricspb.DistributionValue_Bucket{ + { + Count: 5, + }, + { + Count: 6, + }, + { + Count: 7, + }, + }, + }, + }, + }, + }, + }, + }, + } +} + +func generateTestSummary(name string) *metricspb.Metric { + return &metricspb.Metric{ + MetricDescriptor: &metricspb.MetricDescriptor{ + Name: name, + Type: metricspb.MetricDescriptor_SUMMARY, + Unit: "Seconds", + LabelKeys: []*metricspb.LabelKey{ + {Key: "label1"}, + }, + }, + Timeseries: []*metricspb.TimeSeries{ + { + LabelValues: []*metricspb.LabelValue{ + {Value: "value1", HasValue: true}, + }, + Points: []*metricspb.Point{ + { + Value: &metricspb.Point_SummaryValue{ + SummaryValue: &metricspb.SummaryValue{ + Sum: &wrappers.DoubleValue{ + Value: 15.0, + }, + Count: &wrappers.Int64Value{ + Value: 5, + }, + Snapshot: &metricspb.SummaryValue_Snapshot{ + Count: &wrappers.Int64Value{ + Value: 5, + }, + Sum: &wrappers.DoubleValue{ + Value: 15.0, + }, + PercentileValues: []*metricspb.SummaryValue_Snapshot_ValueAtPercentile{ + { + Percentile: 0.0, + Value: 1, + }, + { + Percentile: 100.0, + Value: 5, + }, + }, + }, + }, + }, + }, + }, + }, + }, + } +} + +func TestIntDataPointSliceAt(t *testing.T) { + timestamp := time.Now().UnixNano() / int64(time.Millisecond) + instrLibName := "cloudwatch-otel" + labels := map[string]string{"label1": "value1"} + rateKeyParams := map[string]string{ + (namespaceKey): "namespace", + (metricNameKey): "foo", + (logGroupKey): "log-group", + (logStreamKey): "log-stream", + } + + testCases := []struct { + testName string + needsCalculateRate bool + value interface{} + }{ + { + "no rate calculation", + false, + int64(-17), + }, + { + "w/ rate calculation", + true, + int64(0), + }, + } + + for _, tc := range testCases { + t.Run(tc.testName, func(t *testing.T) { + testDPS := pdata.NewIntDataPointSlice() + testDPS.Resize(1) + testDP := testDPS.At(0) + testDP.SetValue(int64(-17)) + testDP.LabelsMap().InitFromMap(labels) + + dps := IntDataPointSlice{ + instrLibName, + rateCalculationMetadata{ + tc.needsCalculateRate, + rateKeyParams, + timestamp, + }, + testDPS, + } + + expectedDP := DataPoint{ + Value: tc.value, + Labels: map[string]string{ + (OTellibDimensionKey): instrLibName, + "label1": "value1", + }, + } + + assert.Equal(t, 1, dps.Len()) + dp := dps.At(0) + assert.Equal(t, expectedDP, dp) + }) + } +} + +func TestDoubleDataPointSliceAt(t *testing.T) { + timestamp := time.Now().UnixNano() / int64(time.Millisecond) + instrLibName := "cloudwatch-otel" + labels := map[string]string{"label1": "value1"} + rateKeyParams := map[string]string{ + (namespaceKey): "namespace", + (metricNameKey): "foo", + (logGroupKey): "log-group", + (logStreamKey): "log-stream", + } + + testCases := []struct { + testName string + needsCalculateRate bool + value interface{} + }{ + { + "no rate calculation", + false, + float64(0.3), + }, + { + "w/ rate calculation", + true, + float64(0), + }, + } + + for _, tc := range testCases { + t.Run(tc.testName, func(t *testing.T) { + testDPS := pdata.NewDoubleDataPointSlice() + testDPS.Resize(1) + testDP := testDPS.At(0) + testDP.SetValue(float64(0.3)) + testDP.LabelsMap().InitFromMap(labels) + + dps := DoubleDataPointSlice{ + instrLibName, + rateCalculationMetadata{ + tc.needsCalculateRate, + rateKeyParams, + timestamp, + }, + testDPS, + } + + expectedDP := DataPoint{ + Value: tc.value, + Labels: map[string]string{ + (OTellibDimensionKey): instrLibName, + "label1": "value1", + }, + } + + assert.Equal(t, 1, dps.Len()) + dp := dps.At(0) + assert.Equal(t, expectedDP, dp) + }) + } +} + +func TestDoubleHistogramDataPointSliceAt(t *testing.T) { + instrLibName := "cloudwatch-otel" + labels := map[string]string{"label1": "value1"} + + testDPS := pdata.NewDoubleHistogramDataPointSlice() + testDPS.Resize(1) + testDP := testDPS.At(0) + testDP.SetCount(uint64(17)) + testDP.SetSum(float64(17.13)) + testDP.SetBucketCounts([]uint64{1, 2, 3}) + testDP.SetExplicitBounds([]float64{1, 2, 3}) + testDP.LabelsMap().InitFromMap(labels) + + dps := DoubleHistogramDataPointSlice{ + instrLibName, + testDPS, + } + + expectedDP := DataPoint{ + Value: &CWMetricStats{ + Min: 1, + Max: 3, + Sum: 17.13, + Count: 17, + }, + Labels: map[string]string{ + (OTellibDimensionKey): instrLibName, + "label1": "value1", + }, + } + + assert.Equal(t, 1, dps.Len()) + dp := dps.At(0) + assert.Equal(t, expectedDP, dp) +} + +func TestDoubleSummaryDataPointSliceAt(t *testing.T) { + instrLibName := "cloudwatch-otel" + labels := map[string]string{"label1": "value1"} + + testDPS := pdata.NewDoubleSummaryDataPointSlice() + testDPS.Resize(1) + testDP := testDPS.At(0) + testDP.SetCount(uint64(17)) + testDP.SetSum(float64(17.13)) + testDP.QuantileValues().Resize(2) + testQuantileValue := testDP.QuantileValues().At(0) + testQuantileValue.SetQuantile(0) + testQuantileValue.SetValue(float64(1)) + testQuantileValue = testDP.QuantileValues().At(1) + testQuantileValue.SetQuantile(100) + testQuantileValue.SetValue(float64(5)) + testDP.LabelsMap().InitFromMap(labels) + + dps := DoubleSummaryDataPointSlice{ + instrLibName, + testDPS, + } + + expectedDP := DataPoint{ + Value: &CWMetricStats{ + Max: 5, + Min: 1, + Count: 17, + Sum: 17.13, + }, + Labels: map[string]string{ + (OTellibDimensionKey): instrLibName, + "label1": "value1", + }, + } + + assert.Equal(t, 1, dps.Len()) + dp := dps.At(0) + assert.Equal(t, expectedDP, dp) +} + +func TestCreateLabels(t *testing.T) { + expectedLabels := map[string]string{ + "a": "A", + "b": "B", + "c": "C", + } + labelsMap := pdata.NewStringMap().InitFromMap(expectedLabels) + + labels := createLabels(labelsMap, noInstrumentationLibraryName) + assert.Equal(t, expectedLabels, labels) + + // With isntrumentation library name + labels = createLabels(labelsMap, "cloudwatch-otel") + expectedLabels[OTellibDimensionKey] = "cloudwatch-otel" + assert.Equal(t, expectedLabels, labels) +} + +func TestCalculateRate(t *testing.T) { + intRateKey := "foo" + doubleRateKey := "bar" + time1 := time.Now().UnixNano() / int64(time.Millisecond) + time2 := time.Unix(0, time1*int64(time.Millisecond)).Add(time.Second*10).UnixNano() / int64(time.Millisecond) + time3 := time.Unix(0, time2*int64(time.Millisecond)).Add(time.Second*10).UnixNano() / int64(time.Millisecond) + + intVal1 := int64(0) + intVal2 := int64(10) + intVal3 := int64(200) + doubleVal1 := 0.0 + doubleVal2 := 5.0 + doubleVal3 := 15.1 + + rate := calculateRate(intRateKey, intVal1, time1) + assert.Equal(t, int64(0), rate) + rate = calculateRate(doubleRateKey, doubleVal1, time1) + assert.Equal(t, float64(0), rate) + + rate = calculateRate(intRateKey, intVal2, time2) + assert.Equal(t, int64(1), rate) + rate = calculateRate(doubleRateKey, doubleVal2, time2) + assert.Equal(t, 0.5, rate) + + // Test change of data type + rate = calculateRate(intRateKey, doubleVal3, time3) + assert.Equal(t, float64(0.51), rate) + rate = calculateRate(doubleRateKey, intVal3, time3) + assert.Equal(t, int64(19), rate) +} + +func TestGetDataPoints(t *testing.T) { + metadata := CWMetricMetadata{ + Namespace: "Namespace", + Timestamp: time.Now().UnixNano() / int64(time.Millisecond), + LogGroup: "log-group", + LogStream: "log-stream", + InstrumentationLibraryName: "cloudwatch-otel", + } + + testCases := []struct { + testName string + metric *metricspb.Metric + expectedDataPoints DataPoints + }{ + { + "Int gauge", + generateTestIntGauge("foo"), + IntDataPointSlice{ + metadata.InstrumentationLibraryName, + rateCalculationMetadata{ + false, + map[string]string{ + (namespaceKey): metadata.Namespace, + (metricNameKey): "foo", + (logGroupKey): metadata.LogGroup, + (logStreamKey): metadata.LogStream, + }, + metadata.Timestamp, + }, + pdata.IntDataPointSlice{}, + }, + }, + { + "Double gauge", + generateTestDoubleGauge("foo"), + DoubleDataPointSlice{ + metadata.InstrumentationLibraryName, + rateCalculationMetadata{ + false, + map[string]string{ + (namespaceKey): metadata.Namespace, + (metricNameKey): "foo", + (logGroupKey): metadata.LogGroup, + (logStreamKey): metadata.LogStream, + }, + metadata.Timestamp, + }, + pdata.DoubleDataPointSlice{}, + }, + }, + { + "Int sum", + generateTestIntSum("foo"), + IntDataPointSlice{ + metadata.InstrumentationLibraryName, + rateCalculationMetadata{ + true, + map[string]string{ + (namespaceKey): metadata.Namespace, + (metricNameKey): "foo", + (logGroupKey): metadata.LogGroup, + (logStreamKey): metadata.LogStream, + }, + metadata.Timestamp, + }, + pdata.IntDataPointSlice{}, + }, + }, + { + "Double sum", + generateTestDoubleSum("foo"), + DoubleDataPointSlice{ + metadata.InstrumentationLibraryName, + rateCalculationMetadata{ + true, + map[string]string{ + (namespaceKey): metadata.Namespace, + (metricNameKey): "foo", + (logGroupKey): metadata.LogGroup, + (logStreamKey): metadata.LogStream, + }, + metadata.Timestamp, + }, + pdata.DoubleDataPointSlice{}, + }, + }, + { + "Double histogram", + generateTestDoubleHistogram("foo"), + DoubleHistogramDataPointSlice{ + metadata.InstrumentationLibraryName, + pdata.DoubleHistogramDataPointSlice{}, + }, + }, + { + "Summary", + generateTestSummary("foo"), + DoubleSummaryDataPointSlice{ + metadata.InstrumentationLibraryName, + pdata.DoubleSummaryDataPointSlice{}, + }, + }, + } + + for _, tc := range testCases { + oc := consumerdata.MetricsData{ + Metrics: []*metricspb.Metric{tc.metric}, + } + + // Retrieve *pdata.Metric + rm := internaldata.OCToMetrics(oc).ResourceMetrics().At(0) + metric := rm.InstrumentationLibraryMetrics().At(0).Metrics().At(0) + + logger := zap.NewNop() + + expectedLabels := pdata.NewStringMap().InitFromMap(map[string]string{"label1": "value1"}) + + t.Run(tc.testName, func(t *testing.T) { + dps := getDataPoints(&metric, metadata, logger) + assert.NotNil(t, dps) + assert.Equal(t, reflect.TypeOf(tc.expectedDataPoints), reflect.TypeOf(dps)) + switch convertedDPS := dps.(type) { + case IntDataPointSlice: + expectedDPS := tc.expectedDataPoints.(IntDataPointSlice) + assert.Equal(t, metadata.InstrumentationLibraryName, convertedDPS.instrumentationLibraryName) + assert.Equal(t, expectedDPS.rateCalculationMetadata, convertedDPS.rateCalculationMetadata) + assert.Equal(t, 1, convertedDPS.Len()) + dp := convertedDPS.IntDataPointSlice.At(0) + assert.Equal(t, int64(1), dp.Value()) + assert.Equal(t, expectedLabels, dp.LabelsMap()) + case DoubleDataPointSlice: + expectedDPS := tc.expectedDataPoints.(DoubleDataPointSlice) + assert.Equal(t, metadata.InstrumentationLibraryName, convertedDPS.instrumentationLibraryName) + assert.Equal(t, expectedDPS.rateCalculationMetadata, convertedDPS.rateCalculationMetadata) + assert.Equal(t, 1, convertedDPS.Len()) + dp := convertedDPS.DoubleDataPointSlice.At(0) + assert.Equal(t, 0.1, dp.Value()) + assert.Equal(t, expectedLabels, dp.LabelsMap()) + case DoubleHistogramDataPointSlice: + assert.Equal(t, metadata.InstrumentationLibraryName, convertedDPS.instrumentationLibraryName) + assert.Equal(t, 1, convertedDPS.Len()) + dp := convertedDPS.DoubleHistogramDataPointSlice.At(0) + assert.Equal(t, 35.0, dp.Sum()) + assert.Equal(t, uint64(18), dp.Count()) + assert.Equal(t, []float64{0, 10}, dp.ExplicitBounds()) + assert.Equal(t, expectedLabels, dp.LabelsMap()) + case DoubleSummaryDataPointSlice: + assert.Equal(t, metadata.InstrumentationLibraryName, convertedDPS.instrumentationLibraryName) + assert.Equal(t, 1, convertedDPS.Len()) + dp := convertedDPS.DoubleSummaryDataPointSlice.At(0) + assert.Equal(t, 15.0, dp.Sum()) + assert.Equal(t, uint64(5), dp.Count()) + assert.Equal(t, 2, dp.QuantileValues().Len()) + assert.Equal(t, float64(1), dp.QuantileValues().At(0).Value()) + assert.Equal(t, float64(5), dp.QuantileValues().At(1).Value()) + } + }) + } + + t.Run("Unhandled metric type", func(t *testing.T) { + metric := pdata.NewMetric() + metric.SetName("foo") + metric.SetUnit("Count") + metric.SetDataType(pdata.MetricDataTypeIntHistogram) + + obs, logs := observer.New(zap.WarnLevel) + logger := zap.New(obs) + + dps := getDataPoints(&metric, metadata, logger) + assert.Nil(t, dps) + + // Test output warning logs + expectedLogs := []observer.LoggedEntry{ + { + Entry: zapcore.Entry{Level: zap.WarnLevel, Message: "Unhandled metric data type."}, + Context: []zapcore.Field{ + zap.String("DataType", "IntHistogram"), + zap.String("Name", "foo"), + zap.String("Unit", "Count"), + }, + }, + } + assert.Equal(t, 1, logs.Len()) + assert.Equal(t, expectedLogs, logs.AllUntimed()) + }) + + t.Run("Nil metric", func(t *testing.T) { + dps := getDataPoints(nil, metadata, zap.NewNop()) + assert.Nil(t, dps) + }) +} + +func BenchmarkGetDataPoints(b *testing.B) { + oc := consumerdata.MetricsData{ + Metrics: []*metricspb.Metric{ + generateTestIntGauge("int-gauge"), + generateTestDoubleGauge("double-gauge"), + generateTestIntSum("int-sum"), + generateTestDoubleSum("double-sum"), + generateTestDoubleHistogram("double-histogram"), + generateTestSummary("summary"), + }, + } + rms := internaldata.OCToMetrics(oc).ResourceMetrics() + metrics := rms.At(0).InstrumentationLibraryMetrics().At(0).Metrics() + numMetrics := metrics.Len() + + metadata := CWMetricMetadata{ + Namespace: "Namespace", + Timestamp: int64(1596151098037), + LogGroup: "log-group", + LogStream: "log-stream", + InstrumentationLibraryName: "cloudwatch-otel", + } + + logger := zap.NewNop() + + b.ResetTimer() + for n := 0; n < b.N; n++ { + for i := 0; i < numMetrics; i++ { + metric := metrics.At(i) + getDataPoints(&metric, metadata, logger) + } + } +} diff --git a/exporter/awsemfexporter/emf_exporter.go b/exporter/awsemfexporter/emf_exporter.go index ed8d8c62ca71e..89d188cc26e12 100644 --- a/exporter/awsemfexporter/emf_exporter.go +++ b/exporter/awsemfexporter/emf_exporter.go @@ -112,63 +112,44 @@ func NewEmfExporter( ) } -func (emf *emfExporter) pushMetricsData(_ context.Context, md pdata.Metrics) (droppedTimeSeries int, err error) { - var cwm []*CWMetrics - var totalDroppedMetrics int - var droppedMetrics int +func (emf *emfExporter) pushMetricsData(_ context.Context, md pdata.Metrics) (totalDroppedMetrics int, err error) { + groupedMetrics := make(map[string]*GroupedMetric) expConfig := emf.config.(*Config) - namespace := expConfig.Namespace - logGroup := "/metrics/default" - logStream := fmt.Sprintf("otel-stream-%s", emf.collectorID) + defaultLogStream := fmt.Sprintf("otel-stream-%s", emf.collectorID) rms := md.ResourceMetrics() for i := 0; i < rms.Len(); i++ { - droppedMetrics = 0 rm := rms.At(i) - cwm, droppedMetrics = TranslateOtToCWMetric(&rm, expConfig) - totalDroppedMetrics = totalDroppedMetrics + droppedMetrics - - if len(cwm) > 0 && len(cwm[0].Measurements) > 0 { - namespace = cwm[0].Measurements[0].Namespace - } - - putLogEvents := TranslateCWMetricToEMF(cwm, expConfig.logger) + TranslateOtToGroupedMetric(&rm, groupedMetrics, expConfig) + } - if namespace != "" { - logGroup = fmt.Sprintf("/metrics/%s", namespace) - } + for _, groupedMetric := range groupedMetrics { + cWMetric := TranslateGroupedMetricToCWMetric(groupedMetric, expConfig) + putLogEvent := TranslateCWMetricToEMF(cWMetric) - // override log group if found it in exp configuration, this configuration has top priority. - // However, in this case, customer won't have correlation experience - if len(expConfig.LogGroupName) > 0 { - logGroup = replacePatterns(expConfig.LogGroupName, rm.Resource().Attributes(), expConfig.logger) - } - if len(expConfig.LogStreamName) > 0 { - logStream = replacePatterns(expConfig.LogStreamName, rm.Resource().Attributes(), expConfig.logger) + logGroup := groupedMetric.Metadata.LogGroup + logStream := groupedMetric.Metadata.LogStream + if logStream == "" { + logStream = defaultLogStream } pusher := emf.getPusher(logGroup, logStream) if pusher != nil { - for _, ple := range putLogEvents { - returnError := pusher.AddLogEntry(ple) - if returnError != nil { - err = wrapErrorIfBadRequest(&returnError) - } - if err != nil { - return totalDroppedMetrics, err - } - } - returnError := pusher.ForceFlush() + returnError := pusher.AddLogEntry(putLogEvent) if returnError != nil { err = wrapErrorIfBadRequest(&returnError) + return } - if err != nil { - return totalDroppedMetrics, err + returnError = pusher.ForceFlush() + if returnError != nil { + err = wrapErrorIfBadRequest(&returnError) + return } } } - return totalDroppedMetrics, nil + + return } func (emf *emfExporter) getPusher(logGroup, logStream string) Pusher { diff --git a/exporter/awsemfexporter/emf_exporter_test.go b/exporter/awsemfexporter/emf_exporter_test.go index 9f34a7e4e0a91..14bb832efd733 100644 --- a/exporter/awsemfexporter/emf_exporter_test.go +++ b/exporter/awsemfexporter/emf_exporter_test.go @@ -189,6 +189,68 @@ func TestConsumeMetricsWithLogGroupStreamConfig(t *testing.T) { assert.NotNil(t, pusher) } +func TestConsumeMetricsWithoutLogGroupStreamConfig(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + factory := NewFactory() + expCfg := factory.CreateDefaultConfig().(*Config) + expCfg.Region = "us-west-2" + expCfg.MaxRetries = defaultRetryCount + exp, err := New(expCfg, component.ExporterCreateParams{Logger: zap.NewNop()}) + assert.Nil(t, err) + assert.NotNil(t, exp) + + mdata := consumerdata.MetricsData{ + Node: &commonpb.Node{}, + Resource: &resourcepb.Resource{ + Labels: map[string]string{ + "resource": "R1", + }, + }, + Metrics: []*metricspb.Metric{ + { + MetricDescriptor: &metricspb.MetricDescriptor{ + Name: "spanCounter", + Description: "Counting all the spans", + Unit: "Count", + Type: metricspb.MetricDescriptor_CUMULATIVE_INT64, + LabelKeys: []*metricspb.LabelKey{ + {Key: "spanName"}, + {Key: "isItAnError"}, + }, + }, + Timeseries: []*metricspb.TimeSeries{ + { + LabelValues: []*metricspb.LabelValue{ + {Value: "testSpan"}, + {Value: "false"}, + }, + Points: []*metricspb.Point{ + { + Timestamp: ×tamp.Timestamp{ + Seconds: 100, + }, + Value: &metricspb.Point_Int64Value{ + Int64Value: 1, + }, + }, + }, + }, + }, + }, + }, + } + md := internaldata.OCToMetrics(mdata) + require.NoError(t, exp.Start(ctx, nil)) + require.Error(t, exp.ConsumeMetrics(ctx, md)) + require.NoError(t, exp.Shutdown(ctx)) + streamToPusherMap, ok := exp.(*emfExporter).groupStreamToPusherMap["/metrics/default"] + assert.True(t, ok) + pusher, ok := streamToPusherMap["otel-stream-"+exp.(*emfExporter).collectorID] + assert.True(t, ok) + assert.NotNil(t, pusher) +} + func TestConsumeMetricsWithLogGroupStreamValidPlaceholder(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -334,8 +396,8 @@ func TestConsumeMetricsWithWrongPlaceholder(t *testing.T) { expCfg := factory.CreateDefaultConfig().(*Config) expCfg.Region = "us-west-2" expCfg.MaxRetries = defaultRetryCount - expCfg.LogGroupName = "test-logGroupName" - expCfg.LogStreamName = "{WrongKey}" + expCfg.LogGroupName = "{WrongGroupKey}" + expCfg.LogStreamName = "{WrongStreamKey}" exp, err := New(expCfg, component.ExporterCreateParams{Logger: zap.NewNop()}) assert.Nil(t, err) assert.NotNil(t, exp) @@ -389,9 +451,9 @@ func TestConsumeMetricsWithWrongPlaceholder(t *testing.T) { require.NoError(t, exp.Start(ctx, nil)) require.Error(t, exp.ConsumeMetrics(ctx, md)) require.NoError(t, exp.Shutdown(ctx)) - streamToPusherMap, ok := exp.(*emfExporter).groupStreamToPusherMap["test-logGroupName"] + streamToPusherMap, ok := exp.(*emfExporter).groupStreamToPusherMap["{WrongGroupKey}"] assert.True(t, ok) - pusher, ok := streamToPusherMap["{WrongKey}"] + pusher, ok := streamToPusherMap["{WrongStreamKey}"] assert.True(t, ok) assert.NotNil(t, pusher) } @@ -486,7 +548,6 @@ func TestNewExporterWithoutConfig(t *testing.T) { assert.Nil(t, exp) assert.NotNil(t, expCfg.logger) } - func TestNewExporterWithMetricDeclarations(t *testing.T) { factory := NewFactory() expCfg := factory.CreateDefaultConfig().(*Config) diff --git a/exporter/awsemfexporter/groupedmetric.go b/exporter/awsemfexporter/groupedmetric.go new file mode 100644 index 0000000000000..c3dc79308dbdb --- /dev/null +++ b/exporter/awsemfexporter/groupedmetric.go @@ -0,0 +1,93 @@ +// Copyright 2020, OpenTelemetry Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package awsemfexporter + +import ( + "strconv" + + "go.opentelemetry.io/collector/consumer/pdata" + "go.uber.org/zap" +) + +const ( + namespaceKey = "CloudWatchNamespace" + timestampKey = "CloudWatchTimestamp" + metricNameKey = "CloudWatchMetricName" + logGroupKey = "CloudWatchLogGroup" + logStreamKey = "CloudWatchLogStream" +) + +// GroupedMetric defines set of metrics with same namespace, timestamp and labels +type GroupedMetric struct { + Labels map[string]string + Metrics map[string]*MetricInfo + Metadata CWMetricMetadata +} + +// MetricInfo defines value and unit for OT Metrics +type MetricInfo struct { + Value interface{} + Unit string +} + +// addToGroupedMetric processes OT metrics and adds them into GroupedMetric buckets +func addToGroupedMetric(pmd *pdata.Metric, groupedMetrics map[string]*GroupedMetric, metadata CWMetricMetadata, logger *zap.Logger) { + if pmd == nil { + return + } + + metricName := pmd.Name() + dps := getDataPoints(pmd, metadata, logger) + if dps == nil || dps.Len() == 0 { + return + } + + for i := 0; i < dps.Len(); i++ { + dp := dps.At(i) + labels := dp.Labels + metric := &MetricInfo{ + Value: dp.Value, + Unit: pmd.Unit(), + } + + // Extra params to use when grouping metrics + groupKeyParams := map[string]string{ + (namespaceKey): metadata.Namespace, + (timestampKey): strconv.FormatInt(metadata.Timestamp, 10), + (logGroupKey): metadata.LogGroup, + (logStreamKey): metadata.LogStream, + } + + groupKey := createMetricKey(labels, groupKeyParams) + if _, ok := groupedMetrics[groupKey]; ok { + // if metricName already exists in metrics map, print warning log + if _, ok := groupedMetrics[groupKey].Metrics[metricName]; ok { + logger.Warn( + "Duplicate metric found", + zap.String("Name", metricName), + zap.Any("Labels", labels), + ) + } else { + groupedMetrics[groupKey].Metrics[metricName] = metric + } + } else { + groupedMetrics[groupKey] = &GroupedMetric{ + Labels: labels, + Metrics: map[string]*MetricInfo{(metricName): metric}, + Metadata: metadata, + } + } + } +} diff --git a/exporter/awsemfexporter/groupedmetric_test.go b/exporter/awsemfexporter/groupedmetric_test.go new file mode 100644 index 0000000000000..4f379b2b5ef5a --- /dev/null +++ b/exporter/awsemfexporter/groupedmetric_test.go @@ -0,0 +1,390 @@ +// Copyright 2020, OpenTelemetry Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package awsemfexporter + +import ( + "testing" + "time" + + commonpb "github.com/census-instrumentation/opencensus-proto/gen-go/agent/common/v1" + metricspb "github.com/census-instrumentation/opencensus-proto/gen-go/metrics/v1" + resourcepb "github.com/census-instrumentation/opencensus-proto/gen-go/resource/v1" + "github.com/stretchr/testify/assert" + "go.opentelemetry.io/collector/consumer/consumerdata" + "go.opentelemetry.io/collector/consumer/pdata" + "go.opentelemetry.io/collector/translator/conventions" + "go.opentelemetry.io/collector/translator/internaldata" + "go.uber.org/zap" + "go.uber.org/zap/zapcore" + "go.uber.org/zap/zaptest/observer" +) + +func TestAddToGroupedMetric(t *testing.T) { + namespace := "namespace" + instrumentationLibName := "cloudwatch-otel" + timestamp := time.Now().UnixNano() / int64(time.Millisecond) + logger := zap.NewNop() + + metadata := CWMetricMetadata{ + Namespace: namespace, + Timestamp: timestamp, + LogGroup: logGroup, + LogStream: logStreamName, + InstrumentationLibraryName: instrumentationLibName, + } + + testCases := []struct { + testName string + metric *metricspb.Metric + expected map[string]*MetricInfo + }{ + { + "Int gauge", + generateTestIntGauge("foo"), + map[string]*MetricInfo{ + "foo": { + Value: int64(1), + Unit: "Count", + }, + }, + }, + { + "Double gauge", + generateTestDoubleGauge("foo"), + map[string]*MetricInfo{ + "foo": { + Value: 0.1, + Unit: "Count", + }, + }, + }, + { + "Int sum", + generateTestIntSum("foo"), + map[string]*MetricInfo{ + "foo": { + Value: int64(0), + Unit: "Count", + }, + }, + }, + { + "Double sum", + generateTestDoubleSum("foo"), + map[string]*MetricInfo{ + "foo": { + Value: float64(0), + Unit: "Count", + }, + }, + }, + { + "Double histogram", + generateTestDoubleHistogram("foo"), + map[string]*MetricInfo{ + "foo": { + Value: &CWMetricStats{ + Min: 0, + Max: 10, + Count: 18, + Sum: 35.0, + }, + Unit: "Seconds", + }, + }, + }, + { + "Summary", + generateTestSummary("foo"), + map[string]*MetricInfo{ + "foo": { + Value: &CWMetricStats{ + Min: 1, + Max: 5, + Count: 5, + Sum: 15, + }, + Unit: "Seconds", + }, + }, + }, + } + + for _, tc := range testCases { + t.Run(tc.testName, func(t *testing.T) { + groupedMetrics := make(map[string]*GroupedMetric) + oc := consumerdata.MetricsData{ + Node: &commonpb.Node{}, + Resource: &resourcepb.Resource{ + Labels: map[string]string{ + conventions.AttributeServiceName: "myServiceName", + conventions.AttributeServiceNamespace: "myServiceNS", + }, + }, + Metrics: []*metricspb.Metric{tc.metric}, + } + + // Retrieve *pdata.Metric + rm := internaldata.OCToMetrics(oc) + rms := rm.ResourceMetrics() + assert.Equal(t, 1, rms.Len()) + ilms := rms.At(0).InstrumentationLibraryMetrics() + assert.Equal(t, 1, ilms.Len()) + metrics := ilms.At(0).Metrics() + assert.Equal(t, 1, metrics.Len()) + metric := metrics.At(0) + + addToGroupedMetric(&metric, groupedMetrics, metadata, zap.NewNop()) + expectedLabels := map[string]string{ + (OTellibDimensionKey): instrumentationLibName, + "label1": "value1", + } + + for _, v := range groupedMetrics { + assert.Equal(t, len(tc.expected), len(v.Metrics)) + assert.Equal(t, tc.expected, v.Metrics) + assert.Equal(t, 2, len(v.Labels)) + assert.Equal(t, metadata, v.Metadata) + assert.Equal(t, expectedLabels, v.Labels) + } + }) + } + + t.Run("Add multiple different metrics", func(t *testing.T) { + groupedMetrics := make(map[string]*GroupedMetric) + oc := consumerdata.MetricsData{ + Node: &commonpb.Node{}, + Resource: &resourcepb.Resource{ + Labels: map[string]string{ + conventions.AttributeServiceName: "myServiceName", + conventions.AttributeServiceNamespace: "myServiceNS", + }, + }, + Metrics: []*metricspb.Metric{ + generateTestIntGauge("int-gauge"), + generateTestDoubleGauge("double-gauge"), + generateTestIntSum("int-sum"), + generateTestDoubleSum("double-sum"), + generateTestDoubleHistogram("double-histogram"), + generateTestSummary("summary"), + }, + } + rm := internaldata.OCToMetrics(oc) + rms := rm.ResourceMetrics() + ilms := rms.At(0).InstrumentationLibraryMetrics() + metrics := ilms.At(0).Metrics() + assert.Equal(t, 6, metrics.Len()) + + for i := 0; i < metrics.Len(); i++ { + metric := metrics.At(i) + addToGroupedMetric(&metric, groupedMetrics, metadata, logger) + } + + assert.Equal(t, 1, len(groupedMetrics)) + for _, group := range groupedMetrics { + assert.Equal(t, 6, len(group.Metrics)) + for metricName, metricInfo := range group.Metrics { + if metricName == "double-histogram" || metricName == "summary" { + assert.Equal(t, "Seconds", metricInfo.Unit) + } else { + assert.Equal(t, "Count", metricInfo.Unit) + } + } + expectedLabels := map[string]string{ + (OTellibDimensionKey): "cloudwatch-otel", + "label1": "value1", + } + assert.Equal(t, expectedLabels, group.Labels) + assert.Equal(t, metadata, group.Metadata) + } + }) + + t.Run("Add same metric but different log group", func(t *testing.T) { + groupedMetrics := make(map[string]*GroupedMetric) + oc := consumerdata.MetricsData{ + Metrics: []*metricspb.Metric{ + generateTestIntGauge("int-gauge"), + }, + } + rm := internaldata.OCToMetrics(oc) + ilms := rm.ResourceMetrics().At(0).InstrumentationLibraryMetrics() + metric := ilms.At(0).Metrics().At(0) + + metricMetadata1 := CWMetricMetadata{ + Namespace: namespace, + Timestamp: timestamp, + LogGroup: "log-group-1", + LogStream: logStreamName, + InstrumentationLibraryName: instrumentationLibName, + } + addToGroupedMetric(&metric, groupedMetrics, metricMetadata1, logger) + + metricMetadata2 := CWMetricMetadata{ + Namespace: namespace, + Timestamp: timestamp, + LogGroup: "log-group-2", + LogStream: logStreamName, + InstrumentationLibraryName: instrumentationLibName, + } + addToGroupedMetric(&metric, groupedMetrics, metricMetadata2, logger) + + assert.Equal(t, 2, len(groupedMetrics)) + seenLogGroup1 := false + seenLogGroup2 := false + for _, group := range groupedMetrics { + assert.Equal(t, 1, len(group.Metrics)) + expectedMetrics := map[string]*MetricInfo{ + "int-gauge": { + Value: int64(1), + Unit: "Count", + }, + } + assert.Equal(t, expectedMetrics, group.Metrics) + expectedLabels := map[string]string{ + (OTellibDimensionKey): "cloudwatch-otel", + "label1": "value1", + } + assert.Equal(t, expectedLabels, group.Labels) + + if group.Metadata.LogGroup == "log-group-2" { + seenLogGroup2 = true + } else if group.Metadata.LogGroup == "log-group-1" { + seenLogGroup1 = true + } + } + assert.True(t, seenLogGroup1) + assert.True(t, seenLogGroup2) + }) + + t.Run("Duplicate metric names", func(t *testing.T) { + groupedMetrics := make(map[string]*GroupedMetric) + oc := consumerdata.MetricsData{ + Resource: &resourcepb.Resource{ + Labels: map[string]string{ + conventions.AttributeServiceName: "myServiceName", + conventions.AttributeServiceNamespace: "myServiceNS", + }, + }, + Metrics: []*metricspb.Metric{ + generateTestIntGauge("foo"), + generateTestDoubleGauge("foo"), + }, + } + rm := internaldata.OCToMetrics(oc) + rms := rm.ResourceMetrics() + ilms := rms.At(0).InstrumentationLibraryMetrics() + metrics := ilms.At(0).Metrics() + assert.Equal(t, 2, metrics.Len()) + + obs, logs := observer.New(zap.WarnLevel) + obsLogger := zap.New(obs) + + for i := 0; i < metrics.Len(); i++ { + metric := metrics.At(i) + addToGroupedMetric(&metric, groupedMetrics, metadata, obsLogger) + } + assert.Equal(t, 1, len(groupedMetrics)) + + labels := map[string]string{ + (OTellibDimensionKey): instrumentationLibName, + "label1": "value1", + } + // Test output warning logs + expectedLogs := []observer.LoggedEntry{ + { + Entry: zapcore.Entry{Level: zap.WarnLevel, Message: "Duplicate metric found"}, + Context: []zapcore.Field{ + zap.String("Name", "foo"), + zap.Any("Labels", labels), + }, + }, + } + assert.Equal(t, 1, logs.Len()) + assert.Equal(t, expectedLogs, logs.AllUntimed()) + }) + + t.Run("Unhandled metric type", func(t *testing.T) { + groupedMetrics := make(map[string]*GroupedMetric) + md := pdata.NewMetrics() + rms := md.ResourceMetrics() + rms.Resize(1) + rms.At(0).InstrumentationLibraryMetrics().Resize(1) + rms.At(0).InstrumentationLibraryMetrics().At(0).Metrics().Resize(1) + metric := rms.At(0).InstrumentationLibraryMetrics().At(0).Metrics().At(0) + metric.SetName("foo") + metric.SetUnit("Count") + metric.SetDataType(pdata.MetricDataTypeIntHistogram) + + obs, logs := observer.New(zap.WarnLevel) + obsLogger := zap.New(obs) + addToGroupedMetric(&metric, groupedMetrics, metadata, obsLogger) + assert.Equal(t, 0, len(groupedMetrics)) + + // Test output warning logs + expectedLogs := []observer.LoggedEntry{ + { + Entry: zapcore.Entry{Level: zap.WarnLevel, Message: "Unhandled metric data type."}, + Context: []zapcore.Field{ + zap.String("DataType", "IntHistogram"), + zap.String("Name", "foo"), + zap.String("Unit", "Count"), + }, + }, + } + assert.Equal(t, 1, logs.Len()) + assert.Equal(t, expectedLogs, logs.AllUntimed()) + }) + + t.Run("Nil metric", func(t *testing.T) { + groupedMetrics := make(map[string]*GroupedMetric) + addToGroupedMetric(nil, groupedMetrics, metadata, logger) + assert.Equal(t, 0, len(groupedMetrics)) + }) +} + +func BenchmarkAddToGroupedMetric(b *testing.B) { + oc := consumerdata.MetricsData{ + Metrics: []*metricspb.Metric{ + generateTestIntGauge("int-gauge"), + generateTestDoubleGauge("double-gauge"), + generateTestIntSum("int-sum"), + generateTestDoubleSum("double-sum"), + generateTestDoubleHistogram("double-histogram"), + generateTestSummary("summary"), + }, + } + rms := internaldata.OCToMetrics(oc).ResourceMetrics() + metrics := rms.At(0).InstrumentationLibraryMetrics().At(0).Metrics() + numMetrics := metrics.Len() + + metadata := CWMetricMetadata{ + Namespace: "Namespace", + Timestamp: int64(1596151098037), + LogGroup: "log-group", + LogStream: "log-stream", + InstrumentationLibraryName: "cloudwatch-otel", + } + + logger := zap.NewNop() + + b.ResetTimer() + for n := 0; n < b.N; n++ { + groupedMetrics := make(map[string]*GroupedMetric) + for i := 0; i < numMetrics; i++ { + metric := metrics.At(i) + addToGroupedMetric(&metric, groupedMetrics, metadata, logger) + } + } +} diff --git a/exporter/awsemfexporter/metric_declaration.go b/exporter/awsemfexporter/metric_declaration.go index 5ce8cfcbe0922..c72248f8abe1a 100644 --- a/exporter/awsemfexporter/metric_declaration.go +++ b/exporter/awsemfexporter/metric_declaration.go @@ -21,7 +21,6 @@ import ( "sort" "strings" - "go.opentelemetry.io/collector/consumer/pdata" "go.uber.org/zap" ) @@ -129,21 +128,20 @@ func (m *MetricDeclaration) Init(logger *zap.Logger) (err error) { return } -// Matches returns true if the given OTLP Metric's name matches any of the Metric -// Declaration's metric name selectors and label matchers. -func (m *MetricDeclaration) Matches(metric *pdata.Metric, labels map[string]string) bool { - // Check if metric matches at least one of the metric name selectors - hasMetricNameMatch := false +// MatchesName returns true if the given OTLP Metric's name matches any of the Metric +// Declaration's metric name selectors. +func (m *MetricDeclaration) MatchesName(metricName string) bool { for _, regex := range m.metricRegexList { - if regex.MatchString(metric.Name()) { - hasMetricNameMatch = true - break + if regex.MatchString(metricName) { + return true } } - if !hasMetricNameMatch { - return false - } + return false +} +// MatchesName returns true if the given OTLP Metric's name matches any of the Metric +// Declaration's label matchers. +func (m *MetricDeclaration) MatchesLabels(labels map[string]string) bool { if len(m.LabelMatchers) == 0 { return true } @@ -158,8 +156,8 @@ func (m *MetricDeclaration) Matches(metric *pdata.Metric, labels map[string]stri return false } -// ExtractDimensions extracts dimensions within the MetricDeclaration that only -// contains labels found in `labels`. Returned order of dimensions are preserved. +// ExtractDimensions filters through the dimensions defined in the given metric declaration and +// returns dimensions that only contains labels from in the given label set. func (m *MetricDeclaration) ExtractDimensions(labels map[string]string) (dimensions [][]string) { for _, dimensionSet := range m.Dimensions { if len(dimensionSet) == 0 { @@ -216,37 +214,3 @@ func (lm *LabelMatcher) getConcatenatedLabels(labels map[string]string) string { } return buf.String() } - -// processMetricDeclarations processes a list of MetricDeclarations and creates a -// list of dimension sets that matches the given `metric`. This list is then aggregated -// together with the rolled-up dimensions. Returned dimension sets -// are deduped and the dimensions in each dimension set are sorted. -// Prerequisite: -// 1. metricDeclarations' dimensions are sorted. -func processMetricDeclarations(metricDeclarations []*MetricDeclaration, metric *pdata.Metric, - labels map[string]string, rolledUpDimensions [][]string) (dimensions [][]string) { - seen := make(map[string]bool) - addDimSet := func(dimSet []string) { - key := strings.Join(dimSet, ",") - // Only add dimension set if not a duplicate - if _, ok := seen[key]; !ok { - dimensions = append(dimensions, dimSet) - seen[key] = true - } - } - // Extract and append dimensions from metric declarations - for _, m := range metricDeclarations { - if m.Matches(metric, labels) { - extractedDims := m.ExtractDimensions(labels) - for _, dimSet := range extractedDims { - addDimSet(dimSet) - } - } - } - // Add on rolled-up dimensions - for _, dimSet := range rolledUpDimensions { - sort.Strings(dimSet) - addDimSet(dimSet) - } - return -} diff --git a/exporter/awsemfexporter/metric_declaration_test.go b/exporter/awsemfexporter/metric_declaration_test.go index 2118bc99e5ea6..fb8841c66372c 100644 --- a/exporter/awsemfexporter/metric_declaration_test.go +++ b/exporter/awsemfexporter/metric_declaration_test.go @@ -18,7 +18,6 @@ import ( "testing" "github.com/stretchr/testify/assert" - "go.opentelemetry.io/collector/consumer/pdata" "go.uber.org/zap" "go.uber.org/zap/zapcore" "go.uber.org/zap/zaptest/observer" @@ -375,7 +374,7 @@ func TestMetricDeclarationInit(t *testing.T) { }) } -func TestMetricDeclarationMatches(t *testing.T) { +func TestMetricDeclarationMatchesName(t *testing.T) { m := &MetricDeclaration{ MetricNameSelectors: []string{"^a+$", "^b.*$", "^ac+a$"}, } @@ -383,36 +382,18 @@ func TestMetricDeclarationMatches(t *testing.T) { err := m.Init(logger) assert.Nil(t, err) - metric := pdata.NewMetric() - metric.SetName("a") - assert.True(t, m.Matches(&metric, nil)) - - metric.SetName("aa") - assert.True(t, m.Matches(&metric, nil)) - - metric.SetName("aaaa") - assert.True(t, m.Matches(&metric, nil)) - - metric.SetName("aaab") - assert.False(t, m.Matches(&metric, nil)) - - metric.SetName("b") - assert.True(t, m.Matches(&metric, nil)) - - metric.SetName("ba") - assert.True(t, m.Matches(&metric, nil)) - - metric.SetName("c") - assert.False(t, m.Matches(&metric, nil)) - - metric.SetName("aca") - assert.True(t, m.Matches(&metric, nil)) - - metric.SetName("accca") - assert.True(t, m.Matches(&metric, nil)) + assert.True(t, m.MatchesName("a")) + assert.True(t, m.MatchesName("aa")) + assert.True(t, m.MatchesName("aaaa")) + assert.False(t, m.MatchesName("aaab")) + assert.True(t, m.MatchesName("b")) + assert.True(t, m.MatchesName("ba")) + assert.False(t, m.MatchesName("c")) + assert.True(t, m.MatchesName("aca")) + assert.True(t, m.MatchesName("accca")) } -func TestMetricDeclarationMatchesWithLabelMatchers(t *testing.T) { +func TestMetricDeclarationMatchesLabels(t *testing.T) { labels := map[string]string{ "label1": "foo", "label2": "bar", @@ -514,8 +495,6 @@ func TestMetricDeclarationMatchesWithLabelMatchers(t *testing.T) { }, } logger := zap.NewNop() - metric := pdata.NewMetric() - metric.SetName("a") for _, tc := range testCases { m := MetricDeclaration{ @@ -525,7 +504,7 @@ func TestMetricDeclarationMatchesWithLabelMatchers(t *testing.T) { t.Run(tc.testName, func(t *testing.T) { err := m.Init(logger) assert.Nil(t, err) - matches := m.Matches(&metric, labels) + matches := m.MatchesLabels(labels) assert.Equal(t, tc.expected, matches) }) } @@ -613,155 +592,3 @@ func TestExtractDimensions(t *testing.T) { }) } } - -func TestProcessMetricDeclarations(t *testing.T) { - metricDeclarations := []*MetricDeclaration{ - { - Dimensions: [][]string{{"dim1", "dim2"}}, - MetricNameSelectors: []string{"a", "b", "c"}, - }, - { - Dimensions: [][]string{{"dim1"}}, - MetricNameSelectors: []string{"aa", "b"}, - }, - { - Dimensions: [][]string{{"dim2", "dim1"}, {"dim1"}}, - MetricNameSelectors: []string{"a"}, - }, - } - logger := zap.NewNop() - for _, m := range metricDeclarations { - err := m.Init(logger) - assert.Nil(t, err) - } - testCases := []struct { - testName string - metricName string - labels map[string]string - rollUpDims [][]string - expectedDims [][]string - }{ - { - "Matching single declaration", - "c", - map[string]string{ - "dim1": "foo", - "dim2": "bar", - }, - nil, - [][]string{ - {"dim1", "dim2"}, - }, - }, - { - "Match single dimension set", - "a", - map[string]string{ - "dim1": "foo", - }, - nil, - [][]string{ - {"dim1"}, - }, - }, - { - "Match single dimension set w/ rolled-up dims", - "a", - map[string]string{ - "dim1": "foo", - "dim3": "car", - }, - [][]string{{"dim1"}, {"dim3"}}, - [][]string{ - {"dim1"}, - {"dim3"}, - }, - }, - { - "Matching multiple declarations", - "b", - map[string]string{ - "dim1": "foo", - "dim2": "bar", - }, - nil, - [][]string{ - {"dim1", "dim2"}, - {"dim1"}, - }, - }, - { - "Matching multiple declarations w/ duplicate", - "a", - map[string]string{ - "dim1": "foo", - "dim2": "bar", - }, - nil, - [][]string{ - {"dim1", "dim2"}, - {"dim1"}, - }, - }, - { - "Matching multiple declarations w/ duplicate + rolled-up dims", - "a", - map[string]string{ - "dim1": "foo", - "dim2": "bar", - "dim3": "car", - }, - [][]string{{"dim2", "dim1"}, {"dim3"}}, - [][]string{ - {"dim1", "dim2"}, - {"dim1"}, - {"dim3"}, - }, - }, - { - "No matching dimension set", - "a", - map[string]string{ - "dim2": "bar", - }, - nil, - nil, - }, - { - "No matching dimension set w/ rolled-up dims", - "a", - map[string]string{ - "dim2": "bar", - }, - [][]string{{"dim2"}}, - [][]string{{"dim2"}}, - }, - { - "No matching metric name", - "c", - map[string]string{ - "dim1": "foo", - }, - nil, - nil, - }, - { - "No matching metric name w/ rolled-up dims", - "c", - map[string]string{ - "dim1": "foo", - }, - [][]string{{"dim1"}}, - [][]string{{"dim1"}}, - }, - } - - for _, tc := range testCases { - metric := pdata.NewMetric() - metric.SetName(tc.metricName) - t.Run(tc.testName, func(t *testing.T) { - dimensions := processMetricDeclarations(metricDeclarations, &metric, tc.labels, tc.rollUpDims) - assert.Equal(t, tc.expectedDims, dimensions) - }) - } -} diff --git a/exporter/awsemfexporter/metric_translator.go b/exporter/awsemfexporter/metric_translator.go index 103052d5c2108..0ba1acc70fa44 100644 --- a/exporter/awsemfexporter/metric_translator.go +++ b/exporter/awsemfexporter/metric_translator.go @@ -15,55 +15,34 @@ package awsemfexporter import ( - "bytes" - "crypto/sha1" "encoding/json" "fmt" - "sort" "time" "go.opentelemetry.io/collector/consumer/pdata" - "go.opentelemetry.io/collector/translator/conventions" "go.uber.org/zap" - - "github.com/open-telemetry/opentelemetry-collector-contrib/exporter/awsemfexporter/mapwithexpiry" ) const ( - CleanInterval = 5 * time.Minute - MinTimeDiff = 50 * time.Millisecond // We assume 50 milli-seconds is the minimal gap between two collected data sample to be valid to calculate delta - // OTel instrumentation lib name as dimension OTellibDimensionKey = "OTelLib" - defaultNameSpace = "default" + defaultNamespace = "default" noInstrumentationLibraryName = "Undefined" - // See: http://docs.aws.amazon.com/AmazonCloudWatchLogs/latest/APIReference/API_PutLogEvents.html - maximumLogEventsPerPut = 10000 - // DimensionRollupOptions ZeroAndSingleDimensionRollup = "ZeroAndSingleDimensionRollup" SingleDimensionRollupOnly = "SingleDimensionRollupOnly" - - FakeMetricValue = 0 ) -var currentState = mapwithexpiry.NewMapWithExpiry(CleanInterval) - -type rateState struct { - value interface{} - timestamp int64 -} - // CWMetrics defines type CWMetrics struct { - Measurements []CwMeasurement + Measurements []CWMeasurement Timestamp int64 Fields map[string]interface{} } -// CwMeasurement defines -type CwMeasurement struct { +// CWMeasurement defines +type CWMeasurement struct { Namespace string Dimensions [][]string Metrics []map[string]string @@ -77,75 +56,21 @@ type CWMetricStats struct { Sum float64 } -// Wrapper interface for: -// - pdata.IntDataPointSlice -// - pdata.DoubleDataPointSlice -// - pdata.IntHistogramDataPointSlice -// - pdata.DoubleHistogramDataPointSlice -// - pdata.DoubleSummaryDataPointSlice -type DataPoints interface { - Len() int - At(int) DataPoint -} - -// DataPoint is a wrapper interface for: -// - pdata.IntDataPoint -// - pdata.DoubleDataPoint -// - pdata.IntHistogramDataPoint -// - pdata.DoubleHistogramDataPoint -// - pdata.DoubleSummaryDataPointSlice -type DataPoint interface { - LabelsMap() pdata.StringMap -} - -// Define wrapper interfaces such that At(i) returns a `DataPoint` -type IntDataPointSlice struct { - pdata.IntDataPointSlice -} -type DoubleDataPointSlice struct { - pdata.DoubleDataPointSlice -} -type DoubleHistogramDataPointSlice struct { - pdata.DoubleHistogramDataPointSlice -} -type DoubleSummaryDataPointSlice struct { - pdata.DoubleSummaryDataPointSlice -} - -func (dps IntDataPointSlice) At(i int) DataPoint { - return dps.IntDataPointSlice.At(i) -} -func (dps DoubleDataPointSlice) At(i int) DataPoint { - return dps.DoubleDataPointSlice.At(i) -} -func (dps DoubleHistogramDataPointSlice) At(i int) DataPoint { - return dps.DoubleHistogramDataPointSlice.At(i) -} -func (dps DoubleSummaryDataPointSlice) At(i int) DataPoint { - return dps.DoubleSummaryDataPointSlice.At(i) +// CWMetricMetadata represents the metadata associated with a given CloudWatch metric +type CWMetricMetadata struct { + Namespace string + Timestamp int64 + LogGroup string + LogStream string + InstrumentationLibraryName string } -// TranslateOtToCWMetric converts OT metrics to CloudWatch Metric format -func TranslateOtToCWMetric(rm *pdata.ResourceMetrics, config *Config) ([]*CWMetrics, int) { - var cwMetricList []*CWMetrics - namespace := config.Namespace +// TranslateOtToGroupedMetric converts OT metrics to Grouped Metric format. +func TranslateOtToGroupedMetric(rm *pdata.ResourceMetrics, groupedMetrics map[string]*GroupedMetric, config *Config) { + timestamp := time.Now().UnixNano() / int64(time.Millisecond) var instrumentationLibName string - - if len(namespace) == 0 { - serviceName, svcNameOk := rm.Resource().Attributes().Get(conventions.AttributeServiceName) - serviceNamespace, svcNsOk := rm.Resource().Attributes().Get(conventions.AttributeServiceNamespace) - if svcNameOk && svcNsOk && serviceName.Type() == pdata.AttributeValueSTRING && serviceNamespace.Type() == pdata.AttributeValueSTRING { - namespace = fmt.Sprintf("%s/%s", serviceNamespace.StringVal(), serviceName.StringVal()) - } else if svcNameOk && serviceName.Type() == pdata.AttributeValueSTRING { - namespace = serviceName.StringVal() - } else if svcNsOk && serviceNamespace.Type() == pdata.AttributeValueSTRING { - namespace = serviceNamespace.StringVal() - } - } - - if len(namespace) == 0 { - namespace = defaultNameSpace - } + cWNamespace := getNamespace(rm, config.Namespace) + logGroup, logStream := getLogInfo(rm, cWNamespace, config) ilms := rm.InstrumentationLibraryMetrics() for j := 0; j < ilms.Len(); j++ { @@ -159,305 +84,228 @@ func TranslateOtToCWMetric(rm *pdata.ResourceMetrics, config *Config) ([]*CWMetr metrics := ilm.Metrics() for k := 0; k < metrics.Len(); k++ { metric := metrics.At(k) - cwMetrics := getCWMetrics(&metric, namespace, instrumentationLibName, config) - cwMetricList = append(cwMetricList, cwMetrics...) + metadata := CWMetricMetadata{ + Namespace: cWNamespace, + Timestamp: timestamp, + LogGroup: logGroup, + LogStream: logStream, + InstrumentationLibraryName: instrumentationLibName, + } + addToGroupedMetric(&metric, groupedMetrics, metadata, config.logger) } } - return cwMetricList, 0 } // TranslateCWMetricToEMF converts CloudWatch Metric format to EMF. -func TranslateCWMetricToEMF(cwMetricLists []*CWMetrics, logger *zap.Logger) []*LogEvent { +func TranslateCWMetricToEMF(cWMetric *CWMetrics) *LogEvent { // convert CWMetric into map format for compatible with PLE input - ples := make([]*LogEvent, 0, maximumLogEventsPerPut) - for _, met := range cwMetricLists { - cwmMap := make(map[string]interface{}) - fieldMap := met.Fields - - if len(met.Measurements) > 0 { - // Create `_aws` section only if there are measurements - cwmMap["CloudWatchMetrics"] = met.Measurements - cwmMap["Timestamp"] = met.Timestamp - fieldMap["_aws"] = cwmMap - } else { - str, _ := json.Marshal(fieldMap) - logger.Debug("Dropped metric due to no matching metric declarations", zap.String("labels", string(str))) - } - - pleMsg, err := json.Marshal(fieldMap) - if err != nil { - continue - } - metricCreationTime := met.Timestamp + cWMetricMap := make(map[string]interface{}) + fieldMap := cWMetric.Fields + + // Create `_aws` section only if there are measurements + if len(cWMetric.Measurements) > 0 { + // Create `_aws` section only if there are measurements + cWMetricMap["CloudWatchMetrics"] = cWMetric.Measurements + cWMetricMap["Timestamp"] = cWMetric.Timestamp + fieldMap["_aws"] = cWMetricMap + } - logEvent := NewLogEvent( - metricCreationTime, - string(pleMsg), - ) - logEvent.LogGeneratedTime = time.Unix(0, metricCreationTime*int64(time.Millisecond)) - ples = append(ples, logEvent) + pleMsg, err := json.Marshal(fieldMap) + if err != nil { + return nil } - return ples + + metricCreationTime := cWMetric.Timestamp + logEvent := NewLogEvent( + metricCreationTime, + string(pleMsg), + ) + logEvent.LogGeneratedTime = time.Unix(0, metricCreationTime*int64(time.Millisecond)) + + return logEvent } -// getCWMetrics translates OTLP Metric to a list of CW Metrics -func getCWMetrics(metric *pdata.Metric, namespace string, instrumentationLibName string, config *Config) (cwMetrics []*CWMetrics) { - if metric == nil { - return +// TranslateGroupedMetricToCWMetric converts Grouped Metric format to CloudWatch Metric format. +func TranslateGroupedMetricToCWMetric(groupedMetric *GroupedMetric, config *Config) *CWMetrics { + labels := groupedMetric.Labels + fields := make(map[string]interface{}, len(labels)+len(groupedMetric.Metrics)) + + // Add labels to fields + for k, v := range labels { + fields[k] = v } - // metric measure data from OT - metricMeasure := make(map[string]string) - metricMeasure["Name"] = metric.Name() - metricMeasure["Unit"] = metric.Unit() - // metric measure slice could include multiple metric measures - metricSlice := []map[string]string{metricMeasure} - - // Retrieve data points - var dps DataPoints - switch metric.DataType() { - case pdata.MetricDataTypeIntGauge: - dps = IntDataPointSlice{metric.IntGauge().DataPoints()} - case pdata.MetricDataTypeDoubleGauge: - dps = DoubleDataPointSlice{metric.DoubleGauge().DataPoints()} - case pdata.MetricDataTypeIntSum: - dps = IntDataPointSlice{metric.IntSum().DataPoints()} - case pdata.MetricDataTypeDoubleSum: - dps = DoubleDataPointSlice{metric.DoubleSum().DataPoints()} - case pdata.MetricDataTypeDoubleHistogram: - dps = DoubleHistogramDataPointSlice{metric.DoubleHistogram().DataPoints()} - case pdata.MetricDataTypeDoubleSummary: - dps = DoubleSummaryDataPointSlice{metric.DoubleSummary().DataPoints()} - default: - config.logger.Warn( - "Unhandled metric data type.", - zap.String("DataType", metric.DataType().String()), - zap.String("Name", metric.Name()), - zap.String("Unit", metric.Unit()), - ) - return + // Add metrics to fields + for metricName, metricInfo := range groupedMetric.Metrics { + fields[metricName] = metricInfo.Value } - if dps.Len() == 0 { - return + var cWMeasurements []CWMeasurement + if len(config.MetricDeclarations) == 0 { + // If there are no metric declarations defined, translate grouped metric + // into the corresponding CW Measurement + cwm := groupedMetricToCWMeasurement(groupedMetric, config) + cWMeasurements = []CWMeasurement{cwm} + } else { + // If metric declarations are defined, filter grouped metric's metrics using + // metric declarations and translate into the corresponding list of CW Measurements + cWMeasurements = groupedMetricToCWMeasurementsWithFilters(groupedMetric, config) } - for m := 0; m < dps.Len(); m++ { - dp := dps.At(m) - cwMetric := buildCWMetric(dp, metric, namespace, metricSlice, instrumentationLibName, config) - if cwMetric != nil { - cwMetrics = append(cwMetrics, cwMetric) - } + + return &CWMetrics{ + Measurements: cWMeasurements, + Timestamp: groupedMetric.Metadata.Timestamp, + Fields: fields, } - return } -// buildCWMetric builds CWMetric from DataPoint -func buildCWMetric(dp DataPoint, pmd *pdata.Metric, namespace string, metricSlice []map[string]string, instrumentationLibName string, config *Config) *CWMetrics { +// groupedMetricToCWMeasurement creates a single CW Measurement from a grouped metric. +func groupedMetricToCWMeasurement(groupedMetric *GroupedMetric, config *Config) CWMeasurement { + labels := groupedMetric.Labels dimensionRollupOption := config.DimensionRollupOption - metricDeclarations := config.MetricDeclarations - - labelsMap := dp.LabelsMap() - labelsSlice := make([]string, labelsMap.Len(), labelsMap.Len()+1) - // `labels` contains label key/value pairs - labels := make(map[string]string, labelsMap.Len()+1) - // `fields` contains metric and dimensions key/value pairs - fields := make(map[string]interface{}, labelsMap.Len()+2) + + // Create a dimension set containing list of label names + dimSet := make([]string, len(labels)) idx := 0 - labelsMap.ForEach(func(k, v string) { - fields[k] = v - labels[k] = v - labelsSlice[idx] = k + for labelName := range labels { + dimSet[idx] = labelName idx++ - }) + } + dimensions := [][]string{dimSet} // Apply single/zero dimension rollup to labels - rollupDimensionArray := dimensionRollup(dimensionRollupOption, labelsSlice, instrumentationLibName) - - // Add OTel instrumentation lib name as an additional dimension if it is defined - if instrumentationLibName != noInstrumentationLibraryName { - labels[OTellibDimensionKey] = instrumentationLibName - fields[OTellibDimensionKey] = instrumentationLibName + rollupDimensionArray := dimensionRollup(dimensionRollupOption, labels) + + if len(rollupDimensionArray) > 0 { + // Perform duplication check for edge case with a single label and single dimension roll-up + _, hasOTelLibKey := labels[OTellibDimensionKey] + isSingleLabel := len(dimSet) <= 1 || (len(dimSet) == 2 && hasOTelLibKey) + singleDimRollup := dimensionRollupOption == SingleDimensionRollupOnly || + dimensionRollupOption == ZeroAndSingleDimensionRollup + if isSingleLabel && singleDimRollup { + // Remove duplicated dimension set before adding on rolled-up dimensions + dimensions = nil + } } - // Create list of dimension sets - var dimensions [][]string - if len(metricDeclarations) > 0 { - // If metric declarations are defined, extract dimension sets from them - dimensions = processMetricDeclarations(metricDeclarations, pmd, labels, rollupDimensionArray) - } else { - // If no metric declarations defined, create a single dimension set containing - // the list of labels - dims := labelsSlice - if instrumentationLibName != noInstrumentationLibraryName { - // If OTel instrumentation lib name is defined, add instrumentation lib - // name as a dimension - dims = append(dims, OTellibDimensionKey) - } + // Add on rolled-up dimensions + dimensions = append(dimensions, rollupDimensionArray...) - if len(rollupDimensionArray) > 0 { - // Perform de-duplication check for edge case with a single label and single roll-up - // is activated - if len(labelsSlice) > 1 || (dimensionRollupOption != SingleDimensionRollupOnly && - dimensionRollupOption != ZeroAndSingleDimensionRollup) { - dimensions = [][]string{dims} - } - dimensions = append(dimensions, rollupDimensionArray...) - } else { - dimensions = [][]string{dims} + metrics := make([]map[string]string, len(groupedMetric.Metrics)) + idx = 0 + for metricName, metricInfo := range groupedMetric.Metrics { + metrics[idx] = map[string]string{ + "Name": metricName, + "Unit": metricInfo.Unit, } + idx++ } - // Build list of CW Measurements - var cwMeasurements []CwMeasurement - if len(dimensions) > 0 { - cwMeasurements = []CwMeasurement{ - { - Namespace: namespace, - Dimensions: dimensions, - Metrics: metricSlice, - }, - } + return CWMeasurement{ + Namespace: groupedMetric.Metadata.Namespace, + Dimensions: dimensions, + Metrics: metrics, } +} - timestamp := time.Now().UnixNano() / int64(time.Millisecond) +// groupedMetricToCWMeasurementsWithFilters filters the grouped metric using the given list of metric +// declarations and returns the corresponding list of CW Measurements. +func groupedMetricToCWMeasurementsWithFilters(groupedMetric *GroupedMetric, config *Config) (cWMeasurements []CWMeasurement) { + labels := groupedMetric.Labels - // Extract metric - var metricVal interface{} - switch metric := dp.(type) { - case pdata.IntDataPoint: - // Put a fake but identical metric value here in order to add metric name into fields - // since calculateRate() needs metric name as one of metric identifiers - fields[pmd.Name()] = int64(FakeMetricValue) - metricVal = metric.Value() - if needsCalculateRate(pmd) { - metricVal = calculateRate(fields, metric.Value(), timestamp) - } - case pdata.DoubleDataPoint: - fields[pmd.Name()] = float64(FakeMetricValue) - metricVal = metric.Value() - if needsCalculateRate(pmd) { - metricVal = calculateRate(fields, metric.Value(), timestamp) - } - case pdata.DoubleHistogramDataPoint: - metricVal = &CWMetricStats{ - Count: metric.Count(), - Sum: metric.Sum(), + // Filter metric declarations by labels + metricDeclarations := make([]*MetricDeclaration, 0, len(config.MetricDeclarations)) + for _, metricDeclaration := range config.MetricDeclarations { + if metricDeclaration.MatchesLabels(labels) { + metricDeclarations = append(metricDeclarations, metricDeclaration) } - case pdata.DoubleSummaryDataPoint: - metricStat := &CWMetricStats{ - Count: metric.Count(), - Sum: metric.Sum(), - } - quantileValues := metric.QuantileValues() - if quantileValues.Len() > 0 { - metricStat.Min = quantileValues.At(0).Value() - metricStat.Max = quantileValues.At(quantileValues.Len() - 1).Value() - } - metricVal = metricStat - } - if metricVal == nil { - return nil } - fields[pmd.Name()] = metricVal - cwMetric := &CWMetrics{ - Measurements: cwMeasurements, - Timestamp: timestamp, - Fields: fields, + // If the whole batch of metrics don't match any metric declarations, drop them + if len(metricDeclarations) == 0 { + labelsStr, _ := json.Marshal(labels) + metricNames := make([]string, 0) + for metricName := range groupedMetric.Metrics { + metricNames = append(metricNames, metricName) + } + config.logger.Debug( + "Dropped batch of metrics: no metric declaration matched labels", + zap.String("Labels", string(labelsStr)), + zap.Strings("Metric Names", metricNames), + ) + return } - return cwMetric -} -// rate is calculated by valDelta / timeDelta -func calculateRate(fields map[string]interface{}, val interface{}, timestamp int64) interface{} { - keys := make([]string, 0, len(fields)) - var b bytes.Buffer - var metricRate interface{} - // hash the key of str: metric + dimension key/value pairs (sorted alpha) - for k := range fields { - keys = append(keys, k) + // Group metrics by matched metric declarations + type metricDeclarationGroup struct { + metricDeclIdxList []int + metrics []map[string]string } - sort.Strings(keys) - for _, k := range keys { - switch v := fields[k].(type) { - case int64: - b.WriteString(k) - continue - case string: - b.WriteString(k) - b.WriteString(v) - default: + + metricDeclGroups := make(map[string]*metricDeclarationGroup) + for metricName, metricInfo := range groupedMetric.Metrics { + // Filter metric declarations by metric name + var metricDeclIdx []int + for i, metricDeclaration := range metricDeclarations { + if metricDeclaration.MatchesName(metricName) { + metricDeclIdx = append(metricDeclIdx, i) + } + } + + if len(metricDeclIdx) == 0 { + config.logger.Debug( + "Dropped metric: no metric declaration matched metric name", + zap.String("Metric name", metricName), + ) continue } - } - h := sha1.New() - h.Write(b.Bytes()) - bs := h.Sum(nil) - hashStr := string(bs) - - // get previous Metric content from map. Need to lock the map until set the new state - currentState.Lock() - if state, ok := currentState.Get(hashStr); ok { - prevStats := state.(*rateState) - deltaTime := timestamp - prevStats.timestamp - var deltaVal interface{} - if _, ok := val.(float64); ok { - deltaVal = val.(float64) - prevStats.value.(float64) - if deltaTime > MinTimeDiff.Milliseconds() && deltaVal.(float64) >= 0 { - metricRate = deltaVal.(float64) * 1e3 / float64(deltaTime) - } + + metric := map[string]string{ + "Name": metricName, + "Unit": metricInfo.Unit, + } + metricDeclKey := fmt.Sprint(metricDeclIdx) + if group, ok := metricDeclGroups[metricDeclKey]; ok { + group.metrics = append(group.metrics, metric) } else { - deltaVal = val.(int64) - prevStats.value.(int64) - if deltaTime > MinTimeDiff.Milliseconds() && deltaVal.(int64) >= 0 { - metricRate = deltaVal.(int64) * 1e3 / deltaTime + metricDeclGroups[metricDeclKey] = &metricDeclarationGroup{ + metricDeclIdxList: metricDeclIdx, + metrics: []map[string]string{metric}, } } } - content := &rateState{ - value: val, - timestamp: timestamp, - } - currentState.Set(hashStr, content) - currentState.Unlock() - if metricRate == nil { - metricRate = 0 - } - return metricRate -} -// dimensionRollup creates rolled-up dimensions from the metric's label set. -func dimensionRollup(dimensionRollupOption string, originalDimensionSlice []string, instrumentationLibName string) [][]string { - var rollupDimensionArray [][]string - var dimensionZero []string - if instrumentationLibName != noInstrumentationLibraryName { - dimensionZero = append(dimensionZero, OTellibDimensionKey) - } - if dimensionRollupOption == ZeroAndSingleDimensionRollup { - //"Zero" dimension rollup - if len(originalDimensionSlice) > 0 { - rollupDimensionArray = append(rollupDimensionArray, dimensionZero) - } + if len(metricDeclGroups) == 0 { + return } - if dimensionRollupOption == ZeroAndSingleDimensionRollup || dimensionRollupOption == SingleDimensionRollupOnly { - //"One" dimension rollup - for _, dimensionKey := range originalDimensionSlice { - rollupDimensionArray = append(rollupDimensionArray, append(dimensionZero, dimensionKey)) + + // Apply single/zero dimension rollup to labels + rollupDimensionArray := dimensionRollup(config.DimensionRollupOption, labels) + + // Translate each group into a CW Measurement + cWMeasurements = make([]CWMeasurement, 0, len(metricDeclGroups)) + for _, group := range metricDeclGroups { + var dimensions [][]string + // Extract dimensions from matched metric declarations + for _, metricDeclIdx := range group.metricDeclIdxList { + dims := metricDeclarations[metricDeclIdx].ExtractDimensions(labels) + dimensions = append(dimensions, dims...) } - } + dimensions = append(dimensions, rollupDimensionArray...) - return rollupDimensionArray -} + // De-duplicate dimensions + dimensions = dedupDimensions(dimensions) -func needsCalculateRate(pmd *pdata.Metric) bool { - switch pmd.DataType() { - case pdata.MetricDataTypeIntSum: - if pmd.IntSum().AggregationTemporality() == pdata.AggregationTemporalityCumulative { - return true - } - case pdata.MetricDataTypeDoubleSum: - if pmd.DoubleSum().AggregationTemporality() == pdata.AggregationTemporalityCumulative { - return true + // Export metrics only with non-empty dimensions list + if len(dimensions) > 0 { + cwm := CWMeasurement{ + Namespace: groupedMetric.Metadata.Namespace, + Dimensions: dimensions, + Metrics: group.metrics, + } + cWMeasurements = append(cWMeasurements, cwm) } } - return false + + return } diff --git a/exporter/awsemfexporter/metric_translator_test.go b/exporter/awsemfexporter/metric_translator_test.go index 642759f74cd6f..c6c13370b1902 100644 --- a/exporter/awsemfexporter/metric_translator_test.go +++ b/exporter/awsemfexporter/metric_translator_test.go @@ -15,12 +15,10 @@ package awsemfexporter import ( - "encoding/json" "io/ioutil" "sort" "strings" "testing" - "time" commonpb "github.com/census-instrumentation/opencensus-proto/gen-go/agent/common/v1" metricspb "github.com/census-instrumentation/opencensus-proto/gen-go/metrics/v1" @@ -240,7 +238,7 @@ func createMetricTestData() consumerdata.MetricsData { Timeseries: []*metricspb.TimeSeries{ { LabelValues: []*metricspb.LabelValue{ - {Value: "testSpan"}, + {Value: "testSpan", HasValue: true}, }, Points: []*metricspb.Point{ { @@ -276,417 +274,255 @@ func createMetricTestData() consumerdata.MetricsData { } } -// Asserts whether dimension sets are equal (i.e. has same sets of dimensions) -func assertDimsEqual(t *testing.T, expected, actual [][]string) { - // Convert to string for easier sorting - expectedStringified := make([]string, len(expected)) - actualStringified := make([]string, len(actual)) - for i, v := range expected { - sort.Strings(v) - expectedStringified[i] = strings.Join(v, ",") +func stringSlicesEqual(expected, actual []string) bool { + if len(expected) != len(actual) { + return false } - for i, v := range actual { + for i, expectedStr := range expected { + if expectedStr != actual[i] { + return false + } + } + return true +} + +// hashDimensions hashes dimensions for equality checking. +func hashDimensions(dims [][]string) []string { + // Convert to string for easier sorting + stringified := make([]string, len(dims)) + for i, v := range dims { sort.Strings(v) - actualStringified[i] = strings.Join(v, ",") + stringified[i] = strings.Join(v, ",") } // Sort across dimension sets for equality checking - sort.Strings(expectedStringified) - sort.Strings(actualStringified) - assert.Equal(t, expectedStringified, actualStringified) + sort.Strings(stringified) + return stringified } -// Asserts whether CW Measurements are equal. -func assertCwMeasurementEqual(t *testing.T, expected, actual CwMeasurement) { - assert.Equal(t, expected.Namespace, actual.Namespace) - assert.Equal(t, expected.Metrics, actual.Metrics) - assertDimsEqual(t, expected.Dimensions, actual.Dimensions) +// hashMetricSlice hashes a metrics slice for equality checking. +func hashMetricSlice(metricSlice []map[string]string) []string { + // Convert to string for easier sorting + stringified := make([]string, len(metricSlice)) + for i, v := range metricSlice { + stringified[i] = v["Name"] + "," + v["Unit"] + } + // Sort across metrics for equality checking + sort.Strings(stringified) + return stringified } -func TestTranslateOtToCWMetricWithInstrLibrary(t *testing.T) { - config := &Config{ - Namespace: "", - DimensionRollupOption: ZeroAndSingleDimensionRollup, - logger: zap.NewNop(), +// assertDimsEqual asserts whether dimension sets are equal +// (i.e. has same sets of dimensions), regardless of order. +func assertDimsEqual(t *testing.T, expected, actual [][]string) { + assert.Equal(t, len(expected), len(actual)) + expectedHashedDimensions := hashDimensions(expected) + actualHashedDimensions := hashDimensions(actual) + assert.Equal(t, expectedHashedDimensions, actualHashedDimensions) +} + +// cWMeasurementEqual returns true if CW Measurements are equal. +func cWMeasurementEqual(expected, actual CWMeasurement) bool { + // Check namespace + if expected.Namespace != actual.Namespace { + return false } - md := createMetricTestData() - rm := internaldata.OCToMetrics(md).ResourceMetrics().At(0) - ilms := rm.InstrumentationLibraryMetrics() - ilm := ilms.At(0) - ilm.InstrumentationLibrary().SetName("cloudwatch-lib") - cwm, totalDroppedMetrics := TranslateOtToCWMetric(&rm, config) - assert.Equal(t, 0, totalDroppedMetrics) - assert.NotNil(t, cwm) - assert.Equal(t, 6, len(cwm)) - assert.Equal(t, 1, len(cwm[0].Measurements)) - met := cwm[0] + // Check metrics + if len(expected.Metrics) != len(actual.Metrics) { + return false + } + expectedHashedMetrics := hashMetricSlice(expected.Metrics) + actualHashedMetrics := hashMetricSlice(actual.Metrics) + if !stringSlicesEqual(expectedHashedMetrics, actualHashedMetrics) { + return false + } - assert.Equal(t, met.Fields["spanCounter"], 0) + // Check dimensions + if len(expected.Dimensions) != len(actual.Dimensions) { + return false + } + expectedHashedDimensions := hashDimensions(expected.Dimensions) + actualHashedDimensions := hashDimensions(actual.Dimensions) + return stringSlicesEqual(expectedHashedDimensions, actualHashedDimensions) +} - expectedMeasurement := CwMeasurement{ - Namespace: "myServiceNS/myServiceName", - Dimensions: [][]string{ - {OTellibDimensionKey, "isItAnError", "spanName"}, - {OTellibDimensionKey}, - {OTellibDimensionKey, "spanName"}, - {OTellibDimensionKey, "isItAnError"}, - }, - Metrics: []map[string]string{ - { - "Name": "spanCounter", - "Unit": "Count", - }, - }, +// assertCWMeasurementEqual asserts whether CW Measurements are equal. +func assertCWMeasurementEqual(t *testing.T, expected, actual CWMeasurement) { + // Check namespace + assert.Equal(t, expected.Namespace, actual.Namespace) + + // Check metrics + assert.Equal(t, len(expected.Metrics), len(actual.Metrics)) + expectedHashSlice := hashMetricSlice(expected.Metrics) + actualHashSlice := hashMetricSlice(actual.Metrics) + assert.Equal(t, expectedHashSlice, actualHashSlice) + + // Check dimensions + assertDimsEqual(t, expected.Dimensions, actual.Dimensions) +} + +// assertCWMeasurementSliceEqual asserts whether CW Measurements are equal, regardless of order. +func assertCWMeasurementSliceEqual(t *testing.T, expected, actual []CWMeasurement) { + assert.Equal(t, len(expected), len(actual)) + seen := make([]bool, len(expected)) + for _, actualMeasurement := range actual { + hasMatch := false + for i, expectedMeasurement := range expected { + if !seen[i] { + if cWMeasurementEqual(actualMeasurement, expectedMeasurement) { + seen[i] = true + hasMatch = true + } + } + } + assert.True(t, hasMatch) } - assertCwMeasurementEqual(t, expectedMeasurement, met.Measurements[0]) } -func TestTranslateOtToCWMetricWithoutInstrLibrary(t *testing.T) { +// assertCWMetricsEqual asserts whether CW Metrics are equal. +func assertCWMetricsEqual(t *testing.T, expected, actual *CWMetrics) { + assert.Equal(t, expected.Timestamp, actual.Timestamp) + assert.Equal(t, expected.Fields, actual.Fields) + assert.Equal(t, len(expected.Measurements), len(actual.Measurements)) + assertCWMeasurementSliceEqual(t, expected.Measurements, actual.Measurements) +} + +func TestTranslateOtToGroupedMetric(t *testing.T) { config := &Config{ Namespace: "", DimensionRollupOption: ZeroAndSingleDimensionRollup, logger: zap.NewNop(), } md := createMetricTestData() - rm := internaldata.OCToMetrics(md).ResourceMetrics().At(0) - cwm, totalDroppedMetrics := TranslateOtToCWMetric(&rm, config) - assert.Equal(t, 0, totalDroppedMetrics) - assert.NotNil(t, cwm) - assert.Equal(t, 6, len(cwm)) - assert.Equal(t, 1, len(cwm[0].Measurements)) - met := cwm[0] - assert.NotContains(t, met.Fields, OTellibDimensionKey) - assert.Equal(t, met.Fields["spanCounter"], 0) + noInstrLibMetric := internaldata.OCToMetrics(md).ResourceMetrics().At(0) + instrLibMetric := internaldata.OCToMetrics(md).ResourceMetrics().At(0) + ilm := instrLibMetric.InstrumentationLibraryMetrics().At(0) + ilm.InstrumentationLibrary().SetName("cloudwatch-lib") - expectedMeasurement := CwMeasurement{ - Namespace: "myServiceNS/myServiceName", - Dimensions: [][]string{ - {"isItAnError", "spanName"}, - {}, - {"spanName"}, - {"isItAnError"}, - }, - Metrics: []map[string]string{ - { - "Name": "spanCounter", - "Unit": "Count", - }, - }, - } - assertCwMeasurementEqual(t, expectedMeasurement, met.Measurements[0]) -} + noNamespaceMetric := internaldata.OCToMetrics(md).ResourceMetrics().At(0) + noNamespaceMetric.Resource().Attributes().Delete(conventions.AttributeServiceNamespace) + noNamespaceMetric.Resource().Attributes().Delete(conventions.AttributeServiceName) -func TestTranslateOtToCWMetricWithNameSpace(t *testing.T) { - config := &Config{ - Namespace: "", - DimensionRollupOption: ZeroAndSingleDimensionRollup, - } - md := consumerdata.MetricsData{ - Node: &commonpb.Node{ - LibraryInfo: &commonpb.LibraryInfo{ExporterVersion: "SomeVersion"}, - }, - Resource: &resourcepb.Resource{ - Labels: map[string]string{ - conventions.AttributeServiceName: "myServiceName", - }, + counterMetrics := map[string]*MetricInfo{ + "spanCounter": { + Value: int64(0), + Unit: "Count", }, - Metrics: []*metricspb.Metric{}, - } - rm := internaldata.OCToMetrics(md).ResourceMetrics().At(0) - cwm, totalDroppedMetrics := TranslateOtToCWMetric(&rm, config) - assert.Equal(t, 0, totalDroppedMetrics) - assert.Nil(t, cwm) - assert.Equal(t, 0, len(cwm)) - md = consumerdata.MetricsData{ - Node: &commonpb.Node{ - LibraryInfo: &commonpb.LibraryInfo{ExporterVersion: "SomeVersion"}, + "spanDoubleCounter": { + Value: float64(0), + Unit: "Count", }, - Resource: &resourcepb.Resource{ - Labels: map[string]string{ - conventions.AttributeServiceNamespace: "myServiceNS", - }, + "spanGaugeCounter": { + Value: int64(1), + Unit: "Count", }, - Metrics: []*metricspb.Metric{ - { - MetricDescriptor: &metricspb.MetricDescriptor{ - Name: "spanCounter", - Description: "Counting all the spans", - Unit: "Count", - Type: metricspb.MetricDescriptor_CUMULATIVE_INT64, - LabelKeys: []*metricspb.LabelKey{ - {Key: "spanName"}, - {Key: "isItAnError"}, - }, - }, - Timeseries: []*metricspb.TimeSeries{ - { - LabelValues: []*metricspb.LabelValue{ - {Value: "testSpan", HasValue: true}, - {Value: "false", HasValue: true}, - }, - Points: []*metricspb.Point{ - { - Timestamp: ×tamp.Timestamp{ - Seconds: 100, - }, - Value: &metricspb.Point_Int64Value{ - Int64Value: 1, - }, - }, - }, - }, - }, - }, - { - MetricDescriptor: &metricspb.MetricDescriptor{ - Name: "spanCounter", - Description: "Counting all the spans", - Unit: "Count", - Type: metricspb.MetricDescriptor_CUMULATIVE_INT64, - }, - Timeseries: []*metricspb.TimeSeries{}, - }, - { - MetricDescriptor: &metricspb.MetricDescriptor{ - Name: "spanGaugeCounter", - Description: "Counting all the spans", - Unit: "Count", - Type: metricspb.MetricDescriptor_GAUGE_INT64, - }, - Timeseries: []*metricspb.TimeSeries{}, - }, - { - MetricDescriptor: &metricspb.MetricDescriptor{ - Name: "spanGaugeDoubleCounter", - Description: "Counting all the spans", - Unit: "Count", - Type: metricspb.MetricDescriptor_GAUGE_DOUBLE, - }, - Timeseries: []*metricspb.TimeSeries{}, - }, - { - MetricDescriptor: &metricspb.MetricDescriptor{ - Name: "spanDoubleCounter", - Description: "Counting all the spans", - Unit: "Count", - Type: metricspb.MetricDescriptor_CUMULATIVE_DOUBLE, - }, - Timeseries: []*metricspb.TimeSeries{}, - }, - { - MetricDescriptor: &metricspb.MetricDescriptor{ - Name: "spanDoubleCounter", - Description: "Counting all the spans", - Unit: "Count", - Type: metricspb.MetricDescriptor_CUMULATIVE_DISTRIBUTION, - }, - Timeseries: []*metricspb.TimeSeries{}, - }, + "spanGaugeDoubleCounter": { + Value: float64(0.1), + Unit: "Count", }, } - rm = internaldata.OCToMetrics(md).ResourceMetrics().At(0) - cwm, totalDroppedMetrics = TranslateOtToCWMetric(&rm, config) - assert.Equal(t, 0, totalDroppedMetrics) - assert.NotNil(t, cwm) - assert.Equal(t, 1, len(cwm)) - - met := cwm[0] - assert.Equal(t, "myServiceNS", met.Measurements[0].Namespace) -} - -func TestTranslateOtToCWMetricWithFiltering(t *testing.T) { - md := consumerdata.MetricsData{ - Node: &commonpb.Node{ - LibraryInfo: &commonpb.LibraryInfo{ExporterVersion: "SomeVersion"}, - }, - Resource: &resourcepb.Resource{ - Labels: map[string]string{ - conventions.AttributeServiceName: "myServiceName", - conventions.AttributeServiceNamespace: "myServiceNS", - }, - }, - Metrics: []*metricspb.Metric{ - { - MetricDescriptor: &metricspb.MetricDescriptor{ - Name: "spanCounter", - Description: "Counting all the spans", - Unit: "Count", - Type: metricspb.MetricDescriptor_CUMULATIVE_INT64, - LabelKeys: []*metricspb.LabelKey{ - {Key: "spanName"}, - {Key: "isItAnError"}, - }, - }, - Timeseries: []*metricspb.TimeSeries{ - { - LabelValues: []*metricspb.LabelValue{ - {Value: "testSpan", HasValue: true}, - {Value: "false", HasValue: true}, - }, - Points: []*metricspb.Point{ - { - Timestamp: ×tamp.Timestamp{ - Seconds: 100, - }, - Value: &metricspb.Point_Int64Value{ - Int64Value: 1, - }, - }, - }, - }, - }, - }, + timerMetrics := map[string]*MetricInfo{ + "spanTimer": { + Value: &CWMetricStats{ + Max: 10, + Min: 0, + Count: 5, + Sum: 15, + }, + Unit: "Seconds", }, } - rm := internaldata.OCToMetrics(md).ResourceMetrics().At(0) - ilm := rm.InstrumentationLibraryMetrics().At(0) - ilm.InstrumentationLibrary().SetName("cloudwatch-lib") - testCases := []struct { - testName string - metricNameSelectors []string - labelMatchers []*LabelMatcher - dimensionRollupOption string - expectedDimensions [][]string - numMeasurements int + testName string + metric *pdata.ResourceMetrics + counterLabels map[string]string + timerLabels map[string]string + expectedNamespace string }{ { - "has match w/ Zero + Single dim rollup", - []string{"spanCounter"}, - nil, - ZeroAndSingleDimensionRollup, - [][]string{ - {"spanName", "isItAnError"}, - {"spanName", OTellibDimensionKey}, - {OTellibDimensionKey, "isItAnError"}, - {OTellibDimensionKey}, + "w/ instrumentation library and namespace", + &instrLibMetric, + map[string]string{ + (OTellibDimensionKey): "cloudwatch-lib", + "isItAnError": "false", + "spanName": "testSpan", }, - 1, - }, - { - "has match w/ no dim rollup", - []string{"spanCounter"}, - nil, - "", - [][]string{ - {"spanName", "isItAnError"}, - {"spanName", OTellibDimensionKey}, + map[string]string{ + (OTellibDimensionKey): "cloudwatch-lib", + "spanName": "testSpan", }, - 1, + "myServiceNS/myServiceName", }, { - "has label match w/ no dim rollup", - []string{"spanCounter"}, - []*LabelMatcher{ - { - LabelNames: []string{"isItAnError", "spanName"}, - Regex: "false;testSpan", - }, + "w/o instrumentation library, w/ namespace", + &noInstrLibMetric, + map[string]string{ + "isItAnError": "false", + "spanName": "testSpan", }, - "", - [][]string{ - {"spanName", "isItAnError"}, - {"spanName", OTellibDimensionKey}, + map[string]string{ + "spanName": "testSpan", }, - 1, + "myServiceNS/myServiceName", }, { - "no label match w/ no dim rollup", - []string{"spanCounter"}, - []*LabelMatcher{ - { - LabelNames: []string{"isItAnError", "spanName"}, - Regex: "true;testSpan", - }, + "w/o instrumentation library and namespace", + &noNamespaceMetric, + map[string]string{ + "isItAnError": "false", + "spanName": "testSpan", }, - "", - nil, - 0, - }, - { - "No match w/ rollup", - []string{"invalid"}, - nil, - ZeroAndSingleDimensionRollup, - [][]string{ - {OTellibDimensionKey, "spanName"}, - {OTellibDimensionKey, "isItAnError"}, - {OTellibDimensionKey}, + map[string]string{ + "spanName": "testSpan", }, - 1, - }, - { - "No match w/ no rollup", - []string{"invalid"}, - nil, - "", - nil, - 0, + defaultNamespace, }, } - logger := zap.NewNop() for _, tc := range testCases { - m := MetricDeclaration{ - Dimensions: [][]string{{"isItAnError", "spanName"}, {"spanName", OTellibDimensionKey}}, - MetricNameSelectors: tc.metricNameSelectors, - LabelMatchers: tc.labelMatchers, - } - config := &Config{ - Namespace: "", - DimensionRollupOption: tc.dimensionRollupOption, - MetricDeclarations: []*MetricDeclaration{&m}, - } t.Run(tc.testName, func(t *testing.T) { - err := m.Init(logger) - assert.Nil(t, err) - cwm, totalDroppedMetrics := TranslateOtToCWMetric(&rm, config) - assert.Equal(t, 0, totalDroppedMetrics) - assert.Equal(t, 1, len(cwm)) - assert.NotNil(t, cwm) - - assert.Equal(t, tc.numMeasurements, len(cwm[0].Measurements)) - - if tc.numMeasurements > 0 { - dimensions := cwm[0].Measurements[0].Dimensions - assertDimsEqual(t, tc.expectedDimensions, dimensions) + groupedMetrics := make(map[string]*GroupedMetric) + TranslateOtToGroupedMetric(tc.metric, groupedMetrics, config) + assert.NotNil(t, groupedMetrics) + assert.Equal(t, 2, len(groupedMetrics)) + + for _, v := range groupedMetrics { + assert.Equal(t, tc.expectedNamespace, v.Metadata.Namespace) + if len(v.Metrics) == 4 { + assert.Equal(t, tc.counterLabels, v.Labels) + assert.Equal(t, counterMetrics, v.Metrics) + } else { + assert.Equal(t, 1, len(v.Metrics)) + assert.Equal(t, tc.timerLabels, v.Labels) + assert.Equal(t, timerMetrics, v.Metrics) + } } }) } - t.Run("No instrumentation library name w/ no dim rollup", func(t *testing.T) { - rm = internaldata.OCToMetrics(md).ResourceMetrics().At(0) - m := MetricDeclaration{ - Dimensions: [][]string{{"isItAnError", "spanName"}, {"spanName", OTellibDimensionKey}}, - MetricNameSelectors: []string{"spanCounter"}, - } - config := &Config{ - Namespace: "", - DimensionRollupOption: "", - MetricDeclarations: []*MetricDeclaration{&m}, - } - err := m.Init(logger) - assert.Nil(t, err) - cwm, totalDroppedMetrics := TranslateOtToCWMetric(&rm, config) - assert.Equal(t, 0, totalDroppedMetrics) - assert.Equal(t, 1, len(cwm)) - assert.NotNil(t, cwm) - - assert.Equal(t, 1, len(cwm[0].Measurements)) - - // No OTelLib present - expectedDims := [][]string{ - {"spanName", "isItAnError"}, + t.Run("No metrics", func(t *testing.T) { + md = consumerdata.MetricsData{ + Node: &commonpb.Node{ + LibraryInfo: &commonpb.LibraryInfo{ExporterVersion: "SomeVersion"}, + }, + Resource: &resourcepb.Resource{ + Labels: map[string]string{ + conventions.AttributeServiceName: "myServiceName", + }, + }, + Metrics: []*metricspb.Metric{}, } - dimensions := cwm[0].Measurements[0].Dimensions - assertDimsEqual(t, expectedDims, dimensions) + rm := internaldata.OCToMetrics(md).ResourceMetrics().At(0) + groupedMetrics := make(map[string]*GroupedMetric) + TranslateOtToGroupedMetric(&rm, groupedMetrics, config) + assert.Equal(t, 0, len(groupedMetrics)) }) } func TestTranslateCWMetricToEMF(t *testing.T) { - cwMeasurement := CwMeasurement{ + cwMeasurement := CWMeasurement{ Namespace: "test-emf", Dimensions: [][]string{{OTellibDimensionKey}, {OTellibDimensionKey, "spanName"}}, Metrics: []map[string]string{{ @@ -703,925 +539,549 @@ func TestTranslateCWMetricToEMF(t *testing.T) { met := &CWMetrics{ Timestamp: timestamp, Fields: fields, - Measurements: []CwMeasurement{cwMeasurement}, + Measurements: []CWMeasurement{cwMeasurement}, } - logger := zap.NewNop() - inputLogEvent := TranslateCWMetricToEMF([]*CWMetrics{met}, logger) + inputLogEvent := TranslateCWMetricToEMF(met) - assert.Equal(t, readFromFile("testdata/testTranslateCWMetricToEMF.json"), *inputLogEvent[0].InputLogEvent.Message, "Expect to be equal") + assert.Equal(t, readFromFile("testdata/testTranslateCWMetricToEMF.json"), *inputLogEvent.InputLogEvent.Message, "Expect to be equal") } -func TestTranslateCWMetricToEMFNoMeasurements(t *testing.T) { +func TestTranslateGroupedMetricToCWMetric(t *testing.T) { timestamp := int64(1596151098037) - fields := make(map[string]interface{}) - fields[OTellibDimensionKey] = "cloudwatch-otel" - fields["spanName"] = "test" - fields["spanCounter"] = 0 - - met := &CWMetrics{ - Timestamp: timestamp, - Fields: fields, - Measurements: nil, - } - obs, logs := observer.New(zap.DebugLevel) - logger := zap.New(obs) - inputLogEvent := TranslateCWMetricToEMF([]*CWMetrics{met}, logger) - expected := "{\"OTelLib\":\"cloudwatch-otel\",\"spanCounter\":0,\"spanName\":\"test\"}" - - assert.Equal(t, expected, *inputLogEvent[0].InputLogEvent.Message) - - // Check logged warning message - fieldsStr, _ := json.Marshal(fields) - expectedLogs := []observer.LoggedEntry{{ - Entry: zapcore.Entry{Level: zap.DebugLevel, Message: "Dropped metric due to no matching metric declarations"}, - Context: []zapcore.Field{zap.String("labels", string(fieldsStr))}, - }} - assert.Equal(t, 1, logs.Len()) - assert.Equal(t, expectedLogs, logs.AllUntimed()) -} - -func TestGetCWMetrics(t *testing.T) { namespace := "Namespace" - OTelLib := OTellibDimensionKey - instrumentationLibName := "InstrLibName" - config := &Config{ - Namespace: "", - DimensionRollupOption: "", - } - testCases := []struct { - testName string - metric *metricspb.Metric - expected []*CWMetrics + testName string + groupedMetric *GroupedMetric + metricDeclarations []*MetricDeclaration + expectedCWMetric *CWMetrics }{ { - "Int gauge", - &metricspb.Metric{ - MetricDescriptor: &metricspb.MetricDescriptor{ - Name: "foo", - Type: metricspb.MetricDescriptor_GAUGE_INT64, - Unit: "Count", - LabelKeys: []*metricspb.LabelKey{ - {Key: "label1"}, - {Key: "label2"}, + "single metric w/o metric declarations", + &GroupedMetric{ + Labels: map[string]string{ + "label1": "value1", + }, + Metrics: map[string]*MetricInfo{ + "metric1": { + Value: 1, + Unit: "Count", }, }, - Timeseries: []*metricspb.TimeSeries{ - { - LabelValues: []*metricspb.LabelValue{ - {Value: "value1", HasValue: true}, - {Value: "value2", HasValue: true}, - }, - Points: []*metricspb.Point{ - { - Value: &metricspb.Point_Int64Value{ - Int64Value: 1, - }, - }, - }, - }, + Metadata: CWMetricMetadata{ + Namespace: namespace, + Timestamp: timestamp, + }, + }, + nil, + &CWMetrics{ + Measurements: []CWMeasurement{ { - LabelValues: []*metricspb.LabelValue{ - {HasValue: false}, - {Value: "value2", HasValue: true}, - }, - Points: []*metricspb.Point{ + Namespace: namespace, + Dimensions: [][]string{{"label1"}}, + Metrics: []map[string]string{ { - Value: &metricspb.Point_Int64Value{ - Int64Value: 3, - }, + "Name": "metric1", + "Unit": "Count", }, }, }, }, + Timestamp: timestamp, + Fields: map[string]interface{}{ + "label1": "value1", + "metric1": 1, + }, }, - []*CWMetrics{ - { - Measurements: []CwMeasurement{ - { - Namespace: namespace, - Dimensions: [][]string{ - {"label1", "label2", OTelLib}, - }, - Metrics: []map[string]string{ - {"Name": "foo", "Unit": "Count"}, - }, - }, - }, - Fields: map[string]interface{}{ - OTelLib: instrumentationLibName, - "foo": int64(1), - "label1": "value1", - "label2": "value2", + }, + { + "single metric w/ metric declarations", + &GroupedMetric{ + Labels: map[string]string{ + "label1": "value1", + }, + Metrics: map[string]*MetricInfo{ + "metric1": { + Value: 1, + Unit: "Count", }, }, + Metadata: CWMetricMetadata{ + Namespace: namespace, + Timestamp: timestamp, + }, + }, + []*MetricDeclaration{ { - Measurements: []CwMeasurement{ - { - Namespace: namespace, - Dimensions: [][]string{ - {"label2", OTelLib}, - }, - Metrics: []map[string]string{ - {"Name": "foo", "Unit": "Count"}, + Dimensions: [][]string{{"label1"}, {"label1", "label2"}}, + MetricNameSelectors: []string{"metric.*"}, + }, + }, + &CWMetrics{ + Measurements: []CWMeasurement{ + { + Namespace: namespace, + Dimensions: [][]string{{"label1"}}, + Metrics: []map[string]string{ + { + "Name": "metric1", + "Unit": "Count", }, }, }, - Fields: map[string]interface{}{ - OTelLib: instrumentationLibName, - "foo": int64(3), - "label2": "value2", - }, + }, + Timestamp: timestamp, + Fields: map[string]interface{}{ + "label1": "value1", + "metric1": 1, }, }, }, { - "Double gauge", - &metricspb.Metric{ - MetricDescriptor: &metricspb.MetricDescriptor{ - Name: "foo", - Type: metricspb.MetricDescriptor_GAUGE_DOUBLE, - Unit: "Count", - LabelKeys: []*metricspb.LabelKey{ - {Key: "label1"}, - {Key: "label2"}, + "multiple metrics w/o metric declarations", + &GroupedMetric{ + Labels: map[string]string{ + "label2": "value2", + "label1": "value1", + }, + Metrics: map[string]*MetricInfo{ + "metric1": { + Value: 1, + Unit: "Count", + }, + "metric2": { + Value: 200, + Unit: "Count", + }, + "metric3": { + Value: 3.14, + Unit: "Seconds", }, }, - Timeseries: []*metricspb.TimeSeries{ + Metadata: CWMetricMetadata{ + Namespace: namespace, + Timestamp: timestamp, + }, + }, + nil, + &CWMetrics{ + Measurements: []CWMeasurement{ { - LabelValues: []*metricspb.LabelValue{ - {Value: "value1", HasValue: true}, - {Value: "value2", HasValue: true}, - }, - Points: []*metricspb.Point{ + Namespace: namespace, + Dimensions: [][]string{{"label1", "label2"}}, + Metrics: []map[string]string{ { - Value: &metricspb.Point_DoubleValue{ - DoubleValue: 0.1, - }, + "Name": "metric1", + "Unit": "Count", }, - }, - }, - { - LabelValues: []*metricspb.LabelValue{ - {HasValue: false}, - {Value: "value2", HasValue: true}, - }, - Points: []*metricspb.Point{ { - Value: &metricspb.Point_DoubleValue{ - DoubleValue: 0.3, - }, + "Name": "metric2", + "Unit": "Count", + }, + { + "Name": "metric3", + "Unit": "Seconds", }, }, }, }, + Timestamp: timestamp, + Fields: map[string]interface{}{ + "label1": "value1", + "label2": "value2", + "metric1": 1, + "metric2": 200, + "metric3": 3.14, + }, }, - []*CWMetrics{ - { - Measurements: []CwMeasurement{ - { - Namespace: namespace, - Dimensions: [][]string{ - {"label1", "label2", OTelLib}, - }, - Metrics: []map[string]string{ - {"Name": "foo", "Unit": "Count"}, - }, - }, + }, + { + "multiple metrics w/ metric declarations", + &GroupedMetric{ + Labels: map[string]string{ + "label2": "value2", + "label1": "value1", + }, + Metrics: map[string]*MetricInfo{ + "metric1": { + Value: 1, + Unit: "Count", + }, + "metric2": { + Value: 200, + Unit: "Count", }, - Fields: map[string]interface{}{ - OTelLib: instrumentationLibName, - "foo": 0.1, - "label1": "value1", - "label2": "value2", + "metric3": { + Value: 3.14, + Unit: "Seconds", }, }, + Metadata: CWMetricMetadata{ + Namespace: namespace, + Timestamp: timestamp, + }, + }, + []*MetricDeclaration{ { - Measurements: []CwMeasurement{ - { - Namespace: namespace, - Dimensions: [][]string{ - {"label2", OTelLib}, - }, - Metrics: []map[string]string{ - {"Name": "foo", "Unit": "Count"}, - }, - }, - }, - Fields: map[string]interface{}{ - OTelLib: instrumentationLibName, - "foo": 0.3, - "label2": "value2", + Dimensions: [][]string{ + {"label1"}, + {"label1", "label3"}, }, + MetricNameSelectors: []string{"metric1"}, }, - }, - }, - { - "Int sum", - &metricspb.Metric{ - MetricDescriptor: &metricspb.MetricDescriptor{ - Name: "foo", - Type: metricspb.MetricDescriptor_CUMULATIVE_INT64, - Unit: "Count", - LabelKeys: []*metricspb.LabelKey{ - {Key: "label1"}, - {Key: "label2"}, + { + Dimensions: [][]string{ + {"label1", "label2"}, + {"label1", "label3"}, }, + MetricNameSelectors: []string{"metric2"}, }, - Timeseries: []*metricspb.TimeSeries{ + }, + &CWMetrics{ + Measurements: []CWMeasurement{ { - LabelValues: []*metricspb.LabelValue{ - {Value: "value1", HasValue: true}, - {Value: "value2", HasValue: true}, - }, - Points: []*metricspb.Point{ + Namespace: namespace, + Dimensions: [][]string{{"label1"}}, + Metrics: []map[string]string{ { - Value: &metricspb.Point_Int64Value{ - Int64Value: 1, - }, + "Name": "metric1", + "Unit": "Count", }, }, }, { - LabelValues: []*metricspb.LabelValue{ - {HasValue: false}, - {Value: "value2", HasValue: true}, - }, - Points: []*metricspb.Point{ + Namespace: namespace, + Dimensions: [][]string{{"label1", "label2"}}, + Metrics: []map[string]string{ { - Value: &metricspb.Point_Int64Value{ - Int64Value: 3, - }, + "Name": "metric2", + "Unit": "Count", }, }, }, }, + Timestamp: timestamp, + Fields: map[string]interface{}{ + "label1": "value1", + "label2": "value2", + "metric1": 1, + "metric2": 200, + "metric3": 3.14, + }, }, - []*CWMetrics{ - { - Measurements: []CwMeasurement{ - { - Namespace: namespace, - Dimensions: [][]string{ - {"label1", "label2", OTelLib}, - }, - Metrics: []map[string]string{ - {"Name": "foo", "Unit": "Count"}, - }, - }, - }, - Fields: map[string]interface{}{ - OTelLib: instrumentationLibName, - "foo": 0, - "label1": "value1", - "label2": "value2", + }, + { + "no metrics", + &GroupedMetric{ + Labels: map[string]string{ + "label1": "value1", + }, + Metrics: nil, + Metadata: CWMetricMetadata{ + Namespace: namespace, + Timestamp: timestamp, + }, + }, + nil, + &CWMetrics{ + Measurements: []CWMeasurement{ + { + Namespace: namespace, + Dimensions: [][]string{{"label1"}}, + Metrics: nil, }, }, - { - Measurements: []CwMeasurement{ - { - Namespace: namespace, - Dimensions: [][]string{ - {"label2", OTelLib}, - }, - Metrics: []map[string]string{ - {"Name": "foo", "Unit": "Count"}, - }, - }, + Timestamp: timestamp, + Fields: map[string]interface{}{ + "label1": "value1", + }, + }, + }, + } + + logger := zap.NewNop() + + for _, tc := range testCases { + t.Run(tc.testName, func(t *testing.T) { + config := &Config{ + MetricDeclarations: tc.metricDeclarations, + DimensionRollupOption: "", + logger: logger, + } + for _, decl := range tc.metricDeclarations { + decl.Init(logger) + } + cWMetric := TranslateGroupedMetricToCWMetric(tc.groupedMetric, config) + assert.NotNil(t, cWMetric) + assertCWMetricsEqual(t, tc.expectedCWMetric, cWMetric) + }) + } +} + +func TestGroupedMetricToCWMeasurement(t *testing.T) { + timestamp := int64(1596151098037) + namespace := "Namespace" + testCases := []struct { + testName string + dimensionRollupOption string + groupedMetric *GroupedMetric + expectedMeasurement CWMeasurement + }{ + { + "single metric, no dim rollup", + "", + &GroupedMetric{ + Labels: map[string]string{ + "label1": "value1", + }, + Metrics: map[string]*MetricInfo{ + "metric1": { + Value: 1, + Unit: "Count", }, - Fields: map[string]interface{}{ - OTelLib: instrumentationLibName, - "foo": 0, - "label2": "value2", + }, + Metadata: CWMetricMetadata{ + Namespace: namespace, + Timestamp: timestamp, + }, + }, + CWMeasurement{ + Namespace: namespace, + Dimensions: [][]string{{"label1"}}, + Metrics: []map[string]string{ + { + "Name": "metric1", + "Unit": "Count", }, }, }, }, { - "Double sum", - &metricspb.Metric{ - MetricDescriptor: &metricspb.MetricDescriptor{ - Name: "foo", - Type: metricspb.MetricDescriptor_CUMULATIVE_DOUBLE, - Unit: "Count", - LabelKeys: []*metricspb.LabelKey{ - {Key: "label1"}, - {Key: "label2"}, + "multiple metrics, no dim rollup", + "", + &GroupedMetric{ + Labels: map[string]string{ + "label2": "value2", + "label1": "value1", + }, + Metrics: map[string]*MetricInfo{ + "metric1": { + Value: 1, + Unit: "Count", + }, + "metric2": { + Value: 200, + Unit: "Count", + }, + "metric3": { + Value: 3.14, + Unit: "Seconds", }, }, - Timeseries: []*metricspb.TimeSeries{ + Metadata: CWMetricMetadata{ + Namespace: namespace, + Timestamp: timestamp, + }, + }, + CWMeasurement{ + Namespace: namespace, + Dimensions: [][]string{{"label1", "label2"}}, + Metrics: []map[string]string{ { - LabelValues: []*metricspb.LabelValue{ - {Value: "value1", HasValue: true}, - {Value: "value2", HasValue: true}, - }, - Points: []*metricspb.Point{ - { - Value: &metricspb.Point_DoubleValue{ - DoubleValue: 0.1, - }, - }, - }, + "Name": "metric1", + "Unit": "Count", }, { - LabelValues: []*metricspb.LabelValue{ - {HasValue: false}, - {Value: "value2", HasValue: true}, - }, - Points: []*metricspb.Point{ - { - Value: &metricspb.Point_DoubleValue{ - DoubleValue: 0.3, - }, - }, - }, + "Name": "metric2", + "Unit": "Count", + }, + { + "Name": "metric3", + "Unit": "Seconds", }, }, }, - []*CWMetrics{ - { - Measurements: []CwMeasurement{ - { - Namespace: namespace, - Dimensions: [][]string{ - {"label1", "label2", OTelLib}, - }, - Metrics: []map[string]string{ - {"Name": "foo", "Unit": "Count"}, - }, - }, - }, - Fields: map[string]interface{}{ - OTelLib: instrumentationLibName, - "foo": 0, - "label1": "value1", - "label2": "value2", + }, + { + "single metric, single dim rollup", + SingleDimensionRollupOnly, + &GroupedMetric{ + Labels: map[string]string{ + "label1": "value1", + }, + Metrics: map[string]*MetricInfo{ + "metric1": { + Value: 1, + Unit: "Count", }, }, - { - Measurements: []CwMeasurement{ - { - Namespace: namespace, - Dimensions: [][]string{ - {"label2", OTelLib}, - }, - Metrics: []map[string]string{ - {"Name": "foo", "Unit": "Count"}, - }, - }, - }, - Fields: map[string]interface{}{ - OTelLib: instrumentationLibName, - "foo": 0, - "label2": "value2", + Metadata: CWMetricMetadata{ + Namespace: namespace, + Timestamp: timestamp, + }, + }, + CWMeasurement{ + Namespace: namespace, + Dimensions: [][]string{{"label1"}}, + Metrics: []map[string]string{ + { + "Name": "metric1", + "Unit": "Count", }, }, }, }, { - "Double histogram", - &metricspb.Metric{ - MetricDescriptor: &metricspb.MetricDescriptor{ - Name: "foo", - Type: metricspb.MetricDescriptor_CUMULATIVE_DISTRIBUTION, - Unit: "Seconds", - LabelKeys: []*metricspb.LabelKey{ - {Key: "label1"}, - {Key: "label2"}, + "multiple metrics, zero & single dim rollup", + ZeroAndSingleDimensionRollup, + &GroupedMetric{ + Labels: map[string]string{ + "label2": "value2", + "label1": "value1", + }, + Metrics: map[string]*MetricInfo{ + "metric1": { + Value: 1, + Unit: "Count", + }, + "metric2": { + Value: 200, + Unit: "Count", + }, + "metric3": { + Value: 3.14, + Unit: "Seconds", }, }, - Timeseries: []*metricspb.TimeSeries{ + Metadata: CWMetricMetadata{ + Namespace: namespace, + Timestamp: timestamp, + }, + }, + CWMeasurement{ + Namespace: namespace, + Dimensions: [][]string{ + {"label1", "label2"}, + {"label1"}, + {"label2"}, + {}, + }, + Metrics: []map[string]string{ { - LabelValues: []*metricspb.LabelValue{ - {Value: "value1", HasValue: true}, - {Value: "value2", HasValue: true}, - }, - Points: []*metricspb.Point{ - { - Value: &metricspb.Point_DistributionValue{ - DistributionValue: &metricspb.DistributionValue{ - Sum: 15.0, - Count: 5, - BucketOptions: &metricspb.DistributionValue_BucketOptions{ - Type: &metricspb.DistributionValue_BucketOptions_Explicit_{ - Explicit: &metricspb.DistributionValue_BucketOptions_Explicit{ - Bounds: []float64{0, 10}, - }, - }, - }, - Buckets: []*metricspb.DistributionValue_Bucket{ - { - Count: 0, - }, - { - Count: 4, - }, - { - Count: 1, - }, - }, - }, - }, - }, - }, + "Name": "metric1", + "Unit": "Count", }, { - LabelValues: []*metricspb.LabelValue{ - {HasValue: false}, - {Value: "value2", HasValue: true}, - }, - Points: []*metricspb.Point{ - { - Value: &metricspb.Point_DistributionValue{ - DistributionValue: &metricspb.DistributionValue{ - Sum: 35.0, - Count: 18, - BucketOptions: &metricspb.DistributionValue_BucketOptions{ - Type: &metricspb.DistributionValue_BucketOptions_Explicit_{ - Explicit: &metricspb.DistributionValue_BucketOptions_Explicit{ - Bounds: []float64{0, 10}, - }, - }, - }, - Buckets: []*metricspb.DistributionValue_Bucket{ - { - Count: 5, - }, - { - Count: 6, - }, - { - Count: 7, - }, - }, - }, - }, - }, - }, - }, - }, - }, - []*CWMetrics{ - { - Measurements: []CwMeasurement{ - { - Namespace: namespace, - Dimensions: [][]string{ - {"label1", "label2", OTelLib}, - }, - Metrics: []map[string]string{ - {"Name": "foo", "Unit": "Seconds"}, - }, - }, - }, - Fields: map[string]interface{}{ - OTelLib: instrumentationLibName, - "foo": &CWMetricStats{ - Sum: 15.0, - Count: 5, - }, - "label1": "value1", - "label2": "value2", - }, - }, - { - Measurements: []CwMeasurement{ - { - Namespace: namespace, - Dimensions: [][]string{ - {"label2", OTelLib}, - }, - Metrics: []map[string]string{ - {"Name": "foo", "Unit": "Seconds"}, - }, - }, + "Name": "metric2", + "Unit": "Count", }, - Fields: map[string]interface{}{ - OTelLib: instrumentationLibName, - "foo": &CWMetricStats{ - Sum: 35.0, - Count: 18, - }, - "label2": "value2", + { + "Name": "metric3", + "Unit": "Seconds", }, }, }, }, { - "Double summary", - &metricspb.Metric{ - MetricDescriptor: &metricspb.MetricDescriptor{ - Name: "foo", - Type: metricspb.MetricDescriptor_SUMMARY, - Unit: "Seconds", - LabelKeys: []*metricspb.LabelKey{ - {Key: "label1"}, - {Key: "label2"}, - }, + "no metrics", + "", + &GroupedMetric{ + Labels: map[string]string{ + "label1": "value1", }, - Timeseries: []*metricspb.TimeSeries{ - { - LabelValues: []*metricspb.LabelValue{ - {Value: "value1", HasValue: true}, - {Value: "value2", HasValue: true}, - }, - Points: []*metricspb.Point{ - { - Value: &metricspb.Point_SummaryValue{ - SummaryValue: &metricspb.SummaryValue{ - Sum: &wrappers.DoubleValue{ - Value: 15.0, - }, - Count: &wrappers.Int64Value{ - Value: 5, - }, - Snapshot: &metricspb.SummaryValue_Snapshot{ - Count: &wrappers.Int64Value{ - Value: 5, - }, - Sum: &wrappers.DoubleValue{ - Value: 15.0, - }, - PercentileValues: []*metricspb.SummaryValue_Snapshot_ValueAtPercentile{ - { - Percentile: 0.0, - Value: 1, - }, - { - Percentile: 100.0, - Value: 5, - }, - }, - }, - }, - }, - }, - }, - }, - { - LabelValues: []*metricspb.LabelValue{ - {HasValue: false}, - {Value: "value2", HasValue: true}, - }, - Points: []*metricspb.Point{ - { - Value: &metricspb.Point_SummaryValue{ - SummaryValue: &metricspb.SummaryValue{ - Sum: &wrappers.DoubleValue{ - Value: 35.0, - }, - Count: &wrappers.Int64Value{ - Value: 18, - }, - Snapshot: &metricspb.SummaryValue_Snapshot{ - Count: &wrappers.Int64Value{ - Value: 18, - }, - Sum: &wrappers.DoubleValue{ - Value: 35.0, - }, - PercentileValues: []*metricspb.SummaryValue_Snapshot_ValueAtPercentile{ - { - Percentile: 0.0, - Value: 0, - }, - { - Percentile: 100.0, - Value: 5, - }, - }, - }, - }, - }, - }, - }, - }, + Metrics: nil, + Metadata: CWMetricMetadata{ + Namespace: namespace, + Timestamp: timestamp, }, }, - []*CWMetrics{ - { - Measurements: []CwMeasurement{ - { - Namespace: namespace, - Dimensions: [][]string{ - {"label1", "label2", OTelLib}, - }, - Metrics: []map[string]string{ - {"Name": "foo", "Unit": "Seconds"}, - }, - }, - }, - Fields: map[string]interface{}{ - OTelLib: instrumentationLibName, - "foo": &CWMetricStats{ - Min: 1, - Max: 5, - Sum: 15.0, - Count: 5, - }, - "label1": "value1", - "label2": "value2", - }, - }, - { - Measurements: []CwMeasurement{ - { - Namespace: namespace, - Dimensions: [][]string{ - {"label2", OTelLib}, - }, - Metrics: []map[string]string{ - {"Name": "foo", "Unit": "Seconds"}, - }, - }, - }, - Fields: map[string]interface{}{ - OTelLib: instrumentationLibName, - "foo": &CWMetricStats{ - Min: 0, - Max: 5, - Sum: 35.0, - Count: 18, - }, - "label2": "value2", - }, - }, + CWMeasurement{ + Namespace: namespace, + Dimensions: [][]string{{"label1"}}, + Metrics: nil, }, }, } for _, tc := range testCases { t.Run(tc.testName, func(t *testing.T) { - oc := consumerdata.MetricsData{ - Node: &commonpb.Node{}, - Resource: &resourcepb.Resource{ - Labels: map[string]string{ - conventions.AttributeServiceName: "myServiceName", - conventions.AttributeServiceNamespace: "myServiceNS", - }, - }, - Metrics: []*metricspb.Metric{tc.metric}, - } - - // Retrieve *pdata.Metric - rms := internaldata.OCToMetrics(oc).ResourceMetrics() - assert.Equal(t, 1, rms.Len()) - ilms := rms.At(0).InstrumentationLibraryMetrics() - assert.Equal(t, 1, ilms.Len()) - metrics := ilms.At(0).Metrics() - assert.Equal(t, 1, metrics.Len()) - metric := metrics.At(0) - - cwMetrics := getCWMetrics(&metric, namespace, instrumentationLibName, config) - assert.Equal(t, len(tc.expected), len(cwMetrics)) - - for i, expected := range tc.expected { - cwMetric := cwMetrics[i] - assert.Equal(t, len(expected.Measurements), len(cwMetric.Measurements)) - for i, expectedMeasurement := range expected.Measurements { - assertCwMeasurementEqual(t, expectedMeasurement, cwMetric.Measurements[i]) - } - assert.Equal(t, len(expected.Fields), len(cwMetric.Fields)) - assert.Equal(t, expected.Fields, cwMetric.Fields) + config := &Config{ + MetricDeclarations: nil, + DimensionRollupOption: tc.dimensionRollupOption, } + cWMeasurement := groupedMetricToCWMeasurement(tc.groupedMetric, config) + assertCWMeasurementEqual(t, tc.expectedMeasurement, cWMeasurement) }) } - t.Run("Unhandled metric type", func(t *testing.T) { - metric := pdata.NewMetric() - metric.SetName("foo") - metric.SetUnit("Count") - metric.SetDataType(pdata.MetricDataTypeIntHistogram) - - obs, logs := observer.New(zap.WarnLevel) - obsConfig := &Config{ - DimensionRollupOption: "", - logger: zap.New(obs), - } - - cwMetrics := getCWMetrics(&metric, namespace, instrumentationLibName, obsConfig) - assert.Nil(t, cwMetrics) - - // Test output warning logs - expectedLogs := []observer.LoggedEntry{ - { - Entry: zapcore.Entry{Level: zap.WarnLevel, Message: "Unhandled metric data type."}, - Context: []zapcore.Field{ - zap.String("DataType", "IntHistogram"), - zap.String("Name", "foo"), - zap.String("Unit", "Count"), - }, - }, - } - assert.Equal(t, 1, logs.Len()) - assert.Equal(t, expectedLogs, logs.AllUntimed()) - }) - - t.Run("Nil metric", func(t *testing.T) { - cwMetrics := getCWMetrics(nil, namespace, instrumentationLibName, config) - assert.Nil(t, cwMetrics) - }) -} - -func TestBuildCWMetric(t *testing.T) { - namespace := "Namespace" - instrLibName := "InstrLibName" - OTelLib := OTellibDimensionKey - config := &Config{ - Namespace: "", - DimensionRollupOption: "", - } - metricSlice := []map[string]string{ - { - "Name": "foo", - "Unit": "", - }, - } - - // Test data types - metric := pdata.NewMetric() - metric.SetName("foo") - - t.Run("Int gauge", func(t *testing.T) { - metric.SetDataType(pdata.MetricDataTypeIntGauge) - dp := pdata.NewIntDataPoint() - dp.LabelsMap().InitFromMap(map[string]string{ - "label1": "value1", - }) - dp.SetValue(int64(-17)) - - cwMetric := buildCWMetric(dp, &metric, namespace, metricSlice, instrLibName, config) - - assert.NotNil(t, cwMetric) - assert.Equal(t, 1, len(cwMetric.Measurements)) - expectedMeasurement := CwMeasurement{ - Namespace: namespace, - Dimensions: [][]string{{"label1", OTelLib}}, - Metrics: metricSlice, - } - assertCwMeasurementEqual(t, expectedMeasurement, cwMetric.Measurements[0]) - expectedFields := map[string]interface{}{ - OTelLib: instrLibName, - "foo": int64(-17), - "label1": "value1", - } - assert.Equal(t, expectedFields, cwMetric.Fields) - }) - - t.Run("Double gauge", func(t *testing.T) { - metric.SetDataType(pdata.MetricDataTypeDoubleGauge) - dp := pdata.NewDoubleDataPoint() - dp.LabelsMap().InitFromMap(map[string]string{ - "label1": "value1", - }) - dp.SetValue(0.3) - - cwMetric := buildCWMetric(dp, &metric, namespace, metricSlice, instrLibName, config) - - assert.NotNil(t, cwMetric) - assert.Equal(t, 1, len(cwMetric.Measurements)) - expectedMeasurement := CwMeasurement{ - Namespace: namespace, - Dimensions: [][]string{{"label1", OTelLib}}, - Metrics: metricSlice, - } - assertCwMeasurementEqual(t, expectedMeasurement, cwMetric.Measurements[0]) - expectedFields := map[string]interface{}{ - OTelLib: instrLibName, - "foo": 0.3, - "label1": "value1", - } - assert.Equal(t, expectedFields, cwMetric.Fields) - }) - - t.Run("Int sum", func(t *testing.T) { - metric.SetDataType(pdata.MetricDataTypeIntSum) - metric.IntSum().SetAggregationTemporality(pdata.AggregationTemporalityCumulative) - dp := pdata.NewIntDataPoint() - dp.LabelsMap().InitFromMap(map[string]string{ - "label1": "value1", - }) - dp.SetValue(int64(-17)) - - cwMetric := buildCWMetric(dp, &metric, namespace, metricSlice, instrLibName, config) - - assert.NotNil(t, cwMetric) - assert.Equal(t, 1, len(cwMetric.Measurements)) - expectedMeasurement := CwMeasurement{ - Namespace: namespace, - Dimensions: [][]string{{"label1", OTelLib}}, - Metrics: metricSlice, - } - assertCwMeasurementEqual(t, expectedMeasurement, cwMetric.Measurements[0]) - expectedFields := map[string]interface{}{ - OTelLib: instrLibName, - "foo": 0, - "label1": "value1", - } - assert.Equal(t, expectedFields, cwMetric.Fields) - }) - - t.Run("Double sum", func(t *testing.T) { - metric.SetDataType(pdata.MetricDataTypeDoubleSum) - metric.DoubleSum().SetAggregationTemporality(pdata.AggregationTemporalityCumulative) - dp := pdata.NewDoubleDataPoint() - dp.LabelsMap().InitFromMap(map[string]string{ - "label1": "value1", - }) - dp.SetValue(0.3) - - cwMetric := buildCWMetric(dp, &metric, namespace, metricSlice, instrLibName, config) - - assert.NotNil(t, cwMetric) - assert.Equal(t, 1, len(cwMetric.Measurements)) - expectedMeasurement := CwMeasurement{ - Namespace: namespace, - Dimensions: [][]string{{"label1", OTelLib}}, - Metrics: metricSlice, - } - assertCwMeasurementEqual(t, expectedMeasurement, cwMetric.Measurements[0]) - expectedFields := map[string]interface{}{ - OTelLib: instrLibName, - "foo": 0, - "label1": "value1", - } - assert.Equal(t, expectedFields, cwMetric.Fields) - }) - - t.Run("Double histogram", func(t *testing.T) { - metric.SetDataType(pdata.MetricDataTypeDoubleHistogram) - dp := pdata.NewDoubleHistogramDataPoint() - dp.LabelsMap().InitFromMap(map[string]string{ - "label1": "value1", - }) - dp.SetCount(uint64(17)) - dp.SetSum(17.13) - dp.SetBucketCounts([]uint64{1, 2, 3}) - dp.SetExplicitBounds([]float64{1, 2, 3}) - - cwMetric := buildCWMetric(dp, &metric, namespace, metricSlice, instrLibName, config) - - assert.NotNil(t, cwMetric) - assert.Equal(t, 1, len(cwMetric.Measurements)) - expectedMeasurement := CwMeasurement{ - Namespace: namespace, - Dimensions: [][]string{{"label1", OTelLib}}, - Metrics: metricSlice, - } - assertCwMeasurementEqual(t, expectedMeasurement, cwMetric.Measurements[0]) - expectedFields := map[string]interface{}{ - OTelLib: instrLibName, - "foo": &CWMetricStats{ - Sum: 17.13, - Count: 17, - }, - "label1": "value1", - } - assert.Equal(t, expectedFields, cwMetric.Fields) - }) - - t.Run("Invalid datapoint type", func(t *testing.T) { - metric.SetDataType(pdata.MetricDataTypeIntGauge) - dp := pdata.NewIntHistogramDataPoint() - - cwMetric := buildCWMetric(dp, &metric, namespace, metricSlice, instrLibName, config) - assert.Nil(t, cwMetric) - }) - // Test rollup options and labels - testCases := []struct { + instrLibName := "cloudwatch-otel" + rollUpTestCases := []struct { testName string labels map[string]string dimensionRollupOption string expectedDims [][]string }{ { - "Single label w/ no rollup", + "Single label, no rollup, no otel dim", map[string]string{"a": "foo"}, "", [][]string{ - {"a", OTelLib}, + {"a"}, }, }, { - "Single label w/ single rollup", + "Single label, no rollup, w/ otel dim", + map[string]string{ + "a": "foo", + (OTellibDimensionKey): instrLibName, + }, + "", + [][]string{ + {"a", OTellibDimensionKey}, + }, + }, + { + "Single label, single rollup, no otel dim", map[string]string{"a": "foo"}, SingleDimensionRollupOnly, [][]string{ - {"a", OTelLib}, + {"a"}, + }, + }, + { + "Single label, single rollup, w/ otel dim", + map[string]string{ + "a": "foo", + (OTellibDimensionKey): instrLibName, + }, + SingleDimensionRollupOnly, + [][]string{ + {"a", OTellibDimensionKey}, }, }, { - "Single label w/ zero + single rollup", + "Single label, zero + single rollup, no otel dim", map[string]string{"a": "foo"}, ZeroAndSingleDimensionRollup, [][]string{ - {"a", OTelLib}, - {OTelLib}, + {"a"}, + {}, + }, + }, + { + "Single label, zero + single rollup, w/ otel dim", + map[string]string{ + "a": "foo", + (OTellibDimensionKey): instrLibName, + }, + ZeroAndSingleDimensionRollup, + [][]string{ + {"a", OTellibDimensionKey}, + {OTellibDimensionKey}, }, }, { - "Multiple label w/ no rollup", + "Multiple label, no rollup, no otel dim", map[string]string{ "a": "foo", "b": "bar", @@ -1629,11 +1089,24 @@ func TestBuildCWMetric(t *testing.T) { }, "", [][]string{ - {"a", "b", "c", OTelLib}, + {"a", "b", "c"}, + }, + }, + { + "Multiple label, no rollup, w/ otel dim", + map[string]string{ + "a": "foo", + "b": "bar", + "c": "car", + (OTellibDimensionKey): instrLibName, + }, + "", + [][]string{ + {"a", "b", "c", OTellibDimensionKey}, }, }, { - "Multiple label w/ rollup", + "Multiple label, rollup, no otel dim", map[string]string{ "a": "foo", "b": "bar", @@ -1641,60 +1114,454 @@ func TestBuildCWMetric(t *testing.T) { }, ZeroAndSingleDimensionRollup, [][]string{ - {"a", "b", "c", OTelLib}, - {OTelLib, "a"}, - {OTelLib, "b"}, - {OTelLib, "c"}, - {OTelLib}, + {"a", "b", "c"}, + {"a"}, + {"b"}, + {"c"}, + {}, + }, + }, + { + "Multiple label, rollup, w/ otel dim", + map[string]string{ + "a": "foo", + "b": "bar", + "c": "car", + (OTellibDimensionKey): instrLibName, + }, + ZeroAndSingleDimensionRollup, + [][]string{ + {"a", "b", "c", OTellibDimensionKey}, + {OTellibDimensionKey, "a"}, + {OTellibDimensionKey, "b"}, + {OTellibDimensionKey, "c"}, + {OTellibDimensionKey}, + }, + }, + } + + for _, tc := range rollUpTestCases { + t.Run(tc.testName, func(t *testing.T) { + groupedMetric := &GroupedMetric{ + Labels: tc.labels, + Metrics: map[string]*MetricInfo{ + "metric1": { + Value: 1, + Unit: "Count", + }, + }, + Metadata: CWMetricMetadata{ + Namespace: namespace, + Timestamp: timestamp, + }, + } + config := &Config{ + DimensionRollupOption: tc.dimensionRollupOption, + } + cWMeasurement := groupedMetricToCWMeasurement(groupedMetric, config) + assertDimsEqual(t, tc.expectedDims, cWMeasurement.Dimensions) + }) + } +} + +func TestGroupedMetricToCWMeasurementsWithFilters(t *testing.T) { + timestamp := int64(1596151098037) + namespace := "Namespace" + + labels := map[string]string{ + "a": "A", + "b": "B", + "c": "C", + } + metrics := map[string]*MetricInfo{ + "metric1": { + Value: 1, + Unit: "Count", + }, + "metric2": { + Value: 200, + Unit: "Count", + }, + "metric3": { + Value: 3.14, + Unit: "Seconds", + }, + } + testCases := []struct { + testName string + metricDeclarations []*MetricDeclaration + expectedMeasurements []CWMeasurement + }{ + { + "single metric declaration", + []*MetricDeclaration{ + { + Dimensions: [][]string{{"a"}, {"a", "c"}, {"b", "d"}}, + MetricNameSelectors: []string{"metric.*"}, + }, + }, + []CWMeasurement{ + { + Namespace: namespace, + Dimensions: [][]string{{"a"}, {"a", "c"}}, + Metrics: []map[string]string{ + { + "Name": "metric1", + "Unit": "Count", + }, + { + "Name": "metric2", + "Unit": "Count", + }, + { + "Name": "metric3", + "Unit": "Seconds", + }, + }, + }, + }, + }, + { + "multiple metric declarations, all unique", + []*MetricDeclaration{ + { + Dimensions: [][]string{{"a"}, {"b", "d"}}, + MetricNameSelectors: []string{"metric.*"}, + }, + { + Dimensions: [][]string{{"a", "c"}, {"b", "d"}}, + MetricNameSelectors: []string{"metric1"}, + }, + { + Dimensions: [][]string{{"a"}, {"b"}, {"b", "d"}}, + MetricNameSelectors: []string{"metric(1|2)"}, + }, + }, + []CWMeasurement{ + { + Namespace: namespace, + Dimensions: [][]string{{"a"}, {"b"}, {"a", "c"}}, + Metrics: []map[string]string{ + { + "Name": "metric1", + "Unit": "Count", + }, + }, + }, + { + Namespace: namespace, + Dimensions: [][]string{{"a"}, {"b"}}, + Metrics: []map[string]string{ + { + "Name": "metric2", + "Unit": "Count", + }, + }, + }, + { + Namespace: namespace, + Dimensions: [][]string{{"a"}}, + Metrics: []map[string]string{ + { + "Name": "metric3", + "Unit": "Seconds", + }, + }, + }, + }, + }, + { + "multiple metric declarations, hybrid", + []*MetricDeclaration{ + { + Dimensions: [][]string{{"a"}, {"b", "d"}}, + MetricNameSelectors: []string{"metric.*"}, + }, + { + Dimensions: [][]string{{"a"}, {"b"}, {"b", "d"}}, + MetricNameSelectors: []string{"metric(1|2)"}, + }, + }, + []CWMeasurement{ + { + Namespace: namespace, + Dimensions: [][]string{{"a"}, {"b"}}, + Metrics: []map[string]string{ + { + "Name": "metric1", + "Unit": "Count", + }, + { + "Name": "metric2", + "Unit": "Count", + }, + }, + }, + { + Namespace: namespace, + Dimensions: [][]string{{"a"}}, + Metrics: []map[string]string{ + { + "Name": "metric3", + "Unit": "Seconds", + }, + }, + }, + }, + }, + { + "some dimensions match", + []*MetricDeclaration{ + { + Dimensions: [][]string{{"b", "d"}}, + MetricNameSelectors: []string{"metric.*"}, + }, + { + Dimensions: [][]string{{"a"}, {"b"}, {"b", "d"}}, + MetricNameSelectors: []string{"metric(1|2)"}, + }, + }, + []CWMeasurement{ + { + Namespace: namespace, + Dimensions: [][]string{{"a"}, {"b"}}, + Metrics: []map[string]string{ + { + "Name": "metric1", + "Unit": "Count", + }, + { + "Name": "metric2", + "Unit": "Count", + }, + }, + }, + }, + }, + { + "no dimension match", + []*MetricDeclaration{ + { + Dimensions: [][]string{{"b", "d"}}, + MetricNameSelectors: []string{"metric.*"}, + }, + }, + nil, + }, + { + "label matchers", + []*MetricDeclaration{ + { + Dimensions: [][]string{{"a"}, {"a", "c"}, {"b", "d"}}, + MetricNameSelectors: []string{"metric.*"}, + LabelMatchers: []*LabelMatcher{ + { + LabelNames: []string{"a", "b", "d"}, + Regex: "A;B;D", + }, + }, + }, + { + Dimensions: [][]string{{"b"}, {"b", "d"}}, + MetricNameSelectors: []string{"metric.*"}, + LabelMatchers: []*LabelMatcher{ + { + LabelNames: []string{"a", "b"}, + Regex: "A;B", + }, + }, + }, + }, + []CWMeasurement{ + { + Namespace: namespace, + Dimensions: [][]string{{"b"}}, + Metrics: []map[string]string{ + { + "Name": "metric1", + "Unit": "Count", + }, + { + "Name": "metric2", + "Unit": "Count", + }, + { + "Name": "metric3", + "Unit": "Seconds", + }, + }, + }, }, }, } + logger := zap.NewNop() + for _, tc := range testCases { t.Run(tc.testName, func(t *testing.T) { - dp := pdata.NewIntDataPoint() - dp.LabelsMap().InitFromMap(tc.labels) - dp.SetValue(int64(-17)) - config = &Config{ - Namespace: namespace, - DimensionRollupOption: tc.dimensionRollupOption, + groupedMetric := &GroupedMetric{ + Labels: labels, + Metrics: metrics, + Metadata: CWMetricMetadata{ + Namespace: namespace, + Timestamp: timestamp, + }, } - - expectedFields := map[string]interface{}{ - OTellibDimensionKey: OTelLib, - "foo": int64(-17), + config := &Config{ + DimensionRollupOption: "", + MetricDeclarations: tc.metricDeclarations, + logger: logger, } - for k, v := range tc.labels { - expectedFields[k] = v + for _, decl := range tc.metricDeclarations { + err := decl.Init(logger) + assert.Nil(t, err) } - expectedMeasurement := CwMeasurement{ - Namespace: namespace, - Dimensions: tc.expectedDims, - Metrics: metricSlice, + + cWMeasurements := groupedMetricToCWMeasurementsWithFilters(groupedMetric, config) + assert.NotNil(t, cWMeasurements) + assert.Equal(t, len(tc.expectedMeasurements), len(cWMeasurements)) + assertCWMeasurementSliceEqual(t, tc.expectedMeasurements, cWMeasurements) + }) + } + + t.Run("No label match", func(t *testing.T) { + groupedMetric := &GroupedMetric{ + Labels: labels, + Metrics: metrics, + Metadata: CWMetricMetadata{ + Namespace: namespace, + Timestamp: timestamp, + }, + } + metricDeclarations := []*MetricDeclaration{ + { + Dimensions: [][]string{{"b"}, {"b", "d"}}, + MetricNameSelectors: []string{"metric.*"}, + LabelMatchers: []*LabelMatcher{ + { + LabelNames: []string{"a", "b"}, + Regex: "A;C", + }, + }, + }, + { + Dimensions: [][]string{{"b"}, {"b", "d"}}, + MetricNameSelectors: []string{"metric.*"}, + LabelMatchers: []*LabelMatcher{ + { + LabelNames: []string{"a", "b"}, + Regex: "a;B", + }, + }, + }, + } + for _, decl := range metricDeclarations { + err := decl.Init(zap.NewNop()) + assert.Nil(t, err) + } + obs, logs := observer.New(zap.DebugLevel) + logger := zap.New(obs) + config := &Config{ + DimensionRollupOption: "", + MetricDeclarations: metricDeclarations, + logger: logger, + } + + cWMeasurements := groupedMetricToCWMeasurementsWithFilters(groupedMetric, config) + assert.Nil(t, cWMeasurements) + + // Test output warning logs + expectedLog := observer.LoggedEntry{ + Entry: zapcore.Entry{Level: zap.DebugLevel, Message: "Dropped batch of metrics: no metric declaration matched labels"}, + Context: []zapcore.Field{ + zap.String("Labels", "{\"a\":\"A\",\"b\":\"B\",\"c\":\"C\"}"), + zap.Strings("Metric Names", []string{"metric1", "metric2", "metric3"}), + }, + } + assert.Equal(t, 1, logs.Len()) + log := logs.AllUntimed()[0] + // Have to perform this hacky equality check because the metric names might not + // be in the right order due to map iteration + assert.Equal(t, expectedLog.Entry, log.Entry) + assert.Equal(t, 2, len(log.Context)) + assert.Equal(t, expectedLog.Context[0], log.Context[0]) + isMatch := false + possibleOrders := []zapcore.Field{ + zap.Strings("Metric Names", []string{"metric1", "metric2", "metric3"}), + zap.Strings("Metric Names", []string{"metric1", "metric3", "metric2"}), + zap.Strings("Metric Names", []string{"metric2", "metric1", "metric3"}), + zap.Strings("Metric Names", []string{"metric2", "metric3", "metric1"}), + zap.Strings("Metric Names", []string{"metric3", "metric2", "metric1"}), + zap.Strings("Metric Names", []string{"metric3", "metric1", "metric2"}), + } + for _, field := range possibleOrders { + if field.Equals(log.Context[1]) { + isMatch = true + break } + } + assert.True(t, isMatch) + }) - cwMetric := buildCWMetric(dp, &metric, namespace, metricSlice, OTelLib, config) + t.Run("No metric name match", func(t *testing.T) { + groupedMetric := &GroupedMetric{ + Labels: labels, + Metrics: metrics, + Metadata: CWMetricMetadata{ + Namespace: namespace, + Timestamp: timestamp, + }, + } + metricDeclarations := []*MetricDeclaration{ + { + Dimensions: [][]string{{"b"}, {"b", "d"}}, + MetricNameSelectors: []string{"metric4"}, + }, + } + for _, decl := range metricDeclarations { + err := decl.Init(zap.NewNop()) + assert.Nil(t, err) + } + obs, logs := observer.New(zap.DebugLevel) + logger := zap.New(obs) + config := &Config{ + DimensionRollupOption: "", + MetricDeclarations: metricDeclarations, + logger: logger, + } - // Check fields - assert.Equal(t, expectedFields, cwMetric.Fields) + cWMeasurements := groupedMetricToCWMeasurementsWithFilters(groupedMetric, config) + assert.Nil(t, cWMeasurements) - // Check CW measurement - assert.Equal(t, 1, len(cwMetric.Measurements)) - assertCwMeasurementEqual(t, expectedMeasurement, cwMetric.Measurements[0]) - }) - } -} + // Test output warning logs + expectedEntry := zapcore.Entry{Level: zap.DebugLevel, Message: "Dropped metric: no metric declaration matched metric name"} + expectedContexts := []zapcore.Field{ + zap.String("Metric name", "metric1"), + zap.String("Metric name", "metric2"), + zap.String("Metric name", "metric3"), + } + assert.Equal(t, 3, logs.Len()) + // Match logs (possibly out of order) + seen := make([]bool, 3) + for _, log := range logs.AllUntimed() { + assert.Equal(t, expectedEntry, log.Entry) + assert.Equal(t, 1, len(log.Context)) + hasMatch := false + for i, expectedCtx := range expectedContexts { + if !seen[i] && log.Context[0].Equals(expectedCtx) { + hasMatch = true + seen[i] = true + break + } + } + assert.True(t, hasMatch) + } + }) -func TestBuildCWMetricWithMetricDeclarations(t *testing.T) { - namespace := "Namespace" - OTelLib := OTellibDimensionKey - instrumentationLibName := "cloudwatch-otel" + // Test metric filtering with various roll-up options metricName := "metric1" - metricValue := int64(-17) - metric := pdata.NewMetric() - metric.SetName(metricName) - metricSlice := []map[string]string{{"Name": metricName}} - testCases := []struct { + instrLibName := "cloudwatch-otel" + rollupTestCases := []struct { testName string labels map[string]string metricDeclarations []*MetricDeclaration @@ -1703,7 +1570,10 @@ func TestBuildCWMetricWithMetricDeclarations(t *testing.T) { }{ { "Single label w/ no rollup", - map[string]string{"a": "foo"}, + map[string]string{ + "a": "foo", + (OTellibDimensionKey): instrLibName, + }, []*MetricDeclaration{ { Dimensions: [][]string{{"a"}}, @@ -1715,19 +1585,25 @@ func TestBuildCWMetricWithMetricDeclarations(t *testing.T) { }, { "Single label + OTelLib w/ no rollup", - map[string]string{"a": "foo"}, + map[string]string{ + "a": "foo", + (OTellibDimensionKey): instrLibName, + }, []*MetricDeclaration{ { - Dimensions: [][]string{{"a", OTelLib}}, + Dimensions: [][]string{{"a", OTellibDimensionKey}}, MetricNameSelectors: []string{metricName}, }, }, "", - [][]string{{"a", OTelLib}}, + [][]string{{"a", OTellibDimensionKey}}, }, { "Single label w/ single rollup", - map[string]string{"a": "foo"}, + map[string]string{ + "a": "foo", + (OTellibDimensionKey): instrLibName, + }, []*MetricDeclaration{ { Dimensions: [][]string{{"a"}}, @@ -1735,11 +1611,14 @@ func TestBuildCWMetricWithMetricDeclarations(t *testing.T) { }, }, SingleDimensionRollupOnly, - [][]string{{"a"}, {"a", OTelLib}}, + [][]string{{"a"}, {"a", OTellibDimensionKey}}, }, { "Single label w/ zero/single rollup", - map[string]string{"a": "foo"}, + map[string]string{ + "a": "foo", + (OTellibDimensionKey): instrLibName, + }, []*MetricDeclaration{ { Dimensions: [][]string{{"a"}}, @@ -1747,23 +1626,30 @@ func TestBuildCWMetricWithMetricDeclarations(t *testing.T) { }, }, ZeroAndSingleDimensionRollup, - [][]string{{"a"}, {"a", OTelLib}, {OTelLib}}, + [][]string{{"a"}, {"a", OTellibDimensionKey}, {OTellibDimensionKey}}, }, { - "No matching metric name", - map[string]string{"a": "foo"}, + "Single label + Otel w/ zero/single rollup", + map[string]string{ + "a": "foo", + (OTellibDimensionKey): instrLibName, + }, []*MetricDeclaration{ { - Dimensions: [][]string{{"a"}}, - MetricNameSelectors: []string{"invalid"}, + Dimensions: [][]string{{"a", OTellibDimensionKey}}, + MetricNameSelectors: []string{metricName}, }, }, - "", - nil, + ZeroAndSingleDimensionRollup, + [][]string{{"a", OTellibDimensionKey}, {OTellibDimensionKey}}, }, { "multiple labels w/ no rollup", - map[string]string{"a": "foo", "b": "bar"}, + map[string]string{ + "a": "foo", + "b": "bar", + (OTellibDimensionKey): instrLibName, + }, []*MetricDeclaration{ { Dimensions: [][]string{{"a"}}, @@ -1775,7 +1661,11 @@ func TestBuildCWMetricWithMetricDeclarations(t *testing.T) { }, { "multiple labels w/ rollup", - map[string]string{"a": "foo", "b": "bar"}, + map[string]string{ + "a": "foo", + "b": "bar", + (OTellibDimensionKey): instrLibName, + }, []*MetricDeclaration{ { Dimensions: [][]string{{"a"}}, @@ -1785,14 +1675,18 @@ func TestBuildCWMetricWithMetricDeclarations(t *testing.T) { ZeroAndSingleDimensionRollup, [][]string{ {"a"}, - {OTelLib, "a"}, - {OTelLib, "b"}, - {OTelLib}, + {OTellibDimensionKey, "a"}, + {OTellibDimensionKey, "b"}, + {OTellibDimensionKey}, }, }, { "multiple labels + multiple dimensions w/ no rollup", - map[string]string{"a": "foo", "b": "bar"}, + map[string]string{ + "a": "foo", + "b": "bar", + (OTellibDimensionKey): instrLibName, + }, []*MetricDeclaration{ { Dimensions: [][]string{{"a", "b"}, {"b"}}, @@ -1803,20 +1697,28 @@ func TestBuildCWMetricWithMetricDeclarations(t *testing.T) { [][]string{{"a", "b"}, {"b"}}, }, { - "multiple labels + multiple dimensions + OTelLib w/ no rollup", - map[string]string{"a": "foo", "b": "bar"}, + "multiple labels + multiple dimensions + OTellibDimensionKey w/ no rollup", + map[string]string{ + "a": "foo", + "b": "bar", + (OTellibDimensionKey): instrLibName, + }, []*MetricDeclaration{ { - Dimensions: [][]string{{"a", "b"}, {"b", OTelLib}, {OTelLib}}, + Dimensions: [][]string{{"a", "b"}, {"b", OTellibDimensionKey}, {OTellibDimensionKey}}, MetricNameSelectors: []string{metricName}, }, }, "", - [][]string{{"a", "b"}, {"b", OTelLib}, {OTelLib}}, + [][]string{{"a", "b"}, {"b", OTellibDimensionKey}, {OTellibDimensionKey}}, }, { "multiple labels + multiple dimensions w/ rollup", - map[string]string{"a": "foo", "b": "bar"}, + map[string]string{ + "a": "foo", + "b": "bar", + (OTellibDimensionKey): instrLibName, + }, []*MetricDeclaration{ { Dimensions: [][]string{{"a", "b"}, {"b"}}, @@ -1827,14 +1729,18 @@ func TestBuildCWMetricWithMetricDeclarations(t *testing.T) { [][]string{ {"a", "b"}, {"b"}, - {OTelLib, "a"}, - {OTelLib, "b"}, - {OTelLib}, + {OTellibDimensionKey, "a"}, + {OTellibDimensionKey, "b"}, + {OTellibDimensionKey}, }, }, { "multiple labels, multiple dimensions w/ invalid dimension", - map[string]string{"a": "foo", "b": "bar"}, + map[string]string{ + "a": "foo", + "b": "bar", + (OTellibDimensionKey): instrLibName, + }, []*MetricDeclaration{ { Dimensions: [][]string{{"a", "b", "c"}, {"b"}}, @@ -1844,14 +1750,19 @@ func TestBuildCWMetricWithMetricDeclarations(t *testing.T) { ZeroAndSingleDimensionRollup, [][]string{ {"b"}, - {OTelLib, "a"}, - {OTelLib, "b"}, - {OTelLib}, + {OTellibDimensionKey, "a"}, + {OTellibDimensionKey, "b"}, + {OTellibDimensionKey}, }, }, { "multiple labels, multiple dimensions w/ missing dimension", - map[string]string{"a": "foo", "b": "bar", "c": "car"}, + map[string]string{ + "a": "foo", + "b": "bar", + "c": "car", + (OTellibDimensionKey): instrLibName, + }, []*MetricDeclaration{ { Dimensions: [][]string{{"a", "b"}, {"b"}}, @@ -1862,15 +1773,20 @@ func TestBuildCWMetricWithMetricDeclarations(t *testing.T) { [][]string{ {"a", "b"}, {"b"}, - {OTelLib, "a"}, - {OTelLib, "b"}, - {OTelLib, "c"}, - {OTelLib}, + {OTellibDimensionKey, "a"}, + {OTellibDimensionKey, "b"}, + {OTellibDimensionKey, "c"}, + {OTellibDimensionKey}, }, }, { "multiple metric declarations w/ no rollup", - map[string]string{"a": "foo", "b": "bar", "c": "car"}, + map[string]string{ + "a": "foo", + "b": "bar", + "c": "car", + (OTellibDimensionKey): instrLibName, + }, []*MetricDeclaration{ { Dimensions: [][]string{{"a", "b"}, {"b"}}, @@ -1896,7 +1812,12 @@ func TestBuildCWMetricWithMetricDeclarations(t *testing.T) { }, { "multiple metric declarations w/ rollup", - map[string]string{"a": "foo", "b": "bar", "c": "car"}, + map[string]string{ + "a": "foo", + "b": "bar", + "c": "car", + (OTellibDimensionKey): instrLibName, + }, []*MetricDeclaration{ { Dimensions: [][]string{{"a", "b"}, {"b"}}, @@ -1915,10 +1836,10 @@ func TestBuildCWMetricWithMetricDeclarations(t *testing.T) { [][]string{ {"a", "b"}, {"b"}, - {OTelLib, "a"}, - {OTelLib, "b"}, - {OTelLib, "c"}, - {OTelLib}, + {OTellibDimensionKey, "a"}, + {OTellibDimensionKey, "b"}, + {OTellibDimensionKey, "c"}, + {OTellibDimensionKey}, {"a", "c"}, {"c"}, {"b", "c"}, @@ -1926,7 +1847,12 @@ func TestBuildCWMetricWithMetricDeclarations(t *testing.T) { }, { "remove measurements with no dimensions", - map[string]string{"a": "foo", "b": "bar", "c": "car"}, + map[string]string{ + "a": "foo", + "b": "bar", + "c": "car", + (OTellibDimensionKey): instrLibName, + }, []*MetricDeclaration{ { Dimensions: [][]string{{"a", "b"}, {"b"}}, @@ -1945,7 +1871,12 @@ func TestBuildCWMetricWithMetricDeclarations(t *testing.T) { }, { "multiple declarations w/ no dimensions", - map[string]string{"a": "foo", "b": "bar", "c": "car"}, + map[string]string{ + "a": "foo", + "b": "bar", + "c": "car", + (OTellibDimensionKey): instrLibName, + }, []*MetricDeclaration{ { Dimensions: [][]string{{"a", "e"}, {"d"}}, @@ -1973,208 +1904,62 @@ func TestBuildCWMetricWithMetricDeclarations(t *testing.T) { }, } - for _, tc := range testCases { + for _, tc := range rollupTestCases { t.Run(tc.testName, func(t *testing.T) { - dp := pdata.NewIntDataPoint() - dp.LabelsMap().InitFromMap(tc.labels) - dp.SetValue(metricValue) - config := &Config{ - Namespace: namespace, - DimensionRollupOption: tc.dimensionRollupOption, - MetricDeclarations: tc.metricDeclarations, + groupedMetric := &GroupedMetric{ + Labels: tc.labels, + Metrics: map[string]*MetricInfo{ + (metricName): { + Value: int64(5), + Unit: "Count", + }, + }, + Metadata: CWMetricMetadata{ + Namespace: namespace, + Timestamp: timestamp, + }, } - logger := zap.NewNop() - for _, m := range tc.metricDeclarations { - err := m.Init(logger) + for _, decl := range tc.metricDeclarations { + err := decl.Init(zap.NewNop()) assert.Nil(t, err) } - - expectedFields := map[string]interface{}{ - OTellibDimensionKey: instrumentationLibName, - metricName: metricValue, - } - for k, v := range tc.labels { - expectedFields[k] = v + config := &Config{ + DimensionRollupOption: tc.dimensionRollupOption, + MetricDeclarations: tc.metricDeclarations, + logger: zap.NewNop(), } - cwMetric := buildCWMetric(dp, &metric, namespace, metricSlice, instrumentationLibName, config) - - // Check fields - assert.Equal(t, expectedFields, cwMetric.Fields) - - // Check CW measurement - if tc.expectedDims == nil { - assert.Equal(t, 0, len(cwMetric.Measurements)) + cWMeasurements := groupedMetricToCWMeasurementsWithFilters(groupedMetric, config) + if len(tc.expectedDims) == 0 { + assert.Equal(t, 0, len(cWMeasurements)) } else { - assert.Equal(t, 1, len(cwMetric.Measurements)) - expectedMeasurement := CwMeasurement{ - Namespace: namespace, - Dimensions: tc.expectedDims, - Metrics: metricSlice, - } - assertCwMeasurementEqual(t, expectedMeasurement, cwMetric.Measurements[0]) + assert.Equal(t, 1, len(cWMeasurements)) + dims := cWMeasurements[0].Dimensions + assertDimsEqual(t, tc.expectedDims, dims) } }) } } -func TestCalculateRate(t *testing.T) { - prevValue := int64(0) - curValue := int64(10) +func TestTranslateCWMetricToEMFNoMeasurements(t *testing.T) { + timestamp := int64(1596151098037) fields := make(map[string]interface{}) fields[OTellibDimensionKey] = "cloudwatch-otel" fields["spanName"] = "test" - fields["spanCounter"] = prevValue - fields["type"] = "Int64" - prevTime := time.Now().UnixNano() / int64(time.Millisecond) - curTime := time.Unix(0, prevTime*int64(time.Millisecond)).Add(time.Second*10).UnixNano() / int64(time.Millisecond) - rate := calculateRate(fields, prevValue, prevTime) - assert.Equal(t, 0, rate) - rate = calculateRate(fields, curValue, curTime) - assert.Equal(t, int64(1), rate) - - prevDoubleValue := 0.0 - curDoubleValue := 5.0 - fields["type"] = "Float64" - rate = calculateRate(fields, prevDoubleValue, prevTime) - assert.Equal(t, 0, rate) - rate = calculateRate(fields, curDoubleValue, curTime) - assert.Equal(t, 0.5, rate) -} - -func TestDimensionRollup(t *testing.T) { - testCases := []struct { - testName string - dimensionRollupOption string - dims []string - instrumentationLibName string - expected [][]string - }{ - { - "no rollup w/o instrumentation library name", - "", - []string{"a", "b", "c"}, - noInstrumentationLibraryName, - nil, - }, - { - "no rollup w/ instrumentation library name", - "", - []string{"a", "b", "c"}, - "cloudwatch-otel", - nil, - }, - { - "single dim w/o instrumentation library name", - SingleDimensionRollupOnly, - []string{"a", "b", "c"}, - noInstrumentationLibraryName, - [][]string{ - {"a"}, - {"b"}, - {"c"}, - }, - }, - { - "single dim w/ instrumentation library name", - SingleDimensionRollupOnly, - []string{"a", "b", "c"}, - "cloudwatch-otel", - [][]string{ - {OTellibDimensionKey, "a"}, - {OTellibDimensionKey, "b"}, - {OTellibDimensionKey, "c"}, - }, - }, - { - "single dim w/o instrumentation library name and only one label", - SingleDimensionRollupOnly, - []string{"a"}, - noInstrumentationLibraryName, - [][]string{{"a"}}, - }, - { - "single dim w/ instrumentation library name and only one label", - SingleDimensionRollupOnly, - []string{"a"}, - "cloudwatch-otel", - [][]string{{OTellibDimensionKey, "a"}}, - }, - { - "zero + single dim w/o instrumentation library name", - ZeroAndSingleDimensionRollup, - []string{"a", "b", "c"}, - noInstrumentationLibraryName, - [][]string{ - {}, - {"a"}, - {"b"}, - {"c"}, - }, - }, - { - "zero + single dim w/ instrumentation library name", - ZeroAndSingleDimensionRollup, - []string{"a", "b", "c", "A"}, - "cloudwatch-otel", - [][]string{ - {OTellibDimensionKey}, - {OTellibDimensionKey, "a"}, - {OTellibDimensionKey, "b"}, - {OTellibDimensionKey, "c"}, - {OTellibDimensionKey, "A"}, - }, - }, - { - "zero dim rollup w/o instrumentation library name and no labels", - ZeroAndSingleDimensionRollup, - []string{}, - noInstrumentationLibraryName, - nil, - }, - { - "zero dim rollup w/ instrumentation library name and no labels", - ZeroAndSingleDimensionRollup, - []string{}, - "cloudwatch-otel", - nil, - }, - } + fields["spanCounter"] = 0 - for _, tc := range testCases { - t.Run(tc.testName, func(t *testing.T) { - rolledUp := dimensionRollup(tc.dimensionRollupOption, tc.dims, tc.instrumentationLibName) - assertDimsEqual(t, tc.expected, rolledUp) - }) + met := &CWMetrics{ + Timestamp: timestamp, + Fields: fields, + Measurements: nil, } -} - -func TestNeedsCalculateRate(t *testing.T) { - metric := pdata.NewMetric() - metric.SetDataType(pdata.MetricDataTypeIntGauge) - assert.False(t, needsCalculateRate(&metric)) - metric.SetDataType(pdata.MetricDataTypeDoubleGauge) - assert.False(t, needsCalculateRate(&metric)) - - metric.SetDataType(pdata.MetricDataTypeIntHistogram) - assert.False(t, needsCalculateRate(&metric)) - metric.SetDataType(pdata.MetricDataTypeDoubleHistogram) - assert.False(t, needsCalculateRate(&metric)) - - metric.SetDataType(pdata.MetricDataTypeIntSum) - metric.IntSum().SetAggregationTemporality(pdata.AggregationTemporalityCumulative) - assert.True(t, needsCalculateRate(&metric)) - metric.IntSum().SetAggregationTemporality(pdata.AggregationTemporalityDelta) - assert.False(t, needsCalculateRate(&metric)) + inputLogEvent := TranslateCWMetricToEMF(met) + expected := "{\"OTelLib\":\"cloudwatch-otel\",\"spanCounter\":0,\"spanName\":\"test\"}" - metric.SetDataType(pdata.MetricDataTypeDoubleSum) - metric.DoubleSum().SetAggregationTemporality(pdata.AggregationTemporalityCumulative) - assert.True(t, needsCalculateRate(&metric)) - metric.DoubleSum().SetAggregationTemporality(pdata.AggregationTemporalityDelta) - assert.False(t, needsCalculateRate(&metric)) + assert.Equal(t, expected, *inputLogEvent.InputLogEvent.Message) } -func BenchmarkTranslateOtToCWMetricWithInstrLibrary(b *testing.B) { +func BenchmarkTranslateOtToGroupedMetricWithInstrLibrary(b *testing.B) { md := createMetricTestData() rm := internaldata.OCToMetrics(md).ResourceMetrics().At(0) ilms := rm.InstrumentationLibraryMetrics() @@ -2183,54 +1968,106 @@ func BenchmarkTranslateOtToCWMetricWithInstrLibrary(b *testing.B) { config := &Config{ Namespace: "", DimensionRollupOption: ZeroAndSingleDimensionRollup, + logger: zap.NewNop(), } b.ResetTimer() for n := 0; n < b.N; n++ { - TranslateOtToCWMetric(&rm, config) + groupedMetric := make(map[string]*GroupedMetric) + TranslateOtToGroupedMetric(&rm, groupedMetric, config) } } -func BenchmarkTranslateOtToCWMetricWithoutInstrLibrary(b *testing.B) { +func BenchmarkTranslateOtToGroupedMetricWithoutInstrLibrary(b *testing.B) { md := createMetricTestData() rm := internaldata.OCToMetrics(md).ResourceMetrics().At(0) config := &Config{ Namespace: "", DimensionRollupOption: ZeroAndSingleDimensionRollup, + logger: zap.NewNop(), } b.ResetTimer() for n := 0; n < b.N; n++ { - TranslateOtToCWMetric(&rm, config) + groupedMetrics := make(map[string]*GroupedMetric) + TranslateOtToGroupedMetric(&rm, groupedMetrics, config) } } -func BenchmarkTranslateOtToCWMetricWithFiltering(b *testing.B) { - md := createMetricTestData() - rm := internaldata.OCToMetrics(md).ResourceMetrics().At(0) - ilms := rm.InstrumentationLibraryMetrics() - ilm := ilms.At(0) - ilm.InstrumentationLibrary().SetName("cloudwatch-lib") - m := MetricDeclaration{ - Dimensions: [][]string{{"spanName"}}, - MetricNameSelectors: []string{"spanCounter", "spanGaugeCounter"}, +func BenchmarkTranslateGroupedMetricToCWMetric(b *testing.B) { + groupedMetric := &GroupedMetric{ + Labels: map[string]string{ + "label1": "value1", + "label2": "value2", + "label3": "value3", + }, + Metrics: map[string]*MetricInfo{ + "metric1": { + Value: 1, + Unit: "Count", + }, + "metric2": { + Value: 200, + Unit: "Seconds", + }, + }, + Metadata: CWMetricMetadata{ + Namespace: "Namespace", + Timestamp: int64(1596151098037), + }, + } + config := &Config{ + MetricDeclarations: nil, + DimensionRollupOption: ZeroAndSingleDimensionRollup, + } + + b.ResetTimer() + for n := 0; n < b.N; n++ { + TranslateGroupedMetricToCWMetric(groupedMetric, config) + } +} + +func BenchmarkTranslateGroupedMetricToCWMetricWithFiltering(b *testing.B) { + groupedMetric := &GroupedMetric{ + Labels: map[string]string{ + "label1": "value1", + "label2": "value2", + "label3": "value3", + }, + Metrics: map[string]*MetricInfo{ + "metric1": { + Value: 1, + Unit: "Count", + }, + "metric2": { + Value: 200, + Unit: "Seconds", + }, + }, + Metadata: CWMetricMetadata{ + Namespace: "Namespace", + Timestamp: int64(1596151098037), + }, + } + m := &MetricDeclaration{ + Dimensions: [][]string{{"label1"}, {"label2"}}, + MetricNameSelectors: []string{"metric1", "metric2"}, } logger := zap.NewNop() m.Init(logger) config := &Config{ - Namespace: "", + MetricDeclarations: []*MetricDeclaration{m}, DimensionRollupOption: ZeroAndSingleDimensionRollup, - MetricDeclarations: []*MetricDeclaration{&m}, } b.ResetTimer() for n := 0; n < b.N; n++ { - TranslateOtToCWMetric(&rm, config) + TranslateGroupedMetricToCWMetric(groupedMetric, config) } } func BenchmarkTranslateCWMetricToEMF(b *testing.B) { - cwMeasurement := CwMeasurement{ + cwMeasurement := CWMeasurement{ Namespace: "test-emf", Dimensions: [][]string{{OTellibDimensionKey}, {OTellibDimensionKey, "spanName"}}, Metrics: []map[string]string{{ @@ -2247,19 +2084,11 @@ func BenchmarkTranslateCWMetricToEMF(b *testing.B) { met := &CWMetrics{ Timestamp: timestamp, Fields: fields, - Measurements: []CwMeasurement{cwMeasurement}, + Measurements: []CWMeasurement{cwMeasurement}, } - logger := zap.NewNop() b.ResetTimer() for n := 0; n < b.N; n++ { - TranslateCWMetricToEMF([]*CWMetrics{met}, logger) - } -} - -func BenchmarkDimensionRollup(b *testing.B) { - dimensions := []string{"a", "b", "c"} - for n := 0; n < b.N; n++ { - dimensionRollup(ZeroAndSingleDimensionRollup, dimensions, "cloudwatch-otel") + TranslateCWMetricToEMF(met) } } diff --git a/exporter/awsemfexporter/pusher_test.go b/exporter/awsemfexporter/pusher_test.go index 79af0dfa75c58..b61a975d37200 100644 --- a/exporter/awsemfexporter/pusher_test.go +++ b/exporter/awsemfexporter/pusher_test.go @@ -79,7 +79,7 @@ func TestLogEventBatch_sortLogEvents(t *testing.T) { logEvent := NewLogEvent( int64(timestamp), fmt.Sprintf("message%v", timestamp)) - fmt.Printf("logEvents[%d].Timetsmap=%d.\n", i, timestamp) + fmt.Printf("logEvents[%d].Timestamp=%d.\n", i, timestamp) logEventBatch.PutLogEventsInput.LogEvents = append(logEventBatch.PutLogEventsInput.LogEvents, logEvent.InputLogEvent) } diff --git a/exporter/awsemfexporter/util.go b/exporter/awsemfexporter/util.go index 7e567e43c1841..9cb9b7306e821 100644 --- a/exporter/awsemfexporter/util.go +++ b/exporter/awsemfexporter/util.go @@ -15,9 +15,12 @@ package awsemfexporter import ( + "fmt" + "sort" "strings" "go.opentelemetry.io/collector/consumer/pdata" + "go.opentelemetry.io/collector/translator/conventions" "go.uber.org/zap" ) @@ -55,3 +58,119 @@ func replace(s, pattern string, value pdata.AttributeValue, logger *zap.Logger) } return strings.Replace(s, pattern, value.StringVal(), -1) } + +// getNamespace retrieves namespace for given set of metrics from user config. +func getNamespace(rm *pdata.ResourceMetrics, namespace string) string { + if len(namespace) == 0 { + serviceName, svcNameOk := rm.Resource().Attributes().Get(conventions.AttributeServiceName) + serviceNamespace, svcNsOk := rm.Resource().Attributes().Get(conventions.AttributeServiceNamespace) + if svcNameOk && svcNsOk && serviceName.Type() == pdata.AttributeValueSTRING && serviceNamespace.Type() == pdata.AttributeValueSTRING { + namespace = fmt.Sprintf("%s/%s", serviceNamespace.StringVal(), serviceName.StringVal()) + } else if svcNameOk && serviceName.Type() == pdata.AttributeValueSTRING { + namespace = serviceName.StringVal() + } else if svcNsOk && serviceNamespace.Type() == pdata.AttributeValueSTRING { + namespace = serviceNamespace.StringVal() + } + } + + if len(namespace) == 0 { + namespace = defaultNamespace + } + return namespace +} + +// getLogInfo retrieves the log group and log stream names from a given set of metrics. +func getLogInfo(rm *pdata.ResourceMetrics, cWNamespace string, config *Config) (logGroup, logStream string) { + if cWNamespace != "" { + logGroup = fmt.Sprintf("/metrics/%s", cWNamespace) + } + + // Override log group/stream if specified in config. However, in this case, customer won't have correlation experience + if len(config.LogGroupName) > 0 { + logGroup = replacePatterns(config.LogGroupName, rm.Resource().Attributes(), config.logger) + } + if len(config.LogStreamName) > 0 { + logStream = replacePatterns(config.LogStreamName, rm.Resource().Attributes(), config.logger) + } + + return +} + +// createMetricKey generates a hashed key from metric labels and additional parameters +func createMetricKey(labels map[string]string, parameters map[string]string) string { + var sb strings.Builder + keys := make([]string, 0, len(labels)+len(parameters)) + values := make(map[string]string, len(labels)+len(parameters)) + + for k, v := range labels { + keys = append(keys, k) + values[k] = v + } + for k, v := range parameters { + keys = append(keys, k) + values[k] = v + } + + sort.Strings(keys) + for i, key := range keys { + keyValuePair := key + ":" + values[key] + sb.WriteString(keyValuePair) + if i < len(keys)-1 { + sb.WriteString(",") + } + } + + return sb.String() +} + +// dedupDimensions removes duplicated dimension sets from the given dimensions. +// Prerequisite: each dimension set is already sorted +func dedupDimensions(dimensions [][]string) (deduped [][]string) { + seen := make(map[string]bool) + for _, dimSet := range dimensions { + key := strings.Join(dimSet, ",") + // Only add dimension set if not a duplicate + if _, ok := seen[key]; !ok { + deduped = append(deduped, dimSet) + seen[key] = true + } + } + return +} + +// dimensionRollup creates rolled-up dimensions from the metric's label set. +// The returned dimensions are sorted in alphabetical order within each dimension set +func dimensionRollup(dimensionRollupOption string, labels map[string]string) [][]string { + var rollupDimensionArray [][]string + dimensionZero := make([]string, 0) + + instrLibName, hasOTelKey := labels[OTellibDimensionKey] + if hasOTelKey { + // If OTel key exists in labels, add it as a zero dimension but remove it + // temporarily from labels as it is not an original label + dimensionZero = []string{OTellibDimensionKey} + delete(labels, OTellibDimensionKey) + } + + if dimensionRollupOption == ZeroAndSingleDimensionRollup { + //"Zero" dimension rollup + if len(labels) > 0 { + rollupDimensionArray = append(rollupDimensionArray, dimensionZero) + } + } + if dimensionRollupOption == ZeroAndSingleDimensionRollup || dimensionRollupOption == SingleDimensionRollupOnly { + //"One" dimension rollup + for labelName := range labels { + dimSet := append(dimensionZero, labelName) + sort.Strings(dimSet) + rollupDimensionArray = append(rollupDimensionArray, dimSet) + } + } + + // Add back OTel key to labels if it was removed + if hasOTelKey { + labels[OTellibDimensionKey] = instrLibName + } + + return rollupDimensionArray +} diff --git a/exporter/awsemfexporter/util_test.go b/exporter/awsemfexporter/util_test.go index cc122a3895f2d..887254376fd6c 100644 --- a/exporter/awsemfexporter/util_test.go +++ b/exporter/awsemfexporter/util_test.go @@ -17,11 +17,31 @@ package awsemfexporter import ( "testing" + commonpb "github.com/census-instrumentation/opencensus-proto/gen-go/agent/common/v1" + resourcepb "github.com/census-instrumentation/opencensus-proto/gen-go/resource/v1" "github.com/stretchr/testify/assert" + "go.opentelemetry.io/collector/consumer/consumerdata" "go.opentelemetry.io/collector/consumer/pdata" + "go.opentelemetry.io/collector/translator/conventions" + "go.opentelemetry.io/collector/translator/internaldata" "go.uber.org/zap" ) +// assertDimsSorted asserts whether each dimension set within dims +// is sorted alphabetically. +func assertDimsSorted(t *testing.T, dims [][]string) { + for _, dimSet := range dims { + if len(dimSet) <= 1 { + continue + } + prevDim := dimSet[0] + for _, dim := range dimSet[1:] { + assert.True(t, prevDim <= dim) + prevDim = dim + } + } +} + func TestReplacePatternValidTaskId(t *testing.T) { logger := zap.NewNop() @@ -111,3 +131,407 @@ func TestReplacePatternNilAttrValue(t *testing.T) { assert.Equal(t, "/aws/ecs/containerinsights/undefined/performance", s) } + +func TestGetNamespace(t *testing.T) { + defaultMetric := createMetricTestData() + testCases := []struct { + testName string + metric consumerdata.MetricsData + configNamespace string + namespace string + }{ + { + "non-empty namespace", + defaultMetric, + "namespace", + "namespace", + }, + { + "empty namespace", + defaultMetric, + "", + "myServiceNS/myServiceName", + }, + { + "empty namespace, no service namespace", + consumerdata.MetricsData{ + Resource: &resourcepb.Resource{ + Labels: map[string]string{ + conventions.AttributeServiceName: "myServiceName", + }, + }, + }, + "", + "myServiceName", + }, + { + "empty namespace, no service name", + consumerdata.MetricsData{ + Resource: &resourcepb.Resource{ + Labels: map[string]string{ + conventions.AttributeServiceNamespace: "myServiceNS", + }, + }, + }, + "", + "myServiceNS", + }, + } + + for _, tc := range testCases { + t.Run(tc.testName, func(t *testing.T) { + rms := internaldata.OCToMetrics(tc.metric) + rm := rms.ResourceMetrics().At(0) + namespace := getNamespace(&rm, tc.configNamespace) + assert.Equal(t, tc.namespace, namespace) + }) + } +} + +func TestGetLogInfo(t *testing.T) { + metric := consumerdata.MetricsData{ + Node: &commonpb.Node{ + ServiceInfo: &commonpb.ServiceInfo{Name: "test-emf"}, + LibraryInfo: &commonpb.LibraryInfo{ExporterVersion: "SomeVersion"}, + }, + Resource: &resourcepb.Resource{ + Labels: map[string]string{ + "aws.ecs.cluster.name": "test-cluster-name", + "aws.ecs.task.id": "test-task-id", + }, + }, + } + rm := internaldata.OCToMetrics(metric).ResourceMetrics().At(0) + + testCases := []struct { + testName string + namespace string + configLogGroup string + configLogStream string + logGroup string + logStream string + }{ + { + "non-empty namespace, no config", + "namespace", + "", + "", + "/metrics/namespace", + "", + }, + { + "empty namespace, no config", + "", + "", + "", + "", + "", + }, + { + "non-empty namespace, config w/o pattern", + "namespace", + "test-logGroupName", + "test-logStreamName", + "test-logGroupName", + "test-logStreamName", + }, + { + "empty namespace, config w/o pattern", + "", + "test-logGroupName", + "test-logStreamName", + "test-logGroupName", + "test-logStreamName", + }, + { + "non-empty namespace, config w/ pattern", + "namespace", + "/aws/ecs/containerinsights/{ClusterName}/performance", + "{TaskId}", + "/aws/ecs/containerinsights/test-cluster-name/performance", + "test-task-id", + }, + { + "empty namespace, config w/ pattern", + "", + "/aws/ecs/containerinsights/{ClusterName}/performance", + "{TaskId}", + "/aws/ecs/containerinsights/test-cluster-name/performance", + "test-task-id", + }, + } + + for _, tc := range testCases { + t.Run(tc.testName, func(t *testing.T) { + config := &Config{ + LogGroupName: tc.configLogGroup, + LogStreamName: tc.configLogStream, + } + logGroup, logStream := getLogInfo(&rm, tc.namespace, config) + assert.Equal(t, tc.logGroup, logGroup) + assert.Equal(t, tc.logStream, logStream) + }) + } +} + +func TestCreateMetricKey(t *testing.T) { + testCases := []struct { + testName string + labels map[string]string + params map[string]string + expectedKey string + }{ + { + "single label w/o params", + map[string]string{ + "a": "A", + }, + nil, + "a:A", + }, + { + "single label w/ params", + map[string]string{ + "a": "A", + }, + map[string]string{ + "param1": "foo", + }, + "a:A,param1:foo", + }, + { + "multiple labels w/o params", + map[string]string{ + "b": "B", + "a": "A", + "c": "C", + }, + nil, + "a:A,b:B,c:C", + }, + { + "multiple labels w/ params", + map[string]string{ + "b": "B", + "a": "A", + "c": "C", + }, + map[string]string{ + "param1": "foo", + "bar": "car", + "apple": "banana", + }, + "a:A,apple:banana,b:B,bar:car,c:C,param1:foo", + }, + } + + for _, tc := range testCases { + t.Run(tc.testName, func(t *testing.T) { + key := createMetricKey(tc.labels, tc.params) + assert.Equal(t, tc.expectedKey, key) + }) + } +} + +func TestDedupDimensions(t *testing.T) { + testCases := []struct { + testName string + dimensions [][]string + deduped [][]string + }{ + { + "single dimension", + [][]string{{"dim1"}}, + [][]string{{"dim1"}}, + }, + { + "multiple dimensions w/o duplicates", + [][]string{{"dim1"}, {"dim2"}, {"dim1", "dim2"}}, + [][]string{{"dim1"}, {"dim2"}, {"dim1", "dim2"}}, + }, + { + "multiple dimensions w/ duplicates", + [][]string{{"dim1"}, {"dim2"}, {"dim1", "dim2"}, {"dim1", "dim2"}}, + [][]string{{"dim1"}, {"dim2"}, {"dim1", "dim2"}}, + }, + } + + for _, tc := range testCases { + t.Run(tc.testName, func(t *testing.T) { + dedupedDims := dedupDimensions(tc.dimensions) + assertDimsEqual(t, tc.deduped, dedupedDims) + }) + } +} + +func TestDimensionRollup(t *testing.T) { + testCases := []struct { + testName string + dimensionRollupOption string + labels map[string]string + expected [][]string + }{ + { + "no rollup w/o instrumentation library name", + "", + map[string]string{ + "a": "A", + "b": "B", + "c": "C", + }, + nil, + }, + { + "no rollup w/ instrumentation library name", + "", + map[string]string{ + "a": "A", + "b": "B", + "c": "C", + (OTellibDimensionKey): "cloudwatch-otel", + }, + nil, + }, + { + "single dim w/o instrumentation library name", + SingleDimensionRollupOnly, + map[string]string{ + "a": "A", + "b": "B", + "c": "C", + }, + [][]string{ + {"a"}, + {"b"}, + {"c"}, + }, + }, + { + "single dim w/ instrumentation library name", + SingleDimensionRollupOnly, + map[string]string{ + "a": "A", + "b": "B", + "c": "C", + (OTellibDimensionKey): "cloudwatch-otel", + }, + [][]string{ + {OTellibDimensionKey, "a"}, + {OTellibDimensionKey, "b"}, + {OTellibDimensionKey, "c"}, + }, + }, + { + "single dim w/o instrumentation library name and only one label", + SingleDimensionRollupOnly, + map[string]string{ + "a": "A", + }, + [][]string{{"a"}}, + }, + { + "single dim w/ instrumentation library name and only one label", + SingleDimensionRollupOnly, + map[string]string{ + "a": "A", + (OTellibDimensionKey): "cloudwatch-otel", + }, + [][]string{{OTellibDimensionKey, "a"}}, + }, + { + "zero + single dim w/o instrumentation library name", + ZeroAndSingleDimensionRollup, + map[string]string{ + "a": "A", + "b": "B", + "c": "C", + }, + [][]string{ + {}, + {"a"}, + {"b"}, + {"c"}, + }, + }, + { + "zero + single dim w/ instrumentation library name", + ZeroAndSingleDimensionRollup, + map[string]string{ + "a": "A", + "b": "B", + "c": "C", + "D": "d", + (OTellibDimensionKey): "cloudwatch-otel", + }, + [][]string{ + {OTellibDimensionKey}, + {OTellibDimensionKey, "a"}, + {OTellibDimensionKey, "b"}, + {OTellibDimensionKey, "c"}, + {OTellibDimensionKey, "D"}, + }, + }, + { + "zero dim rollup w/o instrumentation library name and no labels", + ZeroAndSingleDimensionRollup, + nil, + nil, + }, + { + "zero dim rollup w/ instrumentation library name and no labels", + ZeroAndSingleDimensionRollup, + map[string]string{ + (OTellibDimensionKey): "cloudwatch-otel", + }, + nil, + }, + } + + for _, tc := range testCases { + t.Run(tc.testName, func(t *testing.T) { + numLabels := len(tc.labels) + rolledUp := dimensionRollup(tc.dimensionRollupOption, tc.labels) + // Ensure dimension sets are sorted + assertDimsSorted(t, rolledUp) + assertDimsEqual(t, tc.expected, rolledUp) + // Ensure labels are not changed + assert.Equal(t, numLabels, len(tc.labels)) + }) + } +} + +func BenchmarkCreateMetricKey(b *testing.B) { + labels := map[string]string{ + "a": "A", + "b": "B", + "c": "C", + (OTellibDimensionKey): "cloudwatch-otel", + } + params := map[string]string{ + "param1": "foo", + "bar": "car", + "apple": "banana", + } + m := make(map[string]string) + b.ResetTimer() + for n := 0; n < b.N; n++ { + key := createMetricKey(labels, params) + m[key] = "1" + if _, ok := m[key]; !ok { + b.FailNow() + } + } +} + +func BenchmarkDimensionRollup(b *testing.B) { + labels := map[string]string{ + "a": "A", + "b": "B", + "c": "C", + (OTellibDimensionKey): "cloudwatch-otel", + } + b.ResetTimer() + for n := 0; n < b.N; n++ { + dimensionRollup(ZeroAndSingleDimensionRollup, labels) + } +} From 9cd18ee5346e703ce8af2150bdeff832da1dd26c Mon Sep 17 00:00:00 2001 From: Raphael Koh Date: Tue, 15 Dec 2020 17:32:29 -0500 Subject: [PATCH 2/3] Use metric timestamp --- exporter/awsemfexporter/datapoint.go | 41 ++++++++--- exporter/awsemfexporter/emf_exporter_test.go | 14 ++-- exporter/awsemfexporter/go.mod | 1 + exporter/awsemfexporter/groupedmetric.go | 4 ++ exporter/awsemfexporter/groupedmetric_test.go | 69 +++++++++++++++++++ exporter/awsemfexporter/util.go | 6 ++ 6 files changed, 117 insertions(+), 18 deletions(-) diff --git a/exporter/awsemfexporter/datapoint.go b/exporter/awsemfexporter/datapoint.go index 6b721152a7a94..dc6e3f01d61b8 100644 --- a/exporter/awsemfexporter/datapoint.go +++ b/exporter/awsemfexporter/datapoint.go @@ -32,8 +32,9 @@ var currentState = mapwithexpiry.NewMapWithExpiry(CleanInterval) // DataPoint represents a processed metric data point type DataPoint struct { - Value interface{} - Labels map[string]string + Value interface{} + Labels map[string]string + Timestamp int64 } // DataPoints is a wrapper interface for: @@ -90,17 +91,24 @@ type DoubleSummaryDataPointSlice struct { func (dps IntDataPointSlice) At(i int) DataPoint { metric := dps.IntDataPointSlice.At(i) labels := createLabels(metric.LabelsMap(), dps.instrumentationLibraryName) + timestamp := unixNanoToMilliseconds(metric.Timestamp()) var metricVal interface{} metricVal = metric.Value() if dps.needsCalculateRate { rateKey := createMetricKey(labels, dps.rateKeyParams) - metricVal = calculateRate(rateKey, metricVal, dps.timestamp) + rateTS := dps.timestamp + if timestamp > 0 { + // Use metric timestamp if available + rateTS = timestamp + } + metricVal = calculateRate(rateKey, metricVal, rateTS) } return DataPoint{ - Value: metricVal, - Labels: labels, + Value: metricVal, + Labels: labels, + Timestamp: timestamp, } } @@ -108,17 +116,24 @@ func (dps IntDataPointSlice) At(i int) DataPoint { func (dps DoubleDataPointSlice) At(i int) DataPoint { metric := dps.DoubleDataPointSlice.At(i) labels := createLabels(metric.LabelsMap(), dps.instrumentationLibraryName) + timestamp := unixNanoToMilliseconds(metric.Timestamp()) var metricVal interface{} metricVal = metric.Value() if dps.needsCalculateRate { rateKey := createMetricKey(labels, dps.rateKeyParams) - metricVal = calculateRate(rateKey, metricVal, dps.timestamp) + rateTS := dps.timestamp + if timestamp > 0 { + // Use metric timestamp if available + rateTS = timestamp + } + metricVal = calculateRate(rateKey, metricVal, rateTS) } return DataPoint{ - Value: metricVal, - Labels: labels, + Value: metricVal, + Labels: labels, + Timestamp: timestamp, } } @@ -126,6 +141,7 @@ func (dps DoubleDataPointSlice) At(i int) DataPoint { func (dps DoubleHistogramDataPointSlice) At(i int) DataPoint { metric := dps.DoubleHistogramDataPointSlice.At(i) labels := createLabels(metric.LabelsMap(), dps.instrumentationLibraryName) + timestamp := unixNanoToMilliseconds(metric.Timestamp()) var minBound, maxBound float64 bucketBounds := metric.ExplicitBounds() @@ -141,7 +157,8 @@ func (dps DoubleHistogramDataPointSlice) At(i int) DataPoint { Count: metric.Count(), Sum: metric.Sum(), }, - Labels: labels, + Labels: labels, + Timestamp: timestamp, } } @@ -149,6 +166,7 @@ func (dps DoubleHistogramDataPointSlice) At(i int) DataPoint { func (dps DoubleSummaryDataPointSlice) At(i int) DataPoint { metric := dps.DoubleSummaryDataPointSlice.At(i) labels := createLabels(metric.LabelsMap(), dps.instrumentationLibraryName) + timestamp := unixNanoToMilliseconds(metric.Timestamp()) metricVal := &CWMetricStats{ Count: metric.Count(), @@ -160,8 +178,9 @@ func (dps DoubleSummaryDataPointSlice) At(i int) DataPoint { } return DataPoint{ - Value: metricVal, - Labels: labels, + Value: metricVal, + Labels: labels, + Timestamp: timestamp, } } diff --git a/exporter/awsemfexporter/emf_exporter_test.go b/exporter/awsemfexporter/emf_exporter_test.go index 14bb832efd733..a6e7c6e13df76 100644 --- a/exporter/awsemfexporter/emf_exporter_test.go +++ b/exporter/awsemfexporter/emf_exporter_test.go @@ -105,7 +105,7 @@ func TestConsumeMetrics(t *testing.T) { Points: []*metricspb.Point{ { Timestamp: ×tamp.Timestamp{ - Seconds: 100, + Seconds: int64(1608068109), }, Value: &metricspb.Point_Int64Value{ Int64Value: 1, @@ -166,7 +166,7 @@ func TestConsumeMetricsWithLogGroupStreamConfig(t *testing.T) { Points: []*metricspb.Point{ { Timestamp: ×tamp.Timestamp{ - Seconds: 100, + Seconds: int64(1608068109), }, Value: &metricspb.Point_Int64Value{ Int64Value: 1, @@ -228,7 +228,7 @@ func TestConsumeMetricsWithoutLogGroupStreamConfig(t *testing.T) { Points: []*metricspb.Point{ { Timestamp: ×tamp.Timestamp{ - Seconds: 100, + Seconds: int64(1608068109), }, Value: &metricspb.Point_Int64Value{ Int64Value: 1, @@ -297,7 +297,7 @@ func TestConsumeMetricsWithLogGroupStreamValidPlaceholder(t *testing.T) { Points: []*metricspb.Point{ { Timestamp: ×tamp.Timestamp{ - Seconds: 100, + Seconds: int64(1608068109), }, Value: &metricspb.Point_Int64Value{ Int64Value: 1, @@ -366,7 +366,7 @@ func TestConsumeMetricsWithOnlyLogStreamPlaceholder(t *testing.T) { Points: []*metricspb.Point{ { Timestamp: ×tamp.Timestamp{ - Seconds: 100, + Seconds: int64(1608068109), }, Value: &metricspb.Point_Int64Value{ Int64Value: 1, @@ -435,7 +435,7 @@ func TestConsumeMetricsWithWrongPlaceholder(t *testing.T) { Points: []*metricspb.Point{ { Timestamp: ×tamp.Timestamp{ - Seconds: 100, + Seconds: int64(1608068109), }, Value: &metricspb.Point_Int64Value{ Int64Value: 1, @@ -512,7 +512,7 @@ func TestPushMetricsDataWithErr(t *testing.T) { Points: []*metricspb.Point{ { Timestamp: ×tamp.Timestamp{ - Seconds: 100, + Seconds: int64(1608068109), }, Value: &metricspb.Point_Int64Value{ Int64Value: 1, diff --git a/exporter/awsemfexporter/go.mod b/exporter/awsemfexporter/go.mod index fb10ca094b74a..30de0df0b1400 100644 --- a/exporter/awsemfexporter/go.mod +++ b/exporter/awsemfexporter/go.mod @@ -10,4 +10,5 @@ require ( github.com/stretchr/testify v1.6.1 go.opentelemetry.io/collector v0.16.1-0.20201207152538-326931de8c32 go.uber.org/zap v1.16.0 + google.golang.org/protobuf v1.25.0 ) diff --git a/exporter/awsemfexporter/groupedmetric.go b/exporter/awsemfexporter/groupedmetric.go index c3dc79308dbdb..319feaeddec12 100644 --- a/exporter/awsemfexporter/groupedmetric.go +++ b/exporter/awsemfexporter/groupedmetric.go @@ -62,6 +62,10 @@ func addToGroupedMetric(pmd *pdata.Metric, groupedMetrics map[string]*GroupedMet Unit: pmd.Unit(), } + if dp.Timestamp > 0 { + metadata.Timestamp = dp.Timestamp + } + // Extra params to use when grouping metrics groupKeyParams := map[string]string{ (namespaceKey): metadata.Namespace, diff --git a/exporter/awsemfexporter/groupedmetric_test.go b/exporter/awsemfexporter/groupedmetric_test.go index 4f379b2b5ef5a..ff34912683005 100644 --- a/exporter/awsemfexporter/groupedmetric_test.go +++ b/exporter/awsemfexporter/groupedmetric_test.go @@ -29,6 +29,7 @@ import ( "go.uber.org/zap" "go.uber.org/zap/zapcore" "go.uber.org/zap/zaptest/observer" + "google.golang.org/protobuf/types/known/timestamppb" ) func TestAddToGroupedMetric(t *testing.T) { @@ -211,6 +212,74 @@ func TestAddToGroupedMetric(t *testing.T) { } }) + t.Run("Add multiple metrics w/ different timestamps", func(t *testing.T) { + groupedMetrics := make(map[string]*GroupedMetric) + oc := consumerdata.MetricsData{ + Node: &commonpb.Node{}, + Resource: &resourcepb.Resource{ + Labels: map[string]string{ + conventions.AttributeServiceName: "myServiceName", + conventions.AttributeServiceNamespace: "myServiceNS", + }, + }, + Metrics: []*metricspb.Metric{ + generateTestIntGauge("int-gauge"), + generateTestDoubleGauge("double-gauge"), + generateTestIntSum("int-sum"), + generateTestSummary("summary"), + }, + } + + timestamp1 := ×tamppb.Timestamp{ + Seconds: int64(1608068109), + Nanos: 347942000, + } + timestamp2 := ×tamppb.Timestamp{ + Seconds: int64(1608068110), + Nanos: 347942000, + } + + // Give int gauge and int-sum the same timestamp + oc.Metrics[0].Timeseries[0].Points[0].Timestamp = timestamp1 + oc.Metrics[2].Timeseries[0].Points[0].Timestamp = timestamp1 + // Give summary a different timestamp + oc.Metrics[3].Timeseries[0].Points[0].Timestamp = timestamp2 + + rm := internaldata.OCToMetrics(oc) + rms := rm.ResourceMetrics() + ilms := rms.At(0).InstrumentationLibraryMetrics() + metrics := ilms.At(0).Metrics() + assert.Equal(t, 4, metrics.Len()) + + for i := 0; i < metrics.Len(); i++ { + metric := metrics.At(i) + addToGroupedMetric(&metric, groupedMetrics, metadata, logger) + } + + assert.Equal(t, 3, len(groupedMetrics)) + for _, group := range groupedMetrics { + for metricName := range group.Metrics { + if metricName == "int-gauge" || metricName == "int-sum" { + assert.Equal(t, 2, len(group.Metrics)) + assert.Equal(t, int64(1608068109347), group.Metadata.Timestamp) + } else if metricName == "summary" { + assert.Equal(t, 1, len(group.Metrics)) + assert.Equal(t, int64(1608068110347), group.Metadata.Timestamp) + } else { + // double-gauge should use the default timestamp + assert.Equal(t, 1, len(group.Metrics)) + assert.Equal(t, "double-gauge", metricName) + assert.Equal(t, timestamp, group.Metadata.Timestamp) + } + } + expectedLabels := map[string]string{ + (OTellibDimensionKey): "cloudwatch-otel", + "label1": "value1", + } + assert.Equal(t, expectedLabels, group.Labels) + } + }) + t.Run("Add same metric but different log group", func(t *testing.T) { groupedMetrics := make(map[string]*GroupedMetric) oc := consumerdata.MetricsData{ diff --git a/exporter/awsemfexporter/util.go b/exporter/awsemfexporter/util.go index 9cb9b7306e821..a99e1782d922b 100644 --- a/exporter/awsemfexporter/util.go +++ b/exporter/awsemfexporter/util.go @@ -18,6 +18,7 @@ import ( "fmt" "sort" "strings" + "time" "go.opentelemetry.io/collector/consumer/pdata" "go.opentelemetry.io/collector/translator/conventions" @@ -174,3 +175,8 @@ func dimensionRollup(dimensionRollupOption string, labels map[string]string) [][ return rollupDimensionArray } + +// unixNanoToMilliseconds converts a timestamp in nanoseconds to milliseconds. +func unixNanoToMilliseconds(timestamp pdata.TimestampUnixNano) int64 { + return int64(uint64(timestamp) / uint64(time.Millisecond)) +} From eb1cde878b4e8e9edbc402658744dd1f9be55817 Mon Sep 17 00:00:00 2001 From: Raphael Koh Date: Wed, 23 Dec 2020 13:43:05 -0500 Subject: [PATCH 3/3] Remove max and min from histogram datapoint --- exporter/awsemfexporter/datapoint.go | 10 +--------- exporter/awsemfexporter/datapoint_test.go | 2 -- exporter/awsemfexporter/groupedmetric_test.go | 2 -- exporter/awsemfexporter/metric_translator_test.go | 2 -- 4 files changed, 1 insertion(+), 15 deletions(-) diff --git a/exporter/awsemfexporter/datapoint.go b/exporter/awsemfexporter/datapoint.go index dc6e3f01d61b8..5293c9445799c 100644 --- a/exporter/awsemfexporter/datapoint.go +++ b/exporter/awsemfexporter/datapoint.go @@ -42,6 +42,7 @@ type DataPoint struct { // - pdata.DoubleDataPointSlice // - pdata.IntHistogramDataPointSlice // - pdata.DoubleHistogramDataPointSlice +// - pdata.DoubleSummaryDataPointSlice type DataPoints interface { Len() int // NOTE: At() is an expensive call as it calculates the metric's value @@ -143,17 +144,8 @@ func (dps DoubleHistogramDataPointSlice) At(i int) DataPoint { labels := createLabels(metric.LabelsMap(), dps.instrumentationLibraryName) timestamp := unixNanoToMilliseconds(metric.Timestamp()) - var minBound, maxBound float64 - bucketBounds := metric.ExplicitBounds() - if len(bucketBounds) > 0 { - minBound = bucketBounds[0] - maxBound = bucketBounds[len(bucketBounds)-1] - } - return DataPoint{ Value: &CWMetricStats{ - Min: minBound, - Max: maxBound, Count: metric.Count(), Sum: metric.Sum(), }, diff --git a/exporter/awsemfexporter/datapoint_test.go b/exporter/awsemfexporter/datapoint_test.go index 70fab67625497..f571377d0cf8a 100644 --- a/exporter/awsemfexporter/datapoint_test.go +++ b/exporter/awsemfexporter/datapoint_test.go @@ -383,8 +383,6 @@ func TestDoubleHistogramDataPointSliceAt(t *testing.T) { expectedDP := DataPoint{ Value: &CWMetricStats{ - Min: 1, - Max: 3, Sum: 17.13, Count: 17, }, diff --git a/exporter/awsemfexporter/groupedmetric_test.go b/exporter/awsemfexporter/groupedmetric_test.go index ff34912683005..864aef038367c 100644 --- a/exporter/awsemfexporter/groupedmetric_test.go +++ b/exporter/awsemfexporter/groupedmetric_test.go @@ -97,8 +97,6 @@ func TestAddToGroupedMetric(t *testing.T) { map[string]*MetricInfo{ "foo": { Value: &CWMetricStats{ - Min: 0, - Max: 10, Count: 18, Sum: 35.0, }, diff --git a/exporter/awsemfexporter/metric_translator_test.go b/exporter/awsemfexporter/metric_translator_test.go index c6c13370b1902..d8e8e74090102 100644 --- a/exporter/awsemfexporter/metric_translator_test.go +++ b/exporter/awsemfexporter/metric_translator_test.go @@ -425,8 +425,6 @@ func TestTranslateOtToGroupedMetric(t *testing.T) { timerMetrics := map[string]*MetricInfo{ "spanTimer": { Value: &CWMetricStats{ - Max: 10, - Min: 0, Count: 5, Sum: 15, },