2020-03-25 14:47:17 -07:00
|
|
|
// Copyright The OpenTelemetry Authors
|
2019-10-29 13:27:22 -07:00
|
|
|
//
|
|
|
|
// 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.
|
|
|
|
|
Replace `Ordered` with an iterator in `export.Labels`. (#567)
* Do not expose a slice of labels in export.Record
This is really an inconvenient implementation detail leak - we may
want to store labels in a different way. Replace it with an iterator -
it does not force us to use slice of key values as a storage in the
long run.
* Add Len to LabelIterator
It may come in handy in several situations, where we don't have access
to export.Labels object, but only to the label iterator.
* Use reflect value label iterator for the fixed labels
* add reset operation to iterator
Makes my life easier when writing a benchmark. Might also be an
alternative to cloning the iterator.
* Add benchmarks for iterators
* Add import comment
* Add clone operation to label iterator
* Move iterator tests to a separate package
* Add tests for cloning iterators
* Pass label iterator to export labels
* Use non-addressable array reflect values
By not using the value created by `reflect.New()`, but rather by
`reflect.ValueOf()`, we get a non-addressable array in the value,
which does not infer an allocation cost when getting an element from
the array.
* Drop zero iterator
This can be substituted by a reflect value iterator that goes over a
value with a zero-sized array.
* Add a simple iterator that implements label iterator
In the long run this will completely replace the LabelIterator
interface.
* Replace reflect value iterator with simple iterator
* Pass label storage to new export labels, not label iterator
* Drop label iterator interface, rename storage iterator to label iterator
* Drop clone operation from iterator
It's a leftover from interface times and now it's pointless - the
iterator is a simple struct, so cloning it is a simple copy.
* Drop Reset from label iterator
The sole existence of Reset was actually for benchmarking convenience.
Now we can just copy the iterator cheaply, so a need for Reset is no
more.
* Drop noop iterator tests
* Move back iterator tests to export package
* Eagerly get the reflect value of ordered labels
So we won't get into problems when several goroutines want to iterate
the same labels at the same time. Not sure if this would be a big
deal, since every goroutine would compute the same reflect.Value, but
concurrent write to the same memory is bad anyway. And it doesn't cost
us any extra allocations anyway.
* Replace NewSliceLabelIterator() with a method of LabelSlice
* Add some documentation
* Documentation fixes
2020-03-19 23:01:34 +01:00
|
|
|
package metric // import "go.opentelemetry.io/otel/sdk/export/metric"
|
2019-11-15 13:01:20 -08:00
|
|
|
|
2019-10-29 13:27:22 -07:00
|
|
|
import (
|
|
|
|
"context"
|
2020-05-18 18:37:41 -07:00
|
|
|
"sync"
|
2020-06-18 10:16:33 -07:00
|
|
|
"time"
|
2019-10-29 13:27:22 -07:00
|
|
|
|
2021-02-18 12:59:37 -05:00
|
|
|
"go.opentelemetry.io/otel/attribute"
|
2020-11-11 16:24:12 +01:00
|
|
|
"go.opentelemetry.io/otel/metric/number"
|
2021-08-11 16:02:28 -07:00
|
|
|
"go.opentelemetry.io/otel/metric/sdkapi"
|
2020-06-18 10:16:33 -07:00
|
|
|
"go.opentelemetry.io/otel/sdk/export/metric/aggregation"
|
2021-09-27 08:51:47 -07:00
|
|
|
"go.opentelemetry.io/otel/sdk/instrumentation"
|
2020-04-24 09:44:21 -07:00
|
|
|
"go.opentelemetry.io/otel/sdk/resource"
|
2019-10-29 13:27:22 -07:00
|
|
|
)
|
|
|
|
|
2020-06-23 12:00:15 -07:00
|
|
|
// Processor is responsible for deciding which kind of aggregation to
|
2020-06-23 10:51:15 -07:00
|
|
|
// use (via AggregatorSelector), gathering exported results from the
|
2019-11-15 13:01:20 -08:00
|
|
|
// SDK during collection, and deciding over which dimensions to group
|
|
|
|
// the exported data.
|
|
|
|
//
|
|
|
|
// The SDK supports binding only one of these interfaces, as it has
|
|
|
|
// the sole responsibility of determining which Aggregator to use for
|
|
|
|
// each record.
|
|
|
|
//
|
2020-06-23 10:51:15 -07:00
|
|
|
// The embedded AggregatorSelector interface is called (concurrently)
|
2019-11-15 13:01:20 -08:00
|
|
|
// in instrumentation context to select the appropriate Aggregator for
|
|
|
|
// an instrument.
|
|
|
|
//
|
|
|
|
// The `Process` method is called during collection in a
|
|
|
|
// single-threaded context from the SDK, after the aggregator is
|
2020-06-23 12:00:15 -07:00
|
|
|
// checkpointed, allowing the processor to build the set of metrics
|
2019-11-15 13:01:20 -08:00
|
|
|
// currently being exported.
|
2020-06-23 12:00:15 -07:00
|
|
|
type Processor interface {
|
2020-06-23 10:51:15 -07:00
|
|
|
// AggregatorSelector is responsible for selecting the
|
2019-11-15 13:01:20 -08:00
|
|
|
// concrete type of Aggregator used for a metric in the SDK.
|
|
|
|
//
|
|
|
|
// This may be a static decision based on fields of the
|
|
|
|
// Descriptor, or it could use an external configuration
|
|
|
|
// source to customize the treatment of each metric
|
|
|
|
// instrument.
|
|
|
|
//
|
|
|
|
// The result from AggregatorSelector.AggregatorFor should be
|
|
|
|
// the same type for a given Descriptor or else nil. The same
|
|
|
|
// type should be returned for a given descriptor, because
|
|
|
|
// Aggregators only know how to Merge with their own type. If
|
|
|
|
// the result is nil, the metric instrument will be disabled.
|
|
|
|
//
|
|
|
|
// Note that the SDK only calls AggregatorFor when new records
|
|
|
|
// require an Aggregator. This does not provide a way to
|
|
|
|
// disable metrics with active records.
|
2020-06-23 10:51:15 -07:00
|
|
|
AggregatorSelector
|
2019-11-15 13:01:20 -08:00
|
|
|
|
|
|
|
// Process is called by the SDK once per internal record,
|
2020-06-18 10:16:33 -07:00
|
|
|
// passing the export Accumulation (a Descriptor, the corresponding
|
2021-06-08 19:10:01 +02:00
|
|
|
// Labels, and the checkpointed Aggregator). This call has no
|
2020-06-09 11:00:50 -07:00
|
|
|
// Context argument because it is expected to perform only
|
2021-06-08 19:10:01 +02:00
|
|
|
// computation. An SDK is not expected to call exporters from
|
2020-06-09 11:00:50 -07:00
|
|
|
// with Process, use a controller for that (see
|
|
|
|
// ./controllers/{pull,push}.
|
2020-09-16 18:09:45 +02:00
|
|
|
Process(accum Accumulation) error
|
2019-11-05 13:08:55 -08:00
|
|
|
}
|
|
|
|
|
2020-06-23 10:51:15 -07:00
|
|
|
// AggregatorSelector supports selecting the kind of Aggregator to
|
2019-11-15 13:01:20 -08:00
|
|
|
// use at runtime for a specific metric instrument.
|
2020-06-23 10:51:15 -07:00
|
|
|
type AggregatorSelector interface {
|
2020-06-13 00:55:01 -07:00
|
|
|
// AggregatorFor allocates a variable number of aggregators of
|
|
|
|
// a kind suitable for the requested export. This method
|
|
|
|
// initializes a `...*Aggregator`, to support making a single
|
|
|
|
// allocation.
|
|
|
|
//
|
|
|
|
// When the call returns without initializing the *Aggregator
|
|
|
|
// to a non-nil value, the metric instrument is explicitly
|
|
|
|
// disabled.
|
|
|
|
//
|
|
|
|
// This must return a consistent type to avoid confusion in
|
|
|
|
// later stages of the metrics export process, i.e., when
|
|
|
|
// Merging or Checkpointing aggregators for a specific
|
|
|
|
// instrument.
|
2019-11-15 13:01:20 -08:00
|
|
|
//
|
|
|
|
// Note: This is context-free because the aggregator should
|
|
|
|
// not relate to the incoming context. This call should not
|
|
|
|
// block.
|
2021-10-14 09:06:22 -07:00
|
|
|
AggregatorFor(descriptor *sdkapi.Descriptor, aggregator ...*Aggregator)
|
2019-11-15 13:01:20 -08:00
|
|
|
}
|
|
|
|
|
2020-08-13 13:12:32 -07:00
|
|
|
// Checkpointer is the interface used by a Controller to coordinate
|
|
|
|
// the Processor with Accumulator(s) and Exporter(s). The
|
|
|
|
// StartCollection() and FinishCollection() methods start and finish a
|
|
|
|
// collection interval. Controllers call the Accumulator(s) during
|
|
|
|
// collection to process Accumulations.
|
|
|
|
type Checkpointer interface {
|
|
|
|
// Processor processes metric data for export. The Process
|
|
|
|
// method is bracketed by StartCollection and FinishCollection
|
|
|
|
// calls. The embedded AggregatorSelector can be called at
|
|
|
|
// any time.
|
|
|
|
Processor
|
|
|
|
|
2021-09-27 08:51:47 -07:00
|
|
|
// Reader returns the current data set. This may be
|
2020-08-13 13:12:32 -07:00
|
|
|
// called before and after collection. The
|
|
|
|
// implementation is required to return the same value
|
2021-09-27 08:51:47 -07:00
|
|
|
// throughout its lifetime, since Reader exposes a
|
2020-08-13 13:12:32 -07:00
|
|
|
// sync.Locker interface. The caller is responsible for
|
2021-09-27 08:51:47 -07:00
|
|
|
// locking the Reader before initiating collection.
|
|
|
|
Reader() Reader
|
2020-08-13 13:12:32 -07:00
|
|
|
|
|
|
|
// StartCollection begins a collection interval.
|
|
|
|
StartCollection()
|
|
|
|
|
|
|
|
// FinishCollection ends a collection interval.
|
|
|
|
FinishCollection() error
|
|
|
|
}
|
|
|
|
|
2021-09-27 08:51:47 -07:00
|
|
|
// CheckpointerFactory is an interface for producing configured
|
|
|
|
// Checkpointer instances.
|
|
|
|
type CheckpointerFactory interface {
|
|
|
|
NewCheckpointer() Checkpointer
|
|
|
|
}
|
|
|
|
|
2019-11-15 13:01:20 -08:00
|
|
|
// Aggregator implements a specific aggregation behavior, e.g., a
|
2021-09-01 13:38:37 -07:00
|
|
|
// behavior to track a sequence of updates to an instrument. Counter
|
2020-05-15 22:11:12 -07:00
|
|
|
// instruments commonly use a simple Sum aggregator, but for the
|
2021-09-01 13:38:37 -07:00
|
|
|
// distribution instruments (Histogram, GaugeObserver) there are a
|
2020-05-15 22:11:12 -07:00
|
|
|
// number of possible aggregators with different cost and accuracy
|
|
|
|
// tradeoffs.
|
2019-11-15 13:01:20 -08:00
|
|
|
//
|
|
|
|
// Note that any Aggregator may be attached to any instrument--this is
|
|
|
|
// the result of the OpenTelemetry API/SDK separation. It is possible
|
2021-09-01 13:38:37 -07:00
|
|
|
// to attach a Sum aggregator to a Histogram instrument or a
|
2020-05-15 22:11:12 -07:00
|
|
|
// MinMaxSumCount aggregator to a Counter instrument.
|
2019-11-05 13:08:55 -08:00
|
|
|
type Aggregator interface {
|
2020-06-18 10:16:33 -07:00
|
|
|
// Aggregation returns an Aggregation interface to access the
|
|
|
|
// current state of this Aggregator. The caller is
|
|
|
|
// responsible for synchronization and must not call any the
|
|
|
|
// other methods in this interface concurrently while using
|
|
|
|
// the Aggregation.
|
|
|
|
Aggregation() aggregation.Aggregation
|
|
|
|
|
2019-10-29 13:27:22 -07:00
|
|
|
// Update receives a new measured value and incorporates it
|
2020-06-13 00:55:01 -07:00
|
|
|
// into the aggregation. Update() calls may be called
|
|
|
|
// concurrently.
|
2019-11-15 13:01:20 -08:00
|
|
|
//
|
|
|
|
// Descriptor.NumberKind() should be consulted to determine
|
|
|
|
// whether the provided number is an int64 or float64.
|
|
|
|
//
|
|
|
|
// The Context argument comes from user-level code and could be
|
2020-06-13 00:55:01 -07:00
|
|
|
// inspected for a `correlation.Map` or `trace.SpanContext`.
|
2021-10-14 09:06:22 -07:00
|
|
|
Update(ctx context.Context, number number.Number, descriptor *sdkapi.Descriptor) error
|
2019-11-15 13:01:20 -08:00
|
|
|
|
2020-06-23 10:41:11 -07:00
|
|
|
// SynchronizedMove is called during collection to finish one
|
2020-06-13 00:55:01 -07:00
|
|
|
// period of aggregation by atomically saving the
|
2020-06-23 10:41:11 -07:00
|
|
|
// currently-updating state into the argument Aggregator AND
|
|
|
|
// resetting the current value to the zero state.
|
2019-11-15 13:01:20 -08:00
|
|
|
//
|
2020-06-23 10:41:11 -07:00
|
|
|
// SynchronizedMove() is called concurrently with Update(). These
|
2020-06-13 00:55:01 -07:00
|
|
|
// two methods must be synchronized with respect to each
|
|
|
|
// other, for correctness.
|
|
|
|
//
|
|
|
|
// After saving a synchronized copy, the Aggregator can be converted
|
|
|
|
// into one or more of the interfaces in the `aggregation` sub-package,
|
|
|
|
// according to kind of Aggregator that was selected.
|
|
|
|
//
|
|
|
|
// This method will return an InconsistentAggregatorError if
|
|
|
|
// this Aggregator cannot be copied into the destination due
|
|
|
|
// to an incompatible type.
|
2019-11-15 13:01:20 -08:00
|
|
|
//
|
2020-06-09 11:00:50 -07:00
|
|
|
// This call has no Context argument because it is expected to
|
|
|
|
// perform only computation.
|
2020-12-10 18:13:08 -08:00
|
|
|
//
|
|
|
|
// When called with a nil `destination`, this Aggregator is reset
|
|
|
|
// and the current value is discarded.
|
2021-10-14 09:06:22 -07:00
|
|
|
SynchronizedMove(destination Aggregator, descriptor *sdkapi.Descriptor) error
|
2019-11-15 13:01:20 -08:00
|
|
|
|
|
|
|
// Merge combines the checkpointed state from the argument
|
2020-06-13 00:55:01 -07:00
|
|
|
// Aggregator into this Aggregator. Merge is not synchronized
|
2020-06-23 10:41:11 -07:00
|
|
|
// with respect to Update or SynchronizedMove.
|
2020-06-13 00:55:01 -07:00
|
|
|
//
|
|
|
|
// The owner of an Aggregator being merged is responsible for
|
|
|
|
// synchronization of both Aggregator states.
|
2021-10-14 09:06:22 -07:00
|
|
|
Merge(aggregator Aggregator, descriptor *sdkapi.Descriptor) error
|
2019-11-15 13:01:20 -08:00
|
|
|
}
|
|
|
|
|
|
|
|
// Exporter handles presentation of the checkpoint of aggregate
|
|
|
|
// metrics. This is the final stage of a metrics export pipeline,
|
|
|
|
// where metric data are formatted for a specific system.
|
|
|
|
type Exporter interface {
|
|
|
|
// Export is called immediately after completing a collection
|
|
|
|
// pass in the SDK.
|
|
|
|
//
|
|
|
|
// The Context comes from the controller that initiated
|
|
|
|
// collection.
|
|
|
|
//
|
2021-09-27 08:51:47 -07:00
|
|
|
// The InstrumentationLibraryReader interface refers to the
|
|
|
|
// Processor that just completed collection.
|
|
|
|
Export(ctx context.Context, resource *resource.Resource, reader InstrumentationLibraryReader) error
|
2020-06-22 22:59:51 -07:00
|
|
|
|
2021-10-15 11:18:36 -07:00
|
|
|
// TemporalitySelector is an interface used by the Processor
|
2020-06-22 22:59:51 -07:00
|
|
|
// in deciding whether to compute Delta or Cumulative
|
|
|
|
// Aggregations when passing Records to this Exporter.
|
2021-10-15 11:18:36 -07:00
|
|
|
aggregation.TemporalitySelector
|
2019-11-15 13:01:20 -08:00
|
|
|
}
|
|
|
|
|
2021-09-27 08:51:47 -07:00
|
|
|
// InstrumentationLibraryReader is an interface for exporters to iterate
|
|
|
|
// over one instrumentation library of metric data at a time.
|
|
|
|
type InstrumentationLibraryReader interface {
|
|
|
|
// ForEach calls the passed function once per instrumentation library,
|
|
|
|
// allowing the caller to emit metrics grouped by the library that
|
|
|
|
// produced them.
|
|
|
|
ForEach(readerFunc func(instrumentation.Library, Reader) error) error
|
|
|
|
}
|
|
|
|
|
|
|
|
// Reader allows a controller to access a complete checkpoint of
|
|
|
|
// aggregated metrics from the Processor for a single library of
|
|
|
|
// metric data. This is passed to the Exporter which may then use
|
|
|
|
// ForEach to iterate over the collection of aggregated metrics.
|
|
|
|
type Reader interface {
|
2019-11-15 13:01:20 -08:00
|
|
|
// ForEach iterates over aggregated checkpoints for all
|
|
|
|
// metrics that were updated during the last collection
|
2020-03-16 16:28:33 -07:00
|
|
|
// period. Each aggregated checkpoint returned by the
|
|
|
|
// function parameter may return an error.
|
2020-06-22 22:59:51 -07:00
|
|
|
//
|
2021-10-15 11:18:36 -07:00
|
|
|
// The TemporalitySelector argument is used to determine
|
2020-06-22 22:59:51 -07:00
|
|
|
// whether the Record is computed using Delta or Cumulative
|
|
|
|
// aggregation.
|
|
|
|
//
|
2020-03-16 16:28:33 -07:00
|
|
|
// ForEach tolerates ErrNoData silently, as this is
|
|
|
|
// expected from the Meter implementation. Any other kind
|
|
|
|
// of error will immediately halt ForEach and return
|
|
|
|
// the error to the caller.
|
2021-10-15 11:18:36 -07:00
|
|
|
ForEach(tempSelector aggregation.TemporalitySelector, recordFunc func(Record) error) error
|
2020-05-18 18:37:41 -07:00
|
|
|
|
|
|
|
// Locker supports locking the checkpoint set. Collection
|
|
|
|
// into the checkpoint set cannot take place (in case of a
|
2020-06-23 12:00:15 -07:00
|
|
|
// stateful processor) while it is locked.
|
2020-05-18 18:37:41 -07:00
|
|
|
//
|
2020-06-23 12:00:15 -07:00
|
|
|
// The Processor attached to the Accumulator MUST be called
|
2020-05-18 18:37:41 -07:00
|
|
|
// with the lock held.
|
|
|
|
sync.Locker
|
|
|
|
|
|
|
|
// RLock acquires a read lock corresponding to this Locker.
|
|
|
|
RLock()
|
|
|
|
// RUnlock releases a read lock corresponding to this Locker.
|
|
|
|
RUnlock()
|
2019-11-15 13:01:20 -08:00
|
|
|
}
|
2019-10-30 22:15:27 -07:00
|
|
|
|
2020-06-18 10:16:33 -07:00
|
|
|
// Metadata contains the common elements for exported metric data that
|
2020-06-23 12:00:15 -07:00
|
|
|
// are shared by the Accumulator->Processor and Processor->Exporter
|
2020-06-18 10:16:33 -07:00
|
|
|
// steps.
|
|
|
|
type Metadata struct {
|
2021-10-14 09:06:22 -07:00
|
|
|
descriptor *sdkapi.Descriptor
|
2021-02-18 12:59:37 -05:00
|
|
|
labels *attribute.Set
|
2020-06-18 10:16:33 -07:00
|
|
|
}
|
|
|
|
|
|
|
|
// Accumulation contains the exported data for a single metric instrument
|
2020-06-23 12:00:15 -07:00
|
|
|
// and label set, as prepared by an Accumulator for the Processor.
|
2020-06-18 10:16:33 -07:00
|
|
|
type Accumulation struct {
|
|
|
|
Metadata
|
2019-11-15 13:01:20 -08:00
|
|
|
aggregator Aggregator
|
2019-10-29 13:27:22 -07:00
|
|
|
}
|
|
|
|
|
2020-06-18 10:16:33 -07:00
|
|
|
// Record contains the exported data for a single metric instrument
|
2020-06-23 12:00:15 -07:00
|
|
|
// and label set, as prepared by the Processor for the Exporter.
|
2020-06-18 10:16:33 -07:00
|
|
|
// This includes the effective start and end time for the aggregation.
|
|
|
|
type Record struct {
|
|
|
|
Metadata
|
|
|
|
aggregation aggregation.Aggregation
|
|
|
|
start time.Time
|
|
|
|
end time.Time
|
|
|
|
}
|
|
|
|
|
|
|
|
// Descriptor describes the metric instrument being exported.
|
2021-10-14 09:06:22 -07:00
|
|
|
func (m Metadata) Descriptor() *sdkapi.Descriptor {
|
2020-06-18 10:16:33 -07:00
|
|
|
return m.descriptor
|
|
|
|
}
|
|
|
|
|
|
|
|
// Labels describes the labels associated with the instrument and the
|
|
|
|
// aggregated data.
|
2021-02-18 12:59:37 -05:00
|
|
|
func (m Metadata) Labels() *attribute.Set {
|
2020-06-18 10:16:33 -07:00
|
|
|
return m.labels
|
|
|
|
}
|
|
|
|
|
|
|
|
// NewAccumulation allows Accumulator implementations to construct new
|
2021-08-12 15:44:58 -07:00
|
|
|
// Accumulations to send to Processors. The Descriptor, Labels,
|
2020-06-18 10:16:33 -07:00
|
|
|
// and Aggregator represent aggregate metric events received over a single
|
|
|
|
// collection period.
|
2021-10-14 09:06:22 -07:00
|
|
|
func NewAccumulation(descriptor *sdkapi.Descriptor, labels *attribute.Set, aggregator Aggregator) Accumulation {
|
2020-06-18 10:16:33 -07:00
|
|
|
return Accumulation{
|
|
|
|
Metadata: Metadata{
|
|
|
|
descriptor: descriptor,
|
|
|
|
labels: labels,
|
|
|
|
},
|
2019-11-15 13:01:20 -08:00
|
|
|
aggregator: aggregator,
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Aggregator returns the checkpointed aggregator. It is safe to
|
|
|
|
// access the checkpointed state without locking.
|
2020-06-18 10:16:33 -07:00
|
|
|
func (r Accumulation) Aggregator() Aggregator {
|
2019-11-15 13:01:20 -08:00
|
|
|
return r.aggregator
|
|
|
|
}
|
|
|
|
|
2020-06-23 12:00:15 -07:00
|
|
|
// NewRecord allows Processor implementations to construct export
|
2020-06-18 10:16:33 -07:00
|
|
|
// records. The Descriptor, Labels, and Aggregator represent
|
|
|
|
// aggregate metric events received over a single collection period.
|
2021-10-14 09:06:22 -07:00
|
|
|
func NewRecord(descriptor *sdkapi.Descriptor, labels *attribute.Set, aggregation aggregation.Aggregation, start, end time.Time) Record {
|
2020-06-18 10:16:33 -07:00
|
|
|
return Record{
|
|
|
|
Metadata: Metadata{
|
|
|
|
descriptor: descriptor,
|
|
|
|
labels: labels,
|
|
|
|
},
|
|
|
|
aggregation: aggregation,
|
|
|
|
start: start,
|
|
|
|
end: end,
|
|
|
|
}
|
2019-11-15 13:01:20 -08:00
|
|
|
}
|
|
|
|
|
2020-06-18 10:16:33 -07:00
|
|
|
// Aggregation returns the aggregation, an interface to the record and
|
|
|
|
// its aggregator, dependent on the kind of both the input and exporter.
|
|
|
|
func (r Record) Aggregation() aggregation.Aggregation {
|
|
|
|
return r.aggregation
|
2019-11-15 13:01:20 -08:00
|
|
|
}
|
2020-05-18 17:44:28 -07:00
|
|
|
|
2020-06-18 10:16:33 -07:00
|
|
|
// StartTime is the start time of the interval covered by this aggregation.
|
|
|
|
func (r Record) StartTime() time.Time {
|
|
|
|
return r.start
|
|
|
|
}
|
|
|
|
|
|
|
|
// EndTime is the end time of the interval covered by this aggregation.
|
|
|
|
func (r Record) EndTime() time.Time {
|
|
|
|
return r.end
|
2020-05-18 17:44:28 -07:00
|
|
|
}
|