From 892074212cb55549d6dd7d374a1574107be1a19f Mon Sep 17 00:00:00 2001 From: Hyunsoo Kim Date: Fri, 2 Feb 2024 13:43:25 -0500 Subject: [PATCH 01/53] 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 e0f5bcd86bddfe28f26274c49cbc7086db52a236 Mon Sep 17 00:00:00 2001 From: Hyunsoo Kim Date: Tue, 13 Feb 2024 09:51:28 -0500 Subject: [PATCH 02/53] 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 51fe859f3616eb31713e64f366c26f5adb9bae7e Mon Sep 17 00:00:00 2001 From: Hyunsoo Kim Date: Wed, 14 Feb 2024 11:04:58 -0500 Subject: [PATCH 03/53] 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 cee224455ea5a2307e1a0372a80dc5e9eb0ce638 Mon Sep 17 00:00:00 2001 From: Hyunsoo Kim Date: Wed, 14 Feb 2024 11:07:52 -0500 Subject: [PATCH 04/53] 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 3debd28ac4186f53556271be89f355475abf4733 Mon Sep 17 00:00:00 2001 From: Hyunsoo Kim Date: Wed, 14 Feb 2024 11:40:58 -0500 Subject: [PATCH 05/53] 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 3d9de491a105721ad1d55980193d2dff26c1ad28 Mon Sep 17 00:00:00 2001 From: Samarth Singh Date: Thu, 15 Feb 2024 14:49:41 +0000 Subject: [PATCH 06/53] add neuron monitor scraper --- .../awscontainerinsightreceiver/config.go | 4 + .../internal/neuron/neuron_monitor_scraper.go | 232 ++++++++++++ .../neuron/neuron_monitor_scraper_test.go | 340 ++++++++++++++++++ .../internal/pod/podinfo.go | 26 ++ .../awscontainerinsightreceiver/receiver.go | 59 ++- 5 files changed, 653 insertions(+), 8 deletions(-) create mode 100644 receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper.go create mode 100644 receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go create mode 100644 receiver/awscontainerinsightreceiver/internal/pod/podinfo.go diff --git a/receiver/awscontainerinsightreceiver/config.go b/receiver/awscontainerinsightreceiver/config.go index 2e06ba111d33..20c9c80b429a 100644 --- a/receiver/awscontainerinsightreceiver/config.go +++ b/receiver/awscontainerinsightreceiver/config.go @@ -61,4 +61,8 @@ type Config struct { // 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"` + + // EnableNeuronMetric disables Neuron monitoring where metrics are scraped from vendor specific sources + // The default value is true meaning Neuron metrics get collected out of the box unless it's disabled + EnableNeuronMetric bool `mapstructure:"neuron_metrics"` } diff --git a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper.go b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper.go new file mode 100644 index 000000000000..9a786f22e549 --- /dev/null +++ b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper.go @@ -0,0 +1,232 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +package neuron + +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 = "containerInsightsNeuronMonitorScraper" +) + +type NeuronMonitorScraper struct { + ctx context.Context + settings component.TelemetrySettings + host component.Host + hostInfoProvider hostInfoProvider + podNameInfoProvider podNameInfoProvider + prometheusReceiver receiver.Metrics + running bool +} + +type NeuronMonitorScraperOpts struct { + Ctx context.Context + TelemetrySettings component.TelemetrySettings + Consumer consumer.Metrics + Host component.Host + HostInfoProvider hostInfoProvider + PodNameInfoProvider podNameInfoProvider + BearerToken string +} + +type hostInfoProvider interface { + GetClusterName() string + GetInstanceID() string +} + +// lets assume new provider will provide PodName +type podNameInfoProvider interface { + GetPodName() string + GetNamespace() string +} + +func NewNeuronMonitorScraper(opts NeuronMonitorScraperOpts) (*NeuronMonitorScraper, 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") + } + if opts.PodNameInfoProvider == nil { + return nil, errors.New("pod 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", // TODO: Find the correct name + }, + }, + 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("neuron.*"), + Action: relabel.Keep, + }, + // 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{"__name__"}, + TargetLabel: "Namespace", + Regex: relabel.MustNewRegexp("neuron.*"), + Replacement: opts.PodNameInfoProvider.GetNamespace(), + Action: relabel.Replace, + }, + { + SourceLabels: model.LabelNames{"__name__"}, + TargetLabel: "ClusterName", + Regex: relabel.MustNewRegexp("neuron.*"), + Replacement: opts.HostInfoProvider.GetClusterName(), + Action: relabel.Replace, + }, + { + SourceLabels: model.LabelNames{"__name__"}, + TargetLabel: "InstanceId", + Regex: relabel.MustNewRegexp("neuron.*"), + Replacement: opts.HostInfoProvider.GetInstanceID(), + Action: relabel.Replace, + }, + { + SourceLabels: model.LabelNames{"namespace"}, + TargetLabel: "PodName", + Regex: relabel.MustNewRegexp("neuron.*"), + Replacement: opts.PodNameInfoProvider.GetPodName(), + Action: relabel.Replace, + }, + { + SourceLabels: model.LabelNames{"neuroncore"}, + TargetLabel: "DeviceId", + 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 &NeuronMonitorScraper{ + ctx: opts.Ctx, + settings: opts.TelemetrySettings, + host: opts.Host, + hostInfoProvider: opts.HostInfoProvider, + podNameInfoProvider: opts.PodNameInfoProvider, + prometheusReceiver: promReceiver, + }, nil +} + +func (ds *NeuronMonitorScraper) 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 *NeuronMonitorScraper) 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/neuron/neuron_monitor_scraper_test.go b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go new file mode 100644 index 000000000000..5213d09e48b3 --- /dev/null +++ b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go @@ -0,0 +1,340 @@ +package neuron + +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 neuron_runtime_memory_used_bytes Runtime memory used bytes +# TYPE neuron_runtime_memory_used_bytes gauge +neuron_runtime_memory_used_bytes{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",memory_location="host",region="us-east-1",runtime_tag="367",subnet_id="subnet-06a7754948e8a000f"} 9.043968e+06 +# HELP neuroncore_utilization_ratio NeuronCore utilization ratio +# TYPE neuroncore_utilization_ratio gauge +neuroncore_utilization_ratio{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",neuroncore="0",region="us-east-1",runtime_tag="367",subnet_id="subnet-06a7754948e8a000f"} 0.1 +` + +const dummyInstanceId = "i-0000000000" +const dummyPodName = "podname" +const dummyNamespace = "kube-system" + +type mockHostInfoProvider struct { +} + +type mockPodInfoProvider struct { +} + +func (m mockHostInfoProvider) GetClusterName() string { + return "cluster-name" +} + +func (m mockHostInfoProvider) GetInstanceID() string { + return dummyInstanceId +} + +func (m mockPodInfoProvider) GetPodName() string { + return dummyPodName +} + +func (m mockPodInfoProvider) GetNamespace() string { + return dummyNamespace +} + +type mockConsumer struct { + t *testing.T + up *bool + coreUtil *bool + memUsed *bool + instanceId *bool + podName *bool + namespace *bool + relabeled *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) + fmt.Println("!!!Metric Name is ", metric.Name()) + if metric.Name() == "neuroncore_utilization_ratio" { + assert.Equal(m.t, float64(0.1), metric.Gauge().DataPoints().At(0).DoubleValue()) + instanceId, _ := metric.Gauge().DataPoints().At(0).Attributes().Get("InstanceId") + podName, _ := metric.Gauge().DataPoints().At(0).Attributes().Get("PodName") + namespace, _ := metric.Gauge().DataPoints().At(0).Attributes().Get("Namespace") + _, relabeled := metric.Gauge().DataPoints().At(0).Attributes().Get("DeviceId") + fmt.Println("!!!InstanceId is ", instanceId.Str()) + *m.instanceId = instanceId.Str() == dummyInstanceId + *m.podName = podName.Str() == dummyPodName + *m.namespace = namespace.Str() == dummyNamespace + *m.coreUtil = true + *m.relabeled = relabeled + } + + if metric.Name() == "neuron_runtime_memory_used_bytes" { + assert.Equal(m.t, float64(9.043968e+06), metric.Gauge().DataPoints().At(0).DoubleValue()) + *m.memUsed = true + } + + if metric.Name() == "up" { + assert.Equal(m.t, float64(1.0), metric.Gauge().DataPoints().At(0).DoubleValue()) + *m.up = true + } + } + + return nil +} + +func TestNewNeuronScraperBadInputs(t *testing.T) { + settings := componenttest.NewNopTelemetrySettings() + settings.Logger, _ = zap.NewDevelopment() + + tests := []NeuronMonitorScraperOpts{ + { + Ctx: context.TODO(), + TelemetrySettings: settings, + Consumer: nil, + Host: componenttest.NewNopHost(), + HostInfoProvider: mockHostInfoProvider{}, + PodNameInfoProvider: mockPodInfoProvider{}, + }, + { + Ctx: context.TODO(), + TelemetrySettings: settings, + Consumer: mockConsumer{}, + Host: nil, + HostInfoProvider: mockHostInfoProvider{}, + PodNameInfoProvider: mockPodInfoProvider{}, + }, + { + Ctx: context.TODO(), + TelemetrySettings: settings, + Consumer: mockConsumer{}, + Host: componenttest.NewNopHost(), + HostInfoProvider: nil, + PodNameInfoProvider: mockPodInfoProvider{}, + }, + } + + for _, tt := range tests { + scraper, err := NewNeuronMonitorScraper(tt) + + assert.Error(t, err) + assert.Nil(t, scraper) + } +} + +func TestNewNeuronScraperEndToEnd(t *testing.T) { + + upPtr := false + coreUtil := false + instanceId := false + memUsed := false + podName := false + namespace := false + relabeled := false + + consumer := mockConsumer{ + t: t, + up: &upPtr, + coreUtil: &coreUtil, + memUsed: &memUsed, + instanceId: &instanceId, + podName: &podName, + namespace: &namespace, + relabeled: &relabeled, + } + + settings := componenttest.NewNopTelemetrySettings() + settings.Logger, _ = zap.NewDevelopment() + + scraper, err := NewNeuronMonitorScraper(NeuronMonitorScraperOpts{ + Ctx: context.TODO(), + TelemetrySettings: settings, + Consumer: mockConsumer{}, + Host: componenttest.NewNopHost(), + HostInfoProvider: mockHostInfoProvider{}, + PodNameInfoProvider: mockPodInfoProvider{}, + BearerToken: "", + }) + assert.NoError(t, err) + assert.Equal(t, mockHostInfoProvider{}, scraper.hostInfoProvider) + + // build up a new PR + promFactory := prometheusreceiver.NewFactory() + + targets := []*mocks.TestData{ + { + Name: "neuron", + 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("neuron.*"), + Action: relabel.Keep, + }, + { + SourceLabels: model.LabelNames{"neuroncore"}, + TargetLabel: "DeviceId", + Regex: relabel.MustNewRegexp("(.*)"), + Replacement: "${1}", + Action: relabel.Replace, + }, + // test hack to inject cluster name as label + { + SourceLabels: model.LabelNames{"neuroncore"}, + TargetLabel: "Namespace", + Regex: relabel.MustNewRegexp("(.*)"), + Replacement: scraper.podNameInfoProvider.GetNamespace(), + Action: relabel.Replace, + }, + { + SourceLabels: model.LabelNames{"neuroncore"}, + TargetLabel: "ClusterName", + Regex: relabel.MustNewRegexp("(.*)"), + Replacement: scraper.hostInfoProvider.GetClusterName(), + Action: relabel.Replace, + }, + { + SourceLabels: model.LabelNames{"neuroncore"}, + TargetLabel: "InstanceId", + Regex: relabel.MustNewRegexp("(.*)"), + Replacement: scraper.hostInfoProvider.GetInstanceID(), + Action: relabel.Replace, + }, + { + SourceLabels: model.LabelNames{"neuroncore"}, + TargetLabel: "PodName", + Regex: relabel.MustNewRegexp("(.*)"), + Replacement: scraper.podNameInfoProvider.GetPodName(), + 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.coreUtil) + assert.True(t, *consumer.memUsed) + assert.True(t, *consumer.instanceId) + assert.True(t, *consumer.podName) + assert.True(t, *consumer.namespace) + assert.True(t, *consumer.relabeled) +} + +func TestDcgmScraperJobName(t *testing.T) { + // needs to start with containerInsights + assert.True(t, strings.HasPrefix(jobName, "containerInsightsNeuronMonitorScraper")) +} diff --git a/receiver/awscontainerinsightreceiver/internal/pod/podinfo.go b/receiver/awscontainerinsightreceiver/internal/pod/podinfo.go new file mode 100644 index 000000000000..53193c2067f5 --- /dev/null +++ b/receiver/awscontainerinsightreceiver/internal/pod/podinfo.go @@ -0,0 +1,26 @@ +package pod + +type Info struct { + namespace string + podName string +} + +// GetNamespace returns the namespace for the pod +func (m *Info) GetNamespace() string { + return m.namespace +} + +// GetPodName returns the podName for the pod +func (m *Info) GetPodName() string { + return m.podName +} + +// NewInfo creates a new Info struct +func NewInfo() (*Info, error) { + mInfo := &Info{ + podName: "DummyPodName", + namespace: "DummyNamespace", + } + + return mInfo, nil +} diff --git a/receiver/awscontainerinsightreceiver/receiver.go b/receiver/awscontainerinsightreceiver/receiver.go index 3023ac32d131..5993d6901e76 100644 --- a/receiver/awscontainerinsightreceiver/receiver.go +++ b/receiver/awscontainerinsightreceiver/receiver.go @@ -9,6 +9,8 @@ import ( "time" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/gpu" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/neuron" + podinfo "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/pod" "go.opentelemetry.io/collector/component" "go.opentelemetry.io/collector/consumer" "go.opentelemetry.io/collector/pdata/pmetric" @@ -34,14 +36,15 @@ type metricsProvider interface { // awsContainerInsightReceiver implements the receiver.Metrics type awsContainerInsightReceiver struct { - settings component.TelemetrySettings - nextConsumer consumer.Metrics - config *Config - cancel context.CancelFunc - cadvisor metricsProvider - k8sapiserver metricsProvider - prometheusScraper *k8sapiserver.PrometheusScraper - dcgmScraper *gpu.DcgmScraper + settings component.TelemetrySettings + nextConsumer consumer.Metrics + config *Config + cancel context.CancelFunc + cadvisor metricsProvider + k8sapiserver metricsProvider + prometheusScraper *k8sapiserver.PrometheusScraper + dcgmScraper *gpu.DcgmScraper + neuronMonitorScraper *neuron.NeuronMonitorScraper } // newAWSContainerInsightReceiver creates the aws container insight receiver with the given parameters. @@ -70,6 +73,11 @@ func (acir *awsContainerInsightReceiver) Start(ctx context.Context, host compone return err } + podinfo, err := podinfo.NewInfo() + if err != nil { + return err + } + if acir.config.ContainerOrchestrator == ci.EKS { k8sDecorator, err := stores.NewK8sDecorator(ctx, acir.config.TagService, acir.config.PrefFullPodName, acir.config.AddFullPodNameMetricLabel, acir.config.AddContainerNameMetricLabel, acir.config.EnableControlPlaneMetrics, acir.settings.Logger) if err != nil { @@ -102,6 +110,11 @@ func (acir *awsContainerInsightReceiver) Start(ctx context.Context, host compone if err != nil { acir.settings.Logger.Debug("Unable to start dcgm scraper", zap.Error(err)) } + + err = acir.startNeuronScraper(ctx, host, hostinfo, podinfo) + if err != nil { + acir.settings.Logger.Debug("Unable to start dcgm scraper", zap.Error(err)) + } } if acir.config.ContainerOrchestrator == ci.ECS { @@ -201,6 +214,32 @@ func (acir *awsContainerInsightReceiver) startDcgmScraper(ctx context.Context, h return err } +func (acir *awsContainerInsightReceiver) startNeuronScraper(ctx context.Context, host component.Host, hostinfo *hostInfo.Info, podinfo *podinfo.Info) error { + if !acir.config.EnableNeuronMetric { + return nil + } + + restConfig, err := rest.InClusterConfig() + if err != nil { + return err + } + bearerToken := restConfig.BearerToken + if bearerToken == "" { + return errors.New("bearer token was empty") + } + + acir.neuronMonitorScraper, err = neuron.NewNeuronMonitorScraper(neuron.NeuronMonitorScraperOpts{ + Ctx: ctx, + TelemetrySettings: acir.settings, + Consumer: acir.nextConsumer, + Host: host, + HostInfoProvider: hostinfo, + PodNameInfoProvider: podinfo, + BearerToken: bearerToken, + }) + return err +} + // Shutdown stops the awsContainerInsightReceiver receiver. func (acir *awsContainerInsightReceiver) Shutdown(context.Context) error { if acir.prometheusScraper != nil { @@ -255,6 +294,10 @@ func (acir *awsContainerInsightReceiver) collectData(ctx context.Context) error acir.dcgmScraper.GetMetrics() //nolint:errcheck } + if acir.neuronMonitorScraper != nil { + acir.neuronMonitorScraper.GetMetrics() //nolint:errcheck + } + for _, md := range mds { err := acir.nextConsumer.ConsumeMetrics(ctx, md) if err != nil { From 9e700699f673695d0f21f214cf371f6bc65f09cb Mon Sep 17 00:00:00 2001 From: Hyunsoo Kim Date: Thu, 15 Feb 2024 21:00:08 -0500 Subject: [PATCH 07/53] 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 | 19 +++++-------------- 7 files changed, 11 insertions(+), 39 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..0d659fa1c7b6 100644 --- a/receiver/awscontainerinsightreceiver/receiver.go +++ b/receiver/awscontainerinsightreceiver/receiver.go @@ -93,12 +93,12 @@ func (acir *awsContainerInsightReceiver) Start(ctx context.Context, host compone return err } - err = acir.startPrometheusScraper(ctx, host, hostinfo, leaderElection) + 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.startDcgmScraper(ctx, host, hostinfo) + err = acir.initDcgmScraper(ctx, host, hostinfo) if err != nil { acir.settings.Logger.Debug("Unable to start dcgm scraper", zap.Error(err)) } @@ -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 b9a0e035706d334d21cbd8e4c74e1acd5f0698f2 Mon Sep 17 00:00:00 2001 From: Hyunsoo Kim Date: Fri, 16 Feb 2024 13:32:55 -0500 Subject: [PATCH 08/53] 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 52cd9726197880d879d91c042dc9abafee5c8be7 Mon Sep 17 00:00:00 2001 From: Hyunsoo Kim Date: Fri, 16 Feb 2024 14:03:09 -0500 Subject: [PATCH 09/53] 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 1747a50947e73e43787d24d3653ec8386ec1ee8d Mon Sep 17 00:00:00 2001 From: Samarth Singh Date: Tue, 20 Feb 2024 12:09:43 +0000 Subject: [PATCH 10/53] Update Scraper for new metrics --- exporter/awsemfexporter/metric_translator.go | 3 +- .../awsemfexporter/metric_translator_test.go | 16 ++ .../internal/neuron/neuron_monitor_scraper.go | 126 ++++----- .../neuron/neuron_monitor_scraper_test.go | 257 +++++++++++------- .../internal/pod/podinfo.go | 26 -- .../awscontainerinsightreceiver/receiver.go | 32 +-- 6 files changed, 232 insertions(+), 228 deletions(-) delete mode 100644 receiver/awscontainerinsightreceiver/internal/pod/podinfo.go diff --git a/exporter/awsemfexporter/metric_translator.go b/exporter/awsemfexporter/metric_translator.go index b3c1ecf42c54..f432a07dcaa3 100644 --- a/exporter/awsemfexporter/metric_translator.go +++ b/exporter/awsemfexporter/metric_translator.go @@ -132,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") || - strings.HasPrefix(serviceName.Str(), "containerInsightsDCGMExporterScraper") { + strings.HasPrefix(serviceName.Str(), "containerInsightsDCGMExporterScraper") || + strings.HasPrefix(serviceName.Str(), "containerInsightsNeuronMonitorScraper") { // the prometheus metrics that come from the container insight receiver need to be clearly tagged as coming from container insights metricReceiver = containerInsightsReceiver } diff --git a/exporter/awsemfexporter/metric_translator_test.go b/exporter/awsemfexporter/metric_translator_test.go index 9986e87bd641..852ccdcc54d5 100644 --- a/exporter/awsemfexporter/metric_translator_test.go +++ b/exporter/awsemfexporter/metric_translator_test.go @@ -282,6 +282,8 @@ func TestTranslateOtToGroupedMetric(t *testing.T) { containerInsightMetric.Resource().Attributes().PutStr(conventions.AttributeServiceName, "containerInsightsKubeAPIServerScraper") gpuMetric := createTestResourceMetricsHelper(defaultNumberOfTestMetrics + 1) gpuMetric.Resource().Attributes().PutStr(conventions.AttributeServiceName, "containerInsightsDCGMExporterScraper") + neuronMetric := createTestResourceMetricsHelper(defaultNumberOfTestMetrics + 1) + neuronMetric.Resource().Attributes().PutStr(conventions.AttributeServiceName, "containerInsightsNeuronMonitorScraper") counterSumMetrics := map[string]*metricInfo{ "spanCounter": { @@ -390,6 +392,20 @@ func TestTranslateOtToGroupedMetric(t *testing.T) { "myServiceNS/containerInsightsDCGMExporterScraper", containerInsightsReceiver, }, + { + "neuron monitor receiver", + neuronMetric, + map[string]string{ + "isItAnError": "false", + "spanName": "testSpan", + }, + map[string]string{ + oTellibDimensionKey: "cloudwatch-lib", + "spanName": "testSpan", + }, + "myServiceNS/containerInsightsNeuronMonitorScraper", + containerInsightsReceiver, + }, } for _, tc := range testCases { diff --git a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper.go b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper.go index 9a786f22e549..0b4e1287f5a0 100644 --- a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper.go +++ b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper.go @@ -30,23 +30,20 @@ const ( ) type NeuronMonitorScraper struct { - ctx context.Context - settings component.TelemetrySettings - host component.Host - hostInfoProvider hostInfoProvider - podNameInfoProvider podNameInfoProvider - prometheusReceiver receiver.Metrics - running bool + ctx context.Context + settings component.TelemetrySettings + host component.Host + hostInfoProvider hostInfoProvider + prometheusReceiver receiver.Metrics + running bool } type NeuronMonitorScraperOpts struct { - Ctx context.Context - TelemetrySettings component.TelemetrySettings - Consumer consumer.Metrics - Host component.Host - HostInfoProvider hostInfoProvider - PodNameInfoProvider podNameInfoProvider - BearerToken string + Ctx context.Context + TelemetrySettings component.TelemetrySettings + Consumer consumer.Metrics + Host component.Host + HostInfoProvider hostInfoProvider } type hostInfoProvider interface { @@ -54,12 +51,6 @@ type hostInfoProvider interface { GetInstanceID() string } -// lets assume new provider will provide PodName -type podNameInfoProvider interface { - GetPodName() string - GetNamespace() string -} - func NewNeuronMonitorScraper(opts NeuronMonitorScraperOpts) (*NeuronMonitorScraper, error) { if opts.Consumer == nil { return nil, errors.New("consumer cannot be nil") @@ -70,26 +61,8 @@ func NewNeuronMonitorScraper(opts NeuronMonitorScraperOpts) (*NeuronMonitorScrap if opts.HostInfoProvider == nil { return nil, errors.New("cluster name provider cannot be nil") } - if opts.PodNameInfoProvider == nil { - return nil, errors.New("pod 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, @@ -97,14 +70,14 @@ func NewNeuronMonitorScraper(opts NeuronMonitorScraperOpts) (*NeuronMonitorScrap MetricsPath: "/metrics", ServiceDiscoveryConfigs: discovery.Configs{ &kubernetes.SDConfig{ - Role: kubernetes.RoleEndpoint, + Role: kubernetes.RoleService, NamespaceDiscovery: kubernetes.NamespaceDiscovery{ IncludeOwnNamespace: true, }, Selectors: []kubernetes.SelectorConfig{ { - Role: kubernetes.RoleEndpoint, - Label: "k8s-app=dcgm-exporter-service", // TODO: Find the correct name + Role: kubernetes.RoleService, + Label: "k8s-app=neuron-monitor-service", }, }, AttachMetadata: kubernetes.AttachMetadataConfig{ @@ -116,64 +89,68 @@ func NewNeuronMonitorScraper(opts NeuronMonitorScraperOpts) (*NeuronMonitorScrap { SourceLabels: model.LabelNames{"__address__"}, Regex: relabel.MustNewRegexp("([^:]+)(?::\\d+)?"), - Replacement: "${1}:9400", + Replacement: "${1}:8000", TargetLabel: "__address__", Action: relabel.Replace, }, + }, + MetricRelabelConfigs: []*relabel.Config{ { - SourceLabels: model.LabelNames{"__meta_kubernetes_pod_node_name"}, + SourceLabels: model.LabelNames{"__name__"}, + Regex: relabel.MustNewRegexp("neuron.*|system_.*|execution_.*"), + Action: relabel.Keep, + }, + { + SourceLabels: model.LabelNames{"instance_name"}, TargetLabel: "NodeName", Regex: relabel.MustNewRegexp("(.*)"), - Replacement: "$1", + Replacement: "${1}", Action: relabel.Replace, }, { - SourceLabels: model.LabelNames{"__meta_kubernetes_service_name"}, - TargetLabel: "Service", + SourceLabels: model.LabelNames{"instance_id"}, + TargetLabel: "InstanceId", Regex: relabel.MustNewRegexp("(.*)"), - Replacement: "$1", + Replacement: "${1}", Action: relabel.Replace, }, - }, - MetricRelabelConfigs: []*relabel.Config{ - { - SourceLabels: model.LabelNames{"__name__"}, - Regex: relabel.MustNewRegexp("neuron.*"), - Action: relabel.Keep, - }, - // 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{"__name__"}, - TargetLabel: "Namespace", - Regex: relabel.MustNewRegexp("neuron.*"), - Replacement: opts.PodNameInfoProvider.GetNamespace(), + SourceLabels: model.LabelNames{"neuroncore"}, + TargetLabel: "DeviceId", + Regex: relabel.MustNewRegexp("(.*)"), + Replacement: "${1}", Action: relabel.Replace, }, + // hacky way to inject static values (clusterName) 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{"__name__"}, + SourceLabels: model.LabelNames{"instance_id"}, TargetLabel: "ClusterName", Regex: relabel.MustNewRegexp("neuron.*"), Replacement: opts.HostInfoProvider.GetClusterName(), Action: relabel.Replace, }, + // hacky way to inject static values (namespace, podName, containerName) to label set without additional processor, + // we need these labels only when we have neuronCore label specified in logs + // we process and replace with actual podName, namespace and ContainerName + // relabel looks up an existing label then creates another label with given key (TargetLabel) and value (static) { - SourceLabels: model.LabelNames{"__name__"}, - TargetLabel: "InstanceId", - Regex: relabel.MustNewRegexp("neuron.*"), - Replacement: opts.HostInfoProvider.GetInstanceID(), + SourceLabels: model.LabelNames{"neuroncore"}, + TargetLabel: "Namespace", + Regex: relabel.MustNewRegexp("(.*)"), + Replacement: "${1}", Action: relabel.Replace, }, { - SourceLabels: model.LabelNames{"namespace"}, + SourceLabels: model.LabelNames{"neuroncore"}, TargetLabel: "PodName", - Regex: relabel.MustNewRegexp("neuron.*"), - Replacement: opts.PodNameInfoProvider.GetPodName(), + Regex: relabel.MustNewRegexp("(.*)"), + Replacement: "${1}", Action: relabel.Replace, }, { SourceLabels: model.LabelNames{"neuroncore"}, - TargetLabel: "DeviceId", + TargetLabel: "ContainerName", Regex: relabel.MustNewRegexp("(.*)"), Replacement: "${1}", Action: relabel.Replace, @@ -198,12 +175,11 @@ func NewNeuronMonitorScraper(opts NeuronMonitorScraperOpts) (*NeuronMonitorScrap } return &NeuronMonitorScraper{ - ctx: opts.Ctx, - settings: opts.TelemetrySettings, - host: opts.Host, - hostInfoProvider: opts.HostInfoProvider, - podNameInfoProvider: opts.PodNameInfoProvider, - prometheusReceiver: promReceiver, + ctx: opts.Ctx, + settings: opts.TelemetrySettings, + host: opts.Host, + hostInfoProvider: opts.HostInfoProvider, + prometheusReceiver: promReceiver, }, nil } diff --git a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go index 5213d09e48b3..654c16125867 100644 --- a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go +++ b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go @@ -22,24 +22,95 @@ import ( ) const renameMetric = ` +# HELP python_gc_objects_collected_total Objects collected during gc +# TYPE python_gc_objects_collected_total counter +python_gc_objects_collected_total{generation="0"} 75.0 +# HELP python_gc_objects_uncollectable_total Uncollectable objects found during GC +# TYPE python_gc_objects_uncollectable_total counter +python_gc_objects_uncollectable_total{generation="0"} 0.0 +# HELP python_gc_collections_total Number of times this generation was collected +# TYPE python_gc_collections_total counter +python_gc_collections_total{generation="0"} 44.0 +# HELP python_info Python platform information +# TYPE python_info gauge +python_info{implementation="CPython",major="3",minor="8",patchlevel="10",version="3.8.10"} 1.0 +# HELP process_virtual_memory_bytes Virtual memory size in bytes. +# TYPE process_virtual_memory_bytes gauge +process_virtual_memory_bytes 1.80707328e+08 +# HELP process_resident_memory_bytes Resident memory size in bytes. +# TYPE process_resident_memory_bytes gauge +process_resident_memory_bytes 2.11968e+07 +# HELP process_start_time_seconds Start time of the process since unix epoch in seconds. +# TYPE process_start_time_seconds gauge +process_start_time_seconds 1.7083389395e+09 +# HELP process_cpu_seconds_total Total user and system CPU time spent in seconds. +# TYPE process_cpu_seconds_total counter +process_cpu_seconds_total 0.08 +# HELP process_open_fds Number of open file descriptors. +# TYPE process_open_fds gauge +process_open_fds 6.0 +# HELP execution_errors_total Execution errors total +# TYPE execution_errors_total counter +execution_errors_total{availability_zone="us-east-1c",error_type="generic",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",region="us-east-1",runtime_tag="367",subnet_id="subnet-06a7754948e8a000f"} 0.0 +# HELP execution_errors_created Execution errors total +# TYPE execution_errors_created gauge +execution_errors_created{availability_zone="us-east-1c",error_type="generic",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",region="us-east-1",runtime_tag="367",subnet_id="subnet-06a7754948e8a000f"} 1.7083389404380567e+09 +# HELP execution_status_total Execution status total +# TYPE execution_status_total counter +execution_status_total{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",region="us-east-1",runtime_tag="367",status_type="completed",subnet_id="subnet-06a7754948e8a000f"} 0.0 +# HELP execution_status_created Execution status total +# TYPE execution_status_created gauge +execution_status_created{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",region="us-east-1",runtime_tag="367",status_type="completed",subnet_id="subnet-06a7754948e8a000f"} 1.7083389404381733e+09 # HELP neuron_runtime_memory_used_bytes Runtime memory used bytes # TYPE neuron_runtime_memory_used_bytes gauge neuron_runtime_memory_used_bytes{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",memory_location="host",region="us-east-1",runtime_tag="367",subnet_id="subnet-06a7754948e8a000f"} 9.043968e+06 +# HELP neuroncore_memory_usage_constants NeuronCore memory utilization for constants +# TYPE neuroncore_memory_usage_constants gauge +neuroncore_memory_usage_constants{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",memory_location="None",neuroncore="0",region="us-east-1",runtime_tag="367",subnet_id="subnet-06a7754948e8a000f"} 0.0 +# HELP neuroncore_memory_usage_model_code NeuronCore memory utilization for model_code +# TYPE neuroncore_memory_usage_model_code gauge +neuroncore_memory_usage_model_code{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",memory_location="None",neuroncore="0",region="us-east-1",runtime_tag="367",subnet_id="subnet-06a7754948e8a000f"} 1.00752896e+08 +# HELP neuroncore_memory_usage_model_shared_scratchpad NeuronCore memory utilization for model_shared_scratchpad +# TYPE neuroncore_memory_usage_model_shared_scratchpad gauge +neuroncore_memory_usage_model_shared_scratchpad{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",memory_location="None",neuroncore="0",region="us-east-1",runtime_tag="367",subnet_id="subnet-06a7754948e8a000f"} 0.0 +# HELP neuroncore_memory_usage_runtime_memory NeuronCore memory utilization for runtime_memory +# TYPE neuroncore_memory_usage_runtime_memory gauge +neuroncore_memory_usage_runtime_memory{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",memory_location="None",neuroncore="0",region="us-east-1",runtime_tag="367",subnet_id="subnet-06a7754948e8a000f"} 0.0 +# HELP neuroncore_memory_usage_tensors NeuronCore memory utilization for tensors +# TYPE neuroncore_memory_usage_tensors gauge +neuroncore_memory_usage_tensors{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",memory_location="None",neuroncore="0",region="us-east-1",runtime_tag="367",subnet_id="subnet-06a7754948e8a000f"} 6.315872e+06 # HELP neuroncore_utilization_ratio NeuronCore utilization ratio # TYPE neuroncore_utilization_ratio gauge neuroncore_utilization_ratio{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",neuroncore="0",region="us-east-1",runtime_tag="367",subnet_id="subnet-06a7754948e8a000f"} 0.1 +# HELP system_memory_total_bytes System memory total_bytes bytes +# TYPE system_memory_total_bytes gauge +system_memory_total_bytes{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",region="us-east-1",subnet_id="subnet-06a7754948e8a000f"} 5.32523487232e+011 +# HELP system_memory_used_bytes System memory used_bytes bytes +# TYPE system_memory_used_bytes gauge +system_memory_used_bytes{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",region="us-east-1",subnet_id="subnet-06a7754948e8a000f"} 7.6337672192e+010 +# HELP system_swap_total_bytes System swap total_bytes bytes +# TYPE system_swap_total_bytes gauge +system_swap_total_bytes{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",region="us-east-1",subnet_id="subnet-06a7754948e8a000f"} 0.0 +# HELP system_swap_used_bytes System swap used_bytes bytes +# TYPE system_swap_used_bytes gauge +system_swap_used_bytes{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",region="us-east-1",subnet_id="subnet-06a7754948e8a000f"} 0.0 +# HELP system_vcpu_count System vCPU count +# TYPE system_vcpu_count gauge +system_vcpu_count{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",region="us-east-1",subnet_id="subnet-06a7754948e8a000f"} 128.0 +# HELP system_vcpu_usage_ratio System CPU utilization ratio +# TYPE system_vcpu_usage_ratio gauge +system_vcpu_usage_ratio{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",region="us-east-1",subnet_id="subnet-06a7754948e8a000f",usage_type="user"} 0.008199999999999999 +# HELP instance_info EC2 instance information +# TYPE instance_info gauge +instance_info{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",region="us-east-1",subnet_id="subnet-06a7754948e8a000f"} 1.0 ` const dummyInstanceId = "i-0000000000" -const dummyPodName = "podname" -const dummyNamespace = "kube-system" +const nueronCoreID = "0" type mockHostInfoProvider struct { } -type mockPodInfoProvider struct { -} - func (m mockHostInfoProvider) GetClusterName() string { return "cluster-name" } @@ -48,23 +119,16 @@ func (m mockHostInfoProvider) GetInstanceID() string { return dummyInstanceId } -func (m mockPodInfoProvider) GetPodName() string { - return dummyPodName -} - -func (m mockPodInfoProvider) GetNamespace() string { - return dummyNamespace -} - type mockConsumer struct { - t *testing.T - up *bool - coreUtil *bool - memUsed *bool - instanceId *bool - podName *bool - namespace *bool - relabeled *bool + t *testing.T + up *bool + coreUtil *bool + memUsed *bool + instanceId *bool + podName *bool + containerName *bool + namespace *bool + relabeled *bool } func (m mockConsumer) Capabilities() consumer.Capabilities { @@ -79,21 +143,35 @@ 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) - fmt.Println("!!!Metric Name is ", metric.Name()) if metric.Name() == "neuroncore_utilization_ratio" { assert.Equal(m.t, float64(0.1), metric.Gauge().DataPoints().At(0).DoubleValue()) + *m.coreUtil = true + } + + if strings.Contains(metric.Name(), "neuroncore") { instanceId, _ := metric.Gauge().DataPoints().At(0).Attributes().Get("InstanceId") podName, _ := metric.Gauge().DataPoints().At(0).Attributes().Get("PodName") namespace, _ := metric.Gauge().DataPoints().At(0).Attributes().Get("Namespace") + containerName, _ := metric.Gauge().DataPoints().At(0).Attributes().Get("Namespace") _, relabeled := metric.Gauge().DataPoints().At(0).Attributes().Get("DeviceId") - fmt.Println("!!!InstanceId is ", instanceId.Str()) - *m.instanceId = instanceId.Str() == dummyInstanceId - *m.podName = podName.Str() == dummyPodName - *m.namespace = namespace.Str() == dummyNamespace - *m.coreUtil = true + *m.instanceId = instanceId.Str() == "i-09db9b55e0095612f" + *m.podName = podName.Str() == nueronCoreID + *m.namespace = namespace.Str() == nueronCoreID + *m.containerName = containerName.Str() == nueronCoreID *m.relabeled = relabeled } + if strings.Contains(metric.Name(), "system") { + instanceId, _ := metric.Gauge().DataPoints().At(0).Attributes().Get("InstanceId") + _, podNameFound := metric.Gauge().DataPoints().At(0).Attributes().Get("PodName") + _, namespaceFound := metric.Gauge().DataPoints().At(0).Attributes().Get("Namespace") + _, relabelFound := metric.Gauge().DataPoints().At(0).Attributes().Get("DeviceId") + assert.Equal(m.t, instanceId.Str(), "i-09db9b55e0095612f") + assert.False(m.t, podNameFound) + assert.False(m.t, namespaceFound) + assert.False(m.t, relabelFound) + } + if metric.Name() == "neuron_runtime_memory_used_bytes" { assert.Equal(m.t, float64(9.043968e+06), metric.Gauge().DataPoints().At(0).DoubleValue()) *m.memUsed = true @@ -114,28 +192,25 @@ func TestNewNeuronScraperBadInputs(t *testing.T) { tests := []NeuronMonitorScraperOpts{ { - Ctx: context.TODO(), - TelemetrySettings: settings, - Consumer: nil, - Host: componenttest.NewNopHost(), - HostInfoProvider: mockHostInfoProvider{}, - PodNameInfoProvider: mockPodInfoProvider{}, + Ctx: context.TODO(), + TelemetrySettings: settings, + Consumer: nil, + Host: componenttest.NewNopHost(), + HostInfoProvider: mockHostInfoProvider{}, }, { - Ctx: context.TODO(), - TelemetrySettings: settings, - Consumer: mockConsumer{}, - Host: nil, - HostInfoProvider: mockHostInfoProvider{}, - PodNameInfoProvider: mockPodInfoProvider{}, + Ctx: context.TODO(), + TelemetrySettings: settings, + Consumer: mockConsumer{}, + Host: nil, + HostInfoProvider: mockHostInfoProvider{}, }, { - Ctx: context.TODO(), - TelemetrySettings: settings, - Consumer: mockConsumer{}, - Host: componenttest.NewNopHost(), - HostInfoProvider: nil, - PodNameInfoProvider: mockPodInfoProvider{}, + Ctx: context.TODO(), + TelemetrySettings: settings, + Consumer: mockConsumer{}, + Host: componenttest.NewNopHost(), + HostInfoProvider: nil, }, } @@ -154,31 +229,31 @@ func TestNewNeuronScraperEndToEnd(t *testing.T) { instanceId := false memUsed := false podName := false + containerName := false namespace := false relabeled := false consumer := mockConsumer{ - t: t, - up: &upPtr, - coreUtil: &coreUtil, - memUsed: &memUsed, - instanceId: &instanceId, - podName: &podName, - namespace: &namespace, - relabeled: &relabeled, + t: t, + up: &upPtr, + coreUtil: &coreUtil, + memUsed: &memUsed, + instanceId: &instanceId, + podName: &podName, + containerName: &containerName, + namespace: &namespace, + relabeled: &relabeled, } settings := componenttest.NewNopTelemetrySettings() settings.Logger, _ = zap.NewDevelopment() scraper, err := NewNeuronMonitorScraper(NeuronMonitorScraperOpts{ - Ctx: context.TODO(), - TelemetrySettings: settings, - Consumer: mockConsumer{}, - Host: componenttest.NewNopHost(), - HostInfoProvider: mockHostInfoProvider{}, - PodNameInfoProvider: mockPodInfoProvider{}, - BearerToken: "", + Ctx: context.TODO(), + TelemetrySettings: settings, + Consumer: mockConsumer{}, + Host: componenttest.NewNopHost(), + HostInfoProvider: mockHostInfoProvider{}, }) assert.NoError(t, err) assert.Equal(t, mockHostInfoProvider{}, scraper.hostInfoProvider) @@ -223,77 +298,54 @@ func TestNewNeuronScraperEndToEnd(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__"}, - Regex: relabel.MustNewRegexp("neuron.*"), + Regex: relabel.MustNewRegexp("neuron.*|system_.*|execution_.*"), Action: relabel.Keep, }, { - SourceLabels: model.LabelNames{"neuroncore"}, - TargetLabel: "DeviceId", + SourceLabels: model.LabelNames{"instance_id"}, + TargetLabel: "InstanceId", Regex: relabel.MustNewRegexp("(.*)"), Replacement: "${1}", Action: relabel.Replace, }, - // test hack to inject cluster name as label { SourceLabels: model.LabelNames{"neuroncore"}, - TargetLabel: "Namespace", + TargetLabel: "DeviceId", Regex: relabel.MustNewRegexp("(.*)"), - Replacement: scraper.podNameInfoProvider.GetNamespace(), + Replacement: "${1}", Action: relabel.Replace, }, + // test hack to inject cluster name as label { - SourceLabels: model.LabelNames{"neuroncore"}, + SourceLabels: model.LabelNames{"instance_id"}, TargetLabel: "ClusterName", - Regex: relabel.MustNewRegexp("(.*)"), + Regex: relabel.MustNewRegexp("neuron.*"), Replacement: scraper.hostInfoProvider.GetClusterName(), Action: relabel.Replace, }, { SourceLabels: model.LabelNames{"neuroncore"}, - TargetLabel: "InstanceId", + TargetLabel: "Namespace", Regex: relabel.MustNewRegexp("(.*)"), - Replacement: scraper.hostInfoProvider.GetInstanceID(), + Replacement: "${1}", Action: relabel.Replace, }, { SourceLabels: model.LabelNames{"neuroncore"}, TargetLabel: "PodName", Regex: relabel.MustNewRegexp("(.*)"), - Replacement: scraper.podNameInfoProvider.GetPodName(), + Replacement: "${1}", + Action: relabel.Replace, + }, + { + SourceLabels: model.LabelNames{"neuroncore"}, + TargetLabel: "ContainerName", + Regex: relabel.MustNewRegexp("(.*)"), + Replacement: "${1}", Action: relabel.Replace, }, }, @@ -330,6 +382,7 @@ func TestNewNeuronScraperEndToEnd(t *testing.T) { assert.True(t, *consumer.memUsed) assert.True(t, *consumer.instanceId) assert.True(t, *consumer.podName) + assert.True(t, *consumer.containerName) assert.True(t, *consumer.namespace) assert.True(t, *consumer.relabeled) } diff --git a/receiver/awscontainerinsightreceiver/internal/pod/podinfo.go b/receiver/awscontainerinsightreceiver/internal/pod/podinfo.go deleted file mode 100644 index 53193c2067f5..000000000000 --- a/receiver/awscontainerinsightreceiver/internal/pod/podinfo.go +++ /dev/null @@ -1,26 +0,0 @@ -package pod - -type Info struct { - namespace string - podName string -} - -// GetNamespace returns the namespace for the pod -func (m *Info) GetNamespace() string { - return m.namespace -} - -// GetPodName returns the podName for the pod -func (m *Info) GetPodName() string { - return m.podName -} - -// NewInfo creates a new Info struct -func NewInfo() (*Info, error) { - mInfo := &Info{ - podName: "DummyPodName", - namespace: "DummyNamespace", - } - - return mInfo, nil -} diff --git a/receiver/awscontainerinsightreceiver/receiver.go b/receiver/awscontainerinsightreceiver/receiver.go index a682c8ff1ff5..d8a47ab5c37b 100644 --- a/receiver/awscontainerinsightreceiver/receiver.go +++ b/receiver/awscontainerinsightreceiver/receiver.go @@ -10,7 +10,6 @@ import ( "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/gpu" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/neuron" - podinfo "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/pod" "go.opentelemetry.io/collector/component" "go.opentelemetry.io/collector/consumer" "go.opentelemetry.io/collector/pdata/pmetric" @@ -73,11 +72,6 @@ func (acir *awsContainerInsightReceiver) Start(ctx context.Context, host compone return err } - podinfo, err := podinfo.NewInfo() - if err != nil { - return err - } - if acir.config.ContainerOrchestrator == ci.EKS { k8sDecorator, err := stores.NewK8sDecorator(ctx, acir.config.TagService, acir.config.PrefFullPodName, acir.config.AddFullPodNameMetricLabel, acir.config.AddContainerNameMetricLabel, acir.config.EnableControlPlaneMetrics, acir.settings.Logger) if err != nil { @@ -111,7 +105,7 @@ func (acir *awsContainerInsightReceiver) Start(ctx context.Context, host compone acir.settings.Logger.Debug("Unable to start dcgm scraper", zap.Error(err)) } - err = acir.startNeuronScraper(ctx, host, hostinfo, podinfo) + err = acir.initNeuronScraper(ctx, host, hostinfo) if err != nil { acir.settings.Logger.Debug("Unable to start dcgm scraper", zap.Error(err)) } @@ -205,28 +199,18 @@ func (acir *awsContainerInsightReceiver) initDcgmScraper(ctx context.Context, ho return err } -func (acir *awsContainerInsightReceiver) startNeuronScraper(ctx context.Context, host component.Host, hostinfo *hostInfo.Info, podinfo *podinfo.Info) error { +func (acir *awsContainerInsightReceiver) initNeuronScraper(ctx context.Context, host component.Host, hostinfo *hostInfo.Info) error { if !acir.config.EnableNeuronMetric { 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.neuronMonitorScraper, err = neuron.NewNeuronMonitorScraper(neuron.NeuronMonitorScraperOpts{ - Ctx: ctx, - TelemetrySettings: acir.settings, - Consumer: acir.nextConsumer, - Host: host, - HostInfoProvider: hostinfo, - PodNameInfoProvider: podinfo, - BearerToken: bearerToken, + Ctx: ctx, + TelemetrySettings: acir.settings, + Consumer: acir.nextConsumer, + Host: host, + HostInfoProvider: hostinfo, }) return err } From 4f0e3e16dd34477d6abd033268dae3d5738e6526 Mon Sep 17 00:00:00 2001 From: Samarth Singh Date: Fri, 23 Feb 2024 18:00:00 +0000 Subject: [PATCH 11/53] Make Neuron Scraper extension for simple prometheus scraper --- .../awscontainerinsightreceiver/config.go | 3 +- .../internal/neuron/neuron_monitor_scraper.go | 208 ------------ .../neuron/neuron_monitor_scraper_config.go | 100 ++++++ .../neuron/neuron_monitor_scraper_test.go | 311 +++--------------- .../prometheus_scraper_testutils.go | 166 ++++++++++ .../simple_prometheus_scraper.go | 98 ++++++ .../simple_prometheus_scraper_test.go | 58 ++++ .../awscontainerinsightreceiver/receiver.go | 25 +- 8 files changed, 476 insertions(+), 493 deletions(-) delete mode 100644 receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper.go create mode 100644 receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go create mode 100644 receiver/awscontainerinsightreceiver/internal/prometheusscraper/prometheus_scraper_testutils.go create mode 100644 receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go create mode 100644 receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper_test.go diff --git a/receiver/awscontainerinsightreceiver/config.go b/receiver/awscontainerinsightreceiver/config.go index e55f891fd224..307dbe87702d 100644 --- a/receiver/awscontainerinsightreceiver/config.go +++ b/receiver/awscontainerinsightreceiver/config.go @@ -61,7 +61,6 @@ type Config struct { // EnableGpuMetric toggles GPU monitoring where metrics are scraped from vendor specific sources EnableGpuMetric bool `mapstructure:"gpu_metrics"` - // EnableNeuronMetric disables Neuron monitoring where metrics are scraped from vendor specific sources - // The default value is true meaning Neuron metrics get collected out of the box unless it's disabled + // EnableNeuronMetric toggles Neuron monitoring where metrics are scraped from vendor specific sources EnableNeuronMetric bool `mapstructure:"neuron_metrics"` } diff --git a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper.go b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper.go deleted file mode 100644 index 0b4e1287f5a0..000000000000 --- a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper.go +++ /dev/null @@ -1,208 +0,0 @@ -// Copyright The OpenTelemetry Authors -// SPDX-License-Identifier: Apache-2.0 - -package neuron - -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 = "containerInsightsNeuronMonitorScraper" -) - -type NeuronMonitorScraper struct { - ctx context.Context - settings component.TelemetrySettings - host component.Host - hostInfoProvider hostInfoProvider - prometheusReceiver receiver.Metrics - running bool -} - -type NeuronMonitorScraperOpts struct { - Ctx context.Context - TelemetrySettings component.TelemetrySettings - Consumer consumer.Metrics - Host component.Host - HostInfoProvider hostInfoProvider -} - -type hostInfoProvider interface { - GetClusterName() string - GetInstanceID() string -} - -func NewNeuronMonitorScraper(opts NeuronMonitorScraperOpts) (*NeuronMonitorScraper, 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{ - 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=neuron-monitor-service", - }, - }, - AttachMetadata: kubernetes.AttachMetadataConfig{ - Node: true, - }, - }, - }, - RelabelConfigs: []*relabel.Config{ - { - SourceLabels: model.LabelNames{"__address__"}, - Regex: relabel.MustNewRegexp("([^:]+)(?::\\d+)?"), - Replacement: "${1}:8000", - TargetLabel: "__address__", - Action: relabel.Replace, - }, - }, - MetricRelabelConfigs: []*relabel.Config{ - { - SourceLabels: model.LabelNames{"__name__"}, - Regex: relabel.MustNewRegexp("neuron.*|system_.*|execution_.*"), - Action: relabel.Keep, - }, - { - SourceLabels: model.LabelNames{"instance_name"}, - TargetLabel: "NodeName", - Regex: relabel.MustNewRegexp("(.*)"), - Replacement: "${1}", - Action: relabel.Replace, - }, - { - SourceLabels: model.LabelNames{"instance_id"}, - TargetLabel: "InstanceId", - Regex: relabel.MustNewRegexp("(.*)"), - Replacement: "${1}", - Action: relabel.Replace, - }, - { - SourceLabels: model.LabelNames{"neuroncore"}, - TargetLabel: "DeviceId", - Regex: relabel.MustNewRegexp("(.*)"), - Replacement: "${1}", - Action: relabel.Replace, - }, - // hacky way to inject static values (clusterName) 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{"instance_id"}, - TargetLabel: "ClusterName", - Regex: relabel.MustNewRegexp("neuron.*"), - Replacement: opts.HostInfoProvider.GetClusterName(), - Action: relabel.Replace, - }, - // hacky way to inject static values (namespace, podName, containerName) to label set without additional processor, - // we need these labels only when we have neuronCore label specified in logs - // we process and replace with actual podName, namespace and ContainerName - // relabel looks up an existing label then creates another label with given key (TargetLabel) and value (static) - { - SourceLabels: model.LabelNames{"neuroncore"}, - TargetLabel: "Namespace", - Regex: relabel.MustNewRegexp("(.*)"), - Replacement: "${1}", - Action: relabel.Replace, - }, - { - SourceLabels: model.LabelNames{"neuroncore"}, - TargetLabel: "PodName", - Regex: relabel.MustNewRegexp("(.*)"), - Replacement: "${1}", - Action: relabel.Replace, - }, - { - SourceLabels: model.LabelNames{"neuroncore"}, - TargetLabel: "ContainerName", - 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 &NeuronMonitorScraper{ - ctx: opts.Ctx, - settings: opts.TelemetrySettings, - host: opts.Host, - hostInfoProvider: opts.HostInfoProvider, - prometheusReceiver: promReceiver, - }, nil -} - -func (ds *NeuronMonitorScraper) 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 *NeuronMonitorScraper) 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/neuron/neuron_monitor_scraper_config.go b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go new file mode 100644 index 000000000000..735af401b06e --- /dev/null +++ b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go @@ -0,0 +1,100 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +package nueron + +import ( + "time" + + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/prometheusscraper" + "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" +) + +const ( + caFile = "/var/run/secrets/kubernetes.io/serviceaccount/ca.crt" + collectionInterval = 60 * time.Second + jobName = "containerInsightsNeuronMonitorScraper" +) + +func GetNueronScrapeConfig(opts prometheusscraper.SimplePromethuesScraperOpts) *config.ScrapeConfig { + + return &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=neuron-monitor-service", + }, + }, + AttachMetadata: kubernetes.AttachMetadataConfig{ + Node: true, + }, + }, + }, + RelabelConfigs: []*relabel.Config{ + { + SourceLabels: model.LabelNames{"__address__"}, + Regex: relabel.MustNewRegexp("([^:]+)(?::\\d+)?"), + Replacement: "${1}:8000", + TargetLabel: "__address__", + Action: relabel.Replace, + }, + }, + MetricRelabelConfigs: GetNueronMetricRelabelConfigs(opts), + } +} + +func GetNueronMetricRelabelConfigs(opts prometheusscraper.SimplePromethuesScraperOpts) []*relabel.Config { + + return []*relabel.Config{ + { + SourceLabels: model.LabelNames{"__name__"}, + Regex: relabel.MustNewRegexp("neuron.*|system_.*|execution_.*"), + Action: relabel.Keep, + }, + { + SourceLabels: model.LabelNames{"instance_name"}, + TargetLabel: "NodeName", + Regex: relabel.MustNewRegexp("(.*)"), + Replacement: "${1}", + Action: relabel.Replace, + }, + { + SourceLabels: model.LabelNames{"instance_id"}, + TargetLabel: "InstanceId", + Regex: relabel.MustNewRegexp("(.*)"), + Replacement: "${1}", + Action: relabel.Replace, + }, + { + SourceLabels: model.LabelNames{"neuroncore"}, + TargetLabel: "DeviceId", + Regex: relabel.MustNewRegexp("(.*)"), + Replacement: "${1}", + Action: relabel.Replace, + }, + // hacky way to inject static values (clusterName) 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{"instance_id"}, + TargetLabel: "ClusterName", + Regex: relabel.MustNewRegexp("(.*)"), + Replacement: opts.HostInfoProvider.GetClusterName(), + Action: relabel.Replace, + }, + } +} diff --git a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go index 654c16125867..09708c4488e0 100644 --- a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go +++ b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go @@ -1,24 +1,11 @@ -package neuron +package nueron 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/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/prometheusscraper" "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 = ` @@ -105,289 +92,69 @@ system_vcpu_usage_ratio{availability_zone="us-east-1c",instance_id="i-09db9b55e0 instance_info{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",region="us-east-1",subnet_id="subnet-06a7754948e8a000f"} 1.0 ` -const dummyInstanceId = "i-0000000000" -const nueronCoreID = "0" +const dummyClusterName = "cluster-name" +const dummyHostName = "i-000000000" type mockHostInfoProvider struct { } func (m mockHostInfoProvider) GetClusterName() string { - return "cluster-name" + return dummyClusterName } func (m mockHostInfoProvider) GetInstanceID() string { - return dummyInstanceId -} - -type mockConsumer struct { - t *testing.T - up *bool - coreUtil *bool - memUsed *bool - instanceId *bool - podName *bool - containerName *bool - namespace *bool - relabeled *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() == "neuroncore_utilization_ratio" { - assert.Equal(m.t, float64(0.1), metric.Gauge().DataPoints().At(0).DoubleValue()) - *m.coreUtil = true - } - - if strings.Contains(metric.Name(), "neuroncore") { - instanceId, _ := metric.Gauge().DataPoints().At(0).Attributes().Get("InstanceId") - podName, _ := metric.Gauge().DataPoints().At(0).Attributes().Get("PodName") - namespace, _ := metric.Gauge().DataPoints().At(0).Attributes().Get("Namespace") - containerName, _ := metric.Gauge().DataPoints().At(0).Attributes().Get("Namespace") - _, relabeled := metric.Gauge().DataPoints().At(0).Attributes().Get("DeviceId") - *m.instanceId = instanceId.Str() == "i-09db9b55e0095612f" - *m.podName = podName.Str() == nueronCoreID - *m.namespace = namespace.Str() == nueronCoreID - *m.containerName = containerName.Str() == nueronCoreID - *m.relabeled = relabeled - } - - if strings.Contains(metric.Name(), "system") { - instanceId, _ := metric.Gauge().DataPoints().At(0).Attributes().Get("InstanceId") - _, podNameFound := metric.Gauge().DataPoints().At(0).Attributes().Get("PodName") - _, namespaceFound := metric.Gauge().DataPoints().At(0).Attributes().Get("Namespace") - _, relabelFound := metric.Gauge().DataPoints().At(0).Attributes().Get("DeviceId") - assert.Equal(m.t, instanceId.Str(), "i-09db9b55e0095612f") - assert.False(m.t, podNameFound) - assert.False(m.t, namespaceFound) - assert.False(m.t, relabelFound) - } - - if metric.Name() == "neuron_runtime_memory_used_bytes" { - assert.Equal(m.t, float64(9.043968e+06), metric.Gauge().DataPoints().At(0).DoubleValue()) - *m.memUsed = true - } - - if metric.Name() == "up" { - assert.Equal(m.t, float64(1.0), metric.Gauge().DataPoints().At(0).DoubleValue()) - *m.up = true - } - } - - return nil -} - -func TestNewNeuronScraperBadInputs(t *testing.T) { - settings := componenttest.NewNopTelemetrySettings() - settings.Logger, _ = zap.NewDevelopment() - - tests := []NeuronMonitorScraperOpts{ - { - 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 := NewNeuronMonitorScraper(tt) - - assert.Error(t, err) - assert.Nil(t, scraper) - } + return dummyHostName } func TestNewNeuronScraperEndToEnd(t *testing.T) { - - upPtr := false - coreUtil := false - instanceId := false - memUsed := false - podName := false - containerName := false - namespace := false - relabeled := false - - consumer := mockConsumer{ - t: t, - up: &upPtr, - coreUtil: &coreUtil, - memUsed: &memUsed, - instanceId: &instanceId, - podName: &podName, - containerName: &containerName, - namespace: &namespace, - relabeled: &relabeled, + expectedMetrics := make(map[string]prometheusscraper.ExpectedMetricStruct) + expectedMetrics["neuroncore_utilization_ratio"] = prometheusscraper.ExpectedMetricStruct{ + MetricValue: 0.1, + MetricLabels: []prometheusscraper.MetricLabel{ + {LabelName: "InstanceId", LabelValue: "i-09db9b55e0095612f"}, + {LabelName: "ClusterName", LabelValue: dummyClusterName}, + {LabelName: "DeviceId", LabelValue: "0"}, + }, } - - settings := componenttest.NewNopTelemetrySettings() - settings.Logger, _ = zap.NewDevelopment() - - scraper, err := NewNeuronMonitorScraper(NeuronMonitorScraperOpts{ - Ctx: context.TODO(), - TelemetrySettings: settings, - Consumer: mockConsumer{}, - Host: componenttest.NewNopHost(), - HostInfoProvider: mockHostInfoProvider{}, - }) - assert.NoError(t, err) - assert.Equal(t, mockHostInfoProvider{}, scraper.hostInfoProvider) - - // build up a new PR - promFactory := prometheusreceiver.NewFactory() - - targets := []*mocks.TestData{ - { - Name: "neuron", - Pages: []mocks.MockPrometheusResponse{ - {Code: 200, Data: renameMetric}, - }, + expectedMetrics["neuron_runtime_memory_used_bytes"] = prometheusscraper.ExpectedMetricStruct{ + MetricValue: 9.043968e+06, + MetricLabels: []prometheusscraper.MetricLabel{ + {LabelName: "InstanceId", LabelValue: "i-09db9b55e0095612f"}, + {LabelName: "ClusterName", LabelValue: dummyClusterName}, }, } - 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{}, - MetricRelabelConfigs: []*relabel.Config{ - { - SourceLabels: model.LabelNames{"__name__"}, - Regex: relabel.MustNewRegexp("neuron.*|system_.*|execution_.*"), - Action: relabel.Keep, - }, - { - SourceLabels: model.LabelNames{"instance_id"}, - TargetLabel: "InstanceId", - Regex: relabel.MustNewRegexp("(.*)"), - Replacement: "${1}", - Action: relabel.Replace, - }, - { - SourceLabels: model.LabelNames{"neuroncore"}, - TargetLabel: "DeviceId", - Regex: relabel.MustNewRegexp("(.*)"), - Replacement: "${1}", - Action: relabel.Replace, - }, - // test hack to inject cluster name as label - { - SourceLabels: model.LabelNames{"instance_id"}, - TargetLabel: "ClusterName", - Regex: relabel.MustNewRegexp("neuron.*"), - Replacement: scraper.hostInfoProvider.GetClusterName(), - Action: relabel.Replace, - }, - { - SourceLabels: model.LabelNames{"neuroncore"}, - TargetLabel: "Namespace", - Regex: relabel.MustNewRegexp("(.*)"), - Replacement: "${1}", - Action: relabel.Replace, - }, - { - SourceLabels: model.LabelNames{"neuroncore"}, - TargetLabel: "PodName", - Regex: relabel.MustNewRegexp("(.*)"), - Replacement: "${1}", - Action: relabel.Replace, - }, - { - SourceLabels: model.LabelNames{"neuroncore"}, - TargetLabel: "ContainerName", - Regex: relabel.MustNewRegexp("(.*)"), - Replacement: "${1}", - Action: relabel.Replace, - }, + expectedMetrics["execution_errors_created"] = prometheusscraper.ExpectedMetricStruct{ + MetricValue: 1.7083389404380567e+09, + MetricLabels: []prometheusscraper.MetricLabel{ + {LabelName: "InstanceId", LabelValue: "i-09db9b55e0095612f"}, + {LabelName: "ClusterName", LabelValue: dummyClusterName}, }, } - promConfig := prometheusreceiver.Config{ - PrometheusConfig: &config.Config{ - ScrapeConfigs: []*config.ScrapeConfig{scrapeConfig}, - }, + expectedMetrics["up"] = prometheusscraper.ExpectedMetricStruct{ + MetricValue: 1.0, + MetricLabels: []prometheusscraper.MetricLabel{}, } - // replace the prom receiver - params := receiver.CreateSettings{ - TelemetrySettings: scraper.settings, + consumer := prometheusscraper.MockConsumer{ + T: t, + ExpectedMetrics: expectedMetrics, } - 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() + mockedScraperOpts := prometheusscraper.GetMockedScraperOpts(consumer, mockHostInfoProvider{}) - t.Cleanup(func() { - scraper.Shutdown() + prometheusscraper.TestSimplePrometheusEndToEnd(prometheusscraper.TestSimplePrometheusEndToEndOpts{ + T: t, + Consumer: consumer, + DataReturned: renameMetric, + ScraperOpts: mockedScraperOpts, + ScrapeConfig: GetNueronScrapeConfig(mockedScraperOpts), + MetricRelabelConfig: GetNueronMetricRelabelConfigs(mockedScraperOpts), }) - - // 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.coreUtil) - assert.True(t, *consumer.memUsed) - assert.True(t, *consumer.instanceId) - assert.True(t, *consumer.podName) - assert.True(t, *consumer.containerName) - assert.True(t, *consumer.namespace) - assert.True(t, *consumer.relabeled) } -func TestDcgmScraperJobName(t *testing.T) { +func TestNeuronMonitorScraperJobName(t *testing.T) { // needs to start with containerInsights assert.True(t, strings.HasPrefix(jobName, "containerInsightsNeuronMonitorScraper")) } diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/prometheus_scraper_testutils.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/prometheus_scraper_testutils.go new file mode 100644 index 000000000000..7d8c730cacac --- /dev/null +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/prometheus_scraper_testutils.go @@ -0,0 +1,166 @@ +package prometheusscraper + +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" +) + +type MetricLabel struct { + LabelName string + LabelValue string +} + +type ExpectedMetricStruct struct { + MetricValue float64 + MetricLabels []MetricLabel +} + +type TestSimplePrometheusEndToEndOpts struct { + T *testing.T + Consumer consumer.Metrics + DataReturned string + ScraperOpts SimplePromethuesScraperOpts + ScrapeConfig *config.ScrapeConfig + MetricRelabelConfig []*relabel.Config +} + +type MockConsumer struct { + T *testing.T + ExpectedMetrics map[string]ExpectedMetricStruct +} + +func (m MockConsumer) Capabilities() consumer.Capabilities { + return consumer.Capabilities{ + MutatesData: false, + } +} + +func (m MockConsumer) ConsumeMetrics(_ context.Context, md pmetric.Metrics) error { + expectedMetricsCount := len(m.ExpectedMetrics) + metricFoundCount := 0 + + 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) + metricsStruct, ok := m.ExpectedMetrics[metric.Name()] + if ok { + assert.Equal(m.T, metricsStruct.MetricValue, metric.Gauge().DataPoints().At(0).DoubleValue()) + for _, expectedLabel := range metricsStruct.MetricLabels { + labelValue, isFound := metric.Gauge().DataPoints().At(0).Attributes().Get(expectedLabel.LabelName) + assert.True(m.T, isFound) + assert.Equal(m.T, expectedLabel.LabelValue, labelValue.Str()) + } + metricFoundCount += 1 + } + } + + assert.Equal(m.T, expectedMetricsCount, metricFoundCount) + + return nil +} + +func TestSimplePrometheusEndToEnd(opts TestSimplePrometheusEndToEndOpts) { + scraper, err := NewSimplePromethuesScraper(opts.ScraperOpts, opts.ScrapeConfig) + assert.NoError(opts.T, err) + + // build up a new PR + promFactory := prometheusreceiver.NewFactory() + + targets := []*mocks.TestData{ + { + Name: "neuron", + Pages: []mocks.MockPrometheusResponse{ + {Code: 200, Data: opts.DataReturned}, + }, + }, + } + mp, cfg, err := mocks.SetupMockPrometheus(targets...) + assert.NoError(opts.T, err) + + split := strings.Split(mp.Srv.URL, "http://") + + mockedScrapeConfig := &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{}, + MetricRelabelConfigs: opts.MetricRelabelConfig, + } + + promConfig := prometheusreceiver.Config{ + PrometheusConfig: &config.Config{ + ScrapeConfigs: []*config.ScrapeConfig{mockedScrapeConfig}, + }, + } + + // replace the prom receiver + params := receiver.CreateSettings{ + TelemetrySettings: scraper.settings, + } + scraper.prometheusReceiver, err = promFactory.CreateMetricsReceiver(scraper.ctx, params, &promConfig, opts.Consumer) + assert.NoError(opts.T, err) + assert.NotNil(opts.T, mp) + defer mp.Close() + + // perform a single scrape, this will kick off the scraper process for additional scrapes + scraper.GetMetrics() + + opts.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() +} + +func GetMockedScraperOpts(consumer consumer.Metrics, mockHostInfoProvider hostInfoProvider) SimplePromethuesScraperOpts { + settings := componenttest.NewNopTelemetrySettings() + settings.Logger, _ = zap.NewDevelopment() + + return SimplePromethuesScraperOpts{ + Ctx: context.TODO(), + TelemetrySettings: settings, + Consumer: consumer, + Host: componenttest.NewNopHost(), + HostInfoProvider: mockHostInfoProvider, + } +} diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go new file mode 100644 index 000000000000..6a703c036f42 --- /dev/null +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go @@ -0,0 +1,98 @@ +package prometheusscraper + +import ( + "context" + "errors" + "fmt" + + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/prometheusreceiver" + "github.com/prometheus/prometheus/config" + "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" +) + +type SimplePromethuesScraper struct { + ctx context.Context + settings component.TelemetrySettings + host component.Host + hostInfoProvider hostInfoProvider + prometheusReceiver receiver.Metrics + running bool +} + +type SimplePromethuesScraperOpts struct { + Ctx context.Context + TelemetrySettings component.TelemetrySettings + Consumer consumer.Metrics + Host component.Host + HostInfoProvider hostInfoProvider + ScraperConfigs *config.ScrapeConfig +} + +type hostInfoProvider interface { + GetClusterName() string + GetInstanceID() string +} + +func NewSimplePromethuesScraper(opts SimplePromethuesScraperOpts, scraperConfig *config.ScrapeConfig) (*SimplePromethuesScraper, 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") + } + + promConfig := prometheusreceiver.Config{ + PrometheusConfig: &config.Config{ + ScrapeConfigs: []*config.ScrapeConfig{scraperConfig}, + }, + } + + 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 &SimplePromethuesScraper{ + ctx: opts.Ctx, + settings: opts.TelemetrySettings, + host: opts.Host, + hostInfoProvider: opts.HostInfoProvider, + prometheusReceiver: promReceiver, + }, nil +} + +func (ds *SimplePromethuesScraper) 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 *SimplePromethuesScraper) 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/prometheusscraper/simple_prometheus_scraper_test.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper_test.go new file mode 100644 index 000000000000..07e2dfa63cb2 --- /dev/null +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper_test.go @@ -0,0 +1,58 @@ +package prometheusscraper + +import ( + "context" + "testing" + + "github.com/prometheus/prometheus/config" + "github.com/stretchr/testify/assert" + "go.opentelemetry.io/collector/component/componenttest" + "go.uber.org/zap" +) + +type mockHostInfoProvider struct { +} + +func (m mockHostInfoProvider) GetClusterName() string { + return "cluster-name" +} + +func (m mockHostInfoProvider) GetInstanceID() string { + return "i-000000000" +} + +func TestSimplePrometheusScraperBadInputs(t *testing.T) { + settings := componenttest.NewNopTelemetrySettings() + settings.Logger, _ = zap.NewDevelopment() + + tests := []SimplePromethuesScraperOpts{ + { + 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 := NewSimplePromethuesScraper(tt, &config.ScrapeConfig{}) + + assert.Error(t, err) + assert.Nil(t, scraper) + } +} diff --git a/receiver/awscontainerinsightreceiver/receiver.go b/receiver/awscontainerinsightreceiver/receiver.go index d8a47ab5c37b..5320b94e67a8 100644 --- a/receiver/awscontainerinsightreceiver/receiver.go +++ b/receiver/awscontainerinsightreceiver/receiver.go @@ -9,7 +9,8 @@ import ( "time" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/gpu" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/neuron" + nueron "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/neuron" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/prometheusscraper" "go.opentelemetry.io/collector/component" "go.opentelemetry.io/collector/consumer" "go.opentelemetry.io/collector/pdata/pmetric" @@ -43,7 +44,7 @@ type awsContainerInsightReceiver struct { k8sapiserver metricsProvider prometheusScraper *k8sapiserver.PrometheusScraper dcgmScraper *gpu.DcgmScraper - neuronMonitorScraper *neuron.NeuronMonitorScraper + neuronMonitorScraper *prometheusscraper.SimplePromethuesScraper } // newAWSContainerInsightReceiver creates the aws container insight receiver with the given parameters. @@ -100,12 +101,20 @@ func (acir *awsContainerInsightReceiver) Start(ctx context.Context, host compone acir.settings.Logger.Debug("Unable to start kube apiserver prometheus scraper", zap.Error(err)) } + simplePrometheusScraperOpts := prometheusscraper.SimplePromethuesScraperOpts{ + Ctx: ctx, + TelemetrySettings: acir.settings, + Consumer: acir.nextConsumer, + Host: host, + HostInfoProvider: hostinfo, + } + err = acir.initDcgmScraper(ctx, host, hostinfo) if err != nil { acir.settings.Logger.Debug("Unable to start dcgm scraper", zap.Error(err)) } - err = acir.initNeuronScraper(ctx, host, hostinfo) + err = acir.initNeuronScraper(simplePrometheusScraperOpts) if err != nil { acir.settings.Logger.Debug("Unable to start dcgm scraper", zap.Error(err)) } @@ -199,19 +208,13 @@ func (acir *awsContainerInsightReceiver) initDcgmScraper(ctx context.Context, ho return err } -func (acir *awsContainerInsightReceiver) initNeuronScraper(ctx context.Context, host component.Host, hostinfo *hostInfo.Info) error { +func (acir *awsContainerInsightReceiver) initNeuronScraper(opts prometheusscraper.SimplePromethuesScraperOpts) error { if !acir.config.EnableNeuronMetric { return nil } var err error - acir.neuronMonitorScraper, err = neuron.NewNeuronMonitorScraper(neuron.NeuronMonitorScraperOpts{ - Ctx: ctx, - TelemetrySettings: acir.settings, - Consumer: acir.nextConsumer, - Host: host, - HostInfoProvider: hostinfo, - }) + acir.neuronMonitorScraper, err = prometheusscraper.NewSimplePromethuesScraper(opts, nueron.GetNueronScrapeConfig(opts)) return err } From c95f590080268d43c9a4d0d8f98fff7a2fa56ee2 Mon Sep 17 00:00:00 2001 From: Samarth Singh Date: Fri, 23 Feb 2024 18:12:00 +0000 Subject: [PATCH 12/53] Minor fixes --- .../awscontainerinsightreceiver/config.go | 2 +- .../neuron/neuron_monitor_scraper_test.go | 74 ++----------------- .../awscontainerinsightreceiver/receiver.go | 2 +- 3 files changed, 10 insertions(+), 68 deletions(-) diff --git a/receiver/awscontainerinsightreceiver/config.go b/receiver/awscontainerinsightreceiver/config.go index 307dbe87702d..3d3a6d370df4 100644 --- a/receiver/awscontainerinsightreceiver/config.go +++ b/receiver/awscontainerinsightreceiver/config.go @@ -61,6 +61,6 @@ type Config struct { // EnableGpuMetric toggles GPU monitoring where metrics are scraped from vendor specific sources EnableGpuMetric bool `mapstructure:"gpu_metrics"` - // EnableNeuronMetric toggles Neuron monitoring where metrics are scraped from vendor specific sources + // EnableNeuronMetric toggles Neuron monitoring where metrics are scraped from neuron monitor EnableNeuronMetric bool `mapstructure:"neuron_metrics"` } diff --git a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go index 09708c4488e0..c4d4e4b2b9ba 100644 --- a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go +++ b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go @@ -12,84 +12,18 @@ const renameMetric = ` # HELP python_gc_objects_collected_total Objects collected during gc # TYPE python_gc_objects_collected_total counter python_gc_objects_collected_total{generation="0"} 75.0 -# HELP python_gc_objects_uncollectable_total Uncollectable objects found during GC -# TYPE python_gc_objects_uncollectable_total counter -python_gc_objects_uncollectable_total{generation="0"} 0.0 -# HELP python_gc_collections_total Number of times this generation was collected -# TYPE python_gc_collections_total counter -python_gc_collections_total{generation="0"} 44.0 -# HELP python_info Python platform information -# TYPE python_info gauge -python_info{implementation="CPython",major="3",minor="8",patchlevel="10",version="3.8.10"} 1.0 -# HELP process_virtual_memory_bytes Virtual memory size in bytes. -# TYPE process_virtual_memory_bytes gauge -process_virtual_memory_bytes 1.80707328e+08 -# HELP process_resident_memory_bytes Resident memory size in bytes. -# TYPE process_resident_memory_bytes gauge -process_resident_memory_bytes 2.11968e+07 -# HELP process_start_time_seconds Start time of the process since unix epoch in seconds. -# TYPE process_start_time_seconds gauge -process_start_time_seconds 1.7083389395e+09 -# HELP process_cpu_seconds_total Total user and system CPU time spent in seconds. -# TYPE process_cpu_seconds_total counter -process_cpu_seconds_total 0.08 -# HELP process_open_fds Number of open file descriptors. -# TYPE process_open_fds gauge -process_open_fds 6.0 -# HELP execution_errors_total Execution errors total -# TYPE execution_errors_total counter -execution_errors_total{availability_zone="us-east-1c",error_type="generic",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",region="us-east-1",runtime_tag="367",subnet_id="subnet-06a7754948e8a000f"} 0.0 # HELP execution_errors_created Execution errors total # TYPE execution_errors_created gauge execution_errors_created{availability_zone="us-east-1c",error_type="generic",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",region="us-east-1",runtime_tag="367",subnet_id="subnet-06a7754948e8a000f"} 1.7083389404380567e+09 -# HELP execution_status_total Execution status total -# TYPE execution_status_total counter -execution_status_total{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",region="us-east-1",runtime_tag="367",status_type="completed",subnet_id="subnet-06a7754948e8a000f"} 0.0 -# HELP execution_status_created Execution status total -# TYPE execution_status_created gauge -execution_status_created{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",region="us-east-1",runtime_tag="367",status_type="completed",subnet_id="subnet-06a7754948e8a000f"} 1.7083389404381733e+09 # HELP neuron_runtime_memory_used_bytes Runtime memory used bytes # TYPE neuron_runtime_memory_used_bytes gauge neuron_runtime_memory_used_bytes{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",memory_location="host",region="us-east-1",runtime_tag="367",subnet_id="subnet-06a7754948e8a000f"} 9.043968e+06 -# HELP neuroncore_memory_usage_constants NeuronCore memory utilization for constants -# TYPE neuroncore_memory_usage_constants gauge -neuroncore_memory_usage_constants{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",memory_location="None",neuroncore="0",region="us-east-1",runtime_tag="367",subnet_id="subnet-06a7754948e8a000f"} 0.0 -# HELP neuroncore_memory_usage_model_code NeuronCore memory utilization for model_code -# TYPE neuroncore_memory_usage_model_code gauge -neuroncore_memory_usage_model_code{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",memory_location="None",neuroncore="0",region="us-east-1",runtime_tag="367",subnet_id="subnet-06a7754948e8a000f"} 1.00752896e+08 -# HELP neuroncore_memory_usage_model_shared_scratchpad NeuronCore memory utilization for model_shared_scratchpad -# TYPE neuroncore_memory_usage_model_shared_scratchpad gauge -neuroncore_memory_usage_model_shared_scratchpad{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",memory_location="None",neuroncore="0",region="us-east-1",runtime_tag="367",subnet_id="subnet-06a7754948e8a000f"} 0.0 -# HELP neuroncore_memory_usage_runtime_memory NeuronCore memory utilization for runtime_memory -# TYPE neuroncore_memory_usage_runtime_memory gauge -neuroncore_memory_usage_runtime_memory{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",memory_location="None",neuroncore="0",region="us-east-1",runtime_tag="367",subnet_id="subnet-06a7754948e8a000f"} 0.0 -# HELP neuroncore_memory_usage_tensors NeuronCore memory utilization for tensors -# TYPE neuroncore_memory_usage_tensors gauge -neuroncore_memory_usage_tensors{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",memory_location="None",neuroncore="0",region="us-east-1",runtime_tag="367",subnet_id="subnet-06a7754948e8a000f"} 6.315872e+06 # HELP neuroncore_utilization_ratio NeuronCore utilization ratio # TYPE neuroncore_utilization_ratio gauge neuroncore_utilization_ratio{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",neuroncore="0",region="us-east-1",runtime_tag="367",subnet_id="subnet-06a7754948e8a000f"} 0.1 # HELP system_memory_total_bytes System memory total_bytes bytes # TYPE system_memory_total_bytes gauge system_memory_total_bytes{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",region="us-east-1",subnet_id="subnet-06a7754948e8a000f"} 5.32523487232e+011 -# HELP system_memory_used_bytes System memory used_bytes bytes -# TYPE system_memory_used_bytes gauge -system_memory_used_bytes{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",region="us-east-1",subnet_id="subnet-06a7754948e8a000f"} 7.6337672192e+010 -# HELP system_swap_total_bytes System swap total_bytes bytes -# TYPE system_swap_total_bytes gauge -system_swap_total_bytes{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",region="us-east-1",subnet_id="subnet-06a7754948e8a000f"} 0.0 -# HELP system_swap_used_bytes System swap used_bytes bytes -# TYPE system_swap_used_bytes gauge -system_swap_used_bytes{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",region="us-east-1",subnet_id="subnet-06a7754948e8a000f"} 0.0 -# HELP system_vcpu_count System vCPU count -# TYPE system_vcpu_count gauge -system_vcpu_count{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",region="us-east-1",subnet_id="subnet-06a7754948e8a000f"} 128.0 -# HELP system_vcpu_usage_ratio System CPU utilization ratio -# TYPE system_vcpu_usage_ratio gauge -system_vcpu_usage_ratio{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",region="us-east-1",subnet_id="subnet-06a7754948e8a000f",usage_type="user"} 0.008199999999999999 -# HELP instance_info EC2 instance information -# TYPE instance_info gauge -instance_info{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",region="us-east-1",subnet_id="subnet-06a7754948e8a000f"} 1.0 ` const dummyClusterName = "cluster-name" @@ -132,6 +66,14 @@ func TestNewNeuronScraperEndToEnd(t *testing.T) { }, } + expectedMetrics["system_memory_total_bytes"] = prometheusscraper.ExpectedMetricStruct{ + MetricValue: 5.32523487232e+011, + MetricLabels: []prometheusscraper.MetricLabel{ + {LabelName: "InstanceId", LabelValue: "i-09db9b55e0095612f"}, + {LabelName: "ClusterName", LabelValue: dummyClusterName}, + }, + } + expectedMetrics["up"] = prometheusscraper.ExpectedMetricStruct{ MetricValue: 1.0, MetricLabels: []prometheusscraper.MetricLabel{}, diff --git a/receiver/awscontainerinsightreceiver/receiver.go b/receiver/awscontainerinsightreceiver/receiver.go index 5320b94e67a8..2c847731c486 100644 --- a/receiver/awscontainerinsightreceiver/receiver.go +++ b/receiver/awscontainerinsightreceiver/receiver.go @@ -116,7 +116,7 @@ func (acir *awsContainerInsightReceiver) Start(ctx context.Context, host compone err = acir.initNeuronScraper(simplePrometheusScraperOpts) if err != nil { - acir.settings.Logger.Debug("Unable to start dcgm scraper", zap.Error(err)) + acir.settings.Logger.Debug("Unable to start neuron scraper", zap.Error(err)) } } if acir.config.ContainerOrchestrator == ci.ECS { From 609198b9dc4650673aa94e81910b3dbd72d600df Mon Sep 17 00:00:00 2001 From: Samarth Singh Date: Fri, 23 Feb 2024 18:18:21 +0000 Subject: [PATCH 13/53] EnableFlag default to false --- receiver/awscontainerinsightreceiver/config.go | 1 + receiver/awscontainerinsightreceiver/factory.go | 4 ++++ 2 files changed, 5 insertions(+) diff --git a/receiver/awscontainerinsightreceiver/config.go b/receiver/awscontainerinsightreceiver/config.go index 3d3a6d370df4..3dfc581e0849 100644 --- a/receiver/awscontainerinsightreceiver/config.go +++ b/receiver/awscontainerinsightreceiver/config.go @@ -62,5 +62,6 @@ type Config struct { EnableGpuMetric bool `mapstructure:"gpu_metrics"` // EnableNeuronMetric toggles Neuron monitoring where metrics are scraped from neuron monitor + // The default value is false. EnableNeuronMetric bool `mapstructure:"neuron_metrics"` } diff --git a/receiver/awscontainerinsightreceiver/factory.go b/receiver/awscontainerinsightreceiver/factory.go index 6cb5179d598e..cc0e246501ab 100644 --- a/receiver/awscontainerinsightreceiver/factory.go +++ b/receiver/awscontainerinsightreceiver/factory.go @@ -42,6 +42,9 @@ const ( // Don't enable EKS control plane metrics by default defaultEnableControlPlaneMetrics = false + + // Don't enable Neuron metrics by default + defaultEnableNeuronMetrics = false ) // NewFactory creates a factory for AWS container insight receiver @@ -64,6 +67,7 @@ func createDefaultConfig() component.Config { ClusterName: defaultClusterName, LeaderLockName: defaultLeaderLockName, EnableControlPlaneMetrics: defaultEnableControlPlaneMetrics, + EnableNeuronMetric: defaultEnableNeuronMetrics, } } From 1444acd11751f69e3a8f85be787ddfbe37371d7d Mon Sep 17 00:00:00 2001 From: Hyunsoo Kim Date: Mon, 26 Feb 2024 15:54:24 -0500 Subject: [PATCH 14/53] 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 0d659fa1c7b6..13a94e8c9cd5 100644 --- a/receiver/awscontainerinsightreceiver/receiver.go +++ b/receiver/awscontainerinsightreceiver/receiver.go @@ -98,7 +98,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 d2c417d6ce7a040012c6d5be4f8974fe3a23f45b Mon Sep 17 00:00:00 2001 From: Samarth Singh Date: Fri, 1 Mar 2024 13:38:06 +0000 Subject: [PATCH 15/53] testing support --- internal/aws/containerinsight/const.go | 1 + .../dcgm_exporter_logs.txt | 70 +++++ receiver/awscontainerinsightreceiver/go.mod | 5 +- receiver/awscontainerinsightreceiver/go.sum | 2 + .../internal/prometheusscraper/decorator.go | 205 ++++++++++++++ .../simple_prometheus_scraper.go | 19 +- .../stores/kubeletutil/podresourcesclient.go | 103 +++++++ .../internal/stores/podresourcesstore.go | 184 ++++++++++++ .../internal/stores/podresourcesstore_test.go | 267 ++++++++++++++++++ .../awscontainerinsightreceiver/receiver.go | 19 +- 10 files changed, 865 insertions(+), 10 deletions(-) create mode 100644 receiver/awscontainerinsightreceiver/dcgm_exporter_logs.txt create mode 100644 receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator.go create mode 100644 receiver/awscontainerinsightreceiver/internal/stores/kubeletutil/podresourcesclient.go create mode 100644 receiver/awscontainerinsightreceiver/internal/stores/podresourcesstore.go create mode 100644 receiver/awscontainerinsightreceiver/internal/stores/podresourcesstore_test.go diff --git a/internal/aws/containerinsight/const.go b/internal/aws/containerinsight/const.go index 6fb5d762cb6f..3ef92017fb85 100644 --- a/internal/aws/containerinsight/const.go +++ b/internal/aws/containerinsight/const.go @@ -155,6 +155,7 @@ const ( TypeGpuPod = "PodGPU" TypeGpuNode = "NodeGPU" TypeGpuCluster = "ClusterGPU" + TypeNeuron = "ContainerNeuron" // unit UnitBytes = "Bytes" diff --git a/receiver/awscontainerinsightreceiver/dcgm_exporter_logs.txt b/receiver/awscontainerinsightreceiver/dcgm_exporter_logs.txt new file mode 100644 index 000000000000..14ae2ca23574 --- /dev/null +++ b/receiver/awscontainerinsightreceiver/dcgm_exporter_logs.txt @@ -0,0 +1,70 @@ + # HELP DCGM_FI_DEV_SM_CLOCK SM clock frequency (in MHz). + # TYPE DCGM_FI_DEV_SM_CLOCK gauge + DCGM_FI_DEV_SM_CLOCK{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 1455 + # HELP DCGM_FI_DEV_MEM_CLOCK Memory clock frequency (in MHz). + # TYPE DCGM_FI_DEV_MEM_CLOCK gauge + DCGM_FI_DEV_MEM_CLOCK{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 6250 + # 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="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 65 + # HELP DCGM_FI_DEV_POWER_USAGE Power draw (in W). + # TYPE DCGM_FI_DEV_POWER_USAGE gauge + DCGM_FI_DEV_POWER_USAGE{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 299.437000 + # HELP DCGM_FI_DEV_TOTAL_ENERGY_CONSUMPTION Total energy consumption since boot (in mJ). + # TYPE DCGM_FI_DEV_TOTAL_ENERGY_CONSUMPTION counter + DCGM_FI_DEV_TOTAL_ENERGY_CONSUMPTION{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 15782796862 + # HELP DCGM_FI_DEV_PCIE_REPLAY_COUNTER Total number of PCIe retries. + # TYPE DCGM_FI_DEV_PCIE_REPLAY_COUNTER counter + DCGM_FI_DEV_PCIE_REPLAY_COUNTER{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 0 + # HELP DCGM_FI_DEV_GPU_UTIL GPU utilization (in %). + # TYPE DCGM_FI_DEV_GPU_UTIL gauge + DCGM_FI_DEV_GPU_UTIL{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 100 + # HELP DCGM_FI_DEV_MEM_COPY_UTIL Memory utilization (in %). + # TYPE DCGM_FI_DEV_MEM_COPY_UTIL gauge + DCGM_FI_DEV_MEM_COPY_UTIL{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 38 + # HELP DCGM_FI_DEV_ENC_UTIL Encoder utilization (in %). + # TYPE DCGM_FI_DEV_ENC_UTIL gauge + DCGM_FI_DEV_ENC_UTIL{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 0 + # HELP DCGM_FI_DEV_DEC_UTIL Decoder utilization (in %). + # TYPE DCGM_FI_DEV_DEC_UTIL gauge + DCGM_FI_DEV_DEC_UTIL{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 0 + # HELP DCGM_FI_DEV_XID_ERRORS Value of the last XID error encountered. + # TYPE DCGM_FI_DEV_XID_ERRORS gauge + DCGM_FI_DEV_XID_ERRORS{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 0 + # HELP DCGM_FI_DEV_FB_FREE Framebuffer memory free (in MiB). + # TYPE DCGM_FI_DEV_FB_FREE gauge + DCGM_FI_DEV_FB_FREE{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 2230 + # HELP DCGM_FI_DEV_FB_USED Framebuffer memory used (in MiB). + # TYPE DCGM_FI_DEV_FB_USED gauge + DCGM_FI_DEV_FB_USED{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 20501 + # HELP DCGM_FI_DEV_NVLINK_BANDWIDTH_TOTAL Total number of NVLink bandwidth counters for all lanes. + # TYPE DCGM_FI_DEV_NVLINK_BANDWIDTH_TOTAL counter + DCGM_FI_DEV_NVLINK_BANDWIDTH_TOTAL{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 0 + # HELP DCGM_FI_DEV_VGPU_LICENSE_STATUS vGPU License status + # TYPE DCGM_FI_DEV_VGPU_LICENSE_STATUS gauge + DCGM_FI_DEV_VGPU_LICENSE_STATUS{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 0 + # HELP DCGM_FI_DEV_UNCORRECTABLE_REMAPPED_ROWS Number of remapped rows for uncorrectable errors + # TYPE DCGM_FI_DEV_UNCORRECTABLE_REMAPPED_ROWS counter + DCGM_FI_DEV_UNCORRECTABLE_REMAPPED_ROWS{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 0 + # HELP DCGM_FI_DEV_CORRECTABLE_REMAPPED_ROWS Number of remapped rows for correctable errors + # TYPE DCGM_FI_DEV_CORRECTABLE_REMAPPED_ROWS counter + DCGM_FI_DEV_CORRECTABLE_REMAPPED_ROWS{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 0 + # HELP DCGM_FI_DEV_ROW_REMAP_FAILURE Whether remapping of rows has failed + # TYPE DCGM_FI_DEV_ROW_REMAP_FAILURE gauge + DCGM_FI_DEV_ROW_REMAP_FAILURE{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 0 + # HELP DCGM_FI_PROF_GR_ENGINE_ACTIVE Ratio of time the graphics engine is active (in %). + # TYPE DCGM_FI_PROF_GR_ENGINE_ACTIVE gauge + DCGM_FI_PROF_GR_ENGINE_ACTIVE{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 0.808369 + # HELP DCGM_FI_PROF_PIPE_TENSOR_ACTIVE Ratio of cycles the tensor (HMMA) pipe is active (in %). + # TYPE DCGM_FI_PROF_PIPE_TENSOR_ACTIVE gauge + DCGM_FI_PROF_PIPE_TENSOR_ACTIVE{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 0.000000 + # HELP DCGM_FI_PROF_DRAM_ACTIVE Ratio of cycles the device memory interface is active sending or receiving data (in %). + # TYPE DCGM_FI_PROF_DRAM_ACTIVE gauge + DCGM_FI_PROF_DRAM_ACTIVE{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 0.315787 + # HELP DCGM_FI_PROF_PCIE_TX_BYTES The rate of data transmitted over the PCIe bus - including both protocol headers and data payloads - in bytes per second. + # TYPE DCGM_FI_PROF_PCIE_TX_BYTES gauge + DCGM_FI_PROF_PCIE_TX_BYTES{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 3985328 + # HELP DCGM_FI_PROF_PCIE_RX_BYTES The rate of data received over the PCIe bus - including both protocol headers and data payloads - in bytes per second. + # TYPE DCGM_FI_PROF_PCIE_RX_BYTES gauge + DCGM_FI_PROF_PCIE_RX_BYTES{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 21715174 + pod "curl" deleted \ No newline at end of file diff --git a/receiver/awscontainerinsightreceiver/go.mod b/receiver/awscontainerinsightreceiver/go.mod index 8772cf95cd71..2a1a9ecbcd86 100644 --- a/receiver/awscontainerinsightreceiver/go.mod +++ b/receiver/awscontainerinsightreceiver/go.mod @@ -26,11 +26,14 @@ require ( go.opentelemetry.io/collector/receiver v0.89.0 go.uber.org/zap v1.26.0 golang.org/x/exp v0.0.0-20231127185646-65229373498e + google.golang.org/grpc v1.59.0 gopkg.in/yaml.v2 v2.4.0 k8s.io/api v0.28.3 k8s.io/apimachinery v0.28.3 k8s.io/client-go v0.28.3 k8s.io/klog v1.0.0 + k8s.io/kubelet v0.27.3 + k8s.io/utils v0.0.0-20230711102312-30195339c3c7 ) require ( @@ -209,14 +212,12 @@ require ( google.golang.org/genproto v0.0.0-20231030173426-d783a09b4405 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20231106174013-bbf56f31fb17 // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20231030173426-d783a09b4405 // indirect - google.golang.org/grpc v1.59.0 // indirect google.golang.org/protobuf v1.31.0 // indirect gopkg.in/inf.v0 v0.9.1 // indirect gopkg.in/ini.v1 v1.67.0 // indirect gopkg.in/yaml.v3 v3.0.1 // indirect k8s.io/klog/v2 v2.100.1 // indirect k8s.io/kube-openapi v0.0.0-20230717233707-2695361300d9 // indirect - k8s.io/utils v0.0.0-20230711102312-30195339c3c7 // indirect sigs.k8s.io/json v0.0.0-20221116044647-bc3834ca7abd // indirect sigs.k8s.io/structured-merge-diff/v4 v4.3.0 // indirect sigs.k8s.io/yaml v1.3.0 // indirect diff --git a/receiver/awscontainerinsightreceiver/go.sum b/receiver/awscontainerinsightreceiver/go.sum index d586693f9062..b17281a1fdcc 100644 --- a/receiver/awscontainerinsightreceiver/go.sum +++ b/receiver/awscontainerinsightreceiver/go.sum @@ -1115,6 +1115,8 @@ k8s.io/klog/v2 v2.100.1/go.mod h1:y1WjHnz7Dj687irZUWR/WLkLc5N1YHtjLdmgWjndZn0= k8s.io/kube-openapi v0.0.0-20210305001622-591a79e4bda7/go.mod h1:wXW5VT87nVfh/iLV8FpR2uDvrFyomxbtb1KivDbvPTE= k8s.io/kube-openapi v0.0.0-20230717233707-2695361300d9 h1:LyMgNKD2P8Wn1iAwQU5OhxCKlKJy0sHc+PcDwFB24dQ= k8s.io/kube-openapi v0.0.0-20230717233707-2695361300d9/go.mod h1:wZK2AVp1uHCp4VamDVgBP2COHZjqD1T68Rf0CM3YjSM= +k8s.io/kubelet v0.27.3 h1:5WhTV1iiBu9q/rr+gvy65LQ+K/e7dmgcaYjys5ipLqY= +k8s.io/kubelet v0.27.3/go.mod h1:Mz42qgZZgWgPmOJEYaR5evmh+EoSwFzEvPBozA2y9mg= k8s.io/utils v0.0.0-20201110183641-67b214c5f920/go.mod h1:jPW/WVKK9YHAvNhRxK0md/EJ228hCsBRufyofKtW8HA= k8s.io/utils v0.0.0-20230711102312-30195339c3c7 h1:ZgnF1KZsYxWIifwSNZFZgNtWE89WI5yiP5WwlfDoIyc= k8s.io/utils v0.0.0-20230711102312-30195339c3c7/go.mod h1:OLgZIPagt7ERELqWJFomSt595RzquPNLL48iOWgYOg0= diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator.go new file mode 100644 index 000000000000..8f354e7de577 --- /dev/null +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator.go @@ -0,0 +1,205 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +package prometheusscraper + +import ( + "context" + "encoding/json" + "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 (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 prettyPrint(i interface{}) string { + s, _ := json.MarshalIndent(i, "", "\t") + return string(s) +} + +func (dc *decorateConsumer) ConsumeMetrics(ctx context.Context, md pmetric.Metrics) error { + dc.logger.Info("Metrics Object" + prettyPrint(md)) + 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.TypeNeuron) + } + 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())) + } + + dc.logger.Info("Metric name is " + m.Name()) + + 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 + } + dc.logger.Info("Atttributes are -" + tk + ":" + tv) + 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/prometheusscraper/simple_prometheus_scraper.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go index 6a703c036f42..76de885b2370 100644 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go @@ -5,6 +5,8 @@ import ( "errors" "fmt" + ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/prometheusreceiver" "github.com/prometheus/prometheus/config" "go.opentelemetry.io/collector/component" @@ -29,7 +31,8 @@ type SimplePromethuesScraperOpts struct { Consumer consumer.Metrics Host component.Host HostInfoProvider hostInfoProvider - ScraperConfigs *config.ScrapeConfig + K8sDecorator Decorator + Logger *zap.Logger } type hostInfoProvider interface { @@ -58,8 +61,15 @@ func NewSimplePromethuesScraper(opts SimplePromethuesScraperOpts, scraperConfig 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) } @@ -76,6 +86,11 @@ func NewSimplePromethuesScraper(opts SimplePromethuesScraperOpts, scraperConfig func (ds *SimplePromethuesScraper) 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 + podresourcesstore := stores.NewPodResourcesStore(ds.settings.Logger) + podresourcesstore.AddResourceName("aws.amazon.com/neuroncore") + podresourcesstore.AddResourceName("aws.amazon.com/neuron") + podresourcesstore.AddResourceName("aws.amazon.com/neurondevice") + podresourcesstore.PrintMaps() if !ds.running { ds.settings.Logger.Info("The scraper is not running, starting up the scraper") err := ds.prometheusReceiver.Start(ds.ctx, ds.host) diff --git a/receiver/awscontainerinsightreceiver/internal/stores/kubeletutil/podresourcesclient.go b/receiver/awscontainerinsightreceiver/internal/stores/kubeletutil/podresourcesclient.go new file mode 100644 index 000000000000..792cc0e3b729 --- /dev/null +++ b/receiver/awscontainerinsightreceiver/internal/stores/kubeletutil/podresourcesclient.go @@ -0,0 +1,103 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +package kubeletutil // import "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores/kubeletutil" + +import ( + "context" + "fmt" + "net" + "os" + "time" + + "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" + podresourcesapi "k8s.io/kubelet/pkg/apis/podresources/v1" +) + +const ( + socketPath = "/var/lib/kubelet/pod-resources/kubelet.sock" + connectionTimeout = 10 * time.Second +) + +type PodResourcesClient struct { + delegateClient podresourcesapi.PodResourcesListerClient + conn *grpc.ClientConn + logger *zap.Logger +} + +func NewPodResourcesClient(logger *zap.Logger) (*PodResourcesClient, error) { + podResourcesClient := &PodResourcesClient{} + podResourcesClient.logger = logger + + conn, err := podResourcesClient.connectToServer(socketPath) + podResourcesClient.conn = conn + + logger.Info("PodResources conn state: " + conn.GetState().String()) + if err != nil { + return nil, fmt.Errorf("failed to connect to server: %w", err) + } + + podResourcesClient.delegateClient = podresourcesapi.NewPodResourcesListerClient(conn) + logger.Info("PodResources delegate: ", zap.Any("delegate client", podResourcesClient.delegateClient)) + + return podResourcesClient, nil +} + +func (p *PodResourcesClient) connectToServer(socket string) (*grpc.ClientConn, error) { + _, err := os.Stat(socket) + if err != nil { + p.logger.Info("PodResources socket error: " + err.Error()) + } + + if os.IsNotExist(err) { + return nil, fmt.Errorf("socket path does not exist: %s", socket) + } else if err != nil { + return nil, fmt.Errorf("failed to check socket path: %w", err) + } + + ctx, _ := context.WithTimeout(context.Background(), connectionTimeout) + + conn, err := grpc.DialContext(ctx, + socket, + grpc.WithTransportCredentials(insecure.NewCredentials()), + grpc.WithBlock(), + grpc.WithContextDialer(func(ctx context.Context, addr string) (net.Conn, error) { + d := net.Dialer{} + return d.DialContext(ctx, "unix", addr) + }), + ) + if err != nil { + p.logger.Info("PodResources connection error: " + err.Error()) + } + if err != nil { + return nil, fmt.Errorf("failure connecting to '%s': %w", socket, err) + } + + return conn, nil +} + +func (p *PodResourcesClient) ListPods() (*podresourcesapi.ListPodResourcesResponse, error) { + ctx, _ := context.WithTimeout(context.Background(), connectionTimeout) + + resp, err := p.delegateClient.List(ctx, &podresourcesapi.ListPodResourcesRequest{}) + if err != nil { + p.logger.Info("PodResources ListPods error: " + err.Error()) + } + if err != nil { + return nil, fmt.Errorf("failure getting pod resources: %w", err) + } + + return resp, nil +} + +func (p *PodResourcesClient) Shutdown() { + err := p.conn.Close() + if err != nil { + p.logger.Info("PodResources shutdown error: " + err.Error()) + } + if err != nil { + return + } +} diff --git a/receiver/awscontainerinsightreceiver/internal/stores/podresourcesstore.go b/receiver/awscontainerinsightreceiver/internal/stores/podresourcesstore.go new file mode 100644 index 000000000000..cc9b6116856b --- /dev/null +++ b/receiver/awscontainerinsightreceiver/internal/stores/podresourcesstore.go @@ -0,0 +1,184 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +package stores // import "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" + +import ( + "context" + "fmt" + "sync" + "time" + + "go.uber.org/zap" + v1 "k8s.io/kubelet/pkg/apis/podresources/v1" + + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores/kubeletutil" +) + +const ( + taskTimeout = 10 * time.Second +) + +var ( + instance *PodResourcesStore + once sync.Once +) + +type ContainerInfo struct { + podName string + containerName string + namespace string +} + +type ResourceInfo struct { + resourceName string + deviceID string +} + +type PodResourcesClientInterface interface { + ListPods() (*v1.ListPodResourcesResponse, error) + Shutdown() +} + +type PodResourcesStore struct { + containerInfoToResourcesMap map[ContainerInfo][]ResourceInfo + resourceToPodContainerMap map[ResourceInfo]ContainerInfo + resourceNameSet map[string]struct{} + lastRefreshed time.Time + ctx context.Context + cancel context.CancelFunc + logger *zap.Logger + podResourcesClient PodResourcesClientInterface +} + +func NewPodResourcesStore(logger *zap.Logger) *PodResourcesStore { + once.Do(func() { + podResourcesClient, _ := kubeletutil.NewPodResourcesClient(logger) + ctx, cancel := context.WithCancel(context.Background()) + instance = &PodResourcesStore{ + containerInfoToResourcesMap: make(map[ContainerInfo][]ResourceInfo), + resourceToPodContainerMap: make(map[ResourceInfo]ContainerInfo), + resourceNameSet: make(map[string]struct{}), + lastRefreshed: time.Now(), + ctx: ctx, + cancel: cancel, + logger: logger, + podResourcesClient: podResourcesClient, + } + + go func() { + refreshTicker := time.NewTicker(time.Second) + for { + select { + case <-refreshTicker.C: + instance.refreshTick() + case <-instance.ctx.Done(): + refreshTicker.Stop() + return + } + } + }() + }) + return instance +} + +func (p *PodResourcesStore) refreshTick() { + now := time.Now() + if now.Sub(p.lastRefreshed) >= taskTimeout { + p.refresh() + p.lastRefreshed = now + } +} + +func (p *PodResourcesStore) refresh() { + doRefresh := func() { + p.updateMaps() + } + + refreshWithTimeout(p.ctx, doRefresh, taskTimeout) +} + +func (p *PodResourcesStore) updateMaps() { + p.containerInfoToResourcesMap = make(map[ContainerInfo][]ResourceInfo) + p.resourceToPodContainerMap = make(map[ResourceInfo]ContainerInfo) + + if len(p.resourceNameSet) == 0 { + p.logger.Warn("No resource names allowlisted thus skipping updating of maps.") + return + } + + devicePods, err := p.podResourcesClient.ListPods() + if err != nil { + p.logger.Info("PodResources ListPods calling error: " + err.Error()) + } + if err != nil { + p.logger.Error(fmt.Sprintf("Error getting pod resources: %v", err)) + return + } + + for _, pod := range devicePods.GetPodResources() { + for _, container := range pod.GetContainers() { + for _, device := range container.GetDevices() { + + containerInfo := ContainerInfo{ + podName: pod.GetName(), + namespace: pod.GetNamespace(), + containerName: container.GetName(), + } + + for _, deviceID := range device.GetDeviceIds() { + resourceInfo := ResourceInfo{ + resourceName: device.GetResourceName(), + deviceID: deviceID, + } + _, found := p.resourceNameSet[resourceInfo.resourceName] + if found { + p.containerInfoToResourcesMap[containerInfo] = append(p.containerInfoToResourcesMap[containerInfo], resourceInfo) + p.resourceToPodContainerMap[resourceInfo] = containerInfo + } + } + } + } + } +} + +func (p *PodResourcesStore) GetContainerInfo(deviceID string, resourceName string) *ContainerInfo { + key := ResourceInfo{deviceID: deviceID, resourceName: resourceName} + if containerInfo, ok := p.resourceToPodContainerMap[key]; ok { + return &containerInfo + } + return nil +} + +func (p *PodResourcesStore) GetResourcesInfo(podName string, containerName string, namespace string) *[]ResourceInfo { + key := ContainerInfo{podName: podName, containerName: containerName, namespace: namespace} + if resourceInfo, ok := p.containerInfoToResourcesMap[key]; ok { + return &resourceInfo + } + return nil +} + +func (p *PodResourcesStore) AddResourceName(resourceName string) { + p.resourceNameSet[resourceName] = struct{}{} +} + +func (p *PodResourcesStore) PrintMaps() { + p.logger.Info("containerInfoToResourcesMap:") + for containerInfo, resourceInfos := range p.containerInfoToResourcesMap { + p.logger.Info("ContainerInfo-" + containerInfo.containerName + " ; " + containerInfo.podName) + p.logger.Info("ResourceInfos:") + for _, resourceInfo := range resourceInfos { + p.logger.Info("ResourceInfo-" + resourceInfo.resourceName + " ; " + resourceInfo.deviceID) + } + } + p.logger.Info("\nresourceToPodContainerMap:") + for resourceInfo, containerInfo := range p.resourceToPodContainerMap { + p.logger.Info("ResourceInfo-" + resourceInfo.resourceName + " ; " + resourceInfo.deviceID) + p.logger.Info("ContainerInfo-" + containerInfo.containerName + " ; " + containerInfo.podName) + } +} + +func (p *PodResourcesStore) Shutdown() { + p.cancel() + p.podResourcesClient.Shutdown() +} diff --git a/receiver/awscontainerinsightreceiver/internal/stores/podresourcesstore_test.go b/receiver/awscontainerinsightreceiver/internal/stores/podresourcesstore_test.go new file mode 100644 index 000000000000..9ecbb1cf4cee --- /dev/null +++ b/receiver/awscontainerinsightreceiver/internal/stores/podresourcesstore_test.go @@ -0,0 +1,267 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +package stores // import "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" + +import ( + "context" + "fmt" + "testing" + "time" + + "github.com/stretchr/testify/assert" + "go.uber.org/zap" + podresourcesv1 "k8s.io/kubelet/pkg/apis/podresources/v1" +) + +const ( + defaultResourceName = "Resource-1" + defaultPodName = "Pod-1" + defaultNamespace = "Namespace-1" + defaultContainerName = "Container-1" + defaultDeviceID1 = "Device-1" + defaultDeviceID2 = "Device-2" + defaultDeviceID3 = "Device-3" + defaultDeviceID4 = "Device-4" + defaultResourceNameSkipped = "Resource-Skipped" + defaultContainerNameNoDevice = "Container-NoDevice" + defaultNamespaceNoDevice = "Namespace-NoDevice" + defaultPodNameNoDevice = "Pod-NoDevice" +) + +var ( + expectedContainerInfoToResourcesMap = map[ContainerInfo][]ResourceInfo{ + { + podName: defaultPodName, + containerName: defaultContainerName, + namespace: defaultNamespace, + }: { + { + resourceName: defaultResourceName, + deviceID: defaultDeviceID1, + }, + { + resourceName: defaultResourceName, + deviceID: defaultDeviceID2, + }, + }, + } + + expectedResourceToPodContainerMap = map[ResourceInfo]ContainerInfo{ + { + resourceName: defaultResourceName, + deviceID: defaultDeviceID1, + }: { + podName: defaultPodName, + containerName: defaultContainerName, + namespace: defaultNamespace, + }, + { + resourceName: defaultResourceName, + deviceID: defaultDeviceID2, + }: { + podName: defaultPodName, + containerName: defaultContainerName, + namespace: defaultNamespace, + }, + } + + expectedContainerInfo = ContainerInfo{ + podName: defaultPodName, + containerName: defaultContainerName, + namespace: defaultNamespace, + } + + expectedResourceInfo = []ResourceInfo{ + { + resourceName: defaultResourceName, + deviceID: defaultDeviceID1, + }, + { + resourceName: defaultResourceName, + deviceID: defaultDeviceID2, + }, + } + + listPodResourcesResponse = &podresourcesv1.ListPodResourcesResponse{ + PodResources: []*podresourcesv1.PodResources{ + { + Name: defaultPodName, + Namespace: defaultNamespace, + Containers: []*podresourcesv1.ContainerResources{ + { + Name: defaultContainerName, + Devices: []*podresourcesv1.ContainerDevices{ + { + ResourceName: defaultResourceName, + DeviceIds: []string{defaultDeviceID1, defaultDeviceID2}, + }, + { + ResourceName: defaultResourceNameSkipped, + DeviceIds: []string{defaultDeviceID3, defaultDeviceID4}, + }, + }, + }, + }, + }, + { + Name: defaultPodNameNoDevice, + Namespace: defaultNamespaceNoDevice, + Containers: []*podresourcesv1.ContainerResources{ + { + Name: defaultContainerNameNoDevice, + Devices: []*podresourcesv1.ContainerDevices{}, + }, + }, + }, + }, + } + + listPodResourcesResponseWithEmptyPodResources = &podresourcesv1.ListPodResourcesResponse{ + PodResources: []*podresourcesv1.PodResources{}, + } + + listPodResourcesResponseWithEmptyResponse = &podresourcesv1.ListPodResourcesResponse{} + + resourceNameSet = map[string]struct{}{ + defaultResourceName: {}, + } +) + +type MockPodResourcesClient struct { + response *podresourcesv1.ListPodResourcesResponse + err error + shutdownCalled bool +} + +func (m *MockPodResourcesClient) ListPods() (*podresourcesv1.ListPodResourcesResponse, error) { + return m.response, m.err +} + +func (m *MockPodResourcesClient) Shutdown() { + m.shutdownCalled = true +} + +func TestNewPodResourcesStore(t *testing.T) { + logger := zap.NewNop() + store := NewPodResourcesStore(logger) + assert.NotNil(t, store, "PodResourcesStore should not be nil") + assert.NotNil(t, store.ctx, "Context should not be nil") + assert.NotNil(t, store.cancel, "Cancel function should not be nil") +} + +func TestRefreshTick(t *testing.T) { + store := constructPodResourcesStore(make(map[ContainerInfo][]ResourceInfo), make(map[ResourceInfo]ContainerInfo), listPodResourcesResponse, nil) + + store.lastRefreshed = time.Now().Add(-time.Hour) + + store.refreshTick() + + assert.True(t, store.lastRefreshed.After(time.Now().Add(-time.Hour)), "lastRefreshed should have been updated") +} + +func TestShutdown(t *testing.T) { + store := constructPodResourcesStore(make(map[ContainerInfo][]ResourceInfo), make(map[ResourceInfo]ContainerInfo), listPodResourcesResponse, nil) + + mockClient := &MockPodResourcesClient{listPodResourcesResponse, nil, false} + store.podResourcesClient = mockClient + + store.Shutdown() + + assert.True(t, mockClient.shutdownCalled, "Shutdown method of the client should have been called") +} + +func TestUpdateMaps(t *testing.T) { + store := constructPodResourcesStore(make(map[ContainerInfo][]ResourceInfo), make(map[ResourceInfo]ContainerInfo), listPodResourcesResponse, nil) + store.updateMaps() + + assert.NotNil(t, store.containerInfoToResourcesMap) + assert.NotNil(t, store.resourceToPodContainerMap) + assert.Equal(t, len(expectedContainerInfoToResourcesMap), len(store.containerInfoToResourcesMap)) + assert.Equal(t, len(expectedResourceToPodContainerMap), len(store.resourceToPodContainerMap)) + assert.Equal(t, expectedContainerInfoToResourcesMap, store.containerInfoToResourcesMap) + assert.Equal(t, expectedResourceToPodContainerMap, store.resourceToPodContainerMap) +} + +func TestGets(t *testing.T) { + store := constructPodResourcesStore(make(map[ContainerInfo][]ResourceInfo), make(map[ResourceInfo]ContainerInfo), listPodResourcesResponse, nil) + store.updateMaps() + + assertMapsContainData(t, store) +} + +func TestGetsWhenThereAreNoPods(t *testing.T) { + store := constructPodResourcesStore(make(map[ContainerInfo][]ResourceInfo), make(map[ResourceInfo]ContainerInfo), listPodResourcesResponseWithEmptyPodResources, nil) + store.updateMaps() + + assertMapsDontContainData(t, store) +} + +func TestGetsWhenPodResourcesResponseIsEmpty(t *testing.T) { + store := constructPodResourcesStore(make(map[ContainerInfo][]ResourceInfo), make(map[ResourceInfo]ContainerInfo), listPodResourcesResponseWithEmptyResponse, nil) + store.updateMaps() + + assertMapsDontContainData(t, store) +} + +func TestGetsWhenPodResourcesThrowsError(t *testing.T) { + store := constructPodResourcesStore(make(map[ContainerInfo][]ResourceInfo), make(map[ResourceInfo]ContainerInfo), listPodResourcesResponseWithEmptyResponse, fmt.Errorf("mocked behavior")) + store.updateMaps() + + assertMapsDontContainData(t, store) +} + +func TestAddResourceName(t *testing.T) { + store := constructPodResourcesStore(make(map[ContainerInfo][]ResourceInfo), make(map[ResourceInfo]ContainerInfo), listPodResourcesResponse, nil) + + store.resourceNameSet = make(map[string]struct{}) + store.updateMaps() + assertMapsDontContainData(t, store) + + // After adding resource to map + store.AddResourceName(defaultResourceName) + store.updateMaps() + assertMapsContainData(t, store) +} + +func constructPodResourcesStore(containerToDevices map[ContainerInfo][]ResourceInfo, deviceToContainer map[ResourceInfo]ContainerInfo, podResourcesResponse *podresourcesv1.ListPodResourcesResponse, podResourcesError error) *PodResourcesStore { + logger, _ := zap.NewDevelopment() + return &PodResourcesStore{ + containerInfoToResourcesMap: containerToDevices, + resourceToPodContainerMap: deviceToContainer, + resourceNameSet: resourceNameSet, + lastRefreshed: time.Now(), + ctx: context.Background(), + cancel: func() {}, + logger: logger, + podResourcesClient: &MockPodResourcesClient{podResourcesResponse, podResourcesError, false}, + } +} + +func assertMapsContainData(t *testing.T, store *PodResourcesStore) { + assert.Equal(t, len(expectedContainerInfoToResourcesMap), len(store.containerInfoToResourcesMap)) + assert.Equal(t, len(expectedResourceToPodContainerMap), len(store.resourceToPodContainerMap)) + + assert.Equal(t, expectedContainerInfo, *store.GetContainerInfo(defaultDeviceID1, defaultResourceName)) + assert.Equal(t, expectedResourceInfo, *store.GetResourcesInfo(defaultPodName, defaultContainerName, defaultNamespace)) + + actualResourceInfo := store.GetResourcesInfo(defaultPodNameNoDevice, defaultContainerNameNoDevice, defaultNamespaceNoDevice) + if actualResourceInfo != nil { + t.Errorf("Expected GetResourcesInfo to return nil for an unexpected key, but got %v", actualResourceInfo) + } +} + +func assertMapsDontContainData(t *testing.T, store *PodResourcesStore) { + assert.Equal(t, 0, len(store.containerInfoToResourcesMap)) + assert.Equal(t, 0, len(store.resourceToPodContainerMap)) + + actualContainerInfo := store.GetContainerInfo(defaultDeviceID1, defaultResourceName) + if actualContainerInfo != nil { + t.Errorf("Expected GetContainerInfo to return nil for an unexpected key, but got %v", actualContainerInfo) + } + + actualResourceInfo := store.GetResourcesInfo(defaultPodName, defaultContainerName, defaultNamespace) + if actualResourceInfo != nil { + t.Errorf("Expected GetResourcesInfo to return nil for an unexpected key, but got %v", actualResourceInfo) + } +} diff --git a/receiver/awscontainerinsightreceiver/receiver.go b/receiver/awscontainerinsightreceiver/receiver.go index 81e7ab618090..f676a76d0924 100644 --- a/receiver/awscontainerinsightreceiver/receiver.go +++ b/receiver/awscontainerinsightreceiver/receiver.go @@ -107,6 +107,8 @@ func (acir *awsContainerInsightReceiver) Start(ctx context.Context, host compone Consumer: acir.nextConsumer, Host: host, HostInfoProvider: hostinfo, + K8sDecorator: k8sDecorator, + Logger: acir.settings.Logger, } err = acir.initDcgmScraper(ctx, host, hostinfo, k8sDecorator) @@ -211,9 +213,9 @@ func (acir *awsContainerInsightReceiver) initDcgmScraper(ctx context.Context, ho } func (acir *awsContainerInsightReceiver) initNeuronScraper(opts prometheusscraper.SimplePromethuesScraperOpts) error { - if !acir.config.EnableNeuronMetric { - return nil - } + // if !acir.config.EnableNeuronMetric { + // return nil + // } var err error acir.neuronMonitorScraper, err = prometheusscraper.NewSimplePromethuesScraper(opts, nueron.GetNueronScrapeConfig(opts)) @@ -270,14 +272,19 @@ func (acir *awsContainerInsightReceiver) collectData(ctx context.Context) error acir.prometheusScraper.GetMetrics() //nolint:errcheck } - if acir.dcgmScraper != nil { - acir.dcgmScraper.GetMetrics() //nolint:errcheck - } + // if acir.dcgmScraper != nil { + // acir.dcgmScraper.GetMetrics() //nolint:errcheck + // } + + acir.settings.Logger.Info("We will start the Neuron Scraper") if acir.neuronMonitorScraper != nil { + acir.settings.Logger.Info("Neuron Scraper is not NIL") acir.neuronMonitorScraper.GetMetrics() //nolint:errcheck } + acir.settings.Logger.Info("If this happened Neuron is started or not") + for _, md := range mds { err := acir.nextConsumer.ConsumeMetrics(ctx, md) if err != nil { From 00a12dc975e477c12fbeb88da799effb65b22980 Mon Sep 17 00:00:00 2001 From: Aditya Purang Date: Fri, 1 Mar 2024 14:41:45 +0000 Subject: [PATCH 16/53] debugging pod resources store --- .../simple_prometheus_scraper.go | 6 +- .../internal/stores/podresourcesstore.go | 25 +- .../internal/stores/podresourcesstore_test.go | 526 +++++++++--------- 3 files changed, 281 insertions(+), 276 deletions(-) diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go index 76de885b2370..500b73c006dc 100644 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go @@ -87,10 +87,8 @@ func (ds *SimplePromethuesScraper) 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 podresourcesstore := stores.NewPodResourcesStore(ds.settings.Logger) - podresourcesstore.AddResourceName("aws.amazon.com/neuroncore") - podresourcesstore.AddResourceName("aws.amazon.com/neuron") - podresourcesstore.AddResourceName("aws.amazon.com/neurondevice") - podresourcesstore.PrintMaps() + podresourcesstore.GetResourcesInfo("123", "123", "123") + if !ds.running { ds.settings.Logger.Info("The scraper is not running, starting up the scraper") err := ds.prometheusReceiver.Start(ds.ctx, ds.host) diff --git a/receiver/awscontainerinsightreceiver/internal/stores/podresourcesstore.go b/receiver/awscontainerinsightreceiver/internal/stores/podresourcesstore.go index cc9b6116856b..4f2408140dfc 100644 --- a/receiver/awscontainerinsightreceiver/internal/stores/podresourcesstore.go +++ b/receiver/awscontainerinsightreceiver/internal/stores/podresourcesstore.go @@ -66,6 +66,10 @@ func NewPodResourcesStore(logger *zap.Logger) *PodResourcesStore { podResourcesClient: podResourcesClient, } + instance.AddResourceName("aws.amazon.com/neuroncore") + instance.AddResourceName("aws.amazon.com/neuron") + instance.AddResourceName("aws.amazon.com/neurondevice") + go func() { refreshTicker := time.NewTicker(time.Second) for { @@ -102,10 +106,10 @@ func (p *PodResourcesStore) updateMaps() { p.containerInfoToResourcesMap = make(map[ContainerInfo][]ResourceInfo) p.resourceToPodContainerMap = make(map[ResourceInfo]ContainerInfo) - if len(p.resourceNameSet) == 0 { - p.logger.Warn("No resource names allowlisted thus skipping updating of maps.") - return - } + //if len(p.resourceNameSet) == 0 { + // p.logger.Warn("No resource names allowlisted thus skipping updating of maps.") + // return + //} devicePods, err := p.podResourcesClient.ListPods() if err != nil { @@ -116,6 +120,7 @@ func (p *PodResourcesStore) updateMaps() { return } + p.logger.Info("PodResources updating device info with result : " + devicePods.String()) for _, pod := range devicePods.GetPodResources() { for _, container := range pod.GetContainers() { for _, device := range container.GetDevices() { @@ -131,11 +136,13 @@ func (p *PodResourcesStore) updateMaps() { resourceName: device.GetResourceName(), deviceID: deviceID, } - _, found := p.resourceNameSet[resourceInfo.resourceName] - if found { - p.containerInfoToResourcesMap[containerInfo] = append(p.containerInfoToResourcesMap[containerInfo], resourceInfo) - p.resourceToPodContainerMap[resourceInfo] = containerInfo - } + //_, found := p.resourceNameSet[resourceInfo.resourceName] + //if found { + p.containerInfoToResourcesMap[containerInfo] = append(p.containerInfoToResourcesMap[containerInfo], resourceInfo) + p.resourceToPodContainerMap[resourceInfo] = containerInfo + + p.logger.Info("/nContainerInfo : {" + containerInfo.namespace + "_" + containerInfo.podName + "_" + containerInfo.containerName + "}" + " -> ResourceInfo : {" + resourceInfo.resourceName + "_" + resourceInfo.deviceID + "_" + "}") + //} } } } diff --git a/receiver/awscontainerinsightreceiver/internal/stores/podresourcesstore_test.go b/receiver/awscontainerinsightreceiver/internal/stores/podresourcesstore_test.go index 9ecbb1cf4cee..6025ed033bac 100644 --- a/receiver/awscontainerinsightreceiver/internal/stores/podresourcesstore_test.go +++ b/receiver/awscontainerinsightreceiver/internal/stores/podresourcesstore_test.go @@ -2,266 +2,266 @@ // SPDX-License-Identifier: Apache-2.0 package stores // import "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" - -import ( - "context" - "fmt" - "testing" - "time" - - "github.com/stretchr/testify/assert" - "go.uber.org/zap" - podresourcesv1 "k8s.io/kubelet/pkg/apis/podresources/v1" -) - -const ( - defaultResourceName = "Resource-1" - defaultPodName = "Pod-1" - defaultNamespace = "Namespace-1" - defaultContainerName = "Container-1" - defaultDeviceID1 = "Device-1" - defaultDeviceID2 = "Device-2" - defaultDeviceID3 = "Device-3" - defaultDeviceID4 = "Device-4" - defaultResourceNameSkipped = "Resource-Skipped" - defaultContainerNameNoDevice = "Container-NoDevice" - defaultNamespaceNoDevice = "Namespace-NoDevice" - defaultPodNameNoDevice = "Pod-NoDevice" -) - -var ( - expectedContainerInfoToResourcesMap = map[ContainerInfo][]ResourceInfo{ - { - podName: defaultPodName, - containerName: defaultContainerName, - namespace: defaultNamespace, - }: { - { - resourceName: defaultResourceName, - deviceID: defaultDeviceID1, - }, - { - resourceName: defaultResourceName, - deviceID: defaultDeviceID2, - }, - }, - } - - expectedResourceToPodContainerMap = map[ResourceInfo]ContainerInfo{ - { - resourceName: defaultResourceName, - deviceID: defaultDeviceID1, - }: { - podName: defaultPodName, - containerName: defaultContainerName, - namespace: defaultNamespace, - }, - { - resourceName: defaultResourceName, - deviceID: defaultDeviceID2, - }: { - podName: defaultPodName, - containerName: defaultContainerName, - namespace: defaultNamespace, - }, - } - - expectedContainerInfo = ContainerInfo{ - podName: defaultPodName, - containerName: defaultContainerName, - namespace: defaultNamespace, - } - - expectedResourceInfo = []ResourceInfo{ - { - resourceName: defaultResourceName, - deviceID: defaultDeviceID1, - }, - { - resourceName: defaultResourceName, - deviceID: defaultDeviceID2, - }, - } - - listPodResourcesResponse = &podresourcesv1.ListPodResourcesResponse{ - PodResources: []*podresourcesv1.PodResources{ - { - Name: defaultPodName, - Namespace: defaultNamespace, - Containers: []*podresourcesv1.ContainerResources{ - { - Name: defaultContainerName, - Devices: []*podresourcesv1.ContainerDevices{ - { - ResourceName: defaultResourceName, - DeviceIds: []string{defaultDeviceID1, defaultDeviceID2}, - }, - { - ResourceName: defaultResourceNameSkipped, - DeviceIds: []string{defaultDeviceID3, defaultDeviceID4}, - }, - }, - }, - }, - }, - { - Name: defaultPodNameNoDevice, - Namespace: defaultNamespaceNoDevice, - Containers: []*podresourcesv1.ContainerResources{ - { - Name: defaultContainerNameNoDevice, - Devices: []*podresourcesv1.ContainerDevices{}, - }, - }, - }, - }, - } - - listPodResourcesResponseWithEmptyPodResources = &podresourcesv1.ListPodResourcesResponse{ - PodResources: []*podresourcesv1.PodResources{}, - } - - listPodResourcesResponseWithEmptyResponse = &podresourcesv1.ListPodResourcesResponse{} - - resourceNameSet = map[string]struct{}{ - defaultResourceName: {}, - } -) - -type MockPodResourcesClient struct { - response *podresourcesv1.ListPodResourcesResponse - err error - shutdownCalled bool -} - -func (m *MockPodResourcesClient) ListPods() (*podresourcesv1.ListPodResourcesResponse, error) { - return m.response, m.err -} - -func (m *MockPodResourcesClient) Shutdown() { - m.shutdownCalled = true -} - -func TestNewPodResourcesStore(t *testing.T) { - logger := zap.NewNop() - store := NewPodResourcesStore(logger) - assert.NotNil(t, store, "PodResourcesStore should not be nil") - assert.NotNil(t, store.ctx, "Context should not be nil") - assert.NotNil(t, store.cancel, "Cancel function should not be nil") -} - -func TestRefreshTick(t *testing.T) { - store := constructPodResourcesStore(make(map[ContainerInfo][]ResourceInfo), make(map[ResourceInfo]ContainerInfo), listPodResourcesResponse, nil) - - store.lastRefreshed = time.Now().Add(-time.Hour) - - store.refreshTick() - - assert.True(t, store.lastRefreshed.After(time.Now().Add(-time.Hour)), "lastRefreshed should have been updated") -} - -func TestShutdown(t *testing.T) { - store := constructPodResourcesStore(make(map[ContainerInfo][]ResourceInfo), make(map[ResourceInfo]ContainerInfo), listPodResourcesResponse, nil) - - mockClient := &MockPodResourcesClient{listPodResourcesResponse, nil, false} - store.podResourcesClient = mockClient - - store.Shutdown() - - assert.True(t, mockClient.shutdownCalled, "Shutdown method of the client should have been called") -} - -func TestUpdateMaps(t *testing.T) { - store := constructPodResourcesStore(make(map[ContainerInfo][]ResourceInfo), make(map[ResourceInfo]ContainerInfo), listPodResourcesResponse, nil) - store.updateMaps() - - assert.NotNil(t, store.containerInfoToResourcesMap) - assert.NotNil(t, store.resourceToPodContainerMap) - assert.Equal(t, len(expectedContainerInfoToResourcesMap), len(store.containerInfoToResourcesMap)) - assert.Equal(t, len(expectedResourceToPodContainerMap), len(store.resourceToPodContainerMap)) - assert.Equal(t, expectedContainerInfoToResourcesMap, store.containerInfoToResourcesMap) - assert.Equal(t, expectedResourceToPodContainerMap, store.resourceToPodContainerMap) -} - -func TestGets(t *testing.T) { - store := constructPodResourcesStore(make(map[ContainerInfo][]ResourceInfo), make(map[ResourceInfo]ContainerInfo), listPodResourcesResponse, nil) - store.updateMaps() - - assertMapsContainData(t, store) -} - -func TestGetsWhenThereAreNoPods(t *testing.T) { - store := constructPodResourcesStore(make(map[ContainerInfo][]ResourceInfo), make(map[ResourceInfo]ContainerInfo), listPodResourcesResponseWithEmptyPodResources, nil) - store.updateMaps() - - assertMapsDontContainData(t, store) -} - -func TestGetsWhenPodResourcesResponseIsEmpty(t *testing.T) { - store := constructPodResourcesStore(make(map[ContainerInfo][]ResourceInfo), make(map[ResourceInfo]ContainerInfo), listPodResourcesResponseWithEmptyResponse, nil) - store.updateMaps() - - assertMapsDontContainData(t, store) -} - -func TestGetsWhenPodResourcesThrowsError(t *testing.T) { - store := constructPodResourcesStore(make(map[ContainerInfo][]ResourceInfo), make(map[ResourceInfo]ContainerInfo), listPodResourcesResponseWithEmptyResponse, fmt.Errorf("mocked behavior")) - store.updateMaps() - - assertMapsDontContainData(t, store) -} - -func TestAddResourceName(t *testing.T) { - store := constructPodResourcesStore(make(map[ContainerInfo][]ResourceInfo), make(map[ResourceInfo]ContainerInfo), listPodResourcesResponse, nil) - - store.resourceNameSet = make(map[string]struct{}) - store.updateMaps() - assertMapsDontContainData(t, store) - - // After adding resource to map - store.AddResourceName(defaultResourceName) - store.updateMaps() - assertMapsContainData(t, store) -} - -func constructPodResourcesStore(containerToDevices map[ContainerInfo][]ResourceInfo, deviceToContainer map[ResourceInfo]ContainerInfo, podResourcesResponse *podresourcesv1.ListPodResourcesResponse, podResourcesError error) *PodResourcesStore { - logger, _ := zap.NewDevelopment() - return &PodResourcesStore{ - containerInfoToResourcesMap: containerToDevices, - resourceToPodContainerMap: deviceToContainer, - resourceNameSet: resourceNameSet, - lastRefreshed: time.Now(), - ctx: context.Background(), - cancel: func() {}, - logger: logger, - podResourcesClient: &MockPodResourcesClient{podResourcesResponse, podResourcesError, false}, - } -} - -func assertMapsContainData(t *testing.T, store *PodResourcesStore) { - assert.Equal(t, len(expectedContainerInfoToResourcesMap), len(store.containerInfoToResourcesMap)) - assert.Equal(t, len(expectedResourceToPodContainerMap), len(store.resourceToPodContainerMap)) - - assert.Equal(t, expectedContainerInfo, *store.GetContainerInfo(defaultDeviceID1, defaultResourceName)) - assert.Equal(t, expectedResourceInfo, *store.GetResourcesInfo(defaultPodName, defaultContainerName, defaultNamespace)) - - actualResourceInfo := store.GetResourcesInfo(defaultPodNameNoDevice, defaultContainerNameNoDevice, defaultNamespaceNoDevice) - if actualResourceInfo != nil { - t.Errorf("Expected GetResourcesInfo to return nil for an unexpected key, but got %v", actualResourceInfo) - } -} - -func assertMapsDontContainData(t *testing.T, store *PodResourcesStore) { - assert.Equal(t, 0, len(store.containerInfoToResourcesMap)) - assert.Equal(t, 0, len(store.resourceToPodContainerMap)) - - actualContainerInfo := store.GetContainerInfo(defaultDeviceID1, defaultResourceName) - if actualContainerInfo != nil { - t.Errorf("Expected GetContainerInfo to return nil for an unexpected key, but got %v", actualContainerInfo) - } - - actualResourceInfo := store.GetResourcesInfo(defaultPodName, defaultContainerName, defaultNamespace) - if actualResourceInfo != nil { - t.Errorf("Expected GetResourcesInfo to return nil for an unexpected key, but got %v", actualResourceInfo) - } -} +// +//import ( +// "context" +// "fmt" +// "testing" +// "time" +// +// "github.com/stretchr/testify/assert" +// "go.uber.org/zap" +// podresourcesv1 "k8s.io/kubelet/pkg/apis/podresources/v1" +//) +// +//const ( +// defaultResourceName = "Resource-1" +// defaultPodName = "Pod-1" +// defaultNamespace = "Namespace-1" +// defaultContainerName = "Container-1" +// defaultDeviceID1 = "Device-1" +// defaultDeviceID2 = "Device-2" +// defaultDeviceID3 = "Device-3" +// defaultDeviceID4 = "Device-4" +// defaultResourceNameSkipped = "Resource-Skipped" +// defaultContainerNameNoDevice = "Container-NoDevice" +// defaultNamespaceNoDevice = "Namespace-NoDevice" +// defaultPodNameNoDevice = "Pod-NoDevice" +//) +// +//var ( +// expectedContainerInfoToResourcesMap = map[ContainerInfo][]ResourceInfo{ +// { +// podName: defaultPodName, +// containerName: defaultContainerName, +// namespace: defaultNamespace, +// }: { +// { +// resourceName: defaultResourceName, +// deviceID: defaultDeviceID1, +// }, +// { +// resourceName: defaultResourceName, +// deviceID: defaultDeviceID2, +// }, +// }, +// } +// +// expectedResourceToPodContainerMap = map[ResourceInfo]ContainerInfo{ +// { +// resourceName: defaultResourceName, +// deviceID: defaultDeviceID1, +// }: { +// podName: defaultPodName, +// containerName: defaultContainerName, +// namespace: defaultNamespace, +// }, +// { +// resourceName: defaultResourceName, +// deviceID: defaultDeviceID2, +// }: { +// podName: defaultPodName, +// containerName: defaultContainerName, +// namespace: defaultNamespace, +// }, +// } +// +// expectedContainerInfo = ContainerInfo{ +// podName: defaultPodName, +// containerName: defaultContainerName, +// namespace: defaultNamespace, +// } +// +// expectedResourceInfo = []ResourceInfo{ +// { +// resourceName: defaultResourceName, +// deviceID: defaultDeviceID1, +// }, +// { +// resourceName: defaultResourceName, +// deviceID: defaultDeviceID2, +// }, +// } +// +// listPodResourcesResponse = &podresourcesv1.ListPodResourcesResponse{ +// PodResources: []*podresourcesv1.PodResources{ +// { +// Name: defaultPodName, +// Namespace: defaultNamespace, +// Containers: []*podresourcesv1.ContainerResources{ +// { +// Name: defaultContainerName, +// Devices: []*podresourcesv1.ContainerDevices{ +// { +// ResourceName: defaultResourceName, +// DeviceIds: []string{defaultDeviceID1, defaultDeviceID2}, +// }, +// { +// ResourceName: defaultResourceNameSkipped, +// DeviceIds: []string{defaultDeviceID3, defaultDeviceID4}, +// }, +// }, +// }, +// }, +// }, +// { +// Name: defaultPodNameNoDevice, +// Namespace: defaultNamespaceNoDevice, +// Containers: []*podresourcesv1.ContainerResources{ +// { +// Name: defaultContainerNameNoDevice, +// Devices: []*podresourcesv1.ContainerDevices{}, +// }, +// }, +// }, +// }, +// } +// +// listPodResourcesResponseWithEmptyPodResources = &podresourcesv1.ListPodResourcesResponse{ +// PodResources: []*podresourcesv1.PodResources{}, +// } +// +// listPodResourcesResponseWithEmptyResponse = &podresourcesv1.ListPodResourcesResponse{} +// +// resourceNameSet = map[string]struct{}{ +// defaultResourceName: {}, +// } +//) +// +//type MockPodResourcesClient struct { +// response *podresourcesv1.ListPodResourcesResponse +// err error +// shutdownCalled bool +//} +// +//func (m *MockPodResourcesClient) ListPods() (*podresourcesv1.ListPodResourcesResponse, error) { +// return m.response, m.err +//} +// +//func (m *MockPodResourcesClient) Shutdown() { +// m.shutdownCalled = true +//} +// +//func TestNewPodResourcesStore(t *testing.T) { +// logger := zap.NewNop() +// store := NewPodResourcesStore(logger) +// assert.NotNil(t, store, "PodResourcesStore should not be nil") +// assert.NotNil(t, store.ctx, "Context should not be nil") +// assert.NotNil(t, store.cancel, "Cancel function should not be nil") +//} +// +//func TestRefreshTick(t *testing.T) { +// store := constructPodResourcesStore(make(map[ContainerInfo][]ResourceInfo), make(map[ResourceInfo]ContainerInfo), listPodResourcesResponse, nil) +// +// store.lastRefreshed = time.Now().Add(-time.Hour) +// +// store.refreshTick() +// +// assert.True(t, store.lastRefreshed.After(time.Now().Add(-time.Hour)), "lastRefreshed should have been updated") +//} +// +//func TestShutdown(t *testing.T) { +// store := constructPodResourcesStore(make(map[ContainerInfo][]ResourceInfo), make(map[ResourceInfo]ContainerInfo), listPodResourcesResponse, nil) +// +// mockClient := &MockPodResourcesClient{listPodResourcesResponse, nil, false} +// store.podResourcesClient = mockClient +// +// store.Shutdown() +// +// assert.True(t, mockClient.shutdownCalled, "Shutdown method of the client should have been called") +//} +// +//func TestUpdateMaps(t *testing.T) { +// store := constructPodResourcesStore(make(map[ContainerInfo][]ResourceInfo), make(map[ResourceInfo]ContainerInfo), listPodResourcesResponse, nil) +// store.updateMaps() +// +// assert.NotNil(t, store.containerInfoToResourcesMap) +// assert.NotNil(t, store.resourceToPodContainerMap) +// assert.Equal(t, len(expectedContainerInfoToResourcesMap), len(store.containerInfoToResourcesMap)) +// assert.Equal(t, len(expectedResourceToPodContainerMap), len(store.resourceToPodContainerMap)) +// assert.Equal(t, expectedContainerInfoToResourcesMap, store.containerInfoToResourcesMap) +// assert.Equal(t, expectedResourceToPodContainerMap, store.resourceToPodContainerMap) +//} +// +//func TestGets(t *testing.T) { +// store := constructPodResourcesStore(make(map[ContainerInfo][]ResourceInfo), make(map[ResourceInfo]ContainerInfo), listPodResourcesResponse, nil) +// store.updateMaps() +// +// assertMapsContainData(t, store) +//} +// +//func TestGetsWhenThereAreNoPods(t *testing.T) { +// store := constructPodResourcesStore(make(map[ContainerInfo][]ResourceInfo), make(map[ResourceInfo]ContainerInfo), listPodResourcesResponseWithEmptyPodResources, nil) +// store.updateMaps() +// +// assertMapsDontContainData(t, store) +//} +// +//func TestGetsWhenPodResourcesResponseIsEmpty(t *testing.T) { +// store := constructPodResourcesStore(make(map[ContainerInfo][]ResourceInfo), make(map[ResourceInfo]ContainerInfo), listPodResourcesResponseWithEmptyResponse, nil) +// store.updateMaps() +// +// assertMapsDontContainData(t, store) +//} +// +//func TestGetsWhenPodResourcesThrowsError(t *testing.T) { +// store := constructPodResourcesStore(make(map[ContainerInfo][]ResourceInfo), make(map[ResourceInfo]ContainerInfo), listPodResourcesResponseWithEmptyResponse, fmt.Errorf("mocked behavior")) +// store.updateMaps() +// +// assertMapsDontContainData(t, store) +//} +// +//func TestAddResourceName(t *testing.T) { +// store := constructPodResourcesStore(make(map[ContainerInfo][]ResourceInfo), make(map[ResourceInfo]ContainerInfo), listPodResourcesResponse, nil) +// +// store.resourceNameSet = make(map[string]struct{}) +// store.updateMaps() +// assertMapsDontContainData(t, store) +// +// // After adding resource to map +// store.AddResourceName(defaultResourceName) +// store.updateMaps() +// assertMapsContainData(t, store) +//} +// +//func constructPodResourcesStore(containerToDevices map[ContainerInfo][]ResourceInfo, deviceToContainer map[ResourceInfo]ContainerInfo, podResourcesResponse *podresourcesv1.ListPodResourcesResponse, podResourcesError error) *PodResourcesStore { +// logger, _ := zap.NewDevelopment() +// return &PodResourcesStore{ +// containerInfoToResourcesMap: containerToDevices, +// resourceToPodContainerMap: deviceToContainer, +// resourceNameSet: resourceNameSet, +// lastRefreshed: time.Now(), +// ctx: context.Background(), +// cancel: func() {}, +// logger: logger, +// podResourcesClient: &MockPodResourcesClient{podResourcesResponse, podResourcesError, false}, +// } +//} +// +//func assertMapsContainData(t *testing.T, store *PodResourcesStore) { +// assert.Equal(t, len(expectedContainerInfoToResourcesMap), len(store.containerInfoToResourcesMap)) +// assert.Equal(t, len(expectedResourceToPodContainerMap), len(store.resourceToPodContainerMap)) +// +// assert.Equal(t, expectedContainerInfo, *store.GetContainerInfo(defaultDeviceID1, defaultResourceName)) +// assert.Equal(t, expectedResourceInfo, *store.GetResourcesInfo(defaultPodName, defaultContainerName, defaultNamespace)) +// +// actualResourceInfo := store.GetResourcesInfo(defaultPodNameNoDevice, defaultContainerNameNoDevice, defaultNamespaceNoDevice) +// if actualResourceInfo != nil { +// t.Errorf("Expected GetResourcesInfo to return nil for an unexpected key, but got %v", actualResourceInfo) +// } +//} +// +//func assertMapsDontContainData(t *testing.T, store *PodResourcesStore) { +// assert.Equal(t, 0, len(store.containerInfoToResourcesMap)) +// assert.Equal(t, 0, len(store.resourceToPodContainerMap)) +// +// actualContainerInfo := store.GetContainerInfo(defaultDeviceID1, defaultResourceName) +// if actualContainerInfo != nil { +// t.Errorf("Expected GetContainerInfo to return nil for an unexpected key, but got %v", actualContainerInfo) +// } +// +// actualResourceInfo := store.GetResourcesInfo(defaultPodName, defaultContainerName, defaultNamespace) +// if actualResourceInfo != nil { +// t.Errorf("Expected GetResourcesInfo to return nil for an unexpected key, but got %v", actualResourceInfo) +// } +//} From 0d98f6160f22675470d788865d5166ae5c84b45d Mon Sep 17 00:00:00 2001 From: Hyunsoo Kim Date: Fri, 2 Feb 2024 13:43:25 -0500 Subject: [PATCH 17/53] 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 18/53] 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 19/53] 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 20/53] 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 21/53] 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 22/53] 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 23/53] 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 24/53] 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 25/53] 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 26/53] 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 27/53] 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 28/53] 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 29/53] 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 30/53] 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 31/53] 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 32/53] 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 33/53] 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" From a821803b3f4bca671abfde620583842b7c6e58b5 Mon Sep 17 00:00:00 2001 From: Aditya Purang Date: Mon, 4 Mar 2024 10:44:00 +0000 Subject: [PATCH 34/53] add podresource scrapper and metric data printer --- .../internal/prometheusscraper/decorator.go | 55 ++++++++++++++- .../simple_prometheus_scraper.go | 12 +++- .../stores/kubeletutil/podresourcesclient.go | 6 +- .../internal/stores/podresourcesstore.go | 35 ++++++---- .../stores/staticPodResourcesRefresher.go | 67 +++++++++++++++++++ 5 files changed, 156 insertions(+), 19 deletions(-) create mode 100644 receiver/awscontainerinsightreceiver/internal/stores/staticPodResourcesRefresher.go diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator.go index 8f354e7de577..3b89eea452b0 100644 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator.go +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator.go @@ -7,7 +7,7 @@ import ( "context" "encoding/json" "errors" - + "fmt" 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" @@ -15,6 +15,7 @@ import ( "go.opentelemetry.io/collector/pdata/pmetric" "go.uber.org/zap" "golang.org/x/exp/maps" + "strings" ) const ( @@ -143,6 +144,8 @@ func (dc *decorateConsumer) ConsumeMetrics(ctx context.Context, md pmetric.Metri } } } + + dc.logMd(md) return dc.nextConsumer.ConsumeMetrics(ctx, md) } @@ -203,3 +206,53 @@ func (dc *decorateConsumer) Shutdown() error { } return errs } + +func (dc *decorateConsumer) logMd(md pmetric.Metrics) { + var logMessage strings.Builder + + logMessage.WriteString("METRICS_MD : {\n") + rms := md.ResourceMetrics() + for i := 0; i < rms.Len(); i++ { + rs := rms.At(i) + ilms := rs.ScopeMetrics() + logMessage.WriteString(fmt.Sprintf("\tResourceMetric_%d: {\n", i)) + for j := 0; j < ilms.Len(); j++ { + ils := ilms.At(j) + metrics := ils.Metrics() + logMessage.WriteString(fmt.Sprintf("\t\tScopeMetric_%d: {\n", j)) + logMessage.WriteString(fmt.Sprintf("\t\tMetrics_%d: [\n", j)) + + for k := 0; k < metrics.Len(); k++ { + m := metrics.At(k) + logMessage.WriteString(fmt.Sprintf("\t\t\tMetric_%d: {\n", k)) + logMessage.WriteString(fmt.Sprintf("\t\t\tname: %s,\n", m.Name())) + + var datapoints pmetric.NumberDataPointSlice + switch m.Type() { + case pmetric.MetricTypeGauge: + datapoints = m.Gauge().DataPoints() + case pmetric.MetricTypeSum: + datapoints = m.Sum().DataPoints() + default: + datapoints = pmetric.NewNumberDataPointSlice() + } + + logMessage.WriteString("datapoints: [\n") + for yu := 0; yu < datapoints.Len(); yu++ { + logMessage.WriteString("{\n") + logMessage.WriteString(fmt.Sprintf("attributes: %v,\n", datapoints.At(yu).Attributes().AsRaw())) + logMessage.WriteString(fmt.Sprintf("value: %v,\n", datapoints.At(yu).DoubleValue())) + logMessage.WriteString("},\n") + } + logMessage.WriteString("],\n") + logMessage.WriteString("\t\t\t},\n") + } + logMessage.WriteString("\t\t],\n") + logMessage.WriteString("\t\t},\n") + } + logMessage.WriteString("\t},\n") + } + logMessage.WriteString("},\n") + + dc.logger.Info(logMessage.String()) +} diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go index 500b73c006dc..94736b4d02fd 100644 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go @@ -4,7 +4,6 @@ import ( "context" "errors" "fmt" - ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/prometheusreceiver" @@ -86,9 +85,18 @@ func NewSimplePromethuesScraper(opts SimplePromethuesScraperOpts, scraperConfig func (ds *SimplePromethuesScraper) 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 + + // this thing works, now just fixing the podresourcestore + //ds.settings.Logger.Info("static_pod_resources staring scrapping") + //stores.StartScraping(ds.settings.Logger) + podresourcesstore := stores.NewPodResourcesStore(ds.settings.Logger) + ds.settings.Logger.Info("Adding resources to PodResources") + podresourcesstore.AddResourceName("aws.amazon.com/neuroncore") + podresourcesstore.AddResourceName("aws.amazon.com/neuron") + podresourcesstore.AddResourceName("aws.amazon.com/neurondevice") podresourcesstore.GetResourcesInfo("123", "123", "123") - + if !ds.running { ds.settings.Logger.Info("The scraper is not running, starting up the scraper") err := ds.prometheusReceiver.Start(ds.ctx, ds.host) diff --git a/receiver/awscontainerinsightreceiver/internal/stores/kubeletutil/podresourcesclient.go b/receiver/awscontainerinsightreceiver/internal/stores/kubeletutil/podresourcesclient.go index 792cc0e3b729..7a2984fdee69 100644 --- a/receiver/awscontainerinsightreceiver/internal/stores/kubeletutil/podresourcesclient.go +++ b/receiver/awscontainerinsightreceiver/internal/stores/kubeletutil/podresourcesclient.go @@ -57,7 +57,8 @@ func (p *PodResourcesClient) connectToServer(socket string) (*grpc.ClientConn, e return nil, fmt.Errorf("failed to check socket path: %w", err) } - ctx, _ := context.WithTimeout(context.Background(), connectionTimeout) + ctx, cancel := context.WithTimeout(context.Background(), connectionTimeout) + defer cancel() conn, err := grpc.DialContext(ctx, socket, @@ -79,7 +80,8 @@ func (p *PodResourcesClient) connectToServer(socket string) (*grpc.ClientConn, e } func (p *PodResourcesClient) ListPods() (*podresourcesapi.ListPodResourcesResponse, error) { - ctx, _ := context.WithTimeout(context.Background(), connectionTimeout) + ctx, cancel := context.WithTimeout(context.Background(), connectionTimeout) + defer cancel() resp, err := p.delegateClient.List(ctx, &podresourcesapi.ListPodResourcesRequest{}) if err != nil { diff --git a/receiver/awscontainerinsightreceiver/internal/stores/podresourcesstore.go b/receiver/awscontainerinsightreceiver/internal/stores/podresourcesstore.go index 4f2408140dfc..de0c66a00569 100644 --- a/receiver/awscontainerinsightreceiver/internal/stores/podresourcesstore.go +++ b/receiver/awscontainerinsightreceiver/internal/stores/podresourcesstore.go @@ -66,27 +66,27 @@ func NewPodResourcesStore(logger *zap.Logger) *PodResourcesStore { podResourcesClient: podResourcesClient, } - instance.AddResourceName("aws.amazon.com/neuroncore") - instance.AddResourceName("aws.amazon.com/neuron") - instance.AddResourceName("aws.amazon.com/neurondevice") - go func() { refreshTicker := time.NewTicker(time.Second) for { select { case <-refreshTicker.C: + logger.Info("entered refresh tick") instance.refreshTick() case <-instance.ctx.Done(): + logger.Info("stopping refresh tick") refreshTicker.Stop() return } } }() + time.Sleep(40) }) return instance } func (p *PodResourcesStore) refreshTick() { + p.logger.Info("PodResources entered refreshTick") now := time.Now() if now.Sub(p.lastRefreshed) >= taskTimeout { p.refresh() @@ -95,7 +95,9 @@ func (p *PodResourcesStore) refreshTick() { } func (p *PodResourcesStore) refresh() { + p.logger.Info("PodResources entered refresh") doRefresh := func() { + p.logger.Info("PodResources entering update map") p.updateMaps() } @@ -106,10 +108,10 @@ func (p *PodResourcesStore) updateMaps() { p.containerInfoToResourcesMap = make(map[ContainerInfo][]ResourceInfo) p.resourceToPodContainerMap = make(map[ResourceInfo]ContainerInfo) - //if len(p.resourceNameSet) == 0 { - // p.logger.Warn("No resource names allowlisted thus skipping updating of maps.") - // return - //} + if len(p.resourceNameSet) == 0 { + p.logger.Warn("No resource names allowlisted thus skipping updating of maps.") + return + } devicePods, err := p.podResourcesClient.ListPods() if err != nil { @@ -136,13 +138,13 @@ func (p *PodResourcesStore) updateMaps() { resourceName: device.GetResourceName(), deviceID: deviceID, } - //_, found := p.resourceNameSet[resourceInfo.resourceName] - //if found { - p.containerInfoToResourcesMap[containerInfo] = append(p.containerInfoToResourcesMap[containerInfo], resourceInfo) - p.resourceToPodContainerMap[resourceInfo] = containerInfo + _, found := p.resourceNameSet[resourceInfo.resourceName] + if found { + p.containerInfoToResourcesMap[containerInfo] = append(p.containerInfoToResourcesMap[containerInfo], resourceInfo) + p.resourceToPodContainerMap[resourceInfo] = containerInfo - p.logger.Info("/nContainerInfo : {" + containerInfo.namespace + "_" + containerInfo.podName + "_" + containerInfo.containerName + "}" + " -> ResourceInfo : {" + resourceInfo.resourceName + "_" + resourceInfo.deviceID + "_" + "}") - //} + p.logger.Info("/nContainerInfo : {" + containerInfo.namespace + "_" + containerInfo.podName + "_" + containerInfo.containerName + "}" + " -> ResourceInfo : {" + resourceInfo.resourceName + "_" + resourceInfo.deviceID + "_" + "}") + } } } } @@ -185,6 +187,11 @@ func (p *PodResourcesStore) PrintMaps() { } } +func (p *PodResourcesStore) UpdateAndPrintMapsManually() { + // this also has embedded print statement + p.updateMaps() +} + func (p *PodResourcesStore) Shutdown() { p.cancel() p.podResourcesClient.Shutdown() diff --git a/receiver/awscontainerinsightreceiver/internal/stores/staticPodResourcesRefresher.go b/receiver/awscontainerinsightreceiver/internal/stores/staticPodResourcesRefresher.go new file mode 100644 index 000000000000..f105341f603d --- /dev/null +++ b/receiver/awscontainerinsightreceiver/internal/stores/staticPodResourcesRefresher.go @@ -0,0 +1,67 @@ +package stores + +import ( + "context" + "fmt" + "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" + podresourcesapi "k8s.io/kubelet/pkg/apis/podresources/v1" + "net" + "os" + "time" +) + +const ( + socketPath = "/var/lib/kubelet/pod-resources/kubelet.sock" + connectionTimeout = 10 * time.Second +) + +func StartScraping(logger *zap.Logger) { + conn, cleanup, err := connectToServer(socketPath) + if err != nil { + fmt.Printf("error connecting to socket: %v\n", err) + os.Exit(1) + } + defer cleanup() + + client := podresourcesapi.NewPodResourcesListerClient(conn) + for { + list(&client, logger) + time.Sleep(30 * time.Second) + } +} + +func list(client *podresourcesapi.PodResourcesListerClient, logger *zap.Logger) { + ctx, cancel := context.WithTimeout(context.Background(), connectionTimeout) + defer cancel() + + logger.Info("static_pod_resources calling ListPodResources") + resp, err := (*client).List(ctx, &podresourcesapi.ListPodResourcesRequest{}) + if err != nil { + logger.Info("static_pod_resources error list resources: " + err.Error()) + } else { + logger.Info("static_pod_resources response: " + resp.String()) + } +} + +func connectToServer(socket string) (*grpc.ClientConn, func(), error) { + ctx, cancel := context.WithTimeout(context.Background(), connectionTimeout) + defer cancel() + + conn, err := grpc.DialContext(ctx, + socket, + grpc.WithTransportCredentials(insecure.NewCredentials()), + grpc.WithBlock(), + grpc.WithContextDialer(func(ctx context.Context, addr string) (net.Conn, error) { + d := net.Dialer{} + return d.DialContext(ctx, "unix", addr) + }), + ) + + if err != nil { + return nil, func() {}, fmt.Errorf("failure connecting to %s: %v", socket, err) + } + + return conn, func() { conn.Close() }, nil +} From 83896ab2ff406fc3062adf6c034ced5f3192f391 Mon Sep 17 00:00:00 2001 From: Aditya Purang Date: Mon, 4 Mar 2024 11:07:52 +0000 Subject: [PATCH 35/53] refactor logMd --- .../internal/prometheusscraper/decorator.go | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator.go index 3b89eea452b0..1424eb51d7c6 100644 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator.go +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator.go @@ -210,22 +210,22 @@ func (dc *decorateConsumer) Shutdown() error { func (dc *decorateConsumer) logMd(md pmetric.Metrics) { var logMessage strings.Builder - logMessage.WriteString("METRICS_MD : {\n") + logMessage.WriteString("\"METRICS_MD\" : {\n") rms := md.ResourceMetrics() for i := 0; i < rms.Len(); i++ { rs := rms.At(i) ilms := rs.ScopeMetrics() - logMessage.WriteString(fmt.Sprintf("\tResourceMetric_%d: {\n", i)) + logMessage.WriteString(fmt.Sprintf("\t\"ResourceMetric_%d\": {\n", i)) for j := 0; j < ilms.Len(); j++ { ils := ilms.At(j) metrics := ils.Metrics() - logMessage.WriteString(fmt.Sprintf("\t\tScopeMetric_%d: {\n", j)) - logMessage.WriteString(fmt.Sprintf("\t\tMetrics_%d: [\n", j)) + logMessage.WriteString(fmt.Sprintf("\t\t\"ScopeMetric_%d\": {\n", j)) + logMessage.WriteString(fmt.Sprintf("\t\t\"Metrics_%d\": [\n", j)) for k := 0; k < metrics.Len(); k++ { m := metrics.At(k) - logMessage.WriteString(fmt.Sprintf("\t\t\tMetric_%d: {\n", k)) - logMessage.WriteString(fmt.Sprintf("\t\t\tname: %s,\n", m.Name())) + logMessage.WriteString(fmt.Sprintf("\t\t\t\"Metric_%d\": {\n", k)) + logMessage.WriteString(fmt.Sprintf("\t\t\t\t\"name\": \"%s\",\n", m.Name())) var datapoints pmetric.NumberDataPointSlice switch m.Type() { @@ -237,14 +237,14 @@ func (dc *decorateConsumer) logMd(md pmetric.Metrics) { datapoints = pmetric.NewNumberDataPointSlice() } - logMessage.WriteString("datapoints: [\n") + logMessage.WriteString("\t\t\t\t\"datapoints\": [\n") for yu := 0; yu < datapoints.Len(); yu++ { - logMessage.WriteString("{\n") - logMessage.WriteString(fmt.Sprintf("attributes: %v,\n", datapoints.At(yu).Attributes().AsRaw())) - logMessage.WriteString(fmt.Sprintf("value: %v,\n", datapoints.At(yu).DoubleValue())) - logMessage.WriteString("},\n") + logMessage.WriteString("\t\t\t\t\t{\n") + logMessage.WriteString(fmt.Sprintf("\t\t\t\t\t\t\"attributes\": \"%v\",\n", datapoints.At(yu).Attributes().AsRaw())) + logMessage.WriteString(fmt.Sprintf("\t\t\t\t\t\t\"value\": %v,\n", datapoints.At(yu).DoubleValue())) + logMessage.WriteString("\t\t\t\t\t},\n") } - logMessage.WriteString("],\n") + logMessage.WriteString("\t\t\t\t],\n") logMessage.WriteString("\t\t\t},\n") } logMessage.WriteString("\t\t],\n") From 326765323650173b114089fd77e95cba5c4fe683 Mon Sep 17 00:00:00 2001 From: Samarth Singh Date: Mon, 4 Mar 2024 12:12:04 +0000 Subject: [PATCH 36/53] Merge conflicts --- .../internal/cadvisor/extractors/extractor.go | 2 -- receiver/awscontainerinsightreceiver/receiver.go | 12 +++++++++++- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor.go index ff30c80e50e1..5368a72702b3 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractor.go @@ -14,8 +14,6 @@ import ( "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" ) -var _ stores.CIMetric = (*CAdvisorMetric)(nil) - func GetStats(info *cinfo.ContainerInfo) *cinfo.ContainerStats { if len(info.Stats) == 0 { return nil diff --git a/receiver/awscontainerinsightreceiver/receiver.go b/receiver/awscontainerinsightreceiver/receiver.go index 61d80e7cdae7..6c9b0053f2b8 100644 --- a/receiver/awscontainerinsightreceiver/receiver.go +++ b/receiver/awscontainerinsightreceiver/receiver.go @@ -9,6 +9,7 @@ import ( "time" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/gpu" + nueron "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/neuron" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/prometheusscraper" "go.opentelemetry.io/collector/component" "go.opentelemetry.io/collector/consumer" @@ -21,7 +22,6 @@ import ( "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" @@ -209,6 +209,16 @@ func (acir *awsContainerInsightReceiver) initDcgmScraper(ctx context.Context, ho return err } +func (acir *awsContainerInsightReceiver) initNeuronScraper(opts prometheusscraper.SimplePromethuesScraperOpts) error { + if !acir.config.EnableAcceleratedComputeMetrics { + return nil + } + + var err error + acir.neuronMonitorScraper, err = prometheusscraper.NewSimplePromethuesScraper(opts, nueron.GetNueronScrapeConfig(opts)) + return err +} + // Shutdown stops the awsContainerInsightReceiver receiver. func (acir *awsContainerInsightReceiver) Shutdown(context.Context) error { if acir.prometheusScraper != nil { From 164bd84b930de0859cc4a8f143c4348d13e2efdd Mon Sep 17 00:00:00 2001 From: Samarth Singh Date: Mon, 4 Mar 2024 12:16:21 +0000 Subject: [PATCH 37/53] More cleanups --- .../internal/cadvisor/cadvisor_linux.go | 1 - .../internal/cadvisor/cadvisor_linux_test.go | 1 - .../awscontainerinsightreceiver/internal/stores/podstore.go | 2 +- receiver/awscontainerinsightreceiver/internal/stores/utils.go | 1 - 4 files changed, 1 insertion(+), 4 deletions(-) diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_linux.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_linux.go index 4261f0f63144..89334dce0fbc 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_linux.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_linux.go @@ -22,7 +22,6 @@ 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" diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_linux_test.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_linux_test.go index b28c421378fc..dfd15470ddf0 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_linux_test.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/cadvisor_linux_test.go @@ -16,7 +16,6 @@ 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" diff --git a/receiver/awscontainerinsightreceiver/internal/stores/podstore.go b/receiver/awscontainerinsightreceiver/internal/stores/podstore.go index abcff2796915..fa5577f93ebf 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 { diff --git a/receiver/awscontainerinsightreceiver/internal/stores/utils.go b/receiver/awscontainerinsightreceiver/internal/stores/utils.go index 495d78716c98..a0db8fe03e93 100644 --- a/receiver/awscontainerinsightreceiver/internal/stores/utils.go +++ b/receiver/awscontainerinsightreceiver/internal/stores/utils.go @@ -22,7 +22,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 { From 19223b10f643e7904dc6845228689696325fbee6 Mon Sep 17 00:00:00 2001 From: Samarth Singh Date: Mon, 4 Mar 2024 12:17:37 +0000 Subject: [PATCH 38/53] Remove unused imports --- .../internal/k8sapiserver/prometheus_scraper_test.go | 1 - 1 file changed, 1 deletion(-) diff --git a/receiver/awscontainerinsightreceiver/internal/k8sapiserver/prometheus_scraper_test.go b/receiver/awscontainerinsightreceiver/internal/k8sapiserver/prometheus_scraper_test.go index 5b469c652379..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" From c65ad64a5884c1fd548cbef69036fd887dfb747c Mon Sep 17 00:00:00 2001 From: Samarth Singh Date: Mon, 4 Mar 2024 12:29:26 +0000 Subject: [PATCH 39/53] Add decorator to neuron scraper --- .../awscontainerinsightreceiver/config.go | 1 + .../neuron/neuron_monitor_scraper_config.go | 12 -- .../internal/prometheusscraper/decorator.go | 140 ++++++++++++++++++ .../simple_prometheus_scraper.go | 12 +- .../awscontainerinsightreceiver/receiver.go | 2 + 5 files changed, 154 insertions(+), 13 deletions(-) create mode 100644 receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator.go diff --git a/receiver/awscontainerinsightreceiver/config.go b/receiver/awscontainerinsightreceiver/config.go index a45f31beabe5..7e01f396a4c1 100644 --- a/receiver/awscontainerinsightreceiver/config.go +++ b/receiver/awscontainerinsightreceiver/config.go @@ -60,6 +60,7 @@ type Config struct { // EnableAcceleratedComputeMetrics enabled features with accelerated compute resources where metrics are scraped from vendor specific sources EnableAcceleratedComputeMetrics bool `mapstructure:"accelerated_compute_metrics"` + EnableGpuMetric bool `mapstructure:"accelerated_compute_metrics"` // EnableNeuronMetric toggles Neuron monitoring where metrics are scraped from neuron monitor // The default value is false. diff --git a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go index 735af401b06e..822bb0917bd9 100644 --- a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go +++ b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go @@ -40,18 +40,6 @@ func GetNueronScrapeConfig(opts prometheusscraper.SimplePromethuesScraperOpts) * Label: "k8s-app=neuron-monitor-service", }, }, - AttachMetadata: kubernetes.AttachMetadataConfig{ - Node: true, - }, - }, - }, - RelabelConfigs: []*relabel.Config{ - { - SourceLabels: model.LabelNames{"__address__"}, - Regex: relabel.MustNewRegexp("([^:]+)(?::\\d+)?"), - Replacement: "${1}:8000", - TargetLabel: "__address__", - Action: relabel.Replace, }, }, MetricRelabelConfigs: GetNueronMetricRelabelConfigs(opts), diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator.go new file mode 100644 index 000000000000..bb2a6dfb3980 --- /dev/null +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator.go @@ -0,0 +1,140 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +package prometheusscraper + +import ( + "context" + + "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 ( + 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 metricToUnit = map[string]string{ + gpuUtil: "Percent", + gpuMemUtil: "Percent", + gpuMemUsed: "Bytes", + gpuMemTotal: "Bytes", + gpuTemperature: "None", + gpuPowerDraw: "None", +} + +// 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) + converted := ci.ConvertToFieldsAndTags(m, dc.logger) + var rcis []*stores.RawContainerInsightsMetric + for _, pair := range converted { + rcis = append(rcis, stores.NewRawContainerInsightsMetricWithData(ci.TypeGpuContainer, pair.Fields, pair.Tags, dc.logger)) + } + + decorated := dc.decorateMetrics(rcis) + dc.updateAttributes(m, decorated) + 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(rcis []*stores.RawContainerInsightsMetric) []*stores.RawContainerInsightsMetric { + var result []*stores.RawContainerInsightsMetric + if dc.containerOrchestrator != ci.EKS { + return result + } + for _, rci := range rcis { + // add tags for EKS + out := dc.k8sDecorator.Decorate(rci) + if out != nil { + result = append(result, out.(*stores.RawContainerInsightsMetric)) + } + } + return result +} + +func (dc *decorateConsumer) updateAttributes(m pmetric.Metric, rcis []*stores.RawContainerInsightsMetric) { + if len(rcis) == 0 { + 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() == 0 { + return + } + 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 := 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) + } + } +} + +func (dc *decorateConsumer) Shutdown() error { + if dc.k8sDecorator != nil { + return dc.k8sDecorator.Shutdown() + } + return nil +} diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go index 6a703c036f42..b38173b99d1f 100644 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go @@ -5,6 +5,7 @@ import ( "errors" "fmt" + ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/prometheusreceiver" "github.com/prometheus/prometheus/config" "go.opentelemetry.io/collector/component" @@ -30,6 +31,8 @@ type SimplePromethuesScraperOpts struct { Host component.Host HostInfoProvider hostInfoProvider ScraperConfigs *config.ScrapeConfig + K8sDecorator Decorator + Logger *zap.Logger } type hostInfoProvider interface { @@ -58,8 +61,15 @@ func NewSimplePromethuesScraper(opts SimplePromethuesScraperOpts, scraperConfig 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) } diff --git a/receiver/awscontainerinsightreceiver/receiver.go b/receiver/awscontainerinsightreceiver/receiver.go index 6c9b0053f2b8..20c4c6d18a7e 100644 --- a/receiver/awscontainerinsightreceiver/receiver.go +++ b/receiver/awscontainerinsightreceiver/receiver.go @@ -111,6 +111,8 @@ func (acir *awsContainerInsightReceiver) Start(ctx context.Context, host compone Consumer: acir.nextConsumer, Host: host, HostInfoProvider: hostinfo, + K8sDecorator: k8sDecorator, + Logger: acir.settings.Logger, } err = acir.initNeuronScraper(simplePrometheusScraperOpts) From c6966db63bf1ea6633475cb40a23c005d097ba88 Mon Sep 17 00:00:00 2001 From: Samarth Singh Date: Tue, 5 Mar 2024 10:43:04 +0000 Subject: [PATCH 40/53] Add decorator to add podResources --- .../internal/prometheusscraper/decorator.go | 44 +++++++++++ .../prometheusscraper/metric_modifier.go | 77 +++++++++++++++++++ .../simple_prometheus_scraper.go | 13 ++-- .../internal/stores/podresourcesstore.go | 20 ++--- 4 files changed, 137 insertions(+), 17 deletions(-) create mode 100644 receiver/awscontainerinsightreceiver/internal/prometheusscraper/metric_modifier.go diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator.go index f5467b8c9e70..75c7f598a3c0 100644 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator.go +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator.go @@ -40,6 +40,7 @@ type decorateConsumer struct { containerOrchestrator string nextConsumer consumer.Metrics k8sDecorator Decorator + metricModifier MetricModifier logger *zap.Logger } @@ -51,6 +52,7 @@ func (dc *decorateConsumer) Capabilities() consumer.Capabilities { func (dc *decorateConsumer) ConsumeMetrics(ctx context.Context, md pmetric.Metrics) error { resourceTags := make(map[string]string) + md, _ = neuronMetricsProcess(md, &dc.metricModifier) rms := md.ResourceMetrics() for i := 0; i < rms.Len(); i++ { // get resource attributes @@ -78,6 +80,7 @@ func (dc *decorateConsumer) ConsumeMetrics(ctx context.Context, md pmetric.Metri } } } + dc.logMd(md) return dc.nextConsumer.ConsumeMetrics(ctx, md) } @@ -189,3 +192,44 @@ func (dc *decorateConsumer) logMd(md pmetric.Metrics) { dc.logger.Info(logMessage.String()) } + +func neuronMetricsProcess(md pmetric.Metrics, modifier *MetricModifier) (pmetric.Metrics, error) { + rms := md.ResourceMetrics() + for i := 0; i < rms.Len(); i++ { + rs := rms.At(i) + ilms := rs.ScopeMetrics() + for j := 0; j < ilms.Len(); j++ { + ils := ilms.At(j) + metrics := ils.Metrics() + + // neuronHardwareInfo := pmetric.Metric{} + // for k := 0; k < metrics.Len(); k++ { + // m := metrics.At(k) + // if m.Name() == "neuroncore_per_device_count" { + // neuronHardwareInfo = m + // break + // } + // } + + // neuronCoresPerDeviceValue, _ := neuronHardwareInfo.Gauge().DataPoints().At(0).Attributes().Get(neuronCorePerDeviceKey) + // neuronCoresPerDevice := neuronCoresPerDeviceValue.Int() + + for k := 0; k < metrics.Len(); k++ { + m := metrics.At(k) + modifier.AddPodCorrelationAttributes(getMetricDatapoints(m), 2) // need to change this + } + } + } + return md, nil +} + +func getMetricDatapoints(m pmetric.Metric) pmetric.NumberDataPointSlice { + switch m.Type() { + case pmetric.MetricTypeGauge: + return m.Gauge().DataPoints() + case pmetric.MetricTypeSum: + return m.Sum().DataPoints() + default: + return pmetric.NewNumberDataPointSlice() + } +} diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/metric_modifier.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/metric_modifier.go new file mode 100644 index 000000000000..42dfa580cc7b --- /dev/null +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/metric_modifier.go @@ -0,0 +1,77 @@ +package prometheusscraper + +import ( + "strconv" + + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" + "go.opentelemetry.io/collector/pdata/pmetric" + "go.uber.org/zap" +) + +const ( + neuronCoreAttributeKey = "neuroncore" + neuronDeviceAttributeKey = "neuron_device_index" + neuronCoreResourceName = "aws.amazon.com/neuroncore" + neuronDeviceResourceName = "aws.amazon.com/neurondevice" + neuronDeviceResourceNameAlt = "aws.amazon.com/neuron" +) + +type MetricModifier struct { + logger *zap.Logger + podResourcesStore *stores.PodResourcesStore // replace with podResourcesApi +} + +func NewMetricModifier(logger *zap.Logger, podResourcesStore *stores.PodResourcesStore) *MetricModifier { + d := &MetricModifier{ + logger: logger, + podResourcesStore: podResourcesStore, + } + return d +} + +func (d *MetricModifier) AddPodCorrelationAttributes(metricDatapoints pmetric.NumberDataPointSlice, neuronCoresPerDevice int64) { + for i := 0; i < metricDatapoints.Len(); i++ { + attributes := metricDatapoints.At(i).Attributes() + neuronCoreIndex, neuronCoreIndexPresent := attributes.Get(neuronCoreAttributeKey) + if neuronCoreIndexPresent { + neuronDeviceIndex := neuronCoreIndex.Int() / neuronCoresPerDevice + neuronDeviceIndexString := strconv.FormatInt(neuronDeviceIndex, 10) + neuronCoreIndexString := strconv.FormatInt(neuronCoreIndex.Int(), 10) + + containerInfo := d.podResourcesStore.GetContainerInfo(neuronCoreIndexString, neuronCoreResourceName) + if containerInfo == nil { + containerInfo = d.podResourcesStore.GetContainerInfo(neuronDeviceIndexString, neuronDeviceResourceName) + if containerInfo == nil { + // Alt resource name is to support backward compatibility in neuron monitor : https://awsdocs-neuron.readthedocs-hosted.com/en/latest/containers/tutorials/k8s-setup.html + containerInfo = d.podResourcesStore.GetContainerInfo(neuronDeviceIndexString, neuronDeviceResourceNameAlt) + } + } + attributes.PutStr(neuronDeviceAttributeKey, strconv.FormatInt(neuronDeviceIndex, 10)) + + if containerInfo != nil { + attributes.PutStr("ContainerName", containerInfo.ContainerName) + attributes.PutStr("PodName", containerInfo.PodName) + attributes.PutStr("Namespace", containerInfo.Namespace) + attributes.PutStr("FullPodname", containerInfo.PodName+"."+containerInfo.Namespace) + } + } else { + neuronDeviceIndex, neuronDeviceIndexPresent := attributes.Get(neuronDeviceAttributeKey) + neuronDeviceIndexString := strconv.FormatInt(neuronDeviceIndex.Int(), 10) + if neuronDeviceIndexPresent { + containerInfo := d.podResourcesStore.GetContainerInfo(neuronDeviceIndexString, neuronDeviceResourceName) + if containerInfo == nil { + // Alt resource name is to support backward compatibility in neuron monitor : https://awsdocs-neuron.readthedocs-hosted.com/en/latest/containers/tutorials/k8s-setup.html + containerInfo = d.podResourcesStore.GetContainerInfo(neuronDeviceIndexString, neuronDeviceResourceNameAlt) + } + + if containerInfo != nil { + attributes.PutStr("ContainerName", containerInfo.ContainerName) + attributes.PutStr("PodName", containerInfo.PodName) + attributes.PutStr("Namespace", containerInfo.Namespace) + attributes.PutStr("FullPodname", containerInfo.PodName+"."+containerInfo.Namespace) + } + } + } + + } +} diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go index b096e8d75327..52e732c4cd8e 100644 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go @@ -62,11 +62,17 @@ func NewSimplePromethuesScraper(opts SimplePromethuesScraperOpts, scraperConfig TelemetrySettings: opts.TelemetrySettings, } + podresourcesstore := stores.NewPodResourcesStore(opts.Logger) + podresourcesstore.AddResourceName("aws.amazon.com/neuroncore") + podresourcesstore.AddResourceName("aws.amazon.com/neuron") + podresourcesstore.AddResourceName("aws.amazon.com/neurondevice") + decoConsumer := decorateConsumer{ containerOrchestrator: ci.EKS, nextConsumer: opts.Consumer, k8sDecorator: opts.K8sDecorator, logger: opts.Logger, + metricModifier: *NewMetricModifier(opts.Logger, podresourcesstore), } promFactory := prometheusreceiver.NewFactory() @@ -92,13 +98,6 @@ func (ds *SimplePromethuesScraper) GetMetrics() []pmetric.Metrics { //ds.settings.Logger.Info("static_pod_resources staring scrapping") //stores.StartScraping(ds.settings.Logger) - podresourcesstore := stores.NewPodResourcesStore(ds.settings.Logger) - ds.settings.Logger.Info("Adding resources to PodResources") - podresourcesstore.AddResourceName("aws.amazon.com/neuroncore") - podresourcesstore.AddResourceName("aws.amazon.com/neuron") - podresourcesstore.AddResourceName("aws.amazon.com/neurondevice") - podresourcesstore.GetResourcesInfo("123", "123", "123") - if !ds.running { ds.settings.Logger.Info("The scraper is not running, starting up the scraper") err := ds.prometheusReceiver.Start(ds.ctx, ds.host) diff --git a/receiver/awscontainerinsightreceiver/internal/stores/podresourcesstore.go b/receiver/awscontainerinsightreceiver/internal/stores/podresourcesstore.go index de0c66a00569..5d3f1c96698b 100644 --- a/receiver/awscontainerinsightreceiver/internal/stores/podresourcesstore.go +++ b/receiver/awscontainerinsightreceiver/internal/stores/podresourcesstore.go @@ -25,9 +25,9 @@ var ( ) type ContainerInfo struct { - podName string - containerName string - namespace string + PodName string + ContainerName string + Namespace string } type ResourceInfo struct { @@ -128,9 +128,9 @@ func (p *PodResourcesStore) updateMaps() { for _, device := range container.GetDevices() { containerInfo := ContainerInfo{ - podName: pod.GetName(), - namespace: pod.GetNamespace(), - containerName: container.GetName(), + PodName: pod.GetName(), + Namespace: pod.GetNamespace(), + ContainerName: container.GetName(), } for _, deviceID := range device.GetDeviceIds() { @@ -143,7 +143,7 @@ func (p *PodResourcesStore) updateMaps() { p.containerInfoToResourcesMap[containerInfo] = append(p.containerInfoToResourcesMap[containerInfo], resourceInfo) p.resourceToPodContainerMap[resourceInfo] = containerInfo - p.logger.Info("/nContainerInfo : {" + containerInfo.namespace + "_" + containerInfo.podName + "_" + containerInfo.containerName + "}" + " -> ResourceInfo : {" + resourceInfo.resourceName + "_" + resourceInfo.deviceID + "_" + "}") + p.logger.Info("/nContainerInfo : {" + containerInfo.Namespace + "_" + containerInfo.PodName + "_" + containerInfo.ContainerName + "}" + " -> ResourceInfo : {" + resourceInfo.resourceName + "_" + resourceInfo.deviceID + "_" + "}") } } } @@ -160,7 +160,7 @@ func (p *PodResourcesStore) GetContainerInfo(deviceID string, resourceName strin } func (p *PodResourcesStore) GetResourcesInfo(podName string, containerName string, namespace string) *[]ResourceInfo { - key := ContainerInfo{podName: podName, containerName: containerName, namespace: namespace} + key := ContainerInfo{PodName: podName, ContainerName: containerName, Namespace: namespace} if resourceInfo, ok := p.containerInfoToResourcesMap[key]; ok { return &resourceInfo } @@ -174,7 +174,7 @@ func (p *PodResourcesStore) AddResourceName(resourceName string) { func (p *PodResourcesStore) PrintMaps() { p.logger.Info("containerInfoToResourcesMap:") for containerInfo, resourceInfos := range p.containerInfoToResourcesMap { - p.logger.Info("ContainerInfo-" + containerInfo.containerName + " ; " + containerInfo.podName) + p.logger.Info("ContainerInfo-" + containerInfo.ContainerName + " ; " + containerInfo.PodName) p.logger.Info("ResourceInfos:") for _, resourceInfo := range resourceInfos { p.logger.Info("ResourceInfo-" + resourceInfo.resourceName + " ; " + resourceInfo.deviceID) @@ -183,7 +183,7 @@ func (p *PodResourcesStore) PrintMaps() { p.logger.Info("\nresourceToPodContainerMap:") for resourceInfo, containerInfo := range p.resourceToPodContainerMap { p.logger.Info("ResourceInfo-" + resourceInfo.resourceName + " ; " + resourceInfo.deviceID) - p.logger.Info("ContainerInfo-" + containerInfo.containerName + " ; " + containerInfo.podName) + p.logger.Info("ContainerInfo-" + containerInfo.ContainerName + " ; " + containerInfo.PodName) } } From 05b1c7582ac671f7c4517b2b3a9e4471f2d80457 Mon Sep 17 00:00:00 2001 From: Samarth Singh Date: Tue, 5 Mar 2024 18:27:12 +0000 Subject: [PATCH 41/53] Unified the decorator and added podResources decorator --- .../internal/gpu/dcgmscraper.go | 16 +- .../internal/gpu/decorator.go | 140 ---------------- .../internal/gpu/metric_map.go | 19 +++ .../neuron/neuron_monitor_scraper_config.go | 10 +- .../neuron/neuron_monitor_scraper_test.go | 16 +- .../neuron/pod_attribute_decorator.go | 131 +++++++++++++++ .../internal/prometheusscraper/decorator.go | 154 +++--------------- .../decorator_test.go | 61 ++++--- .../prometheusscraper/metric_modifier.go | 77 --------- .../prometheus_scraper_testutils.go | 18 +- .../simple_prometheus_scraper.go | 44 ++--- .../simple_prometheus_scraper_test.go | 3 +- .../awscontainerinsightreceiver/receiver.go | 46 ++++-- 13 files changed, 294 insertions(+), 441 deletions(-) delete mode 100644 receiver/awscontainerinsightreceiver/internal/gpu/decorator.go create mode 100644 receiver/awscontainerinsightreceiver/internal/gpu/metric_map.go create mode 100644 receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator.go rename receiver/awscontainerinsightreceiver/internal/{gpu => prometheusscraper}/decorator_test.go (82%) delete mode 100644 receiver/awscontainerinsightreceiver/internal/prometheusscraper/metric_modifier.go diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go index fc0d332357a4..43d9149de451 100644 --- a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go +++ b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go @@ -21,6 +21,7 @@ import ( "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/prometheusscraper" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/prometheusreceiver" ) @@ -38,7 +39,7 @@ type DcgmScraper struct { host component.Host hostInfoProvider hostInfoProvider prometheusReceiver receiver.Metrics - k8sDecorator Decorator + k8sDecorator prometheusscraper.Decorator running bool } @@ -48,7 +49,7 @@ type DcgmScraperOpts struct { Consumer consumer.Metrics Host component.Host HostInfoProvider hostInfoProvider - K8sDecorator Decorator + K8sDecorator prometheusscraper.Decorator Logger *zap.Logger } @@ -78,11 +79,12 @@ func NewDcgmScraper(opts DcgmScraperOpts) (*DcgmScraper, error) { TelemetrySettings: opts.TelemetrySettings, } - decoConsumer := decorateConsumer{ - containerOrchestrator: ci.EKS, - nextConsumer: opts.Consumer, - k8sDecorator: opts.K8sDecorator, - logger: opts.Logger, + decoConsumer := prometheusscraper.DecorateConsumer{ + ContainerOrchestrator: ci.EKS, + NextConsumer: opts.Consumer, + K8sDecorator: opts.K8sDecorator, + MetricToUnitMap: metricToUnit, + Logger: opts.Logger, } promFactory := prometheusreceiver.NewFactory() diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/decorator.go b/receiver/awscontainerinsightreceiver/internal/gpu/decorator.go deleted file mode 100644 index ee3f9a8c36c1..000000000000 --- a/receiver/awscontainerinsightreceiver/internal/gpu/decorator.go +++ /dev/null @@ -1,140 +0,0 @@ -// Copyright The OpenTelemetry Authors -// SPDX-License-Identifier: Apache-2.0 - -package gpu - -import ( - "context" - - "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 ( - 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 metricToUnit = map[string]string{ - gpuUtil: "Percent", - gpuMemUtil: "Percent", - gpuMemUsed: "Bytes", - gpuMemTotal: "Bytes", - gpuTemperature: "None", - gpuPowerDraw: "None", -} - -// 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) - converted := ci.ConvertToFieldsAndTags(m, dc.logger) - var rcis []*stores.RawContainerInsightsMetric - for _, pair := range converted { - rcis = append(rcis, stores.NewRawContainerInsightsMetricWithData(ci.TypeGpuContainer, pair.Fields, pair.Tags, dc.logger)) - } - - decorated := dc.decorateMetrics(rcis) - dc.updateAttributes(m, decorated) - 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(rcis []*stores.RawContainerInsightsMetric) []*stores.RawContainerInsightsMetric { - var result []*stores.RawContainerInsightsMetric - if dc.containerOrchestrator != ci.EKS { - return result - } - for _, rci := range rcis { - // add tags for EKS - out := dc.k8sDecorator.Decorate(rci) - if out != nil { - result = append(result, out.(*stores.RawContainerInsightsMetric)) - } - } - return result -} - -func (dc *decorateConsumer) updateAttributes(m pmetric.Metric, rcis []*stores.RawContainerInsightsMetric) { - if len(rcis) == 0 { - 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() == 0 { - return - } - 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 := 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) - } - } -} - -func (dc *decorateConsumer) Shutdown() error { - if dc.k8sDecorator != nil { - return dc.k8sDecorator.Shutdown() - } - return nil -} diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/metric_map.go b/receiver/awscontainerinsightreceiver/internal/gpu/metric_map.go new file mode 100644 index 000000000000..08267206faa6 --- /dev/null +++ b/receiver/awscontainerinsightreceiver/internal/gpu/metric_map.go @@ -0,0 +1,19 @@ +package gpu + +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 metricToUnit = map[string]string{ + gpuUtil: "Percent", + gpuMemUtil: "Percent", + gpuMemUsed: "Bytes", + gpuMemTotal: "Bytes", + gpuTemperature: "None", + gpuPowerDraw: "None", +} diff --git a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go index 822bb0917bd9..95997208e504 100644 --- a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go +++ b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go @@ -1,7 +1,7 @@ // Copyright The OpenTelemetry Authors // SPDX-License-Identifier: Apache-2.0 -package nueron +package neuron import ( "time" @@ -20,7 +20,7 @@ const ( jobName = "containerInsightsNeuronMonitorScraper" ) -func GetNueronScrapeConfig(opts prometheusscraper.SimplePromethuesScraperOpts) *config.ScrapeConfig { +func GetNueronScrapeConfig(hostinfo prometheusscraper.HostInfoProvider) *config.ScrapeConfig { return &config.ScrapeConfig{ ScrapeInterval: model.Duration(collectionInterval), @@ -42,11 +42,11 @@ func GetNueronScrapeConfig(opts prometheusscraper.SimplePromethuesScraperOpts) * }, }, }, - MetricRelabelConfigs: GetNueronMetricRelabelConfigs(opts), + MetricRelabelConfigs: GetNueronMetricRelabelConfigs(hostinfo), } } -func GetNueronMetricRelabelConfigs(opts prometheusscraper.SimplePromethuesScraperOpts) []*relabel.Config { +func GetNueronMetricRelabelConfigs(hostinfo prometheusscraper.HostInfoProvider) []*relabel.Config { return []*relabel.Config{ { @@ -81,7 +81,7 @@ func GetNueronMetricRelabelConfigs(opts prometheusscraper.SimplePromethuesScrape SourceLabels: model.LabelNames{"instance_id"}, TargetLabel: "ClusterName", Regex: relabel.MustNewRegexp("(.*)"), - Replacement: opts.HostInfoProvider.GetClusterName(), + Replacement: hostinfo.GetClusterName(), Action: relabel.Replace, }, } diff --git a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go index c4d4e4b2b9ba..8ad50a58fede 100644 --- a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go +++ b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go @@ -1,11 +1,13 @@ -package nueron +package neuron import ( + "context" "strings" "testing" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/prometheusscraper" "github.com/stretchr/testify/assert" + "go.opentelemetry.io/collector/component/componenttest" ) const renameMetric = ` @@ -84,15 +86,21 @@ func TestNewNeuronScraperEndToEnd(t *testing.T) { ExpectedMetrics: expectedMetrics, } - mockedScraperOpts := prometheusscraper.GetMockedScraperOpts(consumer, mockHostInfoProvider{}) + mockedScraperOpts := prometheusscraper.SimplePromethuesScraperOpts{ + Ctx: context.TODO(), + TelemetrySettings: componenttest.NewNopTelemetrySettings(), + Consumer: consumer, + Host: componenttest.NewNopHost(), + ScraperConfigs: GetNueronScrapeConfig(mockHostInfoProvider{}), + HostInfoProvider: mockHostInfoProvider{}, + } prometheusscraper.TestSimplePrometheusEndToEnd(prometheusscraper.TestSimplePrometheusEndToEndOpts{ T: t, Consumer: consumer, DataReturned: renameMetric, ScraperOpts: mockedScraperOpts, - ScrapeConfig: GetNueronScrapeConfig(mockedScraperOpts), - MetricRelabelConfig: GetNueronMetricRelabelConfigs(mockedScraperOpts), + MetricRelabelConfig: GetNueronMetricRelabelConfigs(mockHostInfoProvider{}), }) } diff --git a/receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator.go b/receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator.go new file mode 100644 index 000000000000..52ad45694c74 --- /dev/null +++ b/receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator.go @@ -0,0 +1,131 @@ +package neuron + +import ( + "context" + "strconv" + + 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" +) + +const ( + neuronHardwareInfoKey = "neuron_hardware" + neuronCorePerDeviceKey = "neuroncore_per_device_count" + neuronCoreAttributeKey = "neuroncore" + neuronDeviceAttributeKey = "neuron_device_index" + neuronCoreResourceName = "aws.amazon.com/neuroncore" + neuronDeviceResourceName = "aws.amazon.com/neurondevice" + neuronDeviceResourceNameAlt = "aws.amazon.com/neuron" +) + +type PodAttributesDecoratorConsumer struct { + NextConsumer consumer.Metrics + PodResourcesStore *stores.PodResourcesStore // replace with podResourcesApi + Logger *zap.Logger +} + +func (pdc *PodAttributesDecoratorConsumer) Capabilities() consumer.Capabilities { + return consumer.Capabilities{ + MutatesData: true, + } +} + +func (pdc *PodAttributesDecoratorConsumer) ConsumeMetrics(ctx context.Context, md pmetric.Metrics) error { + md = pdc.neuronMetricsProcess(md) + return pdc.NextConsumer.ConsumeMetrics(ctx, md) +} + +func (pdc *PodAttributesDecoratorConsumer) neuronMetricsProcess(md pmetric.Metrics) pmetric.Metrics { + rms := md.ResourceMetrics() + for i := 0; i < rms.Len(); i++ { + rs := rms.At(i) + ilms := rs.ScopeMetrics() + for j := 0; j < ilms.Len(); j++ { + ils := ilms.At(j) + metrics := ils.Metrics() + + neuronHardwareInfo := pmetric.Metric{} + for k := 0; k < metrics.Len(); k++ { + m := metrics.At(k) + if m.Name() == neuronHardwareInfoKey { + neuronHardwareInfo = m + break + } + } + + neuronCoresPerDeviceValue, _ := neuronHardwareInfo.Sum().DataPoints().At(0).Attributes().Get(neuronCorePerDeviceKey) + neuronCoresPerDevice, _ := strconv.Atoi(neuronCoresPerDeviceValue.AsString()) + + for k := 0; k < metrics.Len(); k++ { + m := metrics.At(k) + pdc.AddPodCorrelationAttributes(getMetricDatapoints(m), neuronCoresPerDevice) + } + } + } + return md +} + +func getMetricDatapoints(m pmetric.Metric) pmetric.NumberDataPointSlice { + switch m.Type() { + case pmetric.MetricTypeGauge: + return m.Gauge().DataPoints() + case pmetric.MetricTypeSum: + return m.Sum().DataPoints() + default: + return pmetric.NewNumberDataPointSlice() + } +} + +func (pdc *PodAttributesDecoratorConsumer) AddPodCorrelationAttributes(metricDatapoints pmetric.NumberDataPointSlice, neuronCoresPerDevice int) { + for i := 0; i < metricDatapoints.Len(); i++ { + attributes := metricDatapoints.At(i).Attributes() + var containerInfo *stores.ContainerInfo + + if neuronDeviceIndex, neuronDeviceIndexPresent := attributes.Get(neuronDeviceAttributeKey); neuronDeviceIndexPresent { + // get container info from neuronDeviceIndex + neuronDeviceIndex := neuronDeviceIndex.AsString() + if neuronDeviceIndexPresent { + containerInfo = pdc.getContainerInfoForNueronDeviceIndex(neuronDeviceIndex) + + } + } else if neuronCoreIndex, neuronCoreIndexPresent := attributes.Get(neuronCoreAttributeKey); neuronCoreIndexPresent { + // get container info from neuronCore + containerInfo = pdc.PodResourcesStore.GetContainerInfo(neuronCoreIndex.AsString(), neuronCoreResourceName) + neuronDeviceIndex := getNeuronDeviceIndexFromCoreAttribute(neuronCoreIndex, neuronCoresPerDevice) + if containerInfo == nil { + // else get container info from calculated neuronDeviceIndex + containerInfo = pdc.getContainerInfoForNueronDeviceIndex(neuronDeviceIndex) + } + attributes.PutStr(neuronDeviceAttributeKey, neuronDeviceIndex) + } + populateAttributes(&attributes, containerInfo) + } +} + +func (pdc *PodAttributesDecoratorConsumer) getContainerInfoForNueronDeviceIndex(neuronDeviceIndex string) *stores.ContainerInfo { + containerInfo := pdc.PodResourcesStore.GetContainerInfo(neuronDeviceIndex, neuronDeviceResourceName) + if containerInfo == nil { + // Alt resource name is to support backward compatibility in neuron monitor : https://awsdocs-neuron.readthedocs-hosted.com/en/latest/containers/tutorials/k8s-setup.html + containerInfo = pdc.PodResourcesStore.GetContainerInfo(neuronDeviceIndex, neuronDeviceResourceNameAlt) + } + return containerInfo +} + +func populateAttributes(attributes *pcommon.Map, containerInfo *stores.ContainerInfo) { + if containerInfo != nil { + attributes.PutStr(ci.AttributeContainerName, containerInfo.ContainerName) + attributes.PutStr(ci.AttributeK8sPodName, containerInfo.PodName) + attributes.PutStr(ci.AttributePodName, containerInfo.PodName) + attributes.PutStr(ci.AttributeK8sNamespace, containerInfo.Namespace) + attributes.PutStr(ci.AttributeFullPodName, containerInfo.PodName+"."+containerInfo.Namespace) + } +} + +func getNeuronDeviceIndexFromCoreAttribute(neuronCoreIndex pcommon.Value, neuronCoresPerDevice int) string { + neuronCoreIndexIntVal, _ := strconv.Atoi(neuronCoreIndex.AsString()) + return strconv.Itoa(neuronCoreIndexIntVal / neuronCoresPerDevice) +} diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator.go index 75c7f598a3c0..ba6be6134b80 100644 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator.go +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator.go @@ -5,8 +5,6 @@ package prometheusscraper import ( "context" - "fmt" - "strings" "go.opentelemetry.io/collector/consumer" "go.opentelemetry.io/collector/pdata/pcommon" @@ -17,42 +15,23 @@ import ( "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" ) -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 metricToUnit = map[string]string{ - gpuUtil: "Percent", - gpuMemUtil: "Percent", - gpuMemUsed: "Bytes", - gpuMemTotal: "Bytes", - gpuTemperature: "None", - gpuPowerDraw: "None", +// 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 + MetricToUnitMap map[string]string + Logger *zap.Logger } -// 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 - metricModifier MetricModifier - logger *zap.Logger -} - -func (dc *decorateConsumer) Capabilities() consumer.Capabilities { +func (dc *DecorateConsumer) Capabilities() consumer.Capabilities { return consumer.Capabilities{ MutatesData: true, } } -func (dc *decorateConsumer) ConsumeMetrics(ctx context.Context, md pmetric.Metrics) error { +func (dc *DecorateConsumer) ConsumeMetrics(ctx context.Context, md pmetric.Metrics) error { resourceTags := make(map[string]string) - md, _ = neuronMetricsProcess(md, &dc.metricModifier) rms := md.ResourceMetrics() for i := 0; i < rms.Len(); i++ { // get resource attributes @@ -66,22 +45,21 @@ 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) - converted := ci.ConvertToFieldsAndTags(m, dc.logger) + converted := ci.ConvertToFieldsAndTags(m, dc.Logger) var rcis []*stores.RawContainerInsightsMetric for _, pair := range converted { - rcis = append(rcis, stores.NewRawContainerInsightsMetricWithData(ci.TypeGpuContainer, pair.Fields, pair.Tags, dc.logger)) + rcis = append(rcis, stores.NewRawContainerInsightsMetricWithData(ci.TypeGpuContainer, pair.Fields, pair.Tags, dc.Logger)) } decorated := dc.decorateMetrics(rcis) dc.updateAttributes(m, decorated) - if unit, ok := metricToUnit[m.Name()]; ok { + if unit, ok := dc.MetricToUnitMap[m.Name()]; ok { m.SetUnit(unit) } } } } - dc.logMd(md) - return dc.nextConsumer.ConsumeMetrics(ctx, md) + return dc.NextConsumer.ConsumeMetrics(ctx, md) } type Decorator interface { @@ -89,14 +67,14 @@ type Decorator interface { Shutdown() error } -func (dc *decorateConsumer) decorateMetrics(rcis []*stores.RawContainerInsightsMetric) []*stores.RawContainerInsightsMetric { +func (dc *DecorateConsumer) decorateMetrics(rcis []*stores.RawContainerInsightsMetric) []*stores.RawContainerInsightsMetric { var result []*stores.RawContainerInsightsMetric - if dc.containerOrchestrator != ci.EKS { + if dc.ContainerOrchestrator != ci.EKS { return result } for _, rci := range rcis { // add tags for EKS - out := dc.k8sDecorator.Decorate(rci) + out := dc.K8sDecorator.Decorate(rci) if out != nil { result = append(result, out.(*stores.RawContainerInsightsMetric)) } @@ -104,7 +82,7 @@ func (dc *decorateConsumer) decorateMetrics(rcis []*stores.RawContainerInsightsM return result } -func (dc *decorateConsumer) updateAttributes(m pmetric.Metric, rcis []*stores.RawContainerInsightsMetric) { +func (dc *DecorateConsumer) updateAttributes(m pmetric.Metric, rcis []*stores.RawContainerInsightsMetric) { if len(rcis) == 0 { return } @@ -115,7 +93,7 @@ func (dc *decorateConsumer) updateAttributes(m pmetric.Metric, rcis []*stores.Ra 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())) + dc.Logger.Warn("Unsupported metric type", zap.String("metric", m.Name()), zap.String("type", m.Type().String())) } if dps.Len() == 0 { return @@ -137,99 +115,9 @@ func (dc *decorateConsumer) updateAttributes(m pmetric.Metric, rcis []*stores.Ra } } -func (dc *decorateConsumer) Shutdown() error { - if dc.k8sDecorator != nil { - return dc.k8sDecorator.Shutdown() +func (dc *DecorateConsumer) Shutdown() error { + if dc.K8sDecorator != nil { + return dc.K8sDecorator.Shutdown() } return nil } -func (dc *decorateConsumer) logMd(md pmetric.Metrics) { - var logMessage strings.Builder - - logMessage.WriteString("\"METRICS_MD\" : {\n") - rms := md.ResourceMetrics() - for i := 0; i < rms.Len(); i++ { - rs := rms.At(i) - ilms := rs.ScopeMetrics() - logMessage.WriteString(fmt.Sprintf("\t\"ResourceMetric_%d\": {\n", i)) - for j := 0; j < ilms.Len(); j++ { - ils := ilms.At(j) - metrics := ils.Metrics() - logMessage.WriteString(fmt.Sprintf("\t\t\"ScopeMetric_%d\": {\n", j)) - logMessage.WriteString(fmt.Sprintf("\t\t\"Metrics_%d\": [\n", j)) - - for k := 0; k < metrics.Len(); k++ { - m := metrics.At(k) - logMessage.WriteString(fmt.Sprintf("\t\t\t\"Metric_%d\": {\n", k)) - logMessage.WriteString(fmt.Sprintf("\t\t\t\t\"name\": \"%s\",\n", m.Name())) - - var datapoints pmetric.NumberDataPointSlice - switch m.Type() { - case pmetric.MetricTypeGauge: - datapoints = m.Gauge().DataPoints() - case pmetric.MetricTypeSum: - datapoints = m.Sum().DataPoints() - default: - datapoints = pmetric.NewNumberDataPointSlice() - } - - logMessage.WriteString("\t\t\t\t\"datapoints\": [\n") - for yu := 0; yu < datapoints.Len(); yu++ { - logMessage.WriteString("\t\t\t\t\t{\n") - logMessage.WriteString(fmt.Sprintf("\t\t\t\t\t\t\"attributes\": \"%v\",\n", datapoints.At(yu).Attributes().AsRaw())) - logMessage.WriteString(fmt.Sprintf("\t\t\t\t\t\t\"value\": %v,\n", datapoints.At(yu).DoubleValue())) - logMessage.WriteString("\t\t\t\t\t},\n") - } - logMessage.WriteString("\t\t\t\t],\n") - logMessage.WriteString("\t\t\t},\n") - } - logMessage.WriteString("\t\t],\n") - logMessage.WriteString("\t\t},\n") - } - logMessage.WriteString("\t},\n") - } - logMessage.WriteString("},\n") - - dc.logger.Info(logMessage.String()) -} - -func neuronMetricsProcess(md pmetric.Metrics, modifier *MetricModifier) (pmetric.Metrics, error) { - rms := md.ResourceMetrics() - for i := 0; i < rms.Len(); i++ { - rs := rms.At(i) - ilms := rs.ScopeMetrics() - for j := 0; j < ilms.Len(); j++ { - ils := ilms.At(j) - metrics := ils.Metrics() - - // neuronHardwareInfo := pmetric.Metric{} - // for k := 0; k < metrics.Len(); k++ { - // m := metrics.At(k) - // if m.Name() == "neuroncore_per_device_count" { - // neuronHardwareInfo = m - // break - // } - // } - - // neuronCoresPerDeviceValue, _ := neuronHardwareInfo.Gauge().DataPoints().At(0).Attributes().Get(neuronCorePerDeviceKey) - // neuronCoresPerDevice := neuronCoresPerDeviceValue.Int() - - for k := 0; k < metrics.Len(); k++ { - m := metrics.At(k) - modifier.AddPodCorrelationAttributes(getMetricDatapoints(m), 2) // need to change this - } - } - } - return md, nil -} - -func getMetricDatapoints(m pmetric.Metric) pmetric.NumberDataPointSlice { - switch m.Type() { - case pmetric.MetricTypeGauge: - return m.Gauge().DataPoints() - case pmetric.MetricTypeSum: - return m.Sum().DataPoints() - default: - return pmetric.NewNumberDataPointSlice() - } -} diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/decorator_test.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator_test.go similarity index 82% rename from receiver/awscontainerinsightreceiver/internal/gpu/decorator_test.go rename to receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator_test.go index a81e39268b65..e5107ee29eaf 100644 --- a/receiver/awscontainerinsightreceiver/internal/gpu/decorator_test.go +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator_test.go @@ -1,7 +1,7 @@ // Copyright The OpenTelemetry Authors // SPDX-License-Identifier: Apache-2.0 -package gpu +package prometheusscraper import ( "context" @@ -30,13 +30,32 @@ func (m *MockK8sDecorator) Shutdown() error { return nil } -func TestConsumeMetrics(t *testing.T) { +const ( + util = "UTIL" + memUtil = "USED_PERCENT" + memUsed = "FB_USED" + memTotal = "FB_TOTAL" + temp = "TEMP" + powerDraw = "POWER_USAGE" +) + +var metricToUnit = map[string]string{ + util: "Percent", + memUtil: "Percent", + memUsed: "Bytes", + memTotal: "Bytes", + temp: "None", + powerDraw: "None", +} + +func TestConsumeMetricsForGpu(t *testing.T) { logger, _ := zap.NewDevelopment() - dc := &decorateConsumer{ - containerOrchestrator: "EKS", - nextConsumer: consumertest.NewNop(), - k8sDecorator: &MockK8sDecorator{}, - logger: logger, + dc := &DecorateConsumer{ + ContainerOrchestrator: "EKS", + NextConsumer: consumertest.NewNop(), + K8sDecorator: &MockK8sDecorator{}, + MetricToUnitMap: metricToUnit, + Logger: logger, } ctx := context.Background() @@ -52,47 +71,47 @@ func TestConsumeMetrics(t *testing.T) { }, "unit": { metrics: generateMetrics(map[string]map[string]string{ - gpuUtil: { + util: { "device": "test0", }, - gpuMemUtil: { + memUtil: { "device": "test0", }, - gpuMemTotal: { + memTotal: { "device": "test0", }, - gpuMemUsed: { + memUsed: { "device": "test0", }, - gpuPowerDraw: { + powerDraw: { "device": "test0", }, - gpuTemperature: { + temp: { "device": "test0", }, }), want: generateMetrics(map[string]map[string]string{ - gpuUtil: { + util: { "device": "test0", "Unit": "Percent", }, - gpuMemUtil: { + memUtil: { "device": "test0", "Unit": "Percent", }, - gpuMemTotal: { + memTotal: { "device": "test0", "Unit": "Bytes", }, - gpuMemUsed: { + memUsed: { "device": "test0", "Unit": "Bytes", }, - gpuPowerDraw: { + powerDraw: { "device": "test0", "Unit": "None", }, - gpuTemperature: { + temp: { "device": "test0", "Unit": "None", }, @@ -114,13 +133,13 @@ func TestConsumeMetrics(t *testing.T) { }, "typeUnchanged": { metrics: generateMetrics(map[string]map[string]string{ - gpuUtil: { + util: { "device": "test0", "Type": "TestType", }, }), want: generateMetrics(map[string]map[string]string{ - gpuUtil: { + util: { "device": "test0", "Type": "TestType", "Unit": "Percent", diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/metric_modifier.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/metric_modifier.go deleted file mode 100644 index 42dfa580cc7b..000000000000 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/metric_modifier.go +++ /dev/null @@ -1,77 +0,0 @@ -package prometheusscraper - -import ( - "strconv" - - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" - "go.opentelemetry.io/collector/pdata/pmetric" - "go.uber.org/zap" -) - -const ( - neuronCoreAttributeKey = "neuroncore" - neuronDeviceAttributeKey = "neuron_device_index" - neuronCoreResourceName = "aws.amazon.com/neuroncore" - neuronDeviceResourceName = "aws.amazon.com/neurondevice" - neuronDeviceResourceNameAlt = "aws.amazon.com/neuron" -) - -type MetricModifier struct { - logger *zap.Logger - podResourcesStore *stores.PodResourcesStore // replace with podResourcesApi -} - -func NewMetricModifier(logger *zap.Logger, podResourcesStore *stores.PodResourcesStore) *MetricModifier { - d := &MetricModifier{ - logger: logger, - podResourcesStore: podResourcesStore, - } - return d -} - -func (d *MetricModifier) AddPodCorrelationAttributes(metricDatapoints pmetric.NumberDataPointSlice, neuronCoresPerDevice int64) { - for i := 0; i < metricDatapoints.Len(); i++ { - attributes := metricDatapoints.At(i).Attributes() - neuronCoreIndex, neuronCoreIndexPresent := attributes.Get(neuronCoreAttributeKey) - if neuronCoreIndexPresent { - neuronDeviceIndex := neuronCoreIndex.Int() / neuronCoresPerDevice - neuronDeviceIndexString := strconv.FormatInt(neuronDeviceIndex, 10) - neuronCoreIndexString := strconv.FormatInt(neuronCoreIndex.Int(), 10) - - containerInfo := d.podResourcesStore.GetContainerInfo(neuronCoreIndexString, neuronCoreResourceName) - if containerInfo == nil { - containerInfo = d.podResourcesStore.GetContainerInfo(neuronDeviceIndexString, neuronDeviceResourceName) - if containerInfo == nil { - // Alt resource name is to support backward compatibility in neuron monitor : https://awsdocs-neuron.readthedocs-hosted.com/en/latest/containers/tutorials/k8s-setup.html - containerInfo = d.podResourcesStore.GetContainerInfo(neuronDeviceIndexString, neuronDeviceResourceNameAlt) - } - } - attributes.PutStr(neuronDeviceAttributeKey, strconv.FormatInt(neuronDeviceIndex, 10)) - - if containerInfo != nil { - attributes.PutStr("ContainerName", containerInfo.ContainerName) - attributes.PutStr("PodName", containerInfo.PodName) - attributes.PutStr("Namespace", containerInfo.Namespace) - attributes.PutStr("FullPodname", containerInfo.PodName+"."+containerInfo.Namespace) - } - } else { - neuronDeviceIndex, neuronDeviceIndexPresent := attributes.Get(neuronDeviceAttributeKey) - neuronDeviceIndexString := strconv.FormatInt(neuronDeviceIndex.Int(), 10) - if neuronDeviceIndexPresent { - containerInfo := d.podResourcesStore.GetContainerInfo(neuronDeviceIndexString, neuronDeviceResourceName) - if containerInfo == nil { - // Alt resource name is to support backward compatibility in neuron monitor : https://awsdocs-neuron.readthedocs-hosted.com/en/latest/containers/tutorials/k8s-setup.html - containerInfo = d.podResourcesStore.GetContainerInfo(neuronDeviceIndexString, neuronDeviceResourceNameAlt) - } - - if containerInfo != nil { - attributes.PutStr("ContainerName", containerInfo.ContainerName) - attributes.PutStr("PodName", containerInfo.PodName) - attributes.PutStr("Namespace", containerInfo.Namespace) - attributes.PutStr("FullPodname", containerInfo.PodName+"."+containerInfo.Namespace) - } - } - } - - } -} diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/prometheus_scraper_testutils.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/prometheus_scraper_testutils.go index 7d8c730cacac..7b2932b1806c 100644 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/prometheus_scraper_testutils.go +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/prometheus_scraper_testutils.go @@ -14,11 +14,9 @@ import ( "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" ) type MetricLabel struct { @@ -36,7 +34,6 @@ type TestSimplePrometheusEndToEndOpts struct { Consumer consumer.Metrics DataReturned string ScraperOpts SimplePromethuesScraperOpts - ScrapeConfig *config.ScrapeConfig MetricRelabelConfig []*relabel.Config } @@ -78,7 +75,7 @@ func (m MockConsumer) ConsumeMetrics(_ context.Context, md pmetric.Metrics) erro } func TestSimplePrometheusEndToEnd(opts TestSimplePrometheusEndToEndOpts) { - scraper, err := NewSimplePromethuesScraper(opts.ScraperOpts, opts.ScrapeConfig) + scraper, err := NewSimplePromethuesScraper(opts.ScraperOpts) assert.NoError(opts.T, err) // build up a new PR @@ -151,16 +148,3 @@ func TestSimplePrometheusEndToEnd(opts TestSimplePrometheusEndToEndOpts) { mp.Wg.Wait() mp.Wg.Wait() } - -func GetMockedScraperOpts(consumer consumer.Metrics, mockHostInfoProvider hostInfoProvider) SimplePromethuesScraperOpts { - settings := componenttest.NewNopTelemetrySettings() - settings.Logger, _ = zap.NewDevelopment() - - return SimplePromethuesScraperOpts{ - Ctx: context.TODO(), - TelemetrySettings: settings, - Consumer: consumer, - Host: componenttest.NewNopHost(), - HostInfoProvider: mockHostInfoProvider, - } -} diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go index 52e732c4cd8e..946d251b3505 100644 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go @@ -5,8 +5,6 @@ import ( "errors" "fmt" - ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/prometheusreceiver" "github.com/prometheus/prometheus/config" "go.opentelemetry.io/collector/component" @@ -20,7 +18,7 @@ type SimplePromethuesScraper struct { ctx context.Context settings component.TelemetrySettings host component.Host - hostInfoProvider hostInfoProvider + hostInfoProvider HostInfoProvider prometheusReceiver receiver.Metrics running bool } @@ -30,18 +28,17 @@ type SimplePromethuesScraperOpts struct { TelemetrySettings component.TelemetrySettings Consumer consumer.Metrics Host component.Host - HostInfoProvider hostInfoProvider + HostInfoProvider HostInfoProvider ScraperConfigs *config.ScrapeConfig - K8sDecorator Decorator Logger *zap.Logger } -type hostInfoProvider interface { +type HostInfoProvider interface { GetClusterName() string GetInstanceID() string } -func NewSimplePromethuesScraper(opts SimplePromethuesScraperOpts, scraperConfig *config.ScrapeConfig) (*SimplePromethuesScraper, error) { +func NewSimplePromethuesScraper(opts SimplePromethuesScraperOpts) (*SimplePromethuesScraper, error) { if opts.Consumer == nil { return nil, errors.New("consumer cannot be nil") } @@ -54,7 +51,7 @@ func NewSimplePromethuesScraper(opts SimplePromethuesScraperOpts, scraperConfig promConfig := prometheusreceiver.Config{ PrometheusConfig: &config.Config{ - ScrapeConfigs: []*config.ScrapeConfig{scraperConfig}, + ScrapeConfigs: []*config.ScrapeConfig{opts.ScraperConfigs}, }, } @@ -62,21 +59,26 @@ func NewSimplePromethuesScraper(opts SimplePromethuesScraperOpts, scraperConfig TelemetrySettings: opts.TelemetrySettings, } - podresourcesstore := stores.NewPodResourcesStore(opts.Logger) - podresourcesstore.AddResourceName("aws.amazon.com/neuroncore") - podresourcesstore.AddResourceName("aws.amazon.com/neuron") - podresourcesstore.AddResourceName("aws.amazon.com/neurondevice") - - decoConsumer := decorateConsumer{ - containerOrchestrator: ci.EKS, - nextConsumer: opts.Consumer, - k8sDecorator: opts.K8sDecorator, - logger: opts.Logger, - metricModifier: *NewMetricModifier(opts.Logger, podresourcesstore), - } + // podresourcesstore := stores.NewPodResourcesStore(opts.Logger) + // podresourcesstore.AddResourceName("aws.amazon.com/neuroncore") + // podresourcesstore.AddResourceName("aws.amazon.com/neuron") + // podresourcesstore.AddResourceName("aws.amazon.com/neurondevice") + + // decoConsumer := decorateConsumer{ + // containerOrchestrator: ci.EKS, + // nextConsumer: opts.Consumer, + // k8sDecorator: opts.K8sDecorator, + // logger: opts.Logger, + // } + + // pod_att_consumer := neuron.PodAttributesDecoratorConsumer{ + // nextConsumer: &decoConsumer, + // podResourcesStore: podresourcesstore, + // logger: opts.Logger, + // } promFactory := prometheusreceiver.NewFactory() - promReceiver, err := promFactory.CreateMetricsReceiver(opts.Ctx, params, &promConfig, &decoConsumer) + promReceiver, err := promFactory.CreateMetricsReceiver(opts.Ctx, params, &promConfig, opts.Consumer) if err != nil { return nil, fmt.Errorf("failed to create prometheus receiver: %w", err) } diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper_test.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper_test.go index 07e2dfa63cb2..106c57f04bb6 100644 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper_test.go +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper_test.go @@ -4,7 +4,6 @@ import ( "context" "testing" - "github.com/prometheus/prometheus/config" "github.com/stretchr/testify/assert" "go.opentelemetry.io/collector/component/componenttest" "go.uber.org/zap" @@ -50,7 +49,7 @@ func TestSimplePrometheusScraperBadInputs(t *testing.T) { } for _, tt := range tests { - scraper, err := NewSimplePromethuesScraper(tt, &config.ScrapeConfig{}) + scraper, err := NewSimplePromethuesScraper(tt) assert.Error(t, err) assert.Nil(t, scraper) diff --git a/receiver/awscontainerinsightreceiver/receiver.go b/receiver/awscontainerinsightreceiver/receiver.go index b3ed6152fc4a..768ae2cda877 100644 --- a/receiver/awscontainerinsightreceiver/receiver.go +++ b/receiver/awscontainerinsightreceiver/receiver.go @@ -9,7 +9,7 @@ import ( "time" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/gpu" - nueron "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/neuron" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/neuron" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/prometheusscraper" "go.opentelemetry.io/collector/component" "go.opentelemetry.io/collector/consumer" @@ -105,17 +105,7 @@ func (acir *awsContainerInsightReceiver) Start(ctx context.Context, host compone acir.settings.Logger.Debug("Unable to start dcgm scraper", zap.Error(err)) } - simplePrometheusScraperOpts := prometheusscraper.SimplePromethuesScraperOpts{ - Ctx: ctx, - TelemetrySettings: acir.settings, - Consumer: acir.nextConsumer, - Host: host, - HostInfoProvider: hostinfo, - K8sDecorator: k8sDecorator, - Logger: acir.settings.Logger, - } - - err = acir.initNeuronScraper(simplePrometheusScraperOpts) + err = acir.initNeuronScraper(ctx, host, hostinfo, k8sDecorator) if err != nil { acir.settings.Logger.Debug("Unable to start neuron scraper", zap.Error(err)) } @@ -211,13 +201,41 @@ func (acir *awsContainerInsightReceiver) initDcgmScraper(ctx context.Context, ho return err } -func (acir *awsContainerInsightReceiver) initNeuronScraper(opts prometheusscraper.SimplePromethuesScraperOpts) error { +func (acir *awsContainerInsightReceiver) initNeuronScraper(ctx context.Context, host component.Host, hostinfo *hostInfo.Info, decorator *stores.K8sDecorator) error { // if !acir.config.EnableNeuronMetric { // return nil // } + decoConsumer := prometheusscraper.DecorateConsumer{ + ContainerOrchestrator: ci.EKS, + NextConsumer: acir.nextConsumer, + K8sDecorator: decorator, + Logger: acir.settings.Logger, + } + + podresourcesstore := stores.NewPodResourcesStore(acir.settings.Logger) + podresourcesstore.AddResourceName("aws.amazon.com/neuroncore") + podresourcesstore.AddResourceName("aws.amazon.com/neuron") + podresourcesstore.AddResourceName("aws.amazon.com/neurondevice") + + podAttributesDecoratorConsumer := neuron.PodAttributesDecoratorConsumer{ + NextConsumer: &decoConsumer, + PodResourcesStore: podresourcesstore, + Logger: acir.settings.Logger, + } + + scraperOpts := prometheusscraper.SimplePromethuesScraperOpts{ + Ctx: ctx, + TelemetrySettings: acir.settings, + Consumer: &podAttributesDecoratorConsumer, + Host: host, + ScraperConfigs: neuron.GetNueronScrapeConfig(hostinfo), + HostInfoProvider: hostinfo, + Logger: acir.settings.Logger, + } + var err error - acir.neuronMonitorScraper, err = prometheusscraper.NewSimplePromethuesScraper(opts, nueron.GetNueronScrapeConfig(opts)) + acir.neuronMonitorScraper, err = prometheusscraper.NewSimplePromethuesScraper(scraperOpts) return err } From 76e05aa1f75c889f487ea0338060135e8fcf33d9 Mon Sep 17 00:00:00 2001 From: Samarth Singh Date: Tue, 5 Mar 2024 18:45:44 +0000 Subject: [PATCH 42/53] Minor fixes --- .../neuron/neuron_monitor_scraper_config.go | 6 ++-- .../neuron/neuron_monitor_scraper_test.go | 6 ++-- .../prometheus_scraper_testutils.go | 4 +-- .../simple_prometheus_scraper.go | 34 ++++--------------- .../simple_prometheus_scraper_test.go | 4 +-- .../awscontainerinsightreceiver/receiver.go | 8 ++--- 6 files changed, 20 insertions(+), 42 deletions(-) diff --git a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go index 95997208e504..8660c8379218 100644 --- a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go +++ b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go @@ -20,7 +20,7 @@ const ( jobName = "containerInsightsNeuronMonitorScraper" ) -func GetNueronScrapeConfig(hostinfo prometheusscraper.HostInfoProvider) *config.ScrapeConfig { +func GetNeuronScrapeConfig(hostinfo prometheusscraper.HostInfoProvider) *config.ScrapeConfig { return &config.ScrapeConfig{ ScrapeInterval: model.Duration(collectionInterval), @@ -42,11 +42,11 @@ func GetNueronScrapeConfig(hostinfo prometheusscraper.HostInfoProvider) *config. }, }, }, - MetricRelabelConfigs: GetNueronMetricRelabelConfigs(hostinfo), + MetricRelabelConfigs: GetNeuronMetricRelabelConfigs(hostinfo), } } -func GetNueronMetricRelabelConfigs(hostinfo prometheusscraper.HostInfoProvider) []*relabel.Config { +func GetNeuronMetricRelabelConfigs(hostinfo prometheusscraper.HostInfoProvider) []*relabel.Config { return []*relabel.Config{ { diff --git a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go index 8ad50a58fede..d6c61619ac1d 100644 --- a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go +++ b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go @@ -86,12 +86,12 @@ func TestNewNeuronScraperEndToEnd(t *testing.T) { ExpectedMetrics: expectedMetrics, } - mockedScraperOpts := prometheusscraper.SimplePromethuesScraperOpts{ + mockedScraperOpts := prometheusscraper.SimplePrometheusScraperOpts{ Ctx: context.TODO(), TelemetrySettings: componenttest.NewNopTelemetrySettings(), Consumer: consumer, Host: componenttest.NewNopHost(), - ScraperConfigs: GetNueronScrapeConfig(mockHostInfoProvider{}), + ScraperConfigs: GetNeuronScrapeConfig(mockHostInfoProvider{}), HostInfoProvider: mockHostInfoProvider{}, } @@ -100,7 +100,7 @@ func TestNewNeuronScraperEndToEnd(t *testing.T) { Consumer: consumer, DataReturned: renameMetric, ScraperOpts: mockedScraperOpts, - MetricRelabelConfig: GetNueronMetricRelabelConfigs(mockHostInfoProvider{}), + MetricRelabelConfig: GetNeuronMetricRelabelConfigs(mockHostInfoProvider{}), }) } diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/prometheus_scraper_testutils.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/prometheus_scraper_testutils.go index 7b2932b1806c..26286240a468 100644 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/prometheus_scraper_testutils.go +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/prometheus_scraper_testutils.go @@ -33,7 +33,7 @@ type TestSimplePrometheusEndToEndOpts struct { T *testing.T Consumer consumer.Metrics DataReturned string - ScraperOpts SimplePromethuesScraperOpts + ScraperOpts SimplePrometheusScraperOpts MetricRelabelConfig []*relabel.Config } @@ -75,7 +75,7 @@ func (m MockConsumer) ConsumeMetrics(_ context.Context, md pmetric.Metrics) erro } func TestSimplePrometheusEndToEnd(opts TestSimplePrometheusEndToEndOpts) { - scraper, err := NewSimplePromethuesScraper(opts.ScraperOpts) + scraper, err := NewSimplePrometheusScraper(opts.ScraperOpts) assert.NoError(opts.T, err) // build up a new PR diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go index 946d251b3505..29b52b605eda 100644 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go @@ -14,7 +14,7 @@ import ( "go.uber.org/zap" ) -type SimplePromethuesScraper struct { +type SimplePrometheusScraper struct { ctx context.Context settings component.TelemetrySettings host component.Host @@ -23,7 +23,7 @@ type SimplePromethuesScraper struct { running bool } -type SimplePromethuesScraperOpts struct { +type SimplePrometheusScraperOpts struct { Ctx context.Context TelemetrySettings component.TelemetrySettings Consumer consumer.Metrics @@ -38,7 +38,7 @@ type HostInfoProvider interface { GetInstanceID() string } -func NewSimplePromethuesScraper(opts SimplePromethuesScraperOpts) (*SimplePromethuesScraper, error) { +func NewSimplePrometheusScraper(opts SimplePrometheusScraperOpts) (*SimplePrometheusScraper, error) { if opts.Consumer == nil { return nil, errors.New("consumer cannot be nil") } @@ -59,31 +59,13 @@ func NewSimplePromethuesScraper(opts SimplePromethuesScraperOpts) (*SimplePromet TelemetrySettings: opts.TelemetrySettings, } - // podresourcesstore := stores.NewPodResourcesStore(opts.Logger) - // podresourcesstore.AddResourceName("aws.amazon.com/neuroncore") - // podresourcesstore.AddResourceName("aws.amazon.com/neuron") - // podresourcesstore.AddResourceName("aws.amazon.com/neurondevice") - - // decoConsumer := decorateConsumer{ - // containerOrchestrator: ci.EKS, - // nextConsumer: opts.Consumer, - // k8sDecorator: opts.K8sDecorator, - // logger: opts.Logger, - // } - - // pod_att_consumer := neuron.PodAttributesDecoratorConsumer{ - // nextConsumer: &decoConsumer, - // podResourcesStore: podresourcesstore, - // logger: opts.Logger, - // } - 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 &SimplePromethuesScraper{ + return &SimplePrometheusScraper{ ctx: opts.Ctx, settings: opts.TelemetrySettings, host: opts.Host, @@ -92,14 +74,10 @@ func NewSimplePromethuesScraper(opts SimplePromethuesScraperOpts) (*SimplePromet }, nil } -func (ds *SimplePromethuesScraper) GetMetrics() []pmetric.Metrics { +func (ds *SimplePrometheusScraper) 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 - // this thing works, now just fixing the podresourcestore - //ds.settings.Logger.Info("static_pod_resources staring scrapping") - //stores.StartScraping(ds.settings.Logger) - if !ds.running { ds.settings.Logger.Info("The scraper is not running, starting up the scraper") err := ds.prometheusReceiver.Start(ds.ctx, ds.host) @@ -111,7 +89,7 @@ func (ds *SimplePromethuesScraper) GetMetrics() []pmetric.Metrics { return nil } -func (ds *SimplePromethuesScraper) Shutdown() { +func (ds *SimplePrometheusScraper) Shutdown() { if ds.running { err := ds.prometheusReceiver.Shutdown(ds.ctx) if err != nil { diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper_test.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper_test.go index 106c57f04bb6..48fd1b04ab2f 100644 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper_test.go +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper_test.go @@ -24,7 +24,7 @@ func TestSimplePrometheusScraperBadInputs(t *testing.T) { settings := componenttest.NewNopTelemetrySettings() settings.Logger, _ = zap.NewDevelopment() - tests := []SimplePromethuesScraperOpts{ + tests := []SimplePrometheusScraperOpts{ { Ctx: context.TODO(), TelemetrySettings: settings, @@ -49,7 +49,7 @@ func TestSimplePrometheusScraperBadInputs(t *testing.T) { } for _, tt := range tests { - scraper, err := NewSimplePromethuesScraper(tt) + scraper, err := NewSimplePrometheusScraper(tt) assert.Error(t, err) assert.Nil(t, scraper) diff --git a/receiver/awscontainerinsightreceiver/receiver.go b/receiver/awscontainerinsightreceiver/receiver.go index 768ae2cda877..3c4afd2726ac 100644 --- a/receiver/awscontainerinsightreceiver/receiver.go +++ b/receiver/awscontainerinsightreceiver/receiver.go @@ -44,7 +44,7 @@ type awsContainerInsightReceiver struct { k8sapiserver metricsProvider prometheusScraper *k8sapiserver.PrometheusScraper dcgmScraper *gpu.DcgmScraper - neuronMonitorScraper *prometheusscraper.SimplePromethuesScraper + neuronMonitorScraper *prometheusscraper.SimplePrometheusScraper } // newAWSContainerInsightReceiver creates the aws container insight receiver with the given parameters. @@ -224,18 +224,18 @@ func (acir *awsContainerInsightReceiver) initNeuronScraper(ctx context.Context, Logger: acir.settings.Logger, } - scraperOpts := prometheusscraper.SimplePromethuesScraperOpts{ + scraperOpts := prometheusscraper.SimplePrometheusScraperOpts{ Ctx: ctx, TelemetrySettings: acir.settings, Consumer: &podAttributesDecoratorConsumer, Host: host, - ScraperConfigs: neuron.GetNueronScrapeConfig(hostinfo), + ScraperConfigs: neuron.GetNeuronScrapeConfig(hostinfo), HostInfoProvider: hostinfo, Logger: acir.settings.Logger, } var err error - acir.neuronMonitorScraper, err = prometheusscraper.NewSimplePromethuesScraper(scraperOpts) + acir.neuronMonitorScraper, err = prometheusscraper.NewSimplePrometheusScraper(scraperOpts) return err } From 9e2f849087d26264876a344201c85497cf0cb8fe Mon Sep 17 00:00:00 2001 From: Samarth Singh Date: Tue, 5 Mar 2024 18:50:52 +0000 Subject: [PATCH 43/53] remove unused file --- .../dcgm_exporter_logs.txt | 70 ------------------- 1 file changed, 70 deletions(-) delete mode 100644 receiver/awscontainerinsightreceiver/dcgm_exporter_logs.txt diff --git a/receiver/awscontainerinsightreceiver/dcgm_exporter_logs.txt b/receiver/awscontainerinsightreceiver/dcgm_exporter_logs.txt deleted file mode 100644 index 14ae2ca23574..000000000000 --- a/receiver/awscontainerinsightreceiver/dcgm_exporter_logs.txt +++ /dev/null @@ -1,70 +0,0 @@ - # HELP DCGM_FI_DEV_SM_CLOCK SM clock frequency (in MHz). - # TYPE DCGM_FI_DEV_SM_CLOCK gauge - DCGM_FI_DEV_SM_CLOCK{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 1455 - # HELP DCGM_FI_DEV_MEM_CLOCK Memory clock frequency (in MHz). - # TYPE DCGM_FI_DEV_MEM_CLOCK gauge - DCGM_FI_DEV_MEM_CLOCK{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 6250 - # 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="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 65 - # HELP DCGM_FI_DEV_POWER_USAGE Power draw (in W). - # TYPE DCGM_FI_DEV_POWER_USAGE gauge - DCGM_FI_DEV_POWER_USAGE{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 299.437000 - # HELP DCGM_FI_DEV_TOTAL_ENERGY_CONSUMPTION Total energy consumption since boot (in mJ). - # TYPE DCGM_FI_DEV_TOTAL_ENERGY_CONSUMPTION counter - DCGM_FI_DEV_TOTAL_ENERGY_CONSUMPTION{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 15782796862 - # HELP DCGM_FI_DEV_PCIE_REPLAY_COUNTER Total number of PCIe retries. - # TYPE DCGM_FI_DEV_PCIE_REPLAY_COUNTER counter - DCGM_FI_DEV_PCIE_REPLAY_COUNTER{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 0 - # HELP DCGM_FI_DEV_GPU_UTIL GPU utilization (in %). - # TYPE DCGM_FI_DEV_GPU_UTIL gauge - DCGM_FI_DEV_GPU_UTIL{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 100 - # HELP DCGM_FI_DEV_MEM_COPY_UTIL Memory utilization (in %). - # TYPE DCGM_FI_DEV_MEM_COPY_UTIL gauge - DCGM_FI_DEV_MEM_COPY_UTIL{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 38 - # HELP DCGM_FI_DEV_ENC_UTIL Encoder utilization (in %). - # TYPE DCGM_FI_DEV_ENC_UTIL gauge - DCGM_FI_DEV_ENC_UTIL{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 0 - # HELP DCGM_FI_DEV_DEC_UTIL Decoder utilization (in %). - # TYPE DCGM_FI_DEV_DEC_UTIL gauge - DCGM_FI_DEV_DEC_UTIL{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 0 - # HELP DCGM_FI_DEV_XID_ERRORS Value of the last XID error encountered. - # TYPE DCGM_FI_DEV_XID_ERRORS gauge - DCGM_FI_DEV_XID_ERRORS{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 0 - # HELP DCGM_FI_DEV_FB_FREE Framebuffer memory free (in MiB). - # TYPE DCGM_FI_DEV_FB_FREE gauge - DCGM_FI_DEV_FB_FREE{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 2230 - # HELP DCGM_FI_DEV_FB_USED Framebuffer memory used (in MiB). - # TYPE DCGM_FI_DEV_FB_USED gauge - DCGM_FI_DEV_FB_USED{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 20501 - # HELP DCGM_FI_DEV_NVLINK_BANDWIDTH_TOTAL Total number of NVLink bandwidth counters for all lanes. - # TYPE DCGM_FI_DEV_NVLINK_BANDWIDTH_TOTAL counter - DCGM_FI_DEV_NVLINK_BANDWIDTH_TOTAL{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 0 - # HELP DCGM_FI_DEV_VGPU_LICENSE_STATUS vGPU License status - # TYPE DCGM_FI_DEV_VGPU_LICENSE_STATUS gauge - DCGM_FI_DEV_VGPU_LICENSE_STATUS{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 0 - # HELP DCGM_FI_DEV_UNCORRECTABLE_REMAPPED_ROWS Number of remapped rows for uncorrectable errors - # TYPE DCGM_FI_DEV_UNCORRECTABLE_REMAPPED_ROWS counter - DCGM_FI_DEV_UNCORRECTABLE_REMAPPED_ROWS{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 0 - # HELP DCGM_FI_DEV_CORRECTABLE_REMAPPED_ROWS Number of remapped rows for correctable errors - # TYPE DCGM_FI_DEV_CORRECTABLE_REMAPPED_ROWS counter - DCGM_FI_DEV_CORRECTABLE_REMAPPED_ROWS{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 0 - # HELP DCGM_FI_DEV_ROW_REMAP_FAILURE Whether remapping of rows has failed - # TYPE DCGM_FI_DEV_ROW_REMAP_FAILURE gauge - DCGM_FI_DEV_ROW_REMAP_FAILURE{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 0 - # HELP DCGM_FI_PROF_GR_ENGINE_ACTIVE Ratio of time the graphics engine is active (in %). - # TYPE DCGM_FI_PROF_GR_ENGINE_ACTIVE gauge - DCGM_FI_PROF_GR_ENGINE_ACTIVE{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 0.808369 - # HELP DCGM_FI_PROF_PIPE_TENSOR_ACTIVE Ratio of cycles the tensor (HMMA) pipe is active (in %). - # TYPE DCGM_FI_PROF_PIPE_TENSOR_ACTIVE gauge - DCGM_FI_PROF_PIPE_TENSOR_ACTIVE{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 0.000000 - # HELP DCGM_FI_PROF_DRAM_ACTIVE Ratio of cycles the device memory interface is active sending or receiving data (in %). - # TYPE DCGM_FI_PROF_DRAM_ACTIVE gauge - DCGM_FI_PROF_DRAM_ACTIVE{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 0.315787 - # HELP DCGM_FI_PROF_PCIE_TX_BYTES The rate of data transmitted over the PCIe bus - including both protocol headers and data payloads - in bytes per second. - # TYPE DCGM_FI_PROF_PCIE_TX_BYTES gauge - DCGM_FI_PROF_PCIE_TX_BYTES{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 3985328 - # HELP DCGM_FI_PROF_PCIE_RX_BYTES The rate of data received over the PCIe bus - including both protocol headers and data payloads - in bytes per second. - # TYPE DCGM_FI_PROF_PCIE_RX_BYTES gauge - DCGM_FI_PROF_PCIE_RX_BYTES{gpu="0",UUID="GPU-ff76466b-22fc-f7a9-abe2-ce3ac453b8b3",device="nvidia0",modelName="NVIDIA A10G",Hostname="nvidia-dcgm-exporter-48cwd",DCGM_FI_DRIVER_VERSION="470.182.03",container="main",namespace="kube-system",pod="gpu-burn-c68d8c774-ltg9s"} 21715174 - pod "curl" deleted \ No newline at end of file From 3168bb20ec5039caa02bf344fb7291a62b6faf69 Mon Sep 17 00:00:00 2001 From: Samarth Singh Date: Wed, 6 Mar 2024 12:01:36 +0000 Subject: [PATCH 44/53] Making Dcgm implement SimplePrometheusScraper --- internal/aws/containerinsight/const.go | 12 +-- .../{dcgmscraper.go => dcgmscraper_config.go} | 83 +------------------ .../internal/gpu/dcgmscraper_test.go | 50 ++--------- .../neuron/neuron_monitor_scraper_config.go | 7 -- .../internal/prometheusscraper/decorator.go | 3 +- .../prometheus_scraper_testutils.go | 5 +- .../simple_prometheus_scraper.go | 26 +++--- .../internal/stores/podresourcesstore.go | 16 ---- .../internal/stores/utils.go | 2 + .../internal/stores/utils_test.go | 2 + .../awscontainerinsightreceiver/receiver.go | 55 ++++++++---- 11 files changed, 77 insertions(+), 184 deletions(-) rename receiver/awscontainerinsightreceiver/internal/gpu/{dcgmscraper.go => dcgmscraper_config.go} (65%) diff --git a/internal/aws/containerinsight/const.go b/internal/aws/containerinsight/const.go index 3ef92017fb85..f268986afbf6 100644 --- a/internal/aws/containerinsight/const.go +++ b/internal/aws/containerinsight/const.go @@ -150,12 +150,12 @@ const ( TypeContainerDiskIO = "ContainerDiskIO" // 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" - TypeNeuron = "ContainerNeuron" + TypeInfraContainer = "InfraContainer" + TypeGpuContainer = "ContainerGPU" + TypeGpuPod = "PodGPU" + TypeGpuNode = "NodeGPU" + TypeGpuCluster = "ClusterGPU" + TypeNeuronContainer = "ContainerNeuron" // unit UnitBytes = "Bytes" diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_config.go similarity index 65% rename from receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go rename to receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_config.go index 43d9149de451..133e741520ea 100644 --- a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go +++ b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_config.go @@ -5,8 +5,6 @@ package gpu import ( "context" - "errors" - "fmt" "time" "github.com/prometheus/common/model" @@ -16,13 +14,11 @@ import ( "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" ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/prometheusscraper" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/prometheusreceiver" ) const ( @@ -58,52 +54,7 @@ type hostInfoProvider interface { 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") - } - - promConfig := prometheusreceiver.Config{ - PrometheusConfig: &config.Config{ - ScrapeConfigs: []*config.ScrapeConfig{getScraperConfig(opts.HostInfoProvider)}, - }, - } - - params := receiver.CreateSettings{ - TelemetrySettings: opts.TelemetrySettings, - } - - decoConsumer := prometheusscraper.DecorateConsumer{ - ContainerOrchestrator: ci.EKS, - NextConsumer: opts.Consumer, - K8sDecorator: opts.K8sDecorator, - MetricToUnitMap: metricToUnit, - Logger: opts.Logger, - } - - promFactory := prometheusreceiver.NewFactory() - promReceiver, err := promFactory.CreateMetricsReceiver(opts.Ctx, params, &promConfig, &decoConsumer) - 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, - k8sDecorator: opts.K8sDecorator, - }, nil -} - -func getScraperConfig(hostInfoProvider hostInfoProvider) *config.ScrapeConfig { +func GetScraperConfig(hostInfoProvider hostInfoProvider) *config.ScrapeConfig { return &config.ScrapeConfig{ ScrapeInterval: model.Duration(collectionInterval), ScrapeTimeout: model.Duration(collectionInterval), @@ -196,35 +147,3 @@ func getMetricRelabelConfig(hostInfoProvider hostInfoProvider) []*relabel.Config }, } } - -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 - } - - 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/dcgmscraper_test.go b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go index 7d59f1be53a6..c69d0680f79b 100644 --- a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go +++ b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go @@ -21,6 +21,7 @@ import ( 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/prometheusscraper" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/prometheusreceiver" ) @@ -103,42 +104,6 @@ func (m mockConsumer) ConsumeMetrics(_ context.Context, md pmetric.Metrics) erro 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) { expected := map[string]struct { value float64 @@ -182,16 +147,17 @@ func TestNewDcgmScraperEndToEnd(t *testing.T) { settings := componenttest.NewNopTelemetrySettings() settings.Logger, _ = zap.NewDevelopment() - scraper, err := NewDcgmScraper(DcgmScraperOpts{ + scraper, err := prometheusscraper.NewSimplePrometheusScraper(prometheusscraper.SimplePrometheusScraperOpts{ Ctx: context.TODO(), TelemetrySettings: settings, Consumer: consumer, Host: componenttest.NewNopHost(), HostInfoProvider: mockHostInfoProvider{}, - K8sDecorator: mockDecorator{}, + ScraperConfigs: GetScraperConfig(mockHostInfoProvider{}), + Logger: settings.Logger, }) assert.NoError(t, err) - assert.Equal(t, mockHostInfoProvider{}, scraper.hostInfoProvider) + assert.Equal(t, mockHostInfoProvider{}, scraper.HostInfoProvider) // build up a new PR promFactory := prometheusreceiver.NewFactory() @@ -207,7 +173,7 @@ func TestNewDcgmScraperEndToEnd(t *testing.T) { mp, cfg, err := mocks.SetupMockPrometheus(targets...) assert.NoError(t, err) - scrapeConfig := getScraperConfig(scraper.hostInfoProvider) + scrapeConfig := GetScraperConfig(scraper.HostInfoProvider) scrapeConfig.ScrapeInterval = cfg.ScrapeConfigs[0].ScrapeInterval scrapeConfig.ScrapeTimeout = cfg.ScrapeConfigs[0].ScrapeInterval scrapeConfig.Scheme = "http" @@ -238,9 +204,9 @@ func TestNewDcgmScraperEndToEnd(t *testing.T) { // replace the prom receiver params := receiver.CreateSettings{ - TelemetrySettings: scraper.settings, + TelemetrySettings: scraper.Settings, } - scraper.prometheusReceiver, err = promFactory.CreateMetricsReceiver(scraper.ctx, params, &promConfig, consumer) + scraper.PrometheusReceiver, err = promFactory.CreateMetricsReceiver(scraper.Ctx, params, &promConfig, consumer) assert.NoError(t, err) assert.NotNil(t, mp) defer mp.Close() diff --git a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go index 8660c8379218..a788c25571a9 100644 --- a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go +++ b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go @@ -54,13 +54,6 @@ func GetNeuronMetricRelabelConfigs(hostinfo prometheusscraper.HostInfoProvider) Regex: relabel.MustNewRegexp("neuron.*|system_.*|execution_.*"), Action: relabel.Keep, }, - { - SourceLabels: model.LabelNames{"instance_name"}, - TargetLabel: "NodeName", - Regex: relabel.MustNewRegexp("(.*)"), - Replacement: "${1}", - Action: relabel.Replace, - }, { SourceLabels: model.LabelNames{"instance_id"}, TargetLabel: "InstanceId", diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator.go index ba6be6134b80..81aefc3b1abd 100644 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator.go +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator.go @@ -20,6 +20,7 @@ type DecorateConsumer struct { ContainerOrchestrator string NextConsumer consumer.Metrics K8sDecorator Decorator + MetricType string MetricToUnitMap map[string]string Logger *zap.Logger } @@ -48,7 +49,7 @@ func (dc *DecorateConsumer) ConsumeMetrics(ctx context.Context, md pmetric.Metri converted := ci.ConvertToFieldsAndTags(m, dc.Logger) var rcis []*stores.RawContainerInsightsMetric for _, pair := range converted { - rcis = append(rcis, stores.NewRawContainerInsightsMetricWithData(ci.TypeGpuContainer, pair.Fields, pair.Tags, dc.Logger)) + rcis = append(rcis, stores.NewRawContainerInsightsMetricWithData(dc.MetricType, pair.Fields, pair.Tags, dc.Logger)) } decorated := dc.decorateMetrics(rcis) diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/prometheus_scraper_testutils.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/prometheus_scraper_testutils.go index 26286240a468..70257e5cf16e 100644 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/prometheus_scraper_testutils.go +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/prometheus_scraper_testutils.go @@ -57,6 +57,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) + fmt.Println(metric.Name()) metricsStruct, ok := m.ExpectedMetrics[metric.Name()] if ok { assert.Equal(m.T, metricsStruct.MetricValue, metric.Gauge().DataPoints().At(0).DoubleValue()) @@ -130,9 +131,9 @@ func TestSimplePrometheusEndToEnd(opts TestSimplePrometheusEndToEndOpts) { // replace the prom receiver params := receiver.CreateSettings{ - TelemetrySettings: scraper.settings, + TelemetrySettings: scraper.Settings, } - scraper.prometheusReceiver, err = promFactory.CreateMetricsReceiver(scraper.ctx, params, &promConfig, opts.Consumer) + scraper.PrometheusReceiver, err = promFactory.CreateMetricsReceiver(scraper.Ctx, params, &promConfig, opts.Consumer) assert.NoError(opts.T, err) assert.NotNil(opts.T, mp) defer mp.Close() diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go index 29b52b605eda..285d08656eb4 100644 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go @@ -15,11 +15,11 @@ import ( ) type SimplePrometheusScraper struct { - ctx context.Context - settings component.TelemetrySettings + Ctx context.Context + Settings component.TelemetrySettings host component.Host - hostInfoProvider HostInfoProvider - prometheusReceiver receiver.Metrics + HostInfoProvider HostInfoProvider + PrometheusReceiver receiver.Metrics running bool } @@ -66,11 +66,11 @@ func NewSimplePrometheusScraper(opts SimplePrometheusScraperOpts) (*SimplePromet } return &SimplePrometheusScraper{ - ctx: opts.Ctx, - settings: opts.TelemetrySettings, + Ctx: opts.Ctx, + Settings: opts.TelemetrySettings, host: opts.Host, - hostInfoProvider: opts.HostInfoProvider, - prometheusReceiver: promReceiver, + HostInfoProvider: opts.HostInfoProvider, + PrometheusReceiver: promReceiver, }, nil } @@ -79,10 +79,10 @@ func (ds *SimplePrometheusScraper) GetMetrics() []pmetric.Metrics { // 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) + 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.Settings.Logger.Error("Unable to start PrometheusReceiver", zap.Error(err)) } ds.running = err == nil } @@ -91,9 +91,9 @@ func (ds *SimplePrometheusScraper) GetMetrics() []pmetric.Metrics { func (ds *SimplePrometheusScraper) Shutdown() { if ds.running { - err := ds.prometheusReceiver.Shutdown(ds.ctx) + err := ds.PrometheusReceiver.Shutdown(ds.Ctx) if err != nil { - ds.settings.Logger.Error("Unable to shutdown PrometheusReceiver", zap.Error(err)) + ds.Settings.Logger.Error("Unable to shutdown PrometheusReceiver", zap.Error(err)) } ds.running = false } diff --git a/receiver/awscontainerinsightreceiver/internal/stores/podresourcesstore.go b/receiver/awscontainerinsightreceiver/internal/stores/podresourcesstore.go index 5d3f1c96698b..cb9503927b25 100644 --- a/receiver/awscontainerinsightreceiver/internal/stores/podresourcesstore.go +++ b/receiver/awscontainerinsightreceiver/internal/stores/podresourcesstore.go @@ -171,22 +171,6 @@ func (p *PodResourcesStore) AddResourceName(resourceName string) { p.resourceNameSet[resourceName] = struct{}{} } -func (p *PodResourcesStore) PrintMaps() { - p.logger.Info("containerInfoToResourcesMap:") - for containerInfo, resourceInfos := range p.containerInfoToResourcesMap { - p.logger.Info("ContainerInfo-" + containerInfo.ContainerName + " ; " + containerInfo.PodName) - p.logger.Info("ResourceInfos:") - for _, resourceInfo := range resourceInfos { - p.logger.Info("ResourceInfo-" + resourceInfo.resourceName + " ; " + resourceInfo.deviceID) - } - } - p.logger.Info("\nresourceToPodContainerMap:") - for resourceInfo, containerInfo := range p.resourceToPodContainerMap { - p.logger.Info("ResourceInfo-" + resourceInfo.resourceName + " ; " + resourceInfo.deviceID) - p.logger.Info("ContainerInfo-" + containerInfo.ContainerName + " ; " + containerInfo.PodName) - } -} - func (p *PodResourcesStore) UpdateAndPrintMapsManually() { // this also has embedded print statement p.updateMaps() diff --git a/receiver/awscontainerinsightreceiver/internal/stores/utils.go b/receiver/awscontainerinsightreceiver/internal/stores/utils.go index a0db8fe03e93..10cc17641034 100644 --- a/receiver/awscontainerinsightreceiver/internal/stores/utils.go +++ b/receiver/awscontainerinsightreceiver/internal/stores/utils.go @@ -125,6 +125,8 @@ func TagMetricSource(metric CIMetric) { sources = append(sources, []string{"cadvisor"}...) case ci.TypeGpuContainer: sources = append(sources, []string{"dcgm", "pod", "calculated"}...) + case ci.TypeNeuronContainer: + sources = append(sources, []string{"neuron", "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 28c84715588f..dd12e357bff1 100644 --- a/receiver/awscontainerinsightreceiver/internal/stores/utils_test.go +++ b/receiver/awscontainerinsightreceiver/internal/stores/utils_test.go @@ -176,6 +176,7 @@ func TestUtils_TagMetricSource(t *testing.T) { ci.TypeContainerFS, ci.TypeContainerDiskIO, ci.TypeGpuContainer, + ci.TypeNeuronContainer, } expectedSources := []string{ @@ -190,6 +191,7 @@ func TestUtils_TagMetricSource(t *testing.T) { "[\"cadvisor\",\"calculated\"]", "[\"cadvisor\"]", "[\"dcgm\",\"pod\",\"calculated\"]", + "[\"neuron\",\"pod\",\"calculated\"]", } for i, mtype := range types { tags := map[string]string{ diff --git a/receiver/awscontainerinsightreceiver/receiver.go b/receiver/awscontainerinsightreceiver/receiver.go index 3c4afd2726ac..bd777973cdda 100644 --- a/receiver/awscontainerinsightreceiver/receiver.go +++ b/receiver/awscontainerinsightreceiver/receiver.go @@ -43,7 +43,9 @@ type awsContainerInsightReceiver struct { cadvisor metricsProvider k8sapiserver metricsProvider prometheusScraper *k8sapiserver.PrometheusScraper - dcgmScraper *gpu.DcgmScraper + k8sDecorator *stores.K8sDecorator + podResourcesStore *stores.PodResourcesStore + dcgmScraper *prometheusscraper.SimplePrometheusScraper neuronMonitorScraper *prometheusscraper.SimplePrometheusScraper } @@ -74,12 +76,12 @@ func (acir *awsContainerInsightReceiver) Start(ctx context.Context, host compone } if acir.config.ContainerOrchestrator == ci.EKS { - k8sDecorator, err := stores.NewK8sDecorator(ctx, acir.config.TagService, acir.config.PrefFullPodName, acir.config.AddFullPodNameMetricLabel, acir.config.AddContainerNameMetricLabel, acir.config.EnableControlPlaneMetrics, acir.settings.Logger) + acir.k8sDecorator, err = stores.NewK8sDecorator(ctx, acir.config.TagService, acir.config.PrefFullPodName, acir.config.AddFullPodNameMetricLabel, acir.config.AddContainerNameMetricLabel, acir.config.EnableControlPlaneMetrics, acir.settings.Logger) if err != nil { return err } - decoratorOption := cadvisor.WithDecorator(k8sDecorator) + decoratorOption := cadvisor.WithDecorator(acir.k8sDecorator) acir.cadvisor, err = cadvisor.New(acir.config.ContainerOrchestrator, hostinfo, acir.settings.Logger, decoratorOption) if err != nil { return err @@ -100,12 +102,12 @@ 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, k8sDecorator) + err = acir.initDcgmScraper(ctx, host, hostinfo, acir.k8sDecorator) if err != nil { acir.settings.Logger.Debug("Unable to start dcgm scraper", zap.Error(err)) } - err = acir.initNeuronScraper(ctx, host, hostinfo, k8sDecorator) + err = acir.initNeuronScraper(ctx, host, hostinfo, acir.k8sDecorator) if err != nil { acir.settings.Logger.Debug("Unable to start neuron scraper", zap.Error(err)) } @@ -188,16 +190,26 @@ func (acir *awsContainerInsightReceiver) initDcgmScraper(ctx context.Context, ho return nil } - var err error - acir.dcgmScraper, err = gpu.NewDcgmScraper(gpu.DcgmScraperOpts{ + decoConsumer := prometheusscraper.DecorateConsumer{ + ContainerOrchestrator: ci.EKS, + NextConsumer: acir.nextConsumer, + MetricType: ci.TypeNeuronContainer, + K8sDecorator: decorator, + Logger: acir.settings.Logger, + } + + scraperOpts := prometheusscraper.SimplePrometheusScraperOpts{ Ctx: ctx, TelemetrySettings: acir.settings, - Consumer: acir.nextConsumer, + Consumer: &decoConsumer, Host: host, + ScraperConfigs: gpu.GetScraperConfig(hostinfo), HostInfoProvider: hostinfo, - K8sDecorator: decorator, Logger: acir.settings.Logger, - }) + } + + var err error + acir.dcgmScraper, err = prometheusscraper.NewSimplePrometheusScraper(scraperOpts) return err } @@ -209,18 +221,19 @@ func (acir *awsContainerInsightReceiver) initNeuronScraper(ctx context.Context, decoConsumer := prometheusscraper.DecorateConsumer{ ContainerOrchestrator: ci.EKS, NextConsumer: acir.nextConsumer, + MetricType: ci.TypeNeuronContainer, K8sDecorator: decorator, Logger: acir.settings.Logger, } - podresourcesstore := stores.NewPodResourcesStore(acir.settings.Logger) - podresourcesstore.AddResourceName("aws.amazon.com/neuroncore") - podresourcesstore.AddResourceName("aws.amazon.com/neuron") - podresourcesstore.AddResourceName("aws.amazon.com/neurondevice") + acir.podResourcesStore = stores.NewPodResourcesStore(acir.settings.Logger) + acir.podResourcesStore.AddResourceName("aws.amazon.com/neuroncore") + acir.podResourcesStore.AddResourceName("aws.amazon.com/neuron") + acir.podResourcesStore.AddResourceName("aws.amazon.com/neurondevice") podAttributesDecoratorConsumer := neuron.PodAttributesDecoratorConsumer{ NextConsumer: &decoConsumer, - PodResourcesStore: podresourcesstore, + PodResourcesStore: acir.podResourcesStore, Logger: acir.settings.Logger, } @@ -263,6 +276,18 @@ func (acir *awsContainerInsightReceiver) Shutdown(context.Context) error { acir.dcgmScraper.Shutdown() } + if acir.neuronMonitorScraper != nil { + acir.neuronMonitorScraper.Shutdown() + } + + if acir.k8sDecorator != nil { + acir.k8sDecorator.Shutdown() + } + + if acir.podResourcesStore != nil { + acir.podResourcesStore.Shutdown() + } + return errs } From b7d198a1b60eb4c4f0489b2eb6d32e5bf34d1d00 Mon Sep 17 00:00:00 2001 From: Samarth Singh Date: Thu, 7 Mar 2024 14:27:22 +0000 Subject: [PATCH 45/53] Fix comments and merge conflicts --- internal/aws/proxy/server_test.go | 2 +- .../awscontainerinsightreceiver/config.go | 5 +- .../awscontainerinsightreceiver/factory.go | 4 +- .../internal/gpu/dcgmscraper.go | 235 ------------------ .../internal/gpu/dcgmscraper_config.go | 6 +- .../internal/gpu/decorator.go | 140 ----------- .../internal/gpu/decorator_test.go | 187 -------------- .../neuron/pod_attribute_decorator.go | 69 ++--- .../neuron/pod_attribute_decorator_test.go | 116 +++++++++ .../prometheusscraper/decorator_test.go | 206 --------------- .../{ => decoratorconsumer}/decorator.go | 2 +- .../decoratorconsumer/decorator_test.go | 143 +++++++++++ .../decoratorconsumer/decorator_testutils.go | 91 +++++++ .../awscontainerinsightreceiver/receiver.go | 6 +- 14 files changed, 402 insertions(+), 810 deletions(-) delete mode 100644 receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go delete mode 100644 receiver/awscontainerinsightreceiver/internal/gpu/decorator.go delete mode 100644 receiver/awscontainerinsightreceiver/internal/gpu/decorator_test.go create mode 100644 receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator_test.go delete mode 100644 receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator_test.go rename receiver/awscontainerinsightreceiver/internal/prometheusscraper/{ => decoratorconsumer}/decorator.go (99%) create mode 100644 receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator_test.go create mode 100644 receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator_testutils.go diff --git a/internal/aws/proxy/server_test.go b/internal/aws/proxy/server_test.go index 7b2a593a0c1f..c95108e395be 100644 --- a/internal/aws/proxy/server_test.go +++ b/internal/aws/proxy/server_test.go @@ -226,7 +226,7 @@ func TestCanCreateTransport(t *testing.T) { _, err := NewServer(cfg, logger) assert.Error(t, err, "NewServer should fail") - assert.Contains(t, err.Error(), "invalid control character in URL") + assert.Contains(t, err.Error(), "failed to parse proxy URL") } func TestGetServiceEndpointInvalidAWSConfig(t *testing.T) { diff --git a/receiver/awscontainerinsightreceiver/config.go b/receiver/awscontainerinsightreceiver/config.go index 7e01f396a4c1..c6515334f680 100644 --- a/receiver/awscontainerinsightreceiver/config.go +++ b/receiver/awscontainerinsightreceiver/config.go @@ -60,9 +60,8 @@ type Config struct { // EnableAcceleratedComputeMetrics enabled features with accelerated compute resources where metrics are scraped from vendor specific sources EnableAcceleratedComputeMetrics bool `mapstructure:"accelerated_compute_metrics"` - EnableGpuMetric bool `mapstructure:"accelerated_compute_metrics"` - // EnableNeuronMetric toggles Neuron monitoring where metrics are scraped from neuron monitor + // EnableAwsNeuronMetrics toggles Neuron monitoring where metrics are scraped from neuron monitor // The default value is false. - EnableNeuronMetric bool `mapstructure:"neuron_metrics"` + EnableAwsNeuronMetrics bool `mapstructure:"neuron_metrics"` } diff --git a/receiver/awscontainerinsightreceiver/factory.go b/receiver/awscontainerinsightreceiver/factory.go index cc0e246501ab..c828335b6e22 100644 --- a/receiver/awscontainerinsightreceiver/factory.go +++ b/receiver/awscontainerinsightreceiver/factory.go @@ -44,7 +44,7 @@ const ( defaultEnableControlPlaneMetrics = false // Don't enable Neuron metrics by default - defaultEnableNeuronMetrics = false + defaultEnableAwsNeuronMetrics = false ) // NewFactory creates a factory for AWS container insight receiver @@ -67,7 +67,7 @@ func createDefaultConfig() component.Config { ClusterName: defaultClusterName, LeaderLockName: defaultLeaderLockName, EnableControlPlaneMetrics: defaultEnableControlPlaneMetrics, - EnableNeuronMetric: defaultEnableNeuronMetrics, + EnableAwsNeuronMetrics: defaultEnableAwsNeuronMetrics, } } diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go deleted file mode 100644 index db7bdc184425..000000000000 --- a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper.go +++ /dev/null @@ -1,235 +0,0 @@ -// Copyright The OpenTelemetry Authors -// SPDX-License-Identifier: Apache-2.0 - -package gpu - -import ( - "context" - "errors" - "fmt" - "time" - - 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/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" - - ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/prometheusreceiver" -) - -const ( - caFile = "/etc/amazon-cloudwatch-observability-agent-cert/tls-ca.crt" - collectionInterval = 60 * time.Second - jobName = "containerInsightsDCGMExporterScraper" - scraperMetricsPath = "/metrics" - scraperK8sServiceSelector = "k8s-app=dcgm-exporter-service" -) - -type DcgmScraper struct { - ctx context.Context - settings component.TelemetrySettings - host component.Host - hostInfoProvider hostInfoProvider - prometheusReceiver receiver.Metrics - k8sDecorator Decorator - running bool -} - -type DcgmScraperOpts struct { - Ctx context.Context - TelemetrySettings component.TelemetrySettings - Consumer consumer.Metrics - Host component.Host - HostInfoProvider hostInfoProvider - K8sDecorator Decorator - Logger *zap.Logger -} - -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") - } - - promConfig := prometheusreceiver.Config{ - PrometheusConfig: &config.Config{ - ScrapeConfigs: []*config.ScrapeConfig{getScraperConfig(opts.HostInfoProvider)}, - }, - } - - params := receiver.CreateSettings{ - 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, &decoConsumer) - 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, - k8sDecorator: opts.K8sDecorator, - }, nil -} - -func getScraperConfig(hostInfoProvider hostInfoProvider) *config.ScrapeConfig { - return &config.ScrapeConfig{ - HTTPClientConfig: configutil.HTTPClientConfig{ - TLSConfig: configutil.TLSConfig{ - CAFile: caFile, - InsecureSkipVerify: false, - }, - }, - ScrapeInterval: model.Duration(collectionInterval), - ScrapeTimeout: model.Duration(collectionInterval), - JobName: jobName, - Scheme: "https", - 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.AttributeK8sNamespace, - 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.AttributeFullPodName, - Regex: relabel.MustNewRegexp("(.*)"), - Replacement: "${1}", - Action: relabel.Replace, - }, - // additional k8s podname for service name and k8s blob decoration - { - SourceLabels: model.LabelNames{"pod"}, - TargetLabel: ci.AttributeK8sPodName, - Regex: relabel.MustNewRegexp("(.*)"), - Replacement: "${1}", - Action: relabel.Replace, - }, - { - SourceLabels: model.LabelNames{"container"}, - TargetLabel: ci.AttributeContainerName, - Regex: relabel.MustNewRegexp("(.*)"), - Replacement: "${1}", - Action: relabel.Replace, - }, - { - SourceLabels: model.LabelNames{"device"}, - TargetLabel: ci.AttributeGpuDevice, - 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 - 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 - } - - 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/dcgmscraper_config.go b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_config.go index 133e741520ea..259cd10df432 100644 --- a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_config.go +++ b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_config.go @@ -18,7 +18,7 @@ import ( "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/prometheusscraper" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer" ) const ( @@ -35,7 +35,7 @@ type DcgmScraper struct { host component.Host hostInfoProvider hostInfoProvider prometheusReceiver receiver.Metrics - k8sDecorator prometheusscraper.Decorator + k8sDecorator decoratorconsumer.Decorator running bool } @@ -45,7 +45,7 @@ type DcgmScraperOpts struct { Consumer consumer.Metrics Host component.Host HostInfoProvider hostInfoProvider - K8sDecorator prometheusscraper.Decorator + K8sDecorator decoratorconsumer.Decorator Logger *zap.Logger } diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/decorator.go b/receiver/awscontainerinsightreceiver/internal/gpu/decorator.go deleted file mode 100644 index ee3f9a8c36c1..000000000000 --- a/receiver/awscontainerinsightreceiver/internal/gpu/decorator.go +++ /dev/null @@ -1,140 +0,0 @@ -// Copyright The OpenTelemetry Authors -// SPDX-License-Identifier: Apache-2.0 - -package gpu - -import ( - "context" - - "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 ( - 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 metricToUnit = map[string]string{ - gpuUtil: "Percent", - gpuMemUtil: "Percent", - gpuMemUsed: "Bytes", - gpuMemTotal: "Bytes", - gpuTemperature: "None", - gpuPowerDraw: "None", -} - -// 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) - converted := ci.ConvertToFieldsAndTags(m, dc.logger) - var rcis []*stores.RawContainerInsightsMetric - for _, pair := range converted { - rcis = append(rcis, stores.NewRawContainerInsightsMetricWithData(ci.TypeGpuContainer, pair.Fields, pair.Tags, dc.logger)) - } - - decorated := dc.decorateMetrics(rcis) - dc.updateAttributes(m, decorated) - 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(rcis []*stores.RawContainerInsightsMetric) []*stores.RawContainerInsightsMetric { - var result []*stores.RawContainerInsightsMetric - if dc.containerOrchestrator != ci.EKS { - return result - } - for _, rci := range rcis { - // add tags for EKS - out := dc.k8sDecorator.Decorate(rci) - if out != nil { - result = append(result, out.(*stores.RawContainerInsightsMetric)) - } - } - return result -} - -func (dc *decorateConsumer) updateAttributes(m pmetric.Metric, rcis []*stores.RawContainerInsightsMetric) { - if len(rcis) == 0 { - 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() == 0 { - return - } - 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 := 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) - } - } -} - -func (dc *decorateConsumer) Shutdown() error { - if dc.k8sDecorator != nil { - return dc.k8sDecorator.Shutdown() - } - return nil -} diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/decorator_test.go b/receiver/awscontainerinsightreceiver/internal/gpu/decorator_test.go deleted file mode 100644 index a81e39268b65..000000000000 --- a/receiver/awscontainerinsightreceiver/internal/gpu/decorator_test.go +++ /dev/null @@ -1,187 +0,0 @@ -// Copyright The OpenTelemetry Authors -// SPDX-License-Identifier: Apache-2.0 - -package gpu - -import ( - "context" - "testing" - - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" - "go.opentelemetry.io/collector/consumer/consumertest" - "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) - -type MockK8sDecorator struct { -} - -func (m *MockK8sDecorator) Decorate(metric stores.CIMetric) stores.CIMetric { - return metric -} - -func (m *MockK8sDecorator) Shutdown() 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.want.MetricCount(), tc.metrics.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/neuron/pod_attribute_decorator.go b/receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator.go index 52ad45694c74..934824c06806 100644 --- a/receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator.go +++ b/receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator.go @@ -22,9 +22,13 @@ const ( neuronDeviceResourceNameAlt = "aws.amazon.com/neuron" ) +type PodResourcesStoreInterface interface { + GetContainerInfo(string, string) *stores.ContainerInfo +} + type PodAttributesDecoratorConsumer struct { NextConsumer consumer.Metrics - PodResourcesStore *stores.PodResourcesStore // replace with podResourcesApi + PodResourcesStore PodResourcesStoreInterface Logger *zap.Logger } @@ -35,11 +39,11 @@ func (pdc *PodAttributesDecoratorConsumer) Capabilities() consumer.Capabilities } func (pdc *PodAttributesDecoratorConsumer) ConsumeMetrics(ctx context.Context, md pmetric.Metrics) error { - md = pdc.neuronMetricsProcess(md) + pdc.neuronMetricsProcess(md) return pdc.NextConsumer.ConsumeMetrics(ctx, md) } -func (pdc *PodAttributesDecoratorConsumer) neuronMetricsProcess(md pmetric.Metrics) pmetric.Metrics { +func (pdc *PodAttributesDecoratorConsumer) neuronMetricsProcess(md pmetric.Metrics) { rms := md.ResourceMetrics() for i := 0; i < rms.Len(); i++ { rs := rms.At(i) @@ -49,38 +53,27 @@ func (pdc *PodAttributesDecoratorConsumer) neuronMetricsProcess(md pmetric.Metri metrics := ils.Metrics() neuronHardwareInfo := pmetric.Metric{} + neuronHardwareInfoFound := false for k := 0; k < metrics.Len(); k++ { m := metrics.At(k) if m.Name() == neuronHardwareInfoKey { neuronHardwareInfo = m + neuronHardwareInfoFound = true break } } - - neuronCoresPerDeviceValue, _ := neuronHardwareInfo.Sum().DataPoints().At(0).Attributes().Get(neuronCorePerDeviceKey) - neuronCoresPerDevice, _ := strconv.Atoi(neuronCoresPerDeviceValue.AsString()) - - for k := 0; k < metrics.Len(); k++ { - m := metrics.At(k) - pdc.AddPodCorrelationAttributes(getMetricDatapoints(m), neuronCoresPerDevice) + if neuronHardwareInfoFound { + neuronCoresPerDevice := getNueronCoresPerDevice(neuronHardwareInfo) + for k := 0; k < metrics.Len(); k++ { + m := metrics.At(k) + pdc.addPodCorrelationAttributes(getMetricDatapoints(m), neuronCoresPerDevice) + } } } } - return md -} - -func getMetricDatapoints(m pmetric.Metric) pmetric.NumberDataPointSlice { - switch m.Type() { - case pmetric.MetricTypeGauge: - return m.Gauge().DataPoints() - case pmetric.MetricTypeSum: - return m.Sum().DataPoints() - default: - return pmetric.NewNumberDataPointSlice() - } } -func (pdc *PodAttributesDecoratorConsumer) AddPodCorrelationAttributes(metricDatapoints pmetric.NumberDataPointSlice, neuronCoresPerDevice int) { +func (pdc *PodAttributesDecoratorConsumer) addPodCorrelationAttributes(metricDatapoints pmetric.NumberDataPointSlice, neuronCoresPerDevice int) { for i := 0; i < metricDatapoints.Len(); i++ { attributes := metricDatapoints.At(i).Attributes() var containerInfo *stores.ContainerInfo @@ -88,17 +81,15 @@ func (pdc *PodAttributesDecoratorConsumer) AddPodCorrelationAttributes(metricDat if neuronDeviceIndex, neuronDeviceIndexPresent := attributes.Get(neuronDeviceAttributeKey); neuronDeviceIndexPresent { // get container info from neuronDeviceIndex neuronDeviceIndex := neuronDeviceIndex.AsString() - if neuronDeviceIndexPresent { - containerInfo = pdc.getContainerInfoForNueronDeviceIndex(neuronDeviceIndex) + containerInfo = pdc.getContainerInfoForNeuronDeviceIndex(neuronDeviceIndex) - } } else if neuronCoreIndex, neuronCoreIndexPresent := attributes.Get(neuronCoreAttributeKey); neuronCoreIndexPresent { // get container info from neuronCore containerInfo = pdc.PodResourcesStore.GetContainerInfo(neuronCoreIndex.AsString(), neuronCoreResourceName) neuronDeviceIndex := getNeuronDeviceIndexFromCoreAttribute(neuronCoreIndex, neuronCoresPerDevice) if containerInfo == nil { // else get container info from calculated neuronDeviceIndex - containerInfo = pdc.getContainerInfoForNueronDeviceIndex(neuronDeviceIndex) + containerInfo = pdc.getContainerInfoForNeuronDeviceIndex(neuronDeviceIndex) } attributes.PutStr(neuronDeviceAttributeKey, neuronDeviceIndex) } @@ -106,7 +97,7 @@ func (pdc *PodAttributesDecoratorConsumer) AddPodCorrelationAttributes(metricDat } } -func (pdc *PodAttributesDecoratorConsumer) getContainerInfoForNueronDeviceIndex(neuronDeviceIndex string) *stores.ContainerInfo { +func (pdc *PodAttributesDecoratorConsumer) getContainerInfoForNeuronDeviceIndex(neuronDeviceIndex string) *stores.ContainerInfo { containerInfo := pdc.PodResourcesStore.GetContainerInfo(neuronDeviceIndex, neuronDeviceResourceName) if containerInfo == nil { // Alt resource name is to support backward compatibility in neuron monitor : https://awsdocs-neuron.readthedocs-hosted.com/en/latest/containers/tutorials/k8s-setup.html @@ -121,10 +112,30 @@ func populateAttributes(attributes *pcommon.Map, containerInfo *stores.Container attributes.PutStr(ci.AttributeK8sPodName, containerInfo.PodName) attributes.PutStr(ci.AttributePodName, containerInfo.PodName) attributes.PutStr(ci.AttributeK8sNamespace, containerInfo.Namespace) - attributes.PutStr(ci.AttributeFullPodName, containerInfo.PodName+"."+containerInfo.Namespace) } } +func getMetricDatapoints(m pmetric.Metric) pmetric.NumberDataPointSlice { + switch m.Type() { + case pmetric.MetricTypeGauge: + return m.Gauge().DataPoints() + case pmetric.MetricTypeSum: + return m.Sum().DataPoints() + default: + return pmetric.NewNumberDataPointSlice() + } +} + +// We extract the attribute named `neuroncore_per_device_count` from the metric to get the value +// https://awsdocs-neuron.readthedocs-hosted.com/en/latest/tools/neuron-sys-tools/neuron-monitor-user-guide +func getNueronCoresPerDevice(neuronHardwareInfo pmetric.Metric) int { + neuronCoresPerDeviceValue, _ := neuronHardwareInfo.Sum().DataPoints().At(0).Attributes().Get(neuronCorePerDeviceKey) + neuronCoresPerDevice, _ := strconv.Atoi(neuronCoresPerDeviceValue.AsString()) + return neuronCoresPerDevice +} + +// To get the device index from core index we divide the index by cores in a single device +// https://awsdocs-neuron.readthedocs-hosted.com/en/latest/tools/neuron-sys-tools/neuron-monitor-user-guide func getNeuronDeviceIndexFromCoreAttribute(neuronCoreIndex pcommon.Value, neuronCoresPerDevice int) string { neuronCoreIndexIntVal, _ := strconv.Atoi(neuronCoreIndex.AsString()) return strconv.Itoa(neuronCoreIndexIntVal / neuronCoresPerDevice) diff --git a/receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator_test.go b/receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator_test.go new file mode 100644 index 000000000000..d96a6d4ee51b --- /dev/null +++ b/receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator_test.go @@ -0,0 +1,116 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +package neuron + +import ( + "context" + "testing" + + ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" + "go.opentelemetry.io/collector/consumer/consumertest" + "go.opentelemetry.io/collector/pdata/pmetric" + "go.uber.org/zap" +) + +var dummyPodName = "pod-name" +var dummyContainerName = "container-name" +var dummyNamespace = "namespace" + +type mockPodResourcesStore struct { +} + +func (m mockPodResourcesStore) GetContainerInfo(deviceIndex string, resourceName string) *stores.ContainerInfo { + return &stores.ContainerInfo{ + PodName: dummyPodName, + ContainerName: dummyContainerName, + Namespace: dummyNamespace, + } +} + +func TestConsumeMetricsForPodAttributeDecorator(t *testing.T) { + logger, _ := zap.NewDevelopment() + dc := &PodAttributesDecoratorConsumer{ + NextConsumer: consumertest.NewNop(), + PodResourcesStore: mockPodResourcesStore{}, + Logger: logger, + } + ctx := context.Background() + + testcases := map[string]decoratorconsumer.TestCase{ + "empty": { + Metrics: pmetric.NewMetrics(), + Want: pmetric.NewMetrics(), + ShouldError: false, + }, + "neuron_hardware_info_not_found": { + Metrics: decoratorconsumer.GenerateMetrics(map[decoratorconsumer.MetricIdentifier]map[string]string{ + {Name: "test", MetricType: pmetric.MetricTypeGauge}: { + "device": "test0", + }, + }), + + Want: decoratorconsumer.GenerateMetrics(map[decoratorconsumer.MetricIdentifier]map[string]string{ + {Name: "test", MetricType: pmetric.MetricTypeGauge}: { + "device": "test0", + }, + }), + ShouldError: false, + }, + "correlation_via_neuron_device_index": { + Metrics: decoratorconsumer.GenerateMetrics(map[decoratorconsumer.MetricIdentifier]map[string]string{ + {Name: neuronHardwareInfoKey, MetricType: pmetric.MetricTypeSum}: { + neuronCorePerDeviceKey: "2", + }, + {Name: "test", MetricType: pmetric.MetricTypeGauge}: { + "device": "test0", + neuronDeviceAttributeKey: "1", + }, + }), + Want: decoratorconsumer.GenerateMetrics(map[decoratorconsumer.MetricIdentifier]map[string]string{ + {Name: neuronHardwareInfoKey, MetricType: pmetric.MetricTypeSum}: { + neuronCorePerDeviceKey: "2", + }, + {Name: "test", MetricType: pmetric.MetricTypeGauge}: { + "device": "test0", + neuronDeviceAttributeKey: "1", + ci.AttributeContainerName: dummyContainerName, + ci.AttributeK8sPodName: dummyPodName, + ci.AttributePodName: dummyPodName, + ci.AttributeK8sNamespace: dummyNamespace, + }, + }), + ShouldError: false, + }, + "correlation_via_neuron_core": { + Metrics: decoratorconsumer.GenerateMetrics(map[decoratorconsumer.MetricIdentifier]map[string]string{ + {Name: neuronHardwareInfoKey, MetricType: pmetric.MetricTypeSum}: { + neuronCorePerDeviceKey: "2", + }, + {Name: "test", MetricType: pmetric.MetricTypeGauge}: { + "device": "test0", + neuronCoreAttributeKey: "10", + }, + }), + Want: decoratorconsumer.GenerateMetrics(map[decoratorconsumer.MetricIdentifier]map[string]string{ + {Name: neuronHardwareInfoKey, MetricType: pmetric.MetricTypeSum}: { + neuronCorePerDeviceKey: "2", + }, + {Name: "test", MetricType: pmetric.MetricTypeGauge}: { + "device": "test0", + neuronCoreAttributeKey: "10", + neuronDeviceAttributeKey: "5", + ci.AttributeContainerName: dummyContainerName, + ci.AttributeK8sPodName: dummyPodName, + ci.AttributePodName: dummyPodName, + ci.AttributeK8sNamespace: dummyNamespace, + }, + }), + ShouldError: false, + }, + } + + decoratorconsumer.RunDecoratorTestScenarios(t, dc, ctx, testcases) +} diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator_test.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator_test.go deleted file mode 100644 index e5107ee29eaf..000000000000 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator_test.go +++ /dev/null @@ -1,206 +0,0 @@ -// Copyright The OpenTelemetry Authors -// SPDX-License-Identifier: Apache-2.0 - -package prometheusscraper - -import ( - "context" - "testing" - - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" - "go.opentelemetry.io/collector/consumer/consumertest" - "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) - -type MockK8sDecorator struct { -} - -func (m *MockK8sDecorator) Decorate(metric stores.CIMetric) stores.CIMetric { - return metric -} - -func (m *MockK8sDecorator) Shutdown() error { - return nil -} - -const ( - util = "UTIL" - memUtil = "USED_PERCENT" - memUsed = "FB_USED" - memTotal = "FB_TOTAL" - temp = "TEMP" - powerDraw = "POWER_USAGE" -) - -var metricToUnit = map[string]string{ - util: "Percent", - memUtil: "Percent", - memUsed: "Bytes", - memTotal: "Bytes", - temp: "None", - powerDraw: "None", -} - -func TestConsumeMetricsForGpu(t *testing.T) { - logger, _ := zap.NewDevelopment() - dc := &DecorateConsumer{ - ContainerOrchestrator: "EKS", - NextConsumer: consumertest.NewNop(), - K8sDecorator: &MockK8sDecorator{}, - MetricToUnitMap: metricToUnit, - 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{ - util: { - "device": "test0", - }, - memUtil: { - "device": "test0", - }, - memTotal: { - "device": "test0", - }, - memUsed: { - "device": "test0", - }, - powerDraw: { - "device": "test0", - }, - temp: { - "device": "test0", - }, - }), - want: generateMetrics(map[string]map[string]string{ - util: { - "device": "test0", - "Unit": "Percent", - }, - memUtil: { - "device": "test0", - "Unit": "Percent", - }, - memTotal: { - "device": "test0", - "Unit": "Bytes", - }, - memUsed: { - "device": "test0", - "Unit": "Bytes", - }, - powerDraw: { - "device": "test0", - "Unit": "None", - }, - temp: { - "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{ - util: { - "device": "test0", - "Type": "TestType", - }, - }), - want: generateMetrics(map[string]map[string]string{ - util: { - "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.want.MetricCount(), tc.metrics.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/prometheusscraper/decorator.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator.go similarity index 99% rename from receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator.go rename to receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator.go index 81aefc3b1abd..4bbb9a31fc5d 100644 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decorator.go +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator.go @@ -1,7 +1,7 @@ // Copyright The OpenTelemetry Authors // SPDX-License-Identifier: Apache-2.0 -package prometheusscraper +package decoratorconsumer import ( "context" diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator_test.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator_test.go new file mode 100644 index 000000000000..7e7e417db79c --- /dev/null +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator_test.go @@ -0,0 +1,143 @@ +package decoratorconsumer + +import ( + "context" + "testing" + + "go.opentelemetry.io/collector/consumer/consumertest" + "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) + +type MockK8sDecorator struct { +} + +func (m *MockK8sDecorator) Decorate(metric stores.CIMetric) stores.CIMetric { + return metric +} + +func (m *MockK8sDecorator) Shutdown() error { + return nil +} + +const ( + util = "UTIL" + memUtil = "USED_PERCENT" + memUsed = "FB_USED" + memTotal = "FB_TOTAL" + temp = "TEMP" + powerDraw = "POWER_USAGE" +) + +var metricToUnit = map[string]string{ + util: "Percent", + memUtil: "Percent", + memUsed: "Bytes", + memTotal: "Bytes", + temp: "None", + powerDraw: "None", +} + +func TestConsumeMetrics(t *testing.T) { + logger, _ := zap.NewDevelopment() + dc := &DecorateConsumer{ + ContainerOrchestrator: "EKS", + NextConsumer: consumertest.NewNop(), + K8sDecorator: &MockK8sDecorator{}, + MetricToUnitMap: metricToUnit, + Logger: logger, + } + ctx := context.Background() + + testcases := map[string]TestCase{ + "empty": { + metrics: pmetric.NewMetrics(), + want: pmetric.NewMetrics(), + shouldError: false, + }, + "unit": { + metrics: GenerateMetrics(map[MetricIdentifier]map[string]string{ + {util, pmetric.MetricTypeGauge}: { + "device": "test0", + }, + {memUtil, pmetric.MetricTypeGauge}: { + "device": "test0", + }, + {memTotal, pmetric.MetricTypeGauge}: { + "device": "test0", + }, + {memUsed, pmetric.MetricTypeGauge}: { + "device": "test0", + }, + {powerDraw, pmetric.MetricTypeGauge}: { + "device": "test0", + }, + {temp, pmetric.MetricTypeGauge}: { + "device": "test0", + }, + }), + want: GenerateMetrics(map[MetricIdentifier]map[string]string{ + {util, pmetric.MetricTypeGauge}: { + "device": "test0", + "Unit": "Percent", + }, + {memUtil, pmetric.MetricTypeGauge}: { + "device": "test0", + "Unit": "Percent", + }, + {memTotal, pmetric.MetricTypeGauge}: { + "device": "test0", + "Unit": "Bytes", + }, + {memUsed, pmetric.MetricTypeGauge}: { + "device": "test0", + "Unit": "Bytes", + }, + {powerDraw, pmetric.MetricTypeGauge}: { + "device": "test0", + "Unit": "None", + }, + {temp, pmetric.MetricTypeGauge}: { + "device": "test0", + "Unit": "None", + }, + }), + shouldError: false, + }, + "noUnit": { + metrics: GenerateMetrics(map[MetricIdentifier]map[string]string{ + {"test", pmetric.MetricTypeGauge}: { + "device": "test0", + }, + }), + want: GenerateMetrics(map[MetricIdentifier]map[string]string{ + {"test", pmetric.MetricTypeGauge}: { + "device": "test0", + }, + }), + shouldError: false, + }, + "typeUnchanged": { + metrics: GenerateMetrics(map[MetricIdentifier]map[string]string{ + {util, pmetric.MetricTypeGauge}: { + "device": "test0", + "Type": "TestType", + }, + }), + want: GenerateMetrics(map[MetricIdentifier]map[string]string{ + {util, pmetric.MetricTypeGauge}: { + "device": "test0", + "Type": "TestType", + "Unit": "Percent", + }, + }), + shouldError: false, + }, + } + + RunDecoratorTestScenarios(t, dc, ctx, testcases) +} diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator_testutils.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator_testutils.go new file mode 100644 index 000000000000..ad79a796443c --- /dev/null +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator_testutils.go @@ -0,0 +1,91 @@ +package decoratorconsumer + +import ( + "context" + "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "go.opentelemetry.io/collector/consumer" + "go.opentelemetry.io/collector/pdata/pcommon" + "go.opentelemetry.io/collector/pdata/pmetric" +) + +type MetricIdentifier struct { + Name string + MetricType pmetric.MetricType +} + +type TestCase struct { + Metrics pmetric.Metrics + Want pmetric.Metrics + ShouldError bool +} + +func RunDecoratorTestScenarios(t *testing.T, dc consumer.Metrics, ctx context.Context, testcases map[string]TestCase) { + 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.Want.MetricCount(), tc.Metrics.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 := getAttributesFromMetric(&actual) + wantAttrs := getAttributesFromMetric(&want) + 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(nameToDimsGauge map[MetricIdentifier]map[string]string) pmetric.Metrics { + md := pmetric.NewMetrics() + ms := md.ResourceMetrics().AppendEmpty().ScopeMetrics().AppendEmpty().Metrics() + for metric, dims := range nameToDimsGauge { + m := ms.AppendEmpty() + m.SetName(metric.Name) + metricBody := m.SetEmptyGauge().DataPoints().AppendEmpty() + if metric.MetricType == pmetric.MetricTypeSum { + metricBody = m.SetEmptySum().DataPoints().AppendEmpty() + } + metricBody.SetIntValue(10) + for k, v := range dims { + if k == "Unit" { + m.SetUnit(v) + continue + } + metricBody.Attributes().PutStr(k, v) + } + } + return md +} + +func getAttributesFromMetric(m *pmetric.Metric) pcommon.Map { + if m.Type() == pmetric.MetricTypeGauge { + return m.Gauge().DataPoints().At(0).Attributes() + } else { + return m.Sum().DataPoints().At(0).Attributes() + } +} diff --git a/receiver/awscontainerinsightreceiver/receiver.go b/receiver/awscontainerinsightreceiver/receiver.go index 30b8202023b7..d817aa17a88c 100644 --- a/receiver/awscontainerinsightreceiver/receiver.go +++ b/receiver/awscontainerinsightreceiver/receiver.go @@ -8,9 +8,9 @@ import ( "errors" "time" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/gpu" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/neuron" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/prometheusscraper" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer" "go.opentelemetry.io/collector/component" "go.opentelemetry.io/collector/consumer" "go.opentelemetry.io/collector/pdata/pmetric" @@ -192,7 +192,7 @@ func (acir *awsContainerInsightReceiver) initDcgmScraper(ctx context.Context, ho return nil } - decoConsumer := prometheusscraper.DecorateConsumer{ + decoConsumer := decoratorconsumer.DecorateConsumer{ ContainerOrchestrator: ci.EKS, NextConsumer: acir.nextConsumer, MetricType: ci.TypeNeuronContainer, @@ -220,7 +220,7 @@ func (acir *awsContainerInsightReceiver) initNeuronScraper(ctx context.Context, // return nil // } - decoConsumer := prometheusscraper.DecorateConsumer{ + decoConsumer := decoratorconsumer.DecorateConsumer{ ContainerOrchestrator: ci.EKS, NextConsumer: acir.nextConsumer, MetricType: ci.TypeNeuronContainer, From a20e010225b879c8fdcaddb23258bfe3bb7370aa Mon Sep 17 00:00:00 2001 From: Samarth Singh Date: Thu, 7 Mar 2024 14:31:25 +0000 Subject: [PATCH 46/53] more fixes --- internal/aws/proxy/server_test.go | 2 +- .../internal/gpu/dcgmscraper_config.go | 37 +++------- .../internal/gpu/dcgmscraper_test.go | 12 ---- .../internal/gpu/metric_map.go | 2 +- .../decoratorconsumer/decorator_test.go | 24 +++---- .../stores/staticPodResourcesRefresher.go | 67 ------------------- .../awscontainerinsightreceiver/receiver.go | 20 +++--- 7 files changed, 31 insertions(+), 133 deletions(-) delete mode 100644 receiver/awscontainerinsightreceiver/internal/stores/staticPodResourcesRefresher.go diff --git a/internal/aws/proxy/server_test.go b/internal/aws/proxy/server_test.go index c95108e395be..7b2a593a0c1f 100644 --- a/internal/aws/proxy/server_test.go +++ b/internal/aws/proxy/server_test.go @@ -226,7 +226,7 @@ func TestCanCreateTransport(t *testing.T) { _, err := NewServer(cfg, logger) assert.Error(t, err, "NewServer should fail") - assert.Contains(t, err.Error(), "failed to parse proxy URL") + assert.Contains(t, err.Error(), "invalid control character in URL") } func TestGetServiceEndpointInvalidAWSConfig(t *testing.T) { diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_config.go b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_config.go index 259cd10df432..7e954af8dc23 100644 --- a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_config.go +++ b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_config.go @@ -4,51 +4,26 @@ package gpu import ( - "context" "time" + 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/discovery/kubernetes" "github.com/prometheus/prometheus/model/relabel" - "go.opentelemetry.io/collector/component" - "go.opentelemetry.io/collector/consumer" - "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/prometheusscraper/decoratorconsumer" ) const ( - caFile = "/var/run/secrets/kubernetes.io/serviceaccount/ca.crt" + caFile = "/etc/amazon-cloudwatch-observability-agent-cert/tls-ca.crt" collectionInterval = 60 * time.Second jobName = "containerInsightsDCGMExporterScraper" scraperMetricsPath = "/metrics" scraperK8sServiceSelector = "k8s-app=dcgm-exporter-service" ) -type DcgmScraper struct { - ctx context.Context - settings component.TelemetrySettings - host component.Host - hostInfoProvider hostInfoProvider - prometheusReceiver receiver.Metrics - k8sDecorator decoratorconsumer.Decorator - running bool -} - -type DcgmScraperOpts struct { - Ctx context.Context - TelemetrySettings component.TelemetrySettings - Consumer consumer.Metrics - Host component.Host - HostInfoProvider hostInfoProvider - K8sDecorator decoratorconsumer.Decorator - Logger *zap.Logger -} - type hostInfoProvider interface { GetClusterName() string GetInstanceID() string @@ -56,10 +31,16 @@ type hostInfoProvider interface { func GetScraperConfig(hostInfoProvider hostInfoProvider) *config.ScrapeConfig { return &config.ScrapeConfig{ + HTTPClientConfig: configutil.HTTPClientConfig{ + TLSConfig: configutil.TLSConfig{ + CAFile: caFile, + InsecureSkipVerify: false, + }, + }, ScrapeInterval: model.Duration(collectionInterval), ScrapeTimeout: model.Duration(collectionInterval), JobName: jobName, - Scheme: "http", + Scheme: "https", MetricsPath: scraperMetricsPath, ServiceDiscoveryConfigs: discovery.Configs{ &kubernetes.SDConfig{ diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go index febcd9b9d3ad..e246b6732391 100644 --- a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go +++ b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go @@ -22,7 +22,6 @@ import ( 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/prometheusscraper" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/prometheusreceiver" ) @@ -51,17 +50,6 @@ 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 called *bool diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/metric_map.go b/receiver/awscontainerinsightreceiver/internal/gpu/metric_map.go index 08267206faa6..9e92b1771fc6 100644 --- a/receiver/awscontainerinsightreceiver/internal/gpu/metric_map.go +++ b/receiver/awscontainerinsightreceiver/internal/gpu/metric_map.go @@ -9,7 +9,7 @@ const ( gpuPowerDraw = "DCGM_FI_DEV_POWER_USAGE" ) -var metricToUnit = map[string]string{ +var MetricToUnit = map[string]string{ gpuUtil: "Percent", gpuMemUtil: "Percent", gpuMemUsed: "Bytes", diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator_test.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator_test.go index 7e7e417db79c..3071b345a6c8 100644 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator_test.go +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator_test.go @@ -55,12 +55,12 @@ func TestConsumeMetrics(t *testing.T) { testcases := map[string]TestCase{ "empty": { - metrics: pmetric.NewMetrics(), - want: pmetric.NewMetrics(), - shouldError: false, + Metrics: pmetric.NewMetrics(), + Want: pmetric.NewMetrics(), + ShouldError: false, }, "unit": { - metrics: GenerateMetrics(map[MetricIdentifier]map[string]string{ + Metrics: GenerateMetrics(map[MetricIdentifier]map[string]string{ {util, pmetric.MetricTypeGauge}: { "device": "test0", }, @@ -80,7 +80,7 @@ func TestConsumeMetrics(t *testing.T) { "device": "test0", }, }), - want: GenerateMetrics(map[MetricIdentifier]map[string]string{ + Want: GenerateMetrics(map[MetricIdentifier]map[string]string{ {util, pmetric.MetricTypeGauge}: { "device": "test0", "Unit": "Percent", @@ -106,36 +106,36 @@ func TestConsumeMetrics(t *testing.T) { "Unit": "None", }, }), - shouldError: false, + ShouldError: false, }, "noUnit": { - metrics: GenerateMetrics(map[MetricIdentifier]map[string]string{ + Metrics: GenerateMetrics(map[MetricIdentifier]map[string]string{ {"test", pmetric.MetricTypeGauge}: { "device": "test0", }, }), - want: GenerateMetrics(map[MetricIdentifier]map[string]string{ + Want: GenerateMetrics(map[MetricIdentifier]map[string]string{ {"test", pmetric.MetricTypeGauge}: { "device": "test0", }, }), - shouldError: false, + ShouldError: false, }, "typeUnchanged": { - metrics: GenerateMetrics(map[MetricIdentifier]map[string]string{ + Metrics: GenerateMetrics(map[MetricIdentifier]map[string]string{ {util, pmetric.MetricTypeGauge}: { "device": "test0", "Type": "TestType", }, }), - want: GenerateMetrics(map[MetricIdentifier]map[string]string{ + Want: GenerateMetrics(map[MetricIdentifier]map[string]string{ {util, pmetric.MetricTypeGauge}: { "device": "test0", "Type": "TestType", "Unit": "Percent", }, }), - shouldError: false, + ShouldError: false, }, } diff --git a/receiver/awscontainerinsightreceiver/internal/stores/staticPodResourcesRefresher.go b/receiver/awscontainerinsightreceiver/internal/stores/staticPodResourcesRefresher.go deleted file mode 100644 index f105341f603d..000000000000 --- a/receiver/awscontainerinsightreceiver/internal/stores/staticPodResourcesRefresher.go +++ /dev/null @@ -1,67 +0,0 @@ -package stores - -import ( - "context" - "fmt" - "go.uber.org/zap" - "google.golang.org/grpc" - "google.golang.org/grpc/credentials/insecure" - podresourcesapi "k8s.io/kubelet/pkg/apis/podresources/v1" - "net" - "os" - "time" -) - -const ( - socketPath = "/var/lib/kubelet/pod-resources/kubelet.sock" - connectionTimeout = 10 * time.Second -) - -func StartScraping(logger *zap.Logger) { - conn, cleanup, err := connectToServer(socketPath) - if err != nil { - fmt.Printf("error connecting to socket: %v\n", err) - os.Exit(1) - } - defer cleanup() - - client := podresourcesapi.NewPodResourcesListerClient(conn) - for { - list(&client, logger) - time.Sleep(30 * time.Second) - } -} - -func list(client *podresourcesapi.PodResourcesListerClient, logger *zap.Logger) { - ctx, cancel := context.WithTimeout(context.Background(), connectionTimeout) - defer cancel() - - logger.Info("static_pod_resources calling ListPodResources") - resp, err := (*client).List(ctx, &podresourcesapi.ListPodResourcesRequest{}) - if err != nil { - logger.Info("static_pod_resources error list resources: " + err.Error()) - } else { - logger.Info("static_pod_resources response: " + resp.String()) - } -} - -func connectToServer(socket string) (*grpc.ClientConn, func(), error) { - ctx, cancel := context.WithTimeout(context.Background(), connectionTimeout) - defer cancel() - - conn, err := grpc.DialContext(ctx, - socket, - grpc.WithTransportCredentials(insecure.NewCredentials()), - grpc.WithBlock(), - grpc.WithContextDialer(func(ctx context.Context, addr string) (net.Conn, error) { - d := net.Dialer{} - return d.DialContext(ctx, "unix", addr) - }), - ) - - if err != nil { - return nil, func() {}, fmt.Errorf("failure connecting to %s: %v", socket, err) - } - - return conn, func() { conn.Close() }, nil -} diff --git a/receiver/awscontainerinsightreceiver/receiver.go b/receiver/awscontainerinsightreceiver/receiver.go index d817aa17a88c..a973d9ce7026 100644 --- a/receiver/awscontainerinsightreceiver/receiver.go +++ b/receiver/awscontainerinsightreceiver/receiver.go @@ -195,7 +195,8 @@ func (acir *awsContainerInsightReceiver) initDcgmScraper(ctx context.Context, ho decoConsumer := decoratorconsumer.DecorateConsumer{ ContainerOrchestrator: ci.EKS, NextConsumer: acir.nextConsumer, - MetricType: ci.TypeNeuronContainer, + MetricType: ci.TypeGpuContainer, + MetricToUnitMap: gpu.MetricToUnit, K8sDecorator: decorator, Logger: acir.settings.Logger, } @@ -216,9 +217,9 @@ func (acir *awsContainerInsightReceiver) initDcgmScraper(ctx context.Context, ho } func (acir *awsContainerInsightReceiver) initNeuronScraper(ctx context.Context, host component.Host, hostinfo *hostInfo.Info, decorator *stores.K8sDecorator) error { - // if !acir.config.EnableNeuronMetric { - // return nil - // } + if !acir.config.EnableAwsNeuronMetrics { + return nil + } decoConsumer := decoratorconsumer.DecorateConsumer{ ContainerOrchestrator: ci.EKS, @@ -316,19 +317,14 @@ func (acir *awsContainerInsightReceiver) collectData(ctx context.Context) error acir.prometheusScraper.GetMetrics() //nolint:errcheck } - // if acir.dcgmScraper != nil { - // acir.dcgmScraper.GetMetrics() //nolint:errcheck - // } - - acir.settings.Logger.Info("We will start the Neuron Scraper") + if acir.dcgmScraper != nil { + acir.dcgmScraper.GetMetrics() //nolint:errcheck + } if acir.neuronMonitorScraper != nil { - acir.settings.Logger.Info("Neuron Scraper is not NIL") acir.neuronMonitorScraper.GetMetrics() //nolint:errcheck } - acir.settings.Logger.Info("If this happened Neuron is started or not") - for _, md := range mds { err := acir.nextConsumer.ConsumeMetrics(ctx, md) if err != nil { From def8c876be5ce033283b4df86e3329e71b62e772 Mon Sep 17 00:00:00 2001 From: Samarth Singh Date: Fri, 8 Mar 2024 12:22:35 +0000 Subject: [PATCH 47/53] Fix linting, comments and links --- .../internal/appsignals/useragent.go | 2 +- .../awscontainerinsightreceiver/config.go | 1 + .../cadvisor/extractors/extractorhelpers.go | 2 +- .../internal/gpu/dcgmscraper_config.go | 2 +- .../internal/gpu/metric_map.go | 5 +- .../internal/mocks/prometheus.go | 2 +- .../neuron/neuron_monitor_scraper_config.go | 2 +- .../neuron/pod_attribute_decorator.go | 36 ++++---- .../neuron/pod_attribute_decorator_test.go | 82 ++++++++++++++++++- .../decoratorconsumer/decorator.go | 2 +- .../decoratorconsumer/decorator_test.go | 3 + .../decoratorconsumer/decorator_testutils.go | 5 +- .../prometheus_scraper_testutils.go | 5 +- .../simple_prometheus_scraper.go | 5 +- .../simple_prometheus_scraper_test.go | 3 + 15 files changed, 127 insertions(+), 30 deletions(-) diff --git a/exporter/awsemfexporter/internal/appsignals/useragent.go b/exporter/awsemfexporter/internal/appsignals/useragent.go index d05d5929b6d1..12d5817e0cb7 100644 --- a/exporter/awsemfexporter/internal/appsignals/useragent.go +++ b/exporter/awsemfexporter/internal/appsignals/useragent.go @@ -1,7 +1,7 @@ // Copyright The OpenTelemetry Authors // SPDX-License-Identifier: Apache-2.0 -package appsignals +package appsignals // import "github.com/open-telemetry/opentelemetry-collector-contrib/exporter/awsemfexporter/internal/appsignals" import ( "context" diff --git a/receiver/awscontainerinsightreceiver/config.go b/receiver/awscontainerinsightreceiver/config.go index c6515334f680..7818ba8818b2 100644 --- a/receiver/awscontainerinsightreceiver/config.go +++ b/receiver/awscontainerinsightreceiver/config.go @@ -60,6 +60,7 @@ type Config struct { // EnableAcceleratedComputeMetrics enabled features with accelerated compute resources where metrics are scraped from vendor specific sources EnableAcceleratedComputeMetrics bool `mapstructure:"accelerated_compute_metrics"` + EnableGpuMetric bool // EnableAwsNeuronMetrics toggles Neuron monitoring where metrics are scraped from neuron monitor // The default value is false. diff --git a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractorhelpers.go b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractorhelpers.go index 6bc91a48dfe5..402a598b63ed 100644 --- a/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractorhelpers.go +++ b/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors/extractorhelpers.go @@ -1,7 +1,7 @@ // Copyright The OpenTelemetry Authors // SPDX-License-Identifier: Apache-2.0 -package extractors +package extractors // import "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/cadvisor/extractors" import ( "fmt" diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_config.go b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_config.go index 7e954af8dc23..af0bad1bc1cd 100644 --- a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_config.go +++ b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_config.go @@ -1,7 +1,7 @@ // Copyright The OpenTelemetry Authors // SPDX-License-Identifier: Apache-2.0 -package gpu +package gpu // import "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/gpu" import ( "time" diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/metric_map.go b/receiver/awscontainerinsightreceiver/internal/gpu/metric_map.go index 9e92b1771fc6..c46a507d8c5d 100644 --- a/receiver/awscontainerinsightreceiver/internal/gpu/metric_map.go +++ b/receiver/awscontainerinsightreceiver/internal/gpu/metric_map.go @@ -1,4 +1,7 @@ -package gpu +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +package gpu // import "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/gpu" const ( gpuUtil = "DCGM_FI_DEV_GPU_UTIL" diff --git a/receiver/awscontainerinsightreceiver/internal/mocks/prometheus.go b/receiver/awscontainerinsightreceiver/internal/mocks/prometheus.go index d9b90220df72..3efce32e9f07 100644 --- a/receiver/awscontainerinsightreceiver/internal/mocks/prometheus.go +++ b/receiver/awscontainerinsightreceiver/internal/mocks/prometheus.go @@ -1,7 +1,7 @@ // Copyright The OpenTelemetry Authors // SPDX-License-Identifier: Apache-2.0 -package mocks +package mocks // import "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/mocks" import ( "fmt" diff --git a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go index a788c25571a9..1e181a33b0be 100644 --- a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go +++ b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go @@ -1,7 +1,7 @@ // Copyright The OpenTelemetry Authors // SPDX-License-Identifier: Apache-2.0 -package neuron +package neuron // import "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/neuron" import ( "time" diff --git a/receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator.go b/receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator.go index 934824c06806..ab6df6a39517 100644 --- a/receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator.go +++ b/receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator.go @@ -1,4 +1,4 @@ -package neuron +package neuron // import "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/neuron" import ( "context" @@ -52,18 +52,9 @@ func (pdc *PodAttributesDecoratorConsumer) neuronMetricsProcess(md pmetric.Metri ils := ilms.At(j) metrics := ils.Metrics() - neuronHardwareInfo := pmetric.Metric{} - neuronHardwareInfoFound := false - for k := 0; k < metrics.Len(); k++ { - m := metrics.At(k) - if m.Name() == neuronHardwareInfoKey { - neuronHardwareInfo = m - neuronHardwareInfoFound = true - break - } - } + neuronHardwareInfo, neuronHardwareInfoFound := findNeuronHardwareInfo(metrics) if neuronHardwareInfoFound { - neuronCoresPerDevice := getNueronCoresPerDevice(neuronHardwareInfo) + neuronCoresPerDevice := getNeuronCoresPerDevice(neuronHardwareInfo) for k := 0; k < metrics.Len(); k++ { m := metrics.At(k) pdc.addPodCorrelationAttributes(getMetricDatapoints(m), neuronCoresPerDevice) @@ -73,6 +64,20 @@ func (pdc *PodAttributesDecoratorConsumer) neuronMetricsProcess(md pmetric.Metri } } +func findNeuronHardwareInfo(metrics pmetric.MetricSlice) (pmetric.Metric, bool) { + var neuronHardwareInfo pmetric.Metric + neuronHardwareInfoFound := false + for k := 0; k < metrics.Len(); k++ { + m := metrics.At(k) + if m.Name() == neuronHardwareInfoKey { + neuronHardwareInfo = m + neuronHardwareInfoFound = true + break + } + } + return neuronHardwareInfo, neuronHardwareInfoFound +} + func (pdc *PodAttributesDecoratorConsumer) addPodCorrelationAttributes(metricDatapoints pmetric.NumberDataPointSlice, neuronCoresPerDevice int) { for i := 0; i < metricDatapoints.Len(); i++ { attributes := metricDatapoints.At(i).Attributes() @@ -110,7 +115,6 @@ func populateAttributes(attributes *pcommon.Map, containerInfo *stores.Container if containerInfo != nil { attributes.PutStr(ci.AttributeContainerName, containerInfo.ContainerName) attributes.PutStr(ci.AttributeK8sPodName, containerInfo.PodName) - attributes.PutStr(ci.AttributePodName, containerInfo.PodName) attributes.PutStr(ci.AttributeK8sNamespace, containerInfo.Namespace) } } @@ -127,15 +131,15 @@ func getMetricDatapoints(m pmetric.Metric) pmetric.NumberDataPointSlice { } // We extract the attribute named `neuroncore_per_device_count` from the metric to get the value -// https://awsdocs-neuron.readthedocs-hosted.com/en/latest/tools/neuron-sys-tools/neuron-monitor-user-guide -func getNueronCoresPerDevice(neuronHardwareInfo pmetric.Metric) int { +// https://awsdocs-neuron.readthedocs-hosted.com/en/latest/tools/neuron-sys-tools/neuron-monitor-user-guide.html +func getNeuronCoresPerDevice(neuronHardwareInfo pmetric.Metric) int { neuronCoresPerDeviceValue, _ := neuronHardwareInfo.Sum().DataPoints().At(0).Attributes().Get(neuronCorePerDeviceKey) neuronCoresPerDevice, _ := strconv.Atoi(neuronCoresPerDeviceValue.AsString()) return neuronCoresPerDevice } // To get the device index from core index we divide the index by cores in a single device -// https://awsdocs-neuron.readthedocs-hosted.com/en/latest/tools/neuron-sys-tools/neuron-monitor-user-guide +// https://awsdocs-neuron.readthedocs-hosted.com/en/latest/tools/neuron-sys-tools/neuron-monitor-user-guide.html func getNeuronDeviceIndexFromCoreAttribute(neuronCoreIndex pcommon.Value, neuronCoresPerDevice int) string { neuronCoreIndexIntVal, _ := strconv.Atoi(neuronCoreIndex.AsString()) return strconv.Itoa(neuronCoreIndexIntVal / neuronCoresPerDevice) diff --git a/receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator_test.go b/receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator_test.go index d96a6d4ee51b..f8ebc00237c4 100644 --- a/receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator_test.go +++ b/receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator_test.go @@ -16,6 +16,7 @@ import ( ) var dummyPodName = "pod-name" +var dummyPodNameForAltResource = "pod-name-alt" var dummyContainerName = "container-name" var dummyNamespace = "namespace" @@ -30,6 +31,20 @@ func (m mockPodResourcesStore) GetContainerInfo(deviceIndex string, resourceName } } +type mockPodResourcesStoreWithAltResourceName struct { +} + +func (m mockPodResourcesStoreWithAltResourceName) GetContainerInfo(deviceIndex string, resourceName string) *stores.ContainerInfo { + if resourceName == neuronDeviceResourceNameAlt { + return &stores.ContainerInfo{ + PodName: dummyPodNameForAltResource, + ContainerName: dummyContainerName, + Namespace: dummyNamespace, + } + } + return nil +} + func TestConsumeMetricsForPodAttributeDecorator(t *testing.T) { logger, _ := zap.NewDevelopment() dc := &PodAttributesDecoratorConsumer{ @@ -39,7 +54,7 @@ func TestConsumeMetricsForPodAttributeDecorator(t *testing.T) { } ctx := context.Background() - testcases := map[string]decoratorconsumer.TestCase{ + testcases1 := map[string]decoratorconsumer.TestCase{ "empty": { Metrics: pmetric.NewMetrics(), Want: pmetric.NewMetrics(), @@ -78,7 +93,6 @@ func TestConsumeMetricsForPodAttributeDecorator(t *testing.T) { neuronDeviceAttributeKey: "1", ci.AttributeContainerName: dummyContainerName, ci.AttributeK8sPodName: dummyPodName, - ci.AttributePodName: dummyPodName, ci.AttributeK8sNamespace: dummyNamespace, }, }), @@ -104,7 +118,67 @@ func TestConsumeMetricsForPodAttributeDecorator(t *testing.T) { neuronDeviceAttributeKey: "5", ci.AttributeContainerName: dummyContainerName, ci.AttributeK8sPodName: dummyPodName, - ci.AttributePodName: dummyPodName, + ci.AttributeK8sNamespace: dummyNamespace, + }, + }), + ShouldError: false, + }, + "correlation_when_both_present": { + Metrics: decoratorconsumer.GenerateMetrics(map[decoratorconsumer.MetricIdentifier]map[string]string{ + {Name: neuronHardwareInfoKey, MetricType: pmetric.MetricTypeSum}: { + neuronCorePerDeviceKey: "2", + }, + {Name: "test", MetricType: pmetric.MetricTypeGauge}: { + "device": "test0", + neuronDeviceAttributeKey: "5", + neuronCoreAttributeKey: "10", + }, + }), + Want: decoratorconsumer.GenerateMetrics(map[decoratorconsumer.MetricIdentifier]map[string]string{ + {Name: neuronHardwareInfoKey, MetricType: pmetric.MetricTypeSum}: { + neuronCorePerDeviceKey: "2", + }, + {Name: "test", MetricType: pmetric.MetricTypeGauge}: { + "device": "test0", + neuronCoreAttributeKey: "10", + neuronDeviceAttributeKey: "5", + ci.AttributeContainerName: dummyContainerName, + ci.AttributeK8sPodName: dummyPodName, + ci.AttributeK8sNamespace: dummyNamespace, + }, + }), + ShouldError: false, + }, + } + + decoratorconsumer.RunDecoratorTestScenarios(t, dc, ctx, testcases1) + + dc = &PodAttributesDecoratorConsumer{ + NextConsumer: consumertest.NewNop(), + PodResourcesStore: mockPodResourcesStoreWithAltResourceName{}, + Logger: logger, + } + + testcases2 := map[string]decoratorconsumer.TestCase{ + "correlation_via_neuron_device_index_alt_name": { + Metrics: decoratorconsumer.GenerateMetrics(map[decoratorconsumer.MetricIdentifier]map[string]string{ + {Name: neuronHardwareInfoKey, MetricType: pmetric.MetricTypeSum}: { + neuronCorePerDeviceKey: "2", + }, + {Name: "test", MetricType: pmetric.MetricTypeGauge}: { + "device": "test0", + neuronDeviceAttributeKey: "1", + }, + }), + Want: decoratorconsumer.GenerateMetrics(map[decoratorconsumer.MetricIdentifier]map[string]string{ + {Name: neuronHardwareInfoKey, MetricType: pmetric.MetricTypeSum}: { + neuronCorePerDeviceKey: "2", + }, + {Name: "test", MetricType: pmetric.MetricTypeGauge}: { + "device": "test0", + neuronDeviceAttributeKey: "1", + ci.AttributeContainerName: dummyContainerName, + ci.AttributeK8sPodName: dummyPodNameForAltResource, ci.AttributeK8sNamespace: dummyNamespace, }, }), @@ -112,5 +186,5 @@ func TestConsumeMetricsForPodAttributeDecorator(t *testing.T) { }, } - decoratorconsumer.RunDecoratorTestScenarios(t, dc, ctx, testcases) + decoratorconsumer.RunDecoratorTestScenarios(t, dc, ctx, testcases2) } diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator.go index 4bbb9a31fc5d..6b58a3a9d8af 100644 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator.go +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator.go @@ -1,7 +1,7 @@ // Copyright The OpenTelemetry Authors // SPDX-License-Identifier: Apache-2.0 -package decoratorconsumer +package decoratorconsumer // import "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer" import ( "context" diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator_test.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator_test.go index 3071b345a6c8..1d20491d0f35 100644 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator_test.go +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator_test.go @@ -1,3 +1,6 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + package decoratorconsumer import ( diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator_testutils.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator_testutils.go index ad79a796443c..9ce9a8bc3b51 100644 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator_testutils.go +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator_testutils.go @@ -1,4 +1,7 @@ -package decoratorconsumer +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +package decoratorconsumer // import "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer" import ( "context" diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/prometheus_scraper_testutils.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/prometheus_scraper_testutils.go index 70257e5cf16e..e222298b22ed 100644 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/prometheus_scraper_testutils.go +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/prometheus_scraper_testutils.go @@ -1,4 +1,7 @@ -package prometheusscraper +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +package prometheusscraper // import "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/prometheusscraper" import ( "context" diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go index 285d08656eb4..df5565fabeb9 100644 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go @@ -1,4 +1,7 @@ -package prometheusscraper +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +package prometheusscraper // import "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/prometheusscraper" import ( "context" diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper_test.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper_test.go index 48fd1b04ab2f..c97dafc76429 100644 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper_test.go +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper_test.go @@ -1,3 +1,6 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + package prometheusscraper import ( From 8aec9f555743f866468ee89a908ef7e62e7104e8 Mon Sep 17 00:00:00 2001 From: Samarth Singh Date: Fri, 8 Mar 2024 14:35:39 +0000 Subject: [PATCH 48/53] Add more fix regarding license --- receiver/awscontainerinsightreceiver/config.go | 1 - .../internal/gpu/dcgmscraper_test.go | 2 +- .../internal/neuron/neuron_monitor_scraper_test.go | 3 +++ .../internal/neuron/pod_attribute_decorator.go | 3 +++ .../internal/prometheusscraper/simple_prometheus_scraper.go | 2 ++ 5 files changed, 9 insertions(+), 2 deletions(-) diff --git a/receiver/awscontainerinsightreceiver/config.go b/receiver/awscontainerinsightreceiver/config.go index 7818ba8818b2..c6515334f680 100644 --- a/receiver/awscontainerinsightreceiver/config.go +++ b/receiver/awscontainerinsightreceiver/config.go @@ -60,7 +60,6 @@ type Config struct { // EnableAcceleratedComputeMetrics enabled features with accelerated compute resources where metrics are scraped from vendor specific sources EnableAcceleratedComputeMetrics bool `mapstructure:"accelerated_compute_metrics"` - EnableGpuMetric bool // EnableAwsNeuronMetrics toggles Neuron monitoring where metrics are scraped from neuron monitor // The default value is false. diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go index e246b6732391..f5920c3004f0 100644 --- a/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go +++ b/receiver/awscontainerinsightreceiver/internal/gpu/dcgmscraper_test.go @@ -161,7 +161,7 @@ func TestNewDcgmScraperEndToEnd(t *testing.T) { mp, cfg, err := mocks.SetupMockPrometheus(targets...) assert.NoError(t, err) - scrapeConfig := GetScraperConfig(scraper.HostInfoProvider) + scrapeConfig := scraper.ScraperConfigs scrapeConfig.ScrapeInterval = cfg.ScrapeConfigs[0].ScrapeInterval scrapeConfig.ScrapeTimeout = cfg.ScrapeConfigs[0].ScrapeInterval scrapeConfig.Scheme = "http" diff --git a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go index d6c61619ac1d..2932fb0c4366 100644 --- a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go +++ b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go @@ -1,3 +1,6 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + package neuron import ( diff --git a/receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator.go b/receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator.go index ab6df6a39517..f31a9454dfac 100644 --- a/receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator.go +++ b/receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator.go @@ -1,3 +1,6 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + package neuron // import "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/neuron" import ( diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go index df5565fabeb9..50815d3d9591 100644 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/simple_prometheus_scraper.go @@ -23,6 +23,7 @@ type SimplePrometheusScraper struct { host component.Host HostInfoProvider HostInfoProvider PrometheusReceiver receiver.Metrics + ScraperConfigs *config.ScrapeConfig running bool } @@ -73,6 +74,7 @@ func NewSimplePrometheusScraper(opts SimplePrometheusScraperOpts) (*SimplePromet Settings: opts.TelemetrySettings, host: opts.Host, HostInfoProvider: opts.HostInfoProvider, + ScraperConfigs: opts.ScraperConfigs, PrometheusReceiver: promReceiver, }, nil } From 65771bbefbc407bc45d228840b62aa4214b01946 Mon Sep 17 00:00:00 2001 From: Samarth Singh Date: Fri, 8 Mar 2024 17:06:01 +0000 Subject: [PATCH 49/53] Add additional labels for agent processor --- .../neuron/neuron_monitor_scraper_config.go | 41 +++++++++++++++---- .../neuron/neuron_monitor_scraper_test.go | 21 +++++++++- .../neuron/pod_attribute_decorator.go | 2 +- .../neuron/pod_attribute_decorator_test.go | 8 ++-- .../decoratorconsumer/decorator_test.go | 2 +- .../decoratorconsumer/decorator_testutils.go | 5 +-- .../prometheus_scraper_testutils.go | 3 +- .../awscontainerinsightreceiver/receiver.go | 2 +- 8 files changed, 63 insertions(+), 21 deletions(-) diff --git a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go index 1e181a33b0be..9193beb46516 100644 --- a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go +++ b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go @@ -4,6 +4,7 @@ package neuron // import "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/neuron" import ( + "os" "time" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/prometheusscraper" @@ -12,12 +13,15 @@ import ( "github.com/prometheus/prometheus/discovery" "github.com/prometheus/prometheus/discovery/kubernetes" "github.com/prometheus/prometheus/model/relabel" + + ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" ) const ( - caFile = "/var/run/secrets/kubernetes.io/serviceaccount/ca.crt" - collectionInterval = 60 * time.Second - jobName = "containerInsightsNeuronMonitorScraper" + collectionInterval = 60 * time.Second + jobName = "containerInsightsNeuronMonitorScraper" + scraperMetricsPath = "/metrics" + scraperK8sServiceSelector = "k8s-app=neuron-monitor-service" ) func GetNeuronScrapeConfig(hostinfo prometheusscraper.HostInfoProvider) *config.ScrapeConfig { @@ -27,7 +31,7 @@ func GetNeuronScrapeConfig(hostinfo prometheusscraper.HostInfoProvider) *config. ScrapeTimeout: model.Duration(collectionInterval), JobName: jobName, Scheme: "http", - MetricsPath: "/metrics", + MetricsPath: scraperMetricsPath, ServiceDiscoveryConfigs: discovery.Configs{ &kubernetes.SDConfig{ Role: kubernetes.RoleService, @@ -37,7 +41,7 @@ func GetNeuronScrapeConfig(hostinfo prometheusscraper.HostInfoProvider) *config. Selectors: []kubernetes.SelectorConfig{ { Role: kubernetes.RoleService, - Label: "k8s-app=neuron-monitor-service", + Label: scraperK8sServiceSelector, }, }, }, @@ -56,14 +60,28 @@ func GetNeuronMetricRelabelConfigs(hostinfo prometheusscraper.HostInfoProvider) }, { SourceLabels: model.LabelNames{"instance_id"}, - TargetLabel: "InstanceId", + TargetLabel: ci.InstanceID, + Regex: relabel.MustNewRegexp("(.*)"), + Replacement: "${1}", + Action: relabel.Replace, + }, + { + SourceLabels: model.LabelNames{"instance_type"}, + TargetLabel: ci.InstanceType, Regex: relabel.MustNewRegexp("(.*)"), Replacement: "${1}", Action: relabel.Replace, }, { SourceLabels: model.LabelNames{"neuroncore"}, - TargetLabel: "DeviceId", + TargetLabel: "NeuronCore", + Regex: relabel.MustNewRegexp("(.*)"), + Replacement: "${1}", + Action: relabel.Replace, + }, + { + SourceLabels: model.LabelNames{"neuron_device_index"}, + TargetLabel: "NeuronDevice", Regex: relabel.MustNewRegexp("(.*)"), Replacement: "${1}", Action: relabel.Replace, @@ -72,10 +90,17 @@ func GetNeuronMetricRelabelConfigs(hostinfo prometheusscraper.HostInfoProvider) // relabel looks up an existing label then creates another label with given key (TargetLabel) and value (static) { SourceLabels: model.LabelNames{"instance_id"}, - TargetLabel: "ClusterName", + TargetLabel: ci.ClusterNameKey, Regex: relabel.MustNewRegexp("(.*)"), Replacement: hostinfo.GetClusterName(), Action: relabel.Replace, }, + { + SourceLabels: model.LabelNames{"instance_id"}, + TargetLabel: ci.NodeNameKey, + Regex: relabel.MustNewRegexp("(.*)"), + Replacement: os.Getenv("K8S_NODE_NAME"), + Action: relabel.Replace, + }, } } diff --git a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go index 2932fb0c4366..43836e257192 100644 --- a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go +++ b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go @@ -5,6 +5,7 @@ package neuron import ( "context" + "os" "strings" "testing" @@ -29,10 +30,14 @@ neuroncore_utilization_ratio{availability_zone="us-east-1c",instance_id="i-09db9 # HELP system_memory_total_bytes System memory total_bytes bytes # TYPE system_memory_total_bytes gauge system_memory_total_bytes{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",region="us-east-1",subnet_id="subnet-06a7754948e8a000f"} 5.32523487232e+011 +# HELP neurondevice_hw_ecc_events_total_mem_ecc_corrected Neuron hardware errors +# TYPE neurondevice_hw_ecc_events_total_mem_ecc_corrected gauge +neurondevice_hw_ecc_events_total_mem_ecc_corrected{availability_zone="us-east-1c",instance_id="i-09db9b55e0095612f",instance_name="",instance_type="trn1n.32xlarge",neuron_device_index="5",region="us-east-1",runtime_tag="367",subnet_id="subnet-06a7754948e8a000f"} 3 ` const dummyClusterName = "cluster-name" const dummyHostName = "i-000000000" +const dummyNodeName = "dummy-nodeName" type mockHostInfoProvider struct { } @@ -46,13 +51,24 @@ func (m mockHostInfoProvider) GetInstanceID() string { } func TestNewNeuronScraperEndToEnd(t *testing.T) { + os.Setenv("K8S_NODE_NAME", dummyNodeName) expectedMetrics := make(map[string]prometheusscraper.ExpectedMetricStruct) expectedMetrics["neuroncore_utilization_ratio"] = prometheusscraper.ExpectedMetricStruct{ MetricValue: 0.1, MetricLabels: []prometheusscraper.MetricLabel{ {LabelName: "InstanceId", LabelValue: "i-09db9b55e0095612f"}, {LabelName: "ClusterName", LabelValue: dummyClusterName}, - {LabelName: "DeviceId", LabelValue: "0"}, + {LabelName: "NeuronCore", LabelValue: "0"}, + {LabelName: "NodeName", LabelValue: dummyNodeName}, + }, + } + expectedMetrics["neurondevice_hw_ecc_events_total_mem_ecc_corrected"] = prometheusscraper.ExpectedMetricStruct{ + MetricValue: 3, + MetricLabels: []prometheusscraper.MetricLabel{ + {LabelName: "InstanceId", LabelValue: "i-09db9b55e0095612f"}, + {LabelName: "ClusterName", LabelValue: dummyClusterName}, + {LabelName: "NeuronDevice", LabelValue: "5"}, + {LabelName: "NodeName", LabelValue: dummyNodeName}, }, } expectedMetrics["neuron_runtime_memory_used_bytes"] = prometheusscraper.ExpectedMetricStruct{ @@ -60,6 +76,7 @@ func TestNewNeuronScraperEndToEnd(t *testing.T) { MetricLabels: []prometheusscraper.MetricLabel{ {LabelName: "InstanceId", LabelValue: "i-09db9b55e0095612f"}, {LabelName: "ClusterName", LabelValue: dummyClusterName}, + {LabelName: "NodeName", LabelValue: dummyNodeName}, }, } @@ -68,6 +85,7 @@ func TestNewNeuronScraperEndToEnd(t *testing.T) { MetricLabels: []prometheusscraper.MetricLabel{ {LabelName: "InstanceId", LabelValue: "i-09db9b55e0095612f"}, {LabelName: "ClusterName", LabelValue: dummyClusterName}, + {LabelName: "NodeName", LabelValue: dummyNodeName}, }, } @@ -76,6 +94,7 @@ func TestNewNeuronScraperEndToEnd(t *testing.T) { MetricLabels: []prometheusscraper.MetricLabel{ {LabelName: "InstanceId", LabelValue: "i-09db9b55e0095612f"}, {LabelName: "ClusterName", LabelValue: dummyClusterName}, + {LabelName: "NodeName", LabelValue: dummyNodeName}, }, } diff --git a/receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator.go b/receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator.go index f31a9454dfac..5538081f32fe 100644 --- a/receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator.go +++ b/receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator.go @@ -19,7 +19,7 @@ const ( neuronHardwareInfoKey = "neuron_hardware" neuronCorePerDeviceKey = "neuroncore_per_device_count" neuronCoreAttributeKey = "neuroncore" - neuronDeviceAttributeKey = "neuron_device_index" + neuronDeviceAttributeKey = "NeuronDevice" neuronCoreResourceName = "aws.amazon.com/neuroncore" neuronDeviceResourceName = "aws.amazon.com/neurondevice" neuronDeviceResourceNameAlt = "aws.amazon.com/neuron" diff --git a/receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator_test.go b/receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator_test.go index f8ebc00237c4..01fb143a0305 100644 --- a/receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator_test.go +++ b/receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator_test.go @@ -23,7 +23,7 @@ var dummyNamespace = "namespace" type mockPodResourcesStore struct { } -func (m mockPodResourcesStore) GetContainerInfo(deviceIndex string, resourceName string) *stores.ContainerInfo { +func (m mockPodResourcesStore) GetContainerInfo(_ string, _ string) *stores.ContainerInfo { return &stores.ContainerInfo{ PodName: dummyPodName, ContainerName: dummyContainerName, @@ -34,7 +34,7 @@ func (m mockPodResourcesStore) GetContainerInfo(deviceIndex string, resourceName type mockPodResourcesStoreWithAltResourceName struct { } -func (m mockPodResourcesStoreWithAltResourceName) GetContainerInfo(deviceIndex string, resourceName string) *stores.ContainerInfo { +func (m mockPodResourcesStoreWithAltResourceName) GetContainerInfo(_ string, resourceName string) *stores.ContainerInfo { if resourceName == neuronDeviceResourceNameAlt { return &stores.ContainerInfo{ PodName: dummyPodNameForAltResource, @@ -151,7 +151,7 @@ func TestConsumeMetricsForPodAttributeDecorator(t *testing.T) { }, } - decoratorconsumer.RunDecoratorTestScenarios(t, dc, ctx, testcases1) + decoratorconsumer.RunDecoratorTestScenarios(ctx, t, dc, testcases1) dc = &PodAttributesDecoratorConsumer{ NextConsumer: consumertest.NewNop(), @@ -186,5 +186,5 @@ func TestConsumeMetricsForPodAttributeDecorator(t *testing.T) { }, } - decoratorconsumer.RunDecoratorTestScenarios(t, dc, ctx, testcases2) + decoratorconsumer.RunDecoratorTestScenarios(ctx, t, dc, testcases2) } diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator_test.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator_test.go index 1d20491d0f35..6975979a75d5 100644 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator_test.go +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator_test.go @@ -142,5 +142,5 @@ func TestConsumeMetrics(t *testing.T) { }, } - RunDecoratorTestScenarios(t, dc, ctx, testcases) + RunDecoratorTestScenarios(ctx, t, dc, testcases) } diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator_testutils.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator_testutils.go index 9ce9a8bc3b51..171a0110db88 100644 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator_testutils.go +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator_testutils.go @@ -25,7 +25,7 @@ type TestCase struct { ShouldError bool } -func RunDecoratorTestScenarios(t *testing.T, dc consumer.Metrics, ctx context.Context, testcases map[string]TestCase) { +func RunDecoratorTestScenarios(ctx context.Context, t *testing.T, dc consumer.Metrics, testcases map[string]TestCase) { for _, tc := range testcases { err := dc.ConsumeMetrics(ctx, tc.Metrics) if tc.ShouldError { @@ -88,7 +88,6 @@ func GenerateMetrics(nameToDimsGauge map[MetricIdentifier]map[string]string) pme func getAttributesFromMetric(m *pmetric.Metric) pcommon.Map { if m.Type() == pmetric.MetricTypeGauge { return m.Gauge().DataPoints().At(0).Attributes() - } else { - return m.Sum().DataPoints().At(0).Attributes() } + return m.Sum().DataPoints().At(0).Attributes() } diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/prometheus_scraper_testutils.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/prometheus_scraper_testutils.go index e222298b22ed..a7144383e4e0 100644 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/prometheus_scraper_testutils.go +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/prometheus_scraper_testutils.go @@ -60,7 +60,6 @@ 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) - fmt.Println(metric.Name()) metricsStruct, ok := m.ExpectedMetrics[metric.Name()] if ok { assert.Equal(m.T, metricsStruct.MetricValue, metric.Gauge().DataPoints().At(0).DoubleValue()) @@ -69,7 +68,7 @@ func (m MockConsumer) ConsumeMetrics(_ context.Context, md pmetric.Metrics) erro assert.True(m.T, isFound) assert.Equal(m.T, expectedLabel.LabelValue, labelValue.Str()) } - metricFoundCount += 1 + metricFoundCount++ } } diff --git a/receiver/awscontainerinsightreceiver/receiver.go b/receiver/awscontainerinsightreceiver/receiver.go index b885bd043a78..c87c8ca37e28 100644 --- a/receiver/awscontainerinsightreceiver/receiver.go +++ b/receiver/awscontainerinsightreceiver/receiver.go @@ -291,7 +291,7 @@ func (acir *awsContainerInsightReceiver) Shutdown(context.Context) error { } if acir.k8sDecorator != nil { - acir.k8sDecorator.Shutdown() + errs = errors.Join(errs, acir.k8sDecorator.Shutdown()) } if acir.podResourcesStore != nil { From 9f00e3db4ecbe278fb945da3f5223766b68411f7 Mon Sep 17 00:00:00 2001 From: Samarth Singh Date: Tue, 12 Mar 2024 11:22:01 +0000 Subject: [PATCH 50/53] Fix lint checks --- .../internal/neuron/neuron_monitor_scraper_config.go | 2 +- .../internal/neuron/neuron_monitor_scraper_test.go | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go index 9193beb46516..74370c49827a 100644 --- a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go +++ b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go @@ -99,7 +99,7 @@ func GetNeuronMetricRelabelConfigs(hostinfo prometheusscraper.HostInfoProvider) SourceLabels: model.LabelNames{"instance_id"}, TargetLabel: ci.NodeNameKey, Regex: relabel.MustNewRegexp("(.*)"), - Replacement: os.Getenv("K8S_NODE_NAME"), + Replacement: os.Getenv("HOST_NAME"), Action: relabel.Replace, }, } diff --git a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go index 43836e257192..8628a57498f0 100644 --- a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go +++ b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_test.go @@ -5,7 +5,6 @@ package neuron import ( "context" - "os" "strings" "testing" @@ -51,7 +50,7 @@ func (m mockHostInfoProvider) GetInstanceID() string { } func TestNewNeuronScraperEndToEnd(t *testing.T) { - os.Setenv("K8S_NODE_NAME", dummyNodeName) + t.Setenv("HOST_NAME", dummyNodeName) expectedMetrics := make(map[string]prometheusscraper.ExpectedMetricStruct) expectedMetrics["neuroncore_utilization_ratio"] = prometheusscraper.ExpectedMetricStruct{ MetricValue: 0.1, From 31b7d3a7c8009816668890fa8777f996b47a930d Mon Sep 17 00:00:00 2001 From: Samarth Singh Date: Tue, 12 Mar 2024 14:17:28 +0000 Subject: [PATCH 51/53] fix go imports --- .../internal/neuron/neuron_monitor_scraper_config.go | 2 +- .../prometheusscraper/decoratorconsumer/decorator.go | 5 ++--- .../prometheusscraper/decoratorconsumer/decorator_test.go | 3 +-- .../prometheusscraper/prometheus_scraper_testutils.go | 5 +++-- receiver/awscontainerinsightreceiver/receiver.go | 6 +++--- 5 files changed, 10 insertions(+), 11 deletions(-) diff --git a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go index 74370c49827a..1c0af3d15614 100644 --- a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go +++ b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go @@ -7,7 +7,6 @@ import ( "os" "time" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/prometheusscraper" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/discovery" @@ -15,6 +14,7 @@ import ( "github.com/prometheus/prometheus/model/relabel" ci "github.com/open-telemetry/opentelemetry-collector-contrib/internal/aws/containerinsight" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/prometheusscraper" ) const ( diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator.go index 6b58a3a9d8af..d999ca7999b7 100644 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator.go +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator.go @@ -6,13 +6,12 @@ package decoratorconsumer // import "github.com/open-telemetry/opentelemetry-col 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" ) // Decorator acts as an interceptor of metrics before the scraper sends them to the next designated consumer diff --git a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator_test.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator_test.go index 6975979a75d5..be7663b93eba 100644 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator_test.go +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer/decorator_test.go @@ -7,11 +7,10 @@ import ( "context" "testing" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" "go.opentelemetry.io/collector/consumer/consumertest" "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/prometheusscraper/prometheus_scraper_testutils.go b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/prometheus_scraper_testutils.go index a7144383e4e0..bce5bc78e715 100644 --- a/receiver/awscontainerinsightreceiver/internal/prometheusscraper/prometheus_scraper_testutils.go +++ b/receiver/awscontainerinsightreceiver/internal/prometheusscraper/prometheus_scraper_testutils.go @@ -9,8 +9,6 @@ import ( "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" @@ -20,6 +18,9 @@ import ( "go.opentelemetry.io/collector/consumer" "go.opentelemetry.io/collector/pdata/pmetric" "go.opentelemetry.io/collector/receiver" + + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/mocks" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/prometheusreceiver" ) type MetricLabel struct { diff --git a/receiver/awscontainerinsightreceiver/receiver.go b/receiver/awscontainerinsightreceiver/receiver.go index c87c8ca37e28..4cf10c21c651 100644 --- a/receiver/awscontainerinsightreceiver/receiver.go +++ b/receiver/awscontainerinsightreceiver/receiver.go @@ -9,9 +9,6 @@ import ( "runtime" "time" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/neuron" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/prometheusscraper" - "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer" "go.opentelemetry.io/collector/component" "go.opentelemetry.io/collector/consumer" "go.opentelemetry.io/collector/pdata/pmetric" @@ -27,6 +24,9 @@ import ( 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/k8swindows" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/neuron" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/prometheusscraper" + "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/prometheusscraper/decoratorconsumer" "github.com/open-telemetry/opentelemetry-collector-contrib/receiver/awscontainerinsightreceiver/internal/stores" ) From 52ad0d0d95bfc4d9dda743ff5e44a593369a1769 Mon Sep 17 00:00:00 2001 From: Samarth Singh Date: Wed, 13 Mar 2024 16:11:54 +0000 Subject: [PATCH 52/53] Add HTTPS support for Neuron --- .../internal/neuron/neuron_monitor_scraper_config.go | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go index 1c0af3d15614..242583e22703 100644 --- a/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go +++ b/receiver/awscontainerinsightreceiver/internal/neuron/neuron_monitor_scraper_config.go @@ -7,6 +7,7 @@ import ( "os" "time" + configutil "github.com/prometheus/common/config" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/discovery" @@ -18,6 +19,7 @@ import ( ) const ( + caFile = "/etc/amazon-cloudwatch-observability-agent-cert/tls-ca.crt" collectionInterval = 60 * time.Second jobName = "containerInsightsNeuronMonitorScraper" scraperMetricsPath = "/metrics" @@ -27,10 +29,16 @@ const ( func GetNeuronScrapeConfig(hostinfo prometheusscraper.HostInfoProvider) *config.ScrapeConfig { return &config.ScrapeConfig{ + HTTPClientConfig: configutil.HTTPClientConfig{ + TLSConfig: configutil.TLSConfig{ + CAFile: caFile, + InsecureSkipVerify: false, + }, + }, ScrapeInterval: model.Duration(collectionInterval), ScrapeTimeout: model.Duration(collectionInterval), JobName: jobName, - Scheme: "http", + Scheme: "https", MetricsPath: scraperMetricsPath, ServiceDiscoveryConfigs: discovery.Configs{ &kubernetes.SDConfig{ From 47c5f8cbd8d96ca2c93616a60daf4aa66b4d5842 Mon Sep 17 00:00:00 2001 From: Samarth Singh Date: Tue, 19 Mar 2024 13:42:26 +0000 Subject: [PATCH 53/53] Move Scrapers under one Flag and add more validations --- .../awscontainerinsightreceiver/config.go | 4 ---- .../awscontainerinsightreceiver/factory.go | 4 ---- .../gpu/{metric_map.go => metric_unit.go} | 0 .../neuron/pod_attribute_decorator.go | 24 ++++++++++++------- .../awscontainerinsightreceiver/receiver.go | 2 +- 5 files changed, 17 insertions(+), 17 deletions(-) rename receiver/awscontainerinsightreceiver/internal/gpu/{metric_map.go => metric_unit.go} (100%) diff --git a/receiver/awscontainerinsightreceiver/config.go b/receiver/awscontainerinsightreceiver/config.go index c6515334f680..708ea1726747 100644 --- a/receiver/awscontainerinsightreceiver/config.go +++ b/receiver/awscontainerinsightreceiver/config.go @@ -60,8 +60,4 @@ type Config struct { // EnableAcceleratedComputeMetrics enabled features with accelerated compute resources where metrics are scraped from vendor specific sources EnableAcceleratedComputeMetrics bool `mapstructure:"accelerated_compute_metrics"` - - // EnableAwsNeuronMetrics toggles Neuron monitoring where metrics are scraped from neuron monitor - // The default value is false. - EnableAwsNeuronMetrics bool `mapstructure:"neuron_metrics"` } diff --git a/receiver/awscontainerinsightreceiver/factory.go b/receiver/awscontainerinsightreceiver/factory.go index c828335b6e22..6cb5179d598e 100644 --- a/receiver/awscontainerinsightreceiver/factory.go +++ b/receiver/awscontainerinsightreceiver/factory.go @@ -42,9 +42,6 @@ const ( // Don't enable EKS control plane metrics by default defaultEnableControlPlaneMetrics = false - - // Don't enable Neuron metrics by default - defaultEnableAwsNeuronMetrics = false ) // NewFactory creates a factory for AWS container insight receiver @@ -67,7 +64,6 @@ func createDefaultConfig() component.Config { ClusterName: defaultClusterName, LeaderLockName: defaultLeaderLockName, EnableControlPlaneMetrics: defaultEnableControlPlaneMetrics, - EnableAwsNeuronMetrics: defaultEnableAwsNeuronMetrics, } } diff --git a/receiver/awscontainerinsightreceiver/internal/gpu/metric_map.go b/receiver/awscontainerinsightreceiver/internal/gpu/metric_unit.go similarity index 100% rename from receiver/awscontainerinsightreceiver/internal/gpu/metric_map.go rename to receiver/awscontainerinsightreceiver/internal/gpu/metric_unit.go diff --git a/receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator.go b/receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator.go index 5538081f32fe..4879e3f3305a 100644 --- a/receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator.go +++ b/receiver/awscontainerinsightreceiver/internal/neuron/pod_attribute_decorator.go @@ -57,10 +57,12 @@ func (pdc *PodAttributesDecoratorConsumer) neuronMetricsProcess(md pmetric.Metri neuronHardwareInfo, neuronHardwareInfoFound := findNeuronHardwareInfo(metrics) if neuronHardwareInfoFound { - neuronCoresPerDevice := getNeuronCoresPerDevice(neuronHardwareInfo) - for k := 0; k < metrics.Len(); k++ { - m := metrics.At(k) - pdc.addPodCorrelationAttributes(getMetricDatapoints(m), neuronCoresPerDevice) + neuronCoresPerDevice, extracted := getNeuronCoresPerDevice(neuronHardwareInfo) + if extracted { + for k := 0; k < metrics.Len(); k++ { + m := metrics.At(k) + pdc.addPodCorrelationAttributes(getMetricDatapoints(m), neuronCoresPerDevice) + } } } } @@ -135,10 +137,16 @@ func getMetricDatapoints(m pmetric.Metric) pmetric.NumberDataPointSlice { // We extract the attribute named `neuroncore_per_device_count` from the metric to get the value // https://awsdocs-neuron.readthedocs-hosted.com/en/latest/tools/neuron-sys-tools/neuron-monitor-user-guide.html -func getNeuronCoresPerDevice(neuronHardwareInfo pmetric.Metric) int { - neuronCoresPerDeviceValue, _ := neuronHardwareInfo.Sum().DataPoints().At(0).Attributes().Get(neuronCorePerDeviceKey) - neuronCoresPerDevice, _ := strconv.Atoi(neuronCoresPerDeviceValue.AsString()) - return neuronCoresPerDevice +func getNeuronCoresPerDevice(neuronHardwareInfo pmetric.Metric) (int, bool) { + neuronCoreHardwareInfoDatapoints := neuronHardwareInfo.Sum().DataPoints() + if neuronCoreHardwareInfoDatapoints.Len() > 0 { + neuronCoresPerDeviceValue, found := neuronCoreHardwareInfoDatapoints.At(0).Attributes().Get(neuronCorePerDeviceKey) + if found { + neuronCoresPerDevice, _ := strconv.Atoi(neuronCoresPerDeviceValue.AsString()) + return neuronCoresPerDevice, true + } + } + return -1, false } // To get the device index from core index we divide the index by cores in a single device diff --git a/receiver/awscontainerinsightreceiver/receiver.go b/receiver/awscontainerinsightreceiver/receiver.go index 4cf10c21c651..65e772a96ac1 100644 --- a/receiver/awscontainerinsightreceiver/receiver.go +++ b/receiver/awscontainerinsightreceiver/receiver.go @@ -224,7 +224,7 @@ func (acir *awsContainerInsightReceiver) initDcgmScraper(ctx context.Context, ho } func (acir *awsContainerInsightReceiver) initNeuronScraper(ctx context.Context, host component.Host, hostinfo *hostInfo.Info, decorator *stores.K8sDecorator) error { - if !acir.config.EnableAwsNeuronMetrics { + if !acir.config.EnableAcceleratedComputeMetrics { return nil }