You've already forked opentelemetry-go
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:
@@ -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
|
||||||
|
@@ -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)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
@@ -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) {
|
||||||
|
716
exporters/otlp/otlp_metric_test.go
Normal file
716
exporters/otlp/otlp_metric_test.go
Normal 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())
|
||||||
|
}
|
||||||
|
}
|
Reference in New Issue
Block a user