From 386ec92bc5bb0005d6af3465b14a821224d1ef91 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Guillermo=20Julia=CC=81n?= Date: Thu, 12 Sep 2024 16:59:37 +0200 Subject: [PATCH] Refactor GoTLS monitor --- pkg/network/usm/ebpf_gotls.go | 337 ++++-------------- pkg/network/usm/ebpf_gotls_helpers.go | 119 +++++++ pkg/network/usm/kafka_monitor_test.go | 6 +- pkg/network/usm/postgres_monitor_test.go | 2 +- .../usm/tests/tracer_usm_linux_test.go | 8 +- pkg/network/usm/usm_grpc_monitor_test.go | 4 +- pkg/network/usm/usm_http2_monitor_test.go | 16 +- 7 files changed, 207 insertions(+), 285 deletions(-) diff --git a/pkg/network/usm/ebpf_gotls.go b/pkg/network/usm/ebpf_gotls.go index e0090e3ba889c..ca0668fc0bef5 100644 --- a/pkg/network/usm/ebpf_gotls.go +++ b/pkg/network/usm/ebpf_gotls.go @@ -8,36 +8,25 @@ package usm import ( - "debug/elf" "errors" "fmt" "io" - "os" - "path/filepath" "regexp" - "strconv" - "sync" "time" - "unsafe" - "github.com/cihub/seelog" "github.com/cilium/ebpf" - "golang.org/x/sys/unix" manager "github.com/DataDog/ebpf-manager" - ddebpf "github.com/DataDog/datadog-agent/pkg/ebpf" + "github.com/DataDog/datadog-agent/pkg/ebpf/uprobes" "github.com/DataDog/datadog-agent/pkg/network/config" "github.com/DataDog/datadog-agent/pkg/network/go/bininspect" "github.com/DataDog/datadog-agent/pkg/network/protocols" - "github.com/DataDog/datadog-agent/pkg/network/protocols/http/gotls" "github.com/DataDog/datadog-agent/pkg/network/protocols/http/gotls/lookup" libtelemetry "github.com/DataDog/datadog-agent/pkg/network/protocols/telemetry" "github.com/DataDog/datadog-agent/pkg/network/usm/buildmode" usmconfig "github.com/DataDog/datadog-agent/pkg/network/usm/config" "github.com/DataDog/datadog-agent/pkg/network/usm/utils" - "github.com/DataDog/datadog-agent/pkg/process/monitor" - "github.com/DataDog/datadog-agent/pkg/util/log" ) const ( @@ -55,6 +44,9 @@ const ( connWriteProbe = "uprobe__crypto_tls_Conn_Write" connWriteRetProbe = "uprobe__crypto_tls_Conn_Write__return" connCloseProbe = "uprobe__crypto_tls_Conn_Close" + + // UsmGoTLSAttacherName holds the name used for the uprobe attacher of go-tls programs. Used for tests. + UsmGoTLSAttacherName = "usm_gotls" ) type uprobesInfo struct { @@ -76,19 +68,10 @@ var functionToProbes = map[string]uprobesInfo{ }, } -var functionsConfig = map[string]bininspect.FunctionConfiguration{ - bininspect.WriteGoTLSFunc: { - IncludeReturnLocations: true, - ParamLookupFunction: lookup.GetWriteParams, - }, - bininspect.ReadGoTLSFunc: { - IncludeReturnLocations: true, - ParamLookupFunction: lookup.GetReadParams, - }, - bininspect.CloseGoTLSFunc: { - IncludeReturnLocations: false, - ParamLookupFunction: lookup.GetCloseParams, - }, +var paramLookupFunctions = map[string]bininspect.ParameterLookupFunction{ + bininspect.WriteGoTLSFunc: lookup.GetWriteParams, + bininspect.ReadGoTLSFunc: lookup.GetReadParams, + bininspect.CloseGoTLSFunc: lookup.GetCloseParams, } var structFieldsLookupFunctions = map[bininspect.FieldIdentifier]bininspect.StructLookupFunction{ @@ -103,31 +86,16 @@ type pid = uint32 // goTLSProgram contains implementation for go-TLS. type goTLSProgram struct { - wg sync.WaitGroup - done chan struct{} - cfg *config.Config - manager *manager.Manager + attacher *uprobes.UprobeAttacher + inspector *GoTLSBinaryInspector + cfg *config.Config + manager *manager.Manager // Path to the process/container's procfs procRoot string - - // eBPF map holding the result of binary analysis, indexed by binaries' - // inodes. - offsetsDataMap *ebpf.Map - - // binAnalysisMetric handles telemetry on the time spent doing binary - // analysis - binAnalysisMetric *libtelemetry.Counter - - // binNoSymbolsMetric counts Golang binaries without symbols. - binNoSymbolsMetric *libtelemetry.Counter - registry *utils.FileRegistry } -// Validate that goTLSProgram implements the Attacher interface. -var _ utils.Attacher = &goTLSProgram{} - var goTLSSpec = &protocols.ProtocolSpec{ Maps: []*manager.Map{ {Name: offsetsDataMap}, @@ -178,14 +146,56 @@ func newGoTLSProgramProtocolFactory(m *manager.Manager) protocols.ProtocolFactor return nil, errors.New("goTLS support requires runtime-compilation or CO-RE to be enabled") } + attacherCfg := uprobes.AttacherConfig{ + EbpfConfig: &c.Config, + Rules: []*uprobes.AttachRule{{ + Targets: uprobes.AttachToExecutable, + ProbesSelector: []manager.ProbesSelector{ + &manager.AllOf{ + Selectors: []manager.ProbesSelector{ + &manager.ProbeSelector{ProbeIdentificationPair: manager.ProbeIdentificationPair{EBPFFuncName: connReadProbe}}, + &manager.ProbeSelector{ProbeIdentificationPair: manager.ProbeIdentificationPair{EBPFFuncName: connReadRetProbe}}, + &manager.ProbeSelector{ProbeIdentificationPair: manager.ProbeIdentificationPair{EBPFFuncName: connWriteProbe}}, + &manager.ProbeSelector{ProbeIdentificationPair: manager.ProbeIdentificationPair{EBPFFuncName: connWriteRetProbe}}, + &manager.ProbeSelector{ProbeIdentificationPair: manager.ProbeIdentificationPair{EBPFFuncName: connCloseProbe}}, + }, + }, + }, + ProbeOptionsOverride: map[string]uprobes.ProbeOptions{ + connReadProbe: {IsManualReturn: false, Symbol: bininspect.ReadGoTLSFunc}, + connReadRetProbe: {IsManualReturn: true, Symbol: bininspect.ReadGoTLSFunc}, + connWriteProbe: {IsManualReturn: false, Symbol: bininspect.WriteGoTLSFunc}, + connWriteRetProbe: {IsManualReturn: true, Symbol: bininspect.WriteGoTLSFunc}, + connCloseProbe: {IsManualReturn: false, Symbol: bininspect.CloseGoTLSFunc}, + }, + }}, + ExcludeTargets: uprobes.ExcludeInternal, + PerformInitialScan: true, + EnablePeriodicScanNewProcesses: false, + } + + if c.GoTLSExcludeSelf { + attacherCfg.ExcludeTargets |= uprobes.ExcludeSelf + } + + inspector := &GoTLSBinaryInspector{ + structFieldsLookupFunctions: structFieldsLookupFunctions, + paramLookupFunctions: paramLookupFunctions, + binAnalysisMetric: libtelemetry.NewCounter("usm.go_tls.analysis_time", libtelemetry.OptPrometheus), + binNoSymbolsMetric: libtelemetry.NewCounter("usm.go_tls.missing_symbols", libtelemetry.OptPrometheus), + } + + attacher, err := uprobes.NewUprobeAttacher(UsmGoTLSAttacherName, attacherCfg, m, nil, inspector) + if err != nil { + return nil, fmt.Errorf("Cannot create uprobe attacher: %w", err) + } + return &goTLSProgram{ - done: make(chan struct{}), - cfg: c, - manager: m, - procRoot: c.ProcRoot, - binAnalysisMetric: libtelemetry.NewCounter("usm.go_tls.analysis_time", libtelemetry.OptPrometheus), - binNoSymbolsMetric: libtelemetry.NewCounter("usm.go_tls.missing_symbols", libtelemetry.OptPrometheus), - registry: utils.NewFileRegistry("go-tls"), + cfg: c, + manager: m, + procRoot: c.ProcRoot, + inspector: inspector, + attacher: attacher, }, nil } } @@ -212,48 +222,21 @@ func (p *goTLSProgram) ConfigureOptions(_ *manager.Manager, options *manager.Opt func (p *goTLSProgram) PreStart(m *manager.Manager) error { var err error - p.offsetsDataMap, _, err = m.GetMap(offsetsDataMap) + p.inspector.offsetsDataMap, _, err = m.GetMap(offsetsDataMap) if err != nil { return fmt.Errorf("could not get offsets_data map: %s", err) } - procMonitor := monitor.GetProcessMonitor() - cleanupExec := procMonitor.SubscribeExec(p.handleProcessStart) - cleanupExit := procMonitor.SubscribeExit(p.handleProcessExit) - - p.wg.Add(1) - go func() { - processSync := time.NewTicker(scanTerminatedProcessesInterval) - - defer func() { - processSync.Stop() - cleanupExec() - cleanupExit() - procMonitor.Stop() - p.registry.Clear() - p.wg.Done() - }() - - for { - select { - case <-p.done: - return - case <-processSync.C: - processSet := p.registry.GetRegisteredProcesses() - deletedPids := monitor.FindDeletedProcesses(processSet) - for deletedPid := range deletedPids { - _ = p.registry.Unregister(deletedPid) - } - } - } - }() + err = p.attacher.Start() + if err != nil { + return fmt.Errorf("could not start attacher: %w", err) + } return nil } -// PostStart registers the goTLS program to the attacher list. +// PostStart is a no-op func (p *goTLSProgram) PostStart(*manager.Manager) error { - utils.AddAttacher(p.Name(), p) return nil } @@ -267,9 +250,7 @@ func (p *goTLSProgram) GetStats() *protocols.ProtocolStats { // Stop terminates goTLS main goroutine. func (p *goTLSProgram) Stop(*manager.Manager) { - close(p.done) - // Waiting for the main event loop to finish. - p.wg.Wait() + p.attacher.Stop() } var ( @@ -295,7 +276,7 @@ func GoTLSAttachPID(pid pid) error { return errors.New("GoTLS is not enabled") } - err := goTLSSpec.Instance.(*goTLSProgram).AttachPID(pid) + err := goTLSSpec.Instance.(*goTLSProgram).attacher.AttachPID(pid) if errors.Is(err, utils.ErrPathIsAlreadyRegistered) { // The process monitor has attached the process before us. return nil @@ -311,183 +292,5 @@ func GoTLSDetachPID(pid pid) error { return errors.New("GoTLS is not enabled") } - return goTLSSpec.Instance.(*goTLSProgram).DetachPID(pid) -} - -// AttachPID attaches the provided PID to the eBPF program. -func (p *goTLSProgram) AttachPID(pid uint32) error { - if p.cfg.GoTLSExcludeSelf && pid == uint32(os.Getpid()) { - return ErrSelfExcluded - } - - pidAsStr := strconv.FormatUint(uint64(pid), 10) - exePath := filepath.Join(p.procRoot, pidAsStr, "exe") - - binPath, err := utils.ResolveSymlink(exePath) - if err != nil { - return err - } - - // Check if the process is datadog's internal process, if so, we don't want to hook the process. - if internalProcessRegex.MatchString(binPath) { - if log.ShouldLog(seelog.DebugLvl) { - log.Debugf("ignoring pid %d, as it is an internal datadog component (%q)", pid, binPath) - } - return ErrInternalDDogProcessRejected - } - - // Check go process - probeList := make([]manager.ProbeIdentificationPair, 0) - return p.registry.Register(binPath, pid, registerCBCreator(p.manager, p.offsetsDataMap, &probeList, p.binAnalysisMetric, p.binNoSymbolsMetric), - unregisterCBCreator(p.manager, &probeList, p.offsetsDataMap), - utils.IgnoreCB) -} - -func registerCBCreator(mgr *manager.Manager, offsetsDataMap *ebpf.Map, probeIDs *[]manager.ProbeIdentificationPair, binAnalysisMetric, binNoSymbolsMetric *libtelemetry.Counter) func(path utils.FilePath) error { - return func(filePath utils.FilePath) error { - start := time.Now() - - f, err := os.Open(filePath.HostPath) - if err != nil { - return fmt.Errorf("could not open file %s, %w", filePath.HostPath, err) - } - defer f.Close() - - elfFile, err := elf.NewFile(f) - if err != nil { - return fmt.Errorf("file %s could not be parsed as an ELF file: %w", filePath.HostPath, err) - } - - inspectionResult, err := bininspect.InspectNewProcessBinary(elfFile, functionsConfig, structFieldsLookupFunctions) - if err != nil { - if errors.Is(err, elf.ErrNoSymbols) { - binNoSymbolsMetric.Add(1) - } - return fmt.Errorf("error extracting inspectoin data from %s: %w", filePath.HostPath, err) - } - - if err := addInspectionResultToMap(offsetsDataMap, filePath.ID, inspectionResult); err != nil { - return fmt.Errorf("failed adding inspection rules: %w", err) - } - - pIDs, err := attachHooks(mgr, inspectionResult, filePath.HostPath, filePath.ID) - if err != nil { - removeInspectionResultFromMap(offsetsDataMap, filePath.ID) - return fmt.Errorf("error while attaching hooks to %s: %w", filePath.HostPath, err) - } - *probeIDs = pIDs - - elapsed := time.Since(start) - - binAnalysisMetric.Add(elapsed.Milliseconds()) - log.Debugf("attached hooks on %s (%v) in %s", filePath.HostPath, filePath.ID, elapsed) - return nil - } -} - -func (p *goTLSProgram) handleProcessExit(pid pid) { - _ = p.DetachPID(pid) -} - -func (p *goTLSProgram) handleProcessStart(pid pid) { - _ = p.AttachPID(pid) -} - -// addInspectionResultToMap runs a binary inspection and adds the result to the -// map that's being read by the probes, indexed by the binary's inode number `ino`. -func addInspectionResultToMap(offsetsDataMap *ebpf.Map, binID utils.PathIdentifier, result *bininspect.Result) error { - offsetsData, err := inspectionResultToProbeData(result) - if err != nil { - return fmt.Errorf("error while parsing inspection result: %w", err) - } - - key := &gotls.TlsBinaryId{ - Id_major: unix.Major(binID.Dev), - Id_minor: unix.Minor(binID.Dev), - Ino: binID.Inode, - } - if err := offsetsDataMap.Put(unsafe.Pointer(key), unsafe.Pointer(&offsetsData)); err != nil { - return fmt.Errorf("could not write binary inspection result to map for binID %v: %w", binID, err) - } - - return nil -} - -func removeInspectionResultFromMap(offsetsDataMap *ebpf.Map, binID utils.PathIdentifier) { - key := &gotls.TlsBinaryId{ - Id_major: unix.Major(binID.Dev), - Id_minor: unix.Minor(binID.Dev), - Ino: binID.Inode, - } - if err := offsetsDataMap.Delete(unsafe.Pointer(key)); err != nil { - log.Errorf("could not remove inspection result from map for ino %v: %s", binID, err) - } -} - -func attachHooks(mgr *manager.Manager, result *bininspect.Result, binPath string, binID utils.PathIdentifier) ([]manager.ProbeIdentificationPair, error) { - uid := getUID(binID) - probeIDs := make([]manager.ProbeIdentificationPair, 0) - - for function, uprobes := range functionToProbes { - if functionsConfig[function].IncludeReturnLocations { - if uprobes.returnInfo == "" { - return nil, fmt.Errorf("function %q configured to include return locations but no return uprobes found in config", function) - } - for i, offset := range result.Functions[function].ReturnLocations { - returnProbeID := manager.ProbeIdentificationPair{ - EBPFFuncName: uprobes.returnInfo, - UID: makeReturnUID(uid, i), - } - newProbe := &manager.Probe{ - ProbeIdentificationPair: returnProbeID, - BinaryPath: binPath, - // Each return probe needs to have a unique uid value, - // so add the index to the binary UID to make an overall UID. - UprobeOffset: offset, - } - if err := mgr.AddHook("", newProbe); err != nil { - return nil, fmt.Errorf("could not add return hook to function %q in offset %d due to: %w", function, offset, err) - } - probeIDs = append(probeIDs, returnProbeID) - ddebpf.AddProgramNameMapping(newProbe.ID(), newProbe.EBPFFuncName, "usm_gotls") - } - } - - if uprobes.functionInfo != "" { - probeID := manager.ProbeIdentificationPair{ - EBPFFuncName: uprobes.functionInfo, - UID: uid, - } - - newProbe := &manager.Probe{ - BinaryPath: binPath, - UprobeOffset: result.Functions[function].EntryLocation, - ProbeIdentificationPair: probeID, - } - if err := mgr.AddHook("", newProbe); err != nil { - return nil, fmt.Errorf("could not add hook for %q in offset %d due to: %w", uprobes.functionInfo, result.Functions[function].EntryLocation, err) - } - probeIDs = append(probeIDs, probeID) - ddebpf.AddProgramNameMapping(newProbe.ID(), newProbe.EBPFFuncName, "usm_gotls") - } - } - - return probeIDs, nil -} - -func unregisterCBCreator(mgr *manager.Manager, probeIDs *[]manager.ProbeIdentificationPair, offsetsDataMap *ebpf.Map) func(path utils.FilePath) error { - return func(path utils.FilePath) error { - if len(*probeIDs) == 0 { - return nil - } - removeInspectionResultFromMap(offsetsDataMap, path.ID) - for _, probeID := range *probeIDs { - err := mgr.DetachHook(probeID) - if err != nil { - log.Errorf("failed detaching hook %s: %s", probeID.UID, err) - } - } - log.Debugf("detached hooks on ino %v", path.ID) - return nil - } + return goTLSSpec.Instance.(*goTLSProgram).attacher.DetachPID(pid) } diff --git a/pkg/network/usm/ebpf_gotls_helpers.go b/pkg/network/usm/ebpf_gotls_helpers.go index 6eb58873fffb2..2b4e2756408e2 100644 --- a/pkg/network/usm/ebpf_gotls_helpers.go +++ b/pkg/network/usm/ebpf_gotls_helpers.go @@ -8,15 +8,134 @@ package usm import ( + "debug/elf" "errors" "fmt" + "os" "reflect" + "time" "unsafe" + "github.com/cilium/ebpf" + "golang.org/x/sys/unix" + + "github.com/DataDog/datadog-agent/pkg/ebpf/uprobes" "github.com/DataDog/datadog-agent/pkg/network/go/bininspect" "github.com/DataDog/datadog-agent/pkg/network/protocols/http/gotls" + libtelemetry "github.com/DataDog/datadog-agent/pkg/network/protocols/telemetry" + "github.com/DataDog/datadog-agent/pkg/network/usm/utils" + "github.com/DataDog/datadog-agent/pkg/util/log" ) +// GoTLSBinaryInspector is a BinaryInspector that inspects Go binaries, dealing with the specifics of Go binaries +// such as the argument passing convention and the lack of uprobes +type GoTLSBinaryInspector struct { + structFieldsLookupFunctions map[bininspect.FieldIdentifier]bininspect.StructLookupFunction + paramLookupFunctions map[string]bininspect.ParameterLookupFunction + + // eBPF map holding the result of binary analysis, indexed by binaries' + // inodes. + offsetsDataMap *ebpf.Map + + // binAnalysisMetric handles telemetry on the time spent doing binary + // analysis + binAnalysisMetric *libtelemetry.Counter + + // binNoSymbolsMetric counts Golang binaries without symbols. + binNoSymbolsMetric *libtelemetry.Counter +} + +// Ensure GoTLSBinaryInspector implements BinaryInspector +var _ uprobes.BinaryInspector = &GoTLSBinaryInspector{} + +// Inspect extracts the metadata required to attach to a Go binary from the ELF file at the given path. +func (p *GoTLSBinaryInspector) Inspect(fpath utils.FilePath, requests []uprobes.SymbolRequest) (map[string]bininspect.FunctionMetadata, bool, error) { + start := time.Now() + + path := fpath.HostPath + f, err := os.Open(path) + if err != nil { + return nil, false, fmt.Errorf("could not open file %s, %w", path, err) + } + defer f.Close() + + elfFile, err := elf.NewFile(f) + if err != nil { + return nil, false, fmt.Errorf("file %s could not be parsed as an ELF file: %w", path, err) + } + + functionsConfig := make(map[string]bininspect.FunctionConfiguration, len(requests)) + for _, req := range requests { + lookupFunc, ok := p.paramLookupFunctions[req.Name] + if !ok { + return nil, false, fmt.Errorf("no parameter lookup function found for function %s", req.Name) + } + + functionsConfig[req.Name] = bininspect.FunctionConfiguration{ + IncludeReturnLocations: req.IncludeReturnLocations, + ParamLookupFunction: lookupFunc, + } + } + + inspectionResult, err := bininspect.InspectNewProcessBinary(elfFile, functionsConfig, p.structFieldsLookupFunctions) + if err != nil { + if errors.Is(err, elf.ErrNoSymbols) { + p.binNoSymbolsMetric.Add(1) + } + return nil, false, fmt.Errorf("error extracting inspection data from %s: %w", path, err) + } + + if err := p.addInspectionResultToMap(fpath.ID, inspectionResult); err != nil { + return nil, false, fmt.Errorf("failed adding inspection rules: %w", err) + } + + elapsed := time.Since(start) + p.binAnalysisMetric.Add(elapsed.Milliseconds()) + + return inspectionResult.Functions, true, nil +} + +// Cleanup removes the inspection result for the binary at the given path from the map. +func (p *GoTLSBinaryInspector) Cleanup(fpath utils.FilePath) { + p.removeInspectionResultFromMap(fpath.ID) +} + +// addInspectionResultToMap runs a binary inspection and adds the result to the +// map that's being read by the probes, indexed by the binary's inode number `ino`. +func (p *GoTLSBinaryInspector) addInspectionResultToMap(binID utils.PathIdentifier, result *bininspect.Result) error { + offsetsData, err := inspectionResultToProbeData(result) + if err != nil { + return fmt.Errorf("error while parsing inspection result: %w", err) + } + + key := &gotls.TlsBinaryId{ + Id_major: unix.Major(binID.Dev), + Id_minor: unix.Minor(binID.Dev), + Ino: binID.Inode, + } + if err := p.offsetsDataMap.Put(unsafe.Pointer(key), unsafe.Pointer(&offsetsData)); err != nil { + return fmt.Errorf("could not write binary inspection result to map for binID %v: %w", binID, err) + } + + return nil +} + +func (p *GoTLSBinaryInspector) removeInspectionResultFromMap(binID utils.PathIdentifier) { + key := &gotls.TlsBinaryId{ + Id_major: unix.Major(binID.Dev), + Id_minor: unix.Minor(binID.Dev), + Ino: binID.Inode, + } + if err := p.offsetsDataMap.Delete(unsafe.Pointer(key)); err != nil { + // Ignore errors for non-existing keys: if the inspect process fails, we won't have added the key to the map + // but the deactivation callback (which calls Cleanup and thus this method) will still be called. So it's normal + // to not find the key in the map. We report other errors though. + if !errors.Is(err, unix.ENOENT) { + log.Errorf("could not remove binary inspection result from map for binID %v: %v", binID, err) + } + } +} + func inspectionResultToProbeData(result *bininspect.Result) (gotls.TlsOffsetsData, error) { closeConnPointer, err := getConnPointer(result, bininspect.CloseGoTLSFunc) if err != nil { diff --git a/pkg/network/usm/kafka_monitor_test.go b/pkg/network/usm/kafka_monitor_test.go index 1c3c5e12702ca..3dd48dbcb6ae8 100644 --- a/pkg/network/usm/kafka_monitor_test.go +++ b/pkg/network/usm/kafka_monitor_test.go @@ -538,7 +538,7 @@ func (s *KafkaProtocolParsingSuite) testKafkaProtocolParsing(t *testing.T, tls b }) monitor := newKafkaMonitor(t, cfg) if tls && cfg.EnableGoTLSSupport { - utils.WaitForProgramsToBeTraced(t, "go-tls", proxyProcess.Process.Pid, utils.ManualTracingFallbackEnabled) + utils.WaitForProgramsToBeTraced(t, UsmGoTLSAttacherName, proxyProcess.Process.Pid, utils.ManualTracingFallbackEnabled) } tt.testBody(t, &tt.context, monitor) }) @@ -1136,7 +1136,7 @@ func testKafkaFetchRaw(t *testing.T, tls bool, apiVersion int) { monitor := newKafkaMonitor(t, getDefaultTestConfiguration(tls)) if tls { - utils.WaitForProgramsToBeTraced(t, "go-tls", proxyPid, utils.ManualTracingFallbackEnabled) + utils.WaitForProgramsToBeTraced(t, UsmGoTLSAttacherName, proxyPid, utils.ManualTracingFallbackEnabled) } for _, tt := range tests { @@ -1361,7 +1361,7 @@ func testKafkaProduceRaw(t *testing.T, tls bool, apiVersion int) { monitor := newKafkaMonitor(t, getDefaultTestConfiguration(tls)) if tls { - utils.WaitForProgramsToBeTraced(t, "go-tls", proxyPid, utils.ManualTracingFallbackEnabled) + utils.WaitForProgramsToBeTraced(t, UsmGoTLSAttacherName, proxyPid, utils.ManualTracingFallbackEnabled) } for _, tt := range tests { diff --git a/pkg/network/usm/postgres_monitor_test.go b/pkg/network/usm/postgres_monitor_test.go index a74ef203f3721..a4db3785674e8 100644 --- a/pkg/network/usm/postgres_monitor_test.go +++ b/pkg/network/usm/postgres_monitor_test.go @@ -185,7 +185,7 @@ func testDecoding(t *testing.T, isTLS bool) { monitor := setupUSMTLSMonitor(t, getPostgresDefaultTestConfiguration(isTLS)) if isTLS { - utils.WaitForProgramsToBeTraced(t, "go-tls", os.Getpid(), utils.ManualTracingFallbackEnabled) + utils.WaitForProgramsToBeTraced(t, UsmGoTLSAttacherName, os.Getpid(), utils.ManualTracingFallbackEnabled) } tests := []postgresParsingTestAttributes{ diff --git a/pkg/network/usm/tests/tracer_usm_linux_test.go b/pkg/network/usm/tests/tracer_usm_linux_test.go index 2da79eadbf80a..c815f763c883c 100644 --- a/pkg/network/usm/tests/tracer_usm_linux_test.go +++ b/pkg/network/usm/tests/tracer_usm_linux_test.go @@ -2409,11 +2409,11 @@ func testProtocolClassificationLinux(t *testing.T, tr *tracer.Tracer, clientHost // Wraps the call to the Go-TLS attach function and waits for the program to be traced. func goTLSAttachPID(t *testing.T, pid int) { t.Helper() - if utils.IsProgramTraced("go-tls", pid) { + if utils.IsProgramTraced(usm.UsmGoTLSAttacherName, pid) { return } require.NoError(t, usm.GoTLSAttachPID(uint32(pid))) - utils.WaitForProgramsToBeTraced(t, "go-tls", pid, utils.ManualTracingFallbackEnabled) + utils.WaitForProgramsToBeTraced(t, usm.UsmGoTLSAttacherName, pid, utils.ManualTracingFallbackEnabled) } // goTLSDetachPID detaches the Go-TLS monitoring from the given PID. @@ -2422,13 +2422,13 @@ func goTLSDetachPID(t *testing.T, pid int) { t.Helper() // The program is not traced; nothing to do. - if !utils.IsProgramTraced("go-tls", pid) { + if !utils.IsProgramTraced(usm.UsmGoTLSAttacherName, pid) { return } require.NoError(t, usm.GoTLSDetachPID(uint32(pid))) require.Eventually(t, func() bool { - return !utils.IsProgramTraced("go-tls", pid) + return !utils.IsProgramTraced(usm.UsmGoTLSAttacherName, pid) }, 5*time.Second, 100*time.Millisecond, "process %v is still traced by Go-TLS after detaching", pid) } diff --git a/pkg/network/usm/usm_grpc_monitor_test.go b/pkg/network/usm/usm_grpc_monitor_test.go index 35ae34a9d46d4..6ca4d43ef39ad 100644 --- a/pkg/network/usm/usm_grpc_monitor_test.go +++ b/pkg/network/usm/usm_grpc_monitor_test.go @@ -115,7 +115,7 @@ func (s *usmGRPCSuite) TestSimpleGRPCScenarios() { usmMonitor := setupUSMTLSMonitor(t, s.getConfig()) if s.isTLS { - utils.WaitForProgramsToBeTraced(t, "go-tls", srv.Process.Pid, utils.ManualTracingFallbackEnabled) + utils.WaitForProgramsToBeTraced(t, UsmGoTLSAttacherName, srv.Process.Pid, utils.ManualTracingFallbackEnabled) } // c is a stream endpoint // a + b are unary endpoints @@ -443,7 +443,7 @@ func (s *usmGRPCSuite) TestLargeBodiesGRPCScenarios() { usmMonitor := setupUSMTLSMonitor(t, s.getConfig()) if s.isTLS { - utils.WaitForProgramsToBeTraced(t, "go-tls", srv.Process.Pid, utils.ManualTracingFallbackEnabled) + utils.WaitForProgramsToBeTraced(t, UsmGoTLSAttacherName, srv.Process.Pid, utils.ManualTracingFallbackEnabled) } // Random string generation is an heavy operation, and it's proportional for the length (15MB) diff --git a/pkg/network/usm/usm_http2_monitor_test.go b/pkg/network/usm/usm_http2_monitor_test.go index 46c3188b09833..da2c474c82d1b 100644 --- a/pkg/network/usm/usm_http2_monitor_test.go +++ b/pkg/network/usm/usm_http2_monitor_test.go @@ -145,7 +145,7 @@ func (s *usmHTTP2Suite) TestHTTP2DynamicTableCleanup() { monitor := setupUSMTLSMonitor(t, cfg) if s.isTLS { - utils.WaitForProgramsToBeTraced(t, "go-tls", proxyProcess.Process.Pid, utils.ManualTracingFallbackEnabled) + utils.WaitForProgramsToBeTraced(t, UsmGoTLSAttacherName, proxyProcess.Process.Pid, utils.ManualTracingFallbackEnabled) } clients := getHTTP2UnixClientArray(2, unixPath) @@ -207,7 +207,7 @@ func (s *usmHTTP2Suite) TestSimpleHTTP2() { monitor := setupUSMTLSMonitor(t, cfg) if s.isTLS { - utils.WaitForProgramsToBeTraced(t, "go-tls", proxyProcess.Process.Pid, utils.ManualTracingFallbackEnabled) + utils.WaitForProgramsToBeTraced(t, UsmGoTLSAttacherName, proxyProcess.Process.Pid, utils.ManualTracingFallbackEnabled) } tests := []struct { @@ -395,7 +395,7 @@ func (s *usmHTTP2Suite) TestHTTP2KernelTelemetry() { t.Run(tt.name, func(t *testing.T) { monitor := setupUSMTLSMonitor(t, cfg) if s.isTLS { - utils.WaitForProgramsToBeTraced(t, "go-tls", proxyProcess.Process.Pid, utils.ManualTracingFallbackEnabled) + utils.WaitForProgramsToBeTraced(t, UsmGoTLSAttacherName, proxyProcess.Process.Pid, utils.ManualTracingFallbackEnabled) } tt.runClients(t, 1) @@ -450,7 +450,7 @@ func (s *usmHTTP2Suite) TestHTTP2ManyDifferentPaths() { monitor := setupUSMTLSMonitor(t, cfg) if s.isTLS { - utils.WaitForProgramsToBeTraced(t, "go-tls", proxyProcess.Process.Pid, utils.ManualTracingFallbackEnabled) + utils.WaitForProgramsToBeTraced(t, UsmGoTLSAttacherName, proxyProcess.Process.Pid, utils.ManualTracingFallbackEnabled) } const ( @@ -516,7 +516,7 @@ func (s *usmHTTP2Suite) TestRawTraffic() { require.NoError(t, proxy.WaitForConnectionReady(unixPath)) if s.isTLS { - utils.WaitForProgramsToBeTraced(t, "go-tls", proxyProcess.Process.Pid, utils.ManualTracingFallbackEnabled) + utils.WaitForProgramsToBeTraced(t, UsmGoTLSAttacherName, proxyProcess.Process.Pid, utils.ManualTracingFallbackEnabled) } tests := []struct { name string @@ -1314,7 +1314,7 @@ func (s *usmHTTP2Suite) TestDynamicTable() { usmMonitor := setupUSMTLSMonitor(t, cfg) if s.isTLS { - utils.WaitForProgramsToBeTraced(t, "go-tls", proxyProcess.Process.Pid, utils.ManualTracingFallbackEnabled) + utils.WaitForProgramsToBeTraced(t, UsmGoTLSAttacherName, proxyProcess.Process.Pid, utils.ManualTracingFallbackEnabled) } c := dialHTTP2Server(t) @@ -1400,7 +1400,7 @@ func (s *usmHTTP2Suite) TestRemainderTable() { t.Run(tt.name, func(t *testing.T) { usmMonitor := setupUSMTLSMonitor(t, cfg) if s.isTLS { - utils.WaitForProgramsToBeTraced(t, "go-tls", proxyProcess.Process.Pid, utils.ManualTracingFallbackEnabled) + utils.WaitForProgramsToBeTraced(t, UsmGoTLSAttacherName, proxyProcess.Process.Pid, utils.ManualTracingFallbackEnabled) } c := dialHTTP2Server(t) @@ -1470,7 +1470,7 @@ func (s *usmHTTP2Suite) TestRawHuffmanEncoding() { t.Run(tt.name, func(t *testing.T) { usmMonitor := setupUSMTLSMonitor(t, cfg) if s.isTLS { - utils.WaitForProgramsToBeTraced(t, "go-tls", proxyProcess.Process.Pid, utils.ManualTracingFallbackEnabled) + utils.WaitForProgramsToBeTraced(t, UsmGoTLSAttacherName, proxyProcess.Process.Pid, utils.ManualTracingFallbackEnabled) } c := dialHTTP2Server(t)