1
0
mirror of https://github.com/open-telemetry/opentelemetry-go.git synced 2025-10-31 00:07:40 +02:00

Refactor Pipeline (#3233)

* Add views field to pipeline

Redundant maps tracking readers to views and readers to pipelines exist
in the pipelineRegistry. Unify these maps by tracing views in pipelines.

* Rename newPipelineRegistries->newPipelineRegistry

* Add Reader as field to pipeline

* Replace createAggregators with resolver facilitator

* Replace create agg funcs with inserter facilitator

* Correct documentation

* Replace pipelineRegistry with []pipeline type

* Rename newPipelineRegistry->newPipelines

* Fix pipeline_registry_test

* Flatten isMonotonic into only use

* Update FIXME into TODO

* Rename instrument provider resolver field to resolve

* Fix comment English

* Fix drop aggregator detection
This commit is contained in:
Tyler Yahn
2022-09-28 08:47:20 -07:00
committed by GitHub
parent 12e16d41e7
commit aca054b075
6 changed files with 264 additions and 215 deletions

View File

@@ -28,7 +28,7 @@ import (
type asyncInt64Provider struct {
scope instrumentation.Scope
registry *pipelineRegistry
resolve *resolver[int64]
}
var _ asyncint64.InstrumentProvider = asyncInt64Provider{}
@@ -37,7 +37,7 @@ var _ asyncint64.InstrumentProvider = asyncInt64Provider{}
func (p asyncInt64Provider) Counter(name string, opts ...instrument.Option) (asyncint64.Counter, error) {
cfg := instrument.NewConfig(opts...)
aggs, err := createAggregators[int64](p.registry, view.Instrument{
aggs, err := p.resolve.Aggregators(view.Instrument{
Scope: p.scope,
Name: name,
Description: cfg.Description(),
@@ -56,7 +56,7 @@ func (p asyncInt64Provider) Counter(name string, opts ...instrument.Option) (asy
func (p asyncInt64Provider) UpDownCounter(name string, opts ...instrument.Option) (asyncint64.UpDownCounter, error) {
cfg := instrument.NewConfig(opts...)
aggs, err := createAggregators[int64](p.registry, view.Instrument{
aggs, err := p.resolve.Aggregators(view.Instrument{
Scope: p.scope,
Name: name,
Description: cfg.Description(),
@@ -74,7 +74,7 @@ func (p asyncInt64Provider) UpDownCounter(name string, opts ...instrument.Option
func (p asyncInt64Provider) Gauge(name string, opts ...instrument.Option) (asyncint64.Gauge, error) {
cfg := instrument.NewConfig(opts...)
aggs, err := createAggregators[int64](p.registry, view.Instrument{
aggs, err := p.resolve.Aggregators(view.Instrument{
Scope: p.scope,
Name: name,
Description: cfg.Description(),
@@ -90,7 +90,7 @@ func (p asyncInt64Provider) Gauge(name string, opts ...instrument.Option) (async
type asyncFloat64Provider struct {
scope instrumentation.Scope
registry *pipelineRegistry
resolve *resolver[float64]
}
var _ asyncfloat64.InstrumentProvider = asyncFloat64Provider{}
@@ -99,7 +99,7 @@ var _ asyncfloat64.InstrumentProvider = asyncFloat64Provider{}
func (p asyncFloat64Provider) Counter(name string, opts ...instrument.Option) (asyncfloat64.Counter, error) {
cfg := instrument.NewConfig(opts...)
aggs, err := createAggregators[float64](p.registry, view.Instrument{
aggs, err := p.resolve.Aggregators(view.Instrument{
Scope: p.scope,
Name: name,
Description: cfg.Description(),
@@ -117,7 +117,7 @@ func (p asyncFloat64Provider) Counter(name string, opts ...instrument.Option) (a
func (p asyncFloat64Provider) UpDownCounter(name string, opts ...instrument.Option) (asyncfloat64.UpDownCounter, error) {
cfg := instrument.NewConfig(opts...)
aggs, err := createAggregators[float64](p.registry, view.Instrument{
aggs, err := p.resolve.Aggregators(view.Instrument{
Scope: p.scope,
Name: name,
Description: cfg.Description(),
@@ -135,7 +135,7 @@ func (p asyncFloat64Provider) UpDownCounter(name string, opts ...instrument.Opti
func (p asyncFloat64Provider) Gauge(name string, opts ...instrument.Option) (asyncfloat64.Gauge, error) {
cfg := instrument.NewConfig(opts...)
aggs, err := createAggregators[float64](p.registry, view.Instrument{
aggs, err := p.resolve.Aggregators(view.Instrument{
Scope: p.scope,
Name: name,
Description: cfg.Description(),
@@ -151,7 +151,7 @@ func (p asyncFloat64Provider) Gauge(name string, opts ...instrument.Option) (asy
type syncInt64Provider struct {
scope instrumentation.Scope
registry *pipelineRegistry
resolve *resolver[int64]
}
var _ syncint64.InstrumentProvider = syncInt64Provider{}
@@ -160,7 +160,7 @@ var _ syncint64.InstrumentProvider = syncInt64Provider{}
func (p syncInt64Provider) Counter(name string, opts ...instrument.Option) (syncint64.Counter, error) {
cfg := instrument.NewConfig(opts...)
aggs, err := createAggregators[int64](p.registry, view.Instrument{
aggs, err := p.resolve.Aggregators(view.Instrument{
Scope: p.scope,
Name: name,
Description: cfg.Description(),
@@ -178,7 +178,7 @@ func (p syncInt64Provider) Counter(name string, opts ...instrument.Option) (sync
func (p syncInt64Provider) UpDownCounter(name string, opts ...instrument.Option) (syncint64.UpDownCounter, error) {
cfg := instrument.NewConfig(opts...)
aggs, err := createAggregators[int64](p.registry, view.Instrument{
aggs, err := p.resolve.Aggregators(view.Instrument{
Scope: p.scope,
Name: name,
Description: cfg.Description(),
@@ -196,7 +196,7 @@ func (p syncInt64Provider) UpDownCounter(name string, opts ...instrument.Option)
func (p syncInt64Provider) Histogram(name string, opts ...instrument.Option) (syncint64.Histogram, error) {
cfg := instrument.NewConfig(opts...)
aggs, err := createAggregators[int64](p.registry, view.Instrument{
aggs, err := p.resolve.Aggregators(view.Instrument{
Scope: p.scope,
Name: name,
Description: cfg.Description(),
@@ -212,7 +212,7 @@ func (p syncInt64Provider) Histogram(name string, opts ...instrument.Option) (sy
type syncFloat64Provider struct {
scope instrumentation.Scope
registry *pipelineRegistry
resolve *resolver[float64]
}
var _ syncfloat64.InstrumentProvider = syncFloat64Provider{}
@@ -221,7 +221,7 @@ var _ syncfloat64.InstrumentProvider = syncFloat64Provider{}
func (p syncFloat64Provider) Counter(name string, opts ...instrument.Option) (syncfloat64.Counter, error) {
cfg := instrument.NewConfig(opts...)
aggs, err := createAggregators[float64](p.registry, view.Instrument{
aggs, err := p.resolve.Aggregators(view.Instrument{
Scope: p.scope,
Name: name,
Description: cfg.Description(),
@@ -239,7 +239,7 @@ func (p syncFloat64Provider) Counter(name string, opts ...instrument.Option) (sy
func (p syncFloat64Provider) UpDownCounter(name string, opts ...instrument.Option) (syncfloat64.UpDownCounter, error) {
cfg := instrument.NewConfig(opts...)
aggs, err := createAggregators[float64](p.registry, view.Instrument{
aggs, err := p.resolve.Aggregators(view.Instrument{
Scope: p.scope,
Name: name,
Description: cfg.Description(),
@@ -257,7 +257,7 @@ func (p syncFloat64Provider) UpDownCounter(name string, opts ...instrument.Optio
func (p syncFloat64Provider) Histogram(name string, opts ...instrument.Option) (syncfloat64.Histogram, error) {
cfg := instrument.NewConfig(opts...)
aggs, err := createAggregators[float64](p.registry, view.Instrument{
aggs, err := p.resolve.Aggregators(view.Instrument{
Scope: p.scope,
Name: name,
Description: cfg.Description(),

View File

@@ -42,7 +42,7 @@ type meterRegistry struct {
meters map[instrumentation.Scope]*meter
registry *pipelineRegistry
pipes pipelines
}
// Get returns a registered meter matching the instrumentation scope if it
@@ -57,7 +57,7 @@ func (r *meterRegistry) Get(s instrumentation.Scope) *meter {
if r.meters == nil {
m := &meter{
Scope: s,
registry: r.registry,
pipes: r.pipes,
}
r.meters = map[instrumentation.Scope]*meter{s: m}
return m
@@ -70,7 +70,7 @@ func (r *meterRegistry) Get(s instrumentation.Scope) *meter {
m = &meter{
Scope: s,
registry: r.registry,
pipes: r.pipes,
}
r.meters[s] = m
return m
@@ -83,7 +83,7 @@ func (r *meterRegistry) Get(s instrumentation.Scope) *meter {
type meter struct {
instrumentation.Scope
registry *pipelineRegistry
pipes pipelines
}
// Compile-time check meter implements metric.Meter.
@@ -91,27 +91,27 @@ var _ metric.Meter = (*meter)(nil)
// AsyncInt64 returns the asynchronous integer instrument provider.
func (m *meter) AsyncInt64() asyncint64.InstrumentProvider {
return asyncInt64Provider{scope: m.Scope, registry: m.registry}
return asyncInt64Provider{scope: m.Scope, resolve: newResolver[int64](m.pipes)}
}
// AsyncFloat64 returns the asynchronous floating-point instrument provider.
func (m *meter) AsyncFloat64() asyncfloat64.InstrumentProvider {
return asyncFloat64Provider{scope: m.Scope, registry: m.registry}
return asyncFloat64Provider{scope: m.Scope, resolve: newResolver[float64](m.pipes)}
}
// RegisterCallback registers the function f to be called when any of the
// insts Collect method is called.
func (m *meter) RegisterCallback(insts []instrument.Asynchronous, f func(context.Context)) error {
m.registry.registerCallback(f)
m.pipes.registerCallback(f)
return nil
}
// SyncInt64 returns the synchronous integer instrument provider.
func (m *meter) SyncInt64() syncint64.InstrumentProvider {
return syncInt64Provider{scope: m.Scope, registry: m.registry}
return syncInt64Provider{scope: m.Scope, resolve: newResolver[int64](m.pipes)}
}
// SyncFloat64 returns the synchronous floating-point instrument provider.
func (m *meter) SyncFloat64() syncfloat64.InstrumentProvider {
return syncFloat64Provider{scope: m.Scope, registry: m.registry}
return syncFloat64Provider{scope: m.Scope, resolve: newResolver[float64](m.pipes)}
}

View File

@@ -30,9 +30,25 @@ import (
"go.opentelemetry.io/otel/sdk/resource"
)
var (
errCreatingAggregators = errors.New("could not create all aggregators")
errIncompatibleAggregation = errors.New("incompatible aggregation")
errUnknownAggregation = errors.New("unrecognized aggregation")
)
type aggregator interface {
Aggregation() metricdata.Aggregation
}
// instrumentID is used to identify multiple instruments being mapped to the
// same aggregator. e.g. using an exact match view with a name=* view. You
// can't use a view.Instrument here because not all Aggregators are comparable.
type instrumentID struct {
scope instrumentation.Scope
name string
description string
}
type instrumentKey struct {
name string
unit unit.Unit
@@ -43,12 +59,14 @@ type instrumentValue struct {
aggregator aggregator
}
func newPipeline(res *resource.Resource) *pipeline {
func newPipeline(res *resource.Resource, reader Reader, views []view.View) *pipeline {
if res == nil {
res = resource.Empty()
}
return &pipeline{
resource: res,
reader: reader,
views: views,
aggregations: make(map[instrumentation.Scope]map[instrumentKey]instrumentValue),
}
}
@@ -61,6 +79,9 @@ func newPipeline(res *resource.Resource) *pipeline {
type pipeline struct {
resource *resource.Resource
reader Reader
views []view.View
sync.Mutex
aggregations map[instrumentation.Scope]map[instrumentKey]instrumentValue
callbacks []func(context.Context)
@@ -155,53 +176,198 @@ func (p *pipeline) produce(ctx context.Context) (metricdata.ResourceMetrics, err
}, nil
}
// pipelineRegistry manages creating pipelines, and aggregators. Meters retrieve
// new Aggregators from a pipelineRegistry.
type pipelineRegistry struct {
views map[Reader][]view.View
pipelines map[Reader]*pipeline
// inserter facilitates inserting of new instruments into a pipeline.
type inserter[N int64 | float64] struct {
pipeline *pipeline
}
func newPipelineRegistries(res *resource.Resource, views map[Reader][]view.View) *pipelineRegistry {
pipelines := map[Reader]*pipeline{}
for rdr := range views {
pipe := &pipeline{resource: res}
rdr.register(pipe)
pipelines[rdr] = pipe
func newInserter[N int64 | float64](p *pipeline) *inserter[N] {
return &inserter[N]{p}
}
return &pipelineRegistry{
views: views,
pipelines: pipelines,
// Instrument inserts instrument inst with instUnit returning the Aggregators
// that need to be updated with measurments for that instrument.
func (i *inserter[N]) Instrument(inst view.Instrument, instUnit unit.Unit) ([]internal.Aggregator[N], error) {
seen := map[instrumentID]struct{}{}
var aggs []internal.Aggregator[N]
errs := &multierror{wrapped: errCreatingAggregators}
for _, v := range i.pipeline.views {
inst, match := v.TransformInstrument(inst)
id := instrumentID{
scope: inst.Scope,
name: inst.Name,
description: inst.Description,
}
if _, ok := seen[id]; ok || !match {
continue
}
if inst.Aggregation == nil {
inst.Aggregation = i.pipeline.reader.aggregation(inst.Kind)
} else if _, ok := inst.Aggregation.(aggregation.Default); ok {
inst.Aggregation = i.pipeline.reader.aggregation(inst.Kind)
}
if err := isAggregatorCompatible(inst.Kind, inst.Aggregation); err != nil {
err = fmt.Errorf("creating aggregator with instrumentKind: %d, aggregation %v: %w", inst.Kind, inst.Aggregation, err)
errs.append(err)
continue
}
agg, err := i.aggregator(inst)
if err != nil {
errs.append(err)
continue
}
if agg == nil { // Drop aggregator.
continue
}
// TODO (#3011): If filtering is done at the instrument level add here.
// This is where the aggregator and the view are both in scope.
aggs = append(aggs, agg)
seen[id] = struct{}{}
err = i.pipeline.addAggregator(inst.Scope, inst.Name, inst.Description, instUnit, agg)
if err != nil {
errs.append(err)
}
}
// TODO(#3224): handle when no views match. Default should be reader
// aggregation returned.
return aggs, errs.errorOrNil()
}
// aggregator returns the Aggregator for an instrument configuration. If the
// instrument defines an unknown aggregation, an error is returned.
func (i *inserter[N]) aggregator(inst view.Instrument) (internal.Aggregator[N], error) {
// TODO (#3011): If filtering is done by the Aggregator it should be passed
// here.
var (
temporality = i.pipeline.reader.temporality(inst.Kind)
monotonic bool
)
switch inst.Kind {
case view.AsyncCounter, view.SyncCounter, view.SyncHistogram:
monotonic = true
}
switch agg := inst.Aggregation.(type) {
case aggregation.Drop:
return nil, nil
case aggregation.LastValue:
return internal.NewLastValue[N](), nil
case aggregation.Sum:
if temporality == metricdata.CumulativeTemporality {
return internal.NewCumulativeSum[N](monotonic), nil
}
return internal.NewDeltaSum[N](monotonic), nil
case aggregation.ExplicitBucketHistogram:
if temporality == metricdata.CumulativeTemporality {
return internal.NewCumulativeHistogram[N](agg), nil
}
return internal.NewDeltaHistogram[N](agg), nil
}
return nil, errUnknownAggregation
}
// isAggregatorCompatible checks if the aggregation can be used by the instrument.
// Current compatibility:
//
// | Instrument Kind | Drop | LastValue | Sum | Histogram | Exponential Histogram |
// |----------------------|------|-----------|-----|-----------|-----------------------|
// | Sync Counter | X | | X | X | X |
// | Sync UpDown Counter | X | | X | | |
// | Sync Histogram | X | | X | X | X |
// | Async Counter | X | | X | | |
// | Async UpDown Counter | X | | X | | |
// | Async Gauge | X | X | | | |.
func isAggregatorCompatible(kind view.InstrumentKind, agg aggregation.Aggregation) error {
switch agg.(type) {
case aggregation.ExplicitBucketHistogram:
if kind == view.SyncCounter || kind == view.SyncHistogram {
return nil
}
// TODO: review need for aggregation check after
// https://github.com/open-telemetry/opentelemetry-specification/issues/2710
return errIncompatibleAggregation
case aggregation.Sum:
switch kind {
case view.AsyncCounter, view.AsyncUpDownCounter, view.SyncCounter, view.SyncHistogram, view.SyncUpDownCounter:
return nil
default:
// TODO: review need for aggregation check after
// https://github.com/open-telemetry/opentelemetry-specification/issues/2710
return errIncompatibleAggregation
}
case aggregation.LastValue:
if kind == view.AsyncGauge {
return nil
}
// TODO: review need for aggregation check after
// https://github.com/open-telemetry/opentelemetry-specification/issues/2710
return errIncompatibleAggregation
case aggregation.Drop:
return nil
default:
// This is used passed checking for default, it should be an error at this point.
return fmt.Errorf("%w: %v", errUnknownAggregation, agg)
}
}
// pipelines is the group of pipelines connecting Readers with instrument
// measurement.
type pipelines []*pipeline
func newPipelines(res *resource.Resource, readers map[Reader][]view.View) pipelines {
pipes := make([]*pipeline, 0, len(readers))
for r, v := range readers {
p := &pipeline{
resource: res,
reader: r,
views: v,
}
r.register(p)
pipes = append(pipes, p)
}
return pipes
}
// TODO (#3053) Only register callbacks if any instrument matches in a view.
func (reg *pipelineRegistry) registerCallback(fn func(context.Context)) {
for _, pipe := range reg.pipelines {
func (p pipelines) registerCallback(fn func(context.Context)) {
for _, pipe := range p {
pipe.addCallback(fn)
}
}
// createAggregators will create all backing aggregators for an instrument.
// It will return an error if an instrument is registered more than once.
// Note: There may be returned aggregators with an error.
func createAggregators[N int64 | float64](reg *pipelineRegistry, inst view.Instrument, instUnit unit.Unit) ([]internal.Aggregator[N], error) {
// resolver facilitates resolving Aggregators an instrument needs to aggregate
// measurements with while updating all pipelines that need to pull from those
// aggregations.
type resolver[N int64 | float64] struct {
inserters []*inserter[N]
}
func newResolver[N int64 | float64](p pipelines) *resolver[N] {
in := make([]*inserter[N], len(p))
for i := range in {
in[i] = newInserter[N](p[i])
}
return &resolver[N]{in}
}
// Aggregators returns the Aggregators instrument inst needs to update when it
// makes a measurement.
func (r *resolver[N]) Aggregators(inst view.Instrument, instUnit unit.Unit) ([]internal.Aggregator[N], error) {
var aggs []internal.Aggregator[N]
errs := &multierror{}
for rdr, views := range reg.views {
pipe := reg.pipelines[rdr]
rdrAggs, err := createAggregatorsForReader[N](rdr, views, inst)
for _, i := range r.inserters {
a, err := i.Instrument(inst, instUnit)
if err != nil {
errs.append(err)
}
for inst, agg := range rdrAggs {
err := pipe.addAggregator(inst.scope, inst.name, inst.description, instUnit, agg)
if err != nil {
errs.append(err)
}
aggs = append(aggs, agg)
}
aggs = append(aggs, a...)
}
return aggs, errs.errorOrNil()
}
@@ -221,126 +387,3 @@ func (m *multierror) errorOrNil() error {
func (m *multierror) append(err error) {
m.errors = append(m.errors, err.Error())
}
// instrumentID is used to identify multiple instruments being mapped to the same aggregator.
// e.g. using an exact match view with a name=* view.
// You can't use a view.Instrument here because not all Aggregators are comparable.
type instrumentID struct {
scope instrumentation.Scope
name string
description string
}
var errCreatingAggregators = errors.New("could not create all aggregators")
func createAggregatorsForReader[N int64 | float64](rdr Reader, views []view.View, inst view.Instrument) (map[instrumentID]internal.Aggregator[N], error) {
aggs := map[instrumentID]internal.Aggregator[N]{}
errs := &multierror{
wrapped: errCreatingAggregators,
}
for _, v := range views {
inst, match := v.TransformInstrument(inst)
ident := instrumentID{
scope: inst.Scope,
name: inst.Name,
description: inst.Description,
}
if _, ok := aggs[ident]; ok || !match {
continue
}
if inst.Aggregation == nil {
inst.Aggregation = rdr.aggregation(inst.Kind)
} else if _, ok := inst.Aggregation.(aggregation.Default); ok {
inst.Aggregation = rdr.aggregation(inst.Kind)
}
if err := isAggregatorCompatible(inst.Kind, inst.Aggregation); err != nil {
err = fmt.Errorf("creating aggregator with instrumentKind: %d, aggregation %v: %w", inst.Kind, inst.Aggregation, err)
errs.append(err)
continue
}
agg := createAggregator[N](inst.Aggregation, rdr.temporality(inst.Kind), isMonotonic(inst.Kind))
if agg != nil {
// TODO (#3011): If filtering is done at the instrument level add here.
// This is where the aggregator and the view are both in scope.
aggs[ident] = agg
}
}
return aggs, errs.errorOrNil()
}
func isMonotonic(kind view.InstrumentKind) bool {
switch kind {
case view.AsyncCounter, view.SyncCounter, view.SyncHistogram:
return true
}
return false
}
// createAggregator takes the config (Aggregation and Temporality) and produces a memory backed Aggregator.
// TODO (#3011): If filterting is done by the Aggregator it should be passed here.
func createAggregator[N int64 | float64](agg aggregation.Aggregation, temporality metricdata.Temporality, monotonic bool) internal.Aggregator[N] {
switch agg := agg.(type) {
case aggregation.Drop:
return nil
case aggregation.LastValue:
return internal.NewLastValue[N]()
case aggregation.Sum:
if temporality == metricdata.CumulativeTemporality {
return internal.NewCumulativeSum[N](monotonic)
}
return internal.NewDeltaSum[N](monotonic)
case aggregation.ExplicitBucketHistogram:
if temporality == metricdata.CumulativeTemporality {
return internal.NewCumulativeHistogram[N](agg)
}
return internal.NewDeltaHistogram[N](agg)
}
return nil
}
// TODO: review need for aggregation check after https://github.com/open-telemetry/opentelemetry-specification/issues/2710
var errIncompatibleAggregation = errors.New("incompatible aggregation")
var errUnknownAggregation = errors.New("unrecognized aggregation")
// is aggregatorCompatible checks if the aggregation can be used by the instrument.
// Current compatibility:
//
// | Instrument Kind | Drop | LastValue | Sum | Histogram | Exponential Histogram |
// |----------------------|------|-----------|-----|-----------|-----------------------|
// | Sync Counter | X | | X | X | X |
// | Sync UpDown Counter | X | | X | | |
// | Sync Histogram | X | | X | X | X |
// | Async Counter | X | | X | | |
// | Async UpDown Counter | X | | X | | |
// | Async Gauge | X | X | | | |.
func isAggregatorCompatible(kind view.InstrumentKind, agg aggregation.Aggregation) error {
switch agg.(type) {
case aggregation.ExplicitBucketHistogram:
if kind == view.SyncCounter || kind == view.SyncHistogram {
return nil
}
return errIncompatibleAggregation
case aggregation.Sum:
switch kind {
case view.AsyncCounter, view.AsyncUpDownCounter, view.SyncCounter, view.SyncHistogram, view.SyncUpDownCounter:
return nil
default:
return errIncompatibleAggregation
}
case aggregation.LastValue:
if kind == view.AsyncGauge {
return nil
}
return errIncompatibleAggregation
case aggregation.Drop:
return nil
default:
// This is used passed checking for default, it should be an error at this point.
return fmt.Errorf("%w: %v", errUnknownAggregation, agg)
}
}

View File

@@ -211,7 +211,8 @@ func testCreateAggregators[N int64 | float64](t *testing.T) {
}
for _, tt := range testcases {
t.Run(tt.name, func(t *testing.T) {
got, err := createAggregatorsForReader[N](tt.reader, tt.views, tt.inst)
i := newInserter[N](newPipeline(nil, tt.reader, tt.views))
got, err := i.Instrument(tt.inst, unit.Dimensionless)
assert.ErrorIs(t, err, tt.wantErr)
require.Len(t, got, tt.wantLen)
for _, agg := range got {
@@ -222,13 +223,12 @@ func testCreateAggregators[N int64 | float64](t *testing.T) {
}
func testInvalidInstrumentShouldPanic[N int64 | float64]() {
reader := NewManualReader()
views := []view.View{{}}
i := newInserter[N](newPipeline(nil, NewManualReader(), []view.View{{}}))
inst := view.Instrument{
Name: "foo",
Kind: view.InstrumentKind(255),
}
_, _ = createAggregatorsForReader[N](reader, views, inst)
_, _ = i.Instrument(inst, unit.Dimensionless)
}
func TestInvalidInstrumentShouldPanic(t *testing.T) {
@@ -323,27 +323,29 @@ func TestPipelineRegistryCreateAggregators(t *testing.T) {
for _, tt := range testCases {
t.Run(tt.name, func(t *testing.T) {
reg := newPipelineRegistries(resource.Empty(), tt.views)
testPipelineRegistryCreateIntAggregators(t, reg, tt.wantCount)
reg = newPipelineRegistries(resource.Empty(), tt.views)
testPipelineRegistryCreateFloatAggregators(t, reg, tt.wantCount)
p := newPipelines(resource.Empty(), tt.views)
testPipelineRegistryResolveIntAggregators(t, p, tt.wantCount)
p = newPipelines(resource.Empty(), tt.views)
testPipelineRegistryResolveFloatAggregators(t, p, tt.wantCount)
})
}
}
func testPipelineRegistryCreateIntAggregators(t *testing.T, reg *pipelineRegistry, wantCount int) {
func testPipelineRegistryResolveIntAggregators(t *testing.T, p pipelines, wantCount int) {
inst := view.Instrument{Name: "foo", Kind: view.SyncCounter}
aggs, err := createAggregators[int64](reg, inst, unit.Dimensionless)
r := newResolver[int64](p)
aggs, err := r.Aggregators(inst, unit.Dimensionless)
assert.NoError(t, err)
require.Len(t, aggs, wantCount)
}
func testPipelineRegistryCreateFloatAggregators(t *testing.T, reg *pipelineRegistry, wantCount int) {
func testPipelineRegistryResolveFloatAggregators(t *testing.T, p pipelines, wantCount int) {
inst := view.Instrument{Name: "foo", Kind: view.SyncCounter}
aggs, err := createAggregators[float64](reg, inst, unit.Dimensionless)
r := newResolver[float64](p)
aggs, err := r.Aggregators(inst, unit.Dimensionless)
assert.NoError(t, err)
require.Len(t, aggs, wantCount)
@@ -356,8 +358,8 @@ func TestPipelineRegistryResource(t *testing.T) {
NewManualReader(): {{}, v},
}
res := resource.NewSchemaless(attribute.String("key", "val"))
reg := newPipelineRegistries(res, views)
for _, p := range reg.pipelines {
pipes := newPipelines(res, views)
for _, p := range pipes {
assert.True(t, res.Equal(p.resource), "resource not set")
}
}
@@ -370,16 +372,18 @@ func TestPipelineRegistryCreateAggregatorsIncompatibleInstrument(t *testing.T) {
{},
},
}
reg := newPipelineRegistries(resource.Empty(), views)
p := newPipelines(resource.Empty(), views)
inst := view.Instrument{Name: "foo", Kind: view.AsyncGauge}
intAggs, err := createAggregators[int64](reg, inst, unit.Dimensionless)
ri := newResolver[int64](p)
intAggs, err := ri.Aggregators(inst, unit.Dimensionless)
assert.Error(t, err)
assert.Len(t, intAggs, 0)
reg = newPipelineRegistries(resource.Empty(), views)
p = newPipelines(resource.Empty(), views)
floatAggs, err := createAggregators[float64](reg, inst, unit.Dimensionless)
rf := newResolver[float64](p)
floatAggs, err := rf.Aggregators(inst, unit.Dimensionless)
assert.Error(t, err)
assert.Len(t, floatAggs, 0)
}
@@ -399,28 +403,30 @@ func TestPipelineRegistryCreateAggregatorsDuplicateErrors(t *testing.T) {
fooInst := view.Instrument{Name: "foo", Kind: view.SyncCounter}
barInst := view.Instrument{Name: "bar", Kind: view.SyncCounter}
reg := newPipelineRegistries(resource.Empty(), views)
p := newPipelines(resource.Empty(), views)
intAggs, err := createAggregators[int64](reg, fooInst, unit.Dimensionless)
ri := newResolver[int64](p)
intAggs, err := ri.Aggregators(fooInst, unit.Dimensionless)
assert.NoError(t, err)
assert.Len(t, intAggs, 1)
// The Rename view should error, because it creates a foo instrument.
intAggs, err = createAggregators[int64](reg, barInst, unit.Dimensionless)
intAggs, err = ri.Aggregators(barInst, unit.Dimensionless)
assert.Error(t, err)
assert.Len(t, intAggs, 2)
// Creating a float foo instrument should error because there is an int foo instrument.
floatAggs, err := createAggregators[float64](reg, fooInst, unit.Dimensionless)
rf := newResolver[float64](p)
floatAggs, err := rf.Aggregators(fooInst, unit.Dimensionless)
assert.Error(t, err)
assert.Len(t, floatAggs, 1)
fooInst = view.Instrument{Name: "foo-float", Kind: view.SyncCounter}
_, err = createAggregators[float64](reg, fooInst, unit.Dimensionless)
_, err = rf.Aggregators(fooInst, unit.Dimensionless)
assert.NoError(t, err)
floatAggs, err = createAggregators[float64](reg, barInst, unit.Dimensionless)
floatAggs, err = rf.Aggregators(barInst, unit.Dimensionless)
assert.Error(t, err)
assert.Len(t, floatAggs, 2)
}

View File

@@ -61,7 +61,7 @@ func TestEmptyPipeline(t *testing.T) {
}
func TestNewPipeline(t *testing.T) {
pipe := newPipeline(nil)
pipe := newPipeline(nil, nil, nil)
output, err := pipe.produce(context.Background())
require.NoError(t, err)
@@ -158,7 +158,7 @@ func TestPipelineDuplicateRegistration(t *testing.T) {
}
for _, tt := range testCases {
t.Run(tt.name, func(t *testing.T) {
pipe := newPipeline(nil)
pipe := newPipeline(nil, nil, nil)
err := pipe.addAggregator(instrumentation.Scope{}, "name", "desc", unit.Dimensionless, testSumAggregator{})
require.NoError(t, err)
@@ -177,7 +177,7 @@ func TestPipelineDuplicateRegistration(t *testing.T) {
func TestPipelineUsesResource(t *testing.T) {
res := resource.NewWithAttributes("noSchema", attribute.String("test", "resource"))
pipe := newPipeline(res)
pipe := newPipeline(res, nil, nil)
output, err := pipe.produce(context.Background())
assert.NoError(t, err)
@@ -185,7 +185,7 @@ func TestPipelineUsesResource(t *testing.T) {
}
func TestPipelineConcurrency(t *testing.T) {
pipe := newPipeline(nil)
pipe := newPipeline(nil, nil, nil)
ctx := context.Background()
var wg sync.WaitGroup

View File

@@ -48,13 +48,13 @@ func NewMeterProvider(options ...Option) *MeterProvider {
flush, sdown := conf.readerSignals()
registry := newPipelineRegistries(conf.res, conf.readers)
registry := newPipelines(conf.res, conf.readers)
return &MeterProvider{
res: conf.res,
meters: meterRegistry{
registry: registry,
pipes: registry,
},
forceFlush: flush,