From 455f2673f1b03572dd7be51d40dd8900a41c62e8 Mon Sep 17 00:00:00 2001 From: Stuart Geipel Date: Tue, 7 Jan 2025 19:07:47 -0500 Subject: [PATCH 1/2] [NPM-4125] Refactor connection direction to use SYN packets (#32572) (cherry picked from commit f07df0a3c1fca0c83b5a15f553bd994091b0c8ac) --- pkg/network/event_common.go | 3 + .../connection/ebpfless/tcp_processor.go | 68 ++++-- .../connection/ebpfless/tcp_processor_test.go | 55 ++++- .../tracer/connection/ebpfless/tcp_utils.go | 17 ++ .../tracer/connection/ebpfless_tracer.go | 195 +++++++++++------- pkg/network/tracer/tracer_linux_test.go | 18 +- 6 files changed, 256 insertions(+), 100 deletions(-) diff --git a/pkg/network/event_common.go b/pkg/network/event_common.go index fc20fb9fa55bed..8a268e4508d2b3 100644 --- a/pkg/network/event_common.go +++ b/pkg/network/event_common.go @@ -80,6 +80,9 @@ type ConnectionFamily uint8 type ConnectionDirection uint8 const ( + // UNKNOWN represents connections where the direction is not known (yet) + UNKNOWN ConnectionDirection = 0 + // INCOMING represents connections inbound to the host INCOMING ConnectionDirection = 1 // incoming diff --git a/pkg/network/tracer/connection/ebpfless/tcp_processor.go b/pkg/network/tracer/connection/ebpfless/tcp_processor.go index 73ea6a1152aa34..15cf29540adb57 100644 --- a/pkg/network/tracer/connection/ebpfless/tcp_processor.go +++ b/pkg/network/tracer/connection/ebpfless/tcp_processor.go @@ -9,7 +9,6 @@ package ebpfless import ( "fmt" - "github.com/DataDog/datadog-agent/pkg/util/log" "syscall" "time" @@ -19,6 +18,7 @@ import ( "github.com/DataDog/datadog-agent/pkg/network" "github.com/DataDog/datadog-agent/pkg/network/config" + "github.com/DataDog/datadog-agent/pkg/util/log" ) type connectionState struct { @@ -61,6 +61,9 @@ type connectionState struct { // Can we make all connections in TCPProcessor have a ConnectionStats no matter what, and // filter them out in GetConnections? lastUpdateEpoch uint64 + + // connDirection has the direction of the connection, if we saw the SYN packet + connDirection network.ConnectionDirection } func (st *connectionState) hasMissedHandshake() bool { @@ -71,9 +74,9 @@ func (st *connectionState) hasMissedHandshake() bool { type TCPProcessor struct { cfg *config.Config // pendingConns contains connections with tcpState == connStatAttempted - pendingConns map[network.ConnectionTuple]*connectionState + pendingConns map[PCAPTuple]*connectionState // establishedConns contains connections with tcpState == connStatEstablished - establishedConns map[network.ConnectionTuple]*connectionState + establishedConns map[PCAPTuple]*connectionState } // TODO make this into a config value @@ -84,8 +87,8 @@ const pendingConnTimeoutNs = uint64(5 * time.Second) func NewTCPProcessor(cfg *config.Config) *TCPProcessor { return &TCPProcessor{ cfg: cfg, - pendingConns: make(map[network.ConnectionTuple]*connectionState, maxPendingConns), - establishedConns: make(map[network.ConnectionTuple]*connectionState, cfg.MaxTrackedConnections), + pendingConns: make(map[PCAPTuple]*connectionState, maxPendingConns), + establishedConns: make(map[PCAPTuple]*connectionState, cfg.MaxTrackedConnections), } } @@ -135,6 +138,10 @@ func (t *TCPProcessor) updateSynFlag(conn *network.ConnectionStats, st *connecti if tcp.RST { return } + // if this is the initial SYN, store the connection direction + if tcp.SYN && !tcp.ACK { + st.connDirection = connDirectionFromPktType(pktType) + } // progress the synStates based off this packet if pktType == unix.PACKET_OUTGOING { st.localSynState.update(tcp.SYN, tcp.ACK) @@ -247,6 +254,10 @@ func (t *TCPProcessor) updateRstFlag(conn *network.ConnectionStats, st *connecti if st.tcpState == connStatAttempted { reason = syscall.ECONNREFUSED } + + if conn.TCPFailures == nil { + conn.TCPFailures = make(map[uint16]uint32) + } conn.TCPFailures[uint16(reason)]++ if st.tcpState == connStatEstablished { @@ -278,7 +289,12 @@ func (t *TCPProcessor) Process(conn *network.ConnectionStats, timestampNs uint64 return ProcessResultNone, nil } - st := t.getConn(conn.ConnectionTuple) + tuple := MakeEbpflessTuple(conn.ConnectionTuple) + st, ok := t.getConn(tuple) + if !ok { + // create a fresh state object that will be stored by moveConn later + st = &connectionState{} + } origState := st.tcpState t.updateSynFlag(conn, st, pktType, tcp, payloadLen) @@ -288,7 +304,7 @@ func (t *TCPProcessor) Process(conn *network.ConnectionStats, timestampNs uint64 stateChanged := st.tcpState != origState if stateChanged { - ok := t.moveConn(conn.ConnectionTuple, st) + ok := t.moveConn(tuple, st) // if the map is full then we are unable to move the connection, report that if !ok { return ProcessResultMapFull, nil @@ -306,26 +322,25 @@ func (t *TCPProcessor) Process(conn *network.ConnectionStats, timestampNs uint64 return ProcessResultNone, nil } -func (t *TCPProcessor) getConn(tuple network.ConnectionTuple) *connectionState { +func (t *TCPProcessor) getConn(tuple PCAPTuple) (*connectionState, bool) { if st, ok := t.establishedConns[tuple]; ok { - return st + return st, true } if st, ok := t.pendingConns[tuple]; ok { - return st + return st, true } - // otherwise, create a fresh state object that will be stored by moveConn later - return &connectionState{} + return nil, false } // RemoveConn clears a ConnectionTuple from its internal state. -func (t *TCPProcessor) RemoveConn(tuple network.ConnectionTuple) { +func (t *TCPProcessor) RemoveConn(tuple PCAPTuple) { delete(t.pendingConns, tuple) delete(t.establishedConns, tuple) } // moveConn moves a connection to the correct map based on its tcpState. // If it had to drop the connection because the target map was full, it returns false. -func (t *TCPProcessor) moveConn(tuple network.ConnectionTuple, st *connectionState) bool { +func (t *TCPProcessor) moveConn(tuple PCAPTuple, st *connectionState) bool { t.RemoveConn(tuple) switch st.tcpState { @@ -365,3 +380,28 @@ func (t *TCPProcessor) CleanupExpiredPendingConns(timestampNs uint64) { } } } + +// MakeEbpflessTuple converts a network.ConnectionTuple to a PCAPTuple. +// See the PCAPTuple doc for more information. +func MakeEbpflessTuple(tuple network.ConnectionTuple) PCAPTuple { + ret := PCAPTuple(tuple) + ret.Pid = 0 + ret.Direction = network.UNKNOWN + return ret +} + +// MakeConnStatsTuple converts a PCAPTuple to a network.ConnectionTuple. +func MakeConnStatsTuple(tuple PCAPTuple) network.ConnectionTuple { + // Direction is still 0, this will get set by the ebpfless tracer in finalizeConnectionDirection + return network.ConnectionTuple(tuple) +} + +// GetConnDirection returns the direction of the connection. +// If the SYN packet was not seen (for a pre-existing connection), it returns ConnDirUnknown. +func (t *TCPProcessor) GetConnDirection(tuple PCAPTuple) (network.ConnectionDirection, bool) { + conn, ok := t.getConn(tuple) + if !ok { + return network.UNKNOWN, false + } + return conn.connDirection, true +} diff --git a/pkg/network/tracer/connection/ebpfless/tcp_processor_test.go b/pkg/network/tracer/connection/ebpfless/tcp_processor_test.go index 44def44a2860c5..866930ddeb421d 100644 --- a/pkg/network/tracer/connection/ebpfless/tcp_processor_test.go +++ b/pkg/network/tracer/connection/ebpfless/tcp_processor_test.go @@ -8,12 +8,13 @@ package ebpfless import ( - "github.com/DataDog/datadog-agent/pkg/network/config" "net" "syscall" "testing" "time" + "github.com/DataDog/datadog-agent/pkg/network/config" + "golang.org/x/sys/unix" "github.com/google/gopacket/layers" @@ -176,6 +177,15 @@ func newTCPTestFixture(t *testing.T) *tcpTestFixture { } } +func (fixture *tcpTestFixture) getConnectionState() *connectionState { + tuple := MakeEbpflessTuple(fixture.conn.ConnectionTuple) + conn, ok := fixture.tcp.getConn(tuple) + if ok { + return conn + } + return &connectionState{} +} + func (fixture *tcpTestFixture) runPkt(pkt testCapture) ProcessResult { if fixture.conn == nil { fixture.conn = makeTCPStates(pkt) @@ -200,9 +210,8 @@ func (fixture *tcpTestFixture) runAgainstState(packets []testCapture, expected [ expectedStrs = append(expectedStrs, labelForState(expected[i])) fixture.runPkt(pkt) - connTuple := fixture.conn.ConnectionTuple - actual := fixture.tcp.getConn(connTuple).tcpState - actualStrs = append(actualStrs, labelForState(actual)) + tcpState := fixture.getConnectionState().tcpState + actualStrs = append(actualStrs, labelForState(tcpState)) } require.Equal(fixture.t, expectedStrs, actualStrs) } @@ -815,3 +824,41 @@ func TestPendingConnExpiry(t *testing.T) { f.tcp.CleanupExpiredPendingConns(now + tenSecNs) require.Empty(t, f.tcp.pendingConns) } + +func TestTCPProcessorConnDirection(t *testing.T) { + pb := newPacketBuilder(lowerSeq, higherSeq) + + t.Run("outgoing", func(t *testing.T) { + f := newTCPTestFixture(t) + capture := []testCapture{ + pb.outgoing(0, 0, 0, SYN), + pb.incoming(0, 0, 1, SYN|ACK), + pb.outgoing(0, 1, 1, ACK), + } + f.runPkts(capture) + + require.Equal(t, network.OUTGOING, f.getConnectionState().connDirection) + }) + t.Run("incoming", func(t *testing.T) { + f := newTCPTestFixture(t) + capture := []testCapture{ + pb.incoming(0, 0, 0, SYN), + pb.outgoing(0, 0, 1, SYN|ACK), + pb.incoming(0, 1, 1, ACK), + } + f.runPkts(capture) + + require.Equal(t, network.INCOMING, f.getConnectionState().connDirection) + }) + t.Run("preexisting", func(t *testing.T) { + f := newTCPTestFixture(t) + capture := []testCapture{ + // just sending data, no SYN + pb.outgoing(1, 10, 10, ACK), + pb.incoming(1, 10, 11, ACK), + } + f.runPkts(capture) + + require.Equal(t, network.UNKNOWN, f.getConnectionState().connDirection) + }) +} diff --git a/pkg/network/tracer/connection/ebpfless/tcp_utils.go b/pkg/network/tracer/connection/ebpfless/tcp_utils.go index 7a30737e734aee..1175c517f1e93f 100644 --- a/pkg/network/tracer/connection/ebpfless/tcp_utils.go +++ b/pkg/network/tracer/connection/ebpfless/tcp_utils.go @@ -16,11 +16,28 @@ import ( "github.com/google/gopacket/layers" + "github.com/DataDog/datadog-agent/pkg/network" "github.com/DataDog/datadog-agent/pkg/telemetry" ) const ebpflessModuleName = "ebpfless_network_tracer" +// PCAPTuple represents a unique key for an ebpfless tracer connection. +// It represents a network.ConnectionTuple with only the fields that are available +// via packet capture: PID and Direction are zeroed out. +type PCAPTuple network.ConnectionTuple + +func connDirectionFromPktType(pktType uint8) network.ConnectionDirection { + switch pktType { + case unix.PACKET_HOST: + return network.INCOMING + case unix.PACKET_OUTGOING: + return network.OUTGOING + default: + return network.UNKNOWN + } +} + // ProcessResult represents what the ebpfless tracer should do with ConnectionStats after processing a packet type ProcessResult uint8 diff --git a/pkg/network/tracer/connection/ebpfless_tracer.go b/pkg/network/tracer/connection/ebpfless_tracer.go index 3eb5a03344b2a1..c095c6eb61c549 100644 --- a/pkg/network/tracer/connection/ebpfless_tracer.go +++ b/pkg/network/tracer/connection/ebpfless_tracer.go @@ -53,15 +53,14 @@ type ebpfLessTracer struct { config *config.Config - packetSrc *filter.AFPacketSource - exit chan struct{} - scratchConn *network.ConnectionStats + packetSrc *filter.AFPacketSource + exit chan struct{} udp *udpProcessor tcp *ebpfless.TCPProcessor // connection maps - conns map[network.ConnectionTuple]*network.ConnectionStats + conns map[ebpfless.PCAPTuple]*network.ConnectionStats boundPorts *ebpfless.BoundPorts cookieHasher *cookieHasher @@ -81,10 +80,9 @@ func newEbpfLessTracer(cfg *config.Config) (*ebpfLessTracer, error) { config: cfg, packetSrc: packetSrc, exit: make(chan struct{}), - scratchConn: &network.ConnectionStats{}, udp: &udpProcessor{}, tcp: ebpfless.NewTCPProcessor(cfg), - conns: make(map[network.ConnectionTuple]*network.ConnectionStats, cfg.MaxTrackedConnections), + conns: make(map[ebpfless.PCAPTuple]*network.ConnectionStats, cfg.MaxTrackedConnections), boundPorts: ebpfless.NewBoundPorts(cfg), cookieHasher: newCookieHasher(), } @@ -151,54 +149,41 @@ func (t *ebpfLessTracer) processConnection( decoded []gopacket.LayerType, closeCallback func(*network.ConnectionStats), ) error { - t.scratchConn.Source, t.scratchConn.Dest = util.Address{}, util.Address{} - t.scratchConn.SPort, t.scratchConn.DPort = 0, 0 - t.scratchConn.TCPFailures = make(map[uint16]uint32) - var ip4Present, ip6Present, udpPresent, tcpPresent bool - for _, layerType := range decoded { - switch layerType { - case layers.LayerTypeIPv4: - t.scratchConn.Source = util.AddressFromNetIP(ip4.SrcIP) - t.scratchConn.Dest = util.AddressFromNetIP(ip4.DstIP) - t.scratchConn.Family = network.AFINET - ip4Present = true - case layers.LayerTypeIPv6: - t.scratchConn.Source = util.AddressFromNetIP(ip6.SrcIP) - t.scratchConn.Dest = util.AddressFromNetIP(ip6.DstIP) - t.scratchConn.Family = network.AFINET6 - ip6Present = true - case layers.LayerTypeTCP: - t.scratchConn.SPort = uint16(tcp.SrcPort) - t.scratchConn.DPort = uint16(tcp.DstPort) - t.scratchConn.Type = network.TCP - tcpPresent = true - case layers.LayerTypeUDP: - t.scratchConn.SPort = uint16(udp.SrcPort) - t.scratchConn.DPort = uint16(udp.DstPort) - t.scratchConn.Type = network.UDP - udpPresent = true - } - } + tuple, flags := buildTuple(pktType, ip4, ip6, udp, tcp, decoded) // check if we have all the basic pieces - if !udpPresent && !tcpPresent { + if !flags.udpPresent && !flags.tcpPresent { log.Debugf("ignoring packet since its not udp or tcp") ebpfLessTracerTelemetry.skippedPackets.Inc("not_tcp_udp") return nil } + if !flags.ip4Present && !flags.ip6Present { + return fmt.Errorf("expected to have an IP layer") + } - t.determineConnectionDirection(t.scratchConn, pktType) - flipSourceDest(t.scratchConn, pktType) + // don't trace families/protocols that are disabled by configuration + switch tuple.Type { + case network.UDP: + if (flags.ip4Present && !t.config.CollectUDPv4Conns) || (flags.ip6Present && !t.config.CollectUDPv6Conns) { + return nil + } + case network.TCP: + if (flags.ip4Present && !t.config.CollectTCPv4Conns) || (flags.ip6Present && !t.config.CollectTCPv6Conns) { + return nil + } + } t.m.Lock() defer t.m.Unlock() - conn := t.conns[t.scratchConn.ConnectionTuple] - if conn == nil { - conn = &network.ConnectionStats{} - *conn = *t.scratchConn - t.cookieHasher.Hash(conn) - conn.Duration = time.Duration(time.Now().UnixNano()) + conn, ok := t.conns[tuple] + isNewConn := !ok + if isNewConn { + conn = &network.ConnectionStats{ + // NOTE: this tuple does not have the connection direction set yet. + // That will be set from determineConnectionDirection later + ConnectionTuple: ebpfless.MakeConnStatsTuple(tuple), + } } var ts int64 @@ -208,22 +193,12 @@ func (t *ebpfLessTracer) processConnection( } conn.LastUpdateEpoch = uint64(ts) - if !ip4Present && !ip6Present { - return nil - } - var result ebpfless.ProcessResult switch conn.Type { case network.UDP: - if (ip4Present && !t.config.CollectUDPv4Conns) || (ip6Present && !t.config.CollectUDPv6Conns) { - return nil - } result = ebpfless.ProcessResultStoreConn err = t.udp.process(conn, pktType, udp) case network.TCP: - if (ip4Present && !t.config.CollectTCPv4Conns) || (ip6Present && !t.config.CollectTCPv6Conns) { - return nil - } result, err = t.tcp.Process(conn, uint64(ts), pktType, ip4, ip6, tcp) default: err = fmt.Errorf("unsupported connection type %d", conn.Type) @@ -240,49 +215,124 @@ func (t *ebpfLessTracer) processConnection( switch result { case ebpfless.ProcessResultNone: case ebpfless.ProcessResultStoreConn: - maxTrackedConns := int(t.config.MaxTrackedConnections) - ok := ebpfless.WriteMapWithSizeLimit(t.conns, conn.ConnectionTuple, conn, maxTrackedConns) - if !ok { - // we don't have enough space to add this connection, remove its TCP state tracking + // if we fail to store this connection at any point, remove its TCP state tracking + storeConnOk := false + defer func() { + if storeConnOk { + return + } if conn.Type == network.TCP { - t.tcp.RemoveConn(conn.ConnectionTuple) + t.tcp.RemoveConn(tuple) } ebpfLessTracerTelemetry.droppedConnections.Inc() + }() + + if isNewConn { + conn.Duration = time.Duration(time.Now().UnixNano()) + direction, err := t.determineConnectionDirection(conn, pktType) + if err != nil { + return err + } + if direction == network.UNKNOWN { + return fmt.Errorf("could not determine connection direction") + } + conn.Direction = direction + + // now that the direction is set, hash the connection + t.cookieHasher.Hash(conn) } + maxTrackedConns := int(t.config.MaxTrackedConnections) + storeConnOk = ebpfless.WriteMapWithSizeLimit(t.conns, tuple, conn, maxTrackedConns) case ebpfless.ProcessResultCloseConn: - delete(t.conns, conn.ConnectionTuple) + delete(t.conns, tuple) closeCallback(conn) case ebpfless.ProcessResultMapFull: - delete(t.conns, conn.ConnectionTuple) + delete(t.conns, tuple) ebpfLessTracerTelemetry.droppedConnections.Inc() } return nil } -func flipSourceDest(conn *network.ConnectionStats, pktType uint8) { +type packetFlags struct { + ip4Present, ip6Present, udpPresent, tcpPresent bool +} + +// buildTuple converts the packet capture layer info into an EbpflessTuple with flags that indicate which layers were present. +func buildTuple(pktType uint8, ip4 *layers.IPv4, ip6 *layers.IPv6, udp *layers.UDP, tcp *layers.TCP, decoded []gopacket.LayerType) (ebpfless.PCAPTuple, packetFlags) { + var tuple ebpfless.PCAPTuple + var flags packetFlags + for _, layerType := range decoded { + switch layerType { + case layers.LayerTypeIPv4: + tuple.Source = util.AddressFromNetIP(ip4.SrcIP) + tuple.Dest = util.AddressFromNetIP(ip4.DstIP) + tuple.Family = network.AFINET + flags.ip4Present = true + case layers.LayerTypeIPv6: + tuple.Source = util.AddressFromNetIP(ip6.SrcIP) + tuple.Dest = util.AddressFromNetIP(ip6.DstIP) + tuple.Family = network.AFINET6 + flags.ip6Present = true + case layers.LayerTypeTCP: + tuple.SPort = uint16(tcp.SrcPort) + tuple.DPort = uint16(tcp.DstPort) + tuple.Type = network.TCP + flags.tcpPresent = true + case layers.LayerTypeUDP: + tuple.SPort = uint16(udp.SrcPort) + tuple.DPort = uint16(udp.DstPort) + tuple.Type = network.UDP + flags.udpPresent = true + } + } + if pktType == unix.PACKET_HOST { - conn.Dest, conn.Source = conn.Source, conn.Dest - conn.DPort, conn.SPort = conn.SPort, conn.DPort + tuple.Dest, tuple.Source = tuple.Source, tuple.Dest + tuple.DPort, tuple.SPort = tuple.SPort, tuple.DPort } + return tuple, flags } -func (t *ebpfLessTracer) determineConnectionDirection(conn *network.ConnectionStats, pktType uint8) { - t.m.Lock() - defer t.m.Unlock() +// determineConnectionDirection returns connection direction using information from the TCP processor. +// If the TCP processor doesn't know the direction, it will attempt to guess. +func (t *ebpfLessTracer) determineConnectionDirection(conn *network.ConnectionStats, pktType uint8) (network.ConnectionDirection, error) { + if conn.Type == network.TCP { + tuple := ebpfless.MakeEbpflessTuple(conn.ConnectionTuple) + dir, ok := t.tcp.GetConnDirection(tuple) + if !ok { + return network.UNKNOWN, fmt.Errorf("finalizeConnectionDirection: expected to find TCP connection for tuple: %+v", tuple) + } + switch dir { + case network.INCOMING: + case network.OUTGOING: + return dir, nil + case network.UNKNOWN: + // This happens when the TCP processor missed the SYN packet. + // Fall through and guess the direction. + } + } ok := t.boundPorts.Find(conn.Type, conn.SPort) if ok { // incoming connection - conn.Direction = network.INCOMING - return + return network.INCOMING, nil + } + // for local connections - the destination could be a bound port + if conn.Dest.Addr.IsLoopback() { + ok := t.boundPorts.Find(conn.Type, conn.DPort) + if ok { + return network.OUTGOING, nil + } } switch pktType { case unix.PACKET_HOST: - conn.Direction = network.INCOMING + return network.INCOMING, nil case unix.PACKET_OUTGOING: - conn.Direction = network.OUTGOING + return network.OUTGOING, nil + default: + return network.UNKNOWN, fmt.Errorf("unknown packet type %d", pktType) } } @@ -342,9 +392,10 @@ func (t *ebpfLessTracer) cleanupPendingConns() error { func (t *ebpfLessTracer) FlushPending() {} func (t *ebpfLessTracer) remove(conn *network.ConnectionStats) error { - delete(t.conns, conn.ConnectionTuple) + tuple := ebpfless.MakeEbpflessTuple(conn.ConnectionTuple) + delete(t.conns, tuple) if conn.Type == network.TCP { - t.tcp.RemoveConn(conn.ConnectionTuple) + t.tcp.RemoveConn(tuple) } return nil } diff --git a/pkg/network/tracer/tracer_linux_test.go b/pkg/network/tracer/tracer_linux_test.go index bcaa7f11ea9367..05612e8af34034 100644 --- a/pkg/network/tracer/tracer_linux_test.go +++ b/pkg/network/tracer/tracer_linux_test.go @@ -2042,13 +2042,12 @@ func (s *TracerSuite) TestPreexistingConnectionDirection() { } m := outgoing.Monotonic - assert.Equal(collect, clientMessageSize, int(m.SentBytes)) - // ebpfless RecvBytes is based off acknowledgements, so it can miss the first - // packet in a pre-existing connection + // skip byte counts in ebpfless: for ebpfless pre-existing connections, + // byte counts will miss the first couple packets while in connStatAttempted. if !tr.config.EnableEbpfless { + assert.Equal(collect, clientMessageSize, int(m.SentBytes)) assert.Equal(collect, serverMessageSize, int(m.RecvBytes)) - } - if !tr.config.EnableEbpfless { + assert.Equal(collect, os.Getpid(), int(outgoing.Pid)) } assert.Equal(collect, addrPort(server.Address()), int(outgoing.DPort)) @@ -2056,13 +2055,12 @@ func (s *TracerSuite) TestPreexistingConnectionDirection() { assert.Equal(collect, network.OUTGOING, outgoing.Direction) m = incoming.Monotonic - // ebpfless RecvBytes is based off acknowledgements, so it can miss the first - // packet in a pre-existing connection + // skip byte counts in ebpfless: for ebpfless pre-existing connections, + // byte counts will miss the first couple packets while in connStatAttempted. if !tr.config.EnableEbpfless { assert.Equal(collect, clientMessageSize, int(m.RecvBytes)) - } - assert.Equal(collect, serverMessageSize, int(m.SentBytes)) - if !tr.config.EnableEbpfless { + assert.Equal(collect, serverMessageSize, int(m.SentBytes)) + assert.Equal(collect, os.Getpid(), int(incoming.Pid)) } assert.Equal(collect, addrPort(server.Address()), int(incoming.SPort)) From 208e04fea8256afe3837da6a975f51e33d55c6cc Mon Sep 17 00:00:00 2001 From: Stuart Geipel Date: Wed, 29 Jan 2025 07:01:23 -0800 Subject: [PATCH 2/2] add release note --- .../notes/fix-ebpfless-preview-723bdc7e5b7e7de8.yaml | 11 +++++++++++ 1 file changed, 11 insertions(+) create mode 100644 releasenotes/notes/fix-ebpfless-preview-723bdc7e5b7e7de8.yaml diff --git a/releasenotes/notes/fix-ebpfless-preview-723bdc7e5b7e7de8.yaml b/releasenotes/notes/fix-ebpfless-preview-723bdc7e5b7e7de8.yaml new file mode 100644 index 00000000000000..525982da643887 --- /dev/null +++ b/releasenotes/notes/fix-ebpfless-preview-723bdc7e5b7e7de8.yaml @@ -0,0 +1,11 @@ +# Each section from every release note are combined when the +# CHANGELOG.rst is rendered. So the text needs to be worded so that +# it does not depend on any information only available in another +# section. This may mean repeating some details, but each section +# must be readable independently of the other. +# +# Each section note must be formatted as reStructuredText. +--- +fixes: + - | + Fixes a connection tracking regression in the 7.62 Fargate preview for Cloud Network Monitoring.