1
0
mirror of https://github.com/open-telemetry/opentelemetry-go.git synced 2025-07-13 01:00:22 +02:00

Metrics: Rename sdk/export/metric.ExportKind to aggregation.Temporality (#2274)

* Rename ExportKind to aggregation.Temporality

* Changelog uhhhhhhh

* Apply suggestions from code review

Co-authored-by: Tyler Yahn <MrAlias@users.noreply.github.com>

* int->uint8

* go generate

* go.sum

* anual edit

Co-authored-by: Tyler Yahn <MrAlias@users.noreply.github.com>
This commit is contained in:
Joshua MacDonald
2021-10-15 11:18:36 -07:00
committed by GitHub
parent e72a235518
commit 6df0de12bc
30 changed files with 372 additions and 352 deletions

View File

@ -11,6 +11,8 @@ This project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0.htm
## Changed ## Changed
- Skip links with invalid span context. (#2275) - Skip links with invalid span context. (#2275)
- Metric SDK `export.ExportKind`, `export.ExportKindSelector` types have been renamed to `aggregation.Temporality` and `aggregation.TemporalitySelector` respectively to keep in line with current specification and protocol along with built-in selectors (e.g., `aggregation.CumulativeTemporalitySelector`, ...). (#2274)
- The Metric `Exporter` interface now requires a `TemporalitySelector` method instead of an `ExportKindSelector`. (#2274)
- Metrics API cleanup. The `metric/sdkapi` package has been created to relocate the API-to-SDK interface: - Metrics API cleanup. The `metric/sdkapi` package has been created to relocate the API-to-SDK interface:
- The following interface types simply moved from `metric` to `metric/sdkapi`: `Descriptor`, `MeterImpl`, `InstrumentImpl`, `SyncImpl`, `BoundSyncImpl`, `AsyncImpl`, `AsyncRunner`, `AsyncSingleRunner`, and `AsyncBatchRunner` - The following interface types simply moved from `metric` to `metric/sdkapi`: `Descriptor`, `MeterImpl`, `InstrumentImpl`, `SyncImpl`, `BoundSyncImpl`, `AsyncImpl`, `AsyncRunner`, `AsyncSingleRunner`, and `AsyncBatchRunner`
- The following struct types moved and are replaced with type aliases, since they are exposed to the user: `Observation`, `Measurement`. - The following struct types moved and are replaced with type aliases, since they are exposed to the user: `Observation`, `Measurement`.

View File

@ -79,7 +79,7 @@ var _ export.Reader = &metricReader{}
// ForEach iterates through the metrics data, synthesizing an // ForEach iterates through the metrics data, synthesizing an
// export.Record with the appropriate aggregation for the exporter. // export.Record with the appropriate aggregation for the exporter.
func (d *metricReader) ForEach(exporter export.ExportKindSelector, f func(export.Record) error) error { func (d *metricReader) ForEach(_ aggregation.TemporalitySelector, f func(export.Record) error) error {
for _, m := range d.metrics { for _, m := range d.metrics {
descriptor, err := convertDescriptor(m.Descriptor) descriptor, err := convertDescriptor(m.Descriptor)
if err != nil { if err != nil {

View File

@ -48,7 +48,7 @@ type fakeExporter struct {
} }
func (f *fakeExporter) Export(ctx context.Context, res *resource.Resource, ilr exportmetric.InstrumentationLibraryReader) error { func (f *fakeExporter) Export(ctx context.Context, res *resource.Resource, ilr exportmetric.InstrumentationLibraryReader) error {
return controllertest.ReadAll(ilr, export.StatelessExportKindSelector(), return controllertest.ReadAll(ilr, aggregation.StatelessTemporalitySelector(),
func(_ instrumentation.Library, record exportmetric.Record) error { func(_ instrumentation.Library, record exportmetric.Record) error {
f.resource = res f.resource = res
f.records = append(f.records, record) f.records = append(f.records, record)

View File

@ -26,7 +26,7 @@ import (
"go.opentelemetry.io/otel/exporters/prometheus" "go.opentelemetry.io/otel/exporters/prometheus"
"go.opentelemetry.io/otel/metric" "go.opentelemetry.io/otel/metric"
"go.opentelemetry.io/otel/metric/global" "go.opentelemetry.io/otel/metric/global"
export "go.opentelemetry.io/otel/sdk/export/metric" "go.opentelemetry.io/otel/sdk/export/metric/aggregation"
"go.opentelemetry.io/otel/sdk/metric/aggregator/histogram" "go.opentelemetry.io/otel/sdk/metric/aggregator/histogram"
controller "go.opentelemetry.io/otel/sdk/metric/controller/basic" controller "go.opentelemetry.io/otel/sdk/metric/controller/basic"
processor "go.opentelemetry.io/otel/sdk/metric/processor/basic" processor "go.opentelemetry.io/otel/sdk/metric/processor/basic"
@ -44,7 +44,7 @@ func initMeter() {
selector.NewWithHistogramDistribution( selector.NewWithHistogramDistribution(
histogram.WithExplicitBoundaries(config.DefaultHistogramBoundaries), histogram.WithExplicitBoundaries(config.DefaultHistogramBoundaries),
), ),
export.CumulativeExportKindSelector(), aggregation.CumulativeTemporalitySelector(),
processor.WithMemory(true), processor.WithMemory(true),
), ),
) )

View File

@ -33,8 +33,8 @@ var (
// Exporter exports metrics data in the OTLP wire format. // Exporter exports metrics data in the OTLP wire format.
type Exporter struct { type Exporter struct {
client Client client Client
exportKindSelector metricsdk.ExportKindSelector temporalitySelector aggregation.TemporalitySelector
mu sync.RWMutex mu sync.RWMutex
started bool started bool
@ -96,8 +96,8 @@ func (e *Exporter) Shutdown(ctx context.Context) error {
return err return err
} }
func (e *Exporter) ExportKindFor(descriptor *sdkapi.Descriptor, aggregatorKind aggregation.Kind) metricsdk.ExportKind { func (e *Exporter) TemporalityFor(descriptor *sdkapi.Descriptor, kind aggregation.Kind) aggregation.Temporality {
return e.exportKindSelector.ExportKindFor(descriptor, aggregatorKind) return e.temporalitySelector.TemporalityFor(descriptor, kind)
} }
var _ metricsdk.Exporter = (*Exporter)(nil) var _ metricsdk.Exporter = (*Exporter)(nil)
@ -114,10 +114,10 @@ func New(ctx context.Context, client Client, opts ...Option) (*Exporter, error)
// NewUnstarted constructs a new Exporter and does not start it. // NewUnstarted constructs a new Exporter and does not start it.
func NewUnstarted(client Client, opts ...Option) *Exporter { func NewUnstarted(client Client, opts ...Option) *Exporter {
cfg := config{ cfg := config{
// Note: the default ExportKindSelector is specified // Note: the default TemporalitySelector is specified
// as Cumulative: // as Cumulative:
// https://github.com/open-telemetry/opentelemetry-specification/issues/731 // https://github.com/open-telemetry/opentelemetry-specification/issues/731
exportKindSelector: metricsdk.CumulativeExportKindSelector(), temporalitySelector: aggregation.CumulativeTemporalitySelector(),
} }
for _, opt := range opts { for _, opt := range opts {
@ -125,8 +125,8 @@ func NewUnstarted(client Client, opts ...Option) *Exporter {
} }
e := &Exporter{ e := &Exporter{
client: client, client: client,
exportKindSelector: cfg.exportKindSelector, temporalitySelector: cfg.temporalitySelector,
} }
return e return e

View File

@ -33,6 +33,7 @@ import (
"go.opentelemetry.io/otel/metric/number" "go.opentelemetry.io/otel/metric/number"
"go.opentelemetry.io/otel/metric/sdkapi" "go.opentelemetry.io/otel/metric/sdkapi"
metricsdk "go.opentelemetry.io/otel/sdk/export/metric" metricsdk "go.opentelemetry.io/otel/sdk/export/metric"
"go.opentelemetry.io/otel/sdk/export/metric/aggregation"
"go.opentelemetry.io/otel/sdk/instrumentation" "go.opentelemetry.io/otel/sdk/instrumentation"
"go.opentelemetry.io/otel/sdk/metric/aggregator/histogram" "go.opentelemetry.io/otel/sdk/metric/aggregator/histogram"
"go.opentelemetry.io/otel/sdk/metric/aggregator/sum" "go.opentelemetry.io/otel/sdk/metric/aggregator/sum"
@ -606,7 +607,7 @@ func TestResourceInstLibMetricGroupingExport(t *testing.T) {
) )
} }
func TestStatelessExportKind(t *testing.T) { func TestStatelessAggregationTemporality(t *testing.T) {
type testcase struct { type testcase struct {
name string name string
instrumentKind sdkapi.InstrumentKind instrumentKind sdkapi.InstrumentKind
@ -624,8 +625,8 @@ func TestStatelessExportKind(t *testing.T) {
runMetricExportTests( runMetricExportTests(
t, t,
[]otlpmetric.Option{ []otlpmetric.Option{
otlpmetric.WithMetricExportKindSelector( otlpmetric.WithMetricAggregationTemporalitySelector(
metricsdk.StatelessExportKindSelector(), aggregation.StatelessTemporalitySelector(),
), ),
}, },
testerAResource, testerAResource,

View File

@ -72,12 +72,12 @@ func toNanos(t time.Time) uint64 {
// InstrumentationLibraryReader transforms all records contained in a checkpoint into // InstrumentationLibraryReader transforms all records contained in a checkpoint into
// batched OTLP ResourceMetrics. // batched OTLP ResourceMetrics.
func InstrumentationLibraryReader(ctx context.Context, exportSelector export.ExportKindSelector, res *resource.Resource, ilmr export.InstrumentationLibraryReader, numWorkers uint) (*metricpb.ResourceMetrics, error) { func InstrumentationLibraryReader(ctx context.Context, temporalitySelector aggregation.TemporalitySelector, res *resource.Resource, ilmr export.InstrumentationLibraryReader, numWorkers uint) (*metricpb.ResourceMetrics, error) {
var ilms []*metricpb.InstrumentationLibraryMetrics var ilms []*metricpb.InstrumentationLibraryMetrics
err := ilmr.ForEach(func(lib instrumentation.Library, mr export.Reader) error { err := ilmr.ForEach(func(lib instrumentation.Library, mr export.Reader) error {
records, errc := source(ctx, exportSelector, mr) records, errc := source(ctx, temporalitySelector, mr)
// Start a fixed number of goroutines to transform records. // Start a fixed number of goroutines to transform records.
transformed := make(chan result) transformed := make(chan result)
@ -86,7 +86,7 @@ func InstrumentationLibraryReader(ctx context.Context, exportSelector export.Exp
for i := uint(0); i < numWorkers; i++ { for i := uint(0); i < numWorkers; i++ {
go func() { go func() {
defer wg.Done() defer wg.Done()
transformer(ctx, exportSelector, records, transformed) transformer(ctx, temporalitySelector, records, transformed)
}() }()
} }
go func() { go func() {
@ -134,14 +134,14 @@ func InstrumentationLibraryReader(ctx context.Context, exportSelector export.Exp
// source starts a goroutine that sends each one of the Records yielded by // source starts a goroutine that sends each one of the Records yielded by
// the Reader on the returned chan. Any error encountered will be sent // the Reader on the returned chan. Any error encountered will be sent
// on the returned error chan after seeding is complete. // on the returned error chan after seeding is complete.
func source(ctx context.Context, exportSelector export.ExportKindSelector, mr export.Reader) (<-chan export.Record, <-chan error) { func source(ctx context.Context, temporalitySelector aggregation.TemporalitySelector, mr export.Reader) (<-chan export.Record, <-chan error) {
errc := make(chan error, 1) errc := make(chan error, 1)
out := make(chan export.Record) out := make(chan export.Record)
// Seed records into process. // Seed records into process.
go func() { go func() {
defer close(out) defer close(out)
// No select is needed since errc is buffered. // No select is needed since errc is buffered.
errc <- mr.ForEach(exportSelector, func(r export.Record) error { errc <- mr.ForEach(temporalitySelector, func(r export.Record) error {
select { select {
case <-ctx.Done(): case <-ctx.Done():
return ErrContextCanceled return ErrContextCanceled
@ -155,9 +155,9 @@ func source(ctx context.Context, exportSelector export.ExportKindSelector, mr ex
// transformer transforms records read from the passed in chan into // transformer transforms records read from the passed in chan into
// OTLP Metrics which are sent on the out chan. // OTLP Metrics which are sent on the out chan.
func transformer(ctx context.Context, exportSelector export.ExportKindSelector, in <-chan export.Record, out chan<- result) { func transformer(ctx context.Context, temporalitySelector aggregation.TemporalitySelector, in <-chan export.Record, out chan<- result) {
for r := range in { for r := range in {
m, err := Record(exportSelector, r) m, err := Record(temporalitySelector, r)
// Propagate errors, but do not send empty results. // Propagate errors, but do not send empty results.
if err == nil && m == nil { if err == nil && m == nil {
continue continue
@ -237,7 +237,7 @@ func sink(ctx context.Context, in <-chan result) ([]*metricpb.Metric, error) {
// Record transforms a Record into an OTLP Metric. An ErrIncompatibleAgg // Record transforms a Record into an OTLP Metric. An ErrIncompatibleAgg
// error is returned if the Record Aggregator is not supported. // error is returned if the Record Aggregator is not supported.
func Record(exportSelector export.ExportKindSelector, r export.Record) (*metricpb.Metric, error) { func Record(temporalitySelector aggregation.TemporalitySelector, r export.Record) (*metricpb.Metric, error) {
agg := r.Aggregation() agg := r.Aggregation()
switch agg.Kind() { switch agg.Kind() {
case aggregation.MinMaxSumCountKind: case aggregation.MinMaxSumCountKind:
@ -252,7 +252,7 @@ func Record(exportSelector export.ExportKindSelector, r export.Record) (*metricp
if !ok { if !ok {
return nil, fmt.Errorf("%w: %T", ErrIncompatibleAgg, agg) return nil, fmt.Errorf("%w: %T", ErrIncompatibleAgg, agg)
} }
return histogramPoint(r, exportSelector.ExportKindFor(r.Descriptor(), aggregation.HistogramKind), h) return histogramPoint(r, temporalitySelector.TemporalityFor(r.Descriptor(), aggregation.HistogramKind), h)
case aggregation.SumKind: case aggregation.SumKind:
s, ok := agg.(aggregation.Sum) s, ok := agg.(aggregation.Sum)
@ -263,7 +263,7 @@ func Record(exportSelector export.ExportKindSelector, r export.Record) (*metricp
if err != nil { if err != nil {
return nil, err return nil, err
} }
return sumPoint(r, sum, r.StartTime(), r.EndTime(), exportSelector.ExportKindFor(r.Descriptor(), aggregation.SumKind), r.Descriptor().InstrumentKind().Monotonic()) return sumPoint(r, sum, r.StartTime(), r.EndTime(), temporalitySelector.TemporalityFor(r.Descriptor(), aggregation.SumKind), r.Descriptor().InstrumentKind().Monotonic())
case aggregation.LastValueKind: case aggregation.LastValueKind:
lv, ok := agg.(aggregation.LastValue) lv, ok := agg.(aggregation.LastValue)
@ -388,17 +388,17 @@ func gaugePoint(record export.Record, num number.Number, start, end time.Time) (
return m, nil return m, nil
} }
func exportKindToTemporality(ek export.ExportKind) metricpb.AggregationTemporality { func sdkTemporalityToTemporality(temporality aggregation.Temporality) metricpb.AggregationTemporality {
switch ek { switch temporality {
case export.DeltaExportKind: case aggregation.DeltaTemporality:
return metricpb.AggregationTemporality_AGGREGATION_TEMPORALITY_DELTA return metricpb.AggregationTemporality_AGGREGATION_TEMPORALITY_DELTA
case export.CumulativeExportKind: case aggregation.CumulativeTemporality:
return metricpb.AggregationTemporality_AGGREGATION_TEMPORALITY_CUMULATIVE return metricpb.AggregationTemporality_AGGREGATION_TEMPORALITY_CUMULATIVE
} }
return metricpb.AggregationTemporality_AGGREGATION_TEMPORALITY_UNSPECIFIED return metricpb.AggregationTemporality_AGGREGATION_TEMPORALITY_UNSPECIFIED
} }
func sumPoint(record export.Record, num number.Number, start, end time.Time, ek export.ExportKind, monotonic bool) (*metricpb.Metric, error) { func sumPoint(record export.Record, num number.Number, start, end time.Time, temporality aggregation.Temporality, monotonic bool) (*metricpb.Metric, error) {
desc := record.Descriptor() desc := record.Descriptor()
labels := record.Labels() labels := record.Labels()
@ -413,7 +413,7 @@ func sumPoint(record export.Record, num number.Number, start, end time.Time, ek
m.Data = &metricpb.Metric_Sum{ m.Data = &metricpb.Metric_Sum{
Sum: &metricpb.Sum{ Sum: &metricpb.Sum{
IsMonotonic: monotonic, IsMonotonic: monotonic,
AggregationTemporality: exportKindToTemporality(ek), AggregationTemporality: sdkTemporalityToTemporality(temporality),
DataPoints: []*metricpb.NumberDataPoint{ DataPoints: []*metricpb.NumberDataPoint{
{ {
Value: &metricpb.NumberDataPoint_AsInt{ Value: &metricpb.NumberDataPoint_AsInt{
@ -430,7 +430,7 @@ func sumPoint(record export.Record, num number.Number, start, end time.Time, ek
m.Data = &metricpb.Metric_Sum{ m.Data = &metricpb.Metric_Sum{
Sum: &metricpb.Sum{ Sum: &metricpb.Sum{
IsMonotonic: monotonic, IsMonotonic: monotonic,
AggregationTemporality: exportKindToTemporality(ek), AggregationTemporality: sdkTemporalityToTemporality(temporality),
DataPoints: []*metricpb.NumberDataPoint{ DataPoints: []*metricpb.NumberDataPoint{
{ {
Value: &metricpb.NumberDataPoint_AsDouble{ Value: &metricpb.NumberDataPoint_AsDouble{
@ -522,7 +522,7 @@ func histogramValues(a aggregation.Histogram) (boundaries []float64, counts []ui
} }
// histogram transforms a Histogram Aggregator into an OTLP Metric. // histogram transforms a Histogram Aggregator into an OTLP Metric.
func histogramPoint(record export.Record, ek export.ExportKind, a aggregation.Histogram) (*metricpb.Metric, error) { func histogramPoint(record export.Record, temporality aggregation.Temporality, a aggregation.Histogram) (*metricpb.Metric, error) {
desc := record.Descriptor() desc := record.Descriptor()
labels := record.Labels() labels := record.Labels()
boundaries, counts, err := histogramValues(a) boundaries, counts, err := histogramValues(a)
@ -546,7 +546,7 @@ func histogramPoint(record export.Record, ek export.ExportKind, a aggregation.Hi
Unit: string(desc.Unit()), Unit: string(desc.Unit()),
Data: &metricpb.Metric_Histogram{ Data: &metricpb.Metric_Histogram{
Histogram: &metricpb.Histogram{ Histogram: &metricpb.Histogram{
AggregationTemporality: exportKindToTemporality(ek), AggregationTemporality: sdkTemporalityToTemporality(temporality),
DataPoints: []*metricpb.HistogramDataPoint{ DataPoints: []*metricpb.HistogramDataPoint{
{ {
Sum: sum.CoerceToFloat64(desc.NumberKind()), Sum: sum.CoerceToFloat64(desc.NumberKind()),

View File

@ -190,7 +190,7 @@ func TestSumIntDataPoints(t *testing.T) {
value, err := ckpt.Sum() value, err := ckpt.Sum()
require.NoError(t, err) require.NoError(t, err)
if m, err := sumPoint(record, value, record.StartTime(), record.EndTime(), export.CumulativeExportKind, true); assert.NoError(t, err) { if m, err := sumPoint(record, value, record.StartTime(), record.EndTime(), aggregation.CumulativeTemporality, true); assert.NoError(t, err) {
assert.Nil(t, m.GetGauge()) assert.Nil(t, m.GetGauge())
assert.Equal(t, &metricpb.Sum{ assert.Equal(t, &metricpb.Sum{
AggregationTemporality: otelCumulative, AggregationTemporality: otelCumulative,
@ -229,7 +229,7 @@ func TestSumFloatDataPoints(t *testing.T) {
value, err := ckpt.Sum() value, err := ckpt.Sum()
require.NoError(t, err) require.NoError(t, err)
if m, err := sumPoint(record, value, record.StartTime(), record.EndTime(), export.DeltaExportKind, false); assert.NoError(t, err) { if m, err := sumPoint(record, value, record.StartTime(), record.EndTime(), aggregation.DeltaTemporality, false); assert.NoError(t, err) {
assert.Nil(t, m.GetGauge()) assert.Nil(t, m.GetGauge())
assert.Equal(t, &metricpb.Sum{ assert.Equal(t, &metricpb.Sum{
IsMonotonic: false, IsMonotonic: false,
@ -367,7 +367,7 @@ func TestSumErrUnknownValueType(t *testing.T) {
value, err := s.Sum() value, err := s.Sum()
require.NoError(t, err) require.NoError(t, err)
_, err = sumPoint(record, value, record.StartTime(), record.EndTime(), export.CumulativeExportKind, true) _, err = sumPoint(record, value, record.StartTime(), record.EndTime(), aggregation.CumulativeTemporality, true)
assert.Error(t, err) assert.Error(t, err)
if !errors.Is(err, ErrUnknownValueType) { if !errors.Is(err, ErrUnknownValueType) {
t.Errorf("expected ErrUnknownValueType, got %v", err) t.Errorf("expected ErrUnknownValueType, got %v", err)
@ -451,7 +451,7 @@ func TestRecordAggregatorIncompatibleErrors(t *testing.T) {
kind: kind, kind: kind,
agg: agg, agg: agg,
} }
return Record(export.CumulativeExportKindSelector(), export.NewRecord(&desc, &labels, test, intervalStart, intervalEnd)) return Record(aggregation.CumulativeTemporalitySelector(), export.NewRecord(&desc, &labels, test, intervalStart, intervalEnd))
} }
mpb, err := makeMpb(aggregation.SumKind, &lastvalue.New(1)[0]) mpb, err := makeMpb(aggregation.SumKind, &lastvalue.New(1)[0])
@ -483,7 +483,7 @@ func TestRecordAggregatorUnexpectedErrors(t *testing.T) {
makeMpb := func(kind aggregation.Kind, agg aggregation.Aggregation) (*metricpb.Metric, error) { makeMpb := func(kind aggregation.Kind, agg aggregation.Aggregation) (*metricpb.Metric, error) {
desc := metrictest.NewDescriptor("things", sdkapi.CounterInstrumentKind, number.Int64Kind) desc := metrictest.NewDescriptor("things", sdkapi.CounterInstrumentKind, number.Int64Kind)
labels := attribute.NewSet() labels := attribute.NewSet()
return Record(export.CumulativeExportKindSelector(), export.NewRecord(&desc, &labels, agg, intervalStart, intervalEnd)) return Record(aggregation.CumulativeTemporalitySelector(), export.NewRecord(&desc, &labels, agg, intervalStart, intervalEnd))
} }
errEx := fmt.Errorf("timeout") errEx := fmt.Errorf("timeout")

View File

@ -30,7 +30,7 @@ import (
"go.opentelemetry.io/otel/metric" "go.opentelemetry.io/otel/metric"
"go.opentelemetry.io/otel/metric/number" "go.opentelemetry.io/otel/metric/number"
"go.opentelemetry.io/otel/metric/sdkapi" "go.opentelemetry.io/otel/metric/sdkapi"
exportmetric "go.opentelemetry.io/otel/sdk/export/metric" "go.opentelemetry.io/otel/sdk/export/metric/aggregation"
controller "go.opentelemetry.io/otel/sdk/metric/controller/basic" controller "go.opentelemetry.io/otel/sdk/metric/controller/basic"
processor "go.opentelemetry.io/otel/sdk/metric/processor/basic" processor "go.opentelemetry.io/otel/sdk/metric/processor/basic"
"go.opentelemetry.io/otel/sdk/metric/selector/simple" "go.opentelemetry.io/otel/sdk/metric/selector/simple"
@ -40,7 +40,7 @@ import (
// themselves. // themselves.
func RunEndToEndTest(ctx context.Context, t *testing.T, exp *otlpmetric.Exporter, mcMetrics Collector) { func RunEndToEndTest(ctx context.Context, t *testing.T, exp *otlpmetric.Exporter, mcMetrics Collector) {
selector := simple.NewWithInexpensiveDistribution() selector := simple.NewWithInexpensiveDistribution()
proc := processor.NewFactory(selector, exportmetric.StatelessExportKindSelector()) proc := processor.NewFactory(selector, aggregation.StatelessTemporalitySelector())
cont := controller.New(proc, controller.WithExporter(exp)) cont := controller.New(proc, controller.WithExporter(exp))
require.NoError(t, cont.Start(ctx)) require.NoError(t, cont.Start(ctx))

View File

@ -14,7 +14,7 @@
package otlpmetric // import "go.opentelemetry.io/otel/exporters/otlp/otlpmetric" package otlpmetric // import "go.opentelemetry.io/otel/exporters/otlp/otlpmetric"
import metricsdk "go.opentelemetry.io/otel/sdk/export/metric" import "go.opentelemetry.io/otel/sdk/export/metric/aggregation"
// Option are setting options passed to an Exporter on creation. // Option are setting options passed to an Exporter on creation.
type Option interface { type Option interface {
@ -28,15 +28,15 @@ func (fn exporterOptionFunc) apply(cfg *config) {
} }
type config struct { type config struct {
exportKindSelector metricsdk.ExportKindSelector temporalitySelector aggregation.TemporalitySelector
} }
// WithMetricExportKindSelector defines the ExportKindSelector used // WithMetricAggregationTemporalitySelector defines the aggregation.TemporalitySelector used
// for selecting AggregationTemporality (i.e., Cumulative vs. Delta // for selecting aggregation.Temporality (i.e., Cumulative vs. Delta
// aggregation). If not specified otherwise, exporter will use a // aggregation). If not specified otherwise, exporter will use a
// cumulative export kind selector. // cumulative temporality selector.
func WithMetricExportKindSelector(selector metricsdk.ExportKindSelector) Option { func WithMetricAggregationTemporalitySelector(selector aggregation.TemporalitySelector) Option {
return exporterOptionFunc(func(cfg *config) { return exporterOptionFunc(func(cfg *config) {
cfg.exportKindSelector = selector cfg.temporalitySelector = selector
}) })
} }

View File

@ -132,9 +132,9 @@ func (e *Exporter) Controller() *controller.Controller {
return e.controller return e.controller
} }
// ExportKindFor implements ExportKindSelector. // TemporalityFor implements TemporalitySelector.
func (e *Exporter) ExportKindFor(desc *sdkapi.Descriptor, kind aggregation.Kind) export.ExportKind { func (e *Exporter) TemporalityFor(desc *sdkapi.Descriptor, kind aggregation.Kind) aggregation.Temporality {
return export.CumulativeExportKindSelector().ExportKindFor(desc, kind) return aggregation.CumulativeTemporalitySelector().TemporalityFor(desc, kind)
} }
// ServeHTTP implements http.Handler. // ServeHTTP implements http.Handler.

View File

@ -27,7 +27,7 @@ import (
"go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/attribute"
"go.opentelemetry.io/otel/exporters/prometheus" "go.opentelemetry.io/otel/exporters/prometheus"
"go.opentelemetry.io/otel/metric" "go.opentelemetry.io/otel/metric"
export "go.opentelemetry.io/otel/sdk/export/metric" "go.opentelemetry.io/otel/sdk/export/metric/aggregation"
"go.opentelemetry.io/otel/sdk/metric/aggregator/histogram" "go.opentelemetry.io/otel/sdk/metric/aggregator/histogram"
controller "go.opentelemetry.io/otel/sdk/metric/controller/basic" controller "go.opentelemetry.io/otel/sdk/metric/controller/basic"
processor "go.opentelemetry.io/otel/sdk/metric/processor/basic" processor "go.opentelemetry.io/otel/sdk/metric/processor/basic"
@ -88,7 +88,7 @@ func newPipeline(config prometheus.Config, options ...controller.Option) (*prome
selector.NewWithHistogramDistribution( selector.NewWithHistogramDistribution(
histogram.WithExplicitBoundaries(config.DefaultHistogramBoundaries), histogram.WithExplicitBoundaries(config.DefaultHistogramBoundaries),
), ),
export.CumulativeExportKindSelector(), aggregation.CumulativeTemporalitySelector(),
processor.WithMemory(true), processor.WithMemory(true),
), ),
options..., options...,

View File

@ -47,8 +47,8 @@ type line struct {
Timestamp *time.Time `json:"Timestamp,omitempty"` Timestamp *time.Time `json:"Timestamp,omitempty"`
} }
func (e *metricExporter) ExportKindFor(desc *sdkapi.Descriptor, kind aggregation.Kind) exportmetric.ExportKind { func (e *metricExporter) TemporalityFor(desc *sdkapi.Descriptor, kind aggregation.Kind) aggregation.Temporality {
return exportmetric.StatelessExportKindSelector().ExportKindFor(desc, kind) return aggregation.StatelessTemporalitySelector().TemporalityFor(desc, kind)
} }
func (e *metricExporter) Export(_ context.Context, res *resource.Resource, reader exportmetric.InstrumentationLibraryReader) error { func (e *metricExporter) Export(_ context.Context, res *resource.Resource, reader exportmetric.InstrumentationLibraryReader) error {

View File

@ -30,7 +30,7 @@ import (
"go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/attribute"
"go.opentelemetry.io/otel/metric" "go.opentelemetry.io/otel/metric"
export "go.opentelemetry.io/otel/sdk/export/metric" "go.opentelemetry.io/otel/sdk/export/metric/aggregation"
controller "go.opentelemetry.io/otel/sdk/metric/controller/basic" controller "go.opentelemetry.io/otel/sdk/metric/controller/basic"
processor "go.opentelemetry.io/otel/sdk/metric/processor/basic" processor "go.opentelemetry.io/otel/sdk/metric/processor/basic"
"go.opentelemetry.io/otel/sdk/metric/processor/processortest" "go.opentelemetry.io/otel/sdk/metric/processor/processortest"
@ -61,7 +61,7 @@ func newFixtureWithResource(t *testing.T, res *resource.Resource, opts ...stdout
t.Fatal("Error building fixture: ", err) t.Fatal("Error building fixture: ", err)
} }
aggSel := processortest.AggregatorSelector() aggSel := processortest.AggregatorSelector()
proc := processor.NewFactory(aggSel, export.StatelessExportKindSelector()) proc := processor.NewFactory(aggSel, aggregation.StatelessTemporalitySelector())
cont := controller.New(proc, cont := controller.New(proc,
controller.WithExporter(exp), controller.WithExporter(exp),
controller.WithResource(res), controller.WithResource(res),
@ -87,7 +87,7 @@ func (fix testFixture) Output() string {
func TestStdoutTimestamp(t *testing.T) { func TestStdoutTimestamp(t *testing.T) {
var buf bytes.Buffer var buf bytes.Buffer
aggSel := processortest.AggregatorSelector() aggSel := processortest.AggregatorSelector()
proc := processor.NewFactory(aggSel, export.CumulativeExportKindSelector()) proc := processor.NewFactory(aggSel, aggregation.CumulativeTemporalitySelector())
exporter, err := stdoutmetric.New( exporter, err := stdoutmetric.New(
stdoutmetric.WithWriter(&buf), stdoutmetric.WithWriter(&buf),
) )

View File

@ -401,6 +401,7 @@ github.com/imdario/mergo v0.3.12 h1:b6R2BslTbIEToALKP7LxUvijTsNI9TAe80pLWN2g/HU=
github.com/imdario/mergo v0.3.12/go.mod h1:jmQim1M+e3UYxmgPu/WyfjB3N3VflVyUjjjwH0dnCYA= github.com/imdario/mergo v0.3.12/go.mod h1:jmQim1M+e3UYxmgPu/WyfjB3N3VflVyUjjjwH0dnCYA=
github.com/inconshreveable/mousetrap v1.0.0 h1:Z8tu5sraLXCXIcARxBp/8cbvlwVa7Z1NHg9XEKhtSvM= github.com/inconshreveable/mousetrap v1.0.0 h1:Z8tu5sraLXCXIcARxBp/8cbvlwVa7Z1NHg9XEKhtSvM=
github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8=
github.com/itchyny/go-flags v1.5.0 h1:Z5q2ist2sfDjDlExVPBrMqlsEDxDR2h4zuOElB0OEYI=
github.com/itchyny/go-flags v1.5.0/go.mod h1:lenkYuCobuxLBAd/HGFE4LRoW8D3B6iXRQfWYJ+MNbA= github.com/itchyny/go-flags v1.5.0/go.mod h1:lenkYuCobuxLBAd/HGFE4LRoW8D3B6iXRQfWYJ+MNbA=
github.com/itchyny/gojq v0.12.5 h1:6SJ1BQ1VAwJAlIvLSIZmqHP/RUEq3qfVWvsRxrqhsD0= github.com/itchyny/gojq v0.12.5 h1:6SJ1BQ1VAwJAlIvLSIZmqHP/RUEq3qfVWvsRxrqhsD0=
github.com/itchyny/gojq v0.12.5/go.mod h1:3e1hZXv+Kwvdp6V9HXpVrvddiHVApi5EDZwS+zLFeiE= github.com/itchyny/gojq v0.12.5/go.mod h1:3e1hZXv+Kwvdp6V9HXpVrvddiHVApi5EDZwS+zLFeiE=

View File

@ -0,0 +1,117 @@
// 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.
//go:generate stringer -type=Temporality
package aggregation // import "go.opentelemetry.io/otel/sdk/export/metric/aggregation"
import (
"go.opentelemetry.io/otel/metric/sdkapi"
)
// Temporality indicates the temporal aggregation exported by an exporter.
// These bits may be OR-d together when multiple exporters are in use.
type Temporality uint8
const (
// CumulativeTemporality indicates that an Exporter expects a
// Cumulative Aggregation.
CumulativeTemporality Temporality = 1
// DeltaTemporality indicates that an Exporter expects a
// Delta Aggregation.
DeltaTemporality Temporality = 2
)
// Includes returns if t includes support for other temporality.
func (t Temporality) Includes(other Temporality) bool {
return t&other != 0
}
// MemoryRequired returns whether an exporter of this temporality requires
// memory to export correctly.
func (t Temporality) MemoryRequired(mkind sdkapi.InstrumentKind) bool {
switch mkind {
case sdkapi.HistogramInstrumentKind, sdkapi.GaugeObserverInstrumentKind,
sdkapi.CounterInstrumentKind, sdkapi.UpDownCounterInstrumentKind:
// Delta-oriented instruments:
return t.Includes(CumulativeTemporality)
case sdkapi.CounterObserverInstrumentKind, sdkapi.UpDownCounterObserverInstrumentKind:
// Cumulative-oriented instruments:
return t.Includes(DeltaTemporality)
}
// Something unexpected is happening--we could panic. This
// will become an error when the exporter tries to access a
// checkpoint, presumably, so let it be.
return false
}
type (
constantTemporalitySelector Temporality
statelessTemporalitySelector struct{}
)
var (
_ TemporalitySelector = constantTemporalitySelector(0)
_ TemporalitySelector = statelessTemporalitySelector{}
)
// ConstantTemporalitySelector returns an TemporalitySelector that returns
// a constant Temporality.
func ConstantTemporalitySelector(t Temporality) TemporalitySelector {
return constantTemporalitySelector(t)
}
// CumulativeTemporalitySelector returns an TemporalitySelector that
// always returns CumulativeTemporality.
func CumulativeTemporalitySelector() TemporalitySelector {
return ConstantTemporalitySelector(CumulativeTemporality)
}
// DeltaTemporalitySelector returns an TemporalitySelector that
// always returns DeltaTemporality.
func DeltaTemporalitySelector() TemporalitySelector {
return ConstantTemporalitySelector(DeltaTemporality)
}
// StatelessTemporalitySelector returns an TemporalitySelector that
// always returns the Temporality that avoids long-term memory
// requirements.
func StatelessTemporalitySelector() TemporalitySelector {
return statelessTemporalitySelector{}
}
// TemporalityFor implements TemporalitySelector.
func (c constantTemporalitySelector) TemporalityFor(_ *sdkapi.Descriptor, _ Kind) Temporality {
return Temporality(c)
}
// TemporalityFor implements TemporalitySelector.
func (s statelessTemporalitySelector) TemporalityFor(desc *sdkapi.Descriptor, kind Kind) Temporality {
if kind == SumKind && desc.InstrumentKind().PrecomputedSum() {
return CumulativeTemporality
}
return DeltaTemporality
}
// TemporalitySelector is a sub-interface of Exporter used to indicate
// whether the Processor should compute Delta or Cumulative
// Aggregations.
type TemporalitySelector interface {
// TemporalityFor should return the correct Temporality that
// should be used when exporting data for the given metric
// instrument and Aggregator kind.
TemporalityFor(descriptor *sdkapi.Descriptor, aggregationKind Kind) Temporality
}

View File

@ -0,0 +1,25 @@
// Code generated by "stringer -type=Temporality"; DO NOT EDIT.
package aggregation // import "go.opentelemetry.io/otel/sdk/export/metric/aggregation"
import "strconv"
func _() {
// An "invalid array index" compiler error signifies that the constant values have changed.
// Re-run the stringer command to generate them again.
var x [1]struct{}
_ = x[CumulativeTemporality-1]
_ = x[DeltaTemporality-2]
}
const _Temporality_name = "CumulativeTemporalityDeltaTemporality"
var _Temporality_index = [...]uint8{0, 21, 37}
func (i Temporality) String() string {
i -= 1
if i >= Temporality(len(_Temporality_index)-1) {
return "Temporality(" + strconv.FormatInt(int64(i+1), 10) + ")"
}
return _Temporality_name[_Temporality_index[i]:_Temporality_index[i+1]]
}

View File

@ -0,0 +1,74 @@
// 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 aggregation
import (
"testing"
"github.com/stretchr/testify/require"
"go.opentelemetry.io/otel/metric/metrictest"
"go.opentelemetry.io/otel/metric/number"
"go.opentelemetry.io/otel/metric/sdkapi"
)
func TestTemporalityIncludes(t *testing.T) {
require.True(t, CumulativeTemporality.Includes(CumulativeTemporality))
require.True(t, DeltaTemporality.Includes(CumulativeTemporality|DeltaTemporality))
}
var deltaMemoryTemporalties = []sdkapi.InstrumentKind{
sdkapi.CounterObserverInstrumentKind,
sdkapi.UpDownCounterObserverInstrumentKind,
}
var cumulativeMemoryTemporalties = []sdkapi.InstrumentKind{
sdkapi.HistogramInstrumentKind,
sdkapi.GaugeObserverInstrumentKind,
sdkapi.CounterInstrumentKind,
sdkapi.UpDownCounterInstrumentKind,
}
func TestTemporalityMemoryRequired(t *testing.T) {
for _, kind := range deltaMemoryTemporalties {
require.True(t, DeltaTemporality.MemoryRequired(kind))
require.False(t, CumulativeTemporality.MemoryRequired(kind))
}
for _, kind := range cumulativeMemoryTemporalties {
require.True(t, CumulativeTemporality.MemoryRequired(kind))
require.False(t, DeltaTemporality.MemoryRequired(kind))
}
}
func TestTemporalitySelectors(t *testing.T) {
cAggTemp := CumulativeTemporalitySelector()
dAggTemp := DeltaTemporalitySelector()
sAggTemp := StatelessTemporalitySelector()
for _, ikind := range append(deltaMemoryTemporalties, cumulativeMemoryTemporalties...) {
desc := metrictest.NewDescriptor("instrument", ikind, number.Int64Kind)
var akind Kind
if ikind.Adding() {
akind = SumKind
} else {
akind = HistogramKind
}
require.Equal(t, CumulativeTemporality, cAggTemp.TemporalityFor(&desc, akind))
require.Equal(t, DeltaTemporality, dAggTemp.TemporalityFor(&desc, akind))
require.False(t, sAggTemp.TemporalityFor(&desc, akind).MemoryRequired(ikind))
}
}

View File

@ -1,25 +0,0 @@
// Code generated by "stringer -type=ExportKind"; DO NOT EDIT.
package metric // import "go.opentelemetry.io/otel/sdk/export/metric"
import "strconv"
func _() {
// An "invalid array index" compiler error signifies that the constant values have changed.
// Re-run the stringer command to generate them again.
var x [1]struct{}
_ = x[CumulativeExportKind-1]
_ = x[DeltaExportKind-2]
}
const _ExportKind_name = "CumulativeExportKindDeltaExportKind"
var _ExportKind_index = [...]uint8{0, 20, 35}
func (i ExportKind) String() string {
i -= 1
if i < 0 || i >= ExportKind(len(_ExportKind_index)-1) {
return "ExportKind(" + strconv.FormatInt(int64(i+1), 10) + ")"
}
return _ExportKind_name[_ExportKind_index[i]:_ExportKind_index[i+1]]
}

View File

@ -1,75 +0,0 @@
// 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 metric
import (
"testing"
"github.com/stretchr/testify/require"
"go.opentelemetry.io/otel/metric/metrictest"
"go.opentelemetry.io/otel/metric/number"
"go.opentelemetry.io/otel/metric/sdkapi"
"go.opentelemetry.io/otel/sdk/export/metric/aggregation"
)
func TestExportKindIncludes(t *testing.T) {
require.True(t, CumulativeExportKind.Includes(CumulativeExportKind))
require.True(t, DeltaExportKind.Includes(CumulativeExportKind|DeltaExportKind))
}
var deltaMemoryKinds = []sdkapi.InstrumentKind{
sdkapi.CounterObserverInstrumentKind,
sdkapi.UpDownCounterObserverInstrumentKind,
}
var cumulativeMemoryKinds = []sdkapi.InstrumentKind{
sdkapi.HistogramInstrumentKind,
sdkapi.GaugeObserverInstrumentKind,
sdkapi.CounterInstrumentKind,
sdkapi.UpDownCounterInstrumentKind,
}
func TestExportKindMemoryRequired(t *testing.T) {
for _, kind := range deltaMemoryKinds {
require.True(t, DeltaExportKind.MemoryRequired(kind))
require.False(t, CumulativeExportKind.MemoryRequired(kind))
}
for _, kind := range cumulativeMemoryKinds {
require.True(t, CumulativeExportKind.MemoryRequired(kind))
require.False(t, DeltaExportKind.MemoryRequired(kind))
}
}
func TestExportKindSelectors(t *testing.T) {
ceks := CumulativeExportKindSelector()
deks := DeltaExportKindSelector()
seks := StatelessExportKindSelector()
for _, ikind := range append(deltaMemoryKinds, cumulativeMemoryKinds...) {
desc := metrictest.NewDescriptor("instrument", ikind, number.Int64Kind)
var akind aggregation.Kind
if ikind.Adding() {
akind = aggregation.SumKind
} else {
akind = aggregation.HistogramKind
}
require.Equal(t, CumulativeExportKind, ceks.ExportKindFor(&desc, akind))
require.Equal(t, DeltaExportKind, deks.ExportKindFor(&desc, akind))
require.False(t, seks.ExportKindFor(&desc, akind).MemoryRequired(ikind))
}
}

View File

@ -12,8 +12,6 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
//go:generate stringer -type=ExportKind
package metric // import "go.opentelemetry.io/otel/sdk/export/metric" package metric // import "go.opentelemetry.io/otel/sdk/export/metric"
import ( import (
@ -219,20 +217,10 @@ type Exporter interface {
// Processor that just completed collection. // Processor that just completed collection.
Export(ctx context.Context, resource *resource.Resource, reader InstrumentationLibraryReader) error Export(ctx context.Context, resource *resource.Resource, reader InstrumentationLibraryReader) error
// ExportKindSelector is an interface used by the Processor // TemporalitySelector is an interface used by the Processor
// in deciding whether to compute Delta or Cumulative // in deciding whether to compute Delta or Cumulative
// Aggregations when passing Records to this Exporter. // Aggregations when passing Records to this Exporter.
ExportKindSelector aggregation.TemporalitySelector
}
// ExportKindSelector is a sub-interface of Exporter used to indicate
// whether the Processor should compute Delta or Cumulative
// Aggregations.
type ExportKindSelector interface {
// ExportKindFor should return the correct ExportKind that
// should be used when exporting data for the given metric
// instrument and Aggregator kind.
ExportKindFor(descriptor *sdkapi.Descriptor, aggregatorKind aggregation.Kind) ExportKind
} }
// InstrumentationLibraryReader is an interface for exporters to iterate // InstrumentationLibraryReader is an interface for exporters to iterate
@ -254,7 +242,7 @@ type Reader interface {
// period. Each aggregated checkpoint returned by the // period. Each aggregated checkpoint returned by the
// function parameter may return an error. // function parameter may return an error.
// //
// The ExportKindSelector argument is used to determine // The TemporalitySelector argument is used to determine
// whether the Record is computed using Delta or Cumulative // whether the Record is computed using Delta or Cumulative
// aggregation. // aggregation.
// //
@ -262,7 +250,7 @@ type Reader interface {
// expected from the Meter implementation. Any other kind // expected from the Meter implementation. Any other kind
// of error will immediately halt ForEach and return // of error will immediately halt ForEach and return
// the error to the caller. // the error to the caller.
ForEach(kindSelector ExportKindSelector, recordFunc func(Record) error) error ForEach(tempSelector aggregation.TemporalitySelector, recordFunc func(Record) error) error
// Locker supports locking the checkpoint set. Collection // Locker supports locking the checkpoint set. Collection
// into the checkpoint set cannot take place (in case of a // into the checkpoint set cannot take place (in case of a
@ -364,90 +352,3 @@ func (r Record) StartTime() time.Time {
func (r Record) EndTime() time.Time { func (r Record) EndTime() time.Time {
return r.end return r.end
} }
// ExportKind indicates the kind of data exported by an exporter.
// These bits may be OR-d together when multiple exporters are in use.
type ExportKind int
const (
// CumulativeExportKind indicates that an Exporter expects a
// Cumulative Aggregation.
CumulativeExportKind ExportKind = 1
// DeltaExportKind indicates that an Exporter expects a
// Delta Aggregation.
DeltaExportKind ExportKind = 2
)
// Includes tests whether `kind` includes a specific kind of
// exporter.
func (kind ExportKind) Includes(has ExportKind) bool {
return kind&has != 0
}
// MemoryRequired returns whether an exporter of this kind requires
// memory to export correctly.
func (kind ExportKind) MemoryRequired(mkind sdkapi.InstrumentKind) bool {
switch mkind {
case sdkapi.HistogramInstrumentKind, sdkapi.GaugeObserverInstrumentKind,
sdkapi.CounterInstrumentKind, sdkapi.UpDownCounterInstrumentKind:
// Delta-oriented instruments:
return kind.Includes(CumulativeExportKind)
case sdkapi.CounterObserverInstrumentKind, sdkapi.UpDownCounterObserverInstrumentKind:
// Cumulative-oriented instruments:
return kind.Includes(DeltaExportKind)
}
// Something unexpected is happening--we could panic. This
// will become an error when the exporter tries to access a
// checkpoint, presumably, so let it be.
return false
}
type (
constantExportKindSelector ExportKind
statelessExportKindSelector struct{}
)
var (
_ ExportKindSelector = constantExportKindSelector(0)
_ ExportKindSelector = statelessExportKindSelector{}
)
// ConstantExportKindSelector returns an ExportKindSelector that returns
// a constant ExportKind, one that is either always cumulative or always delta.
func ConstantExportKindSelector(kind ExportKind) ExportKindSelector {
return constantExportKindSelector(kind)
}
// CumulativeExportKindSelector returns an ExportKindSelector that
// always returns CumulativeExportKind.
func CumulativeExportKindSelector() ExportKindSelector {
return ConstantExportKindSelector(CumulativeExportKind)
}
// DeltaExportKindSelector returns an ExportKindSelector that
// always returns DeltaExportKind.
func DeltaExportKindSelector() ExportKindSelector {
return ConstantExportKindSelector(DeltaExportKind)
}
// StatelessExportKindSelector returns an ExportKindSelector that
// always returns the ExportKind that avoids long-term memory
// requirements.
func StatelessExportKindSelector() ExportKindSelector {
return statelessExportKindSelector{}
}
// ExportKindFor implements ExportKindSelector.
func (c constantExportKindSelector) ExportKindFor(_ *sdkapi.Descriptor, _ aggregation.Kind) ExportKind {
return ExportKind(c)
}
// ExportKindFor implements ExportKindSelector.
func (s statelessExportKindSelector) ExportKindFor(desc *sdkapi.Descriptor, kind aggregation.Kind) ExportKind {
if kind == aggregation.SumKind && desc.InstrumentKind().PrecomputedSum() {
return CumulativeExportKind
}
return DeltaExportKind
}

View File

@ -45,7 +45,7 @@ func getMap(t *testing.T, cont *controller.Controller) map[string]float64 {
require.NoError(t, cont.ForEach( require.NoError(t, cont.ForEach(
func(_ instrumentation.Library, reader export.Reader) error { func(_ instrumentation.Library, reader export.Reader) error {
return reader.ForEach( return reader.ForEach(
export.CumulativeExportKindSelector(), aggregation.CumulativeTemporalitySelector(),
func(record export.Record) error { func(record export.Record) error {
return out.AddRecord(record) return out.AddRecord(record)
}, },
@ -115,7 +115,7 @@ func TestControllerUsesResource(t *testing.T) {
} }
for _, c := range cases { for _, c := range cases {
t.Run(fmt.Sprintf("case-%s", c.name), func(t *testing.T) { t.Run(fmt.Sprintf("case-%s", c.name), func(t *testing.T) {
sel := export.CumulativeExportKindSelector() sel := aggregation.CumulativeTemporalitySelector()
exp := processortest.New(sel, attribute.DefaultEncoder()) exp := processortest.New(sel, attribute.DefaultEncoder())
cont := controller.New( cont := controller.New(
processor.NewFactory( processor.NewFactory(
@ -145,7 +145,7 @@ func TestStartNoExporter(t *testing.T) {
cont := controller.New( cont := controller.New(
processor.NewFactory( processor.NewFactory(
processortest.AggregatorSelector(), processortest.AggregatorSelector(),
export.CumulativeExportKindSelector(), aggregation.CumulativeTemporalitySelector(),
), ),
controller.WithCollectPeriod(time.Second), controller.WithCollectPeriod(time.Second),
controller.WithResource(resource.Empty()), controller.WithResource(resource.Empty()),
@ -214,7 +214,7 @@ func TestObserverCanceled(t *testing.T) {
cont := controller.New( cont := controller.New(
processor.NewFactory( processor.NewFactory(
processortest.AggregatorSelector(), processortest.AggregatorSelector(),
export.CumulativeExportKindSelector(), aggregation.CumulativeTemporalitySelector(),
), ),
controller.WithCollectPeriod(0), controller.WithCollectPeriod(0),
controller.WithCollectTimeout(time.Millisecond), controller.WithCollectTimeout(time.Millisecond),
@ -246,7 +246,7 @@ func TestObserverContext(t *testing.T) {
cont := controller.New( cont := controller.New(
processor.NewFactory( processor.NewFactory(
processortest.AggregatorSelector(), processortest.AggregatorSelector(),
export.CumulativeExportKindSelector(), aggregation.CumulativeTemporalitySelector(),
), ),
controller.WithCollectTimeout(0), controller.WithCollectTimeout(0),
controller.WithResource(resource.Empty()), controller.WithResource(resource.Empty()),
@ -278,7 +278,7 @@ type blockingExporter struct {
func newBlockingExporter() *blockingExporter { func newBlockingExporter() *blockingExporter {
return &blockingExporter{ return &blockingExporter{
exporter: processortest.New( exporter: processortest.New(
export.CumulativeExportKindSelector(), aggregation.CumulativeTemporalitySelector(),
attribute.DefaultEncoder(), attribute.DefaultEncoder(),
), ),
} }
@ -296,11 +296,8 @@ func (b *blockingExporter) Export(ctx context.Context, res *resource.Resource, o
return err return err
} }
func (*blockingExporter) ExportKindFor( func (*blockingExporter) TemporalityFor(*sdkapi.Descriptor, aggregation.Kind) aggregation.Temporality {
*sdkapi.Descriptor, return aggregation.CumulativeTemporality
aggregation.Kind,
) export.ExportKind {
return export.CumulativeExportKind
} }
func TestExportTimeout(t *testing.T) { func TestExportTimeout(t *testing.T) {
@ -308,7 +305,7 @@ func TestExportTimeout(t *testing.T) {
cont := controller.New( cont := controller.New(
processor.NewFactory( processor.NewFactory(
processortest.AggregatorSelector(), processortest.AggregatorSelector(),
export.CumulativeExportKindSelector(), aggregation.CumulativeTemporalitySelector(),
), ),
controller.WithCollectPeriod(time.Second), controller.WithCollectPeriod(time.Second),
controller.WithPushTimeout(time.Millisecond), controller.WithPushTimeout(time.Millisecond),
@ -357,7 +354,7 @@ func TestExportTimeout(t *testing.T) {
func TestCollectAfterStopThenStartAgain(t *testing.T) { func TestCollectAfterStopThenStartAgain(t *testing.T) {
exp := processortest.New( exp := processortest.New(
export.CumulativeExportKindSelector(), aggregation.CumulativeTemporalitySelector(),
attribute.DefaultEncoder(), attribute.DefaultEncoder(),
) )
cont := controller.New( cont := controller.New(
@ -436,7 +433,7 @@ func TestCollectAfterStopThenStartAgain(t *testing.T) {
func TestRegistryFunction(t *testing.T) { func TestRegistryFunction(t *testing.T) {
exp := processortest.New( exp := processortest.New(
export.CumulativeExportKindSelector(), aggregation.CumulativeTemporalitySelector(),
attribute.DefaultEncoder(), attribute.DefaultEncoder(),
) )
cont := controller.New( cont := controller.New(

View File

@ -24,7 +24,7 @@ import (
"go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/attribute"
"go.opentelemetry.io/otel/metric" "go.opentelemetry.io/otel/metric"
export "go.opentelemetry.io/otel/sdk/export/metric" "go.opentelemetry.io/otel/sdk/export/metric/aggregation"
controller "go.opentelemetry.io/otel/sdk/metric/controller/basic" controller "go.opentelemetry.io/otel/sdk/metric/controller/basic"
"go.opentelemetry.io/otel/sdk/metric/controller/controllertest" "go.opentelemetry.io/otel/sdk/metric/controller/controllertest"
processor "go.opentelemetry.io/otel/sdk/metric/processor/basic" processor "go.opentelemetry.io/otel/sdk/metric/processor/basic"
@ -36,7 +36,7 @@ func TestPullNoCollect(t *testing.T) {
puller := controller.New( puller := controller.New(
processor.NewFactory( processor.NewFactory(
processortest.AggregatorSelector(), processortest.AggregatorSelector(),
export.CumulativeExportKindSelector(), aggregation.CumulativeTemporalitySelector(),
processor.WithMemory(true), processor.WithMemory(true),
), ),
controller.WithCollectPeriod(0), controller.WithCollectPeriod(0),
@ -51,7 +51,7 @@ func TestPullNoCollect(t *testing.T) {
require.NoError(t, puller.Collect(ctx)) require.NoError(t, puller.Collect(ctx))
records := processortest.NewOutput(attribute.DefaultEncoder()) records := processortest.NewOutput(attribute.DefaultEncoder())
require.NoError(t, controllertest.ReadAll(puller, export.CumulativeExportKindSelector(), records.AddInstrumentationLibraryRecord)) require.NoError(t, controllertest.ReadAll(puller, aggregation.CumulativeTemporalitySelector(), records.AddInstrumentationLibraryRecord))
require.EqualValues(t, map[string]float64{ require.EqualValues(t, map[string]float64{
"counter.sum/A=B/": 10, "counter.sum/A=B/": 10,
@ -61,7 +61,7 @@ func TestPullNoCollect(t *testing.T) {
require.NoError(t, puller.Collect(ctx)) require.NoError(t, puller.Collect(ctx))
records = processortest.NewOutput(attribute.DefaultEncoder()) records = processortest.NewOutput(attribute.DefaultEncoder())
require.NoError(t, controllertest.ReadAll(puller, export.CumulativeExportKindSelector(), records.AddInstrumentationLibraryRecord)) require.NoError(t, controllertest.ReadAll(puller, aggregation.CumulativeTemporalitySelector(), records.AddInstrumentationLibraryRecord))
require.EqualValues(t, map[string]float64{ require.EqualValues(t, map[string]float64{
"counter.sum/A=B/": 20, "counter.sum/A=B/": 20,
@ -72,7 +72,7 @@ func TestPullWithCollect(t *testing.T) {
puller := controller.New( puller := controller.New(
processor.NewFactory( processor.NewFactory(
processortest.AggregatorSelector(), processortest.AggregatorSelector(),
export.CumulativeExportKindSelector(), aggregation.CumulativeTemporalitySelector(),
processor.WithMemory(true), processor.WithMemory(true),
), ),
controller.WithCollectPeriod(time.Second), controller.WithCollectPeriod(time.Second),
@ -89,7 +89,7 @@ func TestPullWithCollect(t *testing.T) {
require.NoError(t, puller.Collect(ctx)) require.NoError(t, puller.Collect(ctx))
records := processortest.NewOutput(attribute.DefaultEncoder()) records := processortest.NewOutput(attribute.DefaultEncoder())
require.NoError(t, controllertest.ReadAll(puller, export.CumulativeExportKindSelector(), records.AddInstrumentationLibraryRecord)) require.NoError(t, controllertest.ReadAll(puller, aggregation.CumulativeTemporalitySelector(), records.AddInstrumentationLibraryRecord))
require.EqualValues(t, map[string]float64{ require.EqualValues(t, map[string]float64{
"counter.sum/A=B/": 10, "counter.sum/A=B/": 10,
@ -100,7 +100,7 @@ func TestPullWithCollect(t *testing.T) {
// Cached value! // Cached value!
require.NoError(t, puller.Collect(ctx)) require.NoError(t, puller.Collect(ctx))
records = processortest.NewOutput(attribute.DefaultEncoder()) records = processortest.NewOutput(attribute.DefaultEncoder())
require.NoError(t, controllertest.ReadAll(puller, export.CumulativeExportKindSelector(), records.AddInstrumentationLibraryRecord)) require.NoError(t, controllertest.ReadAll(puller, aggregation.CumulativeTemporalitySelector(), records.AddInstrumentationLibraryRecord))
require.EqualValues(t, map[string]float64{ require.EqualValues(t, map[string]float64{
"counter.sum/A=B/": 10, "counter.sum/A=B/": 10,
@ -112,7 +112,7 @@ func TestPullWithCollect(t *testing.T) {
// Re-computed value! // Re-computed value!
require.NoError(t, puller.Collect(ctx)) require.NoError(t, puller.Collect(ctx))
records = processortest.NewOutput(attribute.DefaultEncoder()) records = processortest.NewOutput(attribute.DefaultEncoder())
require.NoError(t, controllertest.ReadAll(puller, export.CumulativeExportKindSelector(), records.AddInstrumentationLibraryRecord)) require.NoError(t, controllertest.ReadAll(puller, aggregation.CumulativeTemporalitySelector(), records.AddInstrumentationLibraryRecord))
require.EqualValues(t, map[string]float64{ require.EqualValues(t, map[string]float64{
"counter.sum/A=B/": 20, "counter.sum/A=B/": 20,

View File

@ -67,7 +67,7 @@ func init() {
func newExporter() *processortest.Exporter { func newExporter() *processortest.Exporter {
return processortest.New( return processortest.New(
export.StatelessExportKindSelector(), aggregation.StatelessTemporalitySelector(),
attribute.DefaultEncoder(), attribute.DefaultEncoder(),
) )
} }

View File

@ -20,6 +20,7 @@ import (
"github.com/benbjohnson/clock" "github.com/benbjohnson/clock"
export "go.opentelemetry.io/otel/sdk/export/metric" export "go.opentelemetry.io/otel/sdk/export/metric"
"go.opentelemetry.io/otel/sdk/export/metric/aggregation"
"go.opentelemetry.io/otel/sdk/instrumentation" "go.opentelemetry.io/otel/sdk/instrumentation"
controllerTime "go.opentelemetry.io/otel/sdk/metric/controller/time" controllerTime "go.opentelemetry.io/otel/sdk/metric/controller/time"
) )
@ -64,7 +65,7 @@ func (t MockTicker) C() <-chan time.Time {
// metric). // metric).
func ReadAll( func ReadAll(
reader export.InstrumentationLibraryReader, reader export.InstrumentationLibraryReader,
kind export.ExportKindSelector, kind aggregation.TemporalitySelector,
apply func(instrumentation.Library, export.Record) error, apply func(instrumentation.Library, export.Record) error,
) error { ) error {
return reader.ForEach(func(library instrumentation.Library, reader export.Reader) error { return reader.ForEach(func(library instrumentation.Library, reader export.Reader) error {

View File

@ -28,7 +28,7 @@ import (
type ( type (
Processor struct { Processor struct {
export.ExportKindSelector aggregation.TemporalitySelector
export.AggregatorSelector export.AggregatorSelector
state state
@ -118,32 +118,32 @@ var _ export.Reader = &state{}
// ErrInconsistentState is returned when the sequence of collection's starts and finishes are incorrectly balanced. // ErrInconsistentState is returned when the sequence of collection's starts and finishes are incorrectly balanced.
var ErrInconsistentState = fmt.Errorf("inconsistent processor state") var ErrInconsistentState = fmt.Errorf("inconsistent processor state")
// ErrInvalidExportKind is returned for unknown metric.ExportKind. // ErrInvalidTemporality is returned for unknown metric.Temporality.
var ErrInvalidExportKind = fmt.Errorf("invalid export kind") var ErrInvalidTemporality = fmt.Errorf("invalid aggregation temporality")
// New returns a basic Processor that is also a Checkpointer using the provided // New returns a basic Processor that is also a Checkpointer using the provided
// AggregatorSelector to select Aggregators. The ExportKindSelector // AggregatorSelector to select Aggregators. The TemporalitySelector
// is consulted to determine the kind(s) of exporter that will consume // is consulted to determine the kind(s) of exporter that will consume
// data, so that this Processor can prepare to compute Delta or // data, so that this Processor can prepare to compute Delta or
// Cumulative Aggregations as needed. // Cumulative Aggregations as needed.
func New(aselector export.AggregatorSelector, eselector export.ExportKindSelector, opts ...Option) *Processor { func New(aselector export.AggregatorSelector, tselector aggregation.TemporalitySelector, opts ...Option) *Processor {
return NewFactory(aselector, eselector, opts...).NewCheckpointer().(*Processor) return NewFactory(aselector, tselector, opts...).NewCheckpointer().(*Processor)
} }
type factory struct { type factory struct {
aselector export.AggregatorSelector aselector export.AggregatorSelector
eselector export.ExportKindSelector tselector aggregation.TemporalitySelector
config config config config
} }
func NewFactory(aselector export.AggregatorSelector, eselector export.ExportKindSelector, opts ...Option) export.CheckpointerFactory { func NewFactory(aselector export.AggregatorSelector, tselector aggregation.TemporalitySelector, opts ...Option) export.CheckpointerFactory {
var config config var config config
for _, opt := range opts { for _, opt := range opts {
opt.applyProcessor(&config) opt.applyProcessor(&config)
} }
return factory{ return factory{
aselector: aselector, aselector: aselector,
eselector: eselector, tselector: tselector,
config: config, config: config,
} }
} }
@ -153,8 +153,8 @@ var _ export.CheckpointerFactory = factory{}
func (f factory) NewCheckpointer() export.Checkpointer { func (f factory) NewCheckpointer() export.Checkpointer {
now := time.Now() now := time.Now()
p := &Processor{ p := &Processor{
AggregatorSelector: f.aselector, AggregatorSelector: f.aselector,
ExportKindSelector: f.eselector, TemporalitySelector: f.tselector,
state: state{ state: state{
values: map[stateKey]*stateValue{}, values: map[stateKey]*stateValue{},
processStart: now, processStart: now,
@ -181,7 +181,7 @@ func (b *Processor) Process(accum export.Accumulation) error {
// Check if there is an existing value. // Check if there is an existing value.
value, ok := b.state.values[key] value, ok := b.state.values[key]
if !ok { if !ok {
stateful := b.ExportKindFor(desc, agg.Aggregation().Kind()).MemoryRequired(desc.InstrumentKind()) stateful := b.TemporalityFor(desc, agg.Aggregation().Kind()).MemoryRequired(desc.InstrumentKind())
newValue := &stateValue{ newValue := &stateValue{
labels: accum.Labels(), labels: accum.Labels(),
@ -227,7 +227,7 @@ func (b *Processor) Process(accum export.Accumulation) error {
// instrument reports a PrecomputedSum to a DeltaExporter or // instrument reports a PrecomputedSum to a DeltaExporter or
// the reverse, a non-PrecomputedSum instrument with a // the reverse, a non-PrecomputedSum instrument with a
// CumulativeExporter. This logic is encapsulated in // CumulativeExporter. This logic is encapsulated in
// ExportKind.MemoryRequired(InstrumentKind). // Temporality.MemoryRequired(InstrumentKind).
// //
// Case (b) occurs when the variable `sameCollection` is true, // Case (b) occurs when the variable `sameCollection` is true,
// indicating that the stateKey for Accumulation has already // indicating that the stateKey for Accumulation has already
@ -340,7 +340,7 @@ func (b *Processor) FinishCollection() error {
// ForEach iterates through the Reader, passing an // ForEach iterates through the Reader, passing an
// export.Record with the appropriate Cumulative or Delta aggregation // export.Record with the appropriate Cumulative or Delta aggregation
// to an exporter. // to an exporter.
func (b *state) ForEach(exporter export.ExportKindSelector, f func(export.Record) error) error { func (b *state) ForEach(exporter aggregation.TemporalitySelector, f func(export.Record) error) error {
if b.startedCollection != b.finishedCollection { if b.startedCollection != b.finishedCollection {
return ErrInconsistentState return ErrInconsistentState
} }
@ -356,9 +356,9 @@ func (b *state) ForEach(exporter export.ExportKindSelector, f func(export.Record
continue continue
} }
ekind := exporter.ExportKindFor(key.descriptor, value.current.Aggregation().Kind()) aggTemp := exporter.TemporalityFor(key.descriptor, value.current.Aggregation().Kind())
switch ekind { switch aggTemp {
case export.CumulativeExportKind: case aggregation.CumulativeTemporality:
// If stateful, the sum has been computed. If stateless, the // If stateful, the sum has been computed. If stateless, the
// input was already cumulative. Either way, use the checkpointed // input was already cumulative. Either way, use the checkpointed
// value: // value:
@ -369,7 +369,7 @@ func (b *state) ForEach(exporter export.ExportKindSelector, f func(export.Record
} }
start = b.processStart start = b.processStart
case export.DeltaExportKind: case aggregation.DeltaTemporality:
// Precomputed sums are a special case. // Precomputed sums are a special case.
if mkind.PrecomputedSum() { if mkind.PrecomputedSum() {
agg = value.delta.Aggregation() agg = value.delta.Aggregation()
@ -379,7 +379,7 @@ func (b *state) ForEach(exporter export.ExportKindSelector, f func(export.Record
start = b.intervalStart start = b.intervalStart
default: default:
return fmt.Errorf("%v: %w", ekind, ErrInvalidExportKind) return fmt.Errorf("%v: %w", aggTemp, ErrInvalidTemporality)
} }
if err := f(export.NewRecord( if err := f(export.NewRecord(

View File

@ -47,7 +47,7 @@ func requireNotAfter(t *testing.T, t1, t2 time.Time) {
// TestProcessor tests all the non-error paths in this package. // TestProcessor tests all the non-error paths in this package.
func TestProcessor(t *testing.T) { func TestProcessor(t *testing.T) {
type exportCase struct { type exportCase struct {
kind export.ExportKind kind aggregation.Temporality
} }
type instrumentCase struct { type instrumentCase struct {
kind sdkapi.InstrumentKind kind sdkapi.InstrumentKind
@ -60,8 +60,8 @@ func TestProcessor(t *testing.T) {
} }
for _, tc := range []exportCase{ for _, tc := range []exportCase{
{kind: export.CumulativeExportKind}, {kind: aggregation.CumulativeTemporality},
{kind: export.DeltaExportKind}, {kind: aggregation.DeltaTemporality},
} { } {
t.Run(tc.kind.String(), func(t *testing.T) { t.Run(tc.kind.String(), func(t *testing.T) {
for _, ic := range []instrumentCase{ for _, ic := range []instrumentCase{
@ -121,7 +121,7 @@ func updateFor(t *testing.T, desc *sdkapi.Descriptor, selector export.Aggregator
func testProcessor( func testProcessor(
t *testing.T, t *testing.T,
ekind export.ExportKind, aggTemp aggregation.Temporality,
mkind sdkapi.InstrumentKind, mkind sdkapi.InstrumentKind,
nkind number.Kind, nkind number.Kind,
akind aggregation.Kind, akind aggregation.Kind,
@ -134,7 +134,7 @@ func testProcessor(
labs2 := []attribute.KeyValue{attribute.String("L2", "V")} labs2 := []attribute.KeyValue{attribute.String("L2", "V")}
testBody := func(t *testing.T, hasMemory bool, nAccum, nCheckpoint int) { testBody := func(t *testing.T, hasMemory bool, nAccum, nCheckpoint int) {
processor := basic.New(selector, export.ConstantExportKindSelector(ekind), basic.WithMemory(hasMemory)) processor := basic.New(selector, aggregation.ConstantTemporalitySelector(aggTemp), basic.WithMemory(hasMemory))
instSuffix := fmt.Sprint(".", strings.ToLower(akind.String())) instSuffix := fmt.Sprint(".", strings.ToLower(akind.String()))
@ -166,7 +166,7 @@ func testProcessor(
_, canSub := subr.(export.Subtractor) _, canSub := subr.(export.Subtractor)
// Allow unsupported subraction case only when it is called for. // Allow unsupported subraction case only when it is called for.
require.True(t, mkind.PrecomputedSum() && ekind == export.DeltaExportKind && !canSub) require.True(t, mkind.PrecomputedSum() && aggTemp == aggregation.DeltaTemporality && !canSub)
return return
} else if err != nil { } else if err != nil {
t.Fatal("unexpected FinishCollection error: ", err) t.Fatal("unexpected FinishCollection error: ", err)
@ -190,7 +190,7 @@ func testProcessor(
// Test the final checkpoint state. // Test the final checkpoint state.
records1 := processorTest.NewOutput(attribute.DefaultEncoder()) records1 := processorTest.NewOutput(attribute.DefaultEncoder())
err = reader.ForEach(export.ConstantExportKindSelector(ekind), records1.AddRecord) err = reader.ForEach(aggregation.ConstantTemporalitySelector(aggTemp), records1.AddRecord)
// Test for an allowed error: // Test for an allowed error:
if err != nil && err != aggregation.ErrNoSubtraction { if err != nil && err != aggregation.ErrNoSubtraction {
@ -203,7 +203,7 @@ func testProcessor(
// number of Accumulators, unless LastValue aggregation. // number of Accumulators, unless LastValue aggregation.
// If a precomputed sum, we expect cumulative inputs. // If a precomputed sum, we expect cumulative inputs.
if mkind.PrecomputedSum() { if mkind.PrecomputedSum() {
if ekind == export.DeltaExportKind && akind != aggregation.LastValueKind { if aggTemp == aggregation.DeltaTemporality && akind != aggregation.LastValueKind {
multiplier = int64(nAccum) multiplier = int64(nAccum)
} else if akind == aggregation.LastValueKind { } else if akind == aggregation.LastValueKind {
multiplier = cumulativeMultiplier multiplier = cumulativeMultiplier
@ -211,7 +211,7 @@ func testProcessor(
multiplier = cumulativeMultiplier * int64(nAccum) multiplier = cumulativeMultiplier * int64(nAccum)
} }
} else { } else {
if ekind == export.CumulativeExportKind && akind != aggregation.LastValueKind { if aggTemp == aggregation.CumulativeTemporality && akind != aggregation.LastValueKind {
multiplier = cumulativeMultiplier * int64(nAccum) multiplier = cumulativeMultiplier * int64(nAccum)
} else if akind == aggregation.LastValueKind { } else if akind == aggregation.LastValueKind {
multiplier = 1 multiplier = 1
@ -223,7 +223,7 @@ func testProcessor(
// Synchronous accumulate results from multiple accumulators, // Synchronous accumulate results from multiple accumulators,
// use that number as the baseline multiplier. // use that number as the baseline multiplier.
multiplier = int64(nAccum) multiplier = int64(nAccum)
if ekind == export.CumulativeExportKind { if aggTemp == aggregation.CumulativeTemporality {
// If a cumulative exporter, include prior checkpoints. // If a cumulative exporter, include prior checkpoints.
multiplier *= cumulativeMultiplier multiplier *= cumulativeMultiplier
} }
@ -265,8 +265,8 @@ func testProcessor(
type bogusExporter struct{} type bogusExporter struct{}
func (bogusExporter) ExportKindFor(*sdkapi.Descriptor, aggregation.Kind) export.ExportKind { func (bogusExporter) TemporalityFor(*sdkapi.Descriptor, aggregation.Kind) aggregation.Temporality {
return 1000000 return 100
} }
func (bogusExporter) Export(context.Context, export.Reader) error { func (bogusExporter) Export(context.Context, export.Reader) error {
@ -275,39 +275,39 @@ func (bogusExporter) Export(context.Context, export.Reader) error {
func TestBasicInconsistent(t *testing.T) { func TestBasicInconsistent(t *testing.T) {
// Test double-start // Test double-start
b := basic.New(processorTest.AggregatorSelector(), export.StatelessExportKindSelector()) b := basic.New(processorTest.AggregatorSelector(), aggregation.StatelessTemporalitySelector())
b.StartCollection() b.StartCollection()
b.StartCollection() b.StartCollection()
require.Equal(t, basic.ErrInconsistentState, b.FinishCollection()) require.Equal(t, basic.ErrInconsistentState, b.FinishCollection())
// Test finish without start // Test finish without start
b = basic.New(processorTest.AggregatorSelector(), export.StatelessExportKindSelector()) b = basic.New(processorTest.AggregatorSelector(), aggregation.StatelessTemporalitySelector())
require.Equal(t, basic.ErrInconsistentState, b.FinishCollection()) require.Equal(t, basic.ErrInconsistentState, b.FinishCollection())
// Test no finish // Test no finish
b = basic.New(processorTest.AggregatorSelector(), export.StatelessExportKindSelector()) b = basic.New(processorTest.AggregatorSelector(), aggregation.StatelessTemporalitySelector())
b.StartCollection() b.StartCollection()
require.Equal( require.Equal(
t, t,
basic.ErrInconsistentState, basic.ErrInconsistentState,
b.ForEach( b.ForEach(
export.StatelessExportKindSelector(), aggregation.StatelessTemporalitySelector(),
func(export.Record) error { return nil }, func(export.Record) error { return nil },
), ),
) )
// Test no start // Test no start
b = basic.New(processorTest.AggregatorSelector(), export.StatelessExportKindSelector()) b = basic.New(processorTest.AggregatorSelector(), aggregation.StatelessTemporalitySelector())
desc := metrictest.NewDescriptor("inst", sdkapi.CounterInstrumentKind, number.Int64Kind) desc := metrictest.NewDescriptor("inst", sdkapi.CounterInstrumentKind, number.Int64Kind)
accum := export.NewAccumulation(&desc, attribute.EmptySet(), aggregatortest.NoopAggregator{}) accum := export.NewAccumulation(&desc, attribute.EmptySet(), aggregatortest.NoopAggregator{})
require.Equal(t, basic.ErrInconsistentState, b.Process(accum)) require.Equal(t, basic.ErrInconsistentState, b.Process(accum))
// Test invalid kind: // Test invalid kind:
b = basic.New(processorTest.AggregatorSelector(), export.StatelessExportKindSelector()) b = basic.New(processorTest.AggregatorSelector(), aggregation.StatelessTemporalitySelector())
b.StartCollection() b.StartCollection()
require.NoError(t, b.Process(accum)) require.NoError(t, b.Process(accum))
require.NoError(t, b.FinishCollection()) require.NoError(t, b.FinishCollection())
@ -316,14 +316,14 @@ func TestBasicInconsistent(t *testing.T) {
bogusExporter{}, bogusExporter{},
func(export.Record) error { return nil }, func(export.Record) error { return nil },
) )
require.True(t, errors.Is(err, basic.ErrInvalidExportKind)) require.True(t, errors.Is(err, basic.ErrInvalidTemporality))
} }
func TestBasicTimestamps(t *testing.T) { func TestBasicTimestamps(t *testing.T) {
beforeNew := time.Now() beforeNew := time.Now()
time.Sleep(time.Nanosecond) time.Sleep(time.Nanosecond)
b := basic.New(processorTest.AggregatorSelector(), export.StatelessExportKindSelector()) b := basic.New(processorTest.AggregatorSelector(), aggregation.StatelessTemporalitySelector())
time.Sleep(time.Nanosecond) time.Sleep(time.Nanosecond)
afterNew := time.Now() afterNew := time.Now()
@ -336,7 +336,7 @@ func TestBasicTimestamps(t *testing.T) {
var start1, end1 time.Time var start1, end1 time.Time
require.NoError(t, b.ForEach(export.StatelessExportKindSelector(), func(rec export.Record) error { require.NoError(t, b.ForEach(aggregation.StatelessTemporalitySelector(), func(rec export.Record) error {
start1 = rec.StartTime() start1 = rec.StartTime()
end1 = rec.EndTime() end1 = rec.EndTime()
return nil return nil
@ -353,7 +353,7 @@ func TestBasicTimestamps(t *testing.T) {
var start2, end2 time.Time var start2, end2 time.Time
require.NoError(t, b.ForEach(export.StatelessExportKindSelector(), func(rec export.Record) error { require.NoError(t, b.ForEach(aggregation.StatelessTemporalitySelector(), func(rec export.Record) error {
start2 = rec.StartTime() start2 = rec.StartTime()
end2 = rec.EndTime() end2 = rec.EndTime()
return nil return nil
@ -370,12 +370,12 @@ func TestBasicTimestamps(t *testing.T) {
} }
func TestStatefulNoMemoryCumulative(t *testing.T) { func TestStatefulNoMemoryCumulative(t *testing.T) {
ekindSel := export.CumulativeExportKindSelector() aggTempSel := aggregation.CumulativeTemporalitySelector()
desc := metrictest.NewDescriptor("inst.sum", sdkapi.CounterInstrumentKind, number.Int64Kind) desc := metrictest.NewDescriptor("inst.sum", sdkapi.CounterInstrumentKind, number.Int64Kind)
selector := processorTest.AggregatorSelector() selector := processorTest.AggregatorSelector()
processor := basic.New(selector, ekindSel, basic.WithMemory(false)) processor := basic.New(selector, aggTempSel, basic.WithMemory(false))
reader := processor.Reader() reader := processor.Reader()
for i := 1; i < 3; i++ { for i := 1; i < 3; i++ {
@ -385,7 +385,7 @@ func TestStatefulNoMemoryCumulative(t *testing.T) {
// Verify zero elements // Verify zero elements
records := processorTest.NewOutput(attribute.DefaultEncoder()) records := processorTest.NewOutput(attribute.DefaultEncoder())
require.NoError(t, reader.ForEach(ekindSel, records.AddRecord)) require.NoError(t, reader.ForEach(aggTempSel, records.AddRecord))
require.EqualValues(t, map[string]float64{}, records.Map()) require.EqualValues(t, map[string]float64{}, records.Map())
// Add 10 // Add 10
@ -395,7 +395,7 @@ func TestStatefulNoMemoryCumulative(t *testing.T) {
// Verify one element // Verify one element
records = processorTest.NewOutput(attribute.DefaultEncoder()) records = processorTest.NewOutput(attribute.DefaultEncoder())
require.NoError(t, reader.ForEach(ekindSel, records.AddRecord)) require.NoError(t, reader.ForEach(aggTempSel, records.AddRecord))
require.EqualValues(t, map[string]float64{ require.EqualValues(t, map[string]float64{
"inst.sum/A=B/": float64(i * 10), "inst.sum/A=B/": float64(i * 10),
}, records.Map()) }, records.Map())
@ -403,12 +403,12 @@ func TestStatefulNoMemoryCumulative(t *testing.T) {
} }
func TestStatefulNoMemoryDelta(t *testing.T) { func TestStatefulNoMemoryDelta(t *testing.T) {
ekindSel := export.DeltaExportKindSelector() aggTempSel := aggregation.DeltaTemporalitySelector()
desc := metrictest.NewDescriptor("inst.sum", sdkapi.CounterObserverInstrumentKind, number.Int64Kind) desc := metrictest.NewDescriptor("inst.sum", sdkapi.CounterObserverInstrumentKind, number.Int64Kind)
selector := processorTest.AggregatorSelector() selector := processorTest.AggregatorSelector()
processor := basic.New(selector, ekindSel, basic.WithMemory(false)) processor := basic.New(selector, aggTempSel, basic.WithMemory(false))
reader := processor.Reader() reader := processor.Reader()
for i := 1; i < 3; i++ { for i := 1; i < 3; i++ {
@ -418,7 +418,7 @@ func TestStatefulNoMemoryDelta(t *testing.T) {
// Verify zero elements // Verify zero elements
records := processorTest.NewOutput(attribute.DefaultEncoder()) records := processorTest.NewOutput(attribute.DefaultEncoder())
require.NoError(t, reader.ForEach(ekindSel, records.AddRecord)) require.NoError(t, reader.ForEach(aggTempSel, records.AddRecord))
require.EqualValues(t, map[string]float64{}, records.Map()) require.EqualValues(t, map[string]float64{}, records.Map())
// Add 10 // Add 10
@ -428,7 +428,7 @@ func TestStatefulNoMemoryDelta(t *testing.T) {
// Verify one element // Verify one element
records = processorTest.NewOutput(attribute.DefaultEncoder()) records = processorTest.NewOutput(attribute.DefaultEncoder())
require.NoError(t, reader.ForEach(ekindSel, records.AddRecord)) require.NoError(t, reader.ForEach(aggTempSel, records.AddRecord))
require.EqualValues(t, map[string]float64{ require.EqualValues(t, map[string]float64{
"inst.sum/A=B/": 10, "inst.sum/A=B/": 10,
}, records.Map()) }, records.Map())
@ -436,15 +436,15 @@ func TestStatefulNoMemoryDelta(t *testing.T) {
} }
func TestMultiObserverSum(t *testing.T) { func TestMultiObserverSum(t *testing.T) {
for _, ekindSel := range []export.ExportKindSelector{ for _, aggTempSel := range []aggregation.TemporalitySelector{
export.CumulativeExportKindSelector(), aggregation.CumulativeTemporalitySelector(),
export.DeltaExportKindSelector(), aggregation.DeltaTemporalitySelector(),
} { } {
desc := metrictest.NewDescriptor("observe.sum", sdkapi.CounterObserverInstrumentKind, number.Int64Kind) desc := metrictest.NewDescriptor("observe.sum", sdkapi.CounterObserverInstrumentKind, number.Int64Kind)
selector := processorTest.AggregatorSelector() selector := processorTest.AggregatorSelector()
processor := basic.New(selector, ekindSel, basic.WithMemory(false)) processor := basic.New(selector, aggTempSel, basic.WithMemory(false))
reader := processor.Reader() reader := processor.Reader()
for i := 1; i < 3; i++ { for i := 1; i < 3; i++ {
@ -457,13 +457,13 @@ func TestMultiObserverSum(t *testing.T) {
// Multiplier is 1 for deltas, otherwise i. // Multiplier is 1 for deltas, otherwise i.
multiplier := i multiplier := i
if ekindSel.ExportKindFor(&desc, aggregation.SumKind) == export.DeltaExportKind { if aggTempSel.TemporalityFor(&desc, aggregation.SumKind) == aggregation.DeltaTemporality {
multiplier = 1 multiplier = 1
} }
// Verify one element // Verify one element
records := processorTest.NewOutput(attribute.DefaultEncoder()) records := processorTest.NewOutput(attribute.DefaultEncoder())
require.NoError(t, reader.ForEach(ekindSel, records.AddRecord)) require.NoError(t, reader.ForEach(aggTempSel, records.AddRecord))
require.EqualValues(t, map[string]float64{ require.EqualValues(t, map[string]float64{
"observe.sum/A=B/": float64(3 * 10 * multiplier), "observe.sum/A=B/": float64(3 * 10 * multiplier),
}, records.Map()) }, records.Map())
@ -473,7 +473,7 @@ func TestMultiObserverSum(t *testing.T) {
func TestCounterObserverEndToEnd(t *testing.T) { func TestCounterObserverEndToEnd(t *testing.T) {
ctx := context.Background() ctx := context.Background()
eselector := export.CumulativeExportKindSelector() eselector := aggregation.CumulativeTemporalitySelector()
proc := basic.New( proc := basic.New(
processorTest.AggregatorSelector(), processorTest.AggregatorSelector(),
eselector, eselector,

View File

@ -82,7 +82,7 @@ type (
// Exporter is a testing implementation of export.Exporter that // Exporter is a testing implementation of export.Exporter that
// assembles its results as a map[string]float64. // assembles its results as a map[string]float64.
Exporter struct { Exporter struct {
export.ExportKindSelector aggregation.TemporalitySelector
output *Output output *Output
exportCount int exportCount int
@ -230,7 +230,7 @@ func NewOutput(labelEncoder attribute.Encoder) *Output {
} }
// ForEach implements export.Reader. // ForEach implements export.Reader.
func (o *Output) ForEach(_ export.ExportKindSelector, ff func(export.Record) error) error { func (o *Output) ForEach(_ aggregation.TemporalitySelector, ff func(export.Record) error) error {
for key, value := range o.m { for key, value := range o.m {
if err := ff(export.NewRecord( if err := ff(export.NewRecord(
key.desc, key.desc,
@ -281,7 +281,7 @@ func (o *Output) AddRecordWithResource(rec export.Record, res *resource.Resource
// is chosen, whichever is implemented by the underlying Aggregator. // is chosen, whichever is implemented by the underlying Aggregator.
func (o *Output) Map() map[string]float64 { func (o *Output) Map() map[string]float64 {
r := make(map[string]float64) r := make(map[string]float64)
err := o.ForEach(export.StatelessExportKindSelector(), func(record export.Record) error { err := o.ForEach(aggregation.StatelessTemporalitySelector(), func(record export.Record) error {
for key, entry := range o.m { for key, entry := range o.m {
encoded := entry.labels.Encoded(o.labelEncoder) encoded := entry.labels.Encoded(o.labelEncoder)
rencoded := entry.resource.Encoded(o.labelEncoder) rencoded := entry.resource.Encoded(o.labelEncoder)
@ -344,10 +344,10 @@ func (o *Output) AddAccumulation(acc export.Accumulation) error {
// //
// Where in the example A=1,B=2 is the encoded labels and R=V is the // Where in the example A=1,B=2 is the encoded labels and R=V is the
// encoded resource value. // encoded resource value.
func New(selector export.ExportKindSelector, encoder attribute.Encoder) *Exporter { func New(selector aggregation.TemporalitySelector, encoder attribute.Encoder) *Exporter {
return &Exporter{ return &Exporter{
ExportKindSelector: selector, TemporalitySelector: selector,
output: NewOutput(encoder), output: NewOutput(encoder),
} }
} }
@ -356,7 +356,7 @@ func (e *Exporter) Export(_ context.Context, res *resource.Resource, ckpt export
defer e.output.Unlock() defer e.output.Unlock()
e.exportCount++ e.exportCount++
return ckpt.ForEach(func(library instrumentation.Library, mr export.Reader) error { return ckpt.ForEach(func(library instrumentation.Library, mr export.Reader) error {
return mr.ForEach(e.ExportKindSelector, func(r export.Record) error { return mr.ForEach(e.TemporalitySelector, func(r export.Record) error {
if e.InjectErr != nil { if e.InjectErr != nil {
if err := e.InjectErr(r); err != nil { if err := e.InjectErr(r); err != nil {
return err return err
@ -433,7 +433,7 @@ type metricReader struct {
var _ export.Reader = &metricReader{} var _ export.Reader = &metricReader{}
func (m *metricReader) ForEach(_ export.ExportKindSelector, fn func(export.Record) error) error { func (m *metricReader) ForEach(_ aggregation.TemporalitySelector, fn func(export.Record) error) error {
for _, record := range m.records { for _, record := range m.records {
if err := fn(record); err != nil && err != aggregation.ErrNoData { if err := fn(record); err != nil && err != aggregation.ErrNoData {
return err return err

View File

@ -23,6 +23,7 @@ import (
"go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/attribute"
"go.opentelemetry.io/otel/metric" "go.opentelemetry.io/otel/metric"
export "go.opentelemetry.io/otel/sdk/export/metric" export "go.opentelemetry.io/otel/sdk/export/metric"
"go.opentelemetry.io/otel/sdk/export/metric/aggregation"
"go.opentelemetry.io/otel/sdk/instrumentation" "go.opentelemetry.io/otel/sdk/instrumentation"
metricsdk "go.opentelemetry.io/otel/sdk/metric" metricsdk "go.opentelemetry.io/otel/sdk/metric"
"go.opentelemetry.io/otel/sdk/metric/processor/processortest" "go.opentelemetry.io/otel/sdk/metric/processor/processortest"
@ -71,7 +72,7 @@ func TestProcessorTesting(t *testing.T) {
// Export the data and validate it again. // Export the data and validate it again.
exporter := processorTest.New( exporter := processorTest.New(
export.StatelessExportKindSelector(), aggregation.StatelessTemporalitySelector(),
attribute.DefaultEncoder(), attribute.DefaultEncoder(),
) )

View File

@ -23,7 +23,7 @@ import (
"go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/attribute"
"go.opentelemetry.io/otel/metric" "go.opentelemetry.io/otel/metric"
"go.opentelemetry.io/otel/metric/sdkapi" "go.opentelemetry.io/otel/metric/sdkapi"
export "go.opentelemetry.io/otel/sdk/export/metric" "go.opentelemetry.io/otel/sdk/export/metric/aggregation"
"go.opentelemetry.io/otel/sdk/instrumentation" "go.opentelemetry.io/otel/sdk/instrumentation"
metricsdk "go.opentelemetry.io/otel/sdk/metric" metricsdk "go.opentelemetry.io/otel/sdk/metric"
"go.opentelemetry.io/otel/sdk/metric/processor/basic" "go.opentelemetry.io/otel/sdk/metric/processor/basic"
@ -91,7 +91,7 @@ func TestFilterProcessor(t *testing.T) {
// Test a filter with the ../basic Processor. // Test a filter with the ../basic Processor.
func TestFilterBasicProcessor(t *testing.T) { func TestFilterBasicProcessor(t *testing.T) {
basicProc := basic.New(processorTest.AggregatorSelector(), export.CumulativeExportKindSelector()) basicProc := basic.New(processorTest.AggregatorSelector(), aggregation.CumulativeTemporalitySelector())
accum := metricsdk.NewAccumulator( accum := metricsdk.NewAccumulator(
reducer.New(testFilter{}, basicProc), reducer.New(testFilter{}, basicProc),
) )