mirror of
https://github.com/status-im/consul.git
synced 2025-01-23 20:19:29 +00:00
091925bcb7
* Move hcp client to subpackage hcpclient (#16800) * [HCP Observability] New MetricsClient (#17100) * Client configured with TLS using HCP config and retry/throttle * Add tests and godoc for metrics client * close body after request * run go mod tidy * Remove one abstraction to use the config from deps * Address PR feedback * remove clone * Extract CloudConfig and mock for future PR * Switch to hclog.FromContext * [HCP Observability] OTELExporter (#17128) * Client configured with TLS using HCP config and retry/throttle * run go mod tidy * Remove one abstraction to use the config from deps * Address PR feedback * Client configured with TLS using HCP config and retry/throttle * run go mod tidy * Create new OTELExporter which uses the MetricsClient Add transform because the conversion is in an /internal package * Fix lint error * early return when there are no metrics * Add NewOTELExporter() function * Downgrade to metrics SDK version: v1.15.0-rc.1 * Fix imports * fix small nits with comments and url.URL * Fix tests by asserting actual error for context cancellation, fix parallel, and make mock more versatile * Cleanup error handling and clarify empty metrics case * Fix input/expected naming in otel_transform_test.go * add comment for metric tracking * Add a general isEmpty method * Add clear error types * update to latest version 1.15.0 of OTEL * [HCP Observability] OTELSink (#17159) * Client configured with TLS using HCP config and retry/throttle * run go mod tidy * Remove one abstraction to use the config from deps * Address PR feedback * Client configured with TLS using HCP config and retry/throttle * run go mod tidy * Create new OTELExporter which uses the MetricsClient Add transform because the conversion is in an /internal package * Fix lint error * early return when there are no metrics * Add NewOTELExporter() function * Downgrade to metrics SDK version: v1.15.0-rc.1 * Fix imports * fix small nits with comments and url.URL * Fix tests by asserting actual error for context cancellation, fix parallel, and make mock more versatile * Cleanup error handling and clarify empty metrics case * Fix input/expected naming in otel_transform_test.go * add comment for metric tracking * Add a general isEmpty method * Add clear error types * update to latest version 1.15.0 of OTEL * Client configured with TLS using HCP config and retry/throttle * run go mod tidy * Remove one abstraction to use the config from deps * Address PR feedback * Initialize OTELSink with sync.Map for all the instrument stores. * Moved PeriodicReader init to NewOtelReader function. This allows us to use a ManualReader for tests. * Switch to mutex instead of sync.Map to avoid type assertion * Add gauge store * Clarify comments * return concrete sink type * Fix lint errors * Move gauge store to be within sink * Use context.TODO,rebase and clenaup opts handling * Rebase onto otl exporter to downgrade metrics API to v1.15.0-rc.1 * Fix imports * Update to latest stable version by rebasing on cc-4933, fix import, remove mutex init, fix opts error messages and use logger from ctx * Add lots of documentation to the OTELSink * Fix gauge store comment and check ok * Add select and ctx.Done() check to gauge callback * use require.Equal for attributes * Fixed import naming * Remove float64 calls and add a NewGaugeStore method * Change name Store to Set in gaugeStore, add concurrency tests in both OTELSink and gauge store * Generate 100 gauge operations * Seperate the labels into goroutines in sink test * Generate kv store for the test case keys to avoid using uuid * Added a race test with 300 samples for OTELSink * Do not pass in waitgroup and use error channel instead. * Using SHA 7dea2225a218872e86d2f580e82c089b321617b0 to avoid build failures in otel * Fix nits * [HCP Observability] Init OTELSink in Telemetry (#17162) * Move hcp client to subpackage hcpclient (#16800) * [HCP Observability] New MetricsClient (#17100) * Client configured with TLS using HCP config and retry/throttle * Add tests and godoc for metrics client * close body after request * run go mod tidy * Remove one abstraction to use the config from deps * Address PR feedback * remove clone * Extract CloudConfig and mock for future PR * Switch to hclog.FromContext * [HCP Observability] New MetricsClient (#17100) * Client configured with TLS using HCP config and retry/throttle * Add tests and godoc for metrics client * close body after request * run go mod tidy * Remove one abstraction to use the config from deps * Address PR feedback * remove clone * Extract CloudConfig and mock for future PR * Switch to hclog.FromContext * [HCP Observability] New MetricsClient (#17100) * Client configured with TLS using HCP config and retry/throttle * Add tests and godoc for metrics client * close body after request * run go mod tidy * Remove one abstraction to use the config from deps * Address PR feedback * remove clone * Extract CloudConfig and mock for future PR * Switch to hclog.FromContext * Client configured with TLS using HCP config and retry/throttle * run go mod tidy * Remove one abstraction to use the config from deps * Address PR feedback * Client configured with TLS using HCP config and retry/throttle * run go mod tidy * Create new OTELExporter which uses the MetricsClient Add transform because the conversion is in an /internal package * Fix lint error * early return when there are no metrics * Add NewOTELExporter() function * Downgrade to metrics SDK version: v1.15.0-rc.1 * Fix imports * fix small nits with comments and url.URL * Fix tests by asserting actual error for context cancellation, fix parallel, and make mock more versatile * Cleanup error handling and clarify empty metrics case * Fix input/expected naming in otel_transform_test.go * add comment for metric tracking * Add a general isEmpty method * Add clear error types * update to latest version 1.15.0 of OTEL * Client configured with TLS using HCP config and retry/throttle * run go mod tidy * Remove one abstraction to use the config from deps * Address PR feedback * Initialize OTELSink with sync.Map for all the instrument stores. * Moved PeriodicReader init to NewOtelReader function. This allows us to use a ManualReader for tests. * Switch to mutex instead of sync.Map to avoid type assertion * Add gauge store * Clarify comments * return concrete sink type * Fix lint errors * Move gauge store to be within sink * Use context.TODO,rebase and clenaup opts handling * Rebase onto otl exporter to downgrade metrics API to v1.15.0-rc.1 * Fix imports * Update to latest stable version by rebasing on cc-4933, fix import, remove mutex init, fix opts error messages and use logger from ctx * Add lots of documentation to the OTELSink * Fix gauge store comment and check ok * Add select and ctx.Done() check to gauge callback * use require.Equal for attributes * Fixed import naming * Remove float64 calls and add a NewGaugeStore method * Change name Store to Set in gaugeStore, add concurrency tests in both OTELSink and gauge store * Generate 100 gauge operations * Seperate the labels into goroutines in sink test * Generate kv store for the test case keys to avoid using uuid * Added a race test with 300 samples for OTELSink * [HCP Observability] OTELExporter (#17128) * Client configured with TLS using HCP config and retry/throttle * run go mod tidy * Remove one abstraction to use the config from deps * Address PR feedback * Client configured with TLS using HCP config and retry/throttle * run go mod tidy * Create new OTELExporter which uses the MetricsClient Add transform because the conversion is in an /internal package * Fix lint error * early return when there are no metrics * Add NewOTELExporter() function * Downgrade to metrics SDK version: v1.15.0-rc.1 * Fix imports * fix small nits with comments and url.URL * Fix tests by asserting actual error for context cancellation, fix parallel, and make mock more versatile * Cleanup error handling and clarify empty metrics case * Fix input/expected naming in otel_transform_test.go * add comment for metric tracking * Add a general isEmpty method * Add clear error types * update to latest version 1.15.0 of OTEL * Do not pass in waitgroup and use error channel instead. * Using SHA 7dea2225a218872e86d2f580e82c089b321617b0 to avoid build failures in otel * Rebase onto otl exporter to downgrade metrics API to v1.15.0-rc.1 * Initialize OTELSink with sync.Map for all the instrument stores. * Added telemetry agent to client and init sink in deps * Fixed client * Initalize sink in deps * init sink in telemetry library * Init deps before telemetry * Use concrete telemetry.OtelSink type * add /v1/metrics * Avoid returning err for telemetry init * move sink init within the IsCloudEnabled() * Use HCPSinkOpts in deps instead * update golden test for configuration file * Switch to using extra sinks in the telemetry library * keep name MetricsConfig * fix log in verifyCCMRegistration * Set logger in context * pass around MetricSink in deps * Fix imports * Rebased onto otel sink pr * Fix URL in test * [HCP Observability] OTELSink (#17159) * Client configured with TLS using HCP config and retry/throttle * run go mod tidy * Remove one abstraction to use the config from deps * Address PR feedback * Client configured with TLS using HCP config and retry/throttle * run go mod tidy * Create new OTELExporter which uses the MetricsClient Add transform because the conversion is in an /internal package * Fix lint error * early return when there are no metrics * Add NewOTELExporter() function * Downgrade to metrics SDK version: v1.15.0-rc.1 * Fix imports * fix small nits with comments and url.URL * Fix tests by asserting actual error for context cancellation, fix parallel, and make mock more versatile * Cleanup error handling and clarify empty metrics case * Fix input/expected naming in otel_transform_test.go * add comment for metric tracking * Add a general isEmpty method * Add clear error types * update to latest version 1.15.0 of OTEL * Client configured with TLS using HCP config and retry/throttle * run go mod tidy * Remove one abstraction to use the config from deps * Address PR feedback * Initialize OTELSink with sync.Map for all the instrument stores. * Moved PeriodicReader init to NewOtelReader function. This allows us to use a ManualReader for tests. * Switch to mutex instead of sync.Map to avoid type assertion * Add gauge store * Clarify comments * return concrete sink type * Fix lint errors * Move gauge store to be within sink * Use context.TODO,rebase and clenaup opts handling * Rebase onto otl exporter to downgrade metrics API to v1.15.0-rc.1 * Fix imports * Update to latest stable version by rebasing on cc-4933, fix import, remove mutex init, fix opts error messages and use logger from ctx * Add lots of documentation to the OTELSink * Fix gauge store comment and check ok * Add select and ctx.Done() check to gauge callback * use require.Equal for attributes * Fixed import naming * Remove float64 calls and add a NewGaugeStore method * Change name Store to Set in gaugeStore, add concurrency tests in both OTELSink and gauge store * Generate 100 gauge operations * Seperate the labels into goroutines in sink test * Generate kv store for the test case keys to avoid using uuid * Added a race test with 300 samples for OTELSink * Do not pass in waitgroup and use error channel instead. * Using SHA 7dea2225a218872e86d2f580e82c089b321617b0 to avoid build failures in otel * Fix nits * pass extraSinks as function param instead * Add default interval as package export * remove verifyCCM func * Add clusterID * Fix import and add t.Parallel() for missing tests * Kick Vercel CI * Remove scheme from endpoint path, and fix error logging * return metrics.MetricSink for sink method * Update SDK * [HCP Observability] Metrics filtering and Labels in Go Metrics sink (#17184) * Move hcp client to subpackage hcpclient (#16800) * [HCP Observability] New MetricsClient (#17100) * Client configured with TLS using HCP config and retry/throttle * Add tests and godoc for metrics client * close body after request * run go mod tidy * Remove one abstraction to use the config from deps * Address PR feedback * remove clone * Extract CloudConfig and mock for future PR * Switch to hclog.FromContext * [HCP Observability] New MetricsClient (#17100) * Client configured with TLS using HCP config and retry/throttle * Add tests and godoc for metrics client * close body after request * run go mod tidy * Remove one abstraction to use the config from deps * Address PR feedback * remove clone * Extract CloudConfig and mock for future PR * Switch to hclog.FromContext * [HCP Observability] New MetricsClient (#17100) * Client configured with TLS using HCP config and retry/throttle * Add tests and godoc for metrics client * close body after request * run go mod tidy * Remove one abstraction to use the config from deps * Address PR feedback * remove clone * Extract CloudConfig and mock for future PR * Switch to hclog.FromContext * Client configured with TLS using HCP config and retry/throttle * run go mod tidy * Remove one abstraction to use the config from deps * Address PR feedback * Client configured with TLS using HCP config and retry/throttle * run go mod tidy * Create new OTELExporter which uses the MetricsClient Add transform because the conversion is in an /internal package * Fix lint error * early return when there are no metrics * Add NewOTELExporter() function * Downgrade to metrics SDK version: v1.15.0-rc.1 * Fix imports * fix small nits with comments and url.URL * Fix tests by asserting actual error for context cancellation, fix parallel, and make mock more versatile * Cleanup error handling and clarify empty metrics case * Fix input/expected naming in otel_transform_test.go * add comment for metric tracking * Add a general isEmpty method * Add clear error types * update to latest version 1.15.0 of OTEL * Client configured with TLS using HCP config and retry/throttle * run go mod tidy * Remove one abstraction to use the config from deps * Address PR feedback * Initialize OTELSink with sync.Map for all the instrument stores. * Moved PeriodicReader init to NewOtelReader function. This allows us to use a ManualReader for tests. * Switch to mutex instead of sync.Map to avoid type assertion * Add gauge store * Clarify comments * return concrete sink type * Fix lint errors * Move gauge store to be within sink * Use context.TODO,rebase and clenaup opts handling * Rebase onto otl exporter to downgrade metrics API to v1.15.0-rc.1 * Fix imports * Update to latest stable version by rebasing on cc-4933, fix import, remove mutex init, fix opts error messages and use logger from ctx * Add lots of documentation to the OTELSink * Fix gauge store comment and check ok * Add select and ctx.Done() check to gauge callback * use require.Equal for attributes * Fixed import naming * Remove float64 calls and add a NewGaugeStore method * Change name Store to Set in gaugeStore, add concurrency tests in both OTELSink and gauge store * Generate 100 gauge operations * Seperate the labels into goroutines in sink test * Generate kv store for the test case keys to avoid using uuid * Added a race test with 300 samples for OTELSink * [HCP Observability] OTELExporter (#17128) * Client configured with TLS using HCP config and retry/throttle * run go mod tidy * Remove one abstraction to use the config from deps * Address PR feedback * Client configured with TLS using HCP config and retry/throttle * run go mod tidy * Create new OTELExporter which uses the MetricsClient Add transform because the conversion is in an /internal package * Fix lint error * early return when there are no metrics * Add NewOTELExporter() function * Downgrade to metrics SDK version: v1.15.0-rc.1 * Fix imports * fix small nits with comments and url.URL * Fix tests by asserting actual error for context cancellation, fix parallel, and make mock more versatile * Cleanup error handling and clarify empty metrics case * Fix input/expected naming in otel_transform_test.go * add comment for metric tracking * Add a general isEmpty method * Add clear error types * update to latest version 1.15.0 of OTEL * Do not pass in waitgroup and use error channel instead. * Using SHA 7dea2225a218872e86d2f580e82c089b321617b0 to avoid build failures in otel * Rebase onto otl exporter to downgrade metrics API to v1.15.0-rc.1 * Initialize OTELSink with sync.Map for all the instrument stores. * Added telemetry agent to client and init sink in deps * Fixed client * Initalize sink in deps * init sink in telemetry library * Init deps before telemetry * Use concrete telemetry.OtelSink type * add /v1/metrics * Avoid returning err for telemetry init * move sink init within the IsCloudEnabled() * Use HCPSinkOpts in deps instead * update golden test for configuration file * Switch to using extra sinks in the telemetry library * keep name MetricsConfig * fix log in verifyCCMRegistration * Set logger in context * pass around MetricSink in deps * Fix imports * Rebased onto otel sink pr * Fix URL in test * [HCP Observability] OTELSink (#17159) * Client configured with TLS using HCP config and retry/throttle * run go mod tidy * Remove one abstraction to use the config from deps * Address PR feedback * Client configured with TLS using HCP config and retry/throttle * run go mod tidy * Create new OTELExporter which uses the MetricsClient Add transform because the conversion is in an /internal package * Fix lint error * early return when there are no metrics * Add NewOTELExporter() function * Downgrade to metrics SDK version: v1.15.0-rc.1 * Fix imports * fix small nits with comments and url.URL * Fix tests by asserting actual error for context cancellation, fix parallel, and make mock more versatile * Cleanup error handling and clarify empty metrics case * Fix input/expected naming in otel_transform_test.go * add comment for metric tracking * Add a general isEmpty method * Add clear error types * update to latest version 1.15.0 of OTEL * Client configured with TLS using HCP config and retry/throttle * run go mod tidy * Remove one abstraction to use the config from deps * Address PR feedback * Initialize OTELSink with sync.Map for all the instrument stores. * Moved PeriodicReader init to NewOtelReader function. This allows us to use a ManualReader for tests. * Switch to mutex instead of sync.Map to avoid type assertion * Add gauge store * Clarify comments * return concrete sink type * Fix lint errors * Move gauge store to be within sink * Use context.TODO,rebase and clenaup opts handling * Rebase onto otl exporter to downgrade metrics API to v1.15.0-rc.1 * Fix imports * Update to latest stable version by rebasing on cc-4933, fix import, remove mutex init, fix opts error messages and use logger from ctx * Add lots of documentation to the OTELSink * Fix gauge store comment and check ok * Add select and ctx.Done() check to gauge callback * use require.Equal for attributes * Fixed import naming * Remove float64 calls and add a NewGaugeStore method * Change name Store to Set in gaugeStore, add concurrency tests in both OTELSink and gauge store * Generate 100 gauge operations * Seperate the labels into goroutines in sink test * Generate kv store for the test case keys to avoid using uuid * Added a race test with 300 samples for OTELSink * Do not pass in waitgroup and use error channel instead. * Using SHA 7dea2225a218872e86d2f580e82c089b321617b0 to avoid build failures in otel * Fix nits * pass extraSinks as function param instead * Add default interval as package export * remove verifyCCM func * Add clusterID * Fix import and add t.Parallel() for missing tests * Kick Vercel CI * Remove scheme from endpoint path, and fix error logging * return metrics.MetricSink for sink method * Update SDK * Added telemetry agent to client and init sink in deps * Add node_id and __replica__ default labels * add function for default labels and set x-hcp-resource-id * Fix labels tests * Commit suggestion for getDefaultLabels Co-authored-by: Joshua Timmons <joshua.timmons1@gmail.com> * Fixed server.id, and t.Parallel() * Make defaultLabels a method on the TelemetryConfig object * Rename FilterList to lowercase filterList * Cleanup filter implemetation by combining regex into a single one, and making the type lowercase * Fix append * use regex directly for filters * Fix x-resource-id test to use mocked value * Fix log.Error formats * Forgot the len(opts.Label) optimization) * Use cfg.NodeID instead --------- Co-authored-by: Joshua Timmons <joshua.timmons1@gmail.com> * remove replic tag (#17484) * [HCP Observability] Add custom metrics for OTEL sink, improve logging, upgrade modules and cleanup metrics client (#17455) * Add custom metrics for Exporter and transform operations * Improve deps logging Run go mod tidy * Upgrade SDK and OTEL * Remove the partial success implemetation and check for HTTP status code in metrics client * Add x-channel * cleanup logs in deps.go based on PR feedback * Change to debug log and lowercase * address test operation feedback * use GetHumanVersion on version * Fix error wrapping * Fix metric names * [HCP Observability] Turn off retries for now until dynamically configurable (#17496) * Remove retries for now until dynamic configuration is possible * Clarify comment * Update changelog * improve changelog --------- Co-authored-by: Joshua Timmons <joshua.timmons1@gmail.com>
246 lines
8.1 KiB
Go
246 lines
8.1 KiB
Go
package telemetry
|
|
|
|
import (
|
|
"bytes"
|
|
"context"
|
|
"fmt"
|
|
"net/url"
|
|
"regexp"
|
|
"strings"
|
|
"sync"
|
|
"time"
|
|
|
|
gometrics "github.com/armon/go-metrics"
|
|
"github.com/hashicorp/go-hclog"
|
|
"go.opentelemetry.io/otel/attribute"
|
|
otelmetric "go.opentelemetry.io/otel/metric"
|
|
otelsdk "go.opentelemetry.io/otel/sdk/metric"
|
|
"go.opentelemetry.io/otel/sdk/resource"
|
|
|
|
"github.com/hashicorp/consul/agent/hcp/client"
|
|
)
|
|
|
|
// DefaultExportInterval is a default time interval between export of aggregated metrics.
|
|
const DefaultExportInterval = 10 * time.Second
|
|
|
|
// OTELSinkOpts is used to provide configuration when initializing an OTELSink using NewOTELSink.
|
|
type OTELSinkOpts struct {
|
|
Reader otelsdk.Reader
|
|
Ctx context.Context
|
|
Filters []string
|
|
Labels map[string]string
|
|
}
|
|
|
|
// OTELSink captures and aggregates telemetry data as per the OpenTelemetry (OTEL) specification.
|
|
// Metric data is exported in OpenTelemetry Protocol (OTLP) wire format.
|
|
// This should be used as a Go Metrics backend, as it implements the MetricsSink interface.
|
|
type OTELSink struct {
|
|
// spaceReplacer cleans the flattened key by removing any spaces.
|
|
spaceReplacer *strings.Replacer
|
|
logger hclog.Logger
|
|
filters *regexp.Regexp
|
|
|
|
// meterProvider is an OTEL MeterProvider, the entrypoint to the OTEL Metrics SDK.
|
|
// It handles reading/export of aggregated metric data.
|
|
// It enables creation and usage of an OTEL Meter.
|
|
meterProvider *otelsdk.MeterProvider
|
|
|
|
// meter is an OTEL Meter, which enables the creation of OTEL instruments.
|
|
meter *otelmetric.Meter
|
|
|
|
// Instrument stores contain an OTEL Instrument per metric name (<name, instrument>)
|
|
// for each gauge, counter and histogram types.
|
|
// An instrument allows us to record a measurement for a particular metric, and continuously aggregates metrics.
|
|
// We lazy load the creation of these intruments until a metric is seen, and use them repeatedly to record measurements.
|
|
gaugeInstruments map[string]otelmetric.Float64ObservableGauge
|
|
counterInstruments map[string]otelmetric.Float64Counter
|
|
histogramInstruments map[string]otelmetric.Float64Histogram
|
|
|
|
// gaugeStore is required to hold last-seen values of gauges
|
|
// This is a workaround, as OTEL currently does not have synchronous gauge instruments.
|
|
// It only allows the registration of "callbacks", which obtain values when the callback is called.
|
|
// We must hold gauge values until the callback is called, when the measurement is exported, and can be removed.
|
|
gaugeStore *gaugeStore
|
|
|
|
mutex sync.Mutex
|
|
}
|
|
|
|
// NewOTELReader returns a configured OTEL PeriodicReader to export metrics every X seconds.
|
|
// It configures the reader with a custom OTELExporter with a MetricsClient to transform and export
|
|
// metrics in OTLP format to an external url.
|
|
func NewOTELReader(client client.MetricsClient, url *url.URL, exportInterval time.Duration) otelsdk.Reader {
|
|
exporter := NewOTELExporter(client, url)
|
|
return otelsdk.NewPeriodicReader(exporter, otelsdk.WithInterval(exportInterval))
|
|
}
|
|
|
|
// NewOTELSink returns a sink which fits the Go Metrics MetricsSink interface.
|
|
// It sets up a MeterProvider and Meter, key pieces of the OTEL Metrics SDK which
|
|
// enable us to create OTEL Instruments to record measurements.
|
|
func NewOTELSink(opts *OTELSinkOpts) (*OTELSink, error) {
|
|
if opts.Reader == nil {
|
|
return nil, fmt.Errorf("ferror: provide valid reader")
|
|
}
|
|
|
|
if opts.Ctx == nil {
|
|
return nil, fmt.Errorf("ferror: provide valid context")
|
|
}
|
|
|
|
logger := hclog.FromContext(opts.Ctx).Named("otel_sink")
|
|
|
|
filterList, err := newFilterRegex(opts.Filters)
|
|
if err != nil {
|
|
logger.Error("Failed to initialize all filters", "error", err)
|
|
}
|
|
|
|
attrs := make([]attribute.KeyValue, 0, len(opts.Labels))
|
|
for k, v := range opts.Labels {
|
|
kv := attribute.KeyValue{
|
|
Key: attribute.Key(k),
|
|
Value: attribute.StringValue(v),
|
|
}
|
|
attrs = append(attrs, kv)
|
|
}
|
|
// Setup OTEL Metrics SDK to aggregate, convert and export metrics periodically.
|
|
res := resource.NewWithAttributes("", attrs...)
|
|
meterProvider := otelsdk.NewMeterProvider(otelsdk.WithResource(res), otelsdk.WithReader(opts.Reader))
|
|
meter := meterProvider.Meter("github.com/hashicorp/consul/agent/hcp/telemetry")
|
|
|
|
return &OTELSink{
|
|
filters: filterList,
|
|
spaceReplacer: strings.NewReplacer(" ", "_"),
|
|
logger: logger,
|
|
meterProvider: meterProvider,
|
|
meter: &meter,
|
|
gaugeStore: NewGaugeStore(),
|
|
gaugeInstruments: make(map[string]otelmetric.Float64ObservableGauge, 0),
|
|
counterInstruments: make(map[string]otelmetric.Float64Counter, 0),
|
|
histogramInstruments: make(map[string]otelmetric.Float64Histogram, 0),
|
|
}, nil
|
|
}
|
|
|
|
// SetGauge emits a Consul gauge metric.
|
|
func (o *OTELSink) SetGauge(key []string, val float32) {
|
|
o.SetGaugeWithLabels(key, val, nil)
|
|
}
|
|
|
|
// AddSample emits a Consul histogram metric.
|
|
func (o *OTELSink) AddSample(key []string, val float32) {
|
|
o.AddSampleWithLabels(key, val, nil)
|
|
}
|
|
|
|
// IncrCounter emits a Consul counter metric.
|
|
func (o *OTELSink) IncrCounter(key []string, val float32) {
|
|
o.IncrCounterWithLabels(key, val, nil)
|
|
}
|
|
|
|
// AddSampleWithLabels emits a Consul gauge metric that gets
|
|
// registed by an OpenTelemetry Histogram instrument.
|
|
func (o *OTELSink) SetGaugeWithLabels(key []string, val float32, labels []gometrics.Label) {
|
|
k := o.flattenKey(key)
|
|
|
|
if !o.filters.MatchString(k) {
|
|
return
|
|
}
|
|
|
|
// Set value in global Gauge store.
|
|
o.gaugeStore.Set(k, float64(val), toAttributes(labels))
|
|
|
|
o.mutex.Lock()
|
|
defer o.mutex.Unlock()
|
|
|
|
// If instrument does not exist, create it and register callback to emit last value in global Gauge store.
|
|
if _, ok := o.gaugeInstruments[k]; !ok {
|
|
// The registration of a callback only needs to happen once, when the instrument is created.
|
|
// The callback will be triggered every export cycle for that metric.
|
|
// It must be explicitly de-registered to be removed (which we do not do), to ensure new gauge values are exported every cycle.
|
|
inst, err := (*o.meter).Float64ObservableGauge(k, otelmetric.WithFloat64Callback(o.gaugeStore.gaugeCallback(k)))
|
|
if err != nil {
|
|
o.logger.Error("Failed to create gauge instrument", "error", err)
|
|
return
|
|
}
|
|
o.gaugeInstruments[k] = inst
|
|
}
|
|
}
|
|
|
|
// AddSampleWithLabels emits a Consul sample metric that gets registed by an OpenTelemetry Histogram instrument.
|
|
func (o *OTELSink) AddSampleWithLabels(key []string, val float32, labels []gometrics.Label) {
|
|
k := o.flattenKey(key)
|
|
|
|
if !o.filters.MatchString(k) {
|
|
return
|
|
}
|
|
|
|
o.mutex.Lock()
|
|
defer o.mutex.Unlock()
|
|
|
|
inst, ok := o.histogramInstruments[k]
|
|
if !ok {
|
|
histogram, err := (*o.meter).Float64Histogram(k)
|
|
if err != nil {
|
|
o.logger.Error("Failed create histogram instrument", "error", err)
|
|
return
|
|
}
|
|
inst = histogram
|
|
o.histogramInstruments[k] = inst
|
|
}
|
|
|
|
attrs := toAttributes(labels)
|
|
inst.Record(context.TODO(), float64(val), otelmetric.WithAttributes(attrs...))
|
|
}
|
|
|
|
// IncrCounterWithLabels emits a Consul counter metric that gets registed by an OpenTelemetry Histogram instrument.
|
|
func (o *OTELSink) IncrCounterWithLabels(key []string, val float32, labels []gometrics.Label) {
|
|
k := o.flattenKey(key)
|
|
|
|
if !o.filters.MatchString(k) {
|
|
return
|
|
}
|
|
|
|
o.mutex.Lock()
|
|
defer o.mutex.Unlock()
|
|
|
|
inst, ok := o.counterInstruments[k]
|
|
if !ok {
|
|
counter, err := (*o.meter).Float64Counter(k)
|
|
if err != nil {
|
|
o.logger.Error("Failed to create counter instrument:", "error", err)
|
|
return
|
|
}
|
|
|
|
inst = counter
|
|
o.counterInstruments[k] = inst
|
|
}
|
|
|
|
attrs := toAttributes(labels)
|
|
inst.Add(context.TODO(), float64(val), otelmetric.WithAttributes(attrs...))
|
|
}
|
|
|
|
// EmitKey unsupported.
|
|
func (o *OTELSink) EmitKey(key []string, val float32) {}
|
|
|
|
// flattenKey key along with its labels.
|
|
func (o *OTELSink) flattenKey(parts []string) string {
|
|
buf := &bytes.Buffer{}
|
|
joined := strings.Join(parts, ".")
|
|
|
|
o.spaceReplacer.WriteString(buf, joined)
|
|
|
|
return buf.String()
|
|
}
|
|
|
|
// toAttributes converts go metrics Labels into OTEL format []attributes.KeyValue
|
|
func toAttributes(labels []gometrics.Label) []attribute.KeyValue {
|
|
if len(labels) == 0 {
|
|
return nil
|
|
}
|
|
attrs := make([]attribute.KeyValue, len(labels))
|
|
for i, label := range labels {
|
|
attrs[i] = attribute.KeyValue{
|
|
Key: attribute.Key(label.Name),
|
|
Value: attribute.StringValue(label.Value),
|
|
}
|
|
}
|
|
|
|
return attrs
|
|
}
|