1
0
mirror of https://github.com/open-telemetry/opentelemetry-go.git synced 2025-09-16 09:26:25 +02:00

Batch Metrics Exported From the OTLP Exporter (#626)

* Add check to sum transform for unknown NumberKind

* Initial batching

* Move CheckpointSet transform to internal package

* Add tests for the Exporter Export method

Check batching and general output exporter ResourceMetrics are correct.

* Check errors in tests

* Apply suggestions from code review

Co-Authored-By: Krzesimir Nowak <qdlacz@gmail.com>

* Use var instead of multiple calls for group IDs

* Fix otlp metric test reporting

Co-authored-by: Krzesimir Nowak <qdlacz@gmail.com>
This commit is contained in:
Tyler Yahn
2020-04-15 12:04:44 -07:00
committed by GitHub
parent a8f7b3247b
commit ebc245b388
4 changed files with 951 additions and 116 deletions

View File

@@ -17,20 +17,212 @@
package transform package transform
import ( import (
"context"
"errors" "errors"
"fmt"
"strings"
"sync"
commonpb "github.com/open-telemetry/opentelemetry-proto/gen/go/common/v1" commonpb "github.com/open-telemetry/opentelemetry-proto/gen/go/common/v1"
metricpb "github.com/open-telemetry/opentelemetry-proto/gen/go/metrics/v1" metricpb "github.com/open-telemetry/opentelemetry-proto/gen/go/metrics/v1"
resourcepb "github.com/open-telemetry/opentelemetry-proto/gen/go/resource/v1"
"go.opentelemetry.io/otel/api/core" "go.opentelemetry.io/otel/api/core"
"go.opentelemetry.io/otel/api/metric" "go.opentelemetry.io/otel/api/metric"
export "go.opentelemetry.io/otel/sdk/export/metric" export "go.opentelemetry.io/otel/sdk/export/metric"
"go.opentelemetry.io/otel/sdk/export/metric/aggregator" "go.opentelemetry.io/otel/sdk/export/metric/aggregator"
"go.opentelemetry.io/otel/sdk/resource"
) )
// ErrUnimplementedAgg is returned when a transformation of an unimplemented var (
// aggregator is attempted. // ErrUnimplementedAgg is returned when a transformation of an unimplemented
var ErrUnimplementedAgg = errors.New("unimplemented aggregator") // aggregator is attempted.
ErrUnimplementedAgg = errors.New("unimplemented aggregator")
// ErrUnknownValueType is returned when a transformation of an unknown value
// is attempted.
ErrUnknownValueType = errors.New("invalid value type")
// ErrContextCanceled is returned when a context cancellation halts a
// transformation.
ErrContextCanceled = errors.New("context canceled")
// ErrTransforming is returned when an unexected error is encoutered transforming.
ErrTransforming = errors.New("transforming failed")
)
// result is the product of transforming Records into OTLP Metrics.
type result struct {
Resource resource.Resource
Library string
Metric *metricpb.Metric
Err error
}
// CheckpointSet transforms all records contained in a checkpoint into
// batched OTLP ResourceMetrics.
func CheckpointSet(ctx context.Context, cps export.CheckpointSet, numWorkers uint) ([]*metricpb.ResourceMetrics, error) {
records, errc := source(ctx, cps)
// Start a fixed number of goroutines to transform records.
transformed := make(chan result)
var wg sync.WaitGroup
wg.Add(int(numWorkers))
for i := uint(0); i < numWorkers; i++ {
go func() {
defer wg.Done()
transformer(ctx, records, transformed)
}()
}
go func() {
wg.Wait()
close(transformed)
}()
// Synchronously collect the transformed records and transmit.
rms, err := sink(ctx, transformed)
if err != nil {
return nil, err
}
// source is complete, check for any errors.
if err := <-errc; err != nil {
return nil, err
}
return rms, nil
}
// source starts a goroutine that sends each one of the Records yielded by
// the CheckpointSet on the returned chan. Any error encoutered will be sent
// on the returned error chan after seeding is complete.
func source(ctx context.Context, cps export.CheckpointSet) (<-chan export.Record, <-chan error) {
errc := make(chan error, 1)
out := make(chan export.Record)
// Seed records into process.
go func() {
defer close(out)
// No select is needed since errc is buffered.
errc <- cps.ForEach(func(r export.Record) error {
select {
case <-ctx.Done():
return ErrContextCanceled
case out <- r:
}
return nil
})
}()
return out, errc
}
// transformer transforms records read from the passed in chan into
// OTLP Metrics which are sent on the out chan.
func transformer(ctx context.Context, in <-chan export.Record, out chan<- result) {
for r := range in {
m, err := Record(r)
// Propagate errors, but do not send empty results.
if err == nil && m == nil {
continue
}
res := result{
Resource: r.Descriptor().Resource(),
Library: r.Descriptor().LibraryName(),
Metric: m,
Err: err,
}
select {
case <-ctx.Done():
return
case out <- res:
}
}
}
// sink collects transformed Records and batches them.
//
// Any errors encoutered transforming input will be reported with an
// ErrTransforming as well as the completed ResourceMetrics. It is up to the
// caller to handle any incorrect data in these ResourceMetrics.
func sink(ctx context.Context, in <-chan result) ([]*metricpb.ResourceMetrics, error) {
var errStrings []string
type resourceBatch struct {
Resource *resourcepb.Resource
// Group by instrumentation library name and then the MetricDescriptor.
InstrumentationLibraryBatches map[string]map[string]*metricpb.Metric
}
// group by unique Resource string.
grouped := make(map[string]resourceBatch)
for res := range in {
if res.Err != nil {
errStrings = append(errStrings, res.Err.Error())
continue
}
rID := res.Resource.String()
rb, ok := grouped[rID]
if !ok {
rb = resourceBatch{
Resource: Resource(&res.Resource),
InstrumentationLibraryBatches: make(map[string]map[string]*metricpb.Metric),
}
grouped[rID] = rb
}
mb, ok := rb.InstrumentationLibraryBatches[res.Library]
if !ok {
mb = make(map[string]*metricpb.Metric)
rb.InstrumentationLibraryBatches[res.Library] = mb
}
mID := res.Metric.GetMetricDescriptor().String()
m, ok := mb[mID]
if !ok {
mb[mID] = res.Metric
continue
}
if len(res.Metric.Int64DataPoints) > 0 {
m.Int64DataPoints = append(m.Int64DataPoints, res.Metric.Int64DataPoints...)
}
if len(res.Metric.DoubleDataPoints) > 0 {
m.DoubleDataPoints = append(m.DoubleDataPoints, res.Metric.DoubleDataPoints...)
}
if len(res.Metric.HistogramDataPoints) > 0 {
m.HistogramDataPoints = append(m.HistogramDataPoints, res.Metric.HistogramDataPoints...)
}
if len(res.Metric.SummaryDataPoints) > 0 {
m.SummaryDataPoints = append(m.SummaryDataPoints, res.Metric.SummaryDataPoints...)
}
}
if len(grouped) == 0 {
return nil, nil
}
var rms []*metricpb.ResourceMetrics
for _, rb := range grouped {
rm := &metricpb.ResourceMetrics{Resource: rb.Resource}
for ilName, mb := range rb.InstrumentationLibraryBatches {
ilm := &metricpb.InstrumentationLibraryMetrics{
Metrics: make([]*metricpb.Metric, 0, len(mb)),
}
if ilName != "" {
ilm.InstrumentationLibrary = &commonpb.InstrumentationLibrary{Name: ilName}
}
for _, m := range mb {
ilm.Metrics = append(ilm.Metrics, m)
}
rm.InstrumentationLibraryMetrics = append(rm.InstrumentationLibraryMetrics, ilm)
}
rms = append(rms, rm)
}
// Report any transform errors.
if len(errStrings) > 0 {
return rms, fmt.Errorf("%w:\n -%s", ErrTransforming, strings.Join(errStrings, "\n -"))
}
return rms, nil
}
// Record transforms a Record into an OTLP Metric. An ErrUnimplementedAgg // Record transforms a Record into an OTLP Metric. An ErrUnimplementedAgg
// error is returned if the Record Aggregator is not supported. // error is returned if the Record Aggregator is not supported.
@@ -42,8 +234,9 @@ func Record(r export.Record) (*metricpb.Metric, error) {
return minMaxSumCount(d, l, a) return minMaxSumCount(d, l, a)
case aggregator.Sum: case aggregator.Sum:
return sum(d, l, a) return sum(d, l, a)
default:
return nil, fmt.Errorf("%w: %v", ErrUnimplementedAgg, a)
} }
return nil, ErrUnimplementedAgg
} }
// sum transforms a Sum Aggregator into an OTLP Metric. // sum transforms a Sum Aggregator into an OTLP Metric.
@@ -73,6 +266,8 @@ func sum(desc *metric.Descriptor, labels export.Labels, a aggregator.Sum) (*metr
m.DoubleDataPoints = []*metricpb.DoubleDataPoint{ m.DoubleDataPoints = []*metricpb.DoubleDataPoint{
{Value: sum.CoerceToFloat64(n)}, {Value: sum.CoerceToFloat64(n)},
} }
default:
return nil, fmt.Errorf("%w: %v", ErrUnknownValueType, n)
} }
return m, nil return m, nil

View File

@@ -16,6 +16,7 @@ package transform
import ( import (
"context" "context"
"errors"
"testing" "testing"
commonpb "github.com/open-telemetry/opentelemetry-proto/gen/go/common/v1" commonpb "github.com/open-telemetry/opentelemetry-proto/gen/go/common/v1"
@@ -282,3 +283,14 @@ func TestSumFloat64DataPoints(t *testing.T) {
assert.Equal(t, []*metricpb.SummaryDataPoint(nil), m.SummaryDataPoints) assert.Equal(t, []*metricpb.SummaryDataPoint(nil), m.SummaryDataPoints)
} }
} }
func TestSumErrUnknownValueType(t *testing.T) {
desc := metric.NewDescriptor("", metric.MeasureKind, core.NumberKind(-1))
labels := export.NewSimpleLabels(export.NoopLabelEncoder{})
s := sumAgg.New()
_, err := sum(&desc, labels, s)
assert.Error(t, err)
if !errors.Is(err, ErrUnknownValueType) {
t.Errorf("expected ErrUnknownValueType, got %v", err)
}
}

View File

@@ -19,7 +19,6 @@ import (
"context" "context"
"errors" "errors"
"fmt" "fmt"
"strings"
"sync" "sync"
"unsafe" "unsafe"
@@ -28,11 +27,9 @@ import (
colmetricpb "github.com/open-telemetry/opentelemetry-proto/gen/go/collector/metrics/v1" colmetricpb "github.com/open-telemetry/opentelemetry-proto/gen/go/collector/metrics/v1"
coltracepb "github.com/open-telemetry/opentelemetry-proto/gen/go/collector/trace/v1" coltracepb "github.com/open-telemetry/opentelemetry-proto/gen/go/collector/trace/v1"
metricpb "github.com/open-telemetry/opentelemetry-proto/gen/go/metrics/v1"
"go.opentelemetry.io/otel/exporters/otlp/internal/transform" "go.opentelemetry.io/otel/exporters/otlp/internal/transform"
metricsdk "go.opentelemetry.io/otel/sdk/export/metric" metricsdk "go.opentelemetry.io/otel/sdk/export/metric"
"go.opentelemetry.io/otel/sdk/export/metric/aggregator"
tracesdk "go.opentelemetry.io/otel/sdk/export/trace" tracesdk "go.opentelemetry.io/otel/sdk/export/trace"
) )
@@ -84,8 +81,11 @@ func NewUnstartedExporter(opts ...ExporterOption) *Exporter {
} }
var ( var (
errAlreadyStarted = errors.New("already started") errAlreadyStarted = errors.New("already started")
errNotStarted = errors.New("not started") errNotStarted = errors.New("not started")
errDisconnected = errors.New("exporter disconnected")
errStopped = errors.New("exporter stopped")
errContextCanceled = errors.New("context canceled")
) )
// Start dials to the collector, establishing a connection to it. It also // Start dials to the collector, establishing a connection to it. It also
@@ -209,133 +209,45 @@ func (e *Exporter) Stop() error {
} }
// Export implements the "go.opentelemetry.io/otel/sdk/export/metric".Exporter // Export implements the "go.opentelemetry.io/otel/sdk/export/metric".Exporter
// interface. It transforms metric Records into OTLP Metrics and transmits them. // interface. It transforms and batches metric Records into OTLP Metrics and
func (e *Exporter) Export(ctx context.Context, cps metricsdk.CheckpointSet) error { // transmits them to the configured collector.
// Seed records into the work processing pool. func (e *Exporter) Export(parent context.Context, cps metricsdk.CheckpointSet) error {
records := make(chan metricsdk.Record) // Unify the parent context Done signal with the exporter stopCh.
go func() { ctx, cancel := context.WithCancel(parent)
_ = cps.ForEach(func(record metricsdk.Record) (err error) { defer cancel()
select { go func(ctx context.Context, cancel context.CancelFunc) {
case <-e.stopCh:
case <-ctx.Done():
case records <- record:
}
return
})
close(records)
}()
// Allow all errors to be collected and returned singularly.
errCh := make(chan error)
var errStrings []string
go func() {
for err := range errCh {
if err != nil {
errStrings = append(errStrings, err.Error())
}
}
}()
// Start the work processing pool.
processed := make(chan *metricpb.Metric)
var wg sync.WaitGroup
for i := uint(0); i < e.c.numWorkers; i++ {
wg.Add(1)
go func() {
defer wg.Done()
e.processMetrics(ctx, processed, errCh, records)
}()
}
go func() {
wg.Wait()
close(processed)
}()
// Synchronosly collected the processed records and transmit.
e.uploadMetrics(ctx, processed, errCh)
// Now that all processing is done, handle any errors seen.
close(errCh)
if len(errStrings) > 0 {
return fmt.Errorf("errors exporting:\n -%s", strings.Join(errStrings, "\n -"))
}
return nil
}
func (e *Exporter) processMetrics(ctx context.Context, out chan<- *metricpb.Metric, errCh chan<- error, in <-chan metricsdk.Record) {
for r := range in {
m, err := transform.Record(r)
if err != nil {
if err == aggregator.ErrNoData {
// The Aggregator was checkpointed before the first value
// was set, skipping.
continue
}
select {
case <-e.stopCh:
return
case <-ctx.Done():
return
case errCh <- err:
continue
}
}
select { select {
case <-e.stopCh:
return
case <-ctx.Done(): case <-ctx.Done():
return case <-e.stopCh:
case out <- m: cancel()
} }
} }(ctx, cancel)
}
func (e *Exporter) uploadMetrics(ctx context.Context, in <-chan *metricpb.Metric, errCh chan<- error) { rms, err := transform.CheckpointSet(ctx, cps, e.c.numWorkers)
var protoMetrics []*metricpb.Metric if err != nil {
for m := range in { return err
protoMetrics = append(protoMetrics, m)
} }
if len(protoMetrics) == 0 {
return
}
if !e.connected() { if !e.connected() {
return return errDisconnected
}
rm := []*metricpb.ResourceMetrics{
{
Resource: nil,
InstrumentationLibraryMetrics: []*metricpb.InstrumentationLibraryMetrics{
{
Metrics: protoMetrics,
},
},
},
} }
select { select {
case <-e.stopCh: case <-e.stopCh:
return return errStopped
case <-ctx.Done(): case <-ctx.Done():
return return errContextCanceled
default: default:
e.senderMu.Lock() e.senderMu.Lock()
_, err := e.metricExporter.Export(ctx, &colmetricpb.ExportMetricsServiceRequest{ _, err := e.metricExporter.Export(ctx, &colmetricpb.ExportMetricsServiceRequest{
ResourceMetrics: rm, ResourceMetrics: rms,
}) })
e.senderMu.Unlock() e.senderMu.Unlock()
if err != nil { if err != nil {
select { return err
case <-e.stopCh:
return
case <-ctx.Done():
return
case errCh <- err:
}
} }
} }
return nil
} }
func (e *Exporter) ExportSpan(ctx context.Context, sd *tracesdk.SpanData) { func (e *Exporter) ExportSpan(ctx context.Context, sd *tracesdk.SpanData) {

View File

@@ -0,0 +1,716 @@
// Copyright The 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 otlp
import (
"context"
"testing"
colmetricpb "github.com/open-telemetry/opentelemetry-proto/gen/go/collector/metrics/v1"
commonpb "github.com/open-telemetry/opentelemetry-proto/gen/go/common/v1"
metricpb "github.com/open-telemetry/opentelemetry-proto/gen/go/metrics/v1"
resourcepb "github.com/open-telemetry/opentelemetry-proto/gen/go/resource/v1"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"go.opentelemetry.io/otel/api/core"
"go.opentelemetry.io/otel/api/key"
"go.opentelemetry.io/otel/api/metric"
metricsdk "go.opentelemetry.io/otel/sdk/export/metric"
"go.opentelemetry.io/otel/sdk/export/metric/aggregator"
"go.opentelemetry.io/otel/sdk/metric/aggregator/minmaxsumcount"
"go.opentelemetry.io/otel/sdk/metric/aggregator/sum"
"go.opentelemetry.io/otel/sdk/resource"
"google.golang.org/grpc"
)
type metricsServiceClientStub struct {
rm []metricpb.ResourceMetrics
}
func (m *metricsServiceClientStub) Export(ctx context.Context, in *colmetricpb.ExportMetricsServiceRequest, opts ...grpc.CallOption) (*colmetricpb.ExportMetricsServiceResponse, error) {
for _, rm := range in.GetResourceMetrics() {
if rm == nil {
continue
}
m.rm = append(m.rm, *rm)
}
return &colmetricpb.ExportMetricsServiceResponse{}, nil
}
func (m *metricsServiceClientStub) ResourceMetrics() []metricpb.ResourceMetrics {
return m.rm
}
func (m *metricsServiceClientStub) Reset() {
m.rm = nil
}
type checkpointSet struct {
records []metricsdk.Record
}
func (m checkpointSet) ForEach(fn func(metricsdk.Record) error) error {
for _, r := range m.records {
if err := fn(r); err != nil && err != aggregator.ErrNoData {
return err
}
}
return nil
}
type record struct {
name string
mKind metric.Kind
nKind core.NumberKind
opts []metric.Option
labels []core.KeyValue
}
var (
baseKeyValues = []core.KeyValue{key.String("host", "test.com")}
cpuKey = core.Key("CPU")
testInstA = resource.New(key.String("instance", "tester-a"))
testInstB = resource.New(key.String("instance", "tester-b"))
cpu1MD = &metricpb.MetricDescriptor{
Name: "int64-count",
Type: metricpb.MetricDescriptor_COUNTER_INT64,
Labels: []*commonpb.StringKeyValue{
{
Key: "host",
Value: "test.com",
},
{
Key: "CPU",
Value: "1",
},
},
}
cpu2MD = &metricpb.MetricDescriptor{
Name: "int64-count",
Type: metricpb.MetricDescriptor_COUNTER_INT64,
Labels: []*commonpb.StringKeyValue{
{
Key: "host",
Value: "test.com",
},
{
Key: "CPU",
Value: "2",
},
},
}
testerAResource = &resourcepb.Resource{
Attributes: []*commonpb.AttributeKeyValue{
{
Key: "instance",
Type: commonpb.AttributeKeyValue_STRING,
StringValue: "tester-a",
},
},
}
testerBResource = &resourcepb.Resource{
Attributes: []*commonpb.AttributeKeyValue{
{
Key: "instance",
Type: commonpb.AttributeKeyValue_STRING,
StringValue: "tester-b",
},
},
}
)
func TestNoGroupingExport(t *testing.T) {
runMetricExportTests(
t,
[]record{
{
"int64-count",
metric.CounterKind,
core.Int64NumberKind,
[]metric.Option{},
append(baseKeyValues, cpuKey.Int(1)),
},
{
"int64-count",
metric.CounterKind,
core.Int64NumberKind,
[]metric.Option{},
append(baseKeyValues, cpuKey.Int(2)),
},
},
[]metricpb.ResourceMetrics{
{
Resource: &resourcepb.Resource{},
InstrumentationLibraryMetrics: []*metricpb.InstrumentationLibraryMetrics{
{
Metrics: []*metricpb.Metric{
{
MetricDescriptor: cpu1MD,
Int64DataPoints: []*metricpb.Int64DataPoint{
{
Value: 11,
},
},
},
{
MetricDescriptor: cpu2MD,
Int64DataPoints: []*metricpb.Int64DataPoint{
{
Value: 11,
},
},
},
},
},
},
},
},
)
}
func TestMeasureMetricGroupingExport(t *testing.T) {
r := record{
"measure",
metric.MeasureKind,
core.Int64NumberKind,
[]metric.Option{},
append(baseKeyValues, cpuKey.Int(1)),
}
expected := []metricpb.ResourceMetrics{
{
Resource: &resourcepb.Resource{},
InstrumentationLibraryMetrics: []*metricpb.InstrumentationLibraryMetrics{
{
Metrics: []*metricpb.Metric{
{
MetricDescriptor: &metricpb.MetricDescriptor{
Name: "measure",
Type: metricpb.MetricDescriptor_SUMMARY,
Labels: []*commonpb.StringKeyValue{
{
Key: "host",
Value: "test.com",
},
{
Key: "CPU",
Value: "1",
},
},
},
SummaryDataPoints: []*metricpb.SummaryDataPoint{
{
Count: 2,
Sum: 11,
PercentileValues: []*metricpb.SummaryDataPoint_ValueAtPercentile{
{
Percentile: 0.0,
Value: 1.0,
},
{
Percentile: 100.0,
Value: 10.0,
},
},
},
{
Count: 2,
Sum: 11,
PercentileValues: []*metricpb.SummaryDataPoint_ValueAtPercentile{
{
Percentile: 0.0,
Value: 1.0,
},
{
Percentile: 100.0,
Value: 10.0,
},
},
},
},
},
},
},
},
},
}
runMetricExportTests(t, []record{r, r}, expected)
//changing the number kind should make no difference.
r.nKind = core.Uint64NumberKind
runMetricExportTests(t, []record{r, r}, expected)
r.nKind = core.Float64NumberKind
runMetricExportTests(t, []record{r, r}, expected)
}
func TestCountInt64MetricGroupingExport(t *testing.T) {
r := record{
"int64-count",
metric.CounterKind,
core.Int64NumberKind,
[]metric.Option{},
append(baseKeyValues, cpuKey.Int(1)),
}
runMetricExportTests(
t,
[]record{r, r},
[]metricpb.ResourceMetrics{
{
Resource: &resourcepb.Resource{},
InstrumentationLibraryMetrics: []*metricpb.InstrumentationLibraryMetrics{
{
Metrics: []*metricpb.Metric{
{
MetricDescriptor: cpu1MD,
Int64DataPoints: []*metricpb.Int64DataPoint{
{
Value: 11,
},
{
Value: 11,
},
},
},
},
},
},
},
},
)
}
func TestCountUint64MetricGroupingExport(t *testing.T) {
r := record{
"uint64-count",
metric.CounterKind,
core.Uint64NumberKind,
[]metric.Option{},
append(baseKeyValues, cpuKey.Int(1)),
}
runMetricExportTests(
t,
[]record{r, r},
[]metricpb.ResourceMetrics{
{
Resource: &resourcepb.Resource{},
InstrumentationLibraryMetrics: []*metricpb.InstrumentationLibraryMetrics{
{
Metrics: []*metricpb.Metric{
{
MetricDescriptor: &metricpb.MetricDescriptor{
Name: "uint64-count",
Type: metricpb.MetricDescriptor_COUNTER_INT64,
Labels: []*commonpb.StringKeyValue{
{
Key: "host",
Value: "test.com",
},
{
Key: "CPU",
Value: "1",
},
},
},
Int64DataPoints: []*metricpb.Int64DataPoint{
{
Value: 11,
},
{
Value: 11,
},
},
},
},
},
},
},
},
)
}
func TestCountFloat64MetricGroupingExport(t *testing.T) {
r := record{
"float64-count",
metric.CounterKind,
core.Float64NumberKind,
[]metric.Option{},
append(baseKeyValues, cpuKey.Int(1)),
}
runMetricExportTests(
t,
[]record{r, r},
[]metricpb.ResourceMetrics{
{
Resource: &resourcepb.Resource{},
InstrumentationLibraryMetrics: []*metricpb.InstrumentationLibraryMetrics{
{
Metrics: []*metricpb.Metric{
{
MetricDescriptor: &metricpb.MetricDescriptor{
Name: "float64-count",
Type: metricpb.MetricDescriptor_COUNTER_DOUBLE,
Labels: []*commonpb.StringKeyValue{
{
Key: "host",
Value: "test.com",
},
{
Key: "CPU",
Value: "1",
},
},
},
DoubleDataPoints: []*metricpb.DoubleDataPoint{
{
Value: 11,
},
{
Value: 11,
},
},
},
},
},
},
},
},
)
}
func TestResourceMetricGroupingExport(t *testing.T) {
runMetricExportTests(
t,
[]record{
{
"int64-count",
metric.CounterKind,
core.Int64NumberKind,
[]metric.Option{metric.WithResource(*testInstA)},
append(baseKeyValues, cpuKey.Int(1)),
},
{
"int64-count",
metric.CounterKind,
core.Int64NumberKind,
[]metric.Option{metric.WithResource(*testInstA)},
append(baseKeyValues, cpuKey.Int(1)),
},
{
"int64-count",
metric.CounterKind,
core.Int64NumberKind,
[]metric.Option{metric.WithResource(*testInstA)},
append(baseKeyValues, cpuKey.Int(2)),
},
{
"int64-count",
metric.CounterKind,
core.Int64NumberKind,
[]metric.Option{metric.WithResource(*testInstB)},
append(baseKeyValues, cpuKey.Int(1)),
},
},
[]metricpb.ResourceMetrics{
{
Resource: testerAResource,
InstrumentationLibraryMetrics: []*metricpb.InstrumentationLibraryMetrics{
{
Metrics: []*metricpb.Metric{
{
MetricDescriptor: cpu1MD,
Int64DataPoints: []*metricpb.Int64DataPoint{
{
Value: 11,
},
{
Value: 11,
},
},
},
{
MetricDescriptor: cpu2MD,
Int64DataPoints: []*metricpb.Int64DataPoint{
{
Value: 11,
},
},
},
},
},
},
},
{
Resource: testerBResource,
InstrumentationLibraryMetrics: []*metricpb.InstrumentationLibraryMetrics{
{
Metrics: []*metricpb.Metric{
{
MetricDescriptor: cpu1MD,
Int64DataPoints: []*metricpb.Int64DataPoint{
{
Value: 11,
},
},
},
},
},
},
},
},
)
}
func TestResourceInstLibMetricGroupingExport(t *testing.T) {
runMetricExportTests(
t,
[]record{
{
"int64-count",
metric.CounterKind,
core.Int64NumberKind,
[]metric.Option{
metric.WithResource(*testInstA),
metric.WithLibraryName("couting-lib"),
},
append(baseKeyValues, cpuKey.Int(1)),
},
{
"int64-count",
metric.CounterKind,
core.Int64NumberKind,
[]metric.Option{
metric.WithResource(*testInstA),
metric.WithLibraryName("couting-lib"),
},
append(baseKeyValues, cpuKey.Int(1)),
},
{
"int64-count",
metric.CounterKind,
core.Int64NumberKind,
[]metric.Option{
metric.WithResource(*testInstA),
metric.WithLibraryName("couting-lib"),
},
append(baseKeyValues, cpuKey.Int(2)),
},
{
"int64-count",
metric.CounterKind,
core.Int64NumberKind,
[]metric.Option{
metric.WithResource(*testInstA),
metric.WithLibraryName("summing-lib"),
},
append(baseKeyValues, cpuKey.Int(1)),
},
{
"int64-count",
metric.CounterKind,
core.Int64NumberKind,
[]metric.Option{
metric.WithResource(*testInstB),
metric.WithLibraryName("couting-lib"),
},
append(baseKeyValues, cpuKey.Int(1)),
},
},
[]metricpb.ResourceMetrics{
{
Resource: testerAResource,
InstrumentationLibraryMetrics: []*metricpb.InstrumentationLibraryMetrics{
{
InstrumentationLibrary: &commonpb.InstrumentationLibrary{
Name: "couting-lib",
},
Metrics: []*metricpb.Metric{
{
MetricDescriptor: cpu1MD,
Int64DataPoints: []*metricpb.Int64DataPoint{
{
Value: 11,
},
{
Value: 11,
},
},
},
{
MetricDescriptor: cpu2MD,
Int64DataPoints: []*metricpb.Int64DataPoint{
{
Value: 11,
},
},
},
},
},
{
InstrumentationLibrary: &commonpb.InstrumentationLibrary{
Name: "summing-lib",
},
Metrics: []*metricpb.Metric{
{
MetricDescriptor: cpu1MD,
Int64DataPoints: []*metricpb.Int64DataPoint{
{
Value: 11,
},
},
},
},
},
},
},
{
Resource: testerBResource,
InstrumentationLibraryMetrics: []*metricpb.InstrumentationLibraryMetrics{
{
InstrumentationLibrary: &commonpb.InstrumentationLibrary{
Name: "couting-lib",
},
Metrics: []*metricpb.Metric{
{
MetricDescriptor: cpu1MD,
Int64DataPoints: []*metricpb.Int64DataPoint{
{
Value: 11,
},
},
},
},
},
},
},
},
)
}
// What works single-threaded should work multi-threaded
func runMetricExportTests(t *testing.T, rs []record, expected []metricpb.ResourceMetrics) {
t.Run("1 goroutine", func(t *testing.T) {
runMetricExportTest(t, NewUnstartedExporter(WorkerCount(1)), rs, expected)
})
t.Run("20 goroutines", func(t *testing.T) {
runMetricExportTest(t, NewUnstartedExporter(WorkerCount(20)), rs, expected)
})
}
func runMetricExportTest(t *testing.T, exp *Exporter, rs []record, expected []metricpb.ResourceMetrics) {
msc := &metricsServiceClientStub{}
exp.metricExporter = msc
exp.started = true
var recs []metricsdk.Record
for _, r := range rs {
desc := metric.NewDescriptor(r.name, r.mKind, r.nKind, r.opts...)
labs := metricsdk.NewSimpleLabels(metricsdk.NewDefaultLabelEncoder(), r.labels...)
var agg metricsdk.Aggregator
switch r.mKind {
case metric.CounterKind:
agg = sum.New()
default:
agg = minmaxsumcount.New(&desc)
}
ctx := context.Background()
switch r.nKind {
case core.Uint64NumberKind:
require.NoError(t, agg.Update(ctx, core.NewUint64Number(1), &desc))
require.NoError(t, agg.Update(ctx, core.NewUint64Number(10), &desc))
case core.Int64NumberKind:
require.NoError(t, agg.Update(ctx, core.NewInt64Number(1), &desc))
require.NoError(t, agg.Update(ctx, core.NewInt64Number(10), &desc))
case core.Float64NumberKind:
require.NoError(t, agg.Update(ctx, core.NewFloat64Number(1), &desc))
require.NoError(t, agg.Update(ctx, core.NewFloat64Number(10), &desc))
default:
t.Fatalf("invalid number kind: %v", r.nKind)
}
agg.Checkpoint(ctx, &desc)
recs = append(recs, metricsdk.NewRecord(&desc, labs, agg))
}
assert.NoError(t, exp.Export(context.Background(), checkpointSet{records: recs}))
// assert.ElementsMatch does not equate nested slices of different order,
// therefore this requires the top level slice to be broken down.
// Build a map of Resource/InstrumentationLibrary pairs to Metrics, from
// that validate the metric elements match for all expected pairs. Finally,
// make we saw all expected pairs.
type key struct {
resource, instrumentationLibrary string
}
got := map[key][]*metricpb.Metric{}
for _, rm := range msc.ResourceMetrics() {
for _, ilm := range rm.InstrumentationLibraryMetrics {
k := key{
resource: rm.GetResource().String(),
instrumentationLibrary: ilm.GetInstrumentationLibrary().String(),
}
got[k] = ilm.GetMetrics()
}
}
seen := map[key]struct{}{}
for _, rm := range expected {
for _, ilm := range rm.InstrumentationLibraryMetrics {
k := key{
resource: rm.GetResource().String(),
instrumentationLibrary: ilm.GetInstrumentationLibrary().String(),
}
seen[k] = struct{}{}
g, ok := got[k]
if !ok {
t.Errorf("missing metrics for:\n\tResource: %s\n\tInstrumentationLibrary: %s\n", k.resource, k.instrumentationLibrary)
continue
}
assert.ElementsMatch(t, ilm.GetMetrics(), g, "metrics did not match for:\n\tResource: %s\n\tInstrumentationLibrary: %s\n", k.resource, k.instrumentationLibrary)
}
}
for k := range got {
if _, ok := seen[k]; !ok {
t.Errorf("did not expect metrics for:\n\tResource: %s\n\tInstrumentationLibrary: %s\n", k.resource, k.instrumentationLibrary)
}
}
}
func TestEmptyMetricExport(t *testing.T) {
msc := &metricsServiceClientStub{}
exp := NewUnstartedExporter()
exp.metricExporter = msc
exp.started = true
for _, test := range []struct {
records []metricsdk.Record
want []metricpb.ResourceMetrics
}{
{
[]metricsdk.Record(nil),
[]metricpb.ResourceMetrics(nil),
},
{
[]metricsdk.Record{},
[]metricpb.ResourceMetrics(nil),
},
} {
msc.Reset()
require.NoError(t, exp.Export(context.Background(), checkpointSet{records: test.records}))
assert.Equal(t, test.want, msc.ResourceMetrics())
}
}