From 7fd542a29f7010ba7e3290f135f25562009fb984 Mon Sep 17 00:00:00 2001 From: Bryce Kahle Date: Tue, 5 Nov 2024 14:12:35 -0800 Subject: [PATCH] pack TCP failure reason into tcp_stats_t --- pkg/network/ebpf/c/co-re/tracer-fentry.c | 4 +- pkg/network/ebpf/c/tracer.c | 17 +- pkg/network/ebpf/c/tracer/events.h | 23 +-- pkg/network/ebpf/c/tracer/maps.h | 3 - pkg/network/ebpf/c/tracer/tracer.h | 6 +- pkg/network/ebpf/kprobe_types.go | 2 - pkg/network/ebpf/kprobe_types_linux.go | 8 +- pkg/network/ebpf/probes/probes.go | 2 - pkg/network/encoding/encoding_test.go | 2 +- pkg/network/encoding/marshal/format.go | 2 +- pkg/network/event_common.go | 2 +- pkg/network/event_windows.go | 2 +- pkg/network/tracer/connection/ebpf_tracer.go | 63 ++++--- .../tracer/connection/ebpfless_tracer.go | 4 - .../failure/failed_conn_consumer.go | 108 ------------ .../tracer/connection/failure/matching.go | 164 ------------------ .../tracer/connection/failure/types.go | 43 ----- .../tracer/connection/kprobe/manager.go | 5 +- .../tracer/connection/kprobe/tracer.go | 18 +- .../tracer/connection/kprobe/tracer_test.go | 8 +- pkg/network/tracer/connection/tracer.go | 3 - .../tracer/connection/util/conn_tracer.go | 28 --- pkg/network/tracer/tracer.go | 4 - pkg/network/tracer/tracer_test.go | 4 +- 24 files changed, 76 insertions(+), 449 deletions(-) delete mode 100644 pkg/network/tracer/connection/failure/failed_conn_consumer.go delete mode 100644 pkg/network/tracer/connection/failure/matching.go delete mode 100644 pkg/network/tracer/connection/failure/types.go diff --git a/pkg/network/ebpf/c/co-re/tracer-fentry.c b/pkg/network/ebpf/c/co-re/tracer-fentry.c index 09259e948047db..5e35f3e4f7adcc 100644 --- a/pkg/network/ebpf/c/co-re/tracer-fentry.c +++ b/pkg/network/ebpf/c/co-re/tracer-fentry.c @@ -244,7 +244,7 @@ int BPF_PROG(tcp_close, struct sock *sk, long timeout) { bpf_map_delete_elem(&tcp_ongoing_connect_pid, &skp_conn); - cleanup_conn(ctx, &t, sk); + cleanup_conn(ctx, &t, sk, 0); return 0; } @@ -543,7 +543,7 @@ static __always_inline int handle_udp_destroy_sock(void *ctx, struct sock *sk) { __u16 lport = 0; if (valid_tuple) { - cleanup_conn(ctx, &tup, sk); + cleanup_conn(ctx, &tup, sk, 0); lport = tup.sport; } else { // get the port for the current sock diff --git a/pkg/network/ebpf/c/tracer.c b/pkg/network/ebpf/c/tracer.c index 909243dba174e1..98c1e0a4c11e2b 100644 --- a/pkg/network/ebpf/c/tracer.c +++ b/pkg/network/ebpf/c/tracer.c @@ -220,7 +220,7 @@ int BPF_BYPASSABLE_KPROBE(kprobe__tcp_done, struct sock *sk) { return 0; } - // connection timeouts will have 0 pids as they are cleaned up by an idle process. + // connection timeouts will have 0 pids as they are cleaned up by an idle process. // resets can also have kernel pids are they are triggered by receiving an RST packet from the server // get the pid from the ongoing failure map in this case, as it should have been set in connect(). else bail pid_ts_t *failed_conn_pid = bpf_map_lookup_elem(&tcp_ongoing_connect_pid, &skp_conn); @@ -237,9 +237,8 @@ int BPF_BYPASSABLE_KPROBE(kprobe__tcp_done, struct sock *sk) { // skip EEXIST errors for telemetry since it is an expected error __u64 timestamp = bpf_ktime_get_ns(); if (bpf_map_update_with_telemetry(conn_close_flushed, &t, ×tamp, BPF_NOEXIST, -EEXIST) == 0) { - cleanup_conn(ctx, &t, sk); + cleanup_conn(ctx, &t, sk, err); increment_telemetry_count(tcp_done_connection_flush); - flush_tcp_failure(ctx, &t, err); } else { bpf_map_delete_elem(&conn_close_flushed, &t); increment_telemetry_count(double_flush_attempts_done); @@ -278,7 +277,7 @@ int BPF_BYPASSABLE_KPROBE(kprobe__tcp_close, struct sock *sk) { bpf_map_delete_elem(&tcp_ongoing_connect_pid, &skp_conn); if (!tcp_failed_connections_enabled()) { - cleanup_conn(ctx, &t, sk); + cleanup_conn(ctx, &t, sk, 0); return 0; } @@ -287,14 +286,16 @@ int BPF_BYPASSABLE_KPROBE(kprobe__tcp_close, struct sock *sk) { // skip EEXIST errors for telemetry since it is an expected error __u64 timestamp = bpf_ktime_get_ns(); if (bpf_map_update_with_telemetry(conn_close_flushed, &t, ×tamp, BPF_NOEXIST, -EEXIST) == 0) { - cleanup_conn(ctx, &t, sk); - increment_telemetry_count(tcp_close_connection_flush); + __u16 tcp_failure_reason = 0; int err = 0; bpf_probe_read_kernel_with_telemetry(&err, sizeof(err), (&sk->sk_err)); if (err == TCP_CONN_FAILED_RESET || err == TCP_CONN_FAILED_TIMEOUT || err == TCP_CONN_FAILED_REFUSED) { increment_telemetry_count(tcp_close_target_failures); - flush_tcp_failure(ctx, &t, err); + tcp_failure_reason = err; } + + cleanup_conn(ctx, &t, sk, tcp_failure_reason); + increment_telemetry_count(tcp_close_connection_flush); } else { bpf_map_delete_elem(&conn_close_flushed, &t); increment_telemetry_count(double_flush_attempts_close); @@ -1047,7 +1048,7 @@ static __always_inline int handle_udp_destroy_sock(void *ctx, struct sock *skp) __u16 lport = 0; if (valid_tuple) { - cleanup_conn(ctx, &tup, skp); + cleanup_conn(ctx, &tup, skp, 0); lport = tup.sport; } else { lport = read_sport(skp); diff --git a/pkg/network/ebpf/c/tracer/events.h b/pkg/network/ebpf/c/tracer/events.h index 3c17a1e1ec683f..56720bdc3a907a 100644 --- a/pkg/network/ebpf/c/tracer/events.h +++ b/pkg/network/ebpf/c/tracer/events.h @@ -44,7 +44,7 @@ __maybe_unused static __always_inline void submit_closed_conn_event(void *ctx, i } } -static __always_inline void cleanup_conn(void *ctx, conn_tuple_t *tup, struct sock *sk) { +static __always_inline void cleanup_conn(void *ctx, conn_tuple_t *tup, struct sock *sk, __u16 tcp_failure_reason) { u32 cpu = bpf_get_smp_processor_id(); // Will hold the full connection data to send through the perf or ring buffer conn_t conn = { .tup = *tup }; @@ -70,6 +70,10 @@ static __always_inline void cleanup_conn(void *ctx, conn_tuple_t *tup, struct so conn.tup.pid = tup->pid; conn.tcp_stats.state_transitions |= (1 << TCP_CLOSE); + conn.tcp_stats.failure_reason = tcp_failure_reason; + if (tcp_failure_reason) { + increment_telemetry_count(tcp_failed_connect); + } } cst = bpf_map_lookup_elem(&conn_stats, &(conn.tup)); @@ -137,23 +141,6 @@ static __always_inline void cleanup_conn(void *ctx, conn_tuple_t *tup, struct so } } -// This function is used to flush the conn_failed_t to the perf or ring buffer. -static __always_inline void flush_tcp_failure(void *ctx, conn_tuple_t *tup, int failure_reason) { - conn_failed_t failure = {}; - failure.tup = *tup; - failure.failure_reason = failure_reason; - - __u64 ringbuffers_enabled = 0; - LOAD_CONSTANT("ringbuffers_enabled", ringbuffers_enabled); - if (ringbuffers_enabled > 0) { - bpf_ringbuf_output(&conn_fail_event, &failure, sizeof(conn_failed_t), 0); - } else { - u32 cpu = bpf_get_smp_processor_id(); - bpf_perf_event_output(ctx, &conn_fail_event, cpu, &failure, sizeof(conn_failed_t)); - } - increment_telemetry_count(tcp_failed_connect); -} - static __always_inline void flush_conn_close_if_full(void *ctx) { u32 cpu = bpf_get_smp_processor_id(); batch_t *batch_ptr = bpf_map_lookup_elem(&conn_close_batch, &cpu); diff --git a/pkg/network/ebpf/c/tracer/maps.h b/pkg/network/ebpf/c/tracer/maps.h index 6ac62657e90f9d..5e1cd31195cc24 100644 --- a/pkg/network/ebpf/c/tracer/maps.h +++ b/pkg/network/ebpf/c/tracer/maps.h @@ -34,9 +34,6 @@ BPF_HASH_MAP(conn_close_flushed, conn_tuple_t, __u64, 16384) */ BPF_PERF_EVENT_ARRAY_MAP(conn_close_event, __u32) -/* Will hold TCP failed connections */ -BPF_PERF_EVENT_ARRAY_MAP(conn_fail_event, __u32) - /* We use this map as a container for batching closed tcp/udp connections * The key represents the CPU core. Ideally we should use a BPF_MAP_TYPE_PERCPU_HASH map * or BPF_MAP_TYPE_PERCPU_ARRAY, but they are not available in diff --git a/pkg/network/ebpf/c/tracer/tracer.h b/pkg/network/ebpf/c/tracer/tracer.h index 1bdcfaa9774f66..d688359dc9d791 100644 --- a/pkg/network/ebpf/c/tracer/tracer.h +++ b/pkg/network/ebpf/c/tracer/tracer.h @@ -70,6 +70,7 @@ typedef struct { // Bit mask containing all TCP state transitions tracked by our tracer __u16 state_transitions; + __u16 failure_reason; } tcp_stats_t; // Full data for a tcp connection @@ -80,11 +81,6 @@ typedef struct { conn_stats_ts_t conn_stats; } conn_t; -typedef struct { - conn_tuple_t tup; - __u32 failure_reason; -} conn_failed_t; - // Must match the number of conn_t objects embedded in the batch_t struct #ifndef CONN_CLOSED_BATCH_SIZE #define CONN_CLOSED_BATCH_SIZE 4 diff --git a/pkg/network/ebpf/kprobe_types.go b/pkg/network/ebpf/kprobe_types.go index 6745cdc0b7fee0..90d7eb1f331ae2 100644 --- a/pkg/network/ebpf/kprobe_types.go +++ b/pkg/network/ebpf/kprobe_types.go @@ -20,7 +20,6 @@ type ConnTuple C.conn_tuple_t type TCPStats C.tcp_stats_t type ConnStats C.conn_stats_ts_t type Conn C.conn_t -type FailedConn C.conn_failed_t type SkpConn C.skp_conn_tuple_t type PidTs C.pid_ts_t type Batch C.batch_t @@ -60,7 +59,6 @@ const TCPFailureConnTimeout = C.TCP_CONN_FAILED_TIMEOUT const TCPFailureConnRefused = C.TCP_CONN_FAILED_REFUSED const SizeofConn = C.sizeof_conn_t -const SizeofFailedConn = C.sizeof_conn_failed_t type ClassificationProgram = uint32 diff --git a/pkg/network/ebpf/kprobe_types_linux.go b/pkg/network/ebpf/kprobe_types_linux.go index 0b56faa4d9b282..bf9bbf38210f38 100644 --- a/pkg/network/ebpf/kprobe_types_linux.go +++ b/pkg/network/ebpf/kprobe_types_linux.go @@ -19,7 +19,7 @@ type TCPStats struct { Rtt_var uint32 Retransmits uint32 State_transitions uint16 - Pad_cgo_0 [2]byte + Failure_reason uint16 } type ConnStats struct { Sent_bytes uint64 @@ -39,11 +39,6 @@ type Conn struct { Tcp_stats TCPStats Conn_stats ConnStats } -type FailedConn struct { - Tup ConnTuple - Reason uint32 - Pad_cgo_0 [4]byte -} type SkpConn struct { Sk uint64 Tup ConnTuple @@ -136,7 +131,6 @@ const TCPFailureConnTimeout = 0x6e const TCPFailureConnRefused = 0x6f const SizeofConn = 0x78 -const SizeofFailedConn = 0x38 type ClassificationProgram = uint32 diff --git a/pkg/network/ebpf/probes/probes.go b/pkg/network/ebpf/probes/probes.go index 8d8cfa52475420..1401b231fe231c 100644 --- a/pkg/network/ebpf/probes/probes.go +++ b/pkg/network/ebpf/probes/probes.go @@ -198,8 +198,6 @@ const ( ConnCloseFlushed BPFMapName = "conn_close_flushed" // ConnCloseEventMap is the map storing connection close events ConnCloseEventMap BPFMapName = "conn_close_event" - // FailedConnEventMap is the map for storing failed connection events - FailedConnEventMap BPFMapName = "conn_fail_event" // TracerStatusMap is the map storing the status of the tracer TracerStatusMap BPFMapName = "tracer_status" // ConntrackStatusMap is the map storing the status of the conntrack diff --git a/pkg/network/encoding/encoding_test.go b/pkg/network/encoding/encoding_test.go index 04b033d8d52699..e0d0eacfde10c3 100644 --- a/pkg/network/encoding/encoding_test.go +++ b/pkg/network/encoding/encoding_test.go @@ -251,7 +251,7 @@ func TestSerialization(t *testing.T) { }, }, }, - TCPFailures: map[uint32]uint32{ + TCPFailures: map[uint16]uint32{ 110: 1, }, }, diff --git a/pkg/network/encoding/marshal/format.go b/pkg/network/encoding/marshal/format.go index 4628a0203e390e..99cf5c4c756940 100644 --- a/pkg/network/encoding/marshal/format.go +++ b/pkg/network/encoding/marshal/format.go @@ -112,7 +112,7 @@ func FormatConnection(builder *model.ConnectionBuilder, conn network.ConnectionS if len(conn.TCPFailures) > 0 { builder.AddTcpFailuresByErrCode(func(w *model.Connection_TcpFailuresByErrCodeEntryBuilder) { for k, v := range conn.TCPFailures { - w.SetKey(k) + w.SetKey(uint32(k)) w.SetValue(v) } }) diff --git a/pkg/network/event_common.go b/pkg/network/event_common.go index e8dcbfd2e77c3b..d8cf93f5925356 100644 --- a/pkg/network/event_common.go +++ b/pkg/network/event_common.go @@ -269,7 +269,7 @@ type ConnectionStats struct { } DNSStats map[dns.Hostname]map[dns.QueryType]dns.Stats // TCPFailures stores the number of failures for a POSIX error code - TCPFailures map[uint32]uint32 + TCPFailures map[uint16]uint32 ConnectionTuple diff --git a/pkg/network/event_windows.go b/pkg/network/event_windows.go index 54ad5399f8c638..a2b065e6d3eea4 100644 --- a/pkg/network/event_windows.go +++ b/pkg/network/event_windows.go @@ -120,7 +120,7 @@ func FlowToConnStat(cs *ConnectionStats, flow *driver.PerFlowData, enableMonoton tf := flow.TCPFlow() if tf != nil { - cs.TCPFailures = make(map[uint32]uint32) + cs.TCPFailures = make(map[uint16]uint32) cs.Monotonic.Retransmits = uint32(tf.RetransmitCount) cs.RTT = uint32(tf.SRTT) cs.RTTVar = uint32(tf.RttVariance) diff --git a/pkg/network/tracer/connection/ebpf_tracer.go b/pkg/network/tracer/connection/ebpf_tracer.go index 84f4ccd984fdd9..84c1d74150d8f4 100644 --- a/pkg/network/tracer/connection/ebpf_tracer.go +++ b/pkg/network/tracer/connection/ebpf_tracer.go @@ -32,7 +32,6 @@ import ( netebpf "github.com/DataDog/datadog-agent/pkg/network/ebpf" "github.com/DataDog/datadog-agent/pkg/network/ebpf/probes" "github.com/DataDog/datadog-agent/pkg/network/protocols" - "github.com/DataDog/datadog-agent/pkg/network/tracer/connection/failure" "github.com/DataDog/datadog-agent/pkg/network/tracer/connection/fentry" "github.com/DataDog/datadog-agent/pkg/network/tracer/connection/kprobe" "github.com/DataDog/datadog-agent/pkg/network/tracer/connection/util" @@ -47,6 +46,7 @@ const ( ) var tcpOngoingConnectMapTTL = 30 * time.Minute.Nanoseconds() +var connClosedFlushMapTTL = 10 * time.Millisecond.Nanoseconds() var EbpfTracerTelemetry = struct { connections telemetry.Gauge @@ -75,6 +75,7 @@ var EbpfTracerTelemetry = struct { PidCollisions *telemetry.StatCounterWrapper iterationDups telemetry.Counter iterationAborts telemetry.Counter + closedConnFlushedCleaned telemetry.Counter lastTcpFailedConnects *atomic.Int64 LastTcpSentMiscounts *atomic.Int64 @@ -120,6 +121,7 @@ var EbpfTracerTelemetry = struct { telemetry.NewStatCounterWrapper(connTracerModuleName, "pid_collisions", []string{}, "Counter measuring number of process collisions"), telemetry.NewCounter(connTracerModuleName, "iteration_dups", []string{}, "Counter measuring the number of connections iterated more than once"), telemetry.NewCounter(connTracerModuleName, "iteration_aborts", []string{}, "Counter measuring how many times ebpf iteration of connection map was aborted"), + telemetry.NewCounter("network_tracer__tcp_failure", "closed_conn_flushed_cleaned", []string{}, "Counter measuring the number of conn_close_flushed entries cleaned in userspace"), atomic.NewInt64(0), atomic.NewInt64(0), atomic.NewInt64(0), @@ -149,11 +151,10 @@ type ebpfTracer struct { // tcp_close events closeConsumer *tcpCloseConsumer - // tcp failure events - failedConnConsumer *failure.TCPFailedConnConsumer // periodically clean the ongoing connection pid map ongoingConnectCleaner *ddebpf.MapCleaner[netebpf.SkpConn, netebpf.PidTs] + connCloseFlushCleaner *ddebpf.MapCleaner[netebpf.ConnTuple, int64] removeTuple *netebpf.ConnTuple @@ -255,25 +256,20 @@ func newEbpfTracer(config *config.Config, _ telemetryComponent.Component) (Trace closeConsumer := newTCPCloseConsumer(connCloseEventHandler, batchMgr) - var failedConnConsumer *failure.TCPFailedConnConsumer // Failed connections are not supported on prebuilt if tracerType == TracerTypeKProbePrebuilt { config.TCPFailedConnectionsEnabled = false } - if config.FailedConnectionsSupported() { - failedConnConsumer = failure.NewFailedConnConsumer(failedConnsHandler, m, config.MaxFailedConnectionsBuffered) - } tr := &ebpfTracer{ - m: m, - config: config, - closeConsumer: closeConsumer, - failedConnConsumer: failedConnConsumer, - removeTuple: &netebpf.ConnTuple{}, - closeTracer: closeTracerFn, - ebpfTracerType: tracerType, - exitTelemetry: make(chan struct{}), - ch: newCookieHasher(), + m: m, + config: config, + closeConsumer: closeConsumer, + removeTuple: &netebpf.ConnTuple{}, + closeTracer: closeTracerFn, + ebpfTracerType: tracerType, + exitTelemetry: make(chan struct{}), + ch: newCookieHasher(), } tr.setupMapCleaner(m) @@ -327,7 +323,6 @@ func (t *ebpfTracer) Start(callback func(*network.ConnectionStats)) (err error) } t.closeConsumer.Start(callback) - t.failedConnConsumer.Start() return nil } @@ -348,13 +343,6 @@ func (t *ebpfTracer) FlushPending() { t.closeConsumer.FlushPending() } -func (t *ebpfTracer) GetFailedConnections() *failure.FailedConns { - if t.failedConnConsumer == nil { - return nil - } - return t.failedConnConsumer.FailedConns -} - func (t *ebpfTracer) Stop() { t.stopOnce.Do(func() { close(t.exitTelemetry) @@ -362,8 +350,8 @@ func (t *ebpfTracer) Stop() { ebpftelemetry.UnregisterTelemetry(t.m) _ = t.m.Stop(manager.CleanAll) t.closeConsumer.Stop() - t.failedConnConsumer.Stop() t.ongoingConnectCleaner.Stop() + t.connCloseFlushCleaner.Stop() if t.closeTracer != nil { t.closeTracer() } @@ -730,6 +718,26 @@ func (t *ebpfTracer) setupMapCleaner(m *manager.Manager) { }) t.ongoingConnectCleaner = tcpOngoingConnectPidCleaner + + if t.config.FailedConnectionsSupported() { + connCloseFlushMap, _, err := m.GetMap(probes.ConnCloseFlushed) + if err != nil { + log.Errorf("error getting %v map: %s", probes.ConnCloseFlushed, err) + } + connCloseFlushCleaner, err := ddebpf.NewMapCleaner[netebpf.ConnTuple, int64](connCloseFlushMap, 1024) + if err != nil { + log.Errorf("error creating map cleaner: %s", err) + return + } + connCloseFlushCleaner.Clean(time.Second*1, nil, nil, func(now int64, _ netebpf.ConnTuple, val int64) bool { + expired := val > 0 && now-val > connClosedFlushMapTTL + if expired { + EbpfTracerTelemetry.closedConnFlushedCleaned.Inc() + } + return expired + }) + t.connCloseFlushCleaner = connCloseFlushCleaner + } } func populateConnStats(stats *network.ConnectionStats, t *netebpf.ConnTuple, s *netebpf.ConnStats, ch *cookieHasher) { @@ -802,5 +810,10 @@ func updateTCPStats(conn *network.ConnectionStats, tcpStats *netebpf.TCPStats) { conn.Monotonic.TCPClosed = tcpStats.State_transitions >> netebpf.Close & 1 conn.RTT = tcpStats.Rtt conn.RTTVar = tcpStats.Rtt_var + if tcpStats.Failure_reason > 0 { + conn.TCPFailures = map[uint16]uint32{ + tcpStats.Failure_reason: 1, + } + } } } diff --git a/pkg/network/tracer/connection/ebpfless_tracer.go b/pkg/network/tracer/connection/ebpfless_tracer.go index 215df996158dc8..ff969edbc13b11 100644 --- a/pkg/network/tracer/connection/ebpfless_tracer.go +++ b/pkg/network/tracer/connection/ebpfless_tracer.go @@ -25,7 +25,6 @@ import ( "github.com/DataDog/datadog-agent/pkg/network/config" "github.com/DataDog/datadog-agent/pkg/network/filter" "github.com/DataDog/datadog-agent/pkg/network/tracer/connection/ebpfless" - "github.com/DataDog/datadog-agent/pkg/network/tracer/connection/failure" "github.com/DataDog/datadog-agent/pkg/process/util" "github.com/DataDog/datadog-agent/pkg/telemetry" "github.com/DataDog/datadog-agent/pkg/util/log" @@ -329,9 +328,6 @@ func (t *ebpfLessTracer) Describe(_ chan<- *prometheus.Desc) {} // Collect returns the current state of all metrics of the collector func (t *ebpfLessTracer) Collect(_ chan<- prometheus.Metric) {} -// GetFailedConnections returns the underlying map used to store failed connections -func (t *ebpfLessTracer) GetFailedConnections() *failure.FailedConns { return nil } - var _ Tracer = &ebpfLessTracer{} type udpProcessor struct { diff --git a/pkg/network/tracer/connection/failure/failed_conn_consumer.go b/pkg/network/tracer/connection/failure/failed_conn_consumer.go deleted file mode 100644 index cb67e1de887677..00000000000000 --- a/pkg/network/tracer/connection/failure/failed_conn_consumer.go +++ /dev/null @@ -1,108 +0,0 @@ -// Unless explicitly stated otherwise all files in this repository are licensed -// under the Apache License Version 2.0. -// This product includes software developed at Datadog (https://www.datadoghq.com/). -// Copyright 2016-present Datadog, Inc. - -//go:build linux_bpf - -// Package failure contains logic specific to TCP failed connection handling -package failure - -import ( - "sync" - "unsafe" - - manager "github.com/DataDog/ebpf-manager" - - ddebpf "github.com/DataDog/datadog-agent/pkg/ebpf" - netebpf "github.com/DataDog/datadog-agent/pkg/network/ebpf" - "github.com/DataDog/datadog-agent/pkg/telemetry" - "github.com/DataDog/datadog-agent/pkg/util/log" -) - -const failedConnConsumerModuleName = "network_tracer__ebpf" - -// Telemetry -var failedConnConsumerTelemetry = struct { - eventsReceived telemetry.Counter - eventsLost telemetry.Counter -}{ - telemetry.NewCounter(failedConnConsumerModuleName, "failed_conn_polling_received", []string{}, "Counter measuring the number of failed connections received"), - telemetry.NewCounter(failedConnConsumerModuleName, "failed_conn_polling_lost", []string{}, "Counter measuring the number of failed connections lost (were transmitted from ebpf but never received)"), -} - -// TCPFailedConnConsumer consumes failed connection events from the kernel -type TCPFailedConnConsumer struct { - eventHandler ddebpf.EventHandler - once sync.Once - closed chan struct{} - FailedConns *FailedConns -} - -// NewFailedConnConsumer creates a new TCPFailedConnConsumer -func NewFailedConnConsumer(eventHandler ddebpf.EventHandler, m *manager.Manager, maxFailedConnsBuffered uint32) *TCPFailedConnConsumer { - return &TCPFailedConnConsumer{ - eventHandler: eventHandler, - closed: make(chan struct{}), - FailedConns: NewFailedConns(m, maxFailedConnsBuffered), - } -} - -// Stop stops the consumer -func (c *TCPFailedConnConsumer) Stop() { - if c == nil { - return - } - c.eventHandler.Stop() - c.once.Do(func() { - close(c.closed) - }) - c.FailedConns.connCloseFlushedCleaner.Stop() -} - -func (c *TCPFailedConnConsumer) extractConn(data []byte) { - failedConn := (*Conn)(unsafe.Pointer(&data[0])) - failedConnConsumerTelemetry.eventsReceived.Inc() - - c.FailedConns.UpsertConn(failedConn) -} - -// Start starts the consumer -func (c *TCPFailedConnConsumer) Start() { - if c == nil { - return - } - - go func() { - dataChannel := c.eventHandler.DataChannel() - lostChannel := c.eventHandler.LostChannel() - for { - select { - - case <-c.closed: - return - case dataEvent, ok := <-dataChannel: - if !ok { - return - } - - l := len(dataEvent.Data) - switch { - case l >= netebpf.SizeofFailedConn: - c.extractConn(dataEvent.Data) - default: - log.Errorf("unknown type received from buffer, skipping. data size=%d, expecting %d", len(dataEvent.Data), netebpf.SizeofFailedConn) - continue - } - failedConnConsumerTelemetry.eventsLost.Inc() - dataEvent.Done() - // lost events only occur when using perf buffers - case lc, ok := <-lostChannel: - if !ok { - return - } - failedConnConsumerTelemetry.eventsLost.Add(float64(lc)) - } - } - }() -} diff --git a/pkg/network/tracer/connection/failure/matching.go b/pkg/network/tracer/connection/failure/matching.go deleted file mode 100644 index 41de01c7f44684..00000000000000 --- a/pkg/network/tracer/connection/failure/matching.go +++ /dev/null @@ -1,164 +0,0 @@ -// Unless explicitly stated otherwise all files in this repository are licensed -// under the Apache License Version 2.0. -// This product includes software developed at Datadog (https://www.datadoghq.com/). -// Copyright 2016-present Datadog, Inc. - -//go:build linux_bpf - -package failure - -import ( - "fmt" - "sync" - "syscall" - "time" - - manager "github.com/DataDog/ebpf-manager" - "golang.org/x/sys/unix" - - ddebpf "github.com/DataDog/datadog-agent/pkg/ebpf" - "github.com/DataDog/datadog-agent/pkg/network" - "github.com/DataDog/datadog-agent/pkg/network/ebpf" - "github.com/DataDog/datadog-agent/pkg/network/ebpf/probes" - "github.com/DataDog/datadog-agent/pkg/telemetry" - "github.com/DataDog/datadog-agent/pkg/util/log" -) - -var ( - telemetryModuleName = "network_tracer__tcp_failure" - connClosedFlushMapTTL = 10 * time.Millisecond.Nanoseconds() -) - -var failureTelemetry = struct { - failedConnMatches telemetry.Counter - failedConnOrphans telemetry.Counter - failedConnsDropped telemetry.Counter - closedConnFlushedCleaned telemetry.Counter -}{ - telemetry.NewCounter(telemetryModuleName, "matches", []string{"type"}, "Counter measuring the number of successful matches of failed connections with closed connections"), - telemetry.NewCounter(telemetryModuleName, "orphans", []string{}, "Counter measuring the number of orphans after associating failed connections with a closed connection"), - telemetry.NewCounter(telemetryModuleName, "dropped", []string{}, "Counter measuring the number of dropped failed connections"), - telemetry.NewCounter(telemetryModuleName, "closed_conn_flushed_cleaned", []string{}, "Counter measuring the number of conn_close_flushed entries cleaned in userspace"), -} - -// FailedConnStats is a wrapper to help document the purpose of the underlying map -type FailedConnStats struct { - CountByErrCode map[uint32]uint32 - Expiry int64 -} - -// String returns a string representation of the failedConnStats -func (t FailedConnStats) String() string { - return fmt.Sprintf( - "FailedConnStats{CountByErrCode: %v, Expiry: %d}", t.CountByErrCode, t.Expiry, - ) -} - -// FailedConns is a struct to hold failed connections -type FailedConns struct { - FailedConnMap map[network.ConnectionTuple]*FailedConnStats - maxFailuresBuffered uint32 - connCloseFlushedCleaner *ddebpf.MapCleaner[ebpf.ConnTuple, int64] - sync.Mutex -} - -// NewFailedConns returns a new FailedConns struct -func NewFailedConns(m *manager.Manager, maxFailedConnsBuffered uint32) *FailedConns { - fc := &FailedConns{ - FailedConnMap: make(map[network.ConnectionTuple]*FailedConnStats), - maxFailuresBuffered: maxFailedConnsBuffered, - } - fc.setupMapCleaner(m) - return fc -} - -// UpsertConn adds or updates the failed connection in the failed connection map -func (fc *FailedConns) UpsertConn(failedConn *Conn) { - if fc == nil { - return - } - - fc.Lock() - defer fc.Unlock() - - if len(fc.FailedConnMap) >= int(fc.maxFailuresBuffered) { - failureTelemetry.failedConnsDropped.Inc() - return - } - connTuple := failedConn.Tuple() - - stats, ok := fc.FailedConnMap[connTuple] - if !ok { - stats = &FailedConnStats{ - CountByErrCode: make(map[uint32]uint32), - } - fc.FailedConnMap[connTuple] = stats - } - - stats.CountByErrCode[failedConn.Reason]++ - stats.Expiry = time.Now().Add(2 * time.Minute).Unix() -} - -// MatchFailedConn increments the failed connection counters for a given connection based on the failed connection map -func (fc *FailedConns) MatchFailedConn(conn *network.ConnectionStats) { - if fc == nil || conn.Type != network.TCP { - return - } - - fc.Lock() - defer fc.Unlock() - - if failedConn, ok := fc.FailedConnMap[conn.ConnectionTuple]; ok { - // found matching failed connection - conn.TCPFailures = failedConn.CountByErrCode - - for errCode := range failedConn.CountByErrCode { - failureTelemetry.failedConnMatches.Add(1, unix.ErrnoName(syscall.Errno(errCode))) - } - delete(fc.FailedConnMap, conn.ConnectionTuple) - } -} - -// RemoveExpired removes expired failed connections from the map -func (fc *FailedConns) RemoveExpired() { - if fc == nil { - return - } - fc.Lock() - defer fc.Unlock() - - now := time.Now().Unix() - removed := 0 - - for connTuple, failedConn := range fc.FailedConnMap { - if failedConn.Expiry < now { - removed++ - delete(fc.FailedConnMap, connTuple) - } - } - - failureTelemetry.failedConnOrphans.Add(float64(removed)) -} - -func (fc *FailedConns) setupMapCleaner(m *manager.Manager) { - connCloseFlushMap, _, err := m.GetMap(probes.ConnCloseFlushed) - if err != nil { - log.Errorf("error getting %v map: %s", probes.ConnCloseFlushed, err) - return - } - mapCleaner, err := ddebpf.NewMapCleaner[ebpf.ConnTuple, int64](connCloseFlushMap, 1024) - if err != nil { - log.Errorf("error creating map cleaner: %s", err) - return - } - - mapCleaner.Clean(time.Second*1, nil, nil, func(now int64, _ ebpf.ConnTuple, val int64) bool { - expired := val > 0 && now-val > connClosedFlushMapTTL - if expired { - failureTelemetry.closedConnFlushedCleaned.Inc() - } - return expired - }) - - fc.connCloseFlushedCleaner = mapCleaner -} diff --git a/pkg/network/tracer/connection/failure/types.go b/pkg/network/tracer/connection/failure/types.go deleted file mode 100644 index 33ff19d928952b..00000000000000 --- a/pkg/network/tracer/connection/failure/types.go +++ /dev/null @@ -1,43 +0,0 @@ -// Unless explicitly stated otherwise all files in this repository are licensed -// under the Apache License Version 2.0. -// This product includes software developed at Datadog (https://www.datadoghq.com/). -// Copyright 2024-present Datadog, Inc. - -//go:build linux_bpf - -package failure - -import ( - "github.com/DataDog/datadog-agent/pkg/network" - "github.com/DataDog/datadog-agent/pkg/network/ebpf" -) - -// Conn represents a failed connection -type Conn ebpf.FailedConn - -// Tuple returns a network.ConnectionTuple -func (c Conn) Tuple() network.ConnectionTuple { - ct := network.ConnectionTuple{ - Source: c.Tup.SourceAddress(), - Dest: c.Tup.DestAddress(), - Pid: c.Tup.Pid, - NetNS: c.Tup.Netns, - SPort: c.Tup.Sport, - DPort: c.Tup.Dport, - } - - if c.Tup.Type() == ebpf.TCP { - ct.Type = network.TCP - } else { - ct.Type = network.UDP - } - - switch c.Tup.Family() { - case ebpf.IPv4: - ct.Family = network.AFINET - case ebpf.IPv6: - ct.Family = network.AFINET6 - } - - return ct -} diff --git a/pkg/network/tracer/connection/kprobe/manager.go b/pkg/network/tracer/connection/kprobe/manager.go index e8c1448e9d11cc..ceb37144b3c8af 100644 --- a/pkg/network/tracer/connection/kprobe/manager.go +++ b/pkg/network/tracer/connection/kprobe/manager.go @@ -61,7 +61,7 @@ var mainProbes = []probes.ProbeFuncName{ probes.UDPSendPageReturn, } -func initManager(mgr *ddebpf.Manager, connCloseEventHandler ddebpf.EventHandler, failedConnsHandler ddebpf.EventHandler, runtimeTracer bool, cfg *config.Config) error { +func initManager(mgr *ddebpf.Manager, connCloseEventHandler ddebpf.EventHandler, runtimeTracer bool, cfg *config.Config) error { mgr.Maps = []*manager.Map{ {Name: probes.ConnMap}, {Name: probes.TCPStatsMap}, @@ -84,9 +84,6 @@ func initManager(mgr *ddebpf.Manager, connCloseEventHandler ddebpf.EventHandler, {Name: probes.TCPCloseProgsMap}, } util.SetupClosedConnHandler(connCloseEventHandler, mgr, cfg) - if cfg.FailedConnectionsSupported() && failedConnsHandler != nil { - util.SetupFailedConnHandler(failedConnsHandler, mgr, cfg) - } for _, funcName := range mainProbes { p := &manager.Probe{ diff --git a/pkg/network/tracer/connection/kprobe/tracer.go b/pkg/network/tracer/connection/kprobe/tracer.go index 6b2ce3ac7cf1dd..d3ce64a3df4b99 100644 --- a/pkg/network/tracer/connection/kprobe/tracer.go +++ b/pkg/network/tracer/connection/kprobe/tracer.go @@ -126,7 +126,7 @@ func LoadTracer(cfg *config.Config, mgrOpts manager.Options, connCloseEventHandl var m *manager.Manager var closeFn func() if err == nil { - m, closeFn, err = coreTracerLoader(cfg, mgrOpts, connCloseEventHandler, failedConnsHandler) + m, closeFn, err = coreTracerLoader(cfg, mgrOpts, connCloseEventHandler) // if it is a verifier error, bail always regardless of // whether a fallback is enabled in config var ve *ebpf.VerifierError @@ -147,7 +147,7 @@ func LoadTracer(cfg *config.Config, mgrOpts manager.Options, connCloseEventHandl } if cfg.EnableRuntimeCompiler && (!cfg.EnableCORE || cfg.AllowRuntimeCompiledFallback) { - m, closeFn, err := rcTracerLoader(cfg, mgrOpts, connCloseEventHandler, failedConnsHandler) + m, closeFn, err := rcTracerLoader(cfg, mgrOpts, connCloseEventHandler) if err == nil { return m, closeFn, TracerTypeRuntimeCompiled, err } @@ -174,9 +174,9 @@ func LoadTracer(cfg *config.Config, mgrOpts manager.Options, connCloseEventHandl return m, closeFn, TracerTypePrebuilt, err } -func loadTracerFromAsset(buf bytecode.AssetReader, runtimeTracer, coreTracer bool, config *config.Config, mgrOpts manager.Options, connCloseEventHandler ddebpf.EventHandler, failedConnsHandler ddebpf.EventHandler) (*manager.Manager, func(), error) { +func loadTracerFromAsset(buf bytecode.AssetReader, runtimeTracer, coreTracer bool, config *config.Config, mgrOpts manager.Options, connCloseEventHandler ddebpf.EventHandler) (*manager.Manager, func(), error) { m := ddebpf.NewManagerWithDefault(&manager.Manager{}, &ebpftelemetry.ErrorsTelemetryModifier{}) - if err := initManager(m, connCloseEventHandler, failedConnsHandler, runtimeTracer, config); err != nil { + if err := initManager(m, connCloseEventHandler, runtimeTracer, config); err != nil { return nil, nil, fmt.Errorf("could not initialize manager: %w", err) } ringbufferEnabled := false @@ -274,7 +274,7 @@ func loadTracerFromAsset(buf bytecode.AssetReader, runtimeTracer, coreTracer boo return m.Manager, closeProtocolClassifierSocketFilterFn, nil } -func loadCORETracer(config *config.Config, mgrOpts manager.Options, connCloseEventHandler ddebpf.EventHandler, failedConnsHandler ddebpf.EventHandler) (*manager.Manager, func(), error) { +func loadCORETracer(config *config.Config, mgrOpts manager.Options, connCloseEventHandler ddebpf.EventHandler) (*manager.Manager, func(), error) { var m *manager.Manager var closeFn func() var err error @@ -285,21 +285,21 @@ func loadCORETracer(config *config.Config, mgrOpts manager.Options, connCloseEve o.DefaultKprobeAttachMethod = mgrOpts.DefaultKprobeAttachMethod o.DefaultKProbeMaxActive = mgrOpts.DefaultKProbeMaxActive o.BypassEnabled = mgrOpts.BypassEnabled - m, closeFn, err = tracerLoaderFromAsset(ar, false, true, config, o, connCloseEventHandler, failedConnsHandler) + m, closeFn, err = tracerLoaderFromAsset(ar, false, true, config, o, connCloseEventHandler) return err }) return m, closeFn, err } -func loadRuntimeCompiledTracer(config *config.Config, mgrOpts manager.Options, connCloseEventHandler ddebpf.EventHandler, failedConnsHandler ddebpf.EventHandler) (*manager.Manager, func(), error) { +func loadRuntimeCompiledTracer(config *config.Config, mgrOpts manager.Options, connCloseEventHandler ddebpf.EventHandler) (*manager.Manager, func(), error) { buf, err := getRuntimeCompiledTracer(config) if err != nil { return nil, nil, err } defer buf.Close() - return tracerLoaderFromAsset(buf, true, false, config, mgrOpts, connCloseEventHandler, failedConnsHandler) + return tracerLoaderFromAsset(buf, true, false, config, mgrOpts, connCloseEventHandler) } func loadPrebuiltTracer(config *config.Config, mgrOpts manager.Options, connCloseEventHandler ddebpf.EventHandler) (*manager.Manager, func(), error) { @@ -318,7 +318,7 @@ func loadPrebuiltTracer(config *config.Config, mgrOpts manager.Options, connClos config.CollectUDPv6Conns = false } - return tracerLoaderFromAsset(buf, false, false, config, mgrOpts, connCloseEventHandler, nil) + return tracerLoaderFromAsset(buf, false, false, config, mgrOpts, connCloseEventHandler) } func isCORETracerSupported() error { diff --git a/pkg/network/tracer/connection/kprobe/tracer_test.go b/pkg/network/tracer/connection/kprobe/tracer_test.go index 4ffa64609bcfcf..4c4a155ce12b95 100644 --- a/pkg/network/tracer/connection/kprobe/tracer_test.go +++ b/pkg/network/tracer/connection/kprobe/tracer_test.go @@ -169,8 +169,8 @@ func testTracerFallbackCOREAndRCErr(t *testing.T) { runFallbackTests(t, "CORE and RC error", true, true, tests) } -func loaderFunc(closeFn func(), err error) func(_ *config.Config, _ manager.Options, _ ddebpf.EventHandler, _ ddebpf.EventHandler) (*manager.Manager, func(), error) { - return func(_ *config.Config, _ manager.Options, _ ddebpf.EventHandler, _ ddebpf.EventHandler) (*manager.Manager, func(), error) { +func loaderFunc(closeFn func(), err error) func(_ *config.Config, _ manager.Options, _ ddebpf.EventHandler) (*manager.Manager, func(), error) { + return func(_ *config.Config, _ manager.Options, _ ddebpf.EventHandler) (*manager.Manager, func(), error) { return nil, closeFn, err } } @@ -251,7 +251,7 @@ func TestCORETracerSupported(t *testing.T) { }) coreCalled := false - coreTracerLoader = func(*config.Config, manager.Options, ddebpf.EventHandler, ddebpf.EventHandler) (*manager.Manager, func(), error) { + coreTracerLoader = func(*config.Config, manager.Options, ddebpf.EventHandler) (*manager.Manager, func(), error) { coreCalled = true return nil, nil, nil } @@ -296,7 +296,7 @@ func TestCORETracerSupported(t *testing.T) { func TestDefaultKprobeMaxActiveSet(t *testing.T) { prevLoader := tracerLoaderFromAsset - tracerLoaderFromAsset = func(_ bytecode.AssetReader, _, _ bool, _ *config.Config, mgrOpts manager.Options, _ ddebpf.EventHandler, _ ddebpf.EventHandler) (*manager.Manager, func(), error) { + tracerLoaderFromAsset = func(_ bytecode.AssetReader, _, _ bool, _ *config.Config, mgrOpts manager.Options, _ ddebpf.EventHandler) (*manager.Manager, func(), error) { assert.Equal(t, mgrOpts.DefaultKProbeMaxActive, 128) return nil, nil, nil } diff --git a/pkg/network/tracer/connection/tracer.go b/pkg/network/tracer/connection/tracer.go index daceae161da793..e1988a278cc1dd 100644 --- a/pkg/network/tracer/connection/tracer.go +++ b/pkg/network/tracer/connection/tracer.go @@ -16,7 +16,6 @@ import ( "github.com/DataDog/datadog-agent/comp/core/telemetry" "github.com/DataDog/datadog-agent/pkg/network" "github.com/DataDog/datadog-agent/pkg/network/config" - "github.com/DataDog/datadog-agent/pkg/network/tracer/connection/failure" ) // TracerType is the type of the underlying tracer @@ -47,8 +46,6 @@ type Tracer interface { GetConnections(buffer *network.ConnectionBuffer, filter func(*network.ConnectionStats) bool) error // FlushPending forces any closed connections waiting for batching to be processed immediately. FlushPending() - // GetFailedConnections returns the underlying map used to store failed connections - GetFailedConnections() *failure.FailedConns // Remove deletes the connection from tracking state. // It does not prevent the connection from re-appearing later, if additional traffic occurs. Remove(conn *network.ConnectionStats) error diff --git a/pkg/network/tracer/connection/util/conn_tracer.go b/pkg/network/tracer/connection/util/conn_tracer.go index b515cebfcf7f56..7b29d71e5befaa 100644 --- a/pkg/network/tracer/connection/util/conn_tracer.go +++ b/pkg/network/tracer/connection/util/conn_tracer.go @@ -42,28 +42,12 @@ func computeDefaultClosedConnRingBufferSize() int { return 8 * toPowerOf2(numCPUs) * os.Getpagesize() } -// computeDefaultFailedConnectionsRingBufferSize is the default buffer size of the ring buffer for closed connection events. -// Must be a power of 2 and a multiple of the page size -func computeDefaultFailedConnectionsRingBufferSize() int { - numCPUs, err := cebpf.PossibleCPU() - if err != nil { - numCPUs = 1 - } - return 8 * toPowerOf2(numCPUs) * os.Getpagesize() -} - // computeDefaultClosedConnPerfBufferSize is the default buffer size of the perf buffer for closed connection events. // Must be a multiple of the page size func computeDefaultClosedConnPerfBufferSize() int { return 8 * os.Getpagesize() } -// computeDefaultFailedConnPerfBufferSize is the default buffer size of the perf buffer for closed connection events. -// Must be a multiple of the page size -func computeDefaultFailedConnPerfBufferSize() int { - return 8 * os.Getpagesize() -} - // EnableRingbuffersViaMapEditor sets up the ring buffer for closed connection events via a map editor func EnableRingbuffersViaMapEditor(mgrOpts *manager.Options) { mgrOpts.MapSpecEditors[probes.ConnCloseEventMap] = manager.MapSpecEditor{ @@ -73,13 +57,6 @@ func EnableRingbuffersViaMapEditor(mgrOpts *manager.Options) { ValueSize: 0, EditorFlag: manager.EditType | manager.EditMaxEntries | manager.EditKeyValue, } - mgrOpts.MapSpecEditors[probes.FailedConnEventMap] = manager.MapSpecEditor{ - Type: cebpf.RingBuf, - MaxEntries: uint32(computeDefaultFailedConnectionsRingBufferSize()), - KeySize: 0, - ValueSize: 0, - EditorFlag: manager.EditType | manager.EditMaxEntries | manager.EditKeyValue, - } } // SetupHandler sets up the closed connection event handler @@ -122,11 +99,6 @@ func SetupHandler(eventHandler ebpf.EventHandler, mgr *ebpf.Manager, cfg *config } } -// SetupFailedConnHandler sets up the closed connection event handler -func SetupFailedConnHandler(connCloseEventHandler ebpf.EventHandler, mgr *ebpf.Manager, cfg *config.Config) { - SetupHandler(connCloseEventHandler, mgr, cfg, computeDefaultFailedConnPerfBufferSize(), probes.FailedConnEventMap) -} - // SetupClosedConnHandler sets up the closed connection event handler func SetupClosedConnHandler(connCloseEventHandler ebpf.EventHandler, mgr *ebpf.Manager, cfg *config.Config) { SetupHandler(connCloseEventHandler, mgr, cfg, computeDefaultClosedConnPerfBufferSize(), probes.ConnCloseEventMap) diff --git a/pkg/network/tracer/tracer.go b/pkg/network/tracer/tracer.go index 1a586b3ad9d841..2ef995137748f9 100644 --- a/pkg/network/tracer/tracer.go +++ b/pkg/network/tracer/tracer.go @@ -329,7 +329,6 @@ func (t *Tracer) storeClosedConnection(cs *network.ConnectionStats) { t.addProcessInfo(cs) tracerTelemetry.closedConns.IncWithTags(cs.Type.Tags()) - t.ebpfTracer.GetFailedConnections().MatchFailedConn(cs) t.state.StoreClosedConnection(cs) } @@ -567,9 +566,6 @@ func (t *Tracer) getConnections(activeBuffer *network.ConnectionBuffer) (latestU // get rid of stale process entries in the cache t.processCache.Trim() - // remove stale failed connections from map - t.ebpfTracer.GetFailedConnections().RemoveExpired() - entryCount := len(activeConnections) if entryCount >= int(t.config.MaxTrackedConnections) { log.Errorf("connection tracking map size has reached the limit of %d. Accurate connection count and data volume metrics will be affected. Increase config value `system_probe_config.max_tracked_connections` to correct this.", t.config.MaxTrackedConnections) diff --git a/pkg/network/tracer/tracer_test.go b/pkg/network/tracer/tracer_test.go index c1ef311086fa9b..e7f30e8b6714b2 100644 --- a/pkg/network/tracer/tracer_test.go +++ b/pkg/network/tracer/tracer_test.go @@ -1378,7 +1378,7 @@ func (s *TracerSuite) TestTCPFailureConnectionResetNoData() { } // findFailedConnection is a utility function to find a failed connection based on specific TCP error codes -func findFailedConnection(t *testing.T, local, remote string, conns *network.Connections, errorCode uint32) *network.ConnectionStats { // nolint:unused +func findFailedConnection(t *testing.T, local, remote string, conns *network.Connections, errorCode uint16) *network.ConnectionStats { // nolint:unused // Extract the address and port from the net.Addr types localAddrPort, err := netip.ParseAddrPort(local) if err != nil { @@ -1401,7 +1401,7 @@ func findFailedConnection(t *testing.T, local, remote string, conns *network.Con } // for some failed connections we don't know the local addr/port so we need to search by remote addr only -func findFailedConnectionByRemoteAddr(remoteAddr string, conns *network.Connections, errorCode uint32) *network.ConnectionStats { +func findFailedConnectionByRemoteAddr(remoteAddr string, conns *network.Connections, errorCode uint16) *network.ConnectionStats { failureFilter := func(cs network.ConnectionStats) bool { return netip.MustParseAddrPort(remoteAddr) == netip.AddrPortFrom(cs.Dest.Addr, cs.DPort) && cs.TCPFailures[errorCode] > 0 }