From 0d98f6160f22675470d788865d5166ae5c84b45d Mon Sep 17 00:00:00 2001 From: Hyunsoo Kim Date: Fri, 2 Feb 2024 13:43:25 -0500 Subject: [PATCH 01/17] add dcgm exporter scraper and move prometheus scraper test mock to mocks package --- exporter/awsemfexporter/config.go | 3 + .../awscontainerinsightreceiver/config.go | 4 + .../internal/gpu/dcgmscraper.go | 220 +++++++++++++ .../internal/gpu/dcgmscraper_test.go | 302 ++++++++++++++++++ .../k8sapiserver/prometheus_scraper_test.go | 17 +- .../prometheus.go} | 74 ++--- .../awscontainerinsightreceiver/receiver.go | 39 +++ 7 files changed, 614 insertions(+), 45 deletions(-) create mode 100644 receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go create mode 100644 receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go rename receiver/awscontainerinsightreceiver/internal/{k8sapiserver/prometheus_helper_test.go => mocks/prometheus.go} (55%) diff --git a/exporter/awsemfexporter/config.go b/exporter/awsemfexporter/config.go index e42b5532852c..6f12efb3c708 100644 --- a/exporter/awsemfexporter/config.go +++ b/exporter/awsemfexporter/config.go @@ -100,6 +100,9 @@ type Config struct { // MiddlewareID is an ID for an extension that can be used to configure the AWS client. MiddlewareID *component.ID `mapstructure:"middleware,omitempty"` + // EnableGpuMetric indicates payloads will include GPU metrics + EnableGpuMetric bool `mapstructure:"gpu_metrics"` + // logger is the Logger used for writing error/warning logs logger *zap.Logger } diff --git a/receiver/awscontainerinsightreceiver/config.go b/receiver/awscontainerinsightreceiver/config.go index 7ac3b351010c..2e06ba111d33 100644 --- a/receiver/awscontainerinsightreceiver/config.go +++ b/receiver/awscontainerinsightreceiver/config.go @@ -57,4 +57,8 @@ type Config struct { // EnableControlPlaneMetrics enables additional metrics sourced from the Kubernetes API server /metrics prometheus endpoint // The default value is false. EnableControlPlaneMetrics bool `mapstructure:"enable_control_plane_metrics"` + + // EnableGpuMetric disables GPU monitoring where metrics are scraped from vendor specific sources + // The default value is true meaning GPU metrics get collected out of the box unless it's disabled + EnableGpuMetric bool `mapstructure:"gpu_metrics"` } diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go new file mode 100644 index 000000000000..0d9c7b32bc60 --- /dev/null +++ b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go @@ -0,0 +1,220 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +package gpu + +import ( + "context" + "errors" + "fmt" + "time" + + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/config" + "github.com/prometheus/prometheus/discovery" + "github.com/prometheus/prometheus/discovery/kubernetes" + "github.com/prometheus/prometheus/model/relabel" + "go.opentelemetry.io/collector/component" + "go.opentelemetry.io/collector/consumer" + "go.opentelemetry.io/collector/pdata/pmetric" + "go.opentelemetry.io/collector/receiver" + "go.uber.org/zap" + + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/prometheusreceiver" +) + +const ( + caFile = "/var/run/secrets/kubernetes.io/serviceaccount/ca.crt" + collectionInterval = 60 * time.Second + jobName = "containerInsightsDCGMExporterScraper" +) + +type DcgmScraper struct { + ctx context.Context + settings component.TelemetrySettings + host component.Host + hostInfoProvider hostInfoProvider + prometheusReceiver receiver.Metrics + running bool +} + +type DcgmScraperOpts struct { + Ctx context.Context + TelemetrySettings component.TelemetrySettings + Consumer consumer.Metrics + Host component.Host + HostInfoProvider hostInfoProvider + BearerToken string +} + +type hostInfoProvider interface { + GetClusterName() string + GetInstanceID() string +} + +func NewDcgmScraper(opts DcgmScraperOpts) (*DcgmScraper, error) { + if opts.Consumer == nil { + return nil, errors.New("consumer cannot be nil") + } + if opts.Host == nil { + return nil, errors.New("host cannot be nil") + } + if opts.HostInfoProvider == nil { + return nil, errors.New("cluster name provider cannot be nil") + } + + scrapeConfig := &config.ScrapeConfig{ + // TLS needs to be enabled between pods communication + // It can further get restricted by adding authentication mechanism to limit the data + //HTTPClientConfig: configutil.HTTPClientConfig{ + // BasicAuth: &configutil.BasicAuth{ + // Username: "", + // Password: "", + // }, + // Authorization: &configutil.Authorization{ + // Type: "basic_auth", + // }, + // TLSConfig: configutil.TLSConfig{ + // CAFile: caFile, + // InsecureSkipVerify: false, + // }, + //}, + ScrapeInterval: model.Duration(collectionInterval), + ScrapeTimeout: model.Duration(collectionInterval), + JobName: jobName, + Scheme: "http", + MetricsPath: "/metrics", + ServiceDiscoveryConfigs: discovery.Configs{ + &kubernetes.SDConfig{ + Role: kubernetes.RoleEndpoint, + NamespaceDiscovery: kubernetes.NamespaceDiscovery{ + IncludeOwnNamespace: true, + }, + Selectors: []kubernetes.SelectorConfig{ + { + Role: kubernetes.RoleEndpoint, + Label: "k8s-app=dcgm-exporter-service", + }, + }, + AttachMetadata: kubernetes.AttachMetadataConfig{ + Node: true, + }, + }, + }, + RelabelConfigs: []*relabel.Config{ + { + SourceLabels: model.LabelNames{"__address__"}, + Regex: relabel.MustNewRegexp("([^:]+)(?::\\d+)?"), + Replacement: "${1}:9400", + TargetLabel: "__address__", + Action: relabel.Replace, + }, + { + SourceLabels: model.LabelNames{"__meta_kubernetes_pod_node_name"}, + TargetLabel: "NodeName", + Regex: relabel.MustNewRegexp("(.*)"), + Replacement: "$1", + Action: relabel.Replace, + }, + { + SourceLabels: model.LabelNames{"__meta_kubernetes_service_name"}, + TargetLabel: "Service", + Regex: relabel.MustNewRegexp("(.*)"), + Replacement: "$1", + Action: relabel.Replace, + }, + }, + MetricRelabelConfigs: []*relabel.Config{ + { + SourceLabels: model.LabelNames{"__name__"}, + Regex: relabel.MustNewRegexp("DCGM_.*"), + Action: relabel.Keep, + }, + { + SourceLabels: model.LabelNames{"namespace"}, + TargetLabel: "Namespace", + Regex: relabel.MustNewRegexp("(.*)"), + Replacement: "${1}", + Action: relabel.Replace, + }, + // hacky way to inject static values (clusterName & instanceId) to label set without additional processor + // relabel looks up an existing label then creates another label with given key (TargetLabel) and value (static) + { + SourceLabels: model.LabelNames{"namespace"}, + TargetLabel: "ClusterName", + Regex: relabel.MustNewRegexp("(.*)"), + Replacement: opts.HostInfoProvider.GetClusterName(), + Action: relabel.Replace, + }, + { + SourceLabels: model.LabelNames{"namespace"}, + TargetLabel: "InstanceId", + Regex: relabel.MustNewRegexp("(.*)"), + Replacement: opts.HostInfoProvider.GetInstanceID(), + Action: relabel.Replace, + }, + { + SourceLabels: model.LabelNames{"pod"}, + TargetLabel: "FullPodName", + Regex: relabel.MustNewRegexp("(.*)"), + Replacement: "${1}", + Action: relabel.Replace, + }, + { + SourceLabels: model.LabelNames{"pod"}, + TargetLabel: "PodName", + Regex: relabel.MustNewRegexp("(.+)-(.+)"), + Replacement: "${1}", + Action: relabel.Replace, + }, + }, + } + + promConfig := prometheusreceiver.Config{ + PrometheusConfig: &config.Config{ + ScrapeConfigs: []*config.ScrapeConfig{scrapeConfig}, + }, + } + + params := receiver.CreateSettings{ + TelemetrySettings: opts.TelemetrySettings, + } + + promFactory := prometheusreceiver.NewFactory() + promReceiver, err := promFactory.CreateMetricsReceiver(opts.Ctx, params, &promConfig, opts.Consumer) + if err != nil { + return nil, fmt.Errorf("failed to create prometheus receiver: %w", err) + } + + return &DcgmScraper{ + ctx: opts.Ctx, + settings: opts.TelemetrySettings, + host: opts.Host, + hostInfoProvider: opts.HostInfoProvider, + prometheusReceiver: promReceiver, + }, nil +} + +func (ds *DcgmScraper) GetMetrics() []pmetric.Metrics { + // This method will never return metrics because the metrics are collected by the scraper. + // This method will ensure the scraper is running + if !ds.running { + ds.settings.Logger.Info("The scraper is not running, starting up the scraper") + err := ds.prometheusReceiver.Start(ds.ctx, ds.host) + if err != nil { + ds.settings.Logger.Error("Unable to start PrometheusReceiver", zap.Error(err)) + } + ds.running = err == nil + } + return nil +} + +func (ds *DcgmScraper) Shutdown() { + if ds.running { + err := ds.prometheusReceiver.Shutdown(ds.ctx) + if err != nil { + ds.settings.Logger.Error("Unable to shutdown PrometheusReceiver", zap.Error(err)) + } + ds.running = false + } +} diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go new file mode 100644 index 000000000000..7a4cd78da3d7 --- /dev/null +++ b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go @@ -0,0 +1,302 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +package gpu + +import ( + "context" + "fmt" + "strings" + "testing" + + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/mocks" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/prometheusreceiver" + configutil "github.com/prometheus/common/config" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/config" + "github.com/prometheus/prometheus/discovery" + "github.com/prometheus/prometheus/model/relabel" + "github.com/stretchr/testify/assert" + "go.opentelemetry.io/collector/component/componenttest" + "go.opentelemetry.io/collector/consumer" + "go.opentelemetry.io/collector/pdata/pmetric" + "go.opentelemetry.io/collector/receiver" + "go.uber.org/zap" +) + +const renameMetric = ` +# HELP DCGM_FI_DEV_GPU_TEMP GPU temperature (in C). +# TYPE DCGM_FI_DEV_GPU_TEMP gauge +DCGM_FI_DEV_GPU_TEMP{gpu="0",UUID="uuid",device="nvidia0",modelName="NVIDIA A10G",Hostname="hostname",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="fullname-hash"} 65 +# HELP DCGM_FI_DEV_GPU_UTIL GPU utilization (in %). +# TYPE DCGM_FI_DEV_GPU_UTIL gauge +DCGM_FI_DEV_GPU_UTIL{gpu="0",UUID="uuid",device="nvidia0",modelName="NVIDIA A10G",Hostname="hostname",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="fullname-hash"} 100 +` + +const dummyInstanceId = "i-0000000000" + +type mockHostInfoProvider struct { +} + +func (m mockHostInfoProvider) GetClusterName() string { + return "cluster-name" +} + +func (m mockHostInfoProvider) GetInstanceID() string { + return dummyInstanceId +} + +type mockConsumer struct { + t *testing.T + up *bool + gpuTemp *bool + gpuUtil *bool + relabeled *bool + podNameParsed *bool + instanceId *bool +} + +func (m mockConsumer) Capabilities() consumer.Capabilities { + return consumer.Capabilities{ + MutatesData: false, + } +} + +func (m mockConsumer) ConsumeMetrics(_ context.Context, md pmetric.Metrics) error { + assert.Equal(m.t, 1, md.ResourceMetrics().Len()) + + scopeMetrics := md.ResourceMetrics().At(0).ScopeMetrics().At(0).Metrics() + for i := 0; i < scopeMetrics.Len(); i++ { + metric := scopeMetrics.At(i) + if metric.Name() == "DCGM_FI_DEV_GPU_UTIL" { + assert.Equal(m.t, float64(100), metric.Gauge().DataPoints().At(0).DoubleValue()) + *m.gpuUtil = true + instanceId, _ := metric.Gauge().DataPoints().At(0).Attributes().Get("InstanceId") + *m.instanceId = instanceId.Str() == dummyInstanceId + } + if metric.Name() == "DCGM_FI_DEV_GPU_TEMP" { + *m.gpuTemp = true + fullPodName, relabeled := metric.Gauge().DataPoints().At(0).Attributes().Get("FullPodName") + splits := strings.Split(fullPodName.Str(), "-") + podName, _ := metric.Gauge().DataPoints().At(0).Attributes().Get("PodName") + *m.podNameParsed = podName.Str() == splits[0] + *m.relabeled = relabeled + } + if metric.Name() == "up" { + assert.Equal(m.t, float64(1), metric.Gauge().DataPoints().At(0).DoubleValue()) + *m.up = true + } + } + + return nil +} + +func TestNewDcgmScraperBadInputs(t *testing.T) { + settings := componenttest.NewNopTelemetrySettings() + settings.Logger, _ = zap.NewDevelopment() + + tests := []DcgmScraperOpts{ + { + Ctx: context.TODO(), + TelemetrySettings: settings, + Consumer: nil, + Host: componenttest.NewNopHost(), + HostInfoProvider: mockHostInfoProvider{}, + }, + { + Ctx: context.TODO(), + TelemetrySettings: settings, + Consumer: mockConsumer{}, + Host: nil, + HostInfoProvider: mockHostInfoProvider{}, + }, + { + Ctx: context.TODO(), + TelemetrySettings: settings, + Consumer: mockConsumer{}, + Host: componenttest.NewNopHost(), + HostInfoProvider: nil, + }, + } + + for _, tt := range tests { + scraper, err := NewDcgmScraper(tt) + + assert.Error(t, err) + assert.Nil(t, scraper) + } +} + +func TestNewDcgmScraperEndToEnd(t *testing.T) { + + upPtr := false + gpuTemp := false + gpuUtil := false + relabeledPod := false + podNameParsed := false + instanceId := false + + consumer := mockConsumer{ + t: t, + up: &upPtr, + gpuTemp: &gpuTemp, + gpuUtil: &gpuUtil, + relabeled: &relabeledPod, + podNameParsed: &podNameParsed, + instanceId: &instanceId, + } + + settings := componenttest.NewNopTelemetrySettings() + settings.Logger, _ = zap.NewDevelopment() + + scraper, err := NewDcgmScraper(DcgmScraperOpts{ + Ctx: context.TODO(), + TelemetrySettings: settings, + Consumer: mockConsumer{}, + Host: componenttest.NewNopHost(), + HostInfoProvider: mockHostInfoProvider{}, + BearerToken: "", + }) + assert.NoError(t, err) + assert.Equal(t, mockHostInfoProvider{}, scraper.hostInfoProvider) + + // build up a new PR + promFactory := prometheusreceiver.NewFactory() + + targets := []*mocks.TestData{ + { + Name: "dcgm", + Pages: []mocks.MockPrometheusResponse{ + {Code: 200, Data: renameMetric}, + }, + }, + } + mp, cfg, err := mocks.SetupMockPrometheus(targets...) + assert.NoError(t, err) + + split := strings.Split(mp.Srv.URL, "http://") + + scrapeConfig := &config.ScrapeConfig{ + HTTPClientConfig: configutil.HTTPClientConfig{ + TLSConfig: configutil.TLSConfig{ + InsecureSkipVerify: true, + }, + }, + ScrapeInterval: cfg.ScrapeConfigs[0].ScrapeInterval, + ScrapeTimeout: cfg.ScrapeConfigs[0].ScrapeInterval, + JobName: fmt.Sprintf("%s/%s", jobName, cfg.ScrapeConfigs[0].MetricsPath), + HonorTimestamps: true, + Scheme: "http", + MetricsPath: cfg.ScrapeConfigs[0].MetricsPath, + ServiceDiscoveryConfigs: discovery.Configs{ + // using dummy static config to avoid service discovery initialization + &discovery.StaticConfig{ + { + Targets: []model.LabelSet{ + { + model.AddressLabel: model.LabelValue(split[1]), + }, + }, + }, + }, + }, + RelabelConfigs: []*relabel.Config{ + // doesn't seem like there is a good way to unit test relabeling rules https://github.com/prometheus/prometheus/issues/8606 + //{ + // SourceLabels: model.LabelNames{"__address__"}, + // Regex: relabel.MustNewRegexp("([^:]+)(?::\\d+)?"), + // Replacement: "${1}:9400", + // TargetLabel: "__address__", + // Action: relabel.Replace, + //}, + //{ + // SourceLabels: model.LabelNames{"__meta_kubernetes_namespace"}, + // TargetLabel: "Namespace", + // Regex: relabel.MustNewRegexp("(.*)"), + // Replacement: "$1", + // Action: relabel.Replace, + //}, + //{ + // SourceLabels: model.LabelNames{"__meta_kubernetes_pod_name"}, + // TargetLabel: "pod", + // Regex: relabel.MustNewRegexp("(.*)"), + // Replacement: "$1", + // Action: relabel.Replace, + //}, + //{ + // SourceLabels: model.LabelNames{"__meta_kubernetes_pod_node_name"}, + // TargetLabel: "NodeName", + // Regex: relabel.MustNewRegexp("(.*)"), + // Replacement: "$1", + // Action: relabel.Replace, + //}, + }, + MetricRelabelConfigs: []*relabel.Config{ + { + SourceLabels: model.LabelNames{"__name__"}, + Regex: relabel.MustNewRegexp("DCGM_.*"), + Action: relabel.Keep, + }, + // test hack to inject cluster name as label + { + SourceLabels: model.LabelNames{"namespace"}, + TargetLabel: "InstanceId", + Regex: relabel.MustNewRegexp("(.*)"), + Replacement: scraper.hostInfoProvider.GetInstanceID(), + Action: relabel.Replace, + }, + { + SourceLabels: model.LabelNames{"pod"}, + TargetLabel: "FullPodName", + Regex: relabel.MustNewRegexp("(.*)"), + Replacement: "${1}", + Action: relabel.Replace, + }, + { + SourceLabels: model.LabelNames{"pod"}, + TargetLabel: "PodName", + Regex: relabel.MustNewRegexp("(.+)-(.+)"), + Replacement: "${1}", + Action: relabel.Replace, + }, + }, + } + + promConfig := prometheusreceiver.Config{ + PrometheusConfig: &config.Config{ + ScrapeConfigs: []*config.ScrapeConfig{scrapeConfig}, + }, + } + + // replace the prom receiver + params := receiver.CreateSettings{ + TelemetrySettings: scraper.settings, + } + scraper.prometheusReceiver, err = promFactory.CreateMetricsReceiver(scraper.ctx, params, &promConfig, consumer) + assert.NoError(t, err) + assert.NotNil(t, mp) + defer mp.Close() + + // perform a single scrape, this will kick off the scraper process for additional scrapes + scraper.GetMetrics() + + t.Cleanup(func() { + scraper.Shutdown() + }) + + // wait for 2 scrapes, one initiated by us, another by the new scraper process + mp.Wg.Wait() + mp.Wg.Wait() + + assert.True(t, *consumer.up) + assert.True(t, *consumer.gpuTemp) + assert.True(t, *consumer.gpuUtil) + assert.True(t, *consumer.relabeled) + assert.True(t, *consumer.podNameParsed) + assert.True(t, *consumer.instanceId) +} + +func TestDcgmScraperJobName(t *testing.T) { + // needs to start with containerInsights + assert.True(t, strings.HasPrefix(jobName, "containerInsightsDCGMExporterScraper")) +} diff --git a/receiver/awscontainerinsightreceiver/internal/k8sapiserver/prometheus_scraper_test.go b/receiver/awscontainerinsightreceiver/internal/k8sapiserver/prometheus_scraper_test.go index 33dc4ba57071..269341d61d80 100644 --- a/receiver/awscontainerinsightreceiver/internal/k8sapiserver/prometheus_scraper_test.go +++ b/receiver/awscontainerinsightreceiver/internal/k8sapiserver/prometheus_scraper_test.go @@ -9,6 +9,7 @@ import ( "strings" "testing" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/mocks" configutil "github.com/prometheus/common/config" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/config" @@ -177,18 +178,18 @@ func TestNewPrometheusScraperEndToEnd(t *testing.T) { // build up a new PR promFactory := prometheusreceiver.NewFactory() - targets := []*testData{ + targets := []*mocks.TestData{ { - name: "prometheus", - pages: []mockPrometheusResponse{ - {code: 200, data: renameMetric}, + Name: "prometheus", + Pages: []mocks.MockPrometheusResponse{ + {Code: 200, Data: renameMetric}, }, }, } - mp, cfg, err := setupMockPrometheus(targets...) + mp, cfg, err := mocks.SetupMockPrometheus(targets...) assert.NoError(t, err) - split := strings.Split(mp.srv.URL, "http://") + split := strings.Split(mp.Srv.URL, "http://") scrapeConfig := &config.ScrapeConfig{ HTTPClientConfig: configutil.HTTPClientConfig{ @@ -261,8 +262,8 @@ func TestNewPrometheusScraperEndToEnd(t *testing.T) { }) // wait for 2 scrapes, one initiated by us, another by the new scraper process - mp.wg.Wait() - mp.wg.Wait() + mp.Wg.Wait() + mp.Wg.Wait() assert.True(t, *consumer.up) assert.True(t, *consumer.httpConnected) diff --git a/receiver/awscontainerinsightreceiver/internal/k8sapiserver/prometheus_helper_test.go b/receiver/awscontainerinsightreceiver/internal/mocks/prometheus.go similarity index 55% rename from receiver/awscontainerinsightreceiver/internal/k8sapiserver/prometheus_helper_test.go rename to receiver/awscontainerinsightreceiver/internal/mocks/prometheus.go index e18142174e16..d9b90220df72 100644 --- a/receiver/awscontainerinsightreceiver/internal/k8sapiserver/prometheus_helper_test.go +++ b/receiver/awscontainerinsightreceiver/internal/mocks/prometheus.go @@ -1,7 +1,7 @@ // Copyright The OpenTelemetry Authors // SPDX-License-Identifier: Apache-2.0 -package k8sapiserver +package mocks import ( "fmt" @@ -17,68 +17,68 @@ import ( "gopkg.in/yaml.v2" ) -type mockPrometheusResponse struct { - code int - data string - useOpenMetrics bool +type MockPrometheusResponse struct { + Code int + Data string + UseOpenMetrics bool } -type mockPrometheus struct { - mu sync.Mutex // mu protects the fields below. - endpoints map[string][]mockPrometheusResponse - accessIndex map[string]*atomic.Int32 - wg *sync.WaitGroup - srv *httptest.Server +type MockPrometheus struct { + Mu sync.Mutex // mu protects the fields below. + Endpoints map[string][]MockPrometheusResponse + AccessIndex map[string]*atomic.Int32 + Wg *sync.WaitGroup + Srv *httptest.Server } -type testData struct { - name string - pages []mockPrometheusResponse +type TestData struct { + Name string + Pages []MockPrometheusResponse } -func (mp *mockPrometheus) Close() { - mp.srv.Close() +func (mp *MockPrometheus) Close() { + mp.Srv.Close() } -func (mp *mockPrometheus) ServeHTTP(rw http.ResponseWriter, req *http.Request) { - mp.mu.Lock() - defer mp.mu.Unlock() - iptr, ok := mp.accessIndex[req.URL.Path] +func (mp *MockPrometheus) ServeHTTP(rw http.ResponseWriter, req *http.Request) { + mp.Mu.Lock() + defer mp.Mu.Unlock() + iptr, ok := mp.AccessIndex[req.URL.Path] if !ok { rw.WriteHeader(404) return } index := int(iptr.Load()) iptr.Add(1) - pages := mp.endpoints[req.URL.Path] + pages := mp.Endpoints[req.URL.Path] if index >= len(pages) { if index == len(pages) { - mp.wg.Done() + mp.Wg.Done() } rw.WriteHeader(404) return } - if pages[index].useOpenMetrics { + if pages[index].UseOpenMetrics { rw.Header().Set("Content-Type", "application/openmetrics-text") } - rw.WriteHeader(pages[index].code) - _, _ = rw.Write([]byte(pages[index].data)) + rw.WriteHeader(pages[index].Code) + _, _ = rw.Write([]byte(pages[index].Data)) } -func setupMockPrometheus(tds ...*testData) (*mockPrometheus, *promcfg.Config, error) { +func SetupMockPrometheus(tds ...*TestData) (*MockPrometheus, *promcfg.Config, error) { jobs := make([]map[string]any, 0, len(tds)) - endpoints := make(map[string][]mockPrometheusResponse) + endpoints := make(map[string][]MockPrometheusResponse) var metricPaths []string for _, t := range tds { - metricPath := fmt.Sprintf("/%s/metrics", t.name) - endpoints[metricPath] = t.pages + metricPath := fmt.Sprintf("/%s/metrics", t.Name) + endpoints[metricPath] = t.Pages metricPaths = append(metricPaths, metricPath) } mp := newMockPrometheus(endpoints) - u, _ := url.Parse(mp.srv.URL) + u, _ := url.Parse(mp.Srv.URL) for i := 0; i < len(tds); i++ { job := make(map[string]any) - job["job_name"] = tds[i].name + job["job_name"] = tds[i].Name job["metrics_path"] = metricPaths[i] job["scrape_interval"] = "1s" job["scrape_timeout"] = "500ms" @@ -99,19 +99,19 @@ func setupMockPrometheus(tds ...*testData) (*mockPrometheus, *promcfg.Config, er return mp, pCfg, err } -func newMockPrometheus(endpoints map[string][]mockPrometheusResponse) *mockPrometheus { +func newMockPrometheus(endpoints map[string][]MockPrometheusResponse) *MockPrometheus { accessIndex := make(map[string]*atomic.Int32) wg := &sync.WaitGroup{} wg.Add(len(endpoints)) for k := range endpoints { accessIndex[k] = &atomic.Int32{} } - mp := &mockPrometheus{ - wg: wg, - accessIndex: accessIndex, - endpoints: endpoints, + mp := &MockPrometheus{ + Wg: wg, + AccessIndex: accessIndex, + Endpoints: endpoints, } srv := httptest.NewServer(mp) - mp.srv = srv + mp.Srv = srv return mp } diff --git a/receiver/awscontainerinsightreceiver/receiver.go b/receiver/awscontainerinsightreceiver/receiver.go index 9972acd01718..3023ac32d131 100644 --- a/receiver/awscontainerinsightreceiver/receiver.go +++ b/receiver/awscontainerinsightreceiver/receiver.go @@ -8,6 +8,7 @@ import ( "errors" "time" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/gpu" "go.opentelemetry.io/collector/component" "go.opentelemetry.io/collector/consumer" "go.opentelemetry.io/collector/pdata/pmetric" @@ -40,6 +41,7 @@ type awsContainerInsightReceiver struct { cadvisor metricsProvider k8sapiserver metricsProvider prometheusScraper *k8sapiserver.PrometheusScraper + dcgmScraper *gpu.DcgmScraper } // newAWSContainerInsightReceiver creates the aws container insight receiver with the given parameters. @@ -95,6 +97,11 @@ func (acir *awsContainerInsightReceiver) Start(ctx context.Context, host compone if err != nil { acir.settings.Logger.Debug("Unable to start kube apiserver prometheus scraper", zap.Error(err)) } + + err = acir.startDcgmScraper(ctx, host, hostinfo) + if err != nil { + acir.settings.Logger.Debug("Unable to start dcgm scraper", zap.Error(err)) + } } if acir.config.ContainerOrchestrator == ci.ECS { @@ -169,6 +176,30 @@ func (acir *awsContainerInsightReceiver) startPrometheusScraper(ctx context.Cont }) return err } +func (acir *awsContainerInsightReceiver) startDcgmScraper(ctx context.Context, host component.Host, hostinfo *hostInfo.Info) error { + if !acir.config.EnableGpuMetric { + return nil + } + + restConfig, err := rest.InClusterConfig() + if err != nil { + return err + } + bearerToken := restConfig.BearerToken + if bearerToken == "" { + return errors.New("bearer token was empty") + } + + acir.dcgmScraper, err = gpu.NewDcgmScraper(gpu.DcgmScraperOpts{ + Ctx: ctx, + TelemetrySettings: acir.settings, + Consumer: acir.nextConsumer, + Host: host, + HostInfoProvider: hostinfo, + BearerToken: bearerToken, + }) + return err +} // Shutdown stops the awsContainerInsightReceiver receiver. func (acir *awsContainerInsightReceiver) Shutdown(context.Context) error { @@ -190,6 +221,10 @@ func (acir *awsContainerInsightReceiver) Shutdown(context.Context) error { errs = errors.Join(errs, acir.cadvisor.Shutdown()) } + if acir.dcgmScraper != nil { + acir.dcgmScraper.Shutdown() + } + return errs } @@ -216,6 +251,10 @@ func (acir *awsContainerInsightReceiver) collectData(ctx context.Context) error acir.prometheusScraper.GetMetrics() //nolint:errcheck } + if acir.dcgmScraper != nil { + acir.dcgmScraper.GetMetrics() //nolint:errcheck + } + for _, md := range mds { err := acir.nextConsumer.ConsumeMetrics(ctx, md) if err != nil { From 66a683ba7fcb53cadfd9cd7370e5281362409969 Mon Sep 17 00:00:00 2001 From: Hyunsoo Kim Date: Tue, 13 Feb 2024 09:51:28 -0500 Subject: [PATCH 02/17] update emf exporter to handle GPU metrics with different metric types update dcgm relabeling rules --- exporter/awsemfexporter/grouped_metric.go | 55 ++++--- exporter/awsemfexporter/metric_translator.go | 11 +- .../awsemfexporter/metric_translator_test.go | 139 ++++++++++++++++++ exporter/awsemfexporter/util.go | 17 +++ .../internal/gpu/dcgmscraper.go | 49 ++++-- .../internal/stores/servicestore.go | 4 + 6 files changed, 241 insertions(+), 34 deletions(-) diff --git a/exporter/awsemfexporter/grouped_metric.go b/exporter/awsemfexporter/grouped_metric.go index c62c723a4a5e..a5d845fdec8d 100644 --- a/exporter/awsemfexporter/grouped_metric.go +++ b/exporter/awsemfexporter/grouped_metric.go @@ -5,6 +5,7 @@ package awsemfexporter // import "github.com/open-telemetry/opentelemetry-collec import ( "encoding/json" + "fmt" "strings" "go.opentelemetry.io/collector/pdata/pmetric" @@ -78,32 +79,52 @@ func addToGroupedMetric(pmd pmetric.Metric, groupedMetrics map[any]*groupedMetri metadata.timestampMs = dp.timestampMs } - // Extra params to use when grouping metrics - groupKey := aws.NewKey(metadata.groupedMetricMetadata, labels) - if _, ok := groupedMetrics[groupKey]; ok { - // if MetricName already exists in metrics map, print warning log - if _, ok := groupedMetrics[groupKey].metrics[dp.name]; ok { - logger.Warn( - "Duplicate metric found", - zap.String("Name", dp.name), - zap.Any("Labels", labels), - ) - } else { - groupedMetrics[groupKey].metrics[dp.name] = metric + if _, ok := labels["GpuDevice"]; ok { + // add the same metric without GpuDevice label to apply different metric types + newLabels := map[string]string{} + for k, v := range labels { + if k == "GpuDevice" { + continue + } + newLabels[k] = v } - } else { - groupedMetrics[groupKey] = &groupedMetric{ - labels: labels, - metrics: map[string]*metricInfo{(dp.name): metric}, - metadata: metadata, + add(groupedMetrics, dp.name, metric, metadata, newLabels, logger) + + // update metric type to (Container|Pod|Node)GPU + if v, ok := labels["Type"]; ok && (v == "Container" || v == "Pod" || v == "Node") { + v = fmt.Sprintf("%sGPU", v) + labels["Type"] = v } } + add(groupedMetrics, dp.name, metric, metadata, labels, logger) } } return nil } +func add(groupedMetrics map[any]*groupedMetric, dpname string, metric *metricInfo, metadata cWMetricMetadata, labels map[string]string, logger *zap.Logger) { + groupKey := aws.NewKey(metadata.groupedMetricMetadata, labels) + if _, ok := groupedMetrics[groupKey]; ok { + // if MetricName already exists in metrics map, print warning log + if _, ok := groupedMetrics[groupKey].metrics[dpname]; ok { + logger.Warn( + "Duplicate metric found", + zap.String("Name", dpname), + zap.Any("Labels", labels), + ) + } else { + groupedMetrics[groupKey].metrics[dpname] = metric + } + } else { + groupedMetrics[groupKey] = &groupedMetric{ + labels: labels, + metrics: map[string]*metricInfo{(dpname): metric}, + metadata: metadata, + } + } +} + type kubernetesObj struct { ContainerName string `json:"container_name,omitempty"` Docker *internalDockerObj `json:"docker,omitempty"` diff --git a/exporter/awsemfexporter/metric_translator.go b/exporter/awsemfexporter/metric_translator.go index b1e37e804805..31a879546be0 100644 --- a/exporter/awsemfexporter/metric_translator.go +++ b/exporter/awsemfexporter/metric_translator.go @@ -32,6 +32,7 @@ const ( containerInsightsReceiver = "awscontainerinsight" attributeReceiver = "receiver" fieldPrometheusMetricType = "prom_metric_type" + gpuInstanceDimensionKey = "GpuDevice" ) var errMissingMetricsForEnhancedContainerInsights = errors.New("nil event detected with EnhancedContainerInsights enabled") @@ -131,7 +132,8 @@ func (mt metricTranslator) translateOTelToGroupedMetric(rm pmetric.ResourceMetri } if serviceName, ok := rm.Resource().Attributes().Get("service.name"); ok { - if strings.HasPrefix(serviceName.Str(), "containerInsightsKubeAPIServerScraper") { + if strings.HasPrefix(serviceName.Str(), "containerInsightsKubeAPIServerScraper") || + strings.HasPrefix(serviceName.Str(), "containerInsightsDCGMExporterScraper") { // the prometheus metrics that come from the container insight receiver need to be clearly tagged as coming from container insights metricReceiver = containerInsightsReceiver } @@ -204,6 +206,13 @@ func translateGroupedMetricToCWMetric(groupedMetric *groupedMetric, config *Conf } } + // filter out other dimension sets to avoid double count of the same metric + if _, ok := fields[gpuInstanceDimensionKey]; ok { + for i := 0; i < len(cWMeasurements); i++ { + cWMeasurements[i].Dimensions = filterDims(cWMeasurements[i].Dimensions, gpuInstanceDimensionKey) + } + } + return &cWMetrics{ measurements: cWMeasurements, timestampMs: groupedMetric.metadata.timestampMs, diff --git a/exporter/awsemfexporter/metric_translator_test.go b/exporter/awsemfexporter/metric_translator_test.go index c82a02086f5f..1d6abc0b6b72 100644 --- a/exporter/awsemfexporter/metric_translator_test.go +++ b/exporter/awsemfexporter/metric_translator_test.go @@ -280,6 +280,8 @@ func TestTranslateOtToGroupedMetric(t *testing.T) { // need to have 1 more metric than the default because the first is not going to be retained because it is a delta metric containerInsightMetric := createTestResourceMetricsHelper(defaultNumberOfTestMetrics + 1) containerInsightMetric.Resource().Attributes().PutStr(conventions.AttributeServiceName, "containerInsightsKubeAPIServerScraper") + gpuMetric := createTestResourceMetricsHelper(defaultNumberOfTestMetrics + 1) + gpuMetric.Resource().Attributes().PutStr(conventions.AttributeServiceName, "containerInsightsDCGMExporterScraper") counterSumMetrics := map[string]*metricInfo{ "spanCounter": { @@ -374,6 +376,20 @@ func TestTranslateOtToGroupedMetric(t *testing.T) { "myServiceNS/containerInsightsKubeAPIServerScraper", containerInsightsReceiver, }, + { + "dcgm receiver", + gpuMetric, + map[string]string{ + "isItAnError": "false", + "spanName": "testSpan", + }, + map[string]string{ + (oTellibDimensionKey): "cloudwatch-lib", + "spanName": "testSpan", + }, + "myServiceNS/containerInsightsDCGMExporterScraper", + containerInsightsReceiver, + }, } for _, tc := range testCases { @@ -618,6 +634,7 @@ func TestTranslateGroupedMetricToCWMetric(t *testing.T) { groupedMetric *groupedMetric metricDeclarations []*MetricDeclaration disableMetricExtraction bool + enableGpuMetrics bool expectedCWMetric *cWMetrics }{ { @@ -641,6 +658,7 @@ func TestTranslateGroupedMetricToCWMetric(t *testing.T) { }, nil, false, + false, &cWMetrics{ measurements: []cWMeasurement{ { @@ -687,6 +705,7 @@ func TestTranslateGroupedMetricToCWMetric(t *testing.T) { }, }, false, + false, &cWMetrics{ measurements: []cWMeasurement{ { @@ -737,6 +756,7 @@ func TestTranslateGroupedMetricToCWMetric(t *testing.T) { }, nil, false, + false, &cWMetrics{ measurements: []cWMeasurement{ { @@ -813,6 +833,7 @@ func TestTranslateGroupedMetricToCWMetric(t *testing.T) { }, }, false, + false, &cWMetrics{ measurements: []cWMeasurement{ { @@ -862,6 +883,7 @@ func TestTranslateGroupedMetricToCWMetric(t *testing.T) { }, nil, false, + false, &cWMetrics{ measurements: []cWMeasurement{ { @@ -899,6 +921,7 @@ func TestTranslateGroupedMetricToCWMetric(t *testing.T) { }, nil, false, + false, &cWMetrics{ measurements: []cWMeasurement{ { @@ -941,6 +964,7 @@ func TestTranslateGroupedMetricToCWMetric(t *testing.T) { }, nil, true, + false, &cWMetrics{ measurements: []cWMeasurement{}, timestampMs: timestamp, @@ -950,6 +974,117 @@ func TestTranslateGroupedMetricToCWMetric(t *testing.T) { }, }, }, + { + "nvidia gpu metrics", + &groupedMetric{ + labels: map[string]string{ + "label1": "value1", + "Type": "Pod", + }, + metrics: map[string]*metricInfo{ + "metric1": { + value: 1, + unit: "Count", + }, + }, + metadata: cWMetricMetadata{ + groupedMetricMetadata: groupedMetricMetadata{ + namespace: namespace, + timestampMs: timestamp, + metricDataType: pmetric.MetricTypeGauge, + }, + receiver: prometheusReceiver, + }, + }, + []*MetricDeclaration{ + { + Dimensions: [][]string{ + {"label1"}, + {"label1", "Type"}, + }, + MetricNameSelectors: []string{"metric1"}, + }, + }, + false, + true, + &cWMetrics{ + measurements: []cWMeasurement{ + { + Namespace: namespace, + Dimensions: [][]string{{"label1"}, {"label1", "Type"}}, + Metrics: []map[string]string{ + { + "Name": "metric1", + "Unit": "Count", + }, + }, + }, + }, + timestampMs: timestamp, + fields: map[string]any{ + "label1": "value1", + "metric1": 1, + fieldPrometheusMetricType: "gauge", + "Type": "Pod", + }, + }, + }, + { + "nvidia gpu metrics w/ GpuDevice", + &groupedMetric{ + labels: map[string]string{ + "label1": "value1", + "Type": "PodGPU", + gpuInstanceDimensionKey: "device0", + }, + metrics: map[string]*metricInfo{ + "metric1": { + value: 1, + unit: "Count", + }, + }, + metadata: cWMetricMetadata{ + groupedMetricMetadata: groupedMetricMetadata{ + namespace: namespace, + timestampMs: timestamp, + metricDataType: pmetric.MetricTypeGauge, + }, + receiver: prometheusReceiver, + }, + }, + []*MetricDeclaration{ + { + Dimensions: [][]string{ + {"label1", "Type", gpuInstanceDimensionKey}, + }, + MetricNameSelectors: []string{"metric1"}, + }, + }, + false, + true, + &cWMetrics{ + measurements: []cWMeasurement{ + { + Namespace: namespace, + Dimensions: [][]string{{"label1", "Type", gpuInstanceDimensionKey}}, + Metrics: []map[string]string{ + { + "Name": "metric1", + "Unit": "Count", + }, + }, + }, + }, + timestampMs: timestamp, + fields: map[string]any{ + "label1": "value1", + "metric1": 1, + fieldPrometheusMetricType: "gauge", + "Type": "PodGPU", + gpuInstanceDimensionKey: "device0", + }, + }, + }, } logger := zap.NewNop() @@ -960,6 +1095,7 @@ func TestTranslateGroupedMetricToCWMetric(t *testing.T) { MetricDeclarations: tc.metricDeclarations, DimensionRollupOption: "", DisableMetricExtraction: tc.disableMetricExtraction, + EnableGpuMetric: tc.enableGpuMetrics, logger: logger, } for _, decl := range tc.metricDeclarations { @@ -968,6 +1104,9 @@ func TestTranslateGroupedMetricToCWMetric(t *testing.T) { } cWMetric := translateGroupedMetricToCWMetric(tc.groupedMetric, config) assert.NotNil(t, cWMetric) + fmt.Println("=============================") + fmt.Println(fmt.Sprintf("%+v", cWMetric)) + fmt.Println("=============================") assertCWMetricsEqual(t, tc.expectedCWMetric, cWMetric) }) } diff --git a/exporter/awsemfexporter/util.go b/exporter/awsemfexporter/util.go index a820fd4d1ed6..511415ba9a20 100644 --- a/exporter/awsemfexporter/util.go +++ b/exporter/awsemfexporter/util.go @@ -172,3 +172,20 @@ func attrMaptoStringMap(attrMap pcommon.Map) map[string]string { }) return strMap } + +func filterDims(dims [][]string, keep string) [][]string { + var filtered [][]string + for _, dimGrp := range dims { + contains := false + for _, dim := range dimGrp { + if dim == keep { + contains = true + break + } + } + if contains { + filtered = append(filtered, dimGrp) + } + } + return filtered +} diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go index 0d9c7b32bc60..344694f65966 100644 --- a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go +++ b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go @@ -86,13 +86,15 @@ func NewDcgmScraper(opts DcgmScraperOpts) (*DcgmScraper, error) { MetricsPath: "/metrics", ServiceDiscoveryConfigs: discovery.Configs{ &kubernetes.SDConfig{ - Role: kubernetes.RoleEndpoint, + //Role: kubernetes.RoleEndpoint, + Role: kubernetes.RoleService, NamespaceDiscovery: kubernetes.NamespaceDiscovery{ IncludeOwnNamespace: true, }, Selectors: []kubernetes.SelectorConfig{ { - Role: kubernetes.RoleEndpoint, + //Role: kubernetes.RoleEndpoint, + Role: kubernetes.RoleService, Label: "k8s-app=dcgm-exporter-service", }, }, @@ -109,20 +111,6 @@ func NewDcgmScraper(opts DcgmScraperOpts) (*DcgmScraper, error) { TargetLabel: "__address__", Action: relabel.Replace, }, - { - SourceLabels: model.LabelNames{"__meta_kubernetes_pod_node_name"}, - TargetLabel: "NodeName", - Regex: relabel.MustNewRegexp("(.*)"), - Replacement: "$1", - Action: relabel.Replace, - }, - { - SourceLabels: model.LabelNames{"__meta_kubernetes_service_name"}, - TargetLabel: "Service", - Regex: relabel.MustNewRegexp("(.*)"), - Replacement: "$1", - Action: relabel.Replace, - }, }, MetricRelabelConfigs: []*relabel.Config{ { @@ -130,6 +118,13 @@ func NewDcgmScraper(opts DcgmScraperOpts) (*DcgmScraper, error) { Regex: relabel.MustNewRegexp("DCGM_.*"), Action: relabel.Keep, }, + { + SourceLabels: model.LabelNames{"Hostname"}, + TargetLabel: "NodeName", + Regex: relabel.MustNewRegexp("(.*)"), + Replacement: "${1}", + Action: relabel.Replace, + }, { SourceLabels: model.LabelNames{"namespace"}, TargetLabel: "Namespace", @@ -167,6 +162,28 @@ func NewDcgmScraper(opts DcgmScraperOpts) (*DcgmScraper, error) { Replacement: "${1}", Action: relabel.Replace, }, + // additional k8s podname for service name decoration + { + SourceLabels: model.LabelNames{"pod"}, + TargetLabel: "K8sPodName", + Regex: relabel.MustNewRegexp("(.*)"), + Replacement: "${1}", + Action: relabel.Replace, + }, + { + SourceLabels: model.LabelNames{"container"}, + TargetLabel: "ContainerName", + Regex: relabel.MustNewRegexp("(.*)"), + Replacement: "${1}", + Action: relabel.Replace, + }, + { + SourceLabels: model.LabelNames{"device"}, + TargetLabel: "GpuDevice", + Regex: relabel.MustNewRegexp("(.*)"), + Replacement: "${1}", + Action: relabel.Replace, + }, }, } diff --git a/receiver/awscontainerinsightreceiver/internal/stores/servicestore.go b/receiver/awscontainerinsightreceiver/internal/stores/servicestore.go index cd1d6e69617c..7805e069583e 100644 --- a/receiver/awscontainerinsightreceiver/internal/stores/servicestore.go +++ b/receiver/awscontainerinsightreceiver/internal/stores/servicestore.go @@ -82,3 +82,7 @@ func addServiceNameTag(metric CIMetric, serviceNames []string) { // TODO handle serviceNames len is larger than 1. We need to duplicate the metric object metric.AddTag(ci.TypeService, serviceNames[0]) } + +func (s *ServiceStore) Shutdown() error { + return nil +} From edc2cee6871619a3c742c11e286c8dd52d8b24b5 Mon Sep 17 00:00:00 2001 From: Hyunsoo Kim Date: Wed, 14 Feb 2024 11:04:58 -0500 Subject: [PATCH 03/17] remove custom logic in emf exporter --- exporter/awsemfexporter/grouped_metric.go | 55 +++------ exporter/awsemfexporter/metric_translator.go | 7 -- .../awsemfexporter/metric_translator_test.go | 111 ------------------ exporter/awsemfexporter/util.go | 17 --- 4 files changed, 17 insertions(+), 173 deletions(-) diff --git a/exporter/awsemfexporter/grouped_metric.go b/exporter/awsemfexporter/grouped_metric.go index a5d845fdec8d..c62c723a4a5e 100644 --- a/exporter/awsemfexporter/grouped_metric.go +++ b/exporter/awsemfexporter/grouped_metric.go @@ -5,7 +5,6 @@ package awsemfexporter // import "github.com/open-telemetry/opentelemetry-collec import ( "encoding/json" - "fmt" "strings" "go.opentelemetry.io/collector/pdata/pmetric" @@ -79,52 +78,32 @@ func addToGroupedMetric(pmd pmetric.Metric, groupedMetrics map[any]*groupedMetri metadata.timestampMs = dp.timestampMs } - if _, ok := labels["GpuDevice"]; ok { - // add the same metric without GpuDevice label to apply different metric types - newLabels := map[string]string{} - for k, v := range labels { - if k == "GpuDevice" { - continue - } - newLabels[k] = v + // Extra params to use when grouping metrics + groupKey := aws.NewKey(metadata.groupedMetricMetadata, labels) + if _, ok := groupedMetrics[groupKey]; ok { + // if MetricName already exists in metrics map, print warning log + if _, ok := groupedMetrics[groupKey].metrics[dp.name]; ok { + logger.Warn( + "Duplicate metric found", + zap.String("Name", dp.name), + zap.Any("Labels", labels), + ) + } else { + groupedMetrics[groupKey].metrics[dp.name] = metric } - add(groupedMetrics, dp.name, metric, metadata, newLabels, logger) - - // update metric type to (Container|Pod|Node)GPU - if v, ok := labels["Type"]; ok && (v == "Container" || v == "Pod" || v == "Node") { - v = fmt.Sprintf("%sGPU", v) - labels["Type"] = v + } else { + groupedMetrics[groupKey] = &groupedMetric{ + labels: labels, + metrics: map[string]*metricInfo{(dp.name): metric}, + metadata: metadata, } } - add(groupedMetrics, dp.name, metric, metadata, labels, logger) } } return nil } -func add(groupedMetrics map[any]*groupedMetric, dpname string, metric *metricInfo, metadata cWMetricMetadata, labels map[string]string, logger *zap.Logger) { - groupKey := aws.NewKey(metadata.groupedMetricMetadata, labels) - if _, ok := groupedMetrics[groupKey]; ok { - // if MetricName already exists in metrics map, print warning log - if _, ok := groupedMetrics[groupKey].metrics[dpname]; ok { - logger.Warn( - "Duplicate metric found", - zap.String("Name", dpname), - zap.Any("Labels", labels), - ) - } else { - groupedMetrics[groupKey].metrics[dpname] = metric - } - } else { - groupedMetrics[groupKey] = &groupedMetric{ - labels: labels, - metrics: map[string]*metricInfo{(dpname): metric}, - metadata: metadata, - } - } -} - type kubernetesObj struct { ContainerName string `json:"container_name,omitempty"` Docker *internalDockerObj `json:"docker,omitempty"` diff --git a/exporter/awsemfexporter/metric_translator.go b/exporter/awsemfexporter/metric_translator.go index 31a879546be0..560421ace089 100644 --- a/exporter/awsemfexporter/metric_translator.go +++ b/exporter/awsemfexporter/metric_translator.go @@ -206,13 +206,6 @@ func translateGroupedMetricToCWMetric(groupedMetric *groupedMetric, config *Conf } } - // filter out other dimension sets to avoid double count of the same metric - if _, ok := fields[gpuInstanceDimensionKey]; ok { - for i := 0; i < len(cWMeasurements); i++ { - cWMeasurements[i].Dimensions = filterDims(cWMeasurements[i].Dimensions, gpuInstanceDimensionKey) - } - } - return &cWMetrics{ measurements: cWMeasurements, timestampMs: groupedMetric.metadata.timestampMs, diff --git a/exporter/awsemfexporter/metric_translator_test.go b/exporter/awsemfexporter/metric_translator_test.go index 1d6abc0b6b72..6f28c18f4f72 100644 --- a/exporter/awsemfexporter/metric_translator_test.go +++ b/exporter/awsemfexporter/metric_translator_test.go @@ -974,117 +974,6 @@ func TestTranslateGroupedMetricToCWMetric(t *testing.T) { }, }, }, - { - "nvidia gpu metrics", - &groupedMetric{ - labels: map[string]string{ - "label1": "value1", - "Type": "Pod", - }, - metrics: map[string]*metricInfo{ - "metric1": { - value: 1, - unit: "Count", - }, - }, - metadata: cWMetricMetadata{ - groupedMetricMetadata: groupedMetricMetadata{ - namespace: namespace, - timestampMs: timestamp, - metricDataType: pmetric.MetricTypeGauge, - }, - receiver: prometheusReceiver, - }, - }, - []*MetricDeclaration{ - { - Dimensions: [][]string{ - {"label1"}, - {"label1", "Type"}, - }, - MetricNameSelectors: []string{"metric1"}, - }, - }, - false, - true, - &cWMetrics{ - measurements: []cWMeasurement{ - { - Namespace: namespace, - Dimensions: [][]string{{"label1"}, {"label1", "Type"}}, - Metrics: []map[string]string{ - { - "Name": "metric1", - "Unit": "Count", - }, - }, - }, - }, - timestampMs: timestamp, - fields: map[string]any{ - "label1": "value1", - "metric1": 1, - fieldPrometheusMetricType: "gauge", - "Type": "Pod", - }, - }, - }, - { - "nvidia gpu metrics w/ GpuDevice", - &groupedMetric{ - labels: map[string]string{ - "label1": "value1", - "Type": "PodGPU", - gpuInstanceDimensionKey: "device0", - }, - metrics: map[string]*metricInfo{ - "metric1": { - value: 1, - unit: "Count", - }, - }, - metadata: cWMetricMetadata{ - groupedMetricMetadata: groupedMetricMetadata{ - namespace: namespace, - timestampMs: timestamp, - metricDataType: pmetric.MetricTypeGauge, - }, - receiver: prometheusReceiver, - }, - }, - []*MetricDeclaration{ - { - Dimensions: [][]string{ - {"label1", "Type", gpuInstanceDimensionKey}, - }, - MetricNameSelectors: []string{"metric1"}, - }, - }, - false, - true, - &cWMetrics{ - measurements: []cWMeasurement{ - { - Namespace: namespace, - Dimensions: [][]string{{"label1", "Type", gpuInstanceDimensionKey}}, - Metrics: []map[string]string{ - { - "Name": "metric1", - "Unit": "Count", - }, - }, - }, - }, - timestampMs: timestamp, - fields: map[string]any{ - "label1": "value1", - "metric1": 1, - fieldPrometheusMetricType: "gauge", - "Type": "PodGPU", - gpuInstanceDimensionKey: "device0", - }, - }, - }, } logger := zap.NewNop() diff --git a/exporter/awsemfexporter/util.go b/exporter/awsemfexporter/util.go index 511415ba9a20..a820fd4d1ed6 100644 --- a/exporter/awsemfexporter/util.go +++ b/exporter/awsemfexporter/util.go @@ -172,20 +172,3 @@ func attrMaptoStringMap(attrMap pcommon.Map) map[string]string { }) return strMap } - -func filterDims(dims [][]string, keep string) [][]string { - var filtered [][]string - for _, dimGrp := range dims { - contains := false - for _, dim := range dimGrp { - if dim == keep { - contains = true - break - } - } - if contains { - filtered = append(filtered, dimGrp) - } - } - return filtered -} From a6443f12bbe43db50f8a82d0e0940e452a7cda00 Mon Sep 17 00:00:00 2001 From: Hyunsoo Kim Date: Wed, 14 Feb 2024 11:07:52 -0500 Subject: [PATCH 04/17] update gpu flag comment --- receiver/awscontainerinsightreceiver/config.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/receiver/awscontainerinsightreceiver/config.go b/receiver/awscontainerinsightreceiver/config.go index 2e06ba111d33..59b3160952b2 100644 --- a/receiver/awscontainerinsightreceiver/config.go +++ b/receiver/awscontainerinsightreceiver/config.go @@ -58,7 +58,7 @@ type Config struct { // The default value is false. EnableControlPlaneMetrics bool `mapstructure:"enable_control_plane_metrics"` - // EnableGpuMetric disables GPU monitoring where metrics are scraped from vendor specific sources + // EnableGpuMetric toggles GPU monitoring where metrics are scraped from vendor specific sources // The default value is true meaning GPU metrics get collected out of the box unless it's disabled EnableGpuMetric bool `mapstructure:"gpu_metrics"` } From 782f24a045c6df152d468db36a985c23792f055d Mon Sep 17 00:00:00 2001 From: Hyunsoo Kim Date: Wed, 14 Feb 2024 11:40:58 -0500 Subject: [PATCH 05/17] remove comments and test codes --- .../awsemfexporter/metric_translator_test.go | 3 -- .../internal/gpu/dcgmscraper.go | 2 -- .../internal/gpu/dcgmscraper_test.go | 32 +------------------ 3 files changed, 1 insertion(+), 36 deletions(-) diff --git a/exporter/awsemfexporter/metric_translator_test.go b/exporter/awsemfexporter/metric_translator_test.go index 6f28c18f4f72..6657ea4ed242 100644 --- a/exporter/awsemfexporter/metric_translator_test.go +++ b/exporter/awsemfexporter/metric_translator_test.go @@ -993,9 +993,6 @@ func TestTranslateGroupedMetricToCWMetric(t *testing.T) { } cWMetric := translateGroupedMetricToCWMetric(tc.groupedMetric, config) assert.NotNil(t, cWMetric) - fmt.Println("=============================") - fmt.Println(fmt.Sprintf("%+v", cWMetric)) - fmt.Println("=============================") assertCWMetricsEqual(t, tc.expectedCWMetric, cWMetric) }) } diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go index 344694f65966..e894d99338aa 100644 --- a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go +++ b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go @@ -86,14 +86,12 @@ func NewDcgmScraper(opts DcgmScraperOpts) (*DcgmScraper, error) { MetricsPath: "/metrics", ServiceDiscoveryConfigs: discovery.Configs{ &kubernetes.SDConfig{ - //Role: kubernetes.RoleEndpoint, Role: kubernetes.RoleService, NamespaceDiscovery: kubernetes.NamespaceDiscovery{ IncludeOwnNamespace: true, }, Selectors: []kubernetes.SelectorConfig{ { - //Role: kubernetes.RoleEndpoint, Role: kubernetes.RoleService, Label: "k8s-app=dcgm-exporter-service", }, diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go index 7a4cd78da3d7..1d59c594b661 100644 --- a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go +++ b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go @@ -200,37 +200,7 @@ func TestNewDcgmScraperEndToEnd(t *testing.T) { }, }, }, - RelabelConfigs: []*relabel.Config{ - // doesn't seem like there is a good way to unit test relabeling rules https://github.com/prometheus/prometheus/issues/8606 - //{ - // SourceLabels: model.LabelNames{"__address__"}, - // Regex: relabel.MustNewRegexp("([^:]+)(?::\\d+)?"), - // Replacement: "${1}:9400", - // TargetLabel: "__address__", - // Action: relabel.Replace, - //}, - //{ - // SourceLabels: model.LabelNames{"__meta_kubernetes_namespace"}, - // TargetLabel: "Namespace", - // Regex: relabel.MustNewRegexp("(.*)"), - // Replacement: "$1", - // Action: relabel.Replace, - //}, - //{ - // SourceLabels: model.LabelNames{"__meta_kubernetes_pod_name"}, - // TargetLabel: "pod", - // Regex: relabel.MustNewRegexp("(.*)"), - // Replacement: "$1", - // Action: relabel.Replace, - //}, - //{ - // SourceLabels: model.LabelNames{"__meta_kubernetes_pod_node_name"}, - // TargetLabel: "NodeName", - // Regex: relabel.MustNewRegexp("(.*)"), - // Replacement: "$1", - // Action: relabel.Replace, - //}, - }, + RelabelConfigs: []*relabel.Config{}, MetricRelabelConfigs: []*relabel.Config{ { SourceLabels: model.LabelNames{"__name__"}, From 18db72ac6e0be2789d955d768726355f3d764ac3 Mon Sep 17 00:00:00 2001 From: Hyunsoo Kim Date: Thu, 15 Feb 2024 21:00:08 -0500 Subject: [PATCH 06/17] remove unused codes and rename scraper init funcs --- exporter/awsemfexporter/config.go | 3 -- exporter/awsemfexporter/metric_translator.go | 1 - .../awsemfexporter/metric_translator_test.go | 17 +++------- .../internal/gpu/dcgmscraper.go | 5 ++- .../internal/gpu/dcgmscraper_test.go | 1 - .../internal/stores/servicestore.go | 4 --- .../awscontainerinsightreceiver/receiver.go | 31 +++++++------------ 7 files changed, 17 insertions(+), 45 deletions(-) diff --git a/exporter/awsemfexporter/config.go b/exporter/awsemfexporter/config.go index 6f12efb3c708..e42b5532852c 100644 --- a/exporter/awsemfexporter/config.go +++ b/exporter/awsemfexporter/config.go @@ -100,9 +100,6 @@ type Config struct { // MiddlewareID is an ID for an extension that can be used to configure the AWS client. MiddlewareID *component.ID `mapstructure:"middleware,omitempty"` - // EnableGpuMetric indicates payloads will include GPU metrics - EnableGpuMetric bool `mapstructure:"gpu_metrics"` - // logger is the Logger used for writing error/warning logs logger *zap.Logger } diff --git a/exporter/awsemfexporter/metric_translator.go b/exporter/awsemfexporter/metric_translator.go index 560421ace089..b3c1ecf42c54 100644 --- a/exporter/awsemfexporter/metric_translator.go +++ b/exporter/awsemfexporter/metric_translator.go @@ -32,7 +32,6 @@ const ( containerInsightsReceiver = "awscontainerinsight" attributeReceiver = "receiver" fieldPrometheusMetricType = "prom_metric_type" - gpuInstanceDimensionKey = "GpuDevice" ) var errMissingMetricsForEnhancedContainerInsights = errors.New("nil event detected with EnhancedContainerInsights enabled") diff --git a/exporter/awsemfexporter/metric_translator_test.go b/exporter/awsemfexporter/metric_translator_test.go index 6657ea4ed242..9986e87bd641 100644 --- a/exporter/awsemfexporter/metric_translator_test.go +++ b/exporter/awsemfexporter/metric_translator_test.go @@ -370,8 +370,8 @@ func TestTranslateOtToGroupedMetric(t *testing.T) { "spanName": "testSpan", }, map[string]string{ - (oTellibDimensionKey): "cloudwatch-lib", - "spanName": "testSpan", + oTellibDimensionKey: "cloudwatch-lib", + "spanName": "testSpan", }, "myServiceNS/containerInsightsKubeAPIServerScraper", containerInsightsReceiver, @@ -384,8 +384,8 @@ func TestTranslateOtToGroupedMetric(t *testing.T) { "spanName": "testSpan", }, map[string]string{ - (oTellibDimensionKey): "cloudwatch-lib", - "spanName": "testSpan", + oTellibDimensionKey: "cloudwatch-lib", + "spanName": "testSpan", }, "myServiceNS/containerInsightsDCGMExporterScraper", containerInsightsReceiver, @@ -634,7 +634,6 @@ func TestTranslateGroupedMetricToCWMetric(t *testing.T) { groupedMetric *groupedMetric metricDeclarations []*MetricDeclaration disableMetricExtraction bool - enableGpuMetrics bool expectedCWMetric *cWMetrics }{ { @@ -658,7 +657,6 @@ func TestTranslateGroupedMetricToCWMetric(t *testing.T) { }, nil, false, - false, &cWMetrics{ measurements: []cWMeasurement{ { @@ -705,7 +703,6 @@ func TestTranslateGroupedMetricToCWMetric(t *testing.T) { }, }, false, - false, &cWMetrics{ measurements: []cWMeasurement{ { @@ -756,7 +753,6 @@ func TestTranslateGroupedMetricToCWMetric(t *testing.T) { }, nil, false, - false, &cWMetrics{ measurements: []cWMeasurement{ { @@ -833,7 +829,6 @@ func TestTranslateGroupedMetricToCWMetric(t *testing.T) { }, }, false, - false, &cWMetrics{ measurements: []cWMeasurement{ { @@ -883,7 +878,6 @@ func TestTranslateGroupedMetricToCWMetric(t *testing.T) { }, nil, false, - false, &cWMetrics{ measurements: []cWMeasurement{ { @@ -921,7 +915,6 @@ func TestTranslateGroupedMetricToCWMetric(t *testing.T) { }, nil, false, - false, &cWMetrics{ measurements: []cWMeasurement{ { @@ -964,7 +957,6 @@ func TestTranslateGroupedMetricToCWMetric(t *testing.T) { }, nil, true, - false, &cWMetrics{ measurements: []cWMeasurement{}, timestampMs: timestamp, @@ -984,7 +976,6 @@ func TestTranslateGroupedMetricToCWMetric(t *testing.T) { MetricDeclarations: tc.metricDeclarations, DimensionRollupOption: "", DisableMetricExtraction: tc.disableMetricExtraction, - EnableGpuMetric: tc.enableGpuMetrics, logger: logger, } for _, decl := range tc.metricDeclarations { diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go index e894d99338aa..e2ef486dd072 100644 --- a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go +++ b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go @@ -44,7 +44,6 @@ type DcgmScraperOpts struct { Consumer consumer.Metrics Host component.Host HostInfoProvider hostInfoProvider - BearerToken string } type hostInfoProvider interface { @@ -135,14 +134,14 @@ func NewDcgmScraper(opts DcgmScraperOpts) (*DcgmScraper, error) { { SourceLabels: model.LabelNames{"namespace"}, TargetLabel: "ClusterName", - Regex: relabel.MustNewRegexp("(.*)"), + Regex: relabel.MustNewRegexp(".*"), Replacement: opts.HostInfoProvider.GetClusterName(), Action: relabel.Replace, }, { SourceLabels: model.LabelNames{"namespace"}, TargetLabel: "InstanceId", - Regex: relabel.MustNewRegexp("(.*)"), + Regex: relabel.MustNewRegexp(".*"), Replacement: opts.HostInfoProvider.GetInstanceID(), Action: relabel.Replace, }, diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go index 1d59c594b661..9d7d244018d3 100644 --- a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go +++ b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go @@ -155,7 +155,6 @@ func TestNewDcgmScraperEndToEnd(t *testing.T) { Consumer: mockConsumer{}, Host: componenttest.NewNopHost(), HostInfoProvider: mockHostInfoProvider{}, - BearerToken: "", }) assert.NoError(t, err) assert.Equal(t, mockHostInfoProvider{}, scraper.hostInfoProvider) diff --git a/receiver/awscontainerinsightreceiver/internal/stores/servicestore.go b/receiver/awscontainerinsightreceiver/internal/stores/servicestore.go index 7805e069583e..cd1d6e69617c 100644 --- a/receiver/awscontainerinsightreceiver/internal/stores/servicestore.go +++ b/receiver/awscontainerinsightreceiver/internal/stores/servicestore.go @@ -82,7 +82,3 @@ func addServiceNameTag(metric CIMetric, serviceNames []string) { // TODO handle serviceNames len is larger than 1. We need to duplicate the metric object metric.AddTag(ci.TypeService, serviceNames[0]) } - -func (s *ServiceStore) Shutdown() error { - return nil -} diff --git a/receiver/awscontainerinsightreceiver/receiver.go b/receiver/awscontainerinsightreceiver/receiver.go index 3023ac32d131..149c9ff19948 100644 --- a/receiver/awscontainerinsightreceiver/receiver.go +++ b/receiver/awscontainerinsightreceiver/receiver.go @@ -92,15 +92,15 @@ func (acir *awsContainerInsightReceiver) Start(ctx context.Context, host compone if err != nil { return err } + err = acir.initPrometheusScraper(ctx, host, hostinfo, leaderElection) + if err != nil { + acir.settings.Logger.Debug("Unable to start kube apiserver prometheus scraper", zap.Error(err)) + } - err = acir.startPrometheusScraper(ctx, host, hostinfo, leaderElection) - if err != nil { - acir.settings.Logger.Debug("Unable to start kube apiserver prometheus scraper", zap.Error(err)) - } - - err = acir.startDcgmScraper(ctx, host, hostinfo) - if err != nil { - acir.settings.Logger.Debug("Unable to start dcgm scraper", zap.Error(err)) + err = acir.initDcgmScraper(ctx, host, hostinfo) + if err != nil { + acir.settings.Logger.Debug("Unable to start dcgm scraper", zap.Error(err)) + } } } if acir.config.ContainerOrchestrator == ci.ECS { @@ -142,7 +142,7 @@ func (acir *awsContainerInsightReceiver) Start(ctx context.Context, host compone return nil } -func (acir *awsContainerInsightReceiver) startPrometheusScraper(ctx context.Context, host component.Host, hostinfo *hostInfo.Info, leaderElection *k8sapiserver.LeaderElection) error { +func (acir *awsContainerInsightReceiver) initPrometheusScraper(ctx context.Context, host component.Host, hostinfo *hostInfo.Info, leaderElection *k8sapiserver.LeaderElection) error { if !acir.config.EnableControlPlaneMetrics { return nil } @@ -176,27 +176,18 @@ func (acir *awsContainerInsightReceiver) startPrometheusScraper(ctx context.Cont }) return err } -func (acir *awsContainerInsightReceiver) startDcgmScraper(ctx context.Context, host component.Host, hostinfo *hostInfo.Info) error { +func (acir *awsContainerInsightReceiver) initDcgmScraper(ctx context.Context, host component.Host, hostinfo *hostInfo.Info) error { if !acir.config.EnableGpuMetric { return nil } - restConfig, err := rest.InClusterConfig() - if err != nil { - return err - } - bearerToken := restConfig.BearerToken - if bearerToken == "" { - return errors.New("bearer token was empty") - } - + var err error acir.dcgmScraper, err = gpu.NewDcgmScraper(gpu.DcgmScraperOpts{ Ctx: ctx, TelemetrySettings: acir.settings, Consumer: acir.nextConsumer, Host: host, HostInfoProvider: hostinfo, - BearerToken: bearerToken, }) return err } From 5d11f3cd3a2d98a9a0853577c8a1e3fd956b9dd3 Mon Sep 17 00:00:00 2001 From: Hyunsoo Kim Date: Fri, 16 Feb 2024 13:32:55 -0500 Subject: [PATCH 07/17] remove comments --- receiver/awscontainerinsightreceiver/config.go | 1 - .../internal/gpu/dcgmscraper.go | 15 --------------- 2 files changed, 16 deletions(-) diff --git a/receiver/awscontainerinsightreceiver/config.go b/receiver/awscontainerinsightreceiver/config.go index 59b3160952b2..df74242e20f7 100644 --- a/receiver/awscontainerinsightreceiver/config.go +++ b/receiver/awscontainerinsightreceiver/config.go @@ -59,6 +59,5 @@ type Config struct { EnableControlPlaneMetrics bool `mapstructure:"enable_control_plane_metrics"` // EnableGpuMetric toggles GPU monitoring where metrics are scraped from vendor specific sources - // The default value is true meaning GPU metrics get collected out of the box unless it's disabled EnableGpuMetric bool `mapstructure:"gpu_metrics"` } diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go index e2ef486dd072..8aa650750c6a 100644 --- a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go +++ b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go @@ -63,21 +63,6 @@ func NewDcgmScraper(opts DcgmScraperOpts) (*DcgmScraper, error) { } scrapeConfig := &config.ScrapeConfig{ - // TLS needs to be enabled between pods communication - // It can further get restricted by adding authentication mechanism to limit the data - //HTTPClientConfig: configutil.HTTPClientConfig{ - // BasicAuth: &configutil.BasicAuth{ - // Username: "", - // Password: "", - // }, - // Authorization: &configutil.Authorization{ - // Type: "basic_auth", - // }, - // TLSConfig: configutil.TLSConfig{ - // CAFile: caFile, - // InsecureSkipVerify: false, - // }, - //}, ScrapeInterval: model.Duration(collectionInterval), ScrapeTimeout: model.Duration(collectionInterval), JobName: jobName, From 9ed67706d440af523f8ebcb2eb31dc00bf14b639 Mon Sep 17 00:00:00 2001 From: Hyunsoo Kim Date: Fri, 16 Feb 2024 14:03:09 -0500 Subject: [PATCH 08/17] add changelog for gpu --- .chloggen-aws/nvidia-gpu.yaml | 24 ++++++++++++++++++++++++ 1 file changed, 24 insertions(+) create mode 100755 .chloggen-aws/nvidia-gpu.yaml diff --git a/.chloggen-aws/nvidia-gpu.yaml b/.chloggen-aws/nvidia-gpu.yaml new file mode 100755 index 000000000000..268849523a1a --- /dev/null +++ b/.chloggen-aws/nvidia-gpu.yaml @@ -0,0 +1,24 @@ +# Use this changelog template to create an entry for release notes. + +# One of 'breaking', 'deprecation', 'new_component', 'enhancement', 'bug_fix' +change_type: 'enhancement' + +# The name of the component, or a single word describing the area of concern, (e.g. filelogreceiver) +component: containerinsightsreceiver + +# A brief description of the change. Surround your text with quotes ("") if it needs to start with a backtick (`). +note: "Adds DCGM scraper to collect NVIDIA GPU metrics" + +# Mandatory: One or more tracking issues related to the change. You can use the PR number here if no issue exists. +issues: [160] + +# (Optional) One or more lines of additional information to render under the primary note. +# These lines will be padded with 2 spaces and then inserted directly into the document. +# Use pipe (|) for multiline entries. +subtext: Supports NVIDIA GPU metrics by adding a new prometheus data scraper in a k8s environment. The new scraper | + relabels the default DCGM labels into existing Container Insights labels. + +# e.g. '[aws]' +# Include 'aws' if the change is done by cwa +# Default: '[user]' +change_logs: [aws] \ No newline at end of file From e8af343374db21108190dd1562af0cb156e027d2 Mon Sep 17 00:00:00 2001 From: Hyunsoo Kim Date: Mon, 26 Feb 2024 15:54:24 -0500 Subject: [PATCH 09/17] add gpu metric consumer that uses k8s decorator for attributes --- internal/aws/containerinsight/const.go | 4 + internal/aws/containerinsight/utils.go | 33 +++ .../internal/cadvisor/cadvisor_linux.go | 5 +- .../internal/cadvisor/cadvisor_linux_test.go | 4 +- .../internal/cadvisor/extractors/extractor.go | 3 + .../internal/gpu/dcgmscraper.go | 29 ++- .../internal/gpu/decorator.go | 204 +++++++++++++++ .../internal/gpu/decorator_test.go | 233 ++++++++++++++++++ .../internal/stores/podstore.go | 4 +- .../internal/stores/store.go | 7 +- .../internal/stores/utils.go | 3 + .../internal/stores/utils_test.go | 2 + .../awscontainerinsightreceiver/receiver.go | 6 +- 13 files changed, 515 insertions(+), 22 deletions(-) create mode 100644 receiver/awscontainerinsightreceiver/internal/gpu/decorator.go create mode 100644 receiver/awscontainerinsightreceiver/internal/gpu/decorator_test.go diff --git a/internal/aws/containerinsight/const.go b/internal/aws/containerinsight/const.go index 38a94360bd8b..6fb5d762cb6f 100644 --- a/internal/aws/containerinsight/const.go +++ b/internal/aws/containerinsight/const.go @@ -151,6 +151,10 @@ const ( // Special type for pause container // because containerd does not set container name pause container name to POD like docker does. TypeInfraContainer = "InfraContainer" + TypeGpuContainer = "ContainerGPU" + TypeGpuPod = "PodGPU" + TypeGpuNode = "NodeGPU" + TypeGpuCluster = "ClusterGPU" // unit UnitBytes = "Bytes" diff --git a/internal/aws/containerinsight/utils.go b/internal/aws/containerinsight/utils.go index 5f734597bfb1..18cf0662559b 100644 --- a/internal/aws/containerinsight/utils.go +++ b/internal/aws/containerinsight/utils.go @@ -166,6 +166,39 @@ func GetUnitForMetric(metric string) string { return metricToUnitMap[metric] } +// ConvertToFieldsAndTags converts OTLP metric to a field containing metric values and a tag containing for decoration +func ConvertToFieldsAndTags(m pmetric.Metric, logger *zap.Logger) (map[string]any, map[string]string) { + fields := make(map[string]any) + tags := make(map[string]string) + if m.Name() == "" { + return fields, tags + } + + // value is not needed for label decoration + fields[m.Name()] = 0 + + var dps pmetric.NumberDataPointSlice + switch m.Type() { + case pmetric.MetricTypeGauge: + dps = m.Gauge().DataPoints() + case pmetric.MetricTypeSum: + dps = m.Sum().DataPoints() + default: + logger.Warn("Unsupported metric type", zap.String("metric", m.Name()), zap.String("type", m.Type().String())) + } + + // should support metrics with more than 1 datapoints? + if dps.Len() > 1 { + logger.Warn("Metric with more than 1 datapoint is not supported", zap.String("metric", m.Name()), zap.Int("datapoints", dps.Len())) + } + attrs := dps.At(0).Attributes() + attrs.Range(func(k string, v pcommon.Value) bool { + tags[k] = v.Str() + return true + }) + return fields, tags +} + // ConvertToOTLPMetrics converts a field containing metric values and a tag containing the relevant labels to OTLP metrics func ConvertToOTLPMetrics(fields map[string]any, tags map[string]string, logger *zap.Logger) pmetric.Metrics { md := pmetric.NewMetrics() diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_linux.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_linux.go index 971217b9c05e..4a4521479faa 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_linux.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_linux.go @@ -22,6 +22,7 @@ import ( cInfo "github.com/google/cadvisor/info/v1" "github.com/google/cadvisor/manager" "github.com/google/cadvisor/utils/sysfs" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "go.opentelemetry.io/collector/pdata/pmetric" "go.uber.org/zap" @@ -109,7 +110,7 @@ type EcsInfo interface { } type Decorator interface { - Decorate(*extractors.CAdvisorMetric) *extractors.CAdvisorMetric + Decorate(stores.CIMetric) stores.CIMetric Shutdown() error } @@ -307,7 +308,7 @@ func (c *Cadvisor) decorateMetrics(cadvisormetrics []*extractors.CAdvisorMetric) out := c.k8sDecorator.Decorate(m) if out != nil { - result = append(result, out) + result = append(result, out.(*extractors.CAdvisorMetric)) } } diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_linux_test.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_linux_test.go index 85451e580ea2..9e38af49c936 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_linux_test.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_linux_test.go @@ -16,10 +16,10 @@ import ( info "github.com/google/cadvisor/info/v1" "github.com/google/cadvisor/manager" "github.com/google/cadvisor/utils/sysfs" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "github.com/stretchr/testify/assert" "go.uber.org/zap" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/cadvisor/testutils" ) @@ -73,7 +73,7 @@ var mockCreateManagerWithError = func(memoryCache *memory.InMemoryCache, sysfs s type MockK8sDecorator struct { } -func (m *MockK8sDecorator) Decorate(metric *extractors.CAdvisorMetric) *extractors.CAdvisorMetric { +func (m *MockK8sDecorator) Decorate(metric stores.CIMetric) stores.CIMetric { return metric } diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor.go index 398ad4805a59..a10c891117e4 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor.go @@ -8,12 +8,15 @@ import ( "time" cinfo "github.com/google/cadvisor/info/v1" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "go.uber.org/zap" ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" awsmetrics "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/metrics" ) +var _ stores.CIMetric = (*CAdvisorMetric)(nil) + func GetStats(info *cinfo.ContainerInfo) *cinfo.ContainerStats { if len(info.Stats) == 0 { return nil diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go index 8aa650750c6a..6f737d978b57 100644 --- a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go +++ b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go @@ -9,6 +9,7 @@ import ( "fmt" "time" + ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/discovery" @@ -35,6 +36,7 @@ type DcgmScraper struct { host component.Host hostInfoProvider hostInfoProvider prometheusReceiver receiver.Metrics + k8sDecorator Decorator running bool } @@ -44,6 +46,8 @@ type DcgmScraperOpts struct { Consumer consumer.Metrics Host component.Host HostInfoProvider hostInfoProvider + K8sDecorator Decorator + Logger *zap.Logger } type hostInfoProvider interface { @@ -137,13 +141,6 @@ func NewDcgmScraper(opts DcgmScraperOpts) (*DcgmScraper, error) { Replacement: "${1}", Action: relabel.Replace, }, - { - SourceLabels: model.LabelNames{"pod"}, - TargetLabel: "PodName", - Regex: relabel.MustNewRegexp("(.+)-(.+)"), - Replacement: "${1}", - Action: relabel.Replace, - }, // additional k8s podname for service name decoration { SourceLabels: model.LabelNames{"pod"}, @@ -179,8 +176,15 @@ func NewDcgmScraper(opts DcgmScraperOpts) (*DcgmScraper, error) { TelemetrySettings: opts.TelemetrySettings, } + decoConsumer := decorateConsumer{ + containerOrchestrator: ci.EKS, + nextConsumer: opts.Consumer, + k8sDecorator: opts.K8sDecorator, + logger: opts.Logger, + } + promFactory := prometheusreceiver.NewFactory() - promReceiver, err := promFactory.CreateMetricsReceiver(opts.Ctx, params, &promConfig, opts.Consumer) + promReceiver, err := promFactory.CreateMetricsReceiver(opts.Ctx, params, &promConfig, &decoConsumer) if err != nil { return nil, fmt.Errorf("failed to create prometheus receiver: %w", err) } @@ -191,6 +195,7 @@ func NewDcgmScraper(opts DcgmScraperOpts) (*DcgmScraper, error) { host: opts.Host, hostInfoProvider: opts.HostInfoProvider, prometheusReceiver: promReceiver, + k8sDecorator: opts.K8sDecorator, }, nil } @@ -205,6 +210,7 @@ func (ds *DcgmScraper) GetMetrics() []pmetric.Metrics { } ds.running = err == nil } + return nil } @@ -216,4 +222,11 @@ func (ds *DcgmScraper) Shutdown() { } ds.running = false } + + if ds.k8sDecorator != nil { + err := ds.k8sDecorator.Shutdown() + if err != nil { + ds.settings.Logger.Error("Unable to shutdown K8sDecorator", zap.Error(err)) + } + } } diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/decorator.go b/receiver/awscontainerinsightreceiver/internal/gpu/decorator.go new file mode 100644 index 000000000000..4c106db52730 --- /dev/null +++ b/receiver/awscontainerinsightreceiver/internal/gpu/decorator.go @@ -0,0 +1,204 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +package gpu + +import ( + "context" + "errors" + + ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" + "go.opentelemetry.io/collector/consumer" + "go.opentelemetry.io/collector/pdata/pcommon" + "go.opentelemetry.io/collector/pdata/pmetric" + "go.uber.org/zap" + "golang.org/x/exp/maps" +) + +const ( + gpuUtil = "DCGM_FI_DEV_GPU_UTIL" + gpuMemUtil = "DCGM_FI_DEV_FB_USED_PERCENT" + gpuMemUsed = "DCGM_FI_DEV_FB_USED" + gpuMemTotal = "DCGM_FI_DEV_FB_TOTAL" + gpuTemperature = "DCGM_FI_DEV_GPU_TEMP" + gpuPowerDraw = "DCGM_FI_DEV_POWER_USAGE" +) + +var _ stores.CIMetric = (*gpuMetric)(nil) + +var metricToUnit = map[string]string{ + gpuUtil: "Percent", + gpuMemUtil: "Percent", + gpuMemUsed: "Bytes", + gpuMemTotal: "Bytes", + gpuTemperature: "None", + gpuPowerDraw: "None", +} + +type gpuMetric struct { + // key/value pairs that are typed and contain the metric (numerical) data + fields map[string]any + // key/value string pairs that are used to identify the metrics + tags map[string]string +} + +func newResourceMetric(mType string, logger *zap.Logger) *gpuMetric { + metric := &gpuMetric{ + fields: make(map[string]any), + tags: make(map[string]string), + } + metric.tags[ci.MetricType] = mType + return metric +} + +func (gr *gpuMetric) GetTags() map[string]string { + return gr.tags +} + +func (gr *gpuMetric) GetFields() map[string]any { + return gr.fields +} + +func (gr *gpuMetric) GetMetricType() string { + return gr.tags[ci.MetricType] +} + +func (gr *gpuMetric) AddTags(tags map[string]string) { + for k, v := range tags { + gr.tags[k] = v + } +} + +func (gr *gpuMetric) HasField(key string) bool { + return gr.fields[key] != nil +} + +func (gr *gpuMetric) AddField(key string, val any) { + gr.fields[key] = val +} + +func (gr *gpuMetric) GetField(key string) any { + return gr.fields[key] +} + +func (gr *gpuMetric) HasTag(key string) bool { + return gr.tags[key] != "" +} + +func (gr *gpuMetric) AddTag(key, val string) { + gr.tags[key] = val +} + +func (gr *gpuMetric) GetTag(key string) string { + return gr.tags[key] +} + +func (gr *gpuMetric) RemoveTag(key string) { + delete(gr.tags, key) +} + +// GPU decorator acts as an interceptor of metrics before the scraper sends them to the next designated consumer +type decorateConsumer struct { + containerOrchestrator string + nextConsumer consumer.Metrics + k8sDecorator Decorator + logger *zap.Logger +} + +func (dc *decorateConsumer) Capabilities() consumer.Capabilities { + return consumer.Capabilities{ + MutatesData: true, + } +} + +func (dc *decorateConsumer) ConsumeMetrics(ctx context.Context, md pmetric.Metrics) error { + resourceTags := make(map[string]string) + rms := md.ResourceMetrics() + for i := 0; i < rms.Len(); i++ { + // get resource attributes + ras := rms.At(i).Resource().Attributes() + ras.Range(func(k string, v pcommon.Value) bool { + resourceTags[k] = v.AsString() + return true + }) + ilms := rms.At(i).ScopeMetrics() + for j := 0; j < ilms.Len(); j++ { + ms := ilms.At(j).Metrics() + for k := 0; k < ms.Len(); k++ { + m := ms.At(k) + fields, tags := ci.ConvertToFieldsAndTags(m, dc.logger) + maps.Copy(tags, resourceTags) + rm := gpuMetric{ + fields: fields, + tags: tags, + } + if !rm.HasTag(ci.MetricType) { + // force type to be Container to decorate with container level labels + rm.AddTag(ci.MetricType, ci.TypeGpuContainer) + } + dc.decorateMetrics([]*gpuMetric{&rm}) + dc.updateAttributes(m, rm) + if unit, ok := metricToUnit[m.Name()]; ok { + m.SetUnit(unit) + } + } + } + } + return dc.nextConsumer.ConsumeMetrics(ctx, md) +} + +type Decorator interface { + Decorate(stores.CIMetric) stores.CIMetric + Shutdown() error +} + +func (dc *decorateConsumer) decorateMetrics(metrics []*gpuMetric) []*gpuMetric { + var result []*gpuMetric + for _, m := range metrics { + // add tags for EKS + if dc.containerOrchestrator == ci.EKS { + out := dc.k8sDecorator.Decorate(m) + if out != nil { + result = append(result, out.(*gpuMetric)) + } + } + } + return result +} + +func (dc *decorateConsumer) updateAttributes(m pmetric.Metric, gm gpuMetric) { + if len(gm.tags) < 1 { + return + } + var dps pmetric.NumberDataPointSlice + switch m.Type() { + case pmetric.MetricTypeGauge: + dps = m.Gauge().DataPoints() + case pmetric.MetricTypeSum: + dps = m.Sum().DataPoints() + default: + dc.logger.Warn("Unsupported metric type", zap.String("metric", m.Name()), zap.String("type", m.Type().String())) + } + + if dps.Len() < 1 { + return + } + attrs := dps.At(0).Attributes() + for tk, tv := range gm.tags { + // type gets set with metrictransformer while duplicating metrics at different resource levels + if tk == ci.MetricType { + continue + } + attrs.PutStr(tk, tv) + } +} + +func (dc *decorateConsumer) Shutdown() error { + var errs error + + if dc.k8sDecorator != nil { + errs = errors.Join(errs, dc.k8sDecorator.Shutdown()) + } + return errs +} diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/decorator_test.go b/receiver/awscontainerinsightreceiver/internal/gpu/decorator_test.go new file mode 100644 index 000000000000..4cbcb44700b5 --- /dev/null +++ b/receiver/awscontainerinsightreceiver/internal/gpu/decorator_test.go @@ -0,0 +1,233 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +package gpu + +import ( + "context" + "testing" + + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "go.opentelemetry.io/collector/consumer" + "go.opentelemetry.io/collector/consumer/consumertest" + "go.opentelemetry.io/collector/pdata/pcommon" + "go.opentelemetry.io/collector/pdata/pmetric" + "go.uber.org/zap" +) + +var _ Decorator = (*MockK8sDecorator)(nil) + +type mockGpuMetric struct { + tags map[string]string + fields map[string]any +} + +func (m *mockGpuMetric) HasField(key string) bool { + return m.fields[key] != nil +} + +func (m *mockGpuMetric) AddField(key string, val any) { + m.fields[key] = val +} + +func (m *mockGpuMetric) GetField(key string) any { + return m.fields[key] +} + +func (m *mockGpuMetric) HasTag(key string) bool { + return m.tags[key] != "" +} + +func (m *mockGpuMetric) AddTag(key, val string) { + m.tags[key] = val +} + +func (m *mockGpuMetric) GetTag(key string) string { + return m.tags[key] +} + +func (m *mockGpuMetric) RemoveTag(key string) { + delete(m.tags, key) +} + +type MockK8sDecorator struct { +} + +func (m *MockK8sDecorator) Decorate(metric stores.CIMetric) stores.CIMetric { + return metric +} + +func (m *MockK8sDecorator) Shutdown() error { + return nil +} + +type mockNextConsumer struct { +} + +func (mc *mockNextConsumer) Capabilities() consumer.Capabilities { + return consumer.Capabilities{ + MutatesData: true, + } +} + +func (mc *mockNextConsumer) ConsumeMetrics(_ context.Context, md pmetric.Metrics) error { + return nil +} + +func TestConsumeMetrics(t *testing.T) { + logger, _ := zap.NewDevelopment() + dc := &decorateConsumer{ + containerOrchestrator: "EKS", + nextConsumer: consumertest.NewNop(), + k8sDecorator: &MockK8sDecorator{}, + logger: logger, + } + ctx := context.Background() + + testcases := map[string]struct { + metrics pmetric.Metrics + want pmetric.Metrics + shouldError bool + }{ + "empty": { + metrics: pmetric.NewMetrics(), + want: pmetric.NewMetrics(), + shouldError: false, + }, + "unit": { + metrics: generateMetrics(map[string]map[string]string{ + gpuUtil: { + "device": "test0", + }, + gpuMemUtil: { + "device": "test0", + }, + gpuMemTotal: { + "device": "test0", + }, + gpuMemUsed: { + "device": "test0", + }, + gpuPowerDraw: { + "device": "test0", + }, + gpuTemperature: { + "device": "test0", + }, + }), + want: generateMetrics(map[string]map[string]string{ + gpuUtil: { + "device": "test0", + "Unit": "Percent", + }, + gpuMemUtil: { + "device": "test0", + "Unit": "Percent", + }, + gpuMemTotal: { + "device": "test0", + "Unit": "Bytes", + }, + gpuMemUsed: { + "device": "test0", + "Unit": "Bytes", + }, + gpuPowerDraw: { + "device": "test0", + "Unit": "None", + }, + gpuTemperature: { + "device": "test0", + "Unit": "None", + }, + }), + shouldError: false, + }, + "noUnit": { + metrics: generateMetrics(map[string]map[string]string{ + "test": { + "device": "test0", + }, + }), + want: generateMetrics(map[string]map[string]string{ + "test": { + "device": "test0", + }, + }), + shouldError: false, + }, + "typeUnchanged": { + metrics: generateMetrics(map[string]map[string]string{ + gpuUtil: { + "device": "test0", + "Type": "TestType", + }, + }), + want: generateMetrics(map[string]map[string]string{ + gpuUtil: { + "device": "test0", + "Type": "TestType", + "Unit": "Percent", + }, + }), + shouldError: false, + }, + } + + for _, tc := range testcases { + err := dc.ConsumeMetrics(ctx, tc.metrics) + if tc.shouldError { + assert.Error(t, err) + return + } + require.NoError(t, err) + assert.Equal(t, tc.metrics.MetricCount(), tc.want.MetricCount()) + if tc.want.MetricCount() == 0 { + continue + } + actuals := tc.metrics.ResourceMetrics().At(0).ScopeMetrics().At(0).Metrics() + actuals.Sort(func(a, b pmetric.Metric) bool { + return a.Name() < b.Name() + }) + wants := tc.want.ResourceMetrics().At(0).ScopeMetrics().At(0).Metrics() + wants.Sort(func(a, b pmetric.Metric) bool { + return a.Name() < b.Name() + }) + for i := 0; i < wants.Len(); i++ { + actual := actuals.At(i) + want := wants.At(i) + assert.Equal(t, want.Name(), actual.Name()) + assert.Equal(t, want.Unit(), actual.Unit()) + actualAttrs := actual.Gauge().DataPoints().At(0).Attributes() + wantAttrs := want.Gauge().DataPoints().At(0).Attributes() + assert.Equal(t, wantAttrs.Len(), actualAttrs.Len()) + wantAttrs.Range(func(k string, v pcommon.Value) bool { + av, ok := actualAttrs.Get(k) + assert.True(t, ok) + assert.Equal(t, v, av) + return true + }) + } + } +} + +func generateMetrics(nameToDims map[string]map[string]string) pmetric.Metrics { + md := pmetric.NewMetrics() + ms := md.ResourceMetrics().AppendEmpty().ScopeMetrics().AppendEmpty().Metrics() + for name, dims := range nameToDims { + m := ms.AppendEmpty() + m.SetName(name) + gauge := m.SetEmptyGauge().DataPoints().AppendEmpty() + gauge.SetIntValue(10) + for k, v := range dims { + if k == "Unit" { + m.SetUnit(v) + continue + } + gauge.Attributes().PutStr(k, v) + } + } + return md +} diff --git a/receiver/awscontainerinsightreceiver/internal/stores/podstore.go b/receiver/awscontainerinsightreceiver/internal/stores/podstore.go index 41b182f181be..46296fc6de5b 100644 --- a/receiver/awscontainerinsightreceiver/internal/stores/podstore.go +++ b/receiver/awscontainerinsightreceiver/internal/stores/podstore.go @@ -483,7 +483,7 @@ func (p *PodStore) decorateMem(metric CIMetric, pod *corev1.Pod) { } func (p *PodStore) addStatus(metric CIMetric, pod *corev1.Pod) { - if metric.GetTag(ci.MetricType) == ci.TypePod { + if metric.GetTag(ci.MetricType) == ci.TypePod || metric.GetTag(ci.MetricType) == ci.TypeGpuPod { metric.AddTag(ci.PodStatus, string(pod.Status.Phase)) if p.includeEnhancedMetrics { @@ -510,7 +510,7 @@ func (p *PodStore) addStatus(metric CIMetric, pod *corev1.Pod) { } p.setPrevMeasurement(ci.TypePod, podKey, prevPodMeasurement{containersRestarts: curContainerRestarts}) } - } else if metric.GetTag(ci.MetricType) == ci.TypeContainer { + } else if metric.GetTag(ci.MetricType) == ci.TypeContainer || metric.GetTag(ci.MetricType) == ci.TypeGpuContainer { if containerName := metric.GetTag(ci.ContainerNamekey); containerName != "" { for _, containerStatus := range pod.Status.ContainerStatuses { if containerStatus.Name == containerName { diff --git a/receiver/awscontainerinsightreceiver/internal/stores/store.go b/receiver/awscontainerinsightreceiver/internal/stores/store.go index 266d80b7a723..6dd0a0a0ff04 100644 --- a/receiver/awscontainerinsightreceiver/internal/stores/store.go +++ b/receiver/awscontainerinsightreceiver/internal/stores/store.go @@ -10,13 +10,8 @@ import ( "time" "go.uber.org/zap" - - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/cadvisor" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors" ) -var _ cadvisor.Decorator = &K8sDecorator{} - // CIMetric represents the raw metric interface for container insights type CIMetric interface { HasField(key string) bool @@ -89,7 +84,7 @@ func NewK8sDecorator(ctx context.Context, tagService bool, prefFullPodName bool, return k, nil } -func (k *K8sDecorator) Decorate(metric *extractors.CAdvisorMetric) *extractors.CAdvisorMetric { +func (k *K8sDecorator) Decorate(metric CIMetric) CIMetric { kubernetesBlob := map[string]any{} for _, store := range k.stores { ok := store.Decorate(k.ctx, metric, kubernetesBlob) diff --git a/receiver/awscontainerinsightreceiver/internal/stores/utils.go b/receiver/awscontainerinsightreceiver/internal/stores/utils.go index c1db9f3f9884..8c66c07a8fbc 100644 --- a/receiver/awscontainerinsightreceiver/internal/stores/utils.go +++ b/receiver/awscontainerinsightreceiver/internal/stores/utils.go @@ -20,6 +20,7 @@ const ( // https://github.com/kubernetes/apimachinery/blob/master/pkg/util/rand/rand.go#L83 kubeAllowedStringAlphaNums = "bcdfghjklmnpqrstvwxz2456789" cronJobAllowedString = "0123456789" + gpuNvidiaKey = "nvidia.com/gpu" ) func createPodKeyFromMetaData(pod *corev1.Pod) string { @@ -121,6 +122,8 @@ func TagMetricSource(metric CIMetric) { sources = append(sources, []string{"cadvisor", "calculated"}...) case ci.TypeContainerDiskIO: sources = append(sources, []string{"cadvisor"}...) + case ci.TypeGpuContainer: + sources = append(sources, []string{"pod", "calculated"}...) } if len(sources) > 0 { diff --git a/receiver/awscontainerinsightreceiver/internal/stores/utils_test.go b/receiver/awscontainerinsightreceiver/internal/stores/utils_test.go index 397d325cc773..57d2c2134cec 100644 --- a/receiver/awscontainerinsightreceiver/internal/stores/utils_test.go +++ b/receiver/awscontainerinsightreceiver/internal/stores/utils_test.go @@ -175,6 +175,7 @@ func TestUtils_TagMetricSource(t *testing.T) { ci.TypeContainer, ci.TypeContainerFS, ci.TypeContainerDiskIO, + ci.TypeGpuContainer, } expectedSources := []string{ @@ -188,6 +189,7 @@ func TestUtils_TagMetricSource(t *testing.T) { "[\"cadvisor\",\"pod\",\"calculated\"]", "[\"cadvisor\",\"calculated\"]", "[\"cadvisor\"]", + "[\"pod\",\"calculated\"]", } for i, mtype := range types { tags := map[string]string{ diff --git a/receiver/awscontainerinsightreceiver/receiver.go b/receiver/awscontainerinsightreceiver/receiver.go index 149c9ff19948..e3b1b6b39a16 100644 --- a/receiver/awscontainerinsightreceiver/receiver.go +++ b/receiver/awscontainerinsightreceiver/receiver.go @@ -97,7 +97,7 @@ func (acir *awsContainerInsightReceiver) Start(ctx context.Context, host compone acir.settings.Logger.Debug("Unable to start kube apiserver prometheus scraper", zap.Error(err)) } - err = acir.initDcgmScraper(ctx, host, hostinfo) + err = acir.initDcgmScraper(ctx, host, hostinfo, k8sDecorator) if err != nil { acir.settings.Logger.Debug("Unable to start dcgm scraper", zap.Error(err)) } @@ -176,7 +176,7 @@ func (acir *awsContainerInsightReceiver) initPrometheusScraper(ctx context.Conte }) return err } -func (acir *awsContainerInsightReceiver) initDcgmScraper(ctx context.Context, host component.Host, hostinfo *hostInfo.Info) error { +func (acir *awsContainerInsightReceiver) initDcgmScraper(ctx context.Context, host component.Host, hostinfo *hostInfo.Info, decorator *stores.K8sDecorator) error { if !acir.config.EnableGpuMetric { return nil } @@ -188,6 +188,8 @@ func (acir *awsContainerInsightReceiver) initDcgmScraper(ctx context.Context, ho Consumer: acir.nextConsumer, Host: host, HostInfoProvider: hostinfo, + K8sDecorator: decorator, + Logger: acir.settings.Logger, }) return err } From 0b7c89538f2576d9681d29db0aedec3251f942cc Mon Sep 17 00:00:00 2001 From: Hyunsoo Kim Date: Tue, 27 Feb 2024 14:50:12 -0500 Subject: [PATCH 10/17] address comments --- internal/aws/containerinsight/utils.go | 8 ++--- .../internal/gpu/decorator.go | 33 ++++++++++--------- 2 files changed, 21 insertions(+), 20 deletions(-) diff --git a/internal/aws/containerinsight/utils.go b/internal/aws/containerinsight/utils.go index 18cf0662559b..bd6a0c862a09 100644 --- a/internal/aws/containerinsight/utils.go +++ b/internal/aws/containerinsight/utils.go @@ -175,7 +175,7 @@ func ConvertToFieldsAndTags(m pmetric.Metric, logger *zap.Logger) (map[string]an } // value is not needed for label decoration - fields[m.Name()] = 0 + fields[m.Name()] = nil var dps pmetric.NumberDataPointSlice switch m.Type() { @@ -188,12 +188,12 @@ func ConvertToFieldsAndTags(m pmetric.Metric, logger *zap.Logger) (map[string]an } // should support metrics with more than 1 datapoints? - if dps.Len() > 1 { - logger.Warn("Metric with more than 1 datapoint is not supported", zap.String("metric", m.Name()), zap.Int("datapoints", dps.Len())) + if dps.Len() == 0 || dps.Len() > 1 { + logger.Warn("Metric has either 0 or more than 1 datapoints", zap.String("metric", m.Name()), zap.Int("datapoints", dps.Len())) } attrs := dps.At(0).Attributes() attrs.Range(func(k string, v pcommon.Value) bool { - tags[k] = v.Str() + tags[k] = v.AsString() return true }) return fields, tags diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/decorator.go b/receiver/awscontainerinsightreceiver/internal/gpu/decorator.go index 4c106db52730..e4f65e113515 100644 --- a/receiver/awscontainerinsightreceiver/internal/gpu/decorator.go +++ b/receiver/awscontainerinsightreceiver/internal/gpu/decorator.go @@ -5,7 +5,6 @@ package gpu import ( "context" - "errors" ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" @@ -25,8 +24,6 @@ const ( gpuPowerDraw = "DCGM_FI_DEV_POWER_USAGE" ) -var _ stores.CIMetric = (*gpuMetric)(nil) - var metricToUnit = map[string]string{ gpuUtil: "Percent", gpuMemUtil: "Percent", @@ -43,7 +40,7 @@ type gpuMetric struct { tags map[string]string } -func newResourceMetric(mType string, logger *zap.Logger) *gpuMetric { +func newGpuMetric(mType string) *gpuMetric { metric := &gpuMetric{ fields: make(map[string]any), tags: make(map[string]string), @@ -52,6 +49,13 @@ func newResourceMetric(mType string, logger *zap.Logger) *gpuMetric { return metric } +func newGpuMetricWithData(mType string, fields map[string]any, tags map[string]string) *gpuMetric { + metric := newGpuMetric(mType) + metric.fields = fields + metric.tags = tags + return metric +} + func (gr *gpuMetric) GetTags() map[string]string { return gr.tags } @@ -129,15 +133,12 @@ func (dc *decorateConsumer) ConsumeMetrics(ctx context.Context, md pmetric.Metri m := ms.At(k) fields, tags := ci.ConvertToFieldsAndTags(m, dc.logger) maps.Copy(tags, resourceTags) - rm := gpuMetric{ - fields: fields, - tags: tags, - } + rm := newGpuMetricWithData(ci.TypeGpuContainer, fields, tags) if !rm.HasTag(ci.MetricType) { // force type to be Container to decorate with container level labels rm.AddTag(ci.MetricType, ci.TypeGpuContainer) } - dc.decorateMetrics([]*gpuMetric{&rm}) + dc.decorateMetrics([]*gpuMetric{rm}) dc.updateAttributes(m, rm) if unit, ok := metricToUnit[m.Name()]; ok { m.SetUnit(unit) @@ -167,8 +168,8 @@ func (dc *decorateConsumer) decorateMetrics(metrics []*gpuMetric) []*gpuMetric { return result } -func (dc *decorateConsumer) updateAttributes(m pmetric.Metric, gm gpuMetric) { - if len(gm.tags) < 1 { +func (dc *decorateConsumer) updateAttributes(m pmetric.Metric, gm *gpuMetric) { + if len(gm.tags) == 0 { return } var dps pmetric.NumberDataPointSlice @@ -181,7 +182,7 @@ func (dc *decorateConsumer) updateAttributes(m pmetric.Metric, gm gpuMetric) { dc.logger.Warn("Unsupported metric type", zap.String("metric", m.Name()), zap.String("type", m.Type().String())) } - if dps.Len() < 1 { + if dps.Len() == 0 { return } attrs := dps.At(0).Attributes() @@ -195,10 +196,10 @@ func (dc *decorateConsumer) updateAttributes(m pmetric.Metric, gm gpuMetric) { } func (dc *decorateConsumer) Shutdown() error { - var errs error - if dc.k8sDecorator != nil { - errs = errors.Join(errs, dc.k8sDecorator.Shutdown()) + return dc.k8sDecorator.Shutdown() } - return errs + return nil } + +var _ stores.CIMetric = (*gpuMetric)(nil) From bc8d26937a73fe7fdfddd79776ef71dbd0ae7173 Mon Sep 17 00:00:00 2001 From: Hyunsoo Kim Date: Tue, 27 Feb 2024 15:12:38 -0500 Subject: [PATCH 11/17] handle the case with no dp --- internal/aws/containerinsight/utils.go | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/internal/aws/containerinsight/utils.go b/internal/aws/containerinsight/utils.go index bd6a0c862a09..1c75501beace 100644 --- a/internal/aws/containerinsight/utils.go +++ b/internal/aws/containerinsight/utils.go @@ -191,11 +191,14 @@ func ConvertToFieldsAndTags(m pmetric.Metric, logger *zap.Logger) (map[string]an if dps.Len() == 0 || dps.Len() > 1 { logger.Warn("Metric has either 0 or more than 1 datapoints", zap.String("metric", m.Name()), zap.Int("datapoints", dps.Len())) } - attrs := dps.At(0).Attributes() - attrs.Range(func(k string, v pcommon.Value) bool { - tags[k] = v.AsString() - return true - }) + + if dps.Len() > 0 { + attrs := dps.At(0).Attributes() + attrs.Range(func(k string, v pcommon.Value) bool { + tags[k] = v.AsString() + return true + }) + } return fields, tags } From 43ba471f48a85903e07eeeaf7c3a45ddc23ef293 Mon Sep 17 00:00:00 2001 From: Hyunsoo Kim Date: Thu, 29 Feb 2024 08:58:40 -0500 Subject: [PATCH 12/17] consolidate CI metrics structs into single RawContainerInsights use constant variables use the same scrape configs in dcgm scraper test remove unnecessary attribute decoration for GPU metrics add dcgm as source for dim --- internal/aws/containerinsight/k8sconst.go | 1 + .../internal/cadvisor/cadvisor_linux.go | 8 +- .../internal/cadvisor/cadvisor_nolinux.go | 2 +- .../cadvisor/container_info_processor.go | 12 +- .../cadvisor/extractors/cpu_extractor.go | 20 +- .../cadvisor/extractors/cpu_extractor_test.go | 2 +- .../cadvisor/extractors/diskio_extractor.go | 16 +- .../extractors/diskio_extractor_test.go | 2 +- .../internal/cadvisor/extractors/extractor.go | 98 ++++----- .../extractors/extractor_helpers_test.go | 14 +- .../cadvisor/extractors/extractor_test.go | 70 +++--- .../cadvisor/extractors/fs_extractor.go | 24 +-- .../cadvisor/extractors/fs_extractor_test.go | 10 +- .../cadvisor/extractors/mem_extractor.go | 40 ++-- .../cadvisor/extractors/mem_extractor_test.go | 2 +- .../cadvisor/extractors/net_extractor.go | 14 +- .../cadvisor/extractors/net_extractor_test.go | 2 +- .../internal/gpu/dcgmscraper.go | 204 +++++++++--------- .../internal/gpu/dcgmscraper_test.go | 191 ++++++++-------- .../internal/gpu/decorator.go | 104 ++------- .../internal/stores/podstore.go | 4 +- .../internal/stores/utils.go | 3 +- 22 files changed, 382 insertions(+), 461 deletions(-) diff --git a/internal/aws/containerinsight/k8sconst.go b/internal/aws/containerinsight/k8sconst.go index d7060ccae583..a5425a52444e 100644 --- a/internal/aws/containerinsight/k8sconst.go +++ b/internal/aws/containerinsight/k8sconst.go @@ -20,6 +20,7 @@ const ( PodStatus = "pod_status" ContainerStatus = "container_status" + GpuDevice = "GpuDevice" ContainerStatusReason = "container_status_reason" ContainerLastTerminationReason = "container_last_termination_reason" diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_linux.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_linux.go index 4a4521479faa..0e4643e0a87e 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_linux.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_linux.go @@ -198,7 +198,7 @@ func (c *Cadvisor) addEbsVolumeInfo(tags map[string]string, ebsVolumeIdsUsedAsPV } } -func (c *Cadvisor) addECSMetrics(cadvisormetrics []*extractors.CAdvisorMetric) { +func (c *Cadvisor) addECSMetrics(cadvisormetrics []*extractors.RawContainerInsightsMetric) { if len(cadvisormetrics) == 0 { c.logger.Warn("cadvisor can't collect any metrics!") @@ -257,9 +257,9 @@ func addECSResources(tags map[string]string) { } } -func (c *Cadvisor) decorateMetrics(cadvisormetrics []*extractors.CAdvisorMetric) []*extractors.CAdvisorMetric { +func (c *Cadvisor) decorateMetrics(cadvisormetrics []*extractors.RawContainerInsightsMetric) []*extractors.RawContainerInsightsMetric { ebsVolumeIdsUsedAsPV := c.hostInfo.ExtractEbsIDsUsedByKubernetes() - var result []*extractors.CAdvisorMetric + var result []*extractors.RawContainerInsightsMetric for _, m := range cadvisormetrics { tags := m.GetTags() c.addEbsVolumeInfo(tags, ebsVolumeIdsUsedAsPV) @@ -308,7 +308,7 @@ func (c *Cadvisor) decorateMetrics(cadvisormetrics []*extractors.CAdvisorMetric) out := c.k8sDecorator.Decorate(m) if out != nil { - result = append(result, out.(*extractors.CAdvisorMetric)) + result = append(result, out.(*extractors.RawContainerInsightsMetric)) } } diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_nolinux.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_nolinux.go index 19999a2f3b95..3f1e23ee234a 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_nolinux.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_nolinux.go @@ -26,7 +26,7 @@ type Cadvisor struct { } type Decorator interface { - Decorate(*extractors.CAdvisorMetric) *extractors.CAdvisorMetric + Decorate(*extractors.RawContainerInsightsMetric) *extractors.RawContainerInsightsMetric Shutdown() error } diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/container_info_processor.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/container_info_processor.go index 2a39db9bde4b..1abd4fd00b87 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/container_info_processor.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/container_info_processor.go @@ -38,8 +38,8 @@ type podKey struct { namespace string } -func processContainers(cInfos []*cInfo.ContainerInfo, mInfo extractors.CPUMemInfoProvider, containerOrchestrator string, logger *zap.Logger) []*extractors.CAdvisorMetric { - var metrics []*extractors.CAdvisorMetric +func processContainers(cInfos []*cInfo.ContainerInfo, mInfo extractors.CPUMemInfoProvider, containerOrchestrator string, logger *zap.Logger) []*extractors.RawContainerInsightsMetric { + var metrics []*extractors.RawContainerInsightsMetric podKeys := make(map[string]podKey) // first iteration of container infos processes individual container info and @@ -88,8 +88,8 @@ func processContainers(cInfos []*cInfo.ContainerInfo, mInfo extractors.CPUMemInf } // processContainers get metrics for individual container and gather information for pod so we can look it up later. -func processContainer(info *cInfo.ContainerInfo, mInfo extractors.CPUMemInfoProvider, containerOrchestrator string, logger *zap.Logger) ([]*extractors.CAdvisorMetric, *podKey, error) { - var result []*extractors.CAdvisorMetric +func processContainer(info *cInfo.ContainerInfo, mInfo extractors.CPUMemInfoProvider, containerOrchestrator string, logger *zap.Logger) ([]*extractors.RawContainerInsightsMetric, *podKey, error) { + var result []*extractors.RawContainerInsightsMetric var pKey *podKey if isContainerInContainer(info.Name) { @@ -165,8 +165,8 @@ func processContainer(info *cInfo.ContainerInfo, mInfo extractors.CPUMemInfoProv return result, pKey, nil } -func processPod(info *cInfo.ContainerInfo, mInfo extractors.CPUMemInfoProvider, podKeys map[string]podKey, logger *zap.Logger) []*extractors.CAdvisorMetric { - var result []*extractors.CAdvisorMetric +func processPod(info *cInfo.ContainerInfo, mInfo extractors.CPUMemInfoProvider, podKeys map[string]podKey, logger *zap.Logger) []*extractors.RawContainerInsightsMetric { + var result []*extractors.RawContainerInsightsMetric if isContainerInContainer(info.Name) { logger.Debug("drop metric because it's nested container", zap.String("name", info.Name)) return result diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/cpu_extractor.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/cpu_extractor.go index e2b8851d6740..6c6956e3ea5f 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/cpu_extractor.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/cpu_extractor.go @@ -24,8 +24,8 @@ func (c *CPUMetricExtractor) HasValue(info *cInfo.ContainerInfo) bool { return info.Spec.HasCpu } -func (c *CPUMetricExtractor) GetValue(info *cInfo.ContainerInfo, mInfo CPUMemInfoProvider, containerType string) []*CAdvisorMetric { - var metrics []*CAdvisorMetric +func (c *CPUMetricExtractor) GetValue(info *cInfo.ContainerInfo, mInfo CPUMemInfoProvider, containerType string) []*RawContainerInsightsMetric { + var metrics []*RawContainerInsightsMetric // Skip infra container and handle node, pod, other containers in pod if containerType == ci.TypeInfraContainer { return metrics @@ -33,20 +33,20 @@ func (c *CPUMetricExtractor) GetValue(info *cInfo.ContainerInfo, mInfo CPUMemInf // When there is more than one stats point, always use the last one curStats := GetStats(info) - metric := newCadvisorMetric(containerType, c.logger) - metric.cgroupPath = info.Name + metric := NewRawContainerInsightsMetric(containerType, c.logger) + metric.ContainerName = info.Name multiplier := float64(decimalToMillicores) - assignRateValueToField(&c.rateCalculator, metric.fields, ci.MetricName(containerType, ci.CPUTotal), info.Name, float64(curStats.Cpu.Usage.Total), curStats.Timestamp, multiplier) - assignRateValueToField(&c.rateCalculator, metric.fields, ci.MetricName(containerType, ci.CPUUser), info.Name, float64(curStats.Cpu.Usage.User), curStats.Timestamp, multiplier) - assignRateValueToField(&c.rateCalculator, metric.fields, ci.MetricName(containerType, ci.CPUSystem), info.Name, float64(curStats.Cpu.Usage.System), curStats.Timestamp, multiplier) + assignRateValueToField(&c.rateCalculator, metric.Fields, ci.MetricName(containerType, ci.CPUTotal), info.Name, float64(curStats.Cpu.Usage.Total), curStats.Timestamp, multiplier) + assignRateValueToField(&c.rateCalculator, metric.Fields, ci.MetricName(containerType, ci.CPUUser), info.Name, float64(curStats.Cpu.Usage.User), curStats.Timestamp, multiplier) + assignRateValueToField(&c.rateCalculator, metric.Fields, ci.MetricName(containerType, ci.CPUSystem), info.Name, float64(curStats.Cpu.Usage.System), curStats.Timestamp, multiplier) numCores := mInfo.GetNumCores() - if metric.fields[ci.MetricName(containerType, ci.CPUTotal)] != nil && numCores != 0 { - metric.fields[ci.MetricName(containerType, ci.CPUUtilization)] = metric.fields[ci.MetricName(containerType, ci.CPUTotal)].(float64) / float64(numCores*decimalToMillicores) * 100 + if metric.Fields[ci.MetricName(containerType, ci.CPUTotal)] != nil && numCores != 0 { + metric.Fields[ci.MetricName(containerType, ci.CPUUtilization)] = metric.Fields[ci.MetricName(containerType, ci.CPUTotal)].(float64) / float64(numCores*decimalToMillicores) * 100 } if containerType == ci.TypeNode || containerType == ci.TypeInstance { - metric.fields[ci.MetricName(containerType, ci.CPULimit)] = numCores * decimalToMillicores + metric.Fields[ci.MetricName(containerType, ci.CPULimit)] = numCores * decimalToMillicores } metrics = append(metrics, metric) diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/cpu_extractor_test.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/cpu_extractor_test.go index a172918706e5..c8a6f52f55ae 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/cpu_extractor_test.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/cpu_extractor_test.go @@ -22,7 +22,7 @@ func TestCPUStats(t *testing.T) { containerType := containerinsight.TypeContainer extractor := NewCPUMetricExtractor(nil) - var cMetrics []*CAdvisorMetric + var cMetrics []*RawContainerInsightsMetric if extractor.HasValue(result[0]) { cMetrics = extractor.GetValue(result[0], MockCPUMemInfo, containerType) } diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/diskio_extractor.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/diskio_extractor.go index 384acf0e5e4f..d1ba1e19220e 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/diskio_extractor.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/diskio_extractor.go @@ -24,8 +24,8 @@ func (d *DiskIOMetricExtractor) HasValue(info *cInfo.ContainerInfo) bool { return info.Spec.HasDiskIo } -func (d *DiskIOMetricExtractor) GetValue(info *cInfo.ContainerInfo, _ CPUMemInfoProvider, containerType string) []*CAdvisorMetric { - var metrics []*CAdvisorMetric +func (d *DiskIOMetricExtractor) GetValue(info *cInfo.ContainerInfo, _ CPUMemInfoProvider, containerType string) []*RawContainerInsightsMetric { + var metrics []*RawContainerInsightsMetric if containerType != ci.TypeNode && containerType != ci.TypeInstance { return metrics } @@ -36,20 +36,20 @@ func (d *DiskIOMetricExtractor) GetValue(info *cInfo.ContainerInfo, _ CPUMemInfo return metrics } -func (d *DiskIOMetricExtractor) extractIoMetrics(curStatsSet []cInfo.PerDiskStats, namePrefix string, containerType string, infoName string, curTime time.Time) []*CAdvisorMetric { - var metrics []*CAdvisorMetric +func (d *DiskIOMetricExtractor) extractIoMetrics(curStatsSet []cInfo.PerDiskStats, namePrefix string, containerType string, infoName string, curTime time.Time) []*RawContainerInsightsMetric { + var metrics []*RawContainerInsightsMetric expectedKey := []string{ci.DiskIOAsync, ci.DiskIOSync, ci.DiskIORead, ci.DiskIOWrite, ci.DiskIOTotal} for _, cur := range curStatsSet { curDevName := devName(cur) - metric := newCadvisorMetric(getDiskIOMetricType(containerType, d.logger), d.logger) - metric.tags[ci.DiskDev] = curDevName + metric := NewRawContainerInsightsMetric(getDiskIOMetricType(containerType, d.logger), d.logger) + metric.Tags[ci.DiskDev] = curDevName for _, key := range expectedKey { if curVal, curOk := cur.Stats[key]; curOk { mname := ci.MetricName(containerType, ioMetricName(namePrefix, key)) - assignRateValueToField(&d.rateCalculator, metric.fields, mname, infoName, float64(curVal), curTime, float64(time.Second)) + assignRateValueToField(&d.rateCalculator, metric.Fields, mname, infoName, float64(curVal), curTime, float64(time.Second)) } } - if len(metric.fields) > 0 { + if len(metric.Fields) > 0 { metrics = append(metrics, metric) } } diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/diskio_extractor_test.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/diskio_extractor_test.go index f8492d8d0c85..a93c5206bf06 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/diskio_extractor_test.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/diskio_extractor_test.go @@ -21,7 +21,7 @@ func TestDiskIOStats(t *testing.T) { containerType := containerinsight.TypeNode extractor := NewDiskIOMetricExtractor(nil) - var cMetrics []*CAdvisorMetric + var cMetrics []*RawContainerInsightsMetric if extractor.HasValue(result[0]) { cMetrics = extractor.GetValue(result[0], nil, containerType) } diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor.go index a10c891117e4..9d6a6c9e8f9f 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor.go @@ -15,8 +15,6 @@ import ( awsmetrics "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/metrics" ) -var _ stores.CIMetric = (*CAdvisorMetric)(nil) - func GetStats(info *cinfo.ContainerInfo) *cinfo.ContainerStats { if len(info.Stats) == 0 { return nil @@ -32,88 +30,90 @@ type CPUMemInfoProvider interface { type MetricExtractor interface { HasValue(*cinfo.ContainerInfo) bool - GetValue(info *cinfo.ContainerInfo, mInfo CPUMemInfoProvider, containerType string) []*CAdvisorMetric + GetValue(info *cinfo.ContainerInfo, mInfo CPUMemInfoProvider, containerType string) []*RawContainerInsightsMetric Shutdown() error } -type CAdvisorMetric struct { +type RawContainerInsightsMetric struct { // source of the metric for debugging merge conflict - cgroupPath string + ContainerName string // key/value pairs that are typed and contain the metric (numerical) data - fields map[string]any + Fields map[string]any // key/value string pairs that are used to identify the metrics - tags map[string]string + Tags map[string]string - logger *zap.Logger + Logger *zap.Logger } -func newCadvisorMetric(mType string, logger *zap.Logger) *CAdvisorMetric { - metric := &CAdvisorMetric{ - fields: make(map[string]any), - tags: make(map[string]string), - logger: logger, +var _ stores.CIMetric = (*RawContainerInsightsMetric)(nil) + +func NewRawContainerInsightsMetric(mType string, logger *zap.Logger) *RawContainerInsightsMetric { + metric := &RawContainerInsightsMetric{ + Fields: make(map[string]any), + Tags: make(map[string]string), + Logger: logger, } - metric.tags[ci.MetricType] = mType + metric.Tags[ci.MetricType] = mType return metric } -func (c *CAdvisorMetric) GetTags() map[string]string { - return c.tags +func (c *RawContainerInsightsMetric) GetTags() map[string]string { + return c.Tags } -func (c *CAdvisorMetric) GetFields() map[string]any { - return c.fields +func (c *RawContainerInsightsMetric) GetFields() map[string]any { + return c.Fields } -func (c *CAdvisorMetric) GetMetricType() string { - return c.tags[ci.MetricType] +func (c *RawContainerInsightsMetric) GetMetricType() string { + return c.Tags[ci.MetricType] } -func (c *CAdvisorMetric) AddTags(tags map[string]string) { +func (c *RawContainerInsightsMetric) AddTags(tags map[string]string) { for k, v := range tags { - c.tags[k] = v + c.Tags[k] = v } } -func (c *CAdvisorMetric) HasField(key string) bool { - return c.fields[key] != nil +func (c *RawContainerInsightsMetric) HasField(key string) bool { + return c.Fields[key] != nil } -func (c *CAdvisorMetric) AddField(key string, val any) { - c.fields[key] = val +func (c *RawContainerInsightsMetric) AddField(key string, val any) { + c.Fields[key] = val } -func (c *CAdvisorMetric) GetField(key string) any { - return c.fields[key] +func (c *RawContainerInsightsMetric) GetField(key string) any { + return c.Fields[key] } -func (c *CAdvisorMetric) HasTag(key string) bool { - return c.tags[key] != "" +func (c *RawContainerInsightsMetric) HasTag(key string) bool { + return c.Tags[key] != "" } -func (c *CAdvisorMetric) AddTag(key, val string) { - c.tags[key] = val +func (c *RawContainerInsightsMetric) AddTag(key, val string) { + c.Tags[key] = val } -func (c *CAdvisorMetric) GetTag(key string) string { - return c.tags[key] +func (c *RawContainerInsightsMetric) GetTag(key string) string { + return c.Tags[key] } -func (c *CAdvisorMetric) RemoveTag(key string) { - delete(c.tags, key) +func (c *RawContainerInsightsMetric) RemoveTag(key string) { + delete(c.Tags, key) } -func (c *CAdvisorMetric) Merge(src *CAdvisorMetric) { - // If there is any conflict, keep the fields with earlier timestamp - for k, v := range src.fields { - if _, ok := c.fields[k]; ok { - c.logger.Debug(fmt.Sprintf("metric being merged has conflict in fields, src: %v, dest: %v \n", *src, *c)) - c.logger.Debug("metric being merged has conflict in fields", zap.String("src", src.cgroupPath), zap.String("dest", c.cgroupPath)) - if c.tags[ci.Timestamp] < src.tags[ci.Timestamp] { +func (c *RawContainerInsightsMetric) Merge(src *RawContainerInsightsMetric) { + // If there is any conflict, keep the Fields with earlier timestamp + for k, v := range src.Fields { + if _, ok := c.Fields[k]; ok { + c.Logger.Debug(fmt.Sprintf("metric being merged has conflict in Fields, src: %v, dest: %v \n", *src, *c)) + c.Logger.Debug("metric being merged has conflict in Fields", zap.String("src", src.ContainerName), zap.String("dest", c.ContainerName)) + if c.Tags[ci.Timestamp] < src.Tags[ci.Timestamp] { continue } } - c.fields[k] = v + c.Fields[k] = v } } @@ -139,9 +139,9 @@ func assignRateValueToField(rateCalculator *awsmetrics.MetricCalculator, fields } // MergeMetrics merges an array of cadvisor metrics based on common metric keys -func MergeMetrics(metrics []*CAdvisorMetric) []*CAdvisorMetric { - result := make([]*CAdvisorMetric, 0, len(metrics)) - metricMap := make(map[string]*CAdvisorMetric) +func MergeMetrics(metrics []*RawContainerInsightsMetric) []*RawContainerInsightsMetric { + result := make([]*RawContainerInsightsMetric, 0, len(metrics)) + metricMap := make(map[string]*RawContainerInsightsMetric) for _, metric := range metrics { if metricKey := getMetricKey(metric); metricKey != "" { if mergedMetric, ok := metricMap[metricKey]; ok { @@ -161,7 +161,7 @@ func MergeMetrics(metrics []*CAdvisorMetric) []*CAdvisorMetric { } // return MetricKey for merge-able metrics -func getMetricKey(metric *CAdvisorMetric) string { +func getMetricKey(metric *RawContainerInsightsMetric) string { metricType := metric.GetMetricType() var metricKey string switch metricType { @@ -176,7 +176,7 @@ func getMetricKey(metric *CAdvisorMetric) string { metricKey = fmt.Sprintf("metricType:%s,podId:%s", ci.TypePod, metric.GetTags()[ci.PodIDKey]) case ci.TypeContainer: // merge cpu, memory metric for type Container - metricKey = fmt.Sprintf("metricType:%s,podId:%s,containerName:%s", ci.TypeContainer, metric.GetTags()[ci.PodIDKey], metric.GetTags()[ci.ContainerNamekey]) + metricKey = fmt.Sprintf("metricType:%s,podId:%s,ContainerName:%s", ci.TypeContainer, metric.GetTags()[ci.PodIDKey], metric.GetTags()[ci.ContainerNamekey]) case ci.TypeInstanceDiskIO: // merge io_serviced, io_service_bytes for type InstanceDiskIO metricKey = fmt.Sprintf("metricType:%s,device:%s", ci.TypeInstanceDiskIO, metric.GetTags()[ci.DiskDev]) diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor_helpers_test.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor_helpers_test.go index b0f6985d102b..40a2c32b412b 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor_helpers_test.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor_helpers_test.go @@ -13,7 +13,7 @@ import ( func AssertContainsTaggedFloat( t *testing.T, - cadvisorMetric *CAdvisorMetric, + cadvisorMetric *RawContainerInsightsMetric, field string, expectedValue float64, delta float64, @@ -32,14 +32,14 @@ func AssertContainsTaggedFloat( } } msg := fmt.Sprintf( - "Could not find field \"%s\" with requested tags within %f of %f, Actual: %f", + "Could not find field \"%s\" with requested Tags within %f of %f, Actual: %f", field, delta, expectedValue, actualValue) assert.Fail(t, msg) } func AssertContainsTaggedInt( t *testing.T, - cadvisorMetric *CAdvisorMetric, + cadvisorMetric *RawContainerInsightsMetric, field string, expectedValue int64, ) { @@ -52,14 +52,14 @@ func AssertContainsTaggedInt( } } msg := fmt.Sprintf( - "Could not find field \"%s\" with requested tags with value: %v, Actual: %v", + "Could not find field \"%s\" with requested Tags with value: %v, Actual: %v", field, expectedValue, actualValue) assert.Fail(t, msg) } func AssertContainsTaggedUint( t *testing.T, - cadvisorMetric *CAdvisorMetric, + cadvisorMetric *RawContainerInsightsMetric, field string, expectedValue uint64, ) { @@ -72,14 +72,14 @@ func AssertContainsTaggedUint( } } msg := fmt.Sprintf( - "Could not find field \"%s\" with requested tags with value: %v, Actual: %v", + "Could not find field \"%s\" with requested Tags with value: %v, Actual: %v", field, expectedValue, actualValue) assert.Fail(t, msg) } func AssertContainsTaggedField( t *testing.T, - cadvisorMetric *CAdvisorMetric, + cadvisorMetric *RawContainerInsightsMetric, expectedFields map[string]any, expectedTags map[string]string, ) { diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor_test.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor_test.go index 79e1ce986820..3a240ed1c90b 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor_test.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor_test.go @@ -14,105 +14,105 @@ import ( ) func TestCAdvisorMetric_Merge(t *testing.T) { - src := &CAdvisorMetric{ - fields: map[string]any{"value1": 1, "value2": 2}, - tags: map[string]string{ci.Timestamp: "1586331559882"}, - logger: zap.NewNop(), + src := &RawContainerInsightsMetric{ + Fields: map[string]any{"value1": 1, "value2": 2}, + Tags: map[string]string{ci.Timestamp: "1586331559882"}, + Logger: zap.NewNop(), } - dest := &CAdvisorMetric{ - fields: map[string]any{"value1": 3, "value3": 3}, - tags: map[string]string{ci.Timestamp: "1586331559973"}, - logger: zap.NewNop(), + dest := &RawContainerInsightsMetric{ + Fields: map[string]any{"value1": 3, "value3": 3}, + Tags: map[string]string{ci.Timestamp: "1586331559973"}, + Logger: zap.NewNop(), } src.Merge(dest) - assert.Equal(t, 3, len(src.fields)) - assert.Equal(t, 1, src.fields["value1"].(int)) + assert.Equal(t, 3, len(src.Fields)) + assert.Equal(t, 1, src.Fields["value1"].(int)) } func TestGetMetricKey(t *testing.T) { - c := &CAdvisorMetric{ - tags: map[string]string{ + c := &RawContainerInsightsMetric{ + Tags: map[string]string{ ci.MetricType: ci.TypeInstance, }, } assert.Equal(t, "metricType:Instance", getMetricKey(c)) - c = &CAdvisorMetric{ - tags: map[string]string{ + c = &RawContainerInsightsMetric{ + Tags: map[string]string{ ci.MetricType: ci.TypeNode, }, } assert.Equal(t, "metricType:Node", getMetricKey(c)) - c = &CAdvisorMetric{ - tags: map[string]string{ + c = &RawContainerInsightsMetric{ + Tags: map[string]string{ ci.MetricType: ci.TypePod, ci.PodIDKey: "podID", }, } assert.Equal(t, "metricType:Pod,podId:podID", getMetricKey(c)) - c = &CAdvisorMetric{ - tags: map[string]string{ + c = &RawContainerInsightsMetric{ + Tags: map[string]string{ ci.MetricType: ci.TypeContainer, ci.PodIDKey: "podID", - ci.ContainerNamekey: "containerName", + ci.ContainerNamekey: "ContainerName", }, } - assert.Equal(t, "metricType:Container,podId:podID,containerName:containerName", getMetricKey(c)) + assert.Equal(t, "metricType:Container,podId:podID,ContainerName:ContainerName", getMetricKey(c)) - c = &CAdvisorMetric{ - tags: map[string]string{ + c = &RawContainerInsightsMetric{ + Tags: map[string]string{ ci.MetricType: ci.TypeInstanceDiskIO, ci.DiskDev: "/abc", }, } assert.Equal(t, "metricType:InstanceDiskIO,device:/abc", getMetricKey(c)) - c = &CAdvisorMetric{ - tags: map[string]string{ + c = &RawContainerInsightsMetric{ + Tags: map[string]string{ ci.MetricType: ci.TypeNodeDiskIO, ci.DiskDev: "/abc", }, } assert.Equal(t, "metricType:NodeDiskIO,device:/abc", getMetricKey(c)) - c = &CAdvisorMetric{} + c = &RawContainerInsightsMetric{} assert.Equal(t, "", getMetricKey(c)) } func TestMergeMetrics(t *testing.T) { - cpuMetrics := &CAdvisorMetric{ - fields: map[string]any{ + cpuMetrics := &RawContainerInsightsMetric{ + Fields: map[string]any{ "node_cpu_usage_total": float64(10), "node_cpu_usage_user": float64(10), }, - tags: map[string]string{ + Tags: map[string]string{ ci.MetricType: ci.TypeNode, }, } - memMetrics := &CAdvisorMetric{ - fields: map[string]any{ + memMetrics := &RawContainerInsightsMetric{ + Fields: map[string]any{ "node_memory_cache": uint(25645056), }, - tags: map[string]string{ + Tags: map[string]string{ ci.MetricType: ci.TypeNode, }, } - metrics := []*CAdvisorMetric{ + metrics := []*RawContainerInsightsMetric{ cpuMetrics, memMetrics, } - expected := &CAdvisorMetric{ - fields: map[string]any{ + expected := &RawContainerInsightsMetric{ + Fields: map[string]any{ "node_cpu_usage_total": float64(10), "node_cpu_usage_user": float64(10), "node_memory_cache": uint(25645056), }, - tags: map[string]string{ + Tags: map[string]string{ ci.MetricType: ci.TypeNode, }, } diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/fs_extractor.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/fs_extractor.go index 6e2f888b461b..3725fbe558df 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/fs_extractor.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/fs_extractor.go @@ -23,17 +23,17 @@ func (f *FileSystemMetricExtractor) HasValue(info *cinfo.ContainerInfo) bool { return info.Spec.HasFilesystem } -func (f *FileSystemMetricExtractor) GetValue(info *cinfo.ContainerInfo, _ CPUMemInfoProvider, containerType string) []*CAdvisorMetric { +func (f *FileSystemMetricExtractor) GetValue(info *cinfo.ContainerInfo, _ CPUMemInfoProvider, containerType string) []*RawContainerInsightsMetric { if containerType == ci.TypePod || containerType == ci.TypeInfraContainer { return nil } containerType = getFSMetricType(containerType, f.logger) stats := GetStats(info) - metrics := make([]*CAdvisorMetric, 0, len(stats.Filesystem)) + metrics := make([]*RawContainerInsightsMetric, 0, len(stats.Filesystem)) for _, v := range stats.Filesystem { - metric := newCadvisorMetric(containerType, f.logger) + metric := NewRawContainerInsightsMetric(containerType, f.logger) if v.Device == "" { continue } @@ -41,23 +41,23 @@ func (f *FileSystemMetricExtractor) GetValue(info *cinfo.ContainerInfo, _ CPUMem continue } - metric.tags[ci.DiskDev] = v.Device - metric.tags[ci.FSType] = v.Type + metric.Tags[ci.DiskDev] = v.Device + metric.Tags[ci.FSType] = v.Type - metric.fields[ci.MetricName(containerType, ci.FSUsage)] = v.Usage - metric.fields[ci.MetricName(containerType, ci.FSCapacity)] = v.Limit - metric.fields[ci.MetricName(containerType, ci.FSAvailable)] = v.Available + metric.Fields[ci.MetricName(containerType, ci.FSUsage)] = v.Usage + metric.Fields[ci.MetricName(containerType, ci.FSCapacity)] = v.Limit + metric.Fields[ci.MetricName(containerType, ci.FSAvailable)] = v.Available if v.Limit != 0 { - metric.fields[ci.MetricName(containerType, ci.FSUtilization)] = float64(v.Usage) / float64(v.Limit) * 100 + metric.Fields[ci.MetricName(containerType, ci.FSUtilization)] = float64(v.Usage) / float64(v.Limit) * 100 } if v.HasInodes { - metric.fields[ci.MetricName(containerType, ci.FSInodes)] = v.Inodes - metric.fields[ci.MetricName(containerType, ci.FSInodesfree)] = v.InodesFree + metric.Fields[ci.MetricName(containerType, ci.FSInodes)] = v.Inodes + metric.Fields[ci.MetricName(containerType, ci.FSInodesfree)] = v.InodesFree } - metric.cgroupPath = info.Name + metric.ContainerName = info.Name metrics = append(metrics, metric) } return metrics diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/fs_extractor_test.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/fs_extractor_test.go index 41dd6a57ef62..96157b818d28 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/fs_extractor_test.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/fs_extractor_test.go @@ -21,7 +21,7 @@ func TestFSStats(t *testing.T) { containerType := containerinsight.TypeContainer extractor := NewFileSystemMetricExtractor(nil) - var cMetrics []*CAdvisorMetric + var cMetrics []*RawContainerInsightsMetric if extractor.HasValue(result[0]) { cMetrics = extractor.GetValue(result[0], nil, containerType) } @@ -125,15 +125,15 @@ func TestFSStatsWithAllowList(t *testing.T) { containerType := containerinsight.TypeContainer extractor := NewFileSystemMetricExtractor(nil) - var cMetrics []*CAdvisorMetric + var cMetrics []*RawContainerInsightsMetric if extractor.HasValue(result[0]) { cMetrics = extractor.GetValue(result[0], nil, containerType) } // There are 3 valid device names which pass the allowlist in testAllowList json. assert.Equal(t, 3, len(cMetrics)) - assert.Equal(t, "tmpfs", cMetrics[0].tags["device"]) - assert.Equal(t, "/dev/xvda1", cMetrics[1].tags["device"]) - assert.Equal(t, "overlay", cMetrics[2].tags["device"]) + assert.Equal(t, "tmpfs", cMetrics[0].Tags["device"]) + assert.Equal(t, "/dev/xvda1", cMetrics[1].Tags["device"]) + assert.Equal(t, "overlay", cMetrics[2].Tags["device"]) } diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/mem_extractor.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/mem_extractor.go index fe7750c39093..a379763b7087 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/mem_extractor.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/mem_extractor.go @@ -22,45 +22,45 @@ func (m *MemMetricExtractor) HasValue(info *cinfo.ContainerInfo) bool { return info.Spec.HasMemory } -func (m *MemMetricExtractor) GetValue(info *cinfo.ContainerInfo, mInfo CPUMemInfoProvider, containerType string) []*CAdvisorMetric { - var metrics []*CAdvisorMetric +func (m *MemMetricExtractor) GetValue(info *cinfo.ContainerInfo, mInfo CPUMemInfoProvider, containerType string) []*RawContainerInsightsMetric { + var metrics []*RawContainerInsightsMetric if containerType == ci.TypeInfraContainer { return metrics } - metric := newCadvisorMetric(containerType, m.logger) - metric.cgroupPath = info.Name + metric := NewRawContainerInsightsMetric(containerType, m.logger) + metric.ContainerName = info.Name curStats := GetStats(info) - metric.fields[ci.MetricName(containerType, ci.MemUsage)] = curStats.Memory.Usage - metric.fields[ci.MetricName(containerType, ci.MemCache)] = curStats.Memory.Cache - metric.fields[ci.MetricName(containerType, ci.MemRss)] = curStats.Memory.RSS - metric.fields[ci.MetricName(containerType, ci.MemMaxusage)] = curStats.Memory.MaxUsage - metric.fields[ci.MetricName(containerType, ci.MemSwap)] = curStats.Memory.Swap - metric.fields[ci.MetricName(containerType, ci.MemFailcnt)] = curStats.Memory.Failcnt - metric.fields[ci.MetricName(containerType, ci.MemMappedfile)] = curStats.Memory.MappedFile - metric.fields[ci.MetricName(containerType, ci.MemWorkingset)] = curStats.Memory.WorkingSet + metric.Fields[ci.MetricName(containerType, ci.MemUsage)] = curStats.Memory.Usage + metric.Fields[ci.MetricName(containerType, ci.MemCache)] = curStats.Memory.Cache + metric.Fields[ci.MetricName(containerType, ci.MemRss)] = curStats.Memory.RSS + metric.Fields[ci.MetricName(containerType, ci.MemMaxusage)] = curStats.Memory.MaxUsage + metric.Fields[ci.MetricName(containerType, ci.MemSwap)] = curStats.Memory.Swap + metric.Fields[ci.MetricName(containerType, ci.MemFailcnt)] = curStats.Memory.Failcnt + metric.Fields[ci.MetricName(containerType, ci.MemMappedfile)] = curStats.Memory.MappedFile + metric.Fields[ci.MetricName(containerType, ci.MemWorkingset)] = curStats.Memory.WorkingSet multiplier := float64(time.Second) - assignRateValueToField(&m.rateCalculator, metric.fields, ci.MetricName(containerType, ci.MemPgfault), info.Name, + assignRateValueToField(&m.rateCalculator, metric.Fields, ci.MetricName(containerType, ci.MemPgfault), info.Name, float64(curStats.Memory.ContainerData.Pgfault), curStats.Timestamp, multiplier) - assignRateValueToField(&m.rateCalculator, metric.fields, ci.MetricName(containerType, ci.MemPgmajfault), info.Name, + assignRateValueToField(&m.rateCalculator, metric.Fields, ci.MetricName(containerType, ci.MemPgmajfault), info.Name, float64(curStats.Memory.ContainerData.Pgmajfault), curStats.Timestamp, multiplier) - assignRateValueToField(&m.rateCalculator, metric.fields, ci.MetricName(containerType, ci.MemHierarchicalPgfault), info.Name, + assignRateValueToField(&m.rateCalculator, metric.Fields, ci.MetricName(containerType, ci.MemHierarchicalPgfault), info.Name, float64(curStats.Memory.HierarchicalData.Pgfault), curStats.Timestamp, multiplier) - assignRateValueToField(&m.rateCalculator, metric.fields, ci.MetricName(containerType, ci.MemHierarchicalPgmajfault), info.Name, + assignRateValueToField(&m.rateCalculator, metric.Fields, ci.MetricName(containerType, ci.MemHierarchicalPgmajfault), info.Name, float64(curStats.Memory.HierarchicalData.Pgmajfault), curStats.Timestamp, multiplier) memoryFailuresTotal := curStats.Memory.ContainerData.Pgfault + curStats.Memory.ContainerData.Pgmajfault - assignRateValueToField(&m.rateCalculator, metric.fields, ci.MetricName(containerType, ci.MemFailuresTotal), info.Name, + assignRateValueToField(&m.rateCalculator, metric.Fields, ci.MetricName(containerType, ci.MemFailuresTotal), info.Name, float64(memoryFailuresTotal), curStats.Timestamp, multiplier) memoryCapacity := mInfo.GetMemoryCapacity() - if metric.fields[ci.MetricName(containerType, ci.MemWorkingset)] != nil && memoryCapacity != 0 { - metric.fields[ci.MetricName(containerType, ci.MemUtilization)] = float64(metric.fields[ci.MetricName(containerType, ci.MemWorkingset)].(uint64)) / float64(memoryCapacity) * 100 + if metric.Fields[ci.MetricName(containerType, ci.MemWorkingset)] != nil && memoryCapacity != 0 { + metric.Fields[ci.MetricName(containerType, ci.MemUtilization)] = float64(metric.Fields[ci.MetricName(containerType, ci.MemWorkingset)].(uint64)) / float64(memoryCapacity) * 100 } if containerType == ci.TypeNode || containerType == ci.TypeInstance { - metric.fields[ci.MetricName(containerType, ci.MemLimit)] = memoryCapacity + metric.Fields[ci.MetricName(containerType, ci.MemLimit)] = memoryCapacity } metrics = append(metrics, metric) diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/mem_extractor_test.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/mem_extractor_test.go index 8e5d2c564f06..ae5b1674e6f0 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/mem_extractor_test.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/mem_extractor_test.go @@ -20,7 +20,7 @@ func TestMemStats(t *testing.T) { containerType := containerinsight.TypeContainer extractor := NewMemMetricExtractor(nil) - var cMetrics []*CAdvisorMetric + var cMetrics []*RawContainerInsightsMetric if extractor.HasValue(result[0]) { cMetrics = extractor.GetValue(result[0], MockCPUMemInfo, containerType) } diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/net_extractor.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/net_extractor.go index 3affa24971fd..5a5af9f3f209 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/net_extractor.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/net_extractor.go @@ -30,7 +30,7 @@ func (n *NetMetricExtractor) HasValue(info *cinfo.ContainerInfo) bool { return info.Spec.HasNetwork } -func (n *NetMetricExtractor) GetValue(info *cinfo.ContainerInfo, _ CPUMemInfoProvider, containerType string) []*CAdvisorMetric { +func (n *NetMetricExtractor) GetValue(info *cinfo.ContainerInfo, _ CPUMemInfoProvider, containerType string) []*RawContainerInsightsMetric { // Just a protection here, there is no Container level Net metrics if containerType == ci.TypePod || containerType == ci.TypeContainer { @@ -47,7 +47,7 @@ func (n *NetMetricExtractor) GetValue(info *cinfo.ContainerInfo, _ CPUMemInfoPro // used for aggregation netIfceMetrics := make([]map[string]any, len(curIfceStats)) - metrics := make([]*CAdvisorMetric, len(curIfceStats)) + metrics := make([]*RawContainerInsightsMetric, len(curIfceStats)) for i, cur := range curIfceStats { mType := getNetMetricType(containerType, n.logger) @@ -70,10 +70,10 @@ func (n *NetMetricExtractor) GetValue(info *cinfo.ContainerInfo, _ CPUMemInfoPro netIfceMetrics[i] = netIfceMetric - metric := newCadvisorMetric(mType, n.logger) - metric.tags[ci.NetIfce] = cur.Name + metric := NewRawContainerInsightsMetric(mType, n.logger) + metric.Tags[ci.NetIfce] = cur.Name for k, v := range netIfceMetric { - metric.fields[ci.MetricName(mType, k)] = v + metric.Fields[ci.MetricName(mType, k)] = v } metrics[i] = metric @@ -81,9 +81,9 @@ func (n *NetMetricExtractor) GetValue(info *cinfo.ContainerInfo, _ CPUMemInfoPro aggregatedFields := ci.SumFields(netIfceMetrics) if len(aggregatedFields) > 0 { - metric := newCadvisorMetric(containerType, n.logger) + metric := NewRawContainerInsightsMetric(containerType, n.logger) for k, v := range aggregatedFields { - metric.fields[ci.MetricName(containerType, k)] = v + metric.Fields[ci.MetricName(containerType, k)] = v } metrics = append(metrics, metric) } diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/net_extractor_test.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/net_extractor_test.go index c5c067286690..8f1d31a8019a 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/net_extractor_test.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/net_extractor_test.go @@ -19,7 +19,7 @@ func TestNetStats(t *testing.T) { containerType := ci.TypeNode extractor := NewNetMetricExtractor(nil) - var cMetrics []*CAdvisorMetric + var cMetrics []*RawContainerInsightsMetric if extractor.HasValue(result[0]) { cMetrics = extractor.GetValue(result[0], nil, containerType) } diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go index 6f737d978b57..03ea107f0b1d 100644 --- a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go +++ b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go @@ -25,9 +25,11 @@ import ( ) const ( - caFile = "/var/run/secrets/kubernetes.io/serviceaccount/ca.crt" - collectionInterval = 60 * time.Second - jobName = "containerInsightsDCGMExporterScraper" + caFile = "/var/run/secrets/kubernetes.io/serviceaccount/ca.crt" + collectionInterval = 60 * time.Second + jobName = "containerInsightsDCGMExporterScraper" + scraperMetricsPath = "/metrics" + scraperK8sServiceSelector = "k8s-app=dcgm-exporter-service" ) type DcgmScraper struct { @@ -66,109 +68,9 @@ func NewDcgmScraper(opts DcgmScraperOpts) (*DcgmScraper, error) { return nil, errors.New("cluster name provider cannot be nil") } - scrapeConfig := &config.ScrapeConfig{ - ScrapeInterval: model.Duration(collectionInterval), - ScrapeTimeout: model.Duration(collectionInterval), - JobName: jobName, - Scheme: "http", - MetricsPath: "/metrics", - ServiceDiscoveryConfigs: discovery.Configs{ - &kubernetes.SDConfig{ - Role: kubernetes.RoleService, - NamespaceDiscovery: kubernetes.NamespaceDiscovery{ - IncludeOwnNamespace: true, - }, - Selectors: []kubernetes.SelectorConfig{ - { - Role: kubernetes.RoleService, - Label: "k8s-app=dcgm-exporter-service", - }, - }, - AttachMetadata: kubernetes.AttachMetadataConfig{ - Node: true, - }, - }, - }, - RelabelConfigs: []*relabel.Config{ - { - SourceLabels: model.LabelNames{"__address__"}, - Regex: relabel.MustNewRegexp("([^:]+)(?::\\d+)?"), - Replacement: "${1}:9400", - TargetLabel: "__address__", - Action: relabel.Replace, - }, - }, - MetricRelabelConfigs: []*relabel.Config{ - { - SourceLabels: model.LabelNames{"__name__"}, - Regex: relabel.MustNewRegexp("DCGM_.*"), - Action: relabel.Keep, - }, - { - SourceLabels: model.LabelNames{"Hostname"}, - TargetLabel: "NodeName", - Regex: relabel.MustNewRegexp("(.*)"), - Replacement: "${1}", - Action: relabel.Replace, - }, - { - SourceLabels: model.LabelNames{"namespace"}, - TargetLabel: "Namespace", - Regex: relabel.MustNewRegexp("(.*)"), - Replacement: "${1}", - Action: relabel.Replace, - }, - // hacky way to inject static values (clusterName & instanceId) to label set without additional processor - // relabel looks up an existing label then creates another label with given key (TargetLabel) and value (static) - { - SourceLabels: model.LabelNames{"namespace"}, - TargetLabel: "ClusterName", - Regex: relabel.MustNewRegexp(".*"), - Replacement: opts.HostInfoProvider.GetClusterName(), - Action: relabel.Replace, - }, - { - SourceLabels: model.LabelNames{"namespace"}, - TargetLabel: "InstanceId", - Regex: relabel.MustNewRegexp(".*"), - Replacement: opts.HostInfoProvider.GetInstanceID(), - Action: relabel.Replace, - }, - { - SourceLabels: model.LabelNames{"pod"}, - TargetLabel: "FullPodName", - Regex: relabel.MustNewRegexp("(.*)"), - Replacement: "${1}", - Action: relabel.Replace, - }, - // additional k8s podname for service name decoration - { - SourceLabels: model.LabelNames{"pod"}, - TargetLabel: "K8sPodName", - Regex: relabel.MustNewRegexp("(.*)"), - Replacement: "${1}", - Action: relabel.Replace, - }, - { - SourceLabels: model.LabelNames{"container"}, - TargetLabel: "ContainerName", - Regex: relabel.MustNewRegexp("(.*)"), - Replacement: "${1}", - Action: relabel.Replace, - }, - { - SourceLabels: model.LabelNames{"device"}, - TargetLabel: "GpuDevice", - Regex: relabel.MustNewRegexp("(.*)"), - Replacement: "${1}", - Action: relabel.Replace, - }, - }, - } - promConfig := prometheusreceiver.Config{ PrometheusConfig: &config.Config{ - ScrapeConfigs: []*config.ScrapeConfig{scrapeConfig}, + ScrapeConfigs: []*config.ScrapeConfig{getScraperConfig(opts.HostInfoProvider)}, }, } @@ -199,6 +101,100 @@ func NewDcgmScraper(opts DcgmScraperOpts) (*DcgmScraper, error) { }, nil } +func getScraperConfig(hostInfoProvider hostInfoProvider) *config.ScrapeConfig { + return &config.ScrapeConfig{ + ScrapeInterval: model.Duration(collectionInterval), + ScrapeTimeout: model.Duration(collectionInterval), + JobName: jobName, + Scheme: "http", + MetricsPath: scraperMetricsPath, + ServiceDiscoveryConfigs: discovery.Configs{ + &kubernetes.SDConfig{ + Role: kubernetes.RoleService, + NamespaceDiscovery: kubernetes.NamespaceDiscovery{ + IncludeOwnNamespace: true, + }, + Selectors: []kubernetes.SelectorConfig{ + { + Role: kubernetes.RoleService, + Label: scraperK8sServiceSelector, + }, + }, + }, + }, + MetricRelabelConfigs: getMetricRelabelConfig(hostInfoProvider), + } +} + +func getMetricRelabelConfig(hostInfoProvider hostInfoProvider) []*relabel.Config { + return []*relabel.Config{ + { + SourceLabels: model.LabelNames{"__name__"}, + Regex: relabel.MustNewRegexp("DCGM_.*"), + Action: relabel.Keep, + }, + { + SourceLabels: model.LabelNames{"Hostname"}, + TargetLabel: ci.NodeNameKey, + Regex: relabel.MustNewRegexp("(.*)"), + Replacement: "${1}", + Action: relabel.Replace, + }, + { + SourceLabels: model.LabelNames{"namespace"}, + TargetLabel: ci.K8sNamespace, + Regex: relabel.MustNewRegexp("(.*)"), + Replacement: "${1}", + Action: relabel.Replace, + }, + // hacky way to inject static values (clusterName & instanceId) to label set without additional processor + // relabel looks up an existing label then creates another label with given key (TargetLabel) and value (static) + { + SourceLabels: model.LabelNames{"namespace"}, + TargetLabel: ci.ClusterNameKey, + Regex: relabel.MustNewRegexp(".*"), + Replacement: hostInfoProvider.GetClusterName(), + Action: relabel.Replace, + }, + { + SourceLabels: model.LabelNames{"namespace"}, + TargetLabel: ci.InstanceID, + Regex: relabel.MustNewRegexp(".*"), + Replacement: hostInfoProvider.GetInstanceID(), + Action: relabel.Replace, + }, + { + SourceLabels: model.LabelNames{"pod"}, + TargetLabel: ci.FullPodNameKey, + Regex: relabel.MustNewRegexp("(.*)"), + Replacement: "${1}", + Action: relabel.Replace, + }, + // additional k8s podname for service name and k8s blob decoration + { + SourceLabels: model.LabelNames{"pod"}, + TargetLabel: ci.K8sPodNameKey, + Regex: relabel.MustNewRegexp("(.*)"), + Replacement: "${1}", + Action: relabel.Replace, + }, + { + SourceLabels: model.LabelNames{"container"}, + TargetLabel: ci.ContainerNamekey, + Regex: relabel.MustNewRegexp("(.*)"), + Replacement: "${1}", + Action: relabel.Replace, + }, + { + SourceLabels: model.LabelNames{"device"}, + TargetLabel: ci.GpuDevice, + Regex: relabel.MustNewRegexp("(.*)"), + Replacement: "${1}", + Action: relabel.Replace, + }, + } +} + func (ds *DcgmScraper) GetMetrics() []pmetric.Metrics { // This method will never return metrics because the metrics are collected by the scraper. // This method will ensure the scraper is running diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go index 9d7d244018d3..43f950d8ad53 100644 --- a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go +++ b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go @@ -5,17 +5,17 @@ package gpu import ( "context" - "fmt" "strings" "testing" + ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/mocks" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/prometheusreceiver" configutil "github.com/prometheus/common/config" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/discovery" - "github.com/prometheus/prometheus/model/relabel" "github.com/stretchr/testify/assert" "go.opentelemetry.io/collector/component/componenttest" "go.opentelemetry.io/collector/consumer" @@ -33,27 +33,39 @@ DCGM_FI_DEV_GPU_TEMP{gpu="0",UUID="uuid",device="nvidia0",modelName="NVIDIA A10G DCGM_FI_DEV_GPU_UTIL{gpu="0",UUID="uuid",device="nvidia0",modelName="NVIDIA A10G",Hostname="hostname",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="fullname-hash"} 100 ` -const dummyInstanceId = "i-0000000000" +const ( + dummyInstanceId = "i-0000000000" + dummyClusterName = "cluster-name" +) type mockHostInfoProvider struct { } func (m mockHostInfoProvider) GetClusterName() string { - return "cluster-name" + return dummyClusterName } func (m mockHostInfoProvider) GetInstanceID() string { return dummyInstanceId } +type mockDecorator struct { +} + +func (m mockDecorator) Decorate(metric stores.CIMetric) stores.CIMetric { + return metric +} + +func (m mockDecorator) Shutdown() error { + return nil +} + type mockConsumer struct { - t *testing.T - up *bool - gpuTemp *bool - gpuUtil *bool - relabeled *bool - podNameParsed *bool - instanceId *bool + t *testing.T + expected map[string]struct { + value float64 + labels map[string]string + } } func (m mockConsumer) Capabilities() consumer.Capabilities { @@ -65,29 +77,25 @@ func (m mockConsumer) Capabilities() consumer.Capabilities { func (m mockConsumer) ConsumeMetrics(_ context.Context, md pmetric.Metrics) error { assert.Equal(m.t, 1, md.ResourceMetrics().Len()) + scrapedMetricCnt := 0 scopeMetrics := md.ResourceMetrics().At(0).ScopeMetrics().At(0).Metrics() for i := 0; i < scopeMetrics.Len(); i++ { metric := scopeMetrics.At(i) - if metric.Name() == "DCGM_FI_DEV_GPU_UTIL" { - assert.Equal(m.t, float64(100), metric.Gauge().DataPoints().At(0).DoubleValue()) - *m.gpuUtil = true - instanceId, _ := metric.Gauge().DataPoints().At(0).Attributes().Get("InstanceId") - *m.instanceId = instanceId.Str() == dummyInstanceId + // skip prometheus metadata metrics including "up" + if !strings.HasPrefix(metric.Name(), "DCGM") { + continue } - if metric.Name() == "DCGM_FI_DEV_GPU_TEMP" { - *m.gpuTemp = true - fullPodName, relabeled := metric.Gauge().DataPoints().At(0).Attributes().Get("FullPodName") - splits := strings.Split(fullPodName.Str(), "-") - podName, _ := metric.Gauge().DataPoints().At(0).Attributes().Get("PodName") - *m.podNameParsed = podName.Str() == splits[0] - *m.relabeled = relabeled - } - if metric.Name() == "up" { - assert.Equal(m.t, float64(1), metric.Gauge().DataPoints().At(0).DoubleValue()) - *m.up = true + metadata, ok := m.expected[metric.Name()] + assert.True(m.t, ok) + assert.Equal(m.t, metadata.value, metric.Gauge().DataPoints().At(0).DoubleValue()) + for k, v := range metadata.labels { + lv, found := metric.Gauge().DataPoints().At(0).Attributes().Get(k) + assert.True(m.t, found) + assert.Equal(m.t, v, lv.AsString()) } + scrapedMetricCnt += 1 } - + assert.Equal(m.t, len(m.expected), scrapedMetricCnt) return nil } @@ -128,22 +136,44 @@ func TestNewDcgmScraperBadInputs(t *testing.T) { } func TestNewDcgmScraperEndToEnd(t *testing.T) { - - upPtr := false - gpuTemp := false - gpuUtil := false - relabeledPod := false - podNameParsed := false - instanceId := false - + expected := map[string]struct { + value float64 + labels map[string]string + }{ + "DCGM_FI_DEV_GPU_TEMP": { + value: 65, + labels: map[string]string{ + ci.NodeNameKey: "hostname", + ci.K8sNamespace: "kube-system", + ci.ClusterNameKey: dummyClusterName, + ci.InstanceID: dummyInstanceId, + ci.FullPodNameKey: "fullname-hash", + ci.K8sPodNameKey: "fullname-hash", + ci.ContainerNamekey: "main", + ci.GpuDevice: "nvidia0", + }, + }, + "DCGM_FI_DEV_GPU_UTIL": { + value: 100, + labels: map[string]string{ + ci.NodeNameKey: "hostname", + ci.K8sNamespace: "kube-system", + ci.ClusterNameKey: dummyClusterName, + ci.InstanceID: dummyInstanceId, + ci.FullPodNameKey: "fullname-hash", + ci.K8sPodNameKey: "fullname-hash", + ci.ContainerNamekey: "main", + ci.GpuDevice: "nvidia0", + }, + }, + "up": { + value: 1, + labels: map[string]string{}, + }, + } consumer := mockConsumer{ - t: t, - up: &upPtr, - gpuTemp: &gpuTemp, - gpuUtil: &gpuUtil, - relabeled: &relabeledPod, - podNameParsed: &podNameParsed, - instanceId: &instanceId, + t: t, + expected: expected, } settings := componenttest.NewNopTelemetrySettings() @@ -155,6 +185,7 @@ func TestNewDcgmScraperEndToEnd(t *testing.T) { Consumer: mockConsumer{}, Host: componenttest.NewNopHost(), HostInfoProvider: mockHostInfoProvider{}, + K8sDecorator: mockDecorator{}, }) assert.NoError(t, err) assert.Equal(t, mockHostInfoProvider{}, scraper.hostInfoProvider) @@ -173,62 +204,27 @@ func TestNewDcgmScraperEndToEnd(t *testing.T) { mp, cfg, err := mocks.SetupMockPrometheus(targets...) assert.NoError(t, err) - split := strings.Split(mp.Srv.URL, "http://") - - scrapeConfig := &config.ScrapeConfig{ - HTTPClientConfig: configutil.HTTPClientConfig{ - TLSConfig: configutil.TLSConfig{ - InsecureSkipVerify: true, - }, + scrapeConfig := getScraperConfig(scraper.hostInfoProvider) + scrapeConfig.ScrapeInterval = cfg.ScrapeConfigs[0].ScrapeInterval + scrapeConfig.ScrapeTimeout = cfg.ScrapeConfigs[0].ScrapeInterval + scrapeConfig.Scheme = "http" + scrapeConfig.MetricsPath = cfg.ScrapeConfigs[0].MetricsPath + scrapeConfig.HTTPClientConfig = configutil.HTTPClientConfig{ + TLSConfig: configutil.TLSConfig{ + InsecureSkipVerify: true, }, - ScrapeInterval: cfg.ScrapeConfigs[0].ScrapeInterval, - ScrapeTimeout: cfg.ScrapeConfigs[0].ScrapeInterval, - JobName: fmt.Sprintf("%s/%s", jobName, cfg.ScrapeConfigs[0].MetricsPath), - HonorTimestamps: true, - Scheme: "http", - MetricsPath: cfg.ScrapeConfigs[0].MetricsPath, - ServiceDiscoveryConfigs: discovery.Configs{ - // using dummy static config to avoid service discovery initialization - &discovery.StaticConfig{ - { - Targets: []model.LabelSet{ - { - model.AddressLabel: model.LabelValue(split[1]), - }, + } + scrapeConfig.ServiceDiscoveryConfigs = discovery.Configs{ + // using dummy static config to avoid service discovery initialization + &discovery.StaticConfig{ + { + Targets: []model.LabelSet{ + { + model.AddressLabel: model.LabelValue(strings.Split(mp.Srv.URL, "http://")[1]), }, }, }, }, - RelabelConfigs: []*relabel.Config{}, - MetricRelabelConfigs: []*relabel.Config{ - { - SourceLabels: model.LabelNames{"__name__"}, - Regex: relabel.MustNewRegexp("DCGM_.*"), - Action: relabel.Keep, - }, - // test hack to inject cluster name as label - { - SourceLabels: model.LabelNames{"namespace"}, - TargetLabel: "InstanceId", - Regex: relabel.MustNewRegexp("(.*)"), - Replacement: scraper.hostInfoProvider.GetInstanceID(), - Action: relabel.Replace, - }, - { - SourceLabels: model.LabelNames{"pod"}, - TargetLabel: "FullPodName", - Regex: relabel.MustNewRegexp("(.*)"), - Replacement: "${1}", - Action: relabel.Replace, - }, - { - SourceLabels: model.LabelNames{"pod"}, - TargetLabel: "PodName", - Regex: relabel.MustNewRegexp("(.+)-(.+)"), - Replacement: "${1}", - Action: relabel.Replace, - }, - }, } promConfig := prometheusreceiver.Config{ @@ -256,13 +252,6 @@ func TestNewDcgmScraperEndToEnd(t *testing.T) { // wait for 2 scrapes, one initiated by us, another by the new scraper process mp.Wg.Wait() mp.Wg.Wait() - - assert.True(t, *consumer.up) - assert.True(t, *consumer.gpuTemp) - assert.True(t, *consumer.gpuUtil) - assert.True(t, *consumer.relabeled) - assert.True(t, *consumer.podNameParsed) - assert.True(t, *consumer.instanceId) } func TestDcgmScraperJobName(t *testing.T) { diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/decorator.go b/receiver/awscontainerinsightreceiver/internal/gpu/decorator.go index e4f65e113515..cec7677a7291 100644 --- a/receiver/awscontainerinsightreceiver/internal/gpu/decorator.go +++ b/receiver/awscontainerinsightreceiver/internal/gpu/decorator.go @@ -7,12 +7,12 @@ import ( "context" ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "go.opentelemetry.io/collector/consumer" "go.opentelemetry.io/collector/pdata/pcommon" "go.opentelemetry.io/collector/pdata/pmetric" "go.uber.org/zap" - "golang.org/x/exp/maps" ) const ( @@ -33,75 +33,6 @@ var metricToUnit = map[string]string{ gpuPowerDraw: "None", } -type gpuMetric struct { - // key/value pairs that are typed and contain the metric (numerical) data - fields map[string]any - // key/value string pairs that are used to identify the metrics - tags map[string]string -} - -func newGpuMetric(mType string) *gpuMetric { - metric := &gpuMetric{ - fields: make(map[string]any), - tags: make(map[string]string), - } - metric.tags[ci.MetricType] = mType - return metric -} - -func newGpuMetricWithData(mType string, fields map[string]any, tags map[string]string) *gpuMetric { - metric := newGpuMetric(mType) - metric.fields = fields - metric.tags = tags - return metric -} - -func (gr *gpuMetric) GetTags() map[string]string { - return gr.tags -} - -func (gr *gpuMetric) GetFields() map[string]any { - return gr.fields -} - -func (gr *gpuMetric) GetMetricType() string { - return gr.tags[ci.MetricType] -} - -func (gr *gpuMetric) AddTags(tags map[string]string) { - for k, v := range tags { - gr.tags[k] = v - } -} - -func (gr *gpuMetric) HasField(key string) bool { - return gr.fields[key] != nil -} - -func (gr *gpuMetric) AddField(key string, val any) { - gr.fields[key] = val -} - -func (gr *gpuMetric) GetField(key string) any { - return gr.fields[key] -} - -func (gr *gpuMetric) HasTag(key string) bool { - return gr.tags[key] != "" -} - -func (gr *gpuMetric) AddTag(key, val string) { - gr.tags[key] = val -} - -func (gr *gpuMetric) GetTag(key string) string { - return gr.tags[key] -} - -func (gr *gpuMetric) RemoveTag(key string) { - delete(gr.tags, key) -} - // GPU decorator acts as an interceptor of metrics before the scraper sends them to the next designated consumer type decorateConsumer struct { containerOrchestrator string @@ -132,14 +63,21 @@ func (dc *decorateConsumer) ConsumeMetrics(ctx context.Context, md pmetric.Metri for k := 0; k < ms.Len(); k++ { m := ms.At(k) fields, tags := ci.ConvertToFieldsAndTags(m, dc.logger) - maps.Copy(tags, resourceTags) - rm := newGpuMetricWithData(ci.TypeGpuContainer, fields, tags) - if !rm.HasTag(ci.MetricType) { + // copy down resource metrics only when it's missing at datapoint + for rtk, rtv := range resourceTags { + if _, ok := tags[rtk]; !ok { + tags[rtk] = rtv + } + } + cim := extractors.NewRawContainerInsightsMetric(ci.TypeGpuContainer, dc.logger) + cim.Fields = fields + cim.Tags = tags + if !cim.HasTag(ci.MetricType) { // force type to be Container to decorate with container level labels - rm.AddTag(ci.MetricType, ci.TypeGpuContainer) + cim.AddTag(ci.MetricType, ci.TypeGpuContainer) } - dc.decorateMetrics([]*gpuMetric{rm}) - dc.updateAttributes(m, rm) + dc.decorateMetrics([]*extractors.RawContainerInsightsMetric{cim}) + dc.updateAttributes(m, cim) if unit, ok := metricToUnit[m.Name()]; ok { m.SetUnit(unit) } @@ -154,22 +92,22 @@ type Decorator interface { Shutdown() error } -func (dc *decorateConsumer) decorateMetrics(metrics []*gpuMetric) []*gpuMetric { - var result []*gpuMetric +func (dc *decorateConsumer) decorateMetrics(metrics []*extractors.RawContainerInsightsMetric) []*extractors.RawContainerInsightsMetric { + var result []*extractors.RawContainerInsightsMetric for _, m := range metrics { // add tags for EKS if dc.containerOrchestrator == ci.EKS { out := dc.k8sDecorator.Decorate(m) if out != nil { - result = append(result, out.(*gpuMetric)) + result = append(result, out.(*extractors.RawContainerInsightsMetric)) } } } return result } -func (dc *decorateConsumer) updateAttributes(m pmetric.Metric, gm *gpuMetric) { - if len(gm.tags) == 0 { +func (dc *decorateConsumer) updateAttributes(m pmetric.Metric, cim *extractors.RawContainerInsightsMetric) { + if len(cim.Tags) == 0 { return } var dps pmetric.NumberDataPointSlice @@ -186,7 +124,7 @@ func (dc *decorateConsumer) updateAttributes(m pmetric.Metric, gm *gpuMetric) { return } attrs := dps.At(0).Attributes() - for tk, tv := range gm.tags { + for tk, tv := range cim.Tags { // type gets set with metrictransformer while duplicating metrics at different resource levels if tk == ci.MetricType { continue @@ -201,5 +139,3 @@ func (dc *decorateConsumer) Shutdown() error { } return nil } - -var _ stores.CIMetric = (*gpuMetric)(nil) diff --git a/receiver/awscontainerinsightreceiver/internal/stores/podstore.go b/receiver/awscontainerinsightreceiver/internal/stores/podstore.go index 46296fc6de5b..41b182f181be 100644 --- a/receiver/awscontainerinsightreceiver/internal/stores/podstore.go +++ b/receiver/awscontainerinsightreceiver/internal/stores/podstore.go @@ -483,7 +483,7 @@ func (p *PodStore) decorateMem(metric CIMetric, pod *corev1.Pod) { } func (p *PodStore) addStatus(metric CIMetric, pod *corev1.Pod) { - if metric.GetTag(ci.MetricType) == ci.TypePod || metric.GetTag(ci.MetricType) == ci.TypeGpuPod { + if metric.GetTag(ci.MetricType) == ci.TypePod { metric.AddTag(ci.PodStatus, string(pod.Status.Phase)) if p.includeEnhancedMetrics { @@ -510,7 +510,7 @@ func (p *PodStore) addStatus(metric CIMetric, pod *corev1.Pod) { } p.setPrevMeasurement(ci.TypePod, podKey, prevPodMeasurement{containersRestarts: curContainerRestarts}) } - } else if metric.GetTag(ci.MetricType) == ci.TypeContainer || metric.GetTag(ci.MetricType) == ci.TypeGpuContainer { + } else if metric.GetTag(ci.MetricType) == ci.TypeContainer { if containerName := metric.GetTag(ci.ContainerNamekey); containerName != "" { for _, containerStatus := range pod.Status.ContainerStatuses { if containerStatus.Name == containerName { diff --git a/receiver/awscontainerinsightreceiver/internal/stores/utils.go b/receiver/awscontainerinsightreceiver/internal/stores/utils.go index 8c66c07a8fbc..f412b3998880 100644 --- a/receiver/awscontainerinsightreceiver/internal/stores/utils.go +++ b/receiver/awscontainerinsightreceiver/internal/stores/utils.go @@ -20,7 +20,6 @@ const ( // https://github.com/kubernetes/apimachinery/blob/master/pkg/util/rand/rand.go#L83 kubeAllowedStringAlphaNums = "bcdfghjklmnpqrstvwxz2456789" cronJobAllowedString = "0123456789" - gpuNvidiaKey = "nvidia.com/gpu" ) func createPodKeyFromMetaData(pod *corev1.Pod) string { @@ -123,7 +122,7 @@ func TagMetricSource(metric CIMetric) { case ci.TypeContainerDiskIO: sources = append(sources, []string{"cadvisor"}...) case ci.TypeGpuContainer: - sources = append(sources, []string{"pod", "calculated"}...) + sources = append(sources, []string{"dcgm", "pod", "calculated"}...) } if len(sources) > 0 { From 5a4f394ca40b86768b2eb79703f0d949baf909e3 Mon Sep 17 00:00:00 2001 From: Hyunsoo Kim Date: Thu, 29 Feb 2024 09:04:32 -0500 Subject: [PATCH 13/17] update feature toggle flag name --- receiver/awscontainerinsightreceiver/config.go | 4 ++-- receiver/awscontainerinsightreceiver/receiver.go | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/receiver/awscontainerinsightreceiver/config.go b/receiver/awscontainerinsightreceiver/config.go index df74242e20f7..fd26bfcd453d 100644 --- a/receiver/awscontainerinsightreceiver/config.go +++ b/receiver/awscontainerinsightreceiver/config.go @@ -58,6 +58,6 @@ type Config struct { // The default value is false. EnableControlPlaneMetrics bool `mapstructure:"enable_control_plane_metrics"` - // EnableGpuMetric toggles GPU monitoring where metrics are scraped from vendor specific sources - EnableGpuMetric bool `mapstructure:"gpu_metrics"` + // EnableAcceleratedComputingMetric toggles GPU monitoring where metrics are scraped from vendor specific sources + EnableAcceleratedComputingMetric bool `mapstructure:"accelerated_computing_metrics"` } diff --git a/receiver/awscontainerinsightreceiver/receiver.go b/receiver/awscontainerinsightreceiver/receiver.go index e3b1b6b39a16..e666ab2d1305 100644 --- a/receiver/awscontainerinsightreceiver/receiver.go +++ b/receiver/awscontainerinsightreceiver/receiver.go @@ -177,7 +177,7 @@ func (acir *awsContainerInsightReceiver) initPrometheusScraper(ctx context.Conte return err } func (acir *awsContainerInsightReceiver) initDcgmScraper(ctx context.Context, host component.Host, hostinfo *hostInfo.Info, decorator *stores.K8sDecorator) error { - if !acir.config.EnableGpuMetric { + if !acir.config.EnableAcceleratedComputingMetric { return nil } From 3ee43f68a71bb37ca14e417b54990cb8c35aaafc Mon Sep 17 00:00:00 2001 From: Hyunsoo Kim Date: Fri, 1 Mar 2024 09:41:18 -0500 Subject: [PATCH 14/17] rename k8s attributes vars with Attribute prefix support multiple datapoints when converting pmetric into fields/tags move RawContainerInsightsMetric struct to stores package update feature toggle flag var name to match json key --- internal/aws/containerinsight/k8sconst.go | 19 ++- internal/aws/containerinsight/utils.go | 33 ++++-- .../awscontainerinsightreceiver/config.go | 4 +- .../cadvisor/container_info_processor.go | 16 +-- .../cadvisor/extractors/cpu_extractor.go | 7 +- .../cadvisor/extractors/cpu_extractor_test.go | 3 +- .../cadvisor/extractors/diskio_extractor.go | 11 +- .../extractors/diskio_extractor_test.go | 3 +- .../internal/cadvisor/extractors/extractor.go | 103 ++-------------- .../extractors/extractor_helpers_test.go | 15 +-- .../cadvisor/extractors/extractor_test.go | 39 +++--- .../cadvisor/extractors/fs_extractor.go | 7 +- .../cadvisor/extractors/fs_extractor_test.go | 5 +- .../cadvisor/extractors/mem_extractor.go | 7 +- .../cadvisor/extractors/mem_extractor_test.go | 3 +- .../cadvisor/extractors/net_extractor.go | 9 +- .../cadvisor/extractors/net_extractor_test.go | 3 +- .../internal/gpu/dcgmscraper.go | 10 +- .../internal/gpu/dcgmscraper_test.go | 46 ++++---- .../internal/gpu/decorator.go | 65 +++++----- .../internal/gpu/decorator_test.go | 35 +----- .../internal/k8sapiserver/k8sapiserver.go | 96 +++++++-------- .../k8sapiserver/k8sapiserver_test.go | 22 ++-- .../k8sapiserver/prometheus_scraper_test.go | 2 + .../internal/stores/podstore.go | 20 ++-- .../internal/stores/podstore_test.go | 110 ++++++++--------- .../internal/stores/servicestore.go | 2 +- .../internal/stores/servicestore_test.go | 6 +- .../internal/stores/utils.go | 111 ++++++++++++++++-- .../internal/stores/utils_test.go | 46 ++++---- .../awscontainerinsightreceiver/receiver.go | 2 +- 31 files changed, 430 insertions(+), 430 deletions(-) diff --git a/internal/aws/containerinsight/k8sconst.go b/internal/aws/containerinsight/k8sconst.go index a5425a52444e..1b03eb31710f 100644 --- a/internal/aws/containerinsight/k8sconst.go +++ b/internal/aws/containerinsight/k8sconst.go @@ -8,19 +8,18 @@ const ( EKS = "eks" KubeSecurePort = "10250" - // attribute names - Kubernetes = "kubernetes" - K8sNamespace = "Namespace" - PodIDKey = "PodId" - PodNameKey = "PodName" - FullPodNameKey = "FullPodName" - K8sPodNameKey = "K8sPodName" - ContainerNamekey = "ContainerName" - ContainerIDkey = "ContainerId" + AttributeKubernetes = "kubernetes" + AttributeK8sNamespace = "Namespace" + AttributePodID = "PodId" + AttributePodName = "PodName" + AttributeFullPodName = "FullPodName" + AttributeK8sPodName = "K8sPodName" + AttributeContainerName = "ContainerName" + AttributeContainerID = "ContainerId" + AttributeGpuDevice = "GpuDevice" PodStatus = "pod_status" ContainerStatus = "container_status" - GpuDevice = "GpuDevice" ContainerStatusReason = "container_status_reason" ContainerLastTerminationReason = "container_last_termination_reason" diff --git a/internal/aws/containerinsight/utils.go b/internal/aws/containerinsight/utils.go index 1c75501beace..3bf794e7862b 100644 --- a/internal/aws/containerinsight/utils.go +++ b/internal/aws/containerinsight/utils.go @@ -166,17 +166,18 @@ func GetUnitForMetric(metric string) string { return metricToUnitMap[metric] } +type FieldsAndTagsPair struct { + Fields map[string]any + Tags map[string]string +} + // ConvertToFieldsAndTags converts OTLP metric to a field containing metric values and a tag containing for decoration -func ConvertToFieldsAndTags(m pmetric.Metric, logger *zap.Logger) (map[string]any, map[string]string) { - fields := make(map[string]any) - tags := make(map[string]string) +func ConvertToFieldsAndTags(m pmetric.Metric, logger *zap.Logger) []FieldsAndTagsPair { + var converted []FieldsAndTagsPair if m.Name() == "" { - return fields, tags + return converted } - // value is not needed for label decoration - fields[m.Name()] = nil - var dps pmetric.NumberDataPointSlice switch m.Type() { case pmetric.MetricTypeGauge: @@ -187,19 +188,25 @@ func ConvertToFieldsAndTags(m pmetric.Metric, logger *zap.Logger) (map[string]an logger.Warn("Unsupported metric type", zap.String("metric", m.Name()), zap.String("type", m.Type().String())) } - // should support metrics with more than 1 datapoints? - if dps.Len() == 0 || dps.Len() > 1 { - logger.Warn("Metric has either 0 or more than 1 datapoints", zap.String("metric", m.Name()), zap.Int("datapoints", dps.Len())) + if dps.Len() == 0 { + logger.Warn("Metric has no datapoint", zap.String("metric", m.Name())) } - if dps.Len() > 0 { - attrs := dps.At(0).Attributes() + for i := 0; i < dps.Len(); i++ { + tags := make(map[string]string) + attrs := dps.At(i).Attributes() attrs.Range(func(k string, v pcommon.Value) bool { tags[k] = v.AsString() return true }) + converted = append(converted, FieldsAndTagsPair{ + Fields: map[string]any{ + m.Name(): nil, // metric value not needed for attribute decoration + }, + Tags: tags, + }) } - return fields, tags + return converted } // ConvertToOTLPMetrics converts a field containing metric values and a tag containing the relevant labels to OTLP metrics diff --git a/receiver/awscontainerinsightreceiver/config.go b/receiver/awscontainerinsightreceiver/config.go index fd26bfcd453d..708ea1726747 100644 --- a/receiver/awscontainerinsightreceiver/config.go +++ b/receiver/awscontainerinsightreceiver/config.go @@ -58,6 +58,6 @@ type Config struct { // The default value is false. EnableControlPlaneMetrics bool `mapstructure:"enable_control_plane_metrics"` - // EnableAcceleratedComputingMetric toggles GPU monitoring where metrics are scraped from vendor specific sources - EnableAcceleratedComputingMetric bool `mapstructure:"accelerated_computing_metrics"` + // EnableAcceleratedComputeMetrics enabled features with accelerated compute resources where metrics are scraped from vendor specific sources + EnableAcceleratedComputeMetrics bool `mapstructure:"accelerated_compute_metrics"` } diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/container_info_processor.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/container_info_processor.go index 1abd4fd00b87..3c6842f60831 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/container_info_processor.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/container_info_processor.go @@ -123,9 +123,9 @@ func processContainer(info *cInfo.ContainerInfo, mInfo extractors.CPUMemInfoProv podPath := path.Dir(info.Name) pKey = &podKey{cgroupPath: podPath, podName: podName, podID: podID, namespace: namespace} - tags[ci.PodIDKey] = podID - tags[ci.K8sPodNameKey] = podName - tags[ci.K8sNamespace] = namespace + tags[ci.AttributePodID] = podID + tags[ci.AttributeK8sPodName] = podName + tags[ci.AttributeK8sNamespace] = namespace switch containerName { // For docker, pause container name is set to POD while containerd does not set it. // See https://github.com/aws/amazon-cloudwatch-agent/issues/188 @@ -134,9 +134,9 @@ func processContainer(info *cInfo.ContainerInfo, mInfo extractors.CPUMemInfoProv // other pod info like CPU, Mem are dealt within in processPod. containerType = ci.TypeInfraContainer default: - tags[ci.ContainerNamekey] = containerName + tags[ci.AttributeContainerName] = containerName containerID := path.Base(info.Name) - tags[ci.ContainerIDkey] = containerID + tags[ci.AttributeContainerID] = containerID pKey.containerIds = []string{containerID} containerType = ci.TypeContainer // TODO(pvasir): wait for upstream fix https://github.com/google/cadvisor/issues/2785 @@ -178,9 +178,9 @@ func processPod(info *cInfo.ContainerInfo, mInfo extractors.CPUMemInfoProvider, } tags := map[string]string{} - tags[ci.PodIDKey] = podKey.podID - tags[ci.K8sPodNameKey] = podKey.podName - tags[ci.K8sNamespace] = podKey.namespace + tags[ci.AttributePodID] = podKey.podID + tags[ci.AttributeK8sPodName] = podKey.podName + tags[ci.AttributeK8sNamespace] = podKey.namespace tags[ci.Timestamp] = strconv.FormatInt(extractors.GetStats(info).Timestamp.UnixNano(), 10) diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/cpu_extractor.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/cpu_extractor.go index 6c6956e3ea5f..de91e0d26bdc 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/cpu_extractor.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/cpu_extractor.go @@ -5,6 +5,7 @@ package extractors // import "github.com/open-telemetry/opentelemetry-collector- import ( cInfo "github.com/google/cadvisor/info/v1" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "go.uber.org/zap" ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" @@ -24,8 +25,8 @@ func (c *CPUMetricExtractor) HasValue(info *cInfo.ContainerInfo) bool { return info.Spec.HasCpu } -func (c *CPUMetricExtractor) GetValue(info *cInfo.ContainerInfo, mInfo CPUMemInfoProvider, containerType string) []*RawContainerInsightsMetric { - var metrics []*RawContainerInsightsMetric +func (c *CPUMetricExtractor) GetValue(info *cInfo.ContainerInfo, mInfo CPUMemInfoProvider, containerType string) []*stores.RawContainerInsightsMetric { + var metrics []*stores.RawContainerInsightsMetric // Skip infra container and handle node, pod, other containers in pod if containerType == ci.TypeInfraContainer { return metrics @@ -33,7 +34,7 @@ func (c *CPUMetricExtractor) GetValue(info *cInfo.ContainerInfo, mInfo CPUMemInf // When there is more than one stats point, always use the last one curStats := GetStats(info) - metric := NewRawContainerInsightsMetric(containerType, c.logger) + metric := stores.NewRawContainerInsightsMetric(containerType, c.logger) metric.ContainerName = info.Name multiplier := float64(decimalToMillicores) assignRateValueToField(&c.rateCalculator, metric.Fields, ci.MetricName(containerType, ci.CPUTotal), info.Name, float64(curStats.Cpu.Usage.Total), curStats.Timestamp, multiplier) diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/cpu_extractor_test.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/cpu_extractor_test.go index c8a6f52f55ae..b41b0142cf42 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/cpu_extractor_test.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/cpu_extractor_test.go @@ -6,6 +6,7 @@ package extractors import ( "testing" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "github.com/stretchr/testify/require" "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" @@ -22,7 +23,7 @@ func TestCPUStats(t *testing.T) { containerType := containerinsight.TypeContainer extractor := NewCPUMetricExtractor(nil) - var cMetrics []*RawContainerInsightsMetric + var cMetrics []*stores.RawContainerInsightsMetric if extractor.HasValue(result[0]) { cMetrics = extractor.GetValue(result[0], MockCPUMemInfo, containerType) } diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/diskio_extractor.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/diskio_extractor.go index d1ba1e19220e..dc3879ea37c3 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/diskio_extractor.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/diskio_extractor.go @@ -9,6 +9,7 @@ import ( "time" cInfo "github.com/google/cadvisor/info/v1" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "go.uber.org/zap" ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" @@ -24,8 +25,8 @@ func (d *DiskIOMetricExtractor) HasValue(info *cInfo.ContainerInfo) bool { return info.Spec.HasDiskIo } -func (d *DiskIOMetricExtractor) GetValue(info *cInfo.ContainerInfo, _ CPUMemInfoProvider, containerType string) []*RawContainerInsightsMetric { - var metrics []*RawContainerInsightsMetric +func (d *DiskIOMetricExtractor) GetValue(info *cInfo.ContainerInfo, _ CPUMemInfoProvider, containerType string) []*stores.RawContainerInsightsMetric { + var metrics []*stores.RawContainerInsightsMetric if containerType != ci.TypeNode && containerType != ci.TypeInstance { return metrics } @@ -36,12 +37,12 @@ func (d *DiskIOMetricExtractor) GetValue(info *cInfo.ContainerInfo, _ CPUMemInfo return metrics } -func (d *DiskIOMetricExtractor) extractIoMetrics(curStatsSet []cInfo.PerDiskStats, namePrefix string, containerType string, infoName string, curTime time.Time) []*RawContainerInsightsMetric { - var metrics []*RawContainerInsightsMetric +func (d *DiskIOMetricExtractor) extractIoMetrics(curStatsSet []cInfo.PerDiskStats, namePrefix string, containerType string, infoName string, curTime time.Time) []*stores.RawContainerInsightsMetric { + var metrics []*stores.RawContainerInsightsMetric expectedKey := []string{ci.DiskIOAsync, ci.DiskIOSync, ci.DiskIORead, ci.DiskIOWrite, ci.DiskIOTotal} for _, cur := range curStatsSet { curDevName := devName(cur) - metric := NewRawContainerInsightsMetric(getDiskIOMetricType(containerType, d.logger), d.logger) + metric := stores.NewRawContainerInsightsMetric(getDiskIOMetricType(containerType, d.logger), d.logger) metric.Tags[ci.DiskDev] = curDevName for _, key := range expectedKey { if curVal, curOk := cur.Stats[key]; curOk { diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/diskio_extractor_test.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/diskio_extractor_test.go index a93c5206bf06..287c66a4426e 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/diskio_extractor_test.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/diskio_extractor_test.go @@ -6,6 +6,7 @@ package extractors import ( "testing" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -21,7 +22,7 @@ func TestDiskIOStats(t *testing.T) { containerType := containerinsight.TypeNode extractor := NewDiskIOMetricExtractor(nil) - var cMetrics []*RawContainerInsightsMetric + var cMetrics []*stores.RawContainerInsightsMetric if extractor.HasValue(result[0]) { cMetrics = extractor.GetValue(result[0], nil, containerType) } diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor.go index 9d6a6c9e8f9f..5007cdca3a74 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor.go @@ -8,11 +8,9 @@ import ( "time" cinfo "github.com/google/cadvisor/info/v1" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" - "go.uber.org/zap" - ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" awsmetrics "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/metrics" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" ) func GetStats(info *cinfo.ContainerInfo) *cinfo.ContainerStats { @@ -30,94 +28,11 @@ type CPUMemInfoProvider interface { type MetricExtractor interface { HasValue(*cinfo.ContainerInfo) bool - GetValue(info *cinfo.ContainerInfo, mInfo CPUMemInfoProvider, containerType string) []*RawContainerInsightsMetric + GetValue(info *cinfo.ContainerInfo, mInfo CPUMemInfoProvider, containerType string) []*stores.RawContainerInsightsMetric Shutdown() error } -type RawContainerInsightsMetric struct { - // source of the metric for debugging merge conflict - ContainerName string - // key/value pairs that are typed and contain the metric (numerical) data - Fields map[string]any - // key/value string pairs that are used to identify the metrics - Tags map[string]string - - Logger *zap.Logger -} - -var _ stores.CIMetric = (*RawContainerInsightsMetric)(nil) - -func NewRawContainerInsightsMetric(mType string, logger *zap.Logger) *RawContainerInsightsMetric { - metric := &RawContainerInsightsMetric{ - Fields: make(map[string]any), - Tags: make(map[string]string), - Logger: logger, - } - metric.Tags[ci.MetricType] = mType - return metric -} - -func (c *RawContainerInsightsMetric) GetTags() map[string]string { - return c.Tags -} - -func (c *RawContainerInsightsMetric) GetFields() map[string]any { - return c.Fields -} - -func (c *RawContainerInsightsMetric) GetMetricType() string { - return c.Tags[ci.MetricType] -} - -func (c *RawContainerInsightsMetric) AddTags(tags map[string]string) { - for k, v := range tags { - c.Tags[k] = v - } -} - -func (c *RawContainerInsightsMetric) HasField(key string) bool { - return c.Fields[key] != nil -} - -func (c *RawContainerInsightsMetric) AddField(key string, val any) { - c.Fields[key] = val -} - -func (c *RawContainerInsightsMetric) GetField(key string) any { - return c.Fields[key] -} - -func (c *RawContainerInsightsMetric) HasTag(key string) bool { - return c.Tags[key] != "" -} - -func (c *RawContainerInsightsMetric) AddTag(key, val string) { - c.Tags[key] = val -} - -func (c *RawContainerInsightsMetric) GetTag(key string) string { - return c.Tags[key] -} - -func (c *RawContainerInsightsMetric) RemoveTag(key string) { - delete(c.Tags, key) -} - -func (c *RawContainerInsightsMetric) Merge(src *RawContainerInsightsMetric) { - // If there is any conflict, keep the Fields with earlier timestamp - for k, v := range src.Fields { - if _, ok := c.Fields[k]; ok { - c.Logger.Debug(fmt.Sprintf("metric being merged has conflict in Fields, src: %v, dest: %v \n", *src, *c)) - c.Logger.Debug("metric being merged has conflict in Fields", zap.String("src", src.ContainerName), zap.String("dest", c.ContainerName)) - if c.Tags[ci.Timestamp] < src.Tags[ci.Timestamp] { - continue - } - } - c.Fields[k] = v - } -} - -func newFloat64RateCalculator() awsmetrics.MetricCalculator { +func NewFloat64RateCalculator() awsmetrics.MetricCalculator { return awsmetrics.NewMetricCalculator(func(prev *awsmetrics.MetricValue, val any, timestamp time.Time) (any, bool) { if prev != nil { deltaNs := timestamp.Sub(prev.Timestamp) @@ -139,9 +54,9 @@ func assignRateValueToField(rateCalculator *awsmetrics.MetricCalculator, fields } // MergeMetrics merges an array of cadvisor metrics based on common metric keys -func MergeMetrics(metrics []*RawContainerInsightsMetric) []*RawContainerInsightsMetric { - result := make([]*RawContainerInsightsMetric, 0, len(metrics)) - metricMap := make(map[string]*RawContainerInsightsMetric) +func MergeMetrics(metrics []*stores.RawContainerInsightsMetric) []*stores.RawContainerInsightsMetric { + result := make([]*stores.RawContainerInsightsMetric, 0, len(metrics)) + metricMap := make(map[string]*stores.RawContainerInsightsMetric) for _, metric := range metrics { if metricKey := getMetricKey(metric); metricKey != "" { if mergedMetric, ok := metricMap[metricKey]; ok { @@ -161,7 +76,7 @@ func MergeMetrics(metrics []*RawContainerInsightsMetric) []*RawContainerInsights } // return MetricKey for merge-able metrics -func getMetricKey(metric *RawContainerInsightsMetric) string { +func getMetricKey(metric *stores.RawContainerInsightsMetric) string { metricType := metric.GetMetricType() var metricKey string switch metricType { @@ -173,10 +88,10 @@ func getMetricKey(metric *RawContainerInsightsMetric) string { metricKey = fmt.Sprintf("metricType:%s", ci.TypeNode) case ci.TypePod: // merge cpu, memory, net metric for type Pod - metricKey = fmt.Sprintf("metricType:%s,podId:%s", ci.TypePod, metric.GetTags()[ci.PodIDKey]) + metricKey = fmt.Sprintf("metricType:%s,podId:%s", ci.TypePod, metric.GetTags()[ci.AttributePodID]) case ci.TypeContainer: // merge cpu, memory metric for type Container - metricKey = fmt.Sprintf("metricType:%s,podId:%s,ContainerName:%s", ci.TypeContainer, metric.GetTags()[ci.PodIDKey], metric.GetTags()[ci.ContainerNamekey]) + metricKey = fmt.Sprintf("metricType:%s,podId:%s,containerName:%s", ci.TypeContainer, metric.GetTags()[ci.AttributePodID], metric.GetTags()[ci.AttributeContainerName]) case ci.TypeInstanceDiskIO: // merge io_serviced, io_service_bytes for type InstanceDiskIO metricKey = fmt.Sprintf("metricType:%s,device:%s", ci.TypeInstanceDiskIO, metric.GetTags()[ci.DiskDev]) diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor_helpers_test.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor_helpers_test.go index 40a2c32b412b..f5bafe09101f 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor_helpers_test.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor_helpers_test.go @@ -8,12 +8,13 @@ import ( "reflect" "testing" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "github.com/stretchr/testify/assert" ) func AssertContainsTaggedFloat( t *testing.T, - cadvisorMetric *RawContainerInsightsMetric, + cadvisorMetric *stores.RawContainerInsightsMetric, field string, expectedValue float64, delta float64, @@ -32,14 +33,14 @@ func AssertContainsTaggedFloat( } } msg := fmt.Sprintf( - "Could not find field \"%s\" with requested Tags within %f of %f, Actual: %f", + "Could not find field \"%s\" with requested tags within %f of %f, Actual: %f", field, delta, expectedValue, actualValue) assert.Fail(t, msg) } func AssertContainsTaggedInt( t *testing.T, - cadvisorMetric *RawContainerInsightsMetric, + cadvisorMetric *stores.RawContainerInsightsMetric, field string, expectedValue int64, ) { @@ -52,14 +53,14 @@ func AssertContainsTaggedInt( } } msg := fmt.Sprintf( - "Could not find field \"%s\" with requested Tags with value: %v, Actual: %v", + "Could not find field \"%s\" with requested tags with value: %v, Actual: %v", field, expectedValue, actualValue) assert.Fail(t, msg) } func AssertContainsTaggedUint( t *testing.T, - cadvisorMetric *RawContainerInsightsMetric, + cadvisorMetric *stores.RawContainerInsightsMetric, field string, expectedValue uint64, ) { @@ -72,14 +73,14 @@ func AssertContainsTaggedUint( } } msg := fmt.Sprintf( - "Could not find field \"%s\" with requested Tags with value: %v, Actual: %v", + "Could not find field \"%s\" with requested tags with value: %v, Actual: %v", field, expectedValue, actualValue) assert.Fail(t, msg) } func AssertContainsTaggedField( t *testing.T, - cadvisorMetric *RawContainerInsightsMetric, + cadvisorMetric *stores.RawContainerInsightsMetric, expectedFields map[string]any, expectedTags map[string]string, ) { diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor_test.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor_test.go index 3a240ed1c90b..90a5963d4067 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor_test.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor_test.go @@ -6,6 +6,7 @@ package extractors import ( "testing" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "go.uber.org/zap" @@ -14,12 +15,12 @@ import ( ) func TestCAdvisorMetric_Merge(t *testing.T) { - src := &RawContainerInsightsMetric{ + src := &stores.RawContainerInsightsMetric{ Fields: map[string]any{"value1": 1, "value2": 2}, Tags: map[string]string{ci.Timestamp: "1586331559882"}, Logger: zap.NewNop(), } - dest := &RawContainerInsightsMetric{ + dest := &stores.RawContainerInsightsMetric{ Fields: map[string]any{"value1": 3, "value3": 3}, Tags: map[string]string{ci.Timestamp: "1586331559973"}, Logger: zap.NewNop(), @@ -30,38 +31,38 @@ func TestCAdvisorMetric_Merge(t *testing.T) { } func TestGetMetricKey(t *testing.T) { - c := &RawContainerInsightsMetric{ + c := &stores.RawContainerInsightsMetric{ Tags: map[string]string{ ci.MetricType: ci.TypeInstance, }, } assert.Equal(t, "metricType:Instance", getMetricKey(c)) - c = &RawContainerInsightsMetric{ + c = &stores.RawContainerInsightsMetric{ Tags: map[string]string{ ci.MetricType: ci.TypeNode, }, } assert.Equal(t, "metricType:Node", getMetricKey(c)) - c = &RawContainerInsightsMetric{ + c = &stores.RawContainerInsightsMetric{ Tags: map[string]string{ - ci.MetricType: ci.TypePod, - ci.PodIDKey: "podID", + ci.MetricType: ci.TypePod, + ci.AttributePodID: "podID", }, } assert.Equal(t, "metricType:Pod,podId:podID", getMetricKey(c)) - c = &RawContainerInsightsMetric{ + c = &stores.RawContainerInsightsMetric{ Tags: map[string]string{ - ci.MetricType: ci.TypeContainer, - ci.PodIDKey: "podID", - ci.ContainerNamekey: "ContainerName", + ci.MetricType: ci.TypeContainer, + ci.AttributePodID: "podID", + ci.AttributeContainerName: "ContainerName", }, } - assert.Equal(t, "metricType:Container,podId:podID,ContainerName:ContainerName", getMetricKey(c)) + assert.Equal(t, "metricType:Container,podId:podID,containerName:ContainerName", getMetricKey(c)) - c = &RawContainerInsightsMetric{ + c = &stores.RawContainerInsightsMetric{ Tags: map[string]string{ ci.MetricType: ci.TypeInstanceDiskIO, ci.DiskDev: "/abc", @@ -69,7 +70,7 @@ func TestGetMetricKey(t *testing.T) { } assert.Equal(t, "metricType:InstanceDiskIO,device:/abc", getMetricKey(c)) - c = &RawContainerInsightsMetric{ + c = &stores.RawContainerInsightsMetric{ Tags: map[string]string{ ci.MetricType: ci.TypeNodeDiskIO, ci.DiskDev: "/abc", @@ -77,12 +78,12 @@ func TestGetMetricKey(t *testing.T) { } assert.Equal(t, "metricType:NodeDiskIO,device:/abc", getMetricKey(c)) - c = &RawContainerInsightsMetric{} + c = &stores.RawContainerInsightsMetric{} assert.Equal(t, "", getMetricKey(c)) } func TestMergeMetrics(t *testing.T) { - cpuMetrics := &RawContainerInsightsMetric{ + cpuMetrics := &stores.RawContainerInsightsMetric{ Fields: map[string]any{ "node_cpu_usage_total": float64(10), "node_cpu_usage_user": float64(10), @@ -92,7 +93,7 @@ func TestMergeMetrics(t *testing.T) { }, } - memMetrics := &RawContainerInsightsMetric{ + memMetrics := &stores.RawContainerInsightsMetric{ Fields: map[string]any{ "node_memory_cache": uint(25645056), }, @@ -101,12 +102,12 @@ func TestMergeMetrics(t *testing.T) { }, } - metrics := []*RawContainerInsightsMetric{ + metrics := []*stores.RawContainerInsightsMetric{ cpuMetrics, memMetrics, } - expected := &RawContainerInsightsMetric{ + expected := &stores.RawContainerInsightsMetric{ Fields: map[string]any{ "node_cpu_usage_total": float64(10), "node_cpu_usage_user": float64(10), diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/fs_extractor.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/fs_extractor.go index 3725fbe558df..26ad4f976295 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/fs_extractor.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/fs_extractor.go @@ -7,6 +7,7 @@ import ( "regexp" cinfo "github.com/google/cadvisor/info/v1" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "go.uber.org/zap" ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" @@ -23,17 +24,17 @@ func (f *FileSystemMetricExtractor) HasValue(info *cinfo.ContainerInfo) bool { return info.Spec.HasFilesystem } -func (f *FileSystemMetricExtractor) GetValue(info *cinfo.ContainerInfo, _ CPUMemInfoProvider, containerType string) []*RawContainerInsightsMetric { +func (f *FileSystemMetricExtractor) GetValue(info *cinfo.ContainerInfo, _ CPUMemInfoProvider, containerType string) []*stores.RawContainerInsightsMetric { if containerType == ci.TypePod || containerType == ci.TypeInfraContainer { return nil } containerType = getFSMetricType(containerType, f.logger) stats := GetStats(info) - metrics := make([]*RawContainerInsightsMetric, 0, len(stats.Filesystem)) + metrics := make([]*stores.RawContainerInsightsMetric, 0, len(stats.Filesystem)) for _, v := range stats.Filesystem { - metric := NewRawContainerInsightsMetric(containerType, f.logger) + metric := stores.NewRawContainerInsightsMetric(containerType, f.logger) if v.Device == "" { continue } diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/fs_extractor_test.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/fs_extractor_test.go index 96157b818d28..e4ed994d1d3f 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/fs_extractor_test.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/fs_extractor_test.go @@ -9,6 +9,7 @@ import ( "testing" cinfo "github.com/google/cadvisor/info/v1" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "github.com/stretchr/testify/assert" "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" @@ -21,7 +22,7 @@ func TestFSStats(t *testing.T) { containerType := containerinsight.TypeContainer extractor := NewFileSystemMetricExtractor(nil) - var cMetrics []*RawContainerInsightsMetric + var cMetrics []*stores.RawContainerInsightsMetric if extractor.HasValue(result[0]) { cMetrics = extractor.GetValue(result[0], nil, containerType) } @@ -125,7 +126,7 @@ func TestFSStatsWithAllowList(t *testing.T) { containerType := containerinsight.TypeContainer extractor := NewFileSystemMetricExtractor(nil) - var cMetrics []*RawContainerInsightsMetric + var cMetrics []*stores.RawContainerInsightsMetric if extractor.HasValue(result[0]) { cMetrics = extractor.GetValue(result[0], nil, containerType) } diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/mem_extractor.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/mem_extractor.go index a379763b7087..083e6e9974fb 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/mem_extractor.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/mem_extractor.go @@ -7,6 +7,7 @@ import ( "time" cinfo "github.com/google/cadvisor/info/v1" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "go.uber.org/zap" ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" @@ -22,13 +23,13 @@ func (m *MemMetricExtractor) HasValue(info *cinfo.ContainerInfo) bool { return info.Spec.HasMemory } -func (m *MemMetricExtractor) GetValue(info *cinfo.ContainerInfo, mInfo CPUMemInfoProvider, containerType string) []*RawContainerInsightsMetric { - var metrics []*RawContainerInsightsMetric +func (m *MemMetricExtractor) GetValue(info *cinfo.ContainerInfo, mInfo CPUMemInfoProvider, containerType string) []*stores.RawContainerInsightsMetric { + var metrics []*stores.RawContainerInsightsMetric if containerType == ci.TypeInfraContainer { return metrics } - metric := NewRawContainerInsightsMetric(containerType, m.logger) + metric := stores.NewRawContainerInsightsMetric(containerType, m.logger) metric.ContainerName = info.Name curStats := GetStats(info) diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/mem_extractor_test.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/mem_extractor_test.go index ae5b1674e6f0..708059ed9d1a 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/mem_extractor_test.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/mem_extractor_test.go @@ -6,6 +6,7 @@ package extractors import ( "testing" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "github.com/stretchr/testify/require" "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" @@ -20,7 +21,7 @@ func TestMemStats(t *testing.T) { containerType := containerinsight.TypeContainer extractor := NewMemMetricExtractor(nil) - var cMetrics []*RawContainerInsightsMetric + var cMetrics []*stores.RawContainerInsightsMetric if extractor.HasValue(result[0]) { cMetrics = extractor.GetValue(result[0], MockCPUMemInfo, containerType) } diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/net_extractor.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/net_extractor.go index 5a5af9f3f209..69b6ec2eb52e 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/net_extractor.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/net_extractor.go @@ -7,6 +7,7 @@ import ( "time" cinfo "github.com/google/cadvisor/info/v1" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "go.uber.org/zap" ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" @@ -30,7 +31,7 @@ func (n *NetMetricExtractor) HasValue(info *cinfo.ContainerInfo) bool { return info.Spec.HasNetwork } -func (n *NetMetricExtractor) GetValue(info *cinfo.ContainerInfo, _ CPUMemInfoProvider, containerType string) []*RawContainerInsightsMetric { +func (n *NetMetricExtractor) GetValue(info *cinfo.ContainerInfo, _ CPUMemInfoProvider, containerType string) []*stores.RawContainerInsightsMetric { // Just a protection here, there is no Container level Net metrics if containerType == ci.TypePod || containerType == ci.TypeContainer { @@ -47,7 +48,7 @@ func (n *NetMetricExtractor) GetValue(info *cinfo.ContainerInfo, _ CPUMemInfoPro // used for aggregation netIfceMetrics := make([]map[string]any, len(curIfceStats)) - metrics := make([]*RawContainerInsightsMetric, len(curIfceStats)) + metrics := make([]*stores.RawContainerInsightsMetric, len(curIfceStats)) for i, cur := range curIfceStats { mType := getNetMetricType(containerType, n.logger) @@ -70,7 +71,7 @@ func (n *NetMetricExtractor) GetValue(info *cinfo.ContainerInfo, _ CPUMemInfoPro netIfceMetrics[i] = netIfceMetric - metric := NewRawContainerInsightsMetric(mType, n.logger) + metric := stores.NewRawContainerInsightsMetric(mType, n.logger) metric.Tags[ci.NetIfce] = cur.Name for k, v := range netIfceMetric { metric.Fields[ci.MetricName(mType, k)] = v @@ -81,7 +82,7 @@ func (n *NetMetricExtractor) GetValue(info *cinfo.ContainerInfo, _ CPUMemInfoPro aggregatedFields := ci.SumFields(netIfceMetrics) if len(aggregatedFields) > 0 { - metric := NewRawContainerInsightsMetric(containerType, n.logger) + metric := stores.NewRawContainerInsightsMetric(containerType, n.logger) for k, v := range aggregatedFields { metric.Fields[ci.MetricName(containerType, k)] = v } diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/net_extractor_test.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/net_extractor_test.go index 8f1d31a8019a..9280d1d3d0f5 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/net_extractor_test.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/net_extractor_test.go @@ -6,6 +6,7 @@ package extractors import ( "testing" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -19,7 +20,7 @@ func TestNetStats(t *testing.T) { containerType := ci.TypeNode extractor := NewNetMetricExtractor(nil) - var cMetrics []*RawContainerInsightsMetric + var cMetrics []*stores.RawContainerInsightsMetric if extractor.HasValue(result[0]) { cMetrics = extractor.GetValue(result[0], nil, containerType) } diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go index 03ea107f0b1d..e7435a34a5b2 100644 --- a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go +++ b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go @@ -142,7 +142,7 @@ func getMetricRelabelConfig(hostInfoProvider hostInfoProvider) []*relabel.Config }, { SourceLabels: model.LabelNames{"namespace"}, - TargetLabel: ci.K8sNamespace, + TargetLabel: ci.AttributeK8sNamespace, Regex: relabel.MustNewRegexp("(.*)"), Replacement: "${1}", Action: relabel.Replace, @@ -165,7 +165,7 @@ func getMetricRelabelConfig(hostInfoProvider hostInfoProvider) []*relabel.Config }, { SourceLabels: model.LabelNames{"pod"}, - TargetLabel: ci.FullPodNameKey, + TargetLabel: ci.AttributeFullPodName, Regex: relabel.MustNewRegexp("(.*)"), Replacement: "${1}", Action: relabel.Replace, @@ -173,21 +173,21 @@ func getMetricRelabelConfig(hostInfoProvider hostInfoProvider) []*relabel.Config // additional k8s podname for service name and k8s blob decoration { SourceLabels: model.LabelNames{"pod"}, - TargetLabel: ci.K8sPodNameKey, + TargetLabel: ci.AttributeK8sPodName, Regex: relabel.MustNewRegexp("(.*)"), Replacement: "${1}", Action: relabel.Replace, }, { SourceLabels: model.LabelNames{"container"}, - TargetLabel: ci.ContainerNamekey, + TargetLabel: ci.AttributeContainerName, Regex: relabel.MustNewRegexp("(.*)"), Replacement: "${1}", Action: relabel.Replace, }, { SourceLabels: model.LabelNames{"device"}, - TargetLabel: ci.GpuDevice, + TargetLabel: ci.AttributeGpuDevice, Regex: relabel.MustNewRegexp("(.*)"), Replacement: "${1}", Action: relabel.Replace, diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go index 43f950d8ad53..b9e875e54271 100644 --- a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go +++ b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go @@ -62,6 +62,7 @@ func (m mockDecorator) Shutdown() error { type mockConsumer struct { t *testing.T + called *bool expected map[string]struct { value float64 labels map[string]string @@ -81,6 +82,7 @@ func (m mockConsumer) ConsumeMetrics(_ context.Context, md pmetric.Metrics) erro scopeMetrics := md.ResourceMetrics().At(0).ScopeMetrics().At(0).Metrics() for i := 0; i < scopeMetrics.Len(); i++ { metric := scopeMetrics.At(i) + // skip prometheus metadata metrics including "up" if !strings.HasPrefix(metric.Name(), "DCGM") { continue @@ -96,6 +98,7 @@ func (m mockConsumer) ConsumeMetrics(_ context.Context, md pmetric.Metrics) erro scrapedMetricCnt += 1 } assert.Equal(m.t, len(m.expected), scrapedMetricCnt) + *m.called = true return nil } @@ -143,36 +146,35 @@ func TestNewDcgmScraperEndToEnd(t *testing.T) { "DCGM_FI_DEV_GPU_TEMP": { value: 65, labels: map[string]string{ - ci.NodeNameKey: "hostname", - ci.K8sNamespace: "kube-system", - ci.ClusterNameKey: dummyClusterName, - ci.InstanceID: dummyInstanceId, - ci.FullPodNameKey: "fullname-hash", - ci.K8sPodNameKey: "fullname-hash", - ci.ContainerNamekey: "main", - ci.GpuDevice: "nvidia0", + ci.NodeNameKey: "hostname", + ci.AttributeK8sNamespace: "kube-system", + ci.ClusterNameKey: dummyClusterName, + ci.InstanceID: dummyInstanceId, + ci.AttributeFullPodName: "fullname-hash", + ci.AttributeK8sPodName: "fullname-hash", + ci.AttributeContainerName: "main", + ci.AttributeGpuDevice: "nvidia0", }, }, "DCGM_FI_DEV_GPU_UTIL": { value: 100, labels: map[string]string{ - ci.NodeNameKey: "hostname", - ci.K8sNamespace: "kube-system", - ci.ClusterNameKey: dummyClusterName, - ci.InstanceID: dummyInstanceId, - ci.FullPodNameKey: "fullname-hash", - ci.K8sPodNameKey: "fullname-hash", - ci.ContainerNamekey: "main", - ci.GpuDevice: "nvidia0", + ci.NodeNameKey: "hostname", + ci.AttributeK8sNamespace: "kube-system", + ci.ClusterNameKey: dummyClusterName, + ci.InstanceID: dummyInstanceId, + ci.AttributeFullPodName: "fullname-hash", + ci.AttributeK8sPodName: "fullname-hash", + ci.AttributeContainerName: "main", + ci.AttributeGpuDevice: "nvidia0", }, }, - "up": { - value: 1, - labels: map[string]string{}, - }, } + + consumerCalled := false consumer := mockConsumer{ t: t, + called: &consumerCalled, expected: expected, } @@ -182,7 +184,7 @@ func TestNewDcgmScraperEndToEnd(t *testing.T) { scraper, err := NewDcgmScraper(DcgmScraperOpts{ Ctx: context.TODO(), TelemetrySettings: settings, - Consumer: mockConsumer{}, + Consumer: consumer, Host: componenttest.NewNopHost(), HostInfoProvider: mockHostInfoProvider{}, K8sDecorator: mockDecorator{}, @@ -252,6 +254,8 @@ func TestNewDcgmScraperEndToEnd(t *testing.T) { // wait for 2 scrapes, one initiated by us, another by the new scraper process mp.Wg.Wait() mp.Wg.Wait() + // make sure the consumer is called at scraping interval + assert.True(t, consumerCalled) } func TestDcgmScraperJobName(t *testing.T) { diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/decorator.go b/receiver/awscontainerinsightreceiver/internal/gpu/decorator.go index cec7677a7291..a3f68c46fa99 100644 --- a/receiver/awscontainerinsightreceiver/internal/gpu/decorator.go +++ b/receiver/awscontainerinsightreceiver/internal/gpu/decorator.go @@ -7,7 +7,6 @@ import ( "context" ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "go.opentelemetry.io/collector/consumer" "go.opentelemetry.io/collector/pdata/pcommon" @@ -62,22 +61,19 @@ func (dc *decorateConsumer) ConsumeMetrics(ctx context.Context, md pmetric.Metri ms := ilms.At(j).Metrics() for k := 0; k < ms.Len(); k++ { m := ms.At(k) - fields, tags := ci.ConvertToFieldsAndTags(m, dc.logger) - // copy down resource metrics only when it's missing at datapoint - for rtk, rtv := range resourceTags { - if _, ok := tags[rtk]; !ok { - tags[rtk] = rtv + converted := ci.ConvertToFieldsAndTags(m, dc.logger) + var rcis []*stores.RawContainerInsightsMetric + for _, pair := range converted { + rci := stores.NewRawContainerInsightsMetricWithData(ci.TypeGpuContainer, pair.Fields, pair.Tags, dc.logger) + if !rci.HasTag(ci.MetricType) { + // force type to be Container to decorate with container level labels + rci.AddTag(ci.MetricType, ci.TypeGpuContainer) } + rcis = append(rcis, stores.NewRawContainerInsightsMetricWithData(ci.TypeGpuContainer, pair.Fields, pair.Tags, dc.logger)) } - cim := extractors.NewRawContainerInsightsMetric(ci.TypeGpuContainer, dc.logger) - cim.Fields = fields - cim.Tags = tags - if !cim.HasTag(ci.MetricType) { - // force type to be Container to decorate with container level labels - cim.AddTag(ci.MetricType, ci.TypeGpuContainer) - } - dc.decorateMetrics([]*extractors.RawContainerInsightsMetric{cim}) - dc.updateAttributes(m, cim) + + decorated := dc.decorateMetrics(rcis) + dc.updateAttributes(m, decorated) if unit, ok := metricToUnit[m.Name()]; ok { m.SetUnit(unit) } @@ -92,22 +88,23 @@ type Decorator interface { Shutdown() error } -func (dc *decorateConsumer) decorateMetrics(metrics []*extractors.RawContainerInsightsMetric) []*extractors.RawContainerInsightsMetric { - var result []*extractors.RawContainerInsightsMetric - for _, m := range metrics { +func (dc *decorateConsumer) decorateMetrics(rcis []*stores.RawContainerInsightsMetric) []*stores.RawContainerInsightsMetric { + var result []*stores.RawContainerInsightsMetric + if dc.containerOrchestrator != ci.EKS { + return result + } + for _, rci := range rcis { // add tags for EKS - if dc.containerOrchestrator == ci.EKS { - out := dc.k8sDecorator.Decorate(m) - if out != nil { - result = append(result, out.(*extractors.RawContainerInsightsMetric)) - } + out := dc.k8sDecorator.Decorate(rci) + if out != nil { + result = append(result, out.(*stores.RawContainerInsightsMetric)) } } return result } -func (dc *decorateConsumer) updateAttributes(m pmetric.Metric, cim *extractors.RawContainerInsightsMetric) { - if len(cim.Tags) == 0 { +func (dc *decorateConsumer) updateAttributes(m pmetric.Metric, rcis []*stores.RawContainerInsightsMetric) { + if len(rcis) == 0 { return } var dps pmetric.NumberDataPointSlice @@ -119,17 +116,23 @@ func (dc *decorateConsumer) updateAttributes(m pmetric.Metric, cim *extractors.R default: dc.logger.Warn("Unsupported metric type", zap.String("metric", m.Name()), zap.String("type", m.Type().String())) } - if dps.Len() == 0 { return } - attrs := dps.At(0).Attributes() - for tk, tv := range cim.Tags { - // type gets set with metrictransformer while duplicating metrics at different resource levels - if tk == ci.MetricType { + for i := 0; i < dps.Len(); i++ { + if i >= len(rcis) { + // this shouldn't be the case, but it helps to avoid panic continue } - attrs.PutStr(tk, tv) + attrs := dps.At(i).Attributes() + tags := rcis[i].Tags + for tk, tv := range tags { + // type gets set with metrictransformer while duplicating metrics at different resource levels + if tk == ci.MetricType { + continue + } + attrs.PutStr(tk, tv) + } } } diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/decorator_test.go b/receiver/awscontainerinsightreceiver/internal/gpu/decorator_test.go index 4cbcb44700b5..4f5cadb8a31a 100644 --- a/receiver/awscontainerinsightreceiver/internal/gpu/decorator_test.go +++ b/receiver/awscontainerinsightreceiver/internal/gpu/decorator_test.go @@ -19,39 +19,6 @@ import ( var _ Decorator = (*MockK8sDecorator)(nil) -type mockGpuMetric struct { - tags map[string]string - fields map[string]any -} - -func (m *mockGpuMetric) HasField(key string) bool { - return m.fields[key] != nil -} - -func (m *mockGpuMetric) AddField(key string, val any) { - m.fields[key] = val -} - -func (m *mockGpuMetric) GetField(key string) any { - return m.fields[key] -} - -func (m *mockGpuMetric) HasTag(key string) bool { - return m.tags[key] != "" -} - -func (m *mockGpuMetric) AddTag(key, val string) { - m.tags[key] = val -} - -func (m *mockGpuMetric) GetTag(key string) string { - return m.tags[key] -} - -func (m *mockGpuMetric) RemoveTag(key string) { - delete(m.tags, key) -} - type MockK8sDecorator struct { } @@ -183,7 +150,7 @@ func TestConsumeMetrics(t *testing.T) { return } require.NoError(t, err) - assert.Equal(t, tc.metrics.MetricCount(), tc.want.MetricCount()) + assert.Equal(t, tc.want.MetricCount(), tc.metrics.MetricCount()) if tc.want.MetricCount() == 0 { continue } diff --git a/receiver/awscontainerinsightreceiver/internal/k8sapiserver/k8sapiserver.go b/receiver/awscontainerinsightreceiver/internal/k8sapiserver/k8sapiserver.go index 6c8a1da2af62..89c32934a608 100644 --- a/receiver/awscontainerinsightreceiver/internal/k8sapiserver/k8sapiserver.go +++ b/receiver/awscontainerinsightreceiver/internal/k8sapiserver/k8sapiserver.go @@ -132,17 +132,17 @@ func (k *K8sAPIServer) getNamespaceMetrics(clusterName, timestampNs string) []pm "namespace_number_of_running_pods": podNum, } attributes := map[string]string{ - ci.ClusterNameKey: clusterName, - ci.MetricType: ci.TypeClusterNamespace, - ci.Timestamp: timestampNs, - ci.K8sNamespace: namespace, - ci.Version: "0", + ci.ClusterNameKey: clusterName, + ci.MetricType: ci.TypeClusterNamespace, + ci.Timestamp: timestampNs, + ci.AttributeK8sNamespace: namespace, + ci.Version: "0", } if k.nodeName != "" { attributes["NodeName"] = k.nodeName } attributes[ci.SourcesKey] = "[\"apiserver\"]" - attributes[ci.Kubernetes] = fmt.Sprintf("{\"namespace_name\":\"%s\"}", namespace) + attributes[ci.AttributeKubernetes] = fmt.Sprintf("{\"namespace_name\":\"%s\"}", namespace) md := ci.ConvertToOTLPMetrics(fields, attributes, k.logger) metrics = append(metrics, md) } @@ -160,18 +160,18 @@ func (k *K8sAPIServer) getDeploymentMetrics(clusterName, timestampNs string) []p ci.StatusReplicasUnavailable: deployment.Status.UnavailableReplicas, // status_replicas_unavailable } attributes := map[string]string{ - ci.ClusterNameKey: clusterName, - ci.MetricType: ci.TypeClusterDeployment, - ci.Timestamp: timestampNs, - ci.PodNameKey: deployment.Name, - ci.K8sNamespace: deployment.Namespace, - ci.Version: "0", + ci.ClusterNameKey: clusterName, + ci.MetricType: ci.TypeClusterDeployment, + ci.Timestamp: timestampNs, + ci.AttributePodName: deployment.Name, + ci.AttributeK8sNamespace: deployment.Namespace, + ci.Version: "0", } if k.nodeName != "" { attributes[ci.NodeNameKey] = k.nodeName } attributes[ci.SourcesKey] = "[\"apiserver\"]" - // attributes[ci.Kubernetes] = fmt.Sprintf("{\"namespace_name\":\"%s\",\"deployment_name\":\"%s\"}", + // attributes[ci.AttributeKubernetes] = fmt.Sprintf("{\"namespace_name\":\"%s\",\"deployment_name\":\"%s\"}", // deployment.Namespace, deployment.Name) md := ci.ConvertToOTLPMetrics(fields, attributes, k.logger) metrics = append(metrics, md) @@ -190,18 +190,18 @@ func (k *K8sAPIServer) getDaemonSetMetrics(clusterName, timestampNs string) []pm ci.ReplicasReady: daemonSet.Status.CurrentNumberScheduled, // replicas_ready } attributes := map[string]string{ - ci.ClusterNameKey: clusterName, - ci.MetricType: ci.TypeClusterDaemonSet, - ci.Timestamp: timestampNs, - ci.PodNameKey: daemonSet.Name, - ci.K8sNamespace: daemonSet.Namespace, - ci.Version: "0", + ci.ClusterNameKey: clusterName, + ci.MetricType: ci.TypeClusterDaemonSet, + ci.Timestamp: timestampNs, + ci.AttributePodName: daemonSet.Name, + ci.AttributeK8sNamespace: daemonSet.Namespace, + ci.Version: "0", } if k.nodeName != "" { attributes[ci.NodeNameKey] = k.nodeName } attributes[ci.SourcesKey] = "[\"apiserver\"]" - // attributes[ci.Kubernetes] = fmt.Sprintf("{\"namespace_name\":\"%s\",\"daemonset_name\":\"%s\"}", + // attributes[ci.AttributeKubernetes] = fmt.Sprintf("{\"namespace_name\":\"%s\",\"daemonset_name\":\"%s\"}", // daemonSet.Namespace, daemonSet.Name) md := ci.ConvertToOTLPMetrics(fields, attributes, k.logger) metrics = append(metrics, md) @@ -216,18 +216,18 @@ func (k *K8sAPIServer) getServiceMetrics(clusterName, timestampNs string) []pmet "service_number_of_running_pods": podNum, } attributes := map[string]string{ - ci.ClusterNameKey: clusterName, - ci.MetricType: ci.TypeClusterService, - ci.Timestamp: timestampNs, - ci.TypeService: service.ServiceName, - ci.K8sNamespace: service.Namespace, - ci.Version: "0", + ci.ClusterNameKey: clusterName, + ci.MetricType: ci.TypeClusterService, + ci.Timestamp: timestampNs, + ci.TypeService: service.ServiceName, + ci.AttributeK8sNamespace: service.Namespace, + ci.Version: "0", } if k.nodeName != "" { attributes["NodeName"] = k.nodeName } attributes[ci.SourcesKey] = "[\"apiserver\"]" - attributes[ci.Kubernetes] = fmt.Sprintf("{\"namespace_name\":\"%s\",\"service_name\":\"%s\"}", + attributes[ci.AttributeKubernetes] = fmt.Sprintf("{\"namespace_name\":\"%s\",\"service_name\":\"%s\"}", service.Namespace, service.ServiceName) md := ci.ConvertToOTLPMetrics(fields, attributes, k.logger) metrics = append(metrics, md) @@ -245,12 +245,12 @@ func (k *K8sAPIServer) getStatefulSetMetrics(clusterName, timestampNs string) [] ci.StatusReplicasAvailable: statefulSet.Status.AvailableReplicas, // status_replicas_available } attributes := map[string]string{ - ci.ClusterNameKey: clusterName, - ci.MetricType: ci.TypeClusterStatefulSet, - ci.Timestamp: timestampNs, - ci.PodNameKey: statefulSet.Name, - ci.K8sNamespace: statefulSet.Namespace, - ci.Version: "0", + ci.ClusterNameKey: clusterName, + ci.MetricType: ci.TypeClusterStatefulSet, + ci.Timestamp: timestampNs, + ci.AttributePodName: statefulSet.Name, + ci.AttributeK8sNamespace: statefulSet.Namespace, + ci.Version: "0", } if k.nodeName != "" { attributes[ci.NodeNameKey] = k.nodeName @@ -272,12 +272,12 @@ func (k *K8sAPIServer) getReplicaSetMetrics(clusterName, timestampNs string) []p ci.StatusReplicasAvailable: replicaSet.Status.AvailableReplicas, // status_replicas_available } attributes := map[string]string{ - ci.ClusterNameKey: clusterName, - ci.MetricType: ci.TypeClusterReplicaSet, - ci.Timestamp: timestampNs, - ci.PodNameKey: replicaSet.Name, - ci.K8sNamespace: replicaSet.Namespace, - ci.Version: "0", + ci.ClusterNameKey: clusterName, + ci.MetricType: ci.TypeClusterReplicaSet, + ci.Timestamp: timestampNs, + ci.AttributePodName: replicaSet.Name, + ci.AttributeK8sNamespace: replicaSet.Namespace, + ci.Version: "0", } if k.nodeName != "" { attributes[ci.NodeNameKey] = k.nodeName @@ -305,12 +305,12 @@ func (k *K8sAPIServer) getPendingPodStatusMetrics(clusterName, timestampNs strin } attributes := map[string]string{ - ci.ClusterNameKey: clusterName, - ci.MetricType: ci.TypePod, - ci.Timestamp: timestampNs, - ci.PodNameKey: podInfo.Name, - ci.K8sNamespace: podInfo.Namespace, - ci.Version: "0", + ci.ClusterNameKey: clusterName, + ci.MetricType: ci.TypePod, + ci.Timestamp: timestampNs, + ci.AttributePodName: podInfo.Name, + ci.AttributeK8sNamespace: podInfo.Namespace, + ci.Version: "0", } podKey := k8sutil.CreatePodKey(podInfo.Namespace, podInfo.Name) @@ -333,7 +333,7 @@ func (k *K8sAPIServer) getPendingPodStatusMetrics(clusterName, timestampNs strin if err != nil { k.logger.Warn("Error parsing kubernetes blob for pod metrics") } else { - attributes[ci.Kubernetes] = string(kubernetesInfo) + attributes[ci.AttributeKubernetes] = string(kubernetesInfo) } } attributes[ci.SourcesKey] = "[\"apiserver\"]" @@ -405,9 +405,9 @@ func (k *K8sAPIServer) getKubernetesBlob(pod *k8sclient.PodInfo, kubernetesBlob } } - attributes[ci.PodNameKey] = podName + attributes[ci.AttributePodName] = podName if k.addFullPodNameMetricLabel { - attributes[ci.FullPodNameKey] = pod.Name + attributes[ci.AttributeFullPodName] = pod.Name kubernetesBlob["pod_name"] = pod.Name } } diff --git a/receiver/awscontainerinsightreceiver/internal/k8sapiserver/k8sapiserver_test.go b/receiver/awscontainerinsightreceiver/internal/k8sapiserver/k8sapiserver_test.go index 2c3cd170d689..104fd147a204 100644 --- a/receiver/awscontainerinsightreceiver/internal/k8sapiserver/k8sapiserver_test.go +++ b/receiver/awscontainerinsightreceiver/internal/k8sapiserver/k8sapiserver_test.go @@ -356,39 +356,39 @@ func TestK8sAPIServer_GetMetrics(t *testing.T) { case ci.TypeClusterService: assertMetricValueEqual(t, metric, "service_number_of_running_pods", int64(1)) assert.Contains(t, []string{"service1", "service2"}, getStringAttrVal(metric, ci.TypeService)) - assert.Equal(t, "kube-system", getStringAttrVal(metric, ci.K8sNamespace)) + assert.Equal(t, "kube-system", getStringAttrVal(metric, ci.AttributeK8sNamespace)) case ci.TypeClusterNamespace: assertMetricValueEqual(t, metric, "namespace_number_of_running_pods", int64(2)) - assert.Equal(t, "default", getStringAttrVal(metric, ci.K8sNamespace)) + assert.Equal(t, "default", getStringAttrVal(metric, ci.AttributeK8sNamespace)) case ci.TypeClusterDeployment: assertMetricValueEqual(t, metric, "replicas_desired", int64(11)) assertMetricValueEqual(t, metric, "replicas_ready", int64(10)) assertMetricValueEqual(t, metric, "status_replicas_available", int64(9)) assertMetricValueEqual(t, metric, "status_replicas_unavailable", int64(2)) - assert.Equal(t, "kube-system", getStringAttrVal(metric, ci.K8sNamespace)) - assert.Equal(t, "deployment1", getStringAttrVal(metric, ci.PodNameKey)) + assert.Equal(t, "kube-system", getStringAttrVal(metric, ci.AttributeK8sNamespace)) + assert.Equal(t, "deployment1", getStringAttrVal(metric, ci.AttributePodName)) assert.Equal(t, "ClusterDeployment", getStringAttrVal(metric, ci.MetricType)) case ci.TypeClusterDaemonSet: assertMetricValueEqual(t, metric, "replicas_desired", int64(7)) assertMetricValueEqual(t, metric, "replicas_ready", int64(6)) assertMetricValueEqual(t, metric, "status_replicas_available", int64(10)) assertMetricValueEqual(t, metric, "status_replicas_unavailable", int64(4)) - assert.Equal(t, "kube-system", getStringAttrVal(metric, ci.K8sNamespace)) - assert.Equal(t, "daemonset1", getStringAttrVal(metric, ci.PodNameKey)) + assert.Equal(t, "kube-system", getStringAttrVal(metric, ci.AttributeK8sNamespace)) + assert.Equal(t, "daemonset1", getStringAttrVal(metric, ci.AttributePodName)) assert.Equal(t, "ClusterDaemonSet", getStringAttrVal(metric, ci.MetricType)) case ci.TypeClusterReplicaSet: assertMetricValueEqual(t, metric, "replicas_desired", int64(9)) assertMetricValueEqual(t, metric, "replicas_ready", int64(4)) assertMetricValueEqual(t, metric, "status_replicas_available", int64(3)) - assert.Equal(t, "kube-system", getStringAttrVal(metric, ci.K8sNamespace)) - assert.Equal(t, "replicaset1", getStringAttrVal(metric, ci.PodNameKey)) + assert.Equal(t, "kube-system", getStringAttrVal(metric, ci.AttributeK8sNamespace)) + assert.Equal(t, "replicaset1", getStringAttrVal(metric, ci.AttributePodName)) assert.Equal(t, "ClusterReplicaSet", getStringAttrVal(metric, ci.MetricType)) case ci.TypeClusterStatefulSet: assertMetricValueEqual(t, metric, "replicas_desired", int64(10)) assertMetricValueEqual(t, metric, "replicas_ready", int64(4)) assertMetricValueEqual(t, metric, "status_replicas_available", int64(1)) - assert.Equal(t, "kube-system", getStringAttrVal(metric, ci.K8sNamespace)) - assert.Equal(t, "statefulset1", getStringAttrVal(metric, ci.PodNameKey)) + assert.Equal(t, "kube-system", getStringAttrVal(metric, ci.AttributeK8sNamespace)) + assert.Equal(t, "statefulset1", getStringAttrVal(metric, ci.AttributePodName)) assert.Equal(t, "ClusterStatefulSet", getStringAttrVal(metric, ci.MetricType)) case ci.TypePod: assertMetricValueEqual(t, metric, "pod_status_pending", int64(1)) @@ -398,7 +398,7 @@ func TestK8sAPIServer_GetMetrics(t *testing.T) { assertMetricValueEqual(t, metric, "pod_status_scheduled", int64(0)) assertMetricValueEqual(t, metric, "pod_status_succeeded", int64(0)) assertMetricValueEqual(t, metric, "pod_status_unknown", int64(0)) - assert.Equal(t, "kube-system", getStringAttrVal(metric, ci.K8sNamespace)) + assert.Equal(t, "kube-system", getStringAttrVal(metric, ci.AttributeK8sNamespace)) assert.Equal(t, "Pending", getStringAttrVal(metric, "pod_status")) assert.Equal(t, "Pod", getStringAttrVal(metric, ci.MetricType)) default: diff --git a/receiver/awscontainerinsightreceiver/internal/k8sapiserver/prometheus_scraper_test.go b/receiver/awscontainerinsightreceiver/internal/k8sapiserver/prometheus_scraper_test.go index 269341d61d80..3a0a01b76db2 100644 --- a/receiver/awscontainerinsightreceiver/internal/k8sapiserver/prometheus_scraper_test.go +++ b/receiver/awscontainerinsightreceiver/internal/k8sapiserver/prometheus_scraper_test.go @@ -60,8 +60,10 @@ func (m mockConsumer) ConsumeMetrics(_ context.Context, md pmetric.Metrics) erro assert.Equal(m.t, 1, md.ResourceMetrics().Len()) scopeMetrics := md.ResourceMetrics().At(0).ScopeMetrics().At(0).Metrics() + fmt.Printf("===== count %d\n", scopeMetrics.Len()) for i := 0; i < scopeMetrics.Len(); i++ { metric := scopeMetrics.At(i) + fmt.Printf("===== count %v\n", metric.Name()) if metric.Name() == "http_connected_total" { assert.Equal(m.t, float64(15), metric.Sum().DataPoints().At(0).DoubleValue()) *m.httpConnected = true diff --git a/receiver/awscontainerinsightreceiver/internal/stores/podstore.go b/receiver/awscontainerinsightreceiver/internal/stores/podstore.go index 41b182f181be..fa5577f93ebf 100644 --- a/receiver/awscontainerinsightreceiver/internal/stores/podstore.go +++ b/receiver/awscontainerinsightreceiver/internal/stores/podstore.go @@ -217,7 +217,7 @@ func (p *PodStore) RefreshTick(ctx context.Context) { func (p *PodStore) Decorate(ctx context.Context, metric CIMetric, kubernetesBlob map[string]any) bool { if metric.GetTag(ci.MetricType) == ci.TypeNode { p.decorateNode(metric) - } else if metric.GetTag(ci.K8sPodNameKey) != "" { + } else if metric.GetTag(ci.AttributeK8sPodName) != "" { podKey := createPodKeyFromMetric(metric) if podKey == "" { p.logger.Error("podKey is unavailable when decorating pod") @@ -414,7 +414,7 @@ func (p *PodStore) decorateCPU(metric CIMetric, pod *corev1.Pod) { // add cpu limit and request for container if metric.HasField(ci.MetricName(ci.TypeContainer, ci.CPUTotal)) { containerCPUTotal := metric.GetField(ci.MetricName(ci.TypeContainer, ci.CPUTotal)) - if containerName := metric.GetTag(ci.ContainerNamekey); containerName != "" { + if containerName := metric.GetTag(ci.AttributeContainerName); containerName != "" { for _, containerSpec := range pod.Spec.Containers { if containerSpec.Name == containerName { if containerCPULimit, ok := getLimitForContainer(cpuKey, containerSpec); ok { @@ -463,7 +463,7 @@ func (p *PodStore) decorateMem(metric CIMetric, pod *corev1.Pod) { memWorkingsetMetric := ci.MetricName(ci.TypeContainer, ci.MemWorkingset) if metric.HasField(memWorkingsetMetric) { containerMemWorkingset := metric.GetField(memWorkingsetMetric) - if containerName := metric.GetTag(ci.ContainerNamekey); containerName != "" { + if containerName := metric.GetTag(ci.AttributeContainerName); containerName != "" { for _, containerSpec := range pod.Spec.Containers { if containerSpec.Name == containerName { if containerMemLimit, ok := getLimitForContainer(memoryKey, containerSpec); ok { @@ -511,7 +511,7 @@ func (p *PodStore) addStatus(metric CIMetric, pod *corev1.Pod) { p.setPrevMeasurement(ci.TypePod, podKey, prevPodMeasurement{containersRestarts: curContainerRestarts}) } } else if metric.GetTag(ci.MetricType) == ci.TypeContainer { - if containerName := metric.GetTag(ci.ContainerNamekey); containerName != "" { + if containerName := metric.GetTag(ci.AttributeContainerName); containerName != "" { for _, containerStatus := range pod.Status.ContainerStatuses { if containerStatus.Name == containerName { switch { @@ -677,10 +677,10 @@ func getRequestForContainer(resource corev1.ResourceName, spec corev1.Container) } func addContainerID(pod *corev1.Pod, metric CIMetric, kubernetesBlob map[string]any, logger *zap.Logger) { - if containerName := metric.GetTag(ci.ContainerNamekey); containerName != "" { + if containerName := metric.GetTag(ci.AttributeContainerName); containerName != "" { rawID := "" for _, container := range pod.Status.ContainerStatuses { - if metric.GetTag(ci.ContainerNamekey) == container.Name { + if metric.GetTag(ci.AttributeContainerName) == container.Name { rawID = container.ContainerID if rawID != "" { ids := strings.Split(rawID, "://") @@ -695,9 +695,9 @@ func addContainerID(pod *corev1.Pod, metric CIMetric, kubernetesBlob map[string] } } if rawID == "" { - kubernetesBlob["container_id"] = metric.GetTag(ci.ContainerIDkey) + kubernetesBlob["container_id"] = metric.GetTag(ci.AttributeContainerID) } - metric.RemoveTag(ci.ContainerIDkey) + metric.RemoveTag(ci.AttributeContainerID) } } @@ -766,9 +766,9 @@ func (p *PodStore) addPodOwnersAndPodName(metric CIMetric, pod *corev1.Pod, kube } } - metric.AddTag(ci.PodNameKey, podName) + metric.AddTag(ci.AttributePodName, podName) if p.addFullPodNameMetricLabel { - metric.AddTag(ci.FullPodNameKey, pod.Name) + metric.AddTag(ci.AttributeFullPodName, pod.Name) } } diff --git a/receiver/awscontainerinsightreceiver/internal/stores/podstore_test.go b/receiver/awscontainerinsightreceiver/internal/stores/podstore_test.go index 456c0d567715..4f18cbeb90b3 100644 --- a/receiver/awscontainerinsightreceiver/internal/stores/podstore_test.go +++ b/receiver/awscontainerinsightreceiver/internal/stores/podstore_test.go @@ -231,7 +231,7 @@ func TestPodStore_decorateCpu(t *testing.T) { assert.Equal(t, float64(1), metric.GetField("pod_cpu_usage_total").(float64)) // test container metrics - tags = map[string]string{ci.MetricType: ci.TypeContainer, ci.ContainerNamekey: "ubuntu"} + tags = map[string]string{ci.MetricType: ci.TypeContainer, ci.AttributeContainerName: "ubuntu"} fields = map[string]any{ci.MetricName(ci.TypeContainer, ci.CPUTotal): float64(1)} metric = generateMetric(fields, tags) podStore.decorateCPU(metric, pod) @@ -264,7 +264,7 @@ func TestPodStore_decorateMem(t *testing.T) { assert.Equal(t, float64(20), metric.GetField("pod_memory_utilization_over_pod_limit").(float64)) assert.Equal(t, uint64(10*1024*1024), metric.GetField("pod_memory_working_set").(uint64)) - tags = map[string]string{ci.MetricType: ci.TypeContainer, ci.ContainerNamekey: "ubuntu"} + tags = map[string]string{ci.MetricType: ci.TypeContainer, ci.AttributeContainerName: "ubuntu"} fields = map[string]any{ci.MetricName(ci.TypeContainer, ci.MemWorkingset): uint64(10 * 1024 * 1024)} metric = generateMetric(fields, tags) @@ -287,7 +287,7 @@ func TestPodStore_previousCleanupLocking(_ *testing.T) { pod := getBaseTestPodInfo() ctx := context.TODO() - tags := map[string]string{ci.MetricType: ci.TypePod, ci.K8sNamespace: "default", ci.K8sPodNameKey: "cpu-limit"} + tags := map[string]string{ci.MetricType: ci.TypePod, ci.AttributeK8sNamespace: "default", ci.AttributeK8sPodName: "cpu-limit"} fields := map[string]any{ci.MetricName(ci.TypePod, ci.CPUTotal): float64(1)} metric := generateMetric(fields, tags) @@ -427,7 +427,7 @@ func TestPodStore_addStatus_enhanced_metrics(t *testing.T) { containerCopy.Name = "ubuntu2" pod.Status.ContainerStatuses = append(pod.Status.ContainerStatuses, containerCopy) - tags := map[string]string{ci.MetricType: ci.TypePod, ci.K8sNamespace: "default", ci.K8sPodNameKey: "cpu-limit"} + tags := map[string]string{ci.MetricType: ci.TypePod, ci.AttributeK8sNamespace: "default", ci.AttributeK8sPodName: "cpu-limit"} fields := map[string]any{ci.MetricName(ci.TypePod, ci.CPUTotal): float64(1)} podStore := getPodStore() podStore.includeEnhancedMetrics = true @@ -440,21 +440,21 @@ func TestPodStore_addStatus_enhanced_metrics(t *testing.T) { assert.Nil(t, val) // set up container defaults - tags = map[string]string{ci.MetricType: ci.TypeContainer, ci.K8sNamespace: "default", ci.K8sPodNameKey: "cpu-limit", ci.ContainerNamekey: "ubuntu"} + tags = map[string]string{ci.MetricType: ci.TypeContainer, ci.AttributeK8sNamespace: "default", ci.AttributeK8sPodName: "cpu-limit", ci.AttributeContainerName: "ubuntu"} metric = generateMetric(fields, tags) podStore.addStatus(metric, pod) assert.Equal(t, "Running", metric.GetTag(ci.ContainerStatus)) val = metric.GetField(ci.ContainerRestartCount) assert.Nil(t, val) // set up the other container - tags = map[string]string{ci.MetricType: ci.TypeContainer, ci.K8sNamespace: "default", ci.K8sPodNameKey: "cpu-limit", ci.ContainerNamekey: "ubuntu2"} + tags = map[string]string{ci.MetricType: ci.TypeContainer, ci.AttributeK8sNamespace: "default", ci.AttributeK8sPodName: "cpu-limit", ci.AttributeContainerName: "ubuntu2"} metric = generateMetric(fields, tags) podStore.addStatus(metric, pod) assert.Equal(t, "Running", metric.GetTag(ci.ContainerStatus)) val = metric.GetField(ci.ContainerRestartCount) assert.Nil(t, val) - tags = map[string]string{ci.MetricType: ci.TypePod, ci.K8sNamespace: "default", ci.K8sPodNameKey: "cpu-limit"} + tags = map[string]string{ci.MetricType: ci.TypePod, ci.AttributeK8sNamespace: "default", ci.AttributeK8sPodName: "cpu-limit"} metric = generateMetric(fields, tags) podStore.addStatus(metric, pod) @@ -475,7 +475,7 @@ func TestPodStore_addStatus_enhanced_metrics(t *testing.T) { pod.Status.ContainerStatuses[1].RestartCount = 1 pod.Status.Phase = "Succeeded" - tags = map[string]string{ci.MetricType: ci.TypePod, ci.K8sNamespace: "default", ci.K8sPodNameKey: "cpu-limit"} + tags = map[string]string{ci.MetricType: ci.TypePod, ci.AttributeK8sNamespace: "default", ci.AttributeK8sPodName: "cpu-limit"} metric = generateMetric(fields, tags) podStore.addStatus(metric, pod) @@ -484,18 +484,18 @@ func TestPodStore_addStatus_enhanced_metrics(t *testing.T) { // update the container metrics // set up container defaults - tags = map[string]string{ci.MetricType: ci.TypeContainer, ci.K8sNamespace: "default", ci.K8sPodNameKey: "cpu-limit", ci.ContainerNamekey: "ubuntu"} + tags = map[string]string{ci.MetricType: ci.TypeContainer, ci.AttributeK8sNamespace: "default", ci.AttributeK8sPodName: "cpu-limit", ci.AttributeContainerName: "ubuntu"} metric = generateMetric(fields, tags) podStore.addStatus(metric, pod) assert.Equal(t, 1, metric.GetField(ci.ContainerRestartCount)) // test the other container - tags = map[string]string{ci.MetricType: ci.TypeContainer, ci.K8sNamespace: "default", ci.K8sPodNameKey: "cpu-limit", ci.ContainerNamekey: "ubuntu2"} + tags = map[string]string{ci.MetricType: ci.TypeContainer, ci.AttributeK8sNamespace: "default", ci.AttributeK8sPodName: "cpu-limit", ci.AttributeContainerName: "ubuntu2"} metric = generateMetric(fields, tags) podStore.addStatus(metric, pod) assert.Equal(t, 1, metric.GetField(ci.ContainerRestartCount)) - tags = map[string]string{ci.MetricType: ci.TypePod, ci.K8sNamespace: "default", ci.K8sPodNameKey: "cpu-limit"} + tags = map[string]string{ci.MetricType: ci.TypePod, ci.AttributeK8sNamespace: "default", ci.AttributeK8sPodName: "cpu-limit"} metric = generateMetric(fields, tags) podStore.addStatus(metric, pod) @@ -507,7 +507,7 @@ func TestPodStore_addStatus_enhanced_metrics(t *testing.T) { pod.Status.ContainerStatuses[1].LastTerminationState.Terminated = nil pod.Status.ContainerStatuses[1].State.Waiting = &corev1.ContainerStateWaiting{Reason: "CrashLoopBackOff"} - tags = map[string]string{ci.MetricType: ci.TypePod, ci.K8sNamespace: "default", ci.K8sPodNameKey: "cpu-limit"} + tags = map[string]string{ci.MetricType: ci.TypePod, ci.AttributeK8sNamespace: "default", ci.AttributeK8sPodName: "cpu-limit"} metric = generateMetric(fields, tags) podStore.addStatus(metric, pod) @@ -523,7 +523,7 @@ func TestPodStore_addStatus_enhanced_metrics(t *testing.T) { pod.Status.ContainerStatuses[0].State.Waiting = &corev1.ContainerStateWaiting{Reason: "ImagePullBackOff"} pod.Status.ContainerStatuses[1].State.Waiting = &corev1.ContainerStateWaiting{Reason: "StartError"} - tags = map[string]string{ci.MetricType: ci.TypePod, ci.K8sNamespace: "default", ci.K8sPodNameKey: "cpu-limit"} + tags = map[string]string{ci.MetricType: ci.TypePod, ci.AttributeK8sNamespace: "default", ci.AttributeK8sPodName: "cpu-limit"} metric = generateMetric(fields, tags) podStore.addStatus(metric, pod) @@ -560,13 +560,13 @@ func TestPodStore_addStatus_enhanced_metrics(t *testing.T) { // test delta of restartCount pod.Status.ContainerStatuses[0].RestartCount = 3 - tags = map[string]string{ci.MetricType: ci.TypePod, ci.K8sNamespace: "default", ci.K8sPodNameKey: "cpu-limit"} + tags = map[string]string{ci.MetricType: ci.TypePod, ci.AttributeK8sNamespace: "default", ci.AttributeK8sPodName: "cpu-limit"} metric = generateMetric(fields, tags) podStore.addStatus(metric, pod) assert.Equal(t, 2, metric.GetField(ci.MetricName(ci.TypePod, ci.ContainerRestartCount))) - tags = map[string]string{ci.MetricType: ci.TypeContainer, ci.K8sNamespace: "default", ci.K8sPodNameKey: "cpu-limit", ci.ContainerNamekey: "ubuntu"} + tags = map[string]string{ci.MetricType: ci.TypeContainer, ci.AttributeK8sNamespace: "default", ci.AttributeK8sPodName: "cpu-limit", ci.AttributeContainerName: "ubuntu"} metric = generateMetric(fields, tags) podStore.addStatus(metric, pod) @@ -575,7 +575,7 @@ func TestPodStore_addStatus_enhanced_metrics(t *testing.T) { func TestPodStore_addStatus_without_enhanced_metrics(t *testing.T) { pod := getBaseTestPodInfo() - tags := map[string]string{ci.MetricType: ci.TypePod, ci.K8sNamespace: "default", ci.K8sPodNameKey: "cpu-limit"} + tags := map[string]string{ci.MetricType: ci.TypePod, ci.AttributeK8sNamespace: "default", ci.AttributeK8sPodName: "cpu-limit"} fields := map[string]any{ci.MetricName(ci.TypePod, ci.CPUTotal): float64(1)} podStore := getPodStore() podStore.includeEnhancedMetrics = false @@ -586,7 +586,7 @@ func TestPodStore_addStatus_without_enhanced_metrics(t *testing.T) { val := metric.GetField(ci.MetricName(ci.TypePod, ci.ContainerRestartCount)) assert.Nil(t, val) - tags = map[string]string{ci.MetricType: ci.TypeContainer, ci.K8sNamespace: "default", ci.K8sPodNameKey: "cpu-limit", ci.ContainerNamekey: "ubuntu"} + tags = map[string]string{ci.MetricType: ci.TypeContainer, ci.AttributeK8sNamespace: "default", ci.AttributeK8sPodName: "cpu-limit", ci.AttributeContainerName: "ubuntu"} metric = generateMetric(fields, tags) podStore.addStatus(metric, pod) @@ -601,14 +601,14 @@ func TestPodStore_addStatus_without_enhanced_metrics(t *testing.T) { pod.Status.ContainerStatuses[0].RestartCount = 1 pod.Status.Phase = "Succeeded" - tags = map[string]string{ci.MetricType: ci.TypePod, ci.K8sNamespace: "default", ci.K8sPodNameKey: "cpu-limit"} + tags = map[string]string{ci.MetricType: ci.TypePod, ci.AttributeK8sNamespace: "default", ci.AttributeK8sPodName: "cpu-limit"} metric = generateMetric(fields, tags) podStore.addStatus(metric, pod) assert.Equal(t, "Succeeded", metric.GetTag(ci.PodStatus)) assert.Equal(t, int(1), metric.GetField(ci.MetricName(ci.TypePod, ci.ContainerRestartCount)).(int)) - tags = map[string]string{ci.MetricType: ci.TypeContainer, ci.K8sNamespace: "default", ci.K8sPodNameKey: "cpu-limit", ci.ContainerNamekey: "ubuntu"} + tags = map[string]string{ci.MetricType: ci.TypeContainer, ci.AttributeK8sNamespace: "default", ci.AttributeK8sPodName: "cpu-limit", ci.AttributeContainerName: "ubuntu"} metric = generateMetric(fields, tags) podStore.addStatus(metric, pod) @@ -620,7 +620,7 @@ func TestPodStore_addStatus_without_enhanced_metrics(t *testing.T) { pod.Status.ContainerStatuses[0].State.Terminated = nil pod.Status.ContainerStatuses[0].State.Waiting = &corev1.ContainerStateWaiting{Reason: "CrashLoopBackOff"} - tags = map[string]string{ci.MetricType: ci.TypeContainer, ci.K8sNamespace: "default", ci.K8sPodNameKey: "cpu-limit", ci.ContainerNamekey: "ubuntu"} + tags = map[string]string{ci.MetricType: ci.TypeContainer, ci.AttributeK8sNamespace: "default", ci.AttributeK8sPodName: "cpu-limit", ci.AttributeContainerName: "ubuntu"} metric = generateMetric(fields, tags) podStore.addStatus(metric, pod) @@ -630,7 +630,7 @@ func TestPodStore_addStatus_without_enhanced_metrics(t *testing.T) { pod.Status.ContainerStatuses[0].State.Waiting = &corev1.ContainerStateWaiting{Reason: "SomeOtherReason"} - tags = map[string]string{ci.MetricType: ci.TypeContainer, ci.K8sNamespace: "default", ci.K8sPodNameKey: "cpu-limit", ci.ContainerNamekey: "ubuntu"} + tags = map[string]string{ci.MetricType: ci.TypeContainer, ci.AttributeK8sNamespace: "default", ci.AttributeK8sPodName: "cpu-limit", ci.AttributeContainerName: "ubuntu"} metric = generateMetric(fields, tags) podStore.addStatus(metric, pod) @@ -640,13 +640,13 @@ func TestPodStore_addStatus_without_enhanced_metrics(t *testing.T) { // test delta of restartCount pod.Status.ContainerStatuses[0].RestartCount = 3 - tags = map[string]string{ci.MetricType: ci.TypePod, ci.K8sNamespace: "default", ci.K8sPodNameKey: "cpu-limit"} + tags = map[string]string{ci.MetricType: ci.TypePod, ci.AttributeK8sNamespace: "default", ci.AttributeK8sPodName: "cpu-limit"} metric = generateMetric(fields, tags) podStore.addStatus(metric, pod) assert.Equal(t, int(2), metric.GetField(ci.MetricName(ci.TypePod, ci.ContainerRestartCount)).(int)) - tags = map[string]string{ci.MetricType: ci.TypeContainer, ci.K8sNamespace: "default", ci.K8sPodNameKey: "cpu-limit", ci.ContainerNamekey: "ubuntu"} + tags = map[string]string{ci.MetricType: ci.TypeContainer, ci.AttributeK8sNamespace: "default", ci.AttributeK8sPodName: "cpu-limit", ci.AttributeContainerName: "ubuntu"} metric = generateMetric(fields, tags) podStore.addStatus(metric, pod) @@ -655,7 +655,7 @@ func TestPodStore_addStatus_without_enhanced_metrics(t *testing.T) { func TestPodStore_addContainerID(t *testing.T) { pod := getBaseTestPodInfo() - tags := map[string]string{ci.ContainerNamekey: "ubuntu", ci.ContainerIDkey: "123"} + tags := map[string]string{ci.AttributeContainerName: "ubuntu", ci.AttributeContainerID: "123"} fields := map[string]any{ci.MetricName(ci.TypePod, ci.CPUTotal): float64(1)} kubernetesBlob := map[string]any{} metric := generateMetric(fields, tags) @@ -664,9 +664,9 @@ func TestPodStore_addContainerID(t *testing.T) { expected := map[string]any{} expected["docker"] = map[string]string{"container_id": "637631e2634ea92c0c1aa5d24734cfe794f09c57933026592c12acafbaf6972c"} assert.Equal(t, expected, kubernetesBlob) - assert.Equal(t, metric.GetTag(ci.ContainerNamekey), "ubuntu") + assert.Equal(t, metric.GetTag(ci.AttributeContainerName), "ubuntu") - tags = map[string]string{ci.ContainerNamekey: "notUbuntu", ci.ContainerIDkey: "123"} + tags = map[string]string{ci.AttributeContainerName: "notUbuntu", ci.AttributeContainerID: "123"} kubernetesBlob = map[string]any{} metric = generateMetric(fields, tags) addContainerID(pod, metric, kubernetesBlob, zap.NewNop()) @@ -674,7 +674,7 @@ func TestPodStore_addContainerID(t *testing.T) { expected = map[string]any{} expected["container_id"] = "123" assert.Equal(t, expected, kubernetesBlob) - assert.Equal(t, metric.GetTag(ci.ContainerNamekey), "notUbuntu") + assert.Equal(t, metric.GetTag(ci.AttributeContainerName), "notUbuntu") } func TestPodStore_addLabel(t *testing.T) { @@ -734,7 +734,7 @@ func (m *mockK8sClient2) GetReplicaSetClient() k8sclient.ReplicaSetClient { func TestPodStore_addPodOwnersAndPodNameFallback(t *testing.T) { podStore := &PodStore{k8sClient: &mockK8sClient1{}} pod := getBaseTestPodInfo() - tags := map[string]string{ci.MetricType: ci.TypePod, ci.ContainerNamekey: "ubuntu"} + tags := map[string]string{ci.MetricType: ci.TypePod, ci.AttributeContainerName: "ubuntu"} fields := map[string]any{ci.MetricName(ci.TypePod, ci.CPUTotal): float64(1)} metric := generateMetric(fields, tags) @@ -748,7 +748,7 @@ func TestPodStore_addPodOwnersAndPodNameFallback(t *testing.T) { expectedOwner := map[string]any{} expectedOwner["pod_owners"] = []any{map[string]string{"owner_kind": ci.Deployment, "owner_name": rsName}} expectedOwnerName := rsName - assert.Equal(t, expectedOwnerName, metric.GetTag(ci.PodNameKey)) + assert.Equal(t, expectedOwnerName, metric.GetTag(ci.AttributePodName)) assert.Equal(t, expectedOwner, kubernetesBlob) // Test Job @@ -761,7 +761,7 @@ func TestPodStore_addPodOwnersAndPodNameFallback(t *testing.T) { podStore.addPodOwnersAndPodName(metric, pod, kubernetesBlob) expectedOwner["pod_owners"] = []any{map[string]string{"owner_kind": ci.CronJob, "owner_name": jobName}} expectedOwnerName = jobName - assert.Equal(t, expectedOwnerName, metric.GetTag(ci.PodNameKey)) + assert.Equal(t, expectedOwnerName, metric.GetTag(ci.AttributePodName)) assert.Equal(t, expectedOwner, kubernetesBlob) } @@ -769,7 +769,7 @@ func TestPodStore_addPodOwnersAndPodName(t *testing.T) { podStore := &PodStore{k8sClient: &mockK8sClient2{}} pod := getBaseTestPodInfo() - tags := map[string]string{ci.MetricType: ci.TypePod, ci.ContainerNamekey: "ubuntu"} + tags := map[string]string{ci.MetricType: ci.TypePod, ci.AttributeContainerName: "ubuntu"} fields := map[string]any{ci.MetricName(ci.TypePod, ci.CPUTotal): float64(1)} // Test DaemonSet @@ -780,7 +780,7 @@ func TestPodStore_addPodOwnersAndPodName(t *testing.T) { expectedOwner := map[string]any{} expectedOwner["pod_owners"] = []any{map[string]string{"owner_kind": ci.DaemonSet, "owner_name": "DaemonSetTest"}} expectedOwnerName := "DaemonSetTest" - assert.Equal(t, expectedOwnerName, metric.GetTag(ci.PodNameKey)) + assert.Equal(t, expectedOwnerName, metric.GetTag(ci.AttributePodName)) assert.Equal(t, expectedOwner, kubernetesBlob) // Test ReplicaSet @@ -792,7 +792,7 @@ func TestPodStore_addPodOwnersAndPodName(t *testing.T) { podStore.addPodOwnersAndPodName(metric, pod, kubernetesBlob) expectedOwner["pod_owners"] = []any{map[string]string{"owner_kind": ci.ReplicaSet, "owner_name": rsName}} expectedOwnerName = rsName - assert.Equal(t, expectedOwnerName, metric.GetTag(ci.PodNameKey)) + assert.Equal(t, expectedOwnerName, metric.GetTag(ci.AttributePodName)) assert.Equal(t, expectedOwner, kubernetesBlob) // Test StatefulSet @@ -804,11 +804,11 @@ func TestPodStore_addPodOwnersAndPodName(t *testing.T) { podStore.addPodOwnersAndPodName(metric, pod, kubernetesBlob) expectedOwner["pod_owners"] = []any{map[string]string{"owner_kind": ci.StatefulSet, "owner_name": ssName}} expectedOwnerName = "cpu-limit" - assert.Equal(t, expectedOwnerName, metric.GetTag(ci.PodNameKey)) + assert.Equal(t, expectedOwnerName, metric.GetTag(ci.AttributePodName)) assert.Equal(t, expectedOwner, kubernetesBlob) // Test ReplicationController - pod.Name = "this should not be in FullPodNameKey" + pod.Name = "this should not be in FullPodName" rcName := "ReplicationControllerTest" pod.OwnerReferences[0].Kind = ci.ReplicationController pod.OwnerReferences[0].Name = rcName @@ -816,8 +816,8 @@ func TestPodStore_addPodOwnersAndPodName(t *testing.T) { podStore.addPodOwnersAndPodName(metric, pod, kubernetesBlob) expectedOwner["pod_owners"] = []any{map[string]string{"owner_kind": ci.ReplicationController, "owner_name": rcName}} expectedOwnerName = rcName - assert.Equal(t, expectedOwnerName, metric.GetTag(ci.PodNameKey)) - assert.Equal(t, "", metric.GetTag(ci.FullPodNameKey)) + assert.Equal(t, expectedOwnerName, metric.GetTag(ci.AttributePodName)) + assert.Equal(t, "", metric.GetTag(ci.AttributeFullPodName)) assert.Equal(t, expectedOwner, kubernetesBlob) // Test Job @@ -833,8 +833,8 @@ func TestPodStore_addPodOwnersAndPodName(t *testing.T) { podStore.addPodOwnersAndPodName(metric, pod, kubernetesBlob) expectedOwner["pod_owners"] = []any{map[string]string{"owner_kind": ci.Job, "owner_name": jobName + surfixHash}} expectedOwnerName = jobName + surfixHash - assert.Equal(t, expectedOwnerName, metric.GetTag(ci.PodNameKey)) - assert.Equal(t, pod.Name, metric.GetTag(ci.FullPodNameKey)) + assert.Equal(t, expectedOwnerName, metric.GetTag(ci.AttributePodName)) + assert.Equal(t, pod.Name, metric.GetTag(ci.AttributeFullPodName)) assert.Equal(t, expectedOwner, kubernetesBlob) podStore.prefFullPodName = false @@ -842,7 +842,7 @@ func TestPodStore_addPodOwnersAndPodName(t *testing.T) { podStore.addPodOwnersAndPodName(metric, pod, kubernetesBlob) expectedOwner["pod_owners"] = []any{map[string]string{"owner_kind": ci.Job, "owner_name": jobName}} expectedOwnerName = jobName - assert.Equal(t, expectedOwnerName, metric.GetTag(ci.PodNameKey)) + assert.Equal(t, expectedOwnerName, metric.GetTag(ci.AttributePodName)) assert.Equal(t, expectedOwner, kubernetesBlob) // Test Deployment @@ -854,7 +854,7 @@ func TestPodStore_addPodOwnersAndPodName(t *testing.T) { podStore.addPodOwnersAndPodName(metric, pod, kubernetesBlob) expectedOwner["pod_owners"] = []any{map[string]string{"owner_kind": ci.Deployment, "owner_name": dpName}} expectedOwnerName = dpName - assert.Equal(t, expectedOwnerName, metric.GetTag(ci.PodNameKey)) + assert.Equal(t, expectedOwnerName, metric.GetTag(ci.AttributePodName)) assert.Equal(t, expectedOwner, kubernetesBlob) // Test CronJob @@ -866,7 +866,7 @@ func TestPodStore_addPodOwnersAndPodName(t *testing.T) { podStore.addPodOwnersAndPodName(metric, pod, kubernetesBlob) expectedOwner["pod_owners"] = []any{map[string]string{"owner_kind": ci.CronJob, "owner_name": cjName}} expectedOwnerName = cjName - assert.Equal(t, expectedOwnerName, metric.GetTag(ci.PodNameKey)) + assert.Equal(t, expectedOwnerName, metric.GetTag(ci.AttributePodName)) assert.Equal(t, expectedOwner, kubernetesBlob) // Test kube-proxy created in kops @@ -877,7 +877,7 @@ func TestPodStore_addPodOwnersAndPodName(t *testing.T) { pod.Name = kpName kubernetesBlob = map[string]any{} podStore.addPodOwnersAndPodName(metric, pod, kubernetesBlob) - assert.Equal(t, kpName, metric.GetTag(ci.PodNameKey)) + assert.Equal(t, kpName, metric.GetTag(ci.AttributePodName)) assert.True(t, len(kubernetesBlob) == 0) podStore.prefFullPodName = false @@ -886,7 +886,7 @@ func TestPodStore_addPodOwnersAndPodName(t *testing.T) { pod.Name = kpName kubernetesBlob = map[string]any{} podStore.addPodOwnersAndPodName(metric, pod, kubernetesBlob) - assert.Equal(t, kubeProxy, metric.GetTag(ci.PodNameKey)) + assert.Equal(t, kubeProxy, metric.GetTag(ci.AttributePodName)) assert.True(t, len(kubernetesBlob) == 0) } @@ -1039,10 +1039,10 @@ func TestPodStore_Decorate(t *testing.T) { // metric with no namespace tags = map[string]string{ - ci.ContainerNamekey: "testContainer", - ci.PodIDKey: "123", - ci.K8sPodNameKey: "testPod", - // ci.K8sNamespace: "testNamespace", + ci.AttributeContainerName: "testContainer", + ci.AttributePodID: "123", + ci.AttributeK8sPodName: "testPod", + // ci.AttributeK8sNamespace: "testNamespace", ci.TypeService: "testService", ci.NodeNameKey: "testNode", } @@ -1054,12 +1054,12 @@ func TestPodStore_Decorate(t *testing.T) { // metric with pod info not in cache tags = map[string]string{ - ci.ContainerNamekey: "testContainer", - ci.K8sPodNameKey: "testPod", - ci.PodIDKey: "123", - ci.K8sNamespace: "testNamespace", - ci.TypeService: "testService", - ci.NodeNameKey: "testNode", + ci.AttributeContainerName: "testContainer", + ci.AttributeK8sPodName: "testPod", + ci.AttributePodID: "123", + ci.AttributeK8sNamespace: "testNamespace", + ci.TypeService: "testService", + ci.NodeNameKey: "testNode", } metric = &mockCIMetric{ tags: tags, @@ -1095,7 +1095,7 @@ func generatePodInfo(sourceFileName string) *corev1.Pod { } func generateRawCIMetric() CIMetric { - tags := map[string]string{ci.MetricType: ci.TypePod, ci.K8sNamespace: "default", ci.K8sPodNameKey: "cpu-limit"} + tags := map[string]string{ci.MetricType: ci.TypePod, ci.AttributeK8sNamespace: "default", ci.AttributeK8sPodName: "cpu-limit"} fields := map[string]any{ci.MetricName(ci.TypePod, ci.CPUTotal): float64(1)} return generateMetric(fields, tags) } diff --git a/receiver/awscontainerinsightreceiver/internal/stores/servicestore.go b/receiver/awscontainerinsightreceiver/internal/stores/servicestore.go index cd1d6e69617c..8d28474b0e39 100644 --- a/receiver/awscontainerinsightreceiver/internal/stores/servicestore.go +++ b/receiver/awscontainerinsightreceiver/internal/stores/servicestore.go @@ -53,7 +53,7 @@ func (s *ServiceStore) RefreshTick(ctx context.Context) { // Decorate decorates metrics and update kubernetesBlob // service info is not mandatory func (s *ServiceStore) Decorate(_ context.Context, metric CIMetric, _ map[string]any) bool { - if metric.HasTag(ci.K8sPodNameKey) { + if metric.HasTag(ci.AttributeK8sPodName) { podKey := createPodKeyFromMetric(metric) if podKey == "" { s.logger.Error("podKey is unavailable when decorating service.", zap.Any("podKey", podKey)) diff --git a/receiver/awscontainerinsightreceiver/internal/stores/servicestore_test.go b/receiver/awscontainerinsightreceiver/internal/stores/servicestore_test.go index a7e1604fe496..785247654c42 100644 --- a/receiver/awscontainerinsightreceiver/internal/stores/servicestore_test.go +++ b/receiver/awscontainerinsightreceiver/internal/stores/servicestore_test.go @@ -37,8 +37,8 @@ func TestServiceStore(t *testing.T) { // test the case when it decorates metrics successfully metric := &mockCIMetric{ tags: map[string]string{ - ci.K8sPodNameKey: "test-pod", - ci.K8sNamespace: "default", + ci.AttributeK8sPodName: "test-pod", + ci.AttributeK8sNamespace: "default", }, } kubernetesBlob := map[string]any{} @@ -49,7 +49,7 @@ func TestServiceStore(t *testing.T) { // test the case when it fails to decorate metrics metric = &mockCIMetric{ tags: map[string]string{ - ci.K8sPodNameKey: "test-pod", + ci.AttributeK8sPodName: "test-pod", }, } kubernetesBlob = map[string]any{} diff --git a/receiver/awscontainerinsightreceiver/internal/stores/utils.go b/receiver/awscontainerinsightreceiver/internal/stores/utils.go index f412b3998880..a0db8fe03e93 100644 --- a/receiver/awscontainerinsightreceiver/internal/stores/utils.go +++ b/receiver/awscontainerinsightreceiver/internal/stores/utils.go @@ -6,9 +6,11 @@ package stores // import "github.com/open-telemetry/opentelemetry-collector-cont import ( "context" "encoding/json" + "fmt" "strings" "time" + "go.uber.org/zap" corev1 "k8s.io/api/core/v1" ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" @@ -29,15 +31,15 @@ func createPodKeyFromMetaData(pod *corev1.Pod) string { } func createPodKeyFromMetric(metric CIMetric) string { - namespace := metric.GetTag(ci.K8sNamespace) - podName := metric.GetTag(ci.K8sPodNameKey) + namespace := metric.GetTag(ci.AttributeK8sNamespace) + podName := metric.GetTag(ci.AttributeK8sPodName) return k8sutil.CreatePodKey(namespace, podName) } func createContainerKeyFromMetric(metric CIMetric) string { - namespace := metric.GetTag(ci.K8sNamespace) - podName := metric.GetTag(ci.K8sPodNameKey) - containerName := metric.GetTag(ci.ContainerNamekey) + namespace := metric.GetTag(ci.AttributeK8sNamespace) + podName := metric.GetTag(ci.AttributeK8sPodName) + containerName := metric.GetTag(ci.AttributeContainerName) return k8sutil.CreateContainerKey(namespace, podName, containerName) } @@ -135,13 +137,13 @@ func TagMetricSource(metric CIMetric) { } func AddKubernetesInfo(metric CIMetric, kubernetesBlob map[string]any, retainContainerNameTag bool) { - needMoveToKubernetes := map[string]string{ci.K8sPodNameKey: "pod_name", ci.PodIDKey: "pod_id"} - needCopyToKubernetes := map[string]string{ci.K8sNamespace: "namespace_name", ci.TypeService: "service_name", ci.NodeNameKey: "host"} + needMoveToKubernetes := map[string]string{ci.AttributeK8sPodName: "pod_name", ci.AttributePodID: "pod_id"} + needCopyToKubernetes := map[string]string{ci.AttributeK8sNamespace: "namespace_name", ci.TypeService: "service_name", ci.NodeNameKey: "host"} if retainContainerNameTag { - needCopyToKubernetes[ci.ContainerNamekey] = "container_name" + needCopyToKubernetes[ci.AttributeContainerName] = "container_name" } else { - needMoveToKubernetes[ci.ContainerNamekey] = "container_name" + needMoveToKubernetes[ci.AttributeContainerName] = "container_name" } for k, v := range needMoveToKubernetes { @@ -161,7 +163,7 @@ func AddKubernetesInfo(metric CIMetric, kubernetesBlob map[string]any, retainCon if err != nil { return } - metric.AddTag(ci.Kubernetes, string(kubernetesInfo)) + metric.AddTag(ci.AttributeKubernetes, string(kubernetesInfo)) } } @@ -176,3 +178,92 @@ func refreshWithTimeout(parentContext context.Context, refresh func(), timeout t <-ctx.Done() cancel() } + +type RawContainerInsightsMetric struct { + // source of the metric for debugging merge conflict + ContainerName string + // key/value pairs that are typed and contain the metric (numerical) data + Fields map[string]any + // key/value string pairs that are used to identify the metrics + Tags map[string]string + Logger *zap.Logger +} + +var _ CIMetric = (*RawContainerInsightsMetric)(nil) + +func NewRawContainerInsightsMetric(mType string, logger *zap.Logger) *RawContainerInsightsMetric { + metric := &RawContainerInsightsMetric{ + Fields: make(map[string]any), + Tags: make(map[string]string), + Logger: logger, + } + metric.Tags[ci.MetricType] = mType + return metric +} + +func NewRawContainerInsightsMetricWithData(mType string, fields map[string]any, tags map[string]string, logger *zap.Logger) *RawContainerInsightsMetric { + metric := NewRawContainerInsightsMetric(mType, logger) + metric.Fields = fields + metric.Tags = tags + return metric +} + +func (c *RawContainerInsightsMetric) GetTags() map[string]string { + return c.Tags +} + +func (c *RawContainerInsightsMetric) GetFields() map[string]any { + return c.Fields +} + +func (c *RawContainerInsightsMetric) GetMetricType() string { + return c.Tags[ci.MetricType] +} + +func (c *RawContainerInsightsMetric) AddTags(tags map[string]string) { + for k, v := range tags { + c.Tags[k] = v + } +} + +func (c *RawContainerInsightsMetric) HasField(key string) bool { + return c.Fields[key] != nil +} + +func (c *RawContainerInsightsMetric) AddField(key string, val any) { + c.Fields[key] = val +} + +func (c *RawContainerInsightsMetric) GetField(key string) any { + return c.Fields[key] +} + +func (c *RawContainerInsightsMetric) HasTag(key string) bool { + return c.Tags[key] != "" +} + +func (c *RawContainerInsightsMetric) AddTag(key, val string) { + c.Tags[key] = val +} + +func (c *RawContainerInsightsMetric) GetTag(key string) string { + return c.Tags[key] +} + +func (c *RawContainerInsightsMetric) RemoveTag(key string) { + delete(c.Tags, key) +} + +func (c *RawContainerInsightsMetric) Merge(src *RawContainerInsightsMetric) { + // If there is any conflict, keep the Fields with earlier timestamp + for k, v := range src.Fields { + if _, ok := c.Fields[k]; ok { + c.Logger.Debug(fmt.Sprintf("metric being merged has conflict in fields, src: %v, dest: %v \n", *src, *c)) + c.Logger.Debug("metric being merged has conflict in fields", zap.String("src", src.ContainerName), zap.String("dest", c.ContainerName)) + if c.Tags[ci.Timestamp] < src.Tags[ci.Timestamp] { + continue + } + } + c.Fields[k] = v + } +} diff --git a/receiver/awscontainerinsightreceiver/internal/stores/utils_test.go b/receiver/awscontainerinsightreceiver/internal/stores/utils_test.go index 57d2c2134cec..28c84715588f 100644 --- a/receiver/awscontainerinsightreceiver/internal/stores/utils_test.go +++ b/receiver/awscontainerinsightreceiver/internal/stores/utils_test.go @@ -106,13 +106,13 @@ func TestUtils_parseCronJobFromJob(t *testing.T) { func TestUtils_addKubernetesInfo(t *testing.T) { fields := map[string]any{ci.MetricName(ci.TypePod, ci.CPUTotal): float64(1)} tags := map[string]string{ - ci.ContainerNamekey: "testContainer", - ci.K8sPodNameKey: "testPod", - ci.PodIDKey: "123", - ci.K8sNamespace: "testNamespace", - ci.TypeService: "testService", - ci.NodeNameKey: "testNode", - ci.Timestamp: strconv.FormatInt(time.Now().UnixNano(), 10), + ci.AttributeContainerName: "testContainer", + ci.AttributeK8sPodName: "testPod", + ci.AttributePodID: "123", + ci.AttributeK8sNamespace: "testNamespace", + ci.TypeService: "testService", + ci.NodeNameKey: "testNode", + ci.Timestamp: strconv.FormatInt(time.Now().UnixNano(), 10), } metric := &mockCIMetric{ @@ -122,10 +122,10 @@ func TestUtils_addKubernetesInfo(t *testing.T) { kubernetesBlob := map[string]any{} AddKubernetesInfo(metric, kubernetesBlob, false) - assert.Equal(t, "", metric.GetTag(ci.ContainerNamekey)) - assert.Equal(t, "", metric.GetTag(ci.K8sPodNameKey)) - assert.Equal(t, "", metric.GetTag(ci.PodIDKey)) - assert.Equal(t, "testNamespace", metric.GetTag(ci.K8sNamespace)) + assert.Equal(t, "", metric.GetTag(ci.AttributeContainerName)) + assert.Equal(t, "", metric.GetTag(ci.AttributeK8sPodName)) + assert.Equal(t, "", metric.GetTag(ci.AttributePodID)) + assert.Equal(t, "testNamespace", metric.GetTag(ci.AttributeK8sNamespace)) assert.Equal(t, "testService", metric.GetTag(ci.TypeService)) assert.Equal(t, "testNode", metric.GetTag(ci.NodeNameKey)) @@ -136,13 +136,13 @@ func TestUtils_addKubernetesInfo(t *testing.T) { func TestUtils_addKubernetesInfoRetainContainerNameTag(t *testing.T) { fields := map[string]any{ci.MetricName(ci.TypePod, ci.CPUTotal): float64(1)} tags := map[string]string{ - ci.ContainerNamekey: "testContainer", - ci.K8sPodNameKey: "testPod", - ci.PodIDKey: "123", - ci.K8sNamespace: "testNamespace", - ci.TypeService: "testService", - ci.NodeNameKey: "testNode", - ci.Timestamp: strconv.FormatInt(time.Now().UnixNano(), 10), + ci.AttributeContainerName: "testContainer", + ci.AttributeK8sPodName: "testPod", + ci.AttributePodID: "123", + ci.AttributeK8sNamespace: "testNamespace", + ci.TypeService: "testService", + ci.NodeNameKey: "testNode", + ci.Timestamp: strconv.FormatInt(time.Now().UnixNano(), 10), } metric := &mockCIMetric{ @@ -152,10 +152,10 @@ func TestUtils_addKubernetesInfoRetainContainerNameTag(t *testing.T) { kubernetesBlob := map[string]any{} AddKubernetesInfo(metric, kubernetesBlob, true) - assert.Equal(t, "testContainer", metric.GetTag(ci.ContainerNamekey)) - assert.Equal(t, "", metric.GetTag(ci.K8sPodNameKey)) - assert.Equal(t, "", metric.GetTag(ci.PodIDKey)) - assert.Equal(t, "testNamespace", metric.GetTag(ci.K8sNamespace)) + assert.Equal(t, "testContainer", metric.GetTag(ci.AttributeContainerName)) + assert.Equal(t, "", metric.GetTag(ci.AttributeK8sPodName)) + assert.Equal(t, "", metric.GetTag(ci.AttributePodID)) + assert.Equal(t, "testNamespace", metric.GetTag(ci.AttributeK8sNamespace)) assert.Equal(t, "testService", metric.GetTag(ci.TypeService)) assert.Equal(t, "testNode", metric.GetTag(ci.NodeNameKey)) @@ -189,7 +189,7 @@ func TestUtils_TagMetricSource(t *testing.T) { "[\"cadvisor\",\"pod\",\"calculated\"]", "[\"cadvisor\",\"calculated\"]", "[\"cadvisor\"]", - "[\"pod\",\"calculated\"]", + "[\"dcgm\",\"pod\",\"calculated\"]", } for i, mtype := range types { tags := map[string]string{ diff --git a/receiver/awscontainerinsightreceiver/receiver.go b/receiver/awscontainerinsightreceiver/receiver.go index e666ab2d1305..660ed7c6d9d5 100644 --- a/receiver/awscontainerinsightreceiver/receiver.go +++ b/receiver/awscontainerinsightreceiver/receiver.go @@ -177,7 +177,7 @@ func (acir *awsContainerInsightReceiver) initPrometheusScraper(ctx context.Conte return err } func (acir *awsContainerInsightReceiver) initDcgmScraper(ctx context.Context, host component.Host, hostinfo *hostInfo.Info, decorator *stores.K8sDecorator) error { - if !acir.config.EnableAcceleratedComputingMetric { + if !acir.config.EnableAcceleratedComputeMetrics { return nil } From 6a2de955034e4c7751cc1b53e7eaefe38b9be08d Mon Sep 17 00:00:00 2001 From: Hyunsoo Kim Date: Fri, 1 Mar 2024 14:35:08 -0500 Subject: [PATCH 15/17] merge conflicts --- .../internal/cadvisor/cadvisor_linux.go | 10 +++++----- .../internal/cadvisor/cadvisor_linux_test.go | 2 +- .../internal/cadvisor/cadvisor_nolinux.go | 4 ++-- .../cadvisor/container_info_processor.go | 13 +++++++------ .../cadvisor/extractors/cpu_extractor.go | 2 +- .../cadvisor/extractors/cpu_extractor_test.go | 2 +- .../cadvisor/extractors/diskio_extractor.go | 2 +- .../extractors/diskio_extractor_test.go | 2 +- .../internal/cadvisor/extractors/extractor.go | 3 ++- .../extractors/extractor_helpers_test.go | 3 ++- .../cadvisor/extractors/extractor_test.go | 2 +- .../cadvisor/extractors/fs_extractor.go | 2 +- .../cadvisor/extractors/fs_extractor_test.go | 2 +- .../cadvisor/extractors/mem_extractor.go | 2 +- .../cadvisor/extractors/mem_extractor_test.go | 2 +- .../cadvisor/extractors/net_extractor.go | 2 +- .../cadvisor/extractors/net_extractor_test.go | 2 +- .../internal/gpu/dcgmscraper.go | 2 +- .../internal/gpu/dcgmscraper_test.go | 9 +++++---- .../internal/gpu/decorator.go | 5 +++-- .../internal/gpu/decorator_test.go | 3 ++- .../k8sapiserver/prometheus_scraper_test.go | 2 +- .../awscontainerinsightreceiver/receiver.go | 18 +++++++++--------- 23 files changed, 51 insertions(+), 45 deletions(-) diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_linux.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_linux.go index 0e4643e0a87e..89334dce0fbc 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_linux.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_linux.go @@ -22,12 +22,12 @@ import ( cInfo "github.com/google/cadvisor/info/v1" "github.com/google/cadvisor/manager" "github.com/google/cadvisor/utils/sysfs" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "go.opentelemetry.io/collector/pdata/pmetric" "go.uber.org/zap" ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" ) // The amount of time for which to keep stats in memory. @@ -198,7 +198,7 @@ func (c *Cadvisor) addEbsVolumeInfo(tags map[string]string, ebsVolumeIdsUsedAsPV } } -func (c *Cadvisor) addECSMetrics(cadvisormetrics []*extractors.RawContainerInsightsMetric) { +func (c *Cadvisor) addECSMetrics(cadvisormetrics []*stores.RawContainerInsightsMetric) { if len(cadvisormetrics) == 0 { c.logger.Warn("cadvisor can't collect any metrics!") @@ -257,9 +257,9 @@ func addECSResources(tags map[string]string) { } } -func (c *Cadvisor) decorateMetrics(cadvisormetrics []*extractors.RawContainerInsightsMetric) []*extractors.RawContainerInsightsMetric { +func (c *Cadvisor) decorateMetrics(cadvisormetrics []*stores.RawContainerInsightsMetric) []*stores.RawContainerInsightsMetric { ebsVolumeIdsUsedAsPV := c.hostInfo.ExtractEbsIDsUsedByKubernetes() - var result []*extractors.RawContainerInsightsMetric + var result []*stores.RawContainerInsightsMetric for _, m := range cadvisormetrics { tags := m.GetTags() c.addEbsVolumeInfo(tags, ebsVolumeIdsUsedAsPV) @@ -308,7 +308,7 @@ func (c *Cadvisor) decorateMetrics(cadvisormetrics []*extractors.RawContainerIns out := c.k8sDecorator.Decorate(m) if out != nil { - result = append(result, out.(*extractors.RawContainerInsightsMetric)) + result = append(result, out.(*stores.RawContainerInsightsMetric)) } } diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_linux_test.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_linux_test.go index 9e38af49c936..dfd15470ddf0 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_linux_test.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_linux_test.go @@ -16,11 +16,11 @@ import ( info "github.com/google/cadvisor/info/v1" "github.com/google/cadvisor/manager" "github.com/google/cadvisor/utils/sysfs" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "github.com/stretchr/testify/assert" "go.uber.org/zap" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/cadvisor/testutils" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" ) type mockCadvisorManager struct { diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_nolinux.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_nolinux.go index 3f1e23ee234a..bf13a14ba649 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_nolinux.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_nolinux.go @@ -10,7 +10,7 @@ import ( "go.opentelemetry.io/collector/pdata/pmetric" "go.uber.org/zap" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" ) // cadvisor doesn't support windows, define the dummy functions @@ -26,7 +26,7 @@ type Cadvisor struct { } type Decorator interface { - Decorate(*extractors.RawContainerInsightsMetric) *extractors.RawContainerInsightsMetric + Decorate(*stores.RawContainerInsightsMetric) *stores.RawContainerInsightsMetric Shutdown() error } diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/container_info_processor.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/container_info_processor.go index 3c6842f60831..b5ee37ce0f35 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/container_info_processor.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/container_info_processor.go @@ -17,6 +17,7 @@ import ( ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" ) const ( @@ -38,8 +39,8 @@ type podKey struct { namespace string } -func processContainers(cInfos []*cInfo.ContainerInfo, mInfo extractors.CPUMemInfoProvider, containerOrchestrator string, logger *zap.Logger) []*extractors.RawContainerInsightsMetric { - var metrics []*extractors.RawContainerInsightsMetric +func processContainers(cInfos []*cInfo.ContainerInfo, mInfo extractors.CPUMemInfoProvider, containerOrchestrator string, logger *zap.Logger) []*stores.RawContainerInsightsMetric { + var metrics []*stores.RawContainerInsightsMetric podKeys := make(map[string]podKey) // first iteration of container infos processes individual container info and @@ -88,8 +89,8 @@ func processContainers(cInfos []*cInfo.ContainerInfo, mInfo extractors.CPUMemInf } // processContainers get metrics for individual container and gather information for pod so we can look it up later. -func processContainer(info *cInfo.ContainerInfo, mInfo extractors.CPUMemInfoProvider, containerOrchestrator string, logger *zap.Logger) ([]*extractors.RawContainerInsightsMetric, *podKey, error) { - var result []*extractors.RawContainerInsightsMetric +func processContainer(info *cInfo.ContainerInfo, mInfo extractors.CPUMemInfoProvider, containerOrchestrator string, logger *zap.Logger) ([]*stores.RawContainerInsightsMetric, *podKey, error) { + var result []*stores.RawContainerInsightsMetric var pKey *podKey if isContainerInContainer(info.Name) { @@ -165,8 +166,8 @@ func processContainer(info *cInfo.ContainerInfo, mInfo extractors.CPUMemInfoProv return result, pKey, nil } -func processPod(info *cInfo.ContainerInfo, mInfo extractors.CPUMemInfoProvider, podKeys map[string]podKey, logger *zap.Logger) []*extractors.RawContainerInsightsMetric { - var result []*extractors.RawContainerInsightsMetric +func processPod(info *cInfo.ContainerInfo, mInfo extractors.CPUMemInfoProvider, podKeys map[string]podKey, logger *zap.Logger) []*stores.RawContainerInsightsMetric { + var result []*stores.RawContainerInsightsMetric if isContainerInContainer(info.Name) { logger.Debug("drop metric because it's nested container", zap.String("name", info.Name)) return result diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/cpu_extractor.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/cpu_extractor.go index de91e0d26bdc..d94161da5844 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/cpu_extractor.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/cpu_extractor.go @@ -5,11 +5,11 @@ package extractors // import "github.com/open-telemetry/opentelemetry-collector- import ( cInfo "github.com/google/cadvisor/info/v1" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "go.uber.org/zap" ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" awsmetrics "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/metrics" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" ) const ( diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/cpu_extractor_test.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/cpu_extractor_test.go index b41b0142cf42..dfd6fdc312d7 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/cpu_extractor_test.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/cpu_extractor_test.go @@ -6,11 +6,11 @@ package extractors import ( "testing" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "github.com/stretchr/testify/require" "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/cadvisor/testutils" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" ) func TestCPUStats(t *testing.T) { diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/diskio_extractor.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/diskio_extractor.go index dc3879ea37c3..b40f48e784dd 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/diskio_extractor.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/diskio_extractor.go @@ -9,11 +9,11 @@ import ( "time" cInfo "github.com/google/cadvisor/info/v1" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "go.uber.org/zap" ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" awsmetrics "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/metrics" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" ) type DiskIOMetricExtractor struct { diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/diskio_extractor_test.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/diskio_extractor_test.go index 287c66a4426e..4d32d2d210e9 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/diskio_extractor_test.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/diskio_extractor_test.go @@ -6,12 +6,12 @@ package extractors import ( "testing" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/cadvisor/testutils" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" ) func TestDiskIOStats(t *testing.T) { diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor.go index 5007cdca3a74..5368a72702b3 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor.go @@ -8,6 +8,7 @@ import ( "time" cinfo "github.com/google/cadvisor/info/v1" + ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" awsmetrics "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/metrics" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" @@ -32,7 +33,7 @@ type MetricExtractor interface { Shutdown() error } -func NewFloat64RateCalculator() awsmetrics.MetricCalculator { +func newFloat64RateCalculator() awsmetrics.MetricCalculator { return awsmetrics.NewMetricCalculator(func(prev *awsmetrics.MetricValue, val any, timestamp time.Time) (any, bool) { if prev != nil { deltaNs := timestamp.Sub(prev.Timestamp) diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor_helpers_test.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor_helpers_test.go index f5bafe09101f..6bc91a48dfe5 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor_helpers_test.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor_helpers_test.go @@ -8,8 +8,9 @@ import ( "reflect" "testing" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "github.com/stretchr/testify/assert" + + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" ) func AssertContainsTaggedFloat( diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor_test.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor_test.go index 90a5963d4067..be6533145309 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor_test.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor_test.go @@ -6,12 +6,12 @@ package extractors import ( "testing" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "go.uber.org/zap" ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" ) func TestCAdvisorMetric_Merge(t *testing.T) { diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/fs_extractor.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/fs_extractor.go index 26ad4f976295..33e1d4f7421d 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/fs_extractor.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/fs_extractor.go @@ -7,10 +7,10 @@ import ( "regexp" cinfo "github.com/google/cadvisor/info/v1" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "go.uber.org/zap" ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" ) var allowedPaths = regexp.MustCompile(`^(tmpfs|\/dev\/.*|overlay)$`) diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/fs_extractor_test.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/fs_extractor_test.go index e4ed994d1d3f..55f04de5d0f2 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/fs_extractor_test.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/fs_extractor_test.go @@ -9,11 +9,11 @@ import ( "testing" cinfo "github.com/google/cadvisor/info/v1" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "github.com/stretchr/testify/assert" "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/cadvisor/testutils" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" ) func TestFSStats(t *testing.T) { diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/mem_extractor.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/mem_extractor.go index 083e6e9974fb..7254edd9ba0e 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/mem_extractor.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/mem_extractor.go @@ -7,11 +7,11 @@ import ( "time" cinfo "github.com/google/cadvisor/info/v1" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "go.uber.org/zap" ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" awsmetrics "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/metrics" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" ) type MemMetricExtractor struct { diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/mem_extractor_test.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/mem_extractor_test.go index 708059ed9d1a..8555641fb12e 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/mem_extractor_test.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/mem_extractor_test.go @@ -6,11 +6,11 @@ package extractors import ( "testing" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "github.com/stretchr/testify/require" "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/cadvisor/testutils" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" ) func TestMemStats(t *testing.T) { diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/net_extractor.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/net_extractor.go index 69b6ec2eb52e..7084ad74f568 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/net_extractor.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/net_extractor.go @@ -7,11 +7,11 @@ import ( "time" cinfo "github.com/google/cadvisor/info/v1" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "go.uber.org/zap" ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" awsmetrics "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/metrics" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" ) type NetMetricExtractor struct { diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/net_extractor_test.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/net_extractor_test.go index 9280d1d3d0f5..d6ef130e7070 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/net_extractor_test.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/net_extractor_test.go @@ -6,12 +6,12 @@ package extractors import ( "testing" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/cadvisor/testutils" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" ) func TestNetStats(t *testing.T) { diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go index e7435a34a5b2..fc0d332357a4 100644 --- a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go +++ b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go @@ -9,7 +9,6 @@ import ( "fmt" "time" - ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/discovery" @@ -21,6 +20,7 @@ import ( "go.opentelemetry.io/collector/receiver" "go.uber.org/zap" + ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/prometheusreceiver" ) diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go index b9e875e54271..38bf588910e8 100644 --- a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go +++ b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go @@ -8,10 +8,6 @@ import ( "strings" "testing" - ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/mocks" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/prometheusreceiver" configutil "github.com/prometheus/common/config" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/config" @@ -22,6 +18,11 @@ import ( "go.opentelemetry.io/collector/pdata/pmetric" "go.opentelemetry.io/collector/receiver" "go.uber.org/zap" + + ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/mocks" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/prometheusreceiver" ) const renameMetric = ` diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/decorator.go b/receiver/awscontainerinsightreceiver/internal/gpu/decorator.go index a3f68c46fa99..58a991e481de 100644 --- a/receiver/awscontainerinsightreceiver/internal/gpu/decorator.go +++ b/receiver/awscontainerinsightreceiver/internal/gpu/decorator.go @@ -6,12 +6,13 @@ package gpu import ( "context" - ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "go.opentelemetry.io/collector/consumer" "go.opentelemetry.io/collector/pdata/pcommon" "go.opentelemetry.io/collector/pdata/pmetric" "go.uber.org/zap" + + ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" ) const ( diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/decorator_test.go b/receiver/awscontainerinsightreceiver/internal/gpu/decorator_test.go index 4f5cadb8a31a..2266e218d424 100644 --- a/receiver/awscontainerinsightreceiver/internal/gpu/decorator_test.go +++ b/receiver/awscontainerinsightreceiver/internal/gpu/decorator_test.go @@ -7,7 +7,6 @@ import ( "context" "testing" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "go.opentelemetry.io/collector/consumer" @@ -15,6 +14,8 @@ import ( "go.opentelemetry.io/collector/pdata/pcommon" "go.opentelemetry.io/collector/pdata/pmetric" "go.uber.org/zap" + + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" ) var _ Decorator = (*MockK8sDecorator)(nil) diff --git a/receiver/awscontainerinsightreceiver/internal/k8sapiserver/prometheus_scraper_test.go b/receiver/awscontainerinsightreceiver/internal/k8sapiserver/prometheus_scraper_test.go index 3a0a01b76db2..0c660112e40e 100644 --- a/receiver/awscontainerinsightreceiver/internal/k8sapiserver/prometheus_scraper_test.go +++ b/receiver/awscontainerinsightreceiver/internal/k8sapiserver/prometheus_scraper_test.go @@ -9,7 +9,6 @@ import ( "strings" "testing" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/mocks" configutil "github.com/prometheus/common/config" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/config" @@ -22,6 +21,7 @@ import ( "go.opentelemetry.io/collector/receiver" "go.uber.org/zap" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/mocks" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/prometheusreceiver" ) diff --git a/receiver/awscontainerinsightreceiver/receiver.go b/receiver/awscontainerinsightreceiver/receiver.go index 660ed7c6d9d5..86867e50dcb6 100644 --- a/receiver/awscontainerinsightreceiver/receiver.go +++ b/receiver/awscontainerinsightreceiver/receiver.go @@ -8,7 +8,6 @@ import ( "errors" "time" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/gpu" "go.opentelemetry.io/collector/component" "go.opentelemetry.io/collector/consumer" "go.opentelemetry.io/collector/pdata/pmetric" @@ -16,6 +15,8 @@ import ( "go.uber.org/zap" "k8s.io/client-go/rest" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/gpu" + ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/k8s/k8sclient" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/cadvisor" @@ -92,15 +93,14 @@ func (acir *awsContainerInsightReceiver) Start(ctx context.Context, host compone if err != nil { return err } - err = acir.initPrometheusScraper(ctx, host, hostinfo, leaderElection) - if err != nil { - acir.settings.Logger.Debug("Unable to start kube apiserver prometheus scraper", zap.Error(err)) - } + err = acir.initPrometheusScraper(ctx, host, hostinfo, leaderElection) + if err != nil { + acir.settings.Logger.Debug("Unable to start kube apiserver prometheus scraper", zap.Error(err)) + } - err = acir.initDcgmScraper(ctx, host, hostinfo, k8sDecorator) - if err != nil { - acir.settings.Logger.Debug("Unable to start dcgm scraper", zap.Error(err)) - } + err = acir.initDcgmScraper(ctx, host, hostinfo, k8sDecorator) + if err != nil { + acir.settings.Logger.Debug("Unable to start dcgm scraper", zap.Error(err)) } } if acir.config.ContainerOrchestrator == ci.ECS { From cc3bf6e0245dac5682b64942c6ffcfd641ab3824 Mon Sep 17 00:00:00 2001 From: Hyunsoo Kim Date: Fri, 1 Mar 2024 14:52:44 -0500 Subject: [PATCH 16/17] remove unused variable and code for setting metric type to it --- .../awscontainerinsightreceiver/internal/gpu/decorator.go | 5 ----- 1 file changed, 5 deletions(-) diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/decorator.go b/receiver/awscontainerinsightreceiver/internal/gpu/decorator.go index 58a991e481de..ee3f9a8c36c1 100644 --- a/receiver/awscontainerinsightreceiver/internal/gpu/decorator.go +++ b/receiver/awscontainerinsightreceiver/internal/gpu/decorator.go @@ -65,11 +65,6 @@ func (dc *decorateConsumer) ConsumeMetrics(ctx context.Context, md pmetric.Metri converted := ci.ConvertToFieldsAndTags(m, dc.logger) var rcis []*stores.RawContainerInsightsMetric for _, pair := range converted { - rci := stores.NewRawContainerInsightsMetricWithData(ci.TypeGpuContainer, pair.Fields, pair.Tags, dc.logger) - if !rci.HasTag(ci.MetricType) { - // force type to be Container to decorate with container level labels - rci.AddTag(ci.MetricType, ci.TypeGpuContainer) - } rcis = append(rcis, stores.NewRawContainerInsightsMetricWithData(ci.TypeGpuContainer, pair.Fields, pair.Tags, dc.logger)) } From 852fff7ff3fa8725b6a2061f4a3e3ba36334a853 Mon Sep 17 00:00:00 2001 From: Hyunsoo Kim Date: Fri, 1 Mar 2024 15:14:45 -0500 Subject: [PATCH 17/17] clean ups --- .../internal/gpu/dcgmscraper_test.go | 10 +++++----- .../internal/gpu/decorator_test.go | 14 -------------- receiver/awscontainerinsightreceiver/receiver.go | 3 +-- 3 files changed, 6 insertions(+), 21 deletions(-) diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go index 38bf588910e8..7d59f1be53a6 100644 --- a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go +++ b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go @@ -35,7 +35,7 @@ DCGM_FI_DEV_GPU_UTIL{gpu="0",UUID="uuid",device="nvidia0",modelName="NVIDIA A10G ` const ( - dummyInstanceId = "i-0000000000" + dummyInstanceID = "i-0000000000" dummyClusterName = "cluster-name" ) @@ -47,7 +47,7 @@ func (m mockHostInfoProvider) GetClusterName() string { } func (m mockHostInfoProvider) GetInstanceID() string { - return dummyInstanceId + return dummyInstanceID } type mockDecorator struct { @@ -96,7 +96,7 @@ func (m mockConsumer) ConsumeMetrics(_ context.Context, md pmetric.Metrics) erro assert.True(m.t, found) assert.Equal(m.t, v, lv.AsString()) } - scrapedMetricCnt += 1 + scrapedMetricCnt++ } assert.Equal(m.t, len(m.expected), scrapedMetricCnt) *m.called = true @@ -150,7 +150,7 @@ func TestNewDcgmScraperEndToEnd(t *testing.T) { ci.NodeNameKey: "hostname", ci.AttributeK8sNamespace: "kube-system", ci.ClusterNameKey: dummyClusterName, - ci.InstanceID: dummyInstanceId, + ci.InstanceID: dummyInstanceID, ci.AttributeFullPodName: "fullname-hash", ci.AttributeK8sPodName: "fullname-hash", ci.AttributeContainerName: "main", @@ -163,7 +163,7 @@ func TestNewDcgmScraperEndToEnd(t *testing.T) { ci.NodeNameKey: "hostname", ci.AttributeK8sNamespace: "kube-system", ci.ClusterNameKey: dummyClusterName, - ci.InstanceID: dummyInstanceId, + ci.InstanceID: dummyInstanceID, ci.AttributeFullPodName: "fullname-hash", ci.AttributeK8sPodName: "fullname-hash", ci.AttributeContainerName: "main", diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/decorator_test.go b/receiver/awscontainerinsightreceiver/internal/gpu/decorator_test.go index 2266e218d424..a81e39268b65 100644 --- a/receiver/awscontainerinsightreceiver/internal/gpu/decorator_test.go +++ b/receiver/awscontainerinsightreceiver/internal/gpu/decorator_test.go @@ -9,7 +9,6 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - "go.opentelemetry.io/collector/consumer" "go.opentelemetry.io/collector/consumer/consumertest" "go.opentelemetry.io/collector/pdata/pcommon" "go.opentelemetry.io/collector/pdata/pmetric" @@ -31,19 +30,6 @@ func (m *MockK8sDecorator) Shutdown() error { return nil } -type mockNextConsumer struct { -} - -func (mc *mockNextConsumer) Capabilities() consumer.Capabilities { - return consumer.Capabilities{ - MutatesData: true, - } -} - -func (mc *mockNextConsumer) ConsumeMetrics(_ context.Context, md pmetric.Metrics) error { - return nil -} - func TestConsumeMetrics(t *testing.T) { logger, _ := zap.NewDevelopment() dc := &decorateConsumer{ diff --git a/receiver/awscontainerinsightreceiver/receiver.go b/receiver/awscontainerinsightreceiver/receiver.go index 86867e50dcb6..1628b10ead67 100644 --- a/receiver/awscontainerinsightreceiver/receiver.go +++ b/receiver/awscontainerinsightreceiver/receiver.go @@ -15,12 +15,11 @@ import ( "go.uber.org/zap" "k8s.io/client-go/rest" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/gpu" - ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/k8s/k8sclient" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/cadvisor" ecsinfo "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/ecsInfo" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/gpu" hostInfo "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/host" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/k8sapiserver" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores"